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 2013/12/17 17:03:44 UTC

[08/13] Push composites support in the storage engine

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
index bb00d23..16e74e3 100644
--- a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
+++ b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
@@ -87,7 +87,7 @@ public class LazilyCompactedRow extends AbstractCompactedRow
             emptyColumnFamily.purgeTombstones(controller.gcBefore);
 
         reducer = new Reducer();
-        merger = Iterators.filter(MergeIterator.get(rows, emptyColumnFamily.getComparator().onDiskAtomComparator, reducer), Predicates.notNull());
+        merger = Iterators.filter(MergeIterator.get(rows, emptyColumnFamily.getComparator().onDiskAtomComparator(), reducer), Predicates.notNull());
     }
 
     private static void removeDeletedAndOldShards(ColumnFamily cf, boolean shouldPurge, DecoratedKey key, CompactionController controller)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 3d52396..bec29d5 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -106,7 +106,7 @@ public class Scrubber implements Closeable
             ByteBuffer nextIndexKey = ByteBufferUtil.readWithShortLength(indexFile);
             {
                 // throw away variable so we don't have a side effect in the assert
-                long firstRowPositionFromIndex = RowIndexEntry.serializer.deserialize(indexFile, sstable.descriptor.version).position;
+                long firstRowPositionFromIndex = sstable.metadata.comparator.rowIndexEntrySerializer().deserialize(indexFile, sstable.descriptor.version).position;
                 assert firstRowPositionFromIndex == 0 : firstRowPositionFromIndex;
             }
 
@@ -146,7 +146,7 @@ public class Scrubber implements Closeable
                     nextIndexKey = indexFile.isEOF() ? null : ByteBufferUtil.readWithShortLength(indexFile);
                     nextRowPositionFromIndex = indexFile.isEOF()
                                              ? dataFile.length()
-                                             : RowIndexEntry.serializer.deserialize(indexFile, sstable.descriptor.version).position;
+                                             : sstable.metadata.comparator.rowIndexEntrySerializer().deserialize(indexFile, sstable.descriptor.version).position;
                 }
                 catch (Throwable th)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/AbstractCType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/AbstractCType.java b/src/java/org/apache/cassandra/db/composites/AbstractCType.java
