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 2014/02/20 17:51:59 UTC

[4/7] Merge branch 'cassandra-2.0' into cassandra-2.1

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/AbstractCType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/AbstractCType.java
index 2b07a46,0000000..d3b8940
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/AbstractCType.java
+++ b/src/java/org/apache/cassandra/db/composites/AbstractCType.java
@@@ -1,336 -1,0 +1,339 @@@
 +/*
 + * 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;
++        if (c1 == null || c1.isEmpty())
++            return c2 == null || c2.isEmpty() ? 0 : -1;
++
++        if (c1.isStatic() != c2.isStatic())
++            return c1.isStatic() ? -1 : 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/63b1ef4e/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
index 96fc1eb,0000000..6d4ee12
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
@@@ -1,370 -1,0 +1,428 @@@
 +/*
 + * 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.config.ColumnDefinition;
 +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<Cell> columnComparator;
 +    private final Comparator<Cell> 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<Cell>()
 +        {
 +            public int compare(Cell c1, Cell c2)
 +            {
 +                return AbstractCellNameType.this.compare(c1.name(), c2.name());
 +            }
 +        };
 +        columnReverseComparator = new Comparator<Cell>()
 +        {
 +            public int compare(Cell c1, Cell 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<Cell> columnComparator()
 +    {
 +        return columnComparator;
 +    }
 +
 +    public Comparator<Cell> 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)
++    public CellName create(Composite prefix, ColumnDefinition column, ByteBuffer collectionElement)
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
 +    public CellName rowMarker(Composite prefix)
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
++    public Composite staticPrefix()
++    {
++        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];
 +            if (c instanceof ByteBuffer)
 +            {
 +                rawComponents[i] = (ByteBuffer)c;
 +            }
 +            else
 +            {
 +                AbstractType<?> type = subtype(i);
 +                // If it's a collection type, we need to find the right collection and use the key comparator (since we're building a cell name)
 +                if (type instanceof ColumnToCollectionType)
 +                {
 +                    assert i > 0;
 +                    type = ((ColumnToCollectionType)type).defined.get(rawComponents[i-1]).nameComparator();
 +                }
 +                rawComponents[i] = ((AbstractType)type).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<Cell> cells)
++            public CQL3Row.RowIterator group(Iterator<Cell> cells)
 +            {
-                 return new AbstractIterator<CQL3Row>()
++                return new DenseRowIterator(cells, now);
++            }
++        };
++    }
++
++    private static class DenseRowIterator extends AbstractIterator<CQL3Row> implements CQL3Row.RowIterator
++    {
++        private final Iterator<Cell> cells;
++        private final long now;
++
++        public DenseRowIterator(Iterator<Cell> cells, long now)
++        {
++            this.cells = cells;
++            this.now = now;
++        }
++
++        public CQL3Row getStaticRow()
++        {
++            // There can't be static columns in dense tables
++            return null;
++        }
++
++        protected CQL3Row computeNext()
++        {
++            while (cells.hasNext())
++            {
++                final Cell cell = cells.next();
++                if (cell.isMarkedForDelete(now))
++                    continue;
++
++                return new CQL3Row()
 +                {
-                     protected CQL3Row computeNext()
++                    public ByteBuffer getClusteringColumn(int i)
++                    {
++                        return cell.name().get(i);
++                    }
++
++                    public Cell getColumn(ColumnIdentifier name)
 +                    {
-                         while (cells.hasNext())
-                         {
-                             final Cell cell = cells.next();
-                             if (cell.isMarkedForDelete(now))
-                                 continue;
- 
-                             return new CQL3Row()
-                             {
-                                 public ByteBuffer getClusteringColumn(int i)
-                                 {
-                                     return cell.name().get(i);
-                                 }
- 
-                                 public Cell getColumn(ColumnIdentifier name)
-                                 {
-                                     return cell;
-                                 }
- 
-                                 public List<Cell> getCollection(ColumnIdentifier name)
-                                 {
-                                     return null;
-                                 }
-                             };
-                         }
-                         return endOfData();
++                        return cell;
++                    }
++
++                    public List<Cell> 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<Cell> cells)
++            public CQL3Row.RowIterator group(Iterator<Cell> cells)
 +            {
-                 return new AbstractIterator<CQL3Row>()
++                return new SparseRowIterator(cells, now);
++            }
++        };
++    }
++
++    private static class SparseRowIterator extends AbstractIterator<CQL3Row> implements CQL3Row.RowIterator
++    {
++        private final Iterator<Cell> cells;
++        private final long now;
++        private final CQL3Row staticRow;
++
++        private Cell nextCell;
++        private CellName previous;
++        private CQL3RowOfSparse currentRow;
++
++        public SparseRowIterator(Iterator<Cell> cells, long now)
++        {
++            this.cells = cells;
++            this.now = now;
++            this.staticRow = hasNextCell() && nextCell.name().isStatic()
++                           ? computeNext()
++                           : null;
++        }
++
++        public CQL3Row getStaticRow()
++        {
++            return staticRow;
++        }
++
++        private boolean hasNextCell()
++        {
++            if (nextCell != null)
++                return true;
++
++            while (cells.hasNext())
++            {
++                Cell cell = cells.next();
++                if (cell.isMarkedForDelete(now))
++                    continue;
++
++                nextCell = cell;
++                return true;
++            }
++            return false;
++        }
++
++        protected CQL3Row computeNext()
++        {
++            while (hasNextCell())
++            {
++                CQL3Row toReturn = null;
++                CellName current = nextCell.name();
++                if (currentRow == null || !current.isSameCQL3RowAs(previous))
 +                {
-                     private CellName previous;
-                     private CQL3RowOfSparse currentRow;
++                    toReturn = currentRow;
++                    currentRow = new CQL3RowOfSparse(current);
++                }
++                currentRow.add(nextCell);
++                nextCell = null;
++                previous = current;
 +
-                     protected CQL3Row computeNext()
-                     {
-                         while (cells.hasNext())
-                         {
-                             final Cell 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();
-                     }
-                 };
++                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, Cell> columns;
 +        private Map<ColumnIdentifier, List<Cell>> collections;
 +
 +        CQL3RowOfSparse(CellName cell)
 +        {
 +            this.cell = cell;
 +        }
 +
 +        public ByteBuffer getClusteringColumn(int i)
 +        {
 +            return cell.get(i);
 +        }
 +
 +        void add(Cell cell)
 +        {
 +            CellName cellName = cell.name();
 +            ColumnIdentifier columnName =  cellName.cql3ColumnName();
 +            if (cellName.isCollectionCell())
 +            {
 +                if (collections == null)
 +                    collections = new HashMap<>();
 +
 +                List<Cell> values = collections.get(columnName);
 +                if (values == null)
 +                {
 +                    values = new ArrayList<Cell>();
 +                    collections.put(columnName, values);
 +                }
 +                values.add(cell);
 +            }
 +            else
 +            {
 +                if (columns == null)
 +                    columns = new HashMap<>();
 +                columns.put(columnName, cell);
 +            }
 +        }
 +
 +        public Cell getColumn(ColumnIdentifier name)
 +        {
 +            return columns == null ? null : columns.get(name);
 +        }
 +
 +        public List<Cell> getCollection(ColumnIdentifier name)
 +        {
 +            return collections == null ? null : collections.get(name);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/AbstractComposite.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/AbstractComposite.java
index e23c560,0000000..fbff930
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/AbstractComposite.java
+++ b/src/java/org/apache/cassandra/db/composites/AbstractComposite.java
@@@ -1,132 -1,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.nio.ByteBuffer;
 +
 +import org.apache.cassandra.db.filter.ColumnSlice;
 +import org.apache.cassandra.db.marshal.AbstractCompositeType;
++import org.apache.cassandra.db.marshal.CompositeType;
 +
 +public abstract class AbstractComposite implements Composite
 +{
 +    public boolean isEmpty()
 +    {
 +        return size() == 0;
 +    }
 +
++    public boolean isStatic()
++    {
++        return false;
++    }
++
 +    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());
++        ByteBuffer result = ByteBuffer.allocate(dataSize() + 3 * size() + (isStatic() ? 2 : 0));
++        if (isStatic())
++            AbstractCompositeType.putShortLength(result, CompositeType.STATIC_MARKER);
++
 +        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())
++        if (size() > c.size() || isStatic() != c.isStatic())
 +            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())
++        if (size() != c.size() || isStatic() != c.isStatic())
 +            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();
++        return h + eoc().hashCode() + (isStatic() ? 1 : 0);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
index 0602f50,0000000..1cb605e
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
@@@ -1,264 -1,0 +1,289 @@@
 +/*
 + * 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;
++
++        boolean isStatic = false;
++        if (CompositeType.isStaticName(bytes))
++        {
++            isStatic = true;
++            idx += 2;
++        }
++
 +        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));
++        return makeWith(elements, i, Composite.EOC.from(eoc), isStatic);
 +    }
 +
 +    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);
++        return (CellName)makeWith(components, components.length, Composite.EOC.NONE, false);
 +    }
 +
-     protected abstract Composite makeWith(ByteBuffer[] components, int size, Composite.EOC eoc);
-     protected abstract Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc);
++    protected abstract Composite makeWith(ByteBuffer[] components, int size, Composite.EOC eoc, boolean isStatic);
++    protected abstract Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc, boolean isStatic);
 +
 +    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;
++        private boolean nextIsStatic;
 +
 +        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;
 +
++            if (nextIsStatic != composite.isStatic())
++                return nextIsStatic ? -1 : 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 int readShort()
++        {
++            return ((nextFull[nextIdx++] & 0xFF) << 8) | (nextFull[nextIdx++] & 0xFF);
++        }
++
 +        private boolean deserializeOne()
 +        {
 +            if (allComponentsDeserialized())
 +                return false;
 +
-             int length = ((nextFull[nextIdx++] & 0xFF) << 8) | (nextFull[nextIdx++] & 0xFF);
++            nextIsStatic = false;
++
++            int length = readShort();
++            if (length == CompositeType.STATIC_MARKER)
++            {
++                nextIsStatic = true;
++                length = readShort();
++            }
++
 +            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);
++            Composite c = type.copyAndMakeWith(nextComponents, nextSize, nextEOC, nextIsStatic);
 +            nextFull = null;
 +            return c;
 +        }
 +
 +        public void skipNext() throws IOException
 +        {
 +            maybeReadNext();
 +            nextFull = null;
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/BoundedComposite.java
index 86cdc4e,0000000..d988add
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
+++ b/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
@@@ -1,102 -1,0 +1,107 @@@
 +/*
 + * 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.utils.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.ObjectSizes;
 +import org.apache.cassandra.utils.memory.PoolAllocator;
 +
 +public class BoundedComposite extends AbstractComposite
 +{
 +    private static final long EMPTY_SIZE = ObjectSizes.measure(new BoundedComposite(null, false));
 +
 +    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 boolean isStatic()
++    {
++        return wrapped.isStatic();
++    }
++
 +    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 unsharedHeapSize()
 +    {
 +        return EMPTY_SIZE + wrapped.unsharedHeapSize();
 +    }
 +
 +    public Composite copy(AbstractAllocator allocator)
 +    {
 +        return new BoundedComposite(wrapped.copy(allocator), isStart);
 +    }
 +
 +    @Override
 +    public void free(PoolAllocator<?> allocator)
 +    {
 +        wrapped.free(allocator);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/CellNameType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CellNameType.java
index c4ddb60,0000000..7128c91
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/CellNameType.java
@@@ -1,202 -1,0 +1,213 @@@
 +/*
 + * 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.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.CQL3Row;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.db.Cell;
 +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();
 +
 +    /**
++     * The prefix to use for static columns.
++     *
++     * Note that create() methods below for creating CellName automatically handle static columns already
++     * for convenience, and so there is not need to pass this prefix for them. There is few other cases
++     * where we need the prefix directly however.
++     */
++    public Composite staticPrefix();
++
++    /**
 +     * 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.
++     * Creates a new CellName given a clustering prefix and a CQL3 column.
 +     *
-      * Note that for dense types, the columnName can be null.
++     * Note that for dense types, the column can be null as a shortcut for designing the only
++     * COMPACT_VALUE column of the table.
 +     */
