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/05/19 12:26:43 UTC

[1/2] git commit: Fixup for CASSANDRA-6914: null handling, ...

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 07dc6b5cc -> 33bd8c207


Fixup for CASSANDRA-6914: null handling, ...

patch by slebresne; reviewed by thobbs for CASSANDRA-7155


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

Branch: refs/heads/cassandra-2.1
Commit: 9d06ea6f144c3a1327d9c1322f1543067852701e
Parents: 58dc488
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Mon May 5 11:19:13 2014 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Mon May 19 10:51:31 2014 +0200

----------------------------------------------------------------------
 .../apache/cassandra/cql3/ColumnCondition.java  | 279 ++++++++++++-------
 .../cql3/statements/CQL3CasConditions.java      |  10 +-
 .../cql3/statements/ModificationStatement.java  |   4 +-
 .../cql3/statements/SelectStatement.java        |  20 +-
 .../cassandra/cql3/statements/Selection.java    |  37 +--
 .../db/marshal/AbstractCompositeType.java       |  63 +----
 .../cassandra/db/marshal/CollectionType.java    |  11 -
 .../cassandra/db/marshal/CompositeType.java     |  10 +-
 .../db/marshal/DynamicCompositeType.java        |  14 +-
 .../apache/cassandra/db/marshal/ListType.java   |  33 +++
 .../apache/cassandra/db/marshal/MapType.java    |  34 +++
 .../apache/cassandra/db/marshal/SetType.java    |   6 +
 .../serializers/CollectionSerializer.java       |   6 -
 .../cassandra/serializers/ListSerializer.java   |   9 +-
 .../cassandra/serializers/MapSerializer.java    |  17 +-
 .../cassandra/serializers/SetSerializer.java    |   9 +-
 .../apache/cassandra/utils/ByteBufferUtil.java  |  37 +++
 .../db/marshal/CollectionTypeTest.java          | 116 ++++++++
 18 files changed, 484 insertions(+), 231 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/cql3/ColumnCondition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnCondition.java b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
index 9fb3390..adc8e3a 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnCondition.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.cql3;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import com.google.common.base.Objects;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterators;
 
@@ -72,132 +73,209 @@ public class ColumnCondition
         value.collectMarkerSpecification(boundNames);
     }
 
-    public ColumnCondition.WithVariables with(List<ByteBuffer> variables)
+    public ColumnCondition.Bound bind(List<ByteBuffer> variables) throws InvalidRequestException
     {
-        return new WithVariables(variables);
+        return column.type instanceof CollectionType
+             ? (collectionElement == null ? new CollectionBound(this, variables) : new ElementAccessBound(this, variables))
+             : new SimpleBound(this, variables);
     }
 
-    public class WithVariables
+    public static abstract class Bound
     {
-        private final List<ByteBuffer> variables;
+        public final CFDefinition.Name column;
 
-        private WithVariables(List<ByteBuffer> variables)
+        protected Bound(CFDefinition.Name column)
         {
-            this.variables = variables;
+            this.column = column;
         }
 
-        public boolean equalsTo(WithVariables other) throws InvalidRequestException
-        {
-            if (!column().equals(other.column()))
-                return false;
-
-            if ((collectionElement() == null) != (other.collectionElement() == null))
-                return false;
-
-            if (collectionElement() != null)
-            {
-                assert column.type instanceof ListType || column.type instanceof MapType;
-                AbstractType<?> comparator = column.type instanceof ListType
-                                           ? Int32Type.instance
-                                           : ((MapType)column.type).keys;
-
-                if (comparator.compare(collectionElement().bindAndGet(variables), other.collectionElement().bindAndGet(variables)) != 0)
-                    return false;
-            }
-
-            return value().bindAndGet(variables).equals(other.value().bindAndGet(other.variables));
-        }
+        /**
+         * Validates whether this condition applies to {@code current}.
+         */
+        public abstract boolean appliesTo(ColumnNameBuilder rowPrefix, ColumnFamily current, long now) throws InvalidRequestException;
 
-        private CFDefinition.Name column()
+        public ByteBuffer getCollectionElementValue()
         {
-            return column;
+            return null;
         }
 
-        private Term collectionElement()
+        protected ColumnNameBuilder copyOrUpdatePrefix(CFMetaData cfm, ColumnNameBuilder rowPrefix)
         {
-            return collectionElement;
+            return column.kind == CFDefinition.Name.Kind.STATIC ? cfm.getStaticColumnNameBuilder() : rowPrefix.copy();
         }
 
-        private Term value()
+        protected boolean equalsValue(ByteBuffer value, Column c, AbstractType<?> type, long now)
         {
-            return value;
+            return value == null
+                 ? c == null || !c.isLive(now)
+                 : c != null && c.isLive(now) && type.compare(c.value(), value) == 0;
         }
 
-        public ByteBuffer getCollectionElementValue() throws InvalidRequestException
+        protected Iterator<Column> collectionColumns(ColumnNameBuilder collectionPrefix, ColumnFamily cf, final long now)
         {
-            return collectionElement == null ? null : collectionElement.bindAndGet(variables);
+            // We are testing for collection equality, so we need to have the expected values *and* only those.
+            ColumnSlice[] collectionSlice = new ColumnSlice[]{ new ColumnSlice(collectionPrefix.build(), collectionPrefix.buildAsEndOfRange()) };
+            // Filter live columns, this makes things simpler afterwards
+            return Iterators.filter(cf.iterator(collectionSlice), new Predicate<Column>()
+            {
+                public boolean apply(Column c)
+                {
+                    // we only care about live columns
+                    return c.isLive(now);
+                }
+            });
         }
+    }
 