new file mode 100644
index 0000000..2b07a46
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/AbstractCType.java
@@ -0,0 +1,336 @@
+/*
+ * 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.composites;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+
+import org.apache.cassandra.db.DeletionInfo;
+import org.apache.cassandra.db.RangeTombstone;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.filter.SliceQueryFilter;
+import org.apache.cassandra.db.marshal.AbstractCompositeType;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.io.ISerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.io.sstable.IndexHelper.IndexInfo;
+
+public abstract class AbstractCType implements CType
+{
+    private final Comparator<Composite> reverseComparator;
+    private final Comparator<IndexInfo> indexComparator;
+    private final Comparator<IndexInfo> indexReverseComparator;
+
+    private final Serializer serializer;
+
+    private final ISerializer<IndexInfo> indexSerializer;
+    private final IVersionedSerializer<ColumnSlice> sliceSerializer;
+    private final IVersionedSerializer<SliceQueryFilter> sliceQueryFilterSerializer;
+    private final DeletionInfo.Serializer deletionInfoSerializer;
+    private final RangeTombstone.Serializer rangeTombstoneSerializer;
+    private final RowIndexEntry.Serializer rowIndexEntrySerializer;
+
+    protected AbstractCType()
+    {
+        reverseComparator = new Comparator<Composite>()
+        {
+            public int compare(Composite c1, Composite c2)
+            {
+                return AbstractCType.this.compare(c2, c1);
+            }
+        };
+        indexComparator = new Comparator<IndexInfo>()
+        {
+            public int compare(IndexInfo o1, IndexInfo o2)
+            {
+                return AbstractCType.this.compare(o1.lastName, o2.lastName);
+            }
+        };
+        indexReverseComparator = new Comparator<IndexInfo>()
+        {
+            public int compare(IndexInfo o1, IndexInfo o2)
+            {
+                return AbstractCType.this.compare(o1.firstName, o2.firstName);
+            }
+        };
+
+        serializer = new Serializer(this);
+
+        indexSerializer = new IndexInfo.Serializer(this);
+        sliceSerializer = new ColumnSlice.Serializer(this);
+        sliceQueryFilterSerializer = new SliceQueryFilter.Serializer(this);
+        deletionInfoSerializer = new DeletionInfo.Serializer(this);
+        rangeTombstoneSerializer = new RangeTombstone.Serializer(this);
+        rowIndexEntrySerializer = new RowIndexEntry.Serializer(this);
+    }
+
+    public int compare(Composite c1, Composite c2)
+    {
+        if (c1 == null)
+            return c2 == null ? 0 : -1;
+
+        ByteBuffer previous = null;
+        int i;
+        int minSize = Math.min(c1.size(), c2.size());
+        for (i = 0; i < minSize; i++)
+        {
+            AbstractType<?> comparator = subtype(i);
+            ByteBuffer value1 = c1.get(i);
+            ByteBuffer value2 = c2.get(i);
+
+            int cmp = comparator.compareCollectionMembers(value1, value2, previous);
+            if (cmp != 0)
+                return cmp;
+
+            previous = value1;
+        }
+
+        if (c1.size() == c2.size())
+        {
+            if (c1.eoc() != c2.eoc())
+            {
+                switch (c1.eoc())
+                {
+                    case START: return -1;
+                    case END:   return 1;
+                    case NONE:  return c2.eoc() == Composite.EOC.START ? 1 : -1;
+                }
+            }
+            return 0;
+        }
+
+        if (i == c1.size())
+        {
+            return c1.eoc() == Composite.EOC.END ? 1 : -1;
+        }
+        else
+        {
+            assert i == c2.size();
+            return c2.eoc() == Composite.EOC.END ? -1 : 1;
+        }
+    }
+
+    public void validate(Composite name)
+    {
+        ByteBuffer previous = null;
+        for (int i = 0; i < name.size(); i++)
+        {
+            AbstractType<?> comparator = subtype(i);
+            ByteBuffer value = name.get(i);
+            comparator.validateCollectionMember(value, previous);
+            previous = value;
+        }
+    }
+
+    public boolean isCompatibleWith(CType previous)
+    {
+        if (this == previous)
+            return true;
+
+        // Extending with new components is fine, shrinking is not
+        if (size() < previous.size())
+            return false;
+
+        for (int i = 0; i < previous.size(); i++)
+        {
+            AbstractType<?> tprev = previous.subtype(i);
+            AbstractType<?> tnew = subtype(i);
+            if (!tnew.isCompatibleWith(tprev))
+                return false;
+        }
+        return true;
+    }
+
+    public String getString(Composite c)
+    {
+        StringBuilder sb = new StringBuilder();
+        for (int i = 0; i < c.size(); i++)
+        {
+            if (i > 0)
+                sb.append(":");
+            sb.append(AbstractCompositeType.escape(subtype(i).getString(c.get(i))));
+        }
+        switch (c.eoc())
+        {
+            case START:
+                sb.append(":_");
+                break;
+            case END:
+                sb.append(":!");
+                break;
+        }
+        return sb.toString();
+    }
+
+    public Composite make(Object... components)
+    {
+        if (components.length > size())
+            throw new IllegalArgumentException("Too many components, max is " + size());
+
+        CBuilder builder = builder();
+        for (int i = 0; i < components.length; i++)
+        {
+            Object obj = components[i];
+            if (obj instanceof ByteBuffer)
+                builder.add((ByteBuffer)obj);
+            else
+                builder.add(obj);
+        }
+        return builder.build();
+    }
+
+    public CType.Serializer serializer()
+    {
+        return serializer;
+    }
+
+    public Comparator<Composite> reverseComparator()
+    {
+        return reverseComparator;
+    }
+
+    public Comparator<IndexInfo> indexComparator()
+    {
+        return indexComparator;
+    }
+
+    public Comparator<IndexInfo> indexReverseComparator()
+    {
+        return indexReverseComparator;
+    }
+
+    public ISerializer<IndexInfo> indexSerializer()
+    {
+        return indexSerializer;
+    }
+
+    public IVersionedSerializer<ColumnSlice> sliceSerializer()
+    {
+        return sliceSerializer;
+    }
+
+    public IVersionedSerializer<SliceQueryFilter> sliceQueryFilterSerializer()
+    {
+        return sliceQueryFilterSerializer;
+    }
+
+    public DeletionInfo.Serializer deletionInfoSerializer()
+    {
+        return deletionInfoSerializer;
+    }
+
+    public RangeTombstone.Serializer rangeTombstoneSerializer()
+    {
+        return rangeTombstoneSerializer;
+    }
+
+    public RowIndexEntry.Serializer rowIndexEntrySerializer()
+    {
+        return rowIndexEntrySerializer;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (o == null)
+            return false;
+
+        if (!getClass().equals(o.getClass()))
+            return false;
+
+        CType c = (CType)o;
+        if (size() != c.size())
+            return false;
+
+        for (int i = 0; i < size(); i++)
+        {
+            if (!subtype(i).equals(c.subtype(i)))
+                return false;
+        }
+        return true;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        int h = 31;
+        for (int i = 0; i < size(); i++)
+            h += subtype(i).hashCode();
+        return h + getClass().hashCode();
+    }
+
+    @Override
+    public String toString()
+    {
+        return asAbstractType().toString();
+    }
+
+    protected static ByteBuffer sliceBytes(ByteBuffer bb, int offs, int length)
+    {
+        ByteBuffer copy = bb.duplicate();
+        copy.position(offs);
+        copy.limit(offs + length);
+        return copy;
+    }
+
+    protected static void checkRemaining(ByteBuffer bb, int offs, int length)
+    {
+        if (offs + length > bb.limit())
+            throw new IllegalArgumentException("Not enough bytes");
+    }
+
+    private static class Serializer implements CType.Serializer
+    {
+        private final CType type;
+
+        public Serializer(CType type)
+        {
+            this.type = type;
+        }
+
+        public void serialize(Composite c, DataOutput out) throws IOException
+        {
+            ByteBufferUtil.writeWithShortLength(c.toByteBuffer(), out);
+        }
+
+        public Composite deserialize(DataInput in) throws IOException
+        {
+            return type.fromByteBuffer(ByteBufferUtil.readWithShortLength(in));
+        }
+
+        public long serializedSize(Composite c, TypeSizes type)
+        {
+            return type.sizeofWithShortLength(c.toByteBuffer());
+        }
+
+        public void skip(DataInput in) throws IOException
+        {
+            ByteBufferUtil.skipShortLength(in);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java b/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
new file mode 100644
index 0000000..420ceab
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
@@ -0,0 +1,356 @@
+/*
+ * 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.composites;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.collect.AbstractIterator;
+import org.apache.cassandra.cql3.CQL3Row;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.filter.IDiskAtomFilter;
+import org.apache.cassandra.db.filter.NamesQueryFilter;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ColumnToCollectionType;
+import org.apache.cassandra.io.ISerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public abstract class AbstractCellNameType extends AbstractCType implements CellNameType
+{
+    private final Comparator<Column> columnComparator;
+    private final Comparator<Column> columnReverseComparator;
+    private final Comparator<OnDiskAtom> onDiskAtomComparator;
+
+    private final ISerializer<CellName> cellSerializer;
+    private final ColumnSerializer columnSerializer;
+    private final OnDiskAtom.Serializer onDiskAtomSerializer;
+    private final IVersionedSerializer<NamesQueryFilter> namesQueryFilterSerializer;
+    private final IVersionedSerializer<IDiskAtomFilter> diskAtomFilterSerializer;
+
+    protected AbstractCellNameType()
+    {
+        columnComparator = new Comparator<Column>()
+        {
+            public int compare(Column c1, Column c2)
+            {
+                return AbstractCellNameType.this.compare(c1.name(), c2.name());
+            }
+        };
+        columnReverseComparator = new Comparator<Column>()
+        {
+            public int compare(Column c1, Column c2)
+            {
+                return AbstractCellNameType.this.compare(c2.name(), c1.name());
+            }
+        };
+        onDiskAtomComparator = new Comparator<OnDiskAtom>()
+        {
+            public int compare(OnDiskAtom c1, OnDiskAtom c2)
+            {
+                int comp = AbstractCellNameType.this.compare(c1.name(), c2.name());
+                if (comp != 0)
+                    return comp;
+
+                if (c1 instanceof RangeTombstone)
+                {
+                    if (c2 instanceof RangeTombstone)
+                    {
+                        RangeTombstone t1 = (RangeTombstone)c1;
+                        RangeTombstone t2 = (RangeTombstone)c2;
+                        int comp2 = AbstractCellNameType.this.compare(t1.max, t2.max);
+                        return comp2 == 0 ? t1.data.compareTo(t2.data) : comp2;
+                    }
+                    else
+                    {
+                        return -1;
+                    }
+                }
+                else
+                {
+                    return c2 instanceof RangeTombstone ? 1 : 0;
+                }
+            }
+        };
+
+        // A trivial wrapped over the composite serializer
+        cellSerializer = new ISerializer<CellName>()
+        {
+            public void serialize(CellName c, DataOutput out) throws IOException
+            {
+                serializer().serialize(c, out);
+            }
+
+            public CellName deserialize(DataInput in) throws IOException
+            {
+                Composite ct = serializer().deserialize(in);
+                if (ct.isEmpty())
+                    throw ColumnSerializer.CorruptColumnException.create(in, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+
+                assert ct instanceof CellName : ct;
+                return (CellName)ct;
+            }
+
+            public long serializedSize(CellName c, TypeSizes type)
+            {
+                return serializer().serializedSize(c, type);
+            }
+        };
+        columnSerializer = new ColumnSerializer(this);
+        onDiskAtomSerializer = new OnDiskAtom.Serializer(this);
+        namesQueryFilterSerializer = new NamesQueryFilter.Serializer(this);
+        diskAtomFilterSerializer = new IDiskAtomFilter.Serializer(this);
+    }
+
+    public Comparator<Column> columnComparator()
+    {
+        return columnComparator;
+    }
+
+    public Comparator<Column> columnReverseComparator()
+    {
+        return columnReverseComparator;
+    }
+
+    public Comparator<OnDiskAtom> onDiskAtomComparator()
+    {
+        return onDiskAtomComparator;
+    }
+
+    public ISerializer<CellName> cellSerializer()
+    {
+        return cellSerializer;
+    }
+
+    public ColumnSerializer columnSerializer()
+    {
+        return columnSerializer;
+    }
+
+    public OnDiskAtom.Serializer onDiskAtomSerializer()
+    {
+        return onDiskAtomSerializer;
+    }
+
+    public IVersionedSerializer<NamesQueryFilter> namesQueryFilterSerializer()
+    {
+        return namesQueryFilterSerializer;
+    }
+
+    public IVersionedSerializer<IDiskAtomFilter> diskAtomFilterSerializer()
+    {
+        return diskAtomFilterSerializer;
+    }
+
+    public CellName cellFromByteBuffer(ByteBuffer bytes)
+    {
+        return (CellName)fromByteBuffer(bytes);
+    }
+
+    public CellName create(Composite prefix, ColumnIdentifier columnName, ByteBuffer collectionElement)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public CellName rowMarker(Composite prefix)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public boolean hasCollections()
+    {
+        return false;
+    }
+
+    public boolean supportCollections()
+    {
+        return false;
+    }
+
+    public ColumnToCollectionType collectionType()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public CellNameType addCollection(ColumnIdentifier columnName, CollectionType newCollection)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Composite make(Object... components)
+    {
+        return components.length == size() ? makeCellName(components) : super.make(components);
+    }
+
+    public CellName makeCellName(Object... components)
+    {
+        ByteBuffer[] rawComponents = new ByteBuffer[components.length];
+        for (int i = 0; i < components.length; i++)
+        {
+            Object c = components[i];
+            rawComponents[i] = c instanceof ByteBuffer ? (ByteBuffer)c : ((AbstractType)subtype(i)).decompose(c);
+        }
+        return makeCellName(rawComponents);
+    }
+
+    protected abstract CellName makeCellName(ByteBuffer[] components);
+
+    protected static CQL3Row.Builder makeDenseCQL3RowBuilder(final long now)
+    {
+        return new CQL3Row.Builder()
+        {
+            public Iterator<CQL3Row> group(final Iterator<Column> cells)
+            {
+                return new AbstractIterator<CQL3Row>()
+                {
+                    protected CQL3Row computeNext()
+                    {
+                        while (cells.hasNext())
+                        {
+                            final Column cell = cells.next();
+                            if (cell.isMarkedForDelete(now))
+                                continue;
+
+                            return new CQL3Row()
+                            {
+                                public ByteBuffer getClusteringColumn(int i)
+                                {
+                                    return cell.name().get(i);
+                                }
+
+                                public Column getColumn(ColumnIdentifier name)
+                                {
+                                    return cell;
+                                }
+
+                                public List<Column> getCollection(ColumnIdentifier name)
+                                {
+                                    return null;
+                                }
+                            };
+                        }
+                        return endOfData();
+                    }
+                };
+            }
+        };
+    }
+
+    protected static CQL3Row.Builder makeSparseCQL3RowBuilder(final long now)
+    {
+        return new CQL3Row.Builder()
+        {
+            public Iterator<CQL3Row> group(final Iterator<Column> cells)
+            {
+                return new AbstractIterator<CQL3Row>()
+                {
+                    private CellName previous;
+                    private CQL3RowOfSparse currentRow;
+
+                    protected CQL3Row computeNext()
+                    {
+                        while (cells.hasNext())
+                        {
+                            final Column cell = cells.next();
+                            if (cell.isMarkedForDelete(now))
+                                continue;
+
+                            CQL3Row toReturn = null;
+                            CellName current = cell.name();
+                            if (currentRow == null || !current.isSameCQL3RowAs(previous))
+                            {
+                                toReturn = currentRow;
+                                currentRow = new CQL3RowOfSparse(current);
+                            }
+                            currentRow.add(cell);
+                            previous = current;
+
+                            if (toReturn != null)
+                                return toReturn;
+                        }
+                        if (currentRow != null)
+                        {
+                            CQL3Row toReturn = currentRow;
+                            currentRow = null;
+                            return toReturn;
+                        }
+                        return endOfData();
+                    }
+                };
+            }
+        };
+    }
+
+    private static class CQL3RowOfSparse implements CQL3Row
+    {
+        private final CellName cell;
+        private Map<ColumnIdentifier, Column> columns;
+        private Map<ColumnIdentifier, List<Column>> collections;
+
+        CQL3RowOfSparse(CellName cell)
+        {
+            this.cell = cell;
+        }
+
+        public ByteBuffer getClusteringColumn(int i)
+        {
+            return cell.get(i);
+        }
+
+        void add(Column cell)
+        {
+            CellName cellName = cell.name();
+            ColumnIdentifier columnName =  cellName.cql3ColumnName();
+            if (cellName.isCollectionCell())
+            {
+                if (collections == null)
+                    collections = new HashMap<>();
+
+                List<Column> values = collections.get(columnName);
+                if (values == null)
+                {
+                    values = new ArrayList<Column>();
+                    collections.put(columnName, values);
+                }
+                values.add(cell);
+            }
+            else
+            {
+                if (columns == null)
+                    columns = new HashMap<>();
+                columns.put(columnName, cell);
+            }
+        }
+
+        public Column getColumn(ColumnIdentifier name)
+        {
+            return columns == null ? null : columns.get(name);
+        }
+
+        public List<Column> getCollection(ColumnIdentifier name)
+        {
+            return collections == null ? null : collections.get(name);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/AbstractComposite.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/AbstractComposite.java b/src/java/org/apache/cassandra/db/composites/AbstractComposite.java
new file mode 100644
index 0000000..e23c560
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/AbstractComposite.java
@@ -0,0 +1,132 @@
+/*
+ * 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.composites;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.marshal.AbstractCompositeType;
+
+public abstract class AbstractComposite implements Composite
+{
+    public boolean isEmpty()
+    {
+        return size() == 0;
+    }
+
+    public EOC eoc()
+    {
+        return EOC.NONE;
+    }
+
+    public Composite start()
+    {
+        return withEOC(EOC.START);
+    }
+
+    public Composite end()
+    {
+        return withEOC(EOC.END);
+    }
+
+    public Composite withEOC(EOC newEoc)
+    {
+        // Note: CompositeBound overwrite this so we assume the EOC of this is NONE
+        switch (newEoc)
+        {
+            case START:
+                return BoundedComposite.startOf(this);
+            case END:
+                return BoundedComposite.endOf(this);
+            default:
+                return this;
+        }
+    }
+
+    public ColumnSlice slice()
+    {
+        return new ColumnSlice(start(), end());
+    }
+
+    public ByteBuffer toByteBuffer()
+    {
+        // This is the legacy format of composites.
+        // See org.apache.cassandra.db.marshal.CompositeType for details.
+        ByteBuffer result = ByteBuffer.allocate(dataSize() + 3 * size());
+        for (int i = 0; i < size(); i++)
+        {
+            ByteBuffer bb = get(i);
+            AbstractCompositeType.putShortLength(result, bb.remaining());
+            result.put(bb.duplicate());
+            result.put((byte)0);
+        }
+        result.flip();
+        return result;
+    }
+
+    public int dataSize()
+    {
+        int size = 0;
+        for (int i = 0; i < size(); i++)
+            size += get(i).remaining();
+        return size;
+    }
+
+    public boolean isPrefixOf(Composite c)
+    {
+        if (size() > c.size())
+            return false;
+
+        for (int i = 0; i < size(); i++)
+        {
+            if (!get(i).equals(c.get(i)))
+                return false;
+        }
+        return true;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if(!(o instanceof Composite))
+            return false;
+
+        Composite c = (Composite)o;
+        if (size() != c.size())
+            return false;
+
+        for (int i = 0; i < size(); i++)
+        {
+            if (!get(i).equals(c.get(i)))
+                return false;
+        }
+        return eoc() == c.eoc();
+    }
+
+    @Override
+    public int hashCode()
+    {
+        int h = 31;
+        for (int i = 0; i < size(); i++)
+            h += get(i).hashCode();
+        return h + eoc().hashCode();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java b/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
new file mode 100644
index 0000000..0602f50
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
@@ -0,0 +1,264 @@
+/*
+ * 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.composites;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+
+public abstract class AbstractCompoundCellNameType extends AbstractCellNameType
+{
+    protected final CompoundCType clusteringType;
+    protected final CompoundCType fullType;
+
+    protected final int clusteringSize;
+    protected final int fullSize;
+
+    protected AbstractCompoundCellNameType(CompoundCType clusteringType, CompoundCType fullType)
+    {
+        this.clusteringType = clusteringType;
+        this.fullType = fullType;
+
+        this.clusteringSize = clusteringType.size();
+        this.fullSize = fullType.size();
+    }
+
+    public int clusteringPrefixSize()
+    {
+        return clusteringSize;
+    }
+
+    public boolean isCompound()
+    {
+        return true;
+    }
+
+    public int size()
+    {
+        return fullSize;
+    }
+
+    public AbstractType<?> subtype(int i)
+    {
+        return fullType.subtype(i);
+    }
+
+    public CBuilder prefixBuilder()
+    {
+        return clusteringType.builder();
+    }
+
+    public CBuilder builder()
+    {
+        return new CompoundCType.CompoundCBuilder(this);
+    }
+
+    @Override
+    public Composite fromByteBuffer(ByteBuffer bytes)
+    {
+        if (!bytes.hasRemaining())
+            return Composites.EMPTY;
+
+        ByteBuffer[] elements = new ByteBuffer[fullSize];
+        int idx = bytes.position(), i = 0;
+        byte eoc = 0;
+        while (idx < bytes.limit())
+        {
+            checkRemaining(bytes, idx, 2);
+            int length = bytes.getShort(idx) & 0xFFFF;
+            idx += 2;
+
+            checkRemaining(bytes, idx, length + 1);
+            elements[i++] = sliceBytes(bytes, idx, length);
+            idx += length;
+            eoc = bytes.get(idx++);
+        }
+
+        return makeWith(elements, i, Composite.EOC.from(eoc));
+    }
+
+    public AbstractType<?> asAbstractType()
+    {
+        return CompositeType.getInstance(fullType.types);
+    }
+
+    public Deserializer newDeserializer(DataInput in)
+    {
+        return new CompositeDeserializer(this, in);
+    }
+
+    protected CellName makeCellName(ByteBuffer[] components)
+    {
+        return (CellName)makeWith(components, components.length, Composite.EOC.NONE);
+    }
+
+    protected abstract Composite makeWith(ByteBuffer[] components, int size, Composite.EOC eoc);
+    protected abstract Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc);
+
+    private static class CompositeDeserializer implements CellNameType.Deserializer
+    {
+        private static byte[] EMPTY = new byte[0];
+
+        private final AbstractCompoundCellNameType type;
+        private final DataInput in;
+
+        private byte[] nextFull;
+        private int nextIdx;
+
+        private final ByteBuffer[] nextComponents;
+        private int nextSize;
+        private Composite.EOC nextEOC;
+
+        public CompositeDeserializer(AbstractCompoundCellNameType type, DataInput in)
+        {
+            this.type = type;
+            this.in = in;
+            this.nextComponents = new ByteBuffer[type.size()];
+        }
+
+        public boolean hasNext() throws IOException
+        {
+            if (nextFull == null)
+                maybeReadNext();
+            return nextFull != EMPTY;
+        }
+
+        public boolean hasUnprocessed() throws IOException
+        {
+            return nextFull != null;
+        }
+
+        public int compareNextTo(Composite composite) throws IOException
+        {
+            maybeReadNext();
+
+            if (composite.isEmpty())
+                return nextFull == EMPTY ? 0 : 1;
+
+            if (nextFull == EMPTY)
+                return -1;
+
+            ByteBuffer previous = null;
+            for (int i = 0; i < composite.size(); i++)
+            {
+                if (!hasComponent(i))
+                    return nextEOC == Composite.EOC.END ? 1 : -1;
+
+                AbstractType<?> comparator = type.subtype(i);
+                ByteBuffer value1 = nextComponents[i];
+                ByteBuffer value2 = composite.get(i);
+
+                int cmp = comparator.compareCollectionMembers(value1, value2, previous);
+                if (cmp != 0)
+                    return cmp;
+
+                previous = value1;
+            }
+
+            // If we have more component than composite
+            if (!allComponentsDeserialized() || composite.size() < nextSize)
+                return composite.eoc() == Composite.EOC.END ? -1 : 1;
+
+            // same size, check eoc
+            if (nextEOC != composite.eoc())
+            {
+                switch (nextEOC)
+                {
+                    case START: return -1;
+                    case END:   return 1;
+                    case NONE:  return composite.eoc() == Composite.EOC.START ? 1 : -1;
+                }
+            }
+
+            return 0;
+        }
+
+        private boolean hasComponent(int i)
+        {
+            while (i >= nextSize && deserializeOne())
+                continue;
+
+            return i < nextSize;
+        }
+
+        private boolean deserializeOne()
+        {
+            if (allComponentsDeserialized())
+                return false;
+
+            int length = ((nextFull[nextIdx++] & 0xFF) << 8) | (nextFull[nextIdx++] & 0xFF);
+            ByteBuffer component = ByteBuffer.wrap(nextFull, nextIdx, length);
+            nextIdx += length;
+            nextComponents[nextSize++] = component;
+            nextEOC = Composite.EOC.from(nextFull[nextIdx++]);
+            return true;
+        }
+
+        private void deserializeAll()
+        {
+            while (deserializeOne())
+                continue;
+        }
+
+        private boolean allComponentsDeserialized()
+        {
+            return nextIdx >= nextFull.length;
+        }
+
+        private void maybeReadNext() throws IOException
+        {
+            if (nextFull != null)
+                return;
+
+            nextIdx = 0;
+            nextSize = 0;
+
+            int length = in.readShort() & 0xFFFF;
+            // Note that empty is ok because it marks the end of row
+            if (length == 0)
+            {
+                nextFull = EMPTY;
+                return;
+            }
+
+            nextFull = new byte[length];
+            in.readFully(nextFull);
+        }
+
+        public Composite readNext() throws IOException
+        {
+            maybeReadNext();
+            if (nextFull == EMPTY)
+                return Composites.EMPTY;
+
+            deserializeAll();
+            Composite c = type.copyAndMakeWith(nextComponents, nextSize, nextEOC);
+            nextFull = null;
+            return c;
+        }
+
+        public void skipNext() throws IOException
+        {
+            maybeReadNext();
+            nextFull = null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/AbstractSimpleCellNameType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/AbstractSimpleCellNameType.java b/src/java/org/apache/cassandra/db/composites/AbstractSimpleCellNameType.java
new file mode 100644
index 0000000..94c0c4d
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/AbstractSimpleCellNameType.java
@@ -0,0 +1,141 @@
+/*
+ * 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.composites;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public abstract class AbstractSimpleCellNameType extends AbstractCellNameType
+{
+    protected final AbstractType<?> type;
+
+    protected AbstractSimpleCellNameType(AbstractType<?> type)
+    {
+        this.type = type;
+    }
+
+    public boolean isCompound()
+    {
+        return false;
+    }
+
+    public int size()
+    {
+        return 1;
+    }
+
+    public AbstractType<?> subtype(int i)
+    {
+        if (i != 0)
+            throw new IllegalArgumentException();
+        return type;
+    }
+
+    protected CellName makeCellName(ByteBuffer[] components)
+    {
+        assert components.length == 1;
+        return cellFromByteBuffer(components[0]);
+    }
+
+    public CBuilder builder()
+    {
+        return new SimpleCType.SimpleCBuilder(this);
+    }
+
+    public AbstractType<?> asAbstractType()
+    {
+        return type;
+    }
+
+    public Deserializer newDeserializer(DataInput in)
+    {
+        return new SimpleDeserializer(this, in);
+    }
+
+    private static class SimpleDeserializer implements CellNameType.Deserializer
+    {
+        private final AbstractSimpleCellNameType type;
+        private ByteBuffer next;
+        private final DataInput in;
+
+        public SimpleDeserializer(AbstractSimpleCellNameType type, DataInput in)
+        {
+            this.type = type;
+            this.in = in;
+        }
+
+        public boolean hasNext() throws IOException
+        {
+            if (next == null)
+                maybeReadNext();
+
+            return next.hasRemaining();
+        }
+
+        public boolean hasUnprocessed() throws IOException
+        {
+            return next != null;
+        }
+
+        public int compareNextTo(Composite composite) throws IOException
+        {
+            maybeReadNext();
+
+            if (composite.isEmpty())
+                return next.hasRemaining() ? 1 : 0;
+
+            return type.subtype(0).compare(next, composite.get(0));
+        }
+
+        private void maybeReadNext() throws IOException
+        {
+            if (next != null)
+                return;
+
+            int length = in.readShort() & 0xFFFF;
+            // Note that empty is ok because it marks the end of row
+            if (length == 0)
+            {
+                next = ByteBufferUtil.EMPTY_BYTE_BUFFER;
+                return;
+            }
+
+            byte[] b = new byte[length];
+            in.readFully(b);
+            next = ByteBuffer.wrap(b);
+        }
+
+        public Composite readNext() throws IOException
+        {
+            maybeReadNext();
+            Composite c = type.fromByteBuffer(next);
+            next = null;
+            return c;
+        }
+
+        public void skipNext() throws IOException
+        {
+            maybeReadNext();
+            next = null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/BoundedComposite.java b/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
new file mode 100644
index 0000000..9e56dc2
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
@@ -0,0 +1,95 @@
+/*
+ * 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.composites;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.ObjectSizes;
+
+public class BoundedComposite extends AbstractComposite
+{
+    private final Composite wrapped;
+    private final boolean isStart;
+
+    private BoundedComposite(Composite wrapped, boolean isStart)
+    {
+        this.wrapped = wrapped;
+        this.isStart = isStart;
+    }
+
+    static Composite startOf(Composite c)
+    {
+        return new BoundedComposite(c, true);
+    }
+
+    static Composite endOf(Composite c)
+    {
+        return new BoundedComposite(c, false);
+    }
+
+    public int size()
+    {
+        return wrapped.size();
+    }
+
+    public ByteBuffer get(int i)
+    {
+        return wrapped.get(i);
+    }
+
+    @Override
+    public EOC eoc()
+    {
+        return isStart ? EOC.START : EOC.END;
+    }
+
+    @Override
+    public Composite withEOC(EOC eoc)
+    {
+        switch (eoc)
+        {
+            case START:
+                return isStart ? this : startOf(wrapped);
+            case END:
+                return isStart ? endOf(wrapped) : this;
+            default:
+                return wrapped;
+        }
+    }
+
+    @Override
+    public ByteBuffer toByteBuffer()
+    {
+        ByteBuffer bb = wrapped.toByteBuffer();
+        bb.put(bb.remaining() - 1, (byte)(isStart ? -1 : 1));
+        return bb;
+    }
+
+    public long memorySize()
+    {
+        return ObjectSizes.getFieldSize(ObjectSizes.getReferenceSize() + TypeSizes.NATIVE.sizeof(isStart))
+             + wrapped.memorySize();
+    }
+
+    public Composite copy(Allocator allocator)
+    {
+        return new BoundedComposite(wrapped.copy(allocator), isStart);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/CBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CBuilder.java b/src/java/org/apache/cassandra/db/composites/CBuilder.java
new file mode 100644
index 0000000..490fb67
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/CBuilder.java
@@ -0,0 +1,34 @@
+/*
+ * 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.composites;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A builder of Composite.
+ */
+public interface CBuilder
+{
+    public int remainingCount();
+
+    public CBuilder add(ByteBuffer value);
+    public CBuilder add(Object value);
+
+    public Composite build();
+    public Composite buildWith(ByteBuffer value);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/CType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CType.java b/src/java/org/apache/cassandra/db/composites/CType.java
new file mode 100644
index 0000000..c2516f5
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/CType.java
@@ -0,0 +1,142 @@
+/*
+ * 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.composites;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+
+import org.apache.cassandra.db.DeletionInfo;
+import org.apache.cassandra.db.RangeTombstone;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.filter.SliceQueryFilter;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.io.ISerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
+
+import static org.apache.cassandra.io.sstable.IndexHelper.IndexInfo;
+
+/**
+ * A type for a Composite.
+ *
+ * There is essentially 2 types of Composite and such of CType:
+ *   1. the "simple" ones, see SimpleCType.
+ *   2. the "truly-composite" ones, see CompositeCType.
+ *
+ * API-wise, a CType is simply a collection of AbstractType with a few utility
+ * methods.
+ */
+public interface CType extends Comparator<Composite>
+{
+    /**
+     * Returns whether this is a "truly-composite" underneath.
+     */
+    public boolean isCompound();
+
+    /**
+     * The number of subtypes for this CType.
+     */
+    public int size();
+
+    /**
+     * Gets a subtype of this CType.
+     */
+    public AbstractType<?> subtype(int i);
+
+    /**
+     * A builder of Composite.
+     */
+    public CBuilder builder();
+
+    /**
+     * Convenience method to build composites from their component.
+     *
+     * The arguments can be either ByteBuffer or actual objects of the type
+     * corresponding to their position.
+     */
+    public Composite make(Object... components);
+
+    /**
+     * Validates a composite.
+     */
+    public void validate(Composite name);
+
+    /**
+     * Converts a composite to a user-readable string.
+     */
+    public String getString(Composite c);
+
+    /**
+     * See AbstractType#isCompatibleWith.
+     */
+    public boolean isCompatibleWith(CType previous);
+
+    /**
+     * Returns a new CType that is equivalent to this CType but with
+     * one of the subtype replaced by the provided new type.
+     */
+    public CType setSubtype(int position, AbstractType<?> newType);
+
+    /**
+     * Deserialize a Composite from a ByteBuffer.
+     *
+     * This is meant for thrift/cql2 to convert the fully serialized buffer we
+     * get from the clients to composites.
+     */
+    public Composite fromByteBuffer(ByteBuffer bb);
+
+    /**
+     * Returns a AbstractType corresponding to this CType for thrift/cql2 sake.
+     *
+     * If the CType is a "simple" one, this just return the wrapped type, otherwise
+     * it returns the corresponding org.apache.cassandra.db.marshal.CompositeType.
+     *
+     * This is only meant to be use for backward compatibility (particularly for
+     * thrift/cql2) but it's not meant to be used internally.
+     */
+    public AbstractType<?> asAbstractType();
+
+
+    /**********************************************************/
+
+    /*
+     * Follows a number of per-CType instances for the Comparator and Serializer used throughout
+     * the code. The reason we need this is that we want the per-CType/per-CellNameType Composite/CellName
+     * serializers, which means the following instances have to depend on the type too.
+     */
+
+    public Comparator<Composite> reverseComparator();
+    public Comparator<IndexInfo> indexComparator();
+    public Comparator<IndexInfo> indexReverseComparator();
+
+    public Serializer serializer();
+
+    public ISerializer<IndexInfo> indexSerializer();
+    public IVersionedSerializer<ColumnSlice> sliceSerializer();
+    public IVersionedSerializer<SliceQueryFilter> sliceQueryFilterSerializer();
+    public DeletionInfo.Serializer deletionInfoSerializer();
+    public RangeTombstone.Serializer rangeTombstoneSerializer();
+    public RowIndexEntry.Serializer rowIndexEntrySerializer();
+
+    public interface Serializer extends ISerializer<Composite>
+    {
+        public void skip(DataInput in) throws IOException;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/CellName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CellName.java b/src/java/org/apache/cassandra/db/composites/CellName.java
new file mode 100644
index 0000000..1ca50eb
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/CellName.java
@@ -0,0 +1,74 @@
+/*
+ * 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.composites;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.utils.Allocator;
+
+/**
+ * A CellName is a Composite, but for which, for the sake of CQL3, we
+ * distinguish different parts: a CellName has first a number of clustering
+ * components, followed by the CQL3 column name, and then possibly followed by
+ * a collection element part.
+ *
+ * The clustering prefix can itself be composed of multiple component. It can
+ * also be empty if the table has no clustering keys. In general, the CQL3
+ * column name follows. However, some type of COMPACT STORAGE layout do not
+ * store the CQL3 column name in the cell name and so this part can be null (we
+ * call "dense" the cells whose name don't store the CQL3 column name).
+ *
+ * Lastly, if the cell is part of a CQL3 collection, we'll have a last
+ * component (a UUID for lists, an element for sets and a key for maps).
+ */
+public interface CellName extends Composite
+{
+    /**
+     * The number of clustering components.
+     *
+     * It can be 0 if the table has no clustering columns, and it can be
+     * equal to size() if the table is dense() (in which case cql3ColumnName()
+     * will be null).
+     */
+    public int clusteringSize();
+
+    /**
+     * The name of the CQL3 column this cell represents.
+     *
+     * Will be null for cells of "dense" tables.
+     */
+    public ColumnIdentifier cql3ColumnName();
+
+    /**
+     * The value of the collection element, or null if the cell is not part
+     * of a collection (i.e. if !isCollectionCell()).
+     */
+    public ByteBuffer collectionElement();
+    public boolean isCollectionCell();
+
+    /**
+     * Whether this cell is part of the same CQL3 row as the other cell.
+     */
+    public boolean isSameCQL3RowAs(CellName other);
+
+    // If cellnames were sharing some prefix components, this will break it, so
+    // we might want to try to do better.
+    @Override
+    public CellName copy(Allocator allocator);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/CellNameType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CellNameType.java b/src/java/org/apache/cassandra/db/composites/CellNameType.java
new file mode 100644
index 0000000..53d89f2
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/CellNameType.java
@@ -0,0 +1,202 @@
+/*
+ * 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.composites;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+
+import org.apache.cassandra.cql3.CQL3Row;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.ColumnSerializer;
+import org.apache.cassandra.db.OnDiskAtom;
+import org.apache.cassandra.db.filter.IDiskAtomFilter;
+import org.apache.cassandra.db.filter.NamesQueryFilter;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ColumnToCollectionType;
+import org.apache.cassandra.io.ISerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
+
+/**
+ * The type of CellNames.
+ *
+ * In the same way that a CellName is a Composite, a CellNameType is a CType, but
+ * with a number of method specific to cell names.
+ *
+ * On top of the dichotomy simple/truly-composite of composites, cell names comes
+ * in 2 variants: "dense" and "sparse". The sparse ones are CellName where one of
+ * the component (the last or second-to-last for collections) is used to store the
+ * CQL3 column name. Dense are those for which it's not the case.
+ *
+ * In other words, we have 4 types of CellName/CellNameType which correspond to the
+ * 4 type of table layout that we need to distinguish:
+ *   1. Simple (non-truly-composite) dense: this is the dynamic thrift CFs whose
+ *      comparator is not composite.
+ *   2. Composite dense: this is the dynamic thrift CFs with a CompositeType comparator.
+ *   3. Simple (non-truly-composite) sparse: this is the thrift static CFs (that
+ *      don't have a composite comparator).
+ *   4. Composite sparse: this is the CQL3 layout (note that this is the only one that
+ *      support collections).
+ */
+public interface CellNameType extends CType
+{
+    /**
+     * Whether or not the cell names for this type are dense.
+     */
+    public boolean isDense();
+
+    /**
+     * The number of clustering columns for the table this is the type of.
+     */
+    public int clusteringPrefixSize();
+
+    /**
+     * A builder for the clustering prefix.
+     */
+    public CBuilder prefixBuilder();
+
+    /**
+     * Whether or not there is some collections defined in this type.
+     */
+    public boolean hasCollections();
+
+    /**
+     * Whether or not this type layout support collections.
+     */
+    public boolean supportCollections();
+
+    /**
+     * The type of the collections (or null if the type has not collections).
+     */
+    public ColumnToCollectionType collectionType();
+
+    /**
+     * Return the new type obtained by adding the new collection type for the provided column name
+     * to this type.
+     */
+    public CellNameType addCollection(ColumnIdentifier columnName, CollectionType newCollection);
+
+    /**
+     * Returns a new CellNameType that is equivalent to this one but with one
+     * of the subtype replaced by the provided new type.
+     */
+    @Override
+    public CellNameType setSubtype(int position, AbstractType<?> newType);
+
+    /**
+     * Creates a row marker for the CQL3 having the provided clustering prefix.
+     *
+     * Note that this is only valid for CQL3 tables (isCompound() and !isDense()) and should
+     * only be called for them.
+     */
+    public CellName rowMarker(Composite prefix);
+
+    /**
+     * Creates a new CellName given a clustering prefix and a CQL3 columnName.
+     *
+     * Note that for dense types, the columnName can be null.
+     */
+    public CellName create(Composite prefix, ColumnIdentifier columnName);
+
+    /**
+     * Creates a new collection CellName given a clustering prefix, a CQL3 columnName and the collection element.
+     */
+    public CellName create(Composite prefix, ColumnIdentifier columnName, ByteBuffer collectionElement);
+
+    /**
+     * Convenience method to create cell names given its components.
+     *
+     * This is equivalent to CType#make() but return a full cell name (and thus
+     * require all the components of the name).
+     */
+    public CellName makeCellName(Object... components);
+
+    /**
+     * Deserialize a Composite from a ByteBuffer.
+     *
+     * This is equilvalent to CType#fromByteBuffer but assumes the buffer is a full cell
+     * name. This is meant for thrift/cql2 to convert the fully serialized buffer we
+     * get from the clients.
+     */
+    public CellName cellFromByteBuffer(ByteBuffer bb);
+
+    /**
+     * Creates a new CQL3Row builder for this type. See CQL3Row for details.
+     */
+    public CQL3Row.Builder CQL3RowBuilder(long now);
+
+    // The two following methods are used to pass the declared regular column names (in CFMetaData)
+    // to the CellNameType. This is only used for optimization sake, see SparseCellNameType.
+    public void addCQL3Column(ColumnIdentifier id);
+    public void removeCQL3Column(ColumnIdentifier id);
+
+    /**
+     * Creates a new Deserializer. This is used by AtomDeserializer to do incremental and on-demand
+     * deserialization of the on disk atoms. See AtomDeserializer for details.
+     */
+    public Deserializer newDeserializer(DataInput in);
+
+    /*
+     * Same as in CType, follows a number of per-CellNameType instances for the Comparator and Serializer used
+     * throughout the code (those that require full CellName versus just Composite).
+     */
+
+    // Ultimately, those might be split into an IVersionedSerializer and an ISSTableSerializer
+    public ISerializer<CellName> cellSerializer();
+
+    public Comparator<Column> columnComparator();
+    public Comparator<Column> columnReverseComparator();
+    public Comparator<OnDiskAtom> onDiskAtomComparator();
+
+    public ColumnSerializer columnSerializer();
+    public OnDiskAtom.Serializer onDiskAtomSerializer();
+    public IVersionedSerializer<NamesQueryFilter> namesQueryFilterSerializer();
+    public IVersionedSerializer<IDiskAtomFilter> diskAtomFilterSerializer();
+
+    public interface Deserializer
+    {
+        /**
+         * Whether this deserializer is done or not, i.e. whether we're reached the end of row marker.
+         */
+        public boolean hasNext() throws IOException;
+
+        /**
+         * Whether or not some name has been read but not consumed by readNext.
+         */
+        public boolean hasUnprocessed() throws IOException;
+
+        /**
+         * Comparare the next name to read to the provided Composite.
+         * This does not consume the next name.
+         */
+        public int compareNextTo(Composite composite) throws IOException;
+
+        /**
+         * Actually consume the next name and return it.
+         */
+        public Composite readNext() throws IOException;
+
+        /**
+         * Skip the next name (consuming it).
+         */
+        public void skipNext() throws IOException;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/CellNames.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CellNames.java b/src/java/org/apache/cassandra/db/composites/CellNames.java
new file mode 100644
index 0000000..15f4e16
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/CellNames.java
@@ -0,0 +1,91 @@
+/*
+ * 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.composites;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ColumnToCollectionType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+
+public abstract class CellNames
+{
+    private CellNames() {}
+
+    public static CellNameType fromAbstractType(AbstractType<?> type, boolean isDense)
+    {
+        if (isDense)
+        {
+            if (type instanceof CompositeType)
+            {
+                return new CompoundDenseCellNameType(((CompositeType)type).types);
+            }
+            else
+            {
+                return new SimpleDenseCellNameType(type);
+            }
+        }
+        else
+        {
+            if (type instanceof CompositeType)
+            {
+                List<AbstractType<?>> types = ((CompositeType)type).types;
+                if (types.get(types.size() - 1) instanceof ColumnToCollectionType)
+                {
+                    assert types.get(types.size() - 2) instanceof UTF8Type;
+                    return new CompoundSparseCellNameType.WithCollection(types.subList(0, types.size() - 2), (ColumnToCollectionType)types.get(types.size() - 1));
+                }
+                else
+                {
+                    assert types.get(types.size() - 1) instanceof UTF8Type;
+                    return new CompoundSparseCellNameType(types.subList(0, types.size() - 1));
+                }
+            }
+            else
+            {
+                return new SimpleSparseCellNameType(type);
+            }
+        }
+    }
+
+    // Mainly for tests and a few cases where we know what we need and didn't wanted to pass the type around.
+    // Avoid in general, prefer the CellNameType methods.
+    public static CellName simpleDense(ByteBuffer bb)
+    {
+        assert bb.hasRemaining();
+        return new SimpleDenseCellName(bb);
+    }
+
+    // Mainly for tests and a few cases where we know what we need and didn't wanted to pass the type around
+    // Avoid in general, prefer the CellNameType methods.
+    public static CellName compositeDense(ByteBuffer... bbs)
+    {
+        return new CompoundDenseCellName(bbs);
+    }
+
+    public static String getColumnsString(CellNameType type, Iterable<Column> columns)
+    {
+        StringBuilder builder = new StringBuilder();
+        for (Column column : columns)
+            builder.append(column.getString(type)).append(",");
+        return builder.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/Composite.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/Composite.java b/src/java/org/apache/cassandra/db/composites/Composite.java
new file mode 100644
index 0000000..7001384
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/Composite.java
@@ -0,0 +1,76 @@
+/*
+ * 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.composites;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cache.IMeasurableMemory;
+import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.utils.Allocator;
+
+/**
+ * A composite value.
+ *
+ * This can be though as a list of ByteBuffer, except that this also include an
+ * 'end-of-component' flag, that allow precise selection of composite ranges.
+ *
+ * We also make a difference between "true" composites and the "simple" ones. The
+ * non-truly composite will have a size() == 1 but differs from true composites with
+ * size() == 1 in the way they are stored. Most code shouldn't have to care about the
+ * difference.
+ */
+public interface Composite extends IMeasurableMemory
+{
+    public enum EOC
+    {
+        START, NONE, END;
+
+        public static EOC from(int eoc)
+        {
+            return eoc == 0 ? NONE : (eoc < 0 ? START : END);
+        }
+
+        public byte toByte()
+        {
+            switch (this)
+            {
+                case START: return (byte)-1;
+                case NONE:  return (byte) 0;
+                case END:   return (byte) 1;
+                default: throw new AssertionError();
+            }
+        }
+    }
+
+    public int size();
+    public boolean isEmpty();
+    public ByteBuffer get(int i);
+
+    public EOC eoc();
+    public Composite withEOC(EOC eoc);
+    public Composite start();
+    public Composite end();
+    public ColumnSlice slice();
+
+    public boolean isPrefixOf(Composite other);
+
+    public ByteBuffer toByteBuffer();
+
+    public int dataSize();
+    public Composite copy(Allocator allocator);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/Composites.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/Composites.java b/src/java/org/apache/cassandra/db/composites/Composites.java
new file mode 100644
index 0000000..e02725c
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/Composites.java
@@ -0,0 +1,110 @@
+/*
+ * 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.composites;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public abstract class Composites
+{
+    private Composites() {}
+
+    public static final Composite EMPTY = new EmptyComposite();
+
+    static final CBuilder EMPTY_BUILDER = new CBuilder()
+    {
+        public int remainingCount() { return 0; }
+
+        public CBuilder add(ByteBuffer value) { throw new IllegalStateException(); }
+        public CBuilder add(Object value) { throw new IllegalStateException(); }
+
+        public Composite build() { return EMPTY; }
+        public Composite buildWith(ByteBuffer value) { throw new IllegalStateException(); }
+    };
+
+    private static class EmptyComposite implements Composite
+    {
+        public boolean isEmpty()
+        {
+            return true;
+        }
+
+        public int size()
+        {
+            return 0;
+        }
+
+        public ByteBuffer get(int i)
+        {
+            throw new IndexOutOfBoundsException();
+        }
+
+        public EOC eoc()
+        {
+            return EOC.NONE;
+        }
+
+        public Composite start()
+        {
+            return this;
+        }
+
+        public Composite end()
+        {
+            return this;
+        }
+
+        public Composite withEOC(EOC newEoc)
+        {
+            return this;
+        }
+
+        public ColumnSlice slice()
+        {
+            return ColumnSlice.ALL_COLUMNS;
+        }
+
+        public ByteBuffer toByteBuffer()
+        {
+            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+        }
+
+        public int dataSize()
+        {
+            return 0;
+        }
+
+        public long memorySize()
+        {
+            return 0;
+        }
+
+        public boolean isPrefixOf(Composite c)
+        {
+            return true;
+        }
+
+        public Composite copy(Allocator allocator)
+        {
+            return this;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/CompoundCType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CompoundCType.java b/src/java/org/apache/cassandra/db/composites/CompoundCType.java
new file mode 100644
index 0000000..bbad550
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/CompoundCType.java
@@ -0,0 +1,158 @@
+/*
+ * 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.composites;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+
+/**
+ * A truly-composite CType.
+ */
+public class CompoundCType extends AbstractCType
+{
+    final List<AbstractType<?>> types;
+
+    // It's up to the caller to pass a list that is effectively immutable
+    public CompoundCType(List<AbstractType<?>> types)
+    {
+        this.types = types;
+    }
+
+    public boolean isCompound()
+    {
+        return true;
+    }
+
+    public int size()
+    {
+        return types.size();
+    }
+
+    public AbstractType<?> subtype(int i)
+    {
+        return types.get(i);
+    }
+
+    public Composite fromByteBuffer(ByteBuffer bytes)
+    {
+        if (!bytes.hasRemaining())
+            return Composites.EMPTY;
+
+        ByteBuffer[] elements = new ByteBuffer[size()];
+        int idx = bytes.position(), i = 0;
+        byte eoc = 0;
+        while (idx < bytes.limit())
+        {
+            checkRemaining(bytes, idx, 2);
+            int length = bytes.getShort(idx) & 0xFFFF;
+            idx += 2;
+
+            checkRemaining(bytes, idx, length + 1);
+            elements[i++] = sliceBytes(bytes, idx, length);
+            idx += length;
+            eoc = bytes.get(idx++);
+        }
+        return new CompoundComposite(elements, i).withEOC(Composite.EOC.from(eoc));
+    }
+
+    public CBuilder builder()
+    {
+        return new CompoundCBuilder(this);
+    }
+
+    public CompoundCType setSubtype(int position, AbstractType<?> newType)
+    {
+        List<AbstractType<?>> newTypes = new ArrayList<AbstractType<?>>(types);
+        newTypes.set(position, newType);
+        return new CompoundCType(newTypes);
+    }
+
+    public AbstractType<?> asAbstractType()
+    {
+        return CompositeType.getInstance(types);
+    }
+
+    public static class CompoundCBuilder implements CBuilder
+    {
+        private final CType type;
+        private final ByteBuffer[] values;
+        private int size;
+        private boolean built;
+
+        public CompoundCBuilder(CType type)
+        {
+            this.type = type;
+            this.values = new ByteBuffer[type.size()];
+        }
+
+        public int remainingCount()
+        {
+            return values.length - size;
+        }
+
+        public CBuilder add(ByteBuffer value)
+        {
+            if (isDone())
+                throw new IllegalStateException();
+            values[size++] = value;
+            return this;
+        }
+
+        public CBuilder add(Object value)
+        {
+            return add(((AbstractType)type.subtype(size)).decompose(value));
+        }
+
+        private boolean isDone()
+        {
+            return remainingCount() == 0 || built;
+        }
+
+        public Composite build()
+        {
+            if (size == 0)
+                return Composites.EMPTY;
+
+            // We don't allow to add more element to a builder that has been built so
+            // that we don't have to copy values.
+            built = true;
+
+            // If the builder is full and we're building a dense cell name, then we can
+            // directly allocate the CellName object as it's complete.
+            if (size == values.length && type instanceof CellNameType && ((CellNameType)type).isDense())
+                return new CompoundDenseCellName(values);
+            return new CompoundComposite(values, size);
+        }
+
+        public Composite buildWith(ByteBuffer value)
+        {
+            ByteBuffer[] newValues = Arrays.copyOf(values, values.length);
+            newValues[size] = value;
+            // Same as above
+            if (size+1 == newValues.length && type instanceof CellNameType && ((CellNameType)type).isDense())
+                return new CompoundDenseCellName(newValues);
+
+            return new CompoundComposite(newValues, size+1);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CompoundComposite.java b/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
new file mode 100644
index 0000000..0a258b3
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
@@ -0,0 +1,70 @@
+/*
+ * 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.composites;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.ObjectSizes;
+
+/**
+ * A "truly-composite" Composite.
+ */
+public class CompoundComposite extends AbstractComposite
+{
+    // We could use a List, but we'll create such object *a lot* and using a array+size is not
+    // all that harder, so we save the List object allocation.
+    final ByteBuffer[] elements;
+    final int size;
+
+    CompoundComposite(ByteBuffer[] elements, int size)
+    {
+        this.elements = elements;
+        this.size = size;
+    }
+
+    public int size()
+    {
+        return size;
+    }
+
+    public ByteBuffer get(int i)
+    {
+        return elements[i];
+    }
+
+    protected ByteBuffer[] elementsCopy(Allocator allocator)
+    {
+        ByteBuffer[] elementsCopy = new ByteBuffer[size];
+        for (int i = 0; i < size; i++)
+            elementsCopy[i] = allocator.clone(elements[i]);
+        return elementsCopy;
+    }
+
+    public long memorySize()
+    {
+        return ObjectSizes.getFieldSize(TypeSizes.NATIVE.sizeof(size))
+             + ObjectSizes.getArraySize(elements);
+    }
+
+    public Composite copy(Allocator allocator)
+    {
+        return new CompoundComposite(elementsCopy(allocator), size);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java b/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
new file mode 100644
index 0000000..10f1a11
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
@@ -0,0 +1,70 @@
+/*
+ * 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.composites;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.ObjectSizes;
+
+public class CompoundDenseCellName extends CompoundComposite implements CellName
+{
+    // Not meant to be used directly, you should use the CellNameType method instead
+    CompoundDenseCellName(ByteBuffer[] elements)
+    {
+        super(elements, elements.length);
+    }
+
+    public int clusteringSize()
+    {
+        return size;
+    }
+
+    public ColumnIdentifier cql3ColumnName()
+    {
+        return null;
+    }
+
+    public ByteBuffer collectionElement()
+    {
+        return null;
+    }
+
+    public boolean isCollectionCell()
+    {
+        return false;
+    }
+
+    public boolean isSameCQL3RowAs(CellName other)
+    {
+        // Dense cell imply one cell by CQL row so no other cell will be the same row.
+        return equals(other);
+    }
+
+    @Override
+    public long memorySize()
+    {
+        return ObjectSizes.getSuperClassFieldSize(super.memorySize());
+    }
+
+    public CellName copy(Allocator allocator)
+    {
+        return new CompoundDenseCellName(elementsCopy(allocator));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java b/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
new file mode 100644
index 0000000..bb7a06e
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
@@ -0,0 +1,86 @@
+/*
+ * 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.composites;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.cassandra.cql3.CQL3Row;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+public class CompoundDenseCellNameType extends AbstractCompoundCellNameType
+{
+    public CompoundDenseCellNameType(List<AbstractType<?>> types)
+    {
+        this(new CompoundCType(types));
+    }
+
+    private CompoundDenseCellNameType(CompoundCType type)
+    {
+        super(type, type);
+    }
+
+    public CellNameType setSubtype(int position, AbstractType<?> newType)
+    {
+        if (position != 0)
+            throw new IllegalArgumentException();
+        return new SimpleDenseCellNameType(newType);
+    }
+
+    public boolean isDense()
+    {
+        return true;
+    }
+
+    public CellName create(Composite prefix, ColumnIdentifier columnName)
+    {
+        // We ignore the columnName because it's just the COMPACT_VALUE name which is not store in the cell name
+        assert prefix.size() == fullSize;
+        if (prefix instanceof CellName)
+            return (CellName)prefix;
+
+        assert prefix instanceof CompoundComposite;
+        CompoundComposite lc = (CompoundComposite)prefix;
+        assert lc.elements.length == lc.size;
+        return new CompoundDenseCellName(lc.elements);
+    }
+
+    protected Composite makeWith(ByteBuffer[] components, int size, Composite.EOC eoc)
+    {
+        if (size < fullSize || eoc != Composite.EOC.NONE)
+            return new CompoundComposite(components, size).withEOC(eoc);
+
+        assert components.length == size;
+        return new CompoundDenseCellName(components);
+    }
+
+    protected Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc)
+    {
+        return makeWith(Arrays.copyOfRange(components, 0, size), size, eoc);
+    }
+
+    public void addCQL3Column(ColumnIdentifier id) {}
+    public void removeCQL3Column(ColumnIdentifier id) {}
+
+    public CQL3Row.Builder CQL3RowBuilder(long now)
+    {
+        return makeDenseCQL3RowBuilder(now);
+    }
+}