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

[30/51] [partial] cassandra git commit: Storage engine refactor, a.k.a CASSANDRA-8099

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
deleted file mode 100644
index 14fa16c..0000000
--- a/src/java/org/apache/cassandra/db/composites/AbstractComposite.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.composites;
-
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.db.marshal.CompositeType;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-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() + (isStatic() ? 2 : 0));
-        if (isStatic())
-            ByteBufferUtil.writeShortLength(result, CompositeType.STATIC_MARKER);
-
-        for (int i = 0; i < size(); i++)
-        {
-            ByteBuffer bb = get(i);
-            ByteBufferUtil.writeShortLength(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(CType type, Composite c)
-    {
-        if (size() > c.size() || isStatic() != c.isStatic())
-            return false;
-
-        for (int i = 0; i < size(); i++)
-        {
-            if (type.subtype(i).compare(get(i), c.get(i)) != 0)
-                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() || 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() + (isStatic() ? 1 : 0);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
deleted file mode 100644
index bf303a7..0000000
--- a/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
+++ /dev/null
@@ -1,295 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.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)
-    {
-        super(isByteOrderComparable(fullType.types));
-        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), 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, false);
-    }
-
-    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 int peekShort()
-        {
-            return ((nextFull[nextIdx] & 0xFF) << 8) | (nextFull[nextIdx+1] & 0xFF);
-        }
-
-        private boolean deserializeOne()
-        {
-            if (allComponentsDeserialized())
-                return false;
-
-            int 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);
-
-            // Is is a static?
-            nextIsStatic = false;
-            if (peekShort() == CompositeType.STATIC_MARKER)
-            {
-                nextIsStatic = true;
-                readShort(); // Skip the static marker
-            }
-        }
-
-        public Composite readNext() throws IOException
-        {
-            maybeReadNext();
-            if (nextFull == EMPTY)
-                return Composites.EMPTY;
-
-            deserializeAll();
-            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/a991b648/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
deleted file mode 100644
index b3f4778..0000000
--- a/src/java/org/apache/cassandra/db/composites/AbstractSimpleCellNameType.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.composites;
-
-import java.io.DataInput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Comparator;
-
-import net.nicoulaj.compilecommand.annotations.Inline;
-import org.apache.cassandra.db.Cell;
-import org.apache.cassandra.db.NativeCell;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public abstract class AbstractSimpleCellNameType extends AbstractCellNameType
-{
-    protected final AbstractType<?> type;
-
-    static final Comparator<Cell> rightNativeCell = new Comparator<Cell>()
-    {
-        public int compare(Cell o1, Cell o2)
-        {
-            return -((NativeCell) o2).compareToSimple(o1.name());
-        }
-    };
-
-    static final Comparator<Cell> neitherNativeCell = new Comparator<Cell>()
-    {
-        public int compare(Cell o1, Cell o2)
-        {
-            return compareUnsigned(o1.name(), o2.name());
-        }
-    };
-
-    // only one or the other of these will ever be used
-    static final Comparator<Object> asymmetricRightNativeCell = new Comparator<Object>()
-    {
-        public int compare(Object o1, Object o2)
-        {
-            return -((NativeCell) o2).compareToSimple((Composite) o1);
-        }
-    };
-
-    static final Comparator<Object> asymmetricNeitherNativeCell = new Comparator<Object>()
-    {
-        public int compare(Object o1, Object o2)
-        {
-            return compareUnsigned((Composite) o1, ((Cell) o2).name());
-        }
-    };
-
-    protected AbstractSimpleCellNameType(AbstractType<?> type)
-    {
-        super(type.isByteOrderComparable());
-        this.type = type;
-    }
-
-    public boolean isCompound()
-    {
-        return false;
-    }
-
-    public int size()
-    {
-        return 1;
-    }
-
-    @Inline
-    static int compareUnsigned(Composite c1, Composite c2)
-    {
-        ByteBuffer b1 = c1.toByteBuffer();
-        ByteBuffer b2 = c2.toByteBuffer();
-        return ByteBufferUtil.compareUnsigned(b1, b2);
-    }
-
-    public int compare(Composite c1, Composite c2)
-    {
-        if (isByteOrderComparable)
-            return compareUnsigned(c1, c2);
-
-        assert !(c1.isEmpty() | c2.isEmpty());
-        return type.compare(c1.get(0), c2.get(0));
-    }
-
-    protected Comparator<Cell> getByteOrderColumnComparator(boolean isRightNative)
-    {
-        if (isRightNative)
-            return rightNativeCell;
-        return neitherNativeCell;
-    }
-
-    protected Comparator<Object> getByteOrderAsymmetricColumnComparator(boolean isRightNative)
-    {
-        if (isRightNative)
-            return asymmetricRightNativeCell;
-        return asymmetricNeitherNativeCell;
-    }
-
-    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/a991b648/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
deleted file mode 100644
index 7f596fe..0000000
--- a/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.composites;
-
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.utils.memory.AbstractAllocator;
-import org.apache.cassandra.utils.ObjectSizes;
-
-/**
- * Wraps another Composite and adds an EOC byte to track whether this is a slice start or end.
- */
-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(CFMetaData cfm, AbstractAllocator allocator)
-    {
-        return new BoundedComposite(wrapped.copy(cfm, allocator), isStart);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
deleted file mode 100644
index 39035cb..0000000
--- a/src/java/org/apache/cassandra/db/composites/CBuilder.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.composites;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-/**
- * 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);
-    public Composite buildWith(List<ByteBuffer> values);
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
deleted file mode 100644
index 7f70313..0000000
--- a/src/java/org/apache/cassandra/db/composites/CType.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.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.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();
-
-    int compare(Composite o1, Composite o2);
-
-    /**
-     * 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 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 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) 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 IVersionedSerializer<ColumnSlice> sliceSerializer();
-    public IVersionedSerializer<SliceQueryFilter> sliceQueryFilterSerializer();
-    public DeletionInfo.Serializer deletionInfoSerializer();
-    public RangeTombstone.Serializer rangeTombstoneSerializer();
-
-    public interface Serializer extends ISerializer<Composite>
-    {
-        public void skip(DataInput in) throws IOException;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
deleted file mode 100644
index 4d778d3..0000000
--- a/src/java/org/apache/cassandra/db/composites/CellName.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.composites;
-
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.utils.memory.AbstractAllocator;
-
-/**
- * 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.
-     * @param metadata
-     */
-    public ColumnIdentifier cql3ColumnName(CFMetaData metadata);
-
-    /**
-     * 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(CellNameType type, 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(CFMetaData cfm, AbstractAllocator allocator);
-
-    public long unsharedHeapSizeExcludingData();
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
deleted file mode 100644
index 6c89660..0000000
--- a/src/java/org/apache/cassandra/db/composites/CellNameType.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.composites;
-
-import java.io.DataInput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Comparator;
-
-import org.apache.cassandra.config.CFMetaData;
-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 does not have any non-frozen collections).
-     */
-    public ColumnToCollectionType collectionType();
-
-    /**
-     * Return the new type obtained by adding/updating to the new collection type for the provided column name
-     * to this type.
-     */
-    public CellNameType addOrUpdateCollection(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 column.
-     *
-     * 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, ColumnDefinition column);
-
-    /**
-     * Creates a new collection CellName given a clustering prefix, a CQL3 column and the collection element.
-     */
-    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 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(CFMetaData metadata, 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(boolean isRightNative);
-    public Comparator<Object> asymmetricColumnComparator(boolean isRightNative);
-    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;
-
-        /**
-         * Compare 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/a991b648/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
deleted file mode 100644
index f73f7a7..0000000
--- a/src/java/org/apache/cassandra/db/composites/CellNames.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.composites;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.db.Cell;
-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)
-                {
-                    // We don't allow collection for super columns, so the "name" type *must* be UTF8
-                    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
-                {
-                    AbstractType<?> nameType = types.get(types.size() - 1);
-                    return new CompoundSparseCellNameType(types.subList(0, types.size() - 1), nameType);
-                }
-            }
-            else
-            {
-                assert type != null;
-                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);
-    }
-
-    public static CellName simpleSparse(ColumnIdentifier identifier)
-    {
-        return new SimpleSparseCellName(identifier);
-    }
-
-    // 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 CellName compositeSparse(ByteBuffer[] bbs, ColumnIdentifier identifier, boolean isStatic)
-    {
-        return new CompoundSparseCellName(bbs, identifier, isStatic);
-    }
-
-    public static CellName compositeSparseWithCollection(ByteBuffer[] bbs, ByteBuffer collectionElement, ColumnIdentifier identifier, boolean isStatic)
-    {
-        return new CompoundSparseCellName.WithCollection(bbs, identifier, collectionElement, isStatic);
-    }
-
-    public static String getColumnsString(CellNameType type, Iterable<Cell> columns)
-    {
-        StringBuilder builder = new StringBuilder();
-        for (Cell cell : columns)
-            builder.append(cell.getString(type)).append(",");
-        return builder.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
deleted file mode 100644
index b15daef..0000000
--- a/src/java/org/apache/cassandra/db/composites/Composite.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.composites;
-
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.cache.IMeasurableMemory;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.utils.memory.AbstractAllocator;
-
-/**
- * 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(-1), NONE(-1), END(1);
-
-        // If composite p has this EOC and is a strict prefix of composite c, then this
-        // the result of the comparison of p and c. Basically, p sorts before c unless
-        // it's EOC is END.
-        public final int prefixComparisonResult;
-
-        private EOC(int prefixComparisonResult)
-        {
-            this.prefixComparisonResult = prefixComparisonResult;
-        }
-
-        public static EOC from(int eoc)
-        {
-            return eoc == 0 ? NONE : (eoc < 0 ? START : END);
-        }
-    }
-
-    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(CType type, Composite other);
-
-    public ByteBuffer toByteBuffer();
-
-    public int dataSize();
-    public Composite copy(CFMetaData cfm, AbstractAllocator allocator);
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
deleted file mode 100644
index fa0df48..0000000
--- a/src/java/org/apache/cassandra/db/composites/Composites.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.composites;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.memory.AbstractAllocator;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-
-public abstract class Composites
-{
-    private Composites() {}
-
-    public static final Composite EMPTY = new EmptyComposite();
-
-    /**
-     * Converts the specified <code>Composites</code> into <code>ByteBuffer</code>s.
-     *
-     * @param composites the composites to convert.
-     * @return the <code>ByteBuffer</code>s corresponding to the specified <code>Composites</code>.
-     */
-    public static List<ByteBuffer> toByteBuffers(List<Composite> composites)
-    {
-        return Lists.transform(composites, new Function<Composite, ByteBuffer>()
-        {
-            public ByteBuffer apply(Composite composite)
-            {
-                return composite.toByteBuffer();
-            }
-        });
-    }
-
-    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(); }
-        public Composite buildWith(List<ByteBuffer> values) { throw new IllegalStateException(); }
-    };
-
-    private static class EmptyComposite implements Composite
-    {
-        public boolean isEmpty()
-        {
-            return true;
-        }
-
-        public int size()
-        {
-            return 0;
-        }
-
-        public ByteBuffer get(int i)
-        {
-            if (i > 0)
-                throw new IndexOutOfBoundsException();
-
-            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
-        }
-
-        public EOC eoc()
-        {
-            return EOC.NONE;
-        }
-
-        public Composite start()
-        {
-            // Note that SimpleCType/AbstractSimpleCellNameType compare method
-            // indirectly rely on the fact that EMPTY == EMPTY.start() == EMPTY.end()
-            // (or more precisely on the fact that the EOC is NONE for all of those).
-            return this;
-        }
-
-        public Composite end()
-        {
-            // Note that SimpleCType/AbstractSimpleCellNameType compare method
-            // indirectly rely on the fact that EMPTY == EMPTY.start() == EMPTY.end()
-            // (or more precisely on the fact that the EOC is NONE for all of those).
-            return this;
-        }
-
-        public Composite withEOC(EOC newEoc)
-        {
-            // Note that SimpleCType/AbstractSimpleCellNameType compare method
-            // indirectly rely on the fact that EMPTY == EMPTY.start() == EMPTY.end()
-            // (or more precisely on the fact that the EOC is NONE for all of those).
-            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(CType type, Composite c)
-        {
-            return true;
-        }
-
-        public Composite copy(CFMetaData cfm, AbstractAllocator allocator)
-        {
-            return this;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java b/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
deleted file mode 100644
index 25a510f..0000000
--- a/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
+++ /dev/null
@@ -1,313 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.composites;
-
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import org.apache.cassandra.db.composites.Composite.EOC;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-import static java.util.Collections.singletonList;
-
-/**
- * Builder that allow to build multiple composites at the same time.
- */
-public final class CompositesBuilder
-{
-    /**
-     * The composite type.
-     */
-    private final CType ctype;
-
-    /**
-     * The elements of the composites
-     */
-    private final List<List<ByteBuffer>> elementsList = new ArrayList<>();
-
-    /**
-     * The number of elements that have been added.
-     */
-    private int size;
-
-    /**
-     * <code>true</code> if the composites have been build, <code>false</code> otherwise.
-     */
-    private boolean built;
-
-    /**
-     * <code>true</code> if the composites contains some <code>null</code> elements.
-     */
-    private boolean containsNull;
-
-    /**
-     * <code>true</code> if some empty collection have been added.
-     */
-    private boolean hasMissingElements;
-
-    /**
-     * <code>true</code> if the composites contains some <code>unset</code> elements.
-     */
-    private boolean containsUnset;
-
-    public CompositesBuilder(CType ctype)
-    {
-        this.ctype = ctype;
-    }
-
-    /**
-     * Adds the specified element to all the composites.
-     * <p>
-     * If this builder contains 2 composites: A-B and A-C a call to this method to add D will result in the composites:
-     * A-B-D and A-C-D.
-     * </p>
-     *
-     * @param value the value of the next element
-     * @return this <code>CompositeBuilder</code>
-     */
-    public CompositesBuilder addElementToAll(ByteBuffer value)
-    {
-        checkUpdateable();
-
-        if (isEmpty())
-            elementsList.add(new ArrayList<ByteBuffer>());
-
-        for (int i = 0, m = elementsList.size(); i < m; i++)
-        {
-            if (value == null)
-                containsNull = true;
-            if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
-                containsUnset = true;
-            elementsList.get(i).add(value);
-        }
-        size++;
-        return this;
-    }
-
-    /**
-     * Adds individually each of the specified elements to the end of all of the existing composites.
-     * <p>
-     * If this builder contains 2 composites: A-B and A-C a call to this method to add D and E will result in the 4
-     * composites: A-B-D, A-B-E, A-C-D and A-C-E.
-     * </p>
-     *
-     * @param values the elements to add
-     * @return this <code>CompositeBuilder</code>
-     */
-    public CompositesBuilder addEachElementToAll(List<ByteBuffer> values)
-    {
-        checkUpdateable();
-
-        if (isEmpty())
-            elementsList.add(new ArrayList<ByteBuffer>());
-
-        if (values.isEmpty())
-        {
-            hasMissingElements = true;
-        }
-        else
-        {
-            for (int i = 0, m = elementsList.size(); i < m; i++)
-            {
-                List<ByteBuffer> oldComposite = elementsList.remove(0);
-
-                for (int j = 0, n = values.size(); j < n; j++)
-                {
-                    List<ByteBuffer> newComposite = new ArrayList<>(oldComposite);
-                    elementsList.add(newComposite);
-
-                    ByteBuffer value = values.get(j);
-
-                    if (value == null)
-                        containsNull = true;
-                    if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
-                        containsUnset = true;
-
-                    newComposite.add(values.get(j));
-                }
-            }
-        }
-        size++;
-        return this;
-    }
-
-
-    /**
-     * Adds individually each of the specified list of elements to the end of all of the existing composites.
-     * <p>
-     * If this builder contains 2 composites: A-B and A-C a call to this method to add [[D, E], [F, G]] will result in the 4
-     * composites: A-B-D-E, A-B-F-G, A-C-D-E and A-C-F-G.
-     * </p>
-     *
-     * @param values the elements to add
-     * @return this <code>CompositeBuilder</code>
-     */
-    public CompositesBuilder addAllElementsToAll(List<List<ByteBuffer>> values)
-    {
-        checkUpdateable();
-
-        if (isEmpty())
-            elementsList.add(new ArrayList<ByteBuffer>());
-
-        if (values.isEmpty())
-        {
-            hasMissingElements = true;
-        }
-        else
-        {
-            for (int i = 0, m = elementsList.size(); i < m; i++)
-            {
-                List<ByteBuffer> oldComposite = elementsList.remove(0);
-
-                for (int j = 0, n = values.size(); j < n; j++)
-                {
-                    List<ByteBuffer> newComposite = new ArrayList<>(oldComposite);
-                    elementsList.add(newComposite);
-
-                    List<ByteBuffer> value = values.get(j);
-
-                    if (value.isEmpty())
-                        hasMissingElements = true;
-
-                    if (value.contains(null))
-                        containsNull = true;
-                    if (value.contains(ByteBufferUtil.UNSET_BYTE_BUFFER))
-                        containsUnset = true;
-
-                    newComposite.addAll(value);
-                }
-            }
-            size += values.get(0).size();
-        }
-        return this;
-    }
-
-    /**
-     * Returns the number of elements that can be added to the composites.
-     *
-     * @return the number of elements that can be added to the composites.
-     */
-    public int remainingCount()
-    {
-        return ctype.size() - size;
-    }
-
-    /**
-     * Checks if some elements can still be added to the composites.
-     *
-     * @return <code>true</code> if it is possible to add more elements to the composites, <code>false</code> otherwise.
-     */
-    public boolean hasRemaining()
-    {
-        return remainingCount() > 0;
-    }
-
-    /**
-     * Checks if this builder is empty.
-     *
-     * @return <code>true</code> if this builder is empty, <code>false</code> otherwise.
-     */
-    public boolean isEmpty()
-    {
-        return elementsList.isEmpty();
-    }
-
-    /**
-     * Checks if the composites contains null elements.
-     *
-     * @return <code>true</code> if the composites contains <code>null</code> elements, <code>false</code> otherwise.
-     */
-    public boolean containsNull()
-    {
-        return containsNull;
-    }
-
-    /**
-     * Checks if some empty list of values have been added
-     * @return <code>true</code> if the composites have some missing elements, <code>false</code> otherwise.
-     */
-    public boolean hasMissingElements()
-    {
-        return hasMissingElements;
-    }
-
-    /**
-     * Checks if the composites contains unset elements.
-     *
-     * @return <code>true</code> if the composites contains <code>unset</code> elements, <code>false</code> otherwise.
-     */
-    public boolean containsUnset()
-    {
-        return containsUnset;
-    }
-
-    /**
-     * Builds the <code>Composites</code>.
-     *
-     * @return the composites
-     */
-    public List<Composite> build()
-    {
-        return buildWithEOC(EOC.NONE);
-    }
-
-    /**
-     * Builds the <code>Composites</code> with the specified EOC.
-     *
-     * @return the composites
-     */
-    public List<Composite> buildWithEOC(EOC eoc)
-    {
-        built = true;
-
-        if (hasMissingElements)
-            return Collections.emptyList();
-
-        CBuilder builder = ctype.builder();
-
-        if (elementsList.isEmpty())
-            return singletonList(builder.build().withEOC(eoc));
-
-        // Use a Set to sort if needed and eliminate duplicates
-        Set<Composite> set = newSet();
-
-        for (int i = 0, m = elementsList.size(); i < m; i++)
-        {
-            List<ByteBuffer> elements = elementsList.get(i);
-            set.add(builder.buildWith(elements).withEOC(eoc));
-        }
-
-        return new ArrayList<>(set);
-    }
-
-    /**
-     * Returns a new <code>Set</code> instance that will be used to eliminate duplicates and sort the results.
-     *
-     * @return a new <code>Set</code> instance.
-     */
-    private Set<Composite> newSet()
-    {
-        return new TreeSet<>(ctype);
-    }
-
-    private void checkUpdateable()
-    {
-        if (!hasRemaining() || built)
-            throw new IllegalStateException("this CompositesBuilder cannot be updated anymore");
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
deleted file mode 100644
index 0458748..0000000
--- a/src/java/org/apache/cassandra/db/composites/CompoundCType.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.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)
-    {
-        super(isByteOrderComparable(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, 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, 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, false);
-        }
-
-        public Composite buildWith(List<ByteBuffer> newValues)
-        {
-            ByteBuffer[] buffers = Arrays.copyOf(values, values.length);
-            int newSize = size;
-            for (ByteBuffer value : newValues)
-                buffers[newSize++] = value;
-
-            if (newSize == buffers.length && type instanceof CellNameType && ((CellNameType)type).isDense())
-                return new CompoundDenseCellName(buffers);
-
-            return new CompoundComposite(buffers, newSize, false);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
deleted file mode 100644
index 7a21b01..0000000
--- a/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.composites;
-
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.utils.ObjectSizes;
-import org.apache.cassandra.utils.memory.AbstractAllocator;
-
-/**
- * A "truly-composite" Composite.
- */
-public class CompoundComposite extends AbstractComposite
-{
-    private static final long HEAP_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, boolean isStatic)
-    {
-        this.elements = elements;
-        this.size = size;
-        this.isStatic = isStatic;
-    }
-
-    public int size()
-    {
-        return size;
-    }
-
-    public ByteBuffer get(int i)
-    {
-        // Note: most consumer should validate that i is within bounds. However, for backward compatibility
-        // reasons, composite dense tables can have names that don't have all their component of the clustering
-        // columns, which may end up here with i > size(). For those calls, it's actually simpler to return null
-        // than to force the caller to special case.
-        return i >= size() ? null : 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 HEAP_SIZE + ObjectSizes.sizeOnHeapOf(elements);
-    }
-
-    public long unsharedHeapSizeExcludingData()
-    {
-        return HEAP_SIZE + ObjectSizes.sizeOnHeapExcludingData(elements);
-    }
-
-    public Composite copy(CFMetaData cfm, AbstractAllocator allocator)
-    {
-        return new CompoundComposite(elementsCopy(allocator), size, isStatic);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
deleted file mode 100644
index 1f471a8..0000000
--- a/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.composites;
-
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.config.CFMetaData;
-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, false);
-    }
-
-    CompoundDenseCellName(ByteBuffer[] elements, int size)
-    {
-        super(elements, size, false);
-    }
-
-    public int clusteringSize()
-    {
-        return size;
-    }
-
-    public ColumnIdentifier cql3ColumnName(CFMetaData metadata)
-    {
-        return null;
-    }
-
-    public ByteBuffer collectionElement()
-    {
-        return null;
-    }
-
-    public boolean isCollectionCell()
-    {
-        return false;
-    }
-
-    public boolean isSameCQL3RowAs(CellNameType type, CellName other)
-    {
-        // Dense cell imply one cell by CQL row so no other cell will be the same row.
-        return type.compare(this, other) == 0;
-    }
-
-    @Override
-    public long unsharedHeapSize()
-    {
-        return HEAP_SIZE + ObjectSizes.sizeOnHeapOf(elements);
-    }
-
-    @Override
-    public long unsharedHeapSizeExcludingData()
-    {
-        return HEAP_SIZE + ObjectSizes.sizeOnHeapExcludingData(elements);
-    }
-
-    public CellName copy(CFMetaData cfm, AbstractAllocator allocator)
-    {
-        return new CompoundDenseCellName(elementsCopy(allocator));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
deleted file mode 100644
index 2e409fb..0000000
--- a/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.composites;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.cassandra.config.CFMetaData;
-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, ColumnDefinition column)
-    {
-        // 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)
-        if (prefix instanceof CellName)
-            return (CellName)prefix;
-
-        // as noted below in makeWith(), compound dense cell names don't have to include all components
-        assert prefix instanceof CompoundComposite;
-        CompoundComposite lc = (CompoundComposite)prefix;
-        return new CompoundDenseCellName(lc.elements, lc.size);
-    }
-
-    protected Composite makeWith(ByteBuffer[] components, int size, Composite.EOC eoc, boolean isStatic)
-    {
-        assert !isStatic;
-        // A composite dense table cell name don't have to have all the component set to qualify as a
-        // proper CellName (for backward compatibility reasons mostly), so always return a cellName
-        CompoundDenseCellName c = new CompoundDenseCellName(components, size);
-        return eoc != Composite.EOC.NONE ? c.withEOC(eoc) : c;
-    }
-
-    protected Composite copyAndMakeWith(ByteBuffer[] components, int size, Composite.EOC eoc, boolean isStatic)
-    {
-        return makeWith(Arrays.copyOfRange(components, 0, size), size, eoc, isStatic);
-    }
-
-    public void addCQL3Column(ColumnIdentifier id) {}
-    public void removeCQL3Column(ColumnIdentifier id) {}
-
-    public CQL3Row.Builder CQL3RowBuilder(CFMetaData metadata, long now)
-    {
-        return makeDenseCQL3RowBuilder(now);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java b/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
deleted file mode 100644
index 03af6d0..0000000
--- a/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db.composites;
-
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.config.CFMetaData;
-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;
-
-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, false));
-
-    protected final ColumnIdentifier columnName;
-
-    // Not meant to be used directly, you should use the CellNameType method instead
-    CompoundSparseCellName(ColumnIdentifier columnName, boolean isStatic)
-    {
-        this(EMPTY_PREFIX, columnName, isStatic);
-    }
-
-    CompoundSparseCellName(ByteBuffer[] elements, ColumnIdentifier columnName, boolean isStatic)
-    {
-        this(elements, elements.length, columnName, isStatic);
-    }
-
-    CompoundSparseCellName(ByteBuffer[] elements, int size, ColumnIdentifier columnName, boolean isStatic)
-    {
-        super(elements, size, isStatic);
-        this.columnName = columnName;
-    }
-
-    @Override
-    public long unsharedHeapSize()
-    {
-        return HEAP_SIZE + ObjectSizes.sizeOnHeapOf(elements);
-    }
-
-    @Override
-    public long unsharedHeapSizeExcludingData()
-    {
-        return HEAP_SIZE + ObjectSizes.sizeOnHeapExcludingData(elements);
-    }
-
-    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(CFMetaData metadata)
-    {
-        return columnName;
-    }
-
-    public ByteBuffer collectionElement()
-    {
-        return null;
-    }
-
-    public boolean isCollectionCell()
-    {
-        return false;
-    }
-
-    public boolean isSameCQL3RowAs(CellNameType type, CellName other)
-    {
-        if (clusteringSize() != other.clusteringSize() || other.isStatic() != isStatic())
-            return false;
-
-        for (int i = 0; i < clusteringSize(); i++)
-        {
-            if (type.subtype(i).compare(elements[i], other.get(i)) != 0)
-                return false;
-        }
-        return true;
-    }
-
-    public CellName copy(CFMetaData cfm, 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, isStatic());
-    }
-
-    public static class WithCollection extends CompoundSparseCellName
-    {
-        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, boolean isStatic)
-        {
-            this(EMPTY_PREFIX, columnName, collectionElement, isStatic);
-        }
-
-        WithCollection(ByteBuffer[] elements, ColumnIdentifier columnName, ByteBuffer collectionElement, boolean isStatic)
-        {
-            this(elements, elements.length, columnName, collectionElement, isStatic);
-        }
-
-        WithCollection(ByteBuffer[] elements, int size, ColumnIdentifier columnName, ByteBuffer collectionElement, boolean isStatic)
-        {
-            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(CFMetaData cfm, 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), isStatic());
-        }
-
-        @Override
-        public long unsharedHeapSize()
-        {
-            return HEAP_SIZE + ObjectSizes.sizeOnHeapOf(elements)
-                   + ObjectSizes.sizeOnHeapExcludingData(collectionElement);
-        }
-
-        @Override
-        public long unsharedHeapSizeExcludingData()
-        {
-            return HEAP_SIZE + ObjectSizes.sizeOnHeapExcludingData(elements)
-                   + ObjectSizes.sizeOnHeapExcludingData(collectionElement);
-        }
-    }
-}
\ No newline at end of file