-     public CellName create(Composite prefix, ColumnIdentifier columnName);
++    public CellName create(Composite prefix, ColumnDefinition column);
 +
 +    /**
-      * Creates a new collection CellName given a clustering prefix, a CQL3 columnName and the collection element.
++     * Creates a new collection CellName given a clustering prefix, a CQL3 column and the collection element.
 +     */
-     public CellName create(Composite prefix, ColumnIdentifier columnName, ByteBuffer collectionElement);
++    public CellName create(Composite prefix, ColumnDefinition column, 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<Cell> columnComparator();
 +    public Comparator<Cell> 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/63b1ef4e/src/java/org/apache/cassandra/db/composites/Composite.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/Composite.java
index c82d170,0000000..99a0d46
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/Composite.java
+++ b/src/java/org/apache/cassandra/db/composites/Composite.java
@@@ -1,78 -1,0 +1,80 @@@
 +/*
 + * 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.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.memory.PoolAllocator;
 +
 +/**
 + * 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 isStatic();
++
 +    public boolean isPrefixOf(Composite other);
 +
 +    public ByteBuffer toByteBuffer();
 +
 +    public int dataSize();
 +    public Composite copy(AbstractAllocator allocator);
 +    public void free(PoolAllocator<?> allocator);
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/Composites.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/Composites.java
index f1b7eab,0000000..902c82a
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/Composites.java
+++ b/src/java/org/apache/cassandra/db/composites/Composites.java
@@@ -1,117 -1,0 +1,122 @@@
 +/*
 + * 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.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.memory.PoolAllocator;
 +
 +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 boolean isStatic()
++        {
++            return false;
++        }
++
 +        public int dataSize()
 +        {
 +            return 0;
 +        }
 +
 +        public long unsharedHeapSize()
 +        {
 +            return 0;
 +        }
 +
 +        public boolean isPrefixOf(Composite c)
 +        {
 +            return true;
 +        }
 +
 +        public Composite copy(AbstractAllocator allocator)
 +        {
 +            return this;
 +        }
 +
 +        @Override
 +        public void free(PoolAllocator<?> allocator)
 +        {
 +        }
 +
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/CompoundCType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CompoundCType.java
index bbad550,0000000..d638f16
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundCType.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundCType.java
@@@ -1,158 -1,0 +1,166 @@@
 +/*
 + * 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;
++
++        boolean isStatic = false;
++        if (CompositeType.isStaticName(bytes))
++        {
++            isStatic = true;
++            idx += 2;
++        }
++
 +        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));
++        return new CompoundComposite(elements, i, isStatic).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);
++            return new CompoundComposite(values, size, false);
 +        }
 +
 +        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);
++            return new CompoundComposite(newValues, size+1, false);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CompoundComposite.java
index 54ec759,0000000..9475cc0
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
@@@ -1,84 -1,0 +1,92 @@@
 +/*
 + * 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.utils.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.ObjectSizes;
++import org.apache.cassandra.utils.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.memory.PoolAllocator;
 +
 +/**
 + * A "truly-composite" Composite.
 + */
 +public class CompoundComposite extends AbstractComposite
 +{
-     private static final long EMPTY_SIZE = ObjectSizes.measure(new CompoundComposite(null, 0));
++    private static final long EMPTY_SIZE = ObjectSizes.measure(new CompoundComposite(null, 0, false));
 +
 +    // 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;
++    final boolean isStatic;
 +
-     CompoundComposite(ByteBuffer[] elements, int size)
++    CompoundComposite(ByteBuffer[] elements, int size, boolean isStatic)
 +    {
 +        this.elements = elements;
 +        this.size = size;
++        this.isStatic = isStatic;
 +    }
 +
 +    public int size()
 +    {
 +        return size;
 +    }
 +
 +    public ByteBuffer get(int i)
 +    {
 +        return elements[i];
 +    }
 +
++    @Override
++    public boolean isStatic()
++    {
++        return isStatic;
++    }
++
 +    protected ByteBuffer[] elementsCopy(AbstractAllocator allocator)
 +    {
 +        ByteBuffer[] elementsCopy = new ByteBuffer[size];
 +        for (int i = 0; i < size; i++)
 +            elementsCopy[i] = allocator.clone(elements[i]);
 +        return elementsCopy;
 +    }
 +
 +    public long unsharedHeapSize()
 +    {
 +        return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(elements);
 +    }
 +
 +    public long excessHeapSizeExcludingData()
 +    {
 +        return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(elements);
 +    }
 +
 +    public Composite copy(AbstractAllocator allocator)
 +    {
-         return new CompoundComposite(elementsCopy(allocator), size);
++        return new CompoundComposite(elementsCopy(allocator), size, isStatic);
 +    }
 +
 +    @Override
 +    public void free(PoolAllocator<?> allocator)
 +    {
 +        for (ByteBuffer element : elements)
 +            allocator.free(element);
 +    }
 +
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
index 26df34b,0000000..e16e390
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
@@@ -1,80 -1,0 +1,80 @@@
 +/*
 + * 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.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.ObjectSizes;
 +
 +public class CompoundDenseCellName extends CompoundComposite implements CellName
 +{
 +
 +    private static final long HEAP_SIZE = ObjectSizes.measure(new CompoundDenseCellName(new ByteBuffer[0]));
 +
 +    // Not meant to be used directly, you should use the CellNameType method instead
 +    CompoundDenseCellName(ByteBuffer[] elements)
 +    {
-         super(elements, elements.length);
++        super(elements, elements.length, false);
 +    }
 +
 +    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 unsharedHeapSize()
 +    {
 +        return HEAP_SIZE + ObjectSizes.sizeOnHeapOf(elements);
 +    }
 +
 +    @Override
 +    public long excessHeapSizeExcludingData()
 +    {
 +        return HEAP_SIZE + ObjectSizes.sizeOnHeapExcludingData(elements);
 +    }
 +
 +    public CellName copy(AbstractAllocator allocator)
 +    {
 +        return new CompoundDenseCellName(elementsCopy(allocator));
 +    }
 +
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
index bb7a06e,0000000..06b25b0
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
@@@ -1,86 -1,0 +1,88 @@@
 +/*
 + * 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.config.ColumnDefinition;
 +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)
++    public CellName create(Composite prefix, ColumnDefinition column)
 +    {
-         // We ignore the columnName because it's just the COMPACT_VALUE name which is not store in the cell name
++        // We ignore the column because it's just the COMPACT_VALUE name which is not store in the cell name (and it can be null anyway)
 +        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)
++    protected Composite makeWith(ByteBuffer[] components, int size, Composite.EOC eoc, boolean isStatic)
 +    {
++        assert !isStatic;
 +        if (size < fullSize || eoc != Composite.EOC.NONE)
-             return new CompoundComposite(components, size).withEOC(eoc);
++            return new CompoundComposite(components, size, false).withEOC(eoc);
 +
 +        assert components.length == size;
 +        return new CompoundDenseCellName(components);
 +    }
 +
-     protected Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc)
++    protected Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc, boolean isStatic)
 +    {
-         return makeWith(Arrays.copyOfRange(components, 0, size), size, eoc);
++        return makeWith(Arrays.copyOfRange(components, 0, size), size, eoc, isStatic);
 +    }
 +
 +    public void addCQL3Column(ColumnIdentifier id) {}
 +    public void removeCQL3Column(ColumnIdentifier id) {}
 +
 +    public CQL3Row.Builder CQL3RowBuilder(long now)
 +    {
 +        return makeDenseCQL3RowBuilder(now);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/63b1ef4e/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
index dc525d6,0000000..0106309
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
@@@ -1,175 -1,0 +1,175 @@@
 +/*
 + * 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.ByteBufferUtil;
 +import org.apache.cassandra.utils.memory.AbstractAllocator;
 +import org.apache.cassandra.utils.ObjectSizes;
 +import org.apache.cassandra.utils.memory.PoolAllocator;
 +
 +public class CompoundSparseCellName extends CompoundComposite implements CellName
 +{
 +    private static final ByteBuffer[] EMPTY_PREFIX = new ByteBuffer[0];
 +
-     private static final long HEAP_SIZE = ObjectSizes.measure(new CompoundSparseCellName(null));
++    private static final long HEAP_SIZE = ObjectSizes.measure(new CompoundSparseCellName(null, false));
 +
 +    protected final ColumnIdentifier columnName;
 +
 +    // Not meant to be used directly, you should use the CellNameType method instead
-     CompoundSparseCellName(ColumnIdentifier columnName)
++    CompoundSparseCellName(ColumnIdentifier columnName, boolean isStatic)
 +    {
-         this(EMPTY_PREFIX, columnName);
++        this(EMPTY_PREFIX, columnName, isStatic);
 +    }
 +
-     CompoundSparseCellName(ByteBuffer[] elements, ColumnIdentifier columnName)
++    CompoundSparseCellName(ByteBuffer[] elements, ColumnIdentifier columnName, boolean isStatic)
 +    {
-         this(elements, elements.length, columnName);
++        this(elements, elements.length, columnName, isStatic);
 +    }
 +
-     CompoundSparseCellName(ByteBuffer[] elements, int size, ColumnIdentifier columnName)
++    CompoundSparseCellName(ByteBuffer[] elements, int size, ColumnIdentifier columnName, boolean isStatic)
 +    {
-         super(elements, size);
++        super(elements, size, isStatic);
 +        this.columnName = columnName;
 +    }
 +
 +    public int size()
 +    {
 +        return size + 1;
 +    }
 +
 +    public ByteBuffer get(int i)
 +    {
 +        return i == size ? columnName.bytes : elements[i];
 +    }
 +
 +    public int clusteringSize()
 +    {
 +        return size;
 +    }
 +
 +    public ColumnIdentifier cql3ColumnName()
 +    {
 +        return columnName;
 +    }
 +
 +    public ByteBuffer collectionElement()
 +    {
 +        return null;
 +    }
 +
 +    public boolean isCollectionCell()
 +    {
 +        return false;
 +    }
 +
 +    public boolean isSameCQL3RowAs(CellName other)
 +    {
-         if (clusteringSize() != other.clusteringSize())
++        if (clusteringSize() != other.clusteringSize() || other.isStatic() != isStatic())
 +            return false;
 +
 +        for (int i = 0; i < clusteringSize(); i++)
 +        {
 +            if (!elements[i].equals(other.get(i)))
 +                return false;
 +        }
 +        return true;
 +    }
 +
 +    public CellName copy(AbstractAllocator allocator)
 +    {
 +        if (elements.length == 0)
 +            return this;
 +
 +        // We don't copy columnName because it's interned in SparseCellNameType
-         return new CompoundSparseCellName(elementsCopy(allocator), columnName);
++        return new CompoundSparseCellName(elementsCopy(allocator), columnName, isStatic());
 +    }
 +
 +    public static class WithCollection extends CompoundSparseCellName
 +    {
-         private static final long HEAP_SIZE = ObjectSizes.measure(new WithCollection(null, ByteBufferUtil.EMPTY_BYTE_BUFFER));
++        private static final long HEAP_SIZE = ObjectSizes.measure(new WithCollection(null, ByteBufferUtil.EMPTY_BYTE_BUFFER, false));
 +
 +        private final ByteBuffer collectionElement;
 +
-         WithCollection(ColumnIdentifier columnName, ByteBuffer collectionElement)
++        WithCollection(ColumnIdentifier columnName, ByteBuffer collectionElement, boolean isStatic)
 +        {
-             this(EMPTY_PREFIX, columnName, collectionElement);
++            this(EMPTY_PREFIX, columnName, collectionElement, isStatic);
 +        }
 +
-         WithCollection(ByteBuffer[] elements, ColumnIdentifier columnName, ByteBuffer collectionElement)
++        WithCollection(ByteBuffer[] elements, ColumnIdentifier columnName, ByteBuffer collectionElement, boolean isStatic)
 +        {
-             this(elements, elements.length, columnName, collectionElement);
++            this(elements, elements.length, columnName, collectionElement, isStatic);
 +        }
 +
-         WithCollection(ByteBuffer[] elements, int size, ColumnIdentifier columnName, ByteBuffer collectionElement)
++        WithCollection(ByteBuffer[] elements, int size, ColumnIdentifier columnName, ByteBuffer collectionElement, boolean isStatic)
 +        {
-             super(elements, size, columnName);
++            super(elements, size, columnName, isStatic);
 +            this.collectionElement = collectionElement;
 +        }
 +
 +        public int size()
 +        {
 +            return size + 2;
 +        }
 +
 +        public ByteBuffer get(int i)
 +        {
 +            return i == size + 1 ? collectionElement : super.get(i);
 +        }
 +
 +        @Override
 +        public ByteBuffer collectionElement()
 +        {
 +            return collectionElement;
 +        }
 +
 +        @Override
 +        public boolean isCollectionCell()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public CellName copy(AbstractAllocator allocator)
 +        {
 +            // We don't copy columnName because it's interned in SparseCellNameType
-             return new CompoundSparseCellName.WithCollection(elements.length == 0 ? elements : elementsCopy(allocator), size, columnName, allocator.clone(collectionElement));
++            return new CompoundSparseCellName.WithCollection(elements.length == 0 ? elements : elementsCopy(allocator), size, columnName, allocator.clone(collectionElement), isStatic());
 +        }
 +
 +        @Override
 +        public long unsharedHeapSize()
 +        {
 +            return super.unsharedHeapSize() + ObjectSizes.sizeOnHeapOf(collectionElement);
 +        }
 +
 +        @Override
 +        public long excessHeapSizeExcludingData()
 +        {
 +            return super.excessHeapSizeExcludingData() + ObjectSizes.sizeOnHeapExcludingData(collectionElement);
 +        }
 +
 +        @Override
 +        public void free(PoolAllocator<?> allocator)
 +        {
 +            super.free(allocator);
 +            allocator.free(collectionElement);
 +        }
 +    }
 +}