You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2015/07/22 18:05:30 UTC

[01/15] cassandra git commit: Simplify some 8099's implementations

Repository: cassandra
Updated Branches:
  refs/heads/trunk 7659ae2ee -> 245759942


http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
index 671ee9c..0f822c1 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -249,17 +249,10 @@ public class SSTableRewriterTest extends SchemaLoader
         {
             for (int i = 0; i < 10000; i++)
             {
-                RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 1,  random(i, 10));
-
-                PartitionUpdate update = null;
-
+                UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, random(i, 10)).withTimestamp(1);
                 for (int j = 0; j < 100; j++)
-                {
-                    builder.clustering("" + j).add("val", ByteBuffer.allocate(1000));
-                    update = builder.buildUpdate();
-                }
-
-                writer.append(update.unfilteredIterator());
+                    builder.newRow("" + j).add("val", ByteBuffer.allocate(1000));
+                writer.append(builder.build().unfilteredIterator());
             }
 
             SSTableReader s = writer.setMaxDataAge(1000).openEarly();
@@ -267,17 +260,10 @@ public class SSTableRewriterTest extends SchemaLoader
             assertFileCounts(dir.list(), 2, 2);
             for (int i = 10000; i < 20000; i++)
             {
-                RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 1,  random(i, 10));
-
-                PartitionUpdate update = null;
-
+                UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, random(i, 10)).withTimestamp(1);
                 for (int j = 0; j < 100; j++)
-                {
-                    builder.clustering("" + j).add("val", ByteBuffer.allocate(1000));
-                    update = builder.buildUpdate();
-                }
-
-                writer.append(update.unfilteredIterator());
+                    builder.newRow("" + j).add("val", ByteBuffer.allocate(1000));
+                writer.append(builder.build().unfilteredIterator());
             }
             SSTableReader s2 = writer.setMaxDataAge(1000).openEarly();
             assertTrue(s.last.compareTo(s2.last) < 0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/schema/DefsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/DefsTest.java b/test/unit/org/apache/cassandra/schema/DefsTest.java
index cf49253..b567bb5 100644
--- a/test/unit/org/apache/cassandra/schema/DefsTest.java
+++ b/test/unit/org/apache/cassandra/schema/DefsTest.java
@@ -99,7 +99,7 @@ public class DefsTest
         for (int i = 0; i < 5; i++)
         {
             ByteBuffer name = ByteBuffer.wrap(new byte[] { (byte)i });
-            cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, name, BytesType.instance, null).setIndex(Integer.toString(i), IndexType.KEYS, null));
+            cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, name, BytesType.instance).setIndex(Integer.toString(i), IndexType.KEYS, null));
         }
 
         cfm.comment("No comment")
@@ -114,12 +114,12 @@ public class DefsTest
         CFMetaData cfNew = cfm.copy();
 
         // add one.
-        ColumnDefinition addIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 5 }), BytesType.instance, null)
+        ColumnDefinition addIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 5 }), BytesType.instance)
                                                        .setIndex("5", IndexType.KEYS, null);
         cfNew.addColumnDefinition(addIndexDef);
 
         // remove one.
-        ColumnDefinition removeIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 0 }), BytesType.instance, null)
+        ColumnDefinition removeIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 0 }), BytesType.instance)
                                                           .setIndex("0", IndexType.KEYS, null);
         assertTrue(cfNew.removeColumnDefinition(removeIndexDef));
 
@@ -530,7 +530,7 @@ public class DefsTest
         // drop the index
         CFMetaData meta = cfs.metadata.copy();
         ColumnDefinition cdOld = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("birthdate"));
-        ColumnDefinition cdNew = ColumnDefinition.regularDef(meta, cdOld.name.bytes, cdOld.type, null);
+        ColumnDefinition cdNew = ColumnDefinition.regularDef(meta, cdOld.name.bytes, cdOld.type);
         meta.addOrReplaceColumnDefinition(cdNew);
         MigrationManager.announceColumnFamilyUpdate(meta, false);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/service/DataResolverTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/DataResolverTest.java b/test/unit/org/apache/cassandra/service/DataResolverTest.java
index d752539..dd57a96 100644
--- a/test/unit/org/apache/cassandra/service/DataResolverTest.java
+++ b/test/unit/org/apache/cassandra/service/DataResolverTest.java
@@ -360,7 +360,7 @@ public class DataResolverTest
         assertEquals(1, messageRecorder.sent.size());
         MessageOut msg = getSentMessage(peer1);
         assertRepairMetadata(msg);
-        assertRepairContainsDeletions(msg, new SimpleDeletionTime(1, nowInSec));
+        assertRepairContainsDeletions(msg, new DeletionTime(1, nowInSec));
         assertRepairContainsNoColumns(msg);
     }
 
@@ -397,19 +397,19 @@ public class DataResolverTest
         assertEquals(4, messageRecorder.sent.size());
         MessageOut msg = getSentMessage(peer1);
         assertRepairMetadata(msg);
-        assertRepairContainsDeletions(msg, new SimpleDeletionTime(2, nowInSec));
+        assertRepairContainsDeletions(msg, new DeletionTime(2, nowInSec));
         assertRepairContainsColumn(msg, "1", "two", "B", 3);
 
         // peer 2 needs the deletion from peer 4 and the row from peer 3
         msg = getSentMessage(peer2);
         assertRepairMetadata(msg);
-        assertRepairContainsDeletions(msg, new SimpleDeletionTime(2, nowInSec));
+        assertRepairContainsDeletions(msg, new DeletionTime(2, nowInSec));
         assertRepairContainsColumn(msg, "1", "two", "B", 3);
 
         // peer 3 needs just the deletion from peer 4
         msg = getSentMessage(peer3);
         assertRepairMetadata(msg);
-        assertRepairContainsDeletions(msg, new SimpleDeletionTime(2, nowInSec));
+        assertRepairContainsDeletions(msg, new DeletionTime(2, nowInSec));
         assertRepairContainsNoColumns(msg);
 
         // peer 4 needs just the row from peer 3
@@ -500,7 +500,7 @@ public class DataResolverTest
     private RangeTombstone tombstone(Object start, Object end, long markedForDeleteAt, int localDeletionTime)
     {
         return new RangeTombstone(Slice.make(cfm.comparator.make(start), cfm.comparator.make(end)),
-                                  new SimpleDeletionTime(markedForDeleteAt, localDeletionTime));
+                                  new DeletionTime(markedForDeleteAt, localDeletionTime));
     }
 
     private UnfilteredPartitionIterator fullPartitionDelete(CFMetaData cfm, DecoratedKey dk, long timestamp, int nowInSec)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java b/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
index 0cf7491..df4b08a 100644
--- a/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
+++ b/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
@@ -24,10 +24,7 @@ import org.junit.Test;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.UnfilteredRowIterators;
-import org.apache.cassandra.db.rows.Cell;
-import org.apache.cassandra.db.rows.Row;
-import org.apache.cassandra.db.rows.RowIterator;
+import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.db.partitions.Partition;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
@@ -51,7 +48,7 @@ public class TriggerExecutorTest
 
         RowIterator rowIterator = UnfilteredRowIterators.filter(mutated.unfilteredIterator(), FBUtilities.nowInSeconds());
 
-        Iterator<Cell> cells = rowIterator.next().iterator();
+        Iterator<Cell> cells = rowIterator.next().cells().iterator();
         assertEquals(bytes("trigger"), cells.next().value());
 
         assertTrue(!rowIterator.hasNext());
@@ -271,22 +268,15 @@ public class TriggerExecutorTest
 
     private static PartitionUpdate makeCf(CFMetaData metadata, String key, String columnValue1, String columnValue2)
     {
-        PartitionUpdate update = new PartitionUpdate(metadata, Util.dk(key), metadata.partitionColumns(), 1);
-
-        LivenessInfo info = SimpleLivenessInfo.forUpdate(FBUtilities.timestampMicros(), LivenessInfo.NO_TTL, FBUtilities.nowInSeconds(), metadata);
-
+        Row.Builder builder = ArrayBackedRow.unsortedBuilder(metadata.partitionColumns().regulars, FBUtilities.nowInSeconds());
+        builder.newRow(Clustering.EMPTY);
+        long ts = FBUtilities.timestampMicros();
         if (columnValue1 != null)
-        {
-            update.writer().writeCell(metadata.getColumnDefinition(bytes("c1")), false, bytes(columnValue1), info, null);
-            update.writer().endOfRow();
-        }
+            builder.addCell(BufferCell.live(metadata, metadata.getColumnDefinition(bytes("c1")), ts, bytes(columnValue1)));
         if (columnValue2 != null)
-        {
-            update.writer().writeCell(metadata.getColumnDefinition(bytes("c2")), false, bytes(columnValue1), info, null);
-            update.writer().endOfRow();
-        }
+            builder.addCell(BufferCell.live(metadata, metadata.getColumnDefinition(bytes("c2")), ts, bytes(columnValue2)));
 
-        return update;
+        return PartitionUpdate.singleRowUpdate(metadata, Util.dk(key), builder.build());
     }
 
     public static class NoOpTrigger implements ITrigger


[12/15] cassandra git commit: Simplify some 8099's implementations

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/Slice.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Slice.java b/src/java/org/apache/cassandra/db/Slice.java
index 05c2977..2ffb91e 100644
--- a/src/java/org/apache/cassandra/db/Slice.java
+++ b/src/java/org/apache/cassandra/db/Slice.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.security.MessageDigest;
@@ -25,6 +24,7 @@ import java.util.*;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.ObjectSizes;
@@ -68,8 +68,8 @@ public class Slice
     private Slice(Bound start, Bound end)
     {
         assert start.isStart() && end.isEnd();
-        this.start = start.takeAlias();
-        this.end = end.takeAlias();
+        this.start = start;
+        this.end = end;
     }
 
     public static Slice make(Bound start, Bound end)
@@ -331,7 +331,7 @@ public class Slice
                  + Bound.serializer.serializedSize(slice.end, version, types);
         }
 
-        public Slice deserialize(DataInput in, int version, List<AbstractType<?>> types) throws IOException
+        public Slice deserialize(DataInputPlus in, int version, List<AbstractType<?>> types) throws IOException
         {
             Bound start = Bound.serializer.deserialize(in, version, types);
             Bound end = Bound.serializer.deserialize(in, version, types);
@@ -346,21 +346,19 @@ public class Slice
      */
     public static class Bound extends AbstractClusteringPrefix
     {
-        private static final long EMPTY_SIZE = ObjectSizes.measure(new Bound(Kind.INCL_START_BOUND, new ByteBuffer[0]));
         public static final Serializer serializer = new Serializer();
 
-        /** The smallest start bound, i.e. the one that starts before any row. */
-        public static final Bound BOTTOM = inclusiveStartOf();
-        /** The biggest end bound, i.e. the one that ends after any row. */
-        public static final Bound TOP = inclusiveEndOf();
-
-        protected final Kind kind;
-        protected final ByteBuffer[] values;
+        /**
+         * The smallest and biggest bound. Note that as range tomstone bounds are (special case) of slice bounds,
+         * we want the BOTTOM and TOP to be the same object, but we alias them here because it's cleaner when dealing
+         * with slices to refer to Slice.Bound.BOTTOM and Slice.Bound.TOP.
+         */
+        public static final Bound BOTTOM = RangeTombstone.Bound.BOTTOM;
+        public static final Bound TOP = RangeTombstone.Bound.TOP;
 
         protected Bound(Kind kind, ByteBuffer[] values)
         {
-            this.kind = kind;
-            this.values = values;
+            super(kind, values);
         }
 
         public static Bound create(Kind kind, ByteBuffer[] values)
@@ -396,22 +394,6 @@ public class Slice
             return create(Kind.EXCL_END_BOUND, values);
         }
 
-        public static Bound exclusiveStartOf(ClusteringPrefix prefix)
-        {
-            ByteBuffer[] values = new ByteBuffer[prefix.size()];
-            for (int i = 0; i < prefix.size(); i++)
-                values[i] = prefix.get(i);
-            return exclusiveStartOf(values);
-        }
-
-        public static Bound inclusiveEndOf(ClusteringPrefix prefix)
-        {
-            ByteBuffer[] values = new ByteBuffer[prefix.size()];
-            for (int i = 0; i < prefix.size(); i++)
-                values[i] = prefix.get(i);
-            return inclusiveEndOf(values);
-        }
-
         public static Bound create(ClusteringComparator comparator, boolean isStart, boolean isInclusive, Object... values)
         {
             CBuilder builder = CBuilder.create(comparator);
@@ -426,21 +408,6 @@ public class Slice
             return builder.buildBound(isStart, isInclusive);
         }
 
-        public Kind kind()
-        {
-            return kind;
-        }
-
-        public int size()
-        {
-            return values.length;
-        }
-
-        public ByteBuffer get(int i)
-        {
-            return values[i];
-        }
-
         public Bound withNewKind(Kind kind)
         {
             assert !kind.isBoundary();
@@ -480,24 +447,6 @@ public class Slice
             return withNewKind(kind().invert());
         }
 
-        public ByteBuffer[] getRawValues()
-        {
-            return values;
-        }
-
-        public void digest(MessageDigest digest)
-        {
-            for (int i = 0; i < size(); i++)
-                digest.update(get(i).duplicate());
-            FBUtilities.updateWithByte(digest, kind().ordinal());
-        }
-
-        public void writeTo(Slice.Bound.Writer writer)
-        {
-            super.writeTo(writer);
-            writer.writeBoundKind(kind());
-        }
-
         // For use by intersects, it's called with the sstable bound opposite to the slice bound
         // (so if the slice bound is a start, it's call with the max sstable bound)
         private int compareTo(ClusteringComparator comparator, List<ByteBuffer> sstableBound)
@@ -544,66 +493,10 @@ public class Slice
             return sb.append(")").toString();
         }
 
-        // Overriding to get a more precise type
-        @Override
-        public Bound takeAlias()
-        {
-            return this;
-        }
-
-        @Override
-        public long unsharedHeapSize()
-        {
-            return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(values);
-        }
-
-        public long unsharedHeapSizeExcludingData()
-        {
-            return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(values);
-        }
-
-        public static Builder builder(int size)
-        {
-            return new Builder(size);
-        }
-
-        public interface Writer extends ClusteringPrefix.Writer
-        {
-            public void writeBoundKind(Kind kind);
-        }
-
-        public static class Builder implements Writer
-        {
-            private final ByteBuffer[] values;
-            private Kind kind;
-            private int idx;
-
-            private Builder(int size)
-            {
-                this.values = new ByteBuffer[size];
-            }
-
-            public void writeClusteringValue(ByteBuffer value)
-            {
-                values[idx++] = value;
-            }
-
-            public void writeBoundKind(Kind kind)
-            {
-                this.kind = kind;
-            }
-
-            public Slice.Bound build()
-            {
-                assert idx == values.length;
-                return Slice.Bound.create(kind, values);
-            }
-        }
-
         /**
          * Serializer for slice bounds.
          * <p>
-         * Contrarily to {@code Clustering}, a slice bound can only be a true prefix of the full clustering, so we actually record
+         * Contrarily to {@code Clustering}, a slice bound can be a true prefix of the full clustering, so we actually record
          * its size.
          */
         public static class Serializer
@@ -622,31 +515,21 @@ public class Slice
                      + ClusteringPrefix.serializer.valuesWithoutSizeSerializedSize(bound, version, types);
             }
 
-            public Slice.Bound deserialize(DataInput in, int version, List<AbstractType<?>> types) throws IOException
+            public Slice.Bound deserialize(DataInputPlus in, int version, List<AbstractType<?>> types) throws IOException
             {
                 Kind kind = Kind.values()[in.readByte()];
                 return deserializeValues(in, kind, version, types);
             }
 
-            public Slice.Bound deserializeValues(DataInput in, Kind kind, int version, List<AbstractType<?>> types) throws IOException
+            public Slice.Bound deserializeValues(DataInputPlus in, Kind kind, int version, List<AbstractType<?>> types) throws IOException
             {
                 int size = in.readUnsignedShort();
                 if (size == 0)
                     return kind.isStart() ? BOTTOM : TOP;
 
-                Builder builder = builder(size);
-                ClusteringPrefix.serializer.deserializeValuesWithoutSize(in, size, version, types, builder);
-                builder.writeBoundKind(kind);
-                return builder.build();
-            }
-
-            public void deserializeValues(DataInput in, Bound.Kind kind, int version, List<AbstractType<?>> types, Writer writer) throws IOException
-            {
-                int size = in.readUnsignedShort();
-                ClusteringPrefix.serializer.deserializeValuesWithoutSize(in, size, version, types, writer);
-                writer.writeBoundKind(kind);
+                ByteBuffer[] values = ClusteringPrefix.serializer.deserializeValuesWithoutSize(in, size, version, types);
+                return Slice.Bound.create(kind, values);
             }
-
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/Slices.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Slices.java b/src/java/org/apache/cassandra/db/Slices.java
index a6c690b..32ca06d 100644
--- a/src/java/org/apache/cassandra/db/Slices.java
+++ b/src/java/org/apache/cassandra/db/Slices.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
@@ -28,6 +27,7 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
 /**
@@ -318,7 +318,7 @@ public abstract class Slices implements Iterable<Slice>
             return size;
         }
 
-        public Slices deserialize(DataInput in, int version, CFMetaData metadata) throws IOException
+        public Slices deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException
         {
             int size = in.readInt();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index e8247a3..df7e7ef 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -1160,7 +1160,7 @@ public final class SystemKeyspace
 
         // delete all previous values with a single range tombstone.
         int nowInSec = FBUtilities.nowInSeconds();
-        update.addRangeTombstone(Slice.make(SizeEstimates.comparator, table), new SimpleDeletionTime(timestamp - 1, nowInSec));
+        update.add(new RangeTombstone(Slice.make(SizeEstimates.comparator, table), new DeletionTime(timestamp - 1, nowInSec)));
 
         // add a CQL row for each primary token range.
         for (Map.Entry<Range<Token>, Pair<Long, Long>> entry : estimates.entrySet())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/TypeSizes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/TypeSizes.java b/src/java/org/apache/cassandra/db/TypeSizes.java
index a9e432f..73766c8 100644
--- a/src/java/org/apache/cassandra/db/TypeSizes.java
+++ b/src/java/org/apache/cassandra/db/TypeSizes.java
@@ -68,6 +68,11 @@ public final class TypeSizes
         return sizeof(value.remaining()) + value.remaining();
     }
 
+    public static int sizeofWithVIntLength(ByteBuffer value)
+    {
+        return sizeofVInt(value.remaining()) + value.remaining();
+    }
+
     public static int sizeof(boolean value)
     {
         return BOOL_SIZE;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
index cf7c2dd..b3709d2 100644
--- a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
+++ b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
@@ -111,8 +111,7 @@ public abstract class UnfilteredDeserializer
         private boolean isReady;
         private boolean isDone;
 
-        private final ReusableRow row;
-        private final RangeTombstoneMarker.Builder markerBuilder;
+        private final Row.Builder builder;
 
         private CurrentDeserializer(CFMetaData metadata,
                                     DataInputPlus in,
@@ -122,8 +121,7 @@ public abstract class UnfilteredDeserializer
             super(metadata, in, helper);
             this.header = header;
             this.clusteringDeserializer = new ClusteringPrefix.Deserializer(metadata.comparator, in, header);
-            this.row = new ReusableRow(metadata.clusteringColumns().size(), header.columns().regulars, true, metadata.isCounter());
-            this.markerBuilder = new RangeTombstoneMarker.Builder(metadata.clusteringColumns().size());
+            this.builder = ArrayBackedRow.sortedBuilder(helper.fetchedRegularColumns(header));
         }
 
         public boolean hasNext() throws IOException
@@ -181,17 +179,13 @@ public abstract class UnfilteredDeserializer
             isReady = false;
             if (UnfilteredSerializer.kind(nextFlags) == Unfiltered.Kind.RANGE_TOMBSTONE_MARKER)
             {
-                markerBuilder.reset();
-                RangeTombstone.Bound.Kind kind = clusteringDeserializer.deserializeNextBound(markerBuilder);
-                UnfilteredSerializer.serializer.deserializeMarkerBody(in, header, kind.isBoundary(), markerBuilder);
-                return markerBuilder.build();
+                RangeTombstone.Bound bound = clusteringDeserializer.deserializeNextBound();
+                return UnfilteredSerializer.serializer.deserializeMarkerBody(in, header, bound);
             }
             else
             {
-                Row.Writer writer = row.writer();
-                clusteringDeserializer.deserializeNextClustering(writer);
-                UnfilteredSerializer.serializer.deserializeRowBody(in, header, helper, nextFlags, writer);
-                return row;
+                builder.newRow(clusteringDeserializer.deserializeNextClustering());
+                return UnfilteredSerializer.serializer.deserializeRowBody(in, header, helper, nextFlags, builder);
             }
         }
 
@@ -205,7 +199,7 @@ public abstract class UnfilteredDeserializer
             }
             else
             {
-                UnfilteredSerializer.serializer.skipRowBody(in, header, helper, nextFlags);
+                UnfilteredSerializer.serializer.skipRowBody(in, header, nextFlags);
             }
         }
 
@@ -221,7 +215,6 @@ public abstract class UnfilteredDeserializer
         private final boolean readAllAsDynamic;
         private boolean skipStatic;
 
-        private int nextFlags;
         private boolean isDone;
         private boolean isStart = true;
 
@@ -254,13 +247,7 @@ public abstract class UnfilteredDeserializer
 
         public boolean hasNext() throws IOException
         {
-            if (nextAtom != null)
-                return true;
-
-            if (isDone)
-                return false;
-
-            return deserializeNextAtom();
+            return nextAtom != null || (!isDone && deserializeNextAtom());
         }
 
         private boolean deserializeNextAtom() throws IOException
@@ -392,6 +379,7 @@ public abstract class UnfilteredDeserializer
             grouper.addAtom(nextAtom);
             while (deserializeNextAtom() && grouper.addAtom(nextAtom))
             {
+                // Nothing to do, deserializeNextAtom() changes nextAtom and it's then added to the grouper
             }
 
             // if this was the first static row, we're done with it. Otherwise, we're also done with static.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
index b406251..8625112 100644
--- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
@@ -38,8 +38,6 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 
 abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
 {
-    private static final Logger logger = LoggerFactory.getLogger(AbstractSSTableIterator.class);
-
     protected final SSTableReader sstable;
     protected final DecoratedKey key;
     protected final DeletionTime partitionLevelDeletion;
@@ -65,7 +63,7 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
         this.sstable = sstable;
         this.key = key;
         this.columns = columnFilter;
-        this.helper = new SerializationHelper(sstable.descriptor.version.correspondingMessagingVersion(), SerializationHelper.Flag.LOCAL, columnFilter);
+        this.helper = new SerializationHelper(sstable.metadata, sstable.descriptor.version.correspondingMessagingVersion(), SerializationHelper.Flag.LOCAL, columnFilter);
         this.isForThrift = isForThrift;
 
         if (indexEntry == null)
@@ -81,7 +79,7 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
             {
                 // We seek to the beginning to the partition if either:
                 //   - the partition is not indexed; we then have a single block to read anyway
-                //     and we need to read the partition deletion time.
+                //     (and we need to read the partition deletion time).
                 //   - we're querying static columns.
                 boolean needSeekAtPartitionStart = !indexEntry.isIndexed() || !columns.fetchedColumns().statics.isEmpty();
 
@@ -104,24 +102,24 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
 
                     // Note that this needs to be called after file != null and after the partitionDeletion has been set, but before readStaticRow
                     // (since it uses it) so we can't move that up (but we'll be able to simplify as soon as we drop support for the old file format).
-                    this.reader = needsReader ? createReader(indexEntry, file, needSeekAtPartitionStart, shouldCloseFile) : null;
+                    this.reader = needsReader ? createReader(indexEntry, file, true, shouldCloseFile) : null;
                     this.staticRow = readStaticRow(sstable, file, helper, columns.fetchedColumns().statics, isForThrift, reader == null ? null : reader.deserializer);
                 }
                 else
                 {
                     this.partitionLevelDeletion = indexEntry.deletionTime();
                     this.staticRow = Rows.EMPTY_STATIC_ROW;
-                    this.reader = needsReader ? createReader(indexEntry, file, needSeekAtPartitionStart, shouldCloseFile) : null;
+                    this.reader = needsReader ? createReader(indexEntry, file, false, shouldCloseFile) : null;
                 }
 
-                if (reader == null && shouldCloseFile)
+                if (reader == null && file != null && shouldCloseFile)
                     file.close();
             }
             catch (IOException e)
             {
                 sstable.markSuspect();
                 String filePath = file.getPath();
-                if (shouldCloseFile && file != null)
+                if (shouldCloseFile)
                 {
                     try
                     {
@@ -164,7 +162,7 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
             if (statics.isEmpty() || isForThrift)
                 return Rows.EMPTY_STATIC_ROW;
 
-            assert sstable.metadata.isStaticCompactTable() && !isForThrift;
+            assert sstable.metadata.isStaticCompactTable();
 
             // As said above, if it's a CQL query and the table is a "static compact", the only exposed columns are the
             // static ones. So we don't have to mark the position to seek back later.
@@ -221,45 +219,13 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
 
     public boolean hasNext()
     {
-        try
-        {
-            return reader != null && reader.hasNext();
-        }
-        catch (IOException e)
-        {
-            try
-            {
-                closeInternal();
-            }
-            catch (IOException suppressed)
-            {
-                e.addSuppressed(suppressed);
-            }
-            sstable.markSuspect();
-            throw new CorruptSSTableException(e, reader.file.getPath());
-        }
+        return reader != null && reader.hasNext();
     }
 
     public Unfiltered next()
     {
-        try
-        {
-            assert reader != null;
-            return reader.next();
-        }
-        catch (IOException e)
-        {
-            try
-            {
-                closeInternal();
-            }
-            catch (IOException suppressed)
-            {
-                e.addSuppressed(suppressed);
-            }
-            sstable.markSuspect();
-            throw new CorruptSSTableException(e, reader.file.getPath());
-        }
+        assert reader != null;
+        return reader.next();
     }
 
     public Iterator<Unfiltered> slice(Slice slice)
@@ -269,7 +235,8 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
             if (reader == null)
                 return Collections.emptyIterator();
 
-            return reader.slice(slice);
+            reader.setForSlice(slice);
+            return reader;
         }
         catch (IOException e)
         {
@@ -317,7 +284,7 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
         }
     }
 
-    protected abstract class Reader
+    protected abstract class Reader implements Iterator<Unfiltered>
     {
         private final boolean shouldCloseFile;
         public FileDataInput file;
@@ -327,12 +294,19 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
         // Records the currently open range tombstone (if any)
         protected DeletionTime openMarker = null;
 
-        protected Reader(FileDataInput file, boolean shouldCloseFile)
+        // !isInit means we have never seeked in the file and thus should seek before reading anything
+        protected boolean isInit;
+
+        protected Reader(FileDataInput file, boolean isInit, boolean shouldCloseFile)
         {
             this.file = file;
+            this.isInit = isInit;
             this.shouldCloseFile = shouldCloseFile;
+
             if (file != null)
                 createDeserializer();
+            else
+                assert !isInit;
         }
 
         private void createDeserializer()
@@ -369,9 +343,62 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
             return toReturn;
         }
 
-        public abstract boolean hasNext() throws IOException;
-        public abstract Unfiltered next() throws IOException;
-        public abstract Iterator<Unfiltered> slice(Slice slice) throws IOException;
+        public boolean hasNext() 
+        {
+            try
+            {
+                if (!isInit)
+                {
+                    init();
+                    isInit = true;
+                }
+
+                return hasNextInternal();
+            }
+            catch (IOException e)
+            {
+                try
+                {
+                    closeInternal();
+                }
+                catch (IOException suppressed)
+                {
+                    e.addSuppressed(suppressed);
+                }
+                sstable.markSuspect();
+                throw new CorruptSSTableException(e, reader.file.getPath());
+            }
+        }
+
+        public Unfiltered next()
+        {
+            try
+            {
+                return nextInternal();
+            }
+            catch (IOException e)
+            {
+                try
+                {
+                    closeInternal();
+                }
+                catch (IOException suppressed)
+                {
+                    e.addSuppressed(suppressed);
+                }
+                sstable.markSuspect();
+                throw new CorruptSSTableException(e, reader.file.getPath());
+            }
+        }
+
+        // Called is hasNext() is called but we haven't been yet initialized
+        protected abstract void init() throws IOException;
+
+        // Set the reader so its hasNext/next methods return values within the provided slice
+        public abstract void setForSlice(Slice slice) throws IOException;
+
+        protected abstract boolean hasNextInternal() throws IOException;
+        protected abstract Unfiltered nextInternal() throws IOException;
 
         public void close() throws IOException
         {
@@ -380,35 +407,61 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
         }
     }
 
-    protected abstract class IndexedReader extends Reader
+    // Used by indexed readers to store where they are of the index.
+    protected static class IndexState
     {
-        protected final RowIndexEntry indexEntry;
-        protected final List<IndexHelper.IndexInfo> indexes;
+        private final Reader reader;
+        private final ClusteringComparator comparator;
 
-        protected int currentIndexIdx = -1;
+        private final RowIndexEntry indexEntry;
+        private final List<IndexHelper.IndexInfo> indexes;
+        private final boolean reversed;
 
-        // Marks the beginning of the block corresponding to currentIndexIdx.
-        protected FileMark mark;
+        private int currentIndexIdx = -1;
 
-        // !isInit means we have never seeked in the file and thus shouldn't read as we could be anywhere
-        protected boolean isInit;
+        // Marks the beginning of the block corresponding to currentIndexIdx.
+        private FileMark mark;
 
-        protected IndexedReader(FileDataInput file, boolean shouldCloseFile, RowIndexEntry indexEntry, boolean isInit)
+        public IndexState(Reader reader, ClusteringComparator comparator, RowIndexEntry indexEntry, boolean reversed)
         {
-            super(file, shouldCloseFile);
+            this.reader = reader;
+            this.comparator = comparator;
             this.indexEntry = indexEntry;
             this.indexes = indexEntry.columnsIndex();
-            this.isInit = isInit;
+            this.reversed = reversed;
+            this.currentIndexIdx = reversed ? indexEntry.columnsIndex().size() : -1;
+        }
+
+        public boolean isDone()
+        {
+            return reversed ? currentIndexIdx < 0 : currentIndexIdx >= indexes.size();
         }
 
-        // Should be called when we're at the beginning of blockIdx.
-        protected void updateBlock(int blockIdx) throws IOException
+        // Sets the reader to the beginning of blockIdx.
+        public void setToBlock(int blockIdx) throws IOException
         {
-            seekToPosition(indexEntry.position + indexes.get(blockIdx).offset);
+            if (blockIdx >= 0 && blockIdx < indexes.size())
+                reader.seekToPosition(indexEntry.position + indexes.get(blockIdx).offset);
 
             currentIndexIdx = blockIdx;
-            openMarker = blockIdx > 0 ? indexes.get(blockIdx - 1).endOpenMarker : null;
-            mark = file.mark();
+            reader.openMarker = blockIdx > 0 ? indexes.get(blockIdx - 1).endOpenMarker : null;
+            mark = reader.file.mark();
+        }
+
+        public int blocksCount()
+        {
+            return indexes.size();
+        }
+
+        // Check if we've crossed an index boundary (based on the mark on the beginning of the index block).
+        public boolean isPastCurrentBlock()
+        {
+            return currentIndexIdx < indexes.size() && reader.file.bytesPastMark(mark) >= currentIndex().width;
+        }
+
+        public int currentBlockIdx()
+        {
+            return currentIndexIdx;
         }
 
         public IndexHelper.IndexInfo currentIndex()
@@ -416,9 +469,16 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
             return indexes.get(currentIndexIdx);
         }
 
-        public IndexHelper.IndexInfo previousIndex()
+        // Finds the index of the first block containing the provided bound, starting at the current index.
+        // Will be -1 if the bound is before any block, and blocksCount() if it is after every block.
+        public int findBlockIndex(Slice.Bound bound)
         {
-            return currentIndexIdx <= 1 ? null : indexes.get(currentIndexIdx - 1);
+            if (bound == Slice.Bound.BOTTOM)
+                return -1;
+            if (bound == Slice.Bound.TOP)
+                return blocksCount();
+
+            return IndexHelper.indexFor(bound, indexes, comparator, reversed, currentIndexIdx);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
index 4fd5205..a58ea3e 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
@@ -18,30 +18,19 @@
 package org.apache.cassandra.db.columniterator;
 
 import java.io.IOException;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-
-import com.google.common.collect.AbstractIterator;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.util.NoSuchElementException;
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.sstable.CorruptSSTableException;
-import org.apache.cassandra.io.sstable.IndexHelper;
 import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  *  A Cell Iterator over SSTable
  */
 public class SSTableIterator extends AbstractSSTableIterator
 {
-    private static final Logger logger = LoggerFactory.getLogger(SSTableIterator.class);
-
     public SSTableIterator(SSTableReader sstable, DecoratedKey key, ColumnFilter columns, boolean isForThrift)
     {
         this(sstable, null, key, sstable.getPosition(key, SSTableReader.Operator.EQ), columns, isForThrift);
@@ -71,222 +60,215 @@ public class SSTableIterator extends AbstractSSTableIterator
 
     private class ForwardReader extends Reader
     {
+        // The start of the current slice. This will be null as soon as we know we've passed that bound.
+        protected Slice.Bound start;
+        // The end of the current slice. Will never be null.
+        protected Slice.Bound end = Slice.Bound.TOP;
+
+        protected Unfiltered next; // the next element to return: this is computed by hasNextInternal().
+
+        protected boolean sliceDone; // set to true once we know we have no more result for the slice. This is in particular
+                                     // used by the indexed reader when we know we can't have results based on the index.
+
         private ForwardReader(FileDataInput file, boolean isAtPartitionStart, boolean shouldCloseFile)
         {
-            super(file, shouldCloseFile);
-            assert isAtPartitionStart;
+            super(file, isAtPartitionStart, shouldCloseFile);
         }
 
-        public boolean hasNext() throws IOException
+        protected void init() throws IOException
         {
-            assert deserializer != null;
-            return deserializer.hasNext();
+            // We should always have been initialized (at the beginning of the partition). Only indexed readers may
+            // have to initialize.
+            throw new IllegalStateException();
         }
 
-        public Unfiltered next() throws IOException
+        public void setForSlice(Slice slice) throws IOException
         {
-            return deserializer.readNext();
+            start = slice.start() == Slice.Bound.BOTTOM ? null : slice.start();
+            end = slice.end();
+
+            sliceDone = false;
+            next = null;
         }
 
-        public Iterator<Unfiltered> slice(final Slice slice) throws IOException
+        // Skip all data that comes before the currently set slice.
+        // Return what should be returned at the end of this, or null if nothing should.
+        private Unfiltered handlePreSliceData() throws IOException
         {
-            return new AbstractIterator<Unfiltered>()
+            // Note that the following comparison is not strict. The reason is that the only cases
+            // where it can be == is if the "next" is a RT start marker (either a '[' of a ')[' boundary),
+            // and if we had a strict inequality and an open RT marker before this, we would issue
+            // the open marker first, and then return then next later, which would send in the
+            // stream both '[' (or '(') and then ')[' for the same clustering value, which is wrong.
+            // By using a non-strict inequality, we avoid that problem (if we do get ')[' for the same
+            // clustering value than the slice, we'll simply record it in 'openMarker').
+            while (deserializer.hasNext() && deserializer.compareNextTo(start) <= 0)
             {
-                private boolean beforeStart = true;
+                if (deserializer.nextIsRow())
+                    deserializer.skipNext();
+                else
+                    updateOpenMarker((RangeTombstoneMarker)deserializer.readNext());
+            }
+
+            Slice.Bound sliceStart = start;
+            start = null;
+
+            // We've reached the beginning of our queried slice. If we have an open marker
+            // we should return that first.
+            if (openMarker != null)
+                return new RangeTombstoneBoundMarker(sliceStart, openMarker);
 
-                protected Unfiltered computeNext()
+            return null;
+        }
+
+        // Compute the next element to return, assuming we're in the middle to the slice
+        // and the next element is either in the slice, or just after it. Returns null
+        // if we're done with the slice.
+        protected Unfiltered computeNext() throws IOException
+        {
+            if (!deserializer.hasNext() || deserializer.compareNextTo(end) > 0)
+                return null;
+
+            Unfiltered next = deserializer.readNext();
+            if (next.kind() == Unfiltered.Kind.RANGE_TOMBSTONE_MARKER)
+                updateOpenMarker((RangeTombstoneMarker)next);
+            return next;
+        }
+
+        protected boolean hasNextInternal() throws IOException
+        {
+            if (next != null)
+                return true;
+
+            if (sliceDone)
+                return false;
+
+            assert deserializer != null;
+
+            if (start != null)
+            {
+                Unfiltered unfiltered = handlePreSliceData();
+                if (unfiltered != null)
                 {
-                    try
-                    {
-                        // While we're before the start of the slice, we can skip row but we should keep
-                        // track of open range tombstones
-                        if (beforeStart)
-                        {
-                            // Note that the following comparison is not strict. The reason is that the only cases
-                            // where it can be == is if the "next" is a RT start marker (either a '[' of a ')[' boundary),
-                            // and if we had a strict inequality and an open RT marker before this, we would issue
-                            // the open marker first, and then return then next later, which would yet in the
-                            // stream both '[' (or '(') and then ')[' for the same clustering value, which is wrong.
-                            // By using a non-strict inequality, we avoid that problem (if we do get ')[' for the same
-                            // clustering value than the slice, we'll simply record it in 'openMarker').
-                            while (deserializer.hasNext() && deserializer.compareNextTo(slice.start()) <= 0)
-                            {
-                                if (deserializer.nextIsRow())
-                                    deserializer.skipNext();
-                                else
-                                    updateOpenMarker((RangeTombstoneMarker)deserializer.readNext());
-                            }
-
-                            beforeStart = false;
-
-                            // We've reached the beginning of our queried slice. If we have an open marker
-                            // we should return that first.
-                            if (openMarker != null)
-                                return new RangeTombstoneBoundMarker(slice.start(), openMarker);
-                        }
-
-                        if (deserializer.hasNext() && deserializer.compareNextTo(slice.end()) <= 0)
-                        {
-                            Unfiltered next = deserializer.readNext();
-                            if (next.kind() == Unfiltered.Kind.RANGE_TOMBSTONE_MARKER)
-                                updateOpenMarker((RangeTombstoneMarker)next);
-                            return next;
-                        }
-
-                        // If we have an open marker, we should close it before finishing
-                        if (openMarker != null)
-                            return new RangeTombstoneBoundMarker(slice.end(), getAndClearOpenMarker());
-
-                        return endOfData();
-                    }
-                    catch (IOException e)
-                    {
-                        try
-                        {
-                            close();
-                        }
-                        catch (IOException suppressed)
-                        {
-                            e.addSuppressed(suppressed);
-                        }
-                        sstable.markSuspect();
-                        throw new CorruptSSTableException(e, file.getPath());
-                    }
+                    next = unfiltered;
+                    return true;
                 }
-            };
+            }
+
+            next = computeNext();
+            if (next != null)
+                return true;
+
+            // If we have an open marker, we should close it before finishing
+            if (openMarker != null)
+            {
+                next = new RangeTombstoneBoundMarker(end, getAndClearOpenMarker());
+                return true;
+            }
+
+            sliceDone = true; // not absolutely necessary but accurate and cheap
+            return false;
+        }
+
+        protected Unfiltered nextInternal() throws IOException
+        {
+            if (!hasNextInternal())
+                throw new NoSuchElementException();
+
+            Unfiltered toReturn = next;
+            next = null;
+            return toReturn;
         }
     }
 
-    private class ForwardIndexedReader extends IndexedReader
+    private class ForwardIndexedReader extends ForwardReader
     {
+        private final IndexState indexState;
+
+        private int lastBlockIdx; // the last index block that has data for the current query
+
         private ForwardIndexedReader(RowIndexEntry indexEntry, FileDataInput file, boolean isAtPartitionStart, boolean shouldCloseFile)
         {
-            super(file, shouldCloseFile, indexEntry, isAtPartitionStart);
+            super(file, isAtPartitionStart, shouldCloseFile);
+            this.indexState = new IndexState(this, sstable.metadata.comparator, indexEntry, false);
+            this.lastBlockIdx = indexState.blocksCount(); // if we never call setForSlice, that's where we want to stop
         }
 
-        public boolean hasNext() throws IOException
+        @Override
+        protected void init() throws IOException
         {
-            // If it's called before we've created the file, create it. This then mean
-            // we're reading from the beginning of the partition.
-            if (!isInit)
-            {
-                seekToPosition(indexEntry.position);
-                ByteBufferUtil.skipShortLength(file); // partition key
-                DeletionTime.serializer.skip(file);   // partition deletion
-                if (sstable.header.hasStatic())
-                    UnfilteredSerializer.serializer.skipStaticRow(file, sstable.header, helper);
-                isInit = true;
-            }
-            return deserializer.hasNext();
+            // If this is called, it means we're calling hasNext() before any call to setForSlice. Which means
+            // we're reading everything from the beginning. So just set us up at the beginning of the first block.
+            indexState.setToBlock(0);
         }
 
-        public Unfiltered next() throws IOException
+        @Override
+        public void setForSlice(Slice slice) throws IOException
         {
-            return deserializer.readNext();
-        }
+            super.setForSlice(slice);
 
-        public Iterator<Unfiltered> slice(final Slice slice) throws IOException
-        {
-            final List<IndexHelper.IndexInfo> indexes = indexEntry.columnsIndex();
+            isInit = true;
 
             // if our previous slicing already got us the biggest row in the sstable, we're done
-            if (currentIndexIdx >= indexes.size())
-                return Collections.emptyIterator();
+            if (indexState.isDone())
+            {
+                sliceDone = true;
+                return;
+            }
 
             // Find the first index block we'll need to read for the slice.
-            final int startIdx = IndexHelper.indexFor(slice.start(), indexes, sstable.metadata.comparator, false, currentIndexIdx);
-            if (startIdx >= indexes.size())
-                return Collections.emptyIterator();
+            int startIdx = indexState.findBlockIndex(slice.start());
+            if (startIdx >= indexState.blocksCount())
+            {
+                sliceDone = true;
+                return;
+            }
 
             // If that's the last block we were reading, we're already where we want to be. Otherwise,
             // seek to that first block
-            if (startIdx != currentIndexIdx)
-                updateBlock(startIdx);
+            if (startIdx != indexState.currentBlockIdx())
+                indexState.setToBlock(startIdx);
 
             // Find the last index block we'll need to read for the slice.
-            final int endIdx = IndexHelper.indexFor(slice.end(), indexes, sstable.metadata.comparator, false, startIdx);
-
-            final IndexHelper.IndexInfo startIndex = currentIndex();
+            lastBlockIdx = indexState.findBlockIndex(slice.end());
 
             // The index search is based on the last name of the index blocks, so at that point we have that:
-            //   1) indexes[startIdx - 1].lastName < slice.start <= indexes[startIdx].lastName
-            //   2) indexes[endIdx - 1].lastName < slice.end <= indexes[endIdx].lastName
-            // so if startIdx == endIdx and slice.end < indexes[startIdx].firstName, we're guaranteed that the
-            // whole slice is between the previous block end and this bloc start, and thus has no corresponding
+            //   1) indexes[currentIdx - 1].lastName < slice.start <= indexes[currentIdx].lastName
+            //   2) indexes[lastBlockIdx - 1].lastName < slice.end <= indexes[lastBlockIdx].lastName
+            // so if currentIdx == lastBlockIdx and slice.end < indexes[currentIdx].firstName, we're guaranteed that the
+            // whole slice is between the previous block end and this block start, and thus has no corresponding
             // data. One exception is if the previous block ends with an openMarker as it will cover our slice
             // and we need to return it.
-            if (startIdx == endIdx && metadata().comparator.compare(slice.end(), startIndex.firstName) < 0 && openMarker == null && sstable.descriptor.version.storeRows())
-                return Collections.emptyIterator();
-
-            return new AbstractIterator<Unfiltered>()
+            if (indexState.currentBlockIdx() == lastBlockIdx
+                && metadata().comparator.compare(slice.end(), indexState.currentIndex().firstName) < 0
+                && openMarker == null
+                && sstable.descriptor.version.storeRows())
             {
-                private boolean beforeStart = true;
-                private int currentIndexIdx = startIdx;
+                sliceDone = true;
+            }
+        }
 
-                protected Unfiltered computeNext()
-                {
-                    try
-                    {
-                        // While we're before the start of the slice, we can skip row but we should keep
-                        // track of open range tombstones
-                        if (beforeStart)
-                        {
-                            // See ForwardReader equivalent method to see why this inequality is not strict.
-                            while (deserializer.hasNext() && deserializer.compareNextTo(slice.start()) <= 0)
-                            {
-                                if (deserializer.nextIsRow())
-                                    deserializer.skipNext();
-                                else
-                                    updateOpenMarker((RangeTombstoneMarker)deserializer.readNext());
-                            }
-
-                            beforeStart = false;
-
-                            // We've reached the beginning of our queried slice. If we have an open marker
-                            // we should return that first.
-                            if (openMarker != null)
-                                return new RangeTombstoneBoundMarker(slice.start(), openMarker);
-                        }
-
-                        // If we've crossed an index block boundary, update our informations
-                        if (currentIndexIdx < indexes.size() && file.bytesPastMark(mark) >= currentIndex().width)
-                            updateBlock(++currentIndexIdx);
-
-                        // Return the next atom unless we've reached the end, or we're beyond our slice
-                        // end (note that unless we're on the last block for the slice, there is no point
-                        // in checking the slice end).
-                        if (currentIndexIdx < indexes.size()
-                            && currentIndexIdx <= endIdx
-                            && deserializer.hasNext()
-                            && (currentIndexIdx != endIdx || deserializer.compareNextTo(slice.end()) <= 0))
-                        {
-                            Unfiltered next = deserializer.readNext();
-                            if (next.kind() == Unfiltered.Kind.RANGE_TOMBSTONE_MARKER)
-                                updateOpenMarker((RangeTombstoneMarker)next);
-                            return next;
-                        }
-
-                        // If we have an open marker, we should close it before finishing
-                        if (openMarker != null)
-                            return new RangeTombstoneBoundMarker(slice.end(), getAndClearOpenMarker());
-
-                        return endOfData();
-                    }
-                    catch (IOException e)
-                    {
-                        try
-                        {
-                            close();
-                        }
-                        catch (IOException suppressed)
-                        {
-                            e.addSuppressed(suppressed);
-                        }
-                        sstable.markSuspect();
-                        throw new CorruptSSTableException(e, file.getPath());
-                    }
-                }
-            };
+        @Override
+        protected Unfiltered computeNext() throws IOException
+        {
+            // Our previous read might have made us cross an index block boundary. If so, update our informations.
+            if (indexState.isPastCurrentBlock())
+                indexState.setToBlock(indexState.currentBlockIdx() + 1);
+
+            // Return the next unfiltered unless we've reached the end, or we're beyond our slice
+            // end (note that unless we're on the last block for the slice, there is no point
+            // in checking the slice end).
+            if (indexState.isDone()
+                || indexState.currentBlockIdx() > lastBlockIdx
+                || !deserializer.hasNext()
+                || (indexState.currentBlockIdx() == lastBlockIdx && deserializer.compareNextTo(end) > 0))
+                return null;
+
+
+            Unfiltered next = deserializer.readNext();
+            if (next.kind() == Unfiltered.Kind.RANGE_TOMBSTONE_MARKER)
+                updateOpenMarker((RangeTombstoneMarker)next);
+            return next;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
index 0e18d4a..e15d330 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
@@ -20,29 +20,19 @@ package org.apache.cassandra.db.columniterator;
 import java.io.IOException;
 import java.util.*;
 
-import com.google.common.collect.AbstractIterator;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.rows.*;
-import org.apache.cassandra.db.partitions.AbstractPartitionData;
+import org.apache.cassandra.db.partitions.AbstractThreadUnsafePartition;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.sstable.CorruptSSTableException;
-import org.apache.cassandra.io.sstable.IndexHelper;
 import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileMark;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  *  A Cell Iterator in reversed clustering order over SSTable
  */
 public class SSTableReversedIterator extends AbstractSSTableIterator
 {
-    private static final Logger logger = LoggerFactory.getLogger(SSTableReversedIterator.class);
-
     public SSTableReversedIterator(SSTableReader sstable, DecoratedKey key, ColumnFilter columns, boolean isForThrift)
     {
         this(sstable, null, key, sstable.getPosition(key, SSTableReader.Operator.EQ), columns, isForThrift);
@@ -70,319 +60,290 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
         return true;
     }
 
-    private ReusablePartitionData createBuffer(int blocksCount)
+    private class ReverseReader extends Reader
     {
-        int estimatedRowCount = 16;
-        int columnCount = metadata().partitionColumns().regulars.columnCount();
-        if (columnCount == 0 || metadata().clusteringColumns().size() == 0)
+        protected ReusablePartitionData buffer;
+        protected Iterator<Unfiltered> iterator;
+
+        private ReverseReader(FileDataInput file, boolean isAtPartitionStart, boolean shouldCloseFile)
         {
-            estimatedRowCount = 1;
+            super(file, isAtPartitionStart, shouldCloseFile);
         }
-        else
+
+        protected ReusablePartitionData createBuffer(int blocksCount)
         {
-            try
+            int estimatedRowCount = 16;
+            int columnCount = metadata().partitionColumns().regulars.columnCount();
+            if (columnCount == 0 || metadata().clusteringColumns().isEmpty())
             {
-                // To avoid wasted resizing we guess-estimate the number of rows we're likely to read. For that
-                // we use the stats on the number of rows per partition for that sstable.
-                // FIXME: so far we only keep stats on cells, so to get a rough estimate on the number of rows,
-                // we divide by the number of regular columns the table has. We should fix once we collect the
-                // stats on rows
-                int estimatedRowsPerPartition = (int)(sstable.getEstimatedColumnCount().percentile(0.75) / columnCount);
-                estimatedRowCount = Math.max(estimatedRowsPerPartition / blocksCount, 1);
+                estimatedRowCount = 1;
             }
-            catch (IllegalStateException e)
+            else
             {
-                // The EstimatedHistogram mean() method can throw this (if it overflows). While such overflow
-                // shouldn't happen, it's not worth taking the risk of letting the exception bubble up.
+                try
+                {
+                    // To avoid wasted resizing we guess-estimate the number of rows we're likely to read. For that
+                    // we use the stats on the number of rows per partition for that sstable.
+                    // FIXME: so far we only keep stats on cells, so to get a rough estimate on the number of rows,
+                    // we divide by the number of regular columns the table has. We should fix once we collect the
+                    // stats on rows
+                    int estimatedRowsPerPartition = (int)(sstable.getEstimatedColumnCount().percentile(0.75) / columnCount);
+                    estimatedRowCount = Math.max(estimatedRowsPerPartition / blocksCount, 1);
+                }
+                catch (IllegalStateException e)
+                {
+                    // The EstimatedHistogram mean() method can throw this (if it overflows). While such overflow
+                    // shouldn't happen, it's not worth taking the risk of letting the exception bubble up.
+                }
             }
+            return new ReusablePartitionData(metadata(), partitionKey(), columns(), estimatedRowCount);
         }
-        return new ReusablePartitionData(metadata(), partitionKey(), DeletionTime.LIVE, columns(), estimatedRowCount);
-    }
-
-    private class ReverseReader extends Reader
-    {
-        private ReusablePartitionData partition;
-        private UnfilteredRowIterator iterator;
 
-        private ReverseReader(FileDataInput file, boolean isAtPartitionStart, boolean shouldCloseFile)
+        protected void init() throws IOException
         {
-            super(file, shouldCloseFile);
-            assert isAtPartitionStart;
+            // We should always have been initialized (at the beginning of the partition). Only indexed readers may
+            // have to initialize.
+            throw new IllegalStateException();
         }
 
-        public boolean hasNext() throws IOException
+        public void setForSlice(Slice slice) throws IOException
         {
-            if (partition == null)
+            // If we have read the data, just create the iterator for the slice. Otherwise, read the data.
+            if (buffer == null)
             {
-                partition = createBuffer(1);
-                partition.populateFrom(this, null, null, new Tester()
-                {
-                    public boolean isDone()
-                    {
-                        return false;
-                    }
-                });
-                iterator = partition.unfilteredIterator(columns, Slices.ALL, true);
+                buffer = createBuffer(1);
+                // Note that we can reuse that buffer between slices (we could alternatively re-read from disk
+                // every time, but that feels more wasteful) so we want to include everything from the beginning.
+                // We can stop at the slice end however since any following slice will be before that.
+                loadFromDisk(null, slice.end());
             }
+            setIterator(slice);
+        }
+
+        protected void setIterator(Slice slice)
+        {
+            assert buffer != null;
+            iterator = buffer.unfilteredIterator(columns, Slices.with(metadata().comparator, slice), true);
+        }
+
+        protected boolean hasNextInternal() throws IOException
+        {
+            // If we've never called setForSlice, we're reading everything
+            if (iterator == null)
+                setForSlice(Slice.ALL);
+
             return iterator.hasNext();
         }
 
-        public Unfiltered next() throws IOException
+        protected Unfiltered nextInternal() throws IOException
         {
             if (!hasNext())
                 throw new NoSuchElementException();
             return iterator.next();
         }
 
-        public Iterator<Unfiltered> slice(final Slice slice) throws IOException
+        protected boolean stopReadingDisk()
         {
-            if (partition == null)
+            return false;
+        }
+
+        // Reads the unfiltered from disk and load them into the reader buffer. It stops reading when either the partition
+        // is fully read, or when stopReadingDisk() returns true.
+        protected void loadFromDisk(Slice.Bound start, Slice.Bound end) throws IOException
+        {
+            buffer.reset();
+
+            // If the start might be in this block, skip everything that comes before it.
+            if (start != null)
             {
-                partition = createBuffer(1);
-                partition.populateFrom(this, slice.start(), slice.end(), new Tester()
+                while (deserializer.hasNext() && deserializer.compareNextTo(start) <= 0 && !stopReadingDisk())
                 {
-                    public boolean isDone()
-                    {
-                        return false;
-                    }
-                });
+                    if (deserializer.nextIsRow())
+                        deserializer.skipNext();
+                    else
+                        updateOpenMarker((RangeTombstoneMarker)deserializer.readNext());
+                }
             }
 
-            return partition.unfilteredIterator(columns, Slices.with(metadata().comparator, slice), true);
-        }
-    }
-
-    private class ReverseIndexedReader extends IndexedReader
-    {
-        private ReusablePartitionData partition;
-        private UnfilteredRowIterator iterator;
-
-        private ReverseIndexedReader(RowIndexEntry indexEntry, FileDataInput file, boolean isAtPartitionStart, boolean shouldCloseFile)
-        {
-            super(file, shouldCloseFile, indexEntry, isAtPartitionStart);
-            this.currentIndexIdx = indexEntry.columnsIndex().size();
-        }
+            // If we have an open marker, it's either one from what we just skipped (if start != null), or it's from the previous index block.
+            if (openMarker != null)
+            {
+                RangeTombstone.Bound markerStart = start == null ? RangeTombstone.Bound.BOTTOM : RangeTombstone.Bound.fromSliceBound(start);
+                buffer.add(new RangeTombstoneBoundMarker(markerStart, openMarker));
+            }
 
-        public boolean hasNext() throws IOException
-        {
-            // If it's called before we've created the file, create it. This then mean
-            // we're reading from the end of the partition.
-            if (!isInit)
+            // Now deserialize everything until we reach our requested end (if we have one)
+            while (deserializer.hasNext()
+                   && (end == null || deserializer.compareNextTo(end) <= 0)
+                   && !stopReadingDisk())
             {
-                seekToPosition(indexEntry.position);
-                ByteBufferUtil.skipShortLength(file); // partition key
-                DeletionTime.serializer.skip(file);   // partition deletion
-                if (sstable.header.hasStatic())
-                    UnfilteredSerializer.serializer.skipStaticRow(file, sstable.header, helper);
-                isInit = true;
+                Unfiltered unfiltered = deserializer.readNext();
+                buffer.add(unfiltered);
+
+                if (unfiltered.isRangeTombstoneMarker())
+                    updateOpenMarker((RangeTombstoneMarker)unfiltered);
             }
 
-            if (partition == null)
+            // If we have an open marker, we should close it before finishing
+            if (openMarker != null)
             {
-                partition = createBuffer(indexes.size());
-                partition.populateFrom(this, null, null, new Tester()
-                {
-                    public boolean isDone()
-                    {
-                        return false;
-                    }
-                });
-                iterator = partition.unfilteredIterator(columns, Slices.ALL, true);
+                // If we have no end and still an openMarker, this means we're indexed and the marker is closed in a following block.
+                RangeTombstone.Bound markerEnd = end == null ? RangeTombstone.Bound.TOP : RangeTombstone.Bound.fromSliceBound(end);
+                buffer.add(new RangeTombstoneBoundMarker(markerEnd, getAndClearOpenMarker()));
             }
 
-            return iterator.hasNext();
+            buffer.build();
         }
+    }
+
+    private class ReverseIndexedReader extends ReverseReader
+    {
+        private final IndexState indexState;
 
-        public Unfiltered next() throws IOException
+        // The slice we're currently iterating over
+        private Slice slice;
+        // The last index block to consider for the slice
+        private int lastBlockIdx;
+
+        private ReverseIndexedReader(RowIndexEntry indexEntry, FileDataInput file, boolean isAtPartitionStart, boolean shouldCloseFile)
         {
-            if (!hasNext())
-                throw new NoSuchElementException();
-            return iterator.next();
+            super(file, isAtPartitionStart, shouldCloseFile);
+            this.indexState = new IndexState(this, sstable.metadata.comparator, indexEntry, true);
         }
 
-        private void prepareBlock(int blockIdx, Slice.Bound start, Slice.Bound end) throws IOException
+        protected void init() throws IOException
         {
-            updateBlock(blockIdx);
-
-            if (partition == null)
-                partition = createBuffer(indexes.size());
-            else
-                partition.clear();
-
-            final FileMark fileMark = mark;
-            final long width = currentIndex().width;
-
-            partition.populateFrom(this, start, end, new Tester()
-            {
-                public boolean isDone()
-                {
-                    return file.bytesPastMark(fileMark) >= width;
-                }
-            });
+            // If this is called, it means we're calling hasNext() before any call to setForSlice. Which means
+            // we're reading everything from the end. So just set us up on the last block.
+            indexState.setToBlock(indexState.blocksCount() - 1);
         }
 
         @Override
-        public Iterator<Unfiltered> slice(final Slice slice) throws IOException
+        public void setForSlice(Slice slice) throws IOException
         {
-            // if our previous slicing already got us the smallest row in the sstable, we're done
-            if (currentIndexIdx < 0)
-                return Collections.emptyIterator();
+            this.slice = slice;
+            isInit = true;
 
-            final List<IndexHelper.IndexInfo> indexes = indexEntry.columnsIndex();
+            // if our previous slicing already got us pas the beginning of the sstable, we're done
+            if (indexState.isDone())
+            {
+                iterator = Collections.emptyIterator();
+                return;
+            }
 
             // Find the first index block we'll need to read for the slice.
-            final int startIdx = IndexHelper.indexFor(slice.end(), indexes, sstable.metadata.comparator, true, currentIndexIdx);
+            int startIdx = indexState.findBlockIndex(slice.end());
             if (startIdx < 0)
-                return Collections.emptyIterator();
+            {
+                iterator = Collections.emptyIterator();
+                return;
+            }
 
-            // Find the last index block we'll need to read for the slice.
-            int lastIdx = IndexHelper.indexFor(slice.start(), indexes, sstable.metadata.comparator, true, startIdx);
+            boolean isCurrentBlock = startIdx == indexState.currentBlockIdx();
+            if (!isCurrentBlock)
+                indexState.setToBlock(startIdx);
 
-            // The index search is by firstname and so lastIdx is such that
-            //   indexes[lastIdx].firstName < slice.start <= indexes[lastIdx + 1].firstName
-            // However, if indexes[lastIdx].lastName < slice.start we can bump lastIdx.
-            if (lastIdx >= 0 && metadata().comparator.compare(indexes.get(lastIdx).lastName, slice.start()) < 0)
-                ++lastIdx;
+            lastBlockIdx = indexState.findBlockIndex(slice.start());
 
-            final int endIdx = lastIdx;
+            if (!isCurrentBlock)
+                readCurrentBlock(true);
 
-            // Because we're reversed, even if it is our current block, we should re-prepare the block since we would
-            // have skipped anything not in the previous slice.
-            prepareBlock(startIdx, slice.start(), slice.end());
+            setIterator(slice);
+        }
 
-            return new AbstractIterator<Unfiltered>()
-            {
-                private Iterator<Unfiltered> currentBlockIterator = partition.unfilteredIterator(columns, Slices.with(metadata().comparator, slice), true);
+        @Override
+        protected boolean hasNextInternal() throws IOException
+        {
+            if (super.hasNextInternal())
+                return true;
+
+            // We have nothing more for our current block, move the previous one.
+            int previousBlockIdx = indexState.currentBlockIdx() - 1;
+            if (previousBlockIdx < 0 || previousBlockIdx < lastBlockIdx)
+                return false;
+
+            // The slice start can be in 
+            indexState.setToBlock(previousBlockIdx);
+            readCurrentBlock(false);
+            setIterator(slice);
+            // since that new block is within the bounds we've computed in setToSlice(), we know there will
+            // always be something matching the slice unless we're on the lastBlockIdx (in which case there
+            // may or may not be results, but if there isn't, we're done for the slice).
+            return iterator.hasNext();
+        }
 
-                protected Unfiltered computeNext()
-                {
-                    try
-                    {
-                        if (currentBlockIterator.hasNext())
-                            return currentBlockIterator.next();
-
-                        --currentIndexIdx;
-                        if (currentIndexIdx < 0 || currentIndexIdx < endIdx)
-                            return endOfData();
-
-                        // Note that since we know we're read blocks backward, there is no point in checking the slice end, so we pass null
-                        prepareBlock(currentIndexIdx, slice.start(), null);
-                        currentBlockIterator = partition.unfilteredIterator(columns, Slices.with(metadata().comparator, slice), true);
-                        return computeNext();
-                    }
-                    catch (IOException e)
-                    {
-                        try
-                        {
-                            close();
-                        }
-                        catch (IOException suppressed)
-                        {
-                            e.addSuppressed(suppressed);
-                        }
-                        sstable.markSuspect();
-                        throw new CorruptSSTableException(e, file.getPath());
-                    }
-                }
-            };
+        /**
+         * Reads the current block, the last one we've set.
+         *
+         * @param canIncludeSliceEnd whether the block can include the slice end.
+         */
+        private void readCurrentBlock(boolean canIncludeSliceEnd) throws IOException
+        {
+            if (buffer == null)
+                buffer = createBuffer(indexState.blocksCount());
+
+            boolean canIncludeSliceStart = indexState.currentBlockIdx() == lastBlockIdx;
+            loadFromDisk(canIncludeSliceStart ? slice.start() : null, canIncludeSliceEnd ? slice.end() : null);
         }
-    }
 
-    private abstract class Tester
-    {
-        public abstract boolean isDone();
+        @Override
+        protected boolean stopReadingDisk()
+        {
+            return indexState.isPastCurrentBlock();
+        }
     }
 
-    private class ReusablePartitionData extends AbstractPartitionData
+    private class ReusablePartitionData extends AbstractThreadUnsafePartition
     {
-        private final Writer rowWriter;
-        private final RangeTombstoneCollector markerWriter;
+        private MutableDeletionInfo.Builder deletionBuilder;
+        private MutableDeletionInfo deletionInfo;
 
         private ReusablePartitionData(CFMetaData metadata,
                                       DecoratedKey partitionKey,
-                                      DeletionTime deletionTime,
                                       PartitionColumns columns,
                                       int initialRowCapacity)
         {
-            super(metadata, partitionKey, deletionTime, columns, initialRowCapacity, false);
+            super(metadata, partitionKey, columns, new ArrayList<>(initialRowCapacity));
+        }
 
-            this.rowWriter = new Writer(true);
-            // Note that even though the iterator handles the reverse case, this object holds the data for a single index bock, and we read index blocks in
-            // forward clustering order.
-            this.markerWriter = new RangeTombstoneCollector(false);
+        public DeletionInfo deletionInfo()
+        {
+            return deletionInfo;
         }
 
-        // Note that this method is here rather than in the readers because we want to use it for both readers and they
-        // don't extend one another
-        private void populateFrom(Reader reader, Slice.Bound start, Slice.Bound end, Tester tester) throws IOException
+        protected boolean canHaveShadowedData()
         {
-            // If we have a start bound, skip everything that comes before it.
-            while (reader.deserializer.hasNext() && start != null && reader.deserializer.compareNextTo(start) <= 0 && !tester.isDone())
-            {
-                if (reader.deserializer.nextIsRow())
-                    reader.deserializer.skipNext();
-                else
-                    reader.updateOpenMarker((RangeTombstoneMarker)reader.deserializer.readNext());
-            }
+            return false;
+        }
 
-            // If we have an open marker, it's either one from what we just skipped (if start != null), or it's from the previous index block.
-            if (reader.openMarker != null)
-            {
-                // If we have no start but still an openMarker, this means we're indexed and it's coming from the previous block
-                Slice.Bound markerStart = start;
-                if (start == null)
-                {
-                    ClusteringPrefix c = ((IndexedReader)reader).previousIndex().lastName;
-                    markerStart = Slice.Bound.exclusiveStartOf(c);
-                }
-                writeMarker(markerStart, reader.openMarker);
-            }
+        public Row staticRow()
+        {
+            return Rows.EMPTY_STATIC_ROW; // we don't actually use that
+        }
 
-            // Now deserialize everything until we reach our requested end (if we have one)
-            while (reader.deserializer.hasNext()
-                   && (end == null || reader.deserializer.compareNextTo(end) <= 0)
-                   && !tester.isDone())
-            {
-                Unfiltered unfiltered = reader.deserializer.readNext();
-                if (unfiltered.kind() == Unfiltered.Kind.ROW)
-                {
-                    ((Row) unfiltered).copyTo(rowWriter);
-                }
-                else
-                {
-                    RangeTombstoneMarker marker = (RangeTombstoneMarker) unfiltered;
-                    reader.updateOpenMarker(marker);
-                    marker.copyTo(markerWriter);
-                }
-            }
+        public RowStats stats()
+        {
+            return RowStats.NO_STATS; // we don't actually use that
+        }
 
-            // If we have an open marker, we should close it before finishing
-            if (reader.openMarker != null)
-            {
-                // If we no end and still an openMarker, this means we're indexed and the marker can be close using the blocks end
-                Slice.Bound markerEnd = end;
-                if (end == null)
-                {
-                    ClusteringPrefix c = ((IndexedReader)reader).currentIndex().lastName;
-                    markerEnd = Slice.Bound.inclusiveEndOf(c);
-                }
-                writeMarker(markerEnd, reader.getAndClearOpenMarker());
-            }
+        public void add(Unfiltered unfiltered)
+        {
+            if (unfiltered.isRow())
+                rows.add((Row)unfiltered);
+            else
+                deletionBuilder.add((RangeTombstoneMarker)unfiltered);
         }
 
-        private void writeMarker(Slice.Bound bound, DeletionTime dt)
+        public void reset()
         {
-            bound.writeTo(markerWriter);
-            markerWriter.writeBoundDeletion(dt);
-            markerWriter.endOfMarker();
+            rows.clear();
+            deletionBuilder = MutableDeletionInfo.builder(partitionLevelDeletion, metadata().comparator, false);
         }
 
-        @Override
-        public void clear()
+        public void build()
         {
-            super.clear();
-            rowWriter.reset();
-            markerWriter.reset();
+            deletionInfo = deletionBuilder.build();
+            deletionBuilder = null;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
index b3cb370..0149582 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
@@ -26,7 +26,6 @@ import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
-import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.metrics.CompactionMetrics;
 
 /**
@@ -34,7 +33,7 @@ import org.apache.cassandra.metrics.CompactionMetrics;
  * <p>
  * On top of the actual merging the source iterators, this class:
  * <ul>
- *   <li>purge gc-able tombstones if possible (see PurgingPartitionIterator below).</li>
+ *   <li>purge gc-able tombstones if possible (see PurgeIterator below).</li>
  *   <li>update 2ndary indexes if necessary (as we don't read-before-write on index updates, index entries are
  *       not deleted on deletion of the base table data, which is ok because we'll fix index inconsistency
  *       on reads. This however mean that potentially obsolete index entries could be kept a long time for
@@ -65,12 +64,9 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
      */
     private final long[] mergeCounters;
 
-    private final UnfilteredPartitionIterator mergedIterator;
+    private final UnfilteredPartitionIterator compacted;
     private final CompactionMetrics metrics;
 
-    // The number of row/RT merged by the iterator
-    private int merged;
-
     public CompactionIterator(OperationType type, List<ISSTableScanner> scanners, CompactionController controller, int nowInSec, UUID compactionId)
     {
         this(type, scanners, controller, nowInSec, compactionId, null);
@@ -96,9 +92,9 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
         if (metrics != null)
             metrics.beginCompaction(this);
 
-        this.mergedIterator = scanners.isEmpty()
-                            ? UnfilteredPartitionIterators.EMPTY
-                            : UnfilteredPartitionIterators.convertExpiredCellsToTombstones(new PurgingPartitionIterator(UnfilteredPartitionIterators.merge(scanners, nowInSec, listener()), controller), nowInSec);
+        this.compacted = scanners.isEmpty()
+                       ? UnfilteredPartitionIterators.EMPTY
+                       : new PurgeIterator(UnfilteredPartitionIterators.merge(scanners, nowInSec, listener()), controller);
     }
 
     public boolean isForThrift()
@@ -143,57 +139,46 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
 
                 CompactionIterator.this.updateCounterFor(merged);
 
-                /*
-                 * The row level listener does 2 things:
-                 *  - It updates 2ndary indexes for deleted/shadowed cells
-                 *  - It updates progress regularly (every UNFILTERED_TO_UPDATE_PROGRESS)
-                 */
-                final SecondaryIndexManager.Updater indexer = type == OperationType.COMPACTION
-                                                            ? controller.cfs.indexManager.gcUpdaterFor(partitionKey, nowInSec)
-                                                            : SecondaryIndexManager.nullUpdater;
+                if (type != OperationType.COMPACTION || !controller.cfs.indexManager.hasIndexes())
+                    return null;
 
-                return new UnfilteredRowIterators.MergeListener()
+                // If we have a 2ndary index, we must update it with deleted/shadowed cells.
+                // TODO: this should probably be done asynchronously and batched.
+                final SecondaryIndexManager.Updater indexer = controller.cfs.indexManager.gcUpdaterFor(partitionKey, nowInSec);
+                final RowDiffListener diffListener = new RowDiffListener()
                 {
-                    private Clustering clustering;
+                    public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original)
+                    {
+                    }
 
-                    public void onMergePartitionLevelDeletion(DeletionTime mergedDeletion, DeletionTime[] versions)
+                    public void onDeletion(int i, Clustering clustering, DeletionTime merged, DeletionTime original)
                     {
                     }
 
-                    public void onMergingRows(Clustering clustering, LivenessInfo mergedInfo, DeletionTime mergedDeletion, Row[] versions)
+                    public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
                     {
-                        this.clustering = clustering;
                     }
 
-                    public void onMergedComplexDeletion(ColumnDefinition c, DeletionTime mergedCompositeDeletion, DeletionTime[] versions)
+                    public void onCell(int i, Clustering clustering, Cell merged, Cell original)
                     {
+                        if (original != null && (merged == null || !merged.isLive(nowInSec)))
+                            indexer.remove(clustering, original);
                     }
+                };
 
-                    public void onMergedCells(Cell mergedCell, Cell[] versions)
+                return new UnfilteredRowIterators.MergeListener()
+                {
+                    public void onMergedPartitionLevelDeletion(DeletionTime mergedDeletion, DeletionTime[] versions)
                     {
-                        if (indexer == SecondaryIndexManager.nullUpdater)
-                            return;
-
-                        for (int i = 0; i < versions.length; i++)
-                        {
-                            Cell version = versions[i];
-                            if (version != null && (mergedCell == null || !mergedCell.equals(version)))
-                                indexer.remove(clustering, version);
-                        }
                     }
 
-                    public void onRowDone()
+                    public void onMergedRows(Row merged, Columns columns, Row[] versions)
                     {
-                        int merged = ++CompactionIterator.this.merged;
-                        if (merged % UNFILTERED_TO_UPDATE_PROGRESS == 0)
-                            updateBytesRead();
+                        Rows.diff(merged, columns, versions, diffListener);
                     }
 
                     public void onMergedRangeTombstoneMarkers(RangeTombstoneMarker mergedMarker, RangeTombstoneMarker[] versions)
                     {
-                        int merged = ++CompactionIterator.this.merged;
-                        if (merged % UNFILTERED_TO_UPDATE_PROGRESS == 0)
-                            updateBytesRead();
                     }
 
                     public void close()
@@ -218,12 +203,12 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
 
     public boolean hasNext()
     {
-        return mergedIterator.hasNext();
+        return compacted.hasNext();
     }
 
     public UnfilteredRowIterator next()
     {
-        return mergedIterator.next();
+        return compacted.next();
     }
 
     public void remove()
@@ -235,7 +220,7 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
     {
         try
         {
-            mergedIterator.close();
+            compacted.close();
         }
         finally
         {
@@ -249,7 +234,7 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
         return this.getCompactionInfo().toString();
     }
 
-    private class PurgingPartitionIterator extends TombstonePurgingPartitionIterator
+    private class PurgeIterator extends PurgingPartitionIterator
     {
         private final CompactionController controller;
 
@@ -257,28 +242,33 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
         private long maxPurgeableTimestamp;
         private boolean hasCalculatedMaxPurgeableTimestamp;
 
-        private PurgingPartitionIterator(UnfilteredPartitionIterator toPurge, CompactionController controller)
+        private long compactedUnfiltered;
+
+        private PurgeIterator(UnfilteredPartitionIterator toPurge, CompactionController controller)
         {
             super(toPurge, controller.gcBefore);
             this.controller = controller;
         }
 
         @Override
-        protected void onEmpty(DecoratedKey key)
+        protected void onEmptyPartitionPostPurge(DecoratedKey key)
         {
             if (type == OperationType.COMPACTION)
                 controller.cfs.invalidateCachedPartition(key);
         }
 
         @Override
-        protected boolean shouldFilter(UnfilteredRowIterator iterator)
+        protected void onNewPartition(DecoratedKey key)
         {
-            currentKey = iterator.partitionKey();
+            currentKey = key;
             hasCalculatedMaxPurgeableTimestamp = false;
+        }
 
-            // TODO: we could be able to skip filtering if UnfilteredRowIterator was giving us some stats
-            // (like the smallest local deletion time).
-            return true;
+        @Override
+        protected void updateProgress()
+        {
+            if ((++compactedUnfiltered) % UNFILTERED_TO_UPDATE_PROGRESS == 0)
+                updateBytesRead();
         }
 
         /*

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/filter/AbstractClusteringIndexFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/AbstractClusteringIndexFilter.java b/src/java/org/apache/cassandra/db/filter/AbstractClusteringIndexFilter.java
index 29ea7fe..ed7584b 100644
--- a/src/java/org/apache/cassandra/db/filter/AbstractClusteringIndexFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/AbstractClusteringIndexFilter.java
@@ -17,13 +17,13 @@
  */
 package org.apache.cassandra.db.filter;
 
-import java.io.DataInput;
 import java.io.IOException;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
 public abstract class AbstractClusteringIndexFilter implements ClusteringIndexFilter
@@ -68,7 +68,7 @@ public abstract class AbstractClusteringIndexFilter implements ClusteringIndexFi
             int i = 0;
             for (ColumnDefinition column : metadata.clusteringColumns())
                 sb.append(i++ == 0 ? "" : ", ").append(column.name).append(column.type instanceof ReversedType ? " ASC" : " DESC");
-            sb.append(")");
+            sb.append(')');
         }
     }
 
@@ -84,7 +84,7 @@ public abstract class AbstractClusteringIndexFilter implements ClusteringIndexFi
             filter.serializeInternal(out, version);
         }
 
-        public ClusteringIndexFilter deserialize(DataInput in, int version, CFMetaData metadata) throws IOException
+        public ClusteringIndexFilter deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException
         {
             Kind kind = Kind.values()[in.readUnsignedByte()];
             boolean reversed = in.readBoolean();
@@ -104,6 +104,6 @@ public abstract class AbstractClusteringIndexFilter implements ClusteringIndexFi
 
     protected static abstract class InternalDeserializer
     {
-        public abstract ClusteringIndexFilter deserialize(DataInput in, int version, CFMetaData metadata, boolean reversed) throws IOException;
+        public abstract ClusteringIndexFilter deserialize(DataInputPlus in, int version, CFMetaData metadata, boolean reversed) throws IOException;
     }
 }


[13/15] cassandra git commit: Simplify some 8099's implementations

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/LivenessInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/LivenessInfo.java b/src/java/org/apache/cassandra/db/LivenessInfo.java
index 89971d1..8f7b1c2 100644
--- a/src/java/org/apache/cassandra/db/LivenessInfo.java
+++ b/src/java/org/apache/cassandra/db/LivenessInfo.java
@@ -17,128 +17,154 @@
  */
 package org.apache.cassandra.db;
 
+import java.util.Objects;
 import java.security.MessageDigest;
 
-import org.apache.cassandra.db.*;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.FBUtilities;
 
 /**
- * Groups the informations necessary to decide the liveness of a given piece of
- * column data.
+ * Stores the information relating to the liveness of the primary key columns of a row.
  * <p>
- * In practice, a {@code LivenessInfo} groups 3 informations:
- *   1) the data timestamp. It is sometimes allowed for a given piece of data to have
- *      no timestamp (for {@link Row#partitionKeyLivenessInfo} more precisely), but if that
- *      is the case it means the data has no liveness info at all.
- *   2) the data ttl if relevant.
- *   2) the data local deletion time if relevant (that is, if either the data has a ttl or is deleted).
+ * A {@code LivenessInfo} can first be empty. If it isn't, it contains at least a timestamp,
+ * which is the timestamp for the row primary key columns. On top of that, the info can be
+ * ttl'ed, in which case the {@code LivenessInfo} also has both a ttl and a local expiration time.
  */
-public interface LivenessInfo extends Aliasable<LivenessInfo>
+public class LivenessInfo
 {
     public static final long NO_TIMESTAMP = Long.MIN_VALUE;
     public static final int NO_TTL = 0;
-    public static final int NO_DELETION_TIME = Integer.MAX_VALUE;
+    public static final int NO_EXPIRATION_TIME = Integer.MAX_VALUE;
 
-    public static final LivenessInfo NONE = new SimpleLivenessInfo(NO_TIMESTAMP, NO_TTL, NO_DELETION_TIME);
+    public static final LivenessInfo EMPTY = new LivenessInfo(NO_TIMESTAMP);
 
-    /**
-     * The timestamp at which the data was inserted or {@link NO_TIMESTAMP}
-     * if it has no timestamp (which may or may not be allowed).
-     *
-     * @return the liveness info timestamp.
-     */
-    public long timestamp();
+    protected final long timestamp;
 
-    /**
-     * Whether this liveness info has a timestamp or not.
-     * <p>
-     * Note that if this return {@code false}, then both {@link #hasTTL} and
-     * {@link #hasLocalDeletionTime} must return {@code false} too.
-     *
-     * @return whether this liveness info has a timestamp or not.
-     */
-    public boolean hasTimestamp();
+    protected LivenessInfo(long timestamp)
+    {
+        this.timestamp = timestamp;
+    }
+
+    public static LivenessInfo create(CFMetaData metadata, long timestamp, int nowInSec)
+    {
+        int defaultTTL = metadata.getDefaultTimeToLive();
+        if (defaultTTL != NO_TTL)
+            return expiring(timestamp, defaultTTL, nowInSec);
+
+        return new LivenessInfo(timestamp);
+    }
+
+    public static LivenessInfo expiring(long timestamp, int ttl, int nowInSec)
+    {
+        return new ExpiringLivenessInfo(timestamp, ttl, nowInSec + ttl);
+    }
+
+    public static LivenessInfo create(CFMetaData metadata, long timestamp, int ttl, int nowInSec)
+    {
+        return ttl == NO_TTL
+             ? create(metadata, timestamp, nowInSec)
+             : expiring(timestamp, ttl, nowInSec);
+    }
+
+    // Note that this ctor ignores the default table ttl and takes the expiration time, not the current time.
+    // Use when you know that's what you want.
+    public static LivenessInfo create(long timestamp, int ttl, int localExpirationTime)
+    {
+        return ttl == NO_TTL ? new LivenessInfo(timestamp) : new ExpiringLivenessInfo(timestamp, ttl, localExpirationTime);
+    }
 
     /**
-     * The ttl (if any) on the data or {@link NO_TTL} if the data is not
-     * expiring.
+     * Whether this liveness info is empty (has no timestamp).
      *
-     * Please note that this value is the TTL that was set originally and is thus not
-     * changing. If you want to figure out how much time the data has before it expires,
-     * then you should use {@link #remainingTTL}.
+     * @return whether this liveness info is empty or not.
      */
-    public int ttl();
+    public boolean isEmpty()
+    {
+        return timestamp == NO_TIMESTAMP;
+    }
 
     /**
-     * Whether this liveness info has a TTL or not.
+     * The timestamp for this liveness info.
      *
-     * @return whether this liveness info has a TTL or not.
+     * @return the liveness info timestamp (or {@link #NO_TIMESTAMP} if the info is empty).
      */
-    public boolean hasTTL();
+    public long timestamp()
+    {
+        return timestamp;
+    }
 
     /**
-     * The deletion time (in seconds) on the data if applicable ({@link NO_DELETION}
-     * otherwise).
-     *
-     * There is 3 cases in practice:
-     *   1) the data is neither deleted nor expiring: it then has neither {@code ttl()}
-     *      nor {@code localDeletionTime()}.
-     *   2) the data is expiring/expired: it then has both a {@code ttl()} and a
-     *      {@code localDeletionTime()}. Whether it's still live or is expired depends
-     *      on the {@code localDeletionTime()}.
-     *   3) the data is deleted: it has no {@code ttl()} but has a
-     *      {@code localDeletionTime()}.
+     * Whether the info has a ttl.
      */
-    public int localDeletionTime();
+    public boolean isExpiring()
+    {
+        return false;
+    }
 
     /**
-     * Whether this liveness info has a local deletion time or not.
+     * The ttl (if any) on the row primary key columns or {@link #NO_TTL} if it is not
+     * expiring.
      *
-     * @return whether this liveness info has a local deletion time or not.
+     * Please note that this value is the TTL that was set originally and is thus not
+     * changing.
      */
-    public boolean hasLocalDeletionTime();
+    public int ttl()
+    {
+        return NO_TTL;
+    }
 
     /**
-     * The actual remaining time to live (in seconds) for the data this is
-     * the liveness information of.
-     *
-     * {@code #ttl} returns the initial TTL sets on the piece of data while this
-     * method computes how much time the data actually has to live given the
-     * current time.
+     * The expiration time (in seconds) if the info is expiring ({@link #NO_EXPIRATION_TIME} otherwise).
      *
-     * @param nowInSec the current time in seconds.
-     * @return the remaining time to live (in seconds) the data has, or
-     * {@code -1} if the data is either expired or not expiring.
      */
-    public int remainingTTL(int nowInSec);
+    public int localExpirationTime()
+    {
+        return NO_EXPIRATION_TIME;
+    }
 
     /**
-     * Checks whether a given piece of data is live given the current time.
+     * Whether that info is still live.
+     *
+     * A {@code LivenessInfo} is live if it is either not expiring, or if its expiration time if after
+     * {@code nowInSec}.
      *
      * @param nowInSec the current time in seconds.
-     * @return whether the data having this liveness info is live or not.
+     * @return whether this liveness info is live or not.
      */
-    public boolean isLive(int nowInSec);
+    public boolean isLive(int nowInSec)
+    {
+        return !isEmpty();
+    }
 
     /**
      * Adds this liveness information to the provided digest.
      *
      * @param digest the digest to add this liveness information to.
      */
-    public void digest(MessageDigest digest);
+    public void digest(MessageDigest digest)
+    {
+        FBUtilities.updateWithLong(digest, timestamp());
+    }
 
     /**
      * Validate the data contained by this liveness information.
      *
      * @throws MarshalException if some of the data is corrupted.
      */
-    public void validate();
+    public void validate()
+    {
+    }
 
     /**
      * The size of the (useful) data this liveness information contains.
      *
      * @return the size of the data this liveness information contains.
      */
-    public int dataSize();
+    public int dataSize()
+    {
+        return TypeSizes.sizeof(timestamp());
+    }
 
     /**
      * Whether this liveness information supersedes another one (that is
@@ -148,31 +174,10 @@ public interface LivenessInfo extends Aliasable<LivenessInfo>
      *
      * @return whether this {@code LivenessInfo} supersedes {@code other}.
      */
-    public boolean supersedes(LivenessInfo other);
-
-    /**
-     * Returns the result of merging this info to another one (that is, it
-     * return this info if it supersedes the other one, or the other one
-     * otherwise).
-     */
-    public LivenessInfo mergeWith(LivenessInfo other);
-
-    /**
-     * Whether this liveness information can be purged.
-     * <p>
-     * A liveness info can be purged if it is not live and hasn't been so
-     * for longer than gcGrace (or more precisely, it's local deletion time
-     * is smaller than gcBefore, which is itself "now - gcGrace").
-     *
-     * @param maxPurgeableTimestamp the biggest timestamp that can be purged.
-     * A liveness info will not be considered purgeable if its timestamp is
-     * greater than this value, even if it mets the other criteria for purging.
-     * @param gcBefore the local deletion time before which deleted/expired
-     * liveness info can be purged.
-     *
-     * @return whether this liveness information can be purged.
-     */
-    public boolean isPurgeable(long maxPurgeableTimestamp, int gcBefore);
+    public boolean supersedes(LivenessInfo other)
+    {
+        return timestamp > other.timestamp;
+    }
 
     /**
      * Returns a copy of this liveness info updated with the provided timestamp.
@@ -182,5 +187,108 @@ public interface LivenessInfo extends Aliasable<LivenessInfo>
      * as timestamp. If it has no timestamp however, this liveness info is returned
      * unchanged.
      */
-    public LivenessInfo withUpdatedTimestamp(long newTimestamp);
+    public LivenessInfo withUpdatedTimestamp(long newTimestamp)
+    {
+        return new LivenessInfo(newTimestamp);
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("[ts=%d]", timestamp);
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if(!(other instanceof LivenessInfo))
+            return false;
+
+        LivenessInfo that = (LivenessInfo)other;
+        return this.timestamp() == that.timestamp()
+            && this.ttl() == that.ttl()
+            && this.localExpirationTime() == that.localExpirationTime();
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(timestamp(), ttl(), localExpirationTime());
+    }
+
+    private static class ExpiringLivenessInfo extends LivenessInfo
+    {
+        private final int ttl;
+        private final int localExpirationTime;
+
+        private ExpiringLivenessInfo(long timestamp, int ttl, int localExpirationTime)
+        {
+            super(timestamp);
+            assert ttl != NO_TTL && localExpirationTime != NO_EXPIRATION_TIME;
+            this.ttl = ttl;
+            this.localExpirationTime = localExpirationTime;
+        }
+
+        @Override
+        public int ttl()
+        {
+            return ttl;
+        }
+
+        @Override
+        public int localExpirationTime()
+        {
+            return localExpirationTime;
+        }
+
+        @Override
+        public boolean isExpiring()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean isLive(int nowInSec)
+        {
+            return nowInSec < localExpirationTime;
+        }
+
+        @Override
+        public void digest(MessageDigest digest)
+        {
+            super.digest(digest);
+            FBUtilities.updateWithInt(digest, localExpirationTime);
+            FBUtilities.updateWithInt(digest, ttl);
+        }
+
+        @Override
+        public void validate()
+        {
+            if (ttl < 0)
+                throw new MarshalException("A TTL should not be negative");
+            if (localExpirationTime < 0)
+                throw new MarshalException("A local expiration time should not be negative");
+        }
+
+        @Override
+        public int dataSize()
+        {
+            return super.dataSize()
+                 + TypeSizes.sizeof(ttl)
+                 + TypeSizes.sizeof(localExpirationTime);
+
+        }
+
+        @Override
+        public LivenessInfo withUpdatedTimestamp(long newTimestamp)
+        {
+            return new ExpiringLivenessInfo(newTimestamp, ttl, localExpirationTime);
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("[ts=%d ttl=%d, let=%d]", timestamp, ttl, localExpirationTime);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/LivenessInfoArray.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/LivenessInfoArray.java b/src/java/org/apache/cassandra/db/LivenessInfoArray.java
deleted file mode 100644
index 24026d8..0000000
--- a/src/java/org/apache/cassandra/db/LivenessInfoArray.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/*
- * 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.util.Arrays;
-
-import org.apache.cassandra.utils.ObjectSizes;
-
-/**
- * Utility class to store an array of liveness info efficiently.
- */
-public class LivenessInfoArray
-{
-    private long[] timestamps;
-    private int[] delTimesAndTTLs;
-
-    public LivenessInfoArray(int initialCapacity)
-    {
-        this.timestamps = new long[initialCapacity];
-        this.delTimesAndTTLs = new int[initialCapacity * 2];
-        clear();
-    }
-
-    public void clear(int i)
-    {
-        timestamps[i] = LivenessInfo.NO_TIMESTAMP;
-        delTimesAndTTLs[2 * i] = LivenessInfo.NO_DELETION_TIME;
-        delTimesAndTTLs[2 * i + 1] = LivenessInfo.NO_TTL;
-    }
-
-    public void set(int i, LivenessInfo info)
-    {
-        set(i, info.timestamp(), info.ttl(), info.localDeletionTime());
-    }
-
-    public void set(int i, long timestamp, int ttl, int localDeletionTime)
-    {
-        this.timestamps[i] = timestamp;
-        this.delTimesAndTTLs[2 * i] = localDeletionTime;
-        this.delTimesAndTTLs[2 * i + 1] = ttl;
-    }
-
-    public long timestamp(int i)
-    {
-        return timestamps[i];
-    }
-
-    public int localDeletionTime(int i)
-    {
-        return delTimesAndTTLs[2 * i];
-    }
-
-    public int ttl(int i)
-    {
-        return delTimesAndTTLs[2 * i + 1];
-    }
-
-    public boolean isLive(int i, int nowInSec)
-    {
-        // See AbstractLivenessInfo.isLive().
-        return localDeletionTime(i) == LivenessInfo.NO_DELETION_TIME
-            || (ttl(i) != LivenessInfo.NO_TTL && nowInSec < localDeletionTime(i));
-    }
-
-    public int size()
-    {
-        return timestamps.length;
-    }
-
-    public void resize(int newSize)
-    {
-        int prevSize = size();
-
-        timestamps = Arrays.copyOf(timestamps, newSize);
-        delTimesAndTTLs = Arrays.copyOf(delTimesAndTTLs, newSize * 2);
-
-        clear(prevSize, newSize);
-    }
-
-    public void swap(int i, int j)
-    {
-        long ts = timestamps[j];
-        int ldt = delTimesAndTTLs[2 * j];
-        int ttl = delTimesAndTTLs[2 * j + 1];
-
-        move(i, j);
-
-        timestamps[i] = ts;
-        delTimesAndTTLs[2 * i] = ldt;
-        delTimesAndTTLs[2 * i + 1] = ttl;
-    }
-
-    public void move(int i, int j)
-    {
-        timestamps[j] = timestamps[i];
-        delTimesAndTTLs[2 * j] = delTimesAndTTLs[2 * i];
-        delTimesAndTTLs[2 * j + 1] = delTimesAndTTLs[2 * i + 1];
-    }
-
-    public void clear()
-    {
-        clear(0, size());
-    }
-
-    private void clear(int from, int to)
-    {
-        Arrays.fill(timestamps, from, to, LivenessInfo.NO_TIMESTAMP);
-        for (int i = from; i < to; i++)
-        {
-            delTimesAndTTLs[2 * i] = LivenessInfo.NO_DELETION_TIME;
-            delTimesAndTTLs[2 * i + 1] = LivenessInfo.NO_TTL;
-        }
-    }
-
-    public int dataSize()
-    {
-        return 16 * size();
-    }
-
-    public long unsharedHeapSize()
-    {
-        return ObjectSizes.sizeOfArray(timestamps)
-             + ObjectSizes.sizeOfArray(delTimesAndTTLs);
-    }
-
-    public static Cursor newCursor()
-    {
-        return new Cursor();
-    }
-
-    public static class Cursor extends AbstractLivenessInfo
-    {
-        private LivenessInfoArray array;
-        private int i;
-
-        public Cursor setTo(LivenessInfoArray array, int i)
-        {
-            this.array = array;
-            this.i = i;
-            return this;
-        }
-
-        public long timestamp()
-        {
-            return array.timestamps[i];
-        }
-
-        public int localDeletionTime()
-        {
-            return array.delTimesAndTTLs[2 * i];
-        }
-
-        public int ttl()
-        {
-            return array.delTimesAndTTLs[2 * i + 1];
-        }
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/MutableDeletionInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/MutableDeletionInfo.java b/src/java/org/apache/cassandra/db/MutableDeletionInfo.java
new file mode 100644
index 0000000..6b19283
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/MutableDeletionInfo.java
@@ -0,0 +1,311 @@
+/*
+ * 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.util.Iterator;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.rows.RowStats;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+
+/**
+ * A mutable implementation of {@code DeletionInfo}.
+ */
+public class MutableDeletionInfo implements DeletionInfo
+{
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new MutableDeletionInfo(0, 0));
+
+    /**
+     * This represents a deletion of the entire partition. We can't represent this within the RangeTombstoneList, so it's
+     * kept separately. This also slightly optimizes the common case of a full partition deletion.
+     */
+    private DeletionTime partitionDeletion;
+
+    /**
+     * A list of range tombstones within the partition.  This is left as null if there are no range tombstones
+     * (to save an allocation (since it's a common case).
+     */
+    private RangeTombstoneList ranges;
+
+    /**
+     * Creates a DeletionInfo with only a top-level (row) tombstone.
+     * @param markedForDeleteAt the time after which the entire row should be considered deleted
+     * @param localDeletionTime what time the deletion write was applied locally (for purposes of
+     *                          purging the tombstone after gc_grace_seconds).
+     */
+    public MutableDeletionInfo(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));
+    }
+
+    public MutableDeletionInfo(DeletionTime partitionDeletion)
+    {
+        this(partitionDeletion, null);
+    }
+
+    public MutableDeletionInfo(DeletionTime partitionDeletion, RangeTombstoneList ranges)
+    {
+        this.partitionDeletion = partitionDeletion;
+        this.ranges = ranges;
+    }
+
+    /**
+     * Returns a new DeletionInfo that has no top-level tombstone or any range tombstones.
+     */
+    public static MutableDeletionInfo live()
+    {
+        return new MutableDeletionInfo(DeletionTime.LIVE);
+    }
+
+    public MutableDeletionInfo mutableCopy()
+    {
+        return new MutableDeletionInfo(partitionDeletion, ranges == null ? null : ranges.copy());
+    }
+
+    public MutableDeletionInfo copy(AbstractAllocator allocator)
+    {
+        RangeTombstoneList rangesCopy = null;
+        if (ranges != null)
+             rangesCopy = ranges.copy(allocator);
+
+        return new MutableDeletionInfo(partitionDeletion, rangesCopy);
+    }
+
+    /**
+     * Returns whether this DeletionInfo is live, that is deletes no columns.
+     */
+    public boolean isLive()
+    {
+        return partitionDeletion.isLive() && (ranges == null || ranges.isEmpty());
+    }
+
+    /**
+     * Potentially replaces the top-level tombstone with another, keeping whichever has the higher markedForDeleteAt
+     * timestamp.
+     * @param newInfo the deletion time to add to this deletion info.
+     */
+    public void add(DeletionTime newInfo)
+    {
+        if (newInfo.supersedes(partitionDeletion))
+            partitionDeletion = newInfo;
+    }
+
+    public void add(RangeTombstone tombstone, ClusteringComparator comparator)
+    {
+        if (ranges == null)
+            ranges = new RangeTombstoneList(comparator, 1);
+
+        ranges.add(tombstone);
+    }
+
+    /**
+     * Combines another DeletionInfo with this one and returns the result.  Whichever top-level tombstone
+     * has the higher markedForDeleteAt timestamp will be kept, along with its localDeletionTime.  The
+     * range tombstones will be combined.
+     *
+     * @return this object.
+     */
+    public DeletionInfo add(DeletionInfo newInfo)
+    {
+        add(newInfo.getPartitionDeletion());
+
+        // We know MutableDeletionInfo is the only impelementation and we're not mutating it, it's just to get access to the
+        // RangeTombstoneList directly.
+        assert newInfo instanceof MutableDeletionInfo;
+        RangeTombstoneList newRanges = ((MutableDeletionInfo)newInfo).ranges;
+
+        if (ranges == null)
+            ranges = newRanges == null ? null : newRanges.copy();
+        else if (newRanges != null)
+            ranges.addAll(newRanges);
+
+        return this;
+    }
+
+    public DeletionTime getPartitionDeletion()
+    {
+        return partitionDeletion;
+    }
+
+    // Use sparingly, not the most efficient thing
+    public Iterator<RangeTombstone> rangeIterator(boolean reversed)
+    {
+        return ranges == null ? Iterators.<RangeTombstone>emptyIterator() : ranges.iterator(reversed);
+    }
+
+    public Iterator<RangeTombstone> rangeIterator(Slice slice, boolean reversed)
+    {
+        return ranges == null ? Iterators.<RangeTombstone>emptyIterator() : ranges.iterator(slice, reversed);
+    }
+
+    public RangeTombstone rangeCovering(Clustering name)
+    {
+        return ranges == null ? null : ranges.search(name);
+    }
+
+    public int dataSize()
+    {
+        int size = TypeSizes.sizeof(partitionDeletion.markedForDeleteAt());
+        return size + (ranges == null ? 0 : ranges.dataSize());
+    }
+
+    public boolean hasRanges()
+    {
+        return ranges != null && !ranges.isEmpty();
+    }
+
+    public int rangeCount()
+    {
+        return hasRanges() ? ranges.size() : 0;
+    }
+
+    public long maxTimestamp()
+    {
+        return ranges == null ? partitionDeletion.markedForDeleteAt() : Math.max(partitionDeletion.markedForDeleteAt(), ranges.maxMarkedAt());
+    }
+
+    /**
+     * Whether this deletion info may modify the provided one if added to it.
+     */
+    public boolean mayModify(DeletionInfo delInfo)
+    {
+        return partitionDeletion.compareTo(delInfo.getPartitionDeletion()) > 0 || hasRanges();
+    }
+
+    @Override
+    public String toString()
+    {
+        if (ranges == null || ranges.isEmpty())
+            return String.format("{%s}", partitionDeletion);
+        else
+            return String.format("{%s, ranges=%s}", partitionDeletion, rangesAsString());
+    }
+
+    private String rangesAsString()
+    {
+        assert !ranges.isEmpty();
+        StringBuilder sb = new StringBuilder();
+        ClusteringComparator cc = ranges.comparator();
+        Iterator<RangeTombstone> iter = rangeIterator(false);
+        while (iter.hasNext())
+        {
+            RangeTombstone i = iter.next();
+            sb.append(i.deletedSlice().toString(cc));
+            sb.append('@');
+            sb.append(i.deletionTime());
+        }
+        return sb.toString();
+    }
+
+    // Updates all the timestamp of the deletion contained in this DeletionInfo to be {@code timestamp}.
+    public DeletionInfo updateAllTimestamp(long timestamp)
+    {
+        if (partitionDeletion.markedForDeleteAt() != Long.MIN_VALUE)
+            partitionDeletion = new DeletionTime(timestamp, partitionDeletion.localDeletionTime());
+
+        if (ranges != null)
+            ranges.updateAllTimestamp(timestamp);
+        return this;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if(!(o instanceof MutableDeletionInfo))
+            return false;
+        MutableDeletionInfo that = (MutableDeletionInfo)o;
+        return partitionDeletion.equals(that.partitionDeletion) && Objects.equal(ranges, that.ranges);
+    }
+
+    @Override
+    public final int hashCode()
+    {
+        return Objects.hashCode(partitionDeletion, ranges);
+    }
+
+    @Override
+    public long unsharedHeapSize()
+    {
+        return EMPTY_SIZE + partitionDeletion.unsharedHeapSize() + (ranges == null ? 0 : ranges.unsharedHeapSize());
+    }
+
+    public void collectStats(RowStats.Collector collector)
+    {
+        collector.update(partitionDeletion);
+        if (ranges != null)
+            ranges.collectStats(collector);
+    }
+
+    public static Builder builder(DeletionTime partitionLevelDeletion, ClusteringComparator comparator, boolean reversed)
+    {
+        return new Builder(partitionLevelDeletion, comparator, reversed);
+    }
+
+    /**
+     * Builds DeletionInfo object from (in order) range tombstone markers.
+     */
+    public static class Builder
+    {
+        private final MutableDeletionInfo deletion;
+        private final ClusteringComparator comparator;
+
+        private final boolean reversed;
+
+        private RangeTombstoneMarker openMarker;
+
+        private Builder(DeletionTime partitionLevelDeletion, ClusteringComparator comparator, boolean reversed)
+        {
+            this.deletion = new MutableDeletionInfo(partitionLevelDeletion);
+            this.comparator = comparator;
+            this.reversed = reversed;
+        }
+
+        public void add(RangeTombstoneMarker marker)
+        {
+            // We need to start by the close case in case that's a boundary
+
+            if (marker.isClose(reversed))
+            {
+                DeletionTime openDeletion = openMarker.openDeletionTime(reversed);
+                assert marker.closeDeletionTime(reversed).equals(openDeletion);
+
+                Slice.Bound open = openMarker.openBound(reversed);
+                Slice.Bound close = marker.closeBound(reversed);
+
+                Slice slice = reversed ? Slice.make(close, open) : Slice.make(open, close);
+                deletion.add(new RangeTombstone(slice, openDeletion), comparator);
+            }
+
+            if (marker.isOpen(reversed))
+            {
+                openMarker = marker;
+            }
+        }
+
+        public MutableDeletionInfo build()
+        {
+            return deletion;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/PartitionColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/PartitionColumns.java b/src/java/org/apache/cassandra/db/PartitionColumns.java
index ef05760..5f1da8a 100644
--- a/src/java/org/apache/cassandra/db/PartitionColumns.java
+++ b/src/java/org/apache/cassandra/db/PartitionColumns.java
@@ -66,6 +66,11 @@ public class PartitionColumns implements Iterable<ColumnDefinition>
         return statics.isEmpty() && regulars.isEmpty();
     }
 
+    public Columns columns(boolean isStatic)
+    {
+        return isStatic ? statics : regulars;
+    }
+
     public boolean contains(ColumnDefinition column)
     {
         return column.isStatic() ? statics.contains(column) : regulars.contains(column);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index 65e38d0..4a3704f 100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -32,6 +31,7 @@ import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.index.SecondaryIndexSearcher;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.metrics.TableMetrics;
@@ -278,11 +278,11 @@ public class PartitionRangeReadCommand extends ReadCommand
 
     private static class Deserializer extends SelectionDeserializer
     {
-        public ReadCommand deserialize(DataInput in, int version, boolean isDigest, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits)
+        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits)
         throws IOException
         {
             DataRange range = DataRange.serializer.deserialize(in, version, metadata);
             return new PartitionRangeReadCommand(isDigest, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, range);
         }
-    };
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/RangeTombstone.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RangeTombstone.java b/src/java/org/apache/cassandra/db/RangeTombstone.java
index df60933..8865b0f 100644
--- a/src/java/org/apache/cassandra/db/RangeTombstone.java
+++ b/src/java/org/apache/cassandra/db/RangeTombstone.java
@@ -17,20 +17,15 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.security.MessageDigest;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.io.ISSTableSerializer;
-import org.apache.cassandra.io.sstable.format.Version;
-import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.utils.Interval;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
  * A range tombstone is a tombstone that covers a slice/range of rows.
@@ -48,7 +43,7 @@ public class RangeTombstone
     public RangeTombstone(Slice slice, DeletionTime deletion)
     {
         this.slice = slice;
-        this.deletion = deletion.takeAlias();
+        this.deletion = deletion;
     }
 
     /**
@@ -73,7 +68,7 @@ public class RangeTombstone
 
     public String toString(ClusteringComparator comparator)
     {
-        return slice.toString(comparator) + "@" + deletion;
+        return slice.toString(comparator) + '@' + deletion;
     }
 
     @Override
@@ -108,9 +103,30 @@ public class RangeTombstone
     {
         public static final Serializer serializer = new Serializer();
 
+        /** The smallest start bound, i.e. the one that starts before any row. */
+        public static final Bound BOTTOM = new Bound(Kind.INCL_START_BOUND, EMPTY_VALUES_ARRAY);
+        /** The biggest end bound, i.e. the one that ends after any row. */
+        public static final Bound TOP = new Bound(Kind.INCL_END_BOUND, EMPTY_VALUES_ARRAY);
+
         public Bound(Kind kind, ByteBuffer[] values)
         {
             super(kind, values);
+            assert values.length > 0 || !kind.isBoundary();
+        }
+
+        public boolean isBoundary()
+        {
+            return kind.isBoundary();
+        }
+
+        public boolean isOpen(boolean reversed)
+        {
+            return kind.isOpen(reversed);
+        }
+
+        public boolean isClose(boolean reversed)
+        {
+            return kind.isClose(reversed);
         }
 
         public static RangeTombstone.Bound inclusiveOpen(boolean reversed, ByteBuffer[] boundValues)
@@ -143,6 +159,19 @@ public class RangeTombstone
             return new Bound(reversed ? Kind.INCL_END_EXCL_START_BOUNDARY : Kind.EXCL_END_INCL_START_BOUNDARY, boundValues);
         }
 
+        public static RangeTombstone.Bound fromSliceBound(Slice.Bound sliceBound)
+        {
+            return new RangeTombstone.Bound(sliceBound.kind(), sliceBound.getRawValues());
+        }
+
+        public RangeTombstone.Bound copy(AbstractAllocator allocator)
+        {
+            ByteBuffer[] newValues = new ByteBuffer[size()];
+            for (int i = 0; i < size(); i++)
+                newValues[i] = allocator.clone(get(i));
+            return new Bound(kind(), newValues);
+        }
+
         @Override
         public Bound withNewKind(Kind kind)
         {
@@ -165,13 +194,15 @@ public class RangeTombstone
                      + ClusteringPrefix.serializer.valuesWithoutSizeSerializedSize(bound, version, types);
             }
 
-            public Kind deserialize(DataInput in, int version, List<AbstractType<?>> types, Writer writer) throws IOException
+            public RangeTombstone.Bound deserialize(DataInputPlus in, int version, List<AbstractType<?>> types) throws IOException
             {
                 Kind kind = Kind.values()[in.readByte()];
-                writer.writeBoundKind(kind);
                 int size = in.readUnsignedShort();
-                ClusteringPrefix.serializer.deserializeValuesWithoutSize(in, size, version, types, writer);
-                return kind;
+                if (size == 0)
+                    return kind.isStart() ? BOTTOM : TOP;
+
+                ByteBuffer[] values = ClusteringPrefix.serializer.deserializeValuesWithoutSize(in, size, version, types);
+                return new RangeTombstone.Bound(kind, values);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/RangeTombstoneList.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RangeTombstoneList.java b/src/java/org/apache/cassandra/db/RangeTombstoneList.java
index 64f0978..96bcdb1 100644
--- a/src/java/org/apache/cassandra/db/RangeTombstoneList.java
+++ b/src/java/org/apache/cassandra/db/RangeTombstoneList.java
@@ -244,7 +244,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
     {
         int idx = searchInternal(clustering, 0, size);
         // No matter what the counter cell's timestamp is, a tombstone always takes precedence. See CASSANDRA-7346.
-        return idx >= 0 && (cell.isCounterCell() || markedAts[idx] >= cell.livenessInfo().timestamp());
+        return idx >= 0 && (cell.isCounterCell() || markedAts[idx] >= cell.timestamp());
     }
 
     /**
@@ -254,7 +254,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
     public DeletionTime searchDeletionTime(Clustering name)
     {
         int idx = searchInternal(name, 0, size);
-        return idx < 0 ? null : new SimpleDeletionTime(markedAts[idx], delTimes[idx]);
+        return idx < 0 ? null : new DeletionTime(markedAts[idx], delTimes[idx]);
     }
 
     public RangeTombstone search(Clustering name)
@@ -300,6 +300,23 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
         return dataSize;
     }
 
+    public long maxMarkedAt()
+    {
+        long max = Long.MIN_VALUE;
+        for (int i = 0; i < size; i++)
+            max = Math.max(max, markedAts[i]);
+        return max;
+    }
+
+    public void collectStats(RowStats.Collector collector)
+    {
+        for (int i = 0; i < size; i++)
+        {
+            collector.updateTimestamp(markedAts[i]);
+            collector.updateLocalDeletionTime(delTimes[i]);
+        }
+    }
+
     public void updateAllTimestamp(long timestamp)
     {
         for (int i = 0; i < size; i++)
@@ -308,22 +325,22 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
 
     private RangeTombstone rangeTombstone(int idx)
     {
-        return new RangeTombstone(Slice.make(starts[idx], ends[idx]), new SimpleDeletionTime(markedAts[idx], delTimes[idx]));
+        return new RangeTombstone(Slice.make(starts[idx], ends[idx]), new DeletionTime(markedAts[idx], delTimes[idx]));
     }
 
     private RangeTombstone rangeTombstoneWithNewStart(int idx, Slice.Bound newStart)
     {
-        return new RangeTombstone(Slice.make(newStart, ends[idx]), new SimpleDeletionTime(markedAts[idx], delTimes[idx]));
+        return new RangeTombstone(Slice.make(newStart, ends[idx]), new DeletionTime(markedAts[idx], delTimes[idx]));
     }
 
     private RangeTombstone rangeTombstoneWithNewEnd(int idx, Slice.Bound newEnd)
     {
-        return new RangeTombstone(Slice.make(starts[idx], newEnd), new SimpleDeletionTime(markedAts[idx], delTimes[idx]));
+        return new RangeTombstone(Slice.make(starts[idx], newEnd), new DeletionTime(markedAts[idx], delTimes[idx]));
     }
 
     private RangeTombstone rangeTombstoneWithNewBounds(int idx, Slice.Bound newStart, Slice.Bound newEnd)
     {
-        return new RangeTombstone(Slice.make(newStart, newEnd), new SimpleDeletionTime(markedAts[idx], delTimes[idx]));
+        return new RangeTombstone(Slice.make(newStart, newEnd), new DeletionTime(markedAts[idx], delTimes[idx]));
     }
 
     public Iterator<RangeTombstone> iterator()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index 99547f0..c3f036a 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -17,11 +17,12 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.DataInput;
 import java.io.IOException;
+import java.util.Iterator;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
@@ -63,7 +64,7 @@ public abstract class ReadCommand implements ReadQuery
 
     protected static abstract class SelectionDeserializer
     {
-        public abstract ReadCommand deserialize(DataInput in, int version, boolean isDigest, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) throws IOException;
+        public abstract ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) throws IOException;
     }
 
     protected enum Kind
@@ -268,22 +269,13 @@ public abstract class ReadCommand implements ReadQuery
 
         try
         {
-            resultIterator = UnfilteredPartitionIterators.convertExpiredCellsToTombstones(resultIterator, nowInSec);
-            resultIterator = withMetricsRecording(withoutExpiredTombstones(resultIterator, cfs), cfs.metric, startTimeNanos);
-
-            // TODO: we should push the dropping of columns down the layers because
-            // 1) it'll be more efficient
-            // 2) it could help us solve #6276
-            // But there is not reason not to do this as a followup so keeping it here for now (we'll have
-            // to be wary of cached row if we move this down the layers)
-            if (!metadata().getDroppedColumns().isEmpty())
-                resultIterator = UnfilteredPartitionIterators.removeDroppedColumns(resultIterator, metadata().getDroppedColumns());
+            resultIterator = withMetricsRecording(withoutPurgeableTombstones(resultIterator, cfs), cfs.metric, startTimeNanos);
 
             // If we've used a 2ndary index, we know the result already satisfy the primary expression used, so
             // no point in checking it again.
             RowFilter updatedFilter = searcher == null
-                                       ? rowFilter()
-                                       : rowFilter().without(searcher.primaryClause(this));
+                                    ? rowFilter()
+                                    : rowFilter().without(searcher.primaryClause(this));
 
             // TODO: We'll currently do filtering by the rowFilter here because it's convenient. However,
             // we'll probably want to optimize by pushing it down the layer (like for dropped columns) as it
@@ -333,26 +325,33 @@ public abstract class ReadCommand implements ReadQuery
             {
                 currentKey = iter.partitionKey();
 
-                return new WrappingUnfilteredRowIterator(iter)
+                return new AlteringUnfilteredRowIterator(iter)
                 {
-                    public Unfiltered next()
+                    @Override
+                    protected Row computeNextStatic(Row row)
                     {
-                        Unfiltered unfiltered = super.next();
-                        if (unfiltered.kind() == Unfiltered.Kind.ROW)
-                        {
-                            Row row = (Row) unfiltered;
-                            if (row.hasLiveData(ReadCommand.this.nowInSec()))
-                                ++liveRows;
-                            for (Cell cell : row)
-                                if (!cell.isLive(ReadCommand.this.nowInSec()))
-                                    countTombstone(row.clustering());
-                        }
-                        else
+                        return computeNext(row);
+                    }
+
+                    @Override
+                    protected Row computeNext(Row row)
+                    {
+                        if (row.hasLiveData(ReadCommand.this.nowInSec()))
+                            ++liveRows;
+
+                        for (Cell cell : row.cells())
                         {
-                            countTombstone(unfiltered.clustering());
+                            if (!cell.isLive(ReadCommand.this.nowInSec()))
+                                countTombstone(row.clustering());
                         }
+                        return row;
+                    }
 
-                        return unfiltered;
+                    @Override
+                    protected RangeTombstoneMarker computeNext(RangeTombstoneMarker marker)
+                    {
+                        countTombstone(marker.clustering());
+                        return marker;
                     }
 
                     private void countTombstone(ClusteringPrefix clustering)
@@ -407,12 +406,12 @@ public abstract class ReadCommand implements ReadQuery
 
     protected abstract void appendCQLWhereClause(StringBuilder sb);
 
-    // Skip expired tombstones. We do this because it's safe to do (post-merge of the memtable and sstable at least), it
-    // can save us some bandwith, and avoid making us throw a TombstoneOverwhelmingException for expired tombstones (which
+    // Skip purgeable tombstones. We do this because it's safe to do (post-merge of the memtable and sstable at least), it
+    // can save us some bandwith, and avoid making us throw a TombstoneOverwhelmingException for purgeable tombstones (which
     // are to some extend an artefact of compaction lagging behind and hence counting them is somewhat unintuitive).
-    protected UnfilteredPartitionIterator withoutExpiredTombstones(UnfilteredPartitionIterator iterator, ColumnFamilyStore cfs)
+    protected UnfilteredPartitionIterator withoutPurgeableTombstones(UnfilteredPartitionIterator iterator, ColumnFamilyStore cfs)
     {
-        return new TombstonePurgingPartitionIterator(iterator, cfs.gcBefore(nowInSec()))
+        return new PurgingPartitionIterator(iterator, cfs.gcBefore(nowInSec()))
         {
             protected long getMaxPurgeableTimestamp()
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/ReusableClustering.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReusableClustering.java b/src/java/org/apache/cassandra/db/ReusableClustering.java
deleted file mode 100644
index e2760aa..0000000
--- a/src/java/org/apache/cassandra/db/ReusableClustering.java
+++ /dev/null
@@ -1,82 +0,0 @@
-
-/*
- * 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.nio.ByteBuffer;
-import java.util.Arrays;
-
-import org.apache.cassandra.utils.ObjectSizes;
-
-public class ReusableClustering extends Clustering
-{
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new ReusableClustering(0));
-
-    protected final ByteBuffer[] values;
-
-    protected ReusableWriter writer;
-
-    public ReusableClustering(int size)
-    {
-        this.values = new ByteBuffer[size];
-    }
-
-    public int size()
-    {
-        return values.length;
-    }
-
-    public ByteBuffer get(int i)
-    {
-        return values[i];
-    }
-
-    public ByteBuffer[] getRawValues()
-    {
-        return values;
-    }
-
-    public Writer writer()
-    {
-        if (writer == null)
-            writer = new ReusableWriter();
-        return writer;
-    }
-
-    public void reset()
-    {
-        Arrays.fill(values, null);
-        if (writer != null)
-            writer.reset();
-    }
-
-    protected class ReusableWriter implements Writer
-    {
-        int idx;
-
-        public void writeClusteringValue(ByteBuffer value)
-        {
-            values[idx++] = value;
-        }
-
-        private void reset()
-        {
-            idx = 0;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/ReusableClusteringPrefix.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReusableClusteringPrefix.java b/src/java/org/apache/cassandra/db/ReusableClusteringPrefix.java
deleted file mode 100644
index d2f19f7..0000000
--- a/src/java/org/apache/cassandra/db/ReusableClusteringPrefix.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.nio.ByteBuffer;
-import java.util.Arrays;
-
-import org.apache.cassandra.utils.ObjectSizes;
-
-// Note that we abuse a bit ReusableClustering to store Slice.Bound infos, but it's convenient so ...
-public class ReusableClusteringPrefix extends ReusableClustering
-{
-    private Kind kind;
-    private int size;
-
-    public ReusableClusteringPrefix(int size)
-    {
-        super(size);
-    }
-
-    public ClusteringPrefix get()
-    {
-        // We use ReusableClusteringPrefix when writing sstables (in ColumnIndex) and we
-        // don't write static clustering there.
-        assert kind != Kind.STATIC_CLUSTERING;
-        if (kind == Kind.CLUSTERING)
-        {
-            assert values.length == size;
-            return this;
-        }
-
-        return Slice.Bound.create(kind, Arrays.copyOfRange(values, 0, size));
-    }
-
-    public void copy(ClusteringPrefix clustering)
-    {
-        kind = clustering.kind();
-        for (int i = 0; i < clustering.size(); i++)
-            values[i] = clustering.get(i);
-        size = clustering.size();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/ReusableLivenessInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReusableLivenessInfo.java b/src/java/org/apache/cassandra/db/ReusableLivenessInfo.java
deleted file mode 100644
index 43530b0..0000000
--- a/src/java/org/apache/cassandra/db/ReusableLivenessInfo.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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;
-
-public class ReusableLivenessInfo extends AbstractLivenessInfo
-{
-    private long timestamp;
-    private int ttl;
-    private int localDeletionTime;
-
-    public ReusableLivenessInfo()
-    {
-        reset();
-    }
-
-    public LivenessInfo setTo(LivenessInfo info)
-    {
-        return setTo(info.timestamp(), info.ttl(), info.localDeletionTime());
-    }
-
-    public LivenessInfo setTo(long timestamp, int ttl, int localDeletionTime)
-    {
-        this.timestamp = timestamp;
-        this.ttl = ttl;
-        this.localDeletionTime = localDeletionTime;
-        return this;
-    }
-
-    public long timestamp()
-    {
-        return timestamp;
-    }
-
-    public int ttl()
-    {
-        return ttl;
-    }
-
-    public int localDeletionTime()
-    {
-        return localDeletionTime;
-    }
-
-    public void reset()
-    {
-        this.timestamp = LivenessInfo.NO_TIMESTAMP;
-        this.ttl = LivenessInfo.NO_TTL;
-        this.localDeletionTime = LivenessInfo.NO_DELETION_TIME;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/RowUpdateBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RowUpdateBuilder.java b/src/java/org/apache/cassandra/db/RowUpdateBuilder.java
index 627321e..7640512 100644
--- a/src/java/org/apache/cassandra/db/RowUpdateBuilder.java
+++ b/src/java/org/apache/cassandra/db/RowUpdateBuilder.java
@@ -43,25 +43,27 @@ public class RowUpdateBuilder
 {
     private final PartitionUpdate update;
 
-    private final LivenessInfo defaultLiveness;
-    private final LivenessInfo deletionLiveness;
+    private final long timestamp;
+    private final int ttl;
+    private final int localDeletionTime;
+
     private final DeletionTime deletionTime;
 
     private final Mutation mutation;
 
-    private Row.Writer regularWriter;
-    private Row.Writer staticWriter;
+    private Row.Builder regularBuilder;
+    private Row.Builder staticBuilder;
 
-    private boolean hasSetClustering;
     private boolean useRowMarker = true;
 
     private RowUpdateBuilder(PartitionUpdate update, long timestamp, int ttl, int localDeletionTime, Mutation mutation)
     {
         this.update = update;
 
-        this.defaultLiveness = SimpleLivenessInfo.forUpdate(timestamp, ttl, localDeletionTime, update.metadata());
-        this.deletionLiveness = SimpleLivenessInfo.forDeletion(timestamp, localDeletionTime);
-        this.deletionTime = new SimpleDeletionTime(timestamp, localDeletionTime);
+        this.timestamp = timestamp;
+        this.ttl = ttl;
+        this.localDeletionTime = localDeletionTime;
+        this.deletionTime = new DeletionTime(timestamp, localDeletionTime);
 
         // note that the created mutation may get further update later on, so we don't use the ctor that create a singletonMap
         // underneath (this class if for convenience, not performance)
@@ -73,31 +75,45 @@ public class RowUpdateBuilder
         this(update, timestamp, ttl, FBUtilities.nowInSeconds(), mutation);
     }
 
-    private Row.Writer writer()
+    private void startRow(Clustering clustering)
     {
-        assert staticWriter == null : "Cannot update both static and non-static columns with the same RowUpdateBuilder object";
-        if (regularWriter == null)
-        {
-            regularWriter = update.writer();
+        assert staticBuilder == null : "Cannot update both static and non-static columns with the same RowUpdateBuilder object";
+        assert regularBuilder == null : "Cannot add the clustering twice to the same row";
+
+        regularBuilder = ArrayBackedRow.unsortedBuilder(update.columns().regulars, FBUtilities.nowInSeconds());
+        regularBuilder.newRow(clustering);
+
+        // If a CQL table, add the "row marker"
+        if (update.metadata().isCQLTable() && useRowMarker)
+            regularBuilder.addPrimaryKeyLivenessInfo(LivenessInfo.create(update.metadata(), timestamp, ttl, localDeletionTime));
+    }
 
-            // If a CQL table, add the "row marker"
-            if (update.metadata().isCQLTable() && useRowMarker)
-                regularWriter.writePartitionKeyLivenessInfo(defaultLiveness);
+    private Row.Builder builder()
+    {
+        assert staticBuilder == null : "Cannot update both static and non-static columns with the same RowUpdateBuilder object";
+        if (regularBuilder == null)
+        {
+            // we don't force people to call clustering() if the table has no clustering, so call it ourselves
+            assert update.metadata().comparator.size() == 0 : "Missing call to clustering()";
+            startRow(Clustering.EMPTY);
         }
-        return regularWriter;
+        return regularBuilder;
     }
 
-    private Row.Writer staticWriter()
+    private Row.Builder staticBuilder()
     {
-        assert regularWriter == null : "Cannot update both static and non-static columns with the same RowUpdateBuilder object";
-        if (staticWriter == null)
-            staticWriter = update.staticWriter();
-        return staticWriter;
+        assert regularBuilder == null : "Cannot update both static and non-static columns with the same RowUpdateBuilder object";
+        if (staticBuilder == null)
+        {
+            staticBuilder = ArrayBackedRow.unsortedBuilder(update.columns().statics, FBUtilities.nowInSeconds());
+            staticBuilder.newRow(Clustering.STATIC_CLUSTERING);
+        }
+        return staticBuilder;
     }
 
-    private Row.Writer writer(ColumnDefinition c)
+    private Row.Builder builder(ColumnDefinition c)
     {
-        return c.isStatic() ? staticWriter() : writer();
+        return c.isStatic() ? staticBuilder() : builder();
     }
 
     public RowUpdateBuilder(CFMetaData metadata, long timestamp, Object partitionKey)
@@ -145,18 +161,17 @@ public class RowUpdateBuilder
     public RowUpdateBuilder clustering(Object... clusteringValues)
     {
         assert clusteringValues.length == update.metadata().comparator.size()
-            : "Invalid clustering values length. Expected: " + update.metadata().comparator.size() + " got: " + clusteringValues.length;
-        hasSetClustering = true;
-        if (clusteringValues.length > 0)
-            Rows.writeClustering(update.metadata().comparator.make(clusteringValues), writer());
+             : "Invalid clustering values length. Expected: " + update.metadata().comparator.size() + " got: " + clusteringValues.length;
+
+        startRow(clusteringValues.length == 0 ? Clustering.EMPTY : update.metadata().comparator.make(clusteringValues));
         return this;
     }
 
     public Mutation build()
     {
-        Row.Writer writer = regularWriter == null ? staticWriter : regularWriter;
-        if (writer != null)
-            writer.endOfRow();
+        Row.Builder builder = regularBuilder == null ? staticBuilder : regularBuilder;
+        if (builder != null)
+            update.add(builder.build());
         return mutation;
     }
 
@@ -170,14 +185,16 @@ public class RowUpdateBuilder
     {
         assert clusteringValues.length == update.metadata().comparator.size() || (clusteringValues.length == 0 && !update.columns().statics.isEmpty());
 
-        Row.Writer writer = clusteringValues.length == update.metadata().comparator.size()
-                          ? update.writer()
-                          : update.staticWriter();
+        boolean isStatic = clusteringValues.length != update.metadata().comparator.size();
+        Row.Builder builder = ArrayBackedRow.sortedBuilder(isStatic ? update.columns().statics : update.columns().regulars);
 
-        if (clusteringValues.length > 0)
-            Rows.writeClustering(update.metadata().comparator.make(clusteringValues), writer);
-        writer.writeRowDeletion(new SimpleDeletionTime(timestamp, FBUtilities.nowInSeconds()));
-        writer.endOfRow();
+        if (isStatic)
+            builder.newRow(Clustering.STATIC_CLUSTERING);
+        else
+            builder.newRow(clusteringValues.length == 0 ? Clustering.EMPTY : update.metadata().comparator.make(clusteringValues));
+        builder.addRowDeletion(new DeletionTime(timestamp, FBUtilities.nowInSeconds()));
+
+        update.add(builder.build());
     }
 
     public static Mutation deleteRow(CFMetaData metadata, long timestamp, Mutation mutation, Object... clusteringValues)
@@ -219,22 +236,21 @@ public class RowUpdateBuilder
     {
         ColumnDefinition c = getDefinition(columnName);
         assert c != null : "Cannot find column " + columnName;
-        assert c.isStatic() || update.metadata().comparator.size() == 0 || hasSetClustering : "Cannot set non static column " + c + " since no clustering has been provided";
+        assert c.isStatic() || update.metadata().comparator.size() == 0 || regularBuilder != null : "Cannot set non static column " + c + " since no clustering has been provided";
         assert c.type.isCollection() && c.type.isMultiCell();
-        writer(c).writeComplexDeletion(c, new SimpleDeletionTime(defaultLiveness.timestamp() - 1, deletionTime.localDeletionTime()));
+        builder(c).addComplexDeletion(c, new DeletionTime(timestamp - 1, localDeletionTime));
         return this;
     }
 
     public RowUpdateBuilder addRangeTombstone(RangeTombstone rt)
     {
-        update.addRangeTombstone(rt);
+        update.add(rt);
         return this;
     }
 
     public RowUpdateBuilder addRangeTombstone(Slice slice)
     {
-        update.addRangeTombstone(slice, deletionTime);
-        return this;
+        return addRangeTombstone(new RangeTombstone(slice, deletionTime));
     }
 
     public RowUpdateBuilder addRangeTombstone(Object start, Object end)
@@ -251,13 +267,17 @@ public class RowUpdateBuilder
         return add(c, value);
     }
 
+    private Cell makeCell(ColumnDefinition c, ByteBuffer value, CellPath path)
+    {
+        return value == null
+             ? BufferCell.tombstone(c, timestamp, localDeletionTime)
+             : (ttl == LivenessInfo.NO_TTL ? BufferCell.live(update.metadata(), c, timestamp, value, path) : BufferCell.expiring(c, timestamp, ttl, localDeletionTime, value, path));
+    }
+
     public RowUpdateBuilder add(ColumnDefinition columnDefinition, Object value)
     {
-        assert columnDefinition.isStatic() || update.metadata().comparator.size() == 0 || hasSetClustering : "Cannot set non static column " + columnDefinition + " since no clustering hasn't been provided";
-        if (value == null)
-            writer(columnDefinition).writeCell(columnDefinition, false, ByteBufferUtil.EMPTY_BYTE_BUFFER, deletionLiveness, null);
-        else
-            writer(columnDefinition).writeCell(columnDefinition, false, bb(value, columnDefinition.type), defaultLiveness, null);
+        assert columnDefinition.isStatic() || update.metadata().comparator.size() == 0 || regularBuilder != null : "Cannot set non static column " + columnDefinition + " since no clustering hasn't been provided";
+        builder(columnDefinition).addCell(makeCell(columnDefinition, bb(value, columnDefinition.type), null));
         return this;
     }
 
@@ -273,8 +293,11 @@ public class RowUpdateBuilder
         return add(columnDefinition, null);
     }
 
-    private ByteBuffer bb(Object value, AbstractType<?> type)
+    private static ByteBuffer bb(Object value, AbstractType<?> type)
     {
+        if (value == null)
+            return null;
+
         if (value instanceof ByteBuffer)
             return (ByteBuffer)value;
 
@@ -306,30 +329,30 @@ public class RowUpdateBuilder
     public RowUpdateBuilder addMapEntry(String columnName, Object key, Object value)
     {
         ColumnDefinition c = getDefinition(columnName);
-        assert c.isStatic() || update.metadata().comparator.size() == 0 || hasSetClustering : "Cannot set non static column " + c + " since no clustering has been provided";
+        assert c.isStatic() || update.metadata().comparator.size() == 0 || regularBuilder != null : "Cannot set non static column " + c + " since no clustering has been provided";
         assert c.type instanceof MapType && c.type.isMultiCell();
         MapType mt = (MapType)c.type;
-        writer(c).writeCell(c, false, bb(value, mt.getValuesType()), defaultLiveness, CellPath.create(bb(key, mt.getKeysType())));
+        builder(c).addCell(makeCell(c, bb(value, mt.getValuesType()), CellPath.create(bb(key, mt.getKeysType()))));
         return this;
     }
 
     public RowUpdateBuilder addListEntry(String columnName, Object value)
     {
         ColumnDefinition c = getDefinition(columnName);
-        assert c.isStatic() || hasSetClustering : "Cannot set non static column " + c + " since no clustering has been provided";
+        assert c.isStatic() || regularBuilder != null : "Cannot set non static column " + c + " since no clustering has been provided";
         assert c.type instanceof ListType && c.type.isMultiCell();
         ListType lt = (ListType)c.type;
-        writer(c).writeCell(c, false, bb(value, lt.getElementsType()), defaultLiveness, CellPath.create(ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes())));
+        builder(c).addCell(makeCell(c, bb(value, lt.getElementsType()), CellPath.create(ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes()))));
         return this;
     }
 
     public RowUpdateBuilder addSetEntry(String columnName, Object value)
     {
         ColumnDefinition c = getDefinition(columnName);
-        assert c.isStatic() || hasSetClustering : "Cannot set non static column " + c + " since no clustering has been provided";
+        assert c.isStatic() || regularBuilder != null : "Cannot set non static column " + c + " since no clustering has been provided";
         assert c.type instanceof SetType && c.type.isMultiCell();
         SetType st = (SetType)c.type;
-        writer(c).writeCell(c, false, ByteBufferUtil.EMPTY_BYTE_BUFFER, defaultLiveness, CellPath.create(bb(value, st.getElementsType())));
+        builder(c).addCell(makeCell(c, ByteBufferUtil.EMPTY_BYTE_BUFFER, CellPath.create(bb(value, st.getElementsType()))));
         return this;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/SerializationHeader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SerializationHeader.java b/src/java/org/apache/cassandra/db/SerializationHeader.java
index c720804..5784260 100644
--- a/src/java/org/apache/cassandra/db/SerializationHeader.java
+++ b/src/java/org/apache/cassandra/db/SerializationHeader.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
@@ -97,7 +96,7 @@ public class SerializationHeader
         // We always use a dense layout for the static row. Having very many static columns with  only a few set at
         // any given time doesn't feel very common at all (and we already optimize the case where no static at all
         // are provided).
-        return isStatic ? false : useSparseColumnLayout;
+        return !isStatic && useSparseColumnLayout;
     }
 
     public static SerializationHeader forKeyCache(CFMetaData metadata)
@@ -159,13 +158,7 @@ public class SerializationHeader
 
     private static List<AbstractType<?>> typesOf(List<ColumnDefinition> columns)
     {
-        return ImmutableList.copyOf(Lists.transform(columns, new Function<ColumnDefinition, AbstractType<?>>()
-        {
-            public AbstractType<?> apply(ColumnDefinition column)
-            {
-                return column.type;
-            }
-        }));
+        return ImmutableList.copyOf(Lists.transform(columns, column -> column.type));
     }
 
     public PartitionColumns columns()
@@ -365,7 +358,7 @@ public class SerializationHeader
             Columns.serializer.serialize(header.columns.regulars, out);
         }
 
-        public SerializationHeader deserializeForMessaging(DataInput in, CFMetaData metadata, boolean hasStatic) throws IOException
+        public SerializationHeader deserializeForMessaging(DataInputPlus in, CFMetaData metadata, boolean hasStatic) throws IOException
         {
             RowStats stats = RowStats.serializer.deserialize(in);
 
@@ -458,7 +451,7 @@ public class SerializationHeader
             return size;
         }
 
-        private void readColumnsWithType(DataInput in, Map<ByteBuffer, AbstractType<?>> typeMap) throws IOException
+        private void readColumnsWithType(DataInputPlus in, Map<ByteBuffer, AbstractType<?>> typeMap) throws IOException
         {
             int length = in.readUnsignedShort();
             for (int i = 0; i < length; i++)
@@ -474,7 +467,7 @@ public class SerializationHeader
             ByteBufferUtil.writeWithLength(UTF8Type.instance.decompose(type.toString()), out);
         }
 
-        private AbstractType<?> readType(DataInput in) throws IOException
+        private AbstractType<?> readType(DataInputPlus in) throws IOException
         {
             ByteBuffer raw = ByteBufferUtil.readWithLength(in);
             return TypeParser.parse(UTF8Type.instance.compose(raw));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/SimpleClustering.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SimpleClustering.java b/src/java/org/apache/cassandra/db/SimpleClustering.java
deleted file mode 100644
index 8b1cb7b..0000000
--- a/src/java/org/apache/cassandra/db/SimpleClustering.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * 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.nio.ByteBuffer;
-
-import org.apache.cassandra.utils.ObjectSizes;
-
-public class SimpleClustering extends Clustering
-{
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new SimpleClustering(new ByteBuffer[0]));
-
-    private final ByteBuffer[] values;
-
-    public SimpleClustering(ByteBuffer... values)
-    {
-        this.values = values;
-    }
-
-    public SimpleClustering(ByteBuffer value)
-    {
-        this(new ByteBuffer[]{ value });
-    }
-
-    public int size()
-    {
-        return values.length;
-    }
-
-    public ByteBuffer get(int i)
-    {
-        return values[i];
-    }
-
-    public ByteBuffer[] getRawValues()
-    {
-        return values;
-    }
-
-    @Override
-    public long unsharedHeapSize()
-    {
-        return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(values);
-    }
-
-    @Override
-    public Clustering takeAlias()
-    {
-        return this;
-    }
-
-    public static Builder builder(int size)
-    {
-        return new Builder(size);
-    }
-
-    public static class Builder implements Writer
-    {
-        private final ByteBuffer[] values;
-        private int idx;
-
-        private Builder(int size)
-        {
-            this.values = new ByteBuffer[size];
-        }
-
-        public void writeClusteringValue(ByteBuffer value)
-        {
-            values[idx++] = value;
-        }
-
-        public SimpleClustering build()
-        {
-            assert idx == values.length;
-            return new SimpleClustering(values);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/SimpleDeletionTime.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SimpleDeletionTime.java b/src/java/org/apache/cassandra/db/SimpleDeletionTime.java
deleted file mode 100644
index 738c5e6..0000000
--- a/src/java/org/apache/cassandra/db/SimpleDeletionTime.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.IOException;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Objects;
-
-import org.apache.cassandra.cache.IMeasurableMemory;
-import org.apache.cassandra.io.ISerializer;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.utils.ObjectSizes;
-
-/**
- * Simple implementation of DeletionTime.
- */
-public class SimpleDeletionTime extends DeletionTime
-{
-    public final long markedForDeleteAt;
-    public final int localDeletionTime;
-
-    @VisibleForTesting
-    public SimpleDeletionTime(long markedForDeleteAt, int localDeletionTime)
-    {
-        this.markedForDeleteAt = markedForDeleteAt;
-        this.localDeletionTime = localDeletionTime;
-    }
-
-    public long markedForDeleteAt()
-    {
-        return markedForDeleteAt;
-    }
-
-    public int localDeletionTime()
-    {
-        return localDeletionTime;
-    }
-
-    public DeletionTime takeAlias()
-    {
-        return this;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/SimpleLivenessInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SimpleLivenessInfo.java b/src/java/org/apache/cassandra/db/SimpleLivenessInfo.java
deleted file mode 100644
index fea1b86..0000000
--- a/src/java/org/apache/cassandra/db/SimpleLivenessInfo.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.util.Objects;
-import java.security.MessageDigest;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.utils.FBUtilities;
-
-public class SimpleLivenessInfo extends AbstractLivenessInfo
-{
-    private final long timestamp;
-    private final int ttl;
-    private final int localDeletionTime;
-
-    // Note that while some code use this ctor, the two following static creation methods
-    // are usually less error prone.
-    SimpleLivenessInfo(long timestamp, int ttl, int localDeletionTime)
-    {
-        this.timestamp = timestamp;
-        this.ttl = ttl;
-        this.localDeletionTime = localDeletionTime;
-    }
-
-    public static SimpleLivenessInfo forUpdate(long timestamp, int ttl, int nowInSec, CFMetaData metadata)
-    {
-        if (ttl == NO_TTL)
-            ttl = metadata.getDefaultTimeToLive();
-
-        return new SimpleLivenessInfo(timestamp, ttl, ttl == NO_TTL ? NO_DELETION_TIME : nowInSec + ttl);
-    }
-
-    public static SimpleLivenessInfo forDeletion(long timestamp, int localDeletionTime)
-    {
-        return new SimpleLivenessInfo(timestamp, NO_TTL, localDeletionTime);
-    }
-
-    public long timestamp()
-    {
-        return timestamp;
-    }
-
-    public int ttl()
-    {
-        return ttl;
-    }
-
-    public int localDeletionTime()
-    {
-        return localDeletionTime;
-    }
-
-    @Override
-    public LivenessInfo takeAlias()
-    {
-        return this;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/SinglePartitionNamesCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionNamesCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionNamesCommand.java
index cb43cd3..53ead14 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionNamesCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionNamesCommand.java
@@ -146,7 +146,7 @@ public class SinglePartitionNamesCommand extends SinglePartitionReadCommand<Clus
 
             try (UnfilteredRowIterator iter = result.unfilteredIterator(columnFilter(), Slices.ALL, false))
             {
-                final Mutation mutation = new Mutation(UnfilteredRowIterators.toUpdate(iter));
+                final Mutation mutation = new Mutation(PartitionUpdate.fromIterator(iter));
                 StageManager.getStage(Stage.MUTATION).execute(new Runnable()
                 {
                     public void run()
@@ -232,7 +232,7 @@ public class SinglePartitionNamesCommand extends SinglePartitionReadCommand<Clus
         // We can remove a row if it has data that is more recent that the next sstable to consider for the data that the query
         // cares about. And the data we care about is 1) the row timestamp (since every query cares if the row exists or not)
         // and 2) the requested columns.
-        if (!row.primaryKeyLivenessInfo().hasTimestamp() || row.primaryKeyLivenessInfo().timestamp() <= sstableTimestamp)
+        if (row.primaryKeyLivenessInfo().isEmpty() || row.primaryKeyLivenessInfo().timestamp() <= sstableTimestamp)
             return false;
 
         for (ColumnDefinition column : requestedColumns)
@@ -242,7 +242,7 @@ public class SinglePartitionNamesCommand extends SinglePartitionReadCommand<Clus
                 return false;
 
             Cell cell = row.getCell(column);
-            if (cell == null || cell.livenessInfo().timestamp() <= sstableTimestamp)
+            if (cell == null || cell.timestamp() <= sstableTimestamp)
                 return false;
         }
         return true;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index f9f583f..80711d6 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.util.*;
 
@@ -29,6 +28,7 @@ import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.metrics.TableMetrics;
 import org.apache.cassandra.service.*;
@@ -483,7 +483,7 @@ public abstract class SinglePartitionReadCommand<F extends ClusteringIndexFilter
 
     private static class Deserializer extends SelectionDeserializer
     {
-        public ReadCommand deserialize(DataInput in, int version, boolean isDigest, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits)
+        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits)
         throws IOException
         {
             DecoratedKey key = StorageService.getPartitioner().decorateKey(metadata.getKeyValidator().readValue(in));


[02/15] cassandra git commit: Simplify some 8099's implementations

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index deb1480..065479b 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -146,7 +146,7 @@ public class ColumnFamilyStoreTest
             public void runMayThrow() throws IOException
             {
                 Row toCheck = Util.getOnlyRowUnfiltered(Util.cmd(cfs, "key1").build());
-                Iterator<Cell> iter = toCheck.iterator();
+                Iterator<Cell> iter = toCheck.cells().iterator();
                 assert(Iterators.size(iter) == 0);
             }
         };
@@ -254,7 +254,7 @@ public class ColumnFamilyStoreTest
         ByteBuffer val = ByteBufferUtil.bytes("val1");
 
         // insert
-        ColumnDefinition newCol = new ColumnDefinition(cfs.metadata, ByteBufferUtil.bytes("val2"), AsciiType.instance, 1, ColumnDefinition.Kind.REGULAR);
+        ColumnDefinition newCol = ColumnDefinition.regularDef(cfs.metadata, ByteBufferUtil.bytes("val2"), AsciiType.instance);
         new RowUpdateBuilder(cfs.metadata, 0, "key1").clustering("Column1").add("val", "val1").build().applyUnsafe();
         new RowUpdateBuilder(cfs.metadata, 0, "key2").clustering("Column1").add("val", "val1").build().applyUnsafe();
         assertRangeCount(cfs, col, val, 2);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CommitLogTest.java b/test/unit/org/apache/cassandra/db/CommitLogTest.java
index 5838e23..21bdd9b 100644
--- a/test/unit/org/apache/cassandra/db/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/CommitLogTest.java
@@ -60,6 +60,7 @@ import org.apache.cassandra.io.util.FileDataInput;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.vint.VIntCoding;
 
 public class CommitLogTest
 {
@@ -230,7 +231,7 @@ public class CommitLogTest
     private static int getMaxRecordDataSize(String keyspace, ByteBuffer key, String cfName, String colName)
     {
         ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(cfName);
-        // We don't want to allocate a size of 0 as this is optimize under the hood and our computation would
+        // We don't want to allocate a size of 0 as this is optimized under the hood and our computation would
         // break testEqualRecordLimit
         int allocSize = 1;
         Mutation rm = new RowUpdateBuilder(cfs.metadata, 0, key)
@@ -239,7 +240,16 @@ public class CommitLogTest
 
         int max = (DatabaseDescriptor.getCommitLogSegmentSize() / 2);
         max -= CommitLogSegment.ENTRY_OVERHEAD_SIZE; // log entry overhead
-        return max - (int) Mutation.serializer.serializedSize(rm, MessagingService.current_version) + allocSize;
+
+        // Note that the size of the value if vint encoded. So we first compute the ovehead of the mutation without the value and it's size
+        int mutationOverhead = (int)Mutation.serializer.serializedSize(rm, MessagingService.current_version) - (VIntCoding.computeVIntSize(allocSize) + allocSize);
+        max -= mutationOverhead;
+
+        // Now, max is the max for both the value and it's size. But we want to know how much we can allocate, i.e. the size of the value.
+        int sizeOfMax = VIntCoding.computeVIntSize(max);
+        max -= sizeOfMax;
+        assert VIntCoding.computeVIntSize(max) == sizeOfMax; // sanity check that we're still encoded with the size we though we would
+        return max;
     }
 
     private static int getMaxRecordDataSize()
@@ -351,7 +361,7 @@ public class CommitLogTest
                 .applyUnsafe();
 
             assertTrue(Util.getOnlyRow(Util.cmd(cfs).columns("val").build())
-                            .iterator().next().value().equals(ByteBufferUtil.bytes("abcd")));
+                            .cells().iterator().next().value().equals(ByteBufferUtil.bytes("abcd")));
 
             cfs.truncateBlocking();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/CounterCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterCellTest.java b/test/unit/org/apache/cassandra/db/CounterCellTest.java
index fb9f9ac..08e0b25 100644
--- a/test/unit/org/apache/cassandra/db/CounterCellTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -29,7 +29,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.rows.AbstractCell;
+import org.apache.cassandra.db.rows.BufferCell;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.CellPath;
 import org.apache.cassandra.db.rows.Cells;
@@ -80,52 +80,13 @@ public class CounterCellTest
         stepLength    = idLength + clockLength + countLength;
     }
 
-    private class TestCounterCell extends AbstractCell
-    {
-        private final ColumnDefinition column;
-        private final ByteBuffer value;
-        private final LivenessInfo info;
-
-        private TestCounterCell(ColumnDefinition column, ByteBuffer value, LivenessInfo info)
-        {
-            this.column = column;
-            this.value = value;
-            this.info = info.takeAlias();
-        }
-
-        public ColumnDefinition column()
-        {
-            return column;
-        }
-
-        public boolean isCounterCell()
-        {
-            return true;
-        }
-
-        public ByteBuffer value()
-        {
-            return value;
-        }
-
-        public LivenessInfo livenessInfo()
-        {
-            return info;
-        }
-
-        public CellPath path()
-        {
-            return null;
-        }
-    }
-
     @Test
     public void testCreate()
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(COUNTER1);
         long delta = 3L;
 
-        TestCounterCell cell = createLegacyCounterCell(cfs, ByteBufferUtil.bytes("c1"), delta, 1, 0, 0);
+        Cell cell = createLegacyCounterCell(cfs, ByteBufferUtil.bytes("val"), delta, 1);
 
         assertEquals(delta, CounterContext.instance().total(cell.value()));
         assertEquals(1, cell.value().getShort(0));
@@ -136,131 +97,115 @@ public class CounterCellTest
 
     }
 
-    private TestCounterCell createLegacyCounterCell(ColumnFamilyStore cfs,
-                                                    ByteBuffer colName,
-                                                    long count,
-                                                    long ts,
-                                                    int ttl,
-                                                    int localDeletion)
+    private Cell createLegacyCounterCell(ColumnFamilyStore cfs, ByteBuffer colName, long count, long ts)
     {
         ColumnDefinition cDef = cfs.metadata.getColumnDefinition(colName);
         ByteBuffer val = CounterContext.instance().createLocal(count);
-        LivenessInfo li = new SimpleLivenessInfo(ts, ttl, localDeletion);
-        return new TestCounterCell(cDef, val, li);
+        return BufferCell.live(cfs.metadata, cDef, ts, val);
     }
 
-    private TestCounterCell createCounterCell(ColumnFamilyStore cfs,
-                                              ByteBuffer colName,
-                                              CounterId id,
-                                              long count,
-                                              long ts,
-                                              int ttl,
-                                              int localDeletion)
+    private Cell createCounterCell(ColumnFamilyStore cfs, ByteBuffer colName, CounterId id, long count, long ts)
     {
         ColumnDefinition cDef = cfs.metadata.getColumnDefinition(colName);
         ByteBuffer val = CounterContext.instance().createGlobal(id, ts, count);
-        LivenessInfo li = new SimpleLivenessInfo(ts, ttl, localDeletion);
-        return new TestCounterCell(cDef, val, li);
+        return BufferCell.live(cfs.metadata, cDef, ts, val);
+    }
+
+    private Cell createCounterCellFromContext(ColumnFamilyStore cfs, ByteBuffer colName, ContextState context, long ts)
+    {
+        ColumnDefinition cDef = cfs.metadata.getColumnDefinition(colName);
+        return BufferCell.live(cfs.metadata, cDef, ts, context.context);
     }
 
-    private TestCounterCell createCounterCellFromContext(ColumnFamilyStore cfs,
-                                                         ByteBuffer colName,
-                                                         ContextState context,
-                                                         long ts,
-                                                         int ttl,
-                                                         int localDeletion)
+    private Cell createDeleted(ColumnFamilyStore cfs, ByteBuffer colName, long ts, int localDeletionTime)
     {
         ColumnDefinition cDef = cfs.metadata.getColumnDefinition(colName);
-        LivenessInfo li = new SimpleLivenessInfo(ts, ttl, localDeletion);
-        return new TestCounterCell(cDef, context.context, li);
+        return BufferCell.tombstone(cDef, ts, localDeletionTime);
     }
 
     @Test
     public void testReconcile()
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(COUNTER1);
-        ColumnDefinition cDef = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1).metadata.getColumnDefinition(ByteBufferUtil.bytes("val"));
-        ByteBuffer col = ByteBufferUtil.bytes("c1");
+        ByteBuffer col = ByteBufferUtil.bytes("val");
 
-        AbstractCell left;
-        AbstractCell right;
+        Cell left;
+        Cell right;
 
         // both deleted, diff deletion time, same ts
-        left = createLegacyCounterCell(cfs, col, 1, 2, 0, 5);
-        right = createLegacyCounterCell(cfs, col, 1, 2, 0, 10);
+        left = createDeleted(cfs, col, 2, 5);
+        right = createDeleted(cfs, col, 2, 10);
         assert Cells.reconcile(left, right, 10) == right;
 
         // diff ts
-        right = createLegacyCounterCell(cfs, col, 1, 1, 0, 10);
+        right = createLegacyCounterCell(cfs, col, 1, 10);
         assert Cells.reconcile(left, right, 10) == left;
 
         // < tombstone
-        left = new CellTest.TestCell(cDef, ByteBufferUtil.bytes(5L), SimpleLivenessInfo.forDeletion(6, 6));
-        right = createLegacyCounterCell(cfs, col, 1, 5, 0, 5);
+        left = createDeleted(cfs, col, 6, 6);
+        right = createLegacyCounterCell(cfs, col, 1, 5);
         assert Cells.reconcile(left, right, 10) == left;
 
         // > tombstone
-        left = new CellTest.TestCell(cDef, ByteBufferUtil.bytes(5L), SimpleLivenessInfo.forDeletion(1, 1));
-        right = createLegacyCounterCell(cfs, col, 1, 5, 0, Integer.MAX_VALUE);
+        left = createDeleted(cfs, col, 1, 1);
+        right = createLegacyCounterCell(cfs, col, 1, 5);
         assert Cells.reconcile(left, right, 10) == left;
 
         // == tombstone
-        left = new CellTest.TestCell(cDef, ByteBufferUtil.bytes(5L), SimpleLivenessInfo.forDeletion(8, 8));
-        right = createLegacyCounterCell(cfs, col, 1, 8, 0, Integer.MAX_VALUE);
+        left = createDeleted(cfs, col, 8, 8);
+        right = createLegacyCounterCell(cfs, col, 1, 8);
         assert Cells.reconcile(left, right, 10) == left;
 
         // live + live
-        left = createLegacyCounterCell(cfs, col, 1, 2, 0, Integer.MAX_VALUE);
-        right = createLegacyCounterCell(cfs, col, 3, 5, 0, Integer.MAX_VALUE);
+        left = createLegacyCounterCell(cfs, col, 1, 2);
+        right = createLegacyCounterCell(cfs, col, 3, 5);
         Cell reconciled = Cells.reconcile(left, right, 10);
         assertEquals(CounterContext.instance().total(reconciled.value()), 4);
-        assertEquals(reconciled.livenessInfo().timestamp(), 5L);
+        assertEquals(reconciled.timestamp(), 5L);
 
         // Add, don't change TS
-        Cell addTen = createLegacyCounterCell(cfs, col, 10, 4, 0, Integer.MAX_VALUE);
+        Cell addTen = createLegacyCounterCell(cfs, col, 10, 4);
         reconciled = Cells.reconcile(reconciled, addTen, 10);
         assertEquals(CounterContext.instance().total(reconciled.value()), 14);
-        assertEquals(reconciled.livenessInfo().timestamp(), 5L);
+        assertEquals(reconciled.timestamp(), 5L);
 
         // Add w/new TS
-        Cell addThree = createLegacyCounterCell(cfs, col, 3, 7, 0, Integer.MAX_VALUE);
+        Cell addThree = createLegacyCounterCell(cfs, col, 3, 7);
         reconciled = Cells.reconcile(reconciled, addThree, 10);
         assertEquals(CounterContext.instance().total(reconciled.value()), 17);
-        assertEquals(reconciled.livenessInfo().timestamp(), 7L);
+        assertEquals(reconciled.timestamp(), 7L);
 
         // Confirm no deletion time
-        assert reconciled.livenessInfo().localDeletionTime() == Integer.MAX_VALUE;
+        assert reconciled.localDeletionTime() == Integer.MAX_VALUE;
 
-        Cell deleted = createLegacyCounterCell(cfs, col, 2, 8, 0, 8);
+        Cell deleted = createDeleted(cfs, col, 8, 8);
         reconciled = Cells.reconcile(reconciled, deleted, 10);
-        assertEquals(2, CounterContext.instance().total(reconciled.value()));
-        assertEquals(reconciled.livenessInfo().timestamp(), 8L);
-        assert reconciled.livenessInfo().localDeletionTime() == 8;
+        assert reconciled.localDeletionTime() == 8;
     }
 
     @Test
     public void testDiff()
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(COUNTER1);
-        ByteBuffer col = ByteBufferUtil.bytes("c1");
+        ByteBuffer col = ByteBufferUtil.bytes("val");
 
-        TestCounterCell leftCell;
-        TestCounterCell rightCell;
+        Cell leftCell;
+        Cell rightCell;
 
         // Equal count
-        leftCell = createLegacyCounterCell(cfs, col, 2, 2, 0, Integer.MAX_VALUE);
-        rightCell = createLegacyCounterCell(cfs, col, 2, 1, 0, Integer.MAX_VALUE);
+        leftCell = createLegacyCounterCell(cfs, col, 2, 2);
+        rightCell = createLegacyCounterCell(cfs, col, 2, 1);
         assertEquals(CounterContext.Relationship.EQUAL, CounterContext.instance().diff(leftCell.value(), rightCell.value()));
 
         // Non-equal count
-        leftCell = createLegacyCounterCell(cfs, col, 1, 2, 0, Integer.MAX_VALUE);
-        rightCell = createLegacyCounterCell(cfs, col, 2, 1, 0, Integer.MAX_VALUE);
+        leftCell = createLegacyCounterCell(cfs, col, 1, 2);
+        rightCell = createLegacyCounterCell(cfs, col, 2, 1);
         assertEquals(CounterContext.Relationship.DISJOINT, CounterContext.instance().diff(leftCell.value(), rightCell.value()));
 
         // timestamp
         CounterId id = CounterId.generate();
-        leftCell = createCounterCell(cfs, col, id, 2, 2, 0, Integer.MAX_VALUE);
-        rightCell = createCounterCell(cfs, col, id, 2, 1, 0, Integer.MAX_VALUE);
+        leftCell = createCounterCell(cfs, col, id, 2, 2);
+        rightCell = createCounterCell(cfs, col, id, 2, 1);
         assertEquals(CounterContext.Relationship.GREATER_THAN, CounterContext.instance().diff(leftCell.value(), rightCell.value()));
 
         ContextState leftContext;
@@ -273,9 +218,9 @@ public class CounterCellTest
         leftContext.writeRemote(CounterId.fromInt(9), 1L, 0L);
         rightContext = ContextState.wrap(ByteBufferUtil.clone(leftContext.context));
 
-        leftCell = createCounterCellFromContext(cfs, col, leftContext, 1, 0, Integer.MAX_VALUE);
-        rightCell = createCounterCellFromContext(cfs, col, rightContext, 1, 0, Integer.MAX_VALUE);
-        assertEquals(CounterContext.Relationship.EQUAL, CounterContext.instance().diff(leftCell.value, rightCell.value));
+        leftCell = createCounterCellFromContext(cfs, col, leftContext, 1);
+        rightCell = createCounterCellFromContext(cfs, col, rightContext, 1);
+        assertEquals(CounterContext.Relationship.EQUAL, CounterContext.instance().diff(leftCell.value(), rightCell.value()));
 
         // greater than: left has superset of nodes (counts equal)
         leftContext = ContextState.allocate(0, 0, 4);
@@ -289,10 +234,10 @@ public class CounterCellTest
         rightContext.writeRemote(CounterId.fromInt(6), 2L, 0L);
         rightContext.writeRemote(CounterId.fromInt(9), 1L, 0L);
 
-        leftCell = createCounterCellFromContext(cfs, col, leftContext, 1, 0, Integer.MAX_VALUE);
-        rightCell = createCounterCellFromContext(cfs, col, rightContext, 1, 0, Integer.MAX_VALUE);
-        assertEquals(CounterContext.Relationship.GREATER_THAN, CounterContext.instance().diff(leftCell.value, rightCell.value));
-        assertEquals(CounterContext.Relationship.LESS_THAN, CounterContext.instance().diff(rightCell.value, leftCell.value));
+        leftCell = createCounterCellFromContext(cfs, col, leftContext, 1);
+        rightCell = createCounterCellFromContext(cfs, col, rightContext, 1);
+        assertEquals(CounterContext.Relationship.GREATER_THAN, CounterContext.instance().diff(leftCell.value(), rightCell.value()));
+        assertEquals(CounterContext.Relationship.LESS_THAN, CounterContext.instance().diff(rightCell.value(), leftCell.value()));
 
         // disjoint: right and left have disjoint node sets
         leftContext = ContextState.allocate(0, 0, 3);
@@ -305,17 +250,17 @@ public class CounterCellTest
         rightContext.writeRemote(CounterId.fromInt(6), 1L, 0L);
         rightContext.writeRemote(CounterId.fromInt(9), 1L, 0L);
 
-        leftCell = createCounterCellFromContext(cfs, col, leftContext, 1, 0, Integer.MAX_VALUE);
-        rightCell = createCounterCellFromContext(cfs, col, rightContext, 1, 0, Integer.MAX_VALUE);
-        assertEquals(CounterContext.Relationship.DISJOINT, CounterContext.instance().diff(leftCell.value, rightCell.value));
-        assertEquals(CounterContext.Relationship.DISJOINT, CounterContext.instance().diff(rightCell.value, leftCell.value));
+        leftCell = createCounterCellFromContext(cfs, col, leftContext, 1);
+        rightCell = createCounterCellFromContext(cfs, col, rightContext, 1);
+        assertEquals(CounterContext.Relationship.DISJOINT, CounterContext.instance().diff(leftCell.value(), rightCell.value()));
+        assertEquals(CounterContext.Relationship.DISJOINT, CounterContext.instance().diff(rightCell.value(), leftCell.value()));
     }
 
     @Test
     public void testUpdateDigest() throws Exception
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(COUNTER1);
-        ByteBuffer col = ByteBufferUtil.bytes("c1");
+        ByteBuffer col = ByteBufferUtil.bytes("val");
 
         MessageDigest digest1 = MessageDigest.getInstance("md5");
         MessageDigest digest2 = MessageDigest.getInstance("md5");
@@ -326,14 +271,13 @@ public class CounterCellTest
         state.writeRemote(CounterId.fromInt(3), 4L, 4L);
         state.writeLocal(CounterId.fromInt(4), 4L, 4L);
 
-        TestCounterCell original = createCounterCellFromContext(cfs, col, state, 5, 0, Integer.MAX_VALUE);
+        Cell original = createCounterCellFromContext(cfs, col, state, 5);
 
         ColumnDefinition cDef = cfs.metadata.getColumnDefinition(col);
-        LivenessInfo li = new SimpleLivenessInfo(5, 0, Integer.MAX_VALUE);
-        TestCounterCell cleared = new TestCounterCell(cDef, CounterContext.instance().clearAllLocal(state.context), li);
+        Cell cleared = BufferCell.live(cfs.metadata, cDef, 5, CounterContext.instance().clearAllLocal(state.context));
 
-        CounterContext.instance().updateDigest(digest1, original.value);
-        CounterContext.instance().updateDigest(digest2, cleared.value);
+        CounterContext.instance().updateDigest(digest1, original.value());
+        CounterContext.instance().updateDigest(digest2, cleared.value());
 
         assert Arrays.equals(digest1.digest(), digest2.digest());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/KeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyspaceTest.java b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
index fdd4f0c..8754209 100644
--- a/test/unit/org/apache/cassandra/db/KeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
@@ -129,8 +129,8 @@ public class KeyspaceTest extends CQLTester
 
     private static void assertRowsInSlice(ColumnFamilyStore cfs, String key, int sliceStart, int sliceEnd, int limit, boolean reversed, String columnValuePrefix)
     {
-        Clustering startClustering = new SimpleClustering(ByteBufferUtil.bytes(sliceStart));
-        Clustering endClustering = new SimpleClustering(ByteBufferUtil.bytes(sliceEnd));
+        Clustering startClustering = new Clustering(ByteBufferUtil.bytes(sliceStart));
+        Clustering endClustering = new Clustering(ByteBufferUtil.bytes(sliceEnd));
         Slices slices = Slices.with(cfs.getComparator(), Slice.make(startClustering, endClustering));
         ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(slices, reversed);
         SinglePartitionSliceCommand command = singlePartitionSlice(cfs, key, filter, limit);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/RangeTombstoneListTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneListTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneListTest.java
index 8a697f4..d6d6d07 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneListTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneListTest.java
@@ -375,7 +375,7 @@ public class RangeTombstoneListTest
 
     private static Clustering clustering(int i)
     {
-        return new SimpleClustering(bb(i));
+        return new Clustering(bb(i));
     }
 
     private static ByteBuffer bb(int i)
@@ -395,12 +395,12 @@ public class RangeTombstoneListTest
 
     private static RangeTombstone rt(int start, boolean startInclusive, int end, boolean endInclusive, long tstamp)
     {
-        return new RangeTombstone(Slice.make(Slice.Bound.create(cmp, true, startInclusive, start), Slice.Bound.create(cmp, false, endInclusive, end)), new SimpleDeletionTime(tstamp, 0));
+        return new RangeTombstone(Slice.make(Slice.Bound.create(cmp, true, startInclusive, start), Slice.Bound.create(cmp, false, endInclusive, end)), new DeletionTime(tstamp, 0));
     }
 
     private static RangeTombstone rt(int start, int end, long tstamp, int delTime)
     {
-        return new RangeTombstone(Slice.make(Slice.Bound.inclusiveStartOf(bb(start)), Slice.Bound.inclusiveEndOf(bb(end))), new SimpleDeletionTime(tstamp, delTime));
+        return new RangeTombstone(Slice.make(Slice.Bound.inclusiveStartOf(bb(start)), Slice.Bound.inclusiveEndOf(bb(end))), new DeletionTime(tstamp, delTime));
     }
 
     private static RangeTombstone rtei(int start, int end, long tstamp)
@@ -410,7 +410,7 @@ public class RangeTombstoneListTest
 
     private static RangeTombstone rtei(int start, int end, long tstamp, int delTime)
     {
-        return new RangeTombstone(Slice.make(Slice.Bound.exclusiveStartOf(bb(start)), Slice.Bound.inclusiveEndOf(bb(end))), new SimpleDeletionTime(tstamp, delTime));
+        return new RangeTombstone(Slice.make(Slice.Bound.exclusiveStartOf(bb(start)), Slice.Bound.inclusiveEndOf(bb(end))), new DeletionTime(tstamp, delTime));
     }
 
     private static RangeTombstone rtie(int start, int end, long tstamp)
@@ -420,6 +420,6 @@ public class RangeTombstoneListTest
 
     private static RangeTombstone rtie(int start, int end, long tstamp, int delTime)
     {
-        return new RangeTombstone(Slice.make(Slice.Bound.inclusiveStartOf(bb(start)), Slice.Bound.exclusiveEndOf(bb(end))), new SimpleDeletionTime(tstamp, delTime));
+        return new RangeTombstone(Slice.make(Slice.Bound.inclusiveStartOf(bb(start)), Slice.Bound.exclusiveEndOf(bb(end))), new DeletionTime(tstamp, delTime));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 8fb0dd0..c83103a 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -56,6 +56,7 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 public class RangeTombstoneTest
@@ -110,19 +111,20 @@ public class RangeTombstoneTest
             cmdBuilder.includeRow(i);
 
         Partition partition = Util.getOnlyPartitionUnfiltered(cmdBuilder.build());
+        int nowInSec = FBUtilities.nowInSeconds();
 
         for (int i : live)
-            assertTrue("Row " + i + " should be live", partition.getRow(new SimpleClustering(bb(i))).hasLiveData(FBUtilities.nowInSeconds()));
+            assertTrue("Row " + i + " should be live", partition.getRow(new Clustering(bb(i))).hasLiveData(nowInSec));
         for (int i : dead)
-            assertTrue("Row " + i + " shouldn't be live", !partition.getRow(new SimpleClustering(bb(i))).hasLiveData(FBUtilities.nowInSeconds()));
+            assertFalse("Row " + i + " shouldn't be live", partition.getRow(new Clustering(bb(i))).hasLiveData(nowInSec));
 
         // Queries by slices
         partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).fromIncl(7).toIncl(30).build());
 
         for (int i : new int[]{ 7, 8, 9, 11, 13, 15, 17, 28, 29, 30 })
-            assertTrue("Row " + i + " should be live", partition.getRow(new SimpleClustering(bb(i))).hasLiveData(FBUtilities.nowInSeconds()));
+            assertTrue("Row " + i + " should be live", partition.getRow(new Clustering(bb(i))).hasLiveData(nowInSec));
         for (int i : new int[]{ 10, 12, 14, 16, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27 })
-            assertTrue("Row " + i + " shouldn't be live", partition.getRow(new SimpleClustering(bb(i))) == null);
+            assertFalse("Row " + i + " shouldn't be live", partition.getRow(new Clustering(bb(i))).hasLiveData(nowInSec));
     }
 
     @Test
@@ -405,24 +407,25 @@ public class RangeTombstoneTest
         cfs.forceBlockingFlush();
 
         Partition partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).build());
+        int nowInSec = FBUtilities.nowInSeconds();
 
         for (int i = 0; i < 5; i++)
-            assertTrue("Row " + i + " should be live", partition.getRow(new SimpleClustering(bb(i))).hasLiveData(FBUtilities.nowInSeconds()));
+            assertTrue("Row " + i + " should be live", partition.getRow(new Clustering(bb(i))).hasLiveData(nowInSec));
         for (int i = 16; i < 20; i++)
-            assertTrue("Row " + i + " should be live", partition.getRow(new SimpleClustering(bb(i))).hasLiveData(FBUtilities.nowInSeconds()));
+            assertTrue("Row " + i + " should be live", partition.getRow(new Clustering(bb(i))).hasLiveData(nowInSec));
         for (int i = 5; i <= 15; i++)
-            assertTrue("Row " + i + " shouldn't be live", partition.getRow(new SimpleClustering(bb(i))) == null);
+            assertFalse("Row " + i + " shouldn't be live", partition.getRow(new Clustering(bb(i))).hasLiveData(nowInSec));
 
         // Compact everything and re-test
         CompactionManager.instance.performMaximal(cfs, false);
         partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).build());
 
         for (int i = 0; i < 5; i++)
-            assertTrue("Row " + i + " should be live", partition.getRow(new SimpleClustering(bb(i))).hasLiveData(FBUtilities.nowInSeconds()));
+            assertTrue("Row " + i + " should be live", partition.getRow(new Clustering(bb(i))).hasLiveData(FBUtilities.nowInSeconds()));
         for (int i = 16; i < 20; i++)
-            assertTrue("Row " + i + " should be live", partition.getRow(new SimpleClustering(bb(i))).hasLiveData(FBUtilities.nowInSeconds()));
+            assertTrue("Row " + i + " should be live", partition.getRow(new Clustering(bb(i))).hasLiveData(FBUtilities.nowInSeconds()));
         for (int i = 5; i <= 15; i++)
-            assertTrue("Row " + i + " shouldn't be live", partition.getRow(new SimpleClustering(bb(i))) == null);
+            assertFalse("Row " + i + " shouldn't be live", partition.getRow(new Clustering(bb(i))).hasLiveData(nowInSec));
     }
 
     @Test
@@ -576,7 +579,7 @@ public class RangeTombstoneTest
         // compacted down to single sstable
         assertEquals(1, cfs.getSSTables().size());
 
-        assertEquals(10, index.deletes.size());
+        assertEquals(8, index.deletes.size());
     }
 
     private static ByteBuffer bb(int i)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/ReadMessageTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ReadMessageTest.java b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
index 2475821..3c53934 100644
--- a/test/unit/org/apache/cassandra/db/ReadMessageTest.java
+++ b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
@@ -219,7 +219,7 @@ public class ReadMessageTest
         {
             for (PartitionUpdate upd : mutation.getPartitionUpdates())
             {
-                Row r = upd.getRow(new SimpleClustering(ByteBufferUtil.bytes("c")));
+                Row r = upd.getRow(new Clustering(ByteBufferUtil.bytes("c")));
                 if (r != null)
                 {
                     if (r.getCell(withCommit) != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java
index 14c9832..9275dae 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java
@@ -82,7 +82,7 @@ public class RecoveryManagerMissingHeaderTest
 
         CommitLog.instance.resetUnsafe(false);
 
-        Assert.assertTrue(AbstractUnfilteredRowIterator.equal(upd1, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace1.getColumnFamilyStore(CF_STANDARD1), dk).build()).unfilteredIterator()));
-        Assert.assertTrue(AbstractUnfilteredRowIterator.equal(upd2, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace2.getColumnFamilyStore(CF_STANDARD3), dk).build()).unfilteredIterator()));
+        Assert.assertTrue(Util.equal(upd1, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace1.getColumnFamilyStore(CF_STANDARD1), dk).build()).unfilteredIterator()));
+        Assert.assertTrue(Util.equal(upd2, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace2.getColumnFamilyStore(CF_STANDARD3), dk).build()).unfilteredIterator()));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
index 9fc37be..baf9466 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
@@ -106,8 +106,8 @@ public class RecoveryManagerTest
         CommitLog.instance.resetUnsafe(false);
 
         DecoratedKey dk = Util.dk("keymulti");
-        Assert.assertTrue(AbstractUnfilteredRowIterator.equal(upd1, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace1.getColumnFamilyStore(CF_STANDARD1), dk).build()).unfilteredIterator()));
-        Assert.assertTrue(AbstractUnfilteredRowIterator.equal(upd2, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace2.getColumnFamilyStore(CF_STANDARD3), dk).build()).unfilteredIterator()));
+        Assert.assertTrue(Util.equal(upd1, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace1.getColumnFamilyStore(CF_STANDARD1), dk).build()).unfilteredIterator()));
+        Assert.assertTrue(Util.equal(upd2, Util.getOnlyPartitionUnfiltered(Util.cmd(keyspace2.getColumnFamilyStore(CF_STANDARD3), dk).build()).unfilteredIterator()));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/RowCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowCacheTest.java b/test/unit/org/apache/cassandra/db/RowCacheTest.java
index 6289c96..883149f 100644
--- a/test/unit/org/apache/cassandra/db/RowCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/RowCacheTest.java
@@ -32,10 +32,7 @@ import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.cache.RowCacheKey;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.rows.Unfiltered;
-import org.apache.cassandra.db.rows.UnfilteredRowIterator;
-import org.apache.cassandra.db.rows.Cell;
-import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.marshal.IntegerType;
@@ -113,10 +110,9 @@ public class RowCacheTest
         for (Unfiltered unfiltered : Util.once(cachedCf.unfilteredIterator(ColumnFilter.selection(cachedCf.columns()), Slices.ALL, false)))
         {
             Row r = (Row) unfiltered;
-
-            for (Cell c : r)
+            for (ColumnData c : r)
             {
-                assertEquals(c.value(), ByteBufferUtil.bytes("val" + 0));
+                assertEquals(((Cell)c).value(), ByteBufferUtil.bytes("val" + 0));
             }
         }
         cachedStore.truncateBlocking();
@@ -156,7 +152,7 @@ public class RowCacheTest
                 Row r = (Row)ai.next();
                 assertFalse(ai.hasNext());
 
-                Iterator<Cell> ci = r.iterator();
+                Iterator<Cell> ci = r.cells().iterator();
                 assert(ci.hasNext());
                 Cell cell = ci.next();
 
@@ -183,7 +179,7 @@ public class RowCacheTest
                 Row r = (Row)ai.next();
                 assertFalse(ai.hasNext());
 
-                Iterator<Cell> ci = r.iterator();
+                Iterator<Cell> ci = r.cells().iterator();
                 assert(ci.hasNext());
                 Cell cell = ci.next();
 
@@ -309,9 +305,9 @@ public class RowCacheTest
 
             assertEquals(r.clustering().get(0), ByteBufferUtil.bytes(values[i].substring(3)));
 
-            for (Cell c : r)
+            for (ColumnData c : r)
             {
-                assertEquals(c.value(), ByteBufferUtil.bytes(values[i]));
+                assertEquals(((Cell)c).value(), ByteBufferUtil.bytes(values[i]));
             }
             i++;
         }
@@ -351,7 +347,7 @@ public class RowCacheTest
         for (int i = offset; i < offset + numberOfRows; i++)
         {
             DecoratedKey key = Util.dk("key" + i);
-            Clustering cl = new SimpleClustering(ByteBufferUtil.bytes("col" + i));
+            Clustering cl = new Clustering(ByteBufferUtil.bytes("col" + i));
             Util.getAll(Util.cmd(store, key).build());
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/RowTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowTest.java b/test/unit/org/apache/cassandra/db/RowTest.java
index f7851f0..9a97483 100644
--- a/test/unit/org/apache/cassandra/db/RowTest.java
+++ b/test/unit/org/apache/cassandra/db/RowTest.java
@@ -107,12 +107,12 @@ public class RowTest
             {
                 RangeTombstoneBoundMarker openMarker = (RangeTombstoneBoundMarker)merged.next();
                 Slice.Bound openBound = openMarker.clustering();
-                DeletionTime openDeletion = new SimpleDeletionTime(openMarker.deletionTime().markedForDeleteAt(),
+                DeletionTime openDeletion = new DeletionTime(openMarker.deletionTime().markedForDeleteAt(),
                                                                    openMarker.deletionTime().localDeletionTime());
 
                 RangeTombstoneBoundMarker closeMarker = (RangeTombstoneBoundMarker)merged.next();
                 Slice.Bound closeBound = closeMarker.clustering();
-                DeletionTime closeDeletion = new SimpleDeletionTime(closeMarker.deletionTime().markedForDeleteAt(),
+                DeletionTime closeDeletion = new DeletionTime(closeMarker.deletionTime().markedForDeleteAt(),
                                                                     closeMarker.deletionTime().localDeletionTime());
 
                 assertEquals(openDeletion, closeDeletion);
@@ -127,16 +127,18 @@ public class RowTest
         ColumnDefinition defA = cfm.getColumnDefinition(new ColumnIdentifier("a", true));
         ColumnDefinition defB = cfm.getColumnDefinition(new ColumnIdentifier("b", true));
 
-        PartitionUpdate update = new PartitionUpdate(cfm, dk, cfm.partitionColumns(), 1);
-        Rows.writeClustering(update.metadata().comparator.make("c1"), update.writer());
-        writeSimpleCellValue(update.writer(), cfm, defA, "a1", 0, nowInSeconds);
-        writeSimpleCellValue(update.writer(), cfm, defA, "a2", 1, nowInSeconds);
-        writeSimpleCellValue(update.writer(), cfm, defB, "b1", 1, nowInSeconds);
-        update.writer().endOfRow();
+        Row.Builder builder = ArrayBackedRow.unsortedBuilder(cfm.partitionColumns().regulars, nowInSeconds);
+        builder.newRow(cfm.comparator.make("c1"));
+        writeSimpleCellValue(builder, cfm, defA, "a1", 0);
+        writeSimpleCellValue(builder, cfm, defA, "a2", 1);
+        writeSimpleCellValue(builder, cfm, defB, "b1", 1);
+        Row row = builder.build();
+
+        PartitionUpdate update = PartitionUpdate.singleRowUpdate(cfm, dk, row);
 
         Unfiltered unfiltered = update.unfilteredIterator().next();
         assertTrue(unfiltered.kind() == Unfiltered.Kind.ROW);
-        Row row = (Row) unfiltered;
+        row = (Row) unfiltered;
         assertEquals("a2", defA.cellValueType().getString(row.getCell(defA).value()));
         assertEquals("b1", defB.cellValueType().getString(row.getCell(defB).value()));
         assertEquals(2, row.columns().columnCount());
@@ -147,12 +149,10 @@ public class RowTest
     {
         int ttl = 1;
         ColumnDefinition def = cfm.getColumnDefinition(new ColumnIdentifier("a", true));
-        PartitionUpdate update = new PartitionUpdate(cfm, dk, cfm.partitionColumns(), 1);
-        Rows.writeClustering(update.metadata().comparator.make("c1"), update.writer());
-        update.writer().writeCell(def, false, ((AbstractType) def.cellValueType()).decompose("a1"),
-                                  SimpleLivenessInfo.forUpdate(0, ttl, nowInSeconds, cfm),
-                                  null);
-        update.writer().endOfRow();
+
+        Cell cell = BufferCell.expiring(def, 0, ttl, nowInSeconds, ((AbstractType) def.cellValueType()).decompose("a1"));
+
+        PartitionUpdate update = PartitionUpdate.singleRowUpdate(cfm, dk, ArrayBackedRow.singleCellRow(cfm.comparator.make("c1"), cell));
         new Mutation(update).applyUnsafe();
 
         // when we read with a nowInSeconds before the cell has expired,
@@ -184,21 +184,15 @@ public class RowTest
     public void writeRangeTombstone(PartitionUpdate update, Object start, Object end, long markedForDeleteAt, int localDeletionTime)
     {
         ClusteringComparator comparator = cfs.getComparator();
-        update.addRangeTombstone(Slice.make(comparator.make(start), comparator.make(end)),
-                                 new SimpleDeletionTime(markedForDeleteAt, localDeletionTime));
+        update.add(new RangeTombstone(Slice.make(comparator.make(start), comparator.make(end)), new DeletionTime(markedForDeleteAt, localDeletionTime)));
     }
 
-    private void writeSimpleCellValue(Row.Writer writer,
+    private void writeSimpleCellValue(Row.Builder builder,
                                       CFMetaData cfm,
                                       ColumnDefinition columnDefinition,
                                       String value,
-                                      long timestamp,
-                                      int nowInSeconds)
+                                      long timestamp)
     {
-        writer.writeCell(columnDefinition,
-                         false,
-                         ((AbstractType) columnDefinition.cellValueType()).decompose(value),
-                         SimpleLivenessInfo.forUpdate(timestamp, LivenessInfo.NO_TTL, nowInSeconds, cfm),
-                         null);
+       builder.addCell(BufferCell.live(cfm, columnDefinition, timestamp, ((AbstractType) columnDefinition.cellValueType()).decompose(value)));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index 6f20ccf..47bfa0c 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -43,6 +43,7 @@ import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.commons.lang3.StringUtils;
 import org.junit.BeforeClass;
@@ -333,7 +334,7 @@ public class ScrubTest
             writer.append(update.unfilteredIterator());
         }
         writer.finish(false);
-        */
+         */
 
         String root = System.getProperty("corrupt-sstable-root");
         assert root != null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
index 3d13a22..d443b8c 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
@@ -24,6 +24,7 @@ package org.apache.cassandra.db.commitlog;
 import java.io.*;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
 import java.util.Properties;
 import java.util.UUID;
 
@@ -131,7 +132,7 @@ public class CommitLogUpgradeTest
             {
                 for (Row row : update)
                 {
-                    for (Cell cell : row)
+                    for (Cell cell : row.cells())
                     {
                         hash = hash(hash, cell.value());
                         ++cells;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
new file mode 100644
index 0000000..1e5c23f
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
@@ -0,0 +1,407 @@
+package org.apache.cassandra.db.rows;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.db.Slice.Bound;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.utils.FBUtilities;
+import static org.junit.Assert.*;
+
+public class RowAndDeletionMergeIteratorTest
+{
+    private static final String KEYSPACE1 = "RowTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    private int nowInSeconds;
+    private DecoratedKey dk;
+    private ColumnFamilyStore cfs;
+    private CFMetaData cfm;
+    private ColumnDefinition defA;
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        CFMetaData cfMetadata = CFMetaData.Builder.create(KEYSPACE1, CF_STANDARD1)
+                                                  .addPartitionKey("key", AsciiType.instance)
+                                                  .addClusteringColumn("col1", Int32Type.instance)
+                                                  .addRegularColumn("a", Int32Type.instance)
+                                                  .build();
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    KeyspaceParams.simple(1),
+                                    cfMetadata);
+
+    }
+
+    @Before
+    public void setup()
+    {
+        nowInSeconds = FBUtilities.nowInSeconds();
+        dk = Util.dk("key0");
+        cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1);
+        cfm = cfs.metadata;
+        defA = cfm.getColumnDefinition(new ColumnIdentifier("a", true));
+    }
+
+    @Test
+    public void testWithNoRangeTombstones()
+    {
+        Iterator<Row> rowIterator = createRowIterator();
+        UnfilteredRowIterator iterator = createMergeIterator(rowIterator, Collections.emptyIterator(), false);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 0);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 1);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 2);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 3);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 4);
+
+        assertFalse(iterator.hasNext());
+    }
+
+    @Test
+    public void testWithOnlyRangeTombstones()
+    {
+        int delTime = nowInSeconds + 1;
+        long timestamp = delTime * 1000;
+
+        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(rt(1, false, 3, false, timestamp, delTime),
+                                                                                       atLeast(4, timestamp, delTime));
+        UnfilteredRowIterator iterator = createMergeIterator(Collections.emptyIterator(), rangeTombstoneIterator, false);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.EXCL_START_BOUND, 1);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.EXCL_END_BOUND, 3);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.INCL_START_BOUND, 4);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), Bound.TOP);
+
+        assertFalse(iterator.hasNext());
+    }
+
+    @Test
+    public void testWithAtMostRangeTombstone()
+    {
+        Iterator<Row> rowIterator = createRowIterator();
+
+        int delTime = nowInSeconds + 1;
+        long timestamp = delTime * 1000;
+
+        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(atMost(0, timestamp, delTime));
+
+        UnfilteredRowIterator iterator = createMergeIterator(rowIterator, rangeTombstoneIterator, false);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), Bound.BOTTOM);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 0);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.INCL_END_BOUND, 0);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 1);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 2);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 3);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 4);
+
+        assertFalse(iterator.hasNext());
+    }
+
+    @Test
+    public void testWithGreaterThanRangeTombstone()
+    {
+        Iterator<Row> rowIterator = createRowIterator();
+
+        int delTime = nowInSeconds + 1;
+        long timestamp = delTime * 1000;
+
+        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(greaterThan(2, timestamp, delTime));
+
+        UnfilteredRowIterator iterator = createMergeIterator(rowIterator, rangeTombstoneIterator, false);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 0);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 1);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 2);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.EXCL_START_BOUND, 2);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 3);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 4);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), Bound.TOP);
+
+        assertFalse(iterator.hasNext());
+    }
+
+    @Test
+    public void testWithAtMostAndGreaterThanRangeTombstone()
+    {
+        Iterator<Row> rowIterator = createRowIterator();
+
+        int delTime = nowInSeconds + 1;
+        long timestamp = delTime * 1000;
+
+        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(atMost(0, timestamp, delTime),
+                                                                                       greaterThan(2, timestamp, delTime));
+
+        UnfilteredRowIterator iterator = createMergeIterator(rowIterator, rangeTombstoneIterator, false);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), Bound.BOTTOM);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 0);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.INCL_END_BOUND, 0);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 1);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 2);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.EXCL_START_BOUND, 2);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 3);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 4);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), Bound.TOP);
+
+        assertFalse(iterator.hasNext());
+    }
+
+    private void assertRtMarker(Unfiltered unfiltered, ClusteringPrefix.Kind kind, int col1)
+    {
+        assertEquals(Unfiltered.Kind.RANGE_TOMBSTONE_MARKER, unfiltered.kind());
+        assertEquals(kind, unfiltered.clustering().kind());
+        assertEquals(bb(col1), unfiltered.clustering().get(0));
+    }
+
+    @Test
+    public void testWithIncludingEndExcludingStartMarker()
+    {
+        Iterator<Row> rowIterator = createRowIterator();
+
+        int delTime = nowInSeconds + 1;
+        long timestamp = delTime * 1000;
+
+        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(atMost(2, timestamp, delTime),
+                                                                                       greaterThan(2, timestamp, delTime));
+
+        UnfilteredRowIterator iterator = createMergeIterator(rowIterator, rangeTombstoneIterator, false);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), Bound.BOTTOM);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 0);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 1);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 2);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.INCL_END_EXCL_START_BOUNDARY, 2);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 3);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 4);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), Bound.TOP);
+
+        assertFalse(iterator.hasNext());
+    }
+
+    @Test
+    public void testWithExcludingEndIncludingStartMarker()
+    {
+        Iterator<Row> rowIterator = createRowIterator();
+
+        int delTime = nowInSeconds + 1;
+        long timestamp = delTime * 1000;
+
+        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(lessThan(2, timestamp, delTime),
+                                                                                       atLeast(2, timestamp, delTime));
+
+        UnfilteredRowIterator iterator = createMergeIterator(rowIterator, rangeTombstoneIterator, false);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), Bound.BOTTOM);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 0);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 1);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.EXCL_END_INCL_START_BOUNDARY, 2);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 2);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 3);
+
+        assertTrue(iterator.hasNext());
+        assertRow(iterator.next(), 4);
+
+        assertTrue(iterator.hasNext());
+        assertRtMarker(iterator.next(), Bound.TOP);
+
+        assertFalse(iterator.hasNext());
+    }
+
+    private void assertRtMarker(Unfiltered unfiltered, Bound bound)
+    {
+        assertEquals(Unfiltered.Kind.RANGE_TOMBSTONE_MARKER, unfiltered.kind());
+        assertEquals(bound, unfiltered.clustering());
+    }
+
+    private void assertRow(Unfiltered unfiltered, int col1)
+    {
+        assertEquals(Unfiltered.Kind.ROW, unfiltered.kind());
+        assertEquals(cfm.comparator.make(col1), unfiltered.clustering());
+    }
+
+    private Iterator<RangeTombstone> createRangeTombstoneIterator(RangeTombstone... tombstones)
+    {
+        RangeTombstoneList list = new RangeTombstoneList(cfm.comparator, 10);
+
+        for (RangeTombstone tombstone : tombstones)
+            list.add(tombstone);
+
+        return list.iterator(Slice.ALL, false);
+    }
+
+    private Iterator<Row> createRowIterator()
+    {
+        PartitionUpdate update = new PartitionUpdate(cfm, dk, cfm.partitionColumns(), 1);
+        for (int i = 0; i < 5; i++)
+            addRow(update, i, i);
+
+        return update.iterator();
+    }
+
+    private UnfilteredRowIterator createMergeIterator(Iterator<Row> rows, Iterator<RangeTombstone> tombstones, boolean reversed)
+    {
+        return new RowAndDeletionMergeIterator(cfm,
+                                               Util.dk("k"),
+                                               DeletionTime.LIVE,
+                                               ColumnFilter.all(cfm),
+                                               Rows.EMPTY_STATIC_ROW,
+                                               reversed,
+                                               RowStats.NO_STATS,
+                                               rows,
+                                               tombstones,
+                                               true);
+    }
+
+    private void addRow(PartitionUpdate update, int col1, int a)
+    {
+        update.add(ArrayBackedRow.singleCellRow(update.metadata().comparator.make(col1), makeCell(cfm, defA, a, 0)));
+    }
+
+    private Cell makeCell(CFMetaData cfm, ColumnDefinition columnDefinition, int value, long timestamp)
+    {
+        return BufferCell.live(cfm, columnDefinition, timestamp, ((AbstractType)columnDefinition.cellValueType()).decompose(value));
+    }
+
+    private static RangeTombstone atLeast(int start, long tstamp, int delTime)
+    {
+        return new RangeTombstone(Slice.make(Slice.Bound.inclusiveStartOf(bb(start)), Slice.Bound.TOP), new DeletionTime(tstamp, delTime));
+    }
+
+    private static RangeTombstone atMost(int end, long tstamp, int delTime)
+    {
+        return new RangeTombstone(Slice.make(Slice.Bound.BOTTOM, Slice.Bound.inclusiveEndOf(bb(end))), new DeletionTime(tstamp, delTime));
+    }
+
+    private static RangeTombstone lessThan(int end, long tstamp, int delTime)
+    {
+        return new RangeTombstone(Slice.make(Slice.Bound.BOTTOM, Slice.Bound.exclusiveEndOf(bb(end))), new DeletionTime(tstamp, delTime));
+    }
+
+    private static RangeTombstone greaterThan(int start, long tstamp, int delTime)
+    {
+        return new RangeTombstone(Slice.make(Slice.Bound.exclusiveStartOf(bb(start)), Slice.Bound.TOP), new DeletionTime(tstamp, delTime));
+    }
+
+    private static RangeTombstone rt(int start, boolean startInclusive, int end, boolean endInclusive, long tstamp, int delTime)
+    {
+        Slice.Bound startBound = startInclusive ? Slice.Bound.inclusiveStartOf(bb(start)) : Slice.Bound.exclusiveStartOf(bb(start));
+        Slice.Bound endBound = endInclusive ? Slice.Bound.inclusiveEndOf(bb(end)) : Slice.Bound.exclusiveEndOf(bb(end));
+
+        return new RangeTombstone(Slice.make(startBound, endBound), new DeletionTime(tstamp, delTime));
+    }
+
+    private static ByteBuffer bb(int i)
+    {
+        return ByteBufferUtil.bytes(i);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/rows/RowAndTombstoneMergeIteratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/RowAndTombstoneMergeIteratorTest.java b/test/unit/org/apache/cassandra/db/rows/RowAndTombstoneMergeIteratorTest.java
deleted file mode 100644
index 88a6f7e..0000000
--- a/test/unit/org/apache/cassandra/db/rows/RowAndTombstoneMergeIteratorTest.java
+++ /dev/null
@@ -1,415 +0,0 @@
-package org.apache.cassandra.db.rows;
-
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.Iterator;
-
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import org.apache.cassandra.db.Slice.Bound;
-import org.apache.cassandra.db.ClusteringPrefix;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.db.SimpleDeletionTime;
-import org.apache.cassandra.db.Slice;
-import org.apache.cassandra.db.RangeTombstone;
-import org.apache.cassandra.db.RangeTombstoneList;
-import org.apache.cassandra.db.LivenessInfo;
-import org.apache.cassandra.db.SimpleLivenessInfo;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.db.partitions.PartitionUpdate;
-import org.apache.cassandra.db.marshal.Int32Type;
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.marshal.AsciiType;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.utils.FBUtilities;
-import static org.junit.Assert.*;
-
-public class RowAndTombstoneMergeIteratorTest
-{
-    private static final String KEYSPACE1 = "RowTest";
-    private static final String CF_STANDARD1 = "Standard1";
-
-    private int nowInSeconds;
-    private DecoratedKey dk;
-    private ColumnFamilyStore cfs;
-    private CFMetaData cfm;
-    private ColumnDefinition defA;
-
-    @BeforeClass
-    public static void defineSchema() throws ConfigurationException
-    {
-        CFMetaData cfMetadata = CFMetaData.Builder.create(KEYSPACE1, CF_STANDARD1)
-                                                  .addPartitionKey("key", AsciiType.instance)
-                                                  .addClusteringColumn("col1", Int32Type.instance)
-                                                  .addRegularColumn("a", Int32Type.instance)
-                                                  .build();
-        SchemaLoader.prepareServer();
-        SchemaLoader.createKeyspace(KEYSPACE1,
-                                    KeyspaceParams.simple(1),
-                                    cfMetadata);
-
-    }
-
-    @Before
-    public void setup()
-    {
-        nowInSeconds = FBUtilities.nowInSeconds();
-        dk = Util.dk("key0");
-        cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1);
-        cfm = cfs.metadata;
-        defA = cfm.getColumnDefinition(new ColumnIdentifier("a", true));
-    }
-
-    @Test
-    public void testWithNoRangeTombstones()
-    {
-        Iterator<Row> rowIterator = createRowIterator();
-
-        RowAndTombstoneMergeIterator iterator = new RowAndTombstoneMergeIterator(cfm.comparator, false)
-                                                          .setTo(rowIterator, Collections.emptyIterator());
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 0);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 1);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 2);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 3);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 4);
-
-        assertFalse(iterator.hasNext());
-    }
-
-    @Test
-    public void testWithOnlyRangeTombstones()
-    {
-        int delTime = nowInSeconds + 1;
-        long timestamp = delTime * 1000;
-
-        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(rt(1, false, 3, false, timestamp, delTime),
-                                                                                       atLeast(4, timestamp, delTime));
-        RowAndTombstoneMergeIterator iterator = new RowAndTombstoneMergeIterator(cfm.comparator, false)
-                                                          .setTo(Collections.emptyIterator(), rangeTombstoneIterator);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.EXCL_START_BOUND, 1);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.EXCL_END_BOUND, 3);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.INCL_START_BOUND, 4);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), Bound.TOP);
-
-        assertFalse(iterator.hasNext());
-    }
-
-    @Test
-    public void testWithAtMostRangeTombstone()
-    {
-        Iterator<Row> rowIterator = createRowIterator();
-
-        int delTime = nowInSeconds + 1;
-        long timestamp = delTime * 1000;
-
-        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(atMost(0, timestamp, delTime));
-
-        RowAndTombstoneMergeIterator iterator = new RowAndTombstoneMergeIterator(cfm.comparator, false)
-                                                          .setTo(rowIterator, rangeTombstoneIterator);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), Bound.BOTTOM);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 0);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.INCL_END_BOUND, 0);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 1);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 2);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 3);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 4);
-
-        assertFalse(iterator.hasNext());
-    }
-
-    @Test
-    public void testWithGreaterThanRangeTombstone()
-    {
-        Iterator<Row> rowIterator = createRowIterator();
-
-        int delTime = nowInSeconds + 1;
-        long timestamp = delTime * 1000;
-
-        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(greaterThan(2, timestamp, delTime));
-
-        RowAndTombstoneMergeIterator iterator = new RowAndTombstoneMergeIterator(cfm.comparator, false)
-                                                          .setTo(rowIterator, rangeTombstoneIterator);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 0);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 1);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 2);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.EXCL_START_BOUND, 2);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 3);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 4);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), Bound.TOP);
-
-        assertFalse(iterator.hasNext());
-    }
-
-    @Test
-    public void testWithAtMostAndGreaterThanRangeTombstone()
-    {
-        Iterator<Row> rowIterator = createRowIterator();
-
-        int delTime = nowInSeconds + 1;
-        long timestamp = delTime * 1000;
-
-        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(atMost(0, timestamp, delTime),
-                                                                                       greaterThan(2, timestamp, delTime));
-
-        RowAndTombstoneMergeIterator iterator = new RowAndTombstoneMergeIterator(cfm.comparator, false)
-                                                          .setTo(rowIterator, rangeTombstoneIterator);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), Bound.BOTTOM);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 0);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.INCL_END_BOUND, 0);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 1);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 2);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.EXCL_START_BOUND, 2);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 3);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 4);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), Bound.TOP);
-
-        assertFalse(iterator.hasNext());
-    }
-
-    private void assertRtMarker(Unfiltered unfiltered, ClusteringPrefix.Kind kind, int col1)    {
-        assertEquals(Unfiltered.Kind.RANGE_TOMBSTONE_MARKER, unfiltered.kind());
-        assertEquals(kind, unfiltered.clustering().kind());
-        assertEquals(bb(col1), unfiltered.clustering().get(0));
-    }
-
-    @Test
-    public void testWithIncludingEndExcludingStartMarker()
-    {
-        Iterator<Row> rowIterator = createRowIterator();
-
-        int delTime = nowInSeconds + 1;
-        long timestamp = delTime * 1000;
-
-        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(atMost(2, timestamp, delTime),
-                                                                                       greaterThan(2, timestamp, delTime));
-
-        RowAndTombstoneMergeIterator iterator = new RowAndTombstoneMergeIterator(cfm.comparator, false)
-                                                          .setTo(rowIterator, rangeTombstoneIterator);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), Bound.BOTTOM);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 0);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 1);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 2);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.INCL_END_EXCL_START_BOUNDARY, 2);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 3);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 4);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), Bound.TOP);
-
-        assertFalse(iterator.hasNext());
-    }
-
-    @Test
-    public void testWithExcludingEndIncludingStartMarker()
-    {
-        Iterator<Row> rowIterator = createRowIterator();
-
-        int delTime = nowInSeconds + 1;
-        long timestamp = delTime * 1000;
-
-        Iterator<RangeTombstone> rangeTombstoneIterator = createRangeTombstoneIterator(lessThan(2, timestamp, delTime),
-                                                                                       atLeast(2, timestamp, delTime));
-
-        RowAndTombstoneMergeIterator iterator = new RowAndTombstoneMergeIterator(cfm.comparator, false)
-                                                          .setTo(rowIterator, rangeTombstoneIterator);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), Bound.BOTTOM);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 0);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 1);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), ClusteringPrefix.Kind.EXCL_END_INCL_START_BOUNDARY, 2);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 2);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 3);
-
-        assertTrue(iterator.hasNext());
-        assertRow(iterator.next(), 4);
-
-        assertTrue(iterator.hasNext());
-        assertRtMarker(iterator.next(), Bound.TOP);
-
-        assertFalse(iterator.hasNext());
-    }
-
-    private void assertRtMarker(Unfiltered unfiltered, Bound bound)
-    {
-        assertEquals(Unfiltered.Kind.RANGE_TOMBSTONE_MARKER, unfiltered.kind());
-        assertEquals(bound, unfiltered.clustering());
-    }
-
-    private void assertRow(Unfiltered unfiltered, int col1)
-    {
-        assertEquals(Unfiltered.Kind.ROW, unfiltered.kind());
-        assertEquals(cfm.comparator.make(col1), unfiltered.clustering());
-    }
-
-    private Iterator<RangeTombstone> createRangeTombstoneIterator(RangeTombstone... tombstones)
-    {
-        RangeTombstoneList list = new RangeTombstoneList(cfm.comparator, 10);
-
-        for (RangeTombstone tombstone : tombstones)
-            list.add(tombstone);
-
-        return list.iterator(Slice.ALL, false);
-    }
-
-    private Iterator<Row> createRowIterator()
-    {
-        PartitionUpdate update = new PartitionUpdate(cfm, dk, cfm.partitionColumns(), 1);
-        for (int i = 0; i < 5; i++)
-            addRow(update, i, i);
-
-        return update.iterator();
-    }
-
-    private void addRow(PartitionUpdate update, int col1, int a)
-    {
-        Rows.writeClustering(update.metadata().comparator.make(col1), update.writer());
-        writeSimpleCellValue(update.writer(), cfm, defA, a, 0, nowInSeconds);
-        update.writer().endOfRow();
-    }
-
-    private void writeSimpleCellValue(Row.Writer writer,
-                                      CFMetaData cfm,
-                                      ColumnDefinition columnDefinition,
-                                      int value,
-                                      long timestamp,
-                                      int nowInSeconds)
-    {
-        writer.writeCell(columnDefinition,
-                         false,
-                         ((AbstractType) columnDefinition.cellValueType()).decompose(value),
-                         SimpleLivenessInfo.forUpdate(timestamp, LivenessInfo.NO_TTL, nowInSeconds, cfm),
-                         null);
-    }
-
-    private static RangeTombstone atLeast(int start, long tstamp, int delTime)
-    {
-        return new RangeTombstone(Slice.make(Slice.Bound.inclusiveStartOf(bb(start)), Slice.Bound.TOP), new SimpleDeletionTime(tstamp, delTime));
-    }
-
-    private static RangeTombstone atMost(int end, long tstamp, int delTime)
-    {
-        return new RangeTombstone(Slice.make(Slice.Bound.BOTTOM, Slice.Bound.inclusiveEndOf(bb(end))), new SimpleDeletionTime(tstamp, delTime));
-    }
-
-    private static RangeTombstone lessThan(int end, long tstamp, int delTime)
-    {
-        return new RangeTombstone(Slice.make(Slice.Bound.BOTTOM, Slice.Bound.exclusiveEndOf(bb(end))), new SimpleDeletionTime(tstamp, delTime));
-    }
-
-    private static RangeTombstone greaterThan(int start, long tstamp, int delTime)
-    {
-        return new RangeTombstone(Slice.make(Slice.Bound.exclusiveStartOf(bb(start)), Slice.Bound.TOP), new SimpleDeletionTime(tstamp, delTime));
-    }
-
-    private static RangeTombstone rt(int start, boolean startInclusive, int end, boolean endInclusive, long tstamp, int delTime)
-    {
-        Slice.Bound startBound = startInclusive ? Slice.Bound.inclusiveStartOf(bb(start)) : Slice.Bound.exclusiveStartOf(bb(start));
-        Slice.Bound endBound = endInclusive ? Slice.Bound.inclusiveEndOf(bb(end)) : Slice.Bound.exclusiveEndOf(bb(end));
-
-        return new RangeTombstone(Slice.make(startBound, endBound), new SimpleDeletionTime(tstamp, delTime));
-    }
-
-    private static ByteBuffer bb(int i)
-    {
-        return ByteBufferUtil.bytes(i);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java b/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java
index a607dca..e869c72 100644
--- a/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java
@@ -138,7 +138,7 @@ public class UnfilteredRowIteratorsMergeTest
     {
         List<UnfilteredRowIterator> us = sources.stream().map(l -> new Source(l.iterator())).collect(Collectors.toList());
         List<Unfiltered> merged = new ArrayList<>();
-        Iterators.addAll(merged, safeIterator(mergeIterators(us, iterations)));
+        Iterators.addAll(merged, mergeIterators(us, iterations));
         return merged;
     }
 
@@ -197,7 +197,7 @@ public class UnfilteredRowIteratorsMergeTest
                     includesEnd = r.nextBoolean();
                 }
                 int deltime = r.nextInt(DEL_RANGE);
-                DeletionTime dt = new SimpleDeletionTime(deltime, deltime);
+                DeletionTime dt = new DeletionTime(deltime, deltime);
                 content.add(new RangeTombstoneBoundMarker(boundFor(pos, true, includesStart), dt));
                 content.add(new RangeTombstoneBoundMarker(boundFor(pos + span, false, includesEnd), dt));
                 prev = pos + span - (includesEnd ? 0 : 1);
@@ -365,173 +365,16 @@ public class UnfilteredRowIteratorsMergeTest
         return Bound.create(Bound.boundKind(start, inclusive), new ByteBuffer[] {Int32Type.instance.decompose(pos)});
     }
 
-    private static SimpleClustering clusteringFor(int i)
+    private static Clustering clusteringFor(int i)
     {
-        return new SimpleClustering(Int32Type.instance.decompose(i));
+        return new Clustering(Int32Type.instance.decompose(i));
     }
 
     static Row emptyRowAt(int pos, Function<Integer, Integer> timeGenerator)
     {
         final Clustering clustering = clusteringFor(pos);
-        final LivenessInfo live = SimpleLivenessInfo.forUpdate(timeGenerator.apply(pos), 0, nowInSec, metadata);
-        return emptyRowAt(clustering, live, DeletionTime.LIVE);
-    }
-
-    public static class TestCell extends AbstractCell
-    {
-        private final ColumnDefinition column;
-        private final ByteBuffer value;
-        private final LivenessInfo info;
-
-        public TestCell(ColumnDefinition column, ByteBuffer value, LivenessInfo info)
-        {
-            this.column = column;
-            this.value = value;
-            this.info = info.takeAlias();
-        }
-
-        @Override
-        public ColumnDefinition column()
-        {
-            return column;
-        }
-
-        @Override
-        public boolean isCounterCell()
-        {
-            return false;
-        }
-
-        @Override
-        public ByteBuffer value()
-        {
-            return value;
-        }
-
-        @Override
-        public LivenessInfo livenessInfo()
-        {
-            return info;
-        }
-
-        @Override
-        public CellPath path()
-        {
-            return null;
-        }
-    }
-
-    static Row emptyRowAt(final Clustering clustering, final LivenessInfo live, final DeletionTime deletion)
-    {
-        final ColumnDefinition columnDef = metadata.getColumnDefinition(new ColumnIdentifier("data", true));
-        final Cell cell = new TestCell(columnDef, clustering.get(0), live);
-
-        return new AbstractRow()
-        {
-            @Override
-            public Columns columns()
-            {
-                return Columns.of(columnDef);
-            }
-
-            @Override
-            public LivenessInfo primaryKeyLivenessInfo()
-            {
-                return live;
-            }
-
-            @Override
-            public DeletionTime deletion()
-            {
-                return deletion;
-            }
-
-            @Override
-            public boolean isEmpty()
-            {
-                return true;
-            }
-
-            @Override
-            public boolean hasComplexDeletion()
-            {
-                return false;
-            }
-
-            @Override
-            public Clustering clustering()
-            {
-                return clustering;
-            }
-
-            @Override
-            public Cell getCell(ColumnDefinition c)
-            {
-                return c == columnDef ? cell : null;
-            }
-
-            @Override
-            public Cell getCell(ColumnDefinition c, CellPath path)
-            {
-                return null;
-            }
-
-            @Override
-            public Iterator<Cell> getCells(ColumnDefinition c)
-            {
-                return Iterators.singletonIterator(cell);
-            }
-
-            @Override
-            public DeletionTime getDeletion(ColumnDefinition c)
-            {
-                return DeletionTime.LIVE;
-            }
-
-            @Override
-            public Iterator<Cell> iterator()
-            {
-                return Iterators.<Cell>emptyIterator();
-            }
-
-            @Override
-            public SearchIterator<ColumnDefinition, ColumnData> searchIterator()
-            {
-                return new SearchIterator<ColumnDefinition, ColumnData>()
-                {
-                    @Override
-                    public boolean hasNext()
-                    {
-                        return false;
-                    }
-
-                    @Override
-                    public ColumnData next(ColumnDefinition column)
-                    {
-                        return null;
-                    }
-                };
-            }
-
-            @Override
-            public Kind kind()
-            {
-                return Unfiltered.Kind.ROW;
-            }
-
-            @Override
-            public Row takeAlias()
-            {
-                return this;
-            }
-
-            @Override
-            public String toString()
-            {
-                return Int32Type.instance.getString(clustering.get(0));
-            }
-        };
-
+        final LivenessInfo live = LivenessInfo.create(metadata, timeGenerator.apply(pos), nowInSec);
+        return ArrayBackedRow.noCellLiveRow(clustering, live);
     }
 
     private void dumpList(List<Unfiltered> list)
@@ -580,33 +423,6 @@ public class UnfilteredRowIteratorsMergeTest
         }
     }
 
-    static RangeTombstoneMarker safeMarker(RangeTombstoneMarker marker)
-    {
-        RangeTombstoneMarker.Builder writer = new RangeTombstoneMarker.Builder(1);
-        marker.copyTo(writer);
-        return writer.build();
-    }
-
-    private static Row safeRow(Row row)
-    {
-        return emptyRowAt(new SimpleClustering(row.clustering().get(0)), row.primaryKeyLivenessInfo(), row.deletion());
-    }
-    
-    public static UnfilteredRowIterator safeIterator(UnfilteredRowIterator iterator)
-    {
-        return new WrappingUnfilteredRowIterator(iterator)
-        {
-            @Override
-            public Unfiltered next()
-            {
-                Unfiltered next = super.next();
-                return next.kind() == Unfiltered.Kind.ROW
-                     ? safeRow((Row) next)
-                     : safeMarker((RangeTombstoneMarker) next);
-            }
-        };
-    }
-
     public void testForInput(String... inputs)
     {
         List<List<Unfiltered>> sources = new ArrayList<>();
@@ -674,6 +490,6 @@ public class UnfilteredRowIteratorsMergeTest
     {
         return new RangeTombstoneBoundMarker(Bound.create(Bound.boundKind(isStart, inclusive),
                                                           new ByteBuffer[] {clusteringFor(pos).get(0)}),
-                                             new SimpleDeletionTime(delTime, delTime));
+                                             new DeletionTime(delTime, delTime));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/io/sstable/IndexHelperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexHelperTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexHelperTest.java
index f4f7de3..c9f268a 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexHelperTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexHelperTest.java
@@ -27,7 +27,7 @@ import org.junit.Test;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.db.ClusteringComparator;
 import org.apache.cassandra.db.ClusteringPrefix;
-import org.apache.cassandra.db.SimpleDeletionTime;
+import org.apache.cassandra.db.DeletionTime;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.db.marshal.LongType;
@@ -48,7 +48,7 @@ public class IndexHelperTest
     @Test
     public void testIndexHelper()
     {
-        SimpleDeletionTime deletionInfo = new SimpleDeletionTime(FBUtilities.timestampMicros(), FBUtilities.nowInSeconds());
+        DeletionTime deletionInfo = new DeletionTime(FBUtilities.timestampMicros(), FBUtilities.nowInSeconds());
 
         List<IndexInfo> indexes = new ArrayList<>();
         indexes.add(new IndexInfo(cn(0L), cn(5L), 0, 0, deletionInfo));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
index f8644bb..5a7c074 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
@@ -28,8 +28,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.SimpleClustering;
+import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.dht.Range;
@@ -107,7 +106,7 @@ public class SSTableLoaderTest
 
         assertEquals(1, partitions.size());
         assertEquals("key1", AsciiType.instance.getString(partitions.get(0).partitionKey().getKey()));
-        assertEquals(ByteBufferUtil.bytes("100"), partitions.get(0).getRow(new SimpleClustering(ByteBufferUtil.bytes("col1")))
+        assertEquals(ByteBufferUtil.bytes("100"), partitions.get(0).getRow(new Clustering(ByteBufferUtil.bytes("col1")))
                                                                    .getCell(cfmeta.getColumnDefinition(ByteBufferUtil.bytes("val")))
                                                                    .value());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 4ffdcd7..13a371c 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -501,7 +501,7 @@ public class SSTableReaderTest
                 public void run()
                 {
                     Row row = Util.getOnlyRowUnfiltered(Util.cmd(store, key).build());
-                    assertEquals(0, ByteBufferUtil.compare(String.format("%3d", index).getBytes(), row.iterator().next().value()));
+                    assertEquals(0, ByteBufferUtil.compare(String.format("%3d", index).getBytes(), row.cells().iterator().next().value()));
                 }
             }));
 


[11/15] cassandra git commit: Simplify some 8099's implementations

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java
index 54feb85..33a0917 100644
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.filter;
 
-import java.io.DataInput;
 import java.io.IOException;
 
 import org.apache.cassandra.config.CFMetaData;
@@ -26,6 +25,7 @@ import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.CachedPartition;
 import org.apache.cassandra.db.partitions.Partition;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
 /**
@@ -146,7 +146,7 @@ public interface ClusteringIndexFilter
     public interface Serializer
     {
         public void serialize(ClusteringIndexFilter filter, DataOutputPlus out, int version) throws IOException;
-        public ClusteringIndexFilter deserialize(DataInput in, int version, CFMetaData metadata) throws IOException;
+        public ClusteringIndexFilter deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException;
         public long serializedSize(ClusteringIndexFilter filter, int version);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
index f2cc46f..13329f3 100644
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.filter;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.util.*;
 
@@ -27,6 +26,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.SearchIterator;
 import org.apache.cassandra.utils.btree.BTreeSet;
@@ -94,6 +94,9 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
 
     public boolean isFullyCoveredBy(CachedPartition partition)
     {
+        if (partition.isEmpty())
+            return false;
+
         // 'partition' contains all columns, so it covers our filter if our last clusterings
         // is smaller than the last in the cache
         return clusterings.comparator().compare(clusterings.last(), partition.lastRow().clustering()) <= 0;
@@ -109,18 +112,18 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
     {
         // Note that we don't filter markers because that's a bit trickier (we don't know in advance until when
         // the range extend) and it's harmless to left them.
-        return new FilteringRowIterator(iterator)
+        return new AlteringUnfilteredRowIterator(iterator)
         {
             @Override
-            public FilteringRow makeRowFilter()
+            public Row computeNextStatic(Row row)
             {
-                return FilteringRow.columnsFilteringRow(columnFilter);
+                return columnFilter.fetchedColumns().statics.isEmpty() ? null : row.filter(columnFilter, iterator.metadata());
             }
 
             @Override
-            protected boolean includeRow(Row row)
+            public Row computeNext(Row row)
             {
-                return clusterings.contains(row.clustering());
+                return clusterings.contains(row.clustering()) ? row.filter(columnFilter, iterator.metadata()) : null;
             }
         };
     }
@@ -214,7 +217,7 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
             sb.append(i++ == 0 ? "" : ", ").append(clustering.toString(metadata));
         if (reversed)
             sb.append(", reversed");
-        return sb.append(")").toString();
+        return sb.append(')').toString();
     }
 
     public String toCQLString(CFMetaData metadata)
@@ -223,7 +226,7 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
             return "";
 
         StringBuilder sb = new StringBuilder();
-        sb.append("(").append(ColumnDefinition.toCQLString(metadata.clusteringColumns())).append(")");
+        sb.append('(').append(ColumnDefinition.toCQLString(metadata.clusteringColumns())).append(')');
         sb.append(clusterings.size() == 1 ? " = " : " IN (");
         int i = 0;
         for (Clustering clustering : clusterings)
@@ -258,13 +261,13 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
 
     private static class NamesDeserializer extends InternalDeserializer
     {
-        public ClusteringIndexFilter deserialize(DataInput in, int version, CFMetaData metadata, boolean reversed) throws IOException
+        public ClusteringIndexFilter deserialize(DataInputPlus in, int version, CFMetaData metadata, boolean reversed) throws IOException
         {
             ClusteringComparator comparator = metadata.comparator;
             BTreeSet.Builder<Clustering> clusterings = BTreeSet.builder(comparator);
             int size = in.readInt();
             for (int i = 0; i < size; i++)
-                clusterings.add(Clustering.serializer.deserialize(in, version, comparator.subtypes()).takeAlias());
+                clusterings.add(Clustering.serializer.deserialize(in, version, comparator.subtypes()));
 
             return new ClusteringIndexNamesFilter(clusterings.build(), reversed);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java
index 8fb319e..4f0e4e2 100644
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.filter;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.util.List;
 import java.nio.ByteBuffer;
@@ -28,6 +27,7 @@ import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.CachedPartition;
 import org.apache.cassandra.db.partitions.Partition;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
 /**
@@ -91,24 +91,24 @@ public class ClusteringIndexSliceFilter extends AbstractClusteringIndexFilter
 
         // Note that we don't filter markers because that's a bit trickier (we don't know in advance until when
         // the range extend) and it's harmless to leave them.
-        return new FilteringRowIterator(iterator)
+        return new AlteringUnfilteredRowIterator(iterator)
         {
             @Override
-            public FilteringRow makeRowFilter()
+            public boolean hasNext()
             {
-                return FilteringRow.columnsFilteringRow(columnFilter);
+                return !tester.isDone() && super.hasNext();
             }
 
             @Override
-            protected boolean includeRow(Row row)
+            public Row computeNextStatic(Row row)
             {
-                return tester.includes(row.clustering());
+                return columnFilter.fetchedColumns().statics.isEmpty() ? null : row.filter(columnFilter, iterator.metadata());
             }
 
             @Override
-            public boolean hasNext()
+            public Row computeNext(Row row)
             {
-                return !tester.isDone() && super.hasNext();
+                return tester.includes(row.clustering()) ? row.filter(columnFilter, iterator.metadata()) : null;
             }
         };
     }
@@ -170,7 +170,7 @@ public class ClusteringIndexSliceFilter extends AbstractClusteringIndexFilter
 
     private static class SliceDeserializer extends InternalDeserializer
     {
-        public ClusteringIndexFilter deserialize(DataInput in, int version, CFMetaData metadata, boolean reversed) throws IOException
+        public ClusteringIndexFilter deserialize(DataInputPlus in, int version, CFMetaData metadata, boolean reversed) throws IOException
         {
             Slices slices = Slices.serializer.deserialize(in, version, metadata);
             return new ClusteringIndexSliceFilter(slices, reversed);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
index b98108d..084bad6 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.filter;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.util.*;
 
@@ -30,8 +29,8 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.CellPath;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * Represents which (non-PK) columns (and optionally which sub-part of a column for complex columns) are selected
@@ -52,15 +51,6 @@ public class ColumnFilter
 {
     public static final Serializer serializer = new Serializer();
 
-    private static final Comparator<ColumnSubselection> valueComparator = new Comparator<ColumnSubselection>()
-    {
-        public int compare(ColumnSubselection s1, ColumnSubselection s2)
-        {
-            assert s1.column().name.equals(s2.column().name);
-            return s1.column().cellPathComparator().compare(s1.minIncludedPath(), s2.minIncludedPath());
-        }
-    };
-
     // Distinguish between the 2 cases described above: if 'isFetchAll' is true, then all columns will be retrieved
     // by the query, but the values for column/cells not selected by 'selection' and 'subSelections' will be skipped.
     // Otherwise, only the column/cells returned by 'selection' and 'subSelections' will be returned at all.
@@ -115,6 +105,11 @@ public class ColumnFilter
         return isFetchAll ? metadata.partitionColumns() : selection;
     }
 
+    public boolean includesAllColumns()
+    {
+        return isFetchAll;
+    }
+
     /**
      * Whether the provided column is selected by this selection.
      */
@@ -144,7 +139,7 @@ public class ColumnFilter
             return true;
 
         for (ColumnSubselection subSel : s)
-            if (subSel.includes(cell.path()))
+            if (subSel.compareInclusionOf(cell.path()) == 0)
                 return true;
 
         return false;
@@ -163,7 +158,7 @@ public class ColumnFilter
             return false;
 
         for (ColumnSubselection subSel : s)
-            if (subSel.includes(path))
+            if (subSel.compareInclusionOf(path) == 0)
                 return false;
 
         return true;
@@ -182,7 +177,7 @@ public class ColumnFilter
         if (s.isEmpty())
             return null;
 
-        return new Tester(s.iterator());
+        return new Tester(isFetchAll, s.iterator());
     }
 
     /**
@@ -205,46 +200,43 @@ public class ColumnFilter
 
     public static class Tester
     {
+        private final boolean isFetchAll;
         private ColumnSubselection current;
         private final Iterator<ColumnSubselection> iterator;
 
-        private Tester(Iterator<ColumnSubselection> iterator)
+        private Tester(boolean isFetchAll, Iterator<ColumnSubselection> iterator)
         {
+            this.isFetchAll = isFetchAll;
             this.iterator = iterator;
         }
 
         public boolean includes(CellPath path)
         {
-            while (current == null)
-            {
-                if (!iterator.hasNext())
-                    return false;
-
-                current = iterator.next();
-                if (current.includes(path))
-                    return true;
-
-                if (current.column().cellPathComparator().compare(current.maxIncludedPath(), path) < 0)
-                    current = null;
-            }
-            return false;
+            return isFetchAll || includedBySubselection(path);
         }
 
         public boolean canSkipValue(CellPath path)
         {
-            while (current == null)
+            return isFetchAll && !includedBySubselection(path);
+        }
+
+        private boolean includedBySubselection(CellPath path)
+        {
+            while (current != null || iterator.hasNext())
             {
-                if (!iterator.hasNext())
-                    return false;
+                if (current == null)
+                    current = iterator.next();
 
-                current = iterator.next();
-                if (current.includes(path))
+                int cmp = current.compareInclusionOf(path);
+                if (cmp == 0) // The path is included
+                    return true;
+                else if (cmp < 0) // The path is before this sub-selection, it's not included by any
                     return false;
 
-                if (current.column().cellPathComparator().compare(current.maxIncludedPath(), path) < 0)
-                    current = null;
+                // the path is after this sub-selection, we need to check the next one.
+                current = null;
             }
-            return true;
+            return false;
         }
     }
 
@@ -302,7 +294,7 @@ public class ColumnFilter
             SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = null;
             if (subSelections != null)
             {
-                s = TreeMultimap.create(Comparator.<ColumnIdentifier>naturalOrder(), valueComparator);
+                s = TreeMultimap.create(Comparator.<ColumnIdentifier>naturalOrder(), Comparator.<ColumnSubselection>naturalOrder());
                 for (ColumnSubselection subSelection : subSelections)
                     s.put(subSelection.column().name, subSelection);
             }
@@ -317,6 +309,9 @@ public class ColumnFilter
         if (selection == null)
             return "*";
 
+        if (selection.isEmpty())
+            return "";
+
         Iterator<ColumnDefinition> defs = selection.selectOrderIterator();
         StringBuilder sb = new StringBuilder();
         appendColumnDef(sb, defs.next());
@@ -351,7 +346,7 @@ public class ColumnFilter
         private static final int HAS_SELECTION_MASK      = 0x02;
         private static final int HAS_SUB_SELECTIONS_MASK = 0x04;
 
-        private int makeHeaderByte(ColumnFilter selection)
+        private static int makeHeaderByte(ColumnFilter selection)
         {
             return (selection.isFetchAll ? IS_FETCH_ALL_MASK : 0)
                  | (selection.selection != null ? HAS_SELECTION_MASK : 0)
@@ -376,7 +371,7 @@ public class ColumnFilter
             }
         }
 
-        public ColumnFilter deserialize(DataInput in, int version, CFMetaData metadata) throws IOException
+        public ColumnFilter deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException
         {
             int header = in.readUnsignedByte();
             boolean isFetchAll = (header & IS_FETCH_ALL_MASK) != 0;
@@ -394,7 +389,7 @@ public class ColumnFilter
             SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections = null;
             if (hasSubSelections)
             {
-                subSelections = TreeMultimap.create(Comparator.<ColumnIdentifier>naturalOrder(), valueComparator);
+                subSelections = TreeMultimap.create(Comparator.<ColumnIdentifier>naturalOrder(), Comparator.<ColumnSubselection>naturalOrder());
                 int size = in.readUnsignedShort();
                 for (int i = 0; i < size; i++)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java b/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java
index 652e27c..e45dbee 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.filter;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Comparator;
@@ -29,6 +28,7 @@ import org.apache.cassandra.db.rows.CellPath;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -38,7 +38,7 @@ import org.apache.cassandra.utils.ByteBufferUtil;
  * This only make sense for complex column. For those, this allow for instance
  * to select only a slice of a map.
  */
-public abstract class ColumnSubselection
+public abstract class ColumnSubselection implements Comparable<ColumnSubselection>
 {
     public static final Serializer serializer = new Serializer();
 
@@ -72,9 +72,19 @@ public abstract class ColumnSubselection
 
     protected abstract Kind kind();
 
-    public abstract CellPath minIncludedPath();
-    public abstract CellPath maxIncludedPath();
-    public abstract boolean includes(CellPath path);
+    protected abstract CellPath comparisonPath();
+
+    public int compareTo(ColumnSubselection other)
+    {
+        assert other.column().name.equals(column().name);
+        return column().cellPathComparator().compare(comparisonPath(), other.comparisonPath());
+    }
+
+    /**
+     * Given a path, return -1 if the path is before anything selected by this subselection, 0 if it is selected by this
+     * subselection and 1 if the path is after anything selected by this subselection.
+     */
+    public abstract int compareInclusionOf(CellPath path);
 
     private static class Slice extends ColumnSubselection
     {
@@ -93,20 +103,20 @@ public abstract class ColumnSubselection
             return Kind.SLICE;
         }
 
-        public CellPath minIncludedPath()
+        public CellPath comparisonPath()
         {
             return from;
         }
 
-        public CellPath maxIncludedPath()
-        {
-            return to;
-        }
-
-        public boolean includes(CellPath path)
+        public int compareInclusionOf(CellPath path)
         {
             Comparator<CellPath> cmp = column.cellPathComparator();
-            return cmp.compare(from, path) <= 0 && cmp.compare(path, to) <= 0;
+            if (cmp.compare(path, from) < 0)
+                return -1;
+            else if (cmp.compare(to, path) < 0)
+                return 1;
+            else
+                return 0;
         }
 
         @Override
@@ -133,20 +143,14 @@ public abstract class ColumnSubselection
             return Kind.ELEMENT;
         }
 
-        public CellPath minIncludedPath()
-        {
-            return element;
-        }
-
-        public CellPath maxIncludedPath()
+        public CellPath comparisonPath()
         {
             return element;
         }
 
-        public boolean includes(CellPath path)
+        public int compareInclusionOf(CellPath path)
         {
-            Comparator<CellPath> cmp = column.cellPathComparator();
-            return cmp.compare(element, path) == 0;
+            return column.cellPathComparator().compare(path, element);
         }
 
         @Override
@@ -180,7 +184,7 @@ public abstract class ColumnSubselection
             throw new AssertionError();
         }
 
-        public ColumnSubselection deserialize(DataInput in, int version, CFMetaData metadata) throws IOException
+        public ColumnSubselection deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException
         {
             ByteBuffer name = ByteBufferUtil.readWithShortLength(in);
             ColumnDefinition column = metadata.getColumnDefinition(name);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/filter/DataLimits.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/DataLimits.java b/src/java/org/apache/cassandra/db/filter/DataLimits.java
index 76e29ec..206afa4 100644
--- a/src/java/org/apache/cassandra/db/filter/DataLimits.java
+++ b/src/java/org/apache/cassandra/db/filter/DataLimits.java
@@ -115,8 +115,7 @@ public abstract class DataLimits
      * The max number of results this limits enforces.
      * <p>
      * Note that the actual definition of "results" depends a bit: for CQL, it's always rows, but for
-     * thrift, it means cells. The {@link #countsCells} allows to distinguish between the two cases if
-     * needed.
+     * thrift, it means cells.
      *
      * @return the maximum number of results this limits enforces.
      */
@@ -124,8 +123,6 @@ public abstract class DataLimits
 
     public abstract int perPartitionCount();
 
-    public abstract boolean countsCells();
-
     public UnfilteredPartitionIterator filter(UnfilteredPartitionIterator iter, int nowInSec)
     {
         return new CountingUnfilteredPartitionIterator(iter, newCounter(nowInSec, false));
@@ -269,11 +266,6 @@ public abstract class DataLimits
             return perPartitionLimit;
         }
 
-        public boolean countsCells()
-        {
-            return false;
-        }
-
         public float estimateTotalResults(ColumnFamilyStore cfs)
         {
             // TODO: we should start storing stats on the number of rows (instead of the number of cells, which
@@ -353,7 +345,7 @@ public abstract class DataLimits
             {
                 sb.append("LIMIT ").append(rowLimit);
                 if (perPartitionLimit != Integer.MAX_VALUE)
-                    sb.append(" ");
+                    sb.append(' ');
             }
 
             if (perPartitionLimit != Integer.MAX_VALUE)
@@ -511,11 +503,6 @@ public abstract class DataLimits
             return cellPerPartitionLimit;
         }
 
-        public boolean countsCells()
-        {
-            return true;
-        }
-
         public float estimateTotalResults(ColumnFamilyStore cfs)
         {
             // remember that getMeansColumns returns a number of cells: we should clean nomenclature
@@ -572,7 +559,7 @@ public abstract class DataLimits
 
             public void newRow(Row row)
             {
-                for (Cell cell : row)
+                for (Cell cell : row.cells())
                 {
                     if (assumeLiveData || cell.isLive(nowInSec))
                     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/filter/RowFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/RowFilter.java b/src/java/org/apache/cassandra/db/filter/RowFilter.java
index 8f34efb..5a49bca 100644
--- a/src/java/org/apache/cassandra/db/filter/RowFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/RowFilter.java
@@ -137,11 +137,11 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
         if (metadata.isCompound())
         {
             List<ByteBuffer> values = CompositeType.splitName(name);
-            return new SimpleClustering(values.toArray(new ByteBuffer[metadata.comparator.size()]));
+            return new Clustering(values.toArray(new ByteBuffer[metadata.comparator.size()]));
         }
         else
         {
-            return new SimpleClustering(name);
+            return new Clustering(name);
         }
     }
 
@@ -165,28 +165,18 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
             super(expressions);
         }
 
-        public UnfilteredPartitionIterator filter(UnfilteredPartitionIterator iter, final int nowInSec)
+        public UnfilteredPartitionIterator filter(UnfilteredPartitionIterator iter, int nowInSec)
         {
             if (expressions.isEmpty())
                 return iter;
 
-            return new WrappingUnfilteredPartitionIterator(iter)
+            return new AlteringUnfilteredPartitionIterator(iter)
             {
-                @Override
-                public UnfilteredRowIterator computeNext(final UnfilteredRowIterator iter)
+                protected Row computeNext(DecoratedKey partitionKey, Row row)
                 {
-                    return new FilteringRowIterator(iter)
-                    {
-                        // We filter tombstones when passing the row to isSatisfiedBy so that the method doesn't have to bother with them.
-                        // (we should however not filter them in the output of the method, hence it's not used as row filter for the
-                        // FilteringRowIterator)
-                        private final TombstoneFilteringRow filter = new TombstoneFilteringRow(nowInSec);
-
-                        protected boolean includeRow(Row row)
-                        {
-                            return CQLFilter.this.isSatisfiedBy(iter.partitionKey(), filter.setTo(row));
-                        }
-                    };
+                    // We filter tombstones when passing the row to isSatisfiedBy so that the method doesn't have to bother with them.
+                    Row purged = row.purge(DeletionPurger.PURGE_ALL, nowInSec);
+                    return purged != null && CQLFilter.this.isSatisfiedBy(partitionKey, purged) ? row : null;
                 }
             };
         }
@@ -515,10 +505,9 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
                     CollectionType<?> type = (CollectionType<?>)column.type;
                     if (column.isComplex())
                     {
-                        Iterator<Cell> iter = row.getCells(column);
-                        while (iter.hasNext())
+                        ComplexColumnData complexData = row.getComplexColumnData(column);
+                        for (Cell cell : complexData)
                         {
-                            Cell cell = iter.next();
                             if (type.kind == CollectionType.Kind.SET)
                             {
                                 if (type.nameComparator().compare(cell.path().get(0), value) == 0)
@@ -720,7 +709,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
 
             // In thrift, we actually allow expression on non-defined columns for the sake of filtering. To accomodate
             // this we create a "fake" definition. This is messy but it works so is probably good enough.
-            return ColumnDefinition.regularDef(metadata, name, metadata.compactValueColumn().type, null);
+            return ColumnDefinition.regularDef(metadata, name, metadata.compactValueColumn().type);
         }
 
         public boolean isSatisfiedBy(DecoratedKey partitionKey, Row row)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
index 794744a..c3a3c08 100644
--- a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
@@ -108,17 +108,16 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
 
     public void deleteForCleanup(ByteBuffer rowKey, Clustering clustering, Cell cell, OpOrder.Group opGroup, int nowInSec)
     {
-        delete(rowKey, clustering, cell.value(), cell.path(), new SimpleDeletionTime(cell.livenessInfo().timestamp(), nowInSec), opGroup);
+        delete(rowKey, clustering, cell.value(), cell.path(), new DeletionTime(cell.timestamp(), nowInSec), opGroup);
     }
 
     public void delete(ByteBuffer rowKey, Clustering clustering, ByteBuffer cellValue, CellPath path, DeletionTime deletion, OpOrder.Group opGroup)
     {
         DecoratedKey valueKey = getIndexKeyFor(getIndexedValue(rowKey, clustering, cellValue, path));
-        PartitionUpdate upd = new PartitionUpdate(indexCfs.metadata, valueKey, PartitionColumns.NONE, 1);
-        Row.Writer writer = upd.writer();
-        Rows.writeClustering(makeIndexClustering(rowKey, clustering, path), writer);
-        writer.writeRowDeletion(deletion);
-        writer.endOfRow();
+
+        Row row = ArrayBackedRow.emptyDeletedRow(makeIndexClustering(rowKey, clustering, path), deletion);
+        PartitionUpdate upd = PartitionUpdate.singleRowUpdate(indexCfs.metadata, valueKey, row);
+
         indexCfs.apply(upd, SecondaryIndexManager.nullUpdater, opGroup, null);
         if (logger.isDebugEnabled())
             logger.debug("removed index entry for cleaned-up value {}:{}", valueKey, upd);
@@ -126,18 +125,16 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
 
     public void insert(ByteBuffer rowKey, Clustering clustering, Cell cell, OpOrder.Group opGroup)
     {
-        insert(rowKey, clustering, cell, cell.livenessInfo(), opGroup);
+        insert(rowKey, clustering, cell, LivenessInfo.create(cell.timestamp(), cell.ttl(), cell.localDeletionTime()), opGroup);
     }
 
     public void insert(ByteBuffer rowKey, Clustering clustering, Cell cell, LivenessInfo info, OpOrder.Group opGroup)
     {
         DecoratedKey valueKey = getIndexKeyFor(getIndexedValue(rowKey, clustering, cell));
 
-        PartitionUpdate upd = new PartitionUpdate(indexCfs.metadata, valueKey, PartitionColumns.NONE, 1);
-        Row.Writer writer = upd.writer();
-        Rows.writeClustering(makeIndexClustering(rowKey, clustering, cell), writer);
-        writer.writePartitionKeyLivenessInfo(info);
-        writer.endOfRow();
+        Row row = ArrayBackedRow.noCellLiveRow(makeIndexClustering(rowKey, clustering, cell), info);
+        PartitionUpdate upd = PartitionUpdate.singleRowUpdate(indexCfs.metadata, valueKey, row);
+
         if (logger.isDebugEnabled())
             logger.debug("applying index row {} in {}", indexCfs.metadata.getKeyValidator().getString(valueKey.getKey()), upd);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
index ab8e688..897aa9c 100644
--- a/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.db.index;
 
 import java.nio.ByteBuffer;
+import java.util.Iterator;
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
@@ -87,17 +88,18 @@ public abstract class PerColumnSecondaryIndex extends SecondaryIndex
             long timestamp = row.primaryKeyLivenessInfo().timestamp();
             int ttl = row.primaryKeyLivenessInfo().ttl();
 
-            for (Cell cell : row)
+            for (Cell cell : row.cells())
             {
-                if (cell.isLive(nowInSec) && cell.livenessInfo().timestamp() > timestamp)
+                if (cell.isLive(nowInSec) && cell.timestamp() > timestamp)
                 {
-                    timestamp = cell.livenessInfo().timestamp();
-                    ttl = cell.livenessInfo().ttl();
+                    timestamp = cell.timestamp();
+                    ttl = cell.ttl();
                 }
             }
             maybeIndex(key.getKey(), clustering, timestamp, ttl, opGroup, nowInSec);
         }
-        for (Cell cell : row)
+
+        for (Cell cell : row.cells())
         {
             if (!indexes(cell.column()))
                 continue;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
index 1bd5452..aaefc9c 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
@@ -18,15 +18,7 @@
 package org.apache.cassandra.db.index;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.IdentityHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentNavigableMap;
@@ -469,7 +461,8 @@ public class SecondaryIndexManager
             if (!row.deletion().isLive())
                 for (PerColumnSecondaryIndex index : indexes)
                     index.maybeDelete(key, clustering, row.deletion(), opGroup);
-            for (Cell cell : row)
+
+            for (Cell cell : row.cells())
             {
                 for (PerColumnSecondaryIndex index : indexes)
                 {
@@ -636,8 +629,7 @@ public class SecondaryIndexManager
         // Completely identical cells (including expiring columns with
         // identical ttl & localExpirationTime) will not get this far due
         // to the oldCell.equals(newCell) in StandardUpdater.update
-        return !oldCell.value().equals(newCell.value())
-            || oldCell.livenessInfo().timestamp() != newCell.livenessInfo().timestamp();
+        return !oldCell.value().equals(newCell.value()) || oldCell.timestamp() != newCell.timestamp();
     }
 
     private Set<String> filterByColumn(Set<String> idxNames)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/index/SecondaryIndexSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndexSearcher.java b/src/java/org/apache/cassandra/db/index/SecondaryIndexSearcher.java
index 1d978a2..d4ca707 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexSearcher.java
@@ -112,7 +112,7 @@ public abstract class SecondaryIndexSearcher
                 NavigableSet<Clustering> requested = ((ClusteringIndexNamesFilter)filter).requestedRows();
                 BTreeSet.Builder<Clustering> clusterings = BTreeSet.builder(index.getIndexComparator());
                 for (Clustering c : requested)
-                    clusterings.add(index.makeIndexClustering(pk, c, (Cell)null).takeAlias());
+                    clusterings.add(index.makeIndexClustering(pk, c, (Cell)null));
                 return new ClusteringIndexNamesFilter(clusterings.build(), filter.isReversed());
             }
             else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
index 7a40a90..e073802 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
@@ -112,11 +112,8 @@ public abstract class CompositesIndex extends AbstractSimplePerColumnSecondaryIn
 
     public void delete(IndexedEntry entry, OpOrder.Group opGroup, int nowInSec)
     {
-        PartitionUpdate upd = new PartitionUpdate(indexCfs.metadata, entry.indexValue, PartitionColumns.NONE, 1);
-        Row.Writer writer = upd.writer();
-        Rows.writeClustering(entry.indexClustering, writer);
-        writer.writeRowDeletion(new SimpleDeletionTime(entry.timestamp, nowInSec));
-        writer.endOfRow();
+        Row row = ArrayBackedRow.emptyDeletedRow(entry.indexClustering, new DeletionTime(entry.timestamp, nowInSec));
+        PartitionUpdate upd = PartitionUpdate.singleRowUpdate(indexCfs.metadata, entry.indexValue, row);
         indexCfs.apply(upd, SecondaryIndexManager.nullUpdater, opGroup, null);
 
         if (logger.isDebugEnabled())
@@ -159,10 +156,10 @@ public abstract class CompositesIndex extends AbstractSimplePerColumnSecondaryIn
         public IndexedEntry(DecoratedKey indexValue, Clustering indexClustering, long timestamp, ByteBuffer indexedKey, Clustering indexedEntryClustering)
         {
             this.indexValue = indexValue;
-            this.indexClustering = indexClustering.takeAlias();
+            this.indexClustering = indexClustering;
             this.timestamp = timestamp;
             this.indexedKey = indexedKey;
-            this.indexedEntryClustering = indexedEntryClustering.takeAlias();
+            this.indexedEntryClustering = indexedEntryClustering;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java
index aa58511..6529ad9 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java
@@ -118,7 +118,7 @@ public class CompositesIndexOnClusteringKey extends CompositesIndex
     public void maybeIndex(ByteBuffer partitionKey, Clustering clustering, long timestamp, int ttl, OpOrder.Group opGroup, int nowInSec)
     {
         if (clustering != Clustering.STATIC_CLUSTERING && clustering.get(columnDef.position()) != null)
-            insert(partitionKey, clustering, null, SimpleLivenessInfo.forUpdate(timestamp, ttl, nowInSec, indexCfs.metadata), opGroup);
+            insert(partitionKey, clustering, null, LivenessInfo.create(indexCfs.metadata, timestamp, ttl, nowInSec), opGroup);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
index 5af842c..30391cf 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
@@ -90,10 +90,9 @@ public class CompositesIndexOnCollectionValue extends CompositesIndex
 
     public boolean isStale(Row data, ByteBuffer indexValue, int nowInSec)
     {
-        Iterator<Cell> iter = data.getCells(columnDef);
-        while (iter.hasNext())
+        ComplexColumnData complexData = data.getComplexColumnData(columnDef);
+        for (Cell cell : complexData)
         {
-            Cell cell = iter.next();
             if (cell.isLive(nowInSec) && ((CollectionType) columnDef.type).valueComparator().compare(indexValue, cell.value()) == 0)
                 return false;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnPartitionKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnPartitionKey.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnPartitionKey.java
index d48e58b..a93f8e1 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnPartitionKey.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnPartitionKey.java
@@ -93,7 +93,7 @@ public class CompositesIndexOnPartitionKey extends CompositesIndex
     @Override
     public void maybeIndex(ByteBuffer partitionKey, Clustering clustering, long timestamp, int ttl, OpOrder.Group opGroup, int nowInSec)
     {
-        insert(partitionKey, clustering, null, SimpleLivenessInfo.forUpdate(timestamp, ttl, nowInSec, indexCfs.metadata), opGroup);
+        insert(partitionKey, clustering, null, LivenessInfo.create(indexCfs.metadata, timestamp, ttl, nowInSec), opGroup);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java b/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
index 029dd3c..ce92164 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
@@ -171,49 +171,20 @@ public class CompositesSearcher extends SecondaryIndexSearcher
                                                      final OpOrder.Group writeOp,
                                                      final int nowInSec)
     {
-        return new WrappingUnfilteredRowIterator(dataIter)
+        return new AlteringUnfilteredRowIterator(dataIter)
         {
             private int entriesIdx;
-            private Unfiltered next;
 
             @Override
-            public boolean hasNext()
-            {
-                return prepareNext();
-            }
-
-            @Override
-            public Unfiltered next()
+            protected Row computeNext(Row row)
             {
-                if (next == null)
-                    prepareNext();
+                CompositesIndex.IndexedEntry entry = findEntry(row.clustering(), writeOp, nowInSec);
+                if (!index.isStale(row, indexValue, nowInSec))
+                    return row;
 
-                Unfiltered toReturn = next;
-                next = null;
-                return toReturn;
-            }
-
-            private boolean prepareNext()
-            {
-                if (next != null)
-                    return true;
-
-                while (super.hasNext())
-                {
-                    next = super.next();
-                    if (next.kind() != Unfiltered.Kind.ROW)
-                        return true;
-
-                    Row row = (Row)next;
-                    CompositesIndex.IndexedEntry entry = findEntry(row.clustering(), writeOp, nowInSec);
-                    if (!index.isStale(row, indexValue, nowInSec))
-                        return true;
-
-                    // The entry is stale: delete the entry and ignore otherwise
-                    index.delete(entry, writeOp, nowInSec);
-                    next = null;
-                }
-                return false;
+                // The entry is stale: delete the entry and ignore otherwise
+                index.delete(entry, writeOp, nowInSec);
+                return null;
             }
 
             private CompositesIndex.IndexedEntry findEntry(Clustering clustering, OpOrder.Group writeOp, int nowInSec)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java b/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
index 6b53640..118fb75 100644
--- a/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
@@ -138,7 +138,7 @@ public class KeysSearcher extends SecondaryIndexSearcher
             // is the indexed name. Ans so we need to materialize the partition.
             ArrayBackedPartition result = ArrayBackedPartition.create(iterator);
             iterator.close();
-            Row data = result.getRow(new SimpleClustering(index.indexedColumn().name.bytes));
+            Row data = result.getRow(new Clustering(index.indexedColumn().name.bytes));
             Cell cell = data == null ? null : data.getCell(baseCfs.metadata.compactValueColumn());
             return deleteIfStale(iterator.partitionKey(), cell, index, indexHit, indexedValue, writeOp, nowInSec)
                  ? null
@@ -173,10 +173,10 @@ public class KeysSearcher extends SecondaryIndexSearcher
         {
             // Index is stale, remove the index entry and ignore
             index.delete(partitionKey.getKey(),
-                         new SimpleClustering(index.indexedColumn().name.bytes),
+                         new Clustering(index.indexedColumn().name.bytes),
                          indexedValue,
                          null,
-                         new SimpleDeletionTime(indexHit.primaryKeyLivenessInfo().timestamp(), nowInSec),
+                         new DeletionTime(indexHit.primaryKeyLivenessInfo().timestamp(), nowInSec),
                          writeOp);
             return true;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/marshal/AbstractType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
index 78ead36..258a8a5 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.marshal;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -36,6 +35,7 @@ import org.apache.cassandra.serializers.MarshalException;
 
 import org.github.jamm.Unmetered;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -325,7 +325,7 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>
         if (valueLengthIfFixed() >= 0)
             out.write(value);
         else
-            ByteBufferUtil.writeWithLength(value, out);
+            ByteBufferUtil.writeWithVIntLength(value, out);
     }
 
     public long writtenLength(ByteBuffer value)
@@ -333,25 +333,25 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>
         assert value.hasRemaining();
         return valueLengthIfFixed() >= 0
              ? value.remaining()
-             : TypeSizes.sizeofWithLength(value);
+             : TypeSizes.sizeofWithVIntLength(value);
     }
 
-    public ByteBuffer readValue(DataInput in) throws IOException
+    public ByteBuffer readValue(DataInputPlus in) throws IOException
     {
         int length = valueLengthIfFixed();
         if (length >= 0)
             return ByteBufferUtil.read(in, length);
         else
-            return ByteBufferUtil.readWithLength(in);
+            return ByteBufferUtil.readWithVIntLength(in);
     }
 
-    public void skipValue(DataInput in) throws IOException
+    public void skipValue(DataInputPlus in) throws IOException
     {
         int length = valueLengthIfFixed();
-        if (length < 0)
-            length = in.readInt();
-
-        FileUtils.skipBytesFully(in, length);
+        if (length >= 0)
+            FileUtils.skipBytesFully(in, length);
+        else
+            ByteBufferUtil.skipWithVIntLength(in);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/marshal/CollectionType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CollectionType.java b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
index a850305..9a096d0 100644
--- a/src/java/org/apache/cassandra/db/marshal/CollectionType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
-import java.io.DataInput;
 import java.io.IOException;
 import java.util.List;
 import java.util.Iterator;
@@ -34,6 +33,7 @@ import org.apache.cassandra.cql3.Sets;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.serializers.CollectionSerializer;
@@ -236,23 +236,22 @@ public abstract class CollectionType<T> extends AbstractType<T>
     {
         public void serialize(CellPath path, DataOutputPlus out) throws IOException
         {
-            ByteBufferUtil.writeWithLength(path.get(0), out);
+            ByteBufferUtil.writeWithVIntLength(path.get(0), out);
         }
 
-        public CellPath deserialize(DataInput in) throws IOException
+        public CellPath deserialize(DataInputPlus in) throws IOException
         {
-            return CellPath.create(ByteBufferUtil.readWithLength(in));
+            return CellPath.create(ByteBufferUtil.readWithVIntLength(in));
         }
 
         public long serializedSize(CellPath path)
         {
-            return TypeSizes.sizeofWithLength(path.get(0));
+            return ByteBufferUtil.serializedSizeWithVIntLength(path.get(0));
         }
 
-        public void skip(DataInput in) throws IOException
+        public void skip(DataInputPlus in) throws IOException
         {
-            int length = in.readInt();
-            FileUtils.skipBytesFully(in, length);
+            ByteBufferUtil.skipWithVIntLength(in);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/partitions/AbstractPartitionData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/AbstractPartitionData.java b/src/java/org/apache/cassandra/db/partitions/AbstractPartitionData.java
deleted file mode 100644
index 6775cf1..0000000
--- a/src/java/org/apache/cassandra/db/partitions/AbstractPartitionData.java
+++ /dev/null
@@ -1,850 +0,0 @@
-/*
- * 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.partitions;
-
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import com.google.common.collect.AbstractIterator;
-import com.google.common.collect.UnmodifiableIterator;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.filter.ColumnFilter;
-import org.apache.cassandra.db.rows.*;
-import org.apache.cassandra.utils.SearchIterator;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Abstract common class for all non-thread safe Partition implementations.
- */
-public abstract class AbstractPartitionData implements Partition, Iterable<Row>
-{
-    private static final Logger logger = LoggerFactory.getLogger(AbstractPartitionData.class);
-
-    protected final CFMetaData metadata;
-    protected final DecoratedKey key;
-
-    protected final DeletionInfo deletionInfo;
-    protected final PartitionColumns columns;
-
-    protected Row staticRow;
-
-    protected int rows;
-
-    // The values for the clustering columns of the rows contained in this partition object. If
-    // clusteringSize is the size of the clustering comparator for this table, clusterings has size
-    // clusteringSize * rows where rows is the number of rows stored, and row i has it's clustering
-    // column values at indexes [clusteringSize * i, clusteringSize * (i + 1)).
-    protected ByteBuffer[] clusterings;
-
-    // The partition key column liveness infos for the rows of this partition (row i has its liveness info at index i).
-    protected final LivenessInfoArray livenessInfos;
-    // The row deletion for the rows of this partition (row i has its row deletion at index i).
-    protected final DeletionTimeArray deletions;
-
-    // The row data (cells data + complex deletions for complex columns) for the rows contained in this partition.
-    protected final RowDataBlock data;
-
-    // Stats over the rows stored in this partition.
-    private final RowStats.Collector statsCollector = new RowStats.Collector();
-
-    // The maximum timestamp for any data contained in this partition.
-    protected long maxTimestamp = Long.MIN_VALUE;
-
-    private AbstractPartitionData(CFMetaData metadata,
-                                    DecoratedKey key,
-                                    DeletionInfo deletionInfo,
-                                    ByteBuffer[] clusterings,
-                                    LivenessInfoArray livenessInfos,
-                                    DeletionTimeArray deletions,
-                                    PartitionColumns columns,
-                                    RowDataBlock data)
-    {
-        this.metadata = metadata;
-        this.key = key;
-        this.deletionInfo = deletionInfo;
-        this.clusterings = clusterings;
-        this.livenessInfos = livenessInfos;
-        this.deletions = deletions;
-        this.columns = columns;
-        this.data = data;
-
-        collectStats(deletionInfo.getPartitionDeletion());
-        Iterator<RangeTombstone> iter = deletionInfo.rangeIterator(false);
-        while (iter.hasNext())
-            collectStats(iter.next().deletionTime());
-    }
-
-    protected AbstractPartitionData(CFMetaData metadata,
-                                    DecoratedKey key,
-                                    DeletionInfo deletionInfo,
-                                    PartitionColumns columns,
-                                    RowDataBlock data,
-                                    int initialRowCapacity)
-    {
-        this(metadata,
-             key,
-             deletionInfo,
-             new ByteBuffer[initialRowCapacity * metadata.clusteringColumns().size()],
-             new LivenessInfoArray(initialRowCapacity),
-             new DeletionTimeArray(initialRowCapacity),
-             columns,
-             data);
-    }
-
-    protected AbstractPartitionData(CFMetaData metadata,
-                                    DecoratedKey key,
-                                    DeletionTime partitionDeletion,
-                                    PartitionColumns columns,
-                                    int initialRowCapacity,
-                                    boolean sortable)
-    {
-        this(metadata,
-             key,
-             new DeletionInfo(partitionDeletion.takeAlias()),
-             columns,
-             new RowDataBlock(columns.regulars, initialRowCapacity, sortable, metadata.isCounter()),
-             initialRowCapacity);
-    }
-
-    private void collectStats(DeletionTime dt)
-    {
-        statsCollector.updateDeletionTime(dt);
-        maxTimestamp = Math.max(maxTimestamp, dt.markedForDeleteAt());
-    }
-
-    private void collectStats(LivenessInfo info)
-    {
-        statsCollector.updateTimestamp(info.timestamp());
-        statsCollector.updateTTL(info.ttl());
-        statsCollector.updateLocalDeletionTime(info.localDeletionTime());
-        maxTimestamp = Math.max(maxTimestamp, info.timestamp());
-    }
-
-    public CFMetaData metadata()
-    {
-        return metadata;
-    }
-
-    public DecoratedKey partitionKey()
-    {
-        return key;
-    }
-
-    public DeletionTime partitionLevelDeletion()
-    {
-        return deletionInfo.getPartitionDeletion();
-    }
-
-    public PartitionColumns columns()
-    {
-        return columns;
-    }
-
-    public Row staticRow()
-    {
-        return staticRow == null ? Rows.EMPTY_STATIC_ROW : staticRow;
-    }
-
-    public RowStats stats()
-    {
-        return statsCollector.get();
-    }
-
-    /**
-     * The deletion info for the partition update.
-     *
-     * <b>warning:</b> the returned object should be used in a read-only fashion. In particular,
-     * it should not be used to add new range tombstones to this deletion. For that,
-     * {@link addRangeTombstone} should be used instead. The reason being that adding directly to
-     * the returned object would bypass some stats collection that {@code addRangeTombstone} does.
-     *
-     * @return the deletion info for the partition update for use as read-only.
-     */
-    public DeletionInfo deletionInfo()
-    {
-        // TODO: it is a tad fragile that deletionInfo can be but shouldn't be modified. We
-        // could add the option of providing a read-only view of a DeletionInfo instead.
-        return deletionInfo;
-    }
-
-    public void addPartitionDeletion(DeletionTime deletionTime)
-    {
-        collectStats(deletionTime);
-        deletionInfo.add(deletionTime);
-    }
-
-    public void addRangeTombstone(Slice deletedSlice, DeletionTime deletion)
-    {
-        addRangeTombstone(new RangeTombstone(deletedSlice, deletion.takeAlias()));
-    }
-
-    public void addRangeTombstone(RangeTombstone range)
-    {
-        collectStats(range.deletionTime());
-        deletionInfo.add(range, metadata.comparator);
-    }
-
-    /**
-     * Swap row i and j.
-     *
-     * This is only used when we need to reorder rows because those were not added in clustering order,
-     * which happens in {@link PartitionUpdate#sort} and {@link ArrayBackedPartition#create}. This method
-     * is public only because {@code PartitionUpdate} needs to implement {@link Sorting.Sortable}, but
-     * it should really only be used by subclasses (and with care) in practice.
-     */
-    public void swap(int i, int j)
-    {
-        int cs = metadata.clusteringColumns().size();
-        for (int k = 0; k < cs; k++)
-        {
-            ByteBuffer tmp = clusterings[j * cs + k];
-            clusterings[j * cs + k] = clusterings[i * cs + k];
-            clusterings[i * cs + k] = tmp;
-        }
-
-        livenessInfos.swap(i, j);
-        deletions.swap(i, j);
-        data.swap(i, j);
-    }
-
-    protected void merge(int i, int j, int nowInSec)
-    {
-        data.merge(i, j, nowInSec);
-        if (livenessInfos.timestamp(i) > livenessInfos.timestamp(j))
-            livenessInfos.move(i, j);
-        if (deletions.supersedes(i, j))
-            deletions.move(i, j);
-    }
-
-    protected void move(int i, int j)
-    {
-        int cs = metadata.clusteringColumns().size();
-        for (int k = 0; k < cs; k++)
-            clusterings[j * cs + k] = clusterings[i * cs + k];
-        data.move(i, j);
-        livenessInfos.move(i, j);
-        deletions.move(i, j);
-    }
-
-    public int rowCount()
-    {
-        return rows;
-    }
-
-    public boolean isEmpty()
-    {
-        return deletionInfo.isLive() && rows == 0 && staticRow().isEmpty();
-    }
-
-    protected void clear()
-    {
-        rows = 0;
-        Arrays.fill(clusterings, null);
-        livenessInfos.clear();
-        deletions.clear();
-        data.clear();
-    }
-
-    @Override
-    public String toString()
-    {
-        StringBuilder sb = new StringBuilder();
-        CFMetaData metadata = metadata();
-        sb.append(String.format("Partition[%s.%s] key=%s columns=%s deletion=%s",
-                    metadata.ksName,
-                    metadata.cfName,
-                    metadata.getKeyValidator().getString(partitionKey().getKey()),
-                    columns(),
-                    deletionInfo));
-
-        if (staticRow() != Rows.EMPTY_STATIC_ROW)
-            sb.append("\n    ").append(staticRow().toString(metadata, true));
-
-        // We use createRowIterator() directly instead of iterator() because that avoids
-        // sorting for PartitionUpdate (which inherit this method) and that is useful because
-        //  1) it can help with debugging and 2) we can't write after sorting but we want to
-        // be able to print an update while we build it (again for debugging)
-        Iterator<Row> iterator = createRowIterator(null, false);
-        while (iterator.hasNext())
-            sb.append("\n    ").append(iterator.next().toString(metadata, true));
-
-        return sb.toString();
-    }
-
-    protected void reverse()
-    {
-        for (int i = 0; i < rows / 2; i++)
-            swap(i, rows - 1 - i);
-    }
-
-    public Row getRow(Clustering clustering)
-    {
-        Row row = searchIterator(ColumnFilter.selection(columns()), false).next(clustering);
-        // Note that for statics, this will never return null, this will return an empty row. However,
-        // it's more consistent for this method to return null if we don't really have a static row.
-        return row == null || (clustering == Clustering.STATIC_CLUSTERING && row.isEmpty()) ? null : row;
-    }
-
-    /**
-     * Returns an iterator that iterators over the rows of this update in clustering order.
-     *
-     * @return an iterator over the rows of this update.
-     */
-    public Iterator<Row> iterator()
-    {
-        return createRowIterator(null, false);
-    }
-
-    public SearchIterator<Clustering, Row> searchIterator(final ColumnFilter columns, boolean reversed)
-    {
-        final RowIterator iter = createRowIterator(columns, reversed);
-        return new SearchIterator<Clustering, Row>()
-        {
-            public boolean hasNext()
-            {
-                return iter.hasNext();
-            }
-
-            public Row next(Clustering key)
-            {
-                if (key == Clustering.STATIC_CLUSTERING)
-                {
-                    if (columns.fetchedColumns().statics.isEmpty() || staticRow().isEmpty())
-                        return Rows.EMPTY_STATIC_ROW;
-
-                    return FilteringRow.columnsFilteringRow(columns).setTo(staticRow());
-                }
-
-                return iter.seekTo(key) ? iter.next() : null;
-            }
-        };
-    }
-
-    public UnfilteredRowIterator unfilteredIterator()
-    {
-        return unfilteredIterator(ColumnFilter.selection(columns()), Slices.ALL, false);
-    }
-
-    public UnfilteredRowIterator unfilteredIterator(ColumnFilter columns, Slices slices, boolean reversed)
-    {
-        return slices.makeSliceIterator(sliceableUnfilteredIterator(columns, reversed));
-    }
-
-    protected SliceableUnfilteredRowIterator sliceableUnfilteredIterator()
-    {
-        return sliceableUnfilteredIterator(ColumnFilter.selection(columns()), false);
-    }
-
-    protected SliceableUnfilteredRowIterator sliceableUnfilteredIterator(final ColumnFilter selection, final boolean reversed)
-    {
-        return new AbstractSliceableIterator(this, selection.fetchedColumns(), reversed)
-        {
-            private final RowIterator rowIterator = createRowIterator(selection, reversed);
-            private RowAndTombstoneMergeIterator mergeIterator = new RowAndTombstoneMergeIterator(metadata.comparator, reversed);
-
-            protected Unfiltered computeNext()
-            {
-                if (!mergeIterator.isSet())
-                    mergeIterator.setTo(rowIterator, deletionInfo.rangeIterator(reversed));
-
-                return mergeIterator.hasNext() ? mergeIterator.next() : endOfData();
-            }
-
-            public Iterator<Unfiltered> slice(Slice slice)
-            {
-                return mergeIterator.setTo(rowIterator.slice(slice), deletionInfo.rangeIterator(slice, reversed));
-            }
-        };
-    }
-
-    private RowIterator createRowIterator(ColumnFilter columns, boolean reversed)
-    {
-        return reversed ? new ReverseRowIterator(columns) : new ForwardRowIterator(columns);
-    }
-
-    /**
-     * An iterator over the rows of this partition that reuse the same row object.
-     */
-    private abstract class RowIterator extends UnmodifiableIterator<Row>
-    {
-        protected final InternalReusableClustering clustering = new InternalReusableClustering();
-        protected final InternalReusableRow reusableRow;
-        protected final FilteringRow filter;
-
-        protected int next;
-
-        protected RowIterator(final ColumnFilter columns)
-        {
-            this.reusableRow = new InternalReusableRow(clustering);
-            this.filter = columns == null ? null : FilteringRow.columnsFilteringRow(columns);
-        }
-
-        /*
-         * Move the iterator so that row {@code name} is returned next by {@code next} if that
-         * row exists. Otherwise the first row sorting after {@code name} will be returned.
-         * Returns whether {@code name} was found or not.
-         */
-        public abstract boolean seekTo(Clustering name);
-
-        public abstract Iterator<Row> slice(Slice slice);
-
-        protected Row setRowTo(int row)
-        {
-            reusableRow.setTo(row);
-            return filter == null ? reusableRow : filter.setTo(reusableRow);
-        }
-
-        /**
-         * Simple binary search.
-         */
-        protected int binarySearch(ClusteringPrefix name, int fromIndex, int toIndex)
-        {
-            int low = fromIndex;
-            int mid = toIndex;
-            int high = mid - 1;
-            int result = -1;
-            while (low <= high)
-            {
-                mid = (low + high) >> 1;
-                if ((result = metadata.comparator.compare(name, clustering.setTo(mid))) > 0)
-                    low = mid + 1;
-                else if (result == 0)
-                    return mid;
-                else
-                    high = mid - 1;
-            }
-            return -mid - (result < 0 ? 1 : 2);
-        }
-    }
-
-    private class ForwardRowIterator extends RowIterator
-    {
-        private ForwardRowIterator(ColumnFilter columns)
-        {
-            super(columns);
-            this.next = 0;
-        }
-
-        public boolean hasNext()
-        {
-            return next < rows;
-        }
-
-        public Row next()
-        {
-            return setRowTo(next++);
-        }
-
-        public boolean seekTo(Clustering name)
-        {
-            if (next >= rows)
-                return false;
-
-            int idx = binarySearch(name, next, rows);
-            next = idx >= 0 ? idx : -idx - 1;
-            return idx >= 0;
-        }
-
-        public Iterator<Row> slice(Slice slice)
-        {
-            int sidx = binarySearch(slice.start(), next, rows);
-            final int start = sidx >= 0 ? sidx : -sidx - 1;
-            if (start >= rows)
-                return Collections.emptyIterator();
-
-            int eidx = binarySearch(slice.end(), start, rows);
-            // The insertion point is the first element greater than slice.end(), so we want the previous index
-            final int end = eidx >= 0 ? eidx : -eidx - 2;
-
-            // Remember the end to speed up potential further slice search
-            next = end;
-
-            if (start > end)
-                return Collections.emptyIterator();
-
-            return new AbstractIterator<Row>()
-            {
-                private int i = start;
-
-                protected Row computeNext()
-                {
-                    if (i >= rows || i > end)
-                        return endOfData();
-
-                    return setRowTo(i++);
-                }
-            };
-        }
-    }
-
-    private class ReverseRowIterator extends RowIterator
-    {
-        private ReverseRowIterator(ColumnFilter columns)
-        {
-            super(columns);
-            this.next = rows - 1;
-        }
-
-        public boolean hasNext()
-        {
-            return next >= 0;
-        }
-
-        public Row next()
-        {
-            return setRowTo(next--);
-        }
-
-        public boolean seekTo(Clustering name)
-        {
-            // We only use that method with forward iterators.
-            throw new UnsupportedOperationException();
-        }
-
-        public Iterator<Row> slice(Slice slice)
-        {
-            int sidx = binarySearch(slice.end(), 0, next + 1);
-            // The insertion point is the first element greater than slice.end(), so we want the previous index
-            final int start = sidx >= 0 ? sidx : -sidx - 2;
-            if (start < 0)
-                return Collections.emptyIterator();
-
-            int eidx = binarySearch(slice.start(), 0, start + 1);
-            final int end = eidx >= 0 ? eidx : -eidx - 1;
-
-            // Remember the end to speed up potential further slice search
-            next = end;
-
-            if (start < end)
-                return Collections.emptyIterator();
-
-            return new AbstractIterator<Row>()
-            {
-                private int i = start;
-
-                protected Row computeNext()
-                {
-                    if (i < 0 || i < end)
-                        return endOfData();
-
-                    return setRowTo(i--);
-                }
-            };
-        }
-    }
-
-    /**
-     * A reusable view over the clustering of this partition.
-     */
-    protected class InternalReusableClustering extends Clustering
-    {
-        final int size = metadata.clusteringColumns().size();
-        private int base;
-
-        public int size()
-        {
-            return size;
-        }
-
-        public Clustering setTo(int row)
-        {
-            base = row * size;
-            return this;
-        }
-
-        public ByteBuffer get(int i)
-        {
-            return clusterings[base + i];
-        }
-
-        public ByteBuffer[] getRawValues()
-        {
-            ByteBuffer[] values = new ByteBuffer[size];
-            for (int i = 0; i < size; i++)
-                values[i] = get(i);
-            return values;
-        }
-    };
-
-    /**
-     * A reusable view over the rows of this partition.
-     */
-    protected class InternalReusableRow extends AbstractReusableRow
-    {
-        private final LivenessInfoArray.Cursor liveness = new LivenessInfoArray.Cursor();
-        private final DeletionTimeArray.Cursor deletion = new DeletionTimeArray.Cursor();
-        private final InternalReusableClustering clustering;
-
-        private int row;
-
-        public InternalReusableRow()
-        {
-            this(new InternalReusableClustering());
-        }
-
-        public InternalReusableRow(InternalReusableClustering clustering)
-        {
-            this.clustering = clustering;
-        }
-
-        protected RowDataBlock data()
-        {
-            return data;
-        }
-
-        public Row setTo(int row)
-        {
-            this.clustering.setTo(row);
-            this.liveness.setTo(livenessInfos, row);
-            this.deletion.setTo(deletions, row);
-            this.row = row;
-            return this;
-        }
-
-        protected int row()
-        {
-            return row;
-        }
-
-        public Clustering clustering()
-        {
-            return clustering;
-        }
-
-        public LivenessInfo primaryKeyLivenessInfo()
-        {
-            return liveness;
-        }
-
-        public DeletionTime deletion()
-        {
-            return deletion;
-        }
-    };
-
-    private static abstract class AbstractSliceableIterator extends AbstractUnfilteredRowIterator implements SliceableUnfilteredRowIterator
-    {
-        private AbstractSliceableIterator(AbstractPartitionData data, PartitionColumns columns, boolean isReverseOrder)
-        {
-            super(data.metadata, data.key, data.partitionLevelDeletion(), columns, data.staticRow(), isReverseOrder, data.stats());
-        }
-    }
-
-    /**
-     * A row writer to add rows to this partition.
-     */
-    protected class Writer extends RowDataBlock.Writer
-    {
-        private int clusteringBase;
-
-        private int simpleColumnsSetInRow;
-        private final Set<ColumnDefinition> complexColumnsSetInRow = new HashSet<>();
-
-        public Writer(boolean inOrderCells)
-        {
-            super(data, inOrderCells);
-        }
-
-        public void writeClusteringValue(ByteBuffer value)
-        {
-            ensureCapacity(row);
-            clusterings[clusteringBase++] = value;
-        }
-
-        public void writePartitionKeyLivenessInfo(LivenessInfo info)
-        {
-            ensureCapacity(row);
-            livenessInfos.set(row, info);
-            collectStats(info);
-        }
-
-        public void writeRowDeletion(DeletionTime deletion)
-        {
-            ensureCapacity(row);
-            if (!deletion.isLive())
-                deletions.set(row, deletion);
-
-            collectStats(deletion);
-        }
-
-        @Override
-        public void writeCell(ColumnDefinition column, boolean isCounter, ByteBuffer value, LivenessInfo info, CellPath path)
-        {
-            ensureCapacity(row);
-            collectStats(info);
-
-            if (column.isComplex())
-                complexColumnsSetInRow.add(column);
-            else
-                ++simpleColumnsSetInRow;
-
-            super.writeCell(column, isCounter, value, info, path);
-        }
-
-        @Override
-        public void writeComplexDeletion(ColumnDefinition c, DeletionTime complexDeletion)
-        {
-            ensureCapacity(row);
-            collectStats(complexDeletion);
-
-            super.writeComplexDeletion(c, complexDeletion);
-        }
-
-        @Override
-        public void endOfRow()
-        {
-            super.endOfRow();
-            ++rows;
-
-            statsCollector.updateColumnSetPerRow(simpleColumnsSetInRow + complexColumnsSetInRow.size());
-
-            simpleColumnsSetInRow = 0;
-            complexColumnsSetInRow.clear();
-        }
-
-        public int currentRow()
-        {
-            return row;
-        }
-
-        private void ensureCapacity(int rowToSet)
-        {
-            int originalCapacity = livenessInfos.size();
-            if (rowToSet < originalCapacity)
-                return;
-
-            int newCapacity = RowDataBlock.computeNewCapacity(originalCapacity, rowToSet);
-
-            int clusteringSize = metadata.clusteringColumns().size();
-
-            clusterings = Arrays.copyOf(clusterings, newCapacity * clusteringSize);
-
-            livenessInfos.resize(newCapacity);
-            deletions.resize(newCapacity);
-        }
-
-        @Override
-        public Writer reset()
-        {
-            super.reset();
-            clusteringBase = 0;
-            simpleColumnsSetInRow = 0;
-            complexColumnsSetInRow.clear();
-            return this;
-        }
-    }
-
-    /**
-     * A range tombstone marker writer to add range tombstone markers to this partition.
-     */
-    protected class RangeTombstoneCollector implements RangeTombstoneMarker.Writer
-    {
-        private final boolean reversed;
-
-        private final ByteBuffer[] nextValues = new ByteBuffer[metadata().comparator.size()];
-        private int size;
-        private RangeTombstone.Bound.Kind nextKind;
-
-        private Slice.Bound openBound;
-        private DeletionTime openDeletion;
-
-        public RangeTombstoneCollector(boolean reversed)
-        {
-            this.reversed = reversed;
-        }
-
-        public void writeClusteringValue(ByteBuffer value)
-        {
-            nextValues[size++] = value;
-        }
-
-        public void writeBoundKind(RangeTombstone.Bound.Kind kind)
-        {
-            nextKind = kind;
-        }
-
-        private ByteBuffer[] getValues()
-        {
-            return Arrays.copyOfRange(nextValues, 0, size);
-        }
-
-        private void open(RangeTombstone.Bound.Kind kind, DeletionTime deletion)
-        {
-            openBound = Slice.Bound.create(kind, getValues());
-            openDeletion = deletion.takeAlias();
-        }
-
-        private void close(RangeTombstone.Bound.Kind kind, DeletionTime deletion)
-        {
-            assert deletion.equals(openDeletion) : "Expected " + openDeletion + " but was "  + deletion;
-            Slice.Bound closeBound = Slice.Bound.create(kind, getValues());
-            Slice slice = reversed
-                        ? Slice.make(closeBound, openBound)
-                        : Slice.make(openBound, closeBound);
-            addRangeTombstone(slice, openDeletion);
-        }
-
-        public void writeBoundDeletion(DeletionTime deletion)
-        {
-            assert !nextKind.isBoundary();
-            if (nextKind.isOpen(reversed))
-                open(nextKind, deletion);
-            else
-                close(nextKind, deletion);
-        }
-
-        public void writeBoundaryDeletion(DeletionTime endDeletion, DeletionTime startDeletion)
-        {
-            assert nextKind.isBoundary();
-            DeletionTime closeTime = reversed ? startDeletion : endDeletion;
-            DeletionTime openTime = reversed ? endDeletion : startDeletion;
-
-            close(nextKind.closeBoundOfBoundary(reversed), closeTime);
-            open(nextKind.openBoundOfBoundary(reversed), openTime);
-        }
-
-        public void endOfMarker()
-        {
-            clear();
-        }
-
-        private void addRangeTombstone(Slice deletionSlice, DeletionTime dt)
-        {
-            AbstractPartitionData.this.addRangeTombstone(deletionSlice, dt);
-        }
-
-        private void clear()
-        {
-            size = 0;
-            Arrays.fill(nextValues, null);
-            nextKind = null;
-        }
-
-        public void reset()
-        {
-            openBound = null;
-            openDeletion = null;
-            clear();
-        }
-    }
-}


[05/15] cassandra git commit: Simplify some 8099's implementations

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/Rows.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Rows.java b/src/java/org/apache/cassandra/db/rows/Rows.java
index 76dcf60..122f7d3 100644
--- a/src/java/org/apache/cassandra/db/rows/Rows.java
+++ b/src/java/org/apache/cassandra/db/rows/Rows.java
@@ -20,14 +20,12 @@ package org.apache.cassandra.db.rows;
 import java.util.*;
 
 import com.google.common.collect.Iterators;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import com.google.common.collect.PeekingIterator;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.partitions.PartitionStatisticsCollector;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
-import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.SearchIterator;
 
 /**
@@ -35,110 +33,198 @@ import org.apache.cassandra.utils.SearchIterator;
  */
 public abstract class Rows
 {
-    private static final Logger logger = LoggerFactory.getLogger(Rows.class);
-
-    private Rows() {}
-
-    public static final Row EMPTY_STATIC_ROW = new AbstractRow()
+    // TODO: we could have a that in a more generic place...
+    private static final SearchIterator<ColumnDefinition, ColumnData> EMPTY_SEARCH_ITERATOR = new SearchIterator<ColumnDefinition, ColumnData>()
     {
-        public Columns columns()
+        public boolean hasNext()
         {
-            return Columns.NONE;
+            return false;
         }
 
-        public LivenessInfo primaryKeyLivenessInfo()
+        public ColumnData next(ColumnDefinition column)
         {
-            return LivenessInfo.NONE;
+            return null;
         }
+    };
 
-        public DeletionTime deletion()
-        {
-            return DeletionTime.LIVE;
-        }
+    private Rows() {}
 
-        public boolean isEmpty()
-        {
-            return true;
-        }
+    public static final Row EMPTY_STATIC_ROW = ArrayBackedRow.emptyRow(Clustering.STATIC_CLUSTERING);
 
-        public boolean hasComplexDeletion()
+    public static Row.Builder copy(Row row, Row.Builder builder)
+    {
+        builder.newRow(row.clustering());
+        builder.addPrimaryKeyLivenessInfo(row.primaryKeyLivenessInfo());
+        builder.addRowDeletion(row.deletion());
+        for (ColumnData cd : row)
         {
-            return false;
+            if (cd.column().isSimple())
+            {
+                builder.addCell((Cell)cd);
+            }
+            else
+            {
+                ComplexColumnData complexData = (ComplexColumnData)cd;
+                builder.addComplexDeletion(complexData.column(), complexData.complexDeletion());
+                for (Cell cell : complexData)
+                    builder.addCell(cell);
+            }
         }
+        return builder;
+    }
 
-        public Clustering clustering()
-        {
-            return Clustering.STATIC_CLUSTERING;
-        }
+    /**
+     * Collect statistics ont a given row.
+     *
+     * @param row the row for which to collect stats.
+     * @param collector the stats collector.
+     * @return the total number of cells in {@code row}.
+     */
+    public static int collectStats(Row row, PartitionStatisticsCollector collector)
+    {
+        assert !row.isEmpty();
 
-        public Cell getCell(ColumnDefinition c)
-        {
-            return null;
-        }
+        collector.update(row.primaryKeyLivenessInfo());
+        collector.update(row.deletion());
 
-        public Cell getCell(ColumnDefinition c, CellPath path)
+        int columnCount = 0;
+        int cellCount = 0;
+        for (ColumnData cd : row)
         {
-            return null;
-        }
+            if (cd.column().isSimple())
+            {
+                ++columnCount;
+                ++cellCount;
+                Cells.collectStats((Cell)cd, collector);
+            }
+            else
+            {
+                ComplexColumnData complexData = (ComplexColumnData)cd;
+                collector.update(complexData.complexDeletion());
+                if (complexData.hasCells())
+                {
+                    ++columnCount;
+                    for (Cell cell : complexData)
+                    {
+                        ++cellCount;
+                        Cells.collectStats(cell, collector);
+                    }
+                }
+            }
 
-        public Iterator<Cell> getCells(ColumnDefinition c)
-        {
-            return null;
         }
+        collector.updateColumnSetPerRow(columnCount);
+        return cellCount;
+    }
 
-        public DeletionTime getDeletion(ColumnDefinition c)
+    /**
+     * Given the result ({@code merged}) of merging multiple {@code inputs}, signals the difference between
+     * each input and {@code merged} to {@code diffListener}.
+     *
+     * @param merged the result of merging {@code inputs}.
+     * @param columns a superset of all the columns in any of {@code merged}/{@code inputs}.
+     * @param inputs the inputs whose merge yielded {@code merged}.
+     * @param diffListener the listener to which to signal the differences between the inputs and the merged
+     * result.
+     */
+    public static void diff(Row merged, Columns columns, Row[] inputs, RowDiffListener diffListener)
+    {
+        Clustering clustering = merged.clustering();
+        LivenessInfo mergedInfo = merged.primaryKeyLivenessInfo().isEmpty() ? null : merged.primaryKeyLivenessInfo();
+        DeletionTime mergedDeletion = merged.deletion().isLive() ? null : merged.deletion();
+        for (int i = 0; i < inputs.length; i++)
         {
-            return DeletionTime.LIVE;
+            Row input = inputs[i];
+            LivenessInfo inputInfo = input == null || input.primaryKeyLivenessInfo().isEmpty() ? null : input.primaryKeyLivenessInfo();
+            DeletionTime inputDeletion = input == null || input.deletion().isLive() ? null : input.deletion();
+
+            if (mergedInfo != null || inputInfo != null)
+                diffListener.onPrimaryKeyLivenessInfo(i, clustering, mergedInfo, inputInfo);
+            if (mergedDeletion != null || inputDeletion != null)
+                diffListener.onDeletion(i, clustering, mergedDeletion, inputDeletion);
         }
 
-        public Iterator<Cell> iterator()
+        SearchIterator<ColumnDefinition, ColumnData> mergedIterator = merged.searchIterator();
+        List<SearchIterator<ColumnDefinition, ColumnData>> inputIterators = new ArrayList<>(inputs.length);
+
+        for (Row row : inputs)
+            inputIterators.add(row == null ? EMPTY_SEARCH_ITERATOR : row.searchIterator());
+
+        Iterator<ColumnDefinition> simpleColumns = columns.simpleColumns();
+        while (simpleColumns.hasNext())
         {
-            return Iterators.<Cell>emptyIterator();
+            ColumnDefinition column = simpleColumns.next();
+            Cell mergedCell = (Cell)mergedIterator.next(column);
+            for (int i = 0; i < inputs.length; i++)
+            {
+                Cell inputCell = (Cell)inputIterators.get(i).next(column);
+                if (mergedCell != null || inputCell != null)
+                    diffListener.onCell(i, clustering, mergedCell, inputCell);
+            }
         }
 
-        public SearchIterator<ColumnDefinition, ColumnData> searchIterator()
+        Iterator<ColumnDefinition> complexColumns = columns.complexColumns();
+        while (complexColumns.hasNext())
         {
-            return new SearchIterator<ColumnDefinition, ColumnData>()
+            ColumnDefinition column = complexColumns.next();
+            ComplexColumnData mergedData = (ComplexColumnData)mergedIterator.next(column);
+            // Doing one input at a time is not the most efficient, but it's a lot simpler for now
+            for (int i = 0; i < inputs.length; i++)
             {
-                public boolean hasNext()
+                ComplexColumnData inputData = (ComplexColumnData)inputIterators.get(i).next(column);
+                if (mergedData == null)
                 {
-                    return false;
+                    if (inputData == null)
+                        continue;
+
+                    // Everything in inputData has been shadowed
+                    if (!inputData.complexDeletion().isLive())
+                        diffListener.onComplexDeletion(i, clustering, column, null, inputData.complexDeletion());
+                    for (Cell inputCell : inputData)
+                        diffListener.onCell(i, clustering, null, inputCell);
                 }
-
-                public ColumnData next(ColumnDefinition column)
+                else if (inputData == null)
                 {
-                    return null;
+                    // Everything in inputData is new
+                    if (!mergedData.complexDeletion().isLive())
+                        diffListener.onComplexDeletion(i, clustering, column, mergedData.complexDeletion(), null);
+                    for (Cell mergedCell : mergedData)
+                        diffListener.onCell(i, clustering, mergedCell, null);
                 }
-            };
-        }
-
-        public Kind kind()
-        {
-            return Unfiltered.Kind.ROW;
-        }
-
-        public Row takeAlias()
-        {
-            return this;
+                else
+                {
+                    PeekingIterator<Cell> mergedCells = Iterators.peekingIterator(mergedData.iterator());
+                    PeekingIterator<Cell> inputCells = Iterators.peekingIterator(inputData.iterator());
+                    while (mergedCells.hasNext() && inputCells.hasNext())
+                    {
+                        int cmp = column.cellPathComparator().compare(mergedCells.peek().path(), inputCells.peek().path());
+                        if (cmp == 0)
+                            diffListener.onCell(i, clustering, mergedCells.next(), inputCells.next());
+                        else if (cmp < 0)
+                            diffListener.onCell(i, clustering, mergedCells.next(), null);
+                        else // cmp > 0
+                            diffListener.onCell(i, clustering, null, inputCells.next());
+                    }
+                    while (mergedCells.hasNext())
+                        diffListener.onCell(i, clustering, mergedCells.next(), null);
+                    while (inputCells.hasNext())
+                        diffListener.onCell(i, clustering, null, inputCells.next());
+                }
+            }
         }
-    };
-
-    public interface SimpleMergeListener
-    {
-        public void onAdded(Cell newCell);
-        public void onRemoved(Cell removedCell);
-        public void onUpdated(Cell existingCell, Cell updatedCell);
     }
 
-    public static void writeClustering(Clustering clustering, Row.Writer writer)
+    public static Row merge(Row row1, Row row2, int nowInSec)
     {
-        for (int i = 0; i < clustering.size(); i++)
-            writer.writeClusteringValue(clustering.get(i));
+        Columns mergedColumns = row1.columns().mergeTo(row2.columns());
+        Row.Builder builder = ArrayBackedRow.sortedBuilder(mergedColumns);
+        merge(row1, row2, mergedColumns, builder, nowInSec, SecondaryIndexManager.nullUpdater);
+        return builder.build();
     }
 
-    public static void merge(Row row1, Row row2, Columns mergedColumns, Row.Writer writer, int nowInSec)
+    public static void merge(Row row1, Row row2, Columns mergedColumns, Row.Builder builder, int nowInSec)
     {
-        merge(row1, row2, mergedColumns, writer, nowInSec, SecondaryIndexManager.nullUpdater);
+        merge(row1, row2, mergedColumns, builder, nowInSec, SecondaryIndexManager.nullUpdater);
     }
 
     // Merge rows in memtable
@@ -146,26 +232,26 @@ public abstract class Rows
     public static long merge(Row existing,
                              Row update,
                              Columns mergedColumns,
-                             Row.Writer writer,
+                             Row.Builder builder,
                              int nowInSec,
                              SecondaryIndexManager.Updater indexUpdater)
     {
         Clustering clustering = existing.clustering();
-        writeClustering(clustering, writer);
+        builder.newRow(clustering);
 
         LivenessInfo existingInfo = existing.primaryKeyLivenessInfo();
         LivenessInfo updateInfo = update.primaryKeyLivenessInfo();
-        LivenessInfo mergedInfo = existingInfo.mergeWith(updateInfo);
+        LivenessInfo mergedInfo = existingInfo.supersedes(updateInfo) ? existingInfo : updateInfo;
 
         long timeDelta = Math.abs(existingInfo.timestamp() - mergedInfo.timestamp());
 
         DeletionTime deletion = existing.deletion().supersedes(update.deletion()) ? existing.deletion() : update.deletion();
 
         if (deletion.deletes(mergedInfo))
-            mergedInfo = LivenessInfo.NONE;
+            mergedInfo = LivenessInfo.EMPTY;
 
-        writer.writePartitionKeyLivenessInfo(mergedInfo);
-        writer.writeRowDeletion(deletion);
+        builder.addPrimaryKeyLivenessInfo(mergedInfo);
+        builder.addRowDeletion(deletion);
 
         indexUpdater.maybeIndex(clustering, mergedInfo.timestamp(), mergedInfo.ttl(), deletion);
 
@@ -178,7 +264,7 @@ public abstract class Rows
                                                             existingCell,
                                                             updateCell,
                                                             deletion,
-                                                            writer,
+                                                            builder,
                                                             nowInSec,
                                                             indexUpdater));
         }
@@ -186,20 +272,22 @@ public abstract class Rows
         for (int i = 0; i < mergedColumns.complexColumnCount(); i++)
         {
             ColumnDefinition c = mergedColumns.getComplex(i);
-            DeletionTime existingDt = existing.getDeletion(c);
-            DeletionTime updateDt = update.getDeletion(c);
+            ComplexColumnData existingData = existing.getComplexColumnData(c);
+            ComplexColumnData updateData = update.getComplexColumnData(c);
+
+            DeletionTime existingDt = existingData == null ? DeletionTime.LIVE : existingData.complexDeletion();
+            DeletionTime updateDt = updateData == null ? DeletionTime.LIVE : updateData.complexDeletion();
             DeletionTime maxDt = existingDt.supersedes(updateDt) ? existingDt : updateDt;
             if (maxDt.supersedes(deletion))
-                writer.writeComplexDeletion(c, maxDt);
+                builder.addComplexDeletion(c, maxDt);
             else
                 maxDt = deletion;
 
-            Iterator<Cell> existingCells = existing.getCells(c);
-            Iterator<Cell> updateCells = update.getCells(c);
-            timeDelta = Math.min(timeDelta, Cells.reconcileComplex(clustering, c, existingCells, updateCells, maxDt, writer, nowInSec, indexUpdater));
+            Iterator<Cell> existingCells = existingData == null ? null : existingData.iterator();
+            Iterator<Cell> updateCells = updateData == null ? null : updateData.iterator();
+            timeDelta = Math.min(timeDelta, Cells.reconcileComplex(clustering, c, existingCells, updateCells, maxDt, builder, nowInSec, indexUpdater));
         }
 
-        writer.endOfRow();
         return timeDelta;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/SerializationHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/SerializationHelper.java b/src/java/org/apache/cassandra/db/rows/SerializationHelper.java
index 56b993c..6b4bc2e 100644
--- a/src/java/org/apache/cassandra/db/rows/SerializationHelper.java
+++ b/src/java/org/apache/cassandra/db/rows/SerializationHelper.java
@@ -18,12 +18,13 @@
 package org.apache.cassandra.db.rows;
 
 import java.nio.ByteBuffer;
+import java.util.*;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.filter.ColumnFilter;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class SerializationHelper
 {
@@ -38,100 +39,90 @@ public class SerializationHelper
      *      when we must ensure that deserializing and reserializing the
      *      result yield the exact same bytes. Streaming uses this.
      */
-    public static enum Flag
+    public enum Flag
     {
-        LOCAL, FROM_REMOTE, PRESERVE_SIZE;
+        LOCAL, FROM_REMOTE, PRESERVE_SIZE
     }
 
     private final Flag flag;
     public final int version;
 
-    private final ReusableLivenessInfo livenessInfo = new ReusableLivenessInfo();
-
-    // The currently read row liveness infos (timestamp, ttl and localDeletionTime).
-    private long rowTimestamp;
-    private int rowTTL;
-    private int rowLocalDeletionTime;
-
     private final ColumnFilter columnsToFetch;
     private ColumnFilter.Tester tester;
 
-    public SerializationHelper(int version, Flag flag, ColumnFilter columnsToFetch)
+    private final Map<ByteBuffer, CFMetaData.DroppedColumn> droppedColumns;
+    private CFMetaData.DroppedColumn currentDroppedComplex;
+
+
+    public SerializationHelper(CFMetaData metadata, int version, Flag flag, ColumnFilter columnsToFetch)
     {
         this.flag = flag;
         this.version = version;
         this.columnsToFetch = columnsToFetch;
+        this.droppedColumns = metadata.getDroppedColumns();
     }
 
-    public SerializationHelper(int version, Flag flag)
+    public SerializationHelper(CFMetaData metadata, int version, Flag flag)
     {
-        this(version, flag, null);
+        this(metadata, version, flag, null);
     }
 
-    public void writePartitionKeyLivenessInfo(Row.Writer writer, long timestamp, int ttl, int localDeletionTime)
+    public Columns fetchedStaticColumns(SerializationHeader header)
     {
-        livenessInfo.setTo(timestamp, ttl, localDeletionTime);
-        writer.writePartitionKeyLivenessInfo(livenessInfo);
-
-        rowTimestamp = timestamp;
-        rowTTL = ttl;
-        rowLocalDeletionTime = localDeletionTime;
+        return columnsToFetch == null ? header.columns().statics : columnsToFetch.fetchedColumns().statics;
     }
 
-    public long getRowTimestamp()
+    public Columns fetchedRegularColumns(SerializationHeader header)
     {
-        return rowTimestamp;
+        return columnsToFetch == null ? header.columns().regulars : columnsToFetch.fetchedColumns().regulars;
     }
 
-    public int getRowTTL()
+    public boolean includes(ColumnDefinition column)
     {
-        return rowTTL;
+        return columnsToFetch == null || columnsToFetch.includes(column);
     }
 
-    public int getRowLocalDeletionTime()
+    public boolean includes(CellPath path)
     {
-        return rowLocalDeletionTime;
+        return path == null || tester == null || tester.includes(path);
     }
 
-    public boolean includes(ColumnDefinition column)
+    public boolean canSkipValue(ColumnDefinition column)
     {
-        return columnsToFetch == null || columnsToFetch.includes(column);
+        return columnsToFetch != null && columnsToFetch.canSkipValue(column);
     }
 
-    public boolean canSkipValue(ColumnDefinition column)
+    public boolean canSkipValue(CellPath path)
     {
-        return columnsToFetch != null && columnsToFetch.canSkipValue(column);
+        return path != null && tester != null && tester.canSkipValue(path);
     }
 
     public void startOfComplexColumn(ColumnDefinition column)
     {
         this.tester = columnsToFetch == null ? null : columnsToFetch.newTester(column);
+        this.currentDroppedComplex = droppedColumns.get(column.name.bytes);
     }
 
-    public void endOfComplexColumn(ColumnDefinition column)
+    public void endOfComplexColumn()
     {
         this.tester = null;
     }
 
-    public void writeCell(Row.Writer writer,
-                          ColumnDefinition column,
-                          boolean isCounter,
-                          ByteBuffer value,
-                          long timestamp,
-                          int localDelTime,
-                          int ttl,
-                          CellPath path)
+    public boolean isDropped(Cell cell, boolean isComplex)
     {
-        livenessInfo.setTo(timestamp, ttl, localDelTime);
+        CFMetaData.DroppedColumn dropped = isComplex ? currentDroppedComplex : droppedColumns.get(cell.column().name.bytes);
+        return dropped != null && cell.timestamp() <= dropped.droppedTime;
+    }
 
-        if (isCounter && ((flag == Flag.FROM_REMOTE || (flag == Flag.LOCAL && CounterContext.instance().shouldClearLocal(value)))))
-            value = CounterContext.instance().clearAllLocal(value);
+    public boolean isDroppedComplexDeletion(DeletionTime complexDeletion)
+    {
+        return currentDroppedComplex != null && complexDeletion.markedForDeleteAt() <= currentDroppedComplex.droppedTime;
+    }
 
-        if (!column.isComplex() || tester == null || tester.includes(path))
-        {
-            if (tester != null && tester.canSkipValue(path))
-                value = ByteBufferUtil.EMPTY_BYTE_BUFFER;
-            writer.writeCell(column, isCounter, value, livenessInfo, path);
-        }
+    public ByteBuffer maybeClearCounterValue(ByteBuffer value)
+    {
+        return flag == Flag.FROM_REMOTE || (flag == Flag.LOCAL && CounterContext.instance().shouldClearLocal(value))
+             ? CounterContext.instance().clearAllLocal(value)
+             : value;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/SimpleRowDataBlock.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/SimpleRowDataBlock.java b/src/java/org/apache/cassandra/db/rows/SimpleRowDataBlock.java
deleted file mode 100644
index 08f37fd..0000000
--- a/src/java/org/apache/cassandra/db/rows/SimpleRowDataBlock.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * 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.rows;
-
-import java.nio.ByteBuffer;
-
-import com.google.common.collect.UnmodifiableIterator;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.utils.ObjectSizes;
-
-/**
- * Holds cells data for the simple columns of one or more rows.
- * <p>
- * In practice, a {@code SimpleRowDataBlock} contains a single {@code CellData} "array" and
- * the (simple) columns for which the {@code SimplerowDataBlock} has data for. The cell for
- * a row i and a column c is stored in the {@code CellData} at index 'i * index(c)'.
- * <p>
- * This  does mean that we store cells in a "dense" way: if column doesn't have a cell for a
- * given row, the correspond index in the cell data array will simple have a {@code null} value.
- * We might want to switch to a more sparse encoding in the future but we keep it simple for
- * now (having a sparse encoding make things a tad more complex because we need to be able to
- * swap the cells for 2 given rows as seen in ComplexRowDataBlock).
- */
-public class SimpleRowDataBlock
-{
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new SimpleRowDataBlock(Columns.NONE, 0, false));
-
-    final Columns columns;
-    final CellData data;
-
-    public SimpleRowDataBlock(Columns columns, int rows, boolean isCounter)
-    {
-        this.columns = columns;
-        this.data = new CellData(rows * columns.simpleColumnCount(), isCounter);
-    }
-
-    public Columns columns()
-    {
-        return columns;
-    }
-
-    // Swap row i and j
-    public void swap(int i, int j)
-    {
-        int s = columns.simpleColumnCount();
-        for (int k = 0; k < s; k++)
-            data.swapCell(i * s + k, j * s + k);
-    }
-
-    // Merge row i into j
-    public void merge(int i, int j, int nowInSec)
-    {
-        int s = columns.simpleColumnCount();
-        for (int k = 0; k < s; k++)
-            data.mergeCell(i * s + k, j * s + k, nowInSec);
-    }
-
-    // Move row i into j
-    public void move(int i, int j)
-    {
-        int s = columns.simpleColumnCount();
-        for (int k = 0; k < s; k++)
-            data.moveCell(i * s + k, j * s + k);
-    }
-
-    public long unsharedHeapSizeExcludingData()
-    {
-        return EMPTY_SIZE + data.unsharedHeapSizeExcludingData();
-    }
-
-    public int dataSize()
-    {
-        return data.dataSize();
-    }
-
-    public CellWriter cellWriter(boolean inOrderCells)
-    {
-        return new CellWriter(inOrderCells);
-    }
-
-    public static CellData.ReusableCell reusableCell()
-    {
-        return new CellData.ReusableCell();
-    }
-
-    public static ReusableIterator reusableIterator()
-    {
-        return new ReusableIterator();
-    }
-
-    public void clear()
-    {
-        data.clear();
-    }
-
-    static class ReusableIterator extends UnmodifiableIterator<Cell>
-    {
-        private SimpleRowDataBlock dataBlock;
-        private final CellData.ReusableCell cell = new CellData.ReusableCell();
-
-        private int base;
-        private int column;
-
-        private ReusableIterator()
-        {
-        }
-
-        public ReusableIterator setTo(SimpleRowDataBlock dataBlock, int row)
-        {
-            this.dataBlock = dataBlock;
-            this.base = dataBlock == null ? -1 : row * dataBlock.columns.simpleColumnCount();
-            this.column = 0;
-            return this;
-        }
-
-        public boolean hasNext()
-        {
-            if (dataBlock == null)
-                return false;
-
-            int columnCount = dataBlock.columns.simpleColumnCount();
-            // iterate over column until we find one with data
-            while (column < columnCount && !dataBlock.data.hasCell(base + column))
-                ++column;
-
-            return column < columnCount;
-        }
-
-        public Cell next()
-        {
-            cell.setTo(dataBlock.data, dataBlock.columns.getSimple(column), base + column);
-            ++column;
-            return cell;
-        }
-    }
-
-    public class CellWriter
-    {
-        private final boolean inOrderCells;
-
-        private int base;
-        private int lastColumnIdx;
-
-        public CellWriter(boolean inOrderCells)
-        {
-            this.inOrderCells = inOrderCells;
-        }
-
-        public void addCell(ColumnDefinition column, ByteBuffer value, LivenessInfo info)
-        {
-            int fromIdx = inOrderCells ? lastColumnIdx : 0;
-            lastColumnIdx = columns.simpleIdx(column, fromIdx);
-            assert lastColumnIdx >= 0 : "Cannot find column " + column.name + " in " + columns + " from " + fromIdx;
-            int idx = base + lastColumnIdx;
-            data.setCell(idx, value, info);
-        }
-
-        public void reset()
-        {
-            base = 0;
-            lastColumnIdx = 0;
-            data.clear();
-        }
-
-        public void endOfRow()
-        {
-            base += columns.simpleColumnCount();
-            lastColumnIdx = 0;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/StaticRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/StaticRow.java b/src/java/org/apache/cassandra/db/rows/StaticRow.java
deleted file mode 100644
index 2ad9fb4..0000000
--- a/src/java/org/apache/cassandra/db/rows/StaticRow.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/*
- * 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.rows;
-
-import java.nio.ByteBuffer;
-import java.util.Iterator;
-
-import org.apache.cassandra.db.*;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.utils.SearchIterator;
-
-public class StaticRow extends AbstractRow
-{
-    private final DeletionTime deletion;
-    private final RowDataBlock data;
-
-    private StaticRow(DeletionTime deletion, RowDataBlock data)
-    {
-        this.deletion = deletion.takeAlias();
-        this.data = data;
-    }
-
-    public Columns columns()
-    {
-        return data.columns();
-    }
-
-    public Cell getCell(ColumnDefinition c)
-    {
-        assert !c.isComplex();
-        if (data.simpleData == null)
-            return null;
-
-        int idx = columns().simpleIdx(c, 0);
-        if (idx < 0)
-            return null;
-
-        return SimpleRowDataBlock.reusableCell().setTo(data.simpleData.data, c, idx);
-    }
-
-    public Cell getCell(ColumnDefinition c, CellPath path)
-    {
-        assert c.isComplex();
-
-        ComplexRowDataBlock dataBlock = data.complexData;
-        if (dataBlock == null)
-            return null;
-
-        int idx = dataBlock.cellIdx(0, c, path);
-        if (idx < 0)
-            return null;
-
-        return SimpleRowDataBlock.reusableCell().setTo(dataBlock.cellData(0), c, idx);
-    }
-
-    public Iterator<Cell> getCells(ColumnDefinition c)
-    {
-        assert c.isComplex();
-        return ComplexRowDataBlock.reusableComplexCells().setTo(data.complexData, 0, c);
-    }
-
-    public boolean hasComplexDeletion()
-    {
-        return data.hasComplexDeletion(0);
-    }
-
-    public DeletionTime getDeletion(ColumnDefinition c)
-    {
-        assert c.isComplex();
-        if (data.complexData == null)
-            return DeletionTime.LIVE;
-
-        int idx = data.complexData.complexDeletionIdx(0, c);
-        return idx < 0
-             ? DeletionTime.LIVE
-             : ComplexRowDataBlock.complexDeletionCursor().setTo(data.complexData.complexDelTimes, idx);
-    }
-
-    public Iterator<Cell> iterator()
-    {
-        return RowDataBlock.reusableIterator().setTo(data, 0);
-    }
-
-    public SearchIterator<ColumnDefinition, ColumnData> searchIterator()
-    {
-        return new SearchIterator<ColumnDefinition, ColumnData>()
-        {
-            private int simpleIdx = 0;
-
-            public boolean hasNext()
-            {
-                // TODO: we can do better, but we expect users to no rely on this anyway
-                return true;
-            }
-
-            public ColumnData next(ColumnDefinition column)
-            {
-                if (column.isComplex())
-                {
-                    // TODO: this is sub-optimal
-
-                    Iterator<Cell> cells = getCells(column);
-                    return cells == null ? null : new ColumnData(column, null, cells, getDeletion(column));
-                }
-                else
-                {
-                    simpleIdx = columns().simpleIdx(column, simpleIdx);
-                    assert simpleIdx >= 0;
-
-                    Cell cell = SimpleRowDataBlock.reusableCell().setTo(data.simpleData.data, column, simpleIdx);
-                    ++simpleIdx;
-                    return cell == null ? null : new ColumnData(column, cell, null, null);
-                }
-            }
-        };
-    }
-
-    public Row takeAlias()
-    {
-        return this;
-    }
-
-    public Clustering clustering()
-    {
-        return Clustering.STATIC_CLUSTERING;
-    }
-
-    public LivenessInfo primaryKeyLivenessInfo()
-    {
-        return LivenessInfo.NONE;
-    }
-
-    public DeletionTime deletion()
-    {
-        return deletion;
-    }
-
-    public static Builder builder(Columns columns, boolean inOrderCells, boolean isCounter)
-    {
-        return new Builder(columns, inOrderCells, isCounter);
-    }
-
-    public static class Builder extends RowDataBlock.Writer
-    {
-        private final RowDataBlock data;
-        private DeletionTime deletion = DeletionTime.LIVE;
-
-        public Builder(Columns columns, boolean inOrderCells, boolean isCounter)
-        {
-            super(inOrderCells);
-            this.data = new RowDataBlock(columns, 1, false, isCounter);
-            updateWriter(data);
-        }
-
-        public void writeClusteringValue(ByteBuffer buffer)
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        public void writePartitionKeyLivenessInfo(LivenessInfo info)
-        {
-            // Static rows are special and don't really have an existence unless they have live cells,
-            // so we shouldn't have any partition key liveness info.
-            assert info.equals(LivenessInfo.NONE);
-        }
-
-        public void writeRowDeletion(DeletionTime deletion)
-        {
-            this.deletion = deletion;
-        }
-
-        public StaticRow build()
-        {
-            return new StaticRow(deletion, data);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/TombstoneFilteringRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/TombstoneFilteringRow.java b/src/java/org/apache/cassandra/db/rows/TombstoneFilteringRow.java
deleted file mode 100644
index a6167ea..0000000
--- a/src/java/org/apache/cassandra/db/rows/TombstoneFilteringRow.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.rows;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.*;
-
-public class TombstoneFilteringRow extends FilteringRow
-{
-    private final int nowInSec;
-
-    public TombstoneFilteringRow(int nowInSec)
-    {
-        this.nowInSec = nowInSec;
-    }
-
-    @Override
-    protected boolean include (LivenessInfo info)
-    {
-        return info.isLive(nowInSec);
-    }
-
-    @Override
-    protected boolean include(DeletionTime dt)
-    {
-        return false;
-    }
-
-    @Override
-    protected boolean include(Cell cell)
-    {
-        return cell.isLive(nowInSec);
-    }
-
-    @Override
-    protected boolean include(ColumnDefinition c, DeletionTime dt)
-    {
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/Unfiltered.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Unfiltered.java b/src/java/org/apache/cassandra/db/rows/Unfiltered.java
index b1692e3..ba03741 100644
--- a/src/java/org/apache/cassandra/db/rows/Unfiltered.java
+++ b/src/java/org/apache/cassandra/db/rows/Unfiltered.java
@@ -57,4 +57,14 @@ public interface Unfiltered extends Clusterable
 
     public String toString(CFMetaData metadata);
     public String toString(CFMetaData metadata, boolean fullDetails);
+
+    default boolean isRow()
+    {
+        return kind() == Kind.ROW;
+    }
+
+    default boolean isRangeTombstoneMarker()
+    {
+        return kind() == Kind.RANGE_TOMBSTONE_MARKER;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java
index 8abd228..129ed50 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.rows;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.io.IOError;
 
@@ -30,7 +29,6 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
 
 /**
  * Serialize/Deserialize an unfiltered row iterator.
@@ -38,7 +36,7 @@ import org.apache.cassandra.utils.FBUtilities;
  * The serialization is composed of a header, follows by the rows and range tombstones of the iterator serialized
  * until we read the end of the partition (see UnfilteredSerializer for details). The header itself
  * is:
- *     <cfid><key><flags><s_header>[<partition_deletion>][<static_row>]
+ *     <cfid><key><flags><s_header>[<partition_deletion>][<static_row>][<row_estimate>]
  * where:
  *     <cfid> is the table cfid.
  *     <key> is the partition key.
@@ -49,23 +47,17 @@ import org.apache.cassandra.utils.FBUtilities;
  *         - has partition deletion: whether or not there is a <partition_deletion> following
  *         - has static row: whether or not there is a <static_row> following
  *         - has row estimate: whether or not there is a <row_estimate> following
- *     <s_header> is the SerializationHeader. More precisely it's
- *           <min_timetamp><min_localDelTime><min_ttl>[<static_columns>]<columns>
- *         where:
- *           - <min_timestamp> is the base timestamp used for delta-encoding timestamps
- *           - <min_localDelTime> is the base localDeletionTime used for delta-encoding local deletion times
- *           - <min_ttl> is the base localDeletionTime used for delta-encoding ttls
- *           - <static_columns> is the static columns if a static row is present. It's
- *             the number of columns as an unsigned short, followed by the column names.
- *           - <columns> is the columns of the rows of the iterator. It's serialized as <static_columns>.
+ *     <s_header> is the {@code SerializationHeader}. It contains in particular the columns contains in the serialized
+ *         iterator as well as other information necessary to decoding the serialized rows
+ *         (see {@code SerializationHeader.Serializer for details}).
  *     <partition_deletion> is the deletion time for the partition (delta-encoded)
  *     <static_row> is the static row for this partition as serialized by UnfilteredSerializer.
- *     <row_estimate> is the (potentially estimated) number of rows serialized. This is only use for
- *         the purpose of some sizing on the receiving end and should not be relied upon too strongly.
+ *     <row_estimate> is the (potentially estimated) number of rows serialized. This is only used for
+ *         the purpose of sizing on the receiving end and should not be relied upon too strongly.
  *
- * !!! Please note that the serialized value depends on the schema and as such should not be used as is if
- *     it might be deserialized after the schema as changed !!!
- * TODO: we should add a flag to include the relevant metadata in the header for commit log etc.....
+ * Please note that the format described above is the on-wire format. On-disk, the format is basically the
+ * same, but the header is written once per sstable, not once per-partition. Further, the actual row and
+ * range tombstones are not written using this class, but rather by {@link ColumnIndex}.
  */
 public class UnfilteredRowIteratorSerializer
 {
@@ -79,11 +71,13 @@ public class UnfilteredRowIteratorSerializer
 
     public static final UnfilteredRowIteratorSerializer serializer = new UnfilteredRowIteratorSerializer();
 
+    // Should only be used for the on-wire format.
     public void serialize(UnfilteredRowIterator iterator, DataOutputPlus out, int version) throws IOException
     {
         serialize(iterator, out, version, -1);
     }
 
+    // Should only be used for the on-wire format.
     public void serialize(UnfilteredRowIterator iterator, DataOutputPlus out, int version, int rowEstimate) throws IOException
     {
         SerializationHeader header = new SerializationHeader(iterator.metadata(),
@@ -92,6 +86,7 @@ public class UnfilteredRowIteratorSerializer
         serialize(iterator, out, header, version, rowEstimate);
     }
 
+    // Should only be used for the on-wire format.
     public void serialize(UnfilteredRowIterator iterator, DataOutputPlus out, SerializationHeader header, int version, int rowEstimate) throws IOException
     {
         CFMetaData.serializer.serialize(iterator.metadata(), out, version);
@@ -129,7 +124,7 @@ public class UnfilteredRowIteratorSerializer
             UnfilteredSerializer.serializer.serialize(staticRow, header, out, version);
 
         if (rowEstimate >= 0)
-            out.writeInt(rowEstimate);
+            out.writeVInt(rowEstimate);
 
         while (iterator.hasNext())
             UnfilteredSerializer.serializer.serialize(iterator.next(), header, out, version);
@@ -137,7 +132,7 @@ public class UnfilteredRowIteratorSerializer
     }
 
     // Please note that this consume the iterator, and as such should not be called unless we have a simple way to
-    // recreate an iterator for both serialize and serializedSize, which is mostly only PartitionUpdate
+    // recreate an iterator for both serialize and serializedSize, which is mostly only PartitionUpdate/ArrayBackedCachedPartition.
     public long serializedSize(UnfilteredRowIterator iterator, int version, int rowEstimate)
     {
         SerializationHeader header = new SerializationHeader(iterator.metadata(),
@@ -166,7 +161,7 @@ public class UnfilteredRowIteratorSerializer
             size += UnfilteredSerializer.serializer.serializedSize(staticRow, header, version);
 
         if (rowEstimate >= 0)
-            size += TypeSizes.sizeof(rowEstimate);
+            size += TypeSizes.sizeofVInt(rowEstimate);
 
         while (iterator.hasNext())
             size += UnfilteredSerializer.serializer.serializedSize(iterator.next(), header, version);
@@ -197,41 +192,29 @@ public class UnfilteredRowIteratorSerializer
 
         Row staticRow = Rows.EMPTY_STATIC_ROW;
         if (hasStatic)
-            staticRow = UnfilteredSerializer.serializer.deserializeStaticRow(in, header, new SerializationHelper(version, flag));
+            staticRow = UnfilteredSerializer.serializer.deserializeStaticRow(in, header, new SerializationHelper(metadata, version, flag));
 
-        int rowEstimate = hasRowEstimate ? in.readInt() : -1;
+        int rowEstimate = hasRowEstimate ? (int)in.readVInt() : -1;
         return new Header(header, metadata, key, isReversed, false, partitionDeletion, staticRow, rowEstimate);
     }
 
-    public void deserialize(DataInput in, SerializationHelper helper, SerializationHeader header, Row.Writer rowWriter, RangeTombstoneMarker.Writer markerWriter) throws IOException
+    public UnfilteredRowIterator deserialize(DataInputPlus in, int version, SerializationHelper.Flag flag, Header header) throws IOException
     {
-        while (UnfilteredSerializer.serializer.deserialize(in, header, helper, rowWriter, markerWriter) != null);
-    }
-
-    public UnfilteredRowIterator deserialize(final DataInputPlus in, int version, SerializationHelper.Flag flag) throws IOException
-    {
-        final Header h = deserializeHeader(in, version, flag);
-
-        if (h.isEmpty)
-            return UnfilteredRowIterators.emptyIterator(h.metadata, h.key, h.isReversed);
-
-        final int clusteringSize = h.metadata.clusteringColumns().size();
-        final SerializationHelper helper = new SerializationHelper(version, flag);
+        if (header.isEmpty)
+            return UnfilteredRowIterators.emptyIterator(header.metadata, header.key, header.isReversed);
 
-        return new AbstractUnfilteredRowIterator(h.metadata, h.key, h.partitionDeletion, h.sHeader.columns(), h.staticRow, h.isReversed, h.sHeader.stats())
+        final SerializationHelper helper = new SerializationHelper(header.metadata, version, flag);
+        final SerializationHeader sHeader = header.sHeader;
+        return new AbstractUnfilteredRowIterator(header.metadata, header.key, header.partitionDeletion, sHeader.columns(), header.staticRow, header.isReversed, sHeader.stats())
         {
-            private final ReusableRow row = new ReusableRow(clusteringSize, h.sHeader.columns().regulars, true, h.metadata.isCounter());
-            private final RangeTombstoneMarker.Builder markerBuilder = new RangeTombstoneMarker.Builder(clusteringSize);
+            private final Row.Builder builder = ArrayBackedRow.sortedBuilder(sHeader.columns().regulars);
 
             protected Unfiltered computeNext()
             {
                 try
                 {
-                    Unfiltered.Kind kind = UnfilteredSerializer.serializer.deserialize(in, h.sHeader, helper, row.writer(), markerBuilder.reset());
-                    if (kind == null)
-                        return endOfData();
-
-                    return kind == Unfiltered.Kind.ROW ? row : markerBuilder.build();
+                    Unfiltered unfiltered = UnfilteredSerializer.serializer.deserialize(in, sHeader, helper, builder);
+                    return unfiltered == null ? endOfData() : unfiltered;
                 }
                 catch (IOException e)
                 {
@@ -241,30 +224,34 @@ public class UnfilteredRowIteratorSerializer
         };
     }
 
+    public UnfilteredRowIterator deserialize(DataInputPlus in, int version, SerializationHelper.Flag flag) throws IOException
+    {
+        return deserialize(in, version, flag,  deserializeHeader(in, version, flag));
+    }
+
     public static void writeDelTime(DeletionTime dt, SerializationHeader header, DataOutputPlus out) throws IOException
     {
-        out.writeLong(header.encodeTimestamp(dt.markedForDeleteAt()));
-        out.writeInt(header.encodeDeletionTime(dt.localDeletionTime()));
+        out.writeVInt(header.encodeTimestamp(dt.markedForDeleteAt()));
+        out.writeVInt(header.encodeDeletionTime(dt.localDeletionTime()));
     }
 
     public static long delTimeSerializedSize(DeletionTime dt, SerializationHeader header)
     {
-        return TypeSizes.sizeof(header.encodeTimestamp(dt.markedForDeleteAt()))
-             + TypeSizes.sizeof(header.encodeDeletionTime(dt.localDeletionTime()));
+        return TypeSizes.sizeofVInt(header.encodeTimestamp(dt.markedForDeleteAt()))
+             + TypeSizes.sizeofVInt(header.encodeDeletionTime(dt.localDeletionTime()));
     }
 
-    public static DeletionTime readDelTime(DataInput in, SerializationHeader header) throws IOException
+    public static DeletionTime readDelTime(DataInputPlus in, SerializationHeader header) throws IOException
     {
-        long markedAt = header.decodeTimestamp(in.readLong());
-        int localDelTime = header.decodeDeletionTime(in.readInt());
-        return new SimpleDeletionTime(markedAt, localDelTime);
+        long markedAt = header.decodeTimestamp(in.readVInt());
+        int localDelTime = header.decodeDeletionTime((int)in.readVInt());
+        return new DeletionTime(markedAt, localDelTime);
     }
 
-    public static void skipDelTime(DataInput in, SerializationHeader header) throws IOException
+    public static void skipDelTime(DataInputPlus in, SerializationHeader header) throws IOException
     {
-        // Note that since we might use VINT, we shouldn't assume the size of a long or an int
-        in.readLong();
-        in.readInt();
+        in.readVInt();
+        in.readVInt();
     }
 
     public static class Header

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
index 2c71cf3..6b6ec67 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db.rows;
 
-import java.nio.ByteBuffer;
 import java.util.*;
 import java.security.MessageDigest;
 
@@ -26,13 +25,10 @@ import org.slf4j.LoggerFactory;
 import com.google.common.collect.AbstractIterator;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.IMergeIterator;
 import org.apache.cassandra.utils.MergeIterator;
@@ -49,13 +45,9 @@ public abstract class UnfilteredRowIterators
 
     public interface MergeListener
     {
-        public void onMergePartitionLevelDeletion(DeletionTime mergedDeletion, DeletionTime[] versions);
-
-        public void onMergingRows(Clustering clustering, LivenessInfo mergedInfo, DeletionTime mergedDeletion, Row[] versions);
-        public void onMergedComplexDeletion(ColumnDefinition c, DeletionTime mergedComplexDeletion, DeletionTime[] versions);
-        public void onMergedCells(Cell mergedCell, Cell[] versions);
-        public void onRowDone();
+        public void onMergedPartitionLevelDeletion(DeletionTime mergedDeletion, DeletionTime[] versions);
 
+        public void onMergedRows(Row merged, Columns columns, Row[] versions);
         public void onMergedRangeTombstoneMarkers(RangeTombstoneMarker merged, RangeTombstoneMarker[] versions);
 
         public void close();
@@ -87,14 +79,13 @@ public abstract class UnfilteredRowIterators
     }
 
     /**
-     * Returns an iterator that is the result of merging other iterators, and using
+     * Returns an iterator that is the result of merging other iterators, and (optionally) using
      * specific MergeListener.
      *
      * Note that this method assumes that there is at least 2 iterators to merge.
      */
     public static UnfilteredRowIterator merge(List<UnfilteredRowIterator> iterators, int nowInSec, MergeListener mergeListener)
     {
-        assert mergeListener != null;
         return UnfilteredRowMergeIterator.create(iterators, nowInSec, mergeListener);
     }
 
@@ -175,10 +166,7 @@ public abstract class UnfilteredRowIterators
         while (iterator.hasNext())
         {
             Unfiltered unfiltered = iterator.next();
-            if (unfiltered.kind() == Unfiltered.Kind.ROW)
-                ((Row) unfiltered).digest(digest);
-            else
-                ((RangeTombstoneMarker) unfiltered).digest(digest);
+            unfiltered.digest(digest);
         }
     }
 
@@ -198,12 +186,12 @@ public abstract class UnfilteredRowIterators
             && iter1.staticRow().equals(iter2.staticRow());
 
         return new AbstractUnfilteredRowIterator(iter1.metadata(),
-                                        iter1.partitionKey(),
-                                        iter1.partitionLevelDeletion(),
-                                        iter1.columns(),
-                                        iter1.staticRow(),
-                                        iter1.isReverseOrder(),
-                                        iter1.stats())
+                                                 iter1.partitionKey(),
+                                                 iter1.partitionLevelDeletion(),
+                                                 iter1.columns(),
+                                                 iter1.staticRow(),
+                                                 iter1.isReverseOrder(),
+                                                 iter1.stats())
         {
             protected Unfiltered computeNext()
             {
@@ -230,155 +218,35 @@ public abstract class UnfilteredRowIterators
 
     public static UnfilteredRowIterator cloningIterator(UnfilteredRowIterator iterator, final AbstractAllocator allocator)
     {
-        return new WrappingUnfilteredRowIterator(iterator)
+        return new AlteringUnfilteredRowIterator(iterator)
         {
-            private final CloningRow cloningRow = new CloningRow();
-            private final RangeTombstoneMarker.Builder markerBuilder = new RangeTombstoneMarker.Builder(iterator.metadata().comparator.size());
-
-            public Unfiltered next()
-            {
-                Unfiltered next = super.next();
-                return next.kind() == Unfiltered.Kind.ROW
-                     ? cloningRow.setTo((Row)next)
-                     : clone((RangeTombstoneMarker)next);
-            }
-
-            private RangeTombstoneMarker clone(RangeTombstoneMarker marker)
-            {
-                markerBuilder.reset();
+            private Row.Builder regularBuilder;
 
-                RangeTombstone.Bound bound = marker.clustering();
-                for (int i = 0; i < bound.size(); i++)
-                    markerBuilder.writeClusteringValue(allocator.clone(bound.get(i)));
-                markerBuilder.writeBoundKind(bound.kind());
-                if (marker.isBoundary())
-                {
-                    RangeTombstoneBoundaryMarker bm = (RangeTombstoneBoundaryMarker)marker;
-                    markerBuilder.writeBoundaryDeletion(bm.endDeletionTime(), bm.startDeletionTime());
-                }
-                else
-                {
-                    markerBuilder.writeBoundDeletion(((RangeTombstoneBoundMarker)marker).deletionTime());
-                }
-                markerBuilder.endOfMarker();
-                return markerBuilder.build();
-            }
-
-            class CloningRow extends WrappingRow
+            @Override
+            protected Row computeNextStatic(Row row)
             {
-                private final CloningClustering cloningClustering = new CloningClustering();
-                private final CloningCell cloningCell = new CloningCell();
-
-                protected Cell filterCell(Cell cell)
-                {
-                    return cloningCell.setTo(cell);
-                }
-
-                @Override
-                public Clustering clustering()
-                {
-                    return cloningClustering.setTo(super.clustering());
-                }
+                Row.Builder staticBuilder = allocator.cloningArrayBackedRowBuilder(columns().statics);
+                return Rows.copy(row, staticBuilder).build();
             }
 
-            class CloningClustering extends Clustering
+            @Override
+            protected Row computeNext(Row row)
             {
-                private Clustering wrapped;
-
-                public Clustering setTo(Clustering wrapped)
-                {
-                    this.wrapped = wrapped;
-                    return this;
-                }
-
-                public int size()
-                {
-                    return wrapped.size();
-                }
-
-                public ByteBuffer get(int i)
-                {
-                    ByteBuffer value = wrapped.get(i);
-                    return value == null ? null : allocator.clone(value);
-                }
+                if (regularBuilder == null)
+                    regularBuilder = allocator.cloningArrayBackedRowBuilder(columns().regulars);
 
-                public ByteBuffer[] getRawValues()
-                {
-                    throw new UnsupportedOperationException();
-                }
+                return Rows.copy(row, regularBuilder).build();
             }
 
-            class CloningCell extends AbstractCell
+            @Override
+            protected RangeTombstoneMarker computeNext(RangeTombstoneMarker marker)
             {
-                private Cell wrapped;
-
-                public Cell setTo(Cell wrapped)
-                {
-                    this.wrapped = wrapped;
-                    return this;
-                }
-
-                public ColumnDefinition column()
-                {
-                    return wrapped.column();
-                }
-
-                public boolean isCounterCell()
-                {
-                    return wrapped.isCounterCell();
-                }
-
-                public ByteBuffer value()
-                {
-                    return allocator.clone(wrapped.value());
-                }
-
-                public LivenessInfo livenessInfo()
-                {
-                    return wrapped.livenessInfo();
-                }
-
-                public CellPath path()
-                {
-                    CellPath path = wrapped.path();
-                    if (path == null)
-                        return null;
-
-                    assert path.size() == 1;
-                    return CellPath.create(allocator.clone(path.get(0)));
-                }
+                return marker.copy(allocator);
             }
         };
     }
 
     /**
-     * Turns the given iterator into an update.
-     *
-     * Warning: this method does not close the provided iterator, it is up to
-     * the caller to close it.
-     */
-    public static PartitionUpdate toUpdate(UnfilteredRowIterator iterator)
-    {
-        PartitionUpdate update = new PartitionUpdate(iterator.metadata(), iterator.partitionKey(), iterator.columns(), 1);
-
-        update.addPartitionDeletion(iterator.partitionLevelDeletion());
-
-        if (iterator.staticRow() != Rows.EMPTY_STATIC_ROW)
-            iterator.staticRow().copyTo(update.staticWriter());
-
-        while (iterator.hasNext())
-        {
-            Unfiltered unfiltered = iterator.next();
-            if (unfiltered.kind() == Unfiltered.Kind.ROW)
-                ((Row) unfiltered).copyTo(update.writer());
-            else
-                ((RangeTombstoneMarker) unfiltered).copyTo(update.markerWriter(iterator.isReverseOrder()));
-        }
-
-        return update;
-    }
-
-    /**
      * Validate that the data of the provided iterator is valid, that is that the values
      * it contains are valid for the type they represent, and more generally that the
      * infos stored are sensible.
@@ -393,15 +261,34 @@ public abstract class UnfilteredRowIterators
      */
     public static UnfilteredRowIterator withValidation(UnfilteredRowIterator iterator, final String filename)
     {
-        return new WrappingUnfilteredRowIterator(iterator)
+        return new AlteringUnfilteredRowIterator(iterator)
         {
-            public Unfiltered next()
+            @Override
+            protected Row computeNextStatic(Row row)
+            {
+                validate(row);
+                return row;
+            }
+
+            @Override
+            protected Row computeNext(Row row)
+            {
+                validate(row);
+                return row;
+            }
+
+            @Override
+            protected RangeTombstoneMarker computeNext(RangeTombstoneMarker marker)
+            {
+                validate(marker);
+                return marker;
+            }
+
+            private void validate(Unfiltered unfiltered)
             {
-                Unfiltered next = super.next();
                 try
                 {
-                    next.validateData(metadata());
-                    return next;
+                    unfiltered.validateData(iterator.metadata());
                 }
                 catch (MarshalException me)
                 {
@@ -412,56 +299,6 @@ public abstract class UnfilteredRowIterators
     }
 
     /**
-     * Convert all expired cells to equivalent tombstones.
-     * <p>
-     * Once a cell expires, it acts exactly as a tombstone and this until it is purged. But in particular that
-     * means we don't care about the value of an expired cell, and it is thus equivalent but more efficient to
-     * replace the expired cell by an equivalent tombstone (that has no value).
-     *
-     * @param iterator the iterator in which to conver expired cells.
-     * @param nowInSec the current time to use to decide if a cell is expired.
-     * @return an iterator that returns the same data than {@code iterator} but with all expired cells converted
-     * to equivalent tombstones.
-     */
-    public static UnfilteredRowIterator convertExpiredCellsToTombstones(UnfilteredRowIterator iterator, final int nowInSec)
-    {
-        return new FilteringRowIterator(iterator)
-        {
-            protected FilteringRow makeRowFilter()
-            {
-                return new FilteringRow()
-                {
-                    @Override
-                    protected Cell filterCell(Cell cell)
-                    {
-                        Cell filtered = super.filterCell(cell);
-                        if (filtered == null)
-                            return null;
-
-                        LivenessInfo info = filtered.livenessInfo();
-                        if (info.hasTTL() && !filtered.isLive(nowInSec))
-                        {
-                            // The column is now expired, we can safely return a simple tombstone. Note that as long as the expiring
-                            // column and the tombstone put together live longer than GC grace seconds, we'll fulfil our responsibility
-                            // to repair. See discussion at
-                            // http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html
-                            return Cells.create(filtered.column(),
-                                                filtered.isCounterCell(),
-                                                ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                SimpleLivenessInfo.forDeletion(info.timestamp(), info.localDeletionTime() - info.ttl()),
-                                                filtered.path());
-                        }
-                        else
-                        {
-                            return filtered;
-                        }
-                    }
-                };
-            }
-        };
-    }
-
-    /**
      * Wraps the provided iterator so it logs the returned atoms for debugging purposes.
      * <p>
      * Note that this is only meant for debugging as this can log a very large amount of
@@ -478,26 +315,28 @@ public abstract class UnfilteredRowIterators
                     iterator.isReverseOrder(),
                     iterator.partitionLevelDeletion().markedForDeleteAt());
 
-        return new WrappingUnfilteredRowIterator(iterator)
+        return new AlteringUnfilteredRowIterator(iterator)
         {
             @Override
-            public Row staticRow()
+            protected Row computeNextStatic(Row row)
             {
-                Row row = super.staticRow();
                 if (!row.isEmpty())
                     logger.info("[{}] {}", id, row.toString(metadata(), fullDetails));
                 return row;
             }
 
             @Override
-            public Unfiltered next()
+            protected Row computeNext(Row row)
             {
-                Unfiltered next = super.next();
-                if (next.kind() == Unfiltered.Kind.ROW)
-                    logger.info("[{}] {}", id, ((Row)next).toString(metadata(), fullDetails));
-                else
-                    logger.info("[{}] {}", id, ((RangeTombstoneMarker)next).toString(metadata()));
-                return next;
+                logger.info("[{}] {}", id, row.toString(metadata(), fullDetails));
+                return row;
+            }
+
+            @Override
+            protected RangeTombstoneMarker computeNext(RangeTombstoneMarker marker)
+            {
+                logger.info("[{}] {}", id, marker.toString(metadata()));
+                return marker;
             }
         };
     }
@@ -526,10 +365,10 @@ public abstract class UnfilteredRowIterators
                   reversed,
                   mergeStats(iterators));
 
-            this.listener = listener;
             this.mergeIterator = MergeIterator.get(iterators,
                                                    reversed ? metadata.comparator.reversed() : metadata.comparator,
-                                                   new MergeReducer(metadata, iterators.size(), reversed, nowInSec));
+                                                   new MergeReducer(iterators.size(), reversed, nowInSec, listener));
+            this.listener = listener;
         }
 
         private static UnfilteredRowMergeIterator create(List<UnfilteredRowIterator> iterators, int nowInSec, MergeListener listener)
@@ -591,7 +430,7 @@ public abstract class UnfilteredRowIterators
                     delTime = iterDeletion;
             }
             if (listener != null && !delTime.isLive())
-                listener.onMergePartitionLevelDeletion(delTime, versions);
+                listener.onMergedPartitionLevelDeletion(delTime, versions);
             return delTime;
         }
 
@@ -605,14 +444,19 @@ public abstract class UnfilteredRowIterators
             if (columns.isEmpty())
                 return Rows.EMPTY_STATIC_ROW;
 
-            Row.Merger merger = Row.Merger.createStatic(metadata, iterators.size(), nowInSec, columns, listener);
+            if (iterators.stream().allMatch(iter -> iter.staticRow().isEmpty()))
+                return Rows.EMPTY_STATIC_ROW;
+
+            Row.Merger merger = new Row.Merger(iterators.size(), nowInSec, columns);
             for (int i = 0; i < iterators.size(); i++)
                 merger.add(i, iterators.get(i).staticRow());
 
-            // Note that we should call 'takeAlias' on the result in theory, but we know that we
-            // won't reuse the merger and so that it's ok not to.
             Row merged = merger.merge(partitionDeletion);
-            return merged == null ? Rows.EMPTY_STATIC_ROW : merged;
+            if (merged == null)
+                merged = Rows.EMPTY_STATIC_ROW;
+            if (listener != null)
+                listener.onMergedRows(merged, columns, merger.mergedRows());
+            return merged;
         }
 
         private static PartitionColumns collectColumns(List<UnfilteredRowIterator> iterators)
@@ -659,26 +503,26 @@ public abstract class UnfilteredRowIterators
                 listener.close();
         }
 
-        /**
-         * Specific reducer for merge operations that rewrite the same reusable
-         * row every time. This also skip cells shadowed by range tombstones when writing.
-         */
         private class MergeReducer extends MergeIterator.Reducer<Unfiltered, Unfiltered>
         {
+            private final MergeListener listener;
+
             private Unfiltered.Kind nextKind;
 
             private final Row.Merger rowMerger;
             private final RangeTombstoneMarker.Merger markerMerger;
 
-            private MergeReducer(CFMetaData metadata, int size, boolean reversed, int nowInSec)
+            private MergeReducer(int size, boolean reversed, int nowInSec, MergeListener listener)
             {
-                this.rowMerger = Row.Merger.createRegular(metadata, size, nowInSec, columns().regulars, listener);
-                this.markerMerger = new RangeTombstoneMarker.Merger(metadata, size, partitionLevelDeletion(), reversed, listener);
+                this.rowMerger = new Row.Merger(size, nowInSec, columns().regulars);
+                this.markerMerger = new RangeTombstoneMarker.Merger(size, partitionLevelDeletion(), reversed);
+                this.listener = listener;
             }
 
             @Override
             public boolean trivialReduceIsTrivial()
             {
+                // If we have a listener, we must signal it even when we have a single version
                 return listener == null;
             }
 
@@ -693,9 +537,20 @@ public abstract class UnfilteredRowIterators
 
             protected Unfiltered getReduced()
             {
-                return nextKind == Unfiltered.Kind.ROW
-                     ? rowMerger.merge(markerMerger.activeDeletion())
-                     : markerMerger.merge();
+                if (nextKind == Unfiltered.Kind.ROW)
+                {
+                    Row merged = rowMerger.merge(markerMerger.activeDeletion());
+                    if (listener != null)
+                        listener.onMergedRows(merged == null ? ArrayBackedRow.emptyRow(rowMerger.mergedClustering()) : merged, columns().regulars, rowMerger.mergedRows());
+                    return merged;
+                }
+                else
+                {
+                    RangeTombstoneMarker merged = markerMerger.merge();
+                    if (merged != null && listener != null)
+                        listener.onMergedRangeTombstoneMarkers(merged, markerMerger.mergedMarkers());
+                    return merged;
+                }
             }
 
             protected void onKeyChange()
@@ -712,13 +567,11 @@ public abstract class UnfilteredRowIterators
     {
         private final UnfilteredRowIterator iter;
         private final int nowInSec;
-        private final TombstoneFilteringRow filter;
 
         public FilteringIterator(UnfilteredRowIterator iter, int nowInSec)
         {
             this.iter = iter;
             this.nowInSec = nowInSec;
-            this.filter = new TombstoneFilteringRow(nowInSec);
         }
 
         public CFMetaData metadata()
@@ -744,7 +597,11 @@ public abstract class UnfilteredRowIterators
         public Row staticRow()
         {
             Row row = iter.staticRow();
-            return row.isEmpty() ? row : new TombstoneFilteringRow(nowInSec).setTo(row);
+            if (row.isEmpty())
+                return Rows.EMPTY_STATIC_ROW;
+
+            row = row.purge(DeletionPurger.PURGE_ALL, nowInSec);
+            return row == null ? Rows.EMPTY_STATIC_ROW : row;
         }
 
         protected Row computeNext()
@@ -752,11 +609,11 @@ public abstract class UnfilteredRowIterators
             while (iter.hasNext())
             {
                 Unfiltered next = iter.next();
-                if (next.kind() != Unfiltered.Kind.ROW)
+                if (next.isRangeTombstoneMarker())
                     continue;
 
-                Row row = filter.setTo((Row)next);
-                if (!row.isEmpty())
+                Row row = ((Row)next).purge(DeletionPurger.PURGE_ALL, nowInSec);
+                if (row != null)
                     return row;
             }
             return endOfData();


[15/15] cassandra git commit: Simplify some 8099's implementations

Posted by sl...@apache.org.
Simplify some 8099's implementations

patch by slebresne; reviewed by iamalesky for CASSANDRA-9705


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/24575994
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/24575994
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/24575994

Branch: refs/heads/trunk
Commit: 2457599427d361314dce4833abeb5cd4915d0b06
Parents: 7659ae2
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Jun 30 15:58:02 2015 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Jul 22 18:05:09 2015 +0200

----------------------------------------------------------------------
 .../org/apache/cassandra/config/CFMetaData.java |   2 +-
 .../cassandra/config/ColumnDefinition.java      |  39 +-
 .../apache/cassandra/cql3/ColumnCondition.java  |   7 +-
 .../org/apache/cassandra/cql3/Constants.java    |  23 +-
 src/java/org/apache/cassandra/cql3/Lists.java   |  69 +-
 src/java/org/apache/cassandra/cql3/Maps.java    |  29 +-
 .../org/apache/cassandra/cql3/Operation.java    |   6 +-
 src/java/org/apache/cassandra/cql3/Sets.java    |  27 +-
 .../apache/cassandra/cql3/UntypedResultSet.java |  14 +-
 .../apache/cassandra/cql3/UpdateParameters.java | 109 ++-
 .../cassandra/cql3/selection/Selection.java     |  13 +-
 .../cql3/statements/AlterTableStatement.java    |   5 +-
 .../cql3/statements/CQL3CasRequest.java         |   2 +-
 .../cql3/statements/DeleteStatement.java        |  25 +-
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../cql3/statements/SelectStatement.java        |   8 +-
 .../cql3/statements/UpdateStatement.java        |  18 +-
 .../cassandra/db/AbstractClusteringPrefix.java  |  47 +-
 .../cassandra/db/AbstractLivenessInfo.java      | 164 ----
 src/java/org/apache/cassandra/db/Aliasable.java |  62 --
 src/java/org/apache/cassandra/db/CBuilder.java  |   6 +-
 .../org/apache/cassandra/db/Clustering.java     |  87 +-
 .../cassandra/db/ClusteringComparator.java      |  71 +-
 .../apache/cassandra/db/ClusteringPrefix.java   | 120 +--
 .../org/apache/cassandra/db/ColumnIndex.java    |  16 +-
 src/java/org/apache/cassandra/db/Columns.java   |  33 +-
 .../apache/cassandra/db/CounterMutation.java    |  10 +-
 src/java/org/apache/cassandra/db/DataRange.java |   4 +-
 .../org/apache/cassandra/db/DeletionInfo.java   | 251 +----
 .../org/apache/cassandra/db/DeletionPurger.java |  35 +
 .../org/apache/cassandra/db/DeletionTime.java   |  39 +-
 .../apache/cassandra/db/DeletionTimeArray.java  | 153 ---
 .../cassandra/db/HintedHandOffManager.java      |  25 +-
 .../org/apache/cassandra/db/LegacyLayout.java   | 123 +--
 .../org/apache/cassandra/db/LivenessInfo.java   | 296 ++++--
 .../apache/cassandra/db/LivenessInfoArray.java  | 174 ----
 .../cassandra/db/MutableDeletionInfo.java       | 311 +++++++
 .../apache/cassandra/db/PartitionColumns.java   |   5 +
 .../cassandra/db/PartitionRangeReadCommand.java |   6 +-
 .../org/apache/cassandra/db/RangeTombstone.java |  59 +-
 .../apache/cassandra/db/RangeTombstoneList.java |  29 +-
 .../org/apache/cassandra/db/ReadCommand.java    |  65 +-
 .../apache/cassandra/db/ReusableClustering.java |  82 --
 .../cassandra/db/ReusableClusteringPrefix.java  |  57 --
 .../cassandra/db/ReusableLivenessInfo.java      |  65 --
 .../apache/cassandra/db/RowUpdateBuilder.java   | 133 +--
 .../cassandra/db/SerializationHeader.java       |  17 +-
 .../apache/cassandra/db/SimpleClustering.java   |  93 --
 .../apache/cassandra/db/SimpleDeletionTime.java |  61 --
 .../apache/cassandra/db/SimpleLivenessInfo.java |  75 --
 .../db/SinglePartitionNamesCommand.java         |   6 +-
 .../db/SinglePartitionReadCommand.java          |   4 +-
 src/java/org/apache/cassandra/db/Slice.java     | 151 +--
 src/java/org/apache/cassandra/db/Slices.java    |   4 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |   2 +-
 src/java/org/apache/cassandra/db/TypeSizes.java |   5 +
 .../cassandra/db/UnfilteredDeserializer.java    |  30 +-
 .../columniterator/AbstractSSTableIterator.java | 196 ++--
 .../db/columniterator/SSTableIterator.java      | 352 ++++---
 .../columniterator/SSTableReversedIterator.java | 447 ++++-----
 .../db/compaction/CompactionIterator.java       |  92 +-
 .../filter/AbstractClusteringIndexFilter.java   |   8 +-
 .../db/filter/ClusteringIndexFilter.java        |   4 +-
 .../db/filter/ClusteringIndexNamesFilter.java   |  23 +-
 .../db/filter/ClusteringIndexSliceFilter.java   |  18 +-
 .../cassandra/db/filter/ColumnFilter.java       |  75 +-
 .../cassandra/db/filter/ColumnSubselection.java |  50 +-
 .../apache/cassandra/db/filter/DataLimits.java  |  19 +-
 .../apache/cassandra/db/filter/RowFilter.java   |  33 +-
 .../AbstractSimplePerColumnSecondaryIndex.java  |  21 +-
 .../db/index/PerColumnSecondaryIndex.java       |  12 +-
 .../db/index/SecondaryIndexManager.java         |  16 +-
 .../db/index/SecondaryIndexSearcher.java        |   2 +-
 .../db/index/composites/CompositesIndex.java    |  11 +-
 .../CompositesIndexOnClusteringKey.java         |   2 +-
 .../CompositesIndexOnCollectionValue.java       |   5 +-
 .../CompositesIndexOnPartitionKey.java          |   2 +-
 .../db/index/composites/CompositesSearcher.java |  45 +-
 .../cassandra/db/index/keys/KeysSearcher.java   |   6 +-
 .../cassandra/db/marshal/AbstractType.java      |  20 +-
 .../cassandra/db/marshal/CollectionType.java    |  15 +-
 .../db/partitions/AbstractPartitionData.java    | 850 -----------------
 .../AbstractThreadUnsafePartition.java          | 393 ++++++++
 .../AlteringUnfilteredPartitionIterator.java    |  72 ++
 .../partitions/ArrayBackedCachedPartition.java  | 217 +++--
 .../db/partitions/ArrayBackedPartition.java     |  82 +-
 .../db/partitions/AtomicBTreePartition.java     | 415 ++-------
 .../CountingUnfilteredRowIterator.java          |   8 +-
 .../db/partitions/FilteredPartition.java        |  89 +-
 .../partitions/FilteringPartitionIterator.java  | 146 ---
 .../PartitionStatisticsCollector.java           |  30 +
 .../db/partitions/PartitionUpdate.java          | 714 +++++++-------
 .../db/partitions/PurgingPartitionIterator.java | 150 +++
 .../TombstonePurgingPartitionIterator.java      | 103 ---
 .../UnfilteredPartitionIterators.java           |  84 +-
 .../apache/cassandra/db/rows/AbstractCell.java  |  70 +-
 .../db/rows/AbstractRangeTombstoneMarker.java   |  27 +-
 .../cassandra/db/rows/AbstractReusableRow.java  | 207 -----
 .../apache/cassandra/db/rows/AbstractRow.java   | 102 +-
 .../db/rows/AbstractUnfilteredRowIterator.java  |  14 -
 .../db/rows/AlteringUnfilteredRowIterator.java  |  98 ++
 .../cassandra/db/rows/ArrayBackedRow.java       | 927 +++++++++++++++++++
 .../apache/cassandra/db/rows/BufferCell.java    | 387 ++++++++
 src/java/org/apache/cassandra/db/rows/Cell.java | 122 +--
 .../org/apache/cassandra/db/rows/CellData.java  | 275 ------
 .../org/apache/cassandra/db/rows/CellPath.java  |  50 +-
 .../org/apache/cassandra/db/rows/Cells.java     | 163 +---
 .../apache/cassandra/db/rows/ColumnData.java    |  95 +-
 .../cassandra/db/rows/ComplexColumnData.java    | 343 +++++++
 .../cassandra/db/rows/ComplexRowDataBlock.java  | 796 ----------------
 .../apache/cassandra/db/rows/FilteringRow.java  | 121 ---
 .../cassandra/db/rows/FilteringRowIterator.java | 126 ---
 .../cassandra/db/rows/MemtableRowData.java      | 204 ----
 .../db/rows/RangeTombstoneBoundMarker.java      |  45 +-
 .../db/rows/RangeTombstoneBoundaryMarker.java   |  44 +-
 .../cassandra/db/rows/RangeTombstoneMarker.java |  87 +-
 .../apache/cassandra/db/rows/ReusableRow.java   | 104 ---
 src/java/org/apache/cassandra/db/rows/Row.java  | 569 ++++++------
 .../db/rows/RowAndDeletionMergeIterator.java    | 191 ++++
 .../db/rows/RowAndTombstoneMergeIterator.java   | 171 ----
 .../apache/cassandra/db/rows/RowDataBlock.java  | 275 ------
 .../cassandra/db/rows/RowDiffListener.java      |  78 ++
 .../apache/cassandra/db/rows/RowIterators.java  |  24 +-
 .../org/apache/cassandra/db/rows/RowStats.java  |  82 +-
 src/java/org/apache/cassandra/db/rows/Rows.java | 258 ++++--
 .../cassandra/db/rows/SerializationHelper.java  |  87 +-
 .../cassandra/db/rows/SimpleRowDataBlock.java   | 188 ----
 .../org/apache/cassandra/db/rows/StaticRow.java | 193 ----
 .../db/rows/TombstoneFilteringRow.java          |  55 --
 .../apache/cassandra/db/rows/Unfiltered.java    |  10 +
 .../rows/UnfilteredRowIteratorSerializer.java   |  97 +-
 .../db/rows/UnfilteredRowIterators.java         | 345 ++-----
 .../cassandra/db/rows/UnfilteredSerializer.java | 489 ++++------
 .../apache/cassandra/db/rows/WrappingRow.java   | 214 -----
 .../db/rows/WrappingUnfilteredRowIterator.java  |  21 +-
 .../cassandra/io/sstable/CQLSSTableWriter.java  |   1 +
 .../io/sstable/SSTableIdentityIterator.java     |   2 +-
 .../io/sstable/SSTableSimpleIterator.java       |  28 +-
 .../io/sstable/SSTableSimpleUnsortedWriter.java |  86 +-
 .../io/sstable/format/big/BigTableWriter.java   |  87 +-
 .../io/sstable/metadata/MetadataCollector.java  |  38 +-
 .../cassandra/schema/LegacySchemaMigrator.java  |   6 +-
 .../apache/cassandra/schema/SchemaKeyspace.java |   2 +-
 .../apache/cassandra/service/DataResolver.java  | 115 +--
 .../service/pager/RangeSliceQueryPager.java     |   2 +-
 .../service/pager/SinglePartitionPager.java     |   2 +-
 .../apache/cassandra/service/paxos/Commit.java  |  50 +-
 .../cassandra/streaming/StreamReader.java       |  25 +-
 .../cassandra/thrift/CassandraServer.java       | 154 +--
 .../cassandra/thrift/ThriftConversion.java      |   7 +-
 .../cassandra/thrift/ThriftResultsMerger.java   | 123 +--
 .../apache/cassandra/utils/ByteBufferUtil.java  |  31 +
 .../org/apache/cassandra/utils/ObjectSizes.java |   3 +
 .../org/apache/cassandra/utils/Sorting.java     | 254 -----
 .../utils/memory/AbstractAllocator.java         |  33 +
 .../utils/memory/MemtableAllocator.java         |  17 +-
 .../utils/memory/MemtableBufferAllocator.java   |  78 +-
 .../cassandra/utils/memory/NativeAllocator.java |  10 +-
 test/data/corrupt-sstables/la-1-big-CRC.db      | Bin 8 -> 8 bytes
 test/data/corrupt-sstables/la-1-big-Data.db     | Bin 280 -> 259 bytes
 .../corrupt-sstables/la-1-big-Digest.adler32    |   2 +-
 test/data/corrupt-sstables/la-1-big-Index.db    | Bin 105 -> 105 bytes
 .../corrupt-sstables/la-1-big-Statistics.db     | Bin 4649 -> 4645 bytes
 test/data/corrupt-sstables/la-1-big-TOC.txt     |   8 +-
 .../db/commitlog/CommitLogStressTest.java       |   2 +-
 .../unit/org/apache/cassandra/SchemaLoader.java |   2 +-
 test/unit/org/apache/cassandra/Util.java        |  20 +-
 .../cassandra/cache/AutoSavingCacheTest.java    |   2 +-
 .../cassandra/config/ColumnDefinitionTest.java  |   4 +-
 .../cassandra/cql3/ColumnConditionTest.java     |  75 +-
 .../apache/cassandra/cql3/SimpleQueryTest.java  |   4 +-
 test/unit/org/apache/cassandra/db/CellTest.java |  73 +-
 .../cassandra/db/ColumnFamilyStoreTest.java     |   4 +-
 .../org/apache/cassandra/db/CommitLogTest.java  |  16 +-
 .../apache/cassandra/db/CounterCellTest.java    | 178 ++--
 .../org/apache/cassandra/db/KeyspaceTest.java   |   4 +-
 .../cassandra/db/RangeTombstoneListTest.java    |  10 +-
 .../apache/cassandra/db/RangeTombstoneTest.java |  25 +-
 .../apache/cassandra/db/ReadMessageTest.java    |   2 +-
 .../db/RecoveryManagerMissingHeaderTest.java    |   4 +-
 .../cassandra/db/RecoveryManagerTest.java       |   4 +-
 .../org/apache/cassandra/db/RowCacheTest.java   |  20 +-
 test/unit/org/apache/cassandra/db/RowTest.java  |  44 +-
 .../unit/org/apache/cassandra/db/ScrubTest.java |   3 +-
 .../db/commitlog/CommitLogUpgradeTest.java      |   3 +-
 .../rows/RowAndDeletionMergeIteratorTest.java   | 407 ++++++++
 .../rows/RowAndTombstoneMergeIteratorTest.java  | 415 ---------
 .../rows/UnfilteredRowIteratorsMergeTest.java   | 198 +---
 .../cassandra/io/sstable/IndexHelperTest.java   |   4 +-
 .../cassandra/io/sstable/SSTableLoaderTest.java |   5 +-
 .../cassandra/io/sstable/SSTableReaderTest.java |   2 +-
 .../io/sstable/SSTableRewriterTest.java         |  26 +-
 .../org/apache/cassandra/schema/DefsTest.java   |   8 +-
 .../cassandra/service/DataResolverTest.java     |  10 +-
 .../cassandra/triggers/TriggerExecutorTest.java |  26 +-
 195 files changed, 7554 insertions(+), 11165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index 3b43960..fbfb7fa 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -715,7 +715,7 @@ public final class CFMetaData
         // it means that it's a dropped column from before 3.0, and in that case using
         // BytesType is fine for what we'll be using it for, even if that's a hack.
         AbstractType<?> type = dropped.type == null ? BytesType.instance : dropped.type;
-        return ColumnDefinition.regularDef(this, name, type, null);
+        return ColumnDefinition.regularDef(this, name, type);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/config/ColumnDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ColumnDefinition.java b/src/java/org/apache/cassandra/config/ColumnDefinition.java
index 432d5a8..54a00f5 100644
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java
@@ -70,7 +70,6 @@ public class ColumnDefinition extends ColumnSpecification implements Comparable<
     private final Integer componentIndex;
 
     private final Comparator<CellPath> cellPathComparator;
-    private final Comparator<Cell> cellComparator;
 
     /**
      * These objects are compared frequently, so we encode several of their comparison components
@@ -103,19 +102,19 @@ public class ColumnDefinition extends ColumnSpecification implements Comparable<
         return new ColumnDefinition(ksName, cfName, ColumnIdentifier.getInterned(name, true),  validator, null, null, null, componentIndex, Kind.CLUSTERING);
     }
 
-    public static ColumnDefinition regularDef(CFMetaData cfm, ByteBuffer name, AbstractType<?> validator, Integer componentIndex)
+    public static ColumnDefinition regularDef(CFMetaData cfm, ByteBuffer name, AbstractType<?> validator)
     {
-        return new ColumnDefinition(cfm, name, validator, componentIndex, Kind.REGULAR);
+        return new ColumnDefinition(cfm, name, validator, null, Kind.REGULAR);
     }
 
-    public static ColumnDefinition regularDef(String ksName, String cfName, String name, AbstractType<?> validator, Integer componentIndex)
+    public static ColumnDefinition regularDef(String ksName, String cfName, String name, AbstractType<?> validator)
     {
-        return new ColumnDefinition(ksName, cfName, ColumnIdentifier.getInterned(name, true), validator, null, null, null, componentIndex, Kind.REGULAR);
+        return new ColumnDefinition(ksName, cfName, ColumnIdentifier.getInterned(name, true), validator, null, null, null, null, Kind.REGULAR);
     }
 
-    public static ColumnDefinition staticDef(CFMetaData cfm, ByteBuffer name, AbstractType<?> validator, Integer componentIndex)
+    public static ColumnDefinition staticDef(CFMetaData cfm, ByteBuffer name, AbstractType<?> validator)
     {
-        return new ColumnDefinition(cfm, name, validator, componentIndex, Kind.STATIC);
+        return new ColumnDefinition(cfm, name, validator, null, Kind.STATIC);
     }
 
     public ColumnDefinition(CFMetaData cfm, ByteBuffer name, AbstractType<?> validator, Integer componentIndex, Kind kind)
@@ -150,12 +149,13 @@ public class ColumnDefinition extends ColumnSpecification implements Comparable<
         super(ksName, cfName, name, validator);
         assert name != null && validator != null && kind != null;
         assert name.isInterned();
+        assert componentIndex == null || kind.isPrimaryKeyKind(); // The componentIndex really only make sense for partition and clustering columns,
+                                                                  // so make sure we don't sneak it for something else since it'd breaks equals()
         this.kind = kind;
         this.indexName = indexName;
         this.componentIndex = componentIndex;
         this.setIndexType(indexType, indexOptions);
         this.cellPathComparator = makeCellPathComparator(kind, validator);
-        this.cellComparator = makeCellComparator(cellPathComparator);
         this.comparisonOrder = comparisonOrder(kind, isComplex(), position());
     }
 
@@ -185,21 +185,6 @@ public class ColumnDefinition extends ColumnSpecification implements Comparable<
         };
     }
 
-    private static Comparator<Cell> makeCellComparator(final Comparator<CellPath> cellPathComparator)
-    {
-        return new Comparator<Cell>()
-        {
-            public int compare(Cell c1, Cell c2)
-            {
-                int cmp = c1.column().compareTo(c2.column());
-                if (cmp != 0 || cellPathComparator == null)
-                    return cmp;
-
-                return cellPathComparator.compare(c1.path(), c2.path());
-            }
-        };
-    }
-
     public ColumnDefinition copy()
     {
         return new ColumnDefinition(ksName, cfName, name, type, indexType, indexOptions, indexName, componentIndex, kind);
@@ -422,14 +407,14 @@ public class ColumnDefinition extends ColumnSpecification implements Comparable<
         return cellPathComparator;
     }
 
-    public Comparator<Cell> cellComparator()
+    public boolean isComplex()
     {
-        return cellComparator;
+        return cellPathComparator != null;
     }
 
-    public boolean isComplex()
+    public boolean isSimple()
     {
-        return cellPathComparator != null;
+        return !isComplex();
     }
 
     public CellPath.Serializer cellPathSerializer()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/ColumnCondition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnCondition.java b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
index 50f79f4..0c2ea2a 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnCondition.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
@@ -38,7 +38,6 @@ import static com.google.common.collect.Lists.newArrayList;
  */
 public class ColumnCondition
 {
-
     public final ColumnDefinition column;
 
     // For collection, when testing the equality of a specific element, null otherwise.
@@ -210,7 +209,11 @@ public class ColumnCondition
     {
         // If we're asking for a complex cells, and we didn't got any row from our read, it's
         // the same as not having any cells for that column.
-        return row == null ? Collections.<Cell>emptyIterator() : row.getCells(column);
+        if (row == null)
+            return Collections.<Cell>emptyIterator();
+
+        ComplexColumnData complexData = row.getComplexColumnData(column);
+        return complexData == null ? Collections.<Cell>emptyIterator() : complexData.iterator();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/Constants.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Constants.java b/src/java/org/apache/cassandra/cql3/Constants.java
index 859b1b5..f10484d 100644
--- a/src/java/org/apache/cassandra/cql3/Constants.java
+++ b/src/java/org/apache/cassandra/cql3/Constants.java
@@ -23,7 +23,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.CounterColumnType;
@@ -318,13 +317,13 @@ public abstract class Constants
             super(column, t);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             ByteBuffer value = t.bindAndGet(params.options);
             if (value == null)
-                params.addTombstone(column, writer);
+                params.addTombstone(column);
             else if (value != ByteBufferUtil.UNSET_BYTE_BUFFER) // use reference equality and not object equality
-                params.addCell(clustering, column, writer, value);
+                params.addCell(column, value);
         }
     }
 
@@ -335,7 +334,7 @@ public abstract class Constants
             super(column, t);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             ByteBuffer bytes = t.bindAndGet(params.options);
             if (bytes == null)
@@ -344,7 +343,7 @@ public abstract class Constants
                 return;
 
             long increment = ByteBufferUtil.toLong(bytes);
-            params.addCounter(column, writer, increment);
+            params.addCounter(column, increment);
         }
     }
 
@@ -355,7 +354,7 @@ public abstract class Constants
             super(column, t);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             ByteBuffer bytes = t.bindAndGet(params.options);
             if (bytes == null)
@@ -367,7 +366,7 @@ public abstract class Constants
             if (increment == Long.MIN_VALUE)
                 throw new InvalidRequestException("The negation of " + increment + " overflows supported counter precision (signed 8 bytes integer)");
 
-            params.addCounter(column, writer, -increment);
+            params.addCounter(column, -increment);
         }
     }
 
@@ -380,12 +379,12 @@ public abstract class Constants
             super(column, null);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             if (column.type.isMultiCell())
-                params.setComplexDeletionTime(column, writer);
+                params.setComplexDeletionTime(column);
             else
-                params.addTombstone(column, writer);
+                params.addTombstone(column);
         }
-    };
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/Lists.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Lists.java b/src/java/org/apache/cassandra/cql3/Lists.java
index 5da2b37..d9dac22 100644
--- a/src/java/org/apache/cassandra/cql3/Lists.java
+++ b/src/java/org/apache/cassandra/cql3/Lists.java
@@ -21,12 +21,9 @@ import static org.apache.cassandra.cql3.Constants.UNSET_VALUE;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 
-import com.google.common.collect.Iterators;
-
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.*;
@@ -72,7 +69,7 @@ public abstract class Lists
             validateAssignableTo(keyspace, receiver);
 
             ColumnSpecification valueSpec = Lists.valueSpecOf(receiver);
-            List<Term> values = new ArrayList<Term>(elements.size());
+            List<Term> values = new ArrayList<>(elements.size());
             boolean allTerminal = true;
             for (Term.Raw rt : elements)
             {
@@ -300,7 +297,7 @@ public abstract class Lists
             super(column, t);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             Term.Terminal value = t.bind(params.options);
             if (value == UNSET_VALUE)
@@ -308,8 +305,8 @@ public abstract class Lists
 
             // delete + append
             if (column.type.isMultiCell())
-                params.setComplexDeletionTimeForOverwrite(column, writer);
-            Appender.doAppend(value, clustering, writer, column, params);
+                params.setComplexDeletionTimeForOverwrite(column);
+            Appender.doAppend(value, column, params);
         }
     }
 
@@ -318,17 +315,8 @@ public abstract class Lists
         if (row == null)
             return 0;
 
-        Iterator<Cell> cells = row.getCells(column);
-        return cells == null ? 0 : Iterators.size(cells);
-    }
-
-    private static Cell existingElement(Row row, ColumnDefinition column, int idx)
-    {
-        assert row != null;
-        Iterator<Cell> cells = row.getCells(column);
-        assert cells != null;
-
-        return Iterators.get(cells, idx);
+        ComplexColumnData complexData = row.getComplexColumnData(column);
+        return complexData == null ? 0 : complexData.cellsCount();
     }
 
     public static class SetterByIndex extends Operation
@@ -354,7 +342,7 @@ public abstract class Lists
             idx.collectMarkerSpecification(boundNames);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             // we should not get here for frozen lists
             assert column.type.isMultiCell() : "Attempted to set an individual element on a frozen list";
@@ -367,7 +355,7 @@ public abstract class Lists
             if (index == ByteBufferUtil.UNSET_BYTE_BUFFER)
                 throw new InvalidRequestException("Invalid unset value for list index");
 
-            Row existingRow = params.getPrefetchedRow(partitionKey, clustering);
+            Row existingRow = params.getPrefetchedRow(partitionKey, params.currentClustering());
             int existingSize = existingSize(existingRow, column);
             int idx = ByteBufferUtil.toInt(index);
             if (existingSize == 0)
@@ -375,10 +363,10 @@ public abstract class Lists
             if (idx < 0 || idx >= existingSize)
                 throw new InvalidRequestException(String.format("List index %d out of bound, list has size %d", idx, existingSize));
 
-            CellPath elementPath = existingElement(existingRow, column, idx).path();
+            CellPath elementPath = existingRow.getComplexColumnData(column).getCellByIndex(idx).path();
             if (value == null)
             {
-                params.addTombstone(column, writer);
+                params.addTombstone(column);
             }
             else if (value != ByteBufferUtil.UNSET_BYTE_BUFFER)
             {
@@ -388,7 +376,7 @@ public abstract class Lists
                                                                     FBUtilities.MAX_UNSIGNED_SHORT,
                                                                     value.remaining()));
 
-                params.addCell(clustering, column, writer, elementPath, value);
+                params.addCell(column, elementPath, value);
             }
         }
     }
@@ -400,14 +388,14 @@ public abstract class Lists
             super(column, t);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             assert column.type.isMultiCell() : "Attempted to append to a frozen list";
             Term.Terminal value = t.bind(params.options);
-            doAppend(value, clustering, writer, column, params);
+            doAppend(value, column, params);
         }
 
-        static void doAppend(Term.Terminal value, Clustering clustering, Row.Writer writer, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
+        static void doAppend(Term.Terminal value, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
         {
             if (column.type.isMultiCell())
             {
@@ -419,16 +407,16 @@ public abstract class Lists
                 for (ByteBuffer buffer : ((Value) value).elements)
                 {
                     ByteBuffer uuid = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes());
-                    params.addCell(clustering, column, writer, CellPath.create(uuid), buffer);
+                    params.addCell(column, CellPath.create(uuid), buffer);
                 }
             }
             else
             {
                 // for frozen lists, we're overwriting the whole cell value
                 if (value == null)
-                    params.addTombstone(column, writer);
+                    params.addTombstone(column);
                 else
-                    params.addCell(clustering, column, writer, value.get(Server.CURRENT_VERSION));
+                    params.addCell(column, value.get(Server.CURRENT_VERSION));
             }
         }
     }
@@ -440,7 +428,7 @@ public abstract class Lists
             super(column, t);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             assert column.type.isMultiCell() : "Attempted to prepend to a frozen list";
             Term.Terminal value = t.bind(params.options);
@@ -454,7 +442,7 @@ public abstract class Lists
             {
                 PrecisionTime pt = PrecisionTime.getNext(time);
                 ByteBuffer uuid = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(pt.millis, pt.nanos));
-                params.addCell(clustering, column, writer, CellPath.create(uuid), toAdd.get(i));
+                params.addCell(column, CellPath.create(uuid), toAdd.get(i));
             }
         }
     }
@@ -472,16 +460,16 @@ public abstract class Lists
             return true;
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             assert column.type.isMultiCell() : "Attempted to delete from a frozen list";
 
             // We want to call bind before possibly returning to reject queries where the value provided is not a list.
             Term.Terminal value = t.bind(params.options);
 
-            Row existingRow = params.getPrefetchedRow(partitionKey, clustering);
-            Iterator<Cell> cells = existingRow == null ? null : existingRow.getCells(column);
-            if (value == null || value == UNSET_VALUE || cells == null)
+            Row existingRow = params.getPrefetchedRow(partitionKey, params.currentClustering());
+            ComplexColumnData complexData = existingRow == null ? null : existingRow.getComplexColumnData(column);
+            if (value == null || value == UNSET_VALUE || complexData == null)
                 return;
 
             // Note: below, we will call 'contains' on this toDiscard list for each element of existingList.
@@ -489,11 +477,10 @@ public abstract class Lists
             // the read-before-write this operation requires limits its usefulness on big lists, so in practice
             // toDiscard will be small and keeping a list will be more efficient.
             List<ByteBuffer> toDiscard = ((Value)value).elements;
-            while (cells.hasNext())
+            for (Cell cell : complexData)
             {
-                Cell cell = cells.next();
                 if (toDiscard.contains(cell.value()))
-                    params.addTombstone(column, writer, cell.path());
+                    params.addTombstone(column, cell.path());
             }
         }
     }
@@ -511,7 +498,7 @@ public abstract class Lists
             return true;
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             assert column.type.isMultiCell() : "Attempted to delete an item by index from a frozen list";
             Term.Terminal index = t.bind(params.options);
@@ -520,7 +507,7 @@ public abstract class Lists
             if (index == Constants.UNSET_VALUE)
                 return;
 
-            Row existingRow = params.getPrefetchedRow(partitionKey, clustering);
+            Row existingRow = params.getPrefetchedRow(partitionKey, params.currentClustering());
             int existingSize = existingSize(existingRow, column);
             int idx = ByteBufferUtil.toInt(index.get(params.options.getProtocolVersion()));
             if (existingSize == 0)
@@ -528,7 +515,7 @@ public abstract class Lists
             if (idx < 0 || idx >= existingSize)
                 throw new InvalidRequestException(String.format("List index %d out of bound, list has size %d", idx, existingSize));
 
-            params.addTombstone(column, writer, existingElement(existingRow, column, idx).path());
+            params.addTombstone(column, existingRow.getComplexColumnData(column).getCellByIndex(idx).path());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/Maps.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Maps.java b/src/java/org/apache/cassandra/cql3/Maps.java
index 2644108..0f0672f 100644
--- a/src/java/org/apache/cassandra/cql3/Maps.java
+++ b/src/java/org/apache/cassandra/cql3/Maps.java
@@ -26,7 +26,6 @@ import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.marshal.MapType;
@@ -290,7 +289,7 @@ public abstract class Maps
             super(column, t);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             Term.Terminal value = t.bind(params.options);
             if (value == UNSET_VALUE)
@@ -298,8 +297,8 @@ public abstract class Maps
 
             // delete + put
             if (column.type.isMultiCell())
-                params.setComplexDeletionTimeForOverwrite(column, writer);
-            Putter.doPut(value, clustering, writer, column, params);
+                params.setComplexDeletionTimeForOverwrite(column);
+            Putter.doPut(value, column, params);
         }
     }
 
@@ -320,7 +319,7 @@ public abstract class Maps
             k.collectMarkerSpecification(boundNames);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             assert column.type.isMultiCell() : "Attempted to set a value for a single key on a frozen map";
             ByteBuffer key = k.bindAndGet(params.options);
@@ -334,7 +333,7 @@ public abstract class Maps
 
             if (value == null)
             {
-                params.addTombstone(column, writer, path);
+                params.addTombstone(column, path);
             }
             else if (value != ByteBufferUtil.UNSET_BYTE_BUFFER)
             {
@@ -344,7 +343,7 @@ public abstract class Maps
                                                                     FBUtilities.MAX_UNSIGNED_SHORT,
                                                                     value.remaining()));
 
-                params.addCell(clustering, column, writer, path, value);
+                params.addCell(column, path, value);
             }
         }
     }
@@ -356,15 +355,15 @@ public abstract class Maps
             super(column, t);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             assert column.type.isMultiCell() : "Attempted to add items to a frozen map";
             Term.Terminal value = t.bind(params.options);
             if (value != UNSET_VALUE)
-                doPut(value, clustering, writer, column, params);
+                doPut(value, column, params);
         }
 
-        static void doPut(Term.Terminal value, Clustering clustering, Row.Writer writer, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
+        static void doPut(Term.Terminal value, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
         {
             if (column.type.isMultiCell())
             {
@@ -373,15 +372,15 @@ public abstract class Maps
 
                 Map<ByteBuffer, ByteBuffer> elements = ((Value) value).map;
                 for (Map.Entry<ByteBuffer, ByteBuffer> entry : elements.entrySet())
-                    params.addCell(clustering, column, writer, CellPath.create(entry.getKey()), entry.getValue());
+                    params.addCell(column, CellPath.create(entry.getKey()), entry.getValue());
             }
             else
             {
                 // for frozen maps, we're overwriting the whole cell
                 if (value == null)
-                    params.addTombstone(column, writer);
+                    params.addTombstone(column);
                 else
-                    params.addCell(clustering, column, writer, value.get(Server.CURRENT_VERSION));
+                    params.addCell(column, value.get(Server.CURRENT_VERSION));
             }
         }
     }
@@ -393,7 +392,7 @@ public abstract class Maps
             super(column, k);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             assert column.type.isMultiCell() : "Attempted to delete a single key in a frozen map";
             Term.Terminal key = t.bind(params.options);
@@ -402,7 +401,7 @@ public abstract class Maps
             if (key == Constants.UNSET_VALUE)
                 throw new InvalidRequestException("Invalid unset map key");
 
-            params.addTombstone(column, writer, CellPath.create(key.get(params.options.getProtocolVersion())));
+            params.addTombstone(column, CellPath.create(key.get(params.options.getProtocolVersion())));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/Operation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Operation.java b/src/java/org/apache/cassandra/cql3/Operation.java
index 5e72e7f..51c8969 100644
--- a/src/java/org/apache/cassandra/cql3/Operation.java
+++ b/src/java/org/apache/cassandra/cql3/Operation.java
@@ -21,9 +21,7 @@ import java.util.Collections;
 
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
@@ -87,11 +85,9 @@ public abstract class Operation
      * Execute the operation.
      *
      * @param partitionKey partition key for the update.
-     * @param clustering the clustering for the row on which the operation applies
-     * @param writer the row update to which to add the updates generated by this operation.
      * @param params parameters of the update.
      */
-    public abstract void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException;
+    public abstract void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException;
 
     /**
      * A parsed raw UPDATE operation.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/Sets.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Sets.java b/src/java/org/apache/cassandra/cql3/Sets.java
index c03005d..7ff3815 100644
--- a/src/java/org/apache/cassandra/cql3/Sets.java
+++ b/src/java/org/apache/cassandra/cql3/Sets.java
@@ -26,7 +26,6 @@ import com.google.common.base.Joiner;
 
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.marshal.*;
@@ -257,7 +256,7 @@ public abstract class Sets
             super(column, t);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             Term.Terminal value = t.bind(params.options);
             if (value == UNSET_VALUE)
@@ -265,8 +264,8 @@ public abstract class Sets
 
             // delete + add
             if (column.type.isMultiCell())
-                params.setComplexDeletionTimeForOverwrite(column, writer);
-            Adder.doAdd(value, clustering, writer, column, params);
+                params.setComplexDeletionTimeForOverwrite(column);
+            Adder.doAdd(value, column, params);
         }
     }
 
@@ -277,15 +276,15 @@ public abstract class Sets
             super(column, t);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             assert column.type.isMultiCell() : "Attempted to add items to a frozen set";
             Term.Terminal value = t.bind(params.options);
             if (value != UNSET_VALUE)
-                doAdd(value, clustering, writer, column, params);
+                doAdd(value, column, params);
         }
 
-        static void doAdd(Term.Terminal value, Clustering clustering, Row.Writer writer, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
+        static void doAdd(Term.Terminal value, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
         {
             if (column.type.isMultiCell())
             {
@@ -297,16 +296,16 @@ public abstract class Sets
                     if (bb == ByteBufferUtil.UNSET_BYTE_BUFFER)
                         continue;
 
-                    params.addCell(clustering, column, writer, CellPath.create(bb), ByteBufferUtil.EMPTY_BYTE_BUFFER);
+                    params.addCell(column, CellPath.create(bb), ByteBufferUtil.EMPTY_BYTE_BUFFER);
                 }
             }
             else
             {
                 // for frozen sets, we're overwriting the whole cell
                 if (value == null)
-                    params.addTombstone(column, writer);
+                    params.addTombstone(column);
                 else
-                    params.addCell(clustering, column, writer, value.get(Server.CURRENT_VERSION));
+                    params.addCell(column, value.get(Server.CURRENT_VERSION));
             }
         }
     }
@@ -319,7 +318,7 @@ public abstract class Sets
             super(column, t);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             assert column.type.isMultiCell() : "Attempted to remove items from a frozen set";
 
@@ -333,7 +332,7 @@ public abstract class Sets
                                       : Collections.singleton(value.get(params.options.getProtocolVersion()));
 
             for (ByteBuffer bb : toDiscard)
-                params.addTombstone(column, writer, CellPath.create(bb));
+                params.addTombstone(column, CellPath.create(bb));
         }
     }
 
@@ -344,14 +343,14 @@ public abstract class Sets
             super(column, k);
         }
 
-        public void execute(DecoratedKey partitionKey, Clustering clustering, Row.Writer writer, UpdateParameters params) throws InvalidRequestException
+        public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException
         {
             assert column.type.isMultiCell() : "Attempted to delete a single element in a frozen set";
             Term.Terminal elt = t.bind(params.options);
             if (elt == null)
                 throw new InvalidRequestException("Invalid null set element");
 
-            params.addTombstone(column, writer, CellPath.create(elt.get(params.options.getProtocolVersion())));
+            params.addTombstone(column, CellPath.create(elt.get(params.options.getProtocolVersion())));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
index 978cb96..97ab60a 100644
--- a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
@@ -234,18 +234,18 @@ public abstract class UntypedResultSet implements Iterable<UntypedResultSet.Row>
 
             for (ColumnDefinition def : metadata.partitionColumns())
             {
-                if (def.isComplex())
-                {
-                    Iterator<Cell> cells = row.getCells(def);
-                    if (cells != null)
-                        data.put(def.name.toString(), ((CollectionType)def.type).serializeForNativeProtocol(def, cells, Server.VERSION_3));
-                }
-                else
+                if (def.isSimple())
                 {
                     Cell cell = row.getCell(def);
                     if (cell != null)
                         data.put(def.name.toString(), cell.value());
                 }
+                else
+                {
+                    ComplexColumnData complexData = row.getComplexColumnData(def);
+                    if (complexData != null)
+                        data.put(def.name.toString(), ((CollectionType)def.type).serializeForNativeProtocol(def, complexData.iterator(), Server.VERSION_3));
+                }
             }
 
             return new Row(data);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/UpdateParameters.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/UpdateParameters.java b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
index 66f6b43..8dcb7e5 100644
--- a/src/java/org/apache/cassandra/cql3/UpdateParameters.java
+++ b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
@@ -29,19 +29,21 @@ import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
- * A simple container that simplify passing parameters for collections methods.
+ * Groups the parameters of an update query, and make building updates easier.
  */
 public class UpdateParameters
 {
     public final CFMetaData metadata;
+    public final PartitionColumns updatedColumns;
     public final QueryOptions options;
 
-    private final LivenessInfo defaultLiveness;
-    private final LivenessInfo deletionLiveness;
+    private final int nowInSec;
+    private final long timestamp;
+    private final int ttl;
+
     private final DeletionTime deletionTime;
 
     private final SecondaryIndexManager indexManager;
@@ -49,16 +51,30 @@ public class UpdateParameters
     // For lists operation that require a read-before-write. Will be null otherwise.
     private final Map<DecoratedKey, Partition> prefetchedRows;
 
-    public UpdateParameters(CFMetaData metadata, QueryOptions options, long timestamp, int ttl, Map<DecoratedKey, Partition> prefetchedRows, boolean validateIndexedColumns)
+    private Row.Builder staticBuilder;
+    private Row.Builder regularBuilder;
+
+    // The builder currently in use. Will alias either staticBuilder or regularBuilder, which are themselves built lazily.
+    private Row.Builder builder;
+
+    public UpdateParameters(CFMetaData metadata,
+                            PartitionColumns updatedColumns,
+                            QueryOptions options,
+                            long timestamp,
+                            int ttl,
+                            Map<DecoratedKey, Partition> prefetchedRows,
+                            boolean validateIndexedColumns)
     throws InvalidRequestException
     {
         this.metadata = metadata;
+        this.updatedColumns = updatedColumns;
         this.options = options;
 
-        int nowInSec = FBUtilities.nowInSeconds();
-        this.defaultLiveness = SimpleLivenessInfo.forUpdate(timestamp, ttl, nowInSec, metadata);
-        this.deletionLiveness = SimpleLivenessInfo.forDeletion(timestamp, nowInSec);
-        this.deletionTime = new SimpleDeletionTime(timestamp, nowInSec);
+        this.nowInSec = FBUtilities.nowInSeconds();
+        this.timestamp = timestamp;
+        this.ttl = ttl;
+
+        this.deletionTime = new DeletionTime(timestamp, nowInSec);
 
         this.prefetchedRows = prefetchedRows;
 
@@ -85,7 +101,7 @@ public class UpdateParameters
             indexManager.validate(partitionKey);
     }
 
-    public void writeClustering(Clustering clustering, Row.Writer writer) throws InvalidRequestException
+    public void newRow(Clustering clustering) throws InvalidRequestException
     {
         if (indexManager != null)
             indexManager.validate(clustering);
@@ -101,66 +117,93 @@ public class UpdateParameters
                 throw new InvalidRequestException("Invalid empty or null value for column " + metadata.clusteringColumns().get(0).name);
         }
 
-        Rows.writeClustering(clustering, writer);
+        if (clustering == Clustering.STATIC_CLUSTERING)
+        {
+            if (staticBuilder == null)
+                staticBuilder = ArrayBackedRow.unsortedBuilder(updatedColumns.statics, nowInSec);
+            builder = staticBuilder;
+        }
+        else
+        {
+            if (regularBuilder == null)
+                regularBuilder = ArrayBackedRow.unsortedBuilder(updatedColumns.regulars, nowInSec);
+            builder = regularBuilder;
+        }
+
+        builder.newRow(clustering);
+    }
+
+    public Clustering currentClustering()
+    {
+        return builder.clustering();
     }
 
-    public void writePartitionKeyLivenessInfo(Row.Writer writer)
+    public void addPrimaryKeyLivenessInfo()
     {
-        writer.writePartitionKeyLivenessInfo(defaultLiveness);
+        builder.addPrimaryKeyLivenessInfo(LivenessInfo.create(metadata, timestamp, ttl, nowInSec));
     }
 
-    public void writeRowDeletion(Row.Writer writer)
+    public void addRowDeletion()
     {
-        writer.writeRowDeletion(deletionTime);
+        builder.addRowDeletion(deletionTime);
     }
 
-    public void addTombstone(ColumnDefinition column, Row.Writer writer) throws InvalidRequestException
+    public void addTombstone(ColumnDefinition column) throws InvalidRequestException
     {
-        addTombstone(column, writer, null);
+        addTombstone(column, null);
     }
 
-    public void addTombstone(ColumnDefinition column, Row.Writer writer, CellPath path) throws InvalidRequestException
+    public void addTombstone(ColumnDefinition column, CellPath path) throws InvalidRequestException
     {
-        writer.writeCell(column, false, ByteBufferUtil.EMPTY_BYTE_BUFFER, deletionLiveness, path);
+        builder.addCell(BufferCell.tombstone(column, timestamp, nowInSec, path));
     }
 
-    public void addCell(Clustering clustering, ColumnDefinition column, Row.Writer writer, ByteBuffer value) throws InvalidRequestException
+    public void addCell(ColumnDefinition column, ByteBuffer value) throws InvalidRequestException
     {
-        addCell(clustering, column, writer, null, value);
+        addCell(column, null, value);
     }
 
-    public void addCell(Clustering clustering, ColumnDefinition column, Row.Writer writer, CellPath path, ByteBuffer value) throws InvalidRequestException
+    public void addCell(ColumnDefinition column, CellPath path, ByteBuffer value) throws InvalidRequestException
     {
         if (indexManager != null)
             indexManager.validate(column, value, path);
 
-        writer.writeCell(column, false, value, defaultLiveness, path);
+        Cell cell = ttl == LivenessInfo.NO_TTL
+                  ? BufferCell.live(metadata, column, timestamp, value, path)
+                  : BufferCell.expiring(column, timestamp, ttl, nowInSec, value, path);
+        builder.addCell(cell);
     }
 
-    public void addCounter(ColumnDefinition column, Row.Writer writer, long increment) throws InvalidRequestException
+    public void addCounter(ColumnDefinition column, long increment) throws InvalidRequestException
     {
-        assert defaultLiveness.ttl() == LivenessInfo.NO_TTL;
+        assert ttl == LivenessInfo.NO_TTL;
 
         // In practice, the actual CounterId (and clock really) that we use doesn't matter, because we will
-        // actually ignore it in CounterMutation when we do the read-before-write to create the actual value
-        // that is applied. In other words, this is not the actual value that will be written to the memtable
+        // ignore it in CounterMutation when we do the read-before-write to create the actual value that is
+        // applied. In other words, this is not the actual value that will be written to the memtable
         // because this will be replaced in CounterMutation.updateWithCurrentValue().
         // As an aside, since we don't care about the CounterId/clock, we used to only send the incremement,
         // but that makes things a bit more complex as this means we need to be able to distinguish inside
         // PartitionUpdate between counter updates that has been processed by CounterMutation and those that
         // haven't.
-        ByteBuffer value = CounterContext.instance().createLocal(increment);
-        writer.writeCell(column, true, value, defaultLiveness, null);
+        builder.addCell(BufferCell.live(metadata, column, timestamp, CounterContext.instance().createLocal(increment)));
+    }
+
+    public void setComplexDeletionTime(ColumnDefinition column)
+    {
+        builder.addComplexDeletion(column, deletionTime);
     }
 
-    public void setComplexDeletionTime(ColumnDefinition column, Row.Writer writer)
+    public void setComplexDeletionTimeForOverwrite(ColumnDefinition column)
     {
-        writer.writeComplexDeletion(column, deletionTime);
+        builder.addComplexDeletion(column, new DeletionTime(deletionTime.markedForDeleteAt() - 1, deletionTime.localDeletionTime()));
     }
 
-    public void setComplexDeletionTimeForOverwrite(ColumnDefinition column, Row.Writer writer)
+    public Row buildRow()
     {
-        writer.writeComplexDeletion(column, new SimpleDeletionTime(deletionTime.markedForDeleteAt() - 1, deletionTime.localDeletionTime()));
+        Row built = builder.build();
+        builder = null; // Resetting to null just so we quickly bad usage where we forget to call newRow() after that.
+        return built;
     }
 
     public DeletionTime deletionTime()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/selection/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selection.java b/src/java/org/apache/cassandra/cql3/selection/Selection.java
index 6a68598..bbb8b25 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java
@@ -307,10 +307,19 @@ public abstract class Selection
             current.add(value(c));
 
             if (timestamps != null)
-                timestamps[current.size() - 1] = c.livenessInfo().timestamp();
+                timestamps[current.size() - 1] = c.timestamp();
 
             if (ttls != null)
-                ttls[current.size() - 1] = c.livenessInfo().remainingTTL(nowInSec);
+                ttls[current.size() - 1] = remainingTTL(c, nowInSec);
+        }
+
+        private int remainingTTL(Cell c, int nowInSec)
+        {
+            if (!c.isExpiring())
+                return -1;
+
+            int remaining = c.localDeletionTime() - nowInSec;
+            return remaining >= 0 ? remaining : -1;
         }
 
         private ByteBuffer value(Cell c)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index ebd3d5b..c6ac7cc 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -141,10 +141,9 @@ public class AlterTableStatement extends SchemaAlteringStatement
                     }
                 }
 
-                Integer componentIndex = cfm.isCompound() ? cfm.comparator.size() : null;
                 cfm.addColumnDefinition(isStatic
-                                        ? ColumnDefinition.staticDef(cfm, columnName.bytes, type, componentIndex)
-                                        : ColumnDefinition.regularDef(cfm, columnName.bytes, type, componentIndex));
+                                        ? ColumnDefinition.staticDef(cfm, columnName.bytes, type)
+                                        : ColumnDefinition.regularDef(cfm, columnName.bytes, type));
                 break;
 
             case ALTER:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
index 9352930..dc70bd2 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
@@ -200,7 +200,7 @@ public class CQL3CasRequest implements CASRequest
         public void applyUpdates(FilteredPartition current, PartitionUpdate updates) throws InvalidRequestException
         {
             Map<DecoratedKey, Partition> map = stmt.requiresRead() ? Collections.<DecoratedKey, Partition>singletonMap(key, current) : null;
-            UpdateParameters params = new UpdateParameters(cfm, options, timestamp, stmt.getTimeToLive(options), map, true);
+            UpdateParameters params = new UpdateParameters(cfm, updates.columns(), options, timestamp, stmt.getTimeToLive(options), map, true);
             stmt.addUpdateForKey(updates, cbuilder, params);
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
index b4d7853..5fdf023 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
@@ -26,7 +26,6 @@ import org.apache.cassandra.cql3.restrictions.Restriction;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.utils.Pair;
@@ -62,16 +61,14 @@ public class DeleteStatement extends ModificationStatement
             // ... or a row deletion ...
             else if (cbuilder.remainingCount() == 0)
             {
-                Clustering clustering = cbuilder.build();
-                Row.Writer writer = update.writer();
-                params.writeClustering(clustering, writer);
-                params.writeRowDeletion(writer);
-                writer.endOfRow();
+                params.newRow(cbuilder.build());
+                params.addRowDeletion();
+                update.add(params.buildRow());
             }
             // ... or a range of rows deletion.
             else
             {
-                update.addRangeTombstone(params.makeRangeTombstone(cbuilder));
+                update.add(params.makeRangeTombstone(cbuilder));
             }
         }
         else
@@ -82,20 +79,18 @@ public class DeleteStatement extends ModificationStatement
                 if (cbuilder.remainingCount() > 0)
                     throw new InvalidRequestException(String.format("Primary key column '%s' must be specified in order to delete column '%s'", getFirstEmptyKey().name, regularDeletions.get(0).column.name));
 
-                Clustering clustering = cbuilder.build();
-                Row.Writer writer = update.writer();
-                params.writeClustering(clustering, writer);
+                params.newRow(cbuilder.build());
                 for (Operation op : regularDeletions)
-                    op.execute(update.partitionKey(), clustering, writer, params);
-                writer.endOfRow();
+                    op.execute(update.partitionKey(), params);
+                update.add(params.buildRow());
             }
 
             if (!staticDeletions.isEmpty())
             {
-                Row.Writer writer = update.staticWriter();
+                params.newRow(Clustering.STATIC_CLUSTERING);
                 for (Operation op : staticDeletions)
-                    op.execute(update.partitionKey(), Clustering.STATIC_CLUSTERING, writer, params);
-                writer.endOfRow();
+                    op.execute(update.partitionKey(), params);
+                update.add(params.buildRow());
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index ee493f5..e827309 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -816,7 +816,7 @@ public abstract class ModificationStatement implements CQLStatement
     {
         // Some lists operation requires reading
         Map<DecoratedKey, Partition> lists = readRequiredLists(keys, clustering, local, options.getConsistency());
-        return new UpdateParameters(cfm, options, getTimestamp(now, options), getTimeToLive(options), lists, true);
+        return new UpdateParameters(cfm, updatedColumns(), options, getTimestamp(now, options), getTimeToLive(options), lists, true);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index b55125a..1a7de37 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -594,7 +594,7 @@ public class SelectStatement implements CQLStatement
 
         ByteBuffer[] keyComponents = getComponents(cfm, partition.partitionKey());
 
-        Row staticRow = partition.staticRow().takeAlias();
+        Row staticRow = partition.staticRow();
         // If there is no rows, then provided the select was a full partition selection
         // (i.e. not a 2ndary index search and there was no condition on clustering columns),
         // we want to include static columns and we're done.
@@ -653,11 +653,11 @@ public class SelectStatement implements CQLStatement
         {
             // Collections are the only complex types we have so far
             assert def.type.isCollection() && def.type.isMultiCell();
-            Iterator<Cell> cells = row.getCells(def);
-            if (cells == null)
+            ComplexColumnData complexData = row.getComplexColumnData(def);
+            if (complexData == null)
                 result.add((ByteBuffer)null);
             else
-                result.add(((CollectionType)def.type).serializeForNativeProtocol(def, cells, protocolVersion));
+                result.add(((CollectionType)def.type).serializeForNativeProtocol(def, complexData.iterator(), protocolVersion));
         }
         else
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
index 6229019..c6bfc98 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@ -23,7 +23,6 @@ import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -54,16 +53,13 @@ public class UpdateStatement extends ModificationStatement
 
         if (updatesRegularRows())
         {
-            Clustering clustering = cbuilder.build();
-            Row.Writer writer = update.writer();
-            params.writeClustering(clustering, writer);
-
+            params.newRow(cbuilder.build());
 
             // We update the row timestamp (ex-row marker) only on INSERT (#6782)
             // Further, COMPACT tables semantic differs from "CQL3" ones in that a row exists only if it has
             // a non-null column, so we don't want to set the row timestamp for them.
             if (type == StatementType.INSERT && cfm.isCQLTable())
-                params.writePartitionKeyLivenessInfo(writer);
+                params.addPrimaryKeyLivenessInfo();
 
             List<Operation> updates = getRegularOperations();
 
@@ -82,17 +78,17 @@ public class UpdateStatement extends ModificationStatement
             }
 
             for (Operation op : updates)
-                op.execute(update.partitionKey(), clustering, writer, params);
+                op.execute(update.partitionKey(), params);
 
-            writer.endOfRow();
+            update.add(params.buildRow());
         }
 
         if (updatesStaticRow())
         {
-            Row.Writer writer = update.staticWriter();
+            params.newRow(Clustering.STATIC_CLUSTERING);
             for (Operation op : getStaticOperations())
-                op.execute(update.partitionKey(), Clustering.STATIC_CLUSTERING, writer, params);
-            writer.endOfRow();
+                op.execute(update.partitionKey(), params);
+            update.add(params.buildRow());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/AbstractClusteringPrefix.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AbstractClusteringPrefix.java b/src/java/org/apache/cassandra/db/AbstractClusteringPrefix.java
index 9ea071d..2631b46 100644
--- a/src/java/org/apache/cassandra/db/AbstractClusteringPrefix.java
+++ b/src/java/org/apache/cassandra/db/AbstractClusteringPrefix.java
@@ -22,14 +22,48 @@ import java.security.MessageDigest;
 import java.util.Objects;
 
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.ObjectSizes;
 
 public abstract class AbstractClusteringPrefix implements ClusteringPrefix
 {
+    protected static final ByteBuffer[] EMPTY_VALUES_ARRAY = new ByteBuffer[0];
+
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new Clustering(EMPTY_VALUES_ARRAY));
+
+    protected final Kind kind;
+    protected final ByteBuffer[] values;
+
+    protected AbstractClusteringPrefix(Kind kind, ByteBuffer[] values)
+    {
+        this.kind = kind;
+        this.values = values;
+    }
+
+    public Kind kind()
+    {
+        return kind;
+    }
+
     public ClusteringPrefix clustering()
     {
         return this;
     }
 
+    public int size()
+    {
+        return values.length;
+    }
+
+    public ByteBuffer get(int i)
+    {
+        return values[i];
+    }
+
+    public ByteBuffer[] getRawValues()
+    {
+        return values;
+    }
+
     public int dataSize()
     {
         int size = 0;
@@ -47,22 +81,19 @@ public abstract class AbstractClusteringPrefix implements ClusteringPrefix
         {
             ByteBuffer bb = get(i);
             if (bb != null)
-            digest.update(bb.duplicate());
+                digest.update(bb.duplicate());
         }
         FBUtilities.updateWithByte(digest, kind().ordinal());
     }
 
-    public void writeTo(Writer writer)
+    public long unsharedHeapSize()
     {
-        for (int i = 0; i < size(); i++)
-            writer.writeClusteringValue(get(i));
+        return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(values);
     }
 
-    public long unsharedHeapSize()
+    public long unsharedHeapSizeExcludingData()
     {
-        // unsharedHeapSize is used inside the cache and in memtables. Implementations that are
-        // safe to use there (SimpleClustering, Slice.Bound.SimpleBound and MemtableRow.* classes) overwrite this.
-        throw new UnsupportedOperationException();
+        return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(values);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/AbstractLivenessInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AbstractLivenessInfo.java b/src/java/org/apache/cassandra/db/AbstractLivenessInfo.java
deleted file mode 100644
index 4f4b5d1..0000000
--- a/src/java/org/apache/cassandra/db/AbstractLivenessInfo.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * 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.util.Objects;
-import java.security.MessageDigest;
-
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.utils.FBUtilities;
-
-/**
- * Base abstract class for {@code LivenessInfo} implementations.
- *
- * All {@code LivenessInfo} should extends this class unless it has a very
- * good reason not to.
- */
-public abstract class AbstractLivenessInfo implements LivenessInfo
-{
-    public boolean hasTimestamp()
-    {
-        return timestamp() != NO_TIMESTAMP;
-    }
-
-    public boolean hasTTL()
-    {
-        return ttl() != NO_TTL;
-    }
-
-    public boolean hasLocalDeletionTime()
-    {
-        return localDeletionTime() != NO_DELETION_TIME;
-    }
-
-    public int remainingTTL(int nowInSec)
-    {
-        if (!hasTTL())
-            return -1;
-
-        int remaining = localDeletionTime() - nowInSec;
-        return remaining >= 0 ? remaining : -1;
-    }
-
-    public boolean isLive(int nowInSec)
-    {
-        // Note that we don't rely on localDeletionTime() only because if we were to, we
-        // could potentially consider a tombstone as a live cell (due to time skew). So
-        // if a cell has a local deletion time and no ttl, it's a tombstone and consider
-        // dead no matter what it's actual local deletion value is.
-        return hasTimestamp() && (!hasLocalDeletionTime() || (hasTTL() && nowInSec < localDeletionTime()));
-    }
-
-    public void digest(MessageDigest digest)
-    {
-        FBUtilities.updateWithLong(digest, timestamp());
-        FBUtilities.updateWithInt(digest, localDeletionTime());
-        FBUtilities.updateWithInt(digest, ttl());
-    }
-
-    public void validate()
-    {
-        if (ttl() < 0)
-            throw new MarshalException("A TTL should not be negative");
-        if (localDeletionTime() < 0)
-            throw new MarshalException("A local deletion time should not be negative");
-        if (hasTTL() && !hasLocalDeletionTime())
-            throw new MarshalException("Shoud not have a TTL without an associated local deletion time");
-    }
-
-    public int dataSize()
-    {
-        int size = 0;
-        if (hasTimestamp())
-            size += TypeSizes.sizeof(timestamp());
-        if (hasTTL())
-            size += TypeSizes.sizeof(ttl());
-        if (hasLocalDeletionTime())
-            size += TypeSizes.sizeof(localDeletionTime());
-        return size;
-
-    }
-
-    public boolean supersedes(LivenessInfo other)
-    {
-        return timestamp() > other.timestamp();
-    }
-
-    public LivenessInfo mergeWith(LivenessInfo other)
-    {
-        return supersedes(other) ? this : other;
-    }
-
-    public LivenessInfo takeAlias()
-    {
-        return new SimpleLivenessInfo(timestamp(), ttl(), localDeletionTime());
-    };
-
-    public LivenessInfo withUpdatedTimestamp(long newTimestamp)
-    {
-        if (!hasTimestamp())
-            return this;
-
-        return new SimpleLivenessInfo(newTimestamp, ttl(), localDeletionTime());
-    }
-
-    public boolean isPurgeable(long maxPurgeableTimestamp, int gcBefore)
-    {
-        return timestamp() < maxPurgeableTimestamp && localDeletionTime() < gcBefore;
-    }
-
-    @Override
-    public String toString()
-    {
-        StringBuilder sb = new StringBuilder();
-        sb.append('[');
-        boolean needSpace = false;
-        if (hasTimestamp())
-        {
-            sb.append("ts=").append(timestamp());
-            needSpace = true;
-        }
-        if (hasTTL())
-        {
-            sb.append(needSpace ? ' ' : "").append("ttl=").append(ttl());
-            needSpace = true;
-        }
-        if (hasLocalDeletionTime())
-            sb.append(needSpace ? ' ' : "").append("ldt=").append(localDeletionTime());
-        sb.append(']');
-        return sb.toString();
-    }
-
-    @Override
-    public boolean equals(Object other)
-    {
-        if(!(other instanceof LivenessInfo))
-            return false;
-
-        LivenessInfo that = (LivenessInfo)other;
-        return this.timestamp() == that.timestamp()
-            && this.ttl() == that.ttl()
-            && this.localDeletionTime() == that.localDeletionTime();
-    }
-
-    @Override
-    public int hashCode()
-    {
-        return Objects.hash(timestamp(), ttl(), localDeletionTime());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/Aliasable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Aliasable.java b/src/java/org/apache/cassandra/db/Aliasable.java
deleted file mode 100644
index a4396fc..0000000
--- a/src/java/org/apache/cassandra/db/Aliasable.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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;
-
-/**
- * This interface marks objects that are only valid in a restricted scope and
- * shouldn't be simply aliased outside of this scope (in other words, you should
- * not keep a reference to the object that escaped said scope as the object will
- * likely become invalid).
- *
- * For instance, most {@link RowIterator} implementation reuse the same {@link
- * Row} object during iteration. This means that the following code would be
- * incorrect.
- * <pre>
- *    RowIterator iter = ...;
- *    Row someRow = null;
- *    while (iter.hasNext())
- *    {
- *        Row row = iter.next();
- *        if (someCondition(row))
- *           someRow = row;         // This isn't safe
- *        doSomethingElse();
- *    }
- *    useRow(someRow);
- * </pre>
- * The problem being that, because the row iterator reuse the same object,
- * {@code someRow} will not point to the row that had met {@code someCondition}
- * at the end of the iteration ({@code someRow} will point to the last iterated
- * row in practice).
- *
- * When code do need to alias such {@code Aliasable} object, it should call the
- * {@code takeAlias} method that will make a copy of the object if necessary.
- *
- * Of course, the {@code takeAlias} should not be abused, as it defeat the purpose
- * of sharing objects in the first place.
- *
- * Also note that some implementation of an {@code Aliasable} object may be
- * safe to alias, in which case its {@code takeAlias} method will be a no-op.
- */
-public interface Aliasable<T>
-{
-    /**
-     * Returns either this object (if it's safe to alias) or a copy of it
-     * (it it isn't safe to alias).
-     */
-    public T takeAlias();
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/CBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CBuilder.java b/src/java/org/apache/cassandra/db/CBuilder.java
index 56cabf1..fe130dc 100644
--- a/src/java/org/apache/cassandra/db/CBuilder.java
+++ b/src/java/org/apache/cassandra/db/CBuilder.java
@@ -163,7 +163,7 @@ public abstract class CBuilder
             built = true;
 
             // Currently, only dense table can leave some clustering column out (see #7990)
-            return size == 0 ? Clustering.EMPTY : new SimpleClustering(values);
+            return size == 0 ? Clustering.EMPTY : new Clustering(values);
         }
 
         public Slice.Bound buildBound(boolean isStart, boolean isInclusive)
@@ -197,7 +197,7 @@ public abstract class CBuilder
 
             ByteBuffer[] newValues = Arrays.copyOf(values, size+1);
             newValues[size] = value;
-            return new SimpleClustering(newValues);
+            return new Clustering(newValues);
         }
 
         public Clustering buildWith(List<ByteBuffer> newValues)
@@ -208,7 +208,7 @@ public abstract class CBuilder
             for (ByteBuffer value : newValues)
                 buffers[newSize++] = value;
 
-            return new SimpleClustering(buffers);
+            return new Clustering(buffers);
         }
 
         public Slice.Bound buildBoundWith(ByteBuffer value, boolean isStart, boolean isInclusive)


[14/15] cassandra git commit: Simplify some 8099's implementations

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/Clustering.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Clustering.java b/src/java/org/apache/cassandra/db/Clustering.java
index 541556b..7754182 100644
--- a/src/java/org/apache/cassandra/db/Clustering.java
+++ b/src/java/org/apache/cassandra/db/Clustering.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
@@ -25,7 +24,9 @@ import java.util.*;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
  * The clustering column values for a row.
@@ -39,7 +40,7 @@ import org.apache.cassandra.io.util.DataOutputPlus;
  * all of the following ones will be too because that's what thrift allows, but it's never assumed by the
  * code so we could start generally allowing nulls for clustering columns if we wanted to).
  */
-public abstract class Clustering extends AbstractClusteringPrefix
+public class Clustering extends AbstractClusteringPrefix
 {
     public static final Serializer serializer = new Serializer();
 
@@ -47,7 +48,7 @@ public abstract class Clustering extends AbstractClusteringPrefix
      * The special cased clustering used by all static rows. It is a special case in the
      * sense that it's always empty, no matter how many clustering columns the table has.
      */
-    public static final Clustering STATIC_CLUSTERING = new EmptyClustering()
+    public static final Clustering STATIC_CLUSTERING = new Clustering(EMPTY_VALUES_ARRAY)
     {
         @Override
         public Kind kind()
@@ -63,19 +64,35 @@ public abstract class Clustering extends AbstractClusteringPrefix
     };
 
     /** Empty clustering for tables having no clustering columns. */
-    public static final Clustering EMPTY = new EmptyClustering();
+    public static final Clustering EMPTY = new Clustering(EMPTY_VALUES_ARRAY)
+    {
+        @Override
+        public String toString(CFMetaData metadata)
+        {
+            return "EMPTY";
+        }
+    };
+
+    public Clustering(ByteBuffer... values)
+    {
+        super(Kind.CLUSTERING, values);
+    }
 
     public Kind kind()
     {
         return Kind.CLUSTERING;
     }
 
-    public Clustering takeAlias()
+    public Clustering copy(AbstractAllocator allocator)
     {
-        ByteBuffer[] values = new ByteBuffer[size()];
+        // Important for STATIC_CLUSTERING (but no point in being wasteful in general).
+        if (size() == 0)
+            return this;
+
+        ByteBuffer[] newValues = new ByteBuffer[size()];
         for (int i = 0; i < size(); i++)
-            values[i] = get(i);
-        return new SimpleClustering(values);
+            newValues[i] = values[i] == null ? null : allocator.clone(values[i]);
+        return new Clustering(newValues);
     }
 
     public String toString(CFMetaData metadata)
@@ -84,7 +101,7 @@ public abstract class Clustering extends AbstractClusteringPrefix
         for (int i = 0; i < size(); i++)
         {
             ColumnDefinition c = metadata.clusteringColumns().get(i);
-            sb.append(i == 0 ? "" : ", ").append(c.name).append("=").append(get(i) == null ? "null" : c.type.getString(get(i)));
+            sb.append(i == 0 ? "" : ", ").append(c.name).append('=').append(get(i) == null ? "null" : c.type.getString(get(i)));
         }
         return sb.toString();
     }
@@ -100,44 +117,6 @@ public abstract class Clustering extends AbstractClusteringPrefix
         return sb.toString();
     }
 
-    private static class EmptyClustering extends Clustering
-    {
-        private static final ByteBuffer[] EMPTY_VALUES_ARRAY = new ByteBuffer[0];
-
-        public int size()
-        {
-            return 0;
-        }
-
-        public ByteBuffer get(int i)
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        public ByteBuffer[] getRawValues()
-        {
-            return EMPTY_VALUES_ARRAY;
-        }
-
-        @Override
-        public Clustering takeAlias()
-        {
-            return this;
-        }
-
-        @Override
-        public long unsharedHeapSize()
-        {
-            return 0;
-        }
-
-        @Override
-        public String toString(CFMetaData metadata)
-        {
-            return "EMPTY";
-        }
-    }
-
     /**
      * Serializer for Clustering object.
      * <p>
@@ -148,6 +127,7 @@ public abstract class Clustering extends AbstractClusteringPrefix
     {
         public void serialize(Clustering clustering, DataOutputPlus out, int version, List<AbstractType<?>> types) throws IOException
         {
+            assert clustering != STATIC_CLUSTERING : "We should never serialize a static clustering";
             ClusteringPrefix.serializer.serializeValuesWithoutSize(clustering, out, version, types);
         }
 
@@ -156,16 +136,13 @@ public abstract class Clustering extends AbstractClusteringPrefix
             return ClusteringPrefix.serializer.valuesWithoutSizeSerializedSize(clustering, version, types);
         }
 
-        public void deserialize(DataInput in, int version, List<AbstractType<?>> types, Writer writer) throws IOException
+        public Clustering deserialize(DataInputPlus in, int version, List<AbstractType<?>> types) throws IOException
         {
-            ClusteringPrefix.serializer.deserializeValuesWithoutSize(in, types.size(), version, types, writer);
-        }
+            if (types.isEmpty())
+                return EMPTY;
 
-        public Clustering deserialize(DataInput in, int version, List<AbstractType<?>> types) throws IOException
-        {
-            SimpleClustering.Builder builder = SimpleClustering.builder(types.size());
-            deserialize(in, version, types, builder);
-            return builder.build();
+            ByteBuffer[] values = ClusteringPrefix.serializer.deserializeValuesWithoutSize(in, types.size(), version, types);
+            return new Clustering(values);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/ClusteringComparator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ClusteringComparator.java b/src/java/org/apache/cassandra/db/ClusteringComparator.java
index 8b01d6f..a5401f0 100644
--- a/src/java/org/apache/cassandra/db/ClusteringComparator.java
+++ b/src/java/org/apache/cassandra/db/ClusteringComparator.java
@@ -25,7 +25,9 @@ import java.util.Objects;
 
 import com.google.common.base.Joiner;
 
+import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.apache.cassandra.io.sstable.IndexHelper.IndexInfo;
@@ -46,9 +48,11 @@ public class ClusteringComparator implements Comparator<Clusterable>
     private final Comparator<IndexInfo> indexReverseComparator;
     private final Comparator<Clusterable> reverseComparator;
 
+    private final Comparator<Row> rowComparator = (r1, r2) -> compare(r1.clustering(), r2.clustering());
+
     public ClusteringComparator(AbstractType<?>... clusteringTypes)
     {
-        this(Arrays.<AbstractType<?>>asList(clusteringTypes));
+        this(Arrays.asList(clusteringTypes));
     }
 
     public ClusteringComparator(List<AbstractType<?>> clusteringTypes)
@@ -56,27 +60,9 @@ public class ClusteringComparator implements Comparator<Clusterable>
         this.clusteringTypes = clusteringTypes;
         this.isByteOrderComparable = isByteOrderComparable(clusteringTypes);
 
-        this.indexComparator = new Comparator<IndexInfo>()
-        {
-            public int compare(IndexInfo o1, IndexInfo o2)
-            {
-                return ClusteringComparator.this.compare(o1.lastName, o2.lastName);
-            }
-        };
-        this.indexReverseComparator = new Comparator<IndexInfo>()
-        {
-            public int compare(IndexInfo o1, IndexInfo o2)
-            {
-                return ClusteringComparator.this.compare(o1.firstName, o2.firstName);
-            }
-        };
-        this.reverseComparator = new Comparator<Clusterable>()
-        {
-            public int compare(Clusterable c1, Clusterable c2)
-            {
-                return ClusteringComparator.this.compare(c2, c1);
-            }
-        };
+        this.indexComparator = (o1, o2) -> ClusteringComparator.this.compare(o1.lastName, o2.lastName);
+        this.indexReverseComparator = (o1, o2) -> ClusteringComparator.this.compare(o1.firstName, o2.firstName);
+        this.reverseComparator = (c1, c2) -> ClusteringComparator.this.compare(c2, c1);
     }
 
     private static boolean isByteOrderComparable(Iterable<AbstractType<?>> types)
@@ -130,11 +116,10 @@ public class ClusteringComparator implements Comparator<Clusterable>
             throw new IllegalArgumentException(String.format("Invalid number of components, expecting %d but got %d", size(), values.length));
 
         CBuilder builder = CBuilder.create(this);
-        for (int i = 0; i < values.length; i++)
+        for (Object val : values)
         {
-            Object val = values[i];
             if (val instanceof ByteBuffer)
-                builder.add((ByteBuffer)val);
+                builder.add((ByteBuffer) val);
             else
                 builder.add(val);
         }
@@ -179,7 +164,7 @@ public class ClusteringComparator implements Comparator<Clusterable>
     public int compareComponent(int i, ByteBuffer v1, ByteBuffer v2)
     {
         if (v1 == null)
-            return v1 == null ? 0 : -1;
+            return v2 == null ? 0 : -1;
         if (v2 == null)
             return 1;
 
@@ -233,6 +218,19 @@ public class ClusteringComparator implements Comparator<Clusterable>
         }
     }
 
+    /**
+     * A comparator for rows.
+     *
+     * A {@code Row} is a {@code Clusterable} so {@code ClusteringComparator} can be used
+     * to compare rows directly, but when we know we deal with rows (and not {@code Clusterable} in
+     * general), this is a little faster because by knowing we compare {@code Clustering} objects,
+     * we know that 1) they all have the same size and 2) they all have the same kind.
+     */
+    public Comparator<Row> rowComparator()
+    {
+        return rowComparator;
+    }
+
     public Comparator<IndexInfo> indexComparator(boolean reversed)
     {
         return reversed ? indexReverseComparator : indexComparator;
@@ -243,27 +241,6 @@ public class ClusteringComparator implements Comparator<Clusterable>
         return reverseComparator;
     }
 
-    /**
-     * Whether the two provided clustering prefix are on the same clustering values.
-     *
-     * @param c1 the first prefix.
-     * @param c2 the second prefix.
-     * @return whether {@code c1} and {@code c2} have the same clustering values (but not necessarily
-     * the same "kind") or not.
-     */
-    public boolean isOnSameClustering(ClusteringPrefix c1, ClusteringPrefix c2)
-    {
-        if (c1.size() != c2.size())
-            return false;
-
-        for (int i = 0; i < c1.size(); i++)
-        {
-            if (compareComponent(i, c1.get(i), c2.get(i)) != 0)
-                return false;
-        }
-        return true;
-    }
-
     @Override
     public String toString()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/ClusteringPrefix.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ClusteringPrefix.java b/src/java/org/apache/cassandra/db/ClusteringPrefix.java
index 3bc7ff8..7b9d582 100644
--- a/src/java/org/apache/cassandra/db/ClusteringPrefix.java
+++ b/src/java/org/apache/cassandra/db/ClusteringPrefix.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.security.MessageDigest;
@@ -27,29 +26,31 @@ import org.apache.cassandra.cache.IMeasurableMemory;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
- * A clustering prefix is basically the unit of what a {@link ClusteringComparator} can compare.
+ * A clustering prefix is the unit of what a {@link ClusteringComparator} can compare.
  * <p>
- * It holds values for the clustering columns of a table (potentially only a prefix of all of them) and it has
+ * It holds values for the clustering columns of a table (potentially only a prefix of all of them) and has
  * a "kind" that allows us to implement slices with inclusive and exclusive bounds.
  * <p>
- * In practice, {@code ClusteringPrefix} is just the common parts to its 2 main subtype: {@link Clustering} and
- * {@link Slice.Bound}, where:
+ * In practice, {@code ClusteringPrefix} is just the common parts to its 3 main subtype: {@link Clustering} and
+ * {@link Slice.Bound}/{@link RangeTombstone.Bound}, where:
  *   1) {@code Clustering} represents the clustering values for a row, i.e. the values for it's clustering columns.
  *   2) {@code Slice.Bound} represents a bound (start or end) of a slice (of rows).
+ *   3) {@code RangeTombstoneBoundMarker.Bound} represents a range tombstone marker "bound".
  * See those classes for more details.
  */
-public interface ClusteringPrefix extends Aliasable<ClusteringPrefix>, IMeasurableMemory, Clusterable
+public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
 {
     public static final Serializer serializer = new Serializer();
 
     /**
      * The kind of clustering prefix this actually is.
      *
-     * The kind {@code STATIC_CLUSTERING} is only implemented by {@link Clustering.STATIC_CLUSTERING} and {@code CLUSTERING} is
+     * The kind {@code STATIC_CLUSTERING} is only implemented by {@link Clustering#STATIC_CLUSTERING} and {@code CLUSTERING} is
      * implemented by the {@link Clustering} class. The rest is used by {@link Slice.Bound} and {@link RangeTombstone.Bound}.
      */
     public enum Kind
@@ -166,12 +167,12 @@ public interface ClusteringPrefix extends Aliasable<ClusteringPrefix>, IMeasurab
 
         public boolean isOpen(boolean reversed)
         {
-            return reversed ? isEnd() : isStart();
+            return isBoundary() || (reversed ? isEnd() : isStart());
         }
 
         public boolean isClose(boolean reversed)
         {
-            return reversed ? isStart() : isEnd();
+            return isBoundary() || (reversed ? isStart() : isEnd());
         }
 
         public Kind closeBoundOfBoundary(boolean reversed)
@@ -211,15 +212,29 @@ public interface ClusteringPrefix extends Aliasable<ClusteringPrefix>, IMeasurab
      */
     public ByteBuffer get(int i);
 
+    /**
+     * Adds the data of this clustering prefix to the provided digest.
+     *
+     * @param digest the digest to which to add this prefix.
+     */
     public void digest(MessageDigest digest);
 
-    // Used to verify if batches goes over a given size
+    /**
+     * The size of the data hold by this prefix.
+     *
+     * @return the size of the data hold by this prefix (this is not the size of the object in memory, just
+     * the size of the data it stores).
+     */
     public int dataSize();
 
+    /**
+     * Generates a proper string representation of the prefix.
+     *
+     * @param metadata the metadata for the table the clustering prefix is of.
+     * @return a human-readable string representation fo this prefix.
+     */
     public String toString(CFMetaData metadata);
 
-    public void writeTo(Writer writer);
-
     /**
      * The values of this prefix as an array.
      * <p>
@@ -231,21 +246,6 @@ public interface ClusteringPrefix extends Aliasable<ClusteringPrefix>, IMeasurab
      */
     public ByteBuffer[] getRawValues();
 
-    /**
-     * Interface for writing a clustering prefix.
-     * <p>
-     * Each value for the prefix should simply be written in order.
-     */
-    public interface Writer
-    {
-        /**
-         * Write the next value to the writer.
-         *
-         * @param value the value to write.
-         */
-        public void writeClusteringValue(ByteBuffer value);
-    }
-
     public static class Serializer
     {
         public void serialize(ClusteringPrefix clustering, DataOutputPlus out, int version, List<AbstractType<?>> types) throws IOException
@@ -263,7 +263,7 @@ public interface ClusteringPrefix extends Aliasable<ClusteringPrefix>, IMeasurab
             }
         }
 
-        public ClusteringPrefix deserialize(DataInput in, int version, List<AbstractType<?>> types) throws IOException
+        public ClusteringPrefix deserialize(DataInputPlus in, int version, List<AbstractType<?>> types) throws IOException
         {
             Kind kind = Kind.values()[in.readByte()];
             // We shouldn't serialize static clusterings
@@ -317,21 +317,20 @@ public interface ClusteringPrefix extends Aliasable<ClusteringPrefix>, IMeasurab
             return size;
         }
 
-        void deserializeValuesWithoutSize(DataInput in, int size, int version, List<AbstractType<?>> types, ClusteringPrefix.Writer writer) throws IOException
+        ByteBuffer[] deserializeValuesWithoutSize(DataInputPlus in, int size, int version, List<AbstractType<?>> types) throws IOException
         {
-            if (size == 0)
-                return;
+            // Callers of this method should handle the case where size = 0 (in all case we want to return a special value anyway).
+            assert size > 0;
 
+            ByteBuffer[] values = new ByteBuffer[size];
             int[] header = readHeader(size, in);
             for (int i = 0; i < size; i++)
             {
-                if (isNull(header, i))
-                    writer.writeClusteringValue(null);
-                else if (isEmpty(header, i))
-                    writer.writeClusteringValue(ByteBufferUtil.EMPTY_BYTE_BUFFER);
-                else
-                    writer.writeClusteringValue(types.get(i).readValue(in));
+                values[i] = isNull(header, i)
+                          ? null
+                          : (isEmpty(header, i) ? ByteBufferUtil.EMPTY_BYTE_BUFFER : types.get(i).readValue(in));
             }
+            return values;
         }
 
         private int headerBytesCount(int size)
@@ -369,7 +368,7 @@ public interface ClusteringPrefix extends Aliasable<ClusteringPrefix>, IMeasurab
             }
         }
 
-        private int[] readHeader(int size, DataInput in) throws IOException
+        private int[] readHeader(int size, DataInputPlus in) throws IOException
         {
             int nbBytes = headerBytesCount(size);
             int[] header = new int[nbBytes];
@@ -378,14 +377,14 @@ public interface ClusteringPrefix extends Aliasable<ClusteringPrefix>, IMeasurab
             return header;
         }
 
-        private boolean isNull(int[] header, int i)
+        private static boolean isNull(int[] header, int i)
         {
             int b = header[i / 4];
             int mask = 1 << ((i % 4) * 2) + 1;
             return (b & mask) != 0;
         }
 
-        private boolean isEmpty(int[] header, int i)
+        private static boolean isEmpty(int[] header, int i)
         {
             int b = header[i / 4];
             int mask = 1 << ((i % 4) * 2);
@@ -405,7 +404,7 @@ public interface ClusteringPrefix extends Aliasable<ClusteringPrefix>, IMeasurab
     public static class Deserializer
     {
         private final ClusteringComparator comparator;
-        private final DataInput in;
+        private final DataInputPlus in;
         private final SerializationHeader serializationHeader;
 
         private boolean nextIsRow;
@@ -414,14 +413,13 @@ public interface ClusteringPrefix extends Aliasable<ClusteringPrefix>, IMeasurab
         private int nextSize;
         private ClusteringPrefix.Kind nextKind;
         private int deserializedSize;
-        private final ByteBuffer[] nextValues;
+        private ByteBuffer[] nextValues;
 
-        public Deserializer(ClusteringComparator comparator, DataInput in, SerializationHeader header)
+        public Deserializer(ClusteringComparator comparator, DataInputPlus in, SerializationHeader header)
         {
             this.comparator = comparator;
             this.in = in;
             this.serializationHeader = header;
-            this.nextValues = new ByteBuffer[comparator.size()];
         }
 
         public void prepare(int flags) throws IOException
@@ -432,6 +430,14 @@ public interface ClusteringPrefix extends Aliasable<ClusteringPrefix>, IMeasurab
             this.nextSize = nextIsRow ? comparator.size() : in.readUnsignedShort();
             this.nextHeader = serializer.readHeader(nextSize, in);
             this.deserializedSize = 0;
+
+            // The point of the deserializer is that some of the clustering prefix won't actually be used (because they are not
+            // within the bounds of the query), and we want to reduce allocation for them. So we only reuse the values array
+            // between elements if 1) we haven't returned the previous element (if we have, nextValues will be null) and 2)
+            // nextValues is of the proper size. Note that the 2nd condition may not hold for range tombstone bounds, but all
+            // rows have a fixed size clustering, so we'll still save in the common case.
+            if (nextValues == null || nextValues.length != nextSize)
+                this.nextValues = new ByteBuffer[nextSize];
         }
 
         public int compareNextTo(Slice.Bound bound) throws IOException
@@ -473,9 +479,9 @@ public interface ClusteringPrefix extends Aliasable<ClusteringPrefix>, IMeasurab
                 return false;
 
             int i = deserializedSize++;
-            nextValues[i] = serializer.isNull(nextHeader, i)
+            nextValues[i] = Serializer.isNull(nextHeader, i)
                           ? null
-                          : (serializer.isEmpty(nextHeader, i) ? ByteBufferUtil.EMPTY_BYTE_BUFFER : serializationHeader.clusteringTypes().get(i).readValue(in));
+                          : (Serializer.isEmpty(nextHeader, i) ? ByteBufferUtil.EMPTY_BYTE_BUFFER : serializationHeader.clusteringTypes().get(i).readValue(in));
             return true;
         }
 
@@ -485,29 +491,31 @@ public interface ClusteringPrefix extends Aliasable<ClusteringPrefix>, IMeasurab
                 continue;
         }
 
-        public RangeTombstone.Bound.Kind deserializeNextBound(RangeTombstone.Bound.Writer writer) throws IOException
+        public RangeTombstone.Bound deserializeNextBound() throws IOException
         {
             assert !nextIsRow;
             deserializeAll();
-            for (int i = 0; i < nextSize; i++)
-                writer.writeClusteringValue(nextValues[i]);
-            writer.writeBoundKind(nextKind);
-            return nextKind;
+            RangeTombstone.Bound bound = new RangeTombstone.Bound(nextKind, nextValues);
+            nextValues = null;
+            return bound;
         }
 
-        public void deserializeNextClustering(Clustering.Writer writer) throws IOException
+        public Clustering deserializeNextClustering() throws IOException
         {
-            assert nextIsRow && nextSize == nextValues.length;
+            assert nextIsRow;
             deserializeAll();
-            for (int i = 0; i < nextSize; i++)
-                writer.writeClusteringValue(nextValues[i]);
+            Clustering clustering = new Clustering(nextValues);
+            nextValues = null;
+            return clustering;
         }
 
         public ClusteringPrefix.Kind skipNext() throws IOException
         {
             for (int i = deserializedSize; i < nextSize; i++)
-                if (!serializer.isNull(nextHeader, i) && !serializer.isEmpty(nextHeader, i))
+            {
+                if (!Serializer.isNull(nextHeader, i) && !Serializer.isEmpty(nextHeader, i))
                     serializationHeader.clusteringTypes().get(i).skipValue(in);
+            }
             return nextKind;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/ColumnIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnIndex.java b/src/java/org/apache/cassandra/db/ColumnIndex.java
index 1a9b92d..52fc48f 100644
--- a/src/java/org/apache/cassandra/db/ColumnIndex.java
+++ b/src/java/org/apache/cassandra/db/ColumnIndex.java
@@ -74,7 +74,7 @@ public class ColumnIndex
         private int written;
 
         private ClusteringPrefix firstClustering;
-        private final ReusableClusteringPrefix lastClustering;
+        private ClusteringPrefix lastClustering;
 
         private DeletionTime openMarker;
 
@@ -90,7 +90,6 @@ public class ColumnIndex
 
             this.result = new ColumnIndex(new ArrayList<IndexHelper.IndexInfo>());
             this.initialPosition = writer.getFilePointer();
-            this.lastClustering = new ReusableClusteringPrefix(iterator.metadata().clusteringColumns().size());
         }
 
         private void writePartitionHeader(UnfilteredRowIterator iterator) throws IOException
@@ -119,7 +118,7 @@ public class ColumnIndex
         private void addIndexBlock()
         {
             IndexHelper.IndexInfo cIndexInfo = new IndexHelper.IndexInfo(firstClustering,
-                                                                         lastClustering.get().takeAlias(),
+                                                                         lastClustering,
                                                                          startPosition,
                                                                          currentPosition() - startPosition,
                                                                          openMarker);
@@ -129,28 +128,27 @@ public class ColumnIndex
 
         private void add(Unfiltered unfiltered) throws IOException
         {
-            lastClustering.copy(unfiltered.clustering());
-            boolean isMarker = unfiltered.kind() == Unfiltered.Kind.RANGE_TOMBSTONE_MARKER;
-
             if (firstClustering == null)
             {
                 // Beginning of an index block. Remember the start and position
-                firstClustering = lastClustering.get().takeAlias();
+                firstClustering = unfiltered.clustering();
                 startPosition = currentPosition();
             }
 
             UnfilteredSerializer.serializer.serialize(unfiltered, header, writer.stream, version);
+            lastClustering = unfiltered.clustering();
             ++written;
 
-            if (isMarker)
+            if (unfiltered.kind() == Unfiltered.Kind.RANGE_TOMBSTONE_MARKER)
             {
-                RangeTombstoneMarker marker = (RangeTombstoneMarker) unfiltered;
+                RangeTombstoneMarker marker = (RangeTombstoneMarker)unfiltered;
                 openMarker = marker.isOpen(false) ? marker.openDeletionTime(false) : null;
             }
 
             // if we hit the column index size that we have to index after, go ahead and index it.
             if (currentPosition() - startPosition >= DatabaseDescriptor.getColumnIndexSize())
                 addIndexBlock();
+
         }
 
         private ColumnIndex close() throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/Columns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Columns.java b/src/java/org/apache/cassandra/db/Columns.java
index 055624b..48a4504 100644
--- a/src/java/org/apache/cassandra/db/Columns.java
+++ b/src/java/org/apache/cassandra/db/Columns.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db;
 import java.io.DataInput;
 import java.io.IOException;
 import java.util.*;
+import java.util.function.Predicate;
 import java.nio.ByteBuffer;
 import java.security.MessageDigest;
 
@@ -329,9 +330,9 @@ public class Columns implements Iterable<ColumnDefinition>
     }
 
     /**
-     * Whether this object is a subset of the provided other {@code Columns object}.
+     * Whether this object is a superset of the provided other {@code Columns object}.
      *
-     * @param other the othere object to test for inclusion in this object.
+     * @param other the other object to test for inclusion in this object.
      *
      * @return whether all the columns of {@code other} are contained by this object.
      */
@@ -439,6 +440,34 @@ public class Columns implements Iterable<ColumnDefinition>
         return new Columns(newColumns);
     }
 
+    /**
+     * Returns a predicate to test whether columns are included in this {@code Columns} object,
+     * assuming that tes tested columns are passed to the predicate in sorted order.
+     *
+     * @return a predicate to test the inclusion of sorted columns in this object.
+     */
+    public Predicate<ColumnDefinition> inOrderInclusionTester()
+    {
+        return new Predicate<ColumnDefinition>()
+        {
+            private int i = 0;
+
+            public boolean test(ColumnDefinition column)
+            {
+                while (i < columns.length)
+                {
+                    int cmp = column.compareTo(columns[i]);
+                    if (cmp < 0)
+                        return false;
+                    i++;
+                    if (cmp == 0)
+                        return true;
+                }
+                return false;
+            }
+        };
+    }
+
     public void digest(MessageDigest digest)
     {
         for (ColumnDefinition c : this)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/CounterMutation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CounterMutation.java b/src/java/org/apache/cassandra/db/CounterMutation.java
index d1830a0..6818513 100644
--- a/src/java/org/apache/cassandra/db/CounterMutation.java
+++ b/src/java/org/apache/cassandra/db/CounterMutation.java
@@ -155,7 +155,7 @@ public class CounterMutation implements IMutation
     /**
      * Returns a wrapper for the Striped#bulkGet() call (via Keyspace#counterLocksFor())
      * Striped#bulkGet() depends on Object#hashCode(), so here we make sure that the cf id and the partition key
-     * all get to be part of the hashCode() calculation, not just the cell name.
+     * all get to be part of the hashCode() calculation.
      */
     private Iterable<Object> getCounterLockKeys()
     {
@@ -167,11 +167,11 @@ public class CounterMutation implements IMutation
                 {
                     public Iterable<Object> apply(final Row row)
                     {
-                        return Iterables.concat(Iterables.transform(row, new Function<Cell, Object>()
+                        return Iterables.concat(Iterables.transform(row, new Function<ColumnData, Object>()
                         {
-                            public Object apply(final Cell cell)
+                            public Object apply(final ColumnData data)
                             {
-                                return Objects.hashCode(update.metadata().cfId, key(), row.clustering(), cell.column(), cell.path());
+                                return Objects.hashCode(update.metadata().cfId, key(), row.clustering(), data.column());
                             }
                         }));
                     }
@@ -238,7 +238,7 @@ public class CounterMutation implements IMutation
         BTreeSet.Builder<Clustering> names = BTreeSet.builder(cfs.metadata.comparator);
         for (PartitionUpdate.CounterMark mark : marks)
         {
-            names.add(mark.clustering().takeAlias());
+            names.add(mark.clustering());
             if (mark.path() == null)
                 builder.add(mark.column());
             else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/DataRange.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DataRange.java b/src/java/org/apache/cassandra/db/DataRange.java
index 0d7a762..358b0ac 100644
--- a/src/java/org/apache/cassandra/db/DataRange.java
+++ b/src/java/org/apache/cassandra/db/DataRange.java
@@ -16,7 +16,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
@@ -26,6 +25,7 @@ import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.dht.*;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessagingService;
 
@@ -372,7 +372,7 @@ public class DataRange
             }
         }
 
-        public DataRange deserialize(DataInput in, int version, CFMetaData metadata) throws IOException
+        public DataRange deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException
         {
             AbstractBounds<PartitionPosition> range = AbstractBounds.rowPositionSerializer.deserialize(in, MessagingService.globalPartitioner(), version);
             ClusteringIndexFilter filter = ClusteringIndexFilter.serializer.deserialize(in, version, metadata);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/DeletionInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletionInfo.java b/src/java/org/apache/cassandra/db/DeletionInfo.java
index a441c48..0b5df06 100644
--- a/src/java/org/apache/cassandra/db/DeletionInfo.java
+++ b/src/java/org/apache/cassandra/db/DeletionInfo.java
@@ -19,255 +19,56 @@ package org.apache.cassandra.db;
 
 import java.util.Iterator;
 
-import com.google.common.base.Objects;
-import com.google.common.collect.Iterators;
-
 import org.apache.cassandra.cache.IMeasurableMemory;
-import org.apache.cassandra.db.rows.Cell;
-import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.db.rows.RowStats;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
  * A combination of a top-level (partition) tombstone and range tombstones describing the deletions
  * within a partition.
+ * <p>
+ * Note that in practice {@link MutableDeletionInfo} is the only concrete implementation of this, however
+ * different parts of the code will return either {@code DeletionInfo} or {@code MutableDeletionInfo} based
+ * on whether it can/should be mutated or not.
+ * <p>
+ * <b>Warning:</b> do not ever cast a {@code DeletionInfo} into a {@code MutableDeletionInfo} to mutate it!!!
+ * TODO: it would be safer to have 2 actual implementation of DeletionInfo, one mutable and one that isn't (I'm
+ * just lazy right this minute).
  */
-public class DeletionInfo implements IMeasurableMemory
+public interface DeletionInfo extends IMeasurableMemory
 {
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new DeletionInfo(0, 0));
-
-    /**
-     * This represents a deletion of the entire partition. We can't represent this within the RangeTombstoneList, so it's
-     * kept separately. This also slightly optimizes the common case of a full partition deletion.
-     */
-    private DeletionTime partitionDeletion;
-
-    /**
-     * A list of range tombstones within the partition.  This is left as null if there are no range tombstones
-     * (to save an allocation (since it's a common case).
-     */
-    private RangeTombstoneList ranges;
-
-    /**
-     * Creates a DeletionInfo with only a top-level (row) tombstone.
-     * @param markedForDeleteAt the time after which the entire row should be considered deleted
-     * @param localDeletionTime what time the deletion write was applied locally (for purposes of
-     *                          purging the tombstone after gc_grace_seconds).
-     */
-    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 SimpleDeletionTime(markedForDeleteAt, localDeletionTime == Integer.MIN_VALUE ? Integer.MAX_VALUE : localDeletionTime));
-    }
-
-    public DeletionInfo(DeletionTime partitionDeletion)
-    {
-        this(partitionDeletion, null);
-    }
-
-    public DeletionInfo(ClusteringComparator comparator, Slice slice, long markedForDeleteAt, int localDeletionTime)
-    {
-        this(DeletionTime.LIVE, new RangeTombstoneList(comparator, 1));
-        ranges.add(slice.start(), slice.end(), markedForDeleteAt, localDeletionTime);
-    }
-
-    public DeletionInfo(DeletionTime partitionDeletion, RangeTombstoneList ranges)
-    {
-        this.partitionDeletion = partitionDeletion.takeAlias();
-        this.ranges = ranges;
-    }
-
-    /**
-     * Returns a new DeletionInfo that has no top-level tombstone or any range tombstones.
-     */
-    public static DeletionInfo live()
-    {
-        return new DeletionInfo(DeletionTime.LIVE);
-    }
-
-    public DeletionInfo copy()
-    {
-        return new DeletionInfo(partitionDeletion, ranges == null ? null : ranges.copy());
-    }
-
-    public DeletionInfo copy(AbstractAllocator allocator)
-    {
-        RangeTombstoneList rangesCopy = null;
-        if (ranges != null)
-             rangesCopy = ranges.copy(allocator);
-
-        return new DeletionInfo(partitionDeletion, rangesCopy);
-    }
+    // Note that while MutableDeletionInfo.live() is mutable, we expose it here as a non-mutable DeletionInfo so sharing is fine.
+    public static final DeletionInfo LIVE = MutableDeletionInfo.live();
 
     /**
      * Returns whether this DeletionInfo is live, that is deletes no columns.
      */
-    public boolean isLive()
-    {
-        return partitionDeletion.isLive() && (ranges == null || ranges.isEmpty());
-    }
+    public boolean isLive();
 
-    /**
-     * Return whether a given cell is deleted by this deletion info.
-     *
-     * @param clustering the clustering for the cell to check.
-     * @param cell the cell to check.
-     * @return true if the cell is deleted, false otherwise
-     */
-    private boolean isDeleted(Clustering clustering, Cell cell)
-    {
-        // If we're live, don't consider anything deleted, even if the cell ends up having as timestamp Long.MIN_VALUE
-        // (which shouldn't happen in practice, but it would invalid to consider it deleted if it does).
-        if (isLive())
-            return false;
+    public DeletionTime getPartitionDeletion();
 
-        if (cell.livenessInfo().timestamp() <= partitionDeletion.markedForDeleteAt())
-            return true;
-
-        // No matter what the counter cell's timestamp is, a tombstone always takes precedence. See CASSANDRA-7346.
-        if (!partitionDeletion.isLive() && cell.isCounterCell())
-            return true;
-
-        return ranges != null && ranges.isDeleted(clustering, cell);
-    }
-
-    /**
-     * Potentially replaces the top-level tombstone with another, keeping whichever has the higher markedForDeleteAt
-     * timestamp.
-     * @param newInfo
-     */
-    public void add(DeletionTime newInfo)
-    {
-        if (newInfo.supersedes(partitionDeletion))
-            partitionDeletion = newInfo;
-    }
-
-    public void add(RangeTombstone tombstone, ClusteringComparator comparator)
-    {
-        if (ranges == null)
-            ranges = new RangeTombstoneList(comparator, 1);
-
-        ranges.add(tombstone);
-    }
-
-    /**
-     * Combines another DeletionInfo with this one and returns the result.  Whichever top-level tombstone
-     * has the higher markedForDeleteAt timestamp will be kept, along with its localDeletionTime.  The
-     * range tombstones will be combined.
-     *
-     * @return this object.
-     */
-    public DeletionInfo add(DeletionInfo newInfo)
-    {
-        add(newInfo.partitionDeletion);
-
-        if (ranges == null)
-            ranges = newInfo.ranges == null ? null : newInfo.ranges.copy();
-        else if (newInfo.ranges != null)
-            ranges.addAll(newInfo.ranges);
+    // Use sparingly, not the most efficient thing
+    public Iterator<RangeTombstone> rangeIterator(boolean reversed);
 
-        return this;
-    }
+    public Iterator<RangeTombstone> rangeIterator(Slice slice, boolean reversed);
 
-    public DeletionTime getPartitionDeletion()
-    {
-        return partitionDeletion;
-    }
+    public RangeTombstone rangeCovering(Clustering name);
 
-    // Use sparingly, not the most efficient thing
-    public Iterator<RangeTombstone> rangeIterator(boolean reversed)
-    {
-        return ranges == null ? Iterators.<RangeTombstone>emptyIterator() : ranges.iterator(reversed);
-    }
+    public void collectStats(RowStats.Collector collector);
 
-    public Iterator<RangeTombstone> rangeIterator(Slice slice, boolean reversed)
-    {
-        return ranges == null ? Iterators.<RangeTombstone>emptyIterator() : ranges.iterator(slice, reversed);
-    }
+    public int dataSize();
 
-    public RangeTombstone rangeCovering(Clustering name)
-    {
-        return ranges == null ? null : ranges.search(name);
-    }
+    public boolean hasRanges();
 
-    public int dataSize()
-    {
-        int size = TypeSizes.sizeof(partitionDeletion.markedForDeleteAt());
-        return size + (ranges == null ? 0 : ranges.dataSize());
-    }
+    public int rangeCount();
 
-    public boolean hasRanges()
-    {
-        return ranges != null && !ranges.isEmpty();
-    }
-
-    public int rangeCount()
-    {
-        return hasRanges() ? ranges.size() : 0;
-    }
+    public long maxTimestamp();
 
     /**
      * Whether this deletion info may modify the provided one if added to it.
      */
-    public boolean mayModify(DeletionInfo delInfo)
-    {
-        return partitionDeletion.compareTo(delInfo.partitionDeletion) > 0 || hasRanges();
-    }
-
-    @Override
-    public String toString()
-    {
-        if (ranges == null || ranges.isEmpty())
-            return String.format("{%s}", partitionDeletion);
-        else
-            return String.format("{%s, ranges=%s}", partitionDeletion, rangesAsString());
-    }
-
-    private String rangesAsString()
-    {
-        assert !ranges.isEmpty();
-        StringBuilder sb = new StringBuilder();
-        ClusteringComparator cc = ranges.comparator();
-        Iterator<RangeTombstone> iter = rangeIterator(false);
-        while (iter.hasNext())
-        {
-            RangeTombstone i = iter.next();
-            sb.append(i.deletedSlice().toString(cc));
-            sb.append("@");
-            sb.append(i.deletionTime());
-        }
-        return sb.toString();
-    }
-
-    // Updates all the timestamp of the deletion contained in this DeletionInfo to be {@code timestamp}.
-    public DeletionInfo updateAllTimestamp(long timestamp)
-    {
-        if (partitionDeletion.markedForDeleteAt() != Long.MIN_VALUE)
-            partitionDeletion = new SimpleDeletionTime(timestamp, partitionDeletion.localDeletionTime());
-
-        if (ranges != null)
-            ranges.updateAllTimestamp(timestamp);
-        return this;
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if(!(o instanceof DeletionInfo))
-            return false;
-        DeletionInfo that = (DeletionInfo)o;
-        return partitionDeletion.equals(that.partitionDeletion) && Objects.equal(ranges, that.ranges);
-    }
-
-    @Override
-    public final int hashCode()
-    {
-        return Objects.hashCode(partitionDeletion, ranges);
-    }
+    public boolean mayModify(DeletionInfo delInfo);
 
-    @Override
-    public long unsharedHeapSize()
-    {
-        return EMPTY_SIZE + partitionDeletion.unsharedHeapSize() + (ranges == null ? 0 : ranges.unsharedHeapSize());
-    }
+    public MutableDeletionInfo mutableCopy();
+    public DeletionInfo copy(AbstractAllocator allocator);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/DeletionPurger.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletionPurger.java b/src/java/org/apache/cassandra/db/DeletionPurger.java
new file mode 100644
index 0000000..d368b69
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/DeletionPurger.java
@@ -0,0 +1,35 @@
+/*
+ * 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;
+
+public interface DeletionPurger
+{
+    public static final DeletionPurger PURGE_ALL = (ts, ldt) -> true;
+
+    public boolean shouldPurge(long timestamp, int localDeletionTime);
+
+    public default boolean shouldPurge(DeletionTime dt)
+    {
+        return !dt.isLive() && shouldPurge(dt.markedForDeleteAt(), dt.localDeletionTime());
+    }
+
+    public default boolean shouldPurge(LivenessInfo liveness, int nowInSec)
+    {
+        return !liveness.isLive(nowInSec) && shouldPurge(liveness.timestamp(), liveness.localExpirationTime());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/DeletionTime.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletionTime.java b/src/java/org/apache/cassandra/db/DeletionTime.java
index 67842a7..3e9ca80 100644
--- a/src/java/org/apache/cassandra/db/DeletionTime.java
+++ b/src/java/org/apache/cassandra/db/DeletionTime.java
@@ -17,13 +17,13 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.security.MessageDigest;
 
 import com.google.common.base.Objects;
 
 import org.apache.cassandra.cache.IMeasurableMemory;
+import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
@@ -34,29 +34,44 @@ import org.apache.cassandra.utils.ObjectSizes;
 /**
  * Information on deletion of a storage engine object.
  */
-public abstract class DeletionTime implements Comparable<DeletionTime>, IMeasurableMemory, Aliasable<DeletionTime>
+public class DeletionTime implements Comparable<DeletionTime>, IMeasurableMemory
 {
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new SimpleDeletionTime(0, 0));
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new DeletionTime(0, 0));
 
     /**
      * A special DeletionTime that signifies that there is no top-level (row) tombstone.
      */
-    public static final DeletionTime LIVE = new SimpleDeletionTime(Long.MIN_VALUE, Integer.MAX_VALUE);
+    public static final DeletionTime LIVE = new DeletionTime(Long.MIN_VALUE, Integer.MAX_VALUE);
 
     public static final Serializer serializer = new Serializer();
 
+    private final long markedForDeleteAt;
+    private final int localDeletionTime;
+
+    public DeletionTime(long markedForDeleteAt, int localDeletionTime)
+    {
+        this.markedForDeleteAt = markedForDeleteAt;
+        this.localDeletionTime = localDeletionTime;
+    }
+
     /**
      * A timestamp (typically in microseconds since the unix epoch, although this is not enforced) after which
      * data should be considered deleted. If set to Long.MIN_VALUE, this implies that the data has not been marked
      * for deletion at all.
      */
-    public abstract long markedForDeleteAt();
+    public long markedForDeleteAt()
+    {
+        return markedForDeleteAt;
+    }
 
     /**
      * The local server timestamp, in seconds since the unix epoch, at which this tombstone was created. This is
      * only used for purposes of purging the tombstone after gc_grace_seconds have elapsed.
      */
-    public abstract int localDeletionTime();
+    public int localDeletionTime()
+    {
+        return localDeletionTime;
+    }
 
     /**
      * Returns whether this DeletionTime is live, that is deletes no columns.
@@ -112,14 +127,14 @@ public abstract class DeletionTime implements Comparable<DeletionTime>, IMeasura
         return markedForDeleteAt() > dt.markedForDeleteAt() || (markedForDeleteAt() == dt.markedForDeleteAt() && localDeletionTime() > dt.localDeletionTime());
     }
 
-    public boolean isPurgeable(long maxPurgeableTimestamp, int gcBefore)
+    public boolean deletes(LivenessInfo info)
     {
-        return markedForDeleteAt() < maxPurgeableTimestamp && localDeletionTime() < gcBefore;
+        return deletes(info.timestamp());
     }
 
-    public boolean deletes(LivenessInfo info)
+    public boolean deletes(Cell cell)
     {
-        return deletes(info.timestamp());
+        return deletes(cell.timestamp());
     }
 
     public boolean deletes(long timestamp)
@@ -151,10 +166,10 @@ public abstract class DeletionTime implements Comparable<DeletionTime>, IMeasura
             long mfda = in.readLong();
             return mfda == Long.MIN_VALUE && ldt == Integer.MAX_VALUE
                  ? LIVE
-                 : new SimpleDeletionTime(mfda, ldt);
+                 : new DeletionTime(mfda, ldt);
         }
 
-        public void skip(DataInput in) throws IOException
+        public void skip(DataInputPlus in) throws IOException
         {
             FileUtils.skipBytesFully(in, 4 + 8);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/DeletionTimeArray.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletionTimeArray.java b/src/java/org/apache/cassandra/db/DeletionTimeArray.java
deleted file mode 100644
index 77eb953..0000000
--- a/src/java/org/apache/cassandra/db/DeletionTimeArray.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * 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.util.Arrays;
-
-import org.apache.cassandra.utils.ObjectSizes;
-
-/**
- * Utility class to store an array of deletion times a bit efficiently.
- */
-public class DeletionTimeArray
-{
-    private long[] markedForDeleteAts;
-    private int[] delTimes;
-
-    public DeletionTimeArray(int initialCapacity)
-    {
-        this.markedForDeleteAts = new long[initialCapacity];
-        this.delTimes = new int[initialCapacity];
-        clear();
-    }
-
-    public void clear(int i)
-    {
-        markedForDeleteAts[i] = Long.MIN_VALUE;
-        delTimes[i] = Integer.MAX_VALUE;
-    }
-
-    public void set(int i, DeletionTime dt)
-    {
-        this.markedForDeleteAts[i] = dt.markedForDeleteAt();
-        this.delTimes[i] = dt.localDeletionTime();
-    }
-
-    public int size()
-    {
-        return markedForDeleteAts.length;
-    }
-
-    public void resize(int newSize)
-    {
-        int prevSize = size();
-
-        markedForDeleteAts = Arrays.copyOf(markedForDeleteAts, newSize);
-        delTimes = Arrays.copyOf(delTimes, newSize);
-
-        Arrays.fill(markedForDeleteAts, prevSize, newSize, Long.MIN_VALUE);
-        Arrays.fill(delTimes, prevSize, newSize, Integer.MAX_VALUE);
-    }
-
-    public boolean supersedes(int i, DeletionTime dt)
-    {
-        return markedForDeleteAts[i] > dt.markedForDeleteAt();
-    }
-
-    public boolean supersedes(int i, int j)
-    {
-        return markedForDeleteAts[i] > markedForDeleteAts[j];
-    }
-
-    public void swap(int i, int j)
-    {
-        long m = markedForDeleteAts[j];
-        int l = delTimes[j];
-
-        move(i, j);
-
-        markedForDeleteAts[i] = m;
-        delTimes[i] = l;
-    }
-
-    public void move(int i, int j)
-    {
-        markedForDeleteAts[j] = markedForDeleteAts[i];
-        delTimes[j] = delTimes[i];
-    }
-
-    public boolean isLive(int i)
-    {
-        return markedForDeleteAts[i] > Long.MIN_VALUE;
-    }
-
-    public void clear()
-    {
-        Arrays.fill(markedForDeleteAts, Long.MIN_VALUE);
-        Arrays.fill(delTimes, Integer.MAX_VALUE);
-    }
-
-    public int dataSize()
-    {
-        return 12 * markedForDeleteAts.length;
-    }
-
-    public long unsharedHeapSize()
-    {
-        return ObjectSizes.sizeOfArray(markedForDeleteAts)
-             + ObjectSizes.sizeOfArray(delTimes);
-    }
-
-    public void copy(DeletionTimeArray other)
-    {
-        assert size() == other.size();
-        for (int i = 0; i < size(); i++)
-        {
-            markedForDeleteAts[i] = other.markedForDeleteAts[i];
-            delTimes[i] = other.delTimes[i];
-        }
-    }
-
-    public static class Cursor extends DeletionTime
-    {
-        private DeletionTimeArray array;
-        private int i;
-
-        public Cursor setTo(DeletionTimeArray array, int i)
-        {
-            this.array = array;
-            this.i = i;
-            return this;
-        }
-
-        public long markedForDeleteAt()
-        {
-            return array.markedForDeleteAts[i];
-        }
-
-        public int localDeletionTime()
-        {
-            return array.delTimes[i];
-        }
-
-        public DeletionTime takeAlias()
-        {
-            return new SimpleDeletionTime(markedForDeleteAt(), localDeletionTime());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/HintedHandOffManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/HintedHandOffManager.java b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
index 38113c8..4501f3c 100644
--- a/src/java/org/apache/cassandra/db/HintedHandOffManager.java
+++ b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
@@ -132,19 +132,13 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
         UUID hintId = UUIDGen.getTimeUUID();
         // serialize the hint with id and version as a composite column name
 
-        PartitionUpdate upd = new PartitionUpdate(SystemKeyspace.Hints,
-                                                  StorageService.getPartitioner().decorateKey(UUIDType.instance.decompose(targetId)),
-                                                  PartitionColumns.of(hintColumn),
-                                                  1);
-
-        Row.Writer writer = upd.writer();
-        Rows.writeClustering(SystemKeyspace.Hints.comparator.make(hintId, MessagingService.current_version), writer);
+        DecoratedKey key = StorageService.getPartitioner().decorateKey(UUIDType.instance.decompose(targetId));
 
+        Clustering clustering = SystemKeyspace.Hints.comparator.make(hintId, MessagingService.current_version);
         ByteBuffer value = ByteBuffer.wrap(FBUtilities.serialize(mutation, Mutation.serializer, MessagingService.current_version));
-        writer.writeCell(hintColumn, false, value, SimpleLivenessInfo.forUpdate(now, ttl, FBUtilities.nowInSeconds(), SystemKeyspace.Hints), null);
-        writer.endOfRow();
+        Cell cell = BufferCell.expiring(hintColumn, now, ttl, FBUtilities.nowInSeconds(), value);
 
-        return new Mutation(upd);
+        return new Mutation(PartitionUpdate.singleRowUpdate(SystemKeyspace.Hints, key, ArrayBackedRow.singleCellRow(clustering, cell)));
     }
 
     /*
@@ -187,13 +181,8 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
     private static void deleteHint(ByteBuffer tokenBytes, Clustering clustering, long timestamp)
     {
         DecoratedKey dk =  StorageService.getPartitioner().decorateKey(tokenBytes);
-
-        PartitionUpdate upd = new PartitionUpdate(SystemKeyspace.Hints, dk, PartitionColumns.of(hintColumn), 1);
-
-        Row.Writer writer = upd.writer();
-        Rows.writeClustering(clustering, writer);
-        Cells.writeTombstone(writer, hintColumn, timestamp, FBUtilities.nowInSeconds());
-
+        Cell cell = BufferCell.tombstone(hintColumn, timestamp, FBUtilities.nowInSeconds());
+        PartitionUpdate upd = PartitionUpdate.singleRowUpdate(SystemKeyspace.Hints, dk, ArrayBackedRow.singleCellRow(clustering, cell));
         new Mutation(upd).applyUnsafe(); // don't bother with commitlog since we're going to flush as soon as we're done with delivery
     }
 
@@ -420,7 +409,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
                 int version = Int32Type.instance.compose(hint.clustering().get(1));
                 Cell cell = hint.getCell(hintColumn);
 
-                final long timestamp = cell.livenessInfo().timestamp();
+                final long timestamp = cell.timestamp();
                 DataInputPlus in = new NIODataInputStream(cell.value(), true);
                 Mutation mutation;
                 try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/LegacyLayout.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/LegacyLayout.java b/src/java/org/apache/cassandra/db/LegacyLayout.java
index f063256..8242ab7 100644
--- a/src/java/org/apache/cassandra/db/LegacyLayout.java
+++ b/src/java/org/apache/cassandra/db/LegacyLayout.java
@@ -31,15 +31,14 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.thrift.ColumnDef;
 import org.apache.cassandra.utils.*;
-import org.apache.hadoop.io.serializer.Serialization;
 
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
@@ -103,7 +102,7 @@ public abstract class LegacyLayout
         if (metadata.isSuper())
         {
             assert superColumnName != null;
-            return decodeForSuperColumn(metadata, new SimpleClustering(superColumnName), cellname);
+            return decodeForSuperColumn(metadata, new Clustering(superColumnName), cellname);
         }
 
         assert superColumnName == null;
@@ -152,7 +151,7 @@ public abstract class LegacyLayout
         {
             // If it's a compact table, it means the column is in fact a "dynamic" one
             if (metadata.isCompactTable())
-                return new LegacyCellName(new SimpleClustering(column), metadata.compactValueColumn(), null);
+                return new LegacyCellName(new Clustering(column), metadata.compactValueColumn(), null);
 
             throw new UnknownColumnException(metadata, column);
         }
@@ -242,7 +241,7 @@ public abstract class LegacyLayout
                                     ? CompositeType.splitName(value)
                                     : Collections.singletonList(value);
 
-        return new SimpleClustering(components.subList(0, Math.min(csize, components.size())).toArray(new ByteBuffer[csize]));
+        return new Clustering(components.subList(0, Math.min(csize, components.size())).toArray(new ByteBuffer[csize]));
     }
 
     public static ByteBuffer encodeClustering(CFMetaData metadata, Clustering clustering)
@@ -276,7 +275,7 @@ public abstract class LegacyLayout
                                                                 DeletionInfo delInfo,
                                                                 Iterator<LegacyCell> cells)
     {
-        SerializationHelper helper = new SerializationHelper(0, SerializationHelper.Flag.LOCAL);
+        SerializationHelper helper = new SerializationHelper(metadata, 0, SerializationHelper.Flag.LOCAL);
         return toUnfilteredRowIterator(metadata, key, LegacyDeletionInfo.from(delInfo), cells, false, helper);
     }
 
@@ -320,22 +319,16 @@ public abstract class LegacyLayout
 
         Iterator<Row> rows = convertToRows(new CellGrouper(metadata, helper), iter, delInfo);
         Iterator<RangeTombstone> ranges = delInfo.deletionInfo.rangeIterator(reversed);
-        final Iterator<Unfiltered> atoms = new RowAndTombstoneMergeIterator(metadata.comparator, reversed)
-                                     .setTo(rows, ranges);
-
-        return new AbstractUnfilteredRowIterator(metadata,
-                                        key,
-                                        delInfo.deletionInfo.getPartitionDeletion(),
-                                        metadata.partitionColumns(),
-                                        staticRow,
-                                        reversed,
-                                        RowStats.NO_STATS)
-        {
-            protected Unfiltered computeNext()
-            {
-                return atoms.hasNext() ? atoms.next() : endOfData();
-            }
-        };
+        return new RowAndDeletionMergeIterator(metadata,
+                                               key,
+                                               delInfo.deletionInfo.getPartitionDeletion(),
+                                               ColumnFilter.all(metadata),
+                                               staticRow,
+                                               reversed,
+                                               RowStats.NO_STATS,
+                                               rows,
+                                               ranges,
+                                               true);
     }
 
     public static Row extractStaticColumns(CFMetaData metadata, DataInputPlus in, Columns statics) throws IOException
@@ -351,7 +344,7 @@ public abstract class LegacyLayout
         for (ColumnDefinition column : statics)
             columnsToFetch.add(column.name.bytes);
 
-        StaticRow.Builder builder = StaticRow.builder(statics, false, metadata.isCounter());
+        Row.Builder builder = ArrayBackedRow.unsortedBuilder(statics, FBUtilities.nowInSeconds());
 
         boolean foundOne = false;
         LegacyAtom atom;
@@ -364,7 +357,7 @@ public abstract class LegacyLayout
                     continue;
 
                 foundOne = true;
-                builder.writeCell(cell.name.column, cell.isCounter(), cell.value, livenessInfo(metadata, cell), null);
+                builder.addCell(new BufferCell(cell.name.column, cell.timestamp, cell.ttl, cell.localDeletionTime, cell.value, null));
             }
             else
             {
@@ -469,7 +462,7 @@ public abstract class LegacyLayout
     {
         return new AbstractIterator<LegacyCell>()
         {
-            private final Iterator<Cell> cells = row.iterator();
+            private final Iterator<Cell> cells = row.cells().iterator();
             // we don't have (and shouldn't have) row markers for compact tables.
             private boolean hasReturnedRowMarker = metadata.isCompactTable();
 
@@ -480,7 +473,7 @@ public abstract class LegacyLayout
                     hasReturnedRowMarker = true;
                     LegacyCellName cellName = new LegacyCellName(row.clustering(), null, null);
                     LivenessInfo info = row.primaryKeyLivenessInfo();
-                    return new LegacyCell(LegacyCell.Kind.REGULAR, cellName, ByteBufferUtil.EMPTY_BYTE_BUFFER, info.timestamp(), info.localDeletionTime(), info.ttl());
+                    return new LegacyCell(LegacyCell.Kind.REGULAR, cellName, ByteBufferUtil.EMPTY_BYTE_BUFFER, info.timestamp(), info.localExpirationTime(), info.ttl());
                 }
 
                 if (!cells.hasNext())
@@ -507,8 +500,7 @@ public abstract class LegacyLayout
         CellPath path = cell.path();
         assert path == null || path.size() == 1;
         LegacyCellName name = new LegacyCellName(clustering, cell.column(), path == null ? null : path.get(0));
-        LivenessInfo info = cell.livenessInfo();
-        return new LegacyCell(kind, name, cell.value(), info.timestamp(), info.localDeletionTime(), info.ttl());
+        return new LegacyCell(kind, name, cell.value(), cell.timestamp(), cell.localDeletionTime(), cell.ttl());
     }
 
     public static RowIterator toRowIterator(final CFMetaData metadata,
@@ -516,17 +508,10 @@ public abstract class LegacyLayout
                                             final Iterator<LegacyCell> cells,
                                             final int nowInSec)
     {
-        SerializationHelper helper = new SerializationHelper(0, SerializationHelper.Flag.LOCAL);
+        SerializationHelper helper = new SerializationHelper(metadata, 0, SerializationHelper.Flag.LOCAL);
         return UnfilteredRowIterators.filter(toUnfilteredRowIterator(metadata, key, LegacyDeletionInfo.live(), cells, false, helper), nowInSec);
     }
 
-    private static LivenessInfo livenessInfo(CFMetaData metadata, LegacyCell cell)
-    {
-        return cell.isTombstone()
-             ? SimpleLivenessInfo.forDeletion(cell.timestamp, cell.localDeletionTime)
-             : SimpleLivenessInfo.forUpdate(cell.timestamp, cell.ttl, cell.localDeletionTime, metadata);
-    }
-
     public static Comparator<LegacyCell> legacyCellComparator(CFMetaData metadata)
     {
         return legacyCellComparator(metadata, false);
@@ -662,7 +647,7 @@ public abstract class LegacyLayout
             ByteBuffer value = ByteBufferUtil.readWithLength(in);
             if (flag == SerializationHelper.Flag.FROM_REMOTE || (flag == SerializationHelper.Flag.LOCAL && CounterContext.instance().shouldClearLocal(value)))
                 value = CounterContext.instance().clearAllLocal(value);
-            return new LegacyCell(LegacyCell.Kind.COUNTER, decodeCellName(metadata, cellname, readAllAsDynamic), value, ts, LivenessInfo.NO_DELETION_TIME, LivenessInfo.NO_TTL);
+            return new LegacyCell(LegacyCell.Kind.COUNTER, decodeCellName(metadata, cellname, readAllAsDynamic), value, ts, Cell.NO_DELETION_TIME, Cell.NO_TTL);
         }
         else if ((mask & EXPIRATION_MASK) != 0)
         {
@@ -678,10 +663,10 @@ public abstract class LegacyLayout
             ByteBuffer value = ByteBufferUtil.readWithLength(in);
             LegacyCellName name = decodeCellName(metadata, cellname, readAllAsDynamic);
             return (mask & COUNTER_UPDATE_MASK) != 0
-                ? new LegacyCell(LegacyCell.Kind.COUNTER, name, CounterContext.instance().createLocal(ByteBufferUtil.toLong(value)), ts, LivenessInfo.NO_DELETION_TIME, LivenessInfo.NO_TTL)
+                ? new LegacyCell(LegacyCell.Kind.COUNTER, name, CounterContext.instance().createLocal(ByteBufferUtil.toLong(value)), ts, Cell.NO_DELETION_TIME, Cell.NO_TTL)
                 : ((mask & DELETION_MASK) == 0
-                        ? new LegacyCell(LegacyCell.Kind.REGULAR, name, value, ts, LivenessInfo.NO_DELETION_TIME, LivenessInfo.NO_TTL)
-                        : new LegacyCell(LegacyCell.Kind.DELETED, name, ByteBufferUtil.EMPTY_BYTE_BUFFER, ts, ByteBufferUtil.toInt(value), LivenessInfo.NO_TTL));
+                        ? new LegacyCell(LegacyCell.Kind.REGULAR, name, value, ts, Cell.NO_DELETION_TIME, Cell.NO_TTL)
+                        : new LegacyCell(LegacyCell.Kind.DELETED, name, ByteBufferUtil.EMPTY_BYTE_BUFFER, ts, ByteBufferUtil.toInt(value), Cell.NO_TTL));
         }
     }
 
@@ -741,10 +726,9 @@ public abstract class LegacyLayout
     public static class CellGrouper
     {
         public final CFMetaData metadata;
-        private final ReusableRow row;
         private final boolean isStatic;
         private final SerializationHelper helper;
-        private Row.Writer writer;
+        private Row.Builder builder;
         private Clustering clustering;
 
         private LegacyRangeTombstone rowDeletion;
@@ -760,10 +744,7 @@ public abstract class LegacyLayout
             this.metadata = metadata;
             this.isStatic = isStatic;
             this.helper = helper;
-            this.row = isStatic ? null : new ReusableRow(metadata.clusteringColumns().size(), metadata.partitionColumns().regulars, false, metadata.isCounter());
-
-            if (isStatic)
-                this.writer = StaticRow.builder(metadata.partitionColumns().statics, false, metadata.isCounter());
+            this.builder = ArrayBackedRow.sortedBuilder(isStatic ? metadata.partitionColumns().statics : metadata.partitionColumns().regulars);
         }
 
         public static CellGrouper staticGrouper(CFMetaData metadata, SerializationHelper helper)
@@ -776,9 +757,6 @@ public abstract class LegacyLayout
             this.clustering = null;
             this.rowDeletion = null;
             this.collectionDeletion = null;
-
-            if (!isStatic)
-                this.writer = row.writer();
         }
 
         public boolean addAtom(LegacyAtom atom)
@@ -797,8 +775,8 @@ public abstract class LegacyLayout
             }
             else if (clustering == null)
             {
-                clustering = cell.name.clustering.takeAlias();
-                clustering.writeTo(writer);
+                clustering = cell.name.clustering;
+                builder.newRow(clustering);
             }
             else if (!clustering.equals(cell.name.clustering))
             {
@@ -809,14 +787,12 @@ public abstract class LegacyLayout
             if (rowDeletion != null && rowDeletion.deletionTime.deletes(cell.timestamp))
                 return true;
 
-            LivenessInfo info = livenessInfo(metadata, cell);
-
             ColumnDefinition column = cell.name.column;
             if (column == null)
             {
                 // It's the row marker
                 assert !cell.value.hasRemaining();
-                helper.writePartitionKeyLivenessInfo(writer, info.timestamp(), info.ttl(), info.localDeletionTime());
+                builder.addPrimaryKeyLivenessInfo(LivenessInfo.create(cell.timestamp, cell.ttl, cell.localDeletionTime));
             }
             else
             {
@@ -833,11 +809,15 @@ public abstract class LegacyLayout
                         // practice and 2) is only used during upgrade, it's probably worth keeping things simple.
                         helper.startOfComplexColumn(column);
                         path = cell.name.collectionElement == null ? null : CellPath.create(cell.name.collectionElement);
+                        if (!helper.includes(path))
+                            return true;
                     }
-                    helper.writeCell(writer, column, cell.isCounter(), cell.value, info.timestamp(), info.localDeletionTime(), info.ttl(), path);
+                    Cell c = new BufferCell(column, cell.timestamp, cell.ttl, cell.localDeletionTime, cell.value, path);
+                    if (!helper.isDropped(c, column.isComplex()))
+                        builder.addCell(c);
                     if (column.isComplex())
                     {
-                        helper.endOfComplexColumn(column);
+                        helper.endOfComplexColumn();
                     }
                 }
             }
@@ -852,9 +832,9 @@ public abstract class LegacyLayout
                 if (clustering != null)
                     return false;
 
-                clustering = tombstone.start.getAsClustering(metadata).takeAlias();
-                clustering.writeTo(writer);
-                writer.writeRowDeletion(tombstone.deletionTime);
+                clustering = tombstone.start.getAsClustering(metadata);
+                builder.newRow(clustering);
+                builder.addRowDeletion(tombstone.deletionTime);
                 rowDeletion = tombstone;
                 return true;
             }
@@ -863,15 +843,15 @@ public abstract class LegacyLayout
             {
                 if (clustering == null)
                 {
-                    clustering = tombstone.start.getAsClustering(metadata).takeAlias();
-                    clustering.writeTo(writer);
+                    clustering = tombstone.start.getAsClustering(metadata);
+                    builder.newRow(clustering);
                 }
                 else if (!clustering.equals(tombstone.start.getAsClustering(metadata)))
                 {
                     return false;
                 }
 
-                writer.writeComplexDeletion(tombstone.start.collectionName, tombstone.deletionTime);
+                builder.addComplexDeletion(tombstone.start.collectionName, tombstone.deletionTime);
                 if (rowDeletion == null || tombstone.deletionTime.supersedes(rowDeletion.deletionTime))
                     collectionDeletion = tombstone;
                 return true;
@@ -881,8 +861,7 @@ public abstract class LegacyLayout
 
         public Row getRow()
         {
-            writer.endOfRow();
-            return isStatic ? ((StaticRow.Builder)writer).build() : row;
+            return builder.build();
         }
     }
 
@@ -947,7 +926,7 @@ public abstract class LegacyLayout
             ByteBuffer[] values = new ByteBuffer[bound.size()];
             for (int i = 0; i < bound.size(); i++)
                 values[i] = bound.get(i);
-            return new SimpleClustering(values);
+            return new Clustering(values);
         }
 
         @Override
@@ -1005,13 +984,13 @@ public abstract class LegacyLayout
         public static LegacyCell regular(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, ByteBuffer value, long timestamp)
         throws UnknownColumnException
         {
-            return new LegacyCell(Kind.REGULAR, decodeCellName(metadata, superColumnName, name), value, timestamp, LivenessInfo.NO_DELETION_TIME, LivenessInfo.NO_TTL);
+            return new LegacyCell(Kind.REGULAR, decodeCellName(metadata, superColumnName, name), value, timestamp, Cell.NO_DELETION_TIME, Cell.NO_TTL);
         }
 
         public static LegacyCell expiring(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, ByteBuffer value, long timestamp, int ttl, int nowInSec)
         throws UnknownColumnException
         {
-            return new LegacyCell(Kind.EXPIRING, decodeCellName(metadata, superColumnName, name), value, timestamp, nowInSec, ttl);
+            return new LegacyCell(Kind.EXPIRING, decodeCellName(metadata, superColumnName, name), value, timestamp, nowInSec + ttl, ttl);
         }
 
         public static LegacyCell tombstone(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long timestamp, int nowInSec)
@@ -1030,7 +1009,7 @@ public abstract class LegacyLayout
 
         public static LegacyCell counter(LegacyCellName name, ByteBuffer value)
         {
-            return new LegacyCell(Kind.COUNTER, name, value, FBUtilities.timestampMicros(), LivenessInfo.NO_DELETION_TIME, LivenessInfo.NO_TTL);
+            return new LegacyCell(Kind.COUNTER, name, value, FBUtilities.timestampMicros(), Cell.NO_DELETION_TIME, Cell.NO_TTL);
         }
 
         public ClusteringPrefix clustering()
@@ -1205,7 +1184,7 @@ public abstract class LegacyLayout
 
         public static LegacyDeletionInfo live()
         {
-            return from(DeletionInfo.live());
+            return from(DeletionInfo.LIVE);
         }
 
         public Iterator<LegacyRangeTombstone> inRowRangeTombstones()
@@ -1228,7 +1207,7 @@ public abstract class LegacyLayout
 
                 int rangeCount = in.readInt();
                 if (rangeCount == 0)
-                    return from(new DeletionInfo(topLevel));
+                    return from(new MutableDeletionInfo(topLevel));
 
                 RangeTombstoneList ranges = new RangeTombstoneList(metadata.comparator, rangeCount);
                 List<LegacyRangeTombstone> inRowTombsones = new ArrayList<>();
@@ -1239,13 +1218,13 @@ public abstract class LegacyLayout
                     int delTime =  in.readInt();
                     long markedAt = in.readLong();
 
-                    LegacyRangeTombstone tombstone = new LegacyRangeTombstone(start, end, new SimpleDeletionTime(markedAt, delTime));
+                    LegacyRangeTombstone tombstone = new LegacyRangeTombstone(start, end, new DeletionTime(markedAt, delTime));
                     if (tombstone.isCollectionTombstone() || tombstone.isRowDeletion(metadata))
                         inRowTombsones.add(tombstone);
                     else
                         ranges.add(start.bound, end.bound, markedAt, delTime);
                 }
-                return new LegacyDeletionInfo(new DeletionInfo(topLevel, ranges), inRowTombsones);
+                return new LegacyDeletionInfo(new MutableDeletionInfo(topLevel, ranges), inRowTombsones);
             }
 
             public long serializedSize(CFMetaData metadata, LegacyDeletionInfo info, TypeSizes typeSizes, int version)


[06/15] cassandra git commit: Simplify some 8099's implementations

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/Row.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Row.java b/src/java/org/apache/cassandra/db/rows/Row.java
index 545da7a..ad21c69 100644
--- a/src/java/org/apache/cassandra/db/rows/Row.java
+++ b/src/java/org/apache/cassandra/db/rows/Row.java
@@ -17,35 +17,31 @@
  */
 package org.apache.cassandra.db.rows;
 
-import java.nio.ByteBuffer;
 import java.util.*;
 
-import com.google.common.collect.Iterators;
-
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.service.paxos.Commit;
 import org.apache.cassandra.utils.MergeIterator;
 import org.apache.cassandra.utils.SearchIterator;
 
 /**
  * Storage engine representation of a row.
  *
- * A row is identified by it's clustering column values (it's an Unfiltered),
- * has row level informations (deletion and partition key liveness infos (see below))
- * and contains data (Cells) regarding the columns it contains.
+ * A row mainly contains the following informations:
+ *   1) Its {@code Clustering}, which holds the values for the clustering columns identifying the row.
+ *   2) Its row level informations: the primary key liveness infos and the row deletion (see
+ *      {@link #primaryKeyLivenessInfo()} and {@link #deletion()} for more details).
+ *   3) Data for the columns it contains, or in other words, it's a (sorted) collection of
+ *      {@code ColumnData}.
  *
- * A row implements {@code WithLivenessInfo} and has thus a timestamp, ttl and
- * local deletion time. Those information do not apply to the row content, they
- * apply to the partition key columns. In other words, the timestamp is the
- * timestamp for the partition key columns: it is what allows to distinguish
- * between a dead row, and a live row but for which only the partition key columns
- * are set. The ttl and local deletion time information are for the case where
- * a TTL is set on those partition key columns. Note however that a row can have
- * live cells but no partition key columns timestamp, because said timestamp (and
- * its corresponding ttl) is only set on INSERT (not UPDATE).
+ * Also note that as for every other storage engine object, a {@code Row} object cannot shadow
+ * it's own data. For instance, a {@code Row} cannot contains a cell that is deleted by its own
+ * row deletion.
  */
-public interface Row extends Unfiltered, Iterable<Cell>, Aliasable<Row>
+public interface Row extends Unfiltered, Iterable<ColumnData>
 {
     /**
      * The clustering values for this row.
@@ -79,17 +75,14 @@ public interface Row extends Unfiltered, Iterable<Cell>, Aliasable<Row>
      * As a row is uniquely identified by its primary key, all its primary key columns
      * share the same {@code LivenessInfo}. This liveness information is what allows us
      * to distinguish between a dead row (it has no live cells and its primary key liveness
-     * info has no timestamp) and a live row but where all non PK columns are null (it has no
-     * live cells, but its primary key liveness has a timestamp). Please note that the ttl
-     * (and local deletion time) of the PK liveness information only apply to the
-     * liveness info timestamp, and not to the content of the row. Also note that because
-     * in practice there is not way to only delete the primary key columns (without deleting
-     * the row itself), the returned {@code LivenessInfo} can only have a local deletion time
-     * if it has a TTL.
+     * info is empty) and a live row but where all non PK columns are null (it has no
+     * live cells, but its primary key liveness is not empty). Please note that the liveness
+     * info (including it's eventually ttl/local deletion time) only apply to the primary key
+     * columns and has no impact on the row content.
      * <p>
-     * Lastly, note that it is possible for a row to have live cells but no PK liveness
-     * info timestamp, because said timestamp is only set on {@code INSERT} (which makes sense
-     * in itself, see #6782) but live cells can be add through {@code UPDATE} even if the row
+     * Note in particular that a row may have live cells but no PK liveness info, because the
+     * primary key liveness informations are only set on {@code INSERT} (which makes sense
+     * in itself, see #6782) but live cells can be added through {@code UPDATE} even if the row
      * wasn't pre-existing (which users are encouraged not to do, but we can't validate).
      */
     public LivenessInfo primaryKeyLivenessInfo();
@@ -102,10 +95,10 @@ public interface Row extends Unfiltered, Iterable<Cell>, Aliasable<Row>
     public boolean isStatic();
 
     /**
-     * Whether the row has no information whatsoever. This means no row infos
-     * (timestamp, ttl, deletion), no cells and no complex deletion info.
+     * Whether the row has no information whatsoever. This means no PK liveness info, no row
+     * deletion, no cells and no complex deletion info.
      *
-     * @return {@code true} if the row has no data whatsoever, {@code false} otherwise.
+     * @return {@code true} if the row has no data, {@code false} otherwise.
      */
     public boolean isEmpty();
 
@@ -115,20 +108,8 @@ public interface Row extends Unfiltered, Iterable<Cell>, Aliasable<Row>
     public boolean hasLiveData(int nowInSec);
 
     /**
-     * Whether or not this row contains any deletion for a complex column. That is if
-     * there is at least one column for which {@code getDeletion} returns a non
-     * live deletion time.
-     */
-    public boolean hasComplexDeletion();
-
-    /**
      * Returns a cell for a simple column.
      *
-     * Calls to this method are allowed to return the same Cell object, and hence the returned
-     * object is only valid until the next getCell/getCells call on the same Row object. You will need
-     * to copy the returned data if you plan on using a reference to the Cell object
-     * longer than that.
-     *
      * @param c the simple column for which to fetch the cell.
      * @return the corresponding cell or {@code null} if the row has no such cell.
      */
@@ -137,11 +118,6 @@ public interface Row extends Unfiltered, Iterable<Cell>, Aliasable<Row>
     /**
      * Return a cell for a given complex column and cell path.
      *
-     * Calls to this method are allowed to return the same Cell object, and hence the returned
-     * object is only valid until the next getCell/getCells call on the same Row object. You will need
-     * to copy the returned data if you plan on using a reference to the Cell object
-     * longer than that.
-     *
      * @param c the complex column for which to fetch the cell.
      * @param path the cell path for which to fetch the cell.
      * @return the corresponding cell or {@code null} if the row has no such cell.
@@ -149,43 +125,35 @@ public interface Row extends Unfiltered, Iterable<Cell>, Aliasable<Row>
     public Cell getCell(ColumnDefinition c, CellPath path);
 
     /**
-     * Returns an iterator on the cells of a complex column c.
-     *
-     * Calls to this method are allowed to return the same iterator object, and
-     * hence the returned object is only valid until the next getCell/getCells call
-     * on the same Row object. You will need to copy the returned data if you
-     * plan on using a reference to the Cell object longer than that.
+     * The data for a complex column.
+     * <p>
+     * The returned object groups all the cells for the column, as well as it's complex deletion (if relevant).
      *
-     * @param c the complex column for which to fetch the cells.
-     * @return an iterator on the cells of complex column {@code c} or {@code null} if the row has no
-     * cells for that column.
+     * @param c the complex column for which to return the complex data.
+     * @return the data for {@code c} or {@code null} is the row has no data for this column.
      */
-    public Iterator<Cell> getCells(ColumnDefinition c);
+    public ComplexColumnData getComplexColumnData(ColumnDefinition c);
 
     /**
-     * Deletion informations for complex columns.
+     * An iterable over the cells of this row.
+     * <p>
+     * The iterable guarantees that cells are returned in order of {@link Cell#comparator}.
      *
-     * @param c the complex column for which to fetch deletion info.
-     * @return the deletion time for complex column {@code c} in this row.
+     * @return an iterable over the cells of this row.
      */
-    public DeletionTime getDeletion(ColumnDefinition c);
+    public Iterable<Cell> cells();
 
     /**
-     * An iterator over the cells of this row.
-     *
-     * The iterator guarantees that for 2 rows of the same partition, columns
-     * are returned in a consistent order in the sense that if the cells for
-     * column c1 is returned before the cells for column c2 by the first iterator,
-     * it is also the case for the 2nd iterator.
-     *
-     * The object returned by a call to next() is only guaranteed to be valid until
-     * the next call to hasNext() or next(). If a consumer wants to keep a
-     * reference on the returned Cell objects for longer than the iteration, it must
-     * make a copy of it explicitly.
+     * Whether the row stores any (non-live) complex deletion for any complex column.
+     */
+    public boolean hasComplexDeletion();
+
+    /**
+     * Whether the row has any deletion info (row deletion, cell tombstone, expired cell or complex deletion).
      *
-     * @return an iterator over the cells of this row.
+     * @param nowInSec the current time in seconds to decid if a cell is expired.
      */
-    public Iterator<Cell> iterator();
+    public boolean hasDeletion(int nowInSec);
 
     /**
      * An iterator to efficiently search data for a given column.
@@ -195,134 +163,167 @@ public interface Row extends Unfiltered, Iterable<Cell>, Aliasable<Row>
     public SearchIterator<ColumnDefinition, ColumnData> searchIterator();
 
     /**
-     * Copy this row to the provided writer.
+     * Returns a copy of this row that:
+     *   1) only includes the data for the column included by {@code filter}.
+     *   2) doesn't include any data that belongs to a dropped column (recorded in {@code metadata}).
+     */
+    public Row filter(ColumnFilter filter, CFMetaData metadata);
+
+    /**
+     * Returns a copy of this row that:
+     *   1) only includes the data for the column included by {@code filter}.
+     *   2) doesn't include any data that belongs to a dropped column (recorded in {@code metadata}).
+     *   3) doesn't include any data that is shadowed/deleted by {@code activeDeletion}.
+     *   4) uses {@code activeDeletion} as row deletion iff {@code setActiveDeletionToRow} and {@code activeDeletion} supersedes the row deletion.
+     */
+    public Row filter(ColumnFilter filter, DeletionTime activeDeletion, boolean setActiveDeletionToRow, CFMetaData metadata);
+
+    /**
+     * Returns a copy of this row without any deletion info that should be purged according to {@code purger}.
      *
-     * @param writer the row writer to write this row to.
+     * @param purger the {@code DeletionPurger} to use to decide what can be purged.
+     * @param nowInSec the current time to decide what is deleted and what isn't (in the case of expired cells).
+     * @return this row but without any deletion info purged by {@code purger}.
+     */
+    public Row purge(DeletionPurger purger, int nowInSec);
+
+    /**
+     * Returns a copy of this row where all counter cells have they "local" shard marked for clearing.
+     */
+    public Row markCounterLocalToBeCleared();
+
+    /**
+     * returns a copy of this row where all live timestamp have been replaced by {@code newTimestamp} and every deletion timestamp
+     * by {@code newTimestamp - 1}. See {@link Commit} for why we need this.
      */
-    public void copyTo(Row.Writer writer);
+    public Row updateAllTimestamp(long newTimestamp);
+
+    public int dataSize();
+
+    public long unsharedHeapSizeExcludingData();
 
     public String toString(CFMetaData metadata, boolean fullDetails);
 
     /**
-     * Interface for writing a row.
+     * Interface for building rows.
      * <p>
-     * Clients of this interface should abid to the following assumptions:
-     *   1) if the row has a non empty clustering (it's not a static one and it doesn't belong to a table without
-     *      clustering columns), then that clustering should be the first thing written (through
-     *      {@link ClusteringPrefix.Writer#writeClusteringValue})).
-     *   2) for a given complex column, calls to {@link #writeCell} are performed consecutively (without
-     *      any call to {@code writeCell} for another column intermingled) and in {@code CellPath} order.
-     *   3) {@link #endOfRow} is always called to end the writing of a given row.
+     * The builder of a row should always abid to the following rules:
+     *   1) {@link #newRow} is always called as the first thing for the row.
+     *   2) {@link #addPrimaryKeyLivenessInfo} and {@link #addRowDeletion}, if called, are called before
+     *      any {@link #addCell}/{@link #addComplexDeletion} call.
+     *   3) {@link #build} is called to construct the new row. The builder can then be reused.
+     *
+     * There is 2 variants of a builder: sorted and unsorted ones. A sorted builder expects user to abid to the
+     * following additional rules:
+     *   4) Calls to {@link #addCell}/{@link #addComplexDeletion} are done in strictly increasing column order.
+     *      In other words, all calls to these methods for a give column {@code c} are done after any call for
+     *      any column before {@code c} and before any call for any column after {@code c}.
+     *   5) Calls to {@link #addCell} are further done in strictly increasing cell order (the one defined by
+     *      {@link Cell#comparator}. That is, for a give column, cells are passed in {@code CellPath} order.
+     *
+     * An unsorted builder will not expect those last rules however: {@link #addCell} and {@link #addComplexDeletion}
+     * can be done in any order. And in particular unsorted builder allows multiple calls for the same column/cell. In
+     * that latter case, the result will follow the usual reconciliation rules (so equal cells are reconciled with
+     * {@link Cells#reconcile} and the "biggest" of multiple complex deletion for the same column wins).
      */
-    public interface Writer extends ClusteringPrefix.Writer
+    public interface Builder
     {
         /**
-         * Writes the livness information for the partition key columns of this row.
+         * Whether the builder is a sorted one or not.
+         *
+         * @return if the builder requires calls to be done in sorted order or not (see above).
+         */
+        public boolean isSorted();
+
+        /**
+         * Prepares the builder to build a new row of clustering {@code clustering}.
+         * <p>
+         * This should always be the first call for a given row.
          *
-         * This call is optional: skipping it is equivalent to calling {@code writePartitionKeyLivenessInfo(LivenessInfo.NONE)}.
+         * @param clustering the clustering for the new row.
+         */
+        public void newRow(Clustering clustering);
+
+        /**
+         * The clustering for the row that is currently being built.
          *
-         * @param info the liveness information for the partition key columns of the written row.
+         * @return the clustering for the row that is currently being built, or {@code null} if {@link #newRow} hasn't
+         * yet been called.
          */
-        public void writePartitionKeyLivenessInfo(LivenessInfo info);
+        public Clustering clustering();
 
         /**
-         * Writes the deletion information for this row.
+         * Adds the liveness information for the partition key columns of this row.
+         *
+         * This call is optional (skipping it is equivalent to calling {@code addPartitionKeyLivenessInfo(LivenessInfo.NONE)}).
+         *
+         * @param info the liveness information for the partition key columns of the built row.
+         */
+        public void addPrimaryKeyLivenessInfo(LivenessInfo info);
+
+        /**
+         * Adds the deletion information for this row.
          *
          * This call is optional and can be skipped if the row is not deleted.
          *
          * @param deletion the row deletion time, or {@code DeletionTime.LIVE} if the row isn't deleted.
          */
-        public void writeRowDeletion(DeletionTime deletion);
+        public void addRowDeletion(DeletionTime deletion);
 
         /**
-         * Writes a cell to the writer.
+         * Adds a cell to this builder.
          *
-         * As mentionned above, add cells for a given column should be added consecutively (and in {@code CellPath} order for complex columns).
-         *
-         * @param column the column for the written cell.
-         * @param isCounter whether or not this is a counter cell.
-         * @param value the value for the cell. For tombstones, which don't have values, this should be an empty buffer.
-         * @param info the cell liveness information.
-         * @param path the {@link CellPath} for complex cells and {@code null} for regular cells.
+         * @param cell the cell to add.
          */
-        public void writeCell(ColumnDefinition column, boolean isCounter, ByteBuffer value, LivenessInfo info, CellPath path);
+        public void addCell(Cell cell);
 
         /**
-         * Writes a deletion for a complex column, that is one that apply to all cells of the complex column.
+         * Adds a complex deletion.
          *
-         * @param column the (complex) column this is a deletion for.
-         * @param complexDeletion the deletion time.
+         * @param column the column for which to add the {@code complexDeletion}.
+         * @param complexDeletion the complex deletion time to add.
          */
-        public void writeComplexDeletion(ColumnDefinition column, DeletionTime complexDeletion);
+        public void addComplexDeletion(ColumnDefinition column, DeletionTime complexDeletion);
 
         /**
-         * Should be called to indicates that the row has been fully written.
+         * Builds and return built row.
+         *
+         * @return the last row built by this builder.
          */
-        public void endOfRow();
+        public Row build();
     }
 
     /**
      * Utility class to help merging rows from multiple inputs (UnfilteredRowIterators).
      */
-    public abstract static class Merger
+    public static class Merger
     {
-        private final CFMetaData metadata;
-        private final int nowInSec;
-        private final UnfilteredRowIterators.MergeListener listener;
         private final Columns columns;
+        private final Row[] rows;
+        private final List<Iterator<ColumnData>> columnDataIterators;
 
         private Clustering clustering;
-        private final Row[] rows;
         private int rowsToMerge;
+        private int lastRowSet = -1;
 
-        private LivenessInfo rowInfo = LivenessInfo.NONE;
-        private DeletionTime rowDeletion = DeletionTime.LIVE;
-
-        private final Cell[] cells;
-        private final List<Iterator<Cell>> complexCells;
-        private final ComplexColumnReducer complexReducer = new ComplexColumnReducer();
+        private final List<ColumnData> dataBuffer = new ArrayList<>();
+        private final ColumnDataReducer columnDataReducer;
 
-        // For the sake of the listener if there is one
-        private final DeletionTime[] complexDelTimes;
-
-        private boolean signaledListenerForRow;
-
-        public static Merger createStatic(CFMetaData metadata, int size, int nowInSec, Columns columns, UnfilteredRowIterators.MergeListener listener)
-        {
-            return new StaticMerger(metadata, size, nowInSec, columns, listener);
-        }
-
-        public static Merger createRegular(CFMetaData metadata, int size, int nowInSec, Columns columns, UnfilteredRowIterators.MergeListener listener)
+        public Merger(int size, int nowInSec, Columns columns)
         {
-            return new RegularMerger(metadata, size, nowInSec, columns, listener);
-        }
-
-        protected Merger(CFMetaData metadata, int size, int nowInSec, Columns columns, UnfilteredRowIterators.MergeListener listener)
-        {
-            this.metadata = metadata;
-            this.nowInSec = nowInSec;
-            this.listener = listener;
             this.columns = columns;
             this.rows = new Row[size];
-            this.complexCells = new ArrayList<>(size);
-
-            this.cells = new Cell[size];
-            this.complexDelTimes = listener == null ? null : new DeletionTime[size];
+            this.columnDataIterators = new ArrayList<>(size);
+            this.columnDataReducer = new ColumnDataReducer(size, nowInSec, columns.hasComplex());
         }
 
         public void clear()
         {
+            dataBuffer.clear();
             Arrays.fill(rows, null);
-            Arrays.fill(cells, null);
-            if (complexDelTimes != null)
-                Arrays.fill(complexDelTimes, null);
-            complexCells.clear();
+            columnDataIterators.clear();
             rowsToMerge = 0;
-
-            rowInfo = LivenessInfo.NONE;
-            rowDeletion = DeletionTime.LIVE;
-
-            signaledListenerForRow = false;
+            lastRowSet = -1;
         }
 
         public void add(int i, Row row)
@@ -330,225 +331,187 @@ public interface Row extends Unfiltered, Iterable<Cell>, Aliasable<Row>
             clustering = row.clustering();
             rows[i] = row;
             ++rowsToMerge;
+            lastRowSet = i;
         }
 
-        protected abstract Row.Writer getWriter();
-        protected abstract Row getRow();
-
         public Row merge(DeletionTime activeDeletion)
         {
             // If for this clustering we have only one row version and have no activeDeletion (i.e. nothing to filter out),
-            // then we can just return that single row (we also should have no listener)
-            if (rowsToMerge == 1 && activeDeletion.isLive() && listener == null)
+            // then we can just return that single row
+            if (rowsToMerge == 1 && activeDeletion.isLive())
             {
-                for (int i = 0; i < rows.length; i++)
-                    if (rows[i] != null)
-                        return rows[i];
-                throw new AssertionError();
+                Row row = rows[lastRowSet];
+                assert row != null;
+                return row;
             }
 
-            Row.Writer writer = getWriter();
-            Rows.writeClustering(clustering, writer);
-
-            for (int i = 0; i < rows.length; i++)
+            LivenessInfo rowInfo = LivenessInfo.EMPTY;
+            DeletionTime rowDeletion = DeletionTime.LIVE;
+            for (Row row : rows)
             {
-                if (rows[i] == null)
+                if (row == null)
                     continue;
 
-                rowInfo = rowInfo.mergeWith(rows[i].primaryKeyLivenessInfo());
-
-                if (rows[i].deletion().supersedes(rowDeletion))
-                    rowDeletion = rows[i].deletion();
+                if (row.primaryKeyLivenessInfo().supersedes(rowInfo))
+                    rowInfo = row.primaryKeyLivenessInfo();
+                if (row.deletion().supersedes(rowDeletion))
+                    rowDeletion = row.deletion();
             }
 
-            if (rowDeletion.supersedes(activeDeletion))
+            if (activeDeletion.supersedes(rowDeletion))
+                rowDeletion = DeletionTime.LIVE;
+            else
                 activeDeletion = rowDeletion;
 
             if (activeDeletion.deletes(rowInfo))
-                rowInfo = LivenessInfo.NONE;
+                rowInfo = LivenessInfo.EMPTY;
 
-            writer.writePartitionKeyLivenessInfo(rowInfo);
-            writer.writeRowDeletion(rowDeletion);
+            for (Row row : rows)
+                columnDataIterators.add(row == null ? Collections.emptyIterator() : row.iterator());
 
-            for (int i = 0; i < columns.simpleColumnCount(); i++)
+            columnDataReducer.setActiveDeletion(activeDeletion);
+            Iterator<ColumnData> merged = MergeIterator.get(columnDataIterators, ColumnData.comparator, columnDataReducer);
+            while (merged.hasNext())
             {
-                ColumnDefinition c = columns.getSimple(i);
-                for (int j = 0; j < rows.length; j++)
-                    cells[j] = rows[j] == null ? null : rows[j].getCell(c);
-
-                reconcileCells(activeDeletion, writer);
+                ColumnData data = merged.next();
+                if (data != null)
+                    dataBuffer.add(data);
             }
 
-            complexReducer.activeDeletion = activeDeletion;
-            complexReducer.writer = writer;
-            for (int i = 0; i < columns.complexColumnCount(); i++)
-            {
-                ColumnDefinition c = columns.getComplex(i);
-
-                DeletionTime maxComplexDeletion = DeletionTime.LIVE;
-                for (int j = 0; j < rows.length; j++)
-                {
-                    if (rows[j] == null)
-                        continue;
-
-                    DeletionTime dt = rows[j].getDeletion(c);
-                    if (complexDelTimes != null)
-                        complexDelTimes[j] = dt;
-
-                    if (dt.supersedes(maxComplexDeletion))
-                        maxComplexDeletion = dt;
-                }
-
-                boolean overrideActive = maxComplexDeletion.supersedes(activeDeletion);
-                maxComplexDeletion =  overrideActive ? maxComplexDeletion : DeletionTime.LIVE;
-                writer.writeComplexDeletion(c, maxComplexDeletion);
-                if (listener != null)
-                    listener.onMergedComplexDeletion(c, maxComplexDeletion, complexDelTimes);
-
-                mergeComplex(overrideActive ? maxComplexDeletion : activeDeletion, c);
-            }
-            writer.endOfRow();
-
-            Row row = getRow();
-            // Because shadowed cells are skipped, the row could be empty. In which case
-            // we return null (we also don't want to signal anything in that case since that
-            // means everything in the row was shadowed and the listener will have been signalled
-            // for whatever shadows it).
-            if (row.isEmpty())
-                return null;
-
-            maybeSignalEndOfRow();
-            return row;
+            // Because some data might have been shadowed by the 'activeDeletion', we could have an empty row
+            return rowInfo.isEmpty() && rowDeletion.isLive() && dataBuffer.isEmpty()
+                 ? null
+                 : ArrayBackedRow.create(clustering, columns, rowInfo, rowDeletion, dataBuffer.size(), dataBuffer.toArray(new ColumnData[dataBuffer.size()]));
         }
 
-        private void maybeSignalListenerForRow()
+        public Clustering mergedClustering()
         {
-            if (listener != null && !signaledListenerForRow)
-            {
-                listener.onMergingRows(clustering, rowInfo, rowDeletion, rows);
-                signaledListenerForRow = true;
-            }
+            return clustering;
         }
 
-        private void maybeSignalListenerForCell(Cell merged, Cell[] versions)
+        public Row[] mergedRows()
         {
-            if (listener != null)
-            {
-                maybeSignalListenerForRow();
-                listener.onMergedCells(merged, versions);
-            }
+            return rows;
         }
 
-        private void maybeSignalEndOfRow()
+        private static class ColumnDataReducer extends MergeIterator.Reducer<ColumnData, ColumnData>
         {
-            if (listener != null)
-            {
-                // If we haven't signaled the listener yet (we had no cells but some deletion info), do it now
-                maybeSignalListenerForRow();
-                listener.onRowDone();
-            }
-        }
+            private final int nowInSec;
 
-        private void reconcileCells(DeletionTime activeDeletion, Row.Writer writer)
-        {
-            Cell reconciled = null;
-            for (int j = 0; j < cells.length; j++)
-            {
-                Cell cell = cells[j];
-                if (cell != null && !activeDeletion.deletes(cell.livenessInfo()))
-                    reconciled = Cells.reconcile(reconciled, cell, nowInSec);
-            }
+            private ColumnDefinition column;
+            private final List<ColumnData> versions;
 
-            if (reconciled != null)
+            private DeletionTime activeDeletion;
+
+            private final ComplexColumnData.Builder complexBuilder;
+            private final List<Iterator<Cell>> complexCells;
+            private final CellReducer cellReducer;
+
+            public ColumnDataReducer(int size, int nowInSec, boolean hasComplex)
             {
-                reconciled.writeTo(writer);
-                maybeSignalListenerForCell(reconciled, cells);
+                this.nowInSec = nowInSec;
+                this.versions = new ArrayList<>(size);
+                this.complexBuilder = hasComplex ? ComplexColumnData.builder() : null;
+                this.complexCells = hasComplex ? new ArrayList<>(size) : null;
+                this.cellReducer = new CellReducer(nowInSec);
             }
-        }
 
-        private void mergeComplex(DeletionTime activeDeletion, ColumnDefinition c)
-        {
-            complexCells.clear();
-            for (int j = 0; j < rows.length; j++)
+            public void setActiveDeletion(DeletionTime activeDeletion)
             {
-                Row row = rows[j];
-                Iterator<Cell> iter = row == null ? null : row.getCells(c);
-                complexCells.add(iter == null ? Iterators.<Cell>emptyIterator() : iter);
+                this.activeDeletion = activeDeletion;
             }
 
-            complexReducer.column = c;
-            complexReducer.activeDeletion = activeDeletion;
-
-            // Note that we use the mergeIterator only to group cells to merge, but we
-            // write the result to the writer directly in the reducer, so all we care
-            // about is iterating over the result.
-            Iterator<Void> iter = MergeIterator.get(complexCells, c.cellComparator(), complexReducer);
-            while (iter.hasNext())
-                iter.next();
-        }
-
-        private class ComplexColumnReducer extends MergeIterator.Reducer<Cell, Void>
-        {
-            private DeletionTime activeDeletion;
-            private Row.Writer writer;
-            private ColumnDefinition column;
-
-            public void reduce(int idx, Cell current)
+            public void reduce(int idx, ColumnData data)
             {
-                cells[idx] = current;
+                column = data.column();
+                versions.add(data);
             }
 
-            protected Void getReduced()
+            protected ColumnData getReduced()
             {
-                reconcileCells(activeDeletion, writer);
-                return null;
+                if (column.isSimple())
+                {
+                    Cell merged = null;
+                    for (ColumnData data : versions)
+                    {
+                        Cell cell = (Cell)data;
+                        if (!activeDeletion.deletes(cell))
+                            merged = merged == null ? cell : Cells.reconcile(merged, cell, nowInSec);
+                    }
+                    return merged;
+                }
+                else
+                {
+                    complexBuilder.newColumn(column);
+                    complexCells.clear();
+                    DeletionTime complexDeletion = DeletionTime.LIVE;
+                    for (ColumnData data : versions)
+                    {
+                        ComplexColumnData cd = (ComplexColumnData)data;
+                        if (cd.complexDeletion().supersedes(complexDeletion))
+                            complexDeletion = cd.complexDeletion();
+                        complexCells.add(cd.iterator());
+                    }
+
+                    if (complexDeletion.supersedes(activeDeletion))
+                    {
+                        cellReducer.setActiveDeletion(complexDeletion);
+                        complexBuilder.addComplexDeletion(complexDeletion);
+                    }
+                    else
+                    {
+                        cellReducer.setActiveDeletion(activeDeletion);
+                    }
+
+                    Iterator<Cell> cells = MergeIterator.get(complexCells, ColumnData.cellComparator, cellReducer);
+                    while (cells.hasNext())
+                    {
+                        Cell merged = cells.next();
+                        if (merged != null)
+                            complexBuilder.addCell(merged);
+                    }
+                    return complexBuilder.build();
+                }
             }
 
             protected void onKeyChange()
             {
-                Arrays.fill(cells, null);
+                versions.clear();
             }
         }
 
-        private static class StaticMerger extends Merger
+        private static class CellReducer extends MergeIterator.Reducer<Cell, Cell>
         {
-            private final StaticRow.Builder builder;
+            private final int nowInSec;
 
-            private StaticMerger(CFMetaData metadata, int size, int nowInSec, Columns columns, UnfilteredRowIterators.MergeListener listener)
-            {
-                super(metadata, size, nowInSec, columns, listener);
-                this.builder = StaticRow.builder(columns, true, metadata.isCounter());
-            }
+            private DeletionTime activeDeletion;
+            private Cell merged;
 
-            protected Row.Writer getWriter()
+            public CellReducer(int nowInSec)
             {
-                return builder;
+                this.nowInSec = nowInSec;
             }
 
-            protected Row getRow()
+            public void setActiveDeletion(DeletionTime activeDeletion)
             {
-                return builder.build();
+                this.activeDeletion = activeDeletion;
+                onKeyChange();
             }
-        }
-
-        private static class RegularMerger extends Merger
-        {
-            private final ReusableRow row;
 
-            private RegularMerger(CFMetaData metadata, int size, int nowInSec, Columns columns, UnfilteredRowIterators.MergeListener listener)
+            public void reduce(int idx, Cell cell)
             {
-                super(metadata, size, nowInSec, columns, listener);
-                this.row = new ReusableRow(metadata.clusteringColumns().size(), columns, true, metadata.isCounter());
+                if (!activeDeletion.deletes(cell))
+                    merged = merged == null ? cell : Cells.reconcile(merged, cell, nowInSec);
             }
 
-            protected Row.Writer getWriter()
+            protected Cell getReduced()
             {
-                return row.writer();
+                return merged;
             }
 
-            protected Row getRow()
+            protected void onKeyChange()
             {
-                return row;
+                merged = null;
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java b/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
new file mode 100644
index 0000000..2a10199
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
@@ -0,0 +1,191 @@
+/*
+ * 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.rows;
+
+import java.util.Comparator;
+import java.util.Iterator;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.filter.ColumnFilter;
+
+/**
+ * An iterator that merges a source of rows with the range tombstone and partition level deletion of a give partition.
+ * <p>
+ * This is used by our {@code Partition} implementations to produce a {@code UnfilteredRowIterator} by merging the rows
+ * and deletion infos that are kept separate. This has also 2 additional role:
+ *   1) this make sure the row returned only includes the columns selected for the resulting iterator.
+ *   2) this (optionally) remove any data that can be shadowed (see commet on 'removeShadowedData' below for more details)
+ */
+public class RowAndDeletionMergeIterator extends AbstractUnfilteredRowIterator
+{
+    // For some of our Partition implementation, we can't guarantee that the deletion information (partition level
+    // deletion and range tombstones) don't shadow data in the rows. If that is the case, this class also take
+    // cares of skipping such shadowed data (since it is the contract of an UnfilteredRowIterator that it doesn't
+    // shadow its own data). Sometimes however, we know this can't happen, in which case we can skip that step.
+    private final boolean removeShadowedData;
+    private final Comparator<Clusterable> comparator;
+    private final ColumnFilter selection;
+
+    private final Iterator<Row> rows;
+    private Row nextRow;
+
+    private final Iterator<RangeTombstone> ranges;
+    private RangeTombstone nextRange;
+
+    // The currently open tombstone. Note that unless this is null, there is no point in checking nextRange.
+    private RangeTombstone openRange;
+
+    public RowAndDeletionMergeIterator(CFMetaData metadata,
+                                       DecoratedKey partitionKey,
+                                       DeletionTime partitionLevelDeletion,
+                                       ColumnFilter selection,
+                                       Row staticRow,
+                                       boolean isReversed,
+                                       RowStats stats,
+                                       Iterator<Row> rows,
+                                       Iterator<RangeTombstone> ranges,
+                                       boolean removeShadowedData)
+    {
+        super(metadata, partitionKey, partitionLevelDeletion, selection.fetchedColumns(), staticRow, isReversed, stats);
+        this.comparator = isReversed ? metadata.comparator.reversed() : metadata.comparator;
+        this.selection = selection;
+        this.removeShadowedData = removeShadowedData;
+        this.rows = rows;
+        this.ranges = ranges;
+    }
+
+    protected Unfiltered computeNext()
+    {
+        while (true)
+        {
+            updateNextRow();
+            if (nextRow == null)
+            {
+                if (openRange != null)
+                    return closeOpenedRange();
+
+                updateNextRange();
+                return nextRange == null ? endOfData() : openRange();
+            }
+
+            // We have a next row
+
+            if (openRange == null)
+            {
+                // We have no currently open tombstone range. So check if we have a next range and if it sorts before this row.
+                // If it does, the opening of that range should go first. Otherwise, the row goes first.
+                updateNextRange();
+                if (nextRange != null && comparator.compare(openBound(nextRange), nextRow.clustering()) < 0)
+                    return openRange();
+
+                Row row = consumeNextRow();
+                // it's possible for the row to be fully shadowed by the current range tombstone
+                if (row != null)
+                    return row;
+            }
+            else
+            {
+                // We have both a next row and a currently opened tombstone. Check which goes first between the range closing and the row.
+                if (comparator.compare(closeBound(openRange), nextRow.clustering()) < 0)
+                    return closeOpenedRange();
+
+                Row row = consumeNextRow();
+                if (row != null)
+                    return row;
+            }
+        }
+    }
+
+    private void updateNextRow()
+    {
+        if (nextRow == null && rows.hasNext())
+            nextRow = rows.next();
+    }
+
+    private void updateNextRange()
+    {
+        while (nextRange == null && ranges.hasNext())
+        {
+            nextRange = ranges.next();
+            if (removeShadowedData && partitionLevelDeletion().supersedes(nextRange.deletionTime()))
+                nextRange = null;
+        }
+    }
+
+    private Row consumeNextRow()
+    {
+        Row row = nextRow;
+        nextRow = null;
+        if (!removeShadowedData)
+            return row.filter(selection, metadata());
+
+        DeletionTime activeDeletion = openRange == null ? partitionLevelDeletion() : openRange.deletionTime();
+        return row.filter(selection, activeDeletion, false, metadata());
+    }
+
+    private RangeTombstone consumeNextRange()
+    {
+        RangeTombstone range = nextRange;
+        nextRange = null;
+        return range;
+    }
+
+    private RangeTombstone consumeOpenRange()
+    {
+        RangeTombstone range = openRange;
+        openRange = null;
+        return range;
+    }
+
+    private Slice.Bound openBound(RangeTombstone range)
+    {
+        return range.deletedSlice().open(isReverseOrder());
+    }
+
+    private Slice.Bound closeBound(RangeTombstone range)
+    {
+        return range.deletedSlice().close(isReverseOrder());
+    }
+
+    private RangeTombstoneMarker closeOpenedRange()
+    {
+        // Check if that close if actually a boundary between markers
+        updateNextRange();
+        RangeTombstoneMarker marker;
+        if (nextRange != null && comparator.compare(closeBound(openRange), openBound(nextRange)) == 0)
+        {
+            marker = RangeTombstoneBoundaryMarker.makeBoundary(isReverseOrder(), closeBound(openRange), openBound(nextRange), openRange.deletionTime(), nextRange.deletionTime());
+            openRange = consumeNextRange();
+        }
+        else
+        {
+            RangeTombstone toClose = consumeOpenRange();
+            marker = new RangeTombstoneBoundMarker(closeBound(toClose), toClose.deletionTime());
+        }
+        return marker;
+    }
+
+    private RangeTombstoneMarker openRange()
+    {
+        assert openRange == null && nextRange != null;
+        openRange = consumeNextRange();
+        return new RangeTombstoneBoundMarker(openBound(openRange), openRange.deletionTime());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/RowAndTombstoneMergeIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowAndTombstoneMergeIterator.java b/src/java/org/apache/cassandra/db/rows/RowAndTombstoneMergeIterator.java
deleted file mode 100644
index 3d204d3..0000000
--- a/src/java/org/apache/cassandra/db/rows/RowAndTombstoneMergeIterator.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * 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.rows;
-
-import java.util.Comparator;
-import java.util.Iterator;
-
-import com.google.common.collect.PeekingIterator;
-import com.google.common.collect.UnmodifiableIterator;
-
-import org.apache.cassandra.db.*;
-
-public class RowAndTombstoneMergeIterator extends UnmodifiableIterator<Unfiltered> implements PeekingIterator<Unfiltered>
-{
-    private final Comparator<Clusterable> comparator;
-    private final boolean reversed;
-
-    private Iterator<Row> rowIter;
-    private Row nextRow;
-
-    private Iterator<RangeTombstone> tombstoneIter;
-    private RangeTombstone nextTombstone;
-    private boolean inTombstone;
-
-    private Unfiltered next;
-
-    public RowAndTombstoneMergeIterator(ClusteringComparator comparator, boolean reversed)
-    {
-        this.comparator = reversed ? comparator.reversed() : comparator;
-        this.reversed = reversed;
-    }
-
-    public RowAndTombstoneMergeIterator setTo(Iterator<Row> rowIter, Iterator<RangeTombstone> tombstoneIter)
-    {
-        this.rowIter = rowIter;
-        this.tombstoneIter = tombstoneIter;
-        this.nextRow = null;
-        this.nextTombstone = null;
-        this.next = null;
-        this.inTombstone = false;
-        return this;
-    }
-
-    public boolean isSet()
-    {
-        return rowIter != null;
-    }
-
-    private void prepareNext()
-    {
-        if (next != null)
-            return;
-
-        if (nextTombstone == null && tombstoneIter.hasNext())
-            nextTombstone = tombstoneIter.next();
-        if (nextRow == null && rowIter.hasNext())
-            nextRow = rowIter.next();
-
-        if (nextTombstone == null)
-        {
-            if (nextRow == null)
-                return;
-
-            next = nextRow;
-            nextRow = null;
-        }
-        else if (nextRow == null)
-        {
-            if (inTombstone)
-            {
-                RangeTombstone rt = nextTombstone;
-                nextTombstone = tombstoneIter.hasNext() ? tombstoneIter.next() : null;
-                // An end and a start makes a boundary if they sort similarly
-                if (nextTombstone != null
-                        && comparator.compare(rt.deletedSlice().close(reversed), nextTombstone.deletedSlice().open(reversed)) == 0)
-                {
-                    next = RangeTombstoneBoundaryMarker.makeBoundary(reversed,
-                                                                     rt.deletedSlice().close(reversed),
-                                                                     nextTombstone.deletedSlice().open(reversed),
-                                                                     rt.deletionTime(),
-                                                                     nextTombstone.deletionTime());
-                }
-                else
-                {
-                    inTombstone = false;
-                    next = new RangeTombstoneBoundMarker(rt.deletedSlice().close(reversed), rt.deletionTime());
-                }
-            }
-            else
-            {
-                inTombstone = true;
-                next = new RangeTombstoneBoundMarker(nextTombstone.deletedSlice().open(reversed), nextTombstone.deletionTime());
-            }
-        }
-        else if (inTombstone)
-        {
-            if (comparator.compare(nextTombstone.deletedSlice().close(reversed), nextRow.clustering()) < 0)
-            {
-                RangeTombstone rt = nextTombstone;
-                nextTombstone = tombstoneIter.hasNext() ? tombstoneIter.next() : null;
-                if (nextTombstone != null
-                        && comparator.compare(rt.deletedSlice().close(reversed), nextTombstone.deletedSlice().open(reversed)) == 0)
-                {
-                    next = RangeTombstoneBoundaryMarker.makeBoundary(reversed,
-                                                                     rt.deletedSlice().close(reversed),
-                                                                     nextTombstone.deletedSlice().open(reversed),
-                                                                     rt.deletionTime(),
-                                                                     nextTombstone.deletionTime());
-                }
-                else
-                {
-                    inTombstone = false;
-                    next = new RangeTombstoneBoundMarker(rt.deletedSlice().close(reversed), rt.deletionTime());
-                }
-            }
-            else
-            {
-                next = nextRow;
-                nextRow = null;
-            }
-        }
-        else
-        {
-            if (comparator.compare(nextTombstone.deletedSlice().open(reversed), nextRow.clustering()) < 0)
-            {
-                inTombstone = true;
-                next = new RangeTombstoneBoundMarker(nextTombstone.deletedSlice().open(reversed), nextTombstone.deletionTime());
-            }
-            else
-            {
-                next = nextRow;
-                nextRow = null;
-            }
-        }
-    }
-
-    public boolean hasNext()
-    {
-        prepareNext();
-        return next != null;
-    }
-
-    public Unfiltered next()
-    {
-        prepareNext();
-        Unfiltered toReturn = next;
-        next = null;
-        return toReturn;
-    }
-
-    public Unfiltered peek()
-    {
-        prepareNext();
-        return next();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/RowDataBlock.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowDataBlock.java b/src/java/org/apache/cassandra/db/rows/RowDataBlock.java
deleted file mode 100644
index b1e2b13..0000000
--- a/src/java/org/apache/cassandra/db/rows/RowDataBlock.java
+++ /dev/null
@@ -1,275 +0,0 @@
-/*
- * 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.rows;
-
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import com.google.common.collect.UnmodifiableIterator;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.utils.ObjectSizes;
-
-/**
- * A {@code RowDataBlock} holds data for one or more row (of a given table). More precisely, it contains
- * cell data  and complex deletion data (for complex columns) and allow access to this data. Please note
- * however that {@code RowDataBlock} only holds the data inside the row, it does not hold the data
- * pertaining to the row itself: clustering, partition key liveness info and row deletion.
- * <p>
- * {@code RowDataBlock} is largely an implementation detail: it is only there to be reused by
- * {@link AbstractPartitionData} and every concrete row implementation.
- */
-public class RowDataBlock
-{
-    private static final Logger logger = LoggerFactory.getLogger(RowDataBlock.class);
-
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new RowDataBlock(Columns.NONE, 0, false, false));
-
-    // We distinguish 2 sub-objects: SimpleRowDataBlock that contains the data for the simple columns only,
-    // and ComplexRowDataBlock that only contains data for complex columns. The reason for having 2 separate
-    // objects is that simple columns are much easier to handle since we have only a single cell per-object
-    // and thus having a more specialized object allow a simpler and more efficient handling.
-    final SimpleRowDataBlock simpleData;
-    final ComplexRowDataBlock complexData;
-
-    public RowDataBlock(Columns columns, int rows, boolean sortable, boolean isCounter)
-    {
-        this.simpleData = columns.hasSimple() ? new SimpleRowDataBlock(columns, rows, isCounter) : null;
-        this.complexData = columns.hasComplex() ? ComplexRowDataBlock.create(columns, rows, sortable, isCounter) : null;
-    }
-
-    public Columns columns()
-    {
-        if (simpleData != null)
-            return simpleData.columns();
-        if (complexData != null)
-            return complexData.columns();
-        return Columns.NONE;
-    }
-
-    /**
-     * Return the cell value for a given column of a given row.
-     *
-     * @param row the row for which to return the cell value.
-     * @param column the column for which to return the cell value.
-     * @param path the cell path for which to return the cell value. Can be null for
-     * simple columns.
-     *
-     * @return the value of the cell of path {@code path} for {@code column} in row {@code row}, or
-     * {@code null} if their is no such cell.
-     */
-    public ByteBuffer getValue(int row, ColumnDefinition column, CellPath path)
-    {
-        if (column.isComplex())
-        {
-            return complexData.getValue(row, column, path);
-        }
-        else
-        {
-            int idx = columns().simpleIdx(column, 0);
-            assert idx >= 0;
-            return simpleData.data.value((row * columns().simpleColumnCount()) + idx);
-        }
-    }
-
-    /**
-     * Sets the cell value for a given simple column of a given row.
-     *
-     * @param row the row for which to set the cell value.
-     * @param column the simple column for which to set the cell value.
-     * @param path the cell path for which to return the cell value. Can be null for
-     * simple columns.
-     * @param value the value to set.
-     */
-    public void setValue(int row, ColumnDefinition column, CellPath path, ByteBuffer value)
-    {
-        if (column.isComplex())
-        {
-            complexData.setValue(row, column, path, value);
-        }
-        else
-        {
-            int idx = columns().simpleIdx(column, 0);
-            assert idx >= 0;
-            simpleData.data.setValue((row * columns().simpleColumnCount()) + idx, value);
-        }
-    }
-
-    public static ReusableIterator reusableIterator()
-    {
-        return new ReusableIterator();
-    }
-
-    // Swap row i and j
-    public void swap(int i, int j)
-    {
-        if (simpleData != null)
-            simpleData.swap(i, j);
-        if (complexData != null)
-            complexData.swap(i, j);
-    }
-
-    // Merge row i into j
-    public void merge(int i, int j, int nowInSec)
-    {
-        if (simpleData != null)
-            simpleData.merge(i, j, nowInSec);
-        if (complexData != null)
-            complexData.merge(i, j, nowInSec);
-    }
-
-    // Move row i into j
-    public void move(int i, int j)
-    {
-        if (simpleData != null)
-            simpleData.move(i, j);
-        if (complexData != null)
-            complexData.move(i, j);
-    }
-
-    public boolean hasComplexDeletion(int row)
-    {
-        return complexData != null && complexData.hasComplexDeletion(row);
-    }
-
-    public long unsharedHeapSizeExcludingData()
-    {
-        return EMPTY_SIZE
-             + (simpleData == null ? 0 : simpleData.unsharedHeapSizeExcludingData())
-             + (complexData == null ? 0 : complexData.unsharedHeapSizeExcludingData());
-    }
-
-    public static int computeNewCapacity(int currentCapacity, int idxToSet)
-    {
-        int newCapacity = currentCapacity == 0 ? 4 : currentCapacity;
-        while (idxToSet >= newCapacity)
-            newCapacity = 1 + (newCapacity * 3) / 2;
-        return newCapacity;
-    }
-
-    public int dataSize()
-    {
-        return (simpleData == null ? 0 : simpleData.dataSize())
-             + (complexData == null ? 0 : complexData.dataSize());
-    }
-
-    public void clear()
-    {
-        if (simpleData != null)
-            simpleData.clear();
-        if (complexData != null)
-            complexData.clear();
-    }
-
-    public abstract static class Writer implements Row.Writer
-    {
-        private final boolean inOrderCells;
-
-        protected int row;
-
-        protected SimpleRowDataBlock.CellWriter simpleWriter;
-        protected ComplexRowDataBlock.CellWriter complexWriter;
-
-        protected Writer(boolean inOrderCells)
-        {
-            this.inOrderCells = inOrderCells;
-        }
-
-        protected Writer(RowDataBlock data, boolean inOrderCells)
-        {
-            this(inOrderCells);
-            updateWriter(data);
-        }
-
-        protected void updateWriter(RowDataBlock data)
-        {
-            this.simpleWriter = data.simpleData == null ? null : data.simpleData.cellWriter(inOrderCells);
-            this.complexWriter = data.complexData == null ? null : data.complexData.cellWriter(inOrderCells);
-        }
-
-        public Writer reset()
-        {
-            row = 0;
-
-            if (simpleWriter != null)
-                simpleWriter.reset();
-            if (complexWriter != null)
-                complexWriter.reset();
-
-            return this;
-        }
-
-        public void writeCell(ColumnDefinition column, boolean isCounter, ByteBuffer value, LivenessInfo info, CellPath path)
-        {
-            if (column.isComplex())
-                complexWriter.addCell(column, value, info, path);
-            else
-                simpleWriter.addCell(column, value, info);
-        }
-
-        public void writeComplexDeletion(ColumnDefinition c, DeletionTime complexDeletion)
-        {
-            if (complexDeletion.isLive())
-                return;
-
-            complexWriter.setComplexDeletion(c, complexDeletion);
-        }
-
-        public void endOfRow()
-        {
-            ++row;
-            if (simpleWriter != null)
-                simpleWriter.endOfRow();
-            if (complexWriter != null)
-                complexWriter.endOfRow();
-        }
-    }
-
-    static class ReusableIterator extends UnmodifiableIterator<Cell> implements Iterator<Cell>
-    {
-        private SimpleRowDataBlock.ReusableIterator simpleIterator;
-        private ComplexRowDataBlock.ReusableIterator complexIterator;
-
-        public ReusableIterator()
-        {
-            this.simpleIterator = SimpleRowDataBlock.reusableIterator();
-            this.complexIterator = ComplexRowDataBlock.reusableIterator();
-        }
-
-        public ReusableIterator setTo(RowDataBlock dataBlock, int row)
-        {
-            simpleIterator.setTo(dataBlock.simpleData, row);
-            complexIterator.setTo(dataBlock.complexData, row);
-            return this;
-        }
-
-        public boolean hasNext()
-        {
-            return simpleIterator.hasNext() || complexIterator.hasNext();
-        }
-
-        public Cell next()
-        {
-            return simpleIterator.hasNext() ? simpleIterator.next() : complexIterator.next();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowDiffListener.java b/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
new file mode 100644
index 0000000..50d6d32
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
@@ -0,0 +1,78 @@
+/*
+ * 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.rows;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.*;
+
+/**
+ * Interface that allows to act on the result of merging multiple rows.
+ *
+ * More precisely, given N rows and the result of merging them, one can call {@link Rows#diff()}
+ * with a {@code RowDiffListener} and that listener will be informed for each input row of the diff between
+ * that input and merge row.
+ */
+public interface RowDiffListener
+{
+    /**
+     * Called for the row primary key liveness info of input {@code i}.
+     *
+     * @param i the input row from which {@code original} is from.
+     * @param clustering the clustering for the row that is merged.
+     * @param merged the primary key liveness info of the merged row. Will be {@code null} if input {@code i} had
+     * a {@code LivenessInfo}, but the merged result don't (i.e. the original info has been shadowed/deleted).
+     * @param original the primary key liveness info of input {@code i}. May be {@code null} if input {@code i}
+     * has not primary key liveness info (i.e. it has {@code LivenessInfo.NONE}) but the merged result has.
+     */
+    public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original);
+
+    /**
+     * Called for the row deletion of input {@code i}.
+     *
+     * @param i the input row from which {@code original} is from.
+     * @param clustering the clustering for the row that is merged.
+     * @param merged the deletion of the merged row. Will be {@code null} if input {@code i} had deletion
+     * but the merged result doesn't (i.e. the deletion has been shadowed).
+     * @param original the deletion of input {@code i}. May be {@code null} if input {@code i} had no deletion but the merged row has.
+     */
+    public void onDeletion(int i, Clustering clustering, DeletionTime merged, DeletionTime original);
+
+    /**
+     * Called for every (non-live) complex deletion of any complex column present in either the merged row of input {@code i}.
+     *
+     * @param i the input row from which {@code original} is from.
+     * @param clustering the clustering for the row that is merged.
+     * @param column the column for which this is a complex deletion of.
+     * @param merged the complex deletion of the merged row. Will be {@code null} if input {@code i} had a complex deletion
+     * for {@code column} but the merged result doesn't (i.e. the deletion has been shadowed).
+     * @param original the complex deletion of input {@code i} for column {@code column}. May be {@code null} if input {@code i}
+     * had no complex deletion but the merged row has.
+     */
+    public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original);
+
+    /**
+     * Called for any cell that is either in the merged row or in input {@code i}.
+     *
+     * @param i the input row from which {@code original} is from.
+     * @param clustering the clustering for the row that is merged.
+     * @param merged the cell of the merged row. Will be {@code null} if input {@code i} had a cell but that cell is no present
+     * in the mergd result (it has been deleted/shadowed).
+     * @param original the cell of input {@code i}. May be {@code null} if input {@code i} had cell corresponding to {@code merged}.
+     */
+    public void onCell(int i, Clustering clustering, Cell merged, Cell original);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/RowIterators.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowIterators.java b/src/java/org/apache/cassandra/db/rows/RowIterators.java
index a3bd913..766cf19 100644
--- a/src/java/org/apache/cassandra/db/rows/RowIterators.java
+++ b/src/java/org/apache/cassandra/db/rows/RowIterators.java
@@ -25,7 +25,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -37,19 +36,6 @@ public abstract class RowIterators
 
     private RowIterators() {}
 
-    public static PartitionUpdate toUpdate(RowIterator iterator)
-    {
-        PartitionUpdate update = new PartitionUpdate(iterator.metadata(), iterator.partitionKey(), iterator.columns(), 1);
-
-        if (iterator.staticRow() != Rows.EMPTY_STATIC_ROW)
-            iterator.staticRow().copyTo(update.staticWriter());
-
-        while (iterator.hasNext())
-            iterator.next().copyTo(update.writer());
-
-        return update;
-    }
-
     public static void digest(RowIterator iterator, MessageDigest digest)
     {
         // TODO: we're not computing digest the same way that old nodes so we'll need
@@ -123,11 +109,11 @@ public abstract class RowIterators
     {
         CFMetaData metadata = iterator.metadata();
         logger.info("[{}] Logging iterator on {}.{}, partition key={}, reversed={}",
-                    new Object[]{ id,
-                                  metadata.ksName,
-                                  metadata.cfName,
-                                  metadata.getKeyValidator().getString(iterator.partitionKey().getKey()),
-                                  iterator.isReverseOrder() });
+                    id,
+                    metadata.ksName,
+                    metadata.cfName,
+                    metadata.getKeyValidator().getString(iterator.partitionKey().getKey()),
+                    iterator.isReverseOrder());
 
         return new WrappingRowIterator(iterator)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/RowStats.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowStats.java b/src/java/org/apache/cassandra/db/rows/RowStats.java
index c672490..5b0d3bd 100644
--- a/src/java/org/apache/cassandra/db/rows/RowStats.java
+++ b/src/java/org/apache/cassandra/db/rows/RowStats.java
@@ -17,17 +17,17 @@
  */
 package org.apache.cassandra.db.rows;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.util.Objects;
 
-import org.apache.cassandra.db.DeletionTime;
-import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.partitions.PartitionStatisticsCollector;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
 import static org.apache.cassandra.db.LivenessInfo.NO_TIMESTAMP;
 import static org.apache.cassandra.db.LivenessInfo.NO_TTL;
-import static org.apache.cassandra.db.LivenessInfo.NO_DELETION_TIME;
+import static org.apache.cassandra.db.LivenessInfo.NO_EXPIRATION_TIME;
 
 /**
  * General statistics on rows (and and tombstones) for a given source.
@@ -45,7 +45,7 @@ import static org.apache.cassandra.db.LivenessInfo.NO_DELETION_TIME;
 public class RowStats
 {
     // We should use this sparingly obviously
-    public static final RowStats NO_STATS = new RowStats(NO_TIMESTAMP, NO_DELETION_TIME, NO_TTL, -1);
+    public static final RowStats NO_STATS = new RowStats(NO_TIMESTAMP, NO_EXPIRATION_TIME, NO_TTL, -1);
 
     public static final Serializer serializer = new Serializer();
 
@@ -74,7 +74,7 @@ public class RowStats
 
     public boolean hasMinLocalDeletionTime()
     {
-        return minLocalDeletionTime != NO_DELETION_TIME;
+        return minLocalDeletionTime != NO_EXPIRATION_TIME;
     }
 
     /**
@@ -89,9 +89,9 @@ public class RowStats
                           ? that.minTimestamp
                           : (that.minTimestamp == NO_TIMESTAMP ? this.minTimestamp : Math.min(this.minTimestamp, that.minTimestamp));
 
-        int minDelTime = this.minLocalDeletionTime == NO_DELETION_TIME
+        int minDelTime = this.minLocalDeletionTime == NO_EXPIRATION_TIME
                        ? that.minLocalDeletionTime
-                       : (that.minLocalDeletionTime == NO_DELETION_TIME ? this.minLocalDeletionTime : Math.min(this.minLocalDeletionTime, that.minLocalDeletionTime));
+                       : (that.minLocalDeletionTime == NO_EXPIRATION_TIME ? this.minLocalDeletionTime : Math.min(this.minLocalDeletionTime, that.minLocalDeletionTime));
 
         int minTTL = this.minTTL == NO_TTL
                    ? that.minTTL
@@ -132,7 +132,7 @@ public class RowStats
         return String.format("RowStats(ts=%d, ldt=%d, ttl=%d, avgColPerRow=%d)", minTimestamp, minLocalDeletionTime, minTTL, avgColumnSetPerRow);
     }
 
-    public static class Collector
+    public static class Collector implements PartitionStatisticsCollector
     {
         private boolean isTimestampSet;
         private long minTimestamp = Long.MAX_VALUE;
@@ -147,6 +147,27 @@ public class RowStats
         private long totalColumnsSet;
         private long rows;
 
+        public void update(LivenessInfo info)
+        {
+            if (info.isEmpty())
+                return;
+
+            updateTimestamp(info.timestamp());
+
+            if (info.isExpiring())
+            {
+                updateTTL(info.ttl());
+                updateLocalDeletionTime(info.localExpirationTime());
+            }
+        }
+
+        public void update(Cell cell)
+        {
+            updateTimestamp(cell.timestamp());
+            updateTTL(cell.ttl());
+            updateLocalDeletionTime(cell.localDeletionTime());
+        }
+
         public void updateTimestamp(long timestamp)
         {
             if (timestamp == NO_TIMESTAMP)
@@ -158,14 +179,14 @@ public class RowStats
 
         public void updateLocalDeletionTime(int deletionTime)
         {
-            if (deletionTime == NO_DELETION_TIME)
+            if (deletionTime == NO_EXPIRATION_TIME)
                 return;
 
             isDelTimeSet = true;
             minDeletionTime = Math.min(minDeletionTime, deletionTime);
         }
 
-        public void updateDeletionTime(DeletionTime deletionTime)
+        public void update(DeletionTime deletionTime)
         {
             if (deletionTime.isLive())
                 return;
@@ -183,7 +204,7 @@ public class RowStats
             minTTL = Math.min(minTTL, ttl);
         }
 
-        public void updateColumnSetPerRow(int columnSetInRow)
+        public void updateColumnSetPerRow(long columnSetInRow)
         {
             updateColumnSetPerRow(columnSetInRow, 1);
         }
@@ -198,12 +219,17 @@ public class RowStats
             this.rows += rows;
         }
 
+        public void updateHasLegacyCounterShards(boolean hasLegacyCounterShards)
+        {
+            // We don't care about this but this come with PartitionStatisticsCollector
+        }
+
         public RowStats get()
         {
             return new RowStats(isTimestampSet ? minTimestamp : NO_TIMESTAMP,
-                                 isDelTimeSet ? minDeletionTime : NO_DELETION_TIME,
-                                 isTTLSet ? minTTL : NO_TTL,
-                                 isColumnSetPerRowSet ? (rows == 0 ? 0 : (int)(totalColumnsSet / rows)) : -1);
+                                isDelTimeSet ? minDeletionTime : NO_EXPIRATION_TIME,
+                                isTTLSet ? minTTL : NO_TTL,
+                                isColumnSetPerRowSet ? (rows == 0 ? 0 : (int)(totalColumnsSet / rows)) : -1);
         }
     }
 
@@ -211,26 +237,26 @@ public class RowStats
     {
         public void serialize(RowStats stats, DataOutputPlus out) throws IOException
         {
-            out.writeLong(stats.minTimestamp);
-            out.writeInt(stats.minLocalDeletionTime);
-            out.writeInt(stats.minTTL);
-            out.writeInt(stats.avgColumnSetPerRow);
+            out.writeVInt(stats.minTimestamp);
+            out.writeVInt(stats.minLocalDeletionTime);
+            out.writeVInt(stats.minTTL);
+            out.writeVInt(stats.avgColumnSetPerRow);
         }
 
         public int serializedSize(RowStats stats)
         {
-            return TypeSizes.sizeof(stats.minTimestamp)
-                 + TypeSizes.sizeof(stats.minLocalDeletionTime)
-                 + TypeSizes.sizeof(stats.minTTL)
-                 + TypeSizes.sizeof(stats.avgColumnSetPerRow);
+            return TypeSizes.sizeofVInt(stats.minTimestamp)
+                 + TypeSizes.sizeofVInt(stats.minLocalDeletionTime)
+                 + TypeSizes.sizeofVInt(stats.minTTL)
+                 + TypeSizes.sizeofVInt(stats.avgColumnSetPerRow);
         }
 
-        public RowStats deserialize(DataInput in) throws IOException
+        public RowStats deserialize(DataInputPlus in) throws IOException
         {
-            long minTimestamp = in.readLong();
-            int minLocalDeletionTime = in.readInt();
-            int minTTL = in.readInt();
-            int avgColumnSetPerRow = in.readInt();
+            long minTimestamp = in.readVInt();
+            int minLocalDeletionTime = (int)in.readVInt();
+            int minTTL = (int)in.readVInt();
+            int avgColumnSetPerRow = (int)in.readVInt();
             return new RowStats(minTimestamp, minLocalDeletionTime, minTTL, avgColumnSetPerRow);
         }
     }


[04/15] cassandra git commit: Simplify some 8099's implementations

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java b/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
index 30923c5..4072f8d 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
@@ -17,32 +17,21 @@
  */
 package org.apache.cassandra.db.rows;
 
-import java.io.DataInput;
 import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.utils.SearchIterator;
 
 /**
- * Serialize/deserialize a single Unfiltered for the intra-node protocol.
+ * Serialize/deserialize a single Unfiltered (both on-wire and on-disk).
  *
- * The encode format for an unfiltered is <flags>(<row>|<marker>) where:
+ * The encoded format for an unfiltered is <flags>(<row>|<marker>) where:
  *
- *   <flags> is a byte whose bits are flags. The rightmost 1st bit is only
- *       set to indicate the end of the partition. The 2nd bit indicates
- *       whether the reminder is a range tombstone marker (otherwise it's a row).
- *       If it's a row then the 3rd bit indicates if it's static, the 4th bit
- *       indicates the presence of a row timestamp, the 5th the presence of a row
- *       ttl, the 6th the presence of row deletion and the 7th indicates the
- *       presence of complex deletion times.
+ *   <flags> is a byte whose bits are flags used by the rest of the serialization. Each
+ *       flag is defined/explained below as the "Unfiltered flags" constants.
  *   <row> is <clustering>[<timestamp>][<ttl>][<deletion>]<sc1>...<sci><cc1>...<ccj> where
  *       <clustering> is the row clustering as serialized by
  *       {@code Clustering.serializer}. Note that static row are an exception and
@@ -50,7 +39,7 @@ import org.apache.cassandra.utils.SearchIterator;
  *       whose presence is determined by the flags. <sci> is the simple columns of the row and <ccj> the
  *       complex ones.  There is actually 2 slightly different possible layout for those
  *       cell: a dense one and a sparse one. Which one is used depends on the serialization
- *       header and more precisely of {@link SerializationHeader.useSparseColumnLayout()}:
+ *       header and more precisely of {@link SerializationHeader#useSparseColumnLayout(boolean)}:
  *         1) in the dense layout, there will be as many <sci> and <ccj> as there is columns
  *            in the serialization header. *Each simple column <sci> will simply be a <cell>
  *            (which might have no value, see below), while each <ccj> will be
@@ -84,27 +73,18 @@ import org.apache.cassandra.utils.SearchIterator;
  */
 public class UnfilteredSerializer
 {
-    private static final Logger logger = LoggerFactory.getLogger(UnfilteredSerializer.class);
-
     public static final UnfilteredSerializer serializer = new UnfilteredSerializer();
 
-    // Unfiltered flags
-    private final static int END_OF_PARTITION     = 0x01;
-    private final static int IS_MARKER            = 0x02;
-    // For rows
-    private final static int IS_STATIC            = 0x04;
-    private final static int HAS_TIMESTAMP        = 0x08;
-    private final static int HAS_TTL              = 0x10;
-    private final static int HAS_DELETION         = 0x20;
-    private final static int HAS_COMPLEX_DELETION = 0x40;
-
-    // Cell flags
-    private final static int PRESENCE_MASK     = 0x01;
-    private final static int DELETION_MASK     = 0x02;
-    private final static int EXPIRATION_MASK   = 0x04;
-    private final static int EMPTY_VALUE_MASK  = 0x08;
-    private final static int USE_ROW_TIMESTAMP = 0x10;
-    private final static int USE_ROW_TTL       = 0x20;
+    /*
+     * Unfiltered flags constants.
+     */
+    private final static int END_OF_PARTITION     = 0x01; // Signal the end of the partition. Nothing follows a <flags> field with that flag.
+    private final static int IS_MARKER            = 0x02; // Whether the encoded unfiltered is a marker or a row. All following markers applies only to rows.
+    private final static int IS_STATIC            = 0x04; // Whether the encoded row is a static.
+    private final static int HAS_TIMESTAMP        = 0x08; // Whether the encoded row has a timestamp (i.e. if row.partitionKeyLivenessInfo().hasTimestamp() == true).
+    private final static int HAS_TTL              = 0x10; // Whether the encoded row has some expiration info (i.e. if row.partitionKeyLivenessInfo().hasTTL() == true).
+    private final static int HAS_DELETION         = 0x20; // Whether the encoded row has some deletion info.
+    private final static int HAS_COMPLEX_DELETION = 0x40; // Whether the encoded row has some complex deletion for at least one of its columns.
 
     public void serialize(Unfiltered unfiltered, SerializationHeader header, DataOutputPlus out, int version)
     throws IOException
@@ -131,9 +111,9 @@ public class UnfilteredSerializer
 
         if (isStatic)
             flags |= IS_STATIC;
-        if (pkLiveness.hasTimestamp())
+        if (!pkLiveness.isEmpty())
             flags |= HAS_TIMESTAMP;
-        if (pkLiveness.hasTTL())
+        if (pkLiveness.isExpiring())
             flags |= HAS_TTL;
         if (!deletion.isLive())
             flags |= HAS_DELETION;
@@ -149,7 +129,7 @@ public class UnfilteredSerializer
         if ((flags & HAS_TTL) != 0)
         {
             out.writeInt(header.encodeTTL(pkLiveness.ttl()));
-            out.writeInt(header.encodeDeletionTime(pkLiveness.localDeletionTime()));
+            out.writeInt(header.encodeDeletionTime(pkLiveness.localExpirationTime()));
         }
         if ((flags & HAS_DELETION) != 0)
             UnfilteredRowIteratorSerializer.writeDelTime(deletion, header, out);
@@ -160,52 +140,49 @@ public class UnfilteredSerializer
         SearchIterator<ColumnDefinition, ColumnData> cells = row.searchIterator();
 
         for (int i = 0; i < simpleCount; i++)
-            writeSimpleColumn(i, cells.next(columns.getSimple(i)), header, out, pkLiveness, useSparse);
+            writeSimpleColumn(i, (Cell)cells.next(columns.getSimple(i)), pkLiveness, header, out, useSparse);
 
         for (int i = simpleCount; i < columns.columnCount(); i++)
-            writeComplexColumn(i, cells.next(columns.getComplex(i - simpleCount)), hasComplexDeletion, header, out, pkLiveness, useSparse);
+            writeComplexColumn(i, (ComplexColumnData)cells.next(columns.getComplex(i - simpleCount)), hasComplexDeletion, pkLiveness, header, out, useSparse);
 
         if (useSparse)
             out.writeShort(-1);
     }
 
-    private void writeSimpleColumn(int idx, ColumnData data, SerializationHeader header, DataOutputPlus out, LivenessInfo rowLiveness, boolean useSparse)
+    private void writeSimpleColumn(int idx, Cell cell, LivenessInfo rowLiveness, SerializationHeader header, DataOutputPlus out, boolean useSparse)
     throws IOException
     {
         if (useSparse)
         {
-            if (data == null)
+            if (cell == null)
                 return;
 
             out.writeShort(idx);
         }
-
-        writeCell(data == null ? null : data.cell(), header, out, rowLiveness);
+        Cell.serializer.serialize(cell, out, rowLiveness, header);
     }
 
-    private void writeComplexColumn(int idx, ColumnData data, boolean hasComplexDeletion, SerializationHeader header, DataOutputPlus out, LivenessInfo rowLiveness, boolean useSparse)
+    private void writeComplexColumn(int idx, ComplexColumnData data, boolean hasComplexDeletion, LivenessInfo rowLiveness, SerializationHeader header, DataOutputPlus out, boolean useSparse)
     throws IOException
     {
-        Iterator<Cell> cells = data == null ? null : data.cells();
-        DeletionTime deletion = data == null ? DeletionTime.LIVE : data.complexDeletion();
-
         if (useSparse)
         {
-            assert hasComplexDeletion || deletion.isLive();
-            if (cells == null && deletion.isLive())
+            if (data == null)
                 return;
 
             out.writeShort(idx);
         }
 
         if (hasComplexDeletion)
-            UnfilteredRowIteratorSerializer.writeDelTime(deletion, header, out);
+            UnfilteredRowIteratorSerializer.writeDelTime(data == null ? DeletionTime.LIVE : data.complexDeletion(), header, out);
 
-        if (cells != null)
-            while (cells.hasNext())
-                writeCell(cells.next(), header, out, rowLiveness);
+        if (data != null)
+        {
+            for (Cell cell : data)
+                Cell.serializer.serialize(cell, out, rowLiveness, header);
+        }
 
-        writeCell(null, header, out, rowLiveness);
+        Cell.serializer.serialize(null, out, rowLiveness, header);
     }
 
     public void serialize(RangeTombstoneMarker marker, SerializationHeader header, DataOutputPlus out, int version)
@@ -245,12 +222,12 @@ public class UnfilteredSerializer
         if (!isStatic)
             size += Clustering.serializer.serializedSize(row.clustering(), version, header.clusteringTypes());
 
-        if (pkLiveness.hasTimestamp())
+        if (!pkLiveness.isEmpty())
             size += TypeSizes.sizeof(header.encodeTimestamp(pkLiveness.timestamp()));
-        if (pkLiveness.hasTTL())
+        if (pkLiveness.isExpiring())
         {
             size += TypeSizes.sizeof(header.encodeTTL(pkLiveness.ttl()));
-            size += TypeSizes.sizeof(header.encodeDeletionTime(pkLiveness.localDeletionTime()));
+            size += TypeSizes.sizeof(header.encodeDeletionTime(pkLiveness.localExpirationTime()));
         }
         if (!deletion.isLive())
             size += UnfilteredRowIteratorSerializer.delTimeSerializedSize(deletion, header);
@@ -261,10 +238,10 @@ public class UnfilteredSerializer
         SearchIterator<ColumnDefinition, ColumnData> cells = row.searchIterator();
 
         for (int i = 0; i < simpleCount; i++)
-            size += sizeOfSimpleColumn(i, cells.next(columns.getSimple(i)), header, pkLiveness, useSparse);
+            size += sizeOfSimpleColumn(i, (Cell)cells.next(columns.getSimple(i)), pkLiveness, header, useSparse);
 
         for (int i = simpleCount; i < columns.columnCount(); i++)
-            size += sizeOfComplexColumn(i, cells.next(columns.getComplex(i - simpleCount)), hasComplexDeletion, header, pkLiveness, useSparse);
+            size += sizeOfComplexColumn(i, (ComplexColumnData)cells.next(columns.getComplex(i - simpleCount)), hasComplexDeletion, pkLiveness, header, useSparse);
 
         if (useSparse)
             size += TypeSizes.sizeof((short)-1);
@@ -272,41 +249,40 @@ public class UnfilteredSerializer
         return size;
     }
 
-    private long sizeOfSimpleColumn(int idx, ColumnData data, SerializationHeader header, LivenessInfo rowLiveness, boolean useSparse)
+    private long sizeOfSimpleColumn(int idx, Cell cell, LivenessInfo rowLiveness, SerializationHeader header, boolean useSparse)
     {
         long size = 0;
         if (useSparse)
         {
-            if (data == null)
+            if (cell == null)
                 return size;
 
             size += TypeSizes.sizeof((short)idx);
         }
-        return size + sizeOfCell(data == null ? null : data.cell(), header, rowLiveness);
+        return size + Cell.serializer.serializedSize(cell, rowLiveness, header);
     }
 
-    private long sizeOfComplexColumn(int idx, ColumnData data, boolean hasComplexDeletion, SerializationHeader header, LivenessInfo rowLiveness, boolean useSparse)
+    private long sizeOfComplexColumn(int idx, ComplexColumnData data, boolean hasComplexDeletion, LivenessInfo rowLiveness, SerializationHeader header, boolean useSparse)
     {
         long size = 0;
-        Iterator<Cell> cells = data == null ? null : data.cells();
-        DeletionTime deletion = data == null ? DeletionTime.LIVE : data.complexDeletion();
         if (useSparse)
         {
-            assert hasComplexDeletion || deletion.isLive();
-            if (cells == null && deletion.isLive())
+            if (data == null)
                 return size;
 
             size += TypeSizes.sizeof((short)idx);
         }
 
         if (hasComplexDeletion)
-            size += UnfilteredRowIteratorSerializer.delTimeSerializedSize(deletion, header);
+            size += UnfilteredRowIteratorSerializer.delTimeSerializedSize(data == null ? DeletionTime.LIVE : data.complexDeletion(), header);
 
-        if (cells != null)
-            while (cells.hasNext())
-                size += sizeOfCell(cells.next(), header, rowLiveness);
+        if (data != null)
+        {
+            for (Cell cell : data)
+                size += Cell.serializer.serializedSize(cell, rowLiveness, header);
+        }
 
-        return size + sizeOfCell(null, header, rowLiveness);
+        return size + Cell.serializer.serializedSize(null, rowLiveness, header);
     }
 
     public long serializedSize(RangeTombstoneMarker marker, SerializationHeader header, int version)
@@ -337,155 +313,157 @@ public class UnfilteredSerializer
         return 1;
     }
 
-    public Unfiltered.Kind deserialize(DataInput in,
-                                 SerializationHeader header,
-                                 SerializationHelper helper,
-                                 Row.Writer rowWriter,
-                                 RangeTombstoneMarker.Writer markerWriter)
+    public Unfiltered deserialize(DataInputPlus in, SerializationHeader header, SerializationHelper helper, Row.Builder builder)
     throws IOException
     {
+        // It wouldn't be wrong per-se to use an unsorted builder, but it would be inefficient so make sure we don't do it by mistake
+        assert builder.isSorted();
+
         int flags = in.readUnsignedByte();
         if (isEndOfPartition(flags))
             return null;
 
         if (kind(flags) == Unfiltered.Kind.RANGE_TOMBSTONE_MARKER)
         {
-            RangeTombstone.Bound.Kind kind = RangeTombstone.Bound.serializer.deserialize(in, helper.version, header.clusteringTypes(), markerWriter);
-            deserializeMarkerBody(in, header, kind.isBoundary(), markerWriter);
-            return Unfiltered.Kind.RANGE_TOMBSTONE_MARKER;
+            RangeTombstone.Bound bound = RangeTombstone.Bound.serializer.deserialize(in, helper.version, header.clusteringTypes());
+            return deserializeMarkerBody(in, header, bound);
         }
         else
         {
             assert !isStatic(flags); // deserializeStaticRow should be used for that.
-            Clustering.serializer.deserialize(in, helper.version, header.clusteringTypes(), rowWriter);
-            deserializeRowBody(in, header, helper, flags, rowWriter);
-            return Unfiltered.Kind.ROW;
+            builder.newRow(Clustering.serializer.deserialize(in, helper.version, header.clusteringTypes()));
+            return deserializeRowBody(in, header, helper, flags, builder);
         }
     }
 
-    public Row deserializeStaticRow(DataInput in, SerializationHeader header, SerializationHelper helper)
+    public Row deserializeStaticRow(DataInputPlus in, SerializationHeader header, SerializationHelper helper)
     throws IOException
     {
         int flags = in.readUnsignedByte();
-        assert !isEndOfPartition(flags) && kind(flags) == Unfiltered.Kind.ROW && isStatic(flags);
-        StaticRow.Builder builder = StaticRow.builder(header.columns().statics, true, header.columns().statics.hasCounters());
-        deserializeRowBody(in, header, helper, flags, builder);
-        return builder.build();
-    }
-
-    public void skipStaticRow(DataInput in, SerializationHeader header, SerializationHelper helper) throws IOException
-    {
-        int flags = in.readUnsignedByte();
-        assert !isEndOfPartition(flags) && kind(flags) == Unfiltered.Kind.ROW && isStatic(flags) : "Flags is " + flags;
-        skipRowBody(in, header, helper, flags);
+        assert !isEndOfPartition(flags) && kind(flags) == Unfiltered.Kind.ROW && isStatic(flags) : flags;
+        Row.Builder builder = ArrayBackedRow.sortedBuilder(helper.fetchedStaticColumns(header));
+        builder.newRow(Clustering.STATIC_CLUSTERING);
+        return deserializeRowBody(in, header, helper, flags, builder);
     }
 
-    public void deserializeMarkerBody(DataInput in,
-                                      SerializationHeader header,
-                                      boolean isBoundary,
-                                      RangeTombstoneMarker.Writer writer)
+    public RangeTombstoneMarker deserializeMarkerBody(DataInputPlus in, SerializationHeader header, RangeTombstone.Bound bound)
     throws IOException
     {
-        if (isBoundary)
-            writer.writeBoundaryDeletion(UnfilteredRowIteratorSerializer.readDelTime(in, header), UnfilteredRowIteratorSerializer.readDelTime(in, header));
-        else
-            writer.writeBoundDeletion(UnfilteredRowIteratorSerializer.readDelTime(in, header));
-        writer.endOfMarker();
-    }
-
-    public void skipMarkerBody(DataInput in, SerializationHeader header, boolean isBoundary) throws IOException
-    {
-        if (isBoundary)
-        {
-            UnfilteredRowIteratorSerializer.skipDelTime(in, header);
-            UnfilteredRowIteratorSerializer.skipDelTime(in, header);
-        }
+        if (bound.isBoundary())
+            return new RangeTombstoneBoundaryMarker(bound, UnfilteredRowIteratorSerializer.readDelTime(in, header), UnfilteredRowIteratorSerializer.readDelTime(in, header));
         else
-        {
-            UnfilteredRowIteratorSerializer.skipDelTime(in, header);
-        }
+            return new RangeTombstoneBoundMarker(bound, UnfilteredRowIteratorSerializer.readDelTime(in, header));
     }
 
-    public void deserializeRowBody(DataInput in,
-                                   SerializationHeader header,
-                                   SerializationHelper helper,
-                                   int flags,
-                                   Row.Writer writer)
+    public Row deserializeRowBody(DataInputPlus in,
+                                  SerializationHeader header,
+                                  SerializationHelper helper,
+                                  int flags,
+                                  Row.Builder builder)
     throws IOException
     {
-        boolean isStatic = isStatic(flags);
-        boolean hasTimestamp = (flags & HAS_TIMESTAMP) != 0;
-        boolean hasTTL = (flags & HAS_TTL) != 0;
-        boolean hasDeletion = (flags & HAS_DELETION) != 0;
-        boolean hasComplexDeletion = (flags & HAS_COMPLEX_DELETION) != 0;
-
-        long timestamp = hasTimestamp ? header.decodeTimestamp(in.readLong()) : LivenessInfo.NO_TIMESTAMP;
-        int ttl = hasTTL ? header.decodeTTL(in.readInt()) : LivenessInfo.NO_TTL;
-        int localDeletionTime = hasTTL ? header.decodeDeletionTime(in.readInt()) : LivenessInfo.NO_DELETION_TIME;
-        DeletionTime deletion = hasDeletion ? UnfilteredRowIteratorSerializer.readDelTime(in, header) : DeletionTime.LIVE;
+        try
+        {
+            boolean isStatic = isStatic(flags);
+            boolean hasTimestamp = (flags & HAS_TIMESTAMP) != 0;
+            boolean hasTTL = (flags & HAS_TTL) != 0;
+            boolean hasDeletion = (flags & HAS_DELETION) != 0;
+            boolean hasComplexDeletion = (flags & HAS_COMPLEX_DELETION) != 0;
+
+            LivenessInfo rowLiveness = LivenessInfo.EMPTY;
+            if (hasTimestamp)
+            {
+                long timestamp = header.decodeTimestamp(in.readLong());
+                int ttl = hasTTL ? header.decodeTTL(in.readInt()) : LivenessInfo.NO_TTL;
+                int localDeletionTime = hasTTL ? header.decodeDeletionTime(in.readInt()) : LivenessInfo.NO_EXPIRATION_TIME;
+                rowLiveness = LivenessInfo.create(timestamp, ttl, localDeletionTime);
+            }
 
-        helper.writePartitionKeyLivenessInfo(writer, timestamp, ttl, localDeletionTime);
-        writer.writeRowDeletion(deletion);
+            builder.addPrimaryKeyLivenessInfo(rowLiveness);
+            builder.addRowDeletion(hasDeletion ? UnfilteredRowIteratorSerializer.readDelTime(in, header) : DeletionTime.LIVE);
 
-        Columns columns = header.columns(isStatic);
-        if (header.useSparseColumnLayout(isStatic))
-        {
-            int count = columns.columnCount();
-            int simpleCount = columns.simpleColumnCount();
-            int i;
-            while ((i = in.readShort()) >= 0)
+            Columns columns = header.columns(isStatic);
+            if (header.useSparseColumnLayout(isStatic))
             {
-                if (i > count)
-                    throw new IOException(String.format("Impossible column index %d, the header has only %d columns defined", i, count));
+                int count = columns.columnCount();
+                int simpleCount = columns.simpleColumnCount();
+                int i;
+                while ((i = in.readShort()) >= 0)
+                {
+                    if (i > count)
+                        throw new IOException(String.format("Impossible column index %d, the header has only %d columns defined", i, count));
+
+                    if (i < simpleCount)
+                        readSimpleColumn(columns.getSimple(i), in, header, helper, builder, rowLiveness);
+                    else
+                        readComplexColumn(columns.getComplex(i - simpleCount), in, header, helper, hasComplexDeletion, builder, rowLiveness);
+                }
+            }
+            else
+            {
+                for (int i = 0; i < columns.simpleColumnCount(); i++)
+                    readSimpleColumn(columns.getSimple(i), in, header, helper, builder, rowLiveness);
 
-                if (i < simpleCount)
-                    readSimpleColumn(columns.getSimple(i), in, header, helper, writer);
-                else
-                    readComplexColumn(columns.getComplex(i - simpleCount), in, header, helper, hasComplexDeletion, writer);
+                for (int i = 0; i < columns.complexColumnCount(); i++)
+                    readComplexColumn(columns.getComplex(i), in, header, helper, hasComplexDeletion, builder, rowLiveness);
             }
+
+                return builder.build();
         }
-        else
+        catch (RuntimeException | AssertionError e)
         {
-            for (int i = 0; i < columns.simpleColumnCount(); i++)
-                readSimpleColumn(columns.getSimple(i), in, header, helper, writer);
-
-            for (int i = 0; i < columns.complexColumnCount(); i++)
-                readComplexColumn(columns.getComplex(i), in, header, helper, hasComplexDeletion, writer);
+            // Corrupted data could be such that it triggers an assertion in the row Builder, or break one of its assumption.
+            // Of course, a bug in said builder could also trigger this, but it's impossible a priori to always make the distinction
+            // between a real bug and data corrupted in just the bad way. Besides, re-throwing as an IOException doesn't hide the
+            // exception, it just make we catch it properly and mark the sstable as corrupted.
+            throw new IOException("Error building row with data deserialized from " + in, e);
         }
-
-        writer.endOfRow();
     }
 
-    private void readSimpleColumn(ColumnDefinition column, DataInput in, SerializationHeader header, SerializationHelper helper, Row.Writer writer)
+    private void readSimpleColumn(ColumnDefinition column, DataInputPlus in, SerializationHeader header, SerializationHelper helper, Row.Builder builder, LivenessInfo rowLiveness)
     throws IOException
     {
         if (helper.includes(column))
-            readCell(column, in, header, helper, writer);
+        {
+            Cell cell = Cell.serializer.deserialize(in, rowLiveness, column, header, helper);
+            if (cell != null && !helper.isDropped(cell, false))
+                builder.addCell(cell);
+        }
         else
-            skipCell(column, in, header);
+        {
+            Cell.serializer.skip(in, column, header);
+        }
     }
 
-    private void readComplexColumn(ColumnDefinition column, DataInput in, SerializationHeader header, SerializationHelper helper, boolean hasComplexDeletion, Row.Writer writer)
+    private void readComplexColumn(ColumnDefinition column, DataInputPlus in, SerializationHeader header, SerializationHelper helper, boolean hasComplexDeletion, Row.Builder builder, LivenessInfo rowLiveness)
     throws IOException
     {
         if (helper.includes(column))
         {
             helper.startOfComplexColumn(column);
-
             if (hasComplexDeletion)
-                writer.writeComplexDeletion(column, UnfilteredRowIteratorSerializer.readDelTime(in, header));
+            {
+                DeletionTime complexDeletion = UnfilteredRowIteratorSerializer.readDelTime(in, header);
+                if (!helper.isDroppedComplexDeletion(complexDeletion))
+                    builder.addComplexDeletion(column, complexDeletion);
+            }
 
-            while (readCell(column, in, header, helper, writer));
+            Cell cell;
+            while ((cell = Cell.serializer.deserialize(in, rowLiveness, column, header, helper)) != null)
+            {
+                if (helper.includes(cell.path()) && !helper.isDropped(cell, true))
+                    builder.addCell(cell);
+            }
 
-            helper.endOfComplexColumn(column);
+            helper.endOfComplexColumn();
         }
         else
         {
-            skipComplexColumn(column, in, header, helper, hasComplexDeletion);
+            skipComplexColumn(in, column, header, hasComplexDeletion);
         }
     }
 
-    public void skipRowBody(DataInput in, SerializationHeader header, SerializationHelper helper, int flags) throws IOException
+    public void skipRowBody(DataInputPlus in, SerializationHeader header, int flags) throws IOException
     {
         boolean isStatic = isStatic(flags);
         boolean hasTimestamp = (flags & HAS_TIMESTAMP) != 0;
@@ -518,28 +496,48 @@ public class UnfilteredSerializer
                     throw new IOException(String.format("Impossible column index %d, the header has only %d columns defined", i, count));
 
                 if (i < simpleCount)
-                    skipCell(columns.getSimple(i), in, header);
+                    Cell.serializer.skip(in, columns.getSimple(i), header);
                 else
-                    skipComplexColumn(columns.getComplex(i - simpleCount), in, header, helper, hasComplexDeletion);
+                    skipComplexColumn(in, columns.getComplex(i - simpleCount), header, hasComplexDeletion);
             }
         }
         else
         {
             for (int i = 0; i < columns.simpleColumnCount(); i++)
-                skipCell(columns.getSimple(i), in, header);
+                Cell.serializer.skip(in, columns.getSimple(i), header);
 
             for (int i = 0; i < columns.complexColumnCount(); i++)
-                skipComplexColumn(columns.getComplex(i), in, header, helper, hasComplexDeletion);
+                skipComplexColumn(in, columns.getComplex(i), header, hasComplexDeletion);
+        }
+    }
+
+    public void skipStaticRow(DataInputPlus in, SerializationHeader header, SerializationHelper helper) throws IOException
+    {
+        int flags = in.readUnsignedByte();
+        assert !isEndOfPartition(flags) && kind(flags) == Unfiltered.Kind.ROW && isStatic(flags) : "Flags is " + flags;
+        skipRowBody(in, header, flags);
+    }
+
+    public void skipMarkerBody(DataInputPlus in, SerializationHeader header, boolean isBoundary) throws IOException
+    {
+        if (isBoundary)
+        {
+            UnfilteredRowIteratorSerializer.skipDelTime(in, header);
+            UnfilteredRowIteratorSerializer.skipDelTime(in, header);
+        }
+        else
+        {
+            UnfilteredRowIteratorSerializer.skipDelTime(in, header);
         }
     }
 
-    private void skipComplexColumn(ColumnDefinition column, DataInput in, SerializationHeader header, SerializationHelper helper, boolean hasComplexDeletion)
+    private void skipComplexColumn(DataInputPlus in, ColumnDefinition column, SerializationHeader header, boolean hasComplexDeletion)
     throws IOException
     {
         if (hasComplexDeletion)
             UnfilteredRowIteratorSerializer.skipDelTime(in, header);
 
-        while (skipCell(column, in, header));
+        while (Cell.serializer.skip(in, column, header));
     }
 
     public static boolean isEndOfPartition(int flags)
@@ -556,151 +554,4 @@ public class UnfilteredSerializer
     {
         return (flags & IS_MARKER) == 0 && (flags & IS_STATIC) != 0;
     }
-
-    private void writeCell(Cell cell, SerializationHeader header, DataOutputPlus out, LivenessInfo rowLiveness)
-    throws IOException
-    {
-        if (cell == null)
-        {
-            out.writeByte((byte)0);
-            return;
-        }
-
-        boolean hasValue = cell.value().hasRemaining();
-        boolean isDeleted = cell.isTombstone();
-        boolean isExpiring = cell.isExpiring();
-        boolean useRowTimestamp = rowLiveness.hasTimestamp() && cell.livenessInfo().timestamp() == rowLiveness.timestamp();
-        boolean useRowTTL = isExpiring && rowLiveness.hasTTL() && cell.livenessInfo().ttl() == rowLiveness.ttl() && cell.livenessInfo().localDeletionTime() == rowLiveness.localDeletionTime();
-        int flags = PRESENCE_MASK;
-        if (!hasValue)
-            flags |= EMPTY_VALUE_MASK;
-
-        if (isDeleted)
-            flags |= DELETION_MASK;
-        else if (isExpiring)
-            flags |= EXPIRATION_MASK;
-
-        if (useRowTimestamp)
-            flags |= USE_ROW_TIMESTAMP;
-        if (useRowTTL)
-            flags |= USE_ROW_TTL;
-
-        out.writeByte((byte)flags);
-
-        if (hasValue)
-            header.getType(cell.column()).writeValue(cell.value(), out);
-
-        if (!useRowTimestamp)
-            out.writeLong(header.encodeTimestamp(cell.livenessInfo().timestamp()));
-
-        if ((isDeleted || isExpiring) && !useRowTTL)
-            out.writeInt(header.encodeDeletionTime(cell.livenessInfo().localDeletionTime()));
-        if (isExpiring && !useRowTTL)
-            out.writeInt(header.encodeTTL(cell.livenessInfo().ttl()));
-
-        if (cell.column().isComplex())
-            cell.column().cellPathSerializer().serialize(cell.path(), out);
-    }
-
-    private long sizeOfCell(Cell cell, SerializationHeader header, LivenessInfo rowLiveness)
-    {
-        long size = 1; // flags
-
-        if (cell == null)
-            return size;
-
-        boolean hasValue = cell.value().hasRemaining();
-        boolean isDeleted = cell.isTombstone();
-        boolean isExpiring = cell.isExpiring();
-        boolean useRowTimestamp = rowLiveness.hasTimestamp() && cell.livenessInfo().timestamp() == rowLiveness.timestamp();
-        boolean useRowTTL = isExpiring && rowLiveness.hasTTL() && cell.livenessInfo().ttl() == rowLiveness.ttl() && cell.livenessInfo().localDeletionTime() == rowLiveness.localDeletionTime();
-
-        if (hasValue)
-            size += header.getType(cell.column()).writtenLength(cell.value());
-
-        if (!useRowTimestamp)
-            size += TypeSizes.sizeof(header.encodeTimestamp(cell.livenessInfo().timestamp()));
-
-        if ((isDeleted || isExpiring) && !useRowTTL)
-            size += TypeSizes.sizeof(header.encodeDeletionTime(cell.livenessInfo().localDeletionTime()));
-        if (isExpiring && !useRowTTL)
-            size += TypeSizes.sizeof(header.encodeTTL(cell.livenessInfo().ttl()));
-
-        if (cell.column().isComplex())
-            size += cell.column().cellPathSerializer().serializedSize(cell.path());
-
-        return size;
-    }
-
-    private boolean readCell(ColumnDefinition column, DataInput in, SerializationHeader header, SerializationHelper helper, Row.Writer writer)
-    throws IOException
-    {
-        int flags = in.readUnsignedByte();
-        if ((flags & PRESENCE_MASK) == 0)
-            return false;
-
-        boolean hasValue = (flags & EMPTY_VALUE_MASK) == 0;
-        boolean isDeleted = (flags & DELETION_MASK) != 0;
-        boolean isExpiring = (flags & EXPIRATION_MASK) != 0;
-        boolean useRowTimestamp = (flags & USE_ROW_TIMESTAMP) != 0;
-        boolean useRowTTL = (flags & USE_ROW_TTL) != 0;
-
-        ByteBuffer value = ByteBufferUtil.EMPTY_BYTE_BUFFER;
-        if (hasValue)
-        {
-            if (helper.canSkipValue(column))
-                header.getType(column).skipValue(in);
-            else
-                value = header.getType(column).readValue(in);
-        }
-
-        long timestamp = useRowTimestamp ? helper.getRowTimestamp() : header.decodeTimestamp(in.readLong());
-
-        int localDelTime = useRowTTL
-                         ? helper.getRowLocalDeletionTime()
-                         : (isDeleted || isExpiring ? header.decodeDeletionTime(in.readInt()) : LivenessInfo.NO_DELETION_TIME);
-
-        int ttl = useRowTTL
-                ? helper.getRowTTL()
-                : (isExpiring ? header.decodeTTL(in.readInt()) : LivenessInfo.NO_TTL);
-
-        CellPath path = column.isComplex()
-                      ? column.cellPathSerializer().deserialize(in)
-                      : null;
-
-        helper.writeCell(writer, column, false, value, timestamp, localDelTime, ttl, path);
-
-        return true;
-    }
-
-    private boolean skipCell(ColumnDefinition column, DataInput in, SerializationHeader header)
-    throws IOException
-    {
-        int flags = in.readUnsignedByte();
-        if ((flags & PRESENCE_MASK) == 0)
-            return false;
-
-        boolean hasValue = (flags & EMPTY_VALUE_MASK) == 0;
-        boolean isDeleted = (flags & DELETION_MASK) != 0;
-        boolean isExpiring = (flags & EXPIRATION_MASK) != 0;
-        boolean useRowTimestamp = (flags & USE_ROW_TIMESTAMP) != 0;
-        boolean useRowTTL = (flags & USE_ROW_TTL) != 0;
-
-        if (hasValue)
-            header.getType(column).skipValue(in);
-
-        if (!useRowTimestamp)
-            in.readLong();
-
-        if (!useRowTTL && (isDeleted || isExpiring))
-            in.readInt();
-
-        if (!useRowTTL && isExpiring)
-            in.readInt();
-
-        if (column.isComplex())
-            column.cellPathSerializer().skip(in);
-
-        return true;
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/WrappingRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/WrappingRow.java b/src/java/org/apache/cassandra/db/rows/WrappingRow.java
deleted file mode 100644
index 5a0cc78..0000000
--- a/src/java/org/apache/cassandra/db/rows/WrappingRow.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * 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.rows;
-
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import com.google.common.collect.UnmodifiableIterator;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.utils.SearchIterator;
-
-public abstract class WrappingRow extends AbstractRow
-{
-    protected Row wrapped;
-
-    private final ReusableIterator cellIterator = new ReusableIterator();
-    private final ReusableSearchIterator cellSearchIterator = new ReusableSearchIterator();
-
-    /**
-     * Apply some filtering/transformation on cells. This function
-     * can return {@code null} in which case the cell will be ignored.
-     */
-    protected abstract Cell filterCell(Cell cell);
-
-    protected DeletionTime filterDeletionTime(DeletionTime deletionTime)
-    {
-        return deletionTime;
-    }
-
-    protected ColumnData filterColumnData(ColumnData data)
-    {
-        if (data.column().isComplex())
-        {
-            Iterator<Cell> cells = cellIterator.setTo(data.cells());
-            DeletionTime dt = filterDeletionTime(data.complexDeletion());
-            return cells == null && dt.isLive()
-                 ? null
-                 : new ColumnData(data.column(), null, cells == null ? Collections.emptyIterator(): cells, dt);
-        }
-        else
-        {
-            Cell filtered = filterCell(data.cell());
-            return filtered == null ? null : new ColumnData(data.column(), filtered, null, null);
-        }
-    }
-
-    public WrappingRow setTo(Row row)
-    {
-        this.wrapped = row;
-        return this;
-    }
-
-    public Unfiltered.Kind kind()
-    {
-        return Unfiltered.Kind.ROW;
-    }
-
-    public Clustering clustering()
-    {
-        return wrapped.clustering();
-    }
-
-    public Columns columns()
-    {
-        return wrapped.columns();
-    }
-
-    public LivenessInfo primaryKeyLivenessInfo()
-    {
-        return wrapped.primaryKeyLivenessInfo();
-    }
-
-    public DeletionTime deletion()
-    {
-        return wrapped.deletion();
-    }
-
-    public boolean hasComplexDeletion()
-    {
-        // Note that because cells can be filtered out/transformed through
-        // filterCell(), we can't rely on wrapped.hasComplexDeletion().
-        for (int i = 0; i < columns().complexColumnCount(); i++)
-            if (!getDeletion(columns().getComplex(i)).isLive())
-                return true;
-        return false;
-    }
-
-    public Cell getCell(ColumnDefinition c)
-    {
-        Cell cell = wrapped.getCell(c);
-        return cell == null ? null : filterCell(cell);
-    }
-
-    public Cell getCell(ColumnDefinition c, CellPath path)
-    {
-        Cell cell = wrapped.getCell(c, path);
-        return cell == null ? null : filterCell(cell);
-    }
-
-    public Iterator<Cell> getCells(ColumnDefinition c)
-    {
-        Iterator<Cell> cells = wrapped.getCells(c);
-        if (cells == null)
-            return null;
-
-        cellIterator.setTo(cells);
-        return cellIterator.hasNext() ? cellIterator : null;
-    }
-
-    public DeletionTime getDeletion(ColumnDefinition c)
-    {
-        return filterDeletionTime(wrapped.getDeletion(c));
-    }
-
-    public Iterator<Cell> iterator()
-    {
-        return cellIterator.setTo(wrapped.iterator());
-    }
-
-    public SearchIterator<ColumnDefinition, ColumnData> searchIterator()
-    {
-        return cellSearchIterator.setTo(wrapped.searchIterator());
-    }
-
-    public Row takeAlias()
-    {
-        boolean isCounter = columns().hasCounters();
-        if (isStatic())
-        {
-            StaticRow.Builder builder = StaticRow.builder(columns(), true, isCounter);
-            copyTo(builder);
-            return builder.build();
-        }
-        else
-        {
-            ReusableRow copy = new ReusableRow(clustering().size(), columns(), true, isCounter);
-            copyTo(copy.writer());
-            return copy;
-        }
-    }
-
-    private class ReusableIterator extends UnmodifiableIterator<Cell>
-    {
-        private Iterator<Cell> iter;
-        private Cell next;
-
-        public ReusableIterator setTo(Iterator<Cell> iter)
-        {
-            this.iter = iter;
-            this.next = null;
-            return this;
-        }
-
-        public boolean hasNext()
-        {
-            while (next == null && iter.hasNext())
-                next = filterCell(iter.next());
-            return next != null;
-        }
-
-        public Cell next()
-        {
-            if (next == null && !hasNext())
-                throw new NoSuchElementException();
-
-            Cell result = next;
-            next = null;
-            return result;
-        }
-    };
-
-    private class ReusableSearchIterator implements SearchIterator<ColumnDefinition, ColumnData>
-    {
-        private SearchIterator<ColumnDefinition, ColumnData> iter;
-
-        public ReusableSearchIterator setTo(SearchIterator<ColumnDefinition, ColumnData> iter)
-        {
-            this.iter = iter;
-            return this;
-        }
-
-        public boolean hasNext()
-        {
-            return iter.hasNext();
-        }
-
-        public ColumnData next(ColumnDefinition column)
-        {
-            ColumnData data = iter.next(column);
-            if (data == null)
-                return null;
-
-            return filterColumnData(data);
-        }
-    };
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/WrappingUnfilteredRowIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/WrappingUnfilteredRowIterator.java b/src/java/org/apache/cassandra/db/rows/WrappingUnfilteredRowIterator.java
index 680e502..ff3f82c 100644
--- a/src/java/org/apache/cassandra/db/rows/WrappingUnfilteredRowIterator.java
+++ b/src/java/org/apache/cassandra/db/rows/WrappingUnfilteredRowIterator.java
@@ -17,16 +17,21 @@
  */
 package org.apache.cassandra.db.rows;
 
+import java.util.NoSuchElementException;
+
 import com.google.common.collect.UnmodifiableIterator;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 
 /**
- * Abstract class to make writing atom iterators that wrap another iterator
+ * Abstract class to make writing unfiltered iterators that wrap another iterator
  * easier. By default, the wrapping iterator simply delegate every call to
- * the wrapped iterator so concrete implementations will override some of the
- * methods.
+ * the wrapped iterator so concrete implementations will have to override
+ * some of the methods.
+ * <p>
+ * Note that if most of what you want to do is modifying/filtering the returned
+ * {@code Unfiltered}, {@link AlteringUnfilteredRowIterator} can be a simpler option.
  */
 public abstract class WrappingUnfilteredRowIterator extends UnmodifiableIterator<Unfiltered>  implements UnfilteredRowIterator
 {
@@ -67,6 +72,11 @@ public abstract class WrappingUnfilteredRowIterator extends UnmodifiableIterator
         return wrapped.staticRow();
     }
 
+    public RowStats stats()
+    {
+        return wrapped.stats();
+    }
+
     public boolean hasNext()
     {
         return wrapped.hasNext();
@@ -77,11 +87,6 @@ public abstract class WrappingUnfilteredRowIterator extends UnmodifiableIterator
         return wrapped.next();
     }
 
-    public RowStats stats()
-    {
-        return wrapped.stats();
-    }
-
     public void close()
     {
         wrapped.close();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index 124547a..43e214b 100644
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@ -209,6 +209,7 @@ public class CQLSSTableWriter implements Closeable
         // Note that we asks indexes to not validate values (the last 'false' arg below) because that triggers a 'Keyspace.open'
         // and that forces a lot of initialization that we don't want.
         UpdateParameters params = new UpdateParameters(insert.cfm,
+                                                       insert.updatedColumns(),
                                                        options,
                                                        insert.getTimestamp(now, options),
                                                        insert.getTimeToLive(options),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
index 70ab99c..f1af85c 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
@@ -52,7 +52,7 @@ public class SSTableIdentityIterator extends AbstractIterator<Unfiltered> implem
         try
         {
             this.partitionLevelDeletion = DeletionTime.serializer.deserialize(file);
-            SerializationHelper helper = new SerializationHelper(sstable.descriptor.version.correspondingMessagingVersion(), SerializationHelper.Flag.LOCAL);
+            SerializationHelper helper = new SerializationHelper(sstable.metadata, sstable.descriptor.version.correspondingMessagingVersion(), SerializationHelper.Flag.LOCAL);
             this.iterator = SSTableSimpleIterator.create(sstable.metadata, file, sstable.header, helper, partitionLevelDeletion);
             this.staticRow = iterator.readStaticRow();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
index 9e2faee..56b621a 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.io.sstable;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.io.IOError;
 import java.util.Iterator;
@@ -42,10 +41,10 @@ import org.apache.cassandra.net.MessagingService;
 public abstract class SSTableSimpleIterator extends AbstractIterator<Unfiltered> implements Iterator<Unfiltered>
 {
     protected final CFMetaData metadata;
-    protected final DataInput in;
+    protected final DataInputPlus in;
     protected final SerializationHelper helper;
 
-    private SSTableSimpleIterator(CFMetaData metadata, DataInput in, SerializationHelper helper)
+    private SSTableSimpleIterator(CFMetaData metadata, DataInputPlus in, SerializationHelper helper)
     {
         this.metadata = metadata;
         this.in = in;
@@ -66,37 +65,26 @@ public abstract class SSTableSimpleIterator extends AbstractIterator<Unfiltered>
     {
         private final SerializationHeader header;
 
-        private final ReusableRow row;
-        private final RangeTombstoneMarker.Builder markerBuilder;
+        private final Row.Builder builder;
 
-        private CurrentFormatIterator(CFMetaData metadata, DataInput in, SerializationHeader header, SerializationHelper helper)
+        private CurrentFormatIterator(CFMetaData metadata, DataInputPlus in, SerializationHeader header, SerializationHelper helper)
         {
             super(metadata, in, helper);
             this.header = header;
-
-            int clusteringSize = metadata.comparator.size();
-            Columns regularColumns = header == null ? metadata.partitionColumns().regulars : header.columns().regulars;
-
-            this.row = new ReusableRow(clusteringSize, regularColumns, true, metadata.isCounter());
-            this.markerBuilder = new RangeTombstoneMarker.Builder(clusteringSize);
+            this.builder = ArrayBackedRow.sortedBuilder(helper.fetchedRegularColumns(header));
         }
 
         public Row readStaticRow() throws IOException
         {
-            return header.hasStatic()
-                ? UnfilteredSerializer.serializer.deserializeStaticRow(in, header, helper)
-                : Rows.EMPTY_STATIC_ROW;
+            return header.hasStatic() ? UnfilteredSerializer.serializer.deserializeStaticRow(in, header, helper) : Rows.EMPTY_STATIC_ROW;
         }
 
         protected Unfiltered computeNext()
         {
             try
             {
-                Unfiltered.Kind kind = UnfilteredSerializer.serializer.deserialize(in, header, helper, row.writer(), markerBuilder.reset());
-
-                return kind == null
-                     ? endOfData()
-                     : (kind == Unfiltered.Kind.ROW ? row : markerBuilder.build());
+                Unfiltered unfiltered = UnfilteredSerializer.serializer.deserialize(in, header, helper, builder);
+                return unfiltered == null ? endOfData() : unfiltered;
             }
             catch (IOException e)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index 5dbe52a..ef3bde1 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -32,7 +32,9 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.RowStats;
+import org.apache.cassandra.db.rows.UnfilteredSerializer;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
@@ -58,6 +60,9 @@ class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
     private final long bufferSize;
     private long currentSize;
 
+    // Used to compute the row serialized size
+    private final SerializationHeader header;
+
     private final BlockingQueue<Buffer> writeQueue = new SynchronousQueue<Buffer>();
     private final DiskWriter diskWriter = new DiskWriter();
 
@@ -65,6 +70,7 @@ class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
     {
         super(directory, metadata, partitioner, columns);
         this.bufferSize = bufferSizeInMB * 1024L * 1024L;
+        this.header = new SerializationHeader(metadata, columns, RowStats.NO_STATS);
         diskWriter.start();
     }
 
@@ -76,42 +82,21 @@ class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
         if (previous == null)
         {
             previous = createPartitionUpdate(key);
-            count(PartitionUpdate.serializer.serializedSize(previous, formatType.info.getLatestVersion().correspondingMessagingVersion()));
+            currentSize += PartitionUpdate.serializer.serializedSize(previous, formatType.info.getLatestVersion().correspondingMessagingVersion());
             previous.allowNewUpdates();
             buffer.put(key, previous);
         }
         return previous;
     }
 
-    private void count(long size)
-    {
-        currentSize += size;
-    }
-
-    private void countCell(ColumnDefinition column, ByteBuffer value, LivenessInfo info, CellPath path)
+    private void countRow(Row row)
     {
-        // Note that the accounting of a cell is a bit inaccurate (it doesn't take some of the file format optimization into account)
+        // Note that the accounting of a row is a bit inaccurate (it doesn't take some of the file format optimization into account)
         // and the maintaining of the bufferSize is in general not perfect. This has always been the case for this class but we should
         // improve that. In particular, what we count is closer to the serialized value, but it's debatable that it's the right thing
         // to count since it will take a lot more space in memory and the bufferSize if first and foremost used to avoid OOM when
         // using this writer.
-
-        count(1); // Each cell has a byte flag on disk
-
-        if (value.hasRemaining())
-            count(column.type.writtenLength(value));
-
-        count(8); // timestamp
-        if (info.hasLocalDeletionTime())
-            count(4);
-        if (info.hasTTL())
-            count(4);
-
-        if (path != null)
-        {
-            assert path.size() == 1;
-            count(2 + path.get(0).remaining());
-        }
+        currentSize += UnfilteredSerializer.serializer.serializedSize(row, header, formatType.info.getLatestVersion().correspondingMessagingVersion());
     }
 
     private void maybeSync() throws SyncException
@@ -134,52 +119,11 @@ class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
         return new PartitionUpdate(metadata, key, columns, 4)
         {
             @Override
-            protected StaticWriter createStaticWriter()
+            public void add(Row row)
             {
-                return new StaticWriter()
-                {
-                    @Override
-                    public void writeCell(ColumnDefinition column, boolean isCounter, ByteBuffer value, LivenessInfo info, CellPath path)
-                    {
-                        super.writeCell(column, isCounter, value, info, path);
-                        countCell(column, value, info, path);
-                    }
-
-                    @Override
-                    public void endOfRow()
-                    {
-                        super.endOfRow();
-                        maybeSync();
-                    }
-                };
-            }
-
-            @Override
-            protected Writer createWriter()
-            {
-                return new RegularWriter()
-                {
-                    @Override
-                    public void writeClusteringValue(ByteBuffer value)
-                    {
-                        super.writeClusteringValue(value);
-                        count(2 + value.remaining());
-                    }
-
-                    @Override
-                    public void writeCell(ColumnDefinition column, boolean isCounter, ByteBuffer value, LivenessInfo info, CellPath path)
-                    {
-                        super.writeCell(column, isCounter, value, info, path);
-                        countCell(column, value, info, path);
-                    }
-
-                    @Override
-                    public void endOfRow()
-                    {
-                        super.endOfRow();
-                        maybeSync();
-                    }
-                };
+                super.add(row);
+                countRow(row);
+                maybeSync();
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
index a991d99..6759293 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.io.sstable.format.big;
 
 import java.io.*;
-import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
@@ -30,7 +29,6 @@ import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.dht.IPartitioner;
@@ -54,11 +52,8 @@ public class BigTableWriter extends SSTableWriter
 {
     private static final Logger logger = LoggerFactory.getLogger(BigTableWriter.class);
 
-    // not very random, but the only value that can't be mistaken for a legal column-name length
-    public static final int END_OF_ROW = 0x0000;
-
     private final IndexWriter iwriter;
-    private SegmentedFile.Builder dbuilder;
+    private final SegmentedFile.Builder dbuilder;
     private final SequentialWriter dataFile;
     private DecoratedKey lastWrittenKey;
     private FileMark dataMark;
@@ -101,8 +96,8 @@ public class BigTableWriter extends SSTableWriter
     private long beforeAppend(DecoratedKey decoratedKey)
     {
         assert decoratedKey != null : "Keys must not be null"; // empty keys ARE allowed b/c of indexed column values
-        if (lastWrittenKey != null && lastWrittenKey.compareTo(decoratedKey) >= 0)
-            throw new RuntimeException("Last written key " + lastWrittenKey + " >= current key " + decoratedKey + " writing into " + getFilename());
+        //if (lastWrittenKey != null && lastWrittenKey.compareTo(decoratedKey) >= 0)
+        //    throw new RuntimeException("Last written key " + lastWrittenKey + " >= current key " + decoratedKey + " writing into " + getFilename());
         return (lastWrittenKey == null) ? 0 : dataFile.getFilePointer();
     }
 
@@ -172,11 +167,10 @@ public class BigTableWriter extends SSTableWriter
         }
     }
 
-    private static class StatsCollector extends WrappingUnfilteredRowIterator
+    private static class StatsCollector extends AlteringUnfilteredRowIterator
     {
-        private int cellCount;
         private final MetadataCollector collector;
-        private final Set<ColumnDefinition> complexColumnsSetInRow = new HashSet<>();
+        private int cellCount;
 
         StatsCollector(UnfilteredRowIterator iter, MetadataCollector collector)
         {
@@ -186,55 +180,36 @@ public class BigTableWriter extends SSTableWriter
         }
 
         @Override
-        public Unfiltered next()
+        protected Row computeNextStatic(Row row)
+        {
+            if (!row.isEmpty())
+                cellCount += Rows.collectStats(row, collector);
+            return row;
+        }
+
+        @Override
+        protected Row computeNext(Row row)
         {
-            Unfiltered unfiltered = super.next();
-            collector.updateClusteringValues(unfiltered.clustering());
+            collector.updateClusteringValues(row.clustering());
+            cellCount += Rows.collectStats(row, collector);
+            return row;
+        }
 
-            switch (unfiltered.kind())
+        @Override
+        protected RangeTombstoneMarker computeNext(RangeTombstoneMarker marker)
+        {
+            collector.updateClusteringValues(marker.clustering());
+            if (marker.isBoundary())
+            {
+                RangeTombstoneBoundaryMarker bm = (RangeTombstoneBoundaryMarker)marker;
+                collector.update(bm.endDeletionTime());
+                collector.update(bm.startDeletionTime());
+            }
+            else
             {
-                case ROW:
-                    Row row = (Row) unfiltered;
-                    collector.update(row.primaryKeyLivenessInfo());
-                    collector.update(row.deletion());
-
-                    int simpleColumnsSet = 0;
-                    complexColumnsSetInRow.clear();
-
-                    for (Cell cell : row)
-                    {
-                        if (cell.column().isComplex())
-                            complexColumnsSetInRow.add(cell.column());
-                        else
-                            ++simpleColumnsSet;
-
-                        ++cellCount;
-                        collector.update(cell.livenessInfo());
-
-                        if (cell.isCounterCell())
-                            collector.updateHasLegacyCounterShards(CounterCells.hasLegacyShards(cell));
-                    }
-
-                    for (int i = 0; i < row.columns().complexColumnCount(); i++)
-                        collector.update(row.getDeletion(row.columns().getComplex(i)));
-
-                    collector.updateColumnSetPerRow(simpleColumnsSet + complexColumnsSetInRow.size());
-
-                    break;
-                case RANGE_TOMBSTONE_MARKER:
-                    if (((RangeTombstoneMarker) unfiltered).isBoundary())
-                    {
-                        RangeTombstoneBoundaryMarker bm = (RangeTombstoneBoundaryMarker)unfiltered;
-                        collector.update(bm.endDeletionTime());
-                        collector.update(bm.startDeletionTime());
-                    }
-                    else
-                    {
-                        collector.update(((RangeTombstoneBoundMarker)unfiltered).deletionTime());
-                    }
-                    break;
+                collector.update(((RangeTombstoneBoundMarker)marker).deletionTime());
             }
-            return unfiltered;
+            return marker;
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
index 2574c62..9b06b53 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
@@ -33,6 +33,8 @@ import com.clearspring.analytics.stream.cardinality.ICardinality;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.partitions.PartitionStatisticsCollector;
+import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.SSTable;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
@@ -41,7 +43,7 @@ import org.apache.cassandra.utils.EstimatedHistogram;
 import org.apache.cassandra.utils.MurmurHash;
 import org.apache.cassandra.utils.StreamingHistogram;
 
-public class MetadataCollector
+public class MetadataCollector implements PartitionStatisticsCollector
 {
     public static final double NO_COMPRESSION_RATIO = -1.0;
 
@@ -89,8 +91,8 @@ public class MetadataCollector
     protected EstimatedHistogram estimatedCellPerPartitionCount = defaultCellPerPartitionCountHistogram();
     protected ReplayPosition replayPosition = ReplayPosition.NONE;
     protected final MinMaxLongTracker timestampTracker = new MinMaxLongTracker();
-    protected final MinMaxIntTracker localDeletionTimeTracker = new MinMaxIntTracker(LivenessInfo.NO_DELETION_TIME, LivenessInfo.NO_DELETION_TIME);
-    protected final MinMaxIntTracker ttlTracker = new MinMaxIntTracker(LivenessInfo.NO_TTL, LivenessInfo.NO_TTL);
+    protected final MinMaxIntTracker localDeletionTimeTracker = new MinMaxIntTracker(Cell.NO_DELETION_TIME, Cell.NO_DELETION_TIME);
+    protected final MinMaxIntTracker ttlTracker = new MinMaxIntTracker(Cell.NO_TTL, Cell.NO_TTL);
     protected double compressionRatio = NO_COMPRESSION_RATIO;
     protected Set<Integer> ancestors = new HashSet<>();
     protected StreamingHistogram estimatedTombstoneDropTime = defaultTombstoneDropTimeHistogram();
@@ -178,34 +180,39 @@ public class MetadataCollector
         return this;
     }
 
-    public MetadataCollector update(LivenessInfo newInfo)
+    public void update(LivenessInfo newInfo)
     {
-        // If the info doesn't have a timestamp, this means the info is basically irrelevant (it's a row
-        // update whose only info we care are the cells info basically).
-        if (newInfo.hasTimestamp())
+        if (newInfo.isEmpty())
+            return;
+
+        updateTimestamp(newInfo.timestamp());
+        if (newInfo.isExpiring())
         {
-            updateTimestamp(newInfo.timestamp());
             updateTTL(newInfo.ttl());
-            updateLocalDeletionTime(newInfo.localDeletionTime());
+            updateLocalDeletionTime(newInfo.localExpirationTime());
         }
-        return this;
     }
 
-    public MetadataCollector update(DeletionTime dt)
+    public void update(Cell cell)
+    {
+        updateTimestamp(cell.timestamp());
+        updateTTL(cell.ttl());
+        updateLocalDeletionTime(cell.localDeletionTime());
+    }
+
+    public void update(DeletionTime dt)
     {
         if (!dt.isLive())
         {
             updateTimestamp(dt.markedForDeleteAt());
             updateLocalDeletionTime(dt.localDeletionTime());
         }
-        return this;
     }
 
-    public MetadataCollector updateColumnSetPerRow(long columnSetInRow)
+    public void updateColumnSetPerRow(long columnSetInRow)
     {
         totalColumnsSet += columnSetInRow;
         ++totalRows;
-        return this;
     }
 
     private void updateTimestamp(long newTimestamp)
@@ -279,10 +286,9 @@ public class MetadataCollector
         return b2;
     }
 
-    public MetadataCollector updateHasLegacyCounterShards(boolean hasLegacyCounterShards)
+    public void updateHasLegacyCounterShards(boolean hasLegacyCounterShards)
     {
         this.hasLegacyCounterShards = this.hasLegacyCounterShards || hasLegacyCounterShards;
-        return this;
     }
 
     public Map<MetadataType, MetadataComponent> finalizeMetadata(String partitioner, double bloomFilterFPChance, long repairedAt, SerializationHeader header)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java b/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
index 879a505..50644bb 100644
--- a/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
+++ b/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
@@ -371,18 +371,18 @@ public final class LegacySchemaMigrator
 
         if (isSuper)
         {
-            defs.add(ColumnDefinition.regularDef(ksName, cfName, CompactTables.SUPER_COLUMN_MAP_COLUMN_STR, MapType.getInstance(subComparator, defaultValidator, true), null));
+            defs.add(ColumnDefinition.regularDef(ksName, cfName, CompactTables.SUPER_COLUMN_MAP_COLUMN_STR, MapType.getInstance(subComparator, defaultValidator, true)));
         }
         else if (isStaticCompactTable)
         {
             defs.add(ColumnDefinition.clusteringKeyDef(ksName, cfName, names.defaultClusteringName(), rawComparator, null));
-            defs.add(ColumnDefinition.regularDef(ksName, cfName, names.defaultCompactValueName(), defaultValidator, null));
+            defs.add(ColumnDefinition.regularDef(ksName, cfName, names.defaultCompactValueName(), defaultValidator));
         }
         else
         {
             // For dense compact tables, we get here if we don't have a compact value column, in which case we should add it
             // (we use EmptyType to recognize that the compact value was not declared by the use (see CreateTableStatement too))
-            defs.add(ColumnDefinition.regularDef(ksName, cfName, names.defaultCompactValueName(), EmptyType.instance, null));
+            defs.add(ColumnDefinition.regularDef(ksName, cfName, names.defaultCompactValueName(), EmptyType.instance));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
index 4228a46..2eb0ac0 100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@ -353,7 +353,7 @@ public final class SchemaKeyspace
                         mutationMap.put(key, mutation);
                     }
 
-                    mutation.add(UnfilteredRowIterators.toUpdate(partition));
+                    mutation.add(PartitionUpdate.fromIterator(partition));
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/service/DataResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/DataResolver.java b/src/java/org/apache/cassandra/service/DataResolver.java
index f164a60..a9024e3 100644
--- a/src/java/org/apache/cassandra/service/DataResolver.java
+++ b/src/java/org/apache/cassandra/service/DataResolver.java
@@ -39,7 +39,7 @@ import org.apache.cassandra.utils.FBUtilities;
 
 public class DataResolver extends ResponseResolver
 {
-    private final List<AsyncOneResponse> repairResults = Collections.synchronizedList(new ArrayList<AsyncOneResponse>());
+    private final List<AsyncOneResponse> repairResults = Collections.synchronizedList(new ArrayList<>());
 
     public DataResolver(Keyspace keyspace, ReadCommand command, ConsistencyLevel consistency, int maxResponseCount)
     {
@@ -162,9 +162,8 @@ public class DataResolver extends ResponseResolver
             private final boolean isReversed;
             private final PartitionUpdate[] repairs = new PartitionUpdate[sources.length];
 
-            private final Row.Writer[] currentRows = new Row.Writer[sources.length];
-            private Clustering currentClustering;
-            private ColumnDefinition currentColumn;
+            private final Row.Builder[] currentRows = new Row.Builder[sources.length];
+            private final RowDiffListener diffListener;
 
             private final Slice.Bound[] markerOpen = new Slice.Bound[sources.length];
             private final DeletionTime[] markerTime = new DeletionTime[sources.length];
@@ -174,87 +173,75 @@ public class DataResolver extends ResponseResolver
                 this.partitionKey = partitionKey;
                 this.columns = columns;
                 this.isReversed = isReversed;
-            }
 
-            private PartitionUpdate update(int i)
-            {
-                PartitionUpdate upd = repairs[i];
-                if (upd == null)
+                this.diffListener = new RowDiffListener()
                 {
-                    upd = new PartitionUpdate(command.metadata(), partitionKey, columns, 1);
-                    repairs[i] = upd;
-                }
-                return upd;
-            }
+                    public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original)
+                    {
+                        if (merged != null && !merged.equals(original))
+                            currentRow(i, clustering).addPrimaryKeyLivenessInfo(merged);
+                    }
 
-            private Row.Writer currentRow(int i)
-            {
-                Row.Writer row = currentRows[i];
-                if (row == null)
-                {
-                    row = currentClustering == Clustering.STATIC_CLUSTERING ? update(i).staticWriter() : update(i).writer();
-                    currentClustering.writeTo(row);
-                    currentRows[i] = row;
-                }
-                return row;
-            }
+                    public void onDeletion(int i, Clustering clustering, DeletionTime merged, DeletionTime original)
+                    {
+                        if (merged != null && !merged.equals(original))
+                            currentRow(i, clustering).addRowDeletion(merged);
+                    }
 
-            public void onMergePartitionLevelDeletion(DeletionTime mergedDeletion, DeletionTime[] versions)
-            {
-                for (int i = 0; i < versions.length; i++)
-                {
-                    DeletionTime version = versions[i];
-                    if (mergedDeletion.supersedes(versions[i]))
-                        update(i).addPartitionDeletion(mergedDeletion);
-                }
-            }
+                    public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
+                    {
+                        if (merged != null && !merged.equals(original))
+                            currentRow(i, clustering).addComplexDeletion(column, merged);
+                    }
 
-            public void onMergingRows(Clustering clustering,
-                                      LivenessInfo mergedInfo,
-                                      DeletionTime mergedDeletion,
-                                      Row[] versions)
-            {
-                currentClustering = clustering;
-                for (int i = 0; i < versions.length; i++)
-                {
-                    Row version = versions[i];
+                    public void onCell(int i, Clustering clustering, Cell merged, Cell original)
+                    {
+                        if (merged != null && !merged.equals(original))
+                            currentRow(i, clustering).addCell(merged);
+                    }
 
-                    if (version == null || mergedInfo.supersedes(version.primaryKeyLivenessInfo()))
-                        currentRow(i).writePartitionKeyLivenessInfo(mergedInfo);
+                };
+            }
 
-                    if (version == null || mergedDeletion.supersedes(version.deletion()))
-                        currentRow(i).writeRowDeletion(mergedDeletion);
-                }
+            private PartitionUpdate update(int i)
+            {
+                if (repairs[i] == null)
+                    repairs[i] = new PartitionUpdate(command.metadata(), partitionKey, columns, 1);
+                return repairs[i];
             }
 
-            public void onMergedComplexDeletion(ColumnDefinition c, DeletionTime mergedCompositeDeletion, DeletionTime[] versions)
+            private Row.Builder currentRow(int i, Clustering clustering)
             {
-                currentColumn = c;
-                for (int i = 0; i < versions.length; i++)
+                if (currentRows[i] == null)
                 {
-                    DeletionTime version = versions[i] == null ? DeletionTime.LIVE : versions[i];
-                    if (mergedCompositeDeletion.supersedes(version))
-                        currentRow(i).writeComplexDeletion(c, mergedCompositeDeletion);
+                    currentRows[i] = ArrayBackedRow.sortedBuilder(clustering == Clustering.STATIC_CLUSTERING ? columns.statics : columns.regulars);
+                    currentRows[i].newRow(clustering);
                 }
+                return currentRows[i];
             }
 
-            public void onMergedCells(Cell mergedCell, Cell[] versions)
+            public void onMergedPartitionLevelDeletion(DeletionTime mergedDeletion, DeletionTime[] versions)
             {
                 for (int i = 0; i < versions.length; i++)
                 {
-                    Cell version = versions[i];
-                    Cell toAdd = version == null ? mergedCell : Cells.diff(mergedCell, version);
-                    if (toAdd != null)
-                        toAdd.writeTo(currentRow(i));
+                    if (mergedDeletion.supersedes(versions[i]))
+                        update(i).addPartitionDeletion(mergedDeletion);
                 }
             }
 
-            public void onRowDone()
+            public void onMergedRows(Row merged, Columns columns, Row[] versions)
             {
+                // If a row was shadowed post merged, it must be by a partition level or range tombstone, and we handle
+                // those case directly in their respective methods (in other words, it would be inefficient to send a row
+                // deletion as repair when we know we've already send a partition level or range tombstone that covers it).
+                if (merged.isEmpty())
+                    return;
+
+                Rows.diff(merged, columns, versions, diffListener);
                 for (int i = 0; i < currentRows.length; i++)
                 {
                     if (currentRows[i] != null)
-                        currentRows[i].endOfRow();
+                        update(i).add(currentRows[i].build());
                 }
                 Arrays.fill(currentRows, null);
             }
@@ -268,12 +255,12 @@ public class DataResolver extends ResponseResolver
                     if (merged.isClose(isReversed) && markerOpen[i] != null)
                     {
                         Slice.Bound open = markerOpen[i];
-                        Slice.Bound close = merged.isBoundary() ? ((RangeTombstoneBoundaryMarker)merged).createCorrespondingCloseBound(isReversed).clustering() : merged.clustering();
-                        update(i).addRangeTombstone(Slice.make(isReversed ? close : open, isReversed ? open : close), markerTime[i]);
+                        Slice.Bound close = merged.closeBound(isReversed);
+                        update(i).add(new RangeTombstone(Slice.make(isReversed ? close : open, isReversed ? open : close), markerTime[i]));
                     }
                     if (merged.isOpen(isReversed) && (marker == null || merged.openDeletionTime(isReversed).supersedes(marker.openDeletionTime(isReversed))))
                     {
-                        markerOpen[i] = merged.isBoundary() ? ((RangeTombstoneBoundaryMarker)merged).createCorrespondingOpenBound(isReversed).clustering() : merged.clustering();
+                        markerOpen[i] = merged.openBound(isReversed);
                         markerTime[i] = merged.openDeletionTime(isReversed);
                     }
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java b/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java
index 6429be0..6c08be0 100644
--- a/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java
@@ -96,7 +96,7 @@ public class RangeSliceQueryPager extends AbstractQueryPager
         if (last != null)
         {
             lastReturnedKey = key;
-            lastReturnedClustering = last.clustering().takeAlias();
+            lastReturnedClustering = last.clustering();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/service/pager/SinglePartitionPager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/SinglePartitionPager.java b/src/java/org/apache/cassandra/service/pager/SinglePartitionPager.java
index 6488641..223c3fd 100644
--- a/src/java/org/apache/cassandra/service/pager/SinglePartitionPager.java
+++ b/src/java/org/apache/cassandra/service/pager/SinglePartitionPager.java
@@ -80,6 +80,6 @@ public class SinglePartitionPager extends AbstractQueryPager
     protected void recordLast(DecoratedKey key, Row last)
     {
         if (last != null)
-            lastReturned = last.clustering().takeAlias();
+            lastReturned = last.clustering();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/service/paxos/Commit.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/paxos/Commit.java b/src/java/org/apache/cassandra/service/paxos/Commit.java
index 9a5e619..579c315 100644
--- a/src/java/org/apache/cassandra/service/paxos/Commit.java
+++ b/src/java/org/apache/cassandra/service/paxos/Commit.java
@@ -27,7 +27,6 @@ import java.util.UUID;
 import com.google.common.base.Objects;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
@@ -63,7 +62,8 @@ public class Commit
 
     public static Commit newProposal(UUID ballot, PartitionUpdate update)
     {
-        return new Commit(ballot, updatesWithPaxosTime(update, ballot));
+        update.updateAllTimestamp(UUIDGen.microsTimestamp(ballot));
+        return new Commit(ballot, update);
     }
 
     public static Commit emptyCommit(DecoratedKey key, CFMetaData metadata)
@@ -83,7 +83,6 @@ public class Commit
 
     public Mutation makeMutation()
     {
-        assert update != null;
         return new Mutation(update);
     }
 
@@ -95,10 +94,7 @@ public class Commit
 
         Commit commit = (Commit) o;
 
-        if (!ballot.equals(commit.ballot)) return false;
-        if (!update.equals(commit.update)) return false;
-
-        return true;
+        return ballot.equals(commit.ballot) && update.equals(commit.update);
     }
 
     @Override
@@ -107,46 +103,6 @@ public class Commit
         return Objects.hashCode(ballot, update);
     }
 
-    private static PartitionUpdate updatesWithPaxosTime(PartitionUpdate update, UUID ballot)
-    {
-        long t = UUIDGen.microsTimestamp(ballot);
-        // Using t-1 for tombstones so deletion doesn't trump newly inserted data (#6069)
-        PartitionUpdate newUpdate = new PartitionUpdate(update.metadata(),
-                                                        update.partitionKey(),
-                                                        update.deletionInfo().updateAllTimestamp(t-1),
-                                                        update.columns(),
-                                                        update.rowCount());
-
-        if (!update.staticRow().isEmpty())
-            copyWithUpdatedTimestamp(update.staticRow(), newUpdate.staticWriter(), t);
-
-        for (Row row : update)
-            copyWithUpdatedTimestamp(row, newUpdate.writer(), t);
-
-        return newUpdate;
-    }
-
-    private static void copyWithUpdatedTimestamp(Row row, Row.Writer writer, long timestamp)
-    {
-        Rows.writeClustering(row.clustering(), writer);
-        writer.writePartitionKeyLivenessInfo(row.primaryKeyLivenessInfo().withUpdatedTimestamp(timestamp));
-        writer.writeRowDeletion(row.deletion());
-
-        for (Cell cell : row)
-            writer.writeCell(cell.column(), cell.isCounterCell(), cell.value(), cell.livenessInfo().withUpdatedTimestamp(timestamp), cell.path());
-
-        for (int i = 0; i < row.columns().complexColumnCount(); i++)
-        {
-            ColumnDefinition c = row.columns().getComplex(i);
-            DeletionTime dt = row.getDeletion(c);
-            // We use t-1 to make sure that on inserting a collection literal, the deletion that comes with it does not
-            // end up deleting the inserted data (see #6069)
-            if (!dt.isLive())
-                writer.writeComplexDeletion(c, new SimpleDeletionTime(timestamp-1, dt.localDeletionTime()));
-        }
-        writer.endOfRow();
-    }
-
     @Override
     public String toString()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReader.java b/src/java/org/apache/cassandra/streaming/StreamReader.java
index 2876f08..ee646aa 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.streaming;
 
 import java.io.*;
-import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
 import java.nio.channels.ReadableByteChannel;
 import java.util.Collection;
@@ -36,7 +35,6 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
-import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableSimpleIterator;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
@@ -184,16 +182,13 @@ public class StreamReader
         private Row staticRow;
         private IOException exception;
 
-        private final CounterFilteredRow counterRow;
-
         public StreamDeserializer(CFMetaData metadata, DataInputPlus in, Version version, SerializationHeader header)
         {
             assert version.storeRows() : "We don't allow streaming from pre-3.0 nodes";
             this.metadata = metadata;
             this.in = in;
-            this.helper = new SerializationHelper(version.correspondingMessagingVersion(), SerializationHelper.Flag.PRESERVE_SIZE);
+            this.helper = new SerializationHelper(metadata, version.correspondingMessagingVersion(), SerializationHelper.Flag.PRESERVE_SIZE);
             this.header = header;
-            this.counterRow = metadata.isCounter() ? new CounterFilteredRow() : null;
         }
 
         public DecoratedKey newPartition() throws IOException
@@ -271,9 +266,7 @@ public class StreamReader
 
         private Row maybeMarkLocalToBeCleared(Row row)
         {
-            return metadata.isCounter()
-                 ? counterRow.setTo(row)
-                 : row;
+            return metadata.isCounter() ? row.markCounterLocalToBeCleared() : row;
         }
 
         public void checkForExceptions() throws IOException
@@ -286,18 +279,4 @@ public class StreamReader
         {
         }
     }
-
-    private static class CounterFilteredRow extends WrappingRow
-    {
-        protected Cell filterCell(Cell cell)
-        {
-            if (!cell.isCounterCell())
-                return cell;
-
-            ByteBuffer marked = CounterContext.instance().markLocalToBeCleared(cell.value());
-            return marked == cell.value()
-                 ? cell
-                 : Cells.create(cell.column(), true, marked, cell.livenessInfo(), cell.path());
-        }
-    }
 }


[08/15] cassandra git commit: Simplify some 8099's implementations

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/ArrayBackedRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/ArrayBackedRow.java b/src/java/org/apache/cassandra/db/rows/ArrayBackedRow.java
new file mode 100644
index 0000000..016b4fa
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/rows/ArrayBackedRow.java
@@ -0,0 +1,927 @@
+/*
+ * 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.rows;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.function.Predicate;
+
+import com.google.common.collect.AbstractIterator;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.utils.SearchIterator;
+import org.apache.cassandra.utils.ObjectSizes;
+
+/**
+ * Immutable implementation of a Row object.
+ */
+public class ArrayBackedRow extends AbstractRow
+{
+    private static final ColumnData[] NO_DATA = new ColumnData[0];
+
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new ArrayBackedRow(Clustering.EMPTY, Columns.NONE, LivenessInfo.EMPTY, DeletionTime.LIVE, 0, NO_DATA, Integer.MAX_VALUE));
+
+    private final Clustering clustering;
+    private final Columns columns;
+    private final LivenessInfo primaryKeyLivenessInfo;
+    private final DeletionTime deletion;
+
+    // The data for each columns present in this row in column sorted order.
+    private final int size;
+    private final ColumnData[] data;
+
+    // We need to filter the tombstones of a row on every read (twice in fact: first to remove purgeable tombstone, and then after reconciliation to remove
+    // all tombstone since we don't return them to the client) as well as on compaction. But it's likely that many rows won't have any tombstone at all, so
+    // we want to speed up that case by not having to iterate/copy the row in this case. We could keep a single boolean telling us if we have tombstones,
+    // but that doesn't work for expiring columns. So instead we keep the deletion time for the first thing in the row to be deleted. This allow at any given
+    // time to know if we have any deleted information or not. If we any "true" tombstone (i.e. not an expiring cell), this value will be forced to
+    // Integer.MIN_VALUE, but if we don't and have expiring cells, this will the time at which the first expiring cell expires. If we have no tombstones and
+    // no expiring cells, this will be Integer.MAX_VALUE;
+    private final int minLocalDeletionTime;
+
+    private ArrayBackedRow(Clustering clustering, Columns columns, LivenessInfo primaryKeyLivenessInfo, DeletionTime deletion, int size, ColumnData[] data, int minLocalDeletionTime)
+    {
+        this.clustering = clustering;
+        this.columns = columns;
+        this.primaryKeyLivenessInfo = primaryKeyLivenessInfo;
+        this.deletion = deletion;
+        this.size = size;
+        this.data = data;
+        this.minLocalDeletionTime = minLocalDeletionTime;
+    }
+
+    // Note that it's often easier/safer to use the sortedBuilder/unsortedBuilder or one of the static creation method below. Only directly useful in a small amount of cases.
+    public static ArrayBackedRow create(Clustering clustering, Columns columns, LivenessInfo primaryKeyLivenessInfo, DeletionTime deletion, int size, ColumnData[] data)
+    {
+        int minDeletionTime = Math.min(minDeletionTime(primaryKeyLivenessInfo), minDeletionTime(deletion));
+        if (minDeletionTime != Integer.MIN_VALUE)
+        {
+            for (int i = 0; i < size; i++)
+                minDeletionTime = Math.min(minDeletionTime, minDeletionTime(data[i]));
+        }
+
+        return new ArrayBackedRow(clustering, columns, primaryKeyLivenessInfo, deletion, size, data, minDeletionTime);
+    }
+
+    public static ArrayBackedRow emptyRow(Clustering clustering)
+    {
+        return new ArrayBackedRow(clustering, Columns.NONE, LivenessInfo.EMPTY, DeletionTime.LIVE, 0, NO_DATA, Integer.MAX_VALUE);
+    }
+
+    public static ArrayBackedRow singleCellRow(Clustering clustering, Cell cell)
+    {
+        if (cell.column().isSimple())
+            return new ArrayBackedRow(clustering, Columns.of(cell.column()), LivenessInfo.EMPTY, DeletionTime.LIVE, 1, new ColumnData[]{ cell }, minDeletionTime(cell));
+
+        ComplexColumnData complexData = new ComplexColumnData(cell.column(), new Cell[]{ cell }, DeletionTime.LIVE);
+        return new ArrayBackedRow(clustering, Columns.of(cell.column()), LivenessInfo.EMPTY, DeletionTime.LIVE, 1, new ColumnData[]{ complexData }, minDeletionTime(cell));
+    }
+
+    public static ArrayBackedRow emptyDeletedRow(Clustering clustering, DeletionTime deletion)
+    {
+        assert !deletion.isLive();
+        return new ArrayBackedRow(clustering, Columns.NONE, LivenessInfo.EMPTY, deletion, 0, NO_DATA, Integer.MIN_VALUE);
+    }
+
+    public static ArrayBackedRow noCellLiveRow(Clustering clustering, LivenessInfo primaryKeyLivenessInfo)
+    {
+        assert !primaryKeyLivenessInfo.isEmpty();
+        return new ArrayBackedRow(clustering, Columns.NONE, primaryKeyLivenessInfo, DeletionTime.LIVE, 0, NO_DATA, minDeletionTime(primaryKeyLivenessInfo));
+    }
+
+    private static int minDeletionTime(Cell cell)
+    {
+        return cell.isTombstone() ? Integer.MIN_VALUE : cell.localDeletionTime();
+    }
+
+    private static int minDeletionTime(LivenessInfo info)
+    {
+        return info.isExpiring() ? info.localExpirationTime() : Integer.MAX_VALUE;
+    }
+
+    private static int minDeletionTime(DeletionTime dt)
+    {
+        return dt.isLive() ? Integer.MAX_VALUE : Integer.MIN_VALUE;
+    }
+
+    private static int minDeletionTime(ComplexColumnData cd)
+    {
+        int min = minDeletionTime(cd.complexDeletion());
+        for (Cell cell : cd)
+            min = Math.min(min, minDeletionTime(cell));
+        return min;
+    }
+
+    private static int minDeletionTime(ColumnData cd)
+    {
+        return cd.column().isSimple() ? minDeletionTime((Cell)cd) : minDeletionTime((ComplexColumnData)cd);
+    }
+
+    public Clustering clustering()
+    {
+        return clustering;
+    }
+
+    public Columns columns()
+    {
+        return columns;
+    }
+
+    public LivenessInfo primaryKeyLivenessInfo()
+    {
+        return primaryKeyLivenessInfo;
+    }
+
+    public DeletionTime deletion()
+    {
+        return deletion;
+    }
+
+    public Cell getCell(ColumnDefinition c)
+    {
+        assert !c.isComplex();
+        int idx = binarySearch(c);
+        return idx < 0 ? null : (Cell)data[idx];
+    }
+
+    public Cell getCell(ColumnDefinition c, CellPath path)
+    {
+        assert c.isComplex();
+        int idx = binarySearch(c);
+        if (idx < 0)
+            return null;
+
+        return ((ComplexColumnData)data[idx]).getCell(path);
+    }
+
+    public ComplexColumnData getComplexColumnData(ColumnDefinition c)
+    {
+        assert c.isComplex();
+        int idx = binarySearch(c);
+        return idx < 0 ? null : (ComplexColumnData)data[idx];
+    }
+
+    public Iterator<ColumnData> iterator()
+    {
+        return new ColumnDataIterator();
+    }
+
+    public Iterable<Cell> cells()
+    {
+        return CellIterator::new;
+    }
+
+    public SearchIterator<ColumnDefinition, ColumnData> searchIterator()
+    {
+        return new ColumnSearchIterator();
+    }
+
+    public Row filter(ColumnFilter filter, CFMetaData metadata)
+    {
+        return filter(filter, DeletionTime.LIVE, false, metadata);
+    }
+
+    public Row filter(ColumnFilter filter, DeletionTime activeDeletion, boolean setActiveDeletionToRow, CFMetaData metadata)
+    {
+        Map<ByteBuffer, CFMetaData.DroppedColumn> droppedColumns = metadata.getDroppedColumns();
+
+        if (filter.includesAllColumns() && (activeDeletion.isLive() || deletion.supersedes(activeDeletion)) && droppedColumns.isEmpty())
+            return this;
+
+        boolean mayHaveShadowed = activeDeletion.supersedes(deletion);
+
+        LivenessInfo newInfo = primaryKeyLivenessInfo;
+        DeletionTime newDeletion = deletion;
+        if (mayHaveShadowed)
+        {
+            if (activeDeletion.deletes(newInfo.timestamp()))
+                newInfo = LivenessInfo.EMPTY;
+            // note that mayHaveShadowed means the activeDeletion shadows the row deletion. So if don't have setActiveDeletionToRow,
+            // the row deletion is shadowed and we shouldn't return it.
+            newDeletion = setActiveDeletionToRow ? activeDeletion : DeletionTime.LIVE;
+        }
+
+        ColumnData[] newData = new ColumnData[size];
+        int newMinDeletionTime = Math.min(minDeletionTime(newInfo), minDeletionTime(newDeletion));
+        Columns columns = filter.fetchedColumns().columns(isStatic());
+        Predicate<ColumnDefinition> inclusionTester = columns.inOrderInclusionTester();
+        int newSize = 0;
+        for (int i = 0; i < size; i++)
+        {
+            ColumnData cd = data[i];
+            ColumnDefinition column = cd.column();
+            if (!inclusionTester.test(column))
+                continue;
+
+            CFMetaData.DroppedColumn dropped = droppedColumns.get(column.name.bytes);
+            if (column.isSimple())
+            {
+                Cell cell = (Cell)cd;
+                if ((dropped == null || cell.timestamp() > dropped.droppedTime) && !(mayHaveShadowed && activeDeletion.deletes(cell)))
+                {
+                    newData[newSize++] = cell;
+                    newMinDeletionTime = Math.min(newMinDeletionTime, minDeletionTime(cell));
+                }
+            }
+            else
+            {
+                ColumnData newCd = ((ComplexColumnData)cd).filter(filter, mayHaveShadowed ? activeDeletion : DeletionTime.LIVE, dropped);
+                if (newCd != null)
+                {
+                    newData[newSize++] = newCd;
+                    newMinDeletionTime = Math.min(newMinDeletionTime, minDeletionTime(newCd));
+                }
+            }
+        }
+
+        if (newSize == 0 && newInfo.isEmpty() && newDeletion.isLive())
+            return null;
+
+        return new ArrayBackedRow(clustering, columns, newInfo, newDeletion, newSize, newData, newMinDeletionTime);
+    }
+
+    public boolean hasComplexDeletion()
+    {
+        // We start by the end cause we know complex columns sort before simple ones
+        for (int i = size - 1; i >= 0; i--)
+        {
+            ColumnData cd = data[i];
+            if (cd.column().isSimple())
+                return false;
+
+            if (!((ComplexColumnData)cd).complexDeletion().isLive())
+                return true;
+        }
+        return false;
+    }
+
+    public Row markCounterLocalToBeCleared()
+    {
+        ColumnData[] newData = null;
+        for (int i = 0; i < size; i++)
+        {
+            ColumnData cd = data[i];
+            ColumnData newCd = cd.column().cellValueType().isCounter()
+                             ? cd.markCounterLocalToBeCleared()
+                             : cd;
+            if (newCd != cd)
+            {
+                if (newData == null)
+                    newData = Arrays.copyOf(data, size);
+                newData[i] = newCd;
+            }
+        }
+
+        return newData == null
+             ? this
+             : new ArrayBackedRow(clustering, columns, primaryKeyLivenessInfo, deletion, size, newData, minLocalDeletionTime);
+    }
+
+    public boolean hasDeletion(int nowInSec)
+    {
+        return nowInSec >= minLocalDeletionTime;
+    }
+
+    /**
+     * Returns a copy of the row where all timestamps for live data have replaced by {@code newTimestamp} and
+     * all deletion timestamp by {@code newTimestamp - 1}.
+     *
+     * This exists for the Paxos path, see {@link PartitionUpdate#updateAllTimestamp} for additional details.
+     */
+    public Row updateAllTimestamp(long newTimestamp)
+    {
+        LivenessInfo newInfo = primaryKeyLivenessInfo.isEmpty() ? primaryKeyLivenessInfo : primaryKeyLivenessInfo.withUpdatedTimestamp(newTimestamp);
+        DeletionTime newDeletion = deletion.isLive() ? deletion : new DeletionTime(newTimestamp - 1, deletion.localDeletionTime());
+
+        ColumnData[] newData = new ColumnData[size];
+        for (int i = 0; i < size; i++)
+            newData[i] = data[i].updateAllTimestamp(newTimestamp);
+
+        return new ArrayBackedRow(clustering, columns, newInfo, newDeletion, size, newData, minLocalDeletionTime);
+    }
+
+    public Row purge(DeletionPurger purger, int nowInSec)
+    {
+        if (!hasDeletion(nowInSec))
+            return this;
+
+        LivenessInfo newInfo = purger.shouldPurge(primaryKeyLivenessInfo, nowInSec) ? LivenessInfo.EMPTY : primaryKeyLivenessInfo;
+        DeletionTime newDeletion = purger.shouldPurge(deletion) ? DeletionTime.LIVE : deletion;
+
+        int newMinDeletionTime = Math.min(minDeletionTime(newInfo), minDeletionTime(newDeletion));
+        ColumnData[] newData = new ColumnData[size];
+        int newSize = 0;
+        for (int i = 0; i < size; i++)
+        {
+            ColumnData purged = data[i].purge(purger, nowInSec);
+            if (purged != null)
+            {
+                newData[newSize++] = purged;
+                newMinDeletionTime = Math.min(newMinDeletionTime, minDeletionTime(purged));
+            }
+        }
+
+        if (newSize == 0 && newInfo.isEmpty() && newDeletion.isLive())
+            return null;
+
+        return new ArrayBackedRow(clustering, columns, newInfo, newDeletion, newSize, newData, newMinDeletionTime);
+    }
+
+    public int dataSize()
+    {
+        int dataSize = clustering.dataSize()
+                     + primaryKeyLivenessInfo.dataSize()
+                     + deletion.dataSize();
+
+        for (int i = 0; i < size; i++)
+            dataSize += data[i].dataSize();
+        return dataSize;
+    }
+
+    public long unsharedHeapSizeExcludingData()
+    {
+        long heapSize = EMPTY_SIZE
+                      + clustering.unsharedHeapSizeExcludingData()
+                      + ObjectSizes.sizeOfArray(data);
+
+        for (int i = 0; i < size; i++)
+            heapSize += data[i].unsharedHeapSizeExcludingData();
+        return heapSize;
+    }
+
+    public static Row.Builder sortedBuilder(Columns columns)
+    {
+        return new SortedBuilder(columns);
+    }
+
+    public static Row.Builder unsortedBuilder(Columns columns, int nowInSec)
+    {
+        return new UnsortedBuilder(columns, nowInSec);
+    }
+
+    // This is only used by PartitionUpdate.CounterMark but other uses should be avoided as much as possible as it breaks our general
+    // assumption that Row objects are immutable. This method should go away post-#6506 in particular.
+    // This method is in particular not exposed by the Row API on purpose.
+    // This method also *assumes* that the cell we're setting already exists.
+    public void setValue(ColumnDefinition column, CellPath path, ByteBuffer value)
+    {
+        int idx = binarySearch(column);
+        assert idx >= 0;
+        if (column.isSimple())
+            data[idx] = ((Cell)data[idx]).withUpdatedValue(value);
+        else
+            ((ComplexColumnData)data[idx]).setValue(path, value);
+    }
+
+    private int binarySearch(ColumnDefinition column)
+    {
+        return binarySearch(column, 0, size);
+    }
+
+    /**
+     * Simple binary search for a given column (in the data list).
+     *
+     * The return value has the exact same meaning that the one of Collections.binarySearch() but
+     * we don't use the later because we're searching for a 'ColumnDefinition' in an array of 'ColumnData'.
+     */
+    private int binarySearch(ColumnDefinition column, int fromIndex, int toIndex)
+    {
+        int low = fromIndex;
+        int mid = toIndex;
+        int high = mid - 1;
+        int result = -1;
+        while (low <= high)
+        {
+            mid = (low + high) >> 1;
+            if ((result = column.compareTo(data[mid].column())) > 0)
+                low = mid + 1;
+            else if (result == 0)
+                return mid;
+            else
+                high = mid - 1;
+        }
+        return -mid - (result < 0 ? 1 : 2);
+    }
+
+    private class ColumnDataIterator extends AbstractIterator<ColumnData>
+    {
+        private int i;
+
+        protected ColumnData computeNext()
+        {
+            return i < size ? data[i++] : endOfData();
+        }
+    }
+
+    private class CellIterator extends AbstractIterator<Cell>
+    {
+        private int i;
+        private Iterator<Cell> complexCells;
+
+        protected Cell computeNext()
+        {
+            while (true)
+            {
+                if (complexCells != null)
+                {
+                    if (complexCells.hasNext())
+                        return complexCells.next();
+
+                    complexCells = null;
+                }
+
+                if (i >= size)
+                    return endOfData();
+
+                ColumnData cd = data[i++];
+                if (cd.column().isComplex())
+                    complexCells = ((ComplexColumnData)cd).iterator();
+                else
+                    return (Cell)cd;
+            }
+        }
+    }
+
+    private class ColumnSearchIterator implements SearchIterator<ColumnDefinition, ColumnData>
+    {
+        // The index at which the next call to "next" should start looking from
+        private int searchFrom = 0;
+
+        public boolean hasNext()
+        {
+            return searchFrom < size;
+        }
+
+        public ColumnData next(ColumnDefinition column)
+        {
+            int idx = binarySearch(column, searchFrom, size);
+            if (idx < 0)
+            {
+                searchFrom = -idx - 1;
+                return null;
+            }
+            else
+            {
+                // We've found it. We'll start after it next time.
+                searchFrom = idx + 1;
+                return data[idx];
+            }
+        }
+    }
+
+    private static abstract class AbstractBuilder implements Row.Builder
+    {
+        protected final Columns columns;
+
+        protected Clustering clustering;
+        protected LivenessInfo primaryKeyLivenessInfo;
+        protected DeletionTime deletion;
+
+        protected List<Cell> cells = new ArrayList<>();
+
+        // For complex column at index i of 'columns', we store at complexDeletions[i] its complex deletion.
+        protected DeletionTime[] complexDeletions;
+        protected int columnsWithComplexDeletion;
+
+        protected AbstractBuilder(Columns columns)
+        {
+            this.columns = columns;
+            this.complexDeletions = new DeletionTime[columns.complexColumnCount()];
+        }
+
+        public void newRow(Clustering clustering)
+        {
+            assert cells.isEmpty(); // Ensures we've properly called build() if we've use this builder before
+            this.clustering = clustering;
+        }
+
+        public Clustering clustering()
+        {
+            return clustering;
+        }
+
+        protected void reset()
+        {
+            this.clustering = null;
+            this.primaryKeyLivenessInfo = LivenessInfo.EMPTY;
+            this.deletion = DeletionTime.LIVE;
+            this.cells.clear();
+            Arrays.fill(this.complexDeletions, null);
+            this.columnsWithComplexDeletion = 0;
+        }
+
+        public void addPrimaryKeyLivenessInfo(LivenessInfo info)
+        {
+            this.primaryKeyLivenessInfo = info;
+        }
+
+        public void addRowDeletion(DeletionTime deletion)
+        {
+            this.deletion = deletion;
+        }
+
+        public void addCell(Cell cell)
+        {
+            assert cell.column().isStatic() == (clustering == Clustering.STATIC_CLUSTERING) : "Column is " + cell.column() + ", clustering = " + clustering;
+            cells.add(cell);
+        }
+
+        public Row build()
+        {
+            Row row = buildInternal();
+            reset();
+            return row;
+        }
+
+        protected abstract Row buildInternal();
+
+        protected Row buildNoCells()
+        {
+            assert cells.isEmpty();
+            int minDeletionTime = Math.min(minDeletionTime(primaryKeyLivenessInfo), minDeletionTime(deletion));
+            if (columnsWithComplexDeletion == 0)
+                return new ArrayBackedRow(clustering, columns, primaryKeyLivenessInfo, deletion, 0, NO_DATA, minDeletionTime);
+
+            ColumnData[] data = new ColumnData[columnsWithComplexDeletion];
+            int size = 0;
+            for (int i = 0; i < complexDeletions.length; i++)
+            {
+                DeletionTime complexDeletion = complexDeletions[i];
+                if (complexDeletion != null)
+                {
+                    assert !complexDeletion.isLive();
+                    data[size++] = new ComplexColumnData(columns.getComplex(i), ComplexColumnData.NO_CELLS, complexDeletion);
+                    minDeletionTime = Integer.MIN_VALUE;
+                }
+            }
+            return new ArrayBackedRow(clustering, columns, primaryKeyLivenessInfo, deletion, size, data, minDeletionTime);
+        }
+    }
+
+    public static class SortedBuilder extends AbstractBuilder
+    {
+        private int columnCount;
+
+        private ColumnDefinition column;
+
+        // The index of the last column for which we've called setColumn if complex.
+        private int complexColumnIndex;
+
+        // For complex column at index i of 'columns', we store at complexColumnCellsCount[i] its number of added cells.
+        private final int[] complexColumnCellsCount;
+
+        protected SortedBuilder(Columns columns)
+        {
+            super(columns);
+            this.complexColumnCellsCount = new int[columns.complexColumnCount()];
+            reset();
+        }
+
+        @Override
+        protected void reset()
+        {
+            super.reset();
+            this.column = null;
+            this.columnCount = 0;
+            this.complexColumnIndex = -1;
+            Arrays.fill(this.complexColumnCellsCount, 0);
+        }
+
+        public boolean isSorted()
+        {
+            return true;
+        }
+
+        private void setColumn(ColumnDefinition column)
+        {
+            int cmp = this.column == null ? -1 : this.column.compareTo(column);
+            assert cmp <= 0 : "current = " + this.column + ", new = " + column;
+            if (cmp != 0)
+            {
+                this.column = column;
+                ++columnCount;
+                if (column.isComplex())
+                    complexColumnIndex = columns.complexIdx(column, complexColumnIndex + 1);
+            }
+        }
+
+        @Override
+        public void addCell(Cell cell)
+        {
+            setColumn(cell.column());
+            super.addCell(cell);
+            if (column.isComplex())
+                complexColumnCellsCount[complexColumnIndex] += 1;
+        }
+
+        @Override
+        public void addComplexDeletion(ColumnDefinition column, DeletionTime complexDeletion)
+        {
+            if (complexDeletion.isLive())
+                return;
+
+            setColumn(column);
+            assert complexDeletions[complexColumnIndex] == null;
+            complexDeletions[complexColumnIndex] = complexDeletion;
+            ++columnsWithComplexDeletion;
+        }
+
+        protected Row buildInternal()
+        {
+            if (cells.isEmpty())
+                return buildNoCells();
+
+            int minDeletionTime = Math.min(minDeletionTime(primaryKeyLivenessInfo), minDeletionTime(deletion));
+
+            ColumnData[] data = new ColumnData[columnCount];
+            int complexIdx = 0;
+            int i = 0;
+            int size = 0;
+            while (i < cells.size())
+            {
+                Cell cell = cells.get(i);
+                ColumnDefinition column = cell.column();
+                if (column.isSimple())
+                {
+                    data[size++] = cell;
+                    minDeletionTime = Math.min(minDeletionTime, minDeletionTime(cell));
+                    ++i;
+                }
+                else
+                {
+                    while (columns.getComplex(complexIdx).compareTo(column) < 0)
+                    {
+                        if (complexDeletions[complexIdx] != null)
+                        {
+                            data[size++] = new ComplexColumnData(columns.getComplex(complexIdx), ComplexColumnData.NO_CELLS, complexDeletions[complexIdx]);
+                            minDeletionTime = Integer.MIN_VALUE;
+                        }
+                        ++complexIdx;
+                    }
+
+                    DeletionTime complexDeletion = complexDeletions[complexIdx];
+                    if (complexDeletion != null)
+                        minDeletionTime = Integer.MIN_VALUE;
+                    int cellCount = complexColumnCellsCount[complexIdx];
+                    Cell[] complexCells = new Cell[cellCount];
+                    for (int j = 0; j < cellCount; j++)
+                    {
+                        Cell complexCell = cells.get(i + j);
+                        complexCells[j] = complexCell;
+                        minDeletionTime = Math.min(minDeletionTime, minDeletionTime(complexCell));
+                    }
+                    i += cellCount;
+
+                    data[size++] = new ComplexColumnData(column, complexCells, complexDeletion == null ? DeletionTime.LIVE : complexDeletion);
+                    ++complexIdx;
+                }
+            }
+            for (int j = complexIdx; j < complexDeletions.length; j++)
+            {
+                if (complexDeletions[j] != null)
+                {
+                    data[size++] = new ComplexColumnData(columns.getComplex(j), ComplexColumnData.NO_CELLS, complexDeletions[j]);
+                    minDeletionTime = Integer.MIN_VALUE;
+                }
+            }
+            assert size == data.length;
+            return new ArrayBackedRow(clustering, columns, primaryKeyLivenessInfo, deletion, size, data, minDeletionTime);
+        }
+    }
+
+    private static class UnsortedBuilder extends AbstractBuilder
+    {
+        private final int nowInSec;
+
+        private UnsortedBuilder(Columns columns, int nowInSec)
+        {
+            super(columns);
+            this.nowInSec = nowInSec;
+            reset();
+        }
+
+        public boolean isSorted()
+        {
+            return false;
+        }
+
+        public void addComplexDeletion(ColumnDefinition column, DeletionTime complexDeletion)
+        {
+            assert column.isComplex();
+            assert column.isStatic() == (clustering == Clustering.STATIC_CLUSTERING);
+
+            if (complexDeletion.isLive())
+                return;
+
+            int complexColumnIndex = columns.complexIdx(column, 0);
+
+            DeletionTime previous = complexDeletions[complexColumnIndex];
+            if (previous == null || complexDeletion.supersedes(previous))
+            {
+                complexDeletions[complexColumnIndex] = complexDeletion;
+                if (previous == null)
+                    ++columnsWithComplexDeletion;
+            }
+        }
+
+        protected Row buildInternal()
+        {
+            // First, the easy cases
+            if (cells.isEmpty())
+                return buildNoCells();
+
+            // Cells have been added in an unsorted way, so sort them first
+            Collections.sort(cells, Cell.comparator);
+
+            // We now need to
+            //  1) merge equal cells together
+            //  2) group the cells for a given complex column together, and include their potential complex deletion time.
+            //     And this without forgetting that some complex columns may have a complex deletion but not cells.
+
+            int addedColumns = countAddedColumns();
+            ColumnData[] data = new ColumnData[addedColumns];
+
+            int nextComplexWithDeletion = findNextComplexWithDeletion(0);
+            ColumnDefinition previousColumn = null;
+
+            int minDeletionTime = Math.min(minDeletionTime(primaryKeyLivenessInfo), minDeletionTime(deletion));
+
+            int i = 0;
+            int size = 0;
+            while (i < cells.size())
+            {
+                Cell cell = cells.get(i++);
+                ColumnDefinition column = cell.column();
+                if (column.isSimple())
+                {
+                    // Either it's a cell for the same column than our previous cell and we merge them together, or it's a new column
+                    if (previousColumn != null && previousColumn.compareTo(column) == 0)
+                        data[size - 1] = Cells.reconcile((Cell)data[size - 1], cell, nowInSec);
+                    else
+                        data[size++] = cell;
+                }
+                else
+                {
+                    // First, collect the complex deletion time for the column we got the first complex column of. We'll
+                    // also find if there is columns that sorts before but had only a complex deletion and add them.
+                    DeletionTime complexDeletion = DeletionTime.LIVE;
+                    while (nextComplexWithDeletion >= 0)
+                    {
+                        int cmp = column.compareTo(columns.getComplex(nextComplexWithDeletion));
+                        if (cmp < 0)
+                        {
+                            // This is after the column we're gonna add cell for. We'll deal with it later
+                            break;
+                        }
+                        else if (cmp > 0)
+                        {
+                            // We have a column that only has a complex deletion and no column. Add its data first
+                            data[size++] = new ComplexColumnData(columns.getComplex(nextComplexWithDeletion), ComplexColumnData.NO_CELLS, complexDeletions[nextComplexWithDeletion]);
+                            minDeletionTime = Integer.MIN_VALUE;
+                            nextComplexWithDeletion = findNextComplexWithDeletion(nextComplexWithDeletion + 1);
+                        }
+                        else // cmp == 0
+                        {
+                            // This is the column we'll about to add cell for. Record the deletion time and break to the cell addition
+                            complexDeletion = complexDeletions[nextComplexWithDeletion];
+                            minDeletionTime = Integer.MIN_VALUE;
+                            nextComplexWithDeletion = findNextComplexWithDeletion(nextComplexWithDeletion + 1);
+                            break;
+                        }
+                    }
+
+                    // Find how many cells the complex column has (cellCount) and the index of the next cell that doesn't belong to it (nextColumnIdx).
+                    int nextColumnIdx = i; // i is on cell following the current one
+                    int cellCount = 1; // We have at least the current cell
+                    Cell previousCell = cell;
+                    while (nextColumnIdx < cells.size())
+                    {
+                        Cell newCell = cells.get(nextColumnIdx);
+                        if (column.compareTo(newCell.column()) != 0)
+                            break;
+
+                        ++nextColumnIdx;
+                        if (column.cellPathComparator().compare(previousCell.path(), newCell.path()) != 0)
+                            ++cellCount;
+                        previousCell = newCell;
+                    }
+                    Cell[] columnCells = new Cell[cellCount];
+                    int complexSize = 0;
+                    columnCells[complexSize++] = cell;
+                    previousCell = cell;
+                    for (int j = i; j < nextColumnIdx; j++)
+                    {
+                        Cell newCell = cells.get(j);
+                        // Either it's a cell for the same path than our previous cell and we merge them together, or it's a new path
+                        if (column.cellPathComparator().compare(previousCell.path(), newCell.path()) == 0)
+                            columnCells[complexSize - 1] = Cells.reconcile(previousCell, newCell, nowInSec);
+                        else
+                            columnCells[complexSize++] = newCell;
+                        previousCell = newCell;
+                    }
+                    i = nextColumnIdx;
+
+                    data[size++] = new ComplexColumnData(column, columnCells, complexDeletion);
+                }
+                previousColumn = column;
+            }
+            // We may still have some complex columns with only a complex deletion
+            while (nextComplexWithDeletion >= 0)
+            {
+                data[size++] = new ComplexColumnData(columns.getComplex(nextComplexWithDeletion), ComplexColumnData.NO_CELLS, complexDeletions[nextComplexWithDeletion]);
+                nextComplexWithDeletion = findNextComplexWithDeletion(nextComplexWithDeletion + 1);
+                minDeletionTime = Integer.MIN_VALUE;
+            }
+            assert size == addedColumns;
+
+            // Reconciliation made it harder to compute minDeletionTime for cells in the loop above, so just do it now if we need to.
+            if (minDeletionTime != Integer.MIN_VALUE)
+            {
+                for (ColumnData cd : data)
+                    minDeletionTime = Math.min(minDeletionTime, minDeletionTime(cd));
+            }
+
+            return new ArrayBackedRow(clustering, columns, primaryKeyLivenessInfo, deletion, size, data, minDeletionTime);
+        }
+
+        private int findNextComplexWithDeletion(int from)
+        {
+            for (int i = from; i < complexDeletions.length; i++)
+            {
+                if (complexDeletions[i] != null)
+                    return i;
+            }
+            return -1;
+        }
+
+        // Should only be called once the cells have been sorted
+        private int countAddedColumns()
+        {
+            int columnCount = 0;
+            int nextComplexWithDeletion = findNextComplexWithDeletion(0);
+            ColumnDefinition previousColumn = null;
+            for (Cell cell : cells)
+            {
+                if (previousColumn != null && previousColumn.compareTo(cell.column()) == 0)
+                    continue;
+
+                ++columnCount;
+                previousColumn = cell.column();
+
+                // We know that simple columns sort before the complex ones, so don't bother with the column having complex deletion
+                // until we've reached the cells of complex columns.
+                if (!previousColumn.isComplex())
+                    continue;
+
+                while (nextComplexWithDeletion >= 0)
+                {
+                    // Check how the column we just counted compared to the next with complex deletion
+                    int cmp = previousColumn.compareTo(columns.getComplex(nextComplexWithDeletion));
+                    if (cmp < 0)
+                    {
+                        // it's before, we'll handle nextColumnWithComplexDeletion later
+                        break;
+                    }
+                    else if (cmp > 0)
+                    {
+                        // it's after. nextColumnWithComplexDeletion has no cell but we should count it
+                        ++columnCount;
+                        nextComplexWithDeletion = findNextComplexWithDeletion(nextComplexWithDeletion + 1);
+                    }
+                    else // cmp == 0
+                    {
+                        // it's the column we just counted. Ignore it and we know we're good with nextComplexWithDeletion for this loop
+                        nextComplexWithDeletion = findNextComplexWithDeletion(nextComplexWithDeletion + 1);
+                        break;
+                    }
+                }
+            }
+            // Anything remaining in complexDeletionColumns are complex columns with no cells but some complex deletion
+            while (nextComplexWithDeletion >= 0)
+            {
+                ++columnCount;
+                nextComplexWithDeletion = findNextComplexWithDeletion(nextComplexWithDeletion + 1);
+            }
+            return columnCount;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/BufferCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/BufferCell.java b/src/java/org/apache/cassandra/db/rows/BufferCell.java
new file mode 100644
index 0000000..c339092
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/rows/BufferCell.java
@@ -0,0 +1,387 @@
+/*
+ * 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.rows;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.marshal.ByteType;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+
+public class BufferCell extends AbstractCell
+{
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new BufferCell(ColumnDefinition.regularDef("", "", "", ByteType.instance), 0L, 0, 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, null));
+
+    private final ColumnDefinition column;
+
+    private final long timestamp;
+    private final int ttl;
+    private final int localDeletionTime;
+
+    private final ByteBuffer value;
+    private final CellPath path;
+
+    public BufferCell(ColumnDefinition column, long timestamp, int ttl, int localDeletionTime, ByteBuffer value, CellPath path)
+    {
+        assert column.isComplex() == (path != null);
+        this.column = column;
+        this.timestamp = timestamp;
+        this.ttl = ttl;
+        this.localDeletionTime = localDeletionTime;
+        this.value = value;
+        this.path = path;
+    }
+
+    public static BufferCell live(CFMetaData metadata, ColumnDefinition column, long timestamp, ByteBuffer value)
+    {
+        return live(metadata, column, timestamp, value, null);
+    }
+
+    public static BufferCell live(CFMetaData metadata, ColumnDefinition column, long timestamp, ByteBuffer value, CellPath path)
+    {
+        if (metadata.getDefaultTimeToLive() != NO_TTL)
+            return expiring(column, timestamp, metadata.getDefaultTimeToLive(), FBUtilities.nowInSeconds(), value, path);
+
+        return new BufferCell(column, timestamp, NO_TTL, NO_DELETION_TIME, value, path);
+    }
+
+    public static BufferCell expiring(ColumnDefinition column, long timestamp, int ttl, int nowInSec, ByteBuffer value)
+    {
+        return expiring(column, timestamp, ttl, nowInSec, value, null);
+    }
+
+    public static BufferCell expiring(ColumnDefinition column, long timestamp, int ttl, int nowInSec, ByteBuffer value, CellPath path)
+    {
+        assert ttl != NO_TTL;
+        return new BufferCell(column, timestamp, ttl, nowInSec + ttl, value, path);
+    }
+
+    public static BufferCell tombstone(ColumnDefinition column, long timestamp, int nowInSec)
+    {
+        return tombstone(column, timestamp, nowInSec, null);
+    }
+
+    public static BufferCell tombstone(ColumnDefinition column, long timestamp, int nowInSec, CellPath path)
+    {
+        return new BufferCell(column, timestamp, NO_TTL, nowInSec, ByteBufferUtil.EMPTY_BYTE_BUFFER, path);
+    }
+
+    public ColumnDefinition column()
+    {
+        return column;
+    }
+
+    public boolean isCounterCell()
+    {
+        return !isTombstone() && column.cellValueType().isCounter();
+    }
+
+    public boolean isLive(int nowInSec)
+    {
+        return localDeletionTime == NO_DELETION_TIME || (ttl != NO_TTL && nowInSec < localDeletionTime);
+    }
+
+    public boolean isTombstone()
+    {
+        return localDeletionTime != NO_DELETION_TIME && ttl == NO_TTL;
+    }
+
+    public boolean isExpiring()
+    {
+        return ttl != NO_TTL;
+    }
+
+    public long timestamp()
+    {
+        return timestamp;
+    }
+
+    public int ttl()
+    {
+        return ttl;
+    }
+
+    public int localDeletionTime()
+    {
+        return localDeletionTime;
+    }
+
+    public ByteBuffer value()
+    {
+        return value;
+    }
+
+    public CellPath path()
+    {
+        return path;
+    }
+
+    public Cell withUpdatedValue(ByteBuffer newValue)
+    {
+        return new BufferCell(column, timestamp, ttl, localDeletionTime, newValue, path);
+    }
+
+    public Cell copy(AbstractAllocator allocator)
+    {
+        if (!value.hasRemaining())
+            return this;
+
+        return new BufferCell(column, timestamp, ttl, localDeletionTime, allocator.clone(value), path == null ? null : path.copy(allocator));
+    }
+
+    public Cell markCounterLocalToBeCleared()
+    {
+        if (!isCounterCell())
+            return this;
+
+        ByteBuffer marked = CounterContext.instance().markLocalToBeCleared(value());
+        return marked == value() ? this : new BufferCell(column, timestamp, ttl, localDeletionTime, marked, path);
+    }
+
+    public Cell purge(DeletionPurger purger, int nowInSec)
+    {
+        if (!isLive(nowInSec))
+        {
+            if (purger.shouldPurge(timestamp, localDeletionTime))
+                return null;
+
+            // We slightly hijack purging to convert expired but not purgeable columns to tombstones. The reason we do that is
+            // that once a column has expired it is equivalent to a tombstone but actually using a tombstone is more compact since
+            // we don't keep the column value. The reason we do it here is that 1) it's somewhat related to dealing with tombstones
+            // so hopefully not too surprising and 2) we want to this and purging at the same places, so it's simpler/more efficient
+            // to do both here.
+            if (isExpiring())
+            {
+                // Note that as long as the expiring column and the tombstone put together live longer than GC grace seconds,
+                // we'll fulfil our responsibility to repair. See discussion at
+                // http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html
+                return BufferCell.tombstone(column, timestamp, localDeletionTime - ttl);
+            }
+        }
+        return this;
+    }
+
+    public Cell updateAllTimestamp(long newTimestamp)
+    {
+        return new BufferCell(column, isTombstone() ? newTimestamp - 1 : newTimestamp, ttl, localDeletionTime, value, path);
+    }
+
+    public int dataSize()
+    {
+        return TypeSizes.sizeof(timestamp)
+             + TypeSizes.sizeof(ttl)
+             + TypeSizes.sizeof(localDeletionTime)
+             + value.remaining()
+             + (path == null ? 0 : path.dataSize());
+    }
+
+    public long unsharedHeapSizeExcludingData()
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(value) + (path == null ? 0 : path.unsharedHeapSizeExcludingData());
+    }
+
+    /**
+     * The serialization format for cell is:
+     *     [ flags ][ timestamp ][ deletion time ][    ttl    ][ path size ][ path ][ value size ][ value ]
+     *     [   1b  ][ 8b (vint) ][   4b (vint)   ][ 4b (vint) ][ 4b (vint) ][  arb ][  4b (vint) ][  arb  ]
+     *
+     * where not all field are always present (in fact, only the [ flags ] are guaranteed to be present). The fields have the following
+     * meaning:
+     *   - [ flags ] is the cell flags. It is a byte for which each bit represents a flag whose meaning is explained below (*_MASK constants)
+     *   - [ timestamp ] is the cell timestamp. Present unless the cell has the USE_TIMESTAMP_MASK.
+     *   - [ deletion time]: the local deletion time for the cell. Present if either the cell is deleted (IS_DELETED_MASK)
+     *       or it is expiring (IS_EXPIRING_MASK) but doesn't have the USE_ROW_TTL_MASK.
+     *   - [ ttl ]: the ttl for the cell. Present if the row is expiring (IS_EXPIRING_MASK) but doesn't have the
+     *       USE_ROW_TTL_MASK.
+     *   - [ value size ] is the size of the [ value ] field. It's present unless either the cell has the HAS_EMPTY_VALUE_MASK, or the value
+     *       for columns of this type have a fixed length.
+     *   - [ path size ] is the size of the [ path ] field. Present iff this is the cell of a complex column.
+     *   - [ value ]: the cell value, unless it has the HAS_EMPTY_VALUE_MASK.
+     *   - [ path ]: the cell path if the column this is a cell of is complex.
+     */
+    static class Serializer implements Cell.Serializer
+    {
+        private final static int PRESENCE_MASK               = 0x01; // Marks the actual presence of a cell. This is used only when serialized on-disk and
+                                                                     // on-wire (i.e. an actual ByteBufferBackedCell instance cannot have this flag set).
+        private final static int IS_DELETED_MASK             = 0x02; // Whether the cell is a tombstone or not.
+        private final static int IS_EXPIRING_MASK            = 0x04; // Whether the cell is expiring.
+        private final static int HAS_EMPTY_VALUE_MASK        = 0x08; // Wether the cell has an empty value. This will be the case for tombstone in particular.
+        private final static int USE_ROW_TIMESTAMP_MASK      = 0x10; // Wether the cell has the same timestamp than the row this is a cell of.
+        private final static int USE_ROW_TTL_MASK            = 0x20; // Wether the cell has the same ttl than the row this is a cell of.
+
+        public void serialize(Cell cell, DataOutputPlus out, LivenessInfo rowLiveness, SerializationHeader header) throws IOException
+        {
+            if (cell == null)
+            {
+                out.writeByte((byte)0);
+                return;
+            }
+
+            boolean hasValue = cell.value().hasRemaining();
+            boolean isDeleted = cell.isTombstone();
+            boolean isExpiring = cell.isExpiring();
+            boolean useRowTimestamp = !rowLiveness.isEmpty() && cell.timestamp() == rowLiveness.timestamp();
+            boolean useRowTTL = isExpiring && rowLiveness.isExpiring() && cell.ttl() == rowLiveness.ttl() && cell.localDeletionTime() == rowLiveness.localExpirationTime();
+            int flags = PRESENCE_MASK;
+            if (!hasValue)
+                flags |= HAS_EMPTY_VALUE_MASK;
+
+            if (isDeleted)
+                flags |= IS_DELETED_MASK;
+            else if (isExpiring)
+                flags |= IS_EXPIRING_MASK;
+
+            if (useRowTimestamp)
+                flags |= USE_ROW_TIMESTAMP_MASK;
+            if (useRowTTL)
+                flags |= USE_ROW_TTL_MASK;
+
+            out.writeByte((byte)flags);
+
+            if (!useRowTimestamp)
+                out.writeVInt(header.encodeTimestamp(cell.timestamp()));
+
+            if ((isDeleted || isExpiring) && !useRowTTL)
+                out.writeVInt(header.encodeDeletionTime(cell.localDeletionTime()));
+            if (isExpiring && !useRowTTL)
+                out.writeVInt(header.encodeTTL(cell.ttl()));
+
+            if (cell.column().isComplex())
+                cell.column().cellPathSerializer().serialize(cell.path(), out);
+
+            if (hasValue)
+                header.getType(cell.column()).writeValue(cell.value(), out);
+        }
+
+        public Cell deserialize(DataInputPlus in, LivenessInfo rowLiveness, ColumnDefinition column, SerializationHeader header, SerializationHelper helper) throws IOException
+        {
+            int flags = in.readUnsignedByte();
+            if ((flags & PRESENCE_MASK) == 0)
+                return null;
+
+            boolean hasValue = (flags & HAS_EMPTY_VALUE_MASK) == 0;
+            boolean isDeleted = (flags & IS_DELETED_MASK) != 0;
+            boolean isExpiring = (flags & IS_EXPIRING_MASK) != 0;
+            boolean useRowTimestamp = (flags & USE_ROW_TIMESTAMP_MASK) != 0;
+            boolean useRowTTL = (flags & USE_ROW_TTL_MASK) != 0;
+
+            long timestamp = useRowTimestamp ? rowLiveness.timestamp() : header.decodeTimestamp(in.readVInt());
+
+            int localDeletionTime = useRowTTL
+                                  ? rowLiveness.localExpirationTime()
+                                  : (isDeleted || isExpiring ? header.decodeDeletionTime((int)in.readVInt()) : NO_DELETION_TIME);
+
+            int ttl = useRowTTL
+                    ? rowLiveness.ttl()
+                    : (isExpiring ? header.decodeTTL((int)in.readVInt()) : NO_TTL);
+
+            CellPath path = column.isComplex()
+                          ? column.cellPathSerializer().deserialize(in)
+                          : null;
+
+            boolean isCounter = localDeletionTime == NO_DELETION_TIME && column.type.isCounter();
+
+            ByteBuffer value = ByteBufferUtil.EMPTY_BYTE_BUFFER;
+            if (hasValue)
+            {
+                if (helper.canSkipValue(column) || (path != null && helper.canSkipValue(path)))
+                {
+                    header.getType(column).skipValue(in);
+                }
+                else
+                {
+                    value = header.getType(column).readValue(in);
+                    if (isCounter)
+                        value = helper.maybeClearCounterValue(value);
+                }
+            }
+
+            return new BufferCell(column, timestamp, ttl, localDeletionTime, value, path);
+        }
+
+        public long serializedSize(Cell cell, LivenessInfo rowLiveness, SerializationHeader header)
+        {
+            long size = 1; // flags
+
+            if (cell == null)
+                return size;
+
+            boolean hasValue = cell.value().hasRemaining();
+            boolean isDeleted = cell.isTombstone();
+            boolean isExpiring = cell.isExpiring();
+            boolean useRowTimestamp = !rowLiveness.isEmpty() && cell.timestamp() == rowLiveness.timestamp();
+            boolean useRowTTL = isExpiring && rowLiveness.isExpiring() && cell.ttl() == rowLiveness.ttl() && cell.localDeletionTime() == rowLiveness.localExpirationTime();
+
+            if (!useRowTimestamp)
+                size += TypeSizes.sizeofVInt(header.encodeTimestamp(cell.timestamp()));
+
+            if ((isDeleted || isExpiring) && !useRowTTL)
+                size += TypeSizes.sizeofVInt(header.encodeDeletionTime(cell.localDeletionTime()));
+            if (isExpiring && !useRowTTL)
+                size += TypeSizes.sizeofVInt(header.encodeTTL(cell.ttl()));
+
+            if (cell.column().isComplex())
+                size += cell.column().cellPathSerializer().serializedSize(cell.path());
+
+            if (hasValue)
+                size += header.getType(cell.column()).writtenLength(cell.value());
+
+            return size;
+        }
+
+        // Returns if the skipped cell was an actual cell (i.e. it had its presence flag).
+        public boolean skip(DataInputPlus in, ColumnDefinition column, SerializationHeader header) throws IOException
+        {
+            int flags = in.readUnsignedByte();
+            if ((flags & PRESENCE_MASK) == 0)
+                return false;
+
+            boolean hasValue = (flags & HAS_EMPTY_VALUE_MASK) == 0;
+            boolean isDeleted = (flags & IS_DELETED_MASK) != 0;
+            boolean isExpiring = (flags & IS_EXPIRING_MASK) != 0;
+            boolean useRowTimestamp = (flags & USE_ROW_TIMESTAMP_MASK) != 0;
+            boolean useRowTTL = (flags & USE_ROW_TTL_MASK) != 0;
+
+            if (!useRowTimestamp)
+                in.readVInt();
+
+            if (!useRowTTL && (isDeleted || isExpiring))
+                in.readVInt();
+
+            if (!useRowTTL && isExpiring)
+                in.readVInt();
+
+            if (column.isComplex())
+                column.cellPathSerializer().skip(in);
+
+            if (hasValue)
+                header.getType(column).skipValue(in);
+
+            return true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/Cell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Cell.java b/src/java/org/apache/cassandra/db/rows/Cell.java
index 80bf901..ccb9708 100644
--- a/src/java/org/apache/cassandra/db/rows/Cell.java
+++ b/src/java/org/apache/cassandra/db/rows/Cell.java
@@ -17,40 +17,41 @@
  */
 package org.apache.cassandra.db.rows;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.security.MessageDigest;
+import java.util.Comparator;
 
 import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.Aliasable;
-import org.apache.cassandra.db.LivenessInfo;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
- * A cell holds a single "simple" value for a given column, as well as "liveness"
- * informations regarding that value.
+ * A cell is our atomic unit for a single value of a single column.
  * <p>
- * The is 2 kind of columns: simple ones and complex ones.
- * Simple columns have only a single associated cell, while complex ones,
- * the one corresponding to non-frozen collections and UDTs, are comprised
- * of multiple cells. For complex columns, the different cells are distinguished
- * by their cell path.
- * <p>
- * We can also distinguish different kind of cells based on the property of their
- * {@link #livenessInfo}:
- *  1) "Normal" cells: their liveness info has no ttl and no deletion time.
- *  2) Expiring cells: their liveness info has both a ttl and a deletion time (the latter
- *    deciding when the cell is actually expired).
- *  3) Tombstones/deleted cells: their liveness info has a deletion time but no ttl. Those
- *     cells don't really have a value but their {@link #value} method return an empty
- *     buffer by convention.
+ * A cell always holds at least a timestamp that gives us how the cell reconcile. We then
+ * have 3 main types of cells:
+ *   1) live regular cells: those will also have a value and, if for a complex column, a path.
+ *   2) expiring cells: on top of regular cells, those have a ttl and a local deletion time (when they are expired).
+ *   3) tombstone cells: those won't have value, but they have a local deletion time (when the tombstone was created).
  */
-public interface Cell extends Aliasable<Cell>
+public interface Cell extends ColumnData
 {
-    /**
-     * The column this cell belongs to.
-     *
-     * @return the column this cell belongs to.
-     */
-    public ColumnDefinition column();
+    public static final int NO_TTL = 0;
+    public static final int NO_DELETION_TIME = Integer.MAX_VALUE;
+
+    public final Comparator<Cell> comparator = (c1, c2) ->
+    {
+        int cmp = c1.column().compareTo(c2.column());
+        if (cmp != 0)
+            return cmp;
+
+        Comparator<CellPath> pathComparator = c1.column().cellPathComparator();
+        return pathComparator == null ? 0 : pathComparator.compare(c1.path(), c2.path());
+    };
+
+    public final Serializer serializer = new BufferCell.Serializer();
 
     /**
      * Whether the cell is a counter cell or not.
@@ -67,12 +68,26 @@ public interface Cell extends Aliasable<Cell>
     public ByteBuffer value();
 
     /**
-     * The liveness info of the cell, that is its timestamp and whether it is
-     * expiring, deleted or none of the above.
+     * The cell timestamp.
+     * <p>
+     * @return the cell timestamp.
+     */
+    public long timestamp();
+
+    /**
+     * The cell ttl.
+     *
+     * @return the cell ttl, or {@code NO_TTL} if the cell isn't an expiring one.
+     */
+    public int ttl();
+
+    /**
+     * The cell local deletion time.
      *
-     * @return the cell {@link LivenessInfo}.
+     * @return the cell local deletion time, or {@code NO_DELETION_TIME} if the cell is neither
+     * a tombstone nor an expiring one.
      */
-    public LivenessInfo livenessInfo();
+    public int localDeletionTime();
 
     /**
      * Whether the cell is a tombstone or not.
@@ -109,34 +124,27 @@ public interface Cell extends Aliasable<Cell>
      */
     public CellPath path();
 
-    /**
-     * Write the cell to the provided writer.
-     *
-     * @param writer the row writer to write the cell to.
-     */
-    public void writeTo(Row.Writer writer);
+    public Cell withUpdatedValue(ByteBuffer newValue);
 
-    /**
-     * Adds the cell to the provided digest.
-     *
-     * @param digest the {@code MessageDigest} to add the cell to.
-     */
-    public void digest(MessageDigest digest);
+    public Cell copy(AbstractAllocator allocator);
 
-    /**
-     * Validate the cell value.
-     *
-     * @throws MarshalException if the cell value is not a valid value for
-     * the column type this is a cell of.
-     */
-    public void validate();
+    @Override
+    // Overrides super type to provide a more precise return type.
+    public Cell markCounterLocalToBeCleared();
 
-    /**
-     * The size of the data hold by this cell.
-     *
-     * This is mainly used to verify if batches goes over a given size.
-     *
-     * @return the size used by the data of this cell.
-     */
-    public int dataSize();
+    @Override
+    // Overrides super type to provide a more precise return type.
+    public Cell purge(DeletionPurger purger, int nowInSec);
+
+    public interface Serializer
+    {
+        public void serialize(Cell cell, DataOutputPlus out, LivenessInfo rowLiveness, SerializationHeader header) throws IOException;
+
+        public Cell deserialize(DataInputPlus in, LivenessInfo rowLiveness, ColumnDefinition column, SerializationHeader header, SerializationHelper helper) throws IOException;
+
+        public long serializedSize(Cell cell, LivenessInfo rowLiveness, SerializationHeader header);
+
+        // Returns if the skipped cell was an actual cell (i.e. it had its presence flag).
+        public boolean skip(DataInputPlus in, ColumnDefinition column, SerializationHeader header) throws IOException;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/CellData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/CellData.java b/src/java/org/apache/cassandra/db/rows/CellData.java
deleted file mode 100644
index 29eac01..0000000
--- a/src/java/org/apache/cassandra/db/rows/CellData.java
+++ /dev/null
@@ -1,275 +0,0 @@
-/*
- * 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.rows;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.utils.ObjectSizes;
-
-/**
- * Contains (non-counter) cell data for one or more rows.
- */
-class CellData
-{
-    private boolean isCounter;
-
-    private ByteBuffer[] values;
-    private final LivenessInfoArray livenessInfos;
-
-    CellData(int initialCellCapacity, boolean isCounter)
-    {
-        this.isCounter = isCounter;
-        this.values = new ByteBuffer[initialCellCapacity];
-        this.livenessInfos = new LivenessInfoArray(initialCellCapacity);
-    }
-
-    public void setCell(int idx, ByteBuffer value, LivenessInfo info)
-    {
-        ensureCapacity(idx);
-        values[idx] = value;
-        livenessInfos.set(idx, info);
-    }
-
-    public boolean hasCell(int idx)
-    {
-        return idx < values.length && values[idx] != null;
-    }
-
-    public ByteBuffer value(int idx)
-    {
-        return values[idx];
-    }
-
-    public void setValue(int idx, ByteBuffer value)
-    {
-        values[idx] = value;
-    }
-
-    private void ensureCapacity(int idxToSet)
-    {
-        int originalCapacity = values.length;
-        if (idxToSet < originalCapacity)
-            return;
-
-        int newCapacity = RowDataBlock.computeNewCapacity(originalCapacity, idxToSet);
-
-        values = Arrays.copyOf(values, newCapacity);
-        livenessInfos.resize(newCapacity);
-    }
-
-    // Swap cell i and j
-    public void swapCell(int i, int j)
-    {
-        ensureCapacity(Math.max(i, j));
-
-        ByteBuffer value = values[j];
-        values[j] = values[i];
-        values[i] = value;
-
-        livenessInfos.swap(i, j);
-    }
-
-    // Merge cell i into j
-    public void mergeCell(int i, int j, int nowInSec)
-    {
-        if (isCounter)
-            mergeCounterCell(this, i, this, j, this, j, nowInSec);
-        else
-            mergeRegularCell(this, i, this, j, this, j, nowInSec);
-    }
-
-    private static boolean handleNoCellCase(CellData d1, int i1, CellData d2, int i2, CellData merged, int iMerged)
-    {
-        if (!d1.hasCell(i1))
-        {
-            if (d2.hasCell(i2))
-                d2.moveCell(i2, merged, iMerged);
-            return true;
-        }
-        if (!d2.hasCell(i2))
-        {
-            d1.moveCell(i1, merged, iMerged);
-            return true;
-        }
-        return false;
-    }
-
-    public static void mergeRegularCell(CellData d1, int i1, CellData d2, int i2, CellData merged, int iMerged, int nowInSec)
-    {
-        if (handleNoCellCase(d1, i1, d2, i2, merged, iMerged))
-            return;
-
-        Conflicts.Resolution res = Conflicts.resolveRegular(d1.livenessInfos.timestamp(i1),
-                                                            d1.livenessInfos.isLive(i1, nowInSec),
-                                                            d1.livenessInfos.localDeletionTime(i1),
-                                                            d1.values[i1],
-                                                            d2.livenessInfos.timestamp(i2),
-                                                            d2.livenessInfos.isLive(i2, nowInSec),
-                                                            d2.livenessInfos.localDeletionTime(i2),
-                                                            d2.values[i2]);
-
-        assert res != Conflicts.Resolution.MERGE;
-        if (res == Conflicts.Resolution.LEFT_WINS)
-            d1.moveCell(i1, merged, iMerged);
-        else
-            d2.moveCell(i2, merged, iMerged);
-    }
-
-    public static void mergeCounterCell(CellData d1, int i1, CellData d2, int i2, CellData merged, int iMerged, int nowInSec)
-    {
-        if (handleNoCellCase(d1, i1, d2, i2, merged, iMerged))
-            return;
-
-        Conflicts.Resolution res = Conflicts.resolveCounter(d1.livenessInfos.timestamp(i1),
-                                                            d1.livenessInfos.isLive(i1, nowInSec),
-                                                            d1.values[i1],
-                                                            d2.livenessInfos.timestamp(i2),
-                                                            d2.livenessInfos.isLive(i2, nowInSec),
-                                                            d2.values[i2]);
-
-        switch (res)
-        {
-            case LEFT_WINS:
-                d1.moveCell(i1, merged, iMerged);
-                break;
-            case RIGHT_WINS:
-                d2.moveCell(i2, merged, iMerged);
-                break;
-            default:
-                merged.values[iMerged] = Conflicts.mergeCounterValues(d1.values[i1], d2.values[i2]);
-                if (d1.livenessInfos.timestamp(i1) > d2.livenessInfos.timestamp(i2))
-                    merged.livenessInfos.set(iMerged, d1.livenessInfos.timestamp(i1), d1.livenessInfos.ttl(i1), d1.livenessInfos.localDeletionTime(i1));
-                else
-                    merged.livenessInfos.set(iMerged, d2.livenessInfos.timestamp(i2), d2.livenessInfos.ttl(i2), d2.livenessInfos.localDeletionTime(i2));
-                break;
-        }
-    }
-
-    // Move cell i into j
-    public void moveCell(int i, int j)
-    {
-        moveCell(i, this, j);
-    }
-
-    public void moveCell(int i, CellData target, int j)
-    {
-        if (!hasCell(i) || (target == this && i == j))
-            return;
-
-        target.ensureCapacity(j);
-
-        target.values[j] = values[i];
-        target.livenessInfos.set(j, livenessInfos.timestamp(i),
-                                    livenessInfos.ttl(i),
-                                    livenessInfos.localDeletionTime(i));
-    }
-
-    public int dataSize()
-    {
-        int size = livenessInfos.dataSize();
-        for (int i = 0; i < values.length; i++)
-            if (values[i] != null)
-                size += values[i].remaining();
-        return size;
-    }
-
-    public void clear()
-    {
-        Arrays.fill(values, null);
-        livenessInfos.clear();
-    }
-
-    public long unsharedHeapSizeExcludingData()
-    {
-        return ObjectSizes.sizeOnHeapExcludingData(values)
-             + livenessInfos.unsharedHeapSize();
-    }
-
-    @Override
-    public String toString()
-    {
-        StringBuilder sb = new StringBuilder();
-        sb.append("CellData(size=").append(values.length);
-        if (isCounter)
-            sb.append(", counter");
-        sb.append("){");
-        LivenessInfoArray.Cursor cursor = LivenessInfoArray.newCursor();
-        for (int i = 0; i < values.length; i++)
-        {
-            if (values[i] == null)
-            {
-                sb.append("[null]");
-                continue;
-            }
-            sb.append("[len(v)=").append(values[i].remaining());
-            sb.append(", info=").append(cursor.setTo(livenessInfos, i));
-            sb.append("]");
-        }
-        return sb.append("}").toString();
-    }
-
-    static class ReusableCell extends AbstractCell
-    {
-        private final LivenessInfoArray.Cursor cursor = LivenessInfoArray.newCursor();
-
-        private CellData data;
-        private ColumnDefinition column;
-        protected int idx;
-
-        ReusableCell setTo(CellData data, ColumnDefinition column, int idx)
-        {
-            if (!data.hasCell(idx))
-                return null;
-
-            this.data = data;
-            this.column = column;
-            this.idx = idx;
-
-            cursor.setTo(data.livenessInfos, idx);
-            return this;
-        }
-
-        public ColumnDefinition column()
-        {
-            return column;
-        }
-
-        public boolean isCounterCell()
-        {
-            return data.isCounter && !cursor.hasLocalDeletionTime();
-        }
-
-        public ByteBuffer value()
-        {
-            return data.value(idx);
-        }
-
-        public LivenessInfo livenessInfo()
-        {
-            return cursor;
-        }
-
-        public CellPath path()
-        {
-            return null;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/CellPath.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/CellPath.java b/src/java/org/apache/cassandra/db/rows/CellPath.java
index 40d525c..68e3c2b 100644
--- a/src/java/org/apache/cassandra/db/rows/CellPath.java
+++ b/src/java/org/apache/cassandra/db/rows/CellPath.java
@@ -17,13 +17,16 @@
  */
 package org.apache.cassandra.db.rows;
 
-import java.io.DataInput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.security.MessageDigest;
 import java.util.Objects;
 
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
  * A path for a cell belonging to a complex column type (non-frozen collection or UDT).
@@ -40,7 +43,7 @@ public abstract class CellPath
     public static CellPath create(ByteBuffer value)
     {
         assert value != null;
-        return new SimpleCellPath(new ByteBuffer[]{ value });
+        return new CollectionCellPath(value);
     }
 
     public int dataSize()
@@ -57,6 +60,10 @@ public abstract class CellPath
             digest.update(get(i).duplicate());
     }
 
+    public abstract CellPath copy(AbstractAllocator allocator);
+
+    public abstract long unsharedHeapSizeExcludingData();
+
     @Override
     public final int hashCode()
     {
@@ -86,28 +93,41 @@ public abstract class CellPath
     public interface Serializer
     {
         public void serialize(CellPath path, DataOutputPlus out) throws IOException;
-        public CellPath deserialize(DataInput in) throws IOException;
+        public CellPath deserialize(DataInputPlus in) throws IOException;
         public long serializedSize(CellPath path);
-        public void skip(DataInput in) throws IOException;
+        public void skip(DataInputPlus in) throws IOException;
     }
 
-    static class SimpleCellPath extends CellPath
+    private static class CollectionCellPath extends CellPath
     {
-        protected final ByteBuffer[] values;
+        private static final long EMPTY_SIZE = ObjectSizes.measure(new CollectionCellPath(ByteBufferUtil.EMPTY_BYTE_BUFFER));
 
-        public SimpleCellPath(ByteBuffer[] values)
+        protected final ByteBuffer value;
+
+        private CollectionCellPath(ByteBuffer value)
         {
-            this.values = values;
+            this.value = value;
         }
 
         public int size()
         {
-            return values.length;
+            return 1;
         }
 
         public ByteBuffer get(int i)
         {
-            return values[i];
+            assert i == 0;
+            return value;
+        }
+
+        public CellPath copy(AbstractAllocator allocator)
+        {
+            return new CollectionCellPath(allocator.clone(value));
+        }
+
+        public long unsharedHeapSizeExcludingData()
+        {
+            return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(value);
         }
     }
 
@@ -122,5 +142,15 @@ public abstract class CellPath
         {
             throw new UnsupportedOperationException();
         }
+
+        public CellPath copy(AbstractAllocator allocator)
+        {
+            return this;
+        }
+
+        public long unsharedHeapSizeExcludingData()
+        {
+            return 0;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/Cells.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Cells.java b/src/java/org/apache/cassandra/db/rows/Cells.java
index 1e329e5..080d640 100644
--- a/src/java/org/apache/cassandra/db/rows/Cells.java
+++ b/src/java/org/apache/cassandra/db/rows/Cells.java
@@ -23,9 +23,8 @@ import java.util.Iterator;
 
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.partitions.PartitionStatisticsCollector;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * Static methods to work on cells.
@@ -35,54 +34,17 @@ public abstract class Cells
     private Cells() {}
 
     /**
-     * Writes a tombstone cell to the provided writer.
+     * Collect statistics ont a given cell.
      *
-     * @param writer the {@code Row.Writer} to write the tombstone to.
-     * @param column the column for the tombstone.
-     * @param timestamp the timestamp for the tombstone.
-     * @param localDeletionTime the local deletion time (in seconds) for the tombstone.
+     * @param cell the cell for which to collect stats.
+     * @param collector the stats collector.
      */
-    public static void writeTombstone(Row.Writer writer, ColumnDefinition column, long timestamp, int localDeletionTime)
+    public static void collectStats(Cell cell, PartitionStatisticsCollector collector)
     {
-        writer.writeCell(column, false, ByteBufferUtil.EMPTY_BYTE_BUFFER, SimpleLivenessInfo.forDeletion(timestamp, localDeletionTime), null);
-    }
-
-    /**
-     * Computes the difference between a cell and the result of merging this
-     * cell to other cells.
-     * <p>
-     * This method is used when cells from multiple sources are merged and we want to
-     * find for a given source if it was up to date for that cell, and if not, what
-     * should be sent to the source to repair it.
-     *
-     * @param merged the cell that is the result of merging multiple source.
-     * @param cell the cell from one of the source that has been merged to yied
-     * {@code merged}.
-     * @return {@code null} if the source having {@code cell} is up-to-date for that
-     * cell, or a cell that applied to the source will "repair" said source otherwise.
-     */
-    public static Cell diff(Cell merged, Cell cell)
-    {
-        // Note that it's enough to check if merged is a counterCell. If it isn't and
-        // cell is one, it means that merged is a tombstone with a greater timestamp
-        // than cell, because that's the only case where reconciling a counter with
-        // a tombstone don't yield a counter. If that's the case, the normal path will
-        // return what it should.
-        if (merged.isCounterCell())
-        {
-            if (merged.livenessInfo().supersedes(cell.livenessInfo()))
-                return merged;
+        collector.update(cell);
 
-            // Reconciliation never returns something with a timestamp strictly lower than its operand. This
-            // means we're in the case where merged.timestamp() == cell.timestamp(). As 1) tombstones
-            // always win over counters (CASSANDRA-7346) and 2) merged is a counter, it follows that cell
-            // can't be a tombstone or merged would be one too.
-            assert !cell.isTombstone();
-
-            CounterContext.Relationship rel = CounterContext.instance().diff(merged.value(), cell.value());
-            return (rel == CounterContext.Relationship.GREATER_THAN || rel == CounterContext.Relationship.DISJOINT) ? merged : null;
-        }
-        return merged.livenessInfo().supersedes(cell.livenessInfo()) ? merged : null;
+        if (cell.isCounterCell())
+            collector.updateHasLegacyCounterShards(CounterCells.hasLegacyShards(cell));
     }
 
     /**
@@ -106,7 +68,7 @@ public abstract class Cells
      * {@code writer}.
      * @param deletion the deletion time that applies to the cells being considered.
      * This deletion time may delete both {@code existing} or {@code update}.
-     * @param writer the row writer to which the result of the reconciliation is written.
+     * @param builder the row builder to which the result of the reconciliation is written.
      * @param nowInSec the current time in seconds (which plays a role during reconciliation
      * because deleted cells always have precedence on timestamp equality and deciding if a
      * cell is a live or not depends on the current time due to expiring cells).
@@ -121,12 +83,12 @@ public abstract class Cells
                                  Cell existing,
                                  Cell update,
                                  DeletionTime deletion,
-                                 Row.Writer writer,
+                                 Row.Builder builder,
                                  int nowInSec,
                                  SecondaryIndexManager.Updater indexUpdater)
     {
-        existing = existing == null || deletion.deletes(existing.livenessInfo()) ? null : existing;
-        update = update == null || deletion.deletes(update.livenessInfo()) ? null : update;
+        existing = existing == null || deletion.deletes(existing) ? null : existing;
+        update = update == null || deletion.deletes(update) ? null : update;
         if (existing == null || update == null)
         {
             if (update != null)
@@ -135,17 +97,17 @@ public abstract class Cells
                 // we'll need to fix that damn 2ndary index API to avoid that.
                 updatePKIndexes(clustering, update, nowInSec, indexUpdater);
                 indexUpdater.insert(clustering, update);
-                update.writeTo(writer);
+                builder.addCell(update);
             }
             else if (existing != null)
             {
-                existing.writeTo(writer);
+                builder.addCell(existing);
             }
             return Long.MAX_VALUE;
         }
 
         Cell reconciled = reconcile(existing, update, nowInSec);
-        reconciled.writeTo(writer);
+        builder.addCell(reconciled);
 
         // Note that this test rely on reconcile returning either 'existing' or 'update'. That's not true for counters but we don't index them
         if (reconciled == update)
@@ -153,13 +115,13 @@ public abstract class Cells
             updatePKIndexes(clustering, update, nowInSec, indexUpdater);
             indexUpdater.update(clustering, existing, reconciled);
         }
-        return Math.abs(existing.livenessInfo().timestamp() - update.livenessInfo().timestamp());
+        return Math.abs(existing.timestamp() - update.timestamp());
     }
 
     private static void updatePKIndexes(Clustering clustering, Cell cell, int nowInSec, SecondaryIndexManager.Updater indexUpdater)
     {
         if (indexUpdater != SecondaryIndexManager.nullUpdater && cell.isLive(nowInSec))
-            indexUpdater.maybeIndex(clustering, cell.livenessInfo().timestamp(), cell.livenessInfo().ttl(), DeletionTime.LIVE);
+            indexUpdater.maybeIndex(clustering, cell.timestamp(), cell.ttl(), DeletionTime.LIVE);
     }
 
     /**
@@ -190,10 +152,10 @@ public abstract class Cells
 
         if (c1.isCounterCell() || c2.isCounterCell())
         {
-            Conflicts.Resolution res = Conflicts.resolveCounter(c1.livenessInfo().timestamp(),
+            Conflicts.Resolution res = Conflicts.resolveCounter(c1.timestamp(),
                                                                 c1.isLive(nowInSec),
                                                                 c1.value(),
-                                                                c2.livenessInfo().timestamp(),
+                                                                c2.timestamp(),
                                                                 c2.isLive(nowInSec),
                                                                 c2.value());
 
@@ -203,26 +165,26 @@ public abstract class Cells
                 case RIGHT_WINS: return c2;
                 default:
                     ByteBuffer merged = Conflicts.mergeCounterValues(c1.value(), c2.value());
-                    LivenessInfo mergedInfo = c1.livenessInfo().mergeWith(c2.livenessInfo());
+                    long timestamp = Math.max(c1.timestamp(), c2.timestamp());
 
                     // We save allocating a new cell object if it turns out that one cell was
                     // a complete superset of the other
-                    if (merged == c1.value() && mergedInfo == c1.livenessInfo())
+                    if (merged == c1.value() && timestamp == c1.timestamp())
                         return c1;
-                    else if (merged == c2.value() && mergedInfo == c2.livenessInfo())
+                    else if (merged == c2.value() && timestamp == c2.timestamp())
                         return c2;
                     else // merge clocks and timestamps.
-                        return create(c1.column(), true, merged, mergedInfo, null);
+                        return new BufferCell(c1.column(), timestamp, Cell.NO_TTL, Cell.NO_DELETION_TIME, merged, c1.path());
             }
         }
 
-        Conflicts.Resolution res = Conflicts.resolveRegular(c1.livenessInfo().timestamp(),
+        Conflicts.Resolution res = Conflicts.resolveRegular(c1.timestamp(),
                                                             c1.isLive(nowInSec),
-                                                            c1.livenessInfo().localDeletionTime(),
+                                                            c1.localDeletionTime(),
                                                             c1.value(),
-                                                            c2.livenessInfo().timestamp(),
+                                                            c2.timestamp(),
                                                             c2.isLive(nowInSec),
-                                                            c2.livenessInfo().localDeletionTime(),
+                                                            c2.localDeletionTime(),
                                                             c2.value());
         assert res != Conflicts.Resolution.MERGE;
         return res == Conflicts.Resolution.LEFT_WINS ? c1 : c2;
@@ -251,7 +213,7 @@ public abstract class Cells
      * {@code existing} to {@code writer}.
      * @param deletion the deletion time that applies to the cells being considered.
      * This deletion time may delete cells in both {@code existing} and {@code update}.
-     * @param writer the row writer to which the result of the reconciliation is written.
+     * @param builder the row build to which the result of the reconciliation is written.
      * @param nowInSec the current time in seconds (which plays a role during reconciliation
      * because deleted cells always have precedence on timestamp equality and deciding if a
      * cell is a live or not depends on the current time due to expiring cells).
@@ -270,7 +232,7 @@ public abstract class Cells
                                         Iterator<Cell> existing,
                                         Iterator<Cell> update,
                                         DeletionTime deletion,
-                                        Row.Writer writer,
+                                        Row.Builder builder,
                                         int nowInSec,
                                         SecondaryIndexManager.Updater indexUpdater)
     {
@@ -285,17 +247,17 @@ public abstract class Cells
                      : comparator.compare(nextExisting.path(), nextUpdate.path()));
             if (cmp < 0)
             {
-                reconcile(clustering, nextExisting, null, deletion, writer, nowInSec, indexUpdater);
+                reconcile(clustering, nextExisting, null, deletion, builder, nowInSec, indexUpdater);
                 nextExisting = getNext(existing);
             }
             else if (cmp > 0)
             {
-                reconcile(clustering, null, nextUpdate, deletion, writer, nowInSec, indexUpdater);
+                reconcile(clustering, null, nextUpdate, deletion, builder, nowInSec, indexUpdater);
                 nextUpdate = getNext(update);
             }
             else
             {
-                timeDelta = Math.min(timeDelta, reconcile(clustering, nextExisting, nextUpdate, deletion, writer, nowInSec, indexUpdater));
+                timeDelta = Math.min(timeDelta, reconcile(clustering, nextExisting, nextUpdate, deletion, builder, nowInSec, indexUpdater));
                 nextExisting = getNext(existing);
                 nextUpdate = getNext(update);
             }
@@ -307,65 +269,4 @@ public abstract class Cells
     {
         return iterator == null || !iterator.hasNext() ? null : iterator.next();
     }
-
-    /**
-     * Creates a simple cell.
-     * <p>
-     * Note that in general cell objects are created by the container they are in and so this method should
-     * only be used in a handful of cases when we know it's the right thing to do.
-     *
-     * @param column the column for the cell to create.
-     * @param isCounter whether the create cell should be a counter one.
-     * @param value the value for the cell.
-     * @param info the liveness info for the cell.
-     * @param path the cell path for the cell.
-     * @return the newly allocated cell object.
-     */
-    public static Cell create(ColumnDefinition column, boolean isCounter, ByteBuffer value, LivenessInfo info, CellPath path)
-    {
-        return new SimpleCell(column, isCounter, value, info, path);
-    }
-
-    private static class SimpleCell extends AbstractCell
-    {
-        private final ColumnDefinition column;
-        private final boolean isCounter;
-        private final ByteBuffer value;
-        private final LivenessInfo info;
-        private final CellPath path;
-
-        private SimpleCell(ColumnDefinition column, boolean isCounter, ByteBuffer value, LivenessInfo info, CellPath path)
-        {
-            this.column = column;
-            this.isCounter = isCounter;
-            this.value = value;
-            this.info = info.takeAlias();
-            this.path = path;
-        }
-
-        public ColumnDefinition column()
-        {
-            return column;
-        }
-
-        public boolean isCounterCell()
-        {
-            return isCounter;
-        }
-
-        public ByteBuffer value()
-        {
-            return value;
-        }
-
-        public LivenessInfo livenessInfo()
-        {
-            return info;
-        }
-
-        public CellPath path()
-        {
-            return path;
-        }
-    }
 }


[09/15] cassandra git commit: Simplify some 8099's implementations

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
index 670b1ae..e805fd2 100644
--- a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
+++ b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
@@ -17,12 +17,12 @@
  */
 package org.apache.cassandra.db.partitions;
 
-import java.io.DataInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
 
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -31,99 +31,185 @@ import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.rows.*;
-import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.util.NIODataInputStream;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Sorting;
+import org.apache.cassandra.utils.MergeIterator;
 
 /**
  * Stores updates made on a partition.
  * <p>
- * A PartitionUpdate object requires that all writes are performed before we
- * try to read the updates (attempts to write to the PartitionUpdate after a
- * read method has been called will result in an exception being thrown).
- * In other words, a Partition is mutable while we do a write and become
- * immutable as soon as it is read.
+ * A PartitionUpdate object requires that all writes/additions are performed before we
+ * try to read the updates (attempts to write to the PartitionUpdate after a read method
+ * has been called will result in an exception being thrown). In other words, a Partition
+ * is mutable while it's written but becomes immutable as soon as it is read.
  * <p>
- * Row updates are added to this update through the {@link #writer} method which
- * returns a {@link Row.Writer}. Multiple rows can be added to this writer as required and
- * those row do not have to added in (clustering) order, and the same row can be added
- * multiple times. Further, for a given row, the writer actually supports intermingling
- * the writing of cells for different complex cells (note that this is usually not supported
- * by {@code Row.Writer} implementations, but is supported here because
- * {@code ModificationStatement} requires that (because we could have multiple {@link Operation}
- * on the same column in a given statement)).
+ * A typical usage is to create a new update ({@code new PartitionUpdate(metadata, key, columns, capacity)})
+ * and then add rows and range tombstones through the {@code add()} methods (the partition
+ * level deletion time can also be set with {@code addPartitionDeletion()}). However, there
+ * is also a few static helper constructor methods for special cases ({@code emptyUpdate()},
+ * {@code fullPartitionDelete} and {@code singleRowUpdate}).
  */
-public class PartitionUpdate extends AbstractPartitionData implements Sorting.Sortable
+public class PartitionUpdate extends AbstractThreadUnsafePartition
 {
     protected static final Logger logger = LoggerFactory.getLogger(PartitionUpdate.class);
 
-    // Records whether the partition update has been sorted (it is the rows contained in the partition
-    // that are sorted since we don't require rows to be added in order). Sorting happens when the
-    // update is read, and writting is rejected as soon as the update is sorted (it's actually possible
-    // to manually allow new update by using allowNewUpdates(), and we could make that more implicit, but
-    // as only triggers really requires it, we keep it simple for now).
-    private boolean isSorted;
-
     public static final PartitionUpdateSerializer serializer = new PartitionUpdateSerializer();
 
-    private final Writer writer;
+    private final int createdAtInSec = FBUtilities.nowInSeconds();
+
+    // Records whether this update is "built", i.e. if the build() method has been called, which
+    // happens when the update is read. Further writing is then rejected though a manual call
+    // to allowNewUpdates() allow new writes. We could make that more implicit but only triggers
+    // really requires that so we keep it simple for now).
+    private boolean isBuilt;
+    private boolean canReOpen = true;
+
+    private final MutableDeletionInfo deletionInfo;
+    private RowStats stats; // will be null if isn't built
+
+    private Row staticRow = Rows.EMPTY_STATIC_ROW;
 
-    // Used by compare for the sake of implementing the Sorting.Sortable interface (which is in turn used
-    // to sort the rows of this update).
-    private final InternalReusableClustering p1 = new InternalReusableClustering();
-    private final InternalReusableClustering p2 = new InternalReusableClustering();
+    private final boolean canHaveShadowedData;
 
     private PartitionUpdate(CFMetaData metadata,
                             DecoratedKey key,
-                            DeletionInfo delInfo,
-                            RowDataBlock data,
                             PartitionColumns columns,
-                            int initialRowCapacity)
+                            Row staticRow,
+                            List<Row> rows,
+                            MutableDeletionInfo deletionInfo,
+                            RowStats stats,
+                            boolean isBuilt,
+                            boolean canHaveShadowedData)
     {
-        super(metadata, key, delInfo, columns, data, initialRowCapacity);
-        this.writer = createWriter();
+        super(metadata, key, columns, rows);
+        this.staticRow = staticRow;
+        this.deletionInfo = deletionInfo;
+        this.stats = stats;
+        this.isBuilt = isBuilt;
+        this.canHaveShadowedData = canHaveShadowedData;
     }
 
     public PartitionUpdate(CFMetaData metadata,
                            DecoratedKey key,
-                           DeletionInfo delInfo,
                            PartitionColumns columns,
                            int initialRowCapacity)
     {
-        this(metadata,
-             key,
-             delInfo,
-             new RowDataBlock(columns.regulars, initialRowCapacity, true, metadata.isCounter()),
-             columns,
-             initialRowCapacity);
+        this(metadata, key, columns, Rows.EMPTY_STATIC_ROW, new ArrayList<>(initialRowCapacity), MutableDeletionInfo.live(), null, false, true);
     }
 
-    public PartitionUpdate(CFMetaData metadata,
-                           DecoratedKey key,
-                           PartitionColumns columns,
-                           int initialRowCapacity)
+    /**
+     * Creates a empty immutable partition update.
+     *
+     * @param metadata the metadata for the created update.
+     * @param key the partition key for the created update.
+     *
+     * @return the newly created empty (and immutable) update.
+     */
+    public static PartitionUpdate emptyUpdate(CFMetaData metadata, DecoratedKey key)
+    {
+        return new PartitionUpdate(metadata, key, PartitionColumns.NONE, Rows.EMPTY_STATIC_ROW, Collections.<Row>emptyList(), MutableDeletionInfo.live(), RowStats.NO_STATS, true, false);
+    }
+
+    /**
+     * Creates an immutable partition update that entirely deletes a given partition.
+     *
+     * @param metadata the metadata for the created update.
+     * @param key the partition key for the partition that the created update should delete.
+     * @param timestamp the timestamp for the deletion.
+     * @param nowInSec the current time in seconds to use as local deletion time for the partition deletion.
+     *
+     * @return the newly created partition deletion update.
+     */
+    public static PartitionUpdate fullPartitionDelete(CFMetaData metadata, DecoratedKey key, long timestamp, int nowInSec)
+    {
+        return new PartitionUpdate(metadata, key, PartitionColumns.NONE, Rows.EMPTY_STATIC_ROW, Collections.<Row>emptyList(), new MutableDeletionInfo(timestamp, nowInSec), RowStats.NO_STATS, true, false);
+    }
+
+    /**
+     * Creates an immutable partition update that contains a single row update.
+     *
+     * @param metadata the metadata for the created update.
+     * @param key the partition key for the partition that the created update should delete.
+     * @param row the row for the update.
+     *
+     * @return the newly created partition update containing only {@code row}.
+     */
+    public static PartitionUpdate singleRowUpdate(CFMetaData metadata, DecoratedKey key, Row row)
+    {
+        return row.isStatic()
+             ? new PartitionUpdate(metadata, key, new PartitionColumns(row.columns(), Columns.NONE), row, Collections.<Row>emptyList(), MutableDeletionInfo.live(), RowStats.NO_STATS, true, false)
+             : new PartitionUpdate(metadata, key, new PartitionColumns(Columns.NONE, row.columns()), Rows.EMPTY_STATIC_ROW, Collections.singletonList(row), MutableDeletionInfo.live(), RowStats.NO_STATS, true, false);
+    }
+
+    /**
+     * Turns the given iterator into an update.
+     *
+     * Warning: this method does not close the provided iterator, it is up to
+     * the caller to close it.
+     */
+    public static PartitionUpdate fromIterator(UnfilteredRowIterator iterator)
+    {
+        CFMetaData metadata = iterator.metadata();
+        boolean reversed = iterator.isReverseOrder();
+
+        List<Row> rows = new ArrayList<>();
+        MutableDeletionInfo.Builder deletionBuilder = MutableDeletionInfo.builder(iterator.partitionLevelDeletion(), metadata.comparator, reversed);
+
+        while (iterator.hasNext())
+        {
+            Unfiltered unfiltered = iterator.next();
+            if (unfiltered.kind() == Unfiltered.Kind.ROW)
+                rows.add((Row)unfiltered);
+            else
+                deletionBuilder.add((RangeTombstoneMarker)unfiltered);
+        }
+
+        if (reversed)
+            Collections.reverse(rows);
+
+        return new PartitionUpdate(metadata, iterator.partitionKey(), iterator.columns(), iterator.staticRow(), rows, deletionBuilder.build(), iterator.stats(), true, false);
+    }
+
+    public static PartitionUpdate fromIterator(RowIterator iterator)
+    {
+        CFMetaData metadata = iterator.metadata();
+        boolean reversed = iterator.isReverseOrder();
+
+        List<Row> rows = new ArrayList<>();
+
+        RowStats.Collector collector = new RowStats.Collector();
+
+        while (iterator.hasNext())
+        {
+            Row row = iterator.next();
+            rows.add(row);
+            Rows.collectStats(row, collector);
+        }
+
+        if (reversed)
+            Collections.reverse(rows);
+
+        return new PartitionUpdate(metadata, iterator.partitionKey(), iterator.columns(), iterator.staticRow(), rows, MutableDeletionInfo.live(), collector.get(), true, false);
+    }
+
+    protected boolean canHaveShadowedData()
     {
-        this(metadata,
-             key,
-             DeletionInfo.live(),
-             columns,
-             initialRowCapacity);
+        return canHaveShadowedData;
     }
 
-    protected Writer createWriter()
+    public Row staticRow()
     {
-        return new RegularWriter();
+        return staticRow;
     }
 
-    protected StaticWriter createStaticWriter()
+    public DeletionInfo deletionInfo()
     {
-        return new StaticWriter();
+        return deletionInfo;
     }
 
     /**
@@ -166,7 +252,7 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
     {
         try (DataOutputBuffer out = new DataOutputBuffer())
         {
-            serializer.serialize(update, out, MessagingService.current_version);
+            serializer.serialize(update, out, version);
             return ByteBuffer.wrap(out.getData(), 0, out.getLength());
         }
         catch (IOException e)
@@ -176,60 +262,6 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
     }
 
     /**
-     * Creates a empty immutable partition update.
-     *
-     * @param metadata the metadata for the created update.
-     * @param key the partition key for the created update.
-     *
-     * @return the newly created empty (and immutable) update.
-     */
-    public static PartitionUpdate emptyUpdate(CFMetaData metadata, DecoratedKey key)
-    {
-        return new PartitionUpdate(metadata, key, PartitionColumns.NONE, 0)
-        {
-            public Row.Writer staticWriter()
-            {
-                throw new UnsupportedOperationException();
-            }
-
-            public Row.Writer writer()
-            {
-                throw new UnsupportedOperationException();
-            }
-
-            public void addPartitionDeletion(DeletionTime deletionTime)
-            {
-                throw new UnsupportedOperationException();
-            }
-
-            public void addRangeTombstone(RangeTombstone range)
-            {
-                throw new UnsupportedOperationException();
-            }
-        };
-    }
-
-    /**
-     * Creates a partition update that entirely deletes a given partition.
-     *
-     * @param metadata the metadata for the created update.
-     * @param key the partition key for the partition that the created update should delete.
-     * @param timestamp the timestamp for the deletion.
-     * @param nowInSec the current time in seconds to use as local deletion time for the partition deletion.
-     *
-     * @return the newly created partition deletion update.
-     */
-    public static PartitionUpdate fullPartitionDelete(CFMetaData metadata, DecoratedKey key, long timestamp, int nowInSec)
-    {
-        return new PartitionUpdate(metadata,
-                                   key,
-                                   new DeletionInfo(timestamp, nowInSec),
-                                   new RowDataBlock(Columns.NONE, 0, true, metadata.isCounter()),
-                                   PartitionColumns.NONE,
-                                   0);
-    }
-
-    /**
      * Merges the provided updates, yielding a new update that incorporates all those updates.
      *
      * @param updates the collection of updates to merge. This shouldn't be empty.
@@ -239,17 +271,30 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
     public static PartitionUpdate merge(Collection<PartitionUpdate> updates)
     {
         assert !updates.isEmpty();
-        if (updates.size() == 1)
+        final int size = updates.size();
+
+        if (size == 1)
             return Iterables.getOnlyElement(updates);
 
-        int totalSize = 0;
+        // Used when merging row to decide of liveness
+        int nowInSec = FBUtilities.nowInSeconds();
+
         PartitionColumns.Builder builder = PartitionColumns.builder();
         DecoratedKey key = null;
         CFMetaData metadata = null;
+        MutableDeletionInfo deletion = MutableDeletionInfo.live();
+        Row staticRow = Rows.EMPTY_STATIC_ROW;
+        List<Iterator<Row>> updateRowIterators = new ArrayList<>(size);
+        RowStats stats = RowStats.NO_STATS;
+
         for (PartitionUpdate update : updates)
         {
-            totalSize += update.rows;
             builder.addAll(update.columns());
+            deletion.add(update.deletionInfo());
+            if (!update.staticRow().isEmpty())
+                staticRow = staticRow == Rows.EMPTY_STATIC_ROW ? update.staticRow() : Rows.merge(staticRow, update.staticRow(), nowInSec);
+            updateRowIterators.add(update.iterator());
+            stats = stats.mergeWith(update.stats());
 
             if (key == null)
                 key = update.partitionKey();
@@ -262,23 +307,70 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
                 assert metadata.cfId.equals(update.metadata().cfId);
         }
 
-        // Used when merging row to decide of liveness
-        int nowInSec = FBUtilities.nowInSeconds();
-        PartitionUpdate newUpdate = new PartitionUpdate(metadata, key, builder.build(), totalSize);
-        for (PartitionUpdate update : updates)
+        PartitionColumns columns = builder.build();
+
+        final Row.Merger merger = new Row.Merger(size, nowInSec, columns.regulars);
+
+        Iterator<Row> merged = MergeIterator.get(updateRowIterators, metadata.comparator, new MergeIterator.Reducer<Row, Row>()
         {
-            newUpdate.deletionInfo.add(update.deletionInfo);
-            if (!update.staticRow().isEmpty())
+            @Override
+            public boolean trivialReduceIsTrivial()
             {
-                if (newUpdate.staticRow().isEmpty())
-                    newUpdate.staticRow = update.staticRow().takeAlias();
-                else
-                    Rows.merge(newUpdate.staticRow(), update.staticRow(), newUpdate.columns().statics, newUpdate.staticWriter(), nowInSec, SecondaryIndexManager.nullUpdater);
+                return true;
             }
-            for (Row row : update)
-                row.copyTo(newUpdate.writer);
-        }
-        return newUpdate;
+
+            public void reduce(int idx, Row current)
+            {
+                merger.add(idx, current);
+            }
+
+            protected Row getReduced()
+            {
+                // Note that while merger.getRow() can theoretically return null, it won't in this case because
+                // we don't pass an "activeDeletion".
+                return merger.merge(DeletionTime.LIVE);
+            }
+
+            @Override
+            protected void onKeyChange()
+            {
+                merger.clear();
+            }
+        });
+
+        List<Row> rows = new ArrayList<>();
+        Iterators.addAll(rows, merged);
+
+        return new PartitionUpdate(metadata, key, columns, staticRow, rows, deletion, stats, true, true);
+    }
+
+    /**
+     * Modify this update to set every timestamp for live data to {@code newTimestamp} and
+     * every deletion timestamp to {@code newTimestamp - 1}.
+     *
+     * There is no reason to use that expect on the Paxos code path, where we need ensure that
+     * anything inserted use the ballot timestamp (to respect the order of update decided by
+     * the Paxos algorithm). We use {@code newTimestamp - 1} for deletions because tombstones
+     * always win on timestamp equality and we don't want to delete our own insertions
+     * (typically, when we overwrite a collection, we first set a complex deletion to delete the
+     * previous collection before adding new elements. If we were to set that complex deletion
+     * to the same timestamp that the new elements, it would delete those elements). And since
+     * tombstones always wins on timestamp equality, using -1 guarantees our deletion will still
+     * delete anything from a previous update.
+     */
+    public void updateAllTimestamp(long newTimestamp)
+    {
+        // We know we won't be updating that update again after this call, and doing is post built is potentially
+        // slightly more efficient (things are more "compact"). So force a build if it hasn't happened yet.
+        maybeBuild();
+
+        deletionInfo.updateAllTimestamp(newTimestamp - 1);
+
+        if (!staticRow.isEmpty())
+            staticRow = staticRow.updateAllTimestamp(newTimestamp);
+
+        for (int i = 0; i < rows.size(); i++)
+            rows.set(i, rows.get(i).updateAllTimestamp(newTimestamp));
     }
 
     /**
@@ -291,7 +383,7 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
      */
     public int operationCount()
     {
-        return rowCount()
+        return rows.size()
              + deletionInfo.rangeCount()
              + (deletionInfo.getPartitionDeletion().isLive() ? 0 : 1);
     }
@@ -303,17 +395,29 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
      */
     public int dataSize()
     {
-        int clusteringSize = metadata().comparator.size();
         int size = 0;
         for (Row row : this)
         {
             size += row.clustering().dataSize();
-            for (Cell cell : row)
-                size += cell.dataSize();
+            for (ColumnData cd : row)
+                size += cd.dataSize();
         }
         return size;
     }
 
+    @Override
+    public int rowCount()
+    {
+        maybeBuild();
+        return super.rowCount();
+    }
+
+    public RowStats stats()
+    {
+        maybeBuild();
+        return stats;
+    }
+
     /**
      * If a partition update has been read (and is thus unmodifiable), a call to this method
      * makes the update modifiable again.
@@ -325,21 +429,17 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
      */
     public synchronized void allowNewUpdates()
     {
+        if (!canReOpen)
+            throw new IllegalStateException("You cannot do more updates on collectCounterMarks has been called");
+
         // This is synchronized to make extra sure things work properly even if this is
         // called concurrently with sort() (which should be avoided in the first place, but
         // better safe than sorry).
-        isSorted = false;
-    }
-
-    @Override
-    public int rowCount()
-    {
-        maybeSort();
-        return super.rowCount();
+        isBuilt = false;
     }
 
     /**
-     * Returns an iterator that iterators over the rows of this update in clustering order.
+     * Returns an iterator that iterates over the rows of this update in clustering order.
      * <p>
      * Note that this might trigger a sorting of the update, and as such the update will not
      * be modifiable anymore after this call.
@@ -349,14 +449,14 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
     @Override
     public Iterator<Row> iterator()
     {
-        maybeSort();
+        maybeBuild();
         return super.iterator();
     }
 
     @Override
     protected SliceableUnfilteredRowIterator sliceableUnfilteredIterator(ColumnFilter columns, boolean reversed)
     {
-        maybeSort();
+        maybeBuild();
         return super.sliceableUnfilteredIterator(columns, reversed);
     }
 
@@ -370,8 +470,8 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
         for (Row row : this)
         {
             metadata().comparator.validate(row.clustering());
-            for (Cell cell : row)
-                cell.validate();
+            for (ColumnData cd : row)
+                cd.validate();
         }
     }
 
@@ -382,6 +482,27 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
      */
     public long maxTimestamp()
     {
+        maybeBuild();
+
+        long maxTimestamp = deletionInfo.maxTimestamp();
+        for (Row row : this)
+        {
+            maxTimestamp = Math.max(maxTimestamp, row.primaryKeyLivenessInfo().timestamp());
+            for (ColumnData cd : row)
+            {
+                if (cd.column().isSimple())
+                {
+                    maxTimestamp = Math.max(maxTimestamp, ((Cell)cd).timestamp());
+                }
+                else
+                {
+                    ComplexColumnData complexData = (ComplexColumnData)cd;
+                    maxTimestamp = Math.max(maxTimestamp, complexData.complexDeletion().markedForDeleteAt());
+                    for (Cell cell : complexData)
+                        maxTimestamp = Math.max(maxTimestamp, cell.timestamp());
+                }
+            }
+        }
         return maxTimestamp;
     }
 
@@ -394,62 +515,73 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
     public List<CounterMark> collectCounterMarks()
     {
         assert metadata().isCounter();
+        maybeBuild();
+        // We will take aliases on the rows of this update, and update them in-place. So we should be sure the
+        // update is no immutable for all intent and purposes.
+        canReOpen = false;
 
-        InternalReusableClustering clustering = new InternalReusableClustering();
         List<CounterMark> l = new ArrayList<>();
-        int i = 0;
-        for (Row row : this)
+        for (Row row : rows)
         {
-            for (Cell cell : row)
+            for (Cell cell : row.cells())
+            {
                 if (cell.isCounterCell())
-                    l.add(new CounterMark(clustering, i, cell.column(), cell.path()));
-            i++;
+                    l.add(new CounterMark(row, cell.column(), cell.path()));
+            }
         }
         return l;
     }
 
-    /**
-     * Returns a row writer for the static row of this partition update.
-     *
-     * @return a row writer for the static row of this partition update. A partition
-     * update contains only one static row so only one row should be written through
-     * this writer (but if multiple rows are added, the latest written one wins).
-     */
-    public Row.Writer staticWriter()
+    private void assertNotBuilt()
     {
-        return createStaticWriter();
+        if (isBuilt)
+            throw new IllegalStateException("An update should not be written again once it has been read");
     }
 
-    /**
-     * Returns a row writer to add (non-static) rows to this partition update.
-     *
-     * @return a row writer to add (non-static) rows to this partition update.
-     * Multiple rows can be successively added this way and the rows added do not have
-     * to be in clustering order. Further, the same row can be added multiple time.
-     *
-     */
-    public Row.Writer writer()
+    public void addPartitionDeletion(DeletionTime deletionTime)
     {
-        if (isSorted)
-            throw new IllegalStateException("An update should not written again once it has been read");
+        assertNotBuilt();
+        deletionInfo.add(deletionTime);
+    }
 
-        return writer;
+    public void add(RangeTombstone range)
+    {
+        assertNotBuilt();
+        deletionInfo.add(range, metadata.comparator);
     }
 
     /**
-     * Returns a range tombstone marker writer to add range tombstones to this
-     * partition update.
-     * <p>
-     * Note that if more convenient, range tombstones can also be added using
-     * {@link addRangeTombstone}.
+     * Adds a row to this update.
+     *
+     * There is no particular assumption made on the order of row added to a partition update. It is further
+     * allowed to add the same row (more precisely, multiple row objects for the same clustering).
      *
-     * @param isReverseOrder whether the range tombstone marker will be provided to the returned writer
-     * in clustering order or in reverse clustering order.
-     * @return a range tombstone marker writer to add range tombstones to this update.
+     * Note however that the columns contained in the added row must be a subset of the columns used when
+     * creating this update.
+     *
+     * @param row the row to add.
      */
-    public RangeTombstoneMarker.Writer markerWriter(boolean isReverseOrder)
+    public void add(Row row)
     {
-        return new RangeTombstoneCollector(isReverseOrder);
+        if (row.isEmpty())
+            return;
+
+        assertNotBuilt();
+
+        if (row.isStatic())
+        {
+            // We test for == first because in most case it'll be true and that is faster
+            assert columns().statics == row.columns() || columns().statics.contains(row.columns());
+            staticRow = staticRow.isEmpty()
+                      ? row
+                      : Rows.merge(staticRow, row, createdAtInSec);
+        }
+        else
+        {
+            // We test for == first because in most case it'll be true and that is faster
+            assert columns().regulars == row.columns() || columns().regulars.contains(row.columns());
+            rows.add(row);
+        }
     }
 
     /**
@@ -459,160 +591,70 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
      */
     public int size()
     {
-        return rows;
+        return rows.size();
     }
 
-    private void maybeSort()
+    private void maybeBuild()
     {
-        if (isSorted)
+        if (isBuilt)
             return;
 
-        sort();
+        build();
     }
 
-    private synchronized void sort()
+    private synchronized void build()
     {
-        if (isSorted)
+        if (isBuilt)
             return;
 
-        if (rows <= 1)
+        if (rows.size() <= 1)
         {
-            isSorted = true;
+            finishBuild();
             return;
         }
 
-        // Sort the rows - will still potentially contain duplicate (non-reconciled) rows
-        Sorting.sort(this);
+        Comparator<Row> comparator = metadata.comparator.rowComparator();
+        // Sort the rows. Because the same row can have been added multiple times, we can still have duplicates after that
+        Collections.sort(rows, comparator);
 
-        // Now find duplicates and merge them together
+        // Now find the duplicates and merge them together
         int previous = 0; // The last element that was set
-        int nowInSec = FBUtilities.nowInSeconds();
-        for (int current = 1; current < rows; current++)
+        for (int current = 1; current < rows.size(); current++)
         {
             // There is really only 2 possible comparison: < 0 or == 0 since we've sorted already
-            int cmp = compare(previous, current);
+            Row previousRow = rows.get(previous);
+            Row currentRow = rows.get(current);
+            int cmp = comparator.compare(previousRow, currentRow);
             if (cmp == 0)
             {
                 // current and previous are the same row. Merge current into previous
                 // (and so previous + 1 will be "free").
-                merge(current, previous, nowInSec);
+                rows.set(previous, Rows.merge(previousRow, currentRow, createdAtInSec));
             }
             else
             {
-                // data[current] != [previous], so move current just after previous if needs be
+                // current != previous, so move current just after previous if needs be
                 ++previous;
                 if (previous != current)
-                    move(current, previous);
+                    rows.set(previous, currentRow);
             }
         }
 
         // previous is on the last value to keep
-        rows = previous + 1;
-
-        isSorted = true;
-    }
+        for (int j = rows.size() - 1; j > previous; j--)
+            rows.remove(j);
 
-    /**
-     * This method is note meant to be used externally: it is only public so this
-     * update conform to the {@link Sorting.Sortable} interface.
-     */
-    public int compare(int i, int j)
-    {
-        return metadata.comparator.compare(p1.setTo(i), p2.setTo(j));
-    }
-
-    protected class StaticWriter extends StaticRow.Builder
-    {
-        protected StaticWriter()
-        {
-            super(columns.statics, false, metadata().isCounter());
-        }
-
-        @Override
-        public void endOfRow()
-        {
-            super.endOfRow();
-            if (staticRow == null)
-            {
-                staticRow = build();
-            }
-            else
-            {
-                StaticRow.Builder builder = StaticRow.builder(columns.statics, true, metadata().isCounter());
-                Rows.merge(staticRow, build(), columns.statics, builder, FBUtilities.nowInSeconds());
-                staticRow = builder.build();
-            }
-        }
+        finishBuild();
     }
 
-    protected class RegularWriter extends Writer
+    private void finishBuild()
     {
-        // For complex column, the writer assumptions is that for a given row, cells of different
-        // complex columns are not intermingled (they also should be in cellPath order). We however
-        // don't yet guarantee that this will be the case for updates (both UpdateStatement and
-        // RowUpdateBuilder can potentially break that assumption; we could change those classes but
-        // that's non trivial, at least for UpdateStatement).
-        // To deal with that problem, we record which complex columns have been updated (for the current
-        // row) and if we detect a violation of our assumption, we switch the row we're writing
-        // into (which is ok because everything will be sorted and merged in maybeSort()).
-        private final Set<ColumnDefinition> updatedComplex = new HashSet();
-        private ColumnDefinition lastUpdatedComplex;
-        private CellPath lastUpdatedComplexPath;
-
-        public RegularWriter()
-        {
-            super(false);
-        }
-
-        @Override
-        public void writeCell(ColumnDefinition column, boolean isCounter, ByteBuffer value, LivenessInfo info, CellPath path)
-        {
-            if (column.isComplex())
-            {
-                if (updatedComplex.contains(column)
-                    && (!column.equals(lastUpdatedComplex) || (column.cellPathComparator().compare(path, lastUpdatedComplexPath)) <= 0))
-                {
-                    // We've updated that complex already, but we've either updated another complex or it's not in order: as this
-                    // break the writer assumption, switch rows.
-                    endOfRow();
-
-                    // Copy the clustering values from the previous row
-                    int clusteringSize = metadata.clusteringColumns().size();
-                    int base = (row - 1) * clusteringSize;
-                    for (int i = 0; i < clusteringSize; i++)
-                        writer.writeClusteringValue(clusterings[base + i]);
-
-                    updatedComplex.clear();
-                }
-
-                lastUpdatedComplex = column;
-                lastUpdatedComplexPath = path;
-                updatedComplex.add(column);
-            }
-            super.writeCell(column, isCounter, value, info, path);
-        }
-
-        @Override
-        public void endOfRow()
-        {
-            super.endOfRow();
-            clear();
-        }
-
-        @Override
-        public Writer reset()
-        {
-            super.reset();
-            clear();
-            return this;
-        }
-
-        private void clear()
-        {
-            updatedComplex.clear();
-            lastUpdatedComplex = null;
-            lastUpdatedComplexPath = null;
-        }
+        RowStats.Collector collector = new RowStats.Collector();
+        deletionInfo.collectStats(collector);
+        for (Row row : rows)
+            Rows.collectStats(row, collector);
+        stats = collector.get();
+        isBuilt = true;
     }
 
     public static class PartitionUpdateSerializer
@@ -648,7 +690,7 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
             try (UnfilteredRowIterator iter = update.sliceableUnfilteredIterator())
             {
                 assert !iter.isReverseOrder();
-                UnfilteredRowIteratorSerializer.serializer.serialize(iter, out, version, update.rows);
+                UnfilteredRowIteratorSerializer.serializer.serialize(iter, out, version, update.rows.size());
             }
         }
 
@@ -666,37 +708,46 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
                 LegacyLayout.LegacyDeletionInfo info = LegacyLayout.LegacyDeletionInfo.serializer.deserialize(metadata, in, version);
                 int size = in.readInt();
                 Iterator<LegacyLayout.LegacyCell> cells = LegacyLayout.deserializeCells(metadata, in, flag, size);
-                SerializationHelper helper = new SerializationHelper(version, flag);
+                SerializationHelper helper = new SerializationHelper(metadata, version, flag);
                 try (UnfilteredRowIterator iterator = LegacyLayout.onWireCellstoUnfilteredRowIterator(metadata, key, info, cells, false, helper))
                 {
-                    return UnfilteredRowIterators.toUpdate(iterator);
+                    return PartitionUpdate.fromIterator(iterator);
                 }
             }
 
             assert key == null; // key is only there for the old format
 
-            UnfilteredRowIteratorSerializer.Header h = UnfilteredRowIteratorSerializer.serializer.deserializeHeader(in, version, flag);
-            if (h.isEmpty)
-                return emptyUpdate(h.metadata, h.key);
+            UnfilteredRowIteratorSerializer.Header header = UnfilteredRowIteratorSerializer.serializer.deserializeHeader(in, version, flag);
+            if (header.isEmpty)
+                return emptyUpdate(header.metadata, header.key);
 
-            assert !h.isReversed;
-            assert h.rowEstimate >= 0;
-            PartitionUpdate upd = new PartitionUpdate(h.metadata,
-                                                      h.key,
-                                                      new DeletionInfo(h.partitionDeletion),
-                                                      new RowDataBlock(h.sHeader.columns().regulars, h.rowEstimate, false, h.metadata.isCounter()),
-                                                      h.sHeader.columns(),
-                                                      h.rowEstimate);
+            assert !header.isReversed;
+            assert header.rowEstimate >= 0;
 
-            upd.staticRow = h.staticRow;
+            MutableDeletionInfo.Builder deletionBuilder = MutableDeletionInfo.builder(header.partitionDeletion, header.metadata.comparator, false);
+            List<Row> rows = new ArrayList<>(header.rowEstimate);
 
-            RangeTombstoneMarker.Writer markerWriter = upd.markerWriter(false);
-            UnfilteredRowIteratorSerializer.serializer.deserialize(in, new SerializationHelper(version, flag), h.sHeader, upd.writer(), markerWriter);
-
-            // Mark sorted after we're read it all since that's what we use in the writer() method to detect bad uses
-            upd.isSorted = true;
+            try (UnfilteredRowIterator partition = UnfilteredRowIteratorSerializer.serializer.deserialize(in, version, flag, header))
+            {
+                while (partition.hasNext())
+                {
+                    Unfiltered unfiltered = partition.next();
+                    if (unfiltered.kind() == Unfiltered.Kind.ROW)
+                        rows.add((Row)unfiltered);
+                    else
+                        deletionBuilder.add((RangeTombstoneMarker)unfiltered);
+                }
+            }
 
-            return upd;
+            return new PartitionUpdate(header.metadata,
+                                       header.key,
+                                       header.sHeader.columns(),
+                                       header.staticRow,
+                                       rows,
+                                       deletionBuilder.build(),
+                                       header.sHeader.stats(),
+                                       true,
+                                       false);
         }
 
         public long serializedSize(PartitionUpdate update, int version)
@@ -719,7 +770,7 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
 
             try (UnfilteredRowIterator iter = update.sliceableUnfilteredIterator())
             {
-                return UnfilteredRowIteratorSerializer.serializer.serializedSize(iter, version, update.rows);
+                return UnfilteredRowIteratorSerializer.serializer.serializedSize(iter, version, update.rows.size());
             }
         }
     }
@@ -729,16 +780,14 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
      * us to update the counter value based on the pre-existing value read during the read-before-write that counters
      * do. See {@link CounterMutation} to understand how this is used.
      */
-    public class CounterMark
+    public static class CounterMark
     {
-        private final InternalReusableClustering clustering;
-        private final int row;
+        private final Row row;
         private final ColumnDefinition column;
         private final CellPath path;
 
-        private CounterMark(InternalReusableClustering clustering, int row, ColumnDefinition column, CellPath path)
+        private CounterMark(Row row, ColumnDefinition column, CellPath path)
         {
-            this.clustering = clustering;
             this.row = row;
             this.column = column;
             this.path = path;
@@ -746,7 +795,7 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
 
         public Clustering clustering()
         {
-            return clustering.setTo(row);
+            return row.clustering();
         }
 
         public ColumnDefinition column()
@@ -761,12 +810,17 @@ public class PartitionUpdate extends AbstractPartitionData implements Sorting.So
 
         public ByteBuffer value()
         {
-            return data.getValue(row, column, path);
+            return path == null
+                 ? row.getCell(column).value()
+                 : row.getCell(column, path).value();
         }
 
         public void setValue(ByteBuffer value)
         {
-            data.setValue(row, column, path, value);
+            // This is a bit of a giant hack as this is the only place where we mutate a Row object. This makes it more efficient
+            // for counters however and this won't be needed post-#6506 so that's probably fine.
+            assert row instanceof ArrayBackedRow;
+            ((ArrayBackedRow)row).setValue(column, path, value);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/partitions/PurgingPartitionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/PurgingPartitionIterator.java b/src/java/org/apache/cassandra/db/partitions/PurgingPartitionIterator.java
new file mode 100644
index 0000000..492fe1d
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/partitions/PurgingPartitionIterator.java
@@ -0,0 +1,150 @@
+/*
+ * 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.partitions;
+
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.rows.*;
+
+public abstract class PurgingPartitionIterator extends WrappingUnfilteredPartitionIterator
+{
+    private final DeletionPurger purger;
+    private final int gcBefore;
+
+    private UnfilteredRowIterator next;
+
+    public PurgingPartitionIterator(UnfilteredPartitionIterator iterator, int gcBefore)
+    {
+        super(iterator);
+        this.gcBefore = gcBefore;
+        this.purger = new DeletionPurger()
+        {
+            public boolean shouldPurge(long timestamp, int localDeletionTime)
+            {
+                return timestamp < getMaxPurgeableTimestamp() && localDeletionTime < gcBefore;
+            }
+        };
+    }
+
+    protected abstract long getMaxPurgeableTimestamp();
+
+    // Called at the beginning of each new partition
+    protected void onNewPartition(DecoratedKey partitionKey)
+    {
+    }
+
+    // Called for each partition that had only purged infos and are empty post-purge.
+    protected void onEmptyPartitionPostPurge(DecoratedKey partitionKey)
+    {
+    }
+
+    // Called for every unfiltered. Meant for CompactionIterator to update progress
+    protected void updateProgress()
+    {
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+        while (next == null && super.hasNext())
+        {
+            UnfilteredRowIterator iterator = super.next();
+            onNewPartition(iterator.partitionKey());
+
+            UnfilteredRowIterator purged = purge(iterator);
+            if (isForThrift() || !purged.isEmpty())
+            {
+                next = purged;
+                return true;
+            }
+
+            onEmptyPartitionPostPurge(purged.partitionKey());
+        }
+        return next != null;
+    }
+
+    @Override
+    public UnfilteredRowIterator next()
+    {
+        UnfilteredRowIterator toReturn = next;
+        next = null;
+        return toReturn;
+    }
+
+    private UnfilteredRowIterator purge(final UnfilteredRowIterator iter)
+    {
+        return new AlteringUnfilteredRowIterator(iter)
+        {
+            @Override
+            public DeletionTime partitionLevelDeletion()
+            {
+                DeletionTime dt = iter.partitionLevelDeletion();
+                return purger.shouldPurge(dt) ? DeletionTime.LIVE : dt;
+            }
+
+            @Override
+            public Row computeNextStatic(Row row)
+            {
+                return row.purge(purger, gcBefore);
+            }
+
+            @Override
+            public Row computeNext(Row row)
+            {
+                return row.purge(purger, gcBefore);
+            }
+
+            @Override
+            public RangeTombstoneMarker computeNext(RangeTombstoneMarker marker)
+            {
+                boolean reversed = isReverseOrder();
+                if (marker.isBoundary())
+                {
+                    // We can only skip the whole marker if both deletion time are purgeable.
+                    // If only one of them is, filterTombstoneMarker will deal with it.
+                    RangeTombstoneBoundaryMarker boundary = (RangeTombstoneBoundaryMarker)marker;
+                    boolean shouldPurgeClose = purger.shouldPurge(boundary.closeDeletionTime(reversed));
+                    boolean shouldPurgeOpen = purger.shouldPurge(boundary.openDeletionTime(reversed));
+
+                    if (shouldPurgeClose)
+                    {
+                        if (shouldPurgeOpen)
+                            return null;
+
+                        return boundary.createCorrespondingOpenMarker(reversed);
+                    }
+
+                    return shouldPurgeOpen
+                         ? boundary.createCorrespondingCloseMarker(reversed)
+                         : marker;
+                }
+                else
+                {
+                    return purger.shouldPurge(((RangeTombstoneBoundMarker)marker).deletionTime()) ? null : marker;
+                }
+            }
+
+            @Override
+            public Unfiltered next()
+            {
+                Unfiltered next = super.next();
+                updateProgress();
+                return next;
+            }
+        };
+    }
+};

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/partitions/TombstonePurgingPartitionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/TombstonePurgingPartitionIterator.java b/src/java/org/apache/cassandra/db/partitions/TombstonePurgingPartitionIterator.java
deleted file mode 100644
index 10022eb..0000000
--- a/src/java/org/apache/cassandra/db/partitions/TombstonePurgingPartitionIterator.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * 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.partitions;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.*;
-
-public abstract class TombstonePurgingPartitionIterator extends FilteringPartitionIterator
-{
-    private final int gcBefore;
-
-    public TombstonePurgingPartitionIterator(UnfilteredPartitionIterator iterator, int gcBefore)
-    {
-        super(iterator);
-        this.gcBefore = gcBefore;
-    }
-
-    protected abstract long getMaxPurgeableTimestamp();
-
-    protected FilteringRow makeRowFilter()
-    {
-        return new FilteringRow()
-        {
-            @Override
-            protected boolean include(LivenessInfo info)
-            {
-                return !info.hasLocalDeletionTime() || !info.isPurgeable(getMaxPurgeableTimestamp(), gcBefore);
-            }
-
-            @Override
-            protected boolean include(DeletionTime dt)
-            {
-                return includeDelTime(dt);
-            }
-
-            @Override
-            protected boolean include(ColumnDefinition c, DeletionTime dt)
-            {
-                return includeDelTime(dt);
-            }
-        };
-    }
-
-    private boolean includeDelTime(DeletionTime dt)
-    {
-        return dt.isLive() || !dt.isPurgeable(getMaxPurgeableTimestamp(), gcBefore);
-    }
-
-    @Override
-    protected boolean includePartitionDeletion(DeletionTime dt)
-    {
-        return includeDelTime(dt);
-    }
-
-    @Override
-    protected boolean includeRangeTombstoneMarker(RangeTombstoneMarker marker)
-    {
-        if (marker.isBoundary())
-        {
-            // We can only skip the whole marker if both deletion time are purgeable.
-            // If only one of them is, filterTombstoneMarker will deal with it.
-            RangeTombstoneBoundaryMarker boundary = (RangeTombstoneBoundaryMarker)marker;
-            return includeDelTime(boundary.endDeletionTime()) || includeDelTime(boundary.startDeletionTime());
-        }
-        else
-        {
-            return includeDelTime(((RangeTombstoneBoundMarker)marker).deletionTime());
-        }
-    }
-
-    @Override
-    protected RangeTombstoneMarker filterRangeTombstoneMarker(RangeTombstoneMarker marker, boolean reversed)
-    {
-        if (!marker.isBoundary())
-            return marker;
-
-        // Note that we know this is called after includeRangeTombstoneMarker. So if one of the deletion time is
-        // purgeable, we know the other one isn't.
-        RangeTombstoneBoundaryMarker boundary = (RangeTombstoneBoundaryMarker)marker;
-        if (!(includeDelTime(boundary.closeDeletionTime(reversed))))
-            return boundary.createCorrespondingCloseBound(reversed);
-        else if (!(includeDelTime(boundary.openDeletionTime(reversed))))
-            return boundary.createCorrespondingOpenBound(reversed);
-        return boundary;
-    }
-
-};

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java b/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java
index 0d3d364..4414f44 100644
--- a/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java
+++ b/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java
@@ -19,13 +19,10 @@ package org.apache.cassandra.db.partitions;
 
 import java.io.IOError;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.security.MessageDigest;
 import java.util.*;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -33,25 +30,14 @@ import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.MergeIterator;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /**
  * Static methods to work with partition iterators.
  */
 public abstract class UnfilteredPartitionIterators
 {
-    private static final Logger logger = LoggerFactory.getLogger(UnfilteredPartitionIterators.class);
-
     private static final Serializer serializer = new Serializer();
 
-    private static final Comparator<UnfilteredRowIterator> partitionComparator = new Comparator<UnfilteredRowIterator>()
-    {
-        public int compare(UnfilteredRowIterator p1, UnfilteredRowIterator p2)
-        {
-            return p1.partitionKey().compareTo(p2.partitionKey());
-        }
-    };
+    private static final Comparator<UnfilteredRowIterator> partitionComparator = (p1, p2) -> p1.partitionKey().compareTo(p2.partitionKey());
 
     public static final UnfilteredPartitionIterator EMPTY = new AbstractUnfilteredPartitionIterator()
     {
@@ -242,28 +228,6 @@ public abstract class UnfilteredPartitionIterators
         };
     }
 
-    /**
-     * Convert all expired cells to equivalent tombstones.
-     * <p>
-     * See {@link UnfilteredRowIterators#convertExpiredCellsToTombstones} for details.
-     *
-     * @param iterator the iterator in which to conver expired cells.
-     * @param nowInSec the current time to use to decide if a cell is expired.
-     * @return an iterator that returns the same data than {@code iterator} but with all expired cells converted
-     * to equivalent tombstones.
-     */
-    public static UnfilteredPartitionIterator convertExpiredCellsToTombstones(UnfilteredPartitionIterator iterator, final int nowInSec)
-    {
-        return new WrappingUnfilteredPartitionIterator(iterator)
-        {
-            @Override
-            protected UnfilteredRowIterator computeNext(UnfilteredRowIterator iter)
-            {
-                return UnfilteredRowIterators.convertExpiredCellsToTombstones(iter, nowInSec);
-            }
-        };
-    }
-
     public static UnfilteredPartitionIterator mergeLazily(final List<? extends UnfilteredPartitionIterator> iterators, final int nowInSec)
     {
         assert !iterators.isEmpty();
@@ -330,52 +294,6 @@ public abstract class UnfilteredPartitionIterators
         };
     }
 
-    public static UnfilteredPartitionIterator removeDroppedColumns(UnfilteredPartitionIterator iterator, final Map<ByteBuffer, CFMetaData.DroppedColumn> droppedColumns)
-    {
-        return new FilteringPartitionIterator(iterator)
-        {
-            @Override
-            protected FilteringRow makeRowFilter()
-            {
-                return new FilteringRow()
-                {
-                    @Override
-                    protected boolean include(Cell cell)
-                    {
-                        return include(cell.column(), cell.livenessInfo().timestamp());
-                    }
-
-                    @Override
-                    protected boolean include(ColumnDefinition c, DeletionTime dt)
-                    {
-                        return include(c, dt.markedForDeleteAt());
-                    }
-
-                    private boolean include(ColumnDefinition column, long timestamp)
-                    {
-                        CFMetaData.DroppedColumn dropped = droppedColumns.get(column.name.bytes);
-                        return dropped == null || timestamp > dropped.droppedTime;
-                    }
-                };
-            }
-
-            @Override
-            protected boolean shouldFilter(UnfilteredRowIterator iterator)
-            {
-                // TODO: We could have row iterators return the smallest timestamp they might return
-                // (which we can get from sstable stats), and ignore any dropping if that smallest
-                // timestamp is bigger that the biggest droppedColumns timestamp.
-
-                // If none of the dropped columns is part of the columns that the iterator actually returns, there is nothing to do;
-                for (ColumnDefinition c : iterator.columns())
-                    if (droppedColumns.containsKey(c.name.bytes))
-                        return true;
-
-                return false;
-            }
-        };
-    }
-
     public static void digest(UnfilteredPartitionIterator iterator, MessageDigest digest)
     {
         try (UnfilteredPartitionIterator iter = iterator)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/AbstractCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractCell.java b/src/java/org/apache/cassandra/db/rows/AbstractCell.java
index c003d6f..807741a 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractCell.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractCell.java
@@ -34,30 +34,12 @@ import org.apache.cassandra.utils.FBUtilities;
  */
 public abstract class AbstractCell implements Cell
 {
-    public boolean isLive(int nowInSec)
-    {
-        return livenessInfo().isLive(nowInSec);
-    }
-
-    public boolean isTombstone()
-    {
-        return livenessInfo().hasLocalDeletionTime() && !livenessInfo().hasTTL();
-    }
-
-    public boolean isExpiring()
-    {
-        return livenessInfo().hasTTL();
-    }
-
-    public void writeTo(Row.Writer writer)
-    {
-        writer.writeCell(column(), isCounterCell(), value(), livenessInfo(), path());
-    }
-
     public void digest(MessageDigest digest)
     {
         digest.update(value().duplicate());
-        livenessInfo().digest(digest);
+        FBUtilities.updateWithLong(digest, timestamp());
+        FBUtilities.updateWithInt(digest, localDeletionTime());
+        FBUtilities.updateWithInt(digest, ttl());
         FBUtilities.updateWithBoolean(digest, isCounterCell());
         if (path() != null)
             path().digest(digest);
@@ -67,7 +49,12 @@ public abstract class AbstractCell implements Cell
     {
         column().validateCellValue(value());
 
-        livenessInfo().validate();
+        if (ttl() < 0)
+            throw new MarshalException("A TTL should not be negative");
+        if (localDeletionTime() < 0)
+            throw new MarshalException("A local deletion time should not be negative");
+        if (isExpiring() && localDeletionTime() == NO_DELETION_TIME)
+            throw new MarshalException("Shoud not have a TTL without an associated local deletion time");
 
         // If cell is a tombstone, it shouldn't have a value.
         if (isTombstone() && value().hasRemaining())
@@ -77,59 +64,58 @@ public abstract class AbstractCell implements Cell
             column().validateCellPath(path());
     }
 
-    public int dataSize()
-    {
-        int size = value().remaining() + livenessInfo().dataSize();
-        if (path() != null)
-            size += path().dataSize();
-        return size;
-
-    }
-
     @Override
     public boolean equals(Object other)
     {
+        if (this == other)
+            return true;
+
         if(!(other instanceof Cell))
             return false;
 
         Cell that = (Cell)other;
         return this.column().equals(that.column())
             && this.isCounterCell() == that.isCounterCell()
+            && this.timestamp() == that.timestamp()
+            && this.ttl() == that.ttl()
+            && this.localDeletionTime() == that.localDeletionTime()
             && Objects.equals(this.value(), that.value())
-            && Objects.equals(this.livenessInfo(), that.livenessInfo())
             && Objects.equals(this.path(), that.path());
     }
 
     @Override
     public int hashCode()
     {
-        return Objects.hash(column(), isCounterCell(), value(), livenessInfo(), path());
+        return Objects.hash(column(), isCounterCell(), timestamp(), ttl(), localDeletionTime(), value(), path());
     }
 
     @Override
     public String toString()
     {
         if (isCounterCell())
-            return String.format("[%s=%d ts=%d]", column().name, CounterContext.instance().total(value()), livenessInfo().timestamp());
+            return String.format("[%s=%d ts=%d]", column().name, CounterContext.instance().total(value()), timestamp());
 
         AbstractType<?> type = column().type;
         if (type instanceof CollectionType && type.isMultiCell())
         {
             CollectionType ct = (CollectionType)type;
-            return String.format("[%s[%s]=%s info=%s]",
+            return String.format("[%s[%s]=%s %s]",
                                  column().name,
                                  ct.nameComparator().getString(path().get(0)),
                                  ct.valueComparator().getString(value()),
-                                 livenessInfo());
+                                 livenessInfoString());
         }
-        return String.format("[%s=%s info=%s]", column().name, type.getString(value()), livenessInfo());
+        return String.format("[%s=%s %s]", column().name, type.getString(value()), livenessInfoString());
     }
 
-    public Cell takeAlias()
+    private String livenessInfoString()
     {
-        // Cell is always used as an Aliasable object but as the code currently
-        // never need to alias a cell outside of its valid scope, we don't yet
-        // need that.
-        throw new UnsupportedOperationException();
+        if (isExpiring())
+            return String.format("ts=%d ttl=%d ldt=%d", timestamp(), ttl(), localDeletionTime());
+        else if (isTombstone())
+            return String.format("ts=%d ldt=%d", timestamp(), localDeletionTime());
+        else
+            return String.format("ts=%d", timestamp());
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java b/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java
index d8256fc..e90e52b 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java
@@ -41,6 +41,21 @@ public abstract class AbstractRangeTombstoneMarker implements RangeTombstoneMark
         return Unfiltered.Kind.RANGE_TOMBSTONE_MARKER;
     }
 
+    public boolean isBoundary()
+    {
+        return bound.isBoundary();
+    }
+
+    public boolean isOpen(boolean reversed)
+    {
+        return bound.isOpen(reversed);
+    }
+
+    public boolean isClose(boolean reversed)
+    {
+        return bound.isClose(reversed);
+    }
+
     public void validateData(CFMetaData metadata)
     {
         Slice.Bound bound = clustering();
@@ -56,16 +71,4 @@ public abstract class AbstractRangeTombstoneMarker implements RangeTombstoneMark
     {
         return toString(metadata);
     }
-
-    protected void copyBoundTo(RangeTombstoneMarker.Writer writer)
-    {
-        for (int i = 0; i < bound.size(); i++)
-            writer.writeClusteringValue(bound.get(i));
-        writer.writeBoundKind(bound.kind());
-    }
-
-    public Unfiltered takeAlias()
-    {
-        return this;
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/AbstractReusableRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractReusableRow.java b/src/java/org/apache/cassandra/db/rows/AbstractReusableRow.java
deleted file mode 100644
index 03aeb88..0000000
--- a/src/java/org/apache/cassandra/db/rows/AbstractReusableRow.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/*
- * 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.rows;
-
-import java.util.Iterator;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.utils.SearchIterator;
-
-public abstract class AbstractReusableRow extends AbstractRow
-{
-    private CellData.ReusableCell simpleCell;
-    private ComplexRowDataBlock.ReusableIterator complexCells;
-    private DeletionTimeArray.Cursor complexDeletionCursor;
-    private RowDataBlock.ReusableIterator iterator;
-
-    public AbstractReusableRow()
-    {
-    }
-
-    protected abstract int row();
-    protected abstract RowDataBlock data();
-
-    private CellData.ReusableCell simpleCell()
-    {
-        if (simpleCell == null)
-            simpleCell = SimpleRowDataBlock.reusableCell();
-        return simpleCell;
-    }
-
-    private ComplexRowDataBlock.ReusableIterator complexCells()
-    {
-        if (complexCells == null)
-            complexCells = ComplexRowDataBlock.reusableComplexCells();
-        return complexCells;
-    }
-
-    private DeletionTimeArray.Cursor complexDeletionCursor()
-    {
-        if (complexDeletionCursor == null)
-            complexDeletionCursor = ComplexRowDataBlock.complexDeletionCursor();
-        return complexDeletionCursor;
-    }
-
-    private RowDataBlock.ReusableIterator reusableIterator()
-    {
-        if (iterator == null)
-            iterator = RowDataBlock.reusableIterator();
-        return iterator;
-    }
-
-    public Columns columns()
-    {
-        return data().columns();
-    }
-
-    public Cell getCell(ColumnDefinition c)
-    {
-        assert !c.isComplex();
-        if (data().simpleData == null)
-            return null;
-
-        int idx = columns().simpleIdx(c, 0);
-        if (idx < 0)
-            return null;
-
-        return simpleCell().setTo(data().simpleData.data, c, (row() * columns().simpleColumnCount()) + idx);
-    }
-
-    public Cell getCell(ColumnDefinition c, CellPath path)
-    {
-        assert c.isComplex();
-
-        ComplexRowDataBlock data = data().complexData;
-        if (data == null)
-            return null;
-
-        int idx = data.cellIdx(row(), c, path);
-        if (idx < 0)
-            return null;
-
-        return simpleCell().setTo(data.cellData(row()), c, idx);
-    }
-
-    public Iterator<Cell> getCells(ColumnDefinition c)
-    {
-        assert c.isComplex();
-        return complexCells().setTo(data().complexData, row(), c);
-    }
-
-    public boolean hasComplexDeletion()
-    {
-        return data().hasComplexDeletion(row());
-    }
-
-    public DeletionTime getDeletion(ColumnDefinition c)
-    {
-        assert c.isComplex();
-        if (data().complexData == null)
-            return DeletionTime.LIVE;
-
-        int idx = data().complexData.complexDeletionIdx(row(), c);
-        return idx < 0
-             ? DeletionTime.LIVE
-             : complexDeletionCursor().setTo(data().complexData.complexDelTimes, idx);
-    }
-
-    public Iterator<Cell> iterator()
-    {
-        return reusableIterator().setTo(data(), row());
-    }
-
-    public SearchIterator<ColumnDefinition, ColumnData> searchIterator()
-    {
-        return new SearchIterator<ColumnDefinition, ColumnData>()
-        {
-            private int simpleIdx = 0;
-
-            public boolean hasNext()
-            {
-                // TODO: we can do better, but we expect users to no rely on this anyway
-                return true;
-            }
-
-            public ColumnData next(ColumnDefinition column)
-            {
-                if (column.isComplex())
-                {
-                    // TODO: this is sub-optimal
-
-                    Iterator<Cell> cells = getCells(column);
-                    return cells == null ? null : new ColumnData(column, null, cells, getDeletion(column));
-                }
-                else
-                {
-                    int idx = columns().simpleIdx(column, simpleIdx);
-                    if (idx < 0)
-                        return null;
-
-                    Cell cell = simpleCell().setTo(data().simpleData.data, column, (row() * columns().simpleColumnCount()) + idx);
-                    simpleIdx = idx + 1;
-                    return cell == null ? null : new ColumnData(column, cell, null, null);
-                }
-            }
-        };
-    }
-
-    public Row takeAlias()
-    {
-        final Clustering clustering = clustering().takeAlias();
-        final LivenessInfo info = primaryKeyLivenessInfo().takeAlias();
-        final DeletionTime deletion = deletion().takeAlias();
-
-        final RowDataBlock data = data();
-        final int row = row();
-
-        return new AbstractReusableRow()
-        {
-            protected RowDataBlock data()
-            {
-                return data;
-            }
-
-            protected int row()
-            {
-                return row;
-            }
-
-            public Clustering clustering()
-            {
-                return clustering;
-            }
-
-            public LivenessInfo primaryKeyLivenessInfo()
-            {
-                return info;
-            }
-
-            public DeletionTime deletion()
-            {
-                return deletion;
-            }
-
-            @Override
-            public Row takeAlias()
-            {
-                return this;
-            }
-        };
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/AbstractRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractRow.java b/src/java/org/apache/cassandra/db/rows/AbstractRow.java
index a99bc78..807d805 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractRow.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractRow.java
@@ -18,11 +18,11 @@ package org.apache.cassandra.db.rows;
 
 import java.nio.ByteBuffer;
 import java.security.MessageDigest;
-import java.util.Iterator;
 import java.util.Objects;
 
+import com.google.common.collect.Iterables;
+
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.serializers.MarshalException;
@@ -46,19 +46,14 @@ public abstract class AbstractRow implements Row
         if (primaryKeyLivenessInfo().isLive(nowInSec))
             return true;
 
-        for (Cell cell : this)
-            if (cell.isLive(nowInSec))
-                return true;
-
-        return false;
+        return Iterables.any(cells(), cell -> cell.isLive(nowInSec));
     }
 
     public boolean isEmpty()
     {
-        return !primaryKeyLivenessInfo().hasTimestamp()
+        return primaryKeyLivenessInfo().isEmpty()
             && deletion().isLive()
-            && !iterator().hasNext()
-            && !hasComplexDeletion();
+            && !iterator().hasNext();
     }
 
     public boolean isStatic()
@@ -74,36 +69,8 @@ public abstract class AbstractRow implements Row
         deletion().digest(digest);
         primaryKeyLivenessInfo().digest(digest);
 
-        Iterator<ColumnDefinition> iter = columns().complexColumns();
-        while (iter.hasNext())
-            getDeletion(iter.next()).digest(digest);
-
-        for (Cell cell : this)
-            cell.digest(digest);
-    }
-
-    /**
-     * Copy this row to the provided writer.
-     *
-     * @param writer the row writer to write this row to.
-     */
-    public void copyTo(Row.Writer writer)
-    {
-        Rows.writeClustering(clustering(), writer);
-        writer.writePartitionKeyLivenessInfo(primaryKeyLivenessInfo());
-        writer.writeRowDeletion(deletion());
-
-        for (Cell cell : this)
-            cell.writeTo(writer);
-
-        for (int i = 0; i < columns().complexColumnCount(); i++)
-        {
-            ColumnDefinition c = columns().getComplex(i);
-            DeletionTime dt = getDeletion(c);
-            if (!dt.isLive())
-                writer.writeComplexDeletion(c, dt);
-        }
-        writer.endOfRow();
+        for (ColumnData cd : this)
+            cd.digest(digest);
     }
 
     public void validateData(CFMetaData metadata)
@@ -120,8 +87,8 @@ public abstract class AbstractRow implements Row
         if (deletion().localDeletionTime() < 0)
             throw new MarshalException("A local deletion time should not be negative");
 
-        for (Cell cell : this)
-            cell.validate();
+        for (ColumnData cd : this)
+            cd.validate();
     }
 
     public String toString(CFMetaData metadata)
@@ -142,33 +109,43 @@ public abstract class AbstractRow implements Row
         }
         sb.append(": ").append(clustering().toString(metadata)).append(" | ");
         boolean isFirst = true;
-        ColumnDefinition prevColumn = null;
-        for (Cell cell : this)
+        for (ColumnData cd : this)
         {
             if (isFirst) isFirst = false; else sb.append(", ");
             if (fullDetails)
             {
-                if (cell.column().isComplex() && !cell.column().equals(prevColumn))
+                if (cd.column().isSimple())
                 {
-                    DeletionTime complexDel = getDeletion(cell.column());
-                    if (!complexDel.isLive())
-                        sb.append("del(").append(cell.column().name).append(")=").append(complexDel).append(", ");
+                    sb.append(cd);
+                }
+                else
+                {
+                    ComplexColumnData complexData = (ComplexColumnData)cd;
+                    if (!complexData.complexDeletion().isLive())
+                        sb.append("del(").append(cd.column().name).append(")=").append(complexData.complexDeletion());
+                    for (Cell cell : complexData)
+                        sb.append(", ").append(cell);
                 }
-                sb.append(cell);
-                prevColumn = cell.column();
             }
             else
             {
-                sb.append(cell.column().name);
-                if (cell.column().type instanceof CollectionType)
+                if (cd.column().isSimple())
                 {
-                    CollectionType ct = (CollectionType)cell.column().type;
-                    sb.append("[").append(ct.nameComparator().getString(cell.path().get(0))).append("]");
-                    sb.append("=").append(ct.valueComparator().getString(cell.value()));
+                    Cell cell = (Cell)cd;
+                    sb.append(cell.column().name).append('=').append(cell.column().type.getString(cell.value()));
                 }
                 else
                 {
-                    sb.append("=").append(cell.column().type.getString(cell.value()));
+                    ComplexColumnData complexData = (ComplexColumnData)cd;
+                    CollectionType ct = (CollectionType)cd.column().type;
+                    sb.append(cd.column().name).append("={");
+                    int i = 0;
+                    for (Cell cell : complexData)
+                    {
+                        sb.append(i++ == 0 ? "" : ", ");
+                        sb.append(ct.nameComparator().getString(cell.path().get(0))).append("->").append(ct.valueComparator().getString(cell.value()));
+                    }
+                    sb.append('}');
                 }
             }
         }
@@ -188,22 +165,15 @@ public abstract class AbstractRow implements Row
              || !this.deletion().equals(that.deletion()))
             return false;
 
-        Iterator<Cell> thisCells = this.iterator();
-        Iterator<Cell> thatCells = that.iterator();
-        while (thisCells.hasNext())
-        {
-            if (!thatCells.hasNext() || !thisCells.next().equals(thatCells.next()))
-                return false;
-        }
-        return !thatCells.hasNext();
+        return Iterables.elementsEqual(this, that);
     }
 
     @Override
     public int hashCode()
     {
         int hash = Objects.hash(clustering(), columns(), primaryKeyLivenessInfo(), deletion());
-        for (Cell cell : this)
-            hash += 31 * cell.hashCode();
+        for (ColumnData cd : this)
+            hash += 31 * cd.hashCode();
         return hash;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/AbstractUnfilteredRowIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractUnfilteredRowIterator.java b/src/java/org/apache/cassandra/db/rows/AbstractUnfilteredRowIterator.java
index 5bfd1a3..b4f849a 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractUnfilteredRowIterator.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractUnfilteredRowIterator.java
@@ -90,18 +90,4 @@ public abstract class AbstractUnfilteredRowIterator extends AbstractIterator<Unf
     public void close()
     {
     }
-
-    public static boolean equal(UnfilteredRowIterator a, UnfilteredRowIterator b)
-    {
-        return Objects.equals(a.columns(), b.columns())
-            && Objects.equals(a.metadata(), b.metadata())
-            && Objects.equals(a.isReverseOrder(), b.isReverseOrder())
-            && Objects.equals(a.partitionKey(), b.partitionKey())
-            && Objects.equals(a.partitionLevelDeletion(), b.partitionLevelDeletion())
-            && Objects.equals(a.staticRow(), b.staticRow())
-            && Objects.equals(a.stats(), b.stats())
-            && Objects.equals(a.metadata(), b.metadata())
-            && Iterators.elementsEqual(a, b);
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/AlteringUnfilteredRowIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/AlteringUnfilteredRowIterator.java b/src/java/org/apache/cassandra/db/rows/AlteringUnfilteredRowIterator.java
new file mode 100644
index 0000000..a390bad
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/rows/AlteringUnfilteredRowIterator.java
@@ -0,0 +1,98 @@
+/*
+ * 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.rows;
+
+import java.util.NoSuchElementException;
+
+import com.google.common.collect.UnmodifiableIterator;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.*;
+
+/**
+ * Class that makes it easier to write unfiltered iterators that filter or modify
+ * the returned unfiltered.
+ *
+ * The methods you want to override are {@code computeNextStatic} and the {@code computeNext} methods.
+ * All of these methods are allowed to return a {@code null} value with the meaning of ignoring
+ * the entry.
+ */
+public abstract class AlteringUnfilteredRowIterator extends WrappingUnfilteredRowIterator
+{
+    private Row staticRow;
+    private Unfiltered next;
+
+    protected AlteringUnfilteredRowIterator(UnfilteredRowIterator wrapped)
+    {
+        super(wrapped);
+    }
+
+    protected Row computeNextStatic(Row row)
+    {
+        return row;
+    }
+
+    protected Row computeNext(Row row)
+    {
+        return row;
+    }
+
+    protected RangeTombstoneMarker computeNext(RangeTombstoneMarker marker)
+    {
+        return marker;
+    }
+
+    public Row staticRow()
+    {
+        if (staticRow == null)
+        {
+            Row row = computeNextStatic(super.staticRow());
+            staticRow = row == null ? Rows.EMPTY_STATIC_ROW : row;
+        }
+        return staticRow;
+    }
+
+    public boolean hasNext()
+    {
+        while (next == null && super.hasNext())
+        {
+            Unfiltered unfiltered = super.next();
+            if (unfiltered.isRow())
+            {
+                Row row = computeNext((Row)unfiltered);
+                if (row != null && !row.isEmpty())
+                    next = row;
+            }
+            else
+            {
+                next = computeNext((RangeTombstoneMarker)unfiltered);
+            }
+        }
+        return next != null;
+    }
+
+    public Unfiltered next()
+    {
+        if (!hasNext())
+            throw new NoSuchElementException();
+
+        Unfiltered toReturn = next;
+        next = null;
+        return toReturn;
+    }
+}


[10/15] cassandra git commit: Simplify some 8099's implementations

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/partitions/AbstractThreadUnsafePartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/AbstractThreadUnsafePartition.java b/src/java/org/apache/cassandra/db/partitions/AbstractThreadUnsafePartition.java
new file mode 100644
index 0000000..d79ab06
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/partitions/AbstractThreadUnsafePartition.java
@@ -0,0 +1,393 @@
+/*
+ * 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.partitions;
+
+import java.util.*;
+
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.utils.SearchIterator;
+
+/**
+ * Abstract common class for all non-thread safe Partition implementations.
+ */
+public abstract class AbstractThreadUnsafePartition implements Partition, Iterable<Row>
+{
+    protected final CFMetaData metadata;
+    protected final DecoratedKey key;
+
+    protected final PartitionColumns columns;
+
+    protected final List<Row> rows;
+
+    protected AbstractThreadUnsafePartition(CFMetaData metadata,
+                                            DecoratedKey key,
+                                            PartitionColumns columns,
+                                            List<Row> rows)
+    {
+        this.metadata = metadata;
+        this.key = key;
+        this.columns = columns;
+        this.rows = rows;
+    }
+
+    public CFMetaData metadata()
+    {
+        return metadata;
+    }
+
+    public DecoratedKey partitionKey()
+    {
+        return key;
+    }
+
+    public DeletionTime partitionLevelDeletion()
+    {
+        return deletionInfo().getPartitionDeletion();
+    }
+
+    public PartitionColumns columns()
+    {
+        return columns;
+    }
+
+    public abstract Row staticRow();
+
+    protected abstract boolean canHaveShadowedData();
+
+    /**
+     * The deletion info for the partition update.
+     *
+     * Note: do not cast the result to a {@code MutableDeletionInfo} to modify it!
+     *
+     * @return the deletion info for the partition update for use as read-only.
+     */
+    public abstract DeletionInfo deletionInfo();
+
+    public int rowCount()
+    {
+        return rows.size();
+    }
+
+    public boolean isEmpty()
+    {
+        return deletionInfo().isLive() && rows.isEmpty() && staticRow().isEmpty();
+    }
+
+    @Override
+    public String toString()
+    {
+        StringBuilder sb = new StringBuilder();
+        CFMetaData metadata = metadata();
+        sb.append(String.format("Partition[%s.%s] key=%s columns=%s%s",
+                                 metadata().ksName,
+                                 metadata().cfName,
+                                 metadata().getKeyValidator().getString(partitionKey().getKey()),
+                                 columns(),
+                                 deletionInfo().isLive() ? "" : " " + deletionInfo()));
+
+        if (staticRow() != Rows.EMPTY_STATIC_ROW)
+            sb.append("\n    ").append(staticRow().toString(metadata, true));
+
+        // We use createRowIterator() directly instead of iterator() because that avoids
+        // sorting for PartitionUpdate (which inherit this method) and that is useful because
+        //  1) it can help with debugging and 2) we can't write after sorting but we want to
+        // be able to print an update while we build it (again for debugging)
+        for (Row row : this)
+            sb.append("\n    ").append(row.toString(metadata, true));
+
+        return sb.toString();
+    }
+
+    public Row getRow(Clustering clustering)
+    {
+        Row row = searchIterator(ColumnFilter.selection(columns()), false).next(clustering);
+        // Note that for statics, this will never return null, this will return an empty row. However,
+        // it's more consistent for this method to return null if we don't really have a static row.
+        return row == null || (clustering == Clustering.STATIC_CLUSTERING && row.isEmpty()) ? null : row;
+    }
+
+    /**
+     * Returns an iterator that iterators over the rows of this update in clustering order.
+     *
+     * @return an iterator over the rows of this partition.
+     */
+    public Iterator<Row> iterator()
+    {
+        return rows.iterator();
+    }
+
+    public SearchIterator<Clustering, Row> searchIterator(final ColumnFilter columns, boolean reversed)
+    {
+        final RowSearcher searcher = reversed ? new ReverseRowSearcher() : new ForwardRowSearcher();
+        return new SearchIterator<Clustering, Row>()
+        {
+            public boolean hasNext()
+            {
+                return !searcher.isDone();
+            }
+
+            public Row next(Clustering clustering)
+            {
+                if (clustering == Clustering.STATIC_CLUSTERING)
+                {
+                    Row staticRow = staticRow();
+                    return staticRow.isEmpty() || columns.fetchedColumns().statics.isEmpty()
+                         ? Rows.EMPTY_STATIC_ROW
+                         : staticRow.filter(columns, partitionLevelDeletion(), true, metadata);
+                }
+
+                Row row = searcher.search(clustering);
+                RangeTombstone rt = deletionInfo().rangeCovering(clustering);
+
+                // A search iterator only return a row, so it doesn't allow to directly account for deletion that should apply to to row
+                // (the partition deletion or the deletion of a range tombstone that covers it). So if needs be, reuse the row deletion
+                // to carry the proper deletion on the row.
+                DeletionTime activeDeletion = partitionLevelDeletion();
+                if (rt != null && rt.deletionTime().supersedes(activeDeletion))
+                    activeDeletion = rt.deletionTime();
+
+                if (row == null)
+                    return activeDeletion.isLive() ? null : ArrayBackedRow.emptyDeletedRow(clustering, activeDeletion);
+
+                return row.filter(columns, activeDeletion, true, metadata);
+            }
+        };
+    }
+
+    public UnfilteredRowIterator unfilteredIterator()
+    {
+        return unfilteredIterator(ColumnFilter.all(metadata()), Slices.ALL, false);
+    }
+
+    public UnfilteredRowIterator unfilteredIterator(ColumnFilter columns, Slices slices, boolean reversed)
+    {
+        return slices.makeSliceIterator(sliceableUnfilteredIterator(columns, reversed));
+    }
+
+    protected SliceableUnfilteredRowIterator sliceableUnfilteredIterator()
+    {
+        return sliceableUnfilteredIterator(ColumnFilter.all(metadata()), false);
+    }
+
+    protected SliceableUnfilteredRowIterator sliceableUnfilteredIterator(ColumnFilter selection, boolean reversed)
+    {
+        return new SliceableIterator(this, selection, reversed);
+    }
+
+    /**
+     * Simple binary search for a given row (in the rows list).
+     *
+     * The return value has the exact same meaning that the one of Collections.binarySearch() but
+     * we don't use the later because we're searching for a 'Clustering' in an array of 'Row' (and while
+     * both are Clusterable, it's slightly faster to use the 'Clustering' comparison (see comment on
+     * ClusteringComparator.rowComparator())).
+     */
+    private int binarySearch(Clustering clustering, int fromIndex, int toIndex)
+    {
+        ClusteringComparator comparator = metadata().comparator;
+        int low = fromIndex;
+        int mid = toIndex;
+        int high = mid - 1;
+        int result = -1;
+        while (low <= high)
+        {
+            mid = (low + high) >> 1;
+            if ((result = comparator.compare(clustering, rows.get(mid).clustering())) > 0)
+                low = mid + 1;
+            else if (result == 0)
+                return mid;
+            else
+                high = mid - 1;
+        }
+        return -mid - (result < 0 ? 1 : 2);
+    }
+
+    private class SliceableIterator extends AbstractUnfilteredRowIterator implements SliceableUnfilteredRowIterator
+    {
+        private final ColumnFilter columns;
+        private RowSearcher searcher;
+
+        private Iterator<Unfiltered> iterator;
+
+        private SliceableIterator(AbstractThreadUnsafePartition partition, ColumnFilter columns, boolean isReverseOrder)
+        {
+            super(partition.metadata(),
+                  partition.partitionKey(),
+                  partition.partitionLevelDeletion(),
+                  columns.fetchedColumns(),
+                  partition.staticRow().isEmpty() ? Rows.EMPTY_STATIC_ROW : partition.staticRow().filter(columns, partition.partitionLevelDeletion(), false, partition.metadata()),
+                  isReverseOrder,
+                  partition.stats());
+            this.columns = columns;
+        }
+
+        protected Unfiltered computeNext()
+        {
+            if (iterator == null)
+                iterator = merge(isReverseOrder ? Lists.reverse(rows).iterator(): iterator(), deletionInfo().rangeIterator(isReverseOrder()));
+
+            return iterator.hasNext() ? iterator.next() : endOfData();
+        }
+
+        public Iterator<Unfiltered> slice(Slice slice)
+        {
+            if (searcher == null)
+                searcher = isReverseOrder() ? new ReverseRowSearcher() : new ForwardRowSearcher();
+            return merge(searcher.slice(slice), deletionInfo().rangeIterator(slice, isReverseOrder()));
+        }
+
+        private Iterator<Unfiltered> merge(Iterator<Row> rows, Iterator<RangeTombstone> ranges)
+        {
+            return new RowAndDeletionMergeIterator(metadata,
+                                                   partitionKey,
+                                                   partitionLevelDeletion,
+                                                   columns,
+                                                   staticRow(),
+                                                   isReverseOrder(),
+                                                   stats(),
+                                                   rows,
+                                                   ranges,
+                                                   canHaveShadowedData());
+        }
+    }
+
+    /**
+     * Utility class to search for rows or slice of rows in order.
+     */
+    private abstract class RowSearcher
+    {
+        public abstract boolean isDone();
+
+        public abstract Row search(Clustering name);
+
+        public abstract Iterator<Row> slice(Slice slice);
+
+        protected int search(Clustering clustering, int from, int to)
+        {
+            return binarySearch(clustering, from, to);
+        }
+
+        protected int search(Slice.Bound bound, int from, int to)
+        {
+            return Collections.binarySearch(rows.subList(from, to), bound, metadata.comparator);
+        }
+    }
+
+    private class ForwardRowSearcher extends RowSearcher
+    {
+        private int nextIdx = 0;
+
+        public boolean isDone()
+        {
+            return nextIdx >= rows.size();
+        }
+
+        public Row search(Clustering name)
+        {
+            if (isDone())
+                return null;
+
+            int idx = search(name, nextIdx, rows.size());
+            if (idx < 0)
+            {
+                nextIdx = -idx - 1;
+                return null;
+            }
+            else
+            {
+                nextIdx = idx + 1;
+                return rows.get(idx);
+            }
+        }
+
+        public Iterator<Row> slice(Slice slice)
+        {
+            // Note that because a Slice.Bound can never sort equally to a Clustering, we know none of the search will
+            // be a match, so we save from testing for it.
+
+            final int start = -search(slice.start(), nextIdx, rows.size()) - 1; // First index to include
+            if (start >= rows.size())
+                return Collections.emptyIterator();
+
+            final int end = -search(slice.end(), start, rows.size()) - 1; // First index to exclude
+
+            // Remember the end to speed up potential further slice search
+            nextIdx = end;
+
+            if (start >= end)
+                return Collections.emptyIterator();
+
+            return rows.subList(start, end).iterator();
+        }
+    }
+
+    private class ReverseRowSearcher extends RowSearcher
+    {
+        private int nextIdx = rows.size() - 1;
+
+        public boolean isDone()
+        {
+            return nextIdx < 0;
+        }
+
+        public Row search(Clustering name)
+        {
+            if (isDone())
+                return null;
+
+            int idx = search(name, 0, nextIdx);
+            if (idx < 0)
+            {
+                // The insertion point is the first element greater than name, so we want start from the previous one next time
+                nextIdx = -idx - 2;
+                return null;
+            }
+            else
+            {
+                nextIdx = idx - 1;
+                return rows.get(idx);
+            }
+        }
+
+        public Iterator<Row> slice(Slice slice)
+        {
+            // Note that because a Slice.Bound can never sort equally to a Clustering, we know none of the search will
+            // be a match, so we save from testing for it.
+
+            // The insertion point is the first element greater than slice.end(), so we want the previous index
+            final int start = -search(slice.end(), 0, nextIdx + 1) - 2;  // First index to include
+            if (start < 0)
+                return Collections.emptyIterator();
+
+            final int end = -search(slice.start(), 0, start + 1) - 2; // First index to exclude
+
+            // Remember the end to speed up potential further slice search
+            nextIdx = end;
+
+            if (start < end)
+                return Collections.emptyIterator();
+
+            return Lists.reverse(rows.subList(end+1, start+1)).iterator();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/partitions/AlteringUnfilteredPartitionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/AlteringUnfilteredPartitionIterator.java b/src/java/org/apache/cassandra/db/partitions/AlteringUnfilteredPartitionIterator.java
new file mode 100644
index 0000000..f7d7222
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/partitions/AlteringUnfilteredPartitionIterator.java
@@ -0,0 +1,72 @@
+/*
+ * 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.partitions;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.rows.*;
+
+/**
+ * A partition iterator that allows to filter/modify the unfiltered from the
+ * underlying iterators.
+ */
+public abstract class AlteringUnfilteredPartitionIterator extends WrappingUnfilteredPartitionIterator
+{
+    protected AlteringUnfilteredPartitionIterator(UnfilteredPartitionIterator wrapped)
+    {
+        super(wrapped);
+    }
+
+    protected Row computeNextStatic(DecoratedKey partitionKey, Row row)
+    {
+        return row;
+    }
+
+    protected Row computeNext(DecoratedKey partitionKey, Row row)
+    {
+        return row;
+    }
+
+    protected RangeTombstoneMarker computeNext(DecoratedKey partitionKey, RangeTombstoneMarker marker)
+    {
+        return marker;
+    }
+
+    @Override
+    protected UnfilteredRowIterator computeNext(UnfilteredRowIterator iter)
+    {
+        final DecoratedKey partitionKey = iter.partitionKey();
+        return new AlteringUnfilteredRowIterator(iter)
+        {
+            protected Row computeNextStatic(Row row)
+            {
+                return AlteringUnfilteredPartitionIterator.this.computeNextStatic(partitionKey, row);
+            }
+
+            protected Row computeNext(Row row)
+            {
+                return AlteringUnfilteredPartitionIterator.this.computeNext(partitionKey, row);
+            }
+
+            protected RangeTombstoneMarker computeNext(RangeTombstoneMarker marker)
+            {
+                return AlteringUnfilteredPartitionIterator.this.computeNext(partitionKey, marker);
+            }
+        };
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/partitions/ArrayBackedCachedPartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/ArrayBackedCachedPartition.java b/src/java/org/apache/cassandra/db/partitions/ArrayBackedCachedPartition.java
index bec8056..f39245b 100644
--- a/src/java/org/apache/cassandra/db/partitions/ArrayBackedCachedPartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/ArrayBackedCachedPartition.java
@@ -18,11 +18,11 @@
 package org.apache.cassandra.db.partitions;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
+import java.util.*;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.filter.DataLimits;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -33,24 +33,31 @@ public class ArrayBackedCachedPartition extends ArrayBackedPartition implements
 {
     private final int createdAtInSec;
 
-    // Note that those fields are really immutable, but we can't easily pass their values to
-    // the ctor so they are not final.
-    private int cachedLiveRows;
-    private int rowsWithNonExpiringCells;
+    private final int cachedLiveRows;
+    private final int rowsWithNonExpiringCells;
 
-    private int nonTombstoneCellCount;
-    private int nonExpiringLiveCells;
+    private final int nonTombstoneCellCount;
+    private final int nonExpiringLiveCells;
 
     private ArrayBackedCachedPartition(CFMetaData metadata,
                                        DecoratedKey partitionKey,
-                                       DeletionTime deletionTime,
                                        PartitionColumns columns,
-                                       int initialRowCapacity,
-                                       boolean sortable,
-                                       int createdAtInSec)
+                                       Row staticRow,
+                                       List<Row> rows,
+                                       DeletionInfo deletionInfo,
+                                       RowStats stats,
+                                       int createdAtInSec,
+                                       int cachedLiveRows,
+                                       int rowsWithNonExpiringCells,
+                                       int nonTombstoneCellCount,
+                                       int nonExpiringLiveCells)
     {
-        super(metadata, partitionKey, deletionTime, columns, initialRowCapacity, sortable);
+        super(metadata, partitionKey, columns, staticRow, rows, deletionInfo, stats);
         this.createdAtInSec = createdAtInSec;
+        this.cachedLiveRows = cachedLiveRows;
+        this.rowsWithNonExpiringCells = rowsWithNonExpiringCells;
+        this.nonTombstoneCellCount = nonTombstoneCellCount;
+        this.nonExpiringLiveCells = nonExpiringLiveCells;
     }
 
     /**
@@ -65,7 +72,7 @@ public class ArrayBackedCachedPartition extends ArrayBackedPartition implements
      */
     public static ArrayBackedCachedPartition create(UnfilteredRowIterator iterator, int nowInSec)
     {
-        return create(iterator, 4, nowInSec);
+        return create(iterator, 16, nowInSec);
     }
 
     /**
@@ -82,30 +89,76 @@ public class ArrayBackedCachedPartition extends ArrayBackedPartition implements
      */
     public static ArrayBackedCachedPartition create(UnfilteredRowIterator iterator, int initialRowCapacity, int nowInSec)
     {
-        ArrayBackedCachedPartition partition = new ArrayBackedCachedPartition(iterator.metadata(),
-                                                                              iterator.partitionKey(),
-                                                                              iterator.partitionLevelDeletion(),
-                                                                              iterator.columns(),
-                                                                              initialRowCapacity,
-                                                                              iterator.isReverseOrder(),
-                                                                              nowInSec);
+        CFMetaData metadata = iterator.metadata();
+        boolean reversed = iterator.isReverseOrder();
 
-        partition.staticRow = iterator.staticRow().takeAlias();
+        List<Row> rows = new ArrayList<>(initialRowCapacity);
+        MutableDeletionInfo.Builder deletionBuilder = MutableDeletionInfo.builder(iterator.partitionLevelDeletion(), metadata.comparator, reversed);
 
-        Writer writer = partition.new Writer(nowInSec);
-        RangeTombstoneCollector markerCollector = partition.new RangeTombstoneCollector(iterator.isReverseOrder());
+        int cachedLiveRows = 0;
+        int rowsWithNonExpiringCells = 0;
 
-        copyAll(iterator, writer, markerCollector, partition);
+        int nonTombstoneCellCount = 0;
+        int nonExpiringLiveCells = 0;
 
-        return partition;
+        while (iterator.hasNext())
+        {
+            Unfiltered unfiltered = iterator.next();
+            if (unfiltered.kind() == Unfiltered.Kind.ROW)
+            {
+                Row row = (Row)unfiltered;
+                rows.add(row);
+
+                // Collect stats
+                if (row.hasLiveData(nowInSec))
+                    ++cachedLiveRows;
+
+                boolean hasNonExpiringCell = false;
+                for (Cell cell : row.cells())
+                {
+                    if (!cell.isTombstone())
+                    {
+                        ++nonTombstoneCellCount;
+                        if (!cell.isExpiring())
+                        {
+                            hasNonExpiringCell = true;
+                            ++nonExpiringLiveCells;
+                        }
+                    }
+                }
+
+                if (hasNonExpiringCell)
+                    ++rowsWithNonExpiringCells;
+            }
+            else
+            {
+                deletionBuilder.add((RangeTombstoneMarker)unfiltered);
+            }
+        }
+
+        if (reversed)
+            Collections.reverse(rows);
+
+        return new ArrayBackedCachedPartition(metadata,
+                                              iterator.partitionKey(),
+                                              iterator.columns(),
+                                              iterator.staticRow(),
+                                              rows,
+                                              deletionBuilder.build(),
+                                              iterator.stats(),
+                                              nowInSec,
+                                              cachedLiveRows,
+                                              rowsWithNonExpiringCells,
+                                              nonTombstoneCellCount,
+                                              nonExpiringLiveCells);
     }
 
     public Row lastRow()
     {
-        if (rows == 0)
+        if (rows.isEmpty())
             return null;
 
-        return new InternalReusableRow().setTo(rows - 1);
+        return rows.get(rows.size() - 1);
     }
 
     /**
@@ -146,62 +199,6 @@ public class ArrayBackedCachedPartition extends ArrayBackedPartition implements
         return nonExpiringLiveCells;
     }
 
-    // Writers that collect the values for 'cachedLiveRows', 'rowsWithNonExpiringCells', 'nonTombstoneCellCount'
-    // and 'nonExpiringLiveCells'.
-    protected class Writer extends AbstractPartitionData.Writer
-    {
-        private final int nowInSec;
-
-        private boolean hasLiveData;
-        private boolean hasNonExpiringCell;
-
-        protected Writer(int nowInSec)
-        {
-            super(true);
-            this.nowInSec = nowInSec;
-        }
-
-        @Override
-        public void writePartitionKeyLivenessInfo(LivenessInfo info)
-        {
-            super.writePartitionKeyLivenessInfo(info);
-            if (info.isLive(nowInSec))
-                hasLiveData = true;
-        }
-
-        @Override
-        public void writeCell(ColumnDefinition column, boolean isCounter, ByteBuffer value, LivenessInfo info, CellPath path)
-        {
-            super.writeCell(column, isCounter, value, info, path);
-
-            if (info.isLive(nowInSec))
-            {
-                hasLiveData = true;
-                if (!info.hasTTL())
-                {
-                    hasNonExpiringCell = true;
-                    ++ArrayBackedCachedPartition.this.nonExpiringLiveCells;
-                }
-            }
-
-            if (!info.hasLocalDeletionTime() || info.hasTTL())
-                ++ArrayBackedCachedPartition.this.nonTombstoneCellCount;
-        }
-
-        @Override
-        public void endOfRow()
-        {
-            super.endOfRow();
-            if (hasLiveData)
-                ++ArrayBackedCachedPartition.this.cachedLiveRows;
-            if (hasNonExpiringCell)
-                ++ArrayBackedCachedPartition.this.rowsWithNonExpiringCells;
-
-            hasLiveData = false;
-            hasNonExpiringCell = false;
-        }
-    }
-
     static class Serializer implements ISerializer<CachedPartition>
     {
         public void serialize(CachedPartition partition, DataOutputPlus out) throws IOException
@@ -210,9 +207,13 @@ public class ArrayBackedCachedPartition extends ArrayBackedPartition implements
             ArrayBackedCachedPartition p = (ArrayBackedCachedPartition)partition;
 
             out.writeInt(p.createdAtInSec);
+            out.writeInt(p.cachedLiveRows);
+            out.writeInt(p.rowsWithNonExpiringCells);
+            out.writeInt(p.nonTombstoneCellCount);
+            out.writeInt(p.nonExpiringLiveCells);
             try (UnfilteredRowIterator iter = p.sliceableUnfilteredIterator())
             {
-                UnfilteredRowIteratorSerializer.serializer.serialize(iter, out, MessagingService.current_version, p.rows);
+                UnfilteredRowIteratorSerializer.serializer.serialize(iter, out, MessagingService.current_version, p.rowCount());
             }
         }
 
@@ -226,18 +227,42 @@ public class ArrayBackedCachedPartition extends ArrayBackedPartition implements
             //      is slightly faster.
 
             int createdAtInSec = in.readInt();
+            int cachedLiveRows = in.readInt();
+            int rowsWithNonExpiringCells = in.readInt();
+            int nonTombstoneCellCount = in.readInt();
+            int nonExpiringLiveCells = in.readInt();
+
+            UnfilteredRowIteratorSerializer.Header header = UnfilteredRowIteratorSerializer.serializer.deserializeHeader(in, MessagingService.current_version, SerializationHelper.Flag.LOCAL);
+            assert !header.isReversed && header.rowEstimate >= 0;
 
-            UnfilteredRowIteratorSerializer.Header h = UnfilteredRowIteratorSerializer.serializer.deserializeHeader(in, MessagingService.current_version, SerializationHelper.Flag.LOCAL);
-            assert !h.isReversed && h.rowEstimate >= 0;
+            MutableDeletionInfo.Builder deletionBuilder = MutableDeletionInfo.builder(header.partitionDeletion, header.metadata.comparator, false);
+            List<Row> rows = new ArrayList<>(header.rowEstimate);
 
-            ArrayBackedCachedPartition partition = new ArrayBackedCachedPartition(h.metadata, h.key, h.partitionDeletion, h.sHeader.columns(), h.rowEstimate, false, createdAtInSec);
-            partition.staticRow = h.staticRow;
+            try (UnfilteredRowIterator partition = UnfilteredRowIteratorSerializer.serializer.deserialize(in, MessagingService.current_version, SerializationHelper.Flag.LOCAL, header))
+            {
+                while (partition.hasNext())
+                {
+                    Unfiltered unfiltered = partition.next();
+                    if (unfiltered.kind() == Unfiltered.Kind.ROW)
+                        rows.add((Row)unfiltered);
+                    else
+                        deletionBuilder.add((RangeTombstoneMarker)unfiltered);
+                }
+            }
 
-            Writer writer = partition.new Writer(createdAtInSec);
-            RangeTombstoneMarker.Writer markerWriter = partition.new RangeTombstoneCollector(false);
+            return new ArrayBackedCachedPartition(header.metadata,
+                                                  header.key,
+                                                  header.sHeader.columns(),
+                                                  header.staticRow,
+                                                  rows,
+                                                  deletionBuilder.build(),
+                                                  header.sHeader.stats(),
+                                                  createdAtInSec,
+                                                  cachedLiveRows,
+                                                  rowsWithNonExpiringCells,
+                                                  nonTombstoneCellCount,
+                                                  nonExpiringLiveCells);
 
-            UnfilteredRowIteratorSerializer.serializer.deserialize(in, new SerializationHelper(MessagingService.current_version, SerializationHelper.Flag.LOCAL), h.sHeader, writer, markerWriter);
-            return partition;
         }
 
         public long serializedSize(CachedPartition partition)
@@ -248,7 +273,11 @@ public class ArrayBackedCachedPartition extends ArrayBackedPartition implements
             try (UnfilteredRowIterator iter = p.sliceableUnfilteredIterator())
             {
                 return TypeSizes.sizeof(p.createdAtInSec)
-                     + UnfilteredRowIteratorSerializer.serializer.serializedSize(iter, MessagingService.current_version, p.rows);
+                     + TypeSizes.sizeof(p.cachedLiveRows)
+                     + TypeSizes.sizeof(p.rowsWithNonExpiringCells)
+                     + TypeSizes.sizeof(p.nonTombstoneCellCount)
+                     + TypeSizes.sizeof(p.nonExpiringLiveCells)
+                     + UnfilteredRowIteratorSerializer.serializer.serializedSize(iter, MessagingService.current_version, p.rowCount());
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/partitions/ArrayBackedPartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/ArrayBackedPartition.java b/src/java/org/apache/cassandra/db/partitions/ArrayBackedPartition.java
index d7f3a88..4485117 100644
--- a/src/java/org/apache/cassandra/db/partitions/ArrayBackedPartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/ArrayBackedPartition.java
@@ -17,28 +17,30 @@
  */
 package org.apache.cassandra.db.partitions;
 
-import java.io.DataInput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
+import java.util.*;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
-import org.apache.cassandra.io.ISerializer;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessagingService;
 
-public class ArrayBackedPartition extends AbstractPartitionData
+public class ArrayBackedPartition extends AbstractThreadUnsafePartition
 {
+    private final Row staticRow;
+    private final DeletionInfo deletionInfo;
+    private final RowStats stats;
+
     protected ArrayBackedPartition(CFMetaData metadata,
                                    DecoratedKey partitionKey,
-                                   DeletionTime deletionTime,
                                    PartitionColumns columns,
-                                   int initialRowCapacity,
-                                   boolean sortable)
+                                   Row staticRow,
+                                   List<Row> rows,
+                                   DeletionInfo deletionInfo,
+                                   RowStats stats)
     {
-        super(metadata, partitionKey, deletionTime, columns, initialRowCapacity, sortable);
+        super(metadata, partitionKey, columns, rows);
+        this.staticRow = staticRow;
+        this.deletionInfo = deletionInfo;
+        this.stats = stats;
     }
 
     /**
@@ -52,7 +54,7 @@ public class ArrayBackedPartition extends AbstractPartitionData
      */
     public static ArrayBackedPartition create(UnfilteredRowIterator iterator)
     {
-        return create(iterator, 4);
+        return create(iterator, 16);
     }
 
     /**
@@ -68,37 +70,45 @@ public class ArrayBackedPartition extends AbstractPartitionData
      */
     public static ArrayBackedPartition create(UnfilteredRowIterator iterator, int initialRowCapacity)
     {
-        ArrayBackedPartition partition = new ArrayBackedPartition(iterator.metadata(),
-                                                                  iterator.partitionKey(),
-                                                                  iterator.partitionLevelDeletion(),
-                                                                  iterator.columns(),
-                                                                  initialRowCapacity,
-                                                                  iterator.isReverseOrder());
-
-        partition.staticRow = iterator.staticRow().takeAlias();
-
-        Writer writer = partition.new Writer(true);
-        RangeTombstoneCollector markerCollector = partition.new RangeTombstoneCollector(iterator.isReverseOrder());
+        CFMetaData metadata = iterator.metadata();
+        boolean reversed = iterator.isReverseOrder();
 
-        copyAll(iterator, writer, markerCollector, partition);
-
-        return partition;
-    }
+        List<Row> rows = new ArrayList<>(initialRowCapacity);
+        MutableDeletionInfo.Builder deletionBuilder = MutableDeletionInfo.builder(iterator.partitionLevelDeletion(), metadata.comparator, reversed);
 
-    protected static void copyAll(UnfilteredRowIterator iterator, Writer writer, RangeTombstoneCollector markerCollector, ArrayBackedPartition partition)
-    {
         while (iterator.hasNext())
         {
             Unfiltered unfiltered = iterator.next();
             if (unfiltered.kind() == Unfiltered.Kind.ROW)
-                ((Row) unfiltered).copyTo(writer);
+                rows.add((Row)unfiltered);
             else
-                ((RangeTombstoneMarker) unfiltered).copyTo(markerCollector);
+                deletionBuilder.add((RangeTombstoneMarker)unfiltered);
         }
 
-        // A Partition (or more precisely AbstractPartitionData) always assumes that its data is in clustering
-        // order. So if we've just added them in reverse clustering order, reverse them.
-        if (iterator.isReverseOrder())
-            partition.reverse();
+        if (reversed)
+            Collections.reverse(rows);
+
+        return new ArrayBackedPartition(metadata, iterator.partitionKey(), iterator.columns(), iterator.staticRow(), rows, deletionBuilder.build(), iterator.stats());
+    }
+
+    protected boolean canHaveShadowedData()
+    {
+        // We only create instances from UnfilteredRowIterator that don't have shadowed data
+        return false;
+    }
+
+    public Row staticRow()
+    {
+        return staticRow;
+    }
+
+    public DeletionInfo deletionInfo()
+    {
+        return deletionInfo;
+    }
+
+    public RowStats stats()
+    {
+        return stats;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
index 6a888a6..1361422 100644
--- a/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
@@ -26,11 +26,8 @@ import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import com.google.common.collect.AbstractIterator;
-import com.google.common.collect.Iterators;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
@@ -89,10 +86,7 @@ public class AtomicBTreePartition implements Partition
 
     private static final AtomicIntegerFieldUpdater<AtomicBTreePartition> wasteTrackerUpdater = AtomicIntegerFieldUpdater.newUpdater(AtomicBTreePartition.class, "wasteTracker");
 
-    private static final DeletionInfo LIVE = DeletionInfo.live();
-    // This is a small optimization: DeletionInfo is mutable, but we know that we will always copy it in that class,
-    // so we can safely alias one DeletionInfo.live() reference and avoid some allocations.
-    private static final Holder EMPTY = new Holder(BTree.empty(), LIVE, null, RowStats.NO_STATS);
+    private static final Holder EMPTY = new Holder(BTree.empty(), DeletionInfo.LIVE, Rows.EMPTY_STATIC_ROW, RowStats.NO_STATS);
 
     private final CFMetaData metadata;
     private final DecoratedKey partitionKey;
@@ -154,146 +148,56 @@ public class AtomicBTreePartition implements Partition
         return row == null || (clustering == Clustering.STATIC_CLUSTERING && row.isEmpty()) ? null : row;
     }
 
+    private Row staticRow(Holder current, ColumnFilter columns, boolean setActiveDeletionToRow)
+    {
+        DeletionTime partitionDeletion = current.deletionInfo.getPartitionDeletion();
+        if (columns.fetchedColumns().statics.isEmpty() || (current.staticRow.isEmpty() && partitionDeletion.isLive()))
+            return Rows.EMPTY_STATIC_ROW;
+
+        Row row = current.staticRow.filter(columns, partitionDeletion, setActiveDeletionToRow, metadata);
+        return row == null ? Rows.EMPTY_STATIC_ROW : row;
+    }
+
     public SearchIterator<Clustering, Row> searchIterator(final ColumnFilter columns, final boolean reversed)
     {
         // TODO: we could optimize comparison for "NativeRow" à la #6755
         final Holder current = ref;
         return new SearchIterator<Clustering, Row>()
         {
-            private final SearchIterator<Clustering, MemtableRowData> rawIter = new BTreeSearchIterator<>(current.tree, metadata.comparator, !reversed);
-            private final MemtableRowData.ReusableRow row = allocator.newReusableRow();
-            private final ReusableFilteringRow filter = new ReusableFilteringRow(columns.fetchedColumns().regulars, columns);
-            private final long partitionDeletion = current.deletionInfo.getPartitionDeletion().markedForDeleteAt();
+            private final SearchIterator<Clustering, Row> rawIter = new BTreeSearchIterator<>(current.tree, metadata.comparator, !reversed);
+            private final DeletionTime partitionDeletion = current.deletionInfo.getPartitionDeletion();
 
             public boolean hasNext()
             {
                 return rawIter.hasNext();
             }
 
-            public Row next(Clustering key)
+            public Row next(Clustering clustering)
             {
-                if (key == Clustering.STATIC_CLUSTERING)
-                    return makeStatic(columns, current, allocator);
+                if (clustering == Clustering.STATIC_CLUSTERING)
+                    return staticRow(current, columns, true);
 
-                MemtableRowData data = rawIter.next(key);
-                // We also need to find if there is a range tombstone covering this key
-                RangeTombstone rt = current.deletionInfo.rangeCovering(key);
+                Row row = rawIter.next(clustering);
+                RangeTombstone rt = current.deletionInfo.rangeCovering(clustering);
 
-                if (data == null)
-                {
-                    // If we have a range tombstone but not data, "fake" the RT by return a row deletion
-                    // corresponding to the tombstone.
-                    if (rt != null && rt.deletionTime().markedForDeleteAt() > partitionDeletion)
-                        return filter.setRowDeletion(rt.deletionTime()).setTo(emptyDeletedRow(key, rt.deletionTime()));
-                    return null;
-                }
+                // A search iterator only return a row, so it doesn't allow to directly account for deletion that should apply to to row
+                // (the partition deletion or the deletion of a range tombstone that covers it). So if needs be, reuse the row deletion
+                // to carry the proper deletion on the row.
+                DeletionTime activeDeletion = partitionDeletion;
+                if (rt != null && rt.deletionTime().supersedes(activeDeletion))
+                    activeDeletion = rt.deletionTime();
 
-                row.setTo(data);
+                if (row == null)
+                    return activeDeletion.isLive() ? null : ArrayBackedRow.emptyDeletedRow(clustering, activeDeletion);
 
-                filter.setRowDeletion(null);
-                if (rt == null || rt.deletionTime().markedForDeleteAt() < partitionDeletion)
-                {
-                    filter.setDeletionTimestamp(partitionDeletion);
-                }
-                else
-                {
-                    filter.setDeletionTimestamp(rt.deletionTime().markedForDeleteAt());
-                    // If we have a range tombstone covering that row and it's bigger than the row deletion itself, then
-                    // we replace the row deletion by the tombstone deletion as a way to return the tombstone.
-                    if (rt.deletionTime().supersedes(row.deletion()))
-                        filter.setRowDeletion(rt.deletionTime());
-                }
-
-                return filter.setTo(row);
-            }
-        };
-    }
-
-    private static Row emptyDeletedRow(Clustering clustering, DeletionTime deletion)
-    {
-        return new AbstractRow()
-        {
-            public Columns columns()
-            {
-                return Columns.NONE;
-            }
-
-            public LivenessInfo primaryKeyLivenessInfo()
-            {
-                return LivenessInfo.NONE;
-            }
-
-            public DeletionTime deletion()
-            {
-                return deletion;
-            }
-
-            public boolean isEmpty()
-            {
-                return true;
-            }
-
-            public boolean hasComplexDeletion()
-            {
-                return false;
-            }
-
-            public Clustering clustering()
-            {
-                return clustering;
-            }
-
-            public Cell getCell(ColumnDefinition c)
-            {
-                return null;
-            }
-
-            public Cell getCell(ColumnDefinition c, CellPath path)
-            {
-                return null;
-            }
-
-            public Iterator<Cell> getCells(ColumnDefinition c)
-            {
-                return null;
-            }
-
-            public DeletionTime getDeletion(ColumnDefinition c)
-            {
-                return DeletionTime.LIVE;
-            }
-
-            public Iterator<Cell> iterator()
-            {
-                return Iterators.<Cell>emptyIterator();
-            }
-
-            public SearchIterator<ColumnDefinition, ColumnData> searchIterator()
-            {
-                return new SearchIterator<ColumnDefinition, ColumnData>()
-                {
-                    public boolean hasNext()
-                    {
-                        return false;
-                    }
-
-                    public ColumnData next(ColumnDefinition column)
-                    {
-                        return null;
-                    }
-                };
-            }
-
-            public Row takeAlias()
-            {
-                return this;
+                return row.filter(columns, activeDeletion, true, metadata);
             }
         };
     }
 
     public UnfilteredRowIterator unfilteredIterator()
     {
-        return unfilteredIterator(ColumnFilter.selection(columns()), Slices.ALL, false);
+        return unfilteredIterator(ColumnFilter.all(metadata()), Slices.ALL, false);
     }
 
     public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, Slices slices, boolean reversed)
@@ -309,7 +213,7 @@ public class AtomicBTreePartition implements Partition
                                                      partitionKey,
                                                      partitionDeletion,
                                                      selection.fetchedColumns(),
-                                                     makeStatic(selection, current, allocator),
+                                                     staticRow(current, selection, false),
                                                      reversed,
                                                      current.stats)
             {
@@ -320,189 +224,51 @@ public class AtomicBTreePartition implements Partition
             };
         }
 
+        Holder current = ref;
+        Row staticRow = staticRow(current, selection, false);
         return slices.size() == 1
-             ? new SingleSliceIterator(metadata, partitionKey, ref, selection, slices.get(0), reversed, allocator)
-             : new SlicesIterator(metadata, partitionKey, ref, selection, slices, reversed, allocator);
+             ? sliceIterator(selection, slices.get(0), reversed, current, staticRow)
+             : new SlicesIterator(metadata, partitionKey, selection, slices, reversed, current, staticRow);
     }
 
-    private static Row makeStatic(ColumnFilter selection, Holder holder, MemtableAllocator allocator)
+    private UnfilteredRowIterator sliceIterator(ColumnFilter selection, Slice slice, boolean reversed, Holder current, Row staticRow)
     {
-        Columns statics = selection.fetchedColumns().statics;
-        if (statics.isEmpty() || holder.staticRow == null)
-            return Rows.EMPTY_STATIC_ROW;
-
-        return new ReusableFilteringRow(statics, selection)
-               .setDeletionTimestamp(holder.deletionInfo.getPartitionDeletion().markedForDeleteAt())
-               .setTo(allocator.newReusableRow().setTo(holder.staticRow));
+        Slice.Bound start = slice.start() == Slice.Bound.BOTTOM ? null : slice.start();
+        Slice.Bound end = slice.end() == Slice.Bound.TOP ? null : slice.end();
+        Iterator<Row> rowIter = BTree.slice(current.tree, metadata.comparator, start, true, end, true, !reversed);
+
+        return new RowAndDeletionMergeIterator(metadata,
+                                               partitionKey,
+                                               current.deletionInfo.getPartitionDeletion(),
+                                               selection,
+                                               staticRow,
+                                               reversed,
+                                               current.stats,
+                                               rowIter,
+                                               current.deletionInfo.rangeIterator(slice, reversed),
+                                               true);
     }
 
-    private static class ReusableFilteringRow extends FilteringRow
+    public class SlicesIterator extends AbstractUnfilteredRowIterator
     {
-        private final Columns columns;
-        private final ColumnFilter selection;
-        private ColumnFilter.Tester tester;
-        private long deletionTimestamp;
-
-        // Used by searchIterator in case the row is covered by a tombstone.
-        private DeletionTime rowDeletion;
-
-        public ReusableFilteringRow(Columns columns, ColumnFilter selection)
-        {
-            this.columns = columns;
-            this.selection = selection;
-        }
-
-        public ReusableFilteringRow setDeletionTimestamp(long timestamp)
-        {
-            this.deletionTimestamp = timestamp;
-            return this;
-        }
-
-        public ReusableFilteringRow setRowDeletion(DeletionTime rowDeletion)
-        {
-            this.rowDeletion = rowDeletion;
-            return this;
-        }
-
-        @Override
-        public DeletionTime deletion()
-        {
-            return rowDeletion == null ? super.deletion() : rowDeletion;
-        }
-
-        @Override
-        protected boolean include(LivenessInfo info)
-        {
-            return info.timestamp() > deletionTimestamp;
-        }
-
-        @Override
-        protected boolean include(ColumnDefinition def)
-        {
-            return columns.contains(def);
-        }
-
-        @Override
-        protected boolean include(DeletionTime dt)
-        {
-            return dt.markedForDeleteAt() > deletionTimestamp;
-        }
-
-        @Override
-        protected boolean include(ColumnDefinition c, DeletionTime dt)
-        {
-            return dt.markedForDeleteAt() > deletionTimestamp;
-        }
-
-        @Override
-        protected boolean include(Cell cell)
-        {
-            return selection.includes(cell);
-        }
-    }
-
-    private static class SingleSliceIterator extends AbstractUnfilteredRowIterator
-    {
-        private final Iterator<Unfiltered> iterator;
-        private final ReusableFilteringRow row;
-
-        private SingleSliceIterator(CFMetaData metadata,
-                                    DecoratedKey key,
-                                    Holder holder,
-                                    ColumnFilter selection,
-                                    Slice slice,
-                                    boolean isReversed,
-                                    MemtableAllocator allocator)
-        {
-            super(metadata,
-                  key,
-                  holder.deletionInfo.getPartitionDeletion(),
-                  selection.fetchedColumns(),
-                  makeStatic(selection, holder, allocator),
-                  isReversed,
-                  holder.stats);
-
-            Iterator<Row> rowIter = rowIter(metadata,
-                                            holder,
-                                            slice,
-                                            !isReversed,
-                                            allocator);
-
-            this.iterator = new RowAndTombstoneMergeIterator(metadata.comparator, isReversed)
-                            .setTo(rowIter, holder.deletionInfo.rangeIterator(slice, isReversed));
-
-            this.row = new ReusableFilteringRow(selection.fetchedColumns().regulars, selection)
-                       .setDeletionTimestamp(partitionLevelDeletion.markedForDeleteAt());
-        }
-
-        private Iterator<Row> rowIter(CFMetaData metadata,
-                                      Holder holder,
-                                      Slice slice,
-                                      boolean forwards,
-                                      final MemtableAllocator allocator)
-        {
-            Slice.Bound start = slice.start() == Slice.Bound.BOTTOM ? null : slice.start();
-            Slice.Bound end = slice.end() == Slice.Bound.TOP ? null : slice.end();
-            final Iterator<MemtableRowData> dataIter = BTree.slice(holder.tree, metadata.comparator, start, true, end, true, forwards);
-            return new AbstractIterator<Row>()
-            {
-                private final MemtableRowData.ReusableRow row = allocator.newReusableRow();
-
-                protected Row computeNext()
-                {
-                    return dataIter.hasNext() ? row.setTo(dataIter.next()) : endOfData();
-                }
-            };
-        }
-
-        protected Unfiltered computeNext()
-        {
-            while (iterator.hasNext())
-            {
-                Unfiltered next = iterator.next();
-                if (next.kind() == Unfiltered.Kind.ROW)
-                {
-                    row.setTo((Row)next);
-                    if (!row.isEmpty())
-                        return row;
-                }
-                else
-                {
-                    RangeTombstoneMarker marker = (RangeTombstoneMarker)next;
-
-                    long deletion = partitionLevelDeletion().markedForDeleteAt();
-                    if (marker.isOpen(isReverseOrder()))
-                        deletion = Math.max(deletion, marker.openDeletionTime(isReverseOrder()).markedForDeleteAt());
-                    row.setDeletionTimestamp(deletion);
-                    return marker;
-                }
-            }
-            return endOfData();
-        }
-    }
-
-    public static class SlicesIterator extends AbstractUnfilteredRowIterator
-    {
-        private final Holder holder;
-        private final MemtableAllocator allocator;
+        private final Holder current;
         private final ColumnFilter selection;
         private final Slices slices;
 
         private int idx;
-        private UnfilteredRowIterator currentSlice;
+        private Iterator<Unfiltered> currentSlice;
 
         private SlicesIterator(CFMetaData metadata,
                                DecoratedKey key,
-                               Holder holder,
                                ColumnFilter selection,
                                Slices slices,
                                boolean isReversed,
-                               MemtableAllocator allocator)
+                               Holder holder,
+                               Row staticRow)
         {
-            super(metadata, key, holder.deletionInfo.getPartitionDeletion(), selection.fetchedColumns(), makeStatic(selection, holder, allocator), isReversed, holder.stats);
-            this.holder = holder;
+            super(metadata, key, holder.deletionInfo.getPartitionDeletion(), selection.fetchedColumns(), staticRow, isReversed, holder.stats);
+            this.current = holder;
             this.selection = selection;
-            this.allocator = allocator;
             this.slices = slices;
         }
 
@@ -516,13 +282,7 @@ public class AtomicBTreePartition implements Partition
                         return endOfData();
 
                     int sliceIdx = isReverseOrder ? slices.size() - idx - 1 : idx;
-                    currentSlice = new SingleSliceIterator(metadata,
-                                                           partitionKey,
-                                                           holder,
-                                                           selection,
-                                                           slices.get(sliceIdx),
-                                                           isReverseOrder,
-                                                           allocator);
+                    currentSlice = sliceIterator(selection, slices.get(sliceIdx), isReverseOrder, current, Rows.EMPTY_STATIC_ROW);
                     idx++;
                 }
 
@@ -565,7 +325,7 @@ public class AtomicBTreePartition implements Partition
                     if (inputDeletionInfoCopy == null)
                         inputDeletionInfoCopy = update.deletionInfo().copy(HeapAllocator.instance);
 
-                    deletionInfo = current.deletionInfo.copy().add(inputDeletionInfoCopy);
+                    deletionInfo = current.deletionInfo.mutableCopy().add(inputDeletionInfoCopy);
                     updater.allocated(deletionInfo.unsharedHeapSize() - current.deletionInfo.unsharedHeapSize());
                 }
                 else
@@ -574,9 +334,9 @@ public class AtomicBTreePartition implements Partition
                 }
 
                 Row newStatic = update.staticRow();
-                MemtableRowData staticRow = newStatic == Rows.EMPTY_STATIC_ROW
-                                          ? current.staticRow
-                                          : (current.staticRow == null ? updater.apply(newStatic) : updater.apply(current.staticRow, newStatic));
+                Row staticRow = newStatic.isEmpty()
+                              ? current.staticRow
+                              : (current.staticRow.isEmpty() ? updater.apply(newStatic) : updater.apply(current.staticRow, newStatic));
                 Object[] tree = BTree.update(current.tree, update.metadata().comparator, update, update.rowCount(), updater);
                 RowStats newStats = current.stats.mergeWith(update.stats());
 
@@ -661,10 +421,10 @@ public class AtomicBTreePartition implements Partition
         final DeletionInfo deletionInfo;
         // the btree of rows
         final Object[] tree;
-        final MemtableRowData staticRow;
+        final Row staticRow;
         final RowStats stats;
 
-        Holder(Object[] tree, DeletionInfo deletionInfo, MemtableRowData staticRow, RowStats stats)
+        Holder(Object[] tree, DeletionInfo deletionInfo, Row staticRow, RowStats stats)
         {
             this.tree = tree;
             this.deletionInfo = deletionInfo;
@@ -679,7 +439,7 @@ public class AtomicBTreePartition implements Partition
     }
 
     // the function we provide to the btree utilities to perform any column replacements
-    private static final class RowUpdater implements UpdateFunction<Row, MemtableRowData>
+    private static final class RowUpdater implements UpdateFunction<Row, Row>
     {
         final AtomicBTreePartition updating;
         final MemtableAllocator allocator;
@@ -687,13 +447,13 @@ public class AtomicBTreePartition implements Partition
         final Updater indexer;
         final int nowInSec;
         Holder ref;
+        Row.Builder regularBuilder;
         long dataSize;
         long heapSize;
         long colUpdateTimeDelta = Long.MAX_VALUE;
-        final MemtableRowData.ReusableRow row;
         final MemtableAllocator.DataReclaimer reclaimer;
-        final MemtableAllocator.RowAllocator rowAllocator;
-        List<MemtableRowData> inserted; // TODO: replace with walk of aborted BTree
+        List<Row> inserted; // TODO: replace with walk of aborted BTree
+
 
         private RowUpdater(AtomicBTreePartition updating, MemtableAllocator allocator, OpOrder.Group writeOp, Updater indexer)
         {
@@ -702,18 +462,25 @@ public class AtomicBTreePartition implements Partition
             this.writeOp = writeOp;
             this.indexer = indexer;
             this.nowInSec = FBUtilities.nowInSeconds();
-            this.row = allocator.newReusableRow();
             this.reclaimer = allocator.reclaimer();
-            this.rowAllocator = allocator.newRowAllocator(updating.metadata(), writeOp);
         }
 
-        public MemtableRowData apply(Row insert)
+        private Row.Builder builder(Clustering clustering)
         {
-            rowAllocator.allocateNewRow(insert.clustering().size(), insert.columns(), insert.isStatic());
-            insert.copyTo(rowAllocator);
-            MemtableRowData data = rowAllocator.allocatedRowData();
+            boolean isStatic = clustering == Clustering.STATIC_CLUSTERING;
+            // We know we only insert/update one static per PartitionUpdate, so no point in saving the builder
+            if (isStatic)
+                return allocator.rowBuilder(updating.metadata(), writeOp, true);
+
+            if (regularBuilder == null)
+                regularBuilder = allocator.rowBuilder(updating.metadata(), writeOp, false);
+            return regularBuilder;
+        }
 
-            insertIntoIndexes(insert);
+        public Row apply(Row insert)
+        {
+            Row data = Rows.copy(insert, builder(insert.clustering())).build();
+            insertIntoIndexes(data);
 
             this.dataSize += data.dataSize();
             this.heapSize += data.unsharedHeapSizeExcludingData();
@@ -723,14 +490,14 @@ public class AtomicBTreePartition implements Partition
             return data;
         }
 
-        public MemtableRowData apply(MemtableRowData existing, Row update)
+        public Row apply(Row existing, Row update)
         {
             Columns mergedColumns = existing.columns().mergeTo(update.columns());
-            rowAllocator.allocateNewRow(update.clustering().size(), mergedColumns, update.isStatic());
 
-            colUpdateTimeDelta = Math.min(colUpdateTimeDelta, Rows.merge(row.setTo(existing), update, mergedColumns, rowAllocator, nowInSec, indexer));
+            Row.Builder builder = builder(existing.clustering());
+            colUpdateTimeDelta = Math.min(colUpdateTimeDelta, Rows.merge(existing, update, mergedColumns, builder, nowInSec, indexer));
 
-            MemtableRowData reconciled = rowAllocator.allocatedRowData();
+            Row reconciled = builder.build();
 
             dataSize += reconciled.dataSize() - existing.dataSize();
             heapSize += reconciled.unsharedHeapSizeExcludingData() - existing.unsharedHeapSizeExcludingData();
@@ -749,7 +516,7 @@ public class AtomicBTreePartition implements Partition
 
             maybeIndexPrimaryKeyColumns(toInsert);
             Clustering clustering = toInsert.clustering();
-            for (Cell cell : toInsert)
+            for (Cell cell : toInsert.cells())
                 indexer.insert(clustering, cell);
         }
 
@@ -761,15 +528,15 @@ public class AtomicBTreePartition implements Partition
             long timestamp = row.primaryKeyLivenessInfo().timestamp();
             int ttl = row.primaryKeyLivenessInfo().ttl();
 
-            for (Cell cell : row)
+            for (Cell cell : row.cells())
             {
-                long cellTimestamp = cell.livenessInfo().timestamp();
+                long cellTimestamp = cell.timestamp();
                 if (cell.isLive(nowInSec))
                 {
                     if (cellTimestamp > timestamp)
                     {
                         timestamp = cellTimestamp;
-                        ttl = cell.livenessInfo().ttl();
+                        ttl = cell.ttl();
                     }
                 }
             }
@@ -783,19 +550,19 @@ public class AtomicBTreePartition implements Partition
             this.heapSize = 0;
             if (inserted != null)
             {
-                for (MemtableRowData row : inserted)
+                for (Row row : inserted)
                     abort(row);
                 inserted.clear();
             }
             reclaimer.cancel();
         }
 
-        protected void abort(MemtableRowData abort)
+        protected void abort(Row abort)
         {
             reclaimer.reclaimImmediately(abort);
         }
 
-        protected void discard(MemtableRowData discard)
+        protected void discard(Row discard)
         {
             reclaimer.reclaim(discard);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/partitions/CountingUnfilteredRowIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/CountingUnfilteredRowIterator.java b/src/java/org/apache/cassandra/db/partitions/CountingUnfilteredRowIterator.java
index acaef5d..e5d1e75 100644
--- a/src/java/org/apache/cassandra/db/partitions/CountingUnfilteredRowIterator.java
+++ b/src/java/org/apache/cassandra/db/partitions/CountingUnfilteredRowIterator.java
@@ -49,10 +49,10 @@ public class CountingUnfilteredRowIterator extends WrappingUnfilteredRowIterator
     @Override
     public Unfiltered next()
     {
-        Unfiltered unfiltered = super.next();
-        if (unfiltered.kind() == Unfiltered.Kind.ROW)
-            counter.newRow((Row) unfiltered);
-        return unfiltered;
+        Unfiltered next = super.next();
+        if (next.isRow())
+            counter.newRow((Row)next);
+        return next;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/partitions/FilteredPartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/FilteredPartition.java b/src/java/org/apache/cassandra/db/partitions/FilteredPartition.java
index 813654d..1cac274 100644
--- a/src/java/org/apache/cassandra/db/partitions/FilteredPartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/FilteredPartition.java
@@ -17,21 +17,24 @@
  */
 package org.apache.cassandra.db.partitions;
 
-import java.util.Iterator;
+import java.util.*;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 
-public class FilteredPartition extends AbstractPartitionData implements Iterable<Row>
+public class FilteredPartition extends AbstractThreadUnsafePartition
 {
+    private final Row staticRow;
+
     private FilteredPartition(CFMetaData metadata,
                               DecoratedKey partitionKey,
                               PartitionColumns columns,
-                              int initialRowCapacity,
-                              boolean sortable)
+                              Row staticRow,
+                              List<Row> rows)
     {
-        super(metadata, partitionKey, DeletionTime.LIVE, columns, initialRowCapacity, sortable);
+        super(metadata, partitionKey, columns, rows);
+        this.staticRow = staticRow;
     }
 
     /**
@@ -42,25 +45,43 @@ public class FilteredPartition extends AbstractPartitionData implements Iterable
      */
     public static FilteredPartition create(RowIterator iterator)
     {
-        FilteredPartition partition = new FilteredPartition(iterator.metadata(),
-                                                            iterator.partitionKey(),
-                                                            iterator.columns(),
-                                                            4,
-                                                            iterator.isReverseOrder());
-
-        partition.staticRow = iterator.staticRow().takeAlias();
+        CFMetaData metadata = iterator.metadata();
+        boolean reversed = iterator.isReverseOrder();
 
-        Writer writer = partition.new Writer(true);
+        List<Row> rows = new ArrayList<>();
 
         while (iterator.hasNext())
-            iterator.next().copyTo(writer);
+        {
+            Unfiltered unfiltered = iterator.next();
+            if (unfiltered.isRow())
+                rows.add((Row)unfiltered);
+        }
+
+        if (reversed)
+            Collections.reverse(rows);
+
+        return new FilteredPartition(metadata, iterator.partitionKey(), iterator.columns(), iterator.staticRow(), rows);
+    }
+
+    protected boolean canHaveShadowedData()
+    {
+        // We only create instances from RowIterator that don't have shadowed data (nor deletion info really)
+        return false;
+    }
 
-        // A Partition (or more precisely AbstractPartitionData) always assumes that its data is in clustering
-        // order. So if we've just added them in reverse clustering order, reverse them.
-        if (iterator.isReverseOrder())
-            partition.reverse();
+    public Row staticRow()
+    {
+        return staticRow;
+    }
 
-        return partition;
+    public DeletionInfo deletionInfo()
+    {
+        return DeletionInfo.LIVE;
+    }
+
+    public RowStats stats()
+    {
+        return RowStats.NO_STATS;
     }
 
     public RowIterator rowIterator()
@@ -90,7 +111,7 @@ public class FilteredPartition extends AbstractPartitionData implements Iterable
 
             public Row staticRow()
             {
-                return staticRow == null ? Rows.EMPTY_STATIC_ROW : staticRow;
+                return FilteredPartition.this.staticRow();
             }
 
             public boolean hasNext()
@@ -117,26 +138,20 @@ public class FilteredPartition extends AbstractPartitionData implements Iterable
     @Override
     public String toString()
     {
-        try (RowIterator iterator = rowIterator())
-        {
-            StringBuilder sb = new StringBuilder();
-            CFMetaData metadata = iterator.metadata();
-            PartitionColumns columns = iterator.columns();
+        StringBuilder sb = new StringBuilder();
 
-            sb.append(String.format("[%s.%s] key=%s columns=%s reversed=%b",
-                                    metadata.ksName,
-                                    metadata.cfName,
-                                    metadata.getKeyValidator().getString(iterator.partitionKey().getKey()),
-                                    columns,
-                                    iterator.isReverseOrder()));
+        sb.append(String.format("[%s.%s] key=%s columns=%s",
+                    metadata.ksName,
+                    metadata.cfName,
+                    metadata.getKeyValidator().getString(partitionKey().getKey()),
+                    columns));
 
-            if (iterator.staticRow() != Rows.EMPTY_STATIC_ROW)
-                sb.append("\n    ").append(iterator.staticRow().toString(metadata));
+        if (staticRow() != Rows.EMPTY_STATIC_ROW)
+            sb.append("\n    ").append(staticRow().toString(metadata));
 
-            while (iterator.hasNext())
-                sb.append("\n    ").append(iterator.next().toString(metadata));
+        for (Row row : this)
+            sb.append("\n    ").append(row.toString(metadata));
 
-            return sb.toString();
-        }
+        return sb.toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/partitions/FilteringPartitionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/FilteringPartitionIterator.java b/src/java/org/apache/cassandra/db/partitions/FilteringPartitionIterator.java
deleted file mode 100644
index c40109b..0000000
--- a/src/java/org/apache/cassandra/db/partitions/FilteringPartitionIterator.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * 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.partitions;
-
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.*;
-
-/**
- * Abstract class to make it easier to write iterators that filter some
- * parts of another iterator (used for purging tombstones and removing dropped columns).
- */
-public abstract class FilteringPartitionIterator extends WrappingUnfilteredPartitionIterator
-{
-    private UnfilteredRowIterator next;
-
-    protected FilteringPartitionIterator(UnfilteredPartitionIterator iter)
-    {
-        super(iter);
-    }
-
-    // The filter to use for filtering row contents. Is null by default to mean no particular filtering
-    // but can be overriden by subclasses. Please see FilteringAtomIterator for details on how this is used.
-    protected FilteringRow makeRowFilter()
-    {
-        return null;
-    }
-
-    // Whether or not we should bother filtering the provided rows iterator. This
-    // exists mainly for preformance
-    protected boolean shouldFilter(UnfilteredRowIterator iterator)
-    {
-        return true;
-    }
-
-    protected boolean includeRangeTombstoneMarker(RangeTombstoneMarker marker)
-    {
-        return true;
-    }
-
-    protected boolean includePartitionDeletion(DeletionTime dt)
-    {
-        return true;
-    }
-
-    // Allows to modify the range tombstone returned. This is called *after* includeRangeTombstoneMarker has been called.
-    protected RangeTombstoneMarker filterRangeTombstoneMarker(RangeTombstoneMarker marker, boolean reversed)
-    {
-        return marker;
-    }
-
-    // Called when a particular partition is skipped due to being empty post filtering
-    protected void onEmpty(DecoratedKey key)
-    {
-    }
-
-    public boolean hasNext()
-    {
-        while (next == null && super.hasNext())
-        {
-            UnfilteredRowIterator iterator = super.next();
-            if (shouldFilter(iterator))
-            {
-                next = new FilteringIterator(iterator);
-                if (!isForThrift() && next.isEmpty())
-                {
-                    onEmpty(iterator.partitionKey());
-                    iterator.close();
-                    next = null;
-                }
-            }
-            else
-            {
-                next = iterator;
-            }
-        }
-        return next != null;
-    }
-
-    public UnfilteredRowIterator next()
-    {
-        UnfilteredRowIterator toReturn = next;
-        next = null;
-        return toReturn;
-    }
-
-    @Override
-    public void close()
-    {
-        try
-        {
-            super.close();
-        }
-        finally
-        {
-            if (next != null)
-                next.close();
-        }
-    }
-
-    private class FilteringIterator extends FilteringRowIterator
-    {
-        private FilteringIterator(UnfilteredRowIterator iterator)
-        {
-            super(iterator);
-        }
-
-        @Override
-        protected FilteringRow makeRowFilter()
-        {
-            return FilteringPartitionIterator.this.makeRowFilter();
-        }
-
-        @Override
-        protected boolean includeRangeTombstoneMarker(RangeTombstoneMarker marker)
-        {
-            return FilteringPartitionIterator.this.includeRangeTombstoneMarker(marker);
-        }
-
-        @Override
-        protected RangeTombstoneMarker filterRangeTombstoneMarker(RangeTombstoneMarker marker, boolean reversed)
-        {
-            return FilteringPartitionIterator.this.filterRangeTombstoneMarker(marker, reversed);
-        }
-
-        @Override
-        protected boolean includePartitionDeletion(DeletionTime dt)
-        {
-            return FilteringPartitionIterator.this.includePartitionDeletion(dt);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/partitions/PartitionStatisticsCollector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionStatisticsCollector.java b/src/java/org/apache/cassandra/db/partitions/PartitionStatisticsCollector.java
new file mode 100644
index 0000000..510b9c8
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/partitions/PartitionStatisticsCollector.java
@@ -0,0 +1,30 @@
+/*
+ * 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.partitions;
+
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.rows.Cell;
+
+public interface PartitionStatisticsCollector
+{
+    public void update(LivenessInfo info);
+    public void update(DeletionTime deletionTime);
+    public void update(Cell cell);
+    public void updateColumnSetPerRow(long columnSetInRow);
+    public void updateHasLegacyCounterShards(boolean hasLegacyCounterShards);
+}


[03/15] cassandra git commit: Simplify some 8099's implementations

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 4399a80..1aadd78 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -108,7 +108,7 @@ public class CassandraServer implements Cassandra.Iface
 
     public List<ColumnOrSuperColumn> thriftifyColumns(CFMetaData metadata, Iterator<LegacyLayout.LegacyCell> cells)
     {
-        ArrayList<ColumnOrSuperColumn> thriftColumns = new ArrayList<ColumnOrSuperColumn>();
+        ArrayList<ColumnOrSuperColumn> thriftColumns = new ArrayList<>();
         while (cells.hasNext())
         {
             LegacyLayout.LegacyCell cell = cells.next();
@@ -142,7 +142,7 @@ public class CassandraServer implements Cassandra.Iface
 
     private List<Column> thriftifyColumnsAsColumns(CFMetaData metadata, Iterator<LegacyLayout.LegacyCell> cells)
     {
-        List<Column> thriftColumns = new ArrayList<Column>();
+        List<Column> thriftColumns = new ArrayList<>();
         while (cells.hasNext())
             thriftColumns.add(thriftifySubColumn(metadata, cells.next()));
         return thriftColumns;
@@ -162,7 +162,7 @@ public class CassandraServer implements Cassandra.Iface
     {
         if (subcolumnsOnly)
         {
-            ArrayList<ColumnOrSuperColumn> thriftSuperColumns = new ArrayList<ColumnOrSuperColumn>();
+            ArrayList<ColumnOrSuperColumn> thriftSuperColumns = new ArrayList<>();
             while (cells.hasNext())
             {
                 LegacyLayout.LegacyCell cell = cells.next();
@@ -185,7 +185,7 @@ public class CassandraServer implements Cassandra.Iface
 
     private List<ColumnOrSuperColumn> thriftifySuperColumns(Iterator<LegacyLayout.LegacyCell> cells, boolean reversed)
     {
-        ArrayList<ColumnOrSuperColumn> thriftSuperColumns = new ArrayList<ColumnOrSuperColumn>();
+        ArrayList<ColumnOrSuperColumn> thriftSuperColumns = new ArrayList<>();
         SuperColumn current = null;
         while (cells.hasNext())
         {
@@ -198,7 +198,7 @@ public class CassandraServer implements Cassandra.Iface
                 if (current != null && reversed)
                     Collections.reverse(current.columns);
 
-                current = new SuperColumn(scName, new ArrayList<Column>());
+                current = new SuperColumn(scName, new ArrayList<>());
                 thriftSuperColumns.add(new ColumnOrSuperColumn().setSuper_column(current));
             }
             current.getColumns().add(thriftifySubColumn(cell, cell.name.superColumnSubName()));
@@ -212,7 +212,7 @@ public class CassandraServer implements Cassandra.Iface
 
     private List<ColumnOrSuperColumn> thriftifyCounterSuperColumns(CFMetaData metadata, Iterator<LegacyLayout.LegacyCell> cells, boolean reversed)
     {
-        ArrayList<ColumnOrSuperColumn> thriftSuperColumns = new ArrayList<ColumnOrSuperColumn>();
+        ArrayList<ColumnOrSuperColumn> thriftSuperColumns = new ArrayList<>();
         CounterSuperColumn current = null;
         while (cells.hasNext())
         {
@@ -225,7 +225,7 @@ public class CassandraServer implements Cassandra.Iface
                 if (current != null && reversed)
                     Collections.reverse(current.columns);
 
-                current = new CounterSuperColumn(scName, new ArrayList<CounterColumn>());
+                current = new CounterSuperColumn(scName, new ArrayList<>());
                 thriftSuperColumns.add(new ColumnOrSuperColumn().setCounter_super_column(current));
             }
             current.getColumns().add(thriftifySubCounter(metadata, cell).setName(cell.name.superColumnSubName()));
@@ -262,7 +262,7 @@ public class CassandraServer implements Cassandra.Iface
     {
         try (PartitionIterator results = read(commands, consistency_level, cState))
         {
-            Map<ByteBuffer, List<ColumnOrSuperColumn>> columnFamiliesMap = new HashMap<ByteBuffer, List<ColumnOrSuperColumn>>();
+            Map<ByteBuffer, List<ColumnOrSuperColumn>> columnFamiliesMap = new HashMap<>();
             while (results.hasNext())
             {
                 try (RowIterator iter = results.next())
@@ -360,7 +360,7 @@ public class CassandraServer implements Cassandra.Iface
     private ClusteringIndexFilter toInternalFilter(CFMetaData metadata, ColumnParent parent, SliceRange range)
     {
         if (metadata.isSuper() && parent.isSetSuper_column())
-            return new ClusteringIndexNamesFilter(FBUtilities.<Clustering>singleton(new SimpleClustering(parent.bufferForSuper_column()), metadata.comparator), range.reversed);
+            return new ClusteringIndexNamesFilter(FBUtilities.singleton(new Clustering(parent.bufferForSuper_column()), metadata.comparator), range.reversed);
         else
             return new ClusteringIndexSliceFilter(makeSlices(metadata, range), range.reversed);
     }
@@ -384,13 +384,13 @@ public class CassandraServer implements Cassandra.Iface
                 {
                     if (parent.isSetSuper_column())
                     {
-                        return new ClusteringIndexNamesFilter(FBUtilities.<Clustering>singleton(new SimpleClustering(parent.bufferForSuper_column()), metadata.comparator), false);
+                        return new ClusteringIndexNamesFilter(FBUtilities.singleton(new Clustering(parent.bufferForSuper_column()), metadata.comparator), false);
                     }
                     else
                     {
                         NavigableSet<Clustering> clusterings = new TreeSet<>(metadata.comparator);
                         for (ByteBuffer bb : predicate.column_names)
-                            clusterings.add(new SimpleClustering(bb));
+                            clusterings.add(new Clustering(bb));
                         return new ClusteringIndexNamesFilter(clusterings, false);
                     }
                 }
@@ -455,7 +455,7 @@ public class CassandraServer implements Cassandra.Iface
             // We only want to include the static columns that are selected by the slices
             for (ColumnDefinition def : columns.statics)
             {
-                if (slices.selects(new SimpleClustering(def.name.bytes)))
+                if (slices.selects(new Clustering(def.name.bytes)))
                     builder.add(def);
             }
             columns = builder.build();
@@ -608,21 +608,20 @@ public class CassandraServer implements Cassandra.Iface
                     builder.select(dynamicDef, CellPath.create(column_path.column));
                     columns = builder.build();
                 }
-                filter = new ClusteringIndexNamesFilter(FBUtilities.<Clustering>singleton(new SimpleClustering(column_path.super_column), metadata.comparator),
+                filter = new ClusteringIndexNamesFilter(FBUtilities.singleton(new Clustering(column_path.super_column), metadata.comparator),
                                                   false);
             }
             else
             {
                 LegacyLayout.LegacyCellName cellname = LegacyLayout.decodeCellName(metadata, column_path.super_column, column_path.column);
                 columns = ColumnFilter.selection(PartitionColumns.of(cellname.column));
-                filter = new ClusteringIndexNamesFilter(FBUtilities.<Clustering>singleton(cellname.clustering, metadata.comparator), false);
+                filter = new ClusteringIndexNamesFilter(FBUtilities.singleton(cellname.clustering, metadata.comparator), false);
             }
 
-            long now = System.currentTimeMillis();
             DecoratedKey dk = StorageService.getPartitioner().decorateKey(key);
             SinglePartitionReadCommand<?> command = SinglePartitionReadCommand.create(true, metadata, FBUtilities.nowInSeconds(), columns, RowFilter.NONE, DataLimits.NONE, dk, filter);
 
-            try (RowIterator result = PartitionIterators.getOnlyElement(read(Arrays.<SinglePartitionReadCommand<?>>asList(command), consistencyLevel, cState), command))
+            try (RowIterator result = PartitionIterators.getOnlyElement(read(Arrays.asList(command), consistencyLevel, cState), command))
             {
                 if (!result.hasNext())
                     throw new NotFoundException();
@@ -768,7 +767,7 @@ public class CassandraServer implements Cassandra.Iface
             String keyspace = cState.getKeyspace();
             cState.hasColumnFamilyAccess(keyspace, column_parent.column_family, Permission.SELECT);
 
-            Map<ByteBuffer, Integer> counts = new HashMap<ByteBuffer, Integer>();
+            Map<ByteBuffer, Integer> counts = new HashMap<>();
             Map<ByteBuffer, List<ColumnOrSuperColumn>> columnFamiliesMap = multigetSliceInternal(keyspace,
                                                                                                  keys,
                                                                                                  column_parent,
@@ -791,6 +790,14 @@ public class CassandraServer implements Cassandra.Iface
         }
     }
 
+    private Cell cellFromColumn(CFMetaData metadata, LegacyLayout.LegacyCellName name, Column column)
+    {
+        CellPath path = name.collectionElement == null ? null : CellPath.create(name.collectionElement);
+        return column.ttl == 0
+             ? BufferCell.live(metadata, name.column, column.timestamp, column.value, path)
+             : BufferCell.expiring(name.column, column.timestamp, column.ttl, FBUtilities.nowInSeconds(), column.value, path);
+    }
+
     private void internal_insert(ByteBuffer key, ColumnParent column_parent, Column column, ConsistencyLevel consistency_level)
     throws RequestValidationException, UnavailableException, TimedOutException
     {
@@ -806,22 +813,17 @@ public class CassandraServer implements Cassandra.Iface
         {
             throw new org.apache.cassandra.exceptions.InvalidRequestException("missing mandatory super column name for super CF " + column_parent.column_family);
         }
-        ThriftValidation.validateColumnNames(metadata, column_parent, Arrays.asList(column.name));
+        ThriftValidation.validateColumnNames(metadata, column_parent, Collections.singletonList(column.name));
         ThriftValidation.validateColumnData(metadata, column_parent.super_column, column);
 
         org.apache.cassandra.db.Mutation mutation;
         try
         {
-            LegacyLayout.LegacyCellName name = LegacyLayout.decodeCellName(metadata, column_parent.super_column, column.name);
-
             DecoratedKey dk = StorageService.getPartitioner().decorateKey(key);
-            PartitionUpdate update = new PartitionUpdate(metadata, dk, PartitionColumns.of(name.column), 1);
 
-            Row.Writer writer = name.column.isStatic() ? update.staticWriter() : update.writer();
-            name.clustering.writeTo(writer);
-            CellPath path = name.collectionElement == null ? null : CellPath.create(name.collectionElement);
-            writer.writeCell(name.column, false, column.value, SimpleLivenessInfo.forUpdate(column.timestamp, column.ttl, FBUtilities.nowInSeconds(), metadata), path);
-            writer.endOfRow();
+            LegacyLayout.LegacyCellName name = LegacyLayout.decodeCellName(metadata, column_parent.super_column, column.name);
+            Cell cell = cellFromColumn(metadata, name, column);
+            PartitionUpdate update = PartitionUpdate.singleRowUpdate(metadata, dk, ArrayBackedRow.singleCellRow(name.clustering, cell));
 
             mutation = new org.apache.cassandra.db.Mutation(update);
         }
@@ -829,7 +831,7 @@ public class CassandraServer implements Cassandra.Iface
         {
             throw new org.apache.cassandra.exceptions.InvalidRequestException(e.getMessage());
         }
-        doInsert(consistency_level, Arrays.asList(mutation));
+        doInsert(consistency_level, Collections.singletonList(mutation));
     }
 
     public void insert(ByteBuffer key, ColumnParent column_parent, Column column, ConsistencyLevel consistency_level)
@@ -896,23 +898,15 @@ public class CassandraServer implements Cassandra.Iface
             if (metadata.isSuper())
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("CAS does not support supercolumns");
 
-            Iterable<ByteBuffer> names = Iterables.transform(updates, new Function<Column, ByteBuffer>()
-            {
-                public ByteBuffer apply(Column column)
-                {
-                    return column.name;
-                }
-            });
+            Iterable<ByteBuffer> names = Iterables.transform(updates, column -> column.name);
             ThriftValidation.validateColumnNames(metadata, new ColumnParent(column_family), names);
             for (Column column : updates)
                 ThriftValidation.validateColumnData(metadata, null, column);
 
-            CFMetaData cfm = Schema.instance.getCFMetaData(cState.getKeyspace(), column_family);
-
             DecoratedKey dk = StorageService.getPartitioner().decorateKey(key);
             int nowInSec = FBUtilities.nowInSeconds();
 
-            PartitionUpdate partitionUpdates = RowIterators.toUpdate(LegacyLayout.toRowIterator(metadata, dk, toLegacyCells(metadata, updates, nowInSec).iterator(), nowInSec));
+            PartitionUpdate partitionUpdates = PartitionUpdate.fromIterator(LegacyLayout.toRowIterator(metadata, dk, toLegacyCells(metadata, updates, nowInSec).iterator(), nowInSec));
 
             FilteredPartition partitionExpected = null;
             if (!expected.isEmpty())
@@ -1096,7 +1090,7 @@ public class CassandraServer implements Cassandra.Iface
                 if (metadata.isCounter())
                     ThriftConversion.fromThrift(consistency_level).validateCounterForWrite(metadata);
 
-                DeletionInfo delInfo = DeletionInfo.live();
+                MutableDeletionInfo delInfo = MutableDeletionInfo.live();
                 List<LegacyLayout.LegacyCell> cells = new ArrayList<>();
                 for (Mutation m : muts)
                 {
@@ -1113,7 +1107,7 @@ public class CassandraServer implements Cassandra.Iface
                 }
 
                 sortAndMerge(metadata, cells, nowInSec);
-                PartitionUpdate update = UnfilteredRowIterators.toUpdate(LegacyLayout.toUnfilteredRowIterator(metadata, dk, delInfo, cells.iterator()));
+                PartitionUpdate update = PartitionUpdate.fromIterator(LegacyLayout.toUnfilteredRowIterator(metadata, dk, delInfo, cells.iterator()));
 
                 org.apache.cassandra.db.Mutation mutation;
                 if (metadata.isCounter())
@@ -1173,12 +1167,12 @@ public class CassandraServer implements Cassandra.Iface
         }
     }
 
-    private void addRange(CFMetaData cfm, DeletionInfo delInfo, Slice.Bound start, Slice.Bound end, long timestamp, int nowInSec)
+    private void addRange(CFMetaData cfm, MutableDeletionInfo delInfo, Slice.Bound start, Slice.Bound end, long timestamp, int nowInSec)
     {
-        delInfo.add(new RangeTombstone(Slice.make(start, end), new SimpleDeletionTime(timestamp, nowInSec)), cfm.comparator);
+        delInfo.add(new RangeTombstone(Slice.make(start, end), new DeletionTime(timestamp, nowInSec)), cfm.comparator);
     }
 
-    private void deleteColumnOrSuperColumn(DeletionInfo delInfo, List<LegacyLayout.LegacyCell> cells, CFMetaData cfm, Deletion del, int nowInSec)
+    private void deleteColumnOrSuperColumn(MutableDeletionInfo delInfo, List<LegacyLayout.LegacyCell> cells, CFMetaData cfm, Deletion del, int nowInSec)
     throws InvalidRequestException
     {
         if (del.predicate != null && del.predicate.column_names != null)
@@ -1227,7 +1221,7 @@ public class CassandraServer implements Cassandra.Iface
             if (del.super_column != null)
                 addRange(cfm, delInfo, Slice.Bound.inclusiveStartOf(del.super_column), Slice.Bound.inclusiveEndOf(del.super_column), del.timestamp, nowInSec);
             else
-                delInfo.add(new SimpleDeletionTime(del.timestamp, nowInSec));
+                delInfo.add(new DeletionTime(del.timestamp, nowInSec));
         }
     }
 
@@ -1320,23 +1314,17 @@ public class CassandraServer implements Cassandra.Iface
         }
         else if (column_path.super_column != null && column_path.column == null)
         {
-            update = new PartitionUpdate(metadata, dk, PartitionColumns.NONE, 1);
-            Row.Writer writer = update.writer();
-            writer.writeClusteringValue(column_path.super_column);
-            writer.writeRowDeletion(new SimpleDeletionTime(timestamp, nowInSec));
-            writer.endOfRow();
+            Row row = ArrayBackedRow.emptyDeletedRow(new Clustering(column_path.super_column), new DeletionTime(timestamp, nowInSec));
+            update = PartitionUpdate.singleRowUpdate(metadata, dk, row);
         }
         else
         {
             try
             {
                 LegacyLayout.LegacyCellName name = LegacyLayout.decodeCellName(metadata, column_path.super_column, column_path.column);
-                update = new PartitionUpdate(metadata, dk, PartitionColumns.of(name.column), 1);
-                Row.Writer writer = name.column.isStatic() ? update.staticWriter() : update.writer();
-                name.clustering.writeTo(writer);
                 CellPath path = name.collectionElement == null ? null : CellPath.create(name.collectionElement);
-                writer.writeCell(name.column, false, ByteBufferUtil.EMPTY_BYTE_BUFFER, SimpleLivenessInfo.forDeletion(timestamp, nowInSec), path);
-                writer.endOfRow();
+                Cell cell = BufferCell.tombstone(name.column, timestamp, nowInSec, path);
+                update = PartitionUpdate.singleRowUpdate(metadata, dk, ArrayBackedRow.singleCellRow(name.clustering, cell));
             }
             catch (UnknownColumnException e)
             {
@@ -1347,9 +1335,9 @@ public class CassandraServer implements Cassandra.Iface
         org.apache.cassandra.db.Mutation mutation = new org.apache.cassandra.db.Mutation(update);
 
         if (isCommutativeOp)
-            doInsert(consistency_level, Arrays.asList(new CounterMutation(mutation, ThriftConversion.fromThrift(consistency_level))));
+            doInsert(consistency_level, Collections.singletonList(new CounterMutation(mutation, ThriftConversion.fromThrift(consistency_level))));
         else
-            doInsert(consistency_level, Arrays.asList(mutation));
+            doInsert(consistency_level, Collections.singletonList(mutation));
     }
 
     public void remove(ByteBuffer key, ColumnPath column_path, long timestamp, ConsistencyLevel consistency_level)
@@ -1483,7 +1471,7 @@ public class CassandraServer implements Cassandra.Iface
                 PartitionPosition end = range.end_key == null
                                 ? p.getTokenFactory().fromString(range.end_token).maxKeyBound()
                                 : PartitionPosition.ForKey.get(range.end_key, p);
-                bounds = new Bounds<PartitionPosition>(PartitionPosition.ForKey.get(range.start_key, p), end);
+                bounds = new Bounds<>(PartitionPosition.ForKey.get(range.start_key, p), end);
             }
             int nowInSec = FBUtilities.nowInSeconds();
             schedule(DatabaseDescriptor.getRangeRpcTimeout());
@@ -1569,7 +1557,7 @@ public class CassandraServer implements Cassandra.Iface
                 PartitionPosition end = range.end_key == null
                                 ? p.getTokenFactory().fromString(range.end_token).maxKeyBound()
                                 : PartitionPosition.ForKey.get(range.end_key, p);
-                bounds = new Bounds<PartitionPosition>(PartitionPosition.ForKey.get(range.start_key, p), end);
+                bounds = new Bounds<>(PartitionPosition.ForKey.get(range.start_key, p), end);
             }
 
             if (range.row_filter != null && !range.row_filter.isEmpty())
@@ -1582,7 +1570,7 @@ public class CassandraServer implements Cassandra.Iface
                 ClusteringIndexFilter filter = new ClusteringIndexSliceFilter(Slices.ALL, false);
                 DataLimits limits = getLimits(range.count, true, Integer.MAX_VALUE);
                 Clustering pageFrom = metadata.isSuper()
-                                    ? new SimpleClustering(start_column)
+                                    ? new Clustering(start_column)
                                     : LegacyLayout.decodeCellName(metadata, start_column).clustering;
                 PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
                                                                               true,
@@ -1624,7 +1612,7 @@ public class CassandraServer implements Cassandra.Iface
     {
         try (PartitionIterator iter = results)
         {
-            List<KeySlice> keySlices = new ArrayList<KeySlice>();
+            List<KeySlice> keySlices = new ArrayList<>();
             while (iter.hasNext())
             {
                 try (RowIterator partition = iter.next())
@@ -1667,8 +1655,8 @@ public class CassandraServer implements Cassandra.Iface
             consistencyLevel.validateForRead(keyspace);
 
             IPartitioner p = StorageService.getPartitioner();
-            AbstractBounds<PartitionPosition> bounds = new Bounds<PartitionPosition>(PartitionPosition.ForKey.get(index_clause.start_key, p),
-                                                                         p.getMinimumToken().minKeyBound());
+            AbstractBounds<PartitionPosition> bounds = new Bounds<>(PartitionPosition.ForKey.get(index_clause.start_key, p),
+                                                                    p.getMinimumToken().minKeyBound());
 
             int nowInSec = FBUtilities.nowInSeconds();
             ColumnFilter columns = makeColumnFilter(metadata, column_parent, column_predicate);
@@ -1706,7 +1694,7 @@ public class CassandraServer implements Cassandra.Iface
         validateLogin();
 
         Set<String> keyspaces = Schema.instance.getKeyspaces();
-        List<KsDef> ksset = new ArrayList<KsDef>(keyspaces.size());
+        List<KsDef> ksset = new ArrayList<>(keyspaces.size());
         for (String ks : keyspaces)
         {
             try
@@ -1778,7 +1766,7 @@ public class CassandraServer implements Cassandra.Iface
     throws TException, InvalidRequestException
     {
         List<CfSplit> splits = describe_splits_ex(cfName, start_token, end_token, keys_per_split);
-        List<String> result = new ArrayList<String>(splits.size() + 1);
+        List<String> result = new ArrayList<>(splits.size() + 1);
 
         result.add(splits.get(0).getStart_token());
         for (CfSplit cfSplit : splits)
@@ -1793,10 +1781,10 @@ public class CassandraServer implements Cassandra.Iface
         try
         {
             Token.TokenFactory tf = StorageService.getPartitioner().getTokenFactory();
-            Range<Token> tr = new Range<Token>(tf.fromString(start_token), tf.fromString(end_token));
+            Range<Token> tr = new Range<>(tf.fromString(start_token), tf.fromString(end_token));
             List<Pair<Range<Token>, Long>> splits =
                     StorageService.instance.getSplits(state().getKeyspace(), cfName, tr, keys_per_split);
-            List<CfSplit> result = new ArrayList<CfSplit>(splits.size());
+            List<CfSplit> result = new ArrayList<>(splits.size());
             for (Pair<Range<Token>, Long> split : splits)
                 result.add(new CfSplit(split.left.left.toString(), split.left.right.toString(), split.right));
             return result;
@@ -2115,17 +2103,13 @@ public class CassandraServer implements Cassandra.Iface
             {
                 LegacyLayout.LegacyCellName name = LegacyLayout.decodeCellName(metadata, column_parent.super_column, column.name);
                 DecoratedKey dk = StorageService.getPartitioner().decorateKey(key);
-                PartitionUpdate update = new PartitionUpdate(metadata, dk, PartitionColumns.of(name.column), 1);
-
-                Row.Writer writer = name.column.isStatic() ? update.staticWriter() : update.writer();
-                name.clustering.writeTo(writer);
-                CellPath path = name.collectionElement == null ? null : CellPath.create(name.collectionElement);
 
                 // See UpdateParameters.addCounter() for more details on this
                 ByteBuffer value = CounterContext.instance().createLocal(column.value);
+                CellPath path = name.collectionElement == null ? null : CellPath.create(name.collectionElement);
+                Cell cell = BufferCell.live(metadata, name.column, FBUtilities.timestampMicros(), value, path);
 
-                writer.writeCell(name.column, true, value, SimpleLivenessInfo.forUpdate(FBUtilities.timestampMicros(), LivenessInfo.NO_TTL, FBUtilities.nowInSeconds(), metadata), path);
-                writer.endOfRow();
+                PartitionUpdate update = PartitionUpdate.singleRowUpdate(metadata, dk, ArrayBackedRow.singleCellRow(name.clustering, cell));
 
                 org.apache.cassandra.db.Mutation mutation = new org.apache.cassandra.db.Mutation(update);
                 doInsert(consistency_level, Arrays.asList(new CounterMutation(mutation, ThriftConversion.fromThrift(consistency_level))));
@@ -2489,7 +2473,7 @@ public class CassandraServer implements Cassandra.Iface
             // Gather the clustering for the expected values and query those.
             BTreeSet.Builder<Clustering> clusterings = BTreeSet.builder(metadata.comparator);
             for (Row row : expected)
-                clusterings.add(row.clustering().takeAlias());
+                clusterings.add(row.clustering());
             PartitionColumns columns = expected.staticRow().isEmpty()
                                      ? metadata.partitionColumns().withoutStatics()
                                      : metadata.partitionColumns();
@@ -2511,11 +2495,29 @@ public class CassandraServer implements Cassandra.Iface
                 if (c == null)
                     return false;
 
-                for (Cell ce : e)
+                SearchIterator<ColumnDefinition, ColumnData> searchIter = c.searchIterator();
+                for (ColumnData expectedData : e)
                 {
-                    Cell cc = c.getCell(ce.column());
-                    if (cc == null || !cc.value().equals(ce.value()))
+                    ColumnDefinition column = expectedData.column();
+                    ColumnData currentData = searchIter.next(column);
+                    if (currentData == null)
                         return false;
+
+                    if (column.isSimple())
+                    {
+                        if (!((Cell)currentData).value().equals(((Cell)expectedData).value()))
+                            return false;
+                    }
+                    else
+                    {
+                        ComplexColumnData currentComplexData = (ComplexColumnData)currentData;
+                        for (Cell expectedCell : (ComplexColumnData)expectedData)
+                        {
+                            Cell currentCell = currentComplexData.getCell(expectedCell.path());
+                            if (currentCell == null || !currentCell.value().equals(expectedCell.value()))
+                                return false;
+                        }
+                    }
                 }
             }
             return true;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/thrift/ThriftConversion.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftConversion.java b/src/java/org/apache/cassandra/thrift/ThriftConversion.java
index d99217d..c9e5062 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftConversion.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftConversion.java
@@ -350,7 +350,7 @@ public class ThriftConversion
         {
             // SuperColumn tables: we use a special map to hold dynamic values within a given super column
             defs.add(ColumnDefinition.clusteringKeyDef(ks, cf, names.defaultClusteringName(), comparator, 0));
-            defs.add(ColumnDefinition.regularDef(ks, cf, CompactTables.SUPER_COLUMN_MAP_COLUMN_STR, MapType.getInstance(subComparator, defaultValidator, true), null));
+            defs.add(ColumnDefinition.regularDef(ks, cf, CompactTables.SUPER_COLUMN_MAP_COLUMN_STR, MapType.getInstance(subComparator, defaultValidator, true)));
         }
         else
         {
@@ -361,7 +361,7 @@ public class ThriftConversion
             for (int i = 0; i < subTypes.size(); i++)
                 defs.add(ColumnDefinition.clusteringKeyDef(ks, cf, names.defaultClusteringName(), subTypes.get(i), i));
 
-            defs.add(ColumnDefinition.regularDef(ks, cf, names.defaultCompactValueName(), defaultValidator, null));
+            defs.add(ColumnDefinition.regularDef(ks, cf, names.defaultCompactValueName(), defaultValidator));
         }
     }
 
@@ -454,7 +454,6 @@ public class ThriftConversion
     {
         boolean isSuper = thriftSubcomparator != null;
         // For super columns, the componentIndex is 1 because the ColumnDefinition applies to the column component.
-        Integer componentIndex = isSuper ? 1 : null;
         AbstractType<?> comparator = thriftSubcomparator == null ? thriftComparator : thriftSubcomparator;
         try
         {
@@ -475,7 +474,7 @@ public class ThriftConversion
                                     thriftColumnDef.index_type == null ? null : org.apache.cassandra.config.IndexType.valueOf(thriftColumnDef.index_type.name()),
                                     thriftColumnDef.index_options,
                                     thriftColumnDef.index_name,
-                                    componentIndex,
+                                    null,
                                     kind);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/thrift/ThriftResultsMerger.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftResultsMerger.java b/src/java/org/apache/cassandra/thrift/ThriftResultsMerger.java
index ccb6e74..9c5a99f 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftResultsMerger.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftResultsMerger.java
@@ -101,10 +101,10 @@ public class ThriftResultsMerger extends WrappingUnfilteredPartitionIterator
         // We initialize lazily to avoid having this iterator fetch the wrapped iterator before it's actually asked for it.
         private boolean isInit;
 
-        private Row staticRow;
-        private int i; // the index of the next column of static row to return
+        private Iterator<Cell> staticCells;
 
-        private ReusableRow nextToMerge;
+        private final Row.Builder builder;
+        private Row nextToMerge;
         private Unfiltered nextFromWrapped;
 
         private PartitionMerger(UnfilteredRowIterator results, int nowInSec)
@@ -112,15 +112,16 @@ public class ThriftResultsMerger extends WrappingUnfilteredPartitionIterator
             super(results);
             assert results.metadata().isStaticCompactTable();
             this.nowInSec = nowInSec;
+            this.builder = ArrayBackedRow.sortedBuilder(results.columns().regulars);
         }
 
         private void init()
         {
             assert !isInit;
-            this.staticRow = super.staticRow();
+            Row staticRow = super.staticRow();
             assert staticRow.columns().complexColumnCount() == 0;
 
-            this.nextToMerge = createReusableRow();
+            staticCells = staticRow.cells().iterator();
             updateNextToMerge();
             isInit = true;
         }
@@ -131,11 +132,6 @@ public class ThriftResultsMerger extends WrappingUnfilteredPartitionIterator
             return Rows.EMPTY_STATIC_ROW;
         }
 
-        private ReusableRow createReusableRow()
-        {
-            return new ReusableRow(metadata().clusteringColumns().size(), metadata().partitionColumns().regulars, true, metadata().isCounter());
-        }
-
         @Override
         public boolean hasNext()
         {
@@ -171,11 +167,9 @@ public class ThriftResultsMerger extends WrappingUnfilteredPartitionIterator
             if (cmp > 0)
                 return consumeNextWrapped();
 
-            // Same row, but we know the row has only a single column so just pick the more recent
+            // Same row, so merge them
             assert nextFromWrapped instanceof Row;
-            ReusableRow row = createReusableRow();
-            Rows.merge((Row)consumeNextWrapped(), consumeNextToMerge(), columns().regulars, row.writer(), nowInSec);
-            return row;
+            return Rows.merge((Row)consumeNextWrapped(), consumeNextToMerge(), nowInSec);
         }
 
         private Unfiltered consumeNextWrapped()
@@ -194,29 +188,26 @@ public class ThriftResultsMerger extends WrappingUnfilteredPartitionIterator
 
         private void updateNextToMerge()
         {
-            while (i < staticRow.columns().simpleColumnCount())
+            if (!staticCells.hasNext())
             {
-                Cell cell = staticRow.getCell(staticRow.columns().getSimple(i++));
-                if (cell != null)
-                {
-                    // Given a static cell, the equivalent row uses the column name as clustering and the
-                    // value as unique cell value.
-                    Row.Writer writer = nextToMerge.writer();
-                    writer.writeClusteringValue(cell.column().name.bytes);
-                    writer.writeCell(metadata().compactValueColumn(), cell.isCounterCell(), cell.value(), cell.livenessInfo(), cell.path());
-                    writer.endOfRow();
-                    return;
-                }
+                // Nothing more to merge.
+                nextToMerge = null;
+                return;
             }
-            // Nothing more to merge.
-            nextToMerge = null;
+
+            Cell cell = staticCells.next();
+
+            // Given a static cell, the equivalent row uses the column name as clustering and the value as unique cell value.
+            builder.newRow(new Clustering(cell.column().name.bytes));
+            builder.addCell(new BufferCell(metadata().compactValueColumn(), cell.timestamp(), cell.ttl(), cell.localDeletionTime(), cell.value(), cell.path()));
+            nextToMerge = builder.build();
         }
     }
 
-    private static class SuperColumnsPartitionMerger extends WrappingUnfilteredRowIterator
+    private static class SuperColumnsPartitionMerger extends AlteringUnfilteredRowIterator
     {
         private final int nowInSec;
-        private final ReusableRow reusableRow;
+        private final Row.Builder builder;
         private final ColumnDefinition superColumnMapColumn;
         private final AbstractType<?> columnComparator;
 
@@ -229,30 +220,23 @@ public class ThriftResultsMerger extends WrappingUnfilteredPartitionIterator
             this.superColumnMapColumn = results.metadata().compactValueColumn();
             assert superColumnMapColumn != null && superColumnMapColumn.type instanceof MapType;
 
-            this.reusableRow = new ReusableRow(results.metadata().clusteringColumns().size(),
-                                               Columns.of(superColumnMapColumn),
-                                               true,
-                                               results.metadata().isCounter());
+            this.builder = ArrayBackedRow.sortedBuilder(Columns.of(superColumnMapColumn));
             this.columnComparator = ((MapType)superColumnMapColumn.type).nameComparator();
         }
 
         @Override
-        public Unfiltered next()
+        protected Row computeNext(Row row)
         {
-            Unfiltered next = super.next();
-            if (next.kind() != Unfiltered.Kind.ROW)
-                return next;
-
-            Row row = (Row)next;
-            Row.Writer writer = reusableRow.writer();
-            row.clustering().writeTo(writer);
-
-            PeekingIterator<Cell> staticCells = Iterators.peekingIterator(makeStaticCellIterator(row));
+            PeekingIterator<Cell> staticCells = Iterators.peekingIterator(simpleCellsIterator(row));
             if (!staticCells.hasNext())
                 return row;
 
-            Iterator<Cell> cells = row.getCells(superColumnMapColumn);
-            PeekingIterator<Cell> dynamicCells = Iterators.peekingIterator(cells.hasNext() ? cells : Collections.<Cell>emptyIterator());
+            builder.newRow(row.clustering());
+
+            ComplexColumnData complexData = row.getComplexColumnData(superColumnMapColumn);
+            PeekingIterator<Cell> dynamicCells = Iterators.peekingIterator(complexData == null ? Collections.<Cell>emptyIterator() : complexData.iterator());
+
+            builder.addComplexDeletion(superColumnMapColumn, complexData.complexDeletion());
 
             while (staticCells.hasNext() && dynamicCells.hasNext())
             {
@@ -260,52 +244,37 @@ public class ThriftResultsMerger extends WrappingUnfilteredPartitionIterator
                 Cell dynamicCell = dynamicCells.peek();
                 int cmp = columnComparator.compare(staticCell.column().name.bytes, dynamicCell.path().get(0));
                 if (cmp < 0)
-                {
-                    staticCell = staticCells.next();
-                    writer.writeCell(superColumnMapColumn, staticCell.isCounterCell(), staticCell.value(), staticCell.livenessInfo(), CellPath.create(staticCell.column().name.bytes));
-                }
+                    builder.addCell(makeDynamicCell(staticCells.next()));
                 else if (cmp > 0)
-                {
-                    dynamicCells.next().writeTo(writer);
-                }
+                    builder.addCell(dynamicCells.next());
                 else
-                {
-                    staticCell = staticCells.next();
-                    Cell toMerge = Cells.create(superColumnMapColumn,
-                                                 staticCell.isCounterCell(),
-                                                 staticCell.value(),
-                                                 staticCell.livenessInfo(),
-                                                 CellPath.create(staticCell.column().name.bytes));
-                    Cells.reconcile(toMerge, dynamicCells.next(), nowInSec).writeTo(writer);
-                }
+                    builder.addCell(Cells.reconcile(makeDynamicCell(staticCells.next()), dynamicCells.next(), nowInSec));
             }
 
             while (staticCells.hasNext())
-            {
-                Cell staticCell = staticCells.next();
-                writer.writeCell(superColumnMapColumn, staticCell.isCounterCell(), staticCell.value(), staticCell.livenessInfo(), CellPath.create(staticCell.column().name.bytes));
-            }
+                builder.addCell(makeDynamicCell(staticCells.next()));
             while (dynamicCells.hasNext())
-            {
-                dynamicCells.next().writeTo(writer);
-            }
+                builder.addCell(dynamicCells.next());
+
+            return builder.build();
+        }
 
-            writer.endOfRow();
-            return reusableRow;
+        private Cell makeDynamicCell(Cell staticCell)
+        {
+            return new BufferCell(superColumnMapColumn, staticCell.timestamp(), staticCell.ttl(), staticCell.localDeletionTime(), staticCell.value(), CellPath.create(staticCell.column().name.bytes));
         }
 
-        private static Iterator<Cell> makeStaticCellIterator(final Row row)
+        private Iterator<Cell> simpleCellsIterator(Row row)
         {
+            final Iterator<Cell> cells = row.cells().iterator();
             return new AbstractIterator<Cell>()
             {
-                private int i;
-
                 protected Cell computeNext()
                 {
-                    while (i < row.columns().simpleColumnCount())
+                    if (cells.hasNext())
                     {
-                        Cell cell = row.getCell(row.columns().getSimple(i++));
-                        if (cell != null)
+                        Cell cell = cells.next();
+                        if (cell.column().isSimple())
                             return cell;
                     }
                     return endOfData();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
index 65ed23c..abc2a37 100644
--- a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
+++ b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
@@ -34,6 +34,7 @@ import java.util.UUID;
 
 import net.nicoulaj.compilecommand.annotations.Inline;
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.util.FileDataInput;
 import org.apache.cassandra.io.util.FileUtils;
@@ -290,6 +291,12 @@ public class ByteBufferUtil
         out.write(bytes);
     }
 
+    public static void writeWithVIntLength(ByteBuffer bytes, DataOutputPlus out) throws IOException
+    {
+        out.writeVInt(bytes.remaining());
+        out.write(bytes);
+    }
+
     public static void writeWithLength(byte[] bytes, DataOutput out) throws IOException
     {
         out.writeInt(bytes.length);
@@ -323,12 +330,36 @@ public class ByteBufferUtil
         return ByteBufferUtil.read(in, length);
     }
 
+    public static ByteBuffer readWithVIntLength(DataInputPlus in) throws IOException
+    {
+        int length = (int)in.readVInt();
+        if (length < 0)
+            throw new IOException("Corrupt (negative) value length encountered");
+
+        return ByteBufferUtil.read(in, length);
+    }
+
     public static int serializedSizeWithLength(ByteBuffer buffer)
     {
         int size = buffer.remaining();
         return TypeSizes.sizeof(size) + size;
     }
 
+    public static int serializedSizeWithVIntLength(ByteBuffer buffer)
+    {
+        int size = buffer.remaining();
+        return TypeSizes.sizeofVInt(size) + size;
+    }
+
+    public static void skipWithVIntLength(DataInputPlus in) throws IOException
+    {
+        int length = (int)in.readVInt();
+        if (length < 0)
+            throw new IOException("Corrupt (negative) value length encountered");
+
+        FileUtils.skipBytesFully(in, length);
+    }
+
     /* @return An unsigned short in an integer. */
     public static int readShortLength(DataInput in) throws IOException
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/utils/ObjectSizes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/ObjectSizes.java b/src/java/org/apache/cassandra/utils/ObjectSizes.java
index e05dcba..e7469c1 100644
--- a/src/java/org/apache/cassandra/utils/ObjectSizes.java
+++ b/src/java/org/apache/cassandra/utils/ObjectSizes.java
@@ -23,6 +23,8 @@ package org.apache.cassandra.utils;
 
 import java.nio.ByteBuffer;
 
+import java.util.ArrayList;
+
 import org.github.jamm.MemoryLayoutSpecification;
 import org.github.jamm.MemoryMeter;
 
@@ -111,6 +113,7 @@ public class ObjectSizes
     {
         return BUFFER_EMPTY_SIZE * array.length + sizeOfArray(array);
     }
+
     /**
      * Memory a byte buffer consumes
      * @param buffer ByteBuffer to calculate in memory size

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/utils/Sorting.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/Sorting.java b/src/java/org/apache/cassandra/utils/Sorting.java
deleted file mode 100644
index b1c0b46..0000000
--- a/src/java/org/apache/cassandra/utils/Sorting.java
+++ /dev/null
@@ -1,254 +0,0 @@
-/*
- * 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.utils;
-
-public abstract class Sorting
-{
-    private Sorting() {}
-
-    /**
-     * Interface that allows to sort elements addressable by index, but without actually requiring those
-     * to elements to be part of a list/array.
-     */
-    public interface Sortable
-    {
-        /**
-         * The number of elements to sort.
-         */
-        public int size();
-
-        /**
-         * Compares the element with index i should sort before the element with index j.
-         */
-        public int compare(int i, int j);
-
-        /**
-         * Swaps element i and j.
-         */
-        public void swap(int i, int j);
-    }
-
-    /**
-     * Sort a sortable.
-     *
-     * The actual algorithm is a direct adaptation of the standard sorting in golang
-     * at http://golang.org/src/pkg/sort/sort.go (comments included).
-     *
-     * It makes one call to data.Len to determine n, and O(n*log(n)) calls to
-     * data.Less and data.Swap. The sort is not guaranteed to be stable.
-     */
-    public static void sort(Sortable data)
-    {
-        // Switch to heapsort if depth of 2*ceil(lg(n+1)) is reached.
-        int n = data.size();
-        int maxDepth = 0;
-        for (int i = n; i > 0; i >>= 1)
-            maxDepth++;
-        maxDepth *= 2;
-        quickSort(data, 0, n, maxDepth);
-    }
-
-    private static void insertionSort(Sortable data, int a, int b)
-    {
-        for (int i = a + 1; i < b; i++)
-            for(int j = i; j > a && data.compare(j, j-1) < 0; j--)
-                data.swap(j, j-1);
-    }
-
-    // siftDown implements the heap property on data[lo, hi).
-    // first is an offset into the array where the root of the heap lies.
-    private static void siftDown(Sortable data, int lo, int hi, int first)
-    {
-        int root = lo;
-        while (true)
-        {
-            int child = 2*root + 1;
-            if (child >= hi)
-                return;
-
-            if (child + 1 < hi && data.compare(first+child, first+child+1) < 0)
-                child++;
-
-            if (data.compare(first+root, first+child) >= 0)
-                return;
-
-            data.swap(first+root, first+child);
-            root = child;
-        }
-    }
-
-    private static void heapSort(Sortable data, int a, int b)
-    {
-        int first = a;
-        int lo = 0;
-        int hi = b - a;
-
-        // Build heap with greatest element at top.
-        for (int i = (hi - 1) / 2; i >= 0; i--)
-            siftDown(data, i, hi, first);
-
-        // Pop elements, largest first, into end of data.
-        for (int i = hi - 1; i >= 0; i--) {
-            data.swap(first, first+i);
-            siftDown(data, lo, i, first);
-        }
-    }
-
-    // Quicksort, following Bentley and McIlroy,
-    // ``Engineering a Sort Function,'' SP&E November 1993.
-
-    // medianOfThree moves the median of the three values data[a], data[b], data[c] into data[a].
-    private static void medianOfThree(Sortable data, int a, int b, int c)
-    {
-        int m0 = b;
-        int m1 = a;
-        int m2 = c;
-        // bubble sort on 3 elements
-        if (data.compare(m1, m0) < 0)
-            data.swap(m1, m0);
-        if (data.compare(m2, m1) < 0)
-            data.swap(m2, m1);
-        if (data.compare(m1, m0) < 0)
-            data.swap(m1, m0);
-        // now data[m0] <= data[m1] <= data[m2]
-    }
-
-    private static void swapRange(Sortable data, int a, int b, int n)
-    {
-        for (int i = 0; i < n; i++)
-            data.swap(a+i, b+i);
-    }
-
-    private static void doPivot(Sortable data, int lo, int hi, int[] result)
-    {
-        int m = lo + (hi-lo)/2; // Written like this to avoid integer overflow.
-        if (hi-lo > 40) {
-            // Tukey's ``Ninther,'' median of three medians of three.
-            int s = (hi - lo) / 8;
-            medianOfThree(data, lo, lo+s, lo+2*s);
-            medianOfThree(data, m, m-s, m+s);
-            medianOfThree(data, hi-1, hi-1-s, hi-1-2*s);
-        }
-        medianOfThree(data, lo, m, hi-1);
-
-        // Invariants are:
-        //    data[lo] = pivot (set up by ChoosePivot)
-        //    data[lo <= i < a] = pivot
-        //    data[a <= i < b] < pivot
-        //    data[b <= i < c] is unexamined
-        //    data[c <= i < d] > pivot
-        //    data[d <= i < hi] = pivot
-        //
-        // Once b meets c, can swap the "= pivot" sections
-        // into the middle of the slice.
-        int pivot = lo;
-        int a = lo+1, b = lo+1, c = hi, d =hi;
-        while (true)
-        {
-            while (b < c)
-            {
-                int cmp = data.compare(b, pivot);
-                if (cmp < 0)  // data[b] < pivot
-                {
-                    b++;
-                }
-                else if (cmp == 0) // data[b] = pivot
-                {
-                    data.swap(a, b);
-                    a++;
-                    b++;
-                }
-                else
-                {
-                    break;
-                }
-            }
-
-            while (b < c)
-            {
-                int cmp = data.compare(pivot, c-1);
-                if (cmp < 0) // data[c-1] > pivot
-                {
-                    c--;
-                }
-                else if (cmp == 0) // data[c-1] = pivot
-                {
-                    data.swap(c-1, d-1);
-                    c--;
-                    d--;
-                }
-                else
-                {
-                    break;
-                }
-            }
-
-            if (b >= c)
-                break;
-
-            // data[b] > pivot; data[c-1] < pivot
-            data.swap(b, c-1);
-            b++;
-            c--;
-        }
-
-        int n = Math.min(b-a, a-lo);
-        swapRange(data, lo, b-n, n);
-
-        n = Math.min(hi-d, d-c);
-        swapRange(data, c, hi-n, n);
-
-        result[0] = lo + b - a;
-        result[1] = hi - (d - c);
-    }
-
-    private static void quickSort(Sortable data, int a, int b, int maxDepth)
-    {
-        int[] buffer = new int[2];
-
-        while (b-a > 7)
-        {
-            if (maxDepth == 0)
-            {
-                heapSort(data, a, b);
-                return;
-            }
-
-            maxDepth--;
-
-            doPivot(data, a, b, buffer);
-            int mlo = buffer[0];
-            int mhi = buffer[1];
-            // Avoiding recursion on the larger subproblem guarantees
-            // a stack depth of at most lg(b-a).
-            if (mlo-a < b-mhi)
-            {
-                quickSort(data, a, mlo, maxDepth);
-                a = mhi; // i.e., quickSort(data, mhi, b)
-            }
-            else
-            {
-                quickSort(data, mhi, b, maxDepth);
-                b = mlo; // i.e., quickSort(data, a, mlo)
-            }
-        }
-
-        if (b-a > 1)
-            insertionSort(data, a, b);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/utils/memory/AbstractAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/AbstractAllocator.java b/src/java/org/apache/cassandra/utils/memory/AbstractAllocator.java
index 0735d6e..a470527 100644
--- a/src/java/org/apache/cassandra/utils/memory/AbstractAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/AbstractAllocator.java
@@ -19,6 +19,11 @@ package org.apache.cassandra.utils.memory;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.Columns;
+import org.apache.cassandra.db.rows.ArrayBackedRow;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public abstract class AbstractAllocator
@@ -40,4 +45,32 @@ public abstract class AbstractAllocator
     }
 
     public abstract ByteBuffer allocate(int size);
+
+    public Row.Builder cloningArrayBackedRowBuilder(Columns columns)
+    {
+        return new CloningArrayBackedRowBuilder(columns, this);
+    }
+
+    private static class CloningArrayBackedRowBuilder extends ArrayBackedRow.SortedBuilder
+    {
+        private final AbstractAllocator allocator;
+
+        private CloningArrayBackedRowBuilder(Columns columns, AbstractAllocator allocator)
+        {
+            super(columns);
+            this.allocator = allocator;
+        }
+
+        @Override
+        public void newRow(Clustering clustering)
+        {
+            super.newRow(clustering.copy(allocator));
+        }
+
+        @Override
+        public void addCell(Cell cell)
+        {
+            super.addCell(cell.copy(allocator));
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java b/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java
index 443aca2..15499ae 100644
--- a/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java
@@ -59,8 +59,7 @@ public abstract class MemtableAllocator
         this.offHeap = offHeap;
     }
 
-    public abstract MemtableRowData.ReusableRow newReusableRow();
-    public abstract RowAllocator newRowAllocator(CFMetaData cfm, OpOrder.Group writeOp);
+    public abstract Row.Builder rowBuilder(CFMetaData metadata, OpOrder.Group opGroup, boolean isStatic);
     public abstract DecoratedKey clone(DecoratedKey key, OpOrder.Group opGroup);
     public abstract DataReclaimer reclaimer();
 
@@ -103,16 +102,10 @@ public abstract class MemtableAllocator
         return state == LifeCycle.LIVE;
     }
 
-    public static interface RowAllocator extends Row.Writer
-    {
-        public void allocateNewRow(int clusteringSize, Columns columns, boolean isStatic);
-        public MemtableRowData allocatedRowData();
-    }
-
     public static interface DataReclaimer
     {
-        public DataReclaimer reclaim(MemtableRowData row);
-        public DataReclaimer reclaimImmediately(MemtableRowData row);
+        public DataReclaimer reclaim(Row row);
+        public DataReclaimer reclaimImmediately(Row row);
         public DataReclaimer reclaimImmediately(DecoratedKey key);
         public void cancel();
         public void commit();
@@ -120,12 +113,12 @@ public abstract class MemtableAllocator
 
     public static final DataReclaimer NO_OP = new DataReclaimer()
     {
-        public DataReclaimer reclaim(MemtableRowData update)
+        public DataReclaimer reclaim(Row update)
         {
             return this;
         }
 
-        public DataReclaimer reclaimImmediately(MemtableRowData update)
+        public DataReclaimer reclaimImmediately(Row update)
         {
             return this;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/utils/memory/MemtableBufferAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/MemtableBufferAllocator.java b/src/java/org/apache/cassandra/utils/memory/MemtableBufferAllocator.java
index 144f439..31df444 100644
--- a/src/java/org/apache/cassandra/utils/memory/MemtableBufferAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/MemtableBufferAllocator.java
@@ -27,20 +27,15 @@ import org.apache.cassandra.utils.concurrent.OpOrder;
 
 public abstract class MemtableBufferAllocator extends MemtableAllocator
 {
-
     protected MemtableBufferAllocator(SubAllocator onHeap, SubAllocator offHeap)
     {
         super(onHeap, offHeap);
     }
 
-    public MemtableRowData.ReusableRow newReusableRow()
+    public Row.Builder rowBuilder(CFMetaData metadata, OpOrder.Group writeOp, boolean isStatic)
     {
-        return MemtableRowData.BufferRowData.createReusableRow();
-    }
-
-    public RowAllocator newRowAllocator(CFMetaData cfm, OpOrder.Group writeOp)
-    {
-        return new RowBufferAllocator(allocator(writeOp), cfm.isCounter());
+        Columns columns = isStatic ? metadata.partitionColumns().statics : metadata.partitionColumns().regulars;
+        return allocator(writeOp).cloningArrayBackedRowBuilder(columns);
     }
 
     public DecoratedKey clone(DecoratedKey key, OpOrder.Group writeOp)
@@ -54,71 +49,4 @@ public abstract class MemtableBufferAllocator extends MemtableAllocator
     {
         return new ContextAllocator(writeOp, this);
     }
-
-    private static class RowBufferAllocator extends RowDataBlock.Writer implements RowAllocator
-    {
-        private final AbstractAllocator allocator;
-        private final boolean isCounter;
-
-        private MemtableRowData.BufferClustering clustering;
-        private int clusteringIdx;
-        private LivenessInfo info;
-        private DeletionTime deletion;
-        private RowDataBlock data;
-
-        private RowBufferAllocator(AbstractAllocator allocator, boolean isCounter)
-        {
-            super(true);
-            this.allocator = allocator;
-            this.isCounter = isCounter;
-        }
-
-        public void allocateNewRow(int clusteringSize, Columns columns, boolean isStatic)
-        {
-            data = new RowDataBlock(columns, 1, false, isCounter);
-            clustering = isStatic ? null : new MemtableRowData.BufferClustering(clusteringSize);
-            clusteringIdx = 0;
-            updateWriter(data);
-        }
-
-        public MemtableRowData allocatedRowData()
-        {
-            MemtableRowData row = new MemtableRowData.BufferRowData(clustering == null ? Clustering.STATIC_CLUSTERING : clustering,
-                                                                    info,
-                                                                    deletion,
-                                                                    data);
-
-            clustering = null;
-            info = LivenessInfo.NONE;
-            deletion = DeletionTime.LIVE;
-            data = null;
-
-            return row;
-        }
-
-        public void writeClusteringValue(ByteBuffer value)
-        {
-            clustering.setClusteringValue(clusteringIdx++, value == null ? null : allocator.clone(value));
-        }
-
-        public void writePartitionKeyLivenessInfo(LivenessInfo info)
-        {
-            this.info = info;
-        }
-
-        public void writeRowDeletion(DeletionTime deletion)
-        {
-            this.deletion = deletion;
-        }
-
-        @Override
-        public void writeCell(ColumnDefinition column, boolean isCounter, ByteBuffer value, LivenessInfo info, CellPath path)
-        {
-            ByteBuffer v = allocator.clone(value);
-            if (column.isComplex())
-                complexWriter.addCell(column, v, info, MemtableRowData.BufferCellPath.clone(path, allocator));
-            else
-                simpleWriter.addCell(column, v, info);
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java b/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
index 7ca859d..7b95430 100644
--- a/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
@@ -27,7 +27,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.NativeDecoratedKey;
-import org.apache.cassandra.db.rows.MemtableRowData;
+import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
 public class NativeAllocator extends MemtableAllocator
@@ -53,13 +53,7 @@ public class NativeAllocator extends MemtableAllocator
         super(pool.onHeap.newAllocator(), pool.offHeap.newAllocator());
     }
 
-    public MemtableRowData.ReusableRow newReusableRow()
-    {
-        // TODO
-        throw new UnsupportedOperationException();
-    }
-
-    public RowAllocator newRowAllocator(CFMetaData cfm, OpOrder.Group writeOp)
+    public Row.Builder rowBuilder(CFMetaData metadata, OpOrder.Group opGroup, boolean isStatic)
     {
         // TODO
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/data/corrupt-sstables/la-1-big-CRC.db
----------------------------------------------------------------------
diff --git a/test/data/corrupt-sstables/la-1-big-CRC.db b/test/data/corrupt-sstables/la-1-big-CRC.db
index f13b9c7..1a0c525 100644
Binary files a/test/data/corrupt-sstables/la-1-big-CRC.db and b/test/data/corrupt-sstables/la-1-big-CRC.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/data/corrupt-sstables/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/corrupt-sstables/la-1-big-Data.db b/test/data/corrupt-sstables/la-1-big-Data.db
index dc516d8..e6c5eb9 100644
Binary files a/test/data/corrupt-sstables/la-1-big-Data.db and b/test/data/corrupt-sstables/la-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/data/corrupt-sstables/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/corrupt-sstables/la-1-big-Digest.adler32 b/test/data/corrupt-sstables/la-1-big-Digest.adler32
index e447277..93deb45 100644
--- a/test/data/corrupt-sstables/la-1-big-Digest.adler32
+++ b/test/data/corrupt-sstables/la-1-big-Digest.adler32
@@ -1 +1 @@
-2370519993
\ No newline at end of file
+3942663153
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/data/corrupt-sstables/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/corrupt-sstables/la-1-big-Index.db b/test/data/corrupt-sstables/la-1-big-Index.db
index 178221e..6e5e352 100644
Binary files a/test/data/corrupt-sstables/la-1-big-Index.db and b/test/data/corrupt-sstables/la-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/data/corrupt-sstables/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/corrupt-sstables/la-1-big-Statistics.db b/test/data/corrupt-sstables/la-1-big-Statistics.db
index 23b76ac..15220e0 100644
Binary files a/test/data/corrupt-sstables/la-1-big-Statistics.db and b/test/data/corrupt-sstables/la-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/data/corrupt-sstables/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/corrupt-sstables/la-1-big-TOC.txt b/test/data/corrupt-sstables/la-1-big-TOC.txt
index 9cbcd44..9ad71ef 100644
--- a/test/data/corrupt-sstables/la-1-big-TOC.txt
+++ b/test/data/corrupt-sstables/la-1-big-TOC.txt
@@ -1,8 +1,8 @@
 Statistics.db
-Filter.db
-Data.db
-Summary.db
-Digest.adler32
 CRC.db
 TOC.txt
+Data.db
 Index.db
+Summary.db
+Digest.adler32
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
index e1dd953..8a63a27 100644
--- a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
+++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
@@ -487,7 +487,7 @@ public class CommitLogStressTest
                     if (!(UTF8Type.instance.compose(row.clustering().get(0)).startsWith("name")))
                         continue;
 
-                    for (Cell cell : row)
+                    for (Cell cell : row.cells())
                     {
                         hash = hash(hash, cell.value());
                         ++cells;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index ce6ac22..12305ef 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -319,7 +319,7 @@ public class SchemaLoader
                 .addPartitionKey("key", AsciiType.instance)
                 .build();
 
-        return cfm.addOrReplaceColumnDefinition(ColumnDefinition.regularDef(ksName, cfName, "indexed", AsciiType.instance, null)
+        return cfm.addOrReplaceColumnDefinition(ColumnDefinition.regularDef(ksName, cfName, "indexed", AsciiType.instance)
                                                                 .setIndex("indexe1", IndexType.CUSTOM, indexOptions));
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index 423b3c0..654b8c6 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -340,7 +340,7 @@ public class Util
                 assert !iterator.hasNext() : "Expecting a single partition but got more";
 
                 assert partition.hasNext() : "Expecting one row in one partition but got an empty partition";
-                Row row = ((Row)partition.next()).takeAlias();
+                Row row = ((Row)partition.next());
                 assert !partition.hasNext() : "Expecting a single row but got more";
                 return row;
             }
@@ -356,7 +356,7 @@ public class Util
             {
                 assert !iterator.hasNext() : "Expecting a single partition but got more";
                 assert partition.hasNext() : "Expecting one row in one partition but got an empty partition";
-                Row row = partition.next().takeAlias();
+                Row row = partition.next();
                 assert !partition.hasNext() : "Expecting a single row but got more";
                 return row;
             }
@@ -444,10 +444,22 @@ public class Util
         return CBuilder.create(new ClusteringComparator(types));
     }
 
+    public static boolean equal(UnfilteredRowIterator a, UnfilteredRowIterator b)
+    {
+        return Objects.equals(a.columns(), b.columns())
+            && Objects.equals(a.metadata(), b.metadata())
+            && Objects.equals(a.isReverseOrder(), b.isReverseOrder())
+            && Objects.equals(a.partitionKey(), b.partitionKey())
+            && Objects.equals(a.partitionLevelDeletion(), b.partitionLevelDeletion())
+            && Objects.equals(a.staticRow(), b.staticRow())
+            && Objects.equals(a.stats(), b.stats())
+            && Iterators.elementsEqual(a, b);
+    }
+
     // moved & refactored from KeyspaceTest in < 3.0
     public static void assertColumns(Row row, String... expectedColumnNames)
     {
-        Iterator<Cell> cells = row == null ? Iterators.<Cell>emptyIterator() : row.iterator();
+        Iterator<Cell> cells = row == null ? Iterators.<Cell>emptyIterator() : row.cells().iterator();
         String[] actual = Iterators.toArray(Iterators.transform(cells, new Function<Cell, String>()
         {
             public String apply(Cell cell)
@@ -472,7 +484,7 @@ public class Util
     {
         assertNotNull(cell);
         assertEquals(0, ByteBufferUtil.compareUnsigned(cell.value(), ByteBufferUtil.bytes(value)));
-        assertEquals(timestamp, cell.livenessInfo().timestamp());
+        assertEquals(timestamp, cell.timestamp());
     }
 
     public static void assertClustering(CFMetaData cfm, Row row, Object... clusteringValue)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java b/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
index fb5d84f..71b87f9 100644
--- a/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
+++ b/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
@@ -56,7 +56,7 @@ public class AutoSavingCacheTest
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1);
         for (int i = 0; i < 2; i++)
         {
-            ColumnDefinition colDef = new ColumnDefinition(cfs.metadata, ByteBufferUtil.bytes("col1"), AsciiType.instance, 0, ColumnDefinition.Kind.REGULAR);
+            ColumnDefinition colDef = ColumnDefinition.regularDef(cfs.metadata, ByteBufferUtil.bytes("col1"), AsciiType.instance);
             RowUpdateBuilder rowBuilder = new RowUpdateBuilder(cfs.metadata, System.currentTimeMillis(), "key1");
             rowBuilder.add(colDef, "val1");
             rowBuilder.build().apply();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java b/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
index 0e5e192..c875165 100644
--- a/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
+++ b/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
@@ -38,10 +38,10 @@ public class ColumnDefinitionTest
                          .addRegularColumn("val", AsciiType.instance)
                          .build();
 
-        ColumnDefinition cd0 = ColumnDefinition.staticDef(cfm, ByteBufferUtil.bytes("TestColumnDefinitionName0"), BytesType.instance, null)
+        ColumnDefinition cd0 = ColumnDefinition.staticDef(cfm, ByteBufferUtil.bytes("TestColumnDefinitionName0"), BytesType.instance)
                                                .setIndex("random index name 0", IndexType.KEYS, null);
 
-        ColumnDefinition cd1 = ColumnDefinition.staticDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition1"), LongType.instance, null);
+        ColumnDefinition cd1 = ColumnDefinition.staticDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition1"), LongType.instance);
 
         testSerializeDeserialize(cfm, cd0);
         testSerializeDeserialize(cfm, cd1);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java b/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
index 9a768de..71524c5 100644
--- a/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
@@ -25,19 +25,20 @@ import org.junit.Test;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.LivenessInfo;
-import org.apache.cassandra.db.rows.AbstractCell;
+import org.apache.cassandra.db.rows.BufferCell;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.CellPath;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.serializers.Int32Serializer;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.UUIDGen;
 
 import static org.junit.Assert.*;
 
 public class ColumnConditionTest
 {
-    public static ByteBuffer UNSET_BYTE_BUFFER = ByteBuffer.wrap(new byte[]{});
+    private static final CellPath LIST_PATH = CellPath.create(ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes()));
 
     public static final ByteBuffer ZERO = Int32Type.instance.fromString("0");
     public static final ByteBuffer ONE = Int32Type.instance.fromString("1");
@@ -51,12 +52,17 @@ public class ColumnConditionTest
         Cell cell = null;
         if (columnValue != null)
         {
-            ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", "c", ListType.getInstance(Int32Type.instance, true), null);
-            cell = new TestCell(definition, null, columnValue, LivenessInfo.NONE);
+            ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", "c", ListType.getInstance(Int32Type.instance, true));
+            cell = testCell(definition, columnValue, LIST_PATH);
         }
         return bound.isSatisfiedByValue(conditionValue, cell, Int32Type.instance, bound.operator);
     }
 
+    private static Cell testCell(ColumnDefinition column, ByteBuffer value, CellPath path)
+    {
+        return new BufferCell(column, 0L, Cell.NO_TTL, Cell.NO_DELETION_TIME, value, path);
+    }
+
     private static void assertThrowsIRE(ColumnCondition.Bound bound, ByteBuffer conditionValue, ByteBuffer columnValue)
     {
         try
@@ -69,7 +75,7 @@ public class ColumnConditionTest
     @Test
     public void testSimpleBoundIsSatisfiedByValue() throws InvalidRequestException
     {
-        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", "c", ListType.getInstance(Int32Type.instance, true), null);
+        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", "c", ListType.getInstance(Int32Type.instance, true));
 
         // EQ
         ColumnCondition condition = ColumnCondition.condition(definition, new Constants.Value(ONE), Operator.EQ);
@@ -157,7 +163,7 @@ public class ColumnConditionTest
         Map<ByteBuffer, CollectionType> typeMap = new HashMap<>();
         typeMap.put(ByteBufferUtil.bytes("c"), ListType.getInstance(Int32Type.instance, true));
 
-        ColumnDefinition definition = new ColumnDefinition(cfm, ByteBufferUtil.bytes("c"), ListType.getInstance(Int32Type.instance, true), 0, ColumnDefinition.Kind.REGULAR);
+        ColumnDefinition definition = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("c"), ListType.getInstance(Int32Type.instance, true));
 
         List<Cell> cells = new ArrayList<>(columnValues.size());
         if (columnValues != null)
@@ -166,7 +172,7 @@ public class ColumnConditionTest
             {
                 ByteBuffer key = Int32Serializer.instance.serialize(i);
                 ByteBuffer value = columnValues.get(i);
-                cells.add(new TestCell(definition, CellPath.create(key), value, LivenessInfo.NONE));
+                cells.add(testCell(definition, value, CellPath.create(key)));
             };
         }
 
@@ -177,7 +183,7 @@ public class ColumnConditionTest
     // sets use the same check as lists
     public void testListCollectionBoundAppliesTo() throws InvalidRequestException
     {
-        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", "c", ListType.getInstance(Int32Type.instance, true), null);
+        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", "c", ListType.getInstance(Int32Type.instance, true));
 
         // EQ
         ColumnCondition condition = ColumnCondition.condition(definition, null, new Lists.Value(Arrays.asList(ONE)), Operator.EQ);
@@ -288,7 +294,7 @@ public class ColumnConditionTest
         CFMetaData cfm = CFMetaData.compile("create table foo(a int PRIMARY KEY, b int, c set<int>)", "ks");
         Map<ByteBuffer, CollectionType> typeMap = new HashMap<>();
         typeMap.put(ByteBufferUtil.bytes("c"), SetType.getInstance(Int32Type.instance, true));
-        ColumnDefinition definition = new ColumnDefinition(cfm, ByteBufferUtil.bytes("c"), SetType.getInstance(Int32Type.instance, true), 0, ColumnDefinition.Kind.REGULAR);
+        ColumnDefinition definition = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("c"), SetType.getInstance(Int32Type.instance, true));
 
         List<Cell> cells = new ArrayList<>(columnValues.size());
         if (columnValues != null)
@@ -296,7 +302,7 @@ public class ColumnConditionTest
             for (int i = 0; i < columnValues.size(); i++)
             {
                 ByteBuffer key = columnValues.get(i);
-                cells.add(new TestCell(definition, CellPath.create(key), ByteBufferUtil.EMPTY_BYTE_BUFFER, LivenessInfo.NONE));
+                cells.add(testCell(definition, ByteBufferUtil.EMPTY_BYTE_BUFFER, CellPath.create(key)));
             };
         }
 
@@ -306,7 +312,7 @@ public class ColumnConditionTest
     @Test
     public void testSetCollectionBoundAppliesTo() throws InvalidRequestException
     {
-        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", "c", ListType.getInstance(Int32Type.instance, true), null);
+        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", "c", ListType.getInstance(Int32Type.instance, true));
 
         // EQ
         ColumnCondition condition = ColumnCondition.condition(definition, null, new Sets.Value(set(ONE)), Operator.EQ);
@@ -420,13 +426,13 @@ public class ColumnConditionTest
         CFMetaData cfm = CFMetaData.compile("create table foo(a int PRIMARY KEY, b map<int, int>)", "ks");
         Map<ByteBuffer, CollectionType> typeMap = new HashMap<>();
         typeMap.put(ByteBufferUtil.bytes("b"), MapType.getInstance(Int32Type.instance, Int32Type.instance, true));
-        ColumnDefinition definition = new ColumnDefinition(cfm, ByteBufferUtil.bytes("b"), MapType.getInstance(Int32Type.instance, Int32Type.instance, true), 0, ColumnDefinition.Kind.REGULAR);
+        ColumnDefinition definition = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("b"), MapType.getInstance(Int32Type.instance, Int32Type.instance, true));
 
         List<Cell> cells = new ArrayList<>(columnValues.size());
         if (columnValues != null)
         {
             for (Map.Entry<ByteBuffer, ByteBuffer> entry : columnValues.entrySet())
-                cells.add(new TestCell(definition, CellPath.create(entry.getKey()), entry.getValue(), LivenessInfo.NONE));
+                cells.add(testCell(definition, entry.getValue(), CellPath.create(entry.getKey())));
         }
 
         return bound.mapAppliesTo(MapType.getInstance(Int32Type.instance, Int32Type.instance, true), cells.iterator(), conditionValues, bound.operator);
@@ -435,7 +441,7 @@ public class ColumnConditionTest
     @Test
     public void testMapCollectionBoundIsSatisfiedByValue() throws InvalidRequestException
     {
-        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", "c", ListType.getInstance(Int32Type.instance, true), null);
+        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", "c", ListType.getInstance(Int32Type.instance, true));
 
         Map<ByteBuffer, ByteBuffer> placeholderMap = new TreeMap<>();
         placeholderMap.put(ONE, ONE);
@@ -573,45 +579,4 @@ public class ColumnConditionTest
         assertTrue(mapAppliesTo(bound, map(ByteBufferUtil.EMPTY_BYTE_BUFFER, ONE), map(ByteBufferUtil.EMPTY_BYTE_BUFFER, ONE)));
         assertTrue(mapAppliesTo(bound, map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER)));
     }
-
-    static class TestCell extends AbstractCell
-    {
-        private final ColumnDefinition column;
-        private final CellPath path;
-        private final ByteBuffer value;
-        private final LivenessInfo info;
-
-        public TestCell(ColumnDefinition column, CellPath path, ByteBuffer value, LivenessInfo info)
-        {
-            this.column = column;
-            this.path = path;
-            this.value = value;
-            this.info = info.takeAlias();
-        }
-
-        public ColumnDefinition column()
-        {
-            return column;
-        }
-
-        public boolean isCounterCell()
-        {
-            return false;
-        }
-
-        public ByteBuffer value()
-        {
-            return value;
-        }
-
-        public LivenessInfo livenessInfo()
-        {
-            return info;
-        }
-
-        public CellPath path()
-        {
-            return path;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java b/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java
index ad0dd7b..052b53d 100644
--- a/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java
+++ b/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java
@@ -307,7 +307,7 @@ public class SimpleQueryTest extends CQLTester
         createTable("CREATE TABLE %s (k text, t int, v1 text, v2 int, PRIMARY KEY (k, t));");
 
         for (int t = 0; t < N; t++)
-                execute("INSERT INTO %s (k, t, v1, v2) values (?, ?, ?, ?)", "key", t, "v" + t, t + 10);
+            execute("INSERT INTO %s (k, t, v1, v2) values (?, ?, ?, ?)", "key", t, "v" + t, t + 10);
 
         flush();
 
@@ -434,7 +434,7 @@ public class SimpleQueryTest extends CQLTester
     @Test
     public void collectionDeletionTest() throws Throwable
     {
-        createTable("CREATE TABLE %s (k text PRIMARY KEY, s set<int>);");
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s set<int>);");
 
         execute("INSERT INTO %s (k, s) VALUES (?, ?)", 1, set(1));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/test/unit/org/apache/cassandra/db/CellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CellTest.java b/test/unit/org/apache/cassandra/db/CellTest.java
index 9560804..e8cb1cb 100644
--- a/test/unit/org/apache/cassandra/db/CellTest.java
+++ b/test/unit/org/apache/cassandra/db/CellTest.java
@@ -60,12 +60,12 @@ public class CellTest
                 // don't test equality for both sides native, as this is based on CellName resolution
                 if (lhs && rhs)
                     continue;
-                Cell a = buildCell(cfm, "a", "a", 1, 1);
-                Cell b = buildCell(cfm, "a", "a", 1, 0);
+                Cell a = expiring(cfm, "val", "a", 1, 1);
+                Cell b = regular(cfm, "val", "a", 1);
                 Assert.assertNotSame(a, b);
                 Assert.assertNotSame(b, a);
 
-                a = deleted(cfm, "a", "a", 1, 1);
+                a = deleted(cfm, "val", 1, 1);
                 Assert.assertNotSame(a, b);
                 Assert.assertNotSame(b, a);
             }
@@ -76,18 +76,18 @@ public class CellTest
     public void testExpiringCellReconile()
     {
         // equal
-        Assert.assertEquals(0, testExpiring("a", "a", 1, 1, null, null, null, null));
+        Assert.assertEquals(0, testExpiring("val", "a", 1, 1, null, null, null, null));
 
         // newer timestamp
-        Assert.assertEquals(-1, testExpiring("a", "a", 2, 1, null, null, 1L, null));
-        Assert.assertEquals(-1, testExpiring("a", "a", 2, 1, null, "b", 1L, 2));
+        Assert.assertEquals(-1, testExpiring("val", "a", 2, 1, null, null, 1L, null));
+        Assert.assertEquals(-1, testExpiring("val", "a", 2, 1, null, "val", 1L, 2));
 
-        Assert.assertEquals(-1, testExpiring("a", "a", 1, 2, null, null, null, 1));
-        Assert.assertEquals(1, testExpiring("a", "a", 1, 2, null, "b", null, 1));
+        Assert.assertEquals(-1, testExpiring("val", "a", 1, 2, null, null, null, 1));
+        Assert.assertEquals(1, testExpiring("val", "a", 1, 2, null, "val", null, 1));
 
         // newer value
-        Assert.assertEquals(-1, testExpiring("a", "b", 2, 1, null, "a", null, null));
-        Assert.assertEquals(-1, testExpiring("a", "b", 2, 1, null, "a", null, 2));
+        Assert.assertEquals(-1, testExpiring("val", "b", 2, 1, null, "a", null, null));
+        Assert.assertEquals(-1, testExpiring("val", "b", 2, 1, null, "a", null, 2));
     }
 
     private int testExpiring(String n1, String v1, long t1, int et1, String n2, String v2, Long t2, Integer et2)
@@ -100,8 +100,8 @@ public class CellTest
             t2 = t1;
         if (et2 == null)
             et2 = et1;
-        Cell c1 = buildCell(cfm, n1, v1, t1, et1);
-        Cell c2 = buildCell(cfm, n2, v2, t2, et2);
+        Cell c1 = expiring(cfm, n1, v1, t1, et1);
+        Cell c2 = expiring(cfm, n2, v2, t2, et2);
 
         int now = FBUtilities.nowInSeconds();
         if (Cells.reconcile(c1, c2, now) == c1)
@@ -109,56 +109,21 @@ public class CellTest
         return Cells.reconcile(c2, c1, now) == c2 ? 1 : 0;
     }
 
-    private Cell buildCell(CFMetaData cfm, String columnName, String value, long timestamp, int ttl)
+    private Cell regular(CFMetaData cfm, String columnName, String value, long timestamp)
     {
         ColumnDefinition cdef = cfm.getColumnDefinition(ByteBufferUtil.bytes(columnName));
-        LivenessInfo info = SimpleLivenessInfo.forUpdate(timestamp, ttl, FBUtilities.nowInSeconds(), cfm);
-        return new TestCell(cdef, ByteBufferUtil.bytes(value), info);
+        return BufferCell.live(cfm, cdef, timestamp, ByteBufferUtil.bytes(value));
     }
 
-    private Cell deleted(CFMetaData cfm, String columnName, String value, int localDeletionTime, long timestamp)
+    private Cell expiring(CFMetaData cfm, String columnName, String value, long timestamp, int localExpirationTime)
     {
         ColumnDefinition cdef = cfm.getColumnDefinition(ByteBufferUtil.bytes(columnName));
-        LivenessInfo info = SimpleLivenessInfo.forDeletion(timestamp, localDeletionTime);
-        return new TestCell(cdef, ByteBufferUtil.bytes(value), info);
+        return new BufferCell(cdef, timestamp, 1, localExpirationTime, ByteBufferUtil.bytes(value), null);
     }
 
-    public static class TestCell extends AbstractCell
+    private Cell deleted(CFMetaData cfm, String columnName, int localDeletionTime, long timestamp)
     {
-        private final ColumnDefinition column;
-        private final ByteBuffer value;
-        private final LivenessInfo info;
-
-        public TestCell(ColumnDefinition column, ByteBuffer value, LivenessInfo info)
-        {
-            this.column = column;
-            this.value = value;
-            this.info = info.takeAlias();
-        }
-
-        public ColumnDefinition column()
-        {
-            return column;
-        }
-
-        public boolean isCounterCell()
-        {
-            return false;
-        }
-
-        public ByteBuffer value()
-        {
-            return value;
-        }
-
-        public LivenessInfo livenessInfo()
-        {
-            return info;
-        }
-
-        public CellPath path()
-        {
-            return null;
-        }
+        ColumnDefinition cdef = cfm.getColumnDefinition(ByteBufferUtil.bytes(columnName));
+        return BufferCell.tombstone(cdef, timestamp, localDeletionTime);
     }
 }


[07/15] cassandra git commit: Simplify some 8099's implementations

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/ColumnData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/ColumnData.java b/src/java/org/apache/cassandra/db/rows/ColumnData.java
index ea472eb..95bad48 100644
--- a/src/java/org/apache/cassandra/db/rows/ColumnData.java
+++ b/src/java/org/apache/cassandra/db/rows/ColumnData.java
@@ -17,45 +17,66 @@
  */
 package org.apache.cassandra.db.rows;
 
-import java.util.Iterator;
+import java.security.MessageDigest;
+import java.util.Comparator;
 
 import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.DeletionPurger;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.serializers.MarshalException;
 
-public class ColumnData
+/**
+ * Generic interface for the data of a given column (inside a row).
+ *
+ * In practice, there is only 2 implementations of this: either {@link Cell} for simple columns
+ * or {@code ComplexColumnData} for complex columns.
+ */
+public interface ColumnData
 {
-    private final ColumnDefinition column;
-    private final Cell cell;
-    private final Iterator<Cell> cells;
-    private final DeletionTime complexDeletion;
-
-    ColumnData(ColumnDefinition column, Cell cell, Iterator<Cell> cells, DeletionTime complexDeletion)
-    {
-        assert column != null && (cell != null || (column.isComplex() && cells != null && complexDeletion != null));
-
-        this.column = column;
-        this.cell = cell;
-        this.cells = cells;
-        this.complexDeletion = complexDeletion;
-    }
-
-    public ColumnDefinition column()
-    {
-        return column;
-    }
-
-    public Cell cell()
-    {
-        return cell;
-    }
-
-    public Iterator<Cell> cells()
-    {
-        return cells;
-    }
-
-    public DeletionTime complexDeletion()
-    {
-        return complexDeletion;
-    }
+    public static final Comparator<ColumnData> comparator = (cd1, cd2) -> cd1.column().compareTo(cd2.column());
+
+    // A comparator for the cells of the *similar* ColumnData, i.e. one that assumes the cells are all for the same column.
+    public static final Comparator<Cell> cellComparator = (c1, c2) -> c1.column().cellPathComparator().compare(c1.path(), c2.path());
+
+    /**
+     * The column this is data for.
+     *
+     * @return the column this is a data for.
+     */
+    public ColumnDefinition column();
+
+    /**
+     * The size of the data hold by this {@code ColumnData}.
+     *
+     * @return the size used by the data of this {@code ColumnData}.
+     */
+    public int dataSize();
+
+    public long unsharedHeapSizeExcludingData();
+
+    /**
+     * Validate the column data.
+     *
+     * @throws MarshalException if the data is not valid.
+     */
+    public void validate();
+
+    /**
+     * Adds the data to the provided digest.
+     *
+     * @param digest the {@code MessageDigest} to add the data to.
+     */
+    public void digest(MessageDigest digest);
+
+    /**
+     * Returns a copy of the data where all timestamps for live data have replaced by {@code newTimestamp} and
+     * all deletion timestamp by {@code newTimestamp - 1}.
+     *
+     * This exists for the Paxos path, see {@link PartitionUpdate#updateAllTimestamp} for additional details.
+     */
+    public ColumnData updateAllTimestamp(long newTimestamp);
+
+    public ColumnData markCounterLocalToBeCleared();
+
+    public ColumnData purge(DeletionPurger purger, int nowInSec);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java b/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
new file mode 100644
index 0000000..d87402a
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
@@ -0,0 +1,343 @@
+/*
+ * 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.rows;
+
+import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+import java.util.*;
+
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.DeletionPurger;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.marshal.ByteType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.utils.ObjectSizes;
+
+/**
+ * The data for a complex column, that is it's cells and potential complex
+ * deletion time.
+ */
+public class ComplexColumnData implements ColumnData, Iterable<Cell>
+{
+    static final Cell[] NO_CELLS = new Cell[0];
+
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new ComplexColumnData(ColumnDefinition.regularDef("", "", "", SetType.getInstance(ByteType.instance, true)), NO_CELLS, new DeletionTime(0, 0)));
+
+    private final ColumnDefinition column;
+
+    // The cells for 'column' sorted by cell path.
+    private final Cell[] cells;
+
+    private final DeletionTime complexDeletion;
+
+    // Only ArrayBackedRow should call this.
+    ComplexColumnData(ColumnDefinition column, Cell[] cells, DeletionTime complexDeletion)
+    {
+        assert column.isComplex();
+        assert cells.length > 0 || !complexDeletion.isLive();
+        this.column = column;
+        this.cells = cells;
+        this.complexDeletion = complexDeletion;
+    }
+
+    public boolean hasCells()
+    {
+        return cellsCount() > 0;
+    }
+
+    public int cellsCount()
+    {
+        return cells.length;
+    }
+
+    public ColumnDefinition column()
+    {
+        return column;
+    }
+
+    public Cell getCell(CellPath path)
+    {
+        int idx = binarySearch(path);
+        return idx < 0 ? null : cells[idx];
+    }
+
+    public Cell getCellByIndex(int i)
+    {
+        assert 0 <= i && i < cells.length;
+        return cells[i];
+    }
+
+    /**
+     * The complex deletion time of the complex column.
+     * <p>
+     * The returned "complex deletion" is a deletion of all the cells of the column. For instance,
+     * for a collection, this correspond to a full collection deletion.
+     * Please note that this deletion says nothing about the individual cells of the complex column:
+     * there can be no complex deletion but some of the individual cells can be deleted.
+     *
+     * @return the complex deletion time for the column this is the data of or {@code DeletionTime.LIVE}
+     * if the column is not deleted.
+     */
+    public DeletionTime complexDeletion()
+    {
+        return complexDeletion;
+    }
+
+    public Iterator<Cell> iterator()
+    {
+        return Iterators.forArray(cells);
+    }
+
+    public int dataSize()
+    {
+        int size = complexDeletion.dataSize();
+        for (Cell cell : cells)
+            size += cell.dataSize();
+        return size;
+    }
+
+    public long unsharedHeapSizeExcludingData()
+    {
+        long heapSize = EMPTY_SIZE + ObjectSizes.sizeOfArray(cells);
+        for (Cell cell : cells)
+            heapSize += cell.unsharedHeapSizeExcludingData();
+        return heapSize;
+    }
+
+    public void validate()
+    {
+        for (Cell cell : cells)
+            cell.validate();
+    }
+
+    public ComplexColumnData filter(ColumnFilter filter, DeletionTime activeDeletion, CFMetaData.DroppedColumn dropped)
+    {
+        ColumnFilter.Tester cellTester = filter.newTester(column);
+        if (cellTester == null && activeDeletion.isLive() && dropped == null)
+            return this;
+
+        DeletionTime newComplexDeletion = activeDeletion.supersedes(complexDeletion) ? DeletionTime.LIVE : complexDeletion;
+
+        int newSize = 0;
+        for (Cell cell : cells)
+        {
+            // The cell must be:
+            //   - Included by the query
+            //   - not shadowed by the active deletion
+            //   - not being for a dropped column
+            if ((cellTester == null || cellTester.includes(cell.path()))
+                 && !activeDeletion.deletes(cell)
+                 && (dropped == null || cell.timestamp() > dropped.droppedTime))
+                ++newSize;
+        }
+
+
+        if (newSize == 0)
+            return newComplexDeletion.isLive() ? null : new ComplexColumnData(column, NO_CELLS, newComplexDeletion);
+
+        if (newSize == cells.length && newComplexDeletion == complexDeletion)
+            return this;
+
+        Cell[] newCells = new Cell[newSize];
+        int j = 0;
+        cellTester = filter.newTester(column); // we need to reste the tester
+        for (Cell cell : cells)
+        {
+            if ((cellTester == null || cellTester.includes(cell.path()))
+                && !activeDeletion.deletes(cell)
+                && (dropped == null || cell.timestamp() > dropped.droppedTime))
+                newCells[j++] = cell;
+        }
+        assert j == newSize;
+
+        return new ComplexColumnData(column, newCells, newComplexDeletion);
+    }
+
+    public void digest(MessageDigest digest)
+    {
+        if (!complexDeletion.isLive())
+            complexDeletion.digest(digest);
+
+        for (Cell cell : cells)
+            cell.digest(digest);
+    }
+
+    public ComplexColumnData markCounterLocalToBeCleared()
+    {
+        Cell[] newCells = null;
+        for (int i = 0; i < cells.length; i++)
+        {
+            Cell cell = cells[i];
+            Cell marked = cell.markCounterLocalToBeCleared();
+            if (marked != cell)
+            {
+                if (newCells == null)
+                    newCells = Arrays.copyOf(cells, cells.length);
+                newCells[i] = marked;
+            }
+        }
+
+        return newCells == null
+             ? this
+             : new ComplexColumnData(column, newCells, complexDeletion);
+    }
+
+    public ComplexColumnData purge(DeletionPurger purger, int nowInSec)
+    {
+        DeletionTime newDeletion = complexDeletion.isLive() || purger.shouldPurge(complexDeletion) ? DeletionTime.LIVE : complexDeletion;
+
+        int newSize = 0;
+        for (Cell cell : cells)
+        {
+            Cell purged = cell.purge(purger, nowInSec);
+            if (purged != null)
+                ++newSize;
+        }
+
+        if (newSize == 0)
+            return newDeletion.isLive() ? null : new ComplexColumnData(column, NO_CELLS, newDeletion);
+
+        if (newDeletion == complexDeletion && newSize == cells.length)
+            return this;
+
+        Cell[] newCells = new Cell[newSize];
+        int j = 0;
+        for (Cell cell : cells)
+        {
+            Cell purged = cell.purge(purger, nowInSec);
+            if (purged != null)
+                newCells[j++] = purged;
+        }
+        assert j == newSize;
+
+        return new ComplexColumnData(column, newCells, newDeletion);
+    }
+
+    public ComplexColumnData updateAllTimestamp(long newTimestamp)
+    {
+        DeletionTime newDeletion = complexDeletion.isLive() ? complexDeletion : new DeletionTime(newTimestamp - 1, complexDeletion.localDeletionTime());
+        Cell[] newCells = new Cell[cells.length];
+        for (int i = 0; i < cells.length; i++)
+            newCells[i] = (Cell)cells[i].updateAllTimestamp(newTimestamp);
+
+        return new ComplexColumnData(column, newCells, newDeletion);
+    }
+
+    // This is the partner in crime of ArrayBackedRow.setValue. The exact warning apply. The short
+    // version is: "don't use that method".
+    void setValue(CellPath path, ByteBuffer value)
+    {
+        int idx = binarySearch(path);
+        assert idx >= 0;
+        cells[idx] = cells[idx].withUpdatedValue(value);
+    }
+
+    private int binarySearch(CellPath path)
+    {
+        return binarySearch(path, 0, cells.length);
+    }
+
+    /**
+     * Simple binary search for a given cell (in the cells array).
+     *
+     * The return value has the exact same meaning that the one of Collections.binarySearch() but
+     * we don't use the later because we're searching for a 'CellPath' in an array of 'Cell'.
+     */
+    private int binarySearch(CellPath path, int fromIndex, int toIndex)
+    {
+        int low = fromIndex;
+        int mid = toIndex;
+        int high = mid - 1;
+        int result = -1;
+        while (low <= high)
+        {
+            mid = (low + high) >> 1;
+            if ((result = column.cellPathComparator().compare(path, cells[mid].path())) > 0)
+                low = mid + 1;
+            else if (result == 0)
+                return mid;
+            else
+                high = mid - 1;
+        }
+        return -mid - (result < 0 ? 1 : 2);
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (this == other)
+            return true;
+
+        if(!(other instanceof ComplexColumnData))
+            return false;
+
+        ComplexColumnData that = (ComplexColumnData)other;
+        return this.column().equals(that.column())
+            && this.complexDeletion().equals(that.complexDeletion)
+            && Arrays.equals(this.cells, that.cells);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(column(), complexDeletion(), cells);
+    }
+
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    public static class Builder
+    {
+        private ColumnDefinition column;
+        private DeletionTime complexDeletion;
+        public final List<Cell> cells = new ArrayList<>();
+
+        public void newColumn(ColumnDefinition column)
+        {
+            this.column = column;
+            this.complexDeletion = DeletionTime.LIVE; // default if writeComplexDeletion is not called
+            this.cells.clear();
+        }
+
+        public void addComplexDeletion(DeletionTime complexDeletion)
+        {
+            this.complexDeletion = complexDeletion;
+        }
+
+        public void addCell(Cell cell)
+        {
+            assert cell.column().equals(column);
+            assert cells.isEmpty() || cell.column().cellPathComparator().compare(cells.get(cells.size() - 1).path(), cell.path()) < 0;
+            cells.add(cell);
+        }
+
+        public ComplexColumnData build()
+        {
+            if (complexDeletion.isLive() && cells.isEmpty())
+                return null;
+
+            return new ComplexColumnData(column, cells.toArray(new Cell[cells.size()]), complexDeletion);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/ComplexRowDataBlock.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/ComplexRowDataBlock.java b/src/java/org/apache/cassandra/db/rows/ComplexRowDataBlock.java
deleted file mode 100644
index 75df874..0000000
--- a/src/java/org/apache/cassandra/db/rows/ComplexRowDataBlock.java
+++ /dev/null
@@ -1,796 +0,0 @@
-/*
- * 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.rows;
-
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import com.google.common.collect.UnmodifiableIterator;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.utils.ObjectSizes;
-
-/**
- * Holds cells data and complex deletions for the complex columns of one or more rows.
- * <p>
- * Contrarily to {@code SimpleRowDataBlock}, each complex column can have multiple cells and
- * we thus can't use a similar dense encoding. Instead, we still store the actual cell data
- * in a {@code CellData} object, but we add a level of indirection (the cellIdx array in
- * {@link ComplexCellBlock}) which for every column of every row stores 2 indexes: the index
- * in the {@code CellData} where the first cell for this column is, and the the index of the
- * last cell (or rather, the index to the first cell that does not belong to that column).
- * <p>
- * What makes this a little bit more complicated however is that in some cases (for
- * {@link PartitionUpdate} typically), we need to be able to swap rows inside a
- * {@code ComplexRowDataBlock} and the extra level of indirection makes that more complex.
- * So in practice, we have 2 separate sub-implementation of a {@code ComplexRowDataBlock}:
- *   - The first one, {@code SimpleComplexRowDataBlock} does not support swapping rows
- *     (and is thus only used when we don't need to) but it uses a single {@code CellData}
- *     for all the rows stored.
- *   - The second one, {@code SortableComplexRowDataBlock}, uses one separate {@code CellData}
- *     per row (in fact, a {@code ComplexCellBlock} which groups the cell data with the
- *     indexing array discussed above) and simply keeps those per-row block in a list. It
- *     is thus less compact in memory but make the swapping of rows trivial.
- */
-public abstract class ComplexRowDataBlock
-{
-    private static final Logger logger = LoggerFactory.getLogger(ComplexRowDataBlock.class);
-
-    private final Columns columns;
-
-    // For each complex column, it's deletion time (if any): the nth complex column of row i
-    // will have it's deletion time at complexDelTimes[(i * ccs) + n] where ccs it the number
-    // of complex columns in 'columns'.
-    final DeletionTimeArray complexDelTimes;
-
-    protected ComplexRowDataBlock(Columns columns, int rows)
-    {
-        this.columns = columns;
-
-        int columnCount = rows * columns.complexColumnCount();
-        this.complexDelTimes = new DeletionTimeArray(columnCount);
-    }
-
-    public static ComplexRowDataBlock create(Columns columns, int rows, boolean sortable, boolean isCounter)
-    {
-        return sortable
-             ? new SortableComplexRowDataBlock(columns, rows, isCounter)
-             : new SimpleComplexRowDataBlock(columns, rows, isCounter);
-    }
-
-    public Columns columns()
-    {
-        return columns;
-    }
-
-    public CellData cellData(int row)
-    {
-        return cellBlock(row).data;
-    }
-
-    public int cellIdx(int row, ColumnDefinition c, CellPath path)
-    {
-        ComplexCellBlock block = cellBlock(row);
-        if (block == null)
-            return -1;
-
-        int base = cellBlockBase(row);
-        int i = base + 2 * columns.complexIdx(c, 0);
-
-        int start = block.cellIdx[i];
-        int end = block.cellIdx[i+1];
-
-        if (i >= block.cellIdx.length || end <= start)
-            return -1;
-
-        return Arrays.binarySearch(block.complexPaths, start, end, path, c.cellPathComparator());
-    }
-
-    // The following methods abstract the fact that we have 2 sub-implementations: both
-    // implementation will use a ComplexCellBlock to store a row, but one will use one
-    // ComplexCellBlock per row, while the other will store all rows into the same block.
-
-    // Returns the cell block for a given row. Can return null if the asked row has no data.
-    protected abstract ComplexCellBlock cellBlock(int row);
-    // Same as cellBlock(), but create the proper block if the row doesn't exists and return it.
-    protected abstract ComplexCellBlock cellBlockForWritting(int row);
-    // The index in the block returned by cellBlock()/cellBlockFroWriting() where the row starts.
-    protected abstract int cellBlockBase(int row);
-
-    protected abstract void swapCells(int i, int j);
-    protected abstract void mergeCells(int i, int j, int nowInSec);
-    protected abstract void moveCells(int i, int j);
-
-    protected abstract long cellDataUnsharedHeapSizeExcludingData();
-    protected abstract int dataCellSize();
-    protected abstract void clearCellData();
-
-    // Swap row i and j
-    public void swap(int i, int j)
-    {
-        swapCells(i, j);
-
-        int s = columns.complexColumnCount();
-        for (int k = 0; k < s; k++)
-            complexDelTimes.swap(i * s + k, j * s + k);
-    }
-
-    // Merge row i into j
-    public void merge(int i, int j, int nowInSec)
-    {
-        assert i > j;
-
-        mergeCells(i, j, nowInSec);
-
-        int s = columns.complexColumnCount();
-        if (i * s >= complexDelTimes.size())
-            return;
-
-        for (int k = 0; k < s; k++)
-            if (complexDelTimes.supersedes(i * s + k, j * s + k))
-                complexDelTimes.move(i * s + k, j * s + k);
-    }
-
-    // Move row i into j
-    public void move(int i, int j)
-    {
-        moveCells(i, j);
-        ensureDelTimesCapacity(Math.max(i, j));
-        int s = columns.complexColumnCount();
-        for (int k = 0; k < s; k++)
-            complexDelTimes.move(i * s + k, j * s + k);
-    }
-
-    public long unsharedHeapSizeExcludingData()
-    {
-        return cellDataUnsharedHeapSizeExcludingData() + complexDelTimes.unsharedHeapSize();
-    }
-
-    public int dataSize()
-    {
-        return dataCellSize() + complexDelTimes.dataSize();
-    }
-
-    public CellWriter cellWriter(boolean inOrderCells)
-    {
-        return new CellWriter(inOrderCells);
-    }
-
-    public int complexDeletionIdx(int row, ColumnDefinition column)
-    {
-        int baseIdx = columns.complexIdx(column, 0);
-        if (baseIdx < 0)
-            return -1;
-
-        int idx = (row * columns.complexColumnCount()) + baseIdx;
-        return idx < complexDelTimes.size() ? idx : -1;
-    }
-
-    public boolean hasComplexDeletion(int row)
-    {
-        int base = row * columns.complexColumnCount();
-        for (int i = base; i < base + columns.complexColumnCount(); i++)
-            if (!complexDelTimes.isLive(i))
-                return true;
-        return false;
-    }
-
-    public ByteBuffer getValue(int row, ColumnDefinition column, CellPath path)
-    {
-        CellData data = cellData(row);
-        assert data != null;
-        int idx = cellIdx(row, column, path);
-        return data.value(idx);
-    }
-
-    public void setValue(int row, ColumnDefinition column, CellPath path, ByteBuffer value)
-    {
-        CellData data = cellData(row);
-        assert data != null;
-        int idx = cellIdx(row, column, path);
-        data.setValue(idx, value);
-    }
-
-    public static ReusableIterator reusableComplexCells()
-    {
-        return new ReusableIterator();
-    }
-
-    public static DeletionTimeArray.Cursor complexDeletionCursor()
-    {
-        return new DeletionTimeArray.Cursor();
-    }
-
-    public static ReusableIterator reusableIterator()
-    {
-        return new ReusableIterator();
-    }
-
-    public void clear()
-    {
-        clearCellData();
-        complexDelTimes.clear();
-    }
-
-    private void ensureDelTimesCapacity(int rowToSet)
-    {
-        int originalCapacity = complexDelTimes.size() / columns.complexColumnCount();
-        if (rowToSet < originalCapacity)
-            return;
-
-        int newCapacity = RowDataBlock.computeNewCapacity(originalCapacity, rowToSet);
-        complexDelTimes.resize(newCapacity * columns.complexColumnCount());
-    }
-
-    /**
-     * Simple sub-implementation that doesn't support swapping/sorting rows.
-     * The cell data for every row is stored in the same contiguous {@code ComplexCellBloc}
-     * object.
-     */
-    private static class SimpleComplexRowDataBlock extends ComplexRowDataBlock
-    {
-        private static final long EMPTY_SIZE = ObjectSizes.measure(new SimpleComplexRowDataBlock(Columns.NONE, 0, false));
-
-        private final ComplexCellBlock cells;
-
-        private SimpleComplexRowDataBlock(Columns columns, int rows, boolean isCounter)
-        {
-            super(columns, rows);
-            this.cells = new ComplexCellBlock(columns, rows, isCounter);
-        }
-
-        protected ComplexCellBlock cellBlock(int row)
-        {
-            return cells;
-        }
-
-        protected ComplexCellBlock cellBlockForWritting(int row)
-        {
-            cells.ensureCapacity(row);
-            return cells;
-        }
-
-        protected int cellBlockBase(int row)
-        {
-            return 2 * row * columns().complexColumnCount();
-        }
-
-        // Swap cells from row i and j
-        public void swapCells(int i, int j)
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        // Merge cells from row i into j
-        public void mergeCells(int i, int j, int nowInSec)
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        // Move cells from row i into j
-        public void moveCells(int i, int j)
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        protected long cellDataUnsharedHeapSizeExcludingData()
-        {
-            return EMPTY_SIZE + cells.unsharedHeapSizeExcludingData();
-        }
-
-        protected int dataCellSize()
-        {
-            return cells.dataSize();
-        }
-
-        protected void clearCellData()
-        {
-            cells.clear();
-        }
-    }
-
-    /**
-     * Sub-implementation that support swapping/sorting rows.
-     * The data for each row is stored in a different {@code ComplexCellBlock} object,
-     * making swapping rows easy.
-     */
-    private static class SortableComplexRowDataBlock extends ComplexRowDataBlock
-    {
-        private static final long EMPTY_SIZE = ObjectSizes.measure(new SortableComplexRowDataBlock(Columns.NONE, 0, false));
-
-        // The cell data for each row.
-        private final List<ComplexCellBlock> cells;
-        private final boolean isCounter;
-
-        private SortableComplexRowDataBlock(Columns columns, int rows, boolean isCounter)
-        {
-            super(columns, rows);
-            this.cells = new ArrayList<>(rows);
-            this.isCounter = isCounter;
-        }
-
-        protected ComplexCellBlock cellBlockForWritting(int row)
-        {
-            if (row < cells.size())
-                return cells.get(row);
-
-            // Make sure the list of size 'row-1' before the insertion, adding nulls if necessary,
-            // so that we do are writing row 'row'
-            ensureCapacity(row-1);
-
-            assert row == cells.size();
-            ComplexCellBlock block = new ComplexCellBlock(columns(), 1, isCounter);
-            cells.add(block);
-            return block;
-        }
-
-        private void ensureCapacity(int row)
-        {
-            while (row >= cells.size())
-                cells.add(null);
-        }
-
-        protected ComplexCellBlock cellBlock(int row)
-        {
-            return row >= cells.size() ? null : cells.get(row);
-        }
-
-        protected int cellBlockBase(int row)
-        {
-            return 0;
-        }
-
-        // Swap row i and j
-        protected void swapCells(int i, int j)
-        {
-            int max = Math.max(i, j);
-            if (max >= cells.size())
-                ensureCapacity(max);
-
-            ComplexCellBlock block = cells.get(j);
-            move(i, j);
-            cells.set(i, block);
-        }
-
-        // Merge row i into j
-        protected void mergeCells(int i, int j, int nowInSec)
-        {
-            assert i > j;
-            if (i >= cells.size())
-                return;
-
-            ComplexCellBlock b1 = cells.get(i);
-            if (b1 == null)
-                return; // nothing to merge into j
-
-            ComplexCellBlock b2 = cells.get(j);
-            if (b2 == null)
-            {
-                cells.set(j, b1);
-                return;
-            }
-
-            ComplexCellBlock merged = new ComplexCellBlock(columns(), 1, isCounter);
-
-            int idxMerged = 0;
-            int s = columns().complexColumnCount();
-            for (int k = 0; k < s; k++)
-            {
-                ColumnDefinition column = columns().getComplex(k);
-                Comparator<CellPath> comparator = column.cellPathComparator();
-
-                merged.cellIdx[2 * k] = idxMerged;
-
-                int idx1 = b1.cellIdx[2 * k];
-                int end1 = b1.cellIdx[2 * k + 1];
-                int idx2 = b2.cellIdx[2 * k];
-                int end2 = b2.cellIdx[2 * k + 1];
-
-                while (idx1 < end1 || idx2 < end2)
-                {
-                    int cmp = idx1 >= end1 ? 1
-                            : (idx2 >= end2 ? -1
-                            : comparator.compare(b1.complexPaths[idx1], b2.complexPaths[idx2]));
-
-                    if (cmp == 0)
-                        merge(b1, idx1++, b2, idx2++, merged, idxMerged++, nowInSec);
-                    else if (cmp < 0)
-                        copy(b1, idx1++, merged, idxMerged++);
-                    else
-                        copy(b2, idx2++, merged, idxMerged++);
-                }
-
-                merged.cellIdx[2 * k + 1] = idxMerged;
-            }
-
-            cells.set(j, merged);
-        }
-
-        private void copy(ComplexCellBlock fromBlock, int fromIdx, ComplexCellBlock toBlock, int toIdx)
-        {
-            fromBlock.data.moveCell(fromIdx, toBlock.data, toIdx);
-            toBlock.ensureComplexPathsCapacity(toIdx);
-            toBlock.complexPaths[toIdx] = fromBlock.complexPaths[fromIdx];
-        }
-
-        private void merge(ComplexCellBlock b1, int idx1, ComplexCellBlock b2, int idx2, ComplexCellBlock mergedBlock, int mergedIdx, int nowInSec)
-        {
-            if (isCounter)
-                CellData.mergeCounterCell(b1.data, idx1, b2.data, idx2, mergedBlock.data, mergedIdx, nowInSec);
-            else
-                CellData.mergeRegularCell(b1.data, idx1, b2.data, idx2, mergedBlock.data, mergedIdx, nowInSec);
-            mergedBlock.ensureComplexPathsCapacity(mergedIdx);
-            mergedBlock.complexPaths[mergedIdx] = b1.complexPaths[idx1];
-        }
-
-        // Move row i into j
-        protected void moveCells(int i, int j)
-        {
-            int max = Math.max(i, j);
-            if (max >= cells.size())
-                ensureCapacity(max);
-
-            cells.set(j, cells.get(i));
-        }
-
-        protected long cellDataUnsharedHeapSizeExcludingData()
-        {
-            long size = EMPTY_SIZE;
-            for (ComplexCellBlock block : cells)
-                if (block != null)
-                    size += block.unsharedHeapSizeExcludingData();
-            return size;
-        }
-
-        protected int dataCellSize()
-        {
-            int size = 0;
-            for (ComplexCellBlock block : cells)
-                if (block != null)
-                    size += block.dataSize();
-            return size;
-        }
-
-        protected void clearCellData()
-        {
-            for (ComplexCellBlock block : cells)
-                if (block != null)
-                    block.clear();
-        }
-    }
-
-    /**
-     * Stores complex column cell data for one or more rows.
-     * <p>
-     * On top of a {@code CellData} object, this stores an index to where the cells
-     * of a given column start and stop in that {@code CellData} object (cellIdx)
-     * as well as the cell path for the cells (since {@code CellData} doesn't have those).
-     */
-    private static class ComplexCellBlock
-    {
-        private final Columns columns;
-
-        /*
-         * For a given complex column c, we have to store an unknown number of
-         * cells. So for each column of each row, we keep pointers (in data)
-         * to the start and end of the cells for this column (cells for a given
-         * columns are thus stored contiguously).
-         * For instance, if columns has 'c' complex columns, the x-th column of
-         * row 'n' will have it's cells in data at indexes
-         *    [cellIdx[2 * (n * c + x)], cellIdx[2 * (n * c + x) + 1])
-         */
-        private int[] cellIdx;
-
-        private final CellData data;
-
-        // The first free idx in data (for writing purposes).
-        private int idx;
-
-        // THe (complex) cells path. This is indexed exactly like the cells in data (so through cellIdx).
-        private CellPath[] complexPaths;
-
-        public ComplexCellBlock(Columns columns, int rows, boolean isCounter)
-        {
-            this.columns = columns;
-
-            int columnCount = columns.complexColumnCount();
-            this.cellIdx = new int[columnCount * 2 * rows];
-
-            // We start with an estimated 4 cells per complex column. The arrays
-            // will grow if needed so this is just a somewhat random estimation.
-            int cellCount =  columnCount * 4;
-            this.data = new CellData(cellCount, isCounter);
-            this.complexPaths = new CellPath[cellCount];
-        }
-
-        public void addCell(int columnIdx, ByteBuffer value, LivenessInfo info, CellPath path, boolean isFirstCell)
-        {
-            if (isFirstCell)
-                cellIdx[columnIdx] = idx;
-            cellIdx[columnIdx + 1] = idx + 1;
-
-            data.setCell(idx, value, info);
-            ensureComplexPathsCapacity(idx);
-            complexPaths[idx] = path;
-            idx++;
-        }
-
-        public long unsharedHeapSizeExcludingData()
-        {
-            long size = ObjectSizes.sizeOfArray(cellIdx)
-                      + data.unsharedHeapSizeExcludingData()
-                      + ObjectSizes.sizeOfArray(complexPaths);
-
-            for (int i = 0; i < complexPaths.length; i++)
-                if (complexPaths[i] != null)
-                    size += ((MemtableRowData.BufferCellPath)complexPaths[i]).unsharedHeapSizeExcludingData();
-            return size;
-        }
-
-        public int dataSize()
-        {
-            int size = data.dataSize() + cellIdx.length * 4;
-
-            for (int i = 0; i < complexPaths.length; i++)
-                if (complexPaths[i] != null)
-                    size += complexPaths[i].dataSize();
-
-            return size;
-        }
-
-        private void ensureCapacity(int rowToSet)
-        {
-            int columnCount = columns.complexColumnCount();
-            int originalCapacity = cellIdx.length / (2 * columnCount);
-            if (rowToSet < originalCapacity)
-                return;
-
-            int newCapacity = RowDataBlock.computeNewCapacity(originalCapacity, rowToSet);
-            cellIdx = Arrays.copyOf(cellIdx, newCapacity * 2 * columnCount);
-        }
-
-        private void ensureComplexPathsCapacity(int idxToSet)
-        {
-            int originalCapacity = complexPaths.length;
-            if (idxToSet < originalCapacity)
-                return;
-
-            int newCapacity = RowDataBlock.computeNewCapacity(originalCapacity, idxToSet);
-            complexPaths = Arrays.copyOf(complexPaths, newCapacity);
-        }
-
-        public void clear()
-        {
-            data.clear();
-            Arrays.fill(cellIdx, 0);
-            Arrays.fill(complexPaths, null);
-            idx = 0;
-        }
-    }
-
-    /**
-     * Simple sublcassing of {@code CellData.ReusableCell} to include the cell path.
-     */
-    private static class ReusableCell extends CellData.ReusableCell
-    {
-        private ComplexCellBlock cellBlock;
-
-        ReusableCell setTo(ComplexCellBlock cellBlock, ColumnDefinition column, int idx)
-        {
-            this.cellBlock = cellBlock;
-            super.setTo(cellBlock.data, column, idx);
-            return this;
-        }
-
-        @Override
-        public CellPath path()
-        {
-            return cellBlock.complexPaths[idx];
-        }
-    }
-
-    /**
-     * An iterator over the complex cells of a given row.
-     * This is used both to iterate over all the (complex) cells of the row, or only on the cells
-     * of a given column within the row.
-     */
-    static class ReusableIterator extends UnmodifiableIterator<Cell>
-    {
-        private ComplexCellBlock cellBlock;
-        private final ReusableCell cell = new ReusableCell();
-
-        // The idx in 'cellBlock' of the row we're iterating over
-        private int rowIdx;
-
-        // columnIdx is the index in 'columns' of the current column we're iterating over.
-        // 'endColumnIdx' is the value of 'columnIdx' at which we should stop iterating.
-        private int columnIdx;
-        private int endColumnIdx;
-
-        // idx is the index in 'cellBlock.data' of the current cell this iterator is on. 'endIdx'
-        // is the index in 'cellBlock.data' of the first cell that does not belong to the current
-        // column we're iterating over (the one pointed by columnIdx).
-        private int idx;
-        private int endIdx;
-
-        private ReusableIterator()
-        {
-        }
-
-        // Sets the iterator for iterating over the cells of 'column' in 'row'
-        public ReusableIterator setTo(ComplexRowDataBlock dataBlock, int row, ColumnDefinition column)
-        {
-            if (dataBlock == null)
-            {
-                this.cellBlock = null;
-                return null;
-            }
-
-            this.cellBlock = dataBlock.cellBlock(row);
-            if (cellBlock == null)
-                return null;
-
-            rowIdx = dataBlock.cellBlockBase(row);
-
-            columnIdx = dataBlock.columns.complexIdx(column, 0);
-            if (columnIdx < 0)
-                return null;
-
-            // We only want the cells of 'column', so stop as soon as we've reach the next column
-            endColumnIdx = columnIdx + 1;
-
-            resetCellIdx();
-
-            return endIdx <= idx ? null : this;
-        }
-
-        // Sets the iterator for iterating over all the cells of 'row'
-        public ReusableIterator setTo(ComplexRowDataBlock dataBlock, int row)
-        {
-            if (dataBlock == null)
-            {
-                this.cellBlock = null;
-                return null;
-            }
-
-            this.cellBlock = dataBlock.cellBlock(row);
-            if (cellBlock == null)
-                return null;
-
-            rowIdx = dataBlock.cellBlockBase(row);
-
-            // We want to iterator over all columns
-            columnIdx = 0;
-            endColumnIdx = dataBlock.columns.complexColumnCount();
-
-            // Not every column might have cells, so set thing up so we're on the
-            // column having cells (with idx and endIdx sets properly for that column)
-            findNextColumnWithCells();
-            return columnIdx < endColumnIdx ? null : this;
-        }
-
-        private void findNextColumnWithCells()
-        {
-            while (columnIdx < endColumnIdx)
-            {
-                resetCellIdx();
-                if (idx < endIdx)
-                    return;
-                ++columnIdx;
-            }
-        }
-
-        // Provided that columnIdx and rowIdx are properly set, sets idx to the first
-        // cells of the pointed column, and endIdx to the first cell not for said column
-        private void resetCellIdx()
-        {
-            int i = rowIdx + 2 * columnIdx;
-            if (i >= cellBlock.cellIdx.length)
-            {
-                idx = 0;
-                endIdx = 0;
-            }
-            else
-            {
-                idx = cellBlock.cellIdx[i];
-                endIdx = cellBlock.cellIdx[i + 1];
-            }
-        }
-
-        public boolean hasNext()
-        {
-            if (cellBlock == null)
-                return false;
-
-            if (columnIdx >= endColumnIdx)
-                return false;
-
-            // checks if we have more cells for the current column
-            if (idx < endIdx)
-                return true;
-
-            // otherwise, find the next column that has cells.
-            ++columnIdx;
-            findNextColumnWithCells();
-
-            return columnIdx < endColumnIdx;
-        }
-
-        public Cell next()
-        {
-            return cell.setTo(cellBlock, cellBlock.columns.getComplex(columnIdx), idx++);
-        }
-    }
-
-    public class CellWriter
-    {
-        private final boolean inOrderCells;
-
-        private int base;
-        private int row;
-        private int lastColumnIdx;
-
-        public CellWriter(boolean inOrderCells)
-        {
-            this.inOrderCells = inOrderCells;
-        }
-
-        public void addCell(ColumnDefinition column, ByteBuffer value, LivenessInfo info, CellPath path)
-        {
-            assert path != null;
-
-            ComplexCellBlock cellBlock = cellBlockForWritting(row);
-
-            lastColumnIdx = columns.complexIdx(column, inOrderCells ? lastColumnIdx : 0);
-            assert lastColumnIdx >= 0 : "Cannot find column " + column.name + " in " + columns;
-
-            int idx = cellBlockBase(row) + 2 * lastColumnIdx;
-
-            int start = cellBlock.cellIdx[idx];
-            int end = cellBlock.cellIdx[idx + 1];
-
-            cellBlock.addCell(idx, value, info, path, end <= start);
-        }
-
-        public void setComplexDeletion(ColumnDefinition column, DeletionTime deletionTime)
-        {
-            int columnIdx = base + columns.complexIdx(column, 0);
-            ensureDelTimesCapacity(row);
-            complexDelTimes.set(columnIdx, deletionTime);
-        }
-
-        public void endOfRow()
-        {
-            base += columns.complexColumnCount();
-            lastColumnIdx = 0;
-            ++row;
-        }
-
-        public void reset()
-        {
-            base = 0;
-            row = 0;
-            lastColumnIdx = 0;
-            clearCellData();
-            complexDelTimes.clear();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/FilteringRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/FilteringRow.java b/src/java/org/apache/cassandra/db/rows/FilteringRow.java
deleted file mode 100644
index fb8f448..0000000
--- a/src/java/org/apache/cassandra/db/rows/FilteringRow.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * 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.rows;
-
-import java.util.Iterator;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.filter.ColumnFilter;
-
-public abstract class FilteringRow extends WrappingRow
-{
-    public static FilteringRow columnsFilteringRow(final Columns toInclude)
-    {
-        return new FilteringRow()
-        {
-            @Override
-            protected boolean include(ColumnDefinition column)
-            {
-                return toInclude.contains(column);
-            }
-        };
-    }
-
-    public static FilteringRow columnsFilteringRow(final ColumnFilter toInclude)
-    {
-        return new FilteringRow()
-        {
-            @Override
-            protected boolean include(ColumnDefinition column)
-            {
-                return toInclude.includes(column);
-            }
-
-            @Override
-            protected boolean include(Cell cell)
-            {
-                return toInclude.includes(cell);
-            }
-        };
-    }
-
-    public FilteringRow setTo(Row row)
-    {
-        super.setTo(row);
-        return this;
-    }
-
-    /**
-     * The following functions are meant to be overriden based on needs.
-     */
-    protected boolean include(Cell cell) { return true; }
-    protected boolean include(LivenessInfo info) { return true; }
-    protected boolean include(DeletionTime dt) { return true; }
-    protected boolean include(ColumnDefinition column) { return true; }
-    protected boolean include(ColumnDefinition c, DeletionTime dt) { return true; }
-
-    // Sublcasses that override this should be careful to call the overriden version first, or this might break FilteringRow (i.e. it might not
-    // filter what it should).
-    @Override
-    protected Cell filterCell(Cell cell)
-    {
-        return include(cell.column()) && include(cell.livenessInfo()) && include(cell) ? cell : null;
-    }
-
-    protected DeletionTime filterDeletionTime(DeletionTime deletion)
-    {
-        return deletion == null || !include(deletion)
-             ? DeletionTime.LIVE
-             : deletion;
-    }
-
-    @Override
-    public LivenessInfo primaryKeyLivenessInfo()
-    {
-        LivenessInfo info = super.primaryKeyLivenessInfo();
-        return include(info) ? info : LivenessInfo.NONE;
-    }
-
-    @Override
-    public DeletionTime deletion()
-    {
-        DeletionTime deletion = super.deletion();
-        return include(deletion) ? deletion : DeletionTime.LIVE;
-    }
-
-    @Override
-    public Iterator<Cell> getCells(ColumnDefinition c)
-    {
-        // slightly speed things up if we know we don't care at all about the column
-        if (!include(c))
-            return null;
-
-        return super.getCells(c);
-    }
-
-    @Override
-    public DeletionTime getDeletion(ColumnDefinition c)
-    {
-        if (!include(c))
-            return DeletionTime.LIVE;
-
-        DeletionTime dt = super.getDeletion(c);
-        return include(c, dt) ? dt : DeletionTime.LIVE;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/FilteringRowIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/FilteringRowIterator.java b/src/java/org/apache/cassandra/db/rows/FilteringRowIterator.java
deleted file mode 100644
index fd1c0a1..0000000
--- a/src/java/org/apache/cassandra/db/rows/FilteringRowIterator.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * 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.rows;
-
-import org.apache.cassandra.db.*;
-
-public class FilteringRowIterator extends WrappingUnfilteredRowIterator
-{
-    private final FilteringRow filter;
-    private Unfiltered next;
-
-    public FilteringRowIterator(UnfilteredRowIterator toFilter)
-    {
-        super(toFilter);
-        this.filter = makeRowFilter();
-    }
-
-    // Subclasses that want to filter withing row should overwrite this. Note that since FilteringRow
-    // is a reusable object, this method won't be called for every filtered row and the same filter will
-    // be used for every regular rows. However, this still can be called twice if we have a static row
-    // to filter, because we don't want to use the same object for them as this makes for weird behavior
-    // if calls to staticRow() are interleaved with hasNext().
-    protected FilteringRow makeRowFilter()
-    {
-        return null;
-    }
-
-    protected boolean includeRangeTombstoneMarker(RangeTombstoneMarker marker)
-    {
-        return true;
-    }
-
-    // Allows to modify the range tombstone returned. This is called *after* includeRangeTombstoneMarker has been called.
-    protected RangeTombstoneMarker filterRangeTombstoneMarker(RangeTombstoneMarker marker, boolean reversed)
-    {
-        return marker;
-    }
-
-    protected boolean includeRow(Row row)
-    {
-        return true;
-    }
-
-    protected boolean includePartitionDeletion(DeletionTime dt)
-    {
-        return true;
-    }
-
-    @Override
-    public DeletionTime partitionLevelDeletion()
-    {
-        DeletionTime dt = wrapped.partitionLevelDeletion();
-        return includePartitionDeletion(dt) ? dt : DeletionTime.LIVE;
-    }
-
-    @Override
-    public Row staticRow()
-    {
-        Row row = super.staticRow();
-        if (row == Rows.EMPTY_STATIC_ROW)
-            return row;
-
-        FilteringRow filter = makeRowFilter();
-        if (filter != null)
-            row = filter.setTo(row);
-
-        return !row.isEmpty() && includeRow(row) ? row : Rows.EMPTY_STATIC_ROW;
-    }
-
-    @Override
-    public boolean hasNext()
-    {
-        if (next != null)
-            return true;
-
-        while (super.hasNext())
-        {
-            Unfiltered unfiltered = super.next();
-            if (unfiltered.kind() == Unfiltered.Kind.ROW)
-            {
-                Row row = filter == null ? (Row) unfiltered : filter.setTo((Row) unfiltered);
-                if (!row.isEmpty() && includeRow(row))
-                {
-                    next = row;
-                    return true;
-                }
-            }
-            else
-            {
-                RangeTombstoneMarker marker = (RangeTombstoneMarker) unfiltered;
-                if (includeRangeTombstoneMarker(marker))
-                {
-                    next = filterRangeTombstoneMarker(marker, isReverseOrder());
-                    return true;
-                }
-            }
-        }
-        return false;
-    }
-
-    @Override
-    public Unfiltered next()
-    {
-        if (next == null)
-            hasNext();
-
-        Unfiltered toReturn = next;
-        next = null;
-        return toReturn;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/MemtableRowData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/MemtableRowData.java b/src/java/org/apache/cassandra/db/rows/MemtableRowData.java
deleted file mode 100644
index cad0765..0000000
--- a/src/java/org/apache/cassandra/db/rows/MemtableRowData.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/*
- * 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.rows;
-
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.utils.ObjectSizes;
-import org.apache.cassandra.utils.memory.AbstractAllocator;
-
-/**
- * Row data stored inside a memtable.
- *
- * This has methods like dataSize and unsharedHeapSizeExcludingData that are
- * specific to memtables.
- */
-public interface MemtableRowData extends Clusterable
-{
-    public Columns columns();
-
-    public int dataSize();
-
-    // returns the size of the Row and all references on the heap, excluding any costs associated with byte arrays
-    // that would be allocated by a clone operation, as these will be accounted for by the allocator
-    public long unsharedHeapSizeExcludingData();
-
-    public interface ReusableRow extends Row
-    {
-        public ReusableRow setTo(MemtableRowData rowData);
-    }
-
-    public class BufferRowData implements MemtableRowData
-    {
-        private static final long EMPTY_SIZE = ObjectSizes.measure(new BufferRowData(null, LivenessInfo.NONE, DeletionTime.LIVE, null));
-
-        private final Clustering clustering;
-        private final LivenessInfo livenessInfo;
-        private final DeletionTime deletion;
-        private final RowDataBlock dataBlock;
-
-        public BufferRowData(Clustering clustering, LivenessInfo livenessInfo, DeletionTime deletion, RowDataBlock dataBlock)
-        {
-            this.clustering = clustering;
-            this.livenessInfo = livenessInfo.takeAlias();
-            this.deletion = deletion.takeAlias();
-            this.dataBlock = dataBlock;
-        }
-
-        public Clustering clustering()
-        {
-            return clustering;
-        }
-
-        public Columns columns()
-        {
-            return dataBlock.columns();
-        }
-
-        public int dataSize()
-        {
-            return clustering.dataSize() + livenessInfo.dataSize() + deletion.dataSize() + dataBlock.dataSize();
-        }
-
-        public long unsharedHeapSizeExcludingData()
-        {
-            return EMPTY_SIZE
-                 + (clustering == Clustering.STATIC_CLUSTERING ? 0 : ((BufferClustering)clustering).unsharedHeapSizeExcludingData())
-                 + dataBlock.unsharedHeapSizeExcludingData();
-        }
-
-        public static ReusableRow createReusableRow()
-        {
-            return new BufferRow();
-        }
-
-        private static class BufferRow extends AbstractReusableRow implements ReusableRow
-        {
-            private BufferRowData rowData;
-
-            private BufferRow()
-            {
-            }
-
-            public ReusableRow setTo(MemtableRowData rowData)
-            {
-                assert rowData instanceof BufferRowData;
-                this.rowData = (BufferRowData)rowData;
-                return this;
-            }
-
-            protected RowDataBlock data()
-            {
-                return rowData.dataBlock;
-            }
-
-            protected int row()
-            {
-                return 0;
-            }
-
-            public Clustering clustering()
-            {
-                return rowData.clustering;
-            }
-
-            public LivenessInfo primaryKeyLivenessInfo()
-            {
-                return rowData.livenessInfo;
-            }
-
-            public DeletionTime deletion()
-            {
-                return rowData.deletion;
-            }
-        }
-    }
-
-    public class BufferClustering extends Clustering
-    {
-        private static final long EMPTY_SIZE = ObjectSizes.measure(new BufferClustering(0));
-
-        private final ByteBuffer[] values;
-
-        public BufferClustering(int size)
-        {
-            this.values = new ByteBuffer[size];
-        }
-
-        public void setClusteringValue(int i, ByteBuffer value)
-        {
-            values[i] = value;
-        }
-
-        public int size()
-        {
-            return values.length;
-        }
-
-        public ByteBuffer get(int i)
-        {
-            return values[i];
-        }
-
-        public ByteBuffer[] getRawValues()
-        {
-            return values;
-        }
-
-        public long unsharedHeapSizeExcludingData()
-        {
-            return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(values);
-        }
-
-        @Override
-        public long unsharedHeapSize()
-        {
-            return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(values);
-        }
-
-        public Clustering takeAlias()
-        {
-            return this;
-        }
-    }
-
-    public class BufferCellPath extends CellPath.SimpleCellPath
-    {
-        private static final long EMPTY_SIZE = ObjectSizes.measure(new BufferCellPath(new ByteBuffer[0]));
-
-        private BufferCellPath(ByteBuffer[] values)
-        {
-            super(values);
-        }
-
-        public static BufferCellPath clone(CellPath path, AbstractAllocator allocator)
-        {
-            int size = path.size();
-            ByteBuffer[] values = new ByteBuffer[size];
-            for (int i = 0; i < size; i++)
-                values[i] = allocator.clone(path.get(0));
-            return new BufferCellPath(values);
-        }
-
-        public long unsharedHeapSizeExcludingData()
-        {
-            return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(values);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundMarker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundMarker.java b/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundMarker.java
index 8b52b0b..b35033d 100644
--- a/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundMarker.java
+++ b/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundMarker.java
@@ -23,6 +23,7 @@ import java.util.Objects;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
  * A range tombstone marker that indicates the bound of a range tombstone (start or end).
@@ -34,7 +35,7 @@ public class RangeTombstoneBoundMarker extends AbstractRangeTombstoneMarker
     public RangeTombstoneBoundMarker(RangeTombstone.Bound bound, DeletionTime deletion)
     {
         super(bound);
-        assert bound.kind().isBound();
+        assert !bound.isBoundary();
         this.deletion = deletion;
     }
 
@@ -43,16 +44,6 @@ public class RangeTombstoneBoundMarker extends AbstractRangeTombstoneMarker
         this(new RangeTombstone.Bound(bound.kind(), bound.getRawValues()), deletion);
     }
 
-    public static RangeTombstoneBoundMarker inclusiveStart(ClusteringPrefix clustering, DeletionTime deletion)
-    {
-        return new RangeTombstoneBoundMarker(new RangeTombstone.Bound(RangeTombstone.Bound.Kind.INCL_START_BOUND, clustering.getRawValues()), deletion);
-    }
-
-    public static RangeTombstoneBoundMarker inclusiveEnd(ClusteringPrefix clustering, DeletionTime deletion)
-    {
-        return new RangeTombstoneBoundMarker(new RangeTombstone.Bound(RangeTombstone.Bound.Kind.INCL_END_BOUND, clustering.getRawValues()), deletion);
-    }
-
     public static RangeTombstoneBoundMarker inclusiveOpen(boolean reversed, ByteBuffer[] boundValues, DeletionTime deletion)
     {
         RangeTombstone.Bound bound = RangeTombstone.Bound.inclusiveOpen(reversed, boundValues);
@@ -90,16 +81,6 @@ public class RangeTombstoneBoundMarker extends AbstractRangeTombstoneMarker
         return deletion;
     }
 
-    public boolean isOpen(boolean reversed)
-    {
-        return bound.kind().isOpen(reversed);
-    }
-
-    public boolean isClose(boolean reversed)
-    {
-        return bound.kind().isClose(reversed);
-    }
-
     public DeletionTime openDeletionTime(boolean reversed)
     {
         if (!isOpen(reversed))
@@ -128,11 +109,19 @@ public class RangeTombstoneBoundMarker extends AbstractRangeTombstoneMarker
         return bound.isInclusive();
     }
 
-    public void copyTo(RangeTombstoneMarker.Writer writer)
+    public RangeTombstone.Bound openBound(boolean reversed)
+    {
+        return isOpen(reversed) ? clustering() : null;
+    }
+
+    public RangeTombstone.Bound closeBound(boolean reversed)
+    {
+        return isClose(reversed) ? clustering() : null;
+    }
+
+    public RangeTombstoneBoundMarker copy(AbstractAllocator allocator)
     {
-        copyBoundTo(writer);
-        writer.writeBoundDeletion(deletion);
-        writer.endOfMarker();
+        return new RangeTombstoneBoundMarker(clustering().copy(allocator), deletion);
     }
 
     public void digest(MessageDigest digest)
@@ -143,11 +132,7 @@ public class RangeTombstoneBoundMarker extends AbstractRangeTombstoneMarker
 
     public String toString(CFMetaData metadata)
     {
-        StringBuilder sb = new StringBuilder();
-        sb.append("Marker ");
-        sb.append(bound.toString(metadata));
-        sb.append("@").append(deletion.markedForDeleteAt());
-        return sb.toString();
+        return "Marker " + bound.toString(metadata) + '@' + deletion.markedForDeleteAt();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundaryMarker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundaryMarker.java b/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundaryMarker.java
index f17515d..06fbf87 100644
--- a/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundaryMarker.java
+++ b/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundaryMarker.java
@@ -23,6 +23,7 @@ import java.util.Objects;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
  * A range tombstone marker that represents a boundary between 2 range tombstones (i.e. it closes one range and open another).
@@ -35,7 +36,7 @@ public class RangeTombstoneBoundaryMarker extends AbstractRangeTombstoneMarker
     public RangeTombstoneBoundaryMarker(RangeTombstone.Bound bound, DeletionTime endDeletion, DeletionTime startDeletion)
     {
         super(bound);
-        assert bound.kind().isBoundary();
+        assert bound.isBoundary();
         this.endDeletion = endDeletion;
         this.startDeletion = startDeletion;
     }
@@ -56,11 +57,6 @@ public class RangeTombstoneBoundaryMarker extends AbstractRangeTombstoneMarker
         return new RangeTombstoneBoundaryMarker(bound, endDeletion, startDeletion);
     }
 
-    public boolean isBoundary()
-    {
-        return true;
-    }
-
     /**
      * The deletion time for the range tombstone this boundary ends (in clustering order).
      */
@@ -92,6 +88,16 @@ public class RangeTombstoneBoundaryMarker extends AbstractRangeTombstoneMarker
         return (bound.kind() == ClusteringPrefix.Kind.EXCL_END_INCL_START_BOUNDARY) ^ reversed;
     }
 
+    public RangeTombstone.Bound openBound(boolean reversed)
+    {
+        return bound.withNewKind(bound.kind().openBoundOfBoundary(reversed));
+    }
+
+    public RangeTombstone.Bound closeBound(boolean reversed)
+    {
+        return bound.withNewKind(bound.kind().closeBoundOfBoundary(reversed));
+    }
+
     public boolean closeIsInclusive(boolean reversed)
     {
         return (bound.kind() == ClusteringPrefix.Kind.INCL_END_EXCL_START_BOUNDARY) ^ reversed;
@@ -109,6 +115,11 @@ public class RangeTombstoneBoundaryMarker extends AbstractRangeTombstoneMarker
         return true;
     }
 
+    public RangeTombstoneBoundaryMarker copy(AbstractAllocator allocator)
+    {
+        return new RangeTombstoneBoundaryMarker(clustering().copy(allocator), endDeletion, startDeletion);
+    }
+
     public static RangeTombstoneBoundaryMarker makeBoundary(boolean reversed, Slice.Bound close, Slice.Bound open, DeletionTime closeDeletion, DeletionTime openDeletion)
     {
         assert RangeTombstone.Bound.Kind.compare(close.kind(), open.kind()) == 0 : "Both bound don't form a boundary";
@@ -118,21 +129,14 @@ public class RangeTombstoneBoundaryMarker extends AbstractRangeTombstoneMarker
              : inclusiveCloseExclusiveOpen(reversed, close.getRawValues(), closeDeletion, openDeletion);
     }
 
-    public RangeTombstoneBoundMarker createCorrespondingCloseBound(boolean reversed)
-    {
-        return new RangeTombstoneBoundMarker(bound.withNewKind(bound.kind().closeBoundOfBoundary(reversed)), endDeletion);
-    }
-
-    public RangeTombstoneBoundMarker createCorrespondingOpenBound(boolean reversed)
+    public RangeTombstoneBoundMarker createCorrespondingCloseMarker(boolean reversed)
     {
-        return new RangeTombstoneBoundMarker(bound.withNewKind(bound.kind().openBoundOfBoundary(reversed)), startDeletion);
+        return new RangeTombstoneBoundMarker(closeBound(reversed), endDeletion);
     }
 
-    public void copyTo(RangeTombstoneMarker.Writer writer)
+    public RangeTombstoneBoundMarker createCorrespondingOpenMarker(boolean reversed)
     {
-        copyBoundTo(writer);
-        writer.writeBoundaryDeletion(endDeletion, startDeletion);
-        writer.endOfMarker();
+        return new RangeTombstoneBoundMarker(openBound(reversed), startDeletion);
     }
 
     public void digest(MessageDigest digest)
@@ -144,11 +148,7 @@ public class RangeTombstoneBoundaryMarker extends AbstractRangeTombstoneMarker
 
     public String toString(CFMetaData metadata)
     {
-        StringBuilder sb = new StringBuilder();
-        sb.append("Marker ");
-        sb.append(bound.toString(metadata));
-        sb.append("@").append(endDeletion.markedForDeleteAt()).append("-").append(startDeletion.markedForDeleteAt());
-        return sb.toString();
+        return String.format("Marker %s@%d-%d", bound.toString(metadata), endDeletion.markedForDeleteAt(), startDeletion.markedForDeleteAt());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/RangeTombstoneMarker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RangeTombstoneMarker.java b/src/java/org/apache/cassandra/db/rows/RangeTombstoneMarker.java
index 380e6b0..5771a86 100644
--- a/src/java/org/apache/cassandra/db/rows/RangeTombstoneMarker.java
+++ b/src/java/org/apache/cassandra/db/rows/RangeTombstoneMarker.java
@@ -22,6 +22,7 @@ import java.util.*;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
  * A marker for a range tombstone bound.
@@ -35,78 +36,18 @@ public interface RangeTombstoneMarker extends Unfiltered
 
     public boolean isBoundary();
 
-    public void copyTo(RangeTombstoneMarker.Writer writer);
-
     public boolean isOpen(boolean reversed);
     public boolean isClose(boolean reversed);
+
     public DeletionTime openDeletionTime(boolean reversed);
     public DeletionTime closeDeletionTime(boolean reversed);
     public boolean openIsInclusive(boolean reversed);
     public boolean closeIsInclusive(boolean reversed);
 
-    public interface Writer extends Slice.Bound.Writer
-    {
-        public void writeBoundDeletion(DeletionTime deletion);
-        public void writeBoundaryDeletion(DeletionTime endDeletion, DeletionTime startDeletion);
-        public void endOfMarker();
-    }
-
-    public static class Builder implements Writer
-    {
-        private final ByteBuffer[] values;
-        private int size;
-
-        private RangeTombstone.Bound.Kind kind;
-        private DeletionTime firstDeletion;
-        private DeletionTime secondDeletion;
-
-        public Builder(int maxClusteringSize)
-        {
-            this.values = new ByteBuffer[maxClusteringSize];
-        }
-
-        public void writeClusteringValue(ByteBuffer value)
-        {
-            values[size++] = value;
-        }
-
-        public void writeBoundKind(RangeTombstone.Bound.Kind kind)
-        {
-            this.kind = kind;
-        }
-
-        public void writeBoundDeletion(DeletionTime deletion)
-        {
-            firstDeletion = deletion;
-        }
-
-        public void writeBoundaryDeletion(DeletionTime endDeletion, DeletionTime startDeletion)
-        {
-            firstDeletion = endDeletion;
-            secondDeletion = startDeletion;
-        }
-
-        public void endOfMarker()
-        {
-        }
-
-        public RangeTombstoneMarker build()
-        {
-            assert kind != null : "Nothing has been written";
-            if (kind.isBoundary())
-                return new RangeTombstoneBoundaryMarker(new RangeTombstone.Bound(kind, Arrays.copyOfRange(values, 0, size)), firstDeletion, secondDeletion);
-            else
-                return new RangeTombstoneBoundMarker(new RangeTombstone.Bound(kind, Arrays.copyOfRange(values, 0, size)), firstDeletion);
-        }
+    public RangeTombstone.Bound openBound(boolean reversed);
+    public RangeTombstone.Bound closeBound(boolean reversed);
 
-        public Builder reset()
-        {
-            Arrays.fill(values, null);
-            size = 0;
-            kind = null;
-            return this;
-        }
-    }
+    public RangeTombstoneMarker copy(AbstractAllocator allocator);
 
     /**
      * Utility class to help merging range tombstone markers coming from multiple inputs (UnfilteredRowIterators).
@@ -123,8 +64,6 @@ public interface RangeTombstoneMarker extends Unfiltered
      */
     public static class Merger
     {
-        private final CFMetaData metadata;
-        private final UnfilteredRowIterators.MergeListener listener;
         private final DeletionTime partitionDeletion;
         private final boolean reversed;
 
@@ -137,10 +76,8 @@ public interface RangeTombstoneMarker extends Unfiltered
         // marker on any iterator.
         private int biggestOpenMarker = -1;
 
-        public Merger(CFMetaData metadata, int size, DeletionTime partitionDeletion, boolean reversed, UnfilteredRowIterators.MergeListener listener)
+        public Merger(int size, DeletionTime partitionDeletion, boolean reversed)
         {
-            this.metadata = metadata;
-            this.listener = listener;
             this.partitionDeletion = partitionDeletion;
             this.reversed = reversed;
 
@@ -202,12 +139,14 @@ public interface RangeTombstoneMarker extends Unfiltered
                        : RangeTombstoneBoundaryMarker.inclusiveCloseExclusiveOpen(reversed, values, previousDeletionTimeInMerged, newDeletionTimeInMerged);
             }
 
-            if (listener != null)
-                listener.onMergedRangeTombstoneMarkers(merged, markers);
-
             return merged;
         }
 
+        public RangeTombstoneMarker[] mergedMarkers()
+        {
+            return markers;
+        }
+
         private DeletionTime currentOpenDeletionTimeInMerged()
         {
             if (biggestOpenMarker < 0)
@@ -215,7 +154,7 @@ public interface RangeTombstoneMarker extends Unfiltered
 
             DeletionTime biggestDeletionTime = openMarkers[biggestOpenMarker];
             // it's only open in the merged iterator if it's not shadowed by the partition level deletion
-            return partitionDeletion.supersedes(biggestDeletionTime) ? DeletionTime.LIVE : biggestDeletionTime.takeAlias();
+            return partitionDeletion.supersedes(biggestDeletionTime) ? DeletionTime.LIVE : biggestDeletionTime;
         }
 
         private void updateOpenMarkers()
@@ -229,7 +168,7 @@ public interface RangeTombstoneMarker extends Unfiltered
                 // Note that we can have boundaries that are both open and close, but in that case all we care about
                 // is what it the open deletion after the marker, so we favor the opening part in this case.
                 if (marker.isOpen(reversed))
-                    openMarkers[i] = marker.openDeletionTime(reversed).takeAlias();
+                    openMarkers[i] = marker.openDeletionTime(reversed);
                 else
                     openMarkers[i] = null;
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/24575994/src/java/org/apache/cassandra/db/rows/ReusableRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/ReusableRow.java b/src/java/org/apache/cassandra/db/rows/ReusableRow.java
deleted file mode 100644
index 0135afc..0000000
--- a/src/java/org/apache/cassandra/db/rows/ReusableRow.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * 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.rows;
-
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.db.*;
-
-public class ReusableRow extends AbstractReusableRow
-{
-    private final ReusableClustering clustering;
-
-    private final ReusableLivenessInfo liveness = new ReusableLivenessInfo();
-
-    private DeletionTime deletion = DeletionTime.LIVE;
-
-    private final RowDataBlock data;
-    private final Writer writer;
-
-    public ReusableRow(int clusteringSize, Columns columns, boolean inOrderCells, boolean isCounter)
-    {
-        this.clustering = new ReusableClustering(clusteringSize);
-        this.data = new RowDataBlock(columns, 1, false, isCounter);
-        this.writer = new Writer(data, inOrderCells);
-    }
-
-    protected RowDataBlock data()
-    {
-        return data;
-    }
-
-    protected int row()
-    {
-        return 0;
-    }
-
-    public Clustering clustering()
-    {
-        return clustering;
-    }
-
-    public LivenessInfo primaryKeyLivenessInfo()
-    {
-        return liveness;
-    }
-
-    public DeletionTime deletion()
-    {
-        return deletion;
-    }
-
-    public Row.Writer writer()
-    {
-        return writer.reset();
-    }
-
-    private class Writer extends RowDataBlock.Writer
-    {
-        public Writer(RowDataBlock data, boolean inOrderCells)
-        {
-            super(data, inOrderCells);
-        }
-
-        public void writeClusteringValue(ByteBuffer buffer)
-        {
-            clustering.writer().writeClusteringValue(buffer);
-        }
-
-        public void writePartitionKeyLivenessInfo(LivenessInfo info)
-        {
-            ReusableRow.this.liveness.setTo(info);
-        }
-
-        public void writeRowDeletion(DeletionTime deletion)
-        {
-            ReusableRow.this.deletion = deletion;
-        }
-
-        @Override
-        public Writer reset()
-        {
-            super.reset();
-            clustering.reset();
-            liveness.reset();
-            deletion = DeletionTime.LIVE;
-            return this;
-        }
-    }
-}