-        private ColumnNameBuilder copyOrUpdatePrefix(CFMetaData cfm, ColumnNameBuilder rowPrefix)
+    private static class SimpleBound extends Bound
+    {
+        public final ByteBuffer value;
+
+        private SimpleBound(ColumnCondition condition, List<ByteBuffer> variables) throws InvalidRequestException
         {
-            return column.kind == CFDefinition.Name.Kind.STATIC ? cfm.getStaticColumnNameBuilder() : rowPrefix.copy();
+            super(condition.column);
+            assert !(column.type instanceof CollectionType) && condition.collectionElement == null;
+            this.value = condition.value.bindAndGet(variables);
         }
 
-        /**
-         * Validates whether this condition applies to {@code current}.
-         */
         public boolean appliesTo(ColumnNameBuilder rowPrefix, ColumnFamily current, long now) throws InvalidRequestException
         {
-            if (column.type instanceof CollectionType)
-                return collectionAppliesTo((CollectionType)column.type, rowPrefix, current, now);
-
-            assert collectionElement == null;
             ColumnNameBuilder prefix = copyOrUpdatePrefix(current.metadata(), rowPrefix);
             ByteBuffer columnName = column.kind == CFDefinition.Name.Kind.VALUE_ALIAS
                                   ? prefix.build()
                                   : prefix.add(column.name.key).build();
 
-            Column c = current.getColumn(columnName);
-            ByteBuffer v = value.bindAndGet(variables);
-            return v == null
-                 ? c == null || !c.isLive(now)
-                 : c != null && c.isLive(now) && column.type.compare(c.value(), v) == 0;
+            return equalsValue(value, current.getColumn(columnName), column.type, now);
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (!(o instanceof SimpleBound))
+                return false;
+
+            SimpleBound that = (SimpleBound)o;
+            if (!column.equals(that.column))
+                return false;
+
+            return value == null || that.value == null
+                 ? value == null && that.value == null
+                 : column.type.compare(value, that.value) == 0;
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Objects.hashCode(column, value);
+        }
+    }
+
+    private static class ElementAccessBound extends Bound
+    {
+        public final ByteBuffer collectionElement;
+        public final ByteBuffer value;
+
+        private ElementAccessBound(ColumnCondition condition, List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            super(condition.column);
+            assert column.type instanceof CollectionType && condition.collectionElement != null;
+            this.collectionElement = condition.collectionElement.bindAndGet(variables);
+            this.value = condition.value.bindAndGet(variables);
         }
 
-        private boolean collectionAppliesTo(CollectionType type, ColumnNameBuilder rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException
+        public boolean appliesTo(ColumnNameBuilder rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException
         {
+            if (collectionElement == null)
+                throw new InvalidRequestException("Invalid null value for " + (column.type instanceof MapType ? "map" : "list") + " element access");
+
             ColumnNameBuilder collectionPrefix = copyOrUpdatePrefix(current.metadata(), rowPrefix).add(column.name.key);
-            Term.Terminal v = value.bind(variables);
+            if (column.type instanceof MapType)
+                return equalsValue(value, current.getColumn(collectionPrefix.add(collectionElement).build()), ((MapType)column.type).values, now);
 
-            // For map element access, we won't iterate over the collection, so deal with that first. In other case, we do.
-            if (collectionElement != null && type instanceof MapType)
-            {
-                ByteBuffer e = collectionElement.bindAndGet(variables);
-                if (e == null)
-                    throw new InvalidRequestException("Invalid null value for map access");
-                return mapElementAppliesTo((MapType)type, current, collectionPrefix, e, v.get(), now);
-            }
+            assert column.type instanceof ListType;
+            int idx = ByteBufferUtil.toInt(collectionElement);
+            if (idx < 0)
+                throw new InvalidRequestException(String.format("Invalid negative list index %d", idx));
 
-            // We are testing for collection equality, so we need to have the expected values *and* only those.
-            ColumnSlice[] collectionSlice = new ColumnSlice[]{ new ColumnSlice(collectionPrefix.build(), collectionPrefix.buildAsEndOfRange()) };
-            // Filter live columns, this makes things simpler afterwards
-            Iterator<Column> iter = Iterators.filter(current.iterator(collectionSlice), new Predicate<Column>()
-            {
-                public boolean apply(Column c)
-                {
-                    // we only care about live columns
-                    return c.isLive(now);
-                }
-            });
+            Iterator<Column> iter = collectionColumns(collectionPrefix, current, now);
+            int adv = Iterators.advance(iter, idx);
+            if (adv != idx || !iter.hasNext())
+                throw new InvalidRequestException(String.format("List index %d out of bound, list has size %d", idx, adv));
 
-            if (v == null)
-                return !iter.hasNext();
+            // We don't support null values inside collections, so a condition like 'IF l[3] = null' can only
+            // be false. We do special case though, as the compare below might mind getting a null.
+            if (value == null)
+                return false;
+
+            return ((ListType)column.type).elements.compare(iter.next().value(), value) == 0;
+        }
+
+        public ByteBuffer getCollectionElementValue()
+        {
+            return collectionElement;
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (!(o instanceof ElementAccessBound))
+                return false;
+
+            ElementAccessBound that = (ElementAccessBound)o;
+            if (!column.equals(that.column))
+                return false;
+
+            if ((collectionElement == null) != (that.collectionElement == null))
+                return false;
 
             if (collectionElement != null)
             {
-                assert type instanceof ListType;
-                ByteBuffer e = collectionElement.bindAndGet(variables);
-                if (e == null)
-                    throw new InvalidRequestException("Invalid null value for list access");
+                assert column.type instanceof ListType || column.type instanceof MapType;
+                AbstractType<?> comparator = column.type instanceof ListType
+                                           ? Int32Type.instance
+                                           : ((MapType)column.type).keys;
 
-                return listElementAppliesTo((ListType)type, iter, e, v.get());
+                if (comparator.compare(collectionElement, that.collectionElement) != 0)
+                    return false;
             }
 
+            return column.type.compare(value, that.value) == 0;
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Objects.hashCode(column, collectionElement, value);
+        }
+    }
+
+    private static class CollectionBound extends Bound
+    {
+        public final Term.Terminal value;
+
+        private CollectionBound(ColumnCondition condition, List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            super(condition.column);
+            assert column.type instanceof CollectionType && condition.collectionElement == null;
+            this.value = condition.value.bind(variables);
+        }
+
+        public boolean appliesTo(ColumnNameBuilder rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException
+        {
+            CollectionType type = (CollectionType)column.type;
+            CFMetaData cfm = current.metadata();
+
+            ColumnNameBuilder collectionPrefix = copyOrUpdatePrefix(cfm, rowPrefix).add(column.name.key);
+
+            Iterator<Column> iter = collectionColumns(collectionPrefix, current, now);
+            if (value == null)
+                return !iter.hasNext();
+
             switch (type.kind)
             {
-                case LIST: return listAppliesTo((ListType)type, current.metadata(), iter, ((Lists.Value)v).elements);
-                case SET: return setAppliesTo((SetType)type, current.metadata(), iter, ((Sets.Value)v).elements);
-                case MAP: return mapAppliesTo((MapType)type, current.metadata(), iter, ((Maps.Value)v).map);
+                case LIST: return listAppliesTo((ListType)type, cfm, iter, ((Lists.Value)value).elements);
+                case SET: return setAppliesTo((SetType)type, cfm, iter, ((Sets.Value)value).elements);
+                case MAP: return mapAppliesTo((MapType)type, cfm, iter, ((Maps.Value)value).map);
             }
             throw new AssertionError();
         }
@@ -217,19 +295,6 @@ public class ColumnCondition
             return !iter.hasNext();
         }
 
-        private boolean listElementAppliesTo(ListType type, Iterator<Column> iter, ByteBuffer element, ByteBuffer value) throws InvalidRequestException
-        {
-            int idx = ByteBufferUtil.toInt(element);
-            if (idx < 0)
-                throw new InvalidRequestException(String.format("Invalid negative list index %d", idx));
-
-            int adv = Iterators.advance(iter, idx);
-            if (adv != idx || !iter.hasNext())
-                throw new InvalidRequestException(String.format("List index %d out of bound, list has size %d", idx, adv));
-
-            return type.elements.compare(iter.next().value(), value) == 0;
-        }
-
         private boolean setAppliesTo(SetType type, CFMetaData cfm, Iterator<Column> iter, Set<ByteBuffer> elements)
         {
             Set<ByteBuffer> remaining = new TreeSet<>(type.elements);
@@ -262,11 +327,31 @@ public class ColumnCondition
             return remaining.isEmpty();
         }
 
-        private boolean mapElementAppliesTo(MapType type, ColumnFamily current, ColumnNameBuilder collectionPrefix, ByteBuffer element, ByteBuffer value, long now)
+        @Override
+        public boolean equals(Object o)
+        {
+            if (!(o instanceof CollectionBound))
+                return false;
+
+            CollectionBound that = (CollectionBound)o;
+            if (!column.equals(that.column))
+                return false;
+
+            // Slightly inefficient because it serialize the collection just for the sake of comparison.
+            // We could improve by adding an equals() method to Lists.Value, Sets.Value and Maps.Value but
+            // this method is only called when there is 2 conditions on the same collection to make sure
+            // both are not incompatible, so overall it's probably not worth the effort.
+            ByteBuffer thisVal = value.get();
+            ByteBuffer thatVal = that.value.get();
+            return thisVal == null || thatVal == null
+                 ? thisVal == null && thatVal == null
+                 : column.type.compare(thisVal, thatVal) == 0;
+        }
+
+        @Override
+        public int hashCode()
         {
-            ByteBuffer name = collectionPrefix.add(element).build();
-            Column c = current.getColumn(name);
-            return c != null && c.isLive(now) && type.values.compare(c.value(), value) == 0;
+            return Objects.hashCode(column, value.get());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java b/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
index 9f67bc0..775a236 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
@@ -167,7 +167,7 @@ public class CQL3CasConditions implements CASConditions
 
     private static class ColumnsConditions extends RowCondition
     {
-        private final Map<Pair<ColumnIdentifier, ByteBuffer>, ColumnCondition.WithVariables> conditions = new HashMap<>();
+        private final Map<Pair<ColumnIdentifier, ByteBuffer>, ColumnCondition.Bound> conditions = new HashMap<>();
 
         private ColumnsConditions(ColumnNameBuilder rowPrefix, long now)
         {
@@ -180,10 +180,10 @@ public class CQL3CasConditions implements CASConditions
             {
                 // We will need the variables in appliesTo but with protocol batches, each condition in this object can have a
                 // different list of variables.
-                ColumnCondition.WithVariables current = condition.with(variables);
-                ColumnCondition.WithVariables previous = conditions.put(Pair.create(condition.column.name, current.getCollectionElementValue()), current);
+                ColumnCondition.Bound current = condition.bind(variables);
+                ColumnCondition.Bound previous = conditions.put(Pair.create(condition.column.name, current.getCollectionElementValue()), current);
                 // If 2 conditions are actually equal, let it slide
-                if (previous != null && !previous.equalsTo(current))
+                if (previous != null && !previous.equals(current))
                     throw new InvalidRequestException("Duplicate and incompatible conditions for column " + condition.column.name);
             }
         }
@@ -193,7 +193,7 @@ public class CQL3CasConditions implements CASConditions
             if (current == null)
                 return conditions.isEmpty();
 
-            for (ColumnCondition.WithVariables condition : conditions.values())
+            for (ColumnCondition.Bound condition : conditions.values())
                 if (!condition.appliesTo(rowPrefix, current, now))
                     return false;
             return true;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index f8c4042..448722e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -649,7 +649,9 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
         }
         else
         {
-            List<CFDefinition.Name> names = new ArrayList<CFDefinition.Name>();
+            // We can have multiple conditions on the same columns (for collections) so use a set
+            // to avoid duplicate, but preserve the order just to it follows the order of IF in the query in general
+            Set<CFDefinition.Name> names = new LinkedHashSet<CFDefinition.Name>();
             // Adding the partition key for batches to disambiguate if the conditions span multipe rows (we don't add them outside
             // of batches for compatibility sakes).
             if (isBatch)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 68adbbd..2468eb9 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -128,9 +128,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         }
 
         // Otherwise, check the selected columns
-        selectsStaticColumns = !Iterables.isEmpty(Iterables.filter(selection.getColumnsList(), isStaticFilter));
+        selectsStaticColumns = !Iterables.isEmpty(Iterables.filter(selection.getColumns(), isStaticFilter));
         selectsOnlyStaticColumns = true;
-        for (CFDefinition.Name name : selection.getColumnsList())
+        for (CFDefinition.Name name : selection.getColumns())
         {
             if (name.kind != CFDefinition.Name.Kind.KEY_ALIAS && name.kind != CFDefinition.Name.Kind.STATIC)
             {
@@ -774,7 +774,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         if (!cfDef.hasCollections)
             return false;
 
-        for (CFDefinition.Name name : selection.getColumnsList())
+        for (CFDefinition.Name name : selection.getColumns())
         {
             if (name.type instanceof CollectionType)
                 return true;
@@ -987,8 +987,8 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             if (!cf.hasOnlyTombstones(now))
             {
                 result.newRow();
-                // selection.getColumnsList() will contain only the partition key components - all of them.
-                for (CFDefinition.Name name : selection.getColumnsList())
+                // selection.getColumns() will contain only the partition key components - all of them.
+                for (CFDefinition.Name name : selection.getColumns())
                     result.add(keyComponents[name.position]);
             }
         }
@@ -1017,7 +1017,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
 
                 result.newRow();
                 // Respect selection order
-                for (CFDefinition.Name name : selection.getColumnsList())
+                for (CFDefinition.Name name : selection.getColumns())
                 {
                     switch (name.kind)
                     {
@@ -1085,7 +1085,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
 
             // Static case: One cqlRow for all columns
             result.newRow();
-            for (CFDefinition.Name name : selection.getColumnsList())
+            for (CFDefinition.Name name : selection.getColumns())
             {
                 if (name.kind == CFDefinition.Name.Kind.KEY_ALIAS)
                     result.add(keyComponents[name.position]);
@@ -1097,7 +1097,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
 
     private boolean hasValueForQuery(ColumnGroupMap staticGroup)
     {
-        for (CFDefinition.Name name : Iterables.filter(selection.getColumnsList(), isStaticFilter))
+        for (CFDefinition.Name name : Iterables.filter(selection.getColumns(), isStaticFilter))
             if (staticGroup.hasValueFor(name.name.key))
                 return true;
         return false;
@@ -1160,7 +1160,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
     {
         // Respect requested order
         result.newRow();
-        for (CFDefinition.Name name : selection.getColumnsList())
+        for (CFDefinition.Name name : selection.getColumns())
         {
             switch (name.kind)
             {
@@ -1258,7 +1258,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                                 : Selection.fromSelectors(cfDef, selectClause);
 
             if (parameters.isDistinct)
-                validateDistinctSelection(selection.getColumnsList(), cfDef.partitionKeys());
+                validateDistinctSelection(selection.getColumns(), cfDef.partitionKeys());
 
             Term prepLimit = null;
             if (limit != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/cql3/statements/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Selection.java b/src/java/org/apache/cassandra/cql3/statements/Selection.java
index 9760311..123ddc3 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.cql3.statements;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
 import org.apache.cassandra.cql3.*;
@@ -36,14 +37,14 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 
 public abstract class Selection
 {
-    private final List<CFDefinition.Name> columnsList;
+    private final Collection<CFDefinition.Name> columns;
     private final List<ColumnSpecification> metadata;
     private final boolean collectTimestamps;
     private final boolean collectTTLs;
 
-    protected Selection(List<CFDefinition.Name> columnsList, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs)
+    protected Selection(Collection<CFDefinition.Name> columns, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs)
     {
-        this.columnsList = columnsList;
+        this.columns = columns;
         this.metadata = metadata;
         this.collectTimestamps = collectTimestamps;
         this.collectTTLs = collectTTLs;
@@ -68,9 +69,9 @@ public abstract class Selection
         return new SimpleSelection(all, true);
     }
 
-    public static Selection forColumns(List<CFDefinition.Name> columnsList)
+    public static Selection forColumns(Collection<CFDefinition.Name> columns)
     {
-        return new SimpleSelection(columnsList, false);
+        return new SimpleSelection(columns, false);
     }
 
     private static boolean isUsingFunction(List<RawSelector> rawSelectors)
@@ -213,7 +214,7 @@ public abstract class Selection
     public List<ColumnIdentifier> regularAndStaticColumnsToFetch()
     {
         List<ColumnIdentifier> toFetch = new ArrayList<ColumnIdentifier>();
-        for (CFDefinition.Name name : columnsList)
+        for (CFDefinition.Name name : columns)
         {
             if (name.kind == CFDefinition.Name.Kind.COLUMN_METADATA || name.kind == CFDefinition.Name.Kind.STATIC)
                 toFetch.add(name.name);
@@ -224,9 +225,9 @@ public abstract class Selection
     /**
      * @return the list of CQL3 columns value this SelectionClause needs.
      */
-    public List<CFDefinition.Name> getColumnsList()
+    public Collection<CFDefinition.Name> getColumns()
     {
-        return columnsList;
+        return columns;
     }
 
     public ResultSetBuilder resultSetBuilder(long now)
@@ -261,8 +262,8 @@ public abstract class Selection
         private ResultSetBuilder(long now)
         {
             this.resultSet = new ResultSet(metadata);
-            this.timestamps = collectTimestamps ? new long[columnsList.size()] : null;
-            this.ttls = collectTTLs ? new int[columnsList.size()] : null;
+            this.timestamps = collectTimestamps ? new long[columns.size()] : null;
+            this.ttls = collectTTLs ? new int[columns.size()] : null;
             this.now = now;
         }
 
@@ -296,7 +297,7 @@ public abstract class Selection
         {
             if (current != null)
                 resultSet.addRow(handleRow(this));
-            current = new ArrayList<ByteBuffer>(columnsList.size());
+            current = new ArrayList<ByteBuffer>(columns.size());
         }
 
         public ResultSet build() throws InvalidRequestException
@@ -315,19 +316,19 @@ public abstract class Selection
     {
         private final boolean isWildcard;
 
-        public SimpleSelection(List<CFDefinition.Name> columnsList, boolean isWildcard)
+        public SimpleSelection(Collection<CFDefinition.Name> columns, boolean isWildcard)
         {
-            this(columnsList, new ArrayList<ColumnSpecification>(columnsList), isWildcard);
+            this(columns, new ArrayList<ColumnSpecification>(columns), isWildcard);
         }
 
-        public SimpleSelection(List<CFDefinition.Name> columnsList, List<ColumnSpecification> metadata, boolean isWildcard)
+        public SimpleSelection(Collection<CFDefinition.Name> columns, List<ColumnSpecification> metadata, boolean isWildcard)
         {
             /*
              * In theory, even a simple selection could have multiple time the same column, so we
-             * could filter those duplicate out of columnsList. But since we're very unlikely to
+             * could filter those duplicate out of columns. But since we're very unlikely to
              * get much duplicate in practice, it's more efficient not to bother.
              */
-            super(columnsList, metadata, false, false);
+            super(columns, metadata, false, false);
             this.isWildcard = isWildcard;
         }
 
@@ -459,9 +460,9 @@ public abstract class Selection
     {
         private final List<Selector> selectors;
 
-        public SelectionWithFunctions(List<CFDefinition.Name> columnsList, List<ColumnSpecification> metadata, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
+        public SelectionWithFunctions(Collection<CFDefinition.Name> columns, List<ColumnSpecification> metadata, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
         {
-            super(columnsList, metadata, collectTimestamps, collectTTLs);
+            super(columns, metadata, collectTimestamps, collectTTLs);
             this.selectors = selectors;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
index c789d9b..de58158 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
@@ -17,15 +17,16 @@
  */
 package org.apache.cassandra.db.marshal;
 
-import org.apache.cassandra.serializers.TypeSerializer;
-import org.apache.cassandra.serializers.BytesSerializer;
-import org.apache.cassandra.serializers.MarshalException;
-
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.serializers.BytesSerializer;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
 /**
  * A class avoiding class duplication between CompositeType and
  * DynamicCompositeType.
@@ -34,44 +35,6 @@ import java.util.List;
  */
 public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
 {
-
-    // changes bb position
-    protected static int getShortLength(ByteBuffer bb)
-    {
-        int length = (bb.get() & 0xFF) << 8;
-        return length | (bb.get() & 0xFF);
-    }
-
-    // Doesn't change bb position
-    protected static int getShortLength(ByteBuffer bb, int position)
-    {
-        int length = (bb.get(position) & 0xFF) << 8;
-        return length | (bb.get(position + 1) & 0xFF);
-    }
-
-    // changes bb position
-    protected static void putShortLength(ByteBuffer bb, int length)
-    {
-        bb.put((byte) ((length >> 8) & 0xFF));
-        bb.put((byte) (length & 0xFF));
-    }
-
-    // changes bb position
-    protected static ByteBuffer getBytes(ByteBuffer bb, int length)
-    {
-        ByteBuffer copy = bb.duplicate();
-        copy.limit(copy.position() + length);
-        bb.position(bb.position() + length);
-        return copy;
-    }
-
-    // changes bb position
-    protected static ByteBuffer getWithShortLength(ByteBuffer bb)
-    {
-        int length = getShortLength(bb);
-        return getBytes(bb, length);
-    }
-
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
         if (o1 == null || !o1.hasRemaining())
@@ -95,8 +58,8 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
         {
             AbstractType<?> comparator = getComparator(i, bb1, bb2);
 
-            ByteBuffer value1 = getWithShortLength(bb1);
-            ByteBuffer value2 = getWithShortLength(bb2);
+            ByteBuffer value1 = ByteBufferUtil.readBytesWithShortLength(bb1);
+            ByteBuffer value2 = ByteBufferUtil.readBytesWithShortLength(bb2);
 
             int cmp = comparator.compareCollectionMembers(value1, value2, previous);
             if (cmp != 0)
@@ -135,7 +98,7 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
         while (bb.remaining() > 0)
         {
             getComparator(i++, bb);
-            l.add(getWithShortLength(bb));
+            l.add(ByteBufferUtil.readBytesWithShortLength(bb));
             bb.get(); // skip end-of-component
         }
         return l.toArray(new ByteBuffer[l.size()]);
@@ -164,7 +127,7 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
         while (bb.remaining() > 0)
         {
             AbstractType comparator = getComparator(i, bb);
-            ByteBuffer value = getWithShortLength(bb);
+            ByteBuffer value = ByteBufferUtil.readBytesWithShortLength(bb);
 
             list.add( new CompositeComponent(comparator,value) );
 
@@ -237,7 +200,7 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
                 sb.append(":");
 
             AbstractType<?> comparator = getAndAppendComparator(i, bb, sb);
-            ByteBuffer value = getWithShortLength(bb);
+            ByteBuffer value = ByteBufferUtil.readBytesWithShortLength(bb);
 
             sb.append(escape(comparator.getString(value)));
 
@@ -284,7 +247,7 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
         for (ByteBuffer component : components)
         {
             comparators.get(i).serializeComparator(bb);
-            putShortLength(bb, component.remaining());
+            ByteBufferUtil.writeShortLength(bb, component.remaining());
             bb.put(component); // it's ok to consume component as we won't use it anymore
             bb.put((byte)0);
             ++i;
@@ -310,11 +273,11 @@ public abstract class AbstractCompositeType extends AbstractType<ByteBuffer>
 
             if (bb.remaining() < 2)
                 throw new MarshalException("Not enough bytes to read value size of component " + i);
-            int length = getShortLength(bb);
+            int length = ByteBufferUtil.readShortLength(bb);
 
             if (bb.remaining() < length)
                 throw new MarshalException("Not enough bytes to read value of component " + i);
-            ByteBuffer value = getBytes(bb, length);
+            ByteBuffer value = ByteBufferUtil.readBytes(bb, length);
 
             comparator.validateCollectionMember(value, previous);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/db/marshal/CollectionType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CollectionType.java b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
index 07c86e0..29b77a0 100644
--- a/src/java/org/apache/cassandra/db/marshal/CollectionType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
@@ -68,11 +68,6 @@ public abstract class CollectionType<T> extends AbstractType<T>
         return sb.toString();
     }
 
-    public int compare(ByteBuffer o1, ByteBuffer o2)
-    {
-        throw new UnsupportedOperationException("CollectionType should not be use directly as a comparator");
-    }
-
     public String getString(ByteBuffer bytes)
     {
         return BytesType.instance.getString(bytes);
@@ -131,12 +126,6 @@ public abstract class CollectionType<T> extends AbstractType<T>
         return pack(buffers, elements, size);
     }
 
-    protected static int getUnsignedShort(ByteBuffer bb)
-    {
-        int length = (bb.get() & 0xFF) << 8;
-        return length | (bb.get() & 0xFF);
-    }
-
     public CQL3Type asCQL3Type()
     {
         return new CQL3Type.Collection(this);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/db/marshal/CompositeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
index 32fc432..2f537fb 100644
--- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
@@ -91,11 +91,11 @@ public class CompositeType extends AbstractCompositeType
         if (bb.remaining() < 2)
             return false;
 
-        int header = getShortLength(bb, bb.position());
+        int header = ByteBufferUtil.getShortLength(bb, bb.position());
         if ((header & 0xFFFF) != STATIC_MARKER)
             return false;
 
-        getShortLength(bb); // Skip header
+        ByteBufferUtil.readShortLength(bb); // Skip header
         return true;
     }
 
@@ -178,7 +178,7 @@ public class CompositeType extends AbstractCompositeType
         int i = 0;
         while (bb.remaining() > 0)
         {
-            ByteBuffer c = getWithShortLength(bb);
+            ByteBuffer c = ByteBufferUtil.readBytesWithShortLength(bb);
             if (i == idx)
                 return c;
 
@@ -197,7 +197,7 @@ public class CompositeType extends AbstractCompositeType
 
     public static boolean isStaticName(ByteBuffer bb)
     {
-        return bb.remaining() >= 2 && (getShortLength(bb, bb.position()) & 0xFFFF) == STATIC_MARKER;
+        return bb.remaining() >= 2 && (ByteBufferUtil.getShortLength(bb, bb.position()) & 0xFFFF) == STATIC_MARKER;
     }
 
     @Override
@@ -363,7 +363,7 @@ public class CompositeType extends AbstractCompositeType
         ByteBuffer out = ByteBuffer.allocate(totalLength);
         for (ByteBuffer bb : buffers)
         {
-            putShortLength(out, bb.remaining());
+            ByteBufferUtil.writeShortLength(out, bb.remaining());
             out.put(bb.duplicate());
             out.put((byte) 0);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java b/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
index 9b56a82..4285d9c 100644
--- a/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
@@ -85,10 +85,10 @@ public class DynamicCompositeType extends AbstractCompositeType
     {
         try
         {
-            int header = getShortLength(bb);
+            int header = ByteBufferUtil.readShortLength(bb);
             if ((header & 0x8000) == 0)
             {
-                String name = ByteBufferUtil.string(getBytes(bb, header));
+                String name = ByteBufferUtil.string(ByteBufferUtil.readBytes(bb, header));
                 return TypeParser.parse(name);
             }
             else
@@ -147,10 +147,10 @@ public class DynamicCompositeType extends AbstractCompositeType
     {
         try
         {
-            int header = getShortLength(bb);
+            int header = ByteBufferUtil.readShortLength(bb);
             if ((header & 0x8000) == 0)
             {
-                String name = ByteBufferUtil.string(getBytes(bb, header));
+                String name = ByteBufferUtil.string(ByteBufferUtil.readBytes(bb, header));
                 sb.append(name).append("@");
                 return TypeParser.parse(name);
             }
@@ -184,13 +184,13 @@ public class DynamicCompositeType extends AbstractCompositeType
         AbstractType<?> comparator = null;
         if (bb.remaining() < 2)
             throw new MarshalException("Not enough bytes to header of the comparator part of component " + i);
-        int header = getShortLength(bb);
+        int header = ByteBufferUtil.readShortLength(bb);
         if ((header & 0x8000) == 0)
         {
             if (bb.remaining() < header)
                 throw new MarshalException("Not enough bytes to read comparator name of component " + i);
 
-            ByteBuffer value = getBytes(bb, header);
+            ByteBuffer value = ByteBufferUtil.readBytes(bb, header);
             try
             {
                 comparator = TypeParser.parse(ByteBufferUtil.string(value));
@@ -308,7 +308,7 @@ public class DynamicCompositeType extends AbstractCompositeType
                 header = 0x8000 | (((byte)comparatorName.charAt(0)) & 0xFF);
             else
                 header = comparatorName.length();
-            putShortLength(bb, header);
+            ByteBufferUtil.writeShortLength(bb, header);
 
             if (!isAlias)
                 bb.put(ByteBufferUtil.bytes(comparatorName));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/db/marshal/ListType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/ListType.java b/src/java/org/apache/cassandra/db/marshal/ListType.java
index 808ba45..4b45bd7 100644
--- a/src/java/org/apache/cassandra/db/marshal/ListType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ListType.java
@@ -26,6 +26,7 @@ import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.ListSerializer;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class ListType<T> extends CollectionType<List<T>>
 {
@@ -77,6 +78,38 @@ public class ListType<T> extends CollectionType<List<T>>
         return serializer;
     }
 
+    @Override
+    public int compare(ByteBuffer o1, ByteBuffer o2)
+    {
+        return compareListOrSet(elements, o1, o2);
+    }
+
+    static int compareListOrSet(AbstractType<?> elementsComparator, ByteBuffer o1, ByteBuffer o2)
+    {
+        // Note that this is only used if the collection is inside an UDT
+        if (o1 == null || !o1.hasRemaining())
+            return o2 == null || !o2.hasRemaining() ? 0 : -1;
+        if (o2 == null || !o2.hasRemaining())
+            return 1;
+
+        ByteBuffer bb1 = o1.duplicate();
+        ByteBuffer bb2 = o2.duplicate();
+
+        int size1 = ByteBufferUtil.readShortLength(bb1);
+        int size2 = ByteBufferUtil.readShortLength(bb2);
+
+        for (int i = 0; i < Math.min(size1, size2); i++)
+        {
+            ByteBuffer v1 = ByteBufferUtil.readBytesWithShortLength(bb1);
+            ByteBuffer v2 = ByteBufferUtil.readBytesWithShortLength(bb2);
+            int cmp = elementsComparator.compare(v1, v2);
+            if (cmp != 0)
+                return cmp;
+        }
+
+        return size1 == size2 ? 0 : (size1 < size2 ? -1 : 1);
+    }
+
     protected void appendToStringBuilder(StringBuilder sb)
     {
         sb.append(getClass().getName()).append(TypeParser.stringifyTypeParameters(Collections.<AbstractType<?>>singletonList(elements)));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/db/marshal/MapType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/MapType.java b/src/java/org/apache/cassandra/db/marshal/MapType.java
index fd96da7..08f795f 100644
--- a/src/java/org/apache/cassandra/db/marshal/MapType.java
+++ b/src/java/org/apache/cassandra/db/marshal/MapType.java
@@ -26,6 +26,7 @@ import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.MapSerializer;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class MapType<K, V> extends CollectionType<Map<K, V>>
 {
@@ -76,6 +77,39 @@ public class MapType<K, V> extends CollectionType<Map<K, V>>
     }
 
     @Override
+    public int compare(ByteBuffer o1, ByteBuffer o2)
+    {
+        // Note that this is only used if the collection is inside an UDT
+        if (o1 == null || !o1.hasRemaining())
+            return o2 == null || !o2.hasRemaining() ? 0 : -1;
+        if (o2 == null || !o2.hasRemaining())
+            return 1;
+
+        ByteBuffer bb1 = o1.duplicate();
+        ByteBuffer bb2 = o2.duplicate();
+
+        int size1 = ByteBufferUtil.readShortLength(bb1);
+        int size2 = ByteBufferUtil.readShortLength(bb2);
+
+        for (int i = 0; i < Math.min(size1, size2); i++)
+        {
+            ByteBuffer k1 = ByteBufferUtil.readBytesWithShortLength(bb1);
+            ByteBuffer k2 = ByteBufferUtil.readBytesWithShortLength(bb2);
+            int cmp = keys.compare(k1, k2);
+            if (cmp != 0)
+                return cmp;
+
+            ByteBuffer v1 = ByteBufferUtil.readBytesWithShortLength(bb1);
+            ByteBuffer v2 = ByteBufferUtil.readBytesWithShortLength(bb2);
+            cmp = values.compare(v1, v2);
+            if (cmp != 0)
+                return cmp;
+        }
+
+        return size1 == size2 ? 0 : (size1 < size2 ? -1 : 1);
+    }
+
+    @Override
     public TypeSerializer<Map<K, V>> getSerializer()
     {
         return serializer;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/db/marshal/SetType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/SetType.java b/src/java/org/apache/cassandra/db/marshal/SetType.java
index c947d26..614ecc7 100644
--- a/src/java/org/apache/cassandra/db/marshal/SetType.java
+++ b/src/java/org/apache/cassandra/db/marshal/SetType.java
@@ -72,6 +72,12 @@ public class SetType<T> extends CollectionType<Set<T>>
         return EmptyType.instance;
     }
 
+    @Override
+    public int compare(ByteBuffer o1, ByteBuffer o2)
+    {
+        return ListType.compareListOrSet(elements, o1, o2);
+    }
+
     public TypeSerializer<Set<T>> getSerializer()
     {
         return serializer;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/serializers/CollectionSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/serializers/CollectionSerializer.java b/src/java/org/apache/cassandra/serializers/CollectionSerializer.java
index 9d4e4a4..83a391d 100644
--- a/src/java/org/apache/cassandra/serializers/CollectionSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/CollectionSerializer.java
@@ -48,10 +48,4 @@ public abstract class CollectionSerializer<T> implements TypeSerializer<T>
             size += 2 + bb.remaining();
         return pack(buffers, elements, size);
     }
-
-    protected static int getUnsignedShort(ByteBuffer bb)
-    {
-        int length = (bb.get() & 0xFF) << 8;
-        return length | (bb.get() & 0xFF);
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/serializers/ListSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/serializers/ListSerializer.java b/src/java/org/apache/cassandra/serializers/ListSerializer.java
index 0272618..59f25d2 100644
--- a/src/java/org/apache/cassandra/serializers/ListSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/ListSerializer.java
@@ -22,6 +22,8 @@ import java.nio.BufferUnderflowException;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.apache.cassandra.utils.ByteBufferUtil;
+
 public class ListSerializer<T> extends CollectionSerializer<List<T>>
 {
     // interning instances
@@ -50,14 +52,11 @@ public class ListSerializer<T> extends CollectionSerializer<List<T>>
         try
         {
             ByteBuffer input = bytes.duplicate();
-            int n = getUnsignedShort(input);
+            int n = ByteBufferUtil.readShortLength(input);
             List<T> l = new ArrayList<T>(n);
             for (int i = 0; i < n; i++)
             {
-                int s = getUnsignedShort(input);
-                byte[] data = new byte[s];
-                input.get(data);
-                ByteBuffer databb = ByteBuffer.wrap(data);
+                ByteBuffer databb = ByteBufferUtil.readBytesWithShortLength(input);
                 elements.validate(databb);
                 l.add(elements.deserialize(databb));
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/serializers/MapSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/serializers/MapSerializer.java b/src/java/org/apache/cassandra/serializers/MapSerializer.java
index f04de6d..f79d07f 100644
--- a/src/java/org/apache/cassandra/serializers/MapSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/MapSerializer.java
@@ -18,12 +18,13 @@
 
 package org.apache.cassandra.serializers;
 
-import org.apache.cassandra.utils.Pair;
-
 import java.nio.BufferUnderflowException;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+
 public class MapSerializer<K, V> extends CollectionSerializer<Map<K, V>>
 {
     // interning instances
@@ -55,20 +56,14 @@ public class MapSerializer<K, V> extends CollectionSerializer<Map<K, V>>
         try
         {
             ByteBuffer input = bytes.duplicate();
-            int n = getUnsignedShort(input);
+            int n = ByteBufferUtil.readShortLength(input);
             Map<K, V> m = new LinkedHashMap<K, V>(n);
             for (int i = 0; i < n; i++)
             {
-                int sk = getUnsignedShort(input);
-                byte[] datak = new byte[sk];
-                input.get(datak);
-                ByteBuffer kbb = ByteBuffer.wrap(datak);
+                ByteBuffer kbb = ByteBufferUtil.readBytesWithShortLength(input);
                 keys.validate(kbb);
 
-                int sv = getUnsignedShort(input);
-                byte[] datav = new byte[sv];
-                input.get(datav);
-                ByteBuffer vbb = ByteBuffer.wrap(datav);
+                ByteBuffer vbb = ByteBufferUtil.readBytesWithShortLength(input);
                 values.validate(vbb);
 
                 m.put(keys.deserialize(kbb), values.deserialize(vbb));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/serializers/SetSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/serializers/SetSerializer.java b/src/java/org/apache/cassandra/serializers/SetSerializer.java
index d424a11..d6d7062 100644
--- a/src/java/org/apache/cassandra/serializers/SetSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/SetSerializer.java
@@ -22,6 +22,8 @@ import java.nio.BufferUnderflowException;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.apache.cassandra.utils.ByteBufferUtil;
+
 public class SetSerializer<T> extends CollectionSerializer<Set<T>>
 {
     // interning instances
@@ -50,14 +52,11 @@ public class SetSerializer<T> extends CollectionSerializer<Set<T>>
         try
         {
             ByteBuffer input = bytes.duplicate();
-            int n = getUnsignedShort(input);
+            int n = ByteBufferUtil.readShortLength(input);
             Set<T> l = new LinkedHashSet<T>(n);
             for (int i = 0; i < n; i++)
             {
-                int s = getUnsignedShort(input);
-                byte[] data = new byte[s];
-                input.get(data);
-                ByteBuffer databb = ByteBuffer.wrap(data);
+                ByteBuffer databb = ByteBufferUtil.readBytesWithShortLength(input);
                 elements.validate(databb);
                 l.add(elements.deserialize(databb));
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
index 491f1fb..4970fe6 100644
--- a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
+++ b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
@@ -558,4 +558,41 @@ public class ByteBufferUtil
     {
         return buf.capacity() > buf.remaining() ? ByteBuffer.wrap(getArray(buf)) : buf;
     }
+
+    // Doesn't change bb position
+    public static int getShortLength(ByteBuffer bb, int position)
+    {
+        int length = (bb.get(position) & 0xFF) << 8;
+        return length | (bb.get(position + 1) & 0xFF);
+    }
+
+    // changes bb position
+    public static int readShortLength(ByteBuffer bb)
+    {
+        int length = (bb.get() & 0xFF) << 8;
+        return length | (bb.get() & 0xFF);
+    }
+
+    // changes bb position
+    public static void writeShortLength(ByteBuffer bb, int length)
+    {
+        bb.put((byte) ((length >> 8) & 0xFF));
+        bb.put((byte) (length & 0xFF));
+    }
+
+    // changes bb position
+    public static ByteBuffer readBytes(ByteBuffer bb, int length)
+    {
+        ByteBuffer copy = bb.duplicate();
+        copy.limit(copy.position() + length);
+        bb.position(bb.position() + length);
+        return copy;
+    }
+
+    // changes bb position
+    public static ByteBuffer readBytesWithShortLength(ByteBuffer bb)
+    {
+        int length = readShortLength(bb);
+        return readBytes(bb, length);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9d06ea6f/test/unit/org/apache/cassandra/db/marshal/CollectionTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/CollectionTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/CollectionTypeTest.java
new file mode 100644
index 0000000..fba4742
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/marshal/CollectionTypeTest.java
@@ -0,0 +1,116 @@
+/**
+ * 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.marshal;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class CollectionTypeTest
+{
+    @Test
+    public void testListComparison()
+    {
+        ListType<String> lt = ListType.getInstance(UTF8Type.instance);
+
+        ByteBuffer[] lists = new ByteBuffer[] {
+            ByteBufferUtil.EMPTY_BYTE_BUFFER,
+            lt.decompose(ImmutableList.<String>of()),
+            lt.decompose(ImmutableList.of("aa")),
+            lt.decompose(ImmutableList.of("bb")),
+            lt.decompose(ImmutableList.of("bb", "cc")),
+            lt.decompose(ImmutableList.of("bb", "dd"))
+        };
+
+        for (int i = 0; i < lists.length; i++)
+            assertEquals(lt.compare(lists[i], lists[i]), 0);
+
+        for (int i = 0; i < lists.length-1; i++)
+        {
+            for (int j = i+1; j < lists.length; j++)
+            {
+                assertEquals(lt.compare(lists[i], lists[j]), -1);
+                assertEquals(lt.compare(lists[j], lists[i]), 1);
+            }
+        }
+    }
+
+    @Test
+    public void testSetComparison()
+    {
+        SetType<String> st = SetType.getInstance(UTF8Type.instance);
+
+        ByteBuffer[] sets = new ByteBuffer[] {
+            ByteBufferUtil.EMPTY_BYTE_BUFFER,
+            st.decompose(ImmutableSet.<String>of()),
+            st.decompose(ImmutableSet.of("aa")),
+            st.decompose(ImmutableSet.of("bb")),
+            st.decompose(ImmutableSet.of("bb", "cc")),
+            st.decompose(ImmutableSet.of("bb", "dd"))
+        };
+
+        for (int i = 0; i < sets.length; i++)
+            assertEquals(st.compare(sets[i], sets[i]), 0);
+
+        for (int i = 0; i < sets.length-1; i++)
+        {
+            for (int j = i+1; j < sets.length; j++)
+            {
+                assertEquals(st.compare(sets[i], sets[j]), -1);
+                assertEquals(st.compare(sets[j], sets[i]), 1);
+            }
+        }
+    }
+
+    @Test
+    public void testMapComparison()
+    {
+        MapType<String, String> mt = MapType.getInstance(UTF8Type.instance, UTF8Type.instance);
+
+        ByteBuffer[] maps = new ByteBuffer[] {
+            ByteBufferUtil.EMPTY_BYTE_BUFFER,
+            mt.decompose(ImmutableMap.<String, String>of()),
+            mt.decompose(ImmutableMap.of("aa", "val1")),
+            mt.decompose(ImmutableMap.of("aa", "val2")),
+            mt.decompose(ImmutableMap.of("bb", "val1")),
+            mt.decompose(ImmutableMap.of("bb", "val1", "cc", "val3")),
+            mt.decompose(ImmutableMap.of("bb", "val1", "dd", "val3")),
+            mt.decompose(ImmutableMap.of("bb", "val1", "dd", "val4"))
+        };
+
+        for (int i = 0; i < maps.length; i++)
+            assertEquals(mt.compare(maps[i], maps[i]), 0);
+
+        for (int i = 0; i < maps.length-1; i++)
+        {
+            for (int j = i+1; j < maps.length; j++)
+            {
+                assertEquals(mt.compare(maps[i], maps[j]), -1);
+                assertEquals(mt.compare(maps[j], maps[i]), 1);
+            }
+        }
+    }
+}


[2/2] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by sl...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
	src/java/org/apache/cassandra/cql3/ColumnCondition.java
	src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
	src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
	src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
	src/java/org/apache/cassandra/cql3/statements/Selection.java
	src/java/org/apache/cassandra/db/marshal/CollectionType.java
	src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
	src/java/org/apache/cassandra/db/marshal/ListType.java
	src/java/org/apache/cassandra/db/marshal/MapType.java
	src/java/org/apache/cassandra/db/marshal/SetType.java
	src/java/org/apache/cassandra/serializers/CollectionSerializer.java
	src/java/org/apache/cassandra/serializers/ListSerializer.java
	src/java/org/apache/cassandra/serializers/MapSerializer.java
	src/java/org/apache/cassandra/serializers/SetSerializer.java
	test/unit/org/apache/cassandra/db/marshal/CollectionTypeTest.java


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

Branch: refs/heads/cassandra-2.1
Commit: 33bd8c207a2b59990019c2cbfcbeceb9b6f8456e
Parents: 07dc6b5 9d06ea6
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Mon May 19 12:26:32 2014 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Mon May 19 12:26:32 2014 +0200

----------------------------------------------------------------------
 .../apache/cassandra/cql3/ColumnCondition.java  | 277 +++++++++++++------
 src/java/org/apache/cassandra/cql3/Lists.java   |  13 +
 src/java/org/apache/cassandra/cql3/Maps.java    |  20 ++
 src/java/org/apache/cassandra/cql3/Sets.java    |  15 +
 .../cql3/statements/CQL3CasConditions.java      |  10 +-
 .../cql3/statements/ModificationStatement.java  |   4 +-
 .../cql3/statements/SelectStatement.java        |  16 +-
 .../cassandra/cql3/statements/Selection.java    |  39 +--
 8 files changed, 272 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/33bd8c20/src/java/org/apache/cassandra/cql3/ColumnCondition.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/ColumnCondition.java
index c2617fe,adc8e3a..ed2b6b4
--- a/src/java/org/apache/cassandra/cql3/ColumnCondition.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
@@@ -74,122 -73,209 +75,196 @@@ public class ColumnConditio
          value.collectMarkerSpecification(boundNames);
      }
  
-     public ColumnCondition.WithOptions with(QueryOptions options)
 -    public ColumnCondition.Bound bind(List<ByteBuffer> variables) throws InvalidRequestException
++    public ColumnCondition.Bound bind(QueryOptions options) throws InvalidRequestException
      {
-         return new WithOptions(options);
+         return column.type instanceof CollectionType
 -             ? (collectionElement == null ? new CollectionBound(this, variables) : new ElementAccessBound(this, variables))
 -             : new SimpleBound(this, variables);
++             ? (collectionElement == null ? new CollectionBound(this, options) : new ElementAccessBound(this, options))
++             : new SimpleBound(this, options);
      }
  
-     public class WithOptions
+     public static abstract class Bound
      {
-         private final QueryOptions options;
 -        public final CFDefinition.Name column;
++        public final ColumnDefinition column;
  
-         private WithOptions(QueryOptions options)
 -        protected Bound(CFDefinition.Name column)
++        protected Bound(ColumnDefinition column)
          {
-             this.options = options;
+             this.column = column;
          }
  
-         public boolean equalsTo(WithOptions other) throws InvalidRequestException
+         /**
+          * Validates whether this condition applies to {@code current}.
+          */
 -        public abstract boolean appliesTo(ColumnNameBuilder rowPrefix, ColumnFamily current, long now) throws InvalidRequestException;
++        public abstract boolean appliesTo(Composite rowPrefix, ColumnFamily current, long now) throws InvalidRequestException;
+ 
+         public ByteBuffer getCollectionElementValue()
          {
-             if (!column().equals(other.column()))
-                 return false;
+             return null;
+         }
  
-             if ((collectionElement() == null) != (other.collectionElement() == null))
-                 return false;
 -        protected ColumnNameBuilder copyOrUpdatePrefix(CFMetaData cfm, ColumnNameBuilder rowPrefix)
 -        {
 -            return column.kind == CFDefinition.Name.Kind.STATIC ? cfm.getStaticColumnNameBuilder() : rowPrefix.copy();
 -        }
 -
 -        protected boolean equalsValue(ByteBuffer value, Column c, AbstractType<?> type, long now)
++        protected boolean equalsValue(ByteBuffer value, Cell c, AbstractType<?> type, long now)
+         {
+             return value == null
+                  ? c == null || !c.isLive(now)
+                  : c != null && c.isLive(now) && type.compare(c.value(), value) == 0;
+         }
  
-             if (collectionElement() != null)
 -        protected Iterator<Column> collectionColumns(ColumnNameBuilder collectionPrefix, ColumnFamily cf, final long now)
++        protected Iterator<Cell> collectionColumns(CellName collection, ColumnFamily cf, final long now)
+         {
+             // We are testing for collection equality, so we need to have the expected values *and* only those.
 -            ColumnSlice[] collectionSlice = new ColumnSlice[]{ new ColumnSlice(collectionPrefix.build(), collectionPrefix.buildAsEndOfRange()) };
++            ColumnSlice[] collectionSlice = new ColumnSlice[]{ collection.slice() };
+             // Filter live columns, this makes things simpler afterwards
 -            return Iterators.filter(cf.iterator(collectionSlice), new Predicate<Column>()
++            return Iterators.filter(cf.iterator(collectionSlice), new Predicate<Cell>()
              {
-                 assert column.type instanceof ListType || column.type instanceof MapType;
-                 AbstractType<?> comparator = column.type instanceof ListType
-                                            ? Int32Type.instance
-                                            : ((MapType)column.type).keys;
 -                public boolean apply(Column c)
++                public boolean apply(Cell c)
+                 {
+                     // we only care about live columns
+                     return c.isLive(now);
+                 }
+             });
+         }
+     }
  
-                 if (comparator.compare(collectionElement().bindAndGet(options), other.collectionElement().bindAndGet(options)) != 0)
-                     return false;
-             }
+     private static class SimpleBound extends Bound
+     {
+         public final ByteBuffer value;
  
-             return value().bindAndGet(options).equals(other.value().bindAndGet(other.options));
 -        private SimpleBound(ColumnCondition condition, List<ByteBuffer> variables) throws InvalidRequestException
++        private SimpleBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
+         {
+             super(condition.column);
+             assert !(column.type instanceof CollectionType) && condition.collectionElement == null;
 -            this.value = condition.value.bindAndGet(variables);
++            this.value = condition.value.bindAndGet(options);
          }
  
-         private ColumnDefinition column()
 -        public boolean appliesTo(ColumnNameBuilder rowPrefix, ColumnFamily current, long now) throws InvalidRequestException
++        public boolean appliesTo(Composite rowPrefix, ColumnFamily current, long now) throws InvalidRequestException
          {
-             return column;
 -            ColumnNameBuilder prefix = copyOrUpdatePrefix(current.metadata(), rowPrefix);
 -            ByteBuffer columnName = column.kind == CFDefinition.Name.Kind.VALUE_ALIAS
 -                                  ? prefix.build()
 -                                  : prefix.add(column.name.key).build();
 -
 -            return equalsValue(value, current.getColumn(columnName), column.type, now);
++            CellName name = current.metadata().comparator.create(rowPrefix, column);
++            return equalsValue(value, current.getColumn(name), column.type, now);
          }
  
-         private Term collectionElement()
+         @Override
+         public boolean equals(Object o)
          {
-             return collectionElement;
+             if (!(o instanceof SimpleBound))
+                 return false;
+ 
+             SimpleBound that = (SimpleBound)o;
+             if (!column.equals(that.column))
+                 return false;
+ 
+             return value == null || that.value == null
+                  ? value == null && that.value == null
+                  : column.type.compare(value, that.value) == 0;
          }
  
-         private Term value()
+         @Override
+         public int hashCode()
          {
-             return value;
+             return Objects.hashCode(column, value);
          }
+     }
  
-         public ByteBuffer getCollectionElementValue() throws InvalidRequestException
+     private static class ElementAccessBound extends Bound
+     {
+         public final ByteBuffer collectionElement;
+         public final ByteBuffer value;
+ 
 -        private ElementAccessBound(ColumnCondition condition, List<ByteBuffer> variables) throws InvalidRequestException
++        private ElementAccessBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
          {
-             return collectionElement == null ? null : collectionElement.bindAndGet(options);
+             super(condition.column);
+             assert column.type instanceof CollectionType && condition.collectionElement != null;
 -            this.collectionElement = condition.collectionElement.bindAndGet(variables);
 -            this.value = condition.value.bindAndGet(variables);
++            this.collectionElement = condition.collectionElement.bindAndGet(options);
++            this.value = condition.value.bindAndGet(options);
          }
  
-         /**
-          * Validates whether this condition applies to {@code current}.
-          */
-         public boolean appliesTo(Composite rowPrefix, ColumnFamily current, long now) throws InvalidRequestException
 -        public boolean appliesTo(ColumnNameBuilder rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException
++        public boolean appliesTo(Composite rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException
          {
-             if (column.type instanceof CollectionType)
-                 return collectionAppliesTo((CollectionType)column.type, rowPrefix, current, now);
+             if (collectionElement == null)
+                 throw new InvalidRequestException("Invalid null value for " + (column.type instanceof MapType ? "map" : "list") + " element access");
  
-             assert collectionElement == null;
-             Cell c = current.getColumn(current.metadata().comparator.create(rowPrefix, column));
-             ByteBuffer v = value.bindAndGet(options);
-             return v == null
-                  ? c == null || !c.isLive(now)
-                  : c != null && c.isLive(now) && c.value().equals(v);
 -            ColumnNameBuilder collectionPrefix = copyOrUpdatePrefix(current.metadata(), rowPrefix).add(column.name.key);
+             if (column.type instanceof MapType)
 -                return equalsValue(value, current.getColumn(collectionPrefix.add(collectionElement).build()), ((MapType)column.type).values, now);
++                return equalsValue(value, current.getColumn(current.metadata().comparator.create(rowPrefix, column, collectionElement)), ((MapType)column.type).values, now);
+ 
+             assert column.type instanceof ListType;
+             int idx = ByteBufferUtil.toInt(collectionElement);
+             if (idx < 0)
+                 throw new InvalidRequestException(String.format("Invalid negative list index %d", idx));
+ 
 -            Iterator<Column> iter = collectionColumns(collectionPrefix, current, now);
++            Iterator<Cell> iter = collectionColumns(current.metadata().comparator.create(rowPrefix, column), current, now);
+             int adv = Iterators.advance(iter, idx);
+             if (adv != idx || !iter.hasNext())
+                 throw new InvalidRequestException(String.format("List index %d out of bound, list has size %d", idx, adv));
+ 
+             // We don't support null values inside collections, so a condition like 'IF l[3] = null' can only
+             // be false. We do special case though, as the compare below might mind getting a null.
+             if (value == null)
+                 return false;
+ 
+             return ((ListType)column.type).elements.compare(iter.next().value(), value) == 0;
          }
  
-         private boolean collectionAppliesTo(CollectionType type, Composite rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException
+         public ByteBuffer getCollectionElementValue()
          {
-             Term.Terminal v = value.bind(options);
+             return collectionElement;
+         }
  
-             // For map element access, we won't iterate over the collection, so deal with that first. In other case, we do.
-             if (collectionElement != null && type instanceof MapType)
-             {
-                 ByteBuffer e = collectionElement.bindAndGet(options);
-                 if (e == null)
-                     throw new InvalidRequestException("Invalid null value for map access");
-                 return mapElementAppliesTo((MapType)type, current, rowPrefix, e, v.get(options), now);
-             }
+         @Override
+         public boolean equals(Object o)
+         {
+             if (!(o instanceof ElementAccessBound))
+                 return false;
  
-             CellName name = current.metadata().comparator.create(rowPrefix, column);
-             // We are testing for collection equality, so we need to have the expected values *and* only those.
-             ColumnSlice[] collectionSlice = new ColumnSlice[]{ name.slice() };
-             // Filter live columns, this makes things simpler afterwards
-             Iterator<Cell> iter = Iterators.filter(current.iterator(collectionSlice), new Predicate<Cell>()
-             {
-                 public boolean apply(Cell c)
-                 {
-                     // we only care about live columns
-                     return c.isLive(now);
-                 }
-             });
+             ElementAccessBound that = (ElementAccessBound)o;
+             if (!column.equals(that.column))
+                 return false;
  
-             if (v == null)
-                 return !iter.hasNext();
+             if ((collectionElement == null) != (that.collectionElement == null))
+                 return false;
  
              if (collectionElement != null)
              {
-                 assert type instanceof ListType;
-                 ByteBuffer e = collectionElement.bindAndGet(options);
-                 if (e == null)
-                     throw new InvalidRequestException("Invalid null value for list access");
+                 assert column.type instanceof ListType || column.type instanceof MapType;
+                 AbstractType<?> comparator = column.type instanceof ListType
+                                            ? Int32Type.instance
+                                            : ((MapType)column.type).keys;
  
-                 return listElementAppliesTo((ListType)type, iter, e, v.get(options));
+                 if (comparator.compare(collectionElement, that.collectionElement) != 0)
+                     return false;
              }
  
+             return column.type.compare(value, that.value) == 0;
+         }
+ 
+         @Override
+         public int hashCode()
+         {
+             return Objects.hashCode(column, collectionElement, value);
+         }
+     }
+ 
+     private static class CollectionBound extends Bound
+     {
+         public final Term.Terminal value;
+ 
 -        private CollectionBound(ColumnCondition condition, List<ByteBuffer> variables) throws InvalidRequestException
++        private CollectionBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
+         {
+             super(condition.column);
+             assert column.type instanceof CollectionType && condition.collectionElement == null;
 -            this.value = condition.value.bind(variables);
++            this.value = condition.value.bind(options);
+         }
+ 
 -        public boolean appliesTo(ColumnNameBuilder rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException
++        public boolean appliesTo(Composite rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException
+         {
+             CollectionType type = (CollectionType)column.type;
 -            CFMetaData cfm = current.metadata();
+ 
 -            ColumnNameBuilder collectionPrefix = copyOrUpdatePrefix(cfm, rowPrefix).add(column.name.key);
 -
 -            Iterator<Column> iter = collectionColumns(collectionPrefix, current, now);
++            Iterator<Cell> iter = collectionColumns(current.metadata().comparator.create(rowPrefix, column), current, now);
+             if (value == null)
+                 return !iter.hasNext();
+ 
              switch (type.kind)
              {
-                 case LIST: return listAppliesTo((ListType)type, iter, ((Lists.Value)v).elements);
-                 case SET: return setAppliesTo((SetType)type, iter, ((Sets.Value)v).elements);
-                 case MAP: return mapAppliesTo((MapType)type, iter, ((Maps.Value)v).map);
 -                case LIST: return listAppliesTo((ListType)type, cfm, iter, ((Lists.Value)value).elements);
 -                case SET: return setAppliesTo((SetType)type, cfm, iter, ((Sets.Value)value).elements);
 -                case MAP: return mapAppliesTo((MapType)type, cfm, iter, ((Maps.Value)value).map);
++                case LIST: return listAppliesTo((ListType)type, iter, ((Lists.Value)value).elements);
++                case SET: return setAppliesTo((SetType)type, iter, ((Sets.Value)value).elements);
++                case MAP: return mapAppliesTo((MapType)type, iter, ((Maps.Value)value).map);
              }
              throw new AssertionError();
          }
@@@ -203,20 -295,7 +278,7 @@@
              return !iter.hasNext();
          }
  
-         private boolean listElementAppliesTo(ListType type, Iterator<Cell> iter, ByteBuffer element, ByteBuffer value) throws InvalidRequestException
-         {
-             int idx = ByteBufferUtil.toInt(element);
-             if (idx < 0)
-                 throw new InvalidRequestException(String.format("Invalid negative list index %d", idx));
- 
-             int adv = Iterators.advance(iter, idx);
-             if (adv != idx || !iter.hasNext())
-                 throw new InvalidRequestException(String.format("List index %d out of bound, list has size %d", idx, adv));
- 
-             return type.elements.compare(iter.next().value(), value) == 0;
-         }
- 
 -        private boolean setAppliesTo(SetType type, CFMetaData cfm, Iterator<Column> iter, Set<ByteBuffer> elements)
 +        private boolean setAppliesTo(SetType type, Iterator<Cell> iter, Set<ByteBuffer> elements)
          {
              Set<ByteBuffer> remaining = new TreeSet<>(type.elements);
              remaining.addAll(elements);
@@@ -248,11 -327,31 +310,48 @@@
              return remaining.isEmpty();
          }
  
-         private boolean mapElementAppliesTo(MapType type, ColumnFamily current, Composite rowPrefix, ByteBuffer element, ByteBuffer value, long now)
+         @Override
+         public boolean equals(Object o)
          {
-             CellName name = current.getComparator().create(rowPrefix, column, element);
-             Cell c = current.getColumn(name);
-             return c != null && c.isLive(now) && type.values.compare(c.value(), value) == 0;
+             if (!(o instanceof CollectionBound))
+                 return false;
+ 
+             CollectionBound that = (CollectionBound)o;
+             if (!column.equals(that.column))
+                 return false;
+ 
 -            // Slightly inefficient because it serialize the collection just for the sake of comparison.
 -            // We could improve by adding an equals() method to Lists.Value, Sets.Value and Maps.Value but
 -            // this method is only called when there is 2 conditions on the same collection to make sure
 -            // both are not incompatible, so overall it's probably not worth the effort.
 -            ByteBuffer thisVal = value.get();
 -            ByteBuffer thatVal = that.value.get();
 -            return thisVal == null || thatVal == null
 -                 ? thisVal == null && thatVal == null
 -                 : column.type.compare(thisVal, thatVal) == 0;
++            if (value == null || that.value == null)
++                return value == null && that.value == null;
++
++            switch (((CollectionType)column.type).kind)
++            {
++                case LIST: return ((Lists.Value)value).equals((ListType)column.type, (Lists.Value)that.value);
++                case SET: return ((Sets.Value)value).equals((SetType)column.type, (Sets.Value)that.value);
++                case MAP: return ((Maps.Value)value).equals((MapType)column.type, (Maps.Value)that.value);
++            }
++            throw new AssertionError();
+         }
+ 
+         @Override
+         public int hashCode()
+         {
 -            return Objects.hashCode(column, value.get());
++            Object val = null;
++            if (value != null)
++            {
++                switch (((CollectionType)column.type).kind)
++                {
++                    case LIST:
++                        val = ((Lists.Value)value).elements.hashCode();
++                        break;
++                    case SET:
++                        val = ((Sets.Value)value).elements.hashCode();
++                        break;
++                    case MAP:
++                        val = ((Maps.Value)value).map.hashCode();
++                        break;
++                }
++            }
++            return Objects.hashCode(column, val);
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33bd8c20/src/java/org/apache/cassandra/cql3/Lists.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Lists.java
index 751ccdb,4ad39db..f12af88
--- a/src/java/org/apache/cassandra/cql3/Lists.java
+++ b/src/java/org/apache/cassandra/cql3/Lists.java
@@@ -20,6 -20,6 +20,7 @@@ package org.apache.cassandra.cql3
  import java.nio.ByteBuffer;
  import java.util.ArrayList;
  import java.util.Collections;
++import java.util.Iterator;
  import java.util.List;
  import java.util.concurrent.atomic.AtomicReference;
  
@@@ -146,10 -144,10 +147,22 @@@ public abstract class List
              }
          }
  
 -        public ByteBuffer get()
 +        public ByteBuffer get(QueryOptions options)
          {
 -            return CollectionType.pack(elements, elements.size());
 +            return CollectionSerializer.pack(elements, elements.size(), options.getProtocolVersion());
 +        }
++
++        public boolean equals(ListType lt, Value v)
++        {
++            if (elements.size() != v.elements.size())
++                return false;
++
++            for (int i = 0; i < elements.size(); i++)
++                if (lt.elements.compare(elements.get(i), v.elements.get(i)) != 0)
++                    return false;
++
++            return true;
+         }
      }
  
      /*

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33bd8c20/src/java/org/apache/cassandra/cql3/Maps.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Maps.java
index 0c4980c,c332999..e6beb7e
--- a/src/java/org/apache/cassandra/cql3/Maps.java
+++ b/src/java/org/apache/cassandra/cql3/Maps.java
@@@ -22,6 -22,6 +22,7 @@@ import java.util.ArrayList
  import java.util.Collections;
  import java.util.Comparator;
  import java.util.HashMap;
++import java.util.Iterator;
  import java.util.LinkedHashMap;
  import java.util.List;
  import java.util.Map;
@@@ -168,8 -165,8 +169,27 @@@ public abstract class Map
                  buffers.add(entry.getKey());
                  buffers.add(entry.getValue());
              }
 -            return CollectionType.pack(buffers, map.size());
 +            return CollectionSerializer.pack(buffers, map.size(), options.getProtocolVersion());
 +        }
++
++        public boolean equals(MapType mt, Value v)
++        {
++            if (map.size() != v.map.size())
++                return false;
++
++            // We use the fact that we know the maps iteration will both be in comparator order
++            Iterator<Map.Entry<ByteBuffer, ByteBuffer>> thisIter = map.entrySet().iterator();
++            Iterator<Map.Entry<ByteBuffer, ByteBuffer>> thatIter = v.map.entrySet().iterator();
++            while (thisIter.hasNext())
++            {
++                Map.Entry<ByteBuffer, ByteBuffer> thisEntry = thisIter.next();
++                Map.Entry<ByteBuffer, ByteBuffer> thatEntry = thatIter.next();
++                if (mt.keys.compare(thisEntry.getKey(), thatEntry.getKey()) != 0 || mt.values.compare(thisEntry.getValue(), thatEntry.getValue()) != 0)
++                    return false;
++            }
++
++            return true;
+         }
      }
  
      // See Lists.DelayedValue

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33bd8c20/src/java/org/apache/cassandra/cql3/Sets.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Sets.java
index 92a3510,69bc3d3..1acaacb
--- a/src/java/org/apache/cassandra/cql3/Sets.java
+++ b/src/java/org/apache/cassandra/cql3/Sets.java
@@@ -22,6 -22,6 +22,7 @@@ import java.util.ArrayList
  import java.util.Collections;
  import java.util.Comparator;
  import java.util.HashSet;
++import java.util.Iterator;
  import java.util.LinkedHashSet;
  import java.util.List;
  import java.util.Set;
@@@ -158,10 -155,10 +159,24 @@@ public abstract class Set
              }
          }
  
 -        public ByteBuffer get()
 +        public ByteBuffer get(QueryOptions options)
          {
 -            return CollectionType.pack(new ArrayList<ByteBuffer>(elements), elements.size());
 +            return CollectionSerializer.pack(new ArrayList<ByteBuffer>(elements), elements.size(), options.getProtocolVersion());
 +        }
++
++        public boolean equals(SetType st, Value v)
++        {
++            if (elements.size() != v.elements.size())
++                return false;
++
++            Iterator<ByteBuffer> thisIter = elements.iterator();
++            Iterator<ByteBuffer> thatIter = v.elements.iterator();
++            while (thisIter.hasNext())
++                if (st.elements.compare(thisIter.next(), thatIter.next()) != 0)
++                    return false;
++
++            return true;
+         }
      }
  
      // See Lists.DelayedValue

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33bd8c20/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
index 5005d2f,775a236..b06b2ee
--- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
@@@ -167,9 -167,9 +167,9 @@@ public class CQL3CasConditions implemen
  
      private static class ColumnsConditions extends RowCondition
      {
-         private final Map<Pair<ColumnIdentifier, ByteBuffer>, ColumnCondition.WithOptions> conditions = new HashMap<>();
+         private final Map<Pair<ColumnIdentifier, ByteBuffer>, ColumnCondition.Bound> conditions = new HashMap<>();
  
 -        private ColumnsConditions(ColumnNameBuilder rowPrefix, long now)
 +        private ColumnsConditions(Composite rowPrefix, long now)
          {
              super(rowPrefix, now);
          }
@@@ -180,10 -180,10 +180,10 @@@
              {
                  // We will need the variables in appliesTo but with protocol batches, each condition in this object can have a
                  // different list of variables.
-                 ColumnCondition.WithOptions current = condition.with(options);
-                 ColumnCondition.WithOptions previous = conditions.put(Pair.create(condition.column.name, current.getCollectionElementValue()), current);
 -                ColumnCondition.Bound current = condition.bind(variables);
++                ColumnCondition.Bound current = condition.bind(options);
+                 ColumnCondition.Bound previous = conditions.put(Pair.create(condition.column.name, current.getCollectionElementValue()), current);
                  // If 2 conditions are actually equal, let it slide
-                 if (previous != null && !previous.equalsTo(current))
+                 if (previous != null && !previous.equals(current))
                      throw new InvalidRequestException("Duplicate and incompatible conditions for column " + condition.column.name);
              }
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33bd8c20/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 7f8b678,448722e..03d4264
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@@ -599,7 -649,9 +599,9 @@@ public abstract class ModificationState
          }
          else
          {
-             List<ColumnDefinition> defs = new ArrayList<>();
+             // We can have multiple conditions on the same columns (for collections) so use a set
+             // to avoid duplicate, but preserve the order just to it follows the order of IF in the query in general
 -            Set<CFDefinition.Name> names = new LinkedHashSet<CFDefinition.Name>();
++            Set<ColumnDefinition> defs = new LinkedHashSet<>();
              // Adding the partition key for batches to disambiguate if the conditions span multipe rows (we don't add them outside
              // of batches for compatibility sakes).
              if (isBatch)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33bd8c20/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index b9ccd1a,2468eb9..55ce6f9
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -126,11 -128,11 +126,11 @@@ public class SelectStatement implement
          }
  
          // Otherwise, check the selected columns
-         selectsStaticColumns = !Iterables.isEmpty(Iterables.filter(selection.getColumnsList(), isStaticFilter));
+         selectsStaticColumns = !Iterables.isEmpty(Iterables.filter(selection.getColumns(), isStaticFilter));
          selectsOnlyStaticColumns = true;
-         for (ColumnDefinition def : selection.getColumnsList())
 -        for (CFDefinition.Name name : selection.getColumns())
++        for (ColumnDefinition def : selection.getColumns())
          {
 -            if (name.kind != CFDefinition.Name.Kind.KEY_ALIAS && name.kind != CFDefinition.Name.Kind.STATIC)
 +            if (def.kind != ColumnDefinition.Kind.PARTITION_KEY && def.kind != ColumnDefinition.Kind.STATIC)
              {
                  selectsOnlyStaticColumns = false;
                  break;
@@@ -733,15 -744,14 +733,15 @@@
              // column (for the case where the row exists but has no columns outside the PK)
              // Two exceptions are "static CF" (non-composite non-compact CF) and "super CF"
              // that don't have marker and for which we must query all columns instead
 -            if (cfDef.isComposite && !cfDef.cfm.isSuper())
 +            if (cfm.comparator.isCompound() && !cfm.isSuper())
              {
                  // marker
 -                columns.add(builder.copy().add(ByteBufferUtil.EMPTY_BYTE_BUFFER).build());
 +                columns.add(cfm.comparator.rowMarker(prefix));
  
                  // selected columns
-                 for (ColumnDefinition def : selection.getColumnsList())
 -                for (ColumnIdentifier id : selection.regularAndStaticColumnsToFetch())
 -                    columns.add(builder.copy().add(id.key).build());
++                for (ColumnDefinition def : selection.getColumns())
 +                    if (def.kind == ColumnDefinition.Kind.REGULAR || def.kind == ColumnDefinition.Kind.STATIC)
 +                        columns.add(cfm.comparator.create(prefix, def));
              }
              else
              {
@@@ -755,12 -771,12 +755,12 @@@
  
      private boolean selectACollection()
      {
 -        if (!cfDef.hasCollections)
 +        if (!cfm.comparator.hasCollections())
              return false;
  
-         for (ColumnDefinition def : selection.getColumnsList())
 -        for (CFDefinition.Name name : selection.getColumns())
++        for (ColumnDefinition def : selection.getColumns())
          {
 -            if (name.type instanceof CollectionType)
 +            if (def.type instanceof CollectionType)
                  return true;
          }
  
@@@ -1024,77 -975,122 +1024,77 @@@
      }
  
      // Used by ModificationStatement for CAS operations
 -    void processColumnFamily(ByteBuffer key, ColumnFamily cf, List<ByteBuffer> variables, long now, Selection.ResultSetBuilder result)
 +    void processColumnFamily(ByteBuffer key, ColumnFamily cf, QueryOptions options, long now, Selection.ResultSetBuilder result)
      throws InvalidRequestException
      {
 -        ByteBuffer[] keyComponents = cfDef.hasCompositeKey
 -                                   ? ((CompositeType)cfDef.cfm.getKeyValidator()).split(key)
 -                                   : new ByteBuffer[]{ key };
 -
 -        if (parameters.isDistinct)
 +        CFMetaData cfm = cf.metadata();
 +        ByteBuffer[] keyComponents = null;
 +        if (cfm.getKeyValidator() instanceof CompositeType)
          {
 -            if (!cf.hasOnlyTombstones(now))
 -            {
 -                result.newRow();
 -                // selection.getColumns() will contain only the partition key components - all of them.
 -                for (CFDefinition.Name name : selection.getColumns())
 -                    result.add(keyComponents[name.position]);
 -            }
 +            keyComponents = ((CompositeType)cfm.getKeyValidator()).split(key);
          }
 -        else if (cfDef.isCompact)
 +        else
          {
 -            // One cqlRow per column
 -            for (Column c : cf)
 -            {
 -                if (c.isMarkedForDelete(now))
 -                    continue;
 -
 -                ByteBuffer[] components = null;
 -                if (cfDef.isComposite)
 -                {
 -                    components = ((CompositeType)cfDef.cfm.comparator).split(c.name());
 -                }
 -                else if (sliceRestriction != null)
 -                {
 -                    Comparator<ByteBuffer> comp = cfDef.cfm.comparator;
 -                    // For dynamic CF, the column could be out of the requested bounds, filter here
 -                    if (!sliceRestriction.isInclusive(Bound.START) && comp.compare(c.name(), sliceRestriction.bound(Bound.START, variables)) == 0)
 -                        continue;
 -                    if (!sliceRestriction.isInclusive(Bound.END) && comp.compare(c.name(), sliceRestriction.bound(Bound.END, variables)) == 0)
 -                        continue;
 -                }
 -
 -                result.newRow();
 -                // Respect selection order
 -                for (CFDefinition.Name name : selection.getColumns())
 -                {
 -                    switch (name.kind)
 -                    {
 -                        case KEY_ALIAS:
 -                            result.add(keyComponents[name.position]);
 -                            break;
 -                        case COLUMN_ALIAS:
 -                            ByteBuffer val = cfDef.isComposite
 -                                           ? (name.position < components.length ? components[name.position] : null)
 -                                           : c.name();
 -                            result.add(val);
 -                            break;
 -                        case VALUE_ALIAS:
 -                            result.add(c);
 -                            break;
 -                        case COLUMN_METADATA:
 -                        case STATIC:
 -                            // This should not happen for compact CF
 -                            throw new AssertionError();
 -                        default:
 -                            throw new AssertionError();
 -                    }
 -                }
 -            }
 +            keyComponents = new ByteBuffer[]{ key };
          }
 -        else if (cfDef.isComposite)
 -        {
 -            // Sparse case: group column in cqlRow when composite prefix is equal
 -            CompositeType composite = (CompositeType)cfDef.cfm.comparator;
  
 -            ColumnGroupMap.Builder builder = new ColumnGroupMap.Builder(composite, cfDef.hasCollections, now);
 +        Iterator<Cell> cells = cf.getSortedColumns().iterator();
 +        if (sliceRestriction != null)
 +            cells = applySliceRestriction(cells, options);
  
 -            for (Column c : cf)
 -            {
 -                if (c.isMarkedForDelete(now))
 -                    continue;
 +        CQL3Row.RowIterator iter = cfm.comparator.CQL3RowBuilder(cfm, now).group(cells);
  
 -                builder.add(c);
 -            }
 -
 -            ColumnGroupMap staticGroup = null;
 -            // Gather up static values first
 -            if (!builder.isEmpty() && builder.firstGroup().isStatic)
 +        // If there is static columns but there is no non-static row, then provided the select was a full
 +        // partition selection (i.e. not a 2ndary index search and there was no condition on clustering columns)
 +        // then we want to include the static columns in the result set (and we're done).
 +        CQL3Row staticRow = iter.getStaticRow();
 +        if (staticRow != null && !iter.hasNext() && !usesSecondaryIndexing && hasNoClusteringColumnsRestriction())
 +        {
 +            result.newRow();
-             for (ColumnDefinition def : selection.getColumnsList())
++            for (ColumnDefinition def : selection.getColumns())
              {
 -                staticGroup = builder.firstGroup();
 -                builder.discardFirst();
 -
 -                // If there was static columns but there is no actual row, then provided the select was a full
 -                // partition selection (i.e. not a 2ndary index search and there was no condition on clustering columns)
 -                // then we want to include the static columns in the result set.
 -                if (builder.isEmpty() && !usesSecondaryIndexing && hasNoClusteringColumnsRestriction() && hasValueForQuery(staticGroup))
 +                switch (def.kind)
                  {
 -                    handleGroup(result, keyComponents, ColumnGroupMap.EMPTY, staticGroup);
 -                    return;
 +                    case PARTITION_KEY:
 +                        result.add(keyComponents[def.position()]);
 +                        break;
 +                    case STATIC:
 +                        addValue(result, def, staticRow, options);
 +                        break;
 +                    default:
 +                        result.add((ByteBuffer)null);
                  }
              }
 -
 -            for (ColumnGroupMap group : builder.groups())
 -                handleGroup(result, keyComponents, group, staticGroup);
 +            return;
          }
 -        else
 +
 +        while (iter.hasNext())
          {
 -            if (cf.hasOnlyTombstones(now))
 -                return;
 +            CQL3Row cql3Row = iter.next();
  
 -            // Static case: One cqlRow for all columns
 +            // Respect requested order
              result.newRow();
 -            for (CFDefinition.Name name : selection.getColumns())
 +            // Respect selection order
-             for (ColumnDefinition def : selection.getColumnsList())
++            for (ColumnDefinition def : selection.getColumns())
              {
 -                if (name.kind == CFDefinition.Name.Kind.KEY_ALIAS)
 -                    result.add(keyComponents[name.position]);
 -                else
 -                    result.add(cf.getColumn(name.name.key));
 +                switch (def.kind)
 +                {
 +                    case PARTITION_KEY:
 +                        result.add(keyComponents[def.position()]);
 +                        break;
 +                    case CLUSTERING_COLUMN:
 +                        result.add(cql3Row.getClusteringColumn(def.position()));
 +                        break;
 +                    case COMPACT_VALUE:
 +                        result.add(cql3Row.getColumn(null));
 +                        break;
 +                    case REGULAR:
 +                        addValue(result, def, cql3Row, options);
 +                        break;
 +                    case STATIC:
 +                        addValue(result, def, staticRow, options);
 +                        break;
 +                }
              }
          }
      }
@@@ -1210,11 -1254,11 +1210,11 @@@
                  throw new InvalidRequestException("Only COUNT(*) and COUNT(1) operations are currently supported.");
  
              Selection selection = selectClause.isEmpty()
 -                                ? Selection.wildcard(cfDef)
 -                                : Selection.fromSelectors(cfDef, selectClause);
 +                                ? Selection.wildcard(cfm)
 +                                : Selection.fromSelectors(cfm, selectClause);
  
              if (parameters.isDistinct)
-                 validateDistinctSelection(selection.getColumnsList(), cfm.partitionKeyColumns());
 -                validateDistinctSelection(selection.getColumns(), cfDef.partitionKeys());
++                validateDistinctSelection(selection.getColumns(), cfm.partitionKeyColumns());
  
              Term prepLimit = null;
              if (limit != null)
@@@ -1537,32 -1606,16 +1537,32 @@@
              return new ParsedStatement.Prepared(stmt, names);
          }
  
 -        private void validateDistinctSelection(Collection<CFDefinition.Name> requestedColumns, Collection<CFDefinition.Name> partitionKey)
 +        private int indexOf(ColumnDefinition def, Selection selection)
 +        {
-             return indexOf(def, selection.getColumnsList().iterator());
++            return indexOf(def, selection.getColumns().iterator());
 +        }
 +
 +        private int indexOf(final ColumnDefinition def, Iterator<ColumnDefinition> defs)
 +        {
 +            return Iterators.indexOf(defs, new Predicate<ColumnDefinition>()
 +                                           {
 +                                               public boolean apply(ColumnDefinition n)
 +                                               {
 +                                                   return def.name.equals(n.name);
 +                                               }
 +                                           });
 +        }
 +
 +        private void validateDistinctSelection(Collection<ColumnDefinition> requestedColumns, Collection<ColumnDefinition> partitionKey)
          throws InvalidRequestException
          {
 -            for (CFDefinition.Name name : requestedColumns)
 -                if (!partitionKey.contains(name))
 -                    throw new InvalidRequestException(String.format("SELECT DISTINCT queries must only request partition key columns (not %s)", name));
 +            for (ColumnDefinition def : requestedColumns)
 +                if (!partitionKey.contains(def))
 +                    throw new InvalidRequestException(String.format("SELECT DISTINCT queries must only request partition key columns (not %s)", def.name));
  
 -            for (CFDefinition.Name name : partitionKey)
 -                if (!requestedColumns.contains(name))
 -                    throw new InvalidRequestException(String.format("SELECT DISTINCT queries must request all the partition key columns (missing %s)", name));
 +            for (ColumnDefinition def : partitionKey)
 +                if (!requestedColumns.contains(def))
 +                    throw new InvalidRequestException(String.format("SELECT DISTINCT queries must request all the partition key columns (missing %s)", def.name));
          }
  
          private boolean containsAlias(final ColumnIdentifier name)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33bd8c20/src/java/org/apache/cassandra/cql3/statements/Selection.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/Selection.java
index af1e621,123ddc3..3769e97
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@@ -19,10 -19,9 +19,11 @@@ package org.apache.cassandra.cql3.state
  
  import java.nio.ByteBuffer;
  import java.util.ArrayList;
+ import java.util.Collection;
  import java.util.List;
  
 +import com.google.common.collect.Iterators;
 +
  import org.apache.cassandra.cql3.*;
  import org.apache.cassandra.cql3.functions.Function;
  import org.apache.cassandra.cql3.functions.Functions;
@@@ -42,15 -37,15 +43,15 @@@ import org.apache.cassandra.utils.ByteB
  
  public abstract class Selection
  {
-     private final List<ColumnDefinition> columnsList;
 -    private final Collection<CFDefinition.Name> columns;
 -    private final List<ColumnSpecification> metadata;
++    private final Collection<ColumnDefinition> columns;
 +    private final ResultSet.Metadata metadata;
      private final boolean collectTimestamps;
      private final boolean collectTTLs;
  
-     protected Selection(List<ColumnDefinition> columnsList, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs)
 -    protected Selection(Collection<CFDefinition.Name> columns, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs)
++    protected Selection(Collection<ColumnDefinition> columns, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs)
      {
-         this.columnsList = columnsList;
+         this.columns = columns;
 -        this.metadata = metadata;
 +        this.metadata = new ResultSet.Metadata(metadata);
          this.collectTimestamps = collectTimestamps;
          this.collectTTLs = collectTTLs;
      }
@@@ -73,18 -69,11 +74,18 @@@
          return new SimpleSelection(all, true);
      }
  
-     public static Selection forColumns(List<ColumnDefinition> columnsList)
 -    public static Selection forColumns(Collection<CFDefinition.Name> columns)
++    public static Selection forColumns(Collection<ColumnDefinition> columns)
      {
-         return new SimpleSelection(columnsList, false);
+         return new SimpleSelection(columns, false);
      }
  
 +    public int addColumnForOrdering(ColumnDefinition c)
 +    {
-         columnsList.add(c);
++        columns.add(c);
 +        metadata.addNonSerializedColumn(c);
-         return columnsList.size() - 1;
++        return columns.size() - 1;
 +    }
 +
      private static boolean isUsingFunction(List<RawSelector> rawSelectors)
      {
          for (RawSelector rawSelector : rawSelectors)
@@@ -248,11 -209,25 +249,11 @@@
      protected abstract List<ByteBuffer> handleRow(ResultSetBuilder rs) throws InvalidRequestException;
  
      /**
 -     * @return the list of CQL3 "regular" (the "COLUMN_METADATA" ones) column names to fetch.
 -     */
 -    public List<ColumnIdentifier> regularAndStaticColumnsToFetch()
 -    {
 -        List<ColumnIdentifier> toFetch = new ArrayList<ColumnIdentifier>();
 -        for (CFDefinition.Name name : columns)
 -        {
 -            if (name.kind == CFDefinition.Name.Kind.COLUMN_METADATA || name.kind == CFDefinition.Name.Kind.STATIC)
 -                toFetch.add(name.name);
 -        }
 -        return toFetch;
 -    }
 -
 -    /**
       * @return the list of CQL3 columns value this SelectionClause needs.
       */
-     public List<ColumnDefinition> getColumnsList()
 -    public Collection<CFDefinition.Name> getColumns()
++    public Collection<ColumnDefinition> getColumns()
      {
-         return columnsList;
+         return columns;
      }
  
      public ResultSetBuilder resultSetBuilder(long now)
@@@ -286,9 -261,9 +287,9 @@@
  
          private ResultSetBuilder(long now)
          {
 -            this.resultSet = new ResultSet(metadata);
 +            this.resultSet = new ResultSet(getResultMetadata(), new ArrayList<List<ByteBuffer>>());
-             this.timestamps = collectTimestamps ? new long[columnsList.size()] : null;
-             this.ttls = collectTTLs ? new int[columnsList.size()] : null;
+             this.timestamps = collectTimestamps ? new long[columns.size()] : null;
+             this.ttls = collectTTLs ? new int[columns.size()] : null;
              this.now = now;
          }
  
@@@ -341,12 -316,12 +342,12 @@@
      {
          private final boolean isWildcard;
  
-         public SimpleSelection(List<ColumnDefinition> columnsList, boolean isWildcard)
 -        public SimpleSelection(Collection<CFDefinition.Name> columns, boolean isWildcard)
++        public SimpleSelection(Collection<ColumnDefinition> columns, boolean isWildcard)
          {
-             this(columnsList, new ArrayList<ColumnSpecification>(columnsList), isWildcard);
+             this(columns, new ArrayList<ColumnSpecification>(columns), isWildcard);
          }
  
-         public SimpleSelection(List<ColumnDefinition> columnsList, List<ColumnSpecification> metadata, boolean isWildcard)
 -        public SimpleSelection(Collection<CFDefinition.Name> columns, List<ColumnSpecification> metadata, boolean isWildcard)
++        public SimpleSelection(Collection<ColumnDefinition> columns, List<ColumnSpecification> metadata, boolean isWildcard)
          {
              /*
               * In theory, even a simple selection could have multiple time the same column, so we
@@@ -522,9 -460,9 +523,9 @@@
      {
          private final List<Selector> selectors;
  
-         public SelectionWithFunctions(List<ColumnDefinition> columnsList, List<ColumnSpecification> metadata, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
 -        public SelectionWithFunctions(Collection<CFDefinition.Name> columns, List<ColumnSpecification> metadata, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
++        public SelectionWithFunctions(Collection<ColumnDefinition> columns, List<ColumnSpecification> metadata, List<Selector> selectors, boolean collectTimestamps, boolean collectTTLs)
          {
-             super(columnsList, metadata, collectTimestamps, collectTTLs);
+             super(columns, metadata, collectTimestamps, collectTTLs);
              this.selectors = selectors;
          }