You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ty...@apache.org on 2014/12/02 20:09:05 UTC

[1/5] cassandra git commit: Refactor SelectStatement and Restrictions

Repository: cassandra
Updated Branches:
  refs/heads/trunk a604b14bf -> 65a7088e7


http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java b/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
deleted file mode 100644
index b6ca640..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
+++ /dev/null
@@ -1,486 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3.statements;
-
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public abstract class SingleColumnRestriction implements Restriction
-{
-    public boolean isMultiColumn()
-    {
-        return false;
-    }
-
-    public static class EQ extends SingleColumnRestriction implements Restriction.EQ
-    {
-        protected final Term value;
-        private final boolean onToken;
-
-        public EQ(Term value, boolean onToken)
-        {
-            this.value = value;
-            this.onToken = onToken;
-        }
-
-        public boolean usesFunction(String ksName, String functionName)
-        {
-            return value != null && value.usesFunction(ksName, functionName);
-        }
-
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-        {
-            return Collections.singletonList(value.bindAndGet(options));
-        }
-
-        public boolean isSlice()
-        {
-            return false;
-        }
-
-        public boolean isEQ()
-        {
-            return true;
-        }
-
-        public boolean isIN()
-        {
-            return false;
-        }
-
-        public boolean isContains()
-        {
-            return false;
-        }
-
-        public boolean isOnToken()
-        {
-            return onToken;
-        }
-
-        public boolean canEvaluateWithSlices()
-        {
-            return true;
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("EQ(%s)%s", value, onToken ? "*" : "");
-        }
-    }
-
-    public static class InWithValues extends SingleColumnRestriction implements Restriction.IN
-    {
-        protected final List<Term> values;
-
-        public InWithValues(List<Term> values)
-        {
-            this.values = values;
-        }
-
-        public boolean usesFunction(String ksName, String functionName)
-        {
-            if (values != null)
-                for (Term value : values)
-                    if (value != null && value.usesFunction(ksName, functionName))
-                        return true;
-            return false;
-        }
-
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-        {
-            List<ByteBuffer> buffers = new ArrayList<>(values.size());
-            for (Term value : values)
-                buffers.add(value.bindAndGet(options));
-            return buffers;
-        }
-
-        public boolean canHaveOnlyOneValue()
-        {
-            return values.size() == 1;
-        }
-
-        public boolean isSlice()
-        {
-            return false;
-        }
-
-        public boolean isEQ()
-        {
-            return false;
-        }
-
-        public boolean isIN()
-        {
-            return true;
-        }
-
-        public boolean isContains()
-        {
-            return false;
-        }
-
-        public boolean isOnToken()
-        {
-            return false;
-        }
-
-        public boolean canEvaluateWithSlices()
-        {
-            return true;
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("IN(%s)", values);
-        }
-    }
-
-    public static class InWithMarker extends SingleColumnRestriction implements Restriction.IN
-    {
-        protected final AbstractMarker marker;
-
-        public InWithMarker(AbstractMarker marker)
-        {
-            this.marker = marker;
-        }
-
-        public boolean usesFunction(String ksName, String functionName)
-        {
-            return false;
-        }
-
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-        {
-            Term.MultiItemTerminal lval = (Term.MultiItemTerminal)marker.bind(options);
-            if (lval == null)
-                throw new InvalidRequestException("Invalid null value for IN restriction");
-            return lval.getElements();
-        }
-
-        public boolean canHaveOnlyOneValue()
-        {
-            return false;
-        }
-
-        public boolean isSlice()
-        {
-            return false;
-        }
-
-        public boolean isEQ()
-        {
-            return false;
-        }
-
-        public boolean isIN()
-        {
-            return true;
-        }
-
-        public boolean isContains()
-        {
-            return false;
-        }
-
-        public boolean isOnToken()
-        {
-            return false;
-        }
-
-        public boolean canEvaluateWithSlices()
-        {
-            return true;
-        }
-
-        @Override
-        public String toString()
-        {
-            return "IN ?";
-        }
-    }
-
-    public static class Slice extends SingleColumnRestriction implements Restriction.Slice
-    {
-        protected final Term[] bounds;
-        protected final boolean[] boundInclusive;
-        protected final boolean onToken;
-
-        public Slice(boolean onToken)
-        {
-            this.bounds = new Term[2];
-            this.boundInclusive = new boolean[2];
-            this.onToken = onToken;
-        }
-
-        public boolean usesFunction(String ksName, String functionName)
-        {
-            for (Term value : bounds)
-                if (value != null && value.usesFunction(ksName, functionName))
-                    return true;
-            return false;
-        }
-
-        public boolean isSlice()
-        {
-            return true;
-        }
-
-        public boolean isEQ()
-        {
-            return false;
-        }
-
-        public boolean isIN()
-        {
-            return false;
-        }
-
-        public boolean isContains()
-        {
-            return false;
-        }
-
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        public boolean isOnToken()
-        {
-            return onToken;
-        }
-
-        public boolean canEvaluateWithSlices()
-        {
-            return true;
-        }
-
-        /** Returns true if the start or end bound (depending on the argument) is set, false otherwise */
-        public boolean hasBound(Bound b)
-        {
-            return bounds[b.idx] != null;
-        }
-
-        public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException
-        {
-            return bounds[b.idx].bindAndGet(options);
-        }
-
-        /** Returns true if the start or end bound (depending on the argument) is inclusive, false otherwise */
-        public boolean isInclusive(Bound b)
-        {
-            return bounds[b.idx] == null || boundInclusive[b.idx];
-        }
-
-        public Operator getRelation(Bound eocBound, Bound inclusiveBound)
-        {
-            switch (eocBound)
-            {
-                case START:
-                    return boundInclusive[inclusiveBound.idx] ? Operator.GTE : Operator.GT;
-                case END:
-                    return boundInclusive[inclusiveBound.idx] ? Operator.LTE : Operator.LT;
-            }
-            throw new AssertionError();
-        }
-
-        public Operator getIndexOperator(Bound b)
-        {
-            switch (b)
-            {
-                case START:
-                    return boundInclusive[b.idx] ? Operator.GTE : Operator.GT;
-                case END:
-                    return boundInclusive[b.idx] ? Operator.LTE : Operator.LT;
-            }
-            throw new AssertionError();
-        }
-
-        public void setBound(ColumnIdentifier name, Operator operator, Term t) throws InvalidRequestException
-        {
-            Bound b;
-            boolean inclusive;
-            switch (operator)
-            {
-                case GT:
-                    b = Bound.START;
-                    inclusive = false;
-                    break;
-                case GTE:
-                    b = Bound.START;
-                    inclusive = true;
-                    break;
-                case LT:
-                    b = Bound.END;
-                    inclusive = false;
-                    break;
-                case LTE:
-                    b = Bound.END;
-                    inclusive = true;
-                    break;
-                default:
-                    throw new AssertionError();
-            }
-
-            if (bounds[b.idx] != null)
-                throw new InvalidRequestException(String.format(
-                        "More than one restriction was found for the %s bound on %s", b.name().toLowerCase(), name));
-
-            bounds[b.idx] = t;
-            boundInclusive[b.idx] = inclusive;
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("SLICE(%s %s, %s %s)%s", boundInclusive[0] ? ">=" : ">",
-                                 bounds[0],
-                                 boundInclusive[1] ? "<=" : "<",
-                                 bounds[1],
-                                 onToken ? "*" : "");
-        }
-    }
-
-    // This holds both CONTAINS and CONTAINS_KEY restriction because we might want to have both of them.
-    public static class Contains extends SingleColumnRestriction
-    {
-        private List<Term> values; // for CONTAINS
-        private List<Term> keys;   // for CONTAINS_KEY
-
-        public boolean usesFunction(String ksName, String functionName)
-        {
-            if (values != null)
-                for (Term value : values)
-                    if (value != null && value.usesFunction(ksName, functionName))
-                        return true;
-            if (keys != null)
-                for (Term key : keys)
-                    if (key != null && key.usesFunction(ksName, functionName))
-                        return true;
-            return false;
-        }
-
-        public boolean hasContains()
-        {
-            return values != null;
-        }
-
-        public boolean hasContainsKey()
-        {
-            return keys != null;
-        }
-
-        public int numberOfValues()
-        {
-            return values == null ? 0 : values.size();
-        }
-
-        public int numberOfKeys()
-        {
-            return keys == null ? 0 : keys.size();
-        }
-
-        public void add(Term t, boolean isKey)
-        {
-            if (isKey)
-                addKey(t);
-            else
-                addValue(t);
-        }
-
-        public void addValue(Term t)
-        {
-            if (values == null)
-                values = new ArrayList<>();
-            values.add(t);
-        }
-
-        public void addKey(Term t)
-        {
-            if (keys == null)
-                keys = new ArrayList<>();
-            keys.add(t);
-        }
-
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-        {
-            if (values == null)
-                return Collections.emptyList();
-
-            List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(values.size());
-            for (Term value : values)
-                buffers.add(value.bindAndGet(options));
-            return buffers;
-        }
-
-        public List<ByteBuffer> keys(QueryOptions options) throws InvalidRequestException
-        {
-            if (keys == null)
-                return Collections.emptyList();
-
-            List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(keys.size());
-            for (Term value : keys)
-                buffers.add(value.bindAndGet(options));
-            return buffers;
-        }
-
-        public boolean isSlice()
-        {
-            return false;
-        }
-
-        public boolean isEQ()
-        {
-            return false;
-        }
-
-        public boolean isIN()
-        {
-            return false;
-        }
-
-        public boolean isContains()
-        {
-            return true;
-        }
-
-        public boolean isOnToken()
-        {
-            return false;
-        }
-
-        public boolean canEvaluateWithSlices()
-        {
-            return false;
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("CONTAINS(values=%s, keys=%s)", values, keys);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/db/composites/Composites.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/Composites.java b/src/java/org/apache/cassandra/db/composites/Composites.java
index f6626e0..fa0df48 100644
--- a/src/java/org/apache/cassandra/db/composites/Composites.java
+++ b/src/java/org/apache/cassandra/db/composites/Composites.java
@@ -22,8 +22,11 @@ import java.util.List;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
 
 public abstract class Composites
 {
@@ -31,6 +34,23 @@ public abstract class Composites
 
     public static final Composite EMPTY = new EmptyComposite();
 
+    /**
+     * Converts the specified <code>Composites</code> into <code>ByteBuffer</code>s.
+     *
+     * @param composites the composites to convert.
+     * @return the <code>ByteBuffer</code>s corresponding to the specified <code>Composites</code>.
+     */
+    public static List<ByteBuffer> toByteBuffers(List<Composite> composites)
+    {
+        return Lists.transform(composites, new Function<Composite, ByteBuffer>()
+        {
+            public ByteBuffer apply(Composite composite)
+            {
+                return composite.toByteBuffer();
+            }
+        });
+    }
+
     static final CBuilder EMPTY_BUILDER = new CBuilder()
     {
         public int remainingCount() { return 0; }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java b/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
index 4542ac5..9a32dcc 100644
--- a/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
+++ b/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.composites;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Comparator;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.TreeSet;
@@ -240,8 +241,8 @@ public final class CompositesBuilder
         if (elementsList.isEmpty())
             return singletonList(builder.build().withEOC(eoc));
 
-        // Use a TreeSet to sort and eliminate duplicates
-        Set<Composite> set = new TreeSet<Composite>(comparator);
+        // Use a Set to sort if needed and eliminate duplicates
+        Set<Composite> set = newSet();
 
         for (int i = 0, m = elementsList.size(); i < m; i++)
         {
@@ -252,6 +253,16 @@ public final class CompositesBuilder
         return new ArrayList<>(set);
     }
 
+    /**
+     * Returns a new <code>Set</code> instance that will be used to eliminate duplicates and sort the results.
+     *
+     * @return a new <code>Set</code> instance.
+     */
+    private Set<Composite> newSet()
+    {
+        return comparator == null ? new LinkedHashSet<Composite>() : new TreeSet<Composite>(comparator);
+    }
+
     private void checkUpdateable()
     {
         if (!hasRemaining() || built)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/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 24ad533..3121899 100644
--- a/src/java/org/apache/cassandra/db/marshal/CollectionType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
@@ -26,6 +26,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Sets;
+
 import org.apache.cassandra.serializers.CollectionSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -43,7 +48,29 @@ public abstract class CollectionType<T> extends AbstractType<T>
 
     public enum Kind
     {
-        MAP, SET, LIST
+        MAP
+        {
+            public ColumnSpecification makeCollectionReceiver(ColumnSpecification collection, boolean isKey)
+            {
+                return isKey ? Maps.keySpecOf(collection) : Maps.valueSpecOf(collection);
+            }
+        },
+        SET
+        {
+            public ColumnSpecification makeCollectionReceiver(ColumnSpecification collection, boolean isKey)
+            {
+                return Sets.valueSpecOf(collection);
+            }
+        },
+        LIST
+        {
+            public ColumnSpecification makeCollectionReceiver(ColumnSpecification collection, boolean isKey)
+            {
+                return Lists.valueSpecOf(collection);
+            }
+        };
+
+        public abstract ColumnSpecification makeCollectionReceiver(ColumnSpecification collection, boolean isKey);
     }
 
     public final Kind kind;
@@ -59,6 +86,11 @@ public abstract class CollectionType<T> extends AbstractType<T>
     @Override
     public abstract CollectionSerializer<T> getSerializer();
 
+    public ColumnSpecification makeCollectionReceiver(ColumnSpecification collection, boolean isKey)
+    {
+        return kind.makeCollectionReceiver(collection, isKey);
+    }
+
     public String getString(ByteBuffer bytes)
     {
         return BytesType.instance.getString(bytes);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/exceptions/UnrecognizedEntityException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/UnrecognizedEntityException.java b/src/java/org/apache/cassandra/exceptions/UnrecognizedEntityException.java
new file mode 100644
index 0000000..e8392e9
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/UnrecognizedEntityException.java
@@ -0,0 +1,49 @@
+/*
+ * 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.exceptions;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.Relation;
+
+/**
+ * Exception thrown when an entity is not recognized within a relation.
+ */
+public final class UnrecognizedEntityException extends InvalidRequestException
+{
+    /**
+     * The unrecognized entity.
+     */
+    public final ColumnIdentifier entity;
+
+    /**
+     * The entity relation.
+     */
+    public final Relation relation;
+
+    /**
+     * Creates a new <code>UnrecognizedEntityException</code>.
+     * @param entity the unrecognized entity
+     * @param relation the entity relation
+     */
+    public UnrecognizedEntityException(ColumnIdentifier entity, Relation relation)
+    {
+        super(String.format("Undefined name %s in where clause ('%s')", entity, relation));
+        this.entity = entity;
+        this.relation = relation;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/test/unit/org/apache/cassandra/cql3/AliasTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/AliasTest.java b/test/unit/org/apache/cassandra/cql3/AliasTest.java
new file mode 100644
index 0000000..132aa04
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/AliasTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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.cql3;
+
+import org.junit.Test;
+
+public class AliasTest extends CQLTester
+{
+    @Test
+    public void testAlias() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int PRIMARY KEY, name text)");
+
+        for (int i = 0; i < 5; i++)
+            execute("INSERT INTO %s (id, name) VALUES (?, ?) USING TTL 10 AND TIMESTAMP 0", i, Integer.toString(i));
+
+        assertInvalidMessage("Aliases aren't allowed in the where clause" ,
+                             "SELECT id AS user_id, name AS user_name FROM %s WHERE user_id = 0");
+
+        // test that select throws a meaningful exception for aliases in order by clause
+        assertInvalidMessage("Aliases are not allowed in order by clause",
+                             "SELECT id AS user_id, name AS user_name FROM %s WHERE id IN (0) ORDER BY user_name");
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java b/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java
index 12d65fa..8f78553 100644
--- a/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java
@@ -26,7 +26,11 @@ public class ContainsRelationTest extends CQLTester
                    row("test", 5, set("lmn"))
         );
 
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS ?", "xyz", "lmn", "notPresent");
+        assertInvalidMessage("Unsupported null value for indexed column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, null);
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
+                             "SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS ?", "xyz", "lmn", "notPresent");
         assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING", "xyz", "lmn", "notPresent"));
     }
 
@@ -52,8 +56,12 @@ public class ContainsRelationTest extends CQLTester
                    row("test", 5, list("lmn"))
         );
 
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?",
-                      "test", 5, "lmn", "notPresent");
+        assertInvalidMessage("Unsupported null value for indexed column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, null);
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?",
+                             "test", 5, "lmn", "notPresent");
         assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING",
                             "test", 5, "lmn", "notPresent"));
     }
@@ -79,13 +87,18 @@ public class ContainsRelationTest extends CQLTester
                    row("test", 5, map("lmn", "foo"))
         );
 
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS KEY ?",
-                      "test", 5, "lmn", "notPresent");
+        assertInvalidMessage("Unsupported null value for indexed column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ?", "test", 5, null);
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS KEY ?",
+                             "test", 5, "lmn", "notPresent");
         assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS KEY ? ALLOW FILTERING",
                             "test", 5, "lmn", "notPresent"));
 
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS ?",
-                      "test", 5, "lmn", "foo");
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS ?",
+                             "test", 5, "lmn", "foo");
     }
 
     @Test
@@ -110,7 +123,11 @@ public class ContainsRelationTest extends CQLTester
                    row("test", 5, map("lmn", "foo"))
         );
 
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?"
+        assertInvalidMessage("Unsupported null value for indexed column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, null);
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?"
                            , "test", 5, "foo", "notPresent");
 
         assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING"
@@ -197,7 +214,8 @@ public class ContainsRelationTest extends CQLTester
         execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
         execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn", "foo2"));
 
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo");
+        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators: 'categories CONTAINS <value>'",
+                             "SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo");
 
         assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn"),
                    row("test", 5, map("lmn", "foo")),
@@ -219,7 +237,8 @@ public class ContainsRelationTest extends CQLTester
         execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
         execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn2", "foo"));
 
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn");
+        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators: 'categories CONTAINS KEY <value>'",
+                             "SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn");
 
         assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo"),
                    row("test", 5, map("lmn", "foo")),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/test/unit/org/apache/cassandra/cql3/FrozenCollectionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/FrozenCollectionsTest.java b/test/unit/org/apache/cassandra/cql3/FrozenCollectionsTest.java
index bf7ccfd..896bc5f 100644
--- a/test/unit/org/apache/cassandra/cql3/FrozenCollectionsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/FrozenCollectionsTest.java
@@ -75,8 +75,8 @@ public class FrozenCollectionsTest extends CQLTester
         );
 
         assertRows(execute("SELECT * FROM %s WHERE k IN ?", list(set(4, 5, 6), set())),
-                row(set(4, 5, 6), 0),
-                row(set(), 0)
+                   row(set(), 0),
+                   row(set(4, 5, 6), 0)
         );
 
         assertRows(execute("SELECT * FROM %s WHERE token(k) >= token(?)", set(4, 5, 6)),
@@ -144,9 +144,9 @@ public class FrozenCollectionsTest extends CQLTester
         );
 
         assertRows(execute("SELECT * FROM %s WHERE k IN ?", list(map(set(4, 5, 6), list(1, 2, 3)), map(), map(set(), list(1, 2, 3)))),
-            row(map(set(4, 5, 6), list(1, 2, 3)), 0),
-            row(map(), 0),
-            row(map(set(), list(1, 2, 3)), 0)
+                   row(map(), 0),
+                   row(map(set(), list(1, 2, 3)), 0),
+                   row(map(set(4, 5, 6), list(1, 2, 3)), 0)
         );
 
         assertRows(execute("SELECT * FROM %s WHERE token(k) >= token(?)", map(set(4, 5, 6), list(1, 2, 3))),
@@ -615,10 +615,10 @@ public class FrozenCollectionsTest extends CQLTester
                              "SELECT * FROM %s WHERE c CONTAINS KEY ?", 1);
 
         // normal indexes on frozen collections don't support CONTAINS or CONTAINS KEY
-        assertInvalidMessage("Cannot restrict column \"b\" by a CONTAINS relation without a secondary index",
+        assertInvalidMessage("Cannot restrict clustering columns by a CONTAINS relation without a secondary index",
                              "SELECT * FROM %s WHERE b CONTAINS ?", 1);
 
-        assertInvalidMessage("Cannot restrict column \"b\" by a CONTAINS relation without a secondary index",
+        assertInvalidMessage("Cannot restrict clustering columns by a CONTAINS relation without a secondary index",
                              "SELECT * FROM %s WHERE b CONTAINS ? ALLOW FILTERING", 1);
 
         assertInvalidMessage("No secondary indexes on the restricted columns support the provided operator",
@@ -627,7 +627,7 @@ public class FrozenCollectionsTest extends CQLTester
         assertInvalidMessage("No secondary indexes on the restricted columns support the provided operator",
                              "SELECT * FROM %s WHERE d CONTAINS KEY ? ALLOW FILTERING", 1);
 
-        assertInvalidMessage("Cannot restrict column \"b\" by a CONTAINS relation without a secondary index",
+        assertInvalidMessage("Cannot restrict clustering columns by a CONTAINS relation without a secondary index",
                              "SELECT * FROM %s WHERE b CONTAINS ? AND d CONTAINS KEY ? ALLOW FILTERING", 1, 1);
 
         // index lookup on b

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
index 4c3ba2a..291afd8 100644
--- a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
@@ -24,56 +24,77 @@ public class MultiColumnRelationTest extends CQLTester
     @Test
     public void testSingleClusteringInvalidQueries() throws Throwable
     {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
         {
             createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
 
             assertInvalidSyntax("SELECT * FROM %s WHERE () = (?, ?)", 1, 2);
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b) = (?) AND (b) > (?)", 0, 0);
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND (b) > (?)", 0, 1);
-            assertInvalid("SELECT * FROM %s WHERE (a, b) = (?, ?)", 0, 0);
+            assertInvalidMessage("b cannot be restricted by more than one relation if it includes an Equal",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b) = (?) AND (b) > (?)", 0, 0);
+            assertInvalidMessage("More than one restriction was found for the start bound on b",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND (b) > (?)", 0, 1);
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                                 "SELECT * FROM %s WHERE (a, b) = (?, ?)", 0, 0);
         }
     }
 
     @Test
     public void testMultiClusteringInvalidQueries() throws Throwable
     {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
         {
             createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
 
             assertInvalidSyntax("SELECT * FROM %s WHERE a = 0 AND (b, c) > ()");
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?, ?)", 1, 2, 3);
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?)", 1, null);
+            assertInvalidMessage("Expected 2 elements in value tuple, but got 3: (?, ?, ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?, ?)", 1, 2, 3);
+            assertInvalidMessage("Invalid null value in condition for column c",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c) > (?, ?)", 1, null);
 
             // Wrong order of columns
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (d, c, b) = (?, ?, ?)", 0, 0, 0);
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (d, c, b) > (?, ?, ?)", 0, 0, 0);
+            assertInvalidMessage("Clustering columns may not be skipped in multi-column relations. They should appear in the PRIMARY KEY order. Got (d, c, b) = (?, ?, ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (d, c, b) = (?, ?, ?)", 0, 0, 0);
+            assertInvalidMessage("Clustering columns may not be skipped in multi-column relations. They should appear in the PRIMARY KEY order. Got (d, c, b) > (?, ?, ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (d, c, b) > (?, ?, ?)", 0, 0, 0);
 
             // Wrong number of values
-            assertInvalid("SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?))", 0, 1);
-            assertInvalid("SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))", 0, 1, 2, 3, 4);
+            assertInvalidMessage("Expected 3 elements in value tuple, but got 2: (?, ?)",
+                                 "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?))", 0, 1);
+            assertInvalidMessage("Expected 3 elements in value tuple, but got 5: (?, ?, ?, ?, ?)",
+                                 "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))", 0, 1, 2, 3, 4);
 
             // Missing first clustering column
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (c, d) = (?, ?)", 0, 0);
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (c, d) > (?, ?)", 0, 0);
+            assertInvalidMessage("Clustering columns may not be skipped in multi-column relations. They should appear in the PRIMARY KEY order. Got (c, d) = (?, ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (c, d) = (?, ?)", 0, 0);
+            assertInvalidMessage("Clustering columns may not be skipped in multi-column relations. They should appear in the PRIMARY KEY order. Got (c, d) > (?, ?)",
+                                 "SELECT * FROM %s WHERE a = 0 AND (c, d) > (?, ?)", 0, 0);
 
             // Nulls
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?))", 1, 2, null);
+            assertInvalidMessage("Invalid null value in condition for column d",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?))", 1, 2, null);
 
             // Wrong type for 'd'
             assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
-
             assertInvalid("SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
 
             // Mix single and tuple inequalities
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND b < ?", 0, 1, 0, 1);
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND c < ?", 0, 1, 0, 1);
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND b > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND c > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
-
-            assertInvalid("SELECT * FROM %s WHERE (a, b, c, d) IN ((?, ?, ?, ?))", 0, 1, 2, 3);
-            assertInvalid("SELECT * FROM %s WHERE (c, d) IN ((?, ?))", 0, 1);
+            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND b < ?", 0, 1, 0, 1);
+            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
+                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND c < ?", 0, 1, 0, 1);
+            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
+                                 "SELECT * FROM %s WHERE a = 0 AND b > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
+            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
+                                 "SELECT * FROM %s WHERE a = 0 AND c > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
+
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                                 "SELECT * FROM %s WHERE (a, b, c, d) IN ((?, ?, ?, ?))", 0, 1, 2, 3);
+            assertInvalidMessage("Clustering columns may not be skipped in multi-column relations. They should appear in the PRIMARY KEY order. Got (c, d) IN ((?, ?))",
+                                 "SELECT * FROM %s WHERE (c, d) IN ((?, ?))", 0, 1);
+
+            assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
+                                 "SELECT * FROM %s WHERE a = ? AND (b, c) in ((?, ?), (?, ?)) AND d > ?",
+                                 0, 0, 0, 0, 0, 0);
 
             assertInvalid("SELECT * FROM %s WHERE a = ? AND (b, c) in ((?, ?), (?, ?)) AND d > ?", 0, 0, 0, 0, 0, 0);
         }
@@ -85,10 +106,12 @@ public class MultiColumnRelationTest extends CQLTester
         for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
         {
             createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)" + compactOption);
-
-            assertInvalid("SELECT * FROM %s WHERE (a) > (?)", 0);
-            assertInvalid("SELECT * FROM %s WHERE (a) = (?)", 0);
-            assertInvalid("SELECT * FROM %s WHERE (b) = (?)", 0);
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                                 "SELECT * FROM %s WHERE (a) > (?)", 0);
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                                 "SELECT * FROM %s WHERE (a) = (?)", 0);
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: b",
+                                 "SELECT * FROM %s WHERE (b) = (?)", 0);
         }
     }
 
@@ -150,7 +173,8 @@ public class MultiColumnRelationTest extends CQLTester
         for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
         {
             createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)" + compactOption);
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b) != (0)");
+            assertInvalidMessage("Unsupported \"!=\" relation: (b) != (0)",
+                    "SELECT * FROM %s WHERE a = 0 AND (b) != (0)");
         }
     }
 
@@ -444,10 +468,10 @@ public class MultiColumnRelationTest extends CQLTester
 
             // same query, but reversed order for the IN values
             assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND (b, c, d) IN (?, ?)", 1, 0, tuple(0, 1, 1), tuple(0, 1, 0)),
-                    row(1, 0, 1, 0),
-                    row(1, 0, 1, 1),
                     row(0, 0, 1, 0),
-                    row(0, 0, 1, 1)
+                    row(0, 0, 1, 1),
+                    row(1, 0, 1, 0),
+                    row(1, 0, 1, 1)
             );
 
             assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) and (b, c) IN ((?, ?))", 0, 1, 0, 1),
@@ -555,4 +579,81 @@ public class MultiColumnRelationTest extends CQLTester
             assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0));
         }
     }
+
+    @Test
+    public void testMultipleClusteringWithIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b, c, d))");
+        createIndex("CREATE INDEX ON %s (b)");
+        createIndex("CREATE INDEX ON %s (e)");
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 0, 1);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 1, 2);
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 1);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 1, 2);
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, 0, 0);
+
+        assertRows(execute("SELECT * FROM %s WHERE (b) = (?)", 1),
+                   row(0, 1, 0, 0, 0),
+                   row(0, 1, 1, 0, 1),
+                   row(0, 1, 1, 1, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) = (?, ?) ALLOW FILTERING", 1, 1),
+                   row(0, 1, 1, 0, 1),
+                   row(0, 1, 1, 1, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) = (?, ?) AND e = ? ALLOW FILTERING", 1, 1, 2),
+                   row(0, 1, 1, 1, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE (b) IN ((?)) AND e = ? ALLOW FILTERING", 1, 2),
+                   row(0, 1, 1, 1, 2));
+
+        assertInvalidMessage("IN restrictions are not supported on indexed columns",
+                             "SELECT * FROM %s WHERE (b) IN ((?), (?)) AND e = ? ALLOW FILTERING", 0, 1, 2);
+
+        assertInvalidMessage("IN restrictions are not supported on indexed columns",
+                             "SELECT * FROM %s WHERE (b, c) IN ((?, ?)) AND e = ? ALLOW FILTERING", 0, 1, 2);
+
+        assertInvalidMessage("IN restrictions are not supported on indexed columns",
+                             "SELECT * FROM %s WHERE (b, c) IN ((?, ?), (?, ?)) AND e = ? ALLOW FILTERING", 0, 1, 1, 1, 2);
+
+        assertInvalidMessage("Slice restrictions are not supported on indexed columns which are part of a multi column relation",
+                             "SELECT * FROM %s WHERE (b) >= (?) AND e = ? ALLOW FILTERING", 1, 2);
+    }
+
+    @Test
+    public void testMultiplePartitionKeyAndMultiClusteringWithIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY ((a, b), c, d, e))");
+        createIndex("CREATE INDEX ON %s (c)");
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 1, 0);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 1, 1);
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 0, 0);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 1, 0);
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 1, 1, 1);
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 2, 0, 0);
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) = (?) ALLOW FILTERING", 0, 1),
+                   row(0, 0, 1, 0, 0),
+                   row(0, 0, 1, 1, 0),
+                   row(0, 0, 1, 1, 1));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) = (?, ?) ALLOW FILTERING", 0, 1, 1),
+                   row(0, 0, 1, 1, 0),
+                   row(0, 0, 1, 1, 1));
+
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) IN ((?, ?)) ALLOW FILTERING", 0, 1, 1);
+
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) ALLOW FILTERING", 0, 1, 1);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java b/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
index 6f9f5e2..39b62e3 100644
--- a/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
@@ -30,17 +30,34 @@ public class SelectWithTokenFunctionTest extends CQLTester
         assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?)", 0), row(0, "a"));
         assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?) and token(a) < token(?)", 0, 1), row(0, "a"));
         assertInvalid("SELECT * FROM %s WHERE token(a) > token(?)", "a");
-        assertInvalid("SELECT * FROM %s WHERE token(a, b) >= token(?, ?)", "b", 0);
-        assertInvalid("SELECT * FROM %s WHERE token(a) >= token(?) and token(a) >= token(?)", 0, 1);
-        assertInvalid("SELECT * FROM %s WHERE token(a) >= token(?) and token(a) = token(?)", 0, 1);
+        assertInvalidMessage("Columns \"a\" cannot be restricted by both a normal relation and a token relation",
+                             "SELECT * FROM %s WHERE token(a) > token(?) AND a = ?", 1, 1);
+        assertInvalidMessage("Columns \"a\" cannot be restricted by both a normal relation and a token relation",
+                             "SELECT * FROM %s WHERE a = ? and token(a) > token(?)", 1, 1);
+        assertInvalidMessage("The token() function must contains only partition key components",
+                             "SELECT * FROM %s WHERE token(a, b) >= token(?, ?)", "b", 0);
+        assertInvalidMessage("More than one restriction was found for the start bound on a",
+                             "SELECT * FROM %s WHERE token(a) >= token(?) and token(a) >= token(?)", 0, 1);
+        assertInvalidMessage("Columns \"a\" cannot be restricted by both an equality and an inequality relation",
+                             "SELECT * FROM %s WHERE token(a) >= token(?) and token(a) = token(?)", 0, 1);
         assertInvalidSyntax("SELECT * FROM %s WHERE token(a) = token(?) and token(a) IN (token(?))", 0, 1);
+
+        assertInvalidMessage("More than one restriction was found for the start bound on a",
+                             "SELECT * FROM %s WHERE token(a) > token(?) AND token(a) > token(?)", 1, 2);
+        assertInvalidMessage("More than one restriction was found for the end bound on a",
+                             "SELECT * FROM %s WHERE token(a) <= token(?) AND token(a) < token(?)", 1, 2);
+        assertInvalidMessage("Columns \"a\" cannot be restricted by both an equality and an inequality relation",
+                             "SELECT * FROM %s WHERE token(a) > token(?) AND token(a) = token(?)", 1, 2);
+        assertInvalidMessage("a cannot be restricted by more than one relation if it includes an Equal",
+                             "SELECT * FROM %s WHERE  token(a) = token(?) AND token(a) > token(?)", 1, 2);
     }
 
     @Test
     public void testTokenFunctionWithPartitionKeyAndClusteringKeyArguments() throws Throwable
     {
         createTable("CREATE TABLE IF NOT EXISTS %s (a int, b text, PRIMARY KEY (a, b))");
-        assertInvalid("SELECT * FROM %s WHERE token(a, b) > token(0, 'c')");
+        assertInvalidMessage("The token() function must contains only partition key components",
+                             "SELECT * FROM %s WHERE token(a, b) > token(0, 'c')");
     }
 
     @Test
@@ -59,8 +76,16 @@ public class SelectWithTokenFunctionTest extends CQLTester
                            0, "d"),
                    row(0, "b"),
                    row(0, "c"));
-        assertInvalid("SELECT * FROM %s WHERE token(a) > token(?) and token(b) > token(?)", 0, "a");
-        assertInvalid("SELECT * FROM %s WHERE token(a) > token(?, ?) and token(a) < token(?, ?) and token(b) > token(?, ?) ", 0, "a", 0, "d", 0, "a");
-        assertInvalid("SELECT * FROM %s WHERE token(b, a) > token(0, 'c')");
+        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
+                             "SELECT * FROM %s WHERE token(a) > token(?) and token(b) > token(?)", 0, "a");
+        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
+                             "SELECT * FROM %s WHERE token(a) > token(?, ?) and token(a) < token(?, ?) and token(b) > token(?, ?) ",
+                             0, "a", 0, "d", 0, "a");
+        assertInvalidMessage("The token function arguments must be in the partition key order: a, b",
+                             "SELECT * FROM %s WHERE token(b, a) > token(0, 'c')");
+        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
+                             "SELECT * FROM %s WHERE token(a, b) > token(?, ?) and token(b) < token(?, ?)", 0, "a", 0, "a");
+        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
+                             "SELECT * FROM %s WHERE token(a) > token(?, ?) and token(b) > token(?, ?)", 0, "a", 0, "a");
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java
index c93147b..112da06 100644
--- a/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/SingleColumnRelationTest.java
@@ -31,9 +31,12 @@ public class SingleColumnRelationTest extends CQLTester
         createIndex("CREATE INDEX ON %s (c)");
         createIndex("CREATE INDEX ON %s (d)");
 
-        assertInvalid("SELECT * FROM %s WHERE a = 0 AND b=?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE a = 0 AND c=?", list(0));
-        assertInvalid("SELECT * FROM %s WHERE a = 0 AND d=?", map(0, 0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '=' relation",
+                             "SELECT * FROM %s WHERE a = 0 AND b=?", set(0));
+        assertInvalidMessage("Collection column 'c' (list<int>) cannot be restricted by a '=' relation",
+                             "SELECT * FROM %s WHERE a = 0 AND c=?", list(0));
+        assertInvalidMessage("Collection column 'd' (map<int, int>) cannot be restricted by a '=' relation",
+                             "SELECT * FROM %s WHERE a = 0 AND d=?", map(0, 0));
     }
 
     @Test
@@ -44,11 +47,16 @@ public class SingleColumnRelationTest extends CQLTester
         execute("INSERT INTO %s (a, b, c) VALUES (0, {0}, 0)");
 
         // non-EQ operators
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b > ?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b >= ?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b < ?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b <= ?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b IN (?)", set(0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '>' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b > ?", set(0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '>=' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b >= ?", set(0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '<' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b < ?", set(0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '<=' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b <= ?", set(0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a 'IN' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b IN (?)", set(0));
     }
 
     @Test
@@ -114,7 +122,8 @@ public class SingleColumnRelationTest extends CQLTester
                    row("first", 2, 6, 2),
                    row("first", 3, 7, 3));
 
-        assertInvalid("select * from %s where a = ? and b in ? and c in ?", "first", null, Arrays.asList(7, 6));
+        assertInvalidMessage("Invalid null value for IN restriction",
+                             "select * from %s where a = ? and b in ? and c in ?", "first", null, Arrays.asList(7, 6));
 
         assertRows(execute("select * from %s where a = ? and c >= ? and b in (?, ?)", "first", 6, 3, 2),
                    row("first", 2, 6, 2),
@@ -128,11 +137,32 @@ public class SingleColumnRelationTest extends CQLTester
 
         assertRows(execute("select * from %s where a = ? and c < ? and b in (?, ?)", "first", 7, 3, 2),
                    row("first", 2, 6, 2));
+//---
+        assertRows(execute("select * from %s where a = ? and c >= ? and c <= ? and b in (?, ?)", "first", 6, 7, 3, 2),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select * from %s where a = ? and c > ? and c <= ? and b in (?, ?)", "first", 6, 7, 3, 2),
+                   row("first", 3, 7, 3));
+
+        assertEmpty(execute("select * from %s where a = ? and c > ? and c < ? and b in (?, ?)", "first", 6, 7, 3, 2));
+
+        assertInvalidMessage("Column \"c\" cannot be restricted by both an equality and an inequality relation",
+                             "select * from %s where a = ? and c > ? and c = ? and b in (?, ?)", "first", 6, 7, 3, 2);
+
+        assertInvalidMessage("c cannot be restricted by more than one relation if it includes an Equal",
+                             "select * from %s where a = ? and c = ? and c > ?  and b in (?, ?)", "first", 6, 7, 3, 2);
 
         assertRows(execute("select * from %s where a = ? and c in (?, ?) and b in (?, ?) order by b DESC",
                            "first", 7, 6, 3, 2),
                    row("first", 3, 7, 3),
                    row("first", 2, 6, 2));
+
+        assertInvalidMessage("More than one restriction was found for the start bound on b",
+                             "select * from %s where a = ? and b > ? and b > ?", "first", 6, 3, 2);
+
+        assertInvalidMessage("More than one restriction was found for the end bound on b",
+                             "select * from %s where a = ? and b < ? and b <= ?", "first", 6, 3, 2);
     }
 
     @Test
@@ -144,8 +174,16 @@ public class SingleColumnRelationTest extends CQLTester
         execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 3, 3);
         execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 4, 4);
 
-        assertInvalid("select * from %s where a in (?, ?)", "first", "second");
-        assertInvalid("select * from %s where a in (?, ?) and b in (?, ?)", "first", "second", 2, 3);
+        assertInvalidMessage("Partition KEY part a cannot be restricted by IN relation (only the last part of the partition key can)",
+                             "select * from %s where a in (?, ?)", "first", "second");
+        assertInvalidMessage("Partition KEY part a cannot be restricted by IN relation (only the last part of the partition key can)",
+                             "select * from %s where a in (?, ?) and b in (?, ?)", "first", "second", 2, 3);
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "select * from %s where a = ?", "first");
+        assertInvalidMessage("b cannot be restricted by more than one relation if it includes a IN",
+                             "select * from %s where a = ? AND b IN (?, ?) AND b = ?", "first", 2, 2, 3);
+        assertInvalidMessage("b cannot be restricted by more than one relation if it includes an Equal",
+                             "select * from %s where a = ? AND b = ? AND b IN (?, ?)", "first", 2, 2, 3);
     }
 
     @Test
@@ -167,4 +205,162 @@ public class SingleColumnRelationTest extends CQLTester
                    row("first", 2, 6, 2),
                    row("first", 3, 7, 3));
     }
+
+    @Test
+    public void testAllowFilteringWithClusteringColumn() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c))");
+
+        execute("INSERT INTO %s (k, c, v) VALUES(?, ?, ?)", 1, 2, 1);
+        execute("INSERT INTO %s (k, c, v) VALUES(?, ?, ?)", 1, 3, 2);
+        execute("INSERT INTO %s (k, c, v) VALUES(?, ?, ?)", 2, 2, 3);
+
+        // Don't require filtering, always allowed
+        assertRows(execute("SELECT * FROM %s WHERE k = ?", 1),
+                   row(1, 2, 1),
+                   row(1, 3, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c > ?", 1, 2), row(1, 3, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c = ?", 1, 2), row(1, 2, 1));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? ALLOW FILTERING", 1),
+                   row(1, 2, 1),
+                   row(1, 3, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c > ? ALLOW FILTERING", 1, 2), row(1, 3, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c = ? ALLOW FILTERING", 1, 2), row(1, 2, 1));
+
+        // Require filtering, allowed only with ALLOW FILTERING
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE c = ?", 2);
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE c > ? AND c <= ?", 2, 4);
+
+        assertRows(execute("SELECT * FROM %s WHERE c = ? ALLOW FILTERING", 2),
+                   row(1, 2, 1),
+                   row(2, 2, 3));
+
+        assertRows(execute("SELECT * FROM %s WHERE c > ? AND c <= ? ALLOW FILTERING", 2, 4), row(1, 3, 2));
+    }
+
+    @Test
+    public void testAllowFilteringWithIndexedColumn() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, a int, b int)");
+        createIndex("CREATE INDEX ON %s(a)");
+
+        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 1, 10, 100);
+        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 2, 20, 200);
+        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 3, 30, 300);
+        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 4, 40, 400);
+
+        // Don't require filtering, always allowed
+        assertRows(execute("SELECT * FROM %s WHERE k = ?", 1), row(1, 10, 100));
+        assertRows(execute("SELECT * FROM %s WHERE a = ?", 20), row(2, 20, 200));
+        assertRows(execute("SELECT * FROM %s WHERE k = ? ALLOW FILTERING", 1), row(1, 10, 100));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? ALLOW FILTERING", 20), row(2, 20, 200));
+
+        assertInvalid("SELECT * FROM %s WHERE a = ? AND b = ?");
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ? ALLOW FILTERING", 20, 200), row(2, 20, 200));
+    }
+
+    @Test
+    public void testIndexQueriesOnComplexPrimaryKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk0 int, pk1 int, ck0 int, ck1 int, ck2 int, value int, PRIMARY KEY ((pk0, pk1), ck0, ck1, ck2))");
+
+        createIndex("CREATE INDEX ON %s (ck1)");
+        createIndex("CREATE INDEX ON %s (ck2)");
+        createIndex("CREATE INDEX ON %s (pk0)");
+        createIndex("CREATE INDEX ON %s (ck0)");
+
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 0, 1, 2, 3, 4, 5);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 1, 2, 3, 4, 5, 0);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 2, 3, 4, 5, 0, 1);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 3, 4, 5, 0, 1, 2);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 4, 5, 0, 1, 2, 3);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 5, 0, 1, 2, 3, 4);
+
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 2"), row(1));
+        assertRows(execute("SELECT value FROM %s WHERE ck0 = 0"), row(3));
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 3 AND pk1 = 4 AND ck1 = 0"), row(2));
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 5 AND pk1 = 0 AND ck0 = 1 AND ck2 = 3 ALLOW FILTERING"), row(4));
+    }
+
+    @Test
+    public void testIndexOnClusteringColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id1 int, id2 int, author text, time bigint, v1 text, v2 text, PRIMARY KEY ((id1, id2), author, time))");
+        createIndex("CREATE INDEX ON %s(time)");
+        createIndex("CREATE INDEX ON %s(id2)");
+
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 0, 'bob', 0, 'A', 'A')");
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 0, 'bob', 1, 'B', 'B')");
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 1, 'bob', 2, 'C', 'C')");
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 0, 'tom', 0, 'D', 'D')");
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 1, 'tom', 1, 'E', 'E')");
+
+        assertRows(execute("SELECT v1 FROM %s WHERE time = 1"), row("B"), row("E"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id2 = 1"), row("C"), row("E"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id1 = 0 AND id2 = 0 AND author = 'bob' AND time = 0"), row("A"));
+
+        // Test for CASSANDRA-8206
+        execute("UPDATE %s SET v2 = null WHERE id1 = 0 AND id2 = 0 AND author = 'bob' AND time = 1");
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id2 = 0"), row("A"), row("B"), row("D"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE time = 1"), row("B"), row("E"));
+
+        assertInvalidMessage("IN restrictions are not supported on indexed columns",
+                             "SELECT v1 FROM %s WHERE id2 = 0 and time IN (1, 2) ALLOW FILTERING");
+    }
+
+    @Test
+    public void testCompositeIndexWithPrimaryKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (blog_id int, time1 int, time2 int, author text, content text, PRIMARY KEY (blog_id, time1, time2))");
+
+        createIndex("CREATE INDEX ON %s(author)");
+
+        String req = "INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, ?)";
+        execute(req, 1, 0, 0, "foo", "bar1");
+        execute(req, 1, 0, 1, "foo", "bar2");
+        execute(req, 2, 1, 0, "foo", "baz");
+        execute(req, 3, 0, 1, "gux", "qux");
+
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE author='foo'"),
+                   row(1, "bar1"),
+                   row(1, "bar2"),
+                   row(2, "baz"));
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 > 0 AND author='foo' ALLOW FILTERING"), row(2, "baz"));
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND author='foo' ALLOW FILTERING"), row(2, "baz"));
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND time2 = 0 AND author='foo' ALLOW FILTERING"),
+                   row(2, "baz"));
+        assertEmpty(execute("SELECT content FROM %s WHERE time1 = 1 AND time2 = 1 AND author='foo' ALLOW FILTERING"));
+        assertEmpty(execute("SELECT content FROM %s WHERE time1 = 1 AND time2 > 0 AND author='foo' ALLOW FILTERING"));
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT content FROM %s WHERE time2 >= 0 AND author='foo'");
+    }
+
+    @Test
+    public void testRangeQueryOnIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int primary key, row int, setid int);");
+        createIndex("CREATE INDEX ON %s (setid)");
+
+        String q = "INSERT INTO %s (id, row, setid) VALUES (?, ?, ?);";
+        execute(q, 0, 0, 0);
+        execute(q, 1, 1, 0);
+        execute(q, 2, 2, 0);
+        execute(q, 3, 3, 0);
+
+        assertInvalidMessage("Cannot execute this query as it might involve data filtering",
+                             "SELECT * FROM %s WHERE setid = 0 AND row < 1;");
+        assertRows(execute("SELECT * FROM %s WHERE setid = 0 AND row < 1 ALLOW FILTERING;"), row(0, 0, 0));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
index deb3082..88ee688 100644
--- a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
@@ -33,7 +33,7 @@ public class ThriftCompatibilityTest extends SchemaLoader
     @BeforeClass
     public static void defineSchema() throws Exception
     {
-        SchemaLoader.prepareServer();
+        // The before class annotation of SchemaLoader will prepare the service so no need to do it here
         SchemaLoader.createKeyspace("thriftcompat",
                                     SimpleStrategy.class,
                                     KSMetaData.optsWithRF(1),


[4/5] cassandra git commit: Refactor SelectStatement and Restrictions

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java b/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
new file mode 100644
index 0000000..d0ed193
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
@@ -0,0 +1,97 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A restriction/clause on a column.
+ * The goal of this class being to group all conditions for a column in a SELECT.
+ */
+public interface Restriction
+{
+    public boolean isOnToken();
+    public boolean isSlice();
+    public boolean isEQ();
+    public boolean isIN();
+    public boolean isContains();
+    public boolean isMultiColumn();
+
+    public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException;
+
+    /**
+     * Returns <code>true</code> if one of the restrictions use the specified function.
+     *
+     * @param ksName the keyspace name
+     * @param functionName the function name
+     * @return <code>true</code> if one of the restrictions use the specified function, <code>false</code> otherwise.
+     */
+    public boolean usesFunction(String ksName, String functionName);
+
+    /**
+     * Checks if the specified bound is set or not.
+     * @param b the bound type
+     * @return <code>true</code> if the specified bound is set, <code>false</code> otherwise
+     */
+    public boolean hasBound(Bound b);
+
+    public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException;
+
+    /**
+     * Checks if the specified bound is inclusive or not.
+     * @param b the bound type
+     * @return <code>true</code> if the specified bound is inclusive, <code>false</code> otherwise
+     */
+    public boolean isInclusive(Bound b);
+
+    /**
+     * Merges this restriction with the specified one.
+     *
+     * @param otherRestriction the restriction to merge into this one
+     * @return the restriction resulting of the merge
+     * @throws InvalidRequestException if the restrictions cannot be merged
+     */
+    public Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException;
+
+    /**
+     * Check if the restriction is on indexed columns.
+     *
+     * @param indexManager the index manager
+     * @return <code>true</code> if the restriction is on indexed columns, <code>false</code>
+     */
+    public boolean hasSupportingIndex(SecondaryIndexManager indexManager);
+
+    /**
+     * Adds to the specified list the <code>IndexExpression</code>s corresponding to this <code>Restriction</code>.
+     *
+     * @param expressions the list to add the <code>IndexExpression</code>s to
+     * @param options the query options
+     * @throws InvalidRequestException if this <code>Restriction</code> cannot be converted into 
+     * <code>IndexExpression</code>s
+     */
+    public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                     QueryOptions options)
+                                     throws InvalidRequestException;
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java
new file mode 100644
index 0000000..cf2555e
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java
@@ -0,0 +1,82 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * Sets of restrictions
+ */
+interface Restrictions
+{
+    /**
+     * Returns the column definitions in position order.
+     * @return the column definitions in position order.
+     */
+    public Collection<ColumnDefinition> getColumnDefs();
+
+    /**
+     * Returns <code>true</code> if one of the restrictions use the specified function.
+     *
+     * @param ksName the keyspace name
+     * @param functionName the function name
+     * @return <code>true</code> if one of the restrictions use the specified function, <code>false</code> otherwise.
+     */
+    public boolean usesFunction(String ksName, String functionName);
+
+    /**
+     * Check if the restriction is on indexed columns.
+     *
+     * @param indexManager the index manager
+     * @return <code>true</code> if the restriction is on indexed columns, <code>false</code>
+     */
+    public boolean hasSupportingIndex(SecondaryIndexManager indexManager);
+
+    /**
+     * Adds to the specified list the <code>IndexExpression</code>s corresponding to this <code>Restriction</code>.
+     *
+     * @param expressions the list to add the <code>IndexExpression</code>s to
+     * @param options the query options
+     * @throws InvalidRequestException if this <code>Restriction</code> cannot be converted into
+     * <code>IndexExpression</code>s
+     */
+    public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                     QueryOptions options)
+                                     throws InvalidRequestException;
+
+    /**
+     * Checks if this <code>SingleColumnPrimaryKeyRestrictions</code> is empty or not.
+     *
+     * @return <code>true</code> if this <code>SingleColumnPrimaryKeyRestrictions</code> is empty, <code>false</code> otherwise.
+     */
+    boolean isEmpty();
+
+    /**
+     * Returns the number of columns that have a restriction.
+     *
+     * @return the number of columns that have a restriction.
+     */
+    public int size();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/restrictions/ReversedPrimaryKeyRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/ReversedPrimaryKeyRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/ReversedPrimaryKeyRestrictions.java
new file mode 100644
index 0000000..9b33161
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/ReversedPrimaryKeyRestrictions.java
@@ -0,0 +1,77 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * <code>PrimaryKeyRestrictions</code> decorator that reverse the slices.
+ */
+final class ReversedPrimaryKeyRestrictions extends ForwardingPrimaryKeyRestrictions
+{
+    /**
+     * The decorated restrictions.
+     */
+    private PrimaryKeyRestrictions restrictions;
+
+    public ReversedPrimaryKeyRestrictions(PrimaryKeyRestrictions restrictions)
+    {
+        this.restrictions = restrictions;
+    }
+
+    @Override
+    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException
+    {
+        return new ReversedPrimaryKeyRestrictions(this.restrictions.mergeWith(restriction));
+    }
+
+    @Override
+    public List<ByteBuffer> bounds(Bound bound, QueryOptions options) throws InvalidRequestException
+    {
+        List<ByteBuffer> buffers = restrictions.bounds(bound.reverse(), options);
+        Collections.reverse(buffers);
+        return buffers;
+    }
+
+    @Override
+    public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
+    {
+        List<Composite> composites = restrictions.boundsAsComposites(bound.reverse(), options);
+        Collections.reverse(composites);
+        return composites;
+    }
+
+    @Override
+    public boolean isInclusive(Bound bound)
+    {
+        return this.restrictions.isInclusive(bound.reverse());
+    }
+
+    @Override
+    protected PrimaryKeyRestrictions getDelegate()
+    {
+        return this.restrictions;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnPrimaryKeyRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnPrimaryKeyRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnPrimaryKeyRestrictions.java
new file mode 100644
index 0000000..3858cdc
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnPrimaryKeyRestrictions.java
@@ -0,0 +1,307 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.composites.CBuilder;
+import org.apache.cassandra.db.composites.CType;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.composites.Composite.EOC;
+import org.apache.cassandra.db.composites.Composites;
+import org.apache.cassandra.db.composites.CompositesBuilder;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+
+/**
+ * A set of single column restrictions on a primary key part (partition key or clustering key).
+ */
+final class SingleColumnPrimaryKeyRestrictions extends AbstractPrimaryKeyRestrictions
+{
+    /**
+     * The composite type.
+     */
+    private final CType ctype;
+
+    /**
+     * The restrictions.
+     */
+    private final SingleColumnRestrictions restrictions;
+
+    /**
+     * <code>true</code> if the restrictions are corresponding to an EQ, <code>false</code> otherwise.
+     */
+    private boolean eq;
+
+    /**
+     * <code>true</code> if the restrictions are corresponding to an IN, <code>false</code> otherwise.
+     */
+    private boolean in;
+
+    /**
+     * <code>true</code> if the restrictions are corresponding to a Slice, <code>false</code> otherwise.
+     */
+    private boolean slice;
+
+    /**
+     * <code>true</code> if the restrictions are corresponding to a Contains, <code>false</code> otherwise.
+     */
+    private boolean contains;
+
+    public SingleColumnPrimaryKeyRestrictions(CType ctype)
+    {
+        this.ctype = ctype;
+        this.restrictions = new SingleColumnRestrictions();
+        this.eq = true;
+    }
+
+    private SingleColumnPrimaryKeyRestrictions(SingleColumnPrimaryKeyRestrictions primaryKeyRestrictions,
+                                               SingleColumnRestriction restriction) throws InvalidRequestException
+    {
+        this.restrictions = primaryKeyRestrictions.restrictions.addRestriction(restriction);
+        this.ctype = primaryKeyRestrictions.ctype;
+
+        if (!primaryKeyRestrictions.isEmpty())
+        {
+            ColumnDefinition lastColumn = primaryKeyRestrictions.restrictions.lastColumn();
+            ColumnDefinition newColumn = restriction.getColumnDef();
+
+            checkFalse(primaryKeyRestrictions.isSlice() && newColumn.position() > lastColumn.position(),
+                       "Clustering column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)",
+                       newColumn.name,
+                       lastColumn.name);
+
+            if (newColumn.position() < lastColumn.position())
+                checkFalse(restriction.isSlice(),
+                           "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)",
+                           restrictions.nextColumn(newColumn).name,
+                           newColumn.name);
+        }
+
+        if (restriction.isSlice() || primaryKeyRestrictions.isSlice())
+            this.slice = true;
+        else if (restriction.isContains() || primaryKeyRestrictions.isContains())
+            this.contains = true;
+        else if (restriction.isIN())
+            this.in = true;
+        else
+            this.eq = true;
+    }
+
+    @Override
+    public boolean isSlice()
+    {
+        return slice;
+    }
+
+    @Override
+    public boolean isEQ()
+    {
+        return eq;
+    }
+
+    @Override
+    public boolean isIN()
+    {
+        return in;
+    }
+
+    @Override
+    public boolean isOnToken()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isContains()
+    {
+        return contains;
+    }
+
+    @Override
+    public boolean isMultiColumn()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean usesFunction(String ksName, String functionName)
+    {
+        return restrictions.usesFunction(ksName, functionName);
+    }
+
+    @Override
+    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException
+    {
+        if (restriction.isMultiColumn())
+        {
+            checkTrue(isEmpty(),
+                      "Mixing single column relations and multi column relations on clustering columns is not allowed");
+            return (PrimaryKeyRestrictions) restriction;
+        }
+
+        if (restriction.isOnToken())
+        {
+            checkTrue(isEmpty(), "Columns \"%s\" cannot be restricted by both a normal relation and a token relation",
+                      ((TokenRestriction) restriction).getColumnNamesAsString());
+            return (PrimaryKeyRestrictions) restriction;
+        }
+
+        return new SingleColumnPrimaryKeyRestrictions(this, (SingleColumnRestriction) restriction);
+    }
+
+    @Override
+    public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
+    {
+        CompositesBuilder builder = new CompositesBuilder(ctype.builder(), ctype);
+        for (ColumnDefinition def : restrictions.getColumnDefs())
+        {
+            Restriction r = restrictions.getRestriction(def);
+            assert !r.isSlice();
+
+            List<ByteBuffer> values = r.values(options);
+
+            if (values.isEmpty())
+                return null;
+
+            builder.addEachElementToAll(values);
+            checkFalse(builder.containsNull(), "Invalid null value for column %s", def.name);
+        }
+
+        return builder.build();
+    }
+
+    @Override
+    public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
+    {
+        CBuilder builder = ctype.builder();
+        List<ColumnDefinition> defs = new ArrayList<>(restrictions.getColumnDefs());
+
+        CompositesBuilder compositeBuilder = new CompositesBuilder(builder, ctype);
+        // The end-of-component of composite doesn't depend on whether the
+        // component type is reversed or not (i.e. the ReversedType is applied
+        // to the component comparator but not to the end-of-component itself),
+        // it only depends on whether the slice is reversed
+        int keyPosition = 0;
+        for (ColumnDefinition def : defs)
+        {
+            // In a restriction, we always have Bound.START < Bound.END for the "base" comparator.
+            // So if we're doing a reverse slice, we must inverse the bounds when giving them as start and end of the slice filter.
+            // But if the actual comparator itself is reversed, we must inversed the bounds too.
+            Bound b = !def.isReversedType() ? bound : bound.reverse();
+            Restriction r = restrictions.getRestriction(def);
+            if (keyPosition != def.position() || r.isContains())
+            {
+                EOC eoc = !compositeBuilder.isEmpty() && bound.isEnd() ? EOC.END : EOC.NONE;
+                return compositeBuilder.buildWithEOC(eoc);
+            }
+            if (r.isSlice())
+            {
+                if (!r.hasBound(b))
+                {
+                    // There wasn't any non EQ relation on that key, we select all records having the preceding component as prefix.
+                    // For composites, if there was preceding component and we're computing the end, we must change the last component
+                    // End-Of-Component, otherwise we would be selecting only one record.
+                    EOC eoc = !compositeBuilder.isEmpty() && bound.isEnd() ? EOC.END : EOC.NONE;
+                    return compositeBuilder.buildWithEOC(eoc);
+                }
+
+                ByteBuffer value = checkNotNull(r.bounds(b, options).get(0), "Invalid null clustering key part %s", r);
+                compositeBuilder.addElementToAll(value);
+                Composite.EOC eoc = eocFor(r, bound, b);
+                return compositeBuilder.buildWithEOC(eoc);
+            }
+
+            compositeBuilder.addEachElementToAll(r.values(options));
+
+            checkFalse(compositeBuilder.containsNull(), "Invalid null clustering key part %s", def.name);
+            keyPosition++;
+        }
+        // Means no relation at all or everything was an equal
+        // Note: if the builder is "full", there is no need to use the end-of-component bit. For columns selection,
+        // it would be harmless to do it. However, we use this method got the partition key too. And when a query
+        // with 2ndary index is done, and with the the partition provided with an EQ, we'll end up here, and in that
+        // case using the eoc would be bad, since for the random partitioner we have no guarantee that
+        // prefix.end() will sort after prefix (see #5240).
+        EOC eoc = bound.isEnd() && compositeBuilder.hasRemaining() ? EOC.END : EOC.NONE;
+        return compositeBuilder.buildWithEOC(eoc);
+    }
+
+    @Override
+    public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+    {
+        return Composites.toByteBuffers(valuesAsComposites(options));
+    }
+
+    @Override
+    public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
+    {
+        return Composites.toByteBuffers(boundsAsComposites(b, options));
+    }
+
+    private static Composite.EOC eocFor(Restriction r, Bound eocBound, Bound inclusiveBound)
+    {
+        if (eocBound.isStart())
+            return r.isInclusive(inclusiveBound) ? Composite.EOC.NONE : Composite.EOC.END;
+
+        return r.isInclusive(inclusiveBound) ? Composite.EOC.END : Composite.EOC.START;
+    }
+
+    @Override
+    public boolean hasBound(Bound b)
+    {
+        if (isEmpty())
+            return false;
+        return restrictions.lastRestriction().hasBound(b);
+    }
+
+    @Override
+    public boolean isInclusive(Bound b)
+    {
+        if (isEmpty())
+            return false;
+        return restrictions.lastRestriction().isInclusive(b);
+    }
+
+    @Override
+    public boolean hasSupportingIndex(SecondaryIndexManager indexManager)
+    {
+        return restrictions.hasSupportingIndex(indexManager);
+    }
+
+    @Override
+    public void addIndexExpressionTo(List<IndexExpression> expressions, QueryOptions options) throws InvalidRequestException
+    {
+        restrictions.addIndexExpressionTo(expressions, options);
+    }
+
+    @Override
+    public Collection<ColumnDefinition> getColumnDefs()
+    {
+        return restrictions.getColumnDefs();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
new file mode 100644
index 0000000..0f0f9c8
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
@@ -0,0 +1,477 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.AbstractMarker;
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+public abstract class SingleColumnRestriction extends AbstractRestriction
+{
+    /**
+     * The definition of the column to which apply the restriction.
+     */
+    protected final ColumnDefinition columnDef;
+
+    public SingleColumnRestriction(ColumnDefinition columnDef)
+    {
+        this.columnDef = columnDef;
+    }
+
+    /**
+     * Returns the definition of the column to which is associated this restriction.
+     * @return the definition of the column to which is associated this restriction
+     */
+    public ColumnDefinition getColumnDef()
+    {
+        return columnDef;
+    }
+
+    @Override
+    public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                     QueryOptions options) throws InvalidRequestException
+    {
+        List<ByteBuffer> values = values(options);
+        checkTrue(values.size() == 1, "IN restrictions are not supported on indexed columns");
+
+        ByteBuffer value = validateIndexedValue(columnDef, values.get(0));
+        expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, value));
+    }
+
+    @Override
+    public boolean hasSupportingIndex(SecondaryIndexManager indexManager)
+    {
+        SecondaryIndex index = indexManager.getIndexForColumn(columnDef.name.bytes);
+        return index != null && isSupportedBy(index);
+    }
+
+    /**
+     * Check if this type of restriction is supported by the specified index.
+     *
+     * @param index the Secondary index
+     * @return <code>true</code> this type of restriction is supported by the specified index,
+     * <code>false</code> otherwise.
+     */
+    protected abstract boolean isSupportedBy(SecondaryIndex index);
+
+    public static final class EQ extends SingleColumnRestriction
+    {
+        private final Term value;
+
+        public EQ(ColumnDefinition columnDef, Term value)
+        {
+            super(columnDef);
+            this.value = value;
+        }
+
+        @Override
+        public boolean usesFunction(String ksName, String functionName)
+        {
+            return usesFunction(value, ksName, functionName);
+        }
+
+        public boolean isEQ()
+        {
+            return true;
+        }
+
+        @Override
+        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+        {
+            return Collections.singletonList(value.bindAndGet(options));
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("EQ(%s)", value);
+        }
+
+        @Override
+        public Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
+        {
+            throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal", columnDef.name);
+        }
+
+        @Override
+        protected boolean isSupportedBy(SecondaryIndex index)
+        {
+            return index.supportsOperator(Operator.EQ);
+        }
+    }
+
+    public static abstract class IN extends SingleColumnRestriction
+    {
+        public IN(ColumnDefinition columnDef)
+        {
+            super(columnDef);
+        }
+
+        @Override
+        public final boolean isIN()
+        {
+            return true;
+        }
+
+        @Override
+        public final Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
+        {
+            throw invalidRequest("%s cannot be restricted by more than one relation if it includes a IN", columnDef.name);
+        }
+
+        @Override
+        protected final boolean isSupportedBy(SecondaryIndex index)
+        {
+            return index.supportsOperator(Operator.IN);
+        }
+    }
+
+    public static class InWithValues extends IN
+    {
+        protected final List<Term> values;
+
+        public InWithValues(ColumnDefinition columnDef, List<Term> values)
+        {
+            super(columnDef);
+            this.values = values;
+        }
+
+        @Override
+        public boolean usesFunction(String ksName, String functionName)
+        {
+            return usesFunction(values, ksName, functionName);
+        }
+
+        @Override
+        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+        {
+            List<ByteBuffer> buffers = new ArrayList<>(values.size());
+            for (Term value : values)
+                buffers.add(value.bindAndGet(options));
+            return buffers;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("IN(%s)", values);
+        }
+    }
+
+    public static class InWithMarker extends IN
+    {
+        protected final AbstractMarker marker;
+
+        public InWithMarker(ColumnDefinition columnDef, AbstractMarker marker)
+        {
+            super(columnDef);
+            this.marker = marker;
+        }
+
+        @Override
+        public boolean usesFunction(String ksName, String functionName)
+        {
+            return false;
+        }
+
+        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+        {
+            Term.MultiItemTerminal lval = (Term.MultiItemTerminal) marker.bind(options);
+            if (lval == null)
+                throw new InvalidRequestException("Invalid null value for IN restriction");
+            return lval.getElements();
+        }
+
+        @Override
+        public String toString()
+        {
+            return "IN ?";
+        }
+    }
+
+    public static class Slice extends SingleColumnRestriction
+    {
+        private final TermSlice slice;
+
+        public Slice(ColumnDefinition columnDef, Bound bound, boolean inclusive, Term term)
+        {
+            super(columnDef);
+            slice = TermSlice.newInstance(bound, inclusive, term);
+        }
+
+        @Override
+        public boolean usesFunction(String ksName, String functionName)
+        {
+            return (slice.hasBound(Bound.START) && usesFunction(slice.bound(Bound.START), ksName, functionName))
+                    || (slice.hasBound(Bound.END) && usesFunction(slice.bound(Bound.END), ksName, functionName));
+        }
+
+        public boolean isSlice()
+        {
+            return true;
+        }
+
+        @Override
+        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean hasBound(Bound b)
+        {
+            return slice.hasBound(b);
+        }
+
+        @Override
+        public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
+        {
+            return Collections.singletonList(slice.bound(b).bindAndGet(options));
+        }
+
+        @Override
+        public boolean isInclusive(Bound b)
+        {
+            return slice.isInclusive(b);
+        }
+
+        @Override
+        public Restriction mergeWith(Restriction otherRestriction)
+        throws InvalidRequestException
+        {
+            checkTrue(otherRestriction.isSlice(),
+                      "Column \"%s\" cannot be restricted by both an equality and an inequality relation",
+                      columnDef.name);
+
+            SingleColumnRestriction.Slice otherSlice = (SingleColumnRestriction.Slice) otherRestriction;
+
+            checkFalse(hasBound(Bound.START) && otherSlice.hasBound(Bound.START),
+                       "More than one restriction was found for the start bound on %s", columnDef.name);
+
+            checkFalse(hasBound(Bound.END) && otherSlice.hasBound(Bound.END),
+                       "More than one restriction was found for the end bound on %s", columnDef.name);
+
+            return new Slice(columnDef,  slice.merge(otherSlice.slice));
+        }
+
+        @Override
+        public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                         QueryOptions options) throws InvalidRequestException
+        {
+            for (Bound b : Bound.values())
+            {
+                if (hasBound(b))
+                {
+                    ByteBuffer value = validateIndexedValue(columnDef, slice.bound(b).bindAndGet(options));
+                    Operator op = slice.getIndexOperator(b);
+                    // If the underlying comparator for name is reversed, we need to reverse the IndexOperator: user operation
+                    // always refer to the "forward" sorting even if the clustering order is reversed, but the 2ndary code does
+                    // use the underlying comparator as is.
+                    op = columnDef.isReversedType() ? op.reverse() : op;
+                    expressions.add(new IndexExpression(columnDef.name.bytes, op, value));
+                }
+            }
+        }
+
+        @Override
+        protected boolean isSupportedBy(SecondaryIndex index)
+        {
+            return slice.isSupportedBy(index);
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("SLICE%s", slice);
+        }
+
+        private Slice(ColumnDefinition columnDef, TermSlice slice)
+        {
+            super(columnDef);
+            this.slice = slice;
+        }
+    }
+
+    // This holds both CONTAINS and CONTAINS_KEY restriction because we might want to have both of them.
+    public static final class Contains extends SingleColumnRestriction
+    {
+        private List<Term> values = new ArrayList<>(); // for CONTAINS
+        private List<Term> keys = new ArrayList<>();  // for CONTAINS_KEY
+
+        public Contains(ColumnDefinition columnDef, Term t, boolean isKey)
+        {
+            super(columnDef);
+            if (isKey)
+                keys.add(t);
+            else
+                values.add(t);
+        }
+
+        @Override
+        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+        {
+            return bindAndGet(values, options);
+        }
+
+        @Override
+        public boolean isContains()
+        {
+            return true;
+        }
+
+        @Override
+        public Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
+        {
+            checkTrue(otherRestriction.isContains(),
+                      "Collection column %s can only be restricted by CONTAINS or CONTAINS KEY",
+                      getColumnDef().name);
+
+            SingleColumnRestriction.Contains newContains = new Contains(getColumnDef());
+
+            copyKeysAndValues(this, newContains);
+            copyKeysAndValues((Contains) otherRestriction, newContains);
+
+            return newContains;
+        }
+
+        @Override
+        public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                         QueryOptions options)
+                                         throws InvalidRequestException
+        {
+            for (ByteBuffer value : values(options))
+            {
+                validateIndexedValue(columnDef, value);
+                expressions.add(new IndexExpression(columnDef.name.bytes, Operator.CONTAINS, value));
+            }
+            for (ByteBuffer key : keys(options))
+            {
+                validateIndexedValue(columnDef, key);
+                expressions.add(new IndexExpression(columnDef.name.bytes, Operator.CONTAINS_KEY, key));
+            }
+        }
+
+        @Override
+        protected boolean isSupportedBy(SecondaryIndex index)
+        {
+            boolean supported = false;
+
+            if (numberOfValues() > 0)
+                supported |= index.supportsOperator(Operator.CONTAINS);
+
+            if (numberOfKeys() > 0)
+                supported |= index.supportsOperator(Operator.CONTAINS_KEY);
+
+            return supported;
+        }
+
+        public int numberOfValues()
+        {
+            return values.size();
+        }
+
+        public int numberOfKeys()
+        {
+            return keys.size();
+        }
+
+        @Override
+        public boolean usesFunction(String ksName, String functionName)
+        {
+            return usesFunction(values, ksName, functionName) || usesFunction(keys, ksName, functionName);
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("CONTAINS(values=%s, keys=%s)", values, keys);
+        }
+
+        @Override
+        public boolean hasBound(Bound b)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean isInclusive(Bound b)
+        {
+            throw new UnsupportedOperationException();
+        }
+        private List<ByteBuffer> keys(QueryOptions options) throws InvalidRequestException
+        {
+            return bindAndGet(keys, options);
+        }
+
+        /**
+         * Binds the query options to the specified terms and returns the resulting values.
+         *
+         * @param terms the terms
+         * @param options the query options
+         * @return the value resulting from binding the query options to the specified terms
+         * @throws InvalidRequestException if a problem occurs while binding the query options
+         */
+        private static List<ByteBuffer> bindAndGet(List<Term> terms, QueryOptions options) throws InvalidRequestException
+        {
+            List<ByteBuffer> buffers = new ArrayList<>(terms.size());
+            for (Term value : terms)
+                buffers.add(value.bindAndGet(options));
+            return buffers;
+        }
+
+        /**
+         * Copies the keys and value from the first <code>Contains</code> to the second one.
+         *
+         * @param from the <code>Contains</code> to copy from
+         * @param to the <code>Contains</code> to copy to
+         */
+        private static void copyKeysAndValues(Contains from, Contains to)
+        {
+            to.values.addAll(from.values);
+            to.keys.addAll(from.keys);
+        }
+
+        private Contains(ColumnDefinition columnDef)
+        {
+            super(columnDef);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestrictions.java
new file mode 100644
index 0000000..ec74cc9
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestrictions.java
@@ -0,0 +1,209 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.util.*;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction.Contains;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * Sets of single column restrictions.
+ */
+final class SingleColumnRestrictions implements Restrictions
+{
+    /**
+     * The comparator used to sort the <code>Restriction</code>s.
+     */
+    private static final Comparator<ColumnDefinition> COLUMN_DEFINITION_COMPARATOR = new Comparator<ColumnDefinition>()
+    {
+        @Override
+        public int compare(ColumnDefinition column, ColumnDefinition otherColumn)
+        {
+            int value = Integer.compare(column.position(), otherColumn.position());
+            return value != 0 ? value : column.name.bytes.compareTo(otherColumn.name.bytes);
+        }
+    };
+
+    /**
+     * The restrictions per column.
+     */
+    protected final TreeMap<ColumnDefinition, Restriction> restrictions;
+
+    public SingleColumnRestrictions()
+    {
+        this(new TreeMap<ColumnDefinition, Restriction>(COLUMN_DEFINITION_COMPARATOR));
+    }
+
+    protected SingleColumnRestrictions(TreeMap<ColumnDefinition, Restriction> restrictions)
+    {
+        this.restrictions = restrictions;
+    }
+
+    @Override
+    public final void addIndexExpressionTo(List<IndexExpression> expressions,
+                                           QueryOptions options) throws InvalidRequestException
+    {
+        for (Restriction restriction : restrictions.values())
+            restriction.addIndexExpressionTo(expressions, options);
+    }
+
+    @Override
+    public final Set<ColumnDefinition> getColumnDefs()
+    {
+        return restrictions.keySet();
+    }
+
+    /**
+     * Returns the restriction associated to the specified column.
+     *
+     * @param columnDef the column definition
+     * @return the restriction associated to the specified column
+     */
+    public Restriction getRestriction(ColumnDefinition columnDef)
+    {
+        return restrictions.get(columnDef);
+    }
+
+    @Override
+    public boolean usesFunction(String ksName, String functionName)
+    {
+        for (Restriction restriction : restrictions.values())
+            if (restriction.usesFunction(ksName, functionName))
+                return true;
+
+        return false;
+    }
+
+    @Override
+    public final boolean isEmpty()
+    {
+        return getColumnDefs().isEmpty();
+    }
+
+    @Override
+    public final int size()
+    {
+        return getColumnDefs().size();
+    }
+
+    /**
+     * Adds the specified restriction to this set of restrictions.
+     *
+     * @param restriction the restriction to add
+     * @return the new set of restrictions
+     * @throws InvalidRequestException if the new restriction cannot be added
+     */
+    public SingleColumnRestrictions addRestriction(SingleColumnRestriction restriction) throws InvalidRequestException
+    {
+        TreeMap<ColumnDefinition, Restriction> newRestrictions = new TreeMap<>(this.restrictions);
+        return new SingleColumnRestrictions(mergeRestrictions(newRestrictions, restriction));
+    }
+
+    private static TreeMap<ColumnDefinition, Restriction> mergeRestrictions(TreeMap<ColumnDefinition, Restriction> restrictions,
+                                                                            Restriction restriction)
+                                                                            throws InvalidRequestException
+    {
+        ColumnDefinition def = ((SingleColumnRestriction) restriction).getColumnDef();
+        Restriction existing = restrictions.get(def);
+        Restriction newRestriction = mergeRestrictions(existing, restriction);
+        restrictions.put(def, newRestriction);
+        return restrictions;
+    }
+
+    @Override
+    public final boolean hasSupportingIndex(SecondaryIndexManager indexManager)
+    {
+        for (Restriction restriction : restrictions.values())
+        {
+            if (restriction.hasSupportingIndex(indexManager))
+                return true;
+        }
+        return false;
+    }
+
+    /**
+     * Returns the column after the specified one.
+     *
+     * @param columnDef the column for which the next one need to be found
+     * @return the column after the specified one.
+     */
+    ColumnDefinition nextColumn(ColumnDefinition columnDef)
+    {
+        return restrictions.tailMap(columnDef, false).firstKey();
+    }
+
+    /**
+     * Returns the definition of the last column.
+     *
+     * @return the definition of the last column.
+     */
+    ColumnDefinition lastColumn()
+    {
+        return isEmpty() ? null : this.restrictions.lastKey();
+    }
+
+    /**
+     * Returns the last restriction.
+     *
+     * @return the last restriction.
+     */
+    Restriction lastRestriction()
+    {
+        return isEmpty() ? null : this.restrictions.lastEntry().getValue();
+    }
+
+    /**
+     * Merges the two specified restrictions.
+     *
+     * @param restriction the first restriction
+     * @param otherRestriction the second restriction
+     * @return the merged restriction
+     * @throws InvalidRequestException if the two restrictions cannot be merged
+     */
+    private static Restriction mergeRestrictions(Restriction restriction,
+                                                 Restriction otherRestriction) throws InvalidRequestException
+    {
+        return restriction == null ? otherRestriction
+                                   : restriction.mergeWith(otherRestriction);
+    }
+
+    /**
+     * Checks if the restrictions contains multiple contains or contains key.
+     *
+     * @return <code>true</code> if the restrictions contains multiple contains or contains key.,
+     * <code>false</code> otherwise
+     */
+    public final boolean hasMultipleContains()
+    {
+        int numberOfContains = 0;
+        for (Restriction restriction : restrictions.values())
+        {
+            if (restriction.isContains())
+            {
+                Contains contains = (Contains) restriction;
+                numberOfContains += (contains.numberOfValues() + contains.numberOfKeys());
+            }
+        }
+        return numberOfContains > 1;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
new file mode 100644
index 0000000..60c7465
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
@@ -0,0 +1,600 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.base.Joiner;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Relation;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.dht.*;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * The restrictions corresponding to the relations specified on the where-clause of CQL query.
+ */
+public final class StatementRestrictions
+{
+    /**
+     * The Column Family meta data
+     */
+    public final CFMetaData cfm;
+
+    /**
+     * Restrictions on partitioning columns
+     */
+    private PrimaryKeyRestrictions partitionKeyRestrictions;
+
+    /**
+     * Restrictions on clustering columns
+     */
+    private PrimaryKeyRestrictions clusteringColumnsRestrictions;
+
+    /**
+     * Restriction on non-primary key columns (i.e. secondary index restrictions)
+     */
+    private SingleColumnRestrictions nonPrimaryKeyRestrictions;
+
+    /**
+     * The restrictions used to build the index expressions
+     */
+    private final List<Restrictions> indexRestrictions = new ArrayList<>();
+
+    /**
+     * <code>true</code> if the secondary index need to be queried, <code>false</code> otherwise
+     */
+    private boolean usesSecondaryIndexing;
+
+    /**
+     * Specify if the query will return a range of partition keys.
+     */
+    private boolean isKeyRange;
+
+    /**
+     * Creates a new empty <code>StatementRestrictions</code>.
+     *
+     * @param cfm the column family meta data
+     * @return a new empty <code>StatementRestrictions</code>.
+     */
+    public static StatementRestrictions empty(CFMetaData cfm)
+    {
+        return new StatementRestrictions(cfm);
+    }
+
+    private StatementRestrictions(CFMetaData cfm)
+    {
+        this.cfm = cfm;
+        this.partitionKeyRestrictions = new SingleColumnPrimaryKeyRestrictions(cfm.getKeyValidatorAsCType());
+        this.clusteringColumnsRestrictions = new SingleColumnPrimaryKeyRestrictions(cfm.comparator);
+        this.nonPrimaryKeyRestrictions = new SingleColumnRestrictions();
+    }
+
+    public StatementRestrictions(CFMetaData cfm,
+            List<Relation> whereClause,
+            VariableSpecifications boundNames,
+            boolean selectsOnlyStaticColumns,
+            boolean selectACollection) throws InvalidRequestException
+    {
+        this.cfm = cfm;
+        this.partitionKeyRestrictions = new SingleColumnPrimaryKeyRestrictions(cfm.getKeyValidatorAsCType());
+        this.clusteringColumnsRestrictions = new SingleColumnPrimaryKeyRestrictions(cfm.comparator);
+        this.nonPrimaryKeyRestrictions = new SingleColumnRestrictions();
+
+        /*
+         * WHERE clause. For a given entity, rules are: - EQ relation conflicts with anything else (including a 2nd EQ)
+         * - Can't have more than one LT(E) relation (resp. GT(E) relation) - IN relation are restricted to row keys
+         * (for now) and conflicts with anything else (we could allow two IN for the same entity but that doesn't seem
+         * very useful) - The value_alias cannot be restricted in any way (we don't support wide rows with indexed value
+         * in CQL so far)
+         */
+        for (Relation relation : whereClause)
+            addRestriction(relation.toRestriction(cfm, boundNames));
+
+        ColumnFamilyStore cfs = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
+        SecondaryIndexManager secondaryIndexManager = cfs.indexManager;
+
+        boolean hasQueriableClusteringColumnIndex = clusteringColumnsRestrictions.hasSupportingIndex(secondaryIndexManager);
+        boolean hasQueriableIndex = hasQueriableClusteringColumnIndex
+                || partitionKeyRestrictions.hasSupportingIndex(secondaryIndexManager)
+                || nonPrimaryKeyRestrictions.hasSupportingIndex(secondaryIndexManager);
+
+        // At this point, the select statement if fully constructed, but we still have a few things to validate
+        processPartitionKeyRestrictions(hasQueriableIndex);
+
+        // Some but not all of the partition key columns have been specified;
+        // hence we need turn these restrictions into index expressions.
+        if (usesSecondaryIndexing)
+            indexRestrictions.add(partitionKeyRestrictions);
+
+        checkFalse(selectsOnlyStaticColumns && hasClusteringColumnsRestriction(),
+                   "Cannot restrict clustering columns when selecting only static columns");
+
+        processClusteringColumnsRestrictions(hasQueriableIndex, selectACollection);
+
+        // Covers indexes on the first clustering column (among others).
+        if (isKeyRange && hasQueriableClusteringColumnIndex)
+            usesSecondaryIndexing = true;
+
+        if (usesSecondaryIndexing)
+        {
+            indexRestrictions.add(clusteringColumnsRestrictions);
+        }
+        else if (clusteringColumnsRestrictions.isContains())
+        {
+            indexRestrictions.add(new ForwardingPrimaryKeyRestrictions() {
+
+                @Override
+                protected PrimaryKeyRestrictions getDelegate()
+                {
+                    return clusteringColumnsRestrictions;
+                }
+
+                @Override
+                public void addIndexExpressionTo(List<IndexExpression> expressions, QueryOptions options) throws InvalidRequestException
+                {
+                    List<IndexExpression> list = new ArrayList<>();
+                    super.addIndexExpressionTo(list, options);
+
+                    for (IndexExpression expression : list)
+                    {
+                        if (expression.isContains() || expression.isContainsKey())
+                            expressions.add(expression);
+                    }
+                }
+            });
+            usesSecondaryIndexing = true;
+        }
+        // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
+        // there is restrictions not covered by the PK.
+        if (!nonPrimaryKeyRestrictions.isEmpty())
+        {
+            usesSecondaryIndexing = true;
+            indexRestrictions.add(nonPrimaryKeyRestrictions);
+        }
+
+        if (usesSecondaryIndexing)
+            validateSecondaryIndexSelections(selectsOnlyStaticColumns);
+    }
+
+    private void addRestriction(Restriction restriction) throws InvalidRequestException
+    {
+        if (restriction.isMultiColumn())
+            clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction);
+        else if (restriction.isOnToken())
+            partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
+        else
+            addSingleColumnRestriction((SingleColumnRestriction) restriction);
+    }
+
+    public boolean usesFunction(String ksName, String functionName)
+    {
+        return  partitionKeyRestrictions.usesFunction(ksName, functionName)
+                || clusteringColumnsRestrictions.usesFunction(ksName, functionName)
+                || nonPrimaryKeyRestrictions.usesFunction(ksName, functionName);
+    }
+
+    private void addSingleColumnRestriction(SingleColumnRestriction restriction) throws InvalidRequestException
+    {
+        ColumnDefinition def = restriction.getColumnDef();
+        if (def.isPartitionKey())
+            partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
+        else if (def.isClusteringColumn())
+            clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction);
+        else
+            nonPrimaryKeyRestrictions = nonPrimaryKeyRestrictions.addRestriction(restriction);
+    }
+
+    /**
+     * Checks if the restrictions on the partition key is an IN restriction.
+     *
+     * @return <code>true</code> the restrictions on the partition key is an IN restriction, <code>false</code>
+     * otherwise.
+     */
+    public boolean keyIsInRelation()
+    {
+        return partitionKeyRestrictions.isIN();
+    }
+
+    /**
+     * Checks if the query request a range of partition keys.
+     *
+     * @return <code>true</code> if the query request a range of partition keys, <code>false</code> otherwise.
+     */
+    public boolean isKeyRange()
+    {
+        return this.isKeyRange;
+    }
+
+    /**
+     * Checks if the secondary index need to be queried.
+     *
+     * @return <code>true</code> if the secondary index need to be queried, <code>false</code> otherwise.
+     */
+    public boolean usesSecondaryIndexing()
+    {
+        return this.usesSecondaryIndexing;
+    }
+
+    private void processPartitionKeyRestrictions(boolean hasQueriableIndex) throws InvalidRequestException
+    {
+        // If there is a queriable index, no special condition are required on the other restrictions.
+        // But we still need to know 2 things:
+        // - If we don't have a queriable index, is the query ok
+        // - Is it queriable without 2ndary index, which is always more efficient
+        // If a component of the partition key is restricted by a relation, all preceding
+        // components must have a EQ. Only the last partition key component can be in IN relation.
+        if (partitionKeyRestrictions.isOnToken())
+        {
+            isKeyRange = true;
+        }
+        else if (hasPartitionKeyUnrestrictedComponents())
+        {
+            if (!partitionKeyRestrictions.isEmpty())
+            {
+                if (!hasQueriableIndex)
+                    throw invalidRequest("Partition key parts: %s must be restricted as other parts are",
+                                         Joiner.on(", ").join(getPartitionKeyUnrestrictedComponents()));
+            }
+
+            isKeyRange = true;
+            usesSecondaryIndexing = hasQueriableIndex;
+        }
+    }
+
+    /**
+     * Checks if the partition key has some unrestricted components.
+     * @return <code>true</code> if the partition key has some unrestricted components, <code>false</code> otherwise.
+     */
+    private boolean hasPartitionKeyUnrestrictedComponents()
+    {
+        return partitionKeyRestrictions.size() <  cfm.partitionKeyColumns().size();
+    }
+
+    /**
+     * Returns the partition key components that are not restricted.
+     * @return the partition key components that are not restricted.
+     */
+    private List<ColumnIdentifier> getPartitionKeyUnrestrictedComponents()
+    {
+        List<ColumnDefinition> list = new ArrayList<>(cfm.partitionKeyColumns());
+        list.removeAll(partitionKeyRestrictions.getColumnDefs());
+        return ColumnDefinition.toIdentifiers(list);
+    }
+
+    /**
+     * Processes the clustering column restrictions.
+     *
+     * @param hasQueriableIndex <code>true</code> if some of the queried data are indexed, <code>false</code> otherwise
+     * @param selectACollection <code>true</code> if the query should return a collection column
+     * @throws InvalidRequestException if the request is invalid
+     */
+    private void processClusteringColumnsRestrictions(boolean hasQueriableIndex,
+                                                      boolean selectACollection) throws InvalidRequestException
+    {
+        checkFalse(clusteringColumnsRestrictions.isIN() && selectACollection,
+                   "Cannot restrict clustering columns by IN relations when a collection is selected by the query");
+        checkFalse(clusteringColumnsRestrictions.isContains() && !hasQueriableIndex,
+                   "Cannot restrict clustering columns by a CONTAINS relation without a secondary index");
+
+        if (hasClusteringColumnsRestriction())
+        {
+            List<ColumnDefinition> clusteringColumns = cfm.clusteringColumns();
+            List<ColumnDefinition> restrictedColumns = new LinkedList<>(clusteringColumnsRestrictions.getColumnDefs());
+
+            for (int i = 0, m = restrictedColumns.size(); i < m; i++)
+            {
+                ColumnDefinition clusteringColumn = clusteringColumns.get(i);
+                ColumnDefinition restrictedColumn = restrictedColumns.get(i);
+
+                if (!clusteringColumn.equals(restrictedColumn))
+                {
+                    checkTrue(hasQueriableIndex,
+                              "PRIMARY KEY column \"%s\" cannot be restricted as preceding column \"%s\" is not restricted",
+                              restrictedColumn.name,
+                              clusteringColumn.name);
+
+                    usesSecondaryIndexing = true; // handle gaps and non-keyrange cases.
+                    break;
+                }
+            }
+        }
+
+        if (clusteringColumnsRestrictions.isContains())
+            usesSecondaryIndexing = true;
+    }
+
+    public List<IndexExpression> getIndexExpressions(QueryOptions options) throws InvalidRequestException
+    {
+        if (!usesSecondaryIndexing || indexRestrictions.isEmpty())
+            return Collections.emptyList();
+
+        List<IndexExpression> expressions = new ArrayList<>();
+        for (Restrictions restrictions : indexRestrictions)
+            restrictions.addIndexExpressionTo(expressions, options);
+
+        return expressions;
+    }
+
+    /**
+     * Returns the partition keys for which the data is requested.
+     *
+     * @param options the query options
+     * @return the partition keys for which the data is requested.
+     * @throws InvalidRequestException if the partition keys cannot be retrieved
+     */
+    public Collection<ByteBuffer> getPartitionKeys(final QueryOptions options) throws InvalidRequestException
+    {
+        return partitionKeyRestrictions.values(options);
+    }
+
+    /**
+     * Returns the specified bound of the partition key.
+     *
+     * @param b the boundary type
+     * @param options the query options
+     * @return the specified bound of the partition key
+     * @throws InvalidRequestException if the boundary cannot be retrieved
+     */
+    private ByteBuffer getPartitionKeyBound(Bound b, QueryOptions options) throws InvalidRequestException
+    {
+        // Deal with unrestricted partition key components (special-casing is required to deal with 2i queries on the
+        // first
+        // component of a composite partition key).
+        if (hasPartitionKeyUnrestrictedComponents())
+            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+
+        // We deal with IN queries for keys in other places, so we know buildBound will return only one result
+        return partitionKeyRestrictions.bounds(b, options).get(0);
+    }
+
+    /**
+     * Returns the partition key bounds.
+     *
+     * @param options the query options
+     * @return the partition key bounds
+     * @throws InvalidRequestException if the query is invalid
+     */
+    public AbstractBounds<RowPosition> getPartitionKeyBounds(QueryOptions options) throws InvalidRequestException
+    {
+        IPartitioner p = StorageService.getPartitioner();
+
+        if (partitionKeyRestrictions.isOnToken())
+        {
+            return getPartitionKeyBoundsForTokenRestrictions(p, options);
+        }
+
+        return getPartitionKeyBounds(p, options);
+    }
+
+    private AbstractBounds<RowPosition> getPartitionKeyBounds(IPartitioner p,
+                                                              QueryOptions options) throws InvalidRequestException
+    {
+        ByteBuffer startKeyBytes = getPartitionKeyBound(Bound.START, options);
+        ByteBuffer finishKeyBytes = getPartitionKeyBound(Bound.END, options);
+
+        RowPosition startKey = RowPosition.ForKey.get(startKeyBytes, p);
+        RowPosition finishKey = RowPosition.ForKey.get(finishKeyBytes, p);
+
+        if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum())
+            return null;
+
+        if (partitionKeyRestrictions.isInclusive(Bound.START))
+        {
+            return partitionKeyRestrictions.isInclusive(Bound.END)
+                    ? new Bounds<>(startKey, finishKey)
+                    : new IncludingExcludingBounds<>(startKey, finishKey);
+        }
+
+        return partitionKeyRestrictions.isInclusive(Bound.END)
+                ? new Range<>(startKey, finishKey)
+                : new ExcludingBounds<>(startKey, finishKey);
+    }
+
+    private AbstractBounds<RowPosition> getPartitionKeyBoundsForTokenRestrictions(IPartitioner p,
+                                                                                  QueryOptions options)
+                                                                                          throws InvalidRequestException
+    {
+        Token startToken = getTokenBound(Bound.START, options, p);
+        Token endToken = getTokenBound(Bound.END, options, p);
+
+        boolean includeStart = partitionKeyRestrictions.isInclusive(Bound.START);
+        boolean includeEnd = partitionKeyRestrictions.isInclusive(Bound.END);
+
+        /*
+         * If we ask SP.getRangeSlice() for (token(200), token(200)], it will happily return the whole ring.
+         * However, wrapping range doesn't really make sense for CQL, and we want to return an empty result in that
+         * case (CASSANDRA-5573). So special case to create a range that is guaranteed to be empty.
+         *
+         * In practice, we want to return an empty result set if either startToken > endToken, or both are equal but
+         * one of the bound is excluded (since [a, a] can contains something, but not (a, a], [a, a) or (a, a)).
+         * Note though that in the case where startToken or endToken is the minimum token, then this special case
+         * rule should not apply.
+         */
+        int cmp = startToken.compareTo(endToken);
+        if (!startToken.isMinimum() && !endToken.isMinimum()
+                && (cmp > 0 || (cmp == 0 && (!includeStart || !includeEnd))))
+            return null;
+
+        RowPosition start = includeStart ? startToken.minKeyBound() : startToken.maxKeyBound();
+        RowPosition end = includeEnd ? endToken.maxKeyBound() : endToken.minKeyBound();
+
+        return new Range<>(start, end);
+    }
+
+    private Token getTokenBound(Bound b, QueryOptions options, IPartitioner p) throws InvalidRequestException
+    {
+        if (!partitionKeyRestrictions.hasBound(b))
+            return p.getMinimumToken();
+
+        ByteBuffer value = partitionKeyRestrictions.bounds(b, options).get(0);
+        checkNotNull(value, "Invalid null token value");
+        return p.getTokenFactory().fromByteArray(value);
+    }
+
+    /**
+     * Checks if the query does not contains any restriction on the clustering columns.
+     *
+     * @return <code>true</code> if the query does not contains any restriction on the clustering columns,
+     * <code>false</code> otherwise.
+     */
+    public boolean hasNoClusteringColumnsRestriction()
+    {
+        return clusteringColumnsRestrictions.isEmpty();
+    }
+
+    // For non-composite slices, we don't support internally the difference between exclusive and
+    // inclusive bounds, so we deal with it manually.
+    public boolean isNonCompositeSliceWithExclusiveBounds()
+    {
+        return !cfm.comparator.isCompound()
+                && clusteringColumnsRestrictions.isSlice()
+                && (!clusteringColumnsRestrictions.isInclusive(Bound.START) || !clusteringColumnsRestrictions.isInclusive(Bound.END));
+    }
+
+    /**
+     * Returns the requested clustering columns as <code>Composite</code>s.
+     *
+     * @param options the query options
+     * @return the requested clustering columns as <code>Composite</code>s
+     * @throws InvalidRequestException if the query is not valid
+     */
+    public List<Composite> getClusteringColumnsAsComposites(QueryOptions options) throws InvalidRequestException
+    {
+        return clusteringColumnsRestrictions.valuesAsComposites(options);
+    }
+
+    /**
+     * Returns the bounds (start or end) of the clustering columns as <code>Composites</code>.
+     *
+     * @param b the bound type
+     * @param options the query options
+     * @return the bounds (start or end) of the clustering columns as <code>Composites</code>
+     * @throws InvalidRequestException if the request is not valid
+     */
+    public List<Composite> getClusteringColumnsBoundsAsComposites(Bound b,
+                                                                  QueryOptions options) throws InvalidRequestException
+    {
+        return clusteringColumnsRestrictions.boundsAsComposites(b, options);
+    }
+
+    /**
+     * Returns the bounds (start or end) of the clustering columns.
+     *
+     * @param b the bound type
+     * @param options the query options
+     * @return the bounds (start or end) of the clustering columns
+     * @throws InvalidRequestException if the request is not valid
+     */
+    public List<ByteBuffer> getClusteringColumnsBounds(Bound b, QueryOptions options) throws InvalidRequestException
+    {
+        return clusteringColumnsRestrictions.bounds(b, options);
+    }
+
+    /**
+     * Checks if the bounds (start or end) of the clustering columns are inclusive.
+     *
+     * @param bound the bound type
+     * @return <code>true</code> if the bounds (start or end) of the clustering columns are inclusive,
+     * <code>false</code> otherwise
+     */
+    public boolean areRequestedBoundsInclusive(Bound bound)
+    {
+        return clusteringColumnsRestrictions.isInclusive(bound);
+    }
+
+    /**
+     * Checks if the query returns a range of columns.
+     *
+     * @return <code>true</code> if the query returns a range of columns, <code>false</code> otherwise.
+     */
+    public boolean isColumnRange()
+    {
+        // Due to CASSANDRA-5762, we always do a slice for CQL3 tables (not dense, composite).
+        // Static CF (non dense but non composite) never entails a column slice however
+        if (!cfm.comparator.isDense())
+            return cfm.comparator.isCompound();
+
+        // Otherwise (i.e. for compact table where we don't have a row marker anyway and thus don't care about
+        // CASSANDRA-5762),
+        // it is a range query if it has at least one the column alias for which no relation is defined or is not EQ.
+        return clusteringColumnsRestrictions.size() < cfm.clusteringColumns().size() || clusteringColumnsRestrictions.isSlice();
+    }
+
+    /**
+     * Checks if the query need to use filtering.
+     * @return <code>true</code> if the query need to use filtering, <code>false</code> otherwise.
+     */
+    public boolean needFiltering()
+    {
+        int numberOfRestrictedColumns = 0;
+        for (Restrictions restrictions : indexRestrictions)
+            numberOfRestrictedColumns += restrictions.size();
+
+        return numberOfRestrictedColumns > 1
+                || (numberOfRestrictedColumns == 0 && !clusteringColumnsRestrictions.isEmpty())
+                || (numberOfRestrictedColumns != 0
+                        && nonPrimaryKeyRestrictions.hasMultipleContains());
+    }
+
+    private void validateSecondaryIndexSelections(boolean selectsOnlyStaticColumns) throws InvalidRequestException
+    {
+        checkFalse(keyIsInRelation(),
+                   "Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
+        // When the user only select static columns, the intent is that we don't query the whole partition but just
+        // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on
+        // static columns
+        // so far, 2i means that you've restricted a non static column, so the query is somewhat non-sensical.
+        checkFalse(selectsOnlyStaticColumns, "Queries using 2ndary indexes don't support selecting only static columns");
+    }
+
+    /**
+     * Checks if the query has some restrictions on the clustering columns.
+     *
+     * @return <code>true</code> if the query has some restrictions on the clustering columns,
+     * <code>false</code> otherwise.
+     */
+    private boolean hasClusteringColumnsRestriction()
+    {
+        return !clusteringColumnsRestrictions.isEmpty();
+    }
+
+    public void reverse()
+    {
+        clusteringColumnsRestrictions = new ReversedPrimaryKeyRestrictions(clusteringColumnsRestrictions);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/restrictions/TermSlice.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/TermSlice.java b/src/java/org/apache/cassandra/cql3/restrictions/TermSlice.java
new file mode 100644
index 0000000..3622220
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/TermSlice.java
@@ -0,0 +1,167 @@
+/*
+ * 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.cql3.restrictions;
+
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.index.SecondaryIndex;
+
+final class TermSlice
+{
+    /**
+     * The slice boundaries.
+     */
+    private final Term[] bounds;
+
+    /**
+     * Specifies if a slice boundary is inclusive or not.
+     */
+    private final boolean[] boundInclusive;
+
+    /**
+     * Creates a new <code>TermSlice</code> with the specified boundaries.
+     *
+     * @param start the lower boundary
+     * @param includeStart <code>true</code> if the lower boundary is inclusive
+     * @param end the upper boundary
+     * @param includeEnd <code>true</code> if the upper boundary is inclusive
+     */
+    private TermSlice(Term start, boolean includeStart, Term end, boolean includeEnd)
+    {
+        bounds = new Term[]{start, end};
+        boundInclusive = new boolean[]{includeStart, includeEnd};
+    }
+
+    /**
+     * Creates a new <code>TermSlice</code> with the specified boundary.
+     *
+     * @param bound the boundary type
+     * @param include <code>true</code> if the boundary is inclusive
+     * @param term the value
+     * @return a new <code>TermSlice</code> instance
+     */
+    public static TermSlice newInstance(Bound bound, boolean include, Term term)
+    {
+        return  bound.isStart() ? new TermSlice(term, include, null, false) 
+                                : new TermSlice(null, false, term, include);
+    }
+
+    /**
+     * Returns the boundary value.
+     *
+     * @param bound the boundary type
+     * @return the boundary value
+     */
+    public Term bound(Bound bound)
+    {
+        return bounds[bound.idx];
+    }
+
+    /**
+     * Checks if this slice has a boundary for the specified type.
+     *
+     * @param b the boundary type
+     * @return <code>true</code> if this slice has a boundary for the specified type, <code>false</code> otherwise.
+     */
+    public boolean hasBound(Bound b)
+    {
+        return bounds[b.idx] != null;
+    }
+
+    /**
+     * Checks if this slice boundary is inclusive for the specified type.
+     *
+     * @param b the boundary type
+     * @return <code>true</code> if this slice boundary is inclusive for the specified type,
+     * <code>false</code> otherwise.
+     */
+    public boolean isInclusive(Bound b)
+    {
+        return bounds[b.idx] == null || boundInclusive[b.idx];
+    }
+
+    /**
+     * Merges this slice with the specified one.
+     *
+     * @param otherSlice the slice to merge to
+     * @return the new slice resulting from the merge
+     */
+    public TermSlice merge(TermSlice otherSlice)
+    {
+        if (hasBound(Bound.START))
+        {
+            assert !otherSlice.hasBound(Bound.START);
+
+            return new TermSlice(bound(Bound.START), 
+                                  isInclusive(Bound.START),
+                                  otherSlice.bound(Bound.END),
+                                  otherSlice.isInclusive(Bound.END));
+        }
+        assert !otherSlice.hasBound(Bound.END);
+
+        return new TermSlice(otherSlice.bound(Bound.START), 
+                              otherSlice.isInclusive(Bound.START),
+                              bound(Bound.END),
+                              isInclusive(Bound.END));
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("(%s %s, %s %s)", boundInclusive[0] ? ">=" : ">",
+                             bounds[0],
+                             boundInclusive[1] ? "<=" : "<",
+                             bounds[1]);
+    }
+
+    /**
+     * Returns the index operator corresponding to the specified boundary.
+     *
+     * @param b the boundary type
+     * @return the index operator corresponding to the specified boundary
+     */
+    public Operator getIndexOperator(Bound b)
+    {
+        if (b.isStart())
+            return boundInclusive[b.idx] ? Operator.GTE : Operator.GT;
+
+        return boundInclusive[b.idx] ? Operator.LTE : Operator.LT;
+    }
+
+    /**
+     * Check if this <code>TermSlice</code> is supported by the specified index.
+     *
+     * @param index the Secondary index
+     * @return <code>true</code> this type of <code>TermSlice</code> is supported by the specified index,
+     * <code>false</code> otherwise.
+     */
+    public boolean isSupportedBy(SecondaryIndex index)
+    {
+        boolean supported = false;
+
+        if (hasBound(Bound.START))
+            supported |= isInclusive(Bound.START) ? index.supportsOperator(Operator.GTE)
+                    : index.supportsOperator(Operator.GT);
+        if (hasBound(Bound.END))
+            supported |= isInclusive(Bound.END) ? index.supportsOperator(Operator.LTE)
+                    : index.supportsOperator(Operator.LT);
+
+        return supported;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
new file mode 100644
index 0000000..85d614e
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
@@ -0,0 +1,224 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.base.Joiner;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * <code>Restriction</code> using the token function.
+ */
+public abstract class TokenRestriction extends AbstractPrimaryKeyRestrictions
+{
+    /**
+     * The definition of the columns to which apply the token restriction.
+     */
+    protected final List<ColumnDefinition> columnDefs;
+
+    /**
+     * Creates a new <code>TokenRestriction</code> that apply to the specified columns.
+     *
+     * @param columnDefs the definition of the columns to which apply the token restriction
+     */
+    public TokenRestriction(List<ColumnDefinition> columnDefs)
+    {
+        this.columnDefs = columnDefs;
+    }
+
+    @Override
+    public  boolean isOnToken()
+    {
+        return true;
+    }
+
+    @Override
+    public Collection<ColumnDefinition> getColumnDefs()
+    {
+        return columnDefs;
+    }
+
+    @Override
+    public boolean hasSupportingIndex(SecondaryIndexManager secondaryIndexManager)
+    {
+        return false;
+    }
+
+    @Override
+    public void addIndexExpressionTo(List<IndexExpression> expressions, QueryOptions options)
+    {
+        throw new UnsupportedOperationException("Index expression cannot be created for token restriction");
+    }
+
+    @Override
+    public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Returns the column names as a comma separated <code>String</code>.
+     *
+     * @return the column names as a comma separated <code>String</code>.
+     */
+    protected final String getColumnNamesAsString()
+    {
+        return Joiner.on(", ").join(ColumnDefinition.toIdentifiers(columnDefs));
+    }
+
+    public static final class EQ extends TokenRestriction
+    {
+        private final Term value;
+
+        public EQ(List<ColumnDefinition> columnDefs, Term value)
+        {
+            super(columnDefs);
+            this.value = value;
+        }
+
+        @Override
+        public boolean isEQ()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean usesFunction(String ksName, String functionName)
+        {
+            return usesFunction(value, ksName, functionName);
+        }
+
+        @Override
+        public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException
+        {
+            throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal",
+                                 Joiner.on(", ").join(ColumnDefinition.toIdentifiers(columnDefs)));
+        }
+
+        @Override
+        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+        {
+            return Collections.singletonList(value.bindAndGet(options));
+        }
+    }
+
+    public static class Slice extends TokenRestriction
+    {
+        private final TermSlice slice;
+
+        public Slice(List<ColumnDefinition> columnDefs, Bound bound, boolean inclusive, Term term)
+        {
+            super(columnDefs);
+            slice = TermSlice.newInstance(bound, inclusive, term);
+        }
+
+        @Override
+        public boolean isSlice()
+        {
+            return true;
+        }
+
+        @Override
+        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean hasBound(Bound b)
+        {
+            return slice.hasBound(b);
+        }
+
+        @Override
+        public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
+        {
+            return Collections.singletonList(slice.bound(b).bindAndGet(options));
+        }
+
+        @Override
+        public boolean usesFunction(String ksName, String functionName)
+        {
+            return (slice.hasBound(Bound.START) && usesFunction(slice.bound(Bound.START), ksName, functionName))
+                    || (slice.hasBound(Bound.END) && usesFunction(slice.bound(Bound.END), ksName, functionName));
+        }
+
+        @Override
+        public boolean isInclusive(Bound b)
+        {
+            return slice.isInclusive(b);
+        }
+
+        @Override
+        public PrimaryKeyRestrictions mergeWith(Restriction otherRestriction)
+        throws InvalidRequestException
+        {
+            if (!otherRestriction.isOnToken())
+                throw invalidRequest("Columns \"%s\" cannot be restricted by both a normal relation and a token relation",
+                                     getColumnNamesAsString());
+
+            if (!otherRestriction.isSlice())
+                throw invalidRequest("Columns \"%s\" cannot be restricted by both an equality and an inequality relation",
+                                     getColumnNamesAsString());
+
+            TokenRestriction.Slice otherSlice = (TokenRestriction.Slice) otherRestriction;
+
+            if (hasBound(Bound.START) && otherSlice.hasBound(Bound.START))
+                throw invalidRequest("More than one restriction was found for the start bound on %s",
+                                     getColumnNamesAsString());
+
+            if (hasBound(Bound.END) && otherSlice.hasBound(Bound.END))
+                throw invalidRequest("More than one restriction was found for the end bound on %s",
+                                     getColumnNamesAsString());
+
+            return new Slice(columnDefs,  slice.merge(otherSlice.slice));
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("SLICE%s", slice);
+        }
+
+        private Slice(List<ColumnDefinition> columnDefs, TermSlice slice)
+        {
+            super(columnDefs);
+            this.slice = slice;
+        }
+    }
+}


[3/5] cassandra git commit: Refactor SelectStatement and Restrictions

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/selection/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selection.java b/src/java/org/apache/cassandra/cql3/selection/Selection.java
index 6ad36e9..e44a39f 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java
@@ -35,17 +35,36 @@ import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 
 public abstract class Selection
 {
+    /**
+     * A predicate that returns <code>true</code> for static columns.
+     */
+    private static final Predicate<ColumnDefinition> STATIC_COLUMN_FILTER = new Predicate<ColumnDefinition>()
+    {
+        public boolean apply(ColumnDefinition def)
+        {
+            return def.isStatic();
+        }
+    };
+
+    private final CFMetaData cfm;
     private final Collection<ColumnDefinition> columns;
     private final ResultSet.Metadata metadata;
     private final boolean collectTimestamps;
     private final boolean collectTTLs;
 
-    protected Selection(Collection<ColumnDefinition> columns, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs)
+    protected Selection(CFMetaData cfm,
+                        Collection<ColumnDefinition> columns,
+                        List<ColumnSpecification> metadata,
+                        boolean collectTimestamps,
+                        boolean collectTTLs)
     {
+        this.cfm = cfm;
         this.columns = columns;
         this.metadata = new ResultSet.Metadata(metadata);
         this.collectTimestamps = collectTimestamps;
@@ -56,6 +75,76 @@ public abstract class Selection
     public boolean isWildcard()
     {
         return false;
+    }    
+
+    /**
+     * Checks if this selection contains static columns.
+     * @return <code>true</code> if this selection contains static columns, <code>false</code> otherwise;
+     */
+    public boolean containsStaticColumns()
+    {
+        if (!cfm.hasStaticColumns())
+            return false;
+
+        if (isWildcard())
+            return true;
+
+        return !Iterables.isEmpty(Iterables.filter(columns, STATIC_COLUMN_FILTER));
+    }
+
+    /**
+     * Checks if this selection contains only static columns.
+     * @return <code>true</code> if this selection contains only static columns, <code>false</code> otherwise;
+     */
+    public boolean containsOnlyStaticColumns()
+    {
+        if (!containsStaticColumns())
+            return false;
+
+        if (isWildcard())
+            return false;
+
+        for (ColumnDefinition def : getColumns())
+        {
+            if (!def.isPartitionKey() && !def.isStatic())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Checks if this selection contains a collection.
+     *
+     * @return <code>true</code> if this selection contains a collection, <code>false</code> otherwise.
+     */
+    public boolean containsACollection()
+    {
+        if (!cfm.comparator.hasCollections())
+            return false;
+
+        for (ColumnDefinition def : getColumns())
+            if (def.type.isCollection() && def.type.isMultiCell())
+                return true;
+
+        return false;
+    }
+
+    /**
+     * Returns the index of the specified column.
+     *
+     * @param def the column definition
+     * @return the index of the specified column
+     */
+    public int indexOf(final ColumnDefinition def)
+    {
+        return Iterators.indexOf(getColumns().iterator(), new Predicate<ColumnDefinition>()
+           {
+               public boolean apply(ColumnDefinition n)
+               {
+                   return def.name.equals(n.name);
+               }
+           });
     }
 
     public ResultSet.Metadata getResultMetadata()
@@ -67,12 +156,12 @@ public abstract class Selection
     {
         List<ColumnDefinition> all = new ArrayList<ColumnDefinition>(cfm.allColumns().size());
         Iterators.addAll(all, cfm.allColumnsInSelectOrder());
-        return new SimpleSelection(all, true);
+        return new SimpleSelection(cfm, all, true);
     }
 
-    public static Selection forColumns(Collection<ColumnDefinition> columns)
+    public static Selection forColumns(CFMetaData cfm, Collection<ColumnDefinition> columns)
     {
-        return new SimpleSelection(columns, false);
+        return new SimpleSelection(cfm, columns, false);
     }
 
     public int addColumnForOrdering(ColumnDefinition c)
@@ -105,8 +194,8 @@ public abstract class Selection
                 SelectorFactories.createFactoriesAndCollectColumnDefinitions(RawSelector.toSelectables(rawSelectors, cfm), cfm, defs);
         List<ColumnSpecification> metadata = collectMetadata(cfm, rawSelectors, factories);
 
-        return processesSelection(rawSelectors) ? new SelectionWithProcessing(defs, metadata, factories)
-                                                : new SimpleSelection(defs, metadata, false);
+        return processesSelection(rawSelectors) ? new SelectionWithProcessing(cfm, defs, metadata, factories)
+                                                : new SimpleSelection(cfm, defs, metadata, false);
     }
 
     private static List<ColumnSpecification> collectMetadata(CFMetaData cfm,
@@ -280,19 +369,22 @@ public abstract class Selection
     {
         private final boolean isWildcard;
 
-        public SimpleSelection(Collection<ColumnDefinition> columns, boolean isWildcard)
+        public SimpleSelection(CFMetaData cfm, Collection<ColumnDefinition> columns, boolean isWildcard)
         {
-            this(columns, new ArrayList<ColumnSpecification>(columns), isWildcard);
+            this(cfm, columns, new ArrayList<ColumnSpecification>(columns), isWildcard);
         }
 
-        public SimpleSelection(Collection<ColumnDefinition> columns, List<ColumnSpecification> metadata, boolean isWildcard)
+        public SimpleSelection(CFMetaData cfm,
+                               Collection<ColumnDefinition> 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 columns. But since we're very unlikely to
              * get much duplicate in practice, it's more efficient not to bother.
              */
-            super(columns, metadata, false, false);
+            super(cfm, columns, metadata, false, false);
             this.isWildcard = isWildcard;
         }
 
@@ -340,17 +432,24 @@ public abstract class Selection
     {
         private final SelectorFactories factories;
 
-        public SelectionWithProcessing(Collection<ColumnDefinition> columns,
+        public SelectionWithProcessing(CFMetaData cfm,
+                                       Collection<ColumnDefinition> columns,
                                        List<ColumnSpecification> metadata,
                                        SelectorFactories factories) throws InvalidRequestException
         {
-            super(columns, metadata, factories.containsWritetimeSelectorFactory(), factories.containsTTLSelectorFactory());
+            super(cfm,
+                  columns,
+                  metadata,
+                  factories.containsWritetimeSelectorFactory(),
+                  factories.containsTTLSelectorFactory());
+
             this.factories = factories;
 
             if (factories.doesAggregation() && !factories.containsOnlyAggregateFunctions())
                 throw new InvalidRequestException("the select clause must either contains only aggregates or none");
         }
 
+        @Override
         public boolean usesFunction(String ksName, String functionName)
         {
             return factories.usesFunction(ksName, functionName);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/statements/Bound.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Bound.java b/src/java/org/apache/cassandra/cql3/statements/Bound.java
index 8aa3a37..7742642 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Bound.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Bound.java
@@ -28,8 +28,18 @@ public enum Bound
         this.idx = idx;
     }
 
-    public static Bound reverse(Bound b)
+    public Bound reverse()
     {
-        return b == START ? END : START;
+        return isStart() ? END : START;
+    }
+
+    public boolean isStart()
+    {
+        return this == START;
+    }
+
+    public boolean isEnd()
+    {
+        return this == END;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
index 33c61e7..ff685cf 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
@@ -23,6 +23,7 @@ import java.util.*;
 import com.google.common.collect.Iterators;
 
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.restrictions.Restriction;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/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 4e39614..8347ef5 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -27,6 +27,8 @@ import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.restrictions.Restriction;
+import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction;
 import org.apache.cassandra.cql3.selection.Selection;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CBuilder;
@@ -237,7 +239,7 @@ public abstract class ModificationStatement implements CQLStatement
 
     public void addKeyValue(ColumnDefinition def, Term value) throws InvalidRequestException
     {
-        addKeyValues(def, new SingleColumnRestriction.EQ(value, false));
+        addKeyValues(def, new SingleColumnRestriction.EQ(def, value));
     }
 
     public void processWhereClause(List<Relation> whereClause, VariableSpecifications names) throws InvalidRequestException
@@ -251,7 +253,7 @@ public abstract class ModificationStatement implements CQLStatement
             }
             SingleColumnRelation rel = (SingleColumnRelation) relation;
 
-            if (rel.onToken)
+            if (rel.onToken())
                 throw new InvalidRequestException(String.format("The token function cannot be used in WHERE clauses for UPDATE and DELETE statements: %s", relation));
 
             ColumnIdentifier id = rel.getEntity().prepare(cfm);
@@ -265,31 +267,9 @@ public abstract class ModificationStatement implements CQLStatement
                 case CLUSTERING_COLUMN:
                     Restriction restriction;
 
-                    if (rel.operator() == Operator.EQ)
+                    if (rel.isEQ() || (def.isPartitionKey() && rel.isIN()))
                     {
-                        Term t = rel.getValue().prepare(keyspace(), def);
-                        t.collectMarkerSpecification(names);
-                        restriction = new SingleColumnRestriction.EQ(t, false);
-                    }
-                    else if (def.kind == ColumnDefinition.Kind.PARTITION_KEY && rel.operator() == Operator.IN)
-                    {
-                        if (rel.getValue() != null)
-                        {
-                            Term t = rel.getValue().prepare(keyspace(), def);
-                            t.collectMarkerSpecification(names);
-                            restriction = new SingleColumnRestriction.InWithMarker((Lists.Marker)t);
-                        }
-                        else
-                        {
-                            List<Term> values = new ArrayList<Term>(rel.getInValues().size());
-                            for (Term.Raw raw : rel.getInValues())
-                            {
-                                Term t = raw.prepare(keyspace(), def);
-                                t.collectMarkerSpecification(names);
-                                values.add(t);
-                            }
-                            restriction = new SingleColumnRestriction.InWithValues(values);
-                        }
+                        restriction = rel.toRestriction(cfm, names);
                     }
                     else
                     {
@@ -623,7 +603,8 @@ public abstract class ModificationStatement implements CQLStatement
             }
             for (ColumnDefinition def : columnsWithConditions)
                 defs.add(def);
-            selection = Selection.forColumns(new ArrayList<>(defs));
+            selection = Selection.forColumns(cfm, new ArrayList<>(defs));
+
         }
 
         long now = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java b/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
deleted file mode 100644
index 96cb905..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3.statements;
-
-import org.apache.cassandra.cql3.AbstractMarker;
-import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.cql3.Term;
-import org.apache.cassandra.cql3.Tuples;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-public interface MultiColumnRestriction extends Restriction
-{
-    public static class EQ extends SingleColumnRestriction.EQ implements MultiColumnRestriction
-    {
-        public EQ(Term value, boolean onToken)
-        {
-            super(value, onToken);
-        }
-
-        public boolean isMultiColumn()
-        {
-            return true;
-        }
-
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-        {
-            Tuples.Value t = (Tuples.Value)value.bind(options);
-            return t.getElements();
-        }
-    }
-
-    public interface IN extends MultiColumnRestriction
-    {
-        public List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException;
-    }
-
-    /**
-     * An IN restriction that has a set of terms for in values.
-     * For example: "SELECT ... WHERE (a, b, c) IN ((1, 2, 3), (4, 5, 6))" or "WHERE (a, b, c) IN (?, ?)"
-     */
-    public static class InWithValues extends SingleColumnRestriction.InWithValues implements MultiColumnRestriction.IN
-    {
-        public InWithValues(List<Term> values)
-        {
-            super(values);
-        }
-
-        public boolean isMultiColumn()
-        {
-            return true;
-        }
-
-        public List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
-        {
-            List<List<ByteBuffer>> buffers = new ArrayList<>(values.size());
-            for (Term value : values)
-            {
-                Term.MultiItemTerminal term = (Term.MultiItemTerminal)value.bind(options);
-                buffers.add(term.getElements());
-            }
-            return buffers;
-        }
-    }
-
-    /**
-     * An IN restriction that uses a single marker for a set of IN values that are tuples.
-     * For example: "SELECT ... WHERE (a, b, c) IN ?"
-     */
-    public static class InWithMarker extends SingleColumnRestriction.InWithMarker implements MultiColumnRestriction.IN
-    {
-        public InWithMarker(AbstractMarker marker)
-        {
-            super(marker);
-        }
-
-        public boolean isMultiColumn()
-        {
-            return true;
-        }
-
-        public List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
-        {
-            Tuples.InMarker inMarker = (Tuples.InMarker)marker;
-            Tuples.InValue inValue = inMarker.bind(options);
-            if (inValue == null)
-                throw new InvalidRequestException("Invalid null value for IN restriction");
-            return inValue.getSplitValues();
-        }
-    }
-
-    public static class Slice extends SingleColumnRestriction.Slice implements MultiColumnRestriction
-    {
-        public Slice(boolean onToken)
-        {
-            super(onToken);
-        }
-
-        public boolean isMultiColumn()
-        {
-            return true;
-        }
-
-        public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException
-        {
-            throw new UnsupportedOperationException("Multicolumn slice restrictions do not support bound()");
-        }
-
-        /**
-         * Similar to bounds(), but returns one ByteBuffer per-component in the bound instead of a single
-         * ByteBuffer to represent the entire bound.
-         */
-        public List<ByteBuffer> componentBounds(Bound b, QueryOptions options) throws InvalidRequestException
-        {
-            Tuples.Value value = (Tuples.Value)bounds[b.idx].bind(options);
-            return value.getElements();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/statements/RequestValidations.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/RequestValidations.java b/src/java/org/apache/cassandra/cql3/statements/RequestValidations.java
new file mode 100644
index 0000000..ce15e15
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/RequestValidations.java
@@ -0,0 +1,194 @@
+/*
+ * 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.cql3.statements;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+import static org.apache.commons.lang3.ArrayUtils.EMPTY_OBJECT_ARRAY;
+
+/**
+ * Utility methods use to perform request validation.
+ */
+public final class RequestValidations
+{
+    /**
+     * Checks that the specified expression is <code>true</code>. If not an <code>InvalidRequestException</code> will
+     * be thrown.
+     *
+     * @param expression the expression to test
+     * @param messageTemplate the error message
+     * @param messageArgs the message arguments
+     * @throws InvalidRequestException if the specified expression is <code>false</code>.
+     */
+    public static void checkTrue(boolean expression, String message) throws InvalidRequestException
+    {
+        checkTrue(expression, message, EMPTY_OBJECT_ARRAY);
+    }
+
+    /**
+     * Checks that the specified expression is <code>true</code>. If not an <code>InvalidRequestException</code> will
+     * be thrown.
+     *
+     * @param expression the expression to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @throws InvalidRequestException if the specified expression is <code>false</code>.
+     */
+    public static void checkTrue(boolean expression,
+                                 String messageTemplate,
+                                 Object... messageArgs)
+                                 throws InvalidRequestException
+    {
+        if (!expression)
+            throw invalidRequest(messageTemplate, messageArgs);
+    }
+
+    /**
+     * Checks that the specified list does not contains duplicates.
+     *
+     * @param list the list to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @throws InvalidRequestException if the specified list contains duplicates.
+     */
+    public static void checkContainsNoDuplicates(List<?> list, String message) throws InvalidRequestException
+    {
+        if (new HashSet<>(list).size() != list.size())
+            throw invalidRequest(message);
+    }
+
+    /**
+     * Checks that the specified list contains only the specified elements.
+     *
+     * @param list the list to test
+     * @param expectedElements the expected elements
+     * @param message the error message
+     * @throws InvalidRequestException if the specified list contains duplicates.
+     */
+    public static <E> void checkContainsOnly(List<E> list,
+                                             List<E> expectedElements,
+                                             String message) throws InvalidRequestException
+    {
+        List<E> copy = new ArrayList<>(list);
+        copy.removeAll(expectedElements);
+        if (!copy.isEmpty())
+            throw invalidRequest(message);
+    }
+
+    /**
+     * Checks that the specified expression is <code>false</code>. If not an <code>InvalidRequestException</code> will
+     * be thrown.
+     *
+     * @param expression the expression to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @throws InvalidRequestException if the specified expression is <code>true</code>.
+     */
+    public static void checkFalse(boolean expression,
+                                  String messageTemplate,
+                                  Object... messageArgs)
+                                  throws InvalidRequestException
+    {
+        checkTrue(!expression, messageTemplate, messageArgs);
+    }
+
+    /**
+     * Checks that the specified expression is <code>false</code>. If not an <code>InvalidRequestException</code> will
+     * be thrown.
+     *
+     * @param expression the expression to test
+     * @param message the error message
+     * @throws InvalidRequestException if the specified expression is <code>true</code>.
+     */
+    public static void checkFalse(boolean expression, String message) throws InvalidRequestException
+    {
+        checkTrue(!expression, message);
+    }
+
+    /**
+     * Checks that the specified object is NOT <code>null</code>.
+     * If it is an <code>InvalidRequestException</code> will be throws.
+     *
+     * @param object the object to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @return the object
+     * @throws InvalidRequestException if the specified object is <code>null</code>.
+     */
+    public static <T> T checkNotNull(T object, String messageTemplate, Object... messageArgs)
+            throws InvalidRequestException
+    {
+        checkTrue(object != null, messageTemplate, messageArgs);
+        return object;
+    }
+
+    /**
+     * Checks that the specified object is <code>null</code>.
+     * If it is not an <code>InvalidRequestException</code> will be throws.
+     *
+     * @param object the object to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @return the object
+     * @throws InvalidRequestException if the specified object is not <code>null</code>.
+     */
+    public static <T> T checkNull(T object, String messageTemplate, Object... messageArgs)
+            throws InvalidRequestException
+    {
+        checkTrue(object == null, messageTemplate, messageArgs);
+        return object;
+    }
+
+    /**
+     * Checks that the specified object is <code>null</code>.
+     * If it is not an <code>InvalidRequestException</code> will be throws.
+     *
+     * @param object the object to test
+     * @param message the error message
+     * @return the object
+     * @throws InvalidRequestException if the specified object is not <code>null</code>.
+     */
+    public static <T> T checkNull(T object, String message) throws InvalidRequestException
+    {
+        return checkNull(object, message, EMPTY_OBJECT_ARRAY);
+    }
+
+    /**
+     * Returns an <code>InvalidRequestException</code> with the specified message.
+     *
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @return an <code>InvalidRequestException</code> with the specified message.
+     */
+    public static InvalidRequestException invalidRequest(String messageTemplate, Object... messageArgs)
+    {
+        return new InvalidRequestException(String.format(messageTemplate, messageArgs));
+    }
+
+    /**
+     * This class must not be instantiated as it only contains static methods.
+     */
+    private RequestValidations()
+    {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/statements/Restriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Restriction.java b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
deleted file mode 100644
index b264156..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/Restriction.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3.statements;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.Operator;
-import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.cql3.Term;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-
-/**
- * A restriction/clause on a column.
- * The goal of this class being to group all conditions for a column in a SELECT.
- */
-public interface Restriction
-{
-    public boolean isOnToken();
-
-    public boolean isSlice();
-    public boolean isEQ();
-    public boolean isIN();
-    public boolean isContains();
-    public boolean isMultiColumn();
-
-    /**
-     * Returns true if, when applied to a clustering column, this restriction can be handled through one or more slices
-     * alone without filtering.  For example, EQ restrictions can be represented as a slice, but CONTAINS cannot.
-     */
-    public boolean canEvaluateWithSlices();
-
-    // Not supported by Slice, but it's convenient to have here
-    public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException;
-
-    boolean usesFunction(String ksName, String functionName);
-
-    public static interface EQ extends Restriction {}
-
-    public static interface IN extends Restriction
-    {
-        public boolean canHaveOnlyOneValue();
-    }
-
-    public static interface Slice extends Restriction
-    {
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException;
-
-        /** Returns true if the start or end bound (depending on the argument) is set, false otherwise */
-        public boolean hasBound(Bound b);
-
-        public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException;
-
-        /** Returns true if the start or end bound (depending on the argument) is inclusive, false otherwise */
-        public boolean isInclusive(Bound b);
-
-        public Operator getRelation(Bound eocBound, Bound inclusiveBound);
-
-        public Operator getIndexOperator(Bound b);
-
-        public void setBound(ColumnIdentifier name, Operator type, Term t) throws InvalidRequestException;
-    }
-}


[5/5] cassandra git commit: Refactor SelectStatement and Restrictions

Posted by ty...@apache.org.
Refactor SelectStatement and Restrictions

Patch by Benjamin Lerer; reviewed by Tyler Hobbs for CASSANDRA-7981


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

Branch: refs/heads/trunk
Commit: 65a7088e71061b876e9cd51140f31c92ded92777
Parents: a604b14
Author: blerer <b_...@hotmail.com>
Authored: Tue Dec 2 13:08:25 2014 -0600
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Tue Dec 2 13:08:25 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |    2 +
 NEWS.txt                                        |    3 +
 .../cassandra/config/ColumnDefinition.java      |   40 +
 .../cassandra/cql3/ColumnSpecification.java     |    6 +
 src/java/org/apache/cassandra/cql3/Cql.g        |    5 +-
 .../cassandra/cql3/MultiColumnRelation.java     |  130 +-
 .../org/apache/cassandra/cql3/Operator.java     |  119 +-
 .../org/apache/cassandra/cql3/Relation.java     |  221 ++-
 .../cassandra/cql3/SingleColumnRelation.java    |  181 +-
 .../apache/cassandra/cql3/TokenRelation.java    |  164 ++
 src/java/org/apache/cassandra/cql3/Tuples.java  |    3 +-
 .../cassandra/cql3/VariableSpecifications.java  |   10 +
 .../AbstractPrimaryKeyRestrictions.java         |   36 +
 .../cql3/restrictions/AbstractRestriction.java  |  129 ++
 .../ForwardingPrimaryKeyRestrictions.java       |  159 ++
 .../restrictions/MultiColumnRestriction.java    |  520 ++++++
 .../restrictions/PrimaryKeyRestrictions.java    |   40 +
 .../cql3/restrictions/Restriction.java          |   97 ++
 .../cql3/restrictions/Restrictions.java         |   82 +
 .../ReversedPrimaryKeyRestrictions.java         |   77 +
 .../SingleColumnPrimaryKeyRestrictions.java     |  307 ++++
 .../restrictions/SingleColumnRestriction.java   |  477 ++++++
 .../restrictions/SingleColumnRestrictions.java  |  209 +++
 .../restrictions/StatementRestrictions.java     |  600 +++++++
 .../cassandra/cql3/restrictions/TermSlice.java  |  167 ++
 .../cql3/restrictions/TokenRestriction.java     |  224 +++
 .../cassandra/cql3/selection/Selection.java     |  123 +-
 .../apache/cassandra/cql3/statements/Bound.java |   14 +-
 .../cql3/statements/DeleteStatement.java        |    1 +
 .../cql3/statements/ModificationStatement.java  |   35 +-
 .../cql3/statements/MultiColumnRestriction.java |  137 --
 .../cql3/statements/RequestValidations.java     |  194 +++
 .../cassandra/cql3/statements/Restriction.java  |   79 -
 .../cql3/statements/SelectStatement.java        | 1597 +++---------------
 .../statements/SingleColumnRestriction.java     |  486 ------
 .../cassandra/db/composites/Composites.java     |   22 +-
 .../db/composites/CompositesBuilder.java        |   15 +-
 .../cassandra/db/marshal/CollectionType.java    |   34 +-
 .../exceptions/UnrecognizedEntityException.java |   49 +
 .../org/apache/cassandra/cql3/AliasTest.java    |   40 +
 .../cassandra/cql3/ContainsRelationTest.java    |   39 +-
 .../cassandra/cql3/FrozenCollectionsTest.java   |   16 +-
 .../cassandra/cql3/MultiColumnRelationTest.java |  161 +-
 .../cql3/SelectWithTokenFunctionTest.java       |   39 +-
 .../cql3/SingleColumnRelationTest.java          |  218 ++-
 .../cassandra/cql3/ThriftCompatibilityTest.java |    2 +-
 46 files changed, 5031 insertions(+), 2278 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 3cb1c0f..6761c31 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,6 @@
 3.0
+ * Refactor SelectStatement, return IN results in natural order instead
+   of IN value list order (CASSANDRA-7981)
  * Support UDTs, tuples, and collections in user-defined
    functions (CASSANDRA-7563)
  * Fix aggregate fn results on empty selection, result column name,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 1d168f0..8d8ebdc 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -33,6 +33,9 @@ Upgrading
      in 2.0.0). Please switch to CQL3 if you haven't already done so.
    - Very large batches will now be rejected (defaults to 50kb). This
      can be customized by modifying batch_size_fail_threshold_in_kb.
+   - The results of CQL3 queries containing an IN restriction will be ordered
+     in the normal order and not anymore in the order in which the column values were
+     specified in the IN restriction.
 
 2.1.2
 =====

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/config/ColumnDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ColumnDefinition.java b/src/java/org/apache/cassandra/config/ColumnDefinition.java
index 10a5a8b..354a6f1 100644
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java
@@ -21,7 +21,9 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
 import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
 
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.*;
@@ -171,11 +173,31 @@ public class ColumnDefinition extends ColumnSpecification
         return componentIndex == null;
     }
 
+    public boolean isPartitionKey()
+    {
+        return kind == Kind.PARTITION_KEY;
+    }
+
+    public boolean isClusteringColumn()
+    {
+        return kind == Kind.CLUSTERING_COLUMN;
+    }
+
     public boolean isStatic()
     {
         return kind == Kind.STATIC;
     }
 
+    public boolean isRegular()
+    {
+        return kind == Kind.REGULAR;
+    }
+
+    public boolean isCompactValue()
+    {
+        return kind == Kind.COMPACT_VALUE;
+    }
+
     // The componentIndex. This never return null however for convenience sake:
     // if componentIndex == null, this return 0. So caller should first check
     // isOnAllComponents() to distinguish if that's a possibility.
@@ -425,4 +447,22 @@ public class ColumnDefinition extends ColumnSpecification
     {
         return indexOptions.containsKey(name);
     }
+
+    /**
+     * Converts the specified column definitions into column identifiers.
+     *
+     * @param definitions the column definitions to convert.
+     * @return the column identifiers corresponding to the specified definitions
+     */
+    public static List<ColumnIdentifier> toIdentifiers(List<ColumnDefinition> definitions)
+    {
+        return Lists.transform(definitions, new Function<ColumnDefinition, ColumnIdentifier>()
+        {
+            @Override
+            public ColumnIdentifier apply(ColumnDefinition columnDef)
+            {
+                return columnDef.name;
+            }
+        });
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
index 2584f85..cc54375 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.cql3;
 
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
 
 public class ColumnSpecification
 {
@@ -44,4 +45,9 @@ public class ColumnSpecification
     {
         return new ColumnSpecification(ksName, cfName, alias, type);
     }
+    
+    public boolean isReversedType()
+    {
+        return type instanceof ReversedType;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 4c051e3..77156f2 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -1091,10 +1091,7 @@ relationType returns [Operator op]
 relation[List<Relation> clauses]
     : name=cident type=relationType t=term { $clauses.add(new SingleColumnRelation(name, type, t)); }
     | K_TOKEN l=tupleOfIdentifiers type=relationType t=term
-        {
-            for (ColumnIdentifier.Raw id : l)
-                $clauses.add(new SingleColumnRelation(id, type, t, true));
-        }
+        { $clauses.add(new TokenRelation(l, type, t)); }
     | name=cident K_IN marker=inMarker
         { $clauses.add(new SingleColumnRelation(name, Operator.IN, marker)); }
     | name=cident K_IN inValues=singleColumnInValues

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
index 37eb69e..d754968 100644
--- a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
@@ -17,8 +17,22 @@
  */
 package org.apache.cassandra.cql3;
 
+import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.Term.MultiColumnRaw;
+import org.apache.cassandra.cql3.Term.Raw;
+import org.apache.cassandra.cql3.restrictions.MultiColumnRestriction;
+import org.apache.cassandra.cql3.restrictions.Restriction;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
 /**
  * A relation using the tuple notation, which typically affects multiple columns.
  * Examples:
@@ -55,6 +69,7 @@ public class MultiColumnRelation extends Relation
      * @param entities the columns on the LHS of the relation
      * @param relationType the relation operator
      * @param valuesOrMarker a Tuples.Literal instance or a Tuples.Raw marker
+     * @return a new <code>MultiColumnRelation</code> instance
      */
     public static MultiColumnRelation createNonInRelation(List<ColumnIdentifier.Raw> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker)
     {
@@ -67,6 +82,7 @@ public class MultiColumnRelation extends Relation
      * For example: "SELECT ... WHERE (a, b) IN ((0, 1), (2, 3))"
      * @param entities the columns on the LHS of the relation
      * @param inValues a list of Tuples.Literal instances or a Tuples.Raw markers
+     * @return a new <code>MultiColumnRelation</code> instance
      */
     public static MultiColumnRelation createInRelation(List<ColumnIdentifier.Raw> entities, List<? extends Term.MultiColumnRaw> inValues)
     {
@@ -78,6 +94,7 @@ public class MultiColumnRelation extends Relation
      * For example: "SELECT ... WHERE (a, b) IN ?"
      * @param entities the columns on the LHS of the relation
      * @param inMarker a single IN marker
+     * @return a new <code>MultiColumnRelation</code> instance
      */
     public static MultiColumnRelation createSingleMarkerInRelation(List<ColumnIdentifier.Raw> entities, Tuples.INRaw inMarker)
     {
@@ -91,54 +108,109 @@ public class MultiColumnRelation extends Relation
 
     /**
      * For non-IN relations, returns the Tuples.Literal or Tuples.Raw marker for a single tuple.
+     * @return a Tuples.Literal for non-IN relations or Tuples.Raw marker for a single tuple.
      */
-    public Term.MultiColumnRaw getValue()
+    private Term.MultiColumnRaw getValue()
     {
-        assert relationType != Operator.IN;
-        return valuesOrMarker;
+        return relationType == Operator.IN ? inMarker : valuesOrMarker;
     }
 
-    /**
-     * For IN relations, returns the list of Tuples.Literal instances or Tuples.Raw markers.
-     * If a single IN marker was used, this will return null;
-     */
-    public List<? extends Term.MultiColumnRaw> getInValues()
+    @Override
+    public boolean isMultiColumn()
     {
+        return true;
+    }
 
-        return inValues;
+    @Override
+    protected Restriction newEQRestriction(CFMetaData cfm,
+                                           VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        List<ColumnDefinition> receivers = receivers(cfm);
+        Term term = toTerm(receivers, getValue(), cfm.ksName, boundNames);
+        return new MultiColumnRestriction.EQ(cfm.comparator, receivers, term);
     }
 
-    /**
-     * For IN relations, returns the single marker for the IN values if there is one, otherwise null.
-     */
-    public Tuples.INRaw getInMarker()
+    @Override
+    protected Restriction newINRestriction(CFMetaData cfm,
+                                           VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        List<ColumnDefinition> receivers = receivers(cfm);
+        List<Term> terms = toTerms(receivers, inValues, cfm.ksName, boundNames);
+        if (terms == null)
+        {
+            Term term = toTerm(receivers, getValue(), cfm.ksName, boundNames);
+            return new MultiColumnRestriction.InWithMarker(cfm.comparator, receivers, (AbstractMarker) term);
+        }
+        return new MultiColumnRestriction.InWithValues(cfm.comparator, receivers, terms);
+    }
+
+    @Override
+    protected Restriction newSliceRestriction(CFMetaData cfm,
+                                              VariableSpecifications boundNames,
+                                              Bound bound,
+                                              boolean inclusive) throws InvalidRequestException
     {
-        return inMarker;
+        List<ColumnDefinition> receivers = receivers(cfm);
+        Term term = toTerm(receivers(cfm), getValue(), cfm.ksName, boundNames);
+        return new MultiColumnRestriction.Slice(cfm.comparator, receivers, bound, inclusive, term);
     }
 
-    public boolean isMultiColumn()
+    @Override
+    protected Restriction newContainsRestriction(CFMetaData cfm,
+                                                 VariableSpecifications boundNames,
+                                                 boolean isKey) throws InvalidRequestException
     {
-        return true;
+        throw invalidRequest("%s cannot be used for Multi-column relations", operator());
     }
 
     @Override
-    public String toString()
+    protected Term toTerm(List<? extends ColumnSpecification> receivers,
+                          Raw raw,
+                          String keyspace,
+                          VariableSpecifications boundNames) throws InvalidRequestException
     {
-        if (relationType == Operator.IN)
+        Term term = ((MultiColumnRaw) raw).prepare(keyspace, receivers);
+        term.collectMarkerSpecification(boundNames);
+        return term;
+    }
+
+    protected List<ColumnDefinition> receivers(CFMetaData cfm) throws InvalidRequestException
+    {
+        List<ColumnDefinition> names = new ArrayList<>(getEntities().size());
+        int previousPosition = -1;
+        for (ColumnIdentifier.Raw raw : getEntities())
         {
-            StringBuilder sb = new StringBuilder(Tuples.tupleToString(entities));
-            sb.append(" IN ");
-            sb.append(inMarker != null ? '?' : Tuples.tupleToString(inValues));
-            return sb.toString();
+            ColumnDefinition def = toColumnDefinition(cfm, raw);
+            checkTrue(def.isClusteringColumn(), "Multi-column relations can only be applied to clustering columns but was applied to: %s", def.name);
+            checkFalse(names.contains(def), "Column \"%s\" appeared twice in a relation: %s", def.name, this);
+
+            // check that no clustering columns were skipped
+            if (def.position() != previousPosition + 1)
+            {
+                checkFalse(previousPosition == -1, "Clustering columns may not be skipped in multi-column relations. " +
+                                                   "They should appear in the PRIMARY KEY order. Got %s", this);
+                throw invalidRequest("Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", this);
+            }
+            names.add(def);
+            previousPosition = def.position();
         }
-        else
+        return names;
+    }
+
+    @Override
+    public String toString()
+    {
+        StringBuilder builder = new StringBuilder(Tuples.tupleToString(entities));
+        if (isIN())
         {
-            StringBuilder sb = new StringBuilder(Tuples.tupleToString(entities));
-            sb.append(" ");
-            sb.append(relationType);
-            sb.append(" ");
-            sb.append(valuesOrMarker);
-            return sb.toString();
+            return builder.append(" IN ")
+                          .append(inMarker != null ? '?' : Tuples.tupleToString(inValues))
+                          .toString();
         }
+        return builder.append(" ")
+                      .append(relationType)
+                      .append(" ")
+                      .append(valuesOrMarker)
+                      .toString();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/Operator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Operator.java b/src/java/org/apache/cassandra/cql3/Operator.java
index 359fcb8..86bcbd3 100644
--- a/src/java/org/apache/cassandra/cql3/Operator.java
+++ b/src/java/org/apache/cassandra/cql3/Operator.java
@@ -23,7 +23,92 @@ import java.io.IOException;
 
 public enum Operator
 {
-    EQ(0), LT(4), LTE(3), GTE(1), GT(2), IN(7), CONTAINS(5), CONTAINS_KEY(6), NEQ(8);
+    EQ(0)
+    {
+        @Override
+        public String toString()
+        {
+            return "=";
+        }
+    },
+    LT(4)
+    {
+        @Override
+        public String toString()
+        {
+            return "<";
+        }
+
+        @Override
+        public Operator reverse()
+        {
+            return GT;
+        }
+    },
+    LTE(3)
+    {
+        @Override
+        public String toString()
+        {
+            return "<=";
+        }
+
+        @Override
+        public Operator reverse()
+        {
+            return GTE;
+        }
+    },
+    GTE(1)
+    {
+        @Override
+        public String toString()
+        {
+            return ">=";
+        }
+
+        @Override
+        public Operator reverse()
+        {
+            return LTE;
+        }
+    },
+    GT(2)
+    {
+        @Override
+        public String toString()
+        {
+            return ">";
+        }
+
+        @Override
+        public Operator reverse()
+        {
+            return LT;
+        }
+    },
+    IN(7)
+    {
+    },
+    CONTAINS(5)
+    {
+    },
+    CONTAINS_KEY(6)
+    {
+        @Override
+        public String toString()
+        {
+            return "CONTAINS KEY";
+        }
+    },
+    NEQ(8)
+    {
+        @Override
+        public String toString()
+        {
+            return "!=";
+        }
+    };
 
     /**
      * The binary representation of this <code>Enum</code> value.
@@ -70,24 +155,16 @@ public enum Operator
     @Override
     public String toString()
     {
-        switch (this)
-        {
-            case EQ:
-                return "=";
-            case LT:
-                return "<";
-            case LTE:
-                return "<=";
-            case GT:
-                return ">";
-            case GTE:
-                return ">=";
-            case NEQ:
-                return "!=";
-            case CONTAINS_KEY:
-                return "CONTAINS KEY";
-            default:
-                return this.name();
-        }
+         return this.name();
+    }
+
+    /**
+     * Returns the reverse operator if this one.
+     *
+     * @return the reverse operator of this one.
+     */
+    public Operator reverse()
+    {
+        return this;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/Relation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Relation.java b/src/java/org/apache/cassandra/cql3/Relation.java
index 91d4100..1337096 100644
--- a/src/java/org/apache/cassandra/cql3/Relation.java
+++ b/src/java/org/apache/cassandra/cql3/Relation.java
@@ -17,6 +17,18 @@
  */
 package org.apache.cassandra.cql3;
 
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.restrictions.Restriction;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.UnrecognizedEntityException;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
 public abstract class Relation {
 
     protected Operator relationType;
@@ -26,5 +38,212 @@ public abstract class Relation {
         return relationType;
     }
 
-    public abstract boolean isMultiColumn();
+    /**
+     * Checks if this relation apply to multiple columns.
+     *
+     * @return <code>true</code> if this relation apply to multiple columns, <code>false</code> otherwise.
+     */
+    public boolean isMultiColumn()
+    {
+        return false;
+    }
+
+    /**
+     * Checks if this relation is a token relation (e.g. <pre>token(a) = token(1)</pre>).
+     *
+     * @return <code>true</code> if this relation is a token relation, <code>false</code> otherwise.
+     */
+    public boolean onToken()
+    {
+        return false;
+    }
+
+    /**
+     * Checks if the operator of this relation is a <code>CONTAINS</code>.
+     * @return <code>true</code>  if the operator of this relation is a <code>CONTAINS</code>, <code>false</code>
+     * otherwise.
+     */
+    public final boolean isContains()
+    {
+        return relationType == Operator.CONTAINS;
+    }
+
+    /**
+     * Checks if the operator of this relation is a <code>CONTAINS_KEY</code>.
+     * @return <code>true</code>  if the operator of this relation is a <code>CONTAINS_KEY</code>, <code>false</code>
+     * otherwise.
+     */
+    public final boolean isContainsKey()
+    {
+        return relationType == Operator.CONTAINS_KEY;
+    }
+
+    /**
+     * Checks if the operator of this relation is a <code>IN</code>.
+     * @return <code>true</code>  if the operator of this relation is a <code>IN</code>, <code>false</code>
+     * otherwise.
+     */
+    public final boolean isIN()
+    {
+        return relationType == Operator.IN;
+    }
+
+    /**
+     * Checks if the operator of this relation is a <code>EQ</code>.
+     * @return <code>true</code>  if the operator of this relation is a <code>EQ</code>, <code>false</code>
+     * otherwise.
+     */
+    public final boolean isEQ()
+    {
+        return relationType == Operator.EQ;
+    }
+
+    /**
+     * Checks if the operator of this relation is a <code>Slice</code> (GT, GTE, LTE, LT).
+     *
+     * @return <code>true</code> if the operator of this relation is a <code>Slice</code>, <code>false</code> otherwise.
+     */
+    public final boolean isSlice()
+    {
+        return relationType == Operator.GT
+                || relationType == Operator.GTE
+                || relationType == Operator.LTE
+                || relationType == Operator.LT;
+    }
+
+    /**
+     * Converts this <code>Relation</code> into a <code>Restriction</code>.
+     *
+     * @param cfm the Column Family meta data
+     * @param boundNames the variables specification where to collect the bind variables
+     * @return the <code>Restriction</code> corresponding to this <code>Relation</code>
+     * @throws InvalidRequestException if this <code>Relation</code> is not valid
+     */
+    public final Restriction toRestriction(CFMetaData cfm,
+                                           VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        switch (relationType)
+        {
+            case EQ: return newEQRestriction(cfm, boundNames);
+            case LT: return newSliceRestriction(cfm, boundNames, Bound.END, false);
+            case LTE: return newSliceRestriction(cfm, boundNames, Bound.END, true);
+            case GTE: return newSliceRestriction(cfm, boundNames, Bound.START, true);
+            case GT: return newSliceRestriction(cfm, boundNames, Bound.START, false);
+            case IN: return newINRestriction(cfm, boundNames);
+            case CONTAINS: return newContainsRestriction(cfm, boundNames, false);
+            case CONTAINS_KEY: return newContainsRestriction(cfm, boundNames, true);
+            default: throw invalidRequest("Unsupported \"!=\" relation: %s", this);
+        }
+    }
+
+    /**
+     * Creates a new EQ restriction instance.
+     *
+     * @param cfm the Column Family meta data
+     * @param boundNames the variables specification where to collect the bind variables
+     * @return a new EQ restriction instance.
+     * @throws InvalidRequestException if the relation cannot be converted into an EQ restriction.
+     */
+    protected abstract Restriction newEQRestriction(CFMetaData cfm,
+                                                    VariableSpecifications boundNames) throws InvalidRequestException;
+
+    /**
+     * Creates a new IN restriction instance.
+     *
+     * @param cfm the Column Family meta data
+     * @param boundNames the variables specification where to collect the bind variables
+     * @return a new IN restriction instance
+     * @throws InvalidRequestException if the relation cannot be converted into an IN restriction.
+     */
+    protected abstract Restriction newINRestriction(CFMetaData cfm,
+                                                    VariableSpecifications boundNames) throws InvalidRequestException;
+
+    /**
+     * Creates a new Slice restriction instance.
+     *
+     * @param cfm the Column Family meta data
+     * @param boundNames the variables specification where to collect the bind variables
+     * @param bound the slice bound
+     * @param inclusive <code>true</code> if the bound is included.
+     * @return a new slice restriction instance
+     * @throws InvalidRequestException if the <code>Relation</code> is not valid
+     */
+    protected abstract Restriction newSliceRestriction(CFMetaData cfm,
+                                                       VariableSpecifications boundNames,
+                                                       Bound bound,
+                                                       boolean inclusive) throws InvalidRequestException;
+
+    /**
+     * Creates a new Contains restriction instance.
+     *
+     * @param cfm the Column Family meta data
+     * @param boundNames the variables specification where to collect the bind variables
+     * @param isKey <code>true</code> if the restriction to create is a CONTAINS KEY
+     * @return a new Contains <code>Restriction</code> instance
+     * @throws InvalidRequestException if the <code>Relation</code> is not valid
+     */
+    protected abstract Restriction newContainsRestriction(CFMetaData cfm,
+                                                          VariableSpecifications boundNames,
+                                                          boolean isKey) throws InvalidRequestException;
+
+    /**
+     * Converts the specified <code>Raw</code> into a <code>Term</code>.
+     * @param receivers the columns to which the values must be associated at
+     * @param raw the raw term to convert
+     * @param keyspace the keyspace name
+     * @param boundNames the variables specification where to collect the bind variables
+     *
+     * @return the <code>Term</code> corresponding to the specified <code>Raw</code>
+     * @throws InvalidRequestException if the <code>Raw</code> term is not valid
+     */
+    protected abstract Term toTerm(List<? extends ColumnSpecification> receivers,
+                                   Term.Raw raw,
+                                   String keyspace,
+                                   VariableSpecifications boundNames)
+                                   throws InvalidRequestException;
+
+    /**
+     * Converts the specified <code>Raw</code> terms into a <code>Term</code>s.
+     * @param receivers the columns to which the values must be associated at
+     * @param raws the raw terms to convert
+     * @param keyspace the keyspace name
+     * @param boundNames the variables specification where to collect the bind variables
+     *
+     * @return the <code>Term</code>s corresponding to the specified <code>Raw</code> terms
+     * @throws InvalidRequestException if the <code>Raw</code> terms are not valid
+     */
+    protected final List<Term> toTerms(List<? extends ColumnSpecification> receivers,
+                                       List<? extends Term.Raw> raws,
+                                       String keyspace,
+                                       VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        if (raws == null)
+            return null;
+
+        List<Term> terms = new ArrayList<>();
+        for (int i = 0, m = raws.size(); i < m; i++)
+            terms.add(toTerm(receivers, raws.get(i), keyspace, boundNames));
+
+        return terms;
+    }
+
+    /**
+     * Converts the specified entity into a column definition.
+     *
+     * @param cfm the column family meta data
+     * @param entity the entity to convert
+     * @return the column definition corresponding to the specified entity
+     * @throws InvalidRequestException if the entity cannot be recognized
+     */
+    protected final ColumnDefinition toColumnDefinition(CFMetaData cfm,
+                                                        ColumnIdentifier.Raw entity) throws InvalidRequestException
+    {
+        ColumnIdentifier identifier = entity.prepare(cfm);
+        ColumnDefinition def = cfm.getColumnDefinition(identifier);
+
+        if (def == null)
+            throw new UnrecognizedEntityException(identifier, this);
+
+        return def;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
index d5109f5..7817d43 100644
--- a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
@@ -17,27 +17,38 @@
  */
 package org.apache.cassandra.cql3;
 
+import java.util.Collections;
 import java.util.List;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.Term.Raw;
+import org.apache.cassandra.cql3.restrictions.Restriction;
+import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+
 /**
  * Relations encapsulate the relationship between an entity of some kind, and
  * a value (term). For example, <key> > "start" or "colname1" = "somevalue".
  *
  */
-public class SingleColumnRelation extends Relation
+public final class SingleColumnRelation extends Relation
 {
     private final ColumnIdentifier.Raw entity;
     private final Term.Raw value;
     private final List<Term.Raw> inValues;
-    public final boolean onToken;
 
-    private SingleColumnRelation(ColumnIdentifier.Raw entity, Operator type, Term.Raw value, List<Term.Raw> inValues, boolean onToken)
+    private SingleColumnRelation(ColumnIdentifier.Raw entity, Operator type, Term.Raw value, List<Term.Raw> inValues)
     {
         this.entity = entity;
         this.relationType = type;
         this.value = value;
         this.inValues = inValues;
-        this.onToken = onToken;
     }
 
     /**
@@ -49,17 +60,12 @@ public class SingleColumnRelation extends Relation
      */
     public SingleColumnRelation(ColumnIdentifier.Raw entity, Operator type, Term.Raw value)
     {
-        this(entity, type, value, null, false);
-    }
-
-    public SingleColumnRelation(ColumnIdentifier.Raw entity, Operator type, Term.Raw value, boolean onToken)
-    {
-        this(entity, type, value, null, onToken);
+        this(entity, type, value, null);
     }
 
     public static SingleColumnRelation createInRelation(ColumnIdentifier.Raw entity, List<Term.Raw> inValues)
     {
-        return new SingleColumnRelation(entity, Operator.IN, null, inValues, false);
+        return new SingleColumnRelation(entity, Operator.IN, null, inValues);
     }
 
     public ColumnIdentifier.Raw getEntity()
@@ -67,21 +73,18 @@ public class SingleColumnRelation extends Relation
         return entity;
     }
 
-    public Term.Raw getValue()
-    {
-        assert relationType != Operator.IN || value == null || value instanceof AbstractMarker.INRaw;
-        return value;
-    }
-
-    public List<Term.Raw> getInValues()
+    @Override
+    protected Term toTerm(List<? extends ColumnSpecification> receivers,
+                          Raw raw,
+                          String keyspace,
+                          VariableSpecifications boundNames)
+                          throws InvalidRequestException
     {
-        assert relationType == Operator.IN;
-        return inValues;
-    }
+        assert receivers.size() == 1;
 
-    public boolean isMultiColumn()
-    {
-        return false;
+        Term term = raw.prepare(keyspace, receivers.get(0));
+        term.collectMarkerSpecification(boundNames);
+        return term;
     }
 
     public SingleColumnRelation withNonStrictOperator()
@@ -97,11 +100,131 @@ public class SingleColumnRelation extends Relation
     @Override
     public String toString()
     {
-        if (relationType == Operator.IN)
+        if (isIN())
             return String.format("%s IN %s", entity, inValues);
-        else if (onToken)
-            return String.format("token(%s) %s %s", entity, relationType, value);
-        else
-            return String.format("%s %s %s", entity, relationType, value);
+
+        return String.format("%s %s %s", entity, relationType, value);
+    }
+
+    @Override
+    protected Restriction newEQRestriction(CFMetaData cfm,
+                                           VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        ColumnDefinition columnDef = toColumnDefinition(cfm, entity);
+        Term term = toTerm(toReceivers(cfm, columnDef), value, cfm.ksName, boundNames);
+        return new SingleColumnRestriction.EQ(columnDef, term);
+    }
+
+    @Override
+    protected Restriction newINRestriction(CFMetaData cfm,
+                                           VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        ColumnDefinition columnDef = cfm.getColumnDefinition(getEntity().prepare(cfm));
+        List<? extends ColumnSpecification> receivers = toReceivers(cfm, columnDef);
+        List<Term> terms = toTerms(receivers, inValues, cfm.ksName, boundNames);
+        if (terms == null)
+        {
+            Term term = toTerm(receivers, value, cfm.ksName, boundNames);
+            return new SingleColumnRestriction.InWithMarker(columnDef, (Lists.Marker) term);
+        }
+        return new SingleColumnRestriction.InWithValues(columnDef, terms);
+    }
+
+    @Override
+    protected Restriction newSliceRestriction(CFMetaData cfm,
+                                              VariableSpecifications boundNames,
+                                              Bound bound,
+                                              boolean inclusive) throws InvalidRequestException
+    {
+        ColumnDefinition columnDef = toColumnDefinition(cfm, entity);
+        Term term = toTerm(toReceivers(cfm, columnDef), value, cfm.ksName, boundNames);
+        return new SingleColumnRestriction.Slice(columnDef, bound, inclusive, term);
+    }
+
+    @Override
+    protected Restriction newContainsRestriction(CFMetaData cfm,
+                                                 VariableSpecifications boundNames,
+                                                 boolean isKey) throws InvalidRequestException
+    {
+        ColumnDefinition columnDef = toColumnDefinition(cfm, entity);
+        Term term = toTerm(toReceivers(cfm, columnDef), value, cfm.ksName, boundNames);
+        return new SingleColumnRestriction.Contains(columnDef, term, isKey);
+    }
+
+    /**
+     * Returns the receivers for this relation.
+     *
+     * @param cfm the Column Family meta data
+     * @param columnDef the column definition
+     * @return the receivers for the specified relation.
+     * @throws InvalidRequestException if the relation is invalid
+     */
+    private List<? extends ColumnSpecification> toReceivers(CFMetaData cfm, ColumnDefinition columnDef) throws InvalidRequestException
+    {
+        ColumnSpecification receiver = columnDef;
+
+        checkFalse(columnDef.isCompactValue(),
+                   "Predicates on the non-primary-key column (%s) of a COMPACT table are not yet supported",
+                   columnDef.name);
+
+        if (isIN())
+        {
+            // For partition keys we only support IN for the last name so far
+            checkFalse(columnDef.isPartitionKey() && !isLastPartitionKey(cfm, columnDef),
+                      "Partition KEY part %s cannot be restricted by IN relation (only the last part of the partition key can)",
+                      columnDef.name);
+
+            // We only allow IN on the row key and the clustering key so far, never on non-PK columns, and this even if
+            // there's an index
+            // Note: for backward compatibility reason, we conside a IN of 1 value the same as a EQ, so we let that
+            // slide.
+            checkFalse(!columnDef.isPrimaryKeyColumn() && !canHaveOnlyOneValue(),
+                       "IN predicates on non-primary-key columns (%s) is not yet supported", columnDef.name);
+        }
+        else if (isSlice())
+        {
+            // Non EQ relation is not supported without token(), even if we have a 2ndary index (since even those
+            // are ordered by partitioner).
+            // Note: In theory we could allow it for 2ndary index queries with ALLOW FILTERING, but that would
+            // probably require some special casing
+            // Note bis: This is also why we don't bother handling the 'tuple' notation of #4851 for keys. If we
+            // lift the limitation for 2ndary
+            // index with filtering, we'll need to handle it though.
+            checkFalse(columnDef.isPartitionKey(), "Only EQ and IN relation are supported on the partition key (unless you use the token() function)");
+        }
+
+        checkFalse(isContainsKey() && !(receiver.type instanceof MapType), "Cannot use CONTAINS KEY on non-map column %s", receiver.name);
+
+        if (receiver.type.isCollection())
+        {
+            // We don't support relations against entire collections (unless they're frozen), like "numbers = {1, 2, 3}"
+            checkFalse(receiver.type.isMultiCell() && !(isContainsKey() || isContains()),
+                       "Collection column '%s' (%s) cannot be restricted by a '%s' relation",
+                       receiver.name,
+                       receiver.type.asCQL3Type(),
+                       operator());
+
+            if (isContainsKey() || isContains())
+                receiver = ((CollectionType<?>) receiver.type).makeCollectionReceiver(receiver, isContainsKey());
+        }
+        return Collections.singletonList(receiver);
+    }
+
+    /**
+     * Checks if the specified column is the last column of the partition key.
+     *
+     * @param cfm the column family meta data
+     * @param columnDef the column to check
+     * @return <code>true</code> if the specified column is the last column of the partition key, <code>false</code>
+     * otherwise.
+     */
+    private static boolean isLastPartitionKey(CFMetaData cfm, ColumnDefinition columnDef)
+    {
+        return columnDef.position() == cfm.partitionKeyColumns().size() - 1;
+    }
+
+    private boolean canHaveOnlyOneValue()
+    {
+        return isEQ() || (isIN() && inValues != null && inValues.size() == 1);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/TokenRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/TokenRelation.java b/src/java/org/apache/cassandra/cql3/TokenRelation.java
new file mode 100644
index 0000000..d1bd265
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/TokenRelation.java
@@ -0,0 +1,164 @@
+/*
+ * 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.cql3;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.base.Joiner;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.Term.Raw;
+import org.apache.cassandra.cql3.restrictions.Restriction;
+import org.apache.cassandra.cql3.restrictions.TokenRestriction;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.StorageService;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkContainsNoDuplicates;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkContainsOnly;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * A relation using the token function.
+ * Examples:
+ * <ul>
+ * <li>SELECT ... WHERE token(a) &gt; token(1)</li>
+ * <li>SELECT ... WHERE token(a, b) &gt; token(1, 3)</li>
+ * </ul>
+ */
+public final class TokenRelation extends Relation
+{
+    private final List<ColumnIdentifier.Raw> entities;
+
+    private final Term.Raw value;
+
+    public TokenRelation(List<ColumnIdentifier.Raw> entities, Operator type, Term.Raw value)
+    {
+        this.entities = entities;
+        this.relationType = type;
+        this.value = value;
+    }
+
+    @Override
+    public boolean onToken()
+    {
+        return true;
+    }
+
+    @Override
+    protected Restriction newEQRestriction(CFMetaData cfm, VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        List<ColumnDefinition> columnDefs = getColumnDefinitions(cfm);
+        Term term = toTerm(toReceivers(cfm, columnDefs), value, cfm.ksName, boundNames);
+        return new TokenRestriction.EQ(columnDefs, term);
+    }
+
+    @Override
+    protected Restriction newINRestriction(CFMetaData cfm, VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        throw invalidRequest("%s cannot be used with the token function", operator());
+    }
+
+    @Override
+    protected Restriction newSliceRestriction(CFMetaData cfm,
+                                              VariableSpecifications boundNames,
+                                              Bound bound,
+                                              boolean inclusive) throws InvalidRequestException
+    {
+        List<ColumnDefinition> columnDefs = getColumnDefinitions(cfm);
+        Term term = toTerm(toReceivers(cfm, columnDefs), value, cfm.ksName, boundNames);
+        return new TokenRestriction.Slice(columnDefs, bound, inclusive, term);
+    }
+
+    @Override
+    protected Restriction newContainsRestriction(CFMetaData cfm, VariableSpecifications boundNames, boolean isKey) throws InvalidRequestException
+    {
+        throw invalidRequest("%s cannot be used with the token function", operator());
+    }
+
+    @Override
+    protected Term toTerm(List<? extends ColumnSpecification> receivers,
+                          Raw raw,
+                          String keyspace,
+                          VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        Term term = raw.prepare(keyspace, receivers.get(0));
+        term.collectMarkerSpecification(boundNames);
+        return term;
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("token(%s) %s %s", Tuples.tupleToString(entities), relationType, value);
+    }
+
+    /**
+     * Returns the definition of the columns to which apply the token restriction.
+     *
+     * @param cfm the column family metadata
+     * @return the definition of the columns to which apply the token restriction.
+     * @throws InvalidRequestException if the entity cannot be resolved
+     */
+    private List<ColumnDefinition> getColumnDefinitions(CFMetaData cfm) throws InvalidRequestException
+    {
+        List<ColumnDefinition> columnDefs = new ArrayList<>();
+        for ( ColumnIdentifier.Raw raw : entities)
+        {
+            columnDefs.add(toColumnDefinition(cfm, raw));
+        }
+        return columnDefs;
+    }
+
+    /**
+     * Returns the receivers for this relation.
+     *
+     * @param cfm the Column Family meta data
+     * @param columnDefs the column definitions
+     * @return the receivers for the specified relation.
+     * @throws InvalidRequestException if the relation is invalid
+     */
+    private static List<? extends ColumnSpecification> toReceivers(CFMetaData cfm,
+                                                                   List<ColumnDefinition> columnDefs)
+                                                                   throws InvalidRequestException
+    {
+
+        if (!columnDefs.equals(cfm.partitionKeyColumns()))
+        {
+            checkTrue(columnDefs.containsAll(cfm.partitionKeyColumns()),
+                      "The token() function must be applied to all partition key components or none of them");
+
+            checkContainsNoDuplicates(columnDefs, "The token() function contains duplicate partition key components");
+
+            checkContainsOnly(columnDefs, cfm.partitionKeyColumns(), "The token() function must contains only partition key components");
+
+            throw invalidRequest("The token function arguments must be in the partition key order: %s",
+                                 Joiner.on(", ").join(ColumnDefinition.toIdentifiers(cfm.partitionKeyColumns())));
+        }
+
+        ColumnDefinition firstColumn = columnDefs.get(0);
+        return Collections.singletonList(new ColumnSpecification(firstColumn.ksName,
+                                                                 firstColumn.cfName,
+                                                                 new ColumnIdentifier("partition key token", true),
+                                                                 StorageService.getPartitioner().getTokenValidator()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/Tuples.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Tuples.java b/src/java/org/apache/cassandra/cql3/Tuples.java
index b203546..a66c534 100644
--- a/src/java/org/apache/cassandra/cql3/Tuples.java
+++ b/src/java/org/apache/cassandra/cql3/Tuples.java
@@ -23,6 +23,7 @@ import java.util.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.cql3.Term.MultiColumnRaw;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.serializers.MarshalException;
@@ -319,7 +320,7 @@ public class Tuples
     /**
      * A raw marker for an IN list of tuples, like "SELECT ... WHERE (a, b, c) IN ?"
      */
-    public static class INRaw extends AbstractMarker.Raw
+    public static class INRaw extends AbstractMarker.Raw implements MultiColumnRaw
     {
         public INRaw(int bindIndex)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/VariableSpecifications.java b/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
index ef78619..0a55ced 100644
--- a/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
+++ b/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.cql3;
 
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 public class VariableSpecifications
@@ -31,6 +32,15 @@ public class VariableSpecifications
         this.specs = new ColumnSpecification[variableNames.size()];
     }
 
+    /**
+     * Returns an empty instance of <code>VariableSpecifications</code>.
+     * @return an empty instance of <code>VariableSpecifications</code>
+     */
+    public static VariableSpecifications empty()
+    {
+        return new VariableSpecifications(Collections.<ColumnIdentifier> emptyList());
+    }
+
     public int size()
     {
         return variableNames.size();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java
new file mode 100644
index 0000000..f137a77
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java
@@ -0,0 +1,36 @@
+/*
+ * 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.cql3.restrictions;
+
+/**
+ * Base class for <code>PrimaryKeyRestrictions</code>.
+ */
+abstract class AbstractPrimaryKeyRestrictions extends AbstractRestriction implements PrimaryKeyRestrictions
+{
+    @Override
+    public final boolean isEmpty()
+    {
+        return getColumnDefs().isEmpty();
+    }
+
+    @Override
+    public final int size()
+    {
+        return getColumnDefs().size();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
new file mode 100644
index 0000000..0ae7b22
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
@@ -0,0 +1,129 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+
+/**
+ * Base class for <code>Restriction</code>s
+ */
+abstract class AbstractRestriction  implements Restriction
+{
+    @Override
+    public  boolean isOnToken()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isMultiColumn()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isSlice()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isEQ()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isIN()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isContains()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean hasBound(Bound b)
+    {
+        return true;
+    }
+
+    @Override
+    public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
+    {
+        return values(options);
+    }
+
+    @Override
+    public boolean isInclusive(Bound b)
+    {
+        return true;
+    }
+
+    protected static ByteBuffer validateIndexedValue(ColumnSpecification columnSpec,
+                                                     ByteBuffer value)
+                                                     throws InvalidRequestException
+    {
+        checkNotNull(value, "Unsupported null value for indexed column %s", columnSpec.name);
+        checkFalse(value.remaining() > 0xFFFF, "Index expression values may not be larger than 64K");
+        return value;
+    }
+
+    /**
+     * Checks if the specified term is using the specified function.
+     *
+     * @param term the term to check
+     * @param ksName the function keyspace name
+     * @param functionName the function name
+     * @return <code>true</code> if the specified term is using the specified function, <code>false</code> otherwise.
+     */
+    protected static final boolean usesFunction(Term term, String ksName, String functionName)
+    {
+        return term != null && term.usesFunction(ksName, functionName);
+    }
+
+    /**
+     * Checks if one of the specified term is using the specified function.
+     *
+     * @param terms the terms to check
+     * @param ksName the function keyspace name
+     * @param functionName the function name
+     * @return <code>true</code> if onee of the specified term is using the specified function, <code>false</code> otherwise.
+     */
+    protected static final boolean usesFunction(List<Term> terms, String ksName, String functionName)
+    {
+        if (terms != null)
+            for (Term value : terms)
+                if (usesFunction(value, ksName, functionName))
+                    return true;
+        return false;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
new file mode 100644
index 0000000..8a57292
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
@@ -0,0 +1,159 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A <code>PrimaryKeyRestrictions</code> which forwards all its method calls to another 
+ * <code>PrimaryKeyRestrictions</code>. Subclasses should override one or more methods to modify the behavior 
+ * of the backing <code>PrimaryKeyRestrictions</code> as desired per the decorator pattern. 
+ */
+abstract class ForwardingPrimaryKeyRestrictions implements PrimaryKeyRestrictions
+{
+    /**
+     * Returns the backing delegate instance that methods are forwarded to.
+     * @return the backing delegate instance that methods are forwarded to.
+     */
+    protected abstract PrimaryKeyRestrictions getDelegate();
+
+    @Override
+    public boolean usesFunction(String ksName, String functionName)
+    {
+        return getDelegate().usesFunction(ksName, functionName);
+    }
+
+    @Override
+    public Collection<ColumnDefinition> getColumnDefs()
+    {
+        return getDelegate().getColumnDefs();
+    }
+
+    @Override
+    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException
+    {
+        return getDelegate().mergeWith(restriction);
+    }
+
+    @Override
+    public boolean hasSupportingIndex(SecondaryIndexManager secondaryIndexManager)
+    {
+        return getDelegate().hasSupportingIndex(secondaryIndexManager);
+    }
+
+    @Override
+    public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+    {
+        return getDelegate().values(options);
+    }
+
+    @Override
+    public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
+    {
+        return getDelegate().valuesAsComposites(options);
+    }
+
+    @Override
+    public List<ByteBuffer> bounds(Bound bound, QueryOptions options) throws InvalidRequestException
+    {
+        return getDelegate().bounds(bound, options);
+    }
+
+    @Override
+    public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
+    {
+        return getDelegate().boundsAsComposites(bound, options);
+    }
+
+    @Override
+    public boolean isInclusive(Bound bound)
+    {
+        return getDelegate().isInclusive(bound.reverse());
+    }
+
+    @Override
+    public boolean isEmpty()
+    {
+        return getDelegate().isEmpty();
+    }
+
+    @Override
+    public int size()
+    {
+        return getDelegate().size();
+    }
+
+    @Override
+    public boolean isOnToken()
+    {
+        return getDelegate().isOnToken();
+    }
+
+    @Override
+    public boolean isSlice()
+    {
+        return getDelegate().isSlice();
+    }
+
+    @Override
+    public boolean isEQ()
+    {
+        return getDelegate().isEQ();
+    }
+
+    @Override
+    public boolean isIN()
+    {
+        return getDelegate().isIN();
+    }
+
+    @Override
+    public boolean isContains()
+    {
+        return getDelegate().isContains();
+    }
+
+    @Override
+    public boolean isMultiColumn()
+    {
+        return getDelegate().isMultiColumn();
+    }
+
+    @Override
+    public boolean hasBound(Bound b)
+    {
+        return getDelegate().hasBound(b);
+    }
+
+    @Override
+    public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                     QueryOptions options) throws InvalidRequestException
+    {
+        getDelegate().addIndexExpressionTo(expressions, options);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
new file mode 100644
index 0000000..e3b3c4c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
@@ -0,0 +1,520 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.AbstractMarker;
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Tuples;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.composites.CBuilder;
+import org.apache.cassandra.db.composites.CType;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.composites.Composites;
+import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+public abstract class MultiColumnRestriction extends AbstractPrimaryKeyRestrictions
+{
+    protected final CType ctype;
+
+    /**
+     * The columns to which the restriction apply.
+     */
+    protected final List<ColumnDefinition> columnDefs;
+
+    public MultiColumnRestriction(CType ctype, List<ColumnDefinition> columnDefs)
+    {
+        this.ctype = ctype;
+        this.columnDefs = columnDefs;
+    }
+
+    @Override
+    public boolean isMultiColumn()
+    {
+        return true;
+    }
+
+    @Override
+    public Collection<ColumnDefinition> getColumnDefs()
+    {
+        return columnDefs;
+    }
+
+    @Override
+    public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
+    {
+        return Composites.toByteBuffers(valuesAsComposites(options));
+    }
+
+    @Override
+    public final PrimaryKeyRestrictions mergeWith(Restriction otherRestriction) throws InvalidRequestException
+    {
+            checkTrue(otherRestriction.isMultiColumn(),
+                      "Mixing single column relations and multi column relations on clustering columns is not allowed");
+            return doMergeWith((PrimaryKeyRestrictions) otherRestriction);
+    }
+
+    protected abstract PrimaryKeyRestrictions doMergeWith(PrimaryKeyRestrictions otherRestriction) throws InvalidRequestException;
+
+    /**
+     * Returns the names of the columns that are specified within this <code>Restrictions</code> and the other one
+     * as a comma separated <code>String</code>.
+     *
+     * @param otherRestrictions the other restrictions
+     * @return the names of the columns that are specified within this <code>Restrictions</code> and the other one
+     * as a comma separated <code>String</code>.
+     */
+    protected final String getColumnsInCommons(Restrictions otherRestrictions)
+    {
+        Set<ColumnDefinition> commons = new HashSet<>(getColumnDefs());
+        commons.retainAll(otherRestrictions.getColumnDefs());
+        StringBuilder builder = new StringBuilder();
+        for (ColumnDefinition columnDefinition : commons)
+        {
+            if (builder.length() != 0)
+                builder.append(" ,");
+            builder.append(columnDefinition.name);
+        }
+        return builder.toString();
+    }
+
+    @Override
+    public final boolean hasSupportingIndex(SecondaryIndexManager indexManager)
+    {
+        for (ColumnDefinition columnDef : columnDefs)
+        {
+            SecondaryIndex index = indexManager.getIndexForColumn(columnDef.name.bytes);
+            if (index != null && isSupportedBy(index))
+                return true;
+        }
+        return false;
+    }
+
+    /**
+     * Check if this type of restriction is supported for the specified column by the specified index.
+     * @param index the Secondary index
+     *
+     * @return <code>true</code> this type of restriction is supported by the specified index,
+     * <code>false</code> otherwise.
+     */
+    protected abstract boolean isSupportedBy(SecondaryIndex index);
+
+    public static class EQ  extends MultiColumnRestriction
+    {
+        protected final Term value;
+
+        public EQ(CType ctype, List<ColumnDefinition> columnDefs, Term value)
+        {
+            super(ctype, columnDefs);
+            this.value = value;
+        }
+
+        @Override
+        public boolean usesFunction(String ksName, String functionName)
+        {
+            return usesFunction(value, ksName, functionName);
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("EQ(%s)", value);
+        }
+
+        @Override
+        public PrimaryKeyRestrictions doMergeWith(PrimaryKeyRestrictions otherRestriction) throws InvalidRequestException
+        {
+            throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal",
+                                 getColumnsInCommons(otherRestriction));
+        }
+
+        @Override
+        public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
+        {
+            return Collections.singletonList(compositeValue(options));
+        }
+
+        @Override
+        public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
+        {
+            Composite prefix = compositeValue(options);
+            return Collections.singletonList(ctype.size() > prefix.size() && bound.isEnd()
+                                             ? prefix.end()
+                                             : prefix);
+        }
+
+        @Override
+        protected boolean isSupportedBy(SecondaryIndex index)
+        {
+            return index.supportsOperator(Operator.EQ);
+        }
+
+        private Composite compositeValue(QueryOptions options) throws InvalidRequestException
+        {
+            CBuilder builder = ctype.builder();
+            Tuples.Value t = ((Tuples.Value) value.bind(options));
+            List<ByteBuffer> values = t.getElements();
+            for (int i = 0; i < values.size(); i++)
+            {
+                ByteBuffer component = checkNotNull(values.get(i),
+                                                    "Invalid null value in condition for column %s",
+                                                    columnDefs.get(i).name);
+                builder.add(component);
+            }
+
+            return builder.build();
+        }
+
+        @Override
+        public final void addIndexExpressionTo(List<IndexExpression> expressions,
+                                               QueryOptions options) throws InvalidRequestException
+        {
+            Tuples.Value t = ((Tuples.Value) value.bind(options));
+            List<ByteBuffer> values = t.getElements();
+            for (int i = 0; i < values.size(); i++)
+            {
+                ColumnDefinition columnDef = columnDefs.get(i);
+                ByteBuffer component = validateIndexedValue(columnDef, values.get(i));
+                expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, component));
+            }
+        }
+    }
+
+    public abstract static class IN extends MultiColumnRestriction
+    {
+        @Override
+        public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
+        {
+            CBuilder builder = ctype.builder();
+            List<List<ByteBuffer>> splitInValues = splitValues(options);
+            // The IN query might not have listed the values in comparator order, so we need to re-sort
+            // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
+            TreeSet<Composite> inValues = new TreeSet<>(ctype);
+            for (List<ByteBuffer> components : splitInValues)
+            {
+                for (int i = 0; i < components.size(); i++)
+                    checkNotNull(components.get(i), "Invalid null value in condition for column " + columnDefs.get(i).name);
+
+                inValues.add(builder.buildWith(components));
+            }
+            return new ArrayList<>(inValues);
+        }
+
+        @Override
+        public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
+        {
+            CBuilder builder = ctype.builder();
+            List<List<ByteBuffer>> splitInValues = splitValues(options);
+            // The IN query might not have listed the values in comparator order, so we need to re-sort
+            // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
+            TreeSet<Composite> inValues = new TreeSet<>(ctype);
+            for (List<ByteBuffer> components : splitInValues)
+            {
+                for (int i = 0; i < components.size(); i++)
+                    checkNotNull(components.get(i), "Invalid null value in condition for column %s", columnDefs.get(i).name);
+
+                Composite prefix = builder.buildWith(components);
+                inValues.add(bound.isEnd() && builder.remainingCount() - components.size() > 0
+                             ? prefix.end()
+                             : prefix);
+            }
+            return new ArrayList<>(inValues);
+        }
+
+        @Override
+        public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                         QueryOptions options) throws InvalidRequestException
+        {
+            List<List<ByteBuffer>> splitInValues = splitValues(options);
+            checkTrue(splitInValues.size() == 1, "IN restrictions are not supported on indexed columns");
+
+            List<ByteBuffer> values = splitInValues.get(0);
+            checkTrue(values.size() == 1, "IN restrictions are not supported on indexed columns");
+
+            ColumnDefinition columnDef = columnDefs.get(0);
+            ByteBuffer component = validateIndexedValue(columnDef, values.get(0));
+            expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, component));
+        }
+
+        public IN(CType ctype, List<ColumnDefinition> columnDefs)
+        {
+            super(ctype, columnDefs);
+        }
+
+        @Override
+        public boolean isIN()
+        {
+            return true;
+        }
+
+        @Override
+        public PrimaryKeyRestrictions doMergeWith(PrimaryKeyRestrictions otherRestrictions) throws InvalidRequestException
+        {
+            throw invalidRequest("%s cannot be restricted by more than one relation if it includes a IN",
+                                 getColumnsInCommons(otherRestrictions));
+        }
+
+        @Override
+        protected boolean isSupportedBy(SecondaryIndex index)
+        {
+            return index.supportsOperator(Operator.IN);
+        }
+
+        protected abstract List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException;
+    }
+
+    /**
+     * An IN restriction that has a set of terms for in values.
+     * For example: "SELECT ... WHERE (a, b, c) IN ((1, 2, 3), (4, 5, 6))" or "WHERE (a, b, c) IN (?, ?)"
+     */
+    public static class InWithValues extends MultiColumnRestriction.IN
+    {
+        protected final List<Term> values;
+
+        public InWithValues(CType ctype, List<ColumnDefinition> columnDefs, List<Term> values)
+        {
+            super(ctype, columnDefs);
+            this.values = values;
+        }
+
+        @Override
+        public boolean usesFunction(String ksName, String functionName)
+        {
+            return usesFunction(values, ksName, functionName);
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("IN(%s)", values);
+        }
+
+        @Override
+        protected List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
+        {
+            List<List<ByteBuffer>> buffers = new ArrayList<>(values.size());
+            for (Term value : values)
+            {
+                Term.MultiItemTerminal term = (Term.MultiItemTerminal) value.bind(options);
+                buffers.add(term.getElements());
+            }
+            return buffers;
+        }
+    }
+
+    /**
+     * An IN restriction that uses a single marker for a set of IN values that are tuples.
+     * For example: "SELECT ... WHERE (a, b, c) IN ?"
+     */
+    public static class InWithMarker extends MultiColumnRestriction.IN
+    {
+        protected final AbstractMarker marker;
+
+        public InWithMarker(CType ctype, List<ColumnDefinition> columnDefs, AbstractMarker marker)
+        {
+            super(ctype, columnDefs);
+            this.marker = marker;
+        }
+
+        @Override
+        public boolean usesFunction(String ksName, String functionName)
+        {
+            return false;
+        }
+
+        @Override
+        public String toString()
+        {
+            return "IN ?";
+        }
+
+        @Override
+        protected List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
+        {
+            Tuples.InMarker inMarker = (Tuples.InMarker) marker;
+            Tuples.InValue inValue = inMarker.bind(options);
+            checkNotNull(inValue, "Invalid null value for IN restriction");
+            return inValue.getSplitValues();
+        }
+    }
+
+    public static class Slice extends MultiColumnRestriction
+    {
+        private final TermSlice slice;
+
+        public Slice(CType ctype, List<ColumnDefinition> columnDefs, Bound bound, boolean inclusive, Term term)
+        {
+            this(ctype, columnDefs, TermSlice.newInstance(bound, inclusive, term));
+        }
+
+        private Slice(CType ctype, List<ColumnDefinition> columnDefs, TermSlice slice)
+        {
+            super(ctype, columnDefs);
+            this.slice = slice;
+        }
+
+        @Override
+        public boolean isSlice()
+        {
+            return true;
+        }
+
+        @Override
+        public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
+        {
+            return Composites.toByteBuffers(boundsAsComposites(b, options));
+        }
+
+        @Override
+        public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
+        {
+            CBuilder builder = ctype.builder();
+            Iterator<ColumnDefinition> iter = columnDefs.iterator();
+            ColumnDefinition firstName = iter.next();
+            // A hack to preserve pre-6875 behavior for tuple-notation slices where the comparator mixes ASCENDING
+            // and DESCENDING orders.  This stores the bound for the first component; we will re-use it for all following
+            // components, even if they don't match the first component's reversal/non-reversal.  Note that this does *not*
+            // guarantee correct query results, it just preserves the previous behavior.
+            Bound firstComponentBound = !firstName.isReversedType() ? bound : bound.reverse();
+
+            if (!hasBound(firstComponentBound))
+            {
+                Composite prefix = builder.build();
+                return Collections.singletonList(builder.remainingCount() > 0 && bound.isEnd()
+                        ? prefix.end()
+                        : prefix);
+            }
+
+            List<ByteBuffer> vals = componentBounds(firstComponentBound, options);
+
+            ByteBuffer v = checkNotNull(vals.get(firstName.position()), "Invalid null value in condition for column %s", firstName.name);
+            builder.add(v);
+
+            while (iter.hasNext())
+            {
+                ColumnDefinition def = iter.next();
+                if (def.position() >= vals.size())
+                    break;
+
+                v = checkNotNull(vals.get(def.position()), "Invalid null value in condition for column %s", def.name);
+                builder.add(v);
+            }
+            Composite.EOC eoc =  eocFor(this, bound, firstComponentBound);
+            return Collections.singletonList(builder.build().withEOC(eoc));
+        }
+
+        @Override
+        public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                         QueryOptions options) throws InvalidRequestException
+        {
+            throw invalidRequest("Slice restrictions are not supported on indexed columns which are part of a multi column relation");
+        }
+
+        @Override
+        protected boolean isSupportedBy(SecondaryIndex index)
+        {
+            return slice.isSupportedBy(index);
+        }
+
+        private static Composite.EOC eocFor(Restriction r, Bound eocBound, Bound inclusiveBound)
+        {
+            if (eocBound.isStart())
+                return r.isInclusive(inclusiveBound) ? Composite.EOC.NONE : Composite.EOC.END;
+
+            return r.isInclusive(inclusiveBound) ? Composite.EOC.END : Composite.EOC.START;
+        }
+
+        @Override
+        public boolean hasBound(Bound b)
+        {
+            return slice.hasBound(b);
+        }
+
+        @Override
+        public boolean usesFunction(String ksName, String functionName)
+        {
+            return (slice.hasBound(Bound.START) && usesFunction(slice.bound(Bound.START), ksName, functionName))
+                    || (slice.hasBound(Bound.END) && usesFunction(slice.bound(Bound.END), ksName, functionName));
+        }
+
+        @Override
+        public boolean isInclusive(Bound b)
+        {
+            return slice.isInclusive(b);
+        }
+
+        @Override
+        public PrimaryKeyRestrictions doMergeWith(PrimaryKeyRestrictions otherRestriction) throws InvalidRequestException
+        {
+            checkTrue(otherRestriction.isSlice(),
+                      "Column \"%s\" cannot be restricted by both an equality and an inequality relation",
+                      getColumnsInCommons(otherRestriction));
+
+            Slice otherSlice = (Slice) otherRestriction;
+
+            checkFalse(hasBound(Bound.START) && otherSlice.hasBound(Bound.START),
+                       "More than one restriction was found for the start bound on %s",
+                       getColumnsInCommons(otherRestriction));
+            checkFalse(hasBound(Bound.END) && otherSlice.hasBound(Bound.END),
+                       "More than one restriction was found for the end bound on %s",
+                       getColumnsInCommons(otherRestriction));
+
+            List<ColumnDefinition> newColumnDefs = size() >= otherSlice.size() ?  columnDefs : otherSlice.columnDefs;
+            return new Slice(ctype,  newColumnDefs, slice.merge(otherSlice.slice));
+        }
+
+        @Override
+        public String toString()
+        {
+            return "SLICE" + slice;
+        }
+
+        /**
+         * Similar to bounds(), but returns one ByteBuffer per-component in the bound instead of a single
+         * ByteBuffer to represent the entire bound.
+         * @param b the bound type
+         * @param options the query options
+         * @return one ByteBuffer per-component in the bound
+         * @throws InvalidRequestException if the components cannot be retrieved
+         */
+        private List<ByteBuffer> componentBounds(Bound b, QueryOptions options) throws InvalidRequestException
+        {
+            Tuples.Value value = (Tuples.Value) slice.bound(b).bind(options);
+            return value.getElements();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java
new file mode 100644
index 0000000..5f977b7
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java
@@ -0,0 +1,40 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.util.List;
+
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A set of restrictions on a primary key part (partition key or clustering key).
+ *
+ */
+interface PrimaryKeyRestrictions extends Restriction, Restrictions
+{
+
+    @Override
+    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException;
+
+    public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException;
+
+    public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException;
+}


[2/5] cassandra git commit: Refactor SelectStatement and Restrictions

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/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 3360d40..022105c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -20,41 +20,50 @@ package org.apache.cassandra.cql3.statements;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import com.google.common.base.Joiner;
 import com.google.common.base.Objects;
 import com.google.common.base.Predicate;
-import com.google.common.collect.AbstractIterator;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 
 import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.statements.SingleColumnRestriction.Contains;
+import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
 import org.apache.cassandra.cql3.selection.RawSelector;
 import org.apache.cassandra.cql3.selection.Selection;
-import org.apache.cassandra.db.composites.*;
-import org.apache.cassandra.db.composites.Composite.EOC;
-import org.apache.cassandra.transport.messages.ResultMessage;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.filter.*;
-import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.composites.Composites;
+import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.filter.IDiskAtomFilter;
+import org.apache.cassandra.db.filter.NamesQueryFilter;
+import org.apache.cassandra.db.filter.SliceQueryFilter;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.dht.*;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageProxy;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.service.pager.*;
-import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.service.pager.Pageable;
+import org.apache.cassandra.service.pager.QueryPager;
+import org.apache.cassandra.service.pager.QueryPagers;
 import org.apache.cassandra.thrift.ThriftValidation;
-import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
 /**
  * Encapsulates a completely parsed SELECT query, including the target
  * column family, expression, result count, and ordering clause.
@@ -70,96 +79,43 @@ public class SelectStatement implements CQLStatement
     private final Selection selection;
     private final Term limit;
 
-    /** Restrictions on partitioning columns */
-    private final Restriction[] keyRestrictions;
-
-    /** Restrictions on clustering columns */
-    private final Restriction[] columnRestrictions;
-
-    /** Restrictions on non-primary key columns (i.e. secondary index restrictions) */
-    private final Map<ColumnIdentifier, Restriction> metadataRestrictions = new HashMap<ColumnIdentifier, Restriction>();
-
-    // All restricted columns not covered by the key or index filter
-    private final Set<ColumnDefinition> restrictedColumns = new HashSet<ColumnDefinition>();
-    private Restriction.Slice sliceRestriction;
-
-    private boolean isReversed;
-    private boolean onToken;
-    private boolean isKeyRange;
-    private boolean keyIsInRelation;
-    private boolean usesSecondaryIndexing;
+    private final StatementRestrictions restrictions;
 
-    private Map<ColumnIdentifier, Integer> orderingIndexes;
+    private final boolean isReversed;
 
-    private boolean selectsStaticColumns;
-    private boolean selectsOnlyStaticColumns;
+    /**
+     * The comparator used to orders results when multiple keys are selected (using IN).
+     */
+    private final Comparator<List<ByteBuffer>> orderingComparator;
 
     // Used by forSelection below
     private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier.Raw, Boolean>emptyMap(), false, false);
 
-    private static final Predicate<ColumnDefinition> isStaticFilter = new Predicate<ColumnDefinition>()
-    {
-        public boolean apply(ColumnDefinition def)
-        {
-            return def.isStatic();
-        }
-    };
-
-    public SelectStatement(CFMetaData cfm, int boundTerms, Parameters parameters, Selection selection, Term limit)
+    public SelectStatement(CFMetaData cfm,
+                           int boundTerms,
+                           Parameters parameters,
+                           Selection selection,
+                           StatementRestrictions restrictions,
+                           boolean isReversed,
+                           Comparator<List<ByteBuffer>> orderingComparator,
+                           Term limit)
     {
         this.cfm = cfm;
         this.boundTerms = boundTerms;
         this.selection = selection;
-        this.keyRestrictions = new Restriction[cfm.partitionKeyColumns().size()];
-        this.columnRestrictions = new Restriction[cfm.clusteringColumns().size()];
+        this.restrictions = restrictions;
+        this.isReversed = isReversed;
+        this.orderingComparator = orderingComparator;
         this.parameters = parameters;
         this.limit = limit;
-
-        // Now gather a few info on whether we should bother with static columns or not for this statement
-        initStaticColumnsInfo();
     }
 
+    @Override
     public boolean usesFunction(String ksName, String functionName)
     {
-        if (selection.usesFunction(ksName, functionName))
-            return true;
-        if (limit != null && limit.usesFunction(ksName, functionName))
-            return true;
-        for (Restriction restriction : metadataRestrictions.values())
-            if (restriction != null && restriction.usesFunction(ksName, functionName))
-                return true;
-        for (Restriction restriction : keyRestrictions)
-            if (restriction != null && restriction.usesFunction(ksName, functionName))
-                return true;
-        for (Restriction restriction : columnRestrictions)
-            if (restriction != null && restriction.usesFunction(ksName, functionName))
-                return true;
-        return false;
-    }
-
-    private void initStaticColumnsInfo()
-    {
-        if (!cfm.hasStaticColumns())
-            return;
-
-        // If it's a wildcard, we do select static but not only them
-        if (selection.isWildcard())
-        {
-            selectsStaticColumns = true;
-            return;
-        }
-
-        // Otherwise, check the selected columns
-        selectsStaticColumns = !Iterables.isEmpty(Iterables.filter(selection.getColumns(), isStaticFilter));
-        selectsOnlyStaticColumns = true;
-        for (ColumnDefinition def : selection.getColumns())
-        {
-            if (def.kind != ColumnDefinition.Kind.PARTITION_KEY && def.kind != ColumnDefinition.Kind.STATIC)
-            {
-                selectsOnlyStaticColumns = false;
-                break;
-            }
-        }
+        return selection.usesFunction(ksName, functionName)
+                || restrictions.usesFunction(ksName, functionName)
+                || (limit != null && limit.usesFunction(ksName, functionName));
     }
 
     // Creates a simple select based on the given selection.
@@ -167,7 +123,14 @@ public class SelectStatement implements CQLStatement
     // queried data through processColumnFamily.
     static SelectStatement forSelection(CFMetaData cfm, Selection selection)
     {
-        return new SelectStatement(cfm, 0, defaultParameters, selection, null);
+        return new SelectStatement(cfm,
+                                   0,
+                                   defaultParameters,
+                                   selection,
+                                   StatementRestrictions.empty(cfm),
+                                   false,
+                                   null,
+                                   null);
     }
 
     public ResultSet.Metadata getResultMetadata()
@@ -193,8 +156,7 @@ public class SelectStatement implements CQLStatement
     public ResultMessage.Rows execute(QueryState state, QueryOptions options) throws RequestExecutionException, RequestValidationException
     {
         ConsistencyLevel cl = options.getConsistency();
-        if (cl == null)
-            throw new InvalidRequestException("Invalid empty consistency level");
+        checkNotNull(cl, "Invalid empty consistency level");
 
         cl.validateForRead(keyspace());
 
@@ -216,13 +178,14 @@ public class SelectStatement implements CQLStatement
         }
 
         QueryPager pager = QueryPagers.pager(command, cl, state.getClientState(), options.getPagingState());
+
         if (selection.isAggregate())
             return pageAggregateQuery(pager, options, pageSize, now);
 
         // We can't properly do post-query ordering if we page (see #6722)
-        if (needsPostQueryOrdering())
-            throw new InvalidRequestException("Cannot page queries with both ORDER BY and a IN restriction on the partition key; you must either remove the "
-                                            + "ORDER BY or the IN and sort client side, or disable paging for this query");
+        checkFalse(needsPostQueryOrdering(),
+                  "Cannot page queries with both ORDER BY and a IN restriction on the partition key;"
+                  + " you must either remove the ORDER BY or the IN and sort client side, or disable paging for this query");
 
         List<Row> page = pager.fetchPage(pageSize);
         ResultMessage.Rows msg = processResults(page, options, limit, now);
@@ -236,7 +199,7 @@ public class SelectStatement implements CQLStatement
     private Pageable getPageableCommand(QueryOptions options, int limit, long now) throws RequestValidationException
     {
         int limitForQuery = updateLimitForQuery(limit);
-        if (isKeyRange || usesSecondaryIndexing)
+        if (restrictions.isKeyRange() || restrictions.usesSecondaryIndexing())
             return getRangeCommand(options, limitForQuery, now);
 
         List<ReadCommand> commands = getSliceCommands(options, limitForQuery, now);
@@ -330,9 +293,7 @@ public class SelectStatement implements CQLStatement
 
     private List<ReadCommand> getSliceCommands(QueryOptions options, int limit, long now) throws RequestValidationException
     {
-        Collection<ByteBuffer> keys = getKeys(options);
-        if (keys.isEmpty()) // in case of IN () for (the last column of) the partition key.
-            return null;
+        Collection<ByteBuffer> keys = restrictions.getPartitionKeys(options);
 
         List<ReadCommand> commands = new ArrayList<>(keys.size());
 
@@ -363,69 +324,12 @@ public class SelectStatement implements CQLStatement
         List<IndexExpression> expressions = getValidatedIndexExpressions(options);
         // The LIMIT provided by the user is the number of CQL row he wants returned.
         // We want to have getRangeSlice to count the number of columns, not the number of keys.
-        AbstractBounds<RowPosition> keyBounds = getKeyBounds(options);
+        AbstractBounds<RowPosition> keyBounds = restrictions.getPartitionKeyBounds(options);
         return keyBounds == null
              ? null
              : new RangeSliceCommand(keyspace(), columnFamily(), now,  filter, keyBounds, expressions, limit, !parameters.isDistinct, false);
     }
 
-    private AbstractBounds<RowPosition> getKeyBounds(QueryOptions options) throws InvalidRequestException
-    {
-        IPartitioner p = StorageService.getPartitioner();
-
-        if (onToken)
-        {
-            Token startToken = getTokenBound(Bound.START, options, p);
-            Token endToken = getTokenBound(Bound.END, options, p);
-
-            boolean includeStart = includeKeyBound(Bound.START);
-            boolean includeEnd = includeKeyBound(Bound.END);
-
-            /*
-             * If we ask SP.getRangeSlice() for (token(200), token(200)], it will happily return the whole ring.
-             * However, wrapping range doesn't really make sense for CQL, and we want to return an empty result
-             * in that case (CASSANDRA-5573). So special case to create a range that is guaranteed to be empty.
-             *
-             * In practice, we want to return an empty result set if either startToken > endToken, or both are
-             * equal but one of the bound is excluded (since [a, a] can contains something, but not (a, a], [a, a)
-             * or (a, a)). Note though that in the case where startToken or endToken is the minimum token, then
-             * this special case rule should not apply.
-             */
-            int cmp = startToken.compareTo(endToken);
-            if (!startToken.isMinimum() && !endToken.isMinimum() && (cmp > 0 || (cmp == 0 && (!includeStart || !includeEnd))))
-                return null;
-
-            RowPosition start = includeStart ? startToken.minKeyBound() : startToken.maxKeyBound();
-            RowPosition end = includeEnd ? endToken.maxKeyBound() : endToken.minKeyBound();
-
-            return new Range<RowPosition>(start, end);
-        }
-        else
-        {
-            ByteBuffer startKeyBytes = getKeyBound(Bound.START, options);
-            ByteBuffer finishKeyBytes = getKeyBound(Bound.END, options);
-
-            RowPosition startKey = RowPosition.ForKey.get(startKeyBytes, p);
-            RowPosition finishKey = RowPosition.ForKey.get(finishKeyBytes, p);
-
-            if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum())
-                return null;
-
-            if (includeKeyBound(Bound.START))
-            {
-                return includeKeyBound(Bound.END)
-                     ? new Bounds<RowPosition>(startKey, finishKey)
-                     : new IncludingExcludingBounds<RowPosition>(startKey, finishKey);
-            }
-            else
-            {
-                return includeKeyBound(Bound.END)
-                     ? new Range<RowPosition>(startKey, finishKey)
-                     : new ExcludingBounds<RowPosition>(startKey, finishKey);
-            }
-        }
-    }
-
     private ColumnSlice makeStaticSlice()
     {
         // Note: we could use staticPrefix.start() for the start bound, but EMPTY gives us the
@@ -444,18 +348,18 @@ public class SelectStatement implements CQLStatement
             // For distinct, we only care about fetching the beginning of each partition. If we don't have
             // static columns, we in fact only care about the first cell, so we query only that (we don't "group").
             // If we do have static columns, we do need to fetch the first full group (to have the static columns values).
-            return new SliceQueryFilter(ColumnSlice.ALL_COLUMNS_ARRAY, false, 1, selectsStaticColumns ? toGroup : -1);
+            return new SliceQueryFilter(ColumnSlice.ALL_COLUMNS_ARRAY, false, 1, selection.containsStaticColumns() ? toGroup : -1);
         }
-        else if (isColumnRange())
+        else if (restrictions.isColumnRange())
         {
-            List<Composite> startBounds = getRequestedBound(Bound.START, options);
-            List<Composite> endBounds = getRequestedBound(Bound.END, options);
+            List<Composite> startBounds = restrictions.getClusteringColumnsBoundsAsComposites(Bound.START, options);
+            List<Composite> endBounds = restrictions.getClusteringColumnsBoundsAsComposites(Bound.END, options);
             assert startBounds.size() == endBounds.size();
 
             // Handles fetching static columns. Note that for 2i, the filter is just used to restrict
             // the part of the index to query so adding the static slice would be useless and confusing.
             // For 2i, static columns are retrieve in CompositesSearcher with each index hit.
-            ColumnSlice staticSlice = selectsStaticColumns && !usesSecondaryIndexing
+            ColumnSlice staticSlice = selection.containsStaticColumns() && !restrictions.usesSecondaryIndexing()
                                     ? makeStaticSlice()
                                     : null;
 
@@ -550,187 +454,41 @@ public class SelectStatement implements CQLStatement
 
     private int getLimit(QueryOptions options) throws InvalidRequestException
     {
-        int l = Integer.MAX_VALUE;
         if (limit != null)
         {
-            ByteBuffer b = limit.bindAndGet(options);
-            if (b == null)
-                throw new InvalidRequestException("Invalid null value of limit");
+            ByteBuffer b = checkNotNull(limit.bindAndGet(options), "Invalid null value of limit");
 
             try
             {
                 Int32Type.instance.validate(b);
-                l = Int32Type.instance.compose(b);
+                int l = Int32Type.instance.compose(b);
+                checkTrue(l > 0, "LIMIT must be strictly positive");
+                return l;
             }
             catch (MarshalException e)
             {
                 throw new InvalidRequestException("Invalid limit value");
             }
         }
-
-        if (l <= 0)
-            throw new InvalidRequestException("LIMIT must be strictly positive");
-
-        return l;
+        return Integer.MAX_VALUE;
     }
 
     private int updateLimitForQuery(int limit)
     {
         // Internally, we don't support exclusive bounds for slices. Instead, we query one more element if necessary
         // and exclude it later (in processColumnFamily)
-        return sliceRestriction != null && (!sliceRestriction.isInclusive(Bound.START) || !sliceRestriction.isInclusive(Bound.END)) && limit != Integer.MAX_VALUE
+        return restrictions.isNonCompositeSliceWithExclusiveBounds() && limit != Integer.MAX_VALUE
              ? limit + 1
              : limit;
     }
 
-    private Collection<ByteBuffer> getKeys(final QueryOptions options) throws InvalidRequestException
-    {
-        List<ByteBuffer> keys = new ArrayList<ByteBuffer>();
-        CBuilder builder = cfm.getKeyValidatorAsCType().builder();
-        for (ColumnDefinition def : cfm.partitionKeyColumns())
-        {
-            Restriction r = keyRestrictions[def.position()];
-            assert r != null && !r.isSlice();
-
-            List<ByteBuffer> values = r.values(options);
-
-            if (builder.remainingCount() == 1)
-            {
-                for (ByteBuffer val : values)
-                {
-                    if (val == null)
-                        throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", def.name));
-                    keys.add(builder.buildWith(val).toByteBuffer());
-                }
-            }
-            else
-            {
-                // Note: for backward compatibility reasons, we let INs with 1 value slide
-                if (values.size() != 1)
-                    throw new InvalidRequestException("IN is only supported on the last column of the partition key");
-                ByteBuffer val = values.get(0);
-                if (val == null)
-                    throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", def.name));
-                builder.add(val);
-            }
-        }
-        return keys;
-    }
-
-    private ByteBuffer getKeyBound(Bound b, QueryOptions options) throws InvalidRequestException
-    {
-        // Deal with unrestricted partition key components (special-casing is required to deal with 2i queries on the first
-        // component of a composite partition key).
-        for (int i = 0; i < keyRestrictions.length; i++)
-            if (keyRestrictions[i] == null)
-                return ByteBufferUtil.EMPTY_BYTE_BUFFER;
-
-        // We deal with IN queries for keys in other places, so we know buildBound will return only one result
-        return buildBound(b, cfm.partitionKeyColumns(), keyRestrictions, false, cfm.getKeyValidatorAsCType(), options).get(0).toByteBuffer();
-    }
-
-    private Token getTokenBound(Bound b, QueryOptions options, IPartitioner p) throws InvalidRequestException
-    {
-        assert onToken;
-
-        Restriction restriction = keyRestrictions[0];
-
-        assert !restriction.isMultiColumn() : "Unexpectedly got a multi-column restriction on a partition key for a range query";
-        SingleColumnRestriction keyRestriction = (SingleColumnRestriction)restriction;
-
-        ByteBuffer value;
-        if (keyRestriction.isEQ())
-        {
-            value = keyRestriction.values(options).get(0);
-        }
-        else
-        {
-            SingleColumnRestriction.Slice slice = (SingleColumnRestriction.Slice)keyRestriction;
-            if (!slice.hasBound(b))
-                return p.getMinimumToken();
-
-            value = slice.bound(b, options);
-        }
-
-        if (value == null)
-            throw new InvalidRequestException("Invalid null token value");
-        return p.getTokenFactory().fromByteArray(value);
-    }
-
-    private boolean includeKeyBound(Bound b)
-    {
-        for (Restriction r : keyRestrictions)
-        {
-            if (r == null)
-                return true;
-            else if (r.isSlice())
-            {
-                assert !r.isMultiColumn() : "Unexpectedly got multi-column restriction on partition key";
-                return ((SingleColumnRestriction.Slice)r).isInclusive(b);
-            }
-        }
-        // All equality
-        return true;
-    }
-
-    private boolean isColumnRange()
-    {
-        // Due to CASSANDRA-5762, we always do a slice for CQL3 tables (not dense, composite).
-        // Static CF (non dense but non composite) never entails a column slice however
-        if (!cfm.comparator.isDense())
-            return cfm.comparator.isCompound();
-
-        // Otherwise (i.e. for compact table where we don't have a row marker anyway and thus don't care about CASSANDRA-5762),
-        // it is a range query if it has at least one the column alias for which no relation is defined or is not EQ.
-        for (Restriction r : columnRestrictions)
-        {
-            if (r == null || r.isSlice())
-                return true;
-        }
-        return false;
-    }
-
     private SortedSet<CellName> getRequestedColumns(QueryOptions options) throws InvalidRequestException
     {
         // Note: getRequestedColumns don't handle static columns, but due to CASSANDRA-5762
         // we always do a slice for CQL3 tables, so it's ok to ignore them here
-        assert !isColumnRange();
-
-        CompositesBuilder builder = new CompositesBuilder(cfm.comparator.prefixBuilder(), cfm.comparator);
-        Iterator<ColumnDefinition> idIter = cfm.clusteringColumns().iterator();
-        for (int i = 0; i < columnRestrictions.length; i++)
-        {
-            Restriction r = columnRestrictions[i];
-            ColumnDefinition def = idIter.next();
-            assert r != null && !r.isSlice();
-
-            if (r.isEQ() || !r.isMultiColumn())
-            {
-                List<ByteBuffer> values = r.values(options);
-                if (values.isEmpty())
-                    return null;
-                builder.addEachElementToAll(values);
-            }
-            else
-            {
-                // we have a multi-column IN restriction
-                List<List<ByteBuffer>> splitValues = ((MultiColumnRestriction.IN) r).splitValues(options);
-                if (splitValues.isEmpty())
-                    return null;
-
-                builder.addAllElementsToAll(splitValues);
-
-                // increment i to skip the remainder of the multicolumn restriction
-                i += splitValues.get(0).size() - 1;
-            }
-
-            if (builder.containsNull())
-                throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s",
-                        def.name));
-        }
-
-        SortedSet<CellName> columns = new TreeSet<>(cfm.comparator);
-        for (Composite composite : builder.build())
+        assert !restrictions.isColumnRange();
+        SortedSet<CellName> columns = new TreeSet<CellName>(cfm.comparator);
+        for (Composite composite : restrictions.getClusteringColumnsAsComposites(options))
             columns.addAll(addSelectedColumns(composite));
         return columns;
     }
@@ -743,10 +501,6 @@ public class SelectStatement implements CQLStatement
         }
         else
         {
-            // Collections require doing a slice query because a given collection is a
-            // non-know set of columns, so we shouldn't get there
-            assert !selectACollection();
-
             SortedSet<CellName> columns = new TreeSet<CellName>(cfm.comparator);
 
             // We need to query the selected column as well as the marker
@@ -760,7 +514,7 @@ public class SelectStatement implements CQLStatement
 
                 // selected columns
                 for (ColumnDefinition def : selection.getColumns())
-                    if (def.kind == ColumnDefinition.Kind.REGULAR || def.kind == ColumnDefinition.Kind.STATIC)
+                    if (def.isRegular() || def.isStatic())
                         columns.add(cfm.comparator.create(prefix, def));
             }
             else
@@ -773,368 +527,45 @@ public class SelectStatement implements CQLStatement
         }
     }
 
-    /** Returns true if a non-frozen collection is selected, false otherwise. */
-    private boolean selectACollection()
-    {
-        if (!cfm.comparator.hasCollections())
-            return false;
-
-        for (ColumnDefinition def : selection.getColumns())
-        {
-            if (def.type.isCollection() && def.type.isMultiCell())
-                return true;
-        }
-
-        return false;
-    }
-
-    private static List<Composite> buildBound(Bound bound,
-                                              List<ColumnDefinition> defs,
-                                              Restriction[] restrictions,
-                                              boolean isReversed,
-                                              CType type,
-                                              QueryOptions options) throws InvalidRequestException
-    {
-        CBuilder builder = type.builder();
-
-        // check the first restriction to see if we're dealing with a multi-column restriction
-        if (!defs.isEmpty())
-        {
-            Restriction firstRestriction = restrictions[0];
-            if (firstRestriction != null && firstRestriction.isMultiColumn())
-            {
-                if (firstRestriction.isSlice())
-                    return buildMultiColumnSliceBound(bound, defs, (MultiColumnRestriction.Slice) firstRestriction, isReversed, builder, options);
-                else if (firstRestriction.isIN())
-                    return buildMultiColumnInBound(bound, defs, (MultiColumnRestriction.IN) firstRestriction, isReversed, builder, type, options);
-                else
-                    return buildMultiColumnEQBound(bound, defs, (MultiColumnRestriction.EQ) firstRestriction, isReversed, builder, options);
-            }
-        }
-
-        CompositesBuilder compositeBuilder = new CompositesBuilder(builder, isReversed ? type.reverseComparator() : type);
-        // The end-of-component of composite doesn't depend on whether the
-        // component type is reversed or not (i.e. the ReversedType is applied
-        // to the component comparator but not to the end-of-component itself),
-        // it only depends on whether the slice is reversed
-        Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
-        for (Iterator<ColumnDefinition> iter = defs.iterator(); iter.hasNext();)
-        {
-            ColumnDefinition def = iter.next();
-
-            // In a restriction, we always have Bound.START < Bound.END for the "base" comparator.
-            // So if we're doing a reverse slice, we must inverse the bounds when giving them as start and end of the slice filter.
-            // But if the actual comparator itself is reversed, we must inversed the bounds too.
-            Bound b = isReversed == isReversedType(def) ? bound : Bound.reverse(bound);
-            Restriction r = restrictions[def.position()];
-            if (isNullRestriction(r, b) || !r.canEvaluateWithSlices())
-            {
-                // There wasn't any non EQ relation on that key, we select all records having the preceding component as prefix.
-                // For composites, if there was preceding component and we're computing the end, we must change the last component
-                // End-Of-Component, otherwise we would be selecting only one record.
-                EOC eoc = !compositeBuilder.isEmpty() && eocBound == Bound.END ? EOC.END : EOC.NONE;
-                return compositeBuilder.buildWithEOC(eoc);
-            }
-            if (r.isSlice())
-            {
-                compositeBuilder.addElementToAll(getSliceValue(r, b, options));
-                Operator relType = ((Restriction.Slice) r).getRelation(eocBound, b);
-                return compositeBuilder.buildWithEOC(eocForRelation(relType));
-            }
-
-            compositeBuilder.addEachElementToAll(r.values(options));
-
-            if (compositeBuilder.containsNull())
-                throw new InvalidRequestException(
-                        String.format("Invalid null clustering key part %s", def.name));
-        }
-        // Means no relation at all or everything was an equal
-        // Note: if the builder is "full", there is no need to use the end-of-component bit. For columns selection,
-        // it would be harmless to do it. However, we use this method got the partition key too. And when a query
-        // with 2ndary index is done, and with the the partition provided with an EQ, we'll end up here, and in that
-        // case using the eoc would be bad, since for the random partitioner we have no guarantee that
-        // prefix.end() will sort after prefix (see #5240).
-        EOC eoc = eocBound == Bound.END && compositeBuilder.hasRemaining() ? EOC.END : EOC.NONE;
-        return compositeBuilder.buildWithEOC(eoc);
-    }
-
-    private static Composite.EOC eocForRelation(Operator op)
-    {
-        switch (op)
-        {
-            case LT:
-                // < X => using startOf(X) as finish bound
-                return Composite.EOC.START;
-            case GT:
-            case LTE:
-                // > X => using endOf(X) as start bound
-                // <= X => using endOf(X) as finish bound
-                return Composite.EOC.END;
-            default:
-                // >= X => using X as start bound (could use START_OF too)
-                // = X => using X
-                return Composite.EOC.NONE;
-        }
-    }
-
-    private static List<Composite> buildMultiColumnSliceBound(Bound bound,
-                                                              List<ColumnDefinition> defs,
-                                                              MultiColumnRestriction.Slice slice,
-                                                              boolean isReversed,
-                                                              CBuilder builder,
-                                                              QueryOptions options) throws InvalidRequestException
-    {
-        Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
-
-        Iterator<ColumnDefinition> iter = defs.iterator();
-        ColumnDefinition firstName = iter.next();
-        // A hack to preserve pre-6875 behavior for tuple-notation slices where the comparator mixes ASCENDING
-        // and DESCENDING orders.  This stores the bound for the first component; we will re-use it for all following
-        // components, even if they don't match the first component's reversal/non-reversal.  Note that this does *not*
-        // guarantee correct query results, it just preserves the previous behavior.
-        Bound firstComponentBound = isReversed == isReversedType(firstName) ? bound : Bound.reverse(bound);
-
-        if (!slice.hasBound(firstComponentBound))
-        {
-            Composite prefix = builder.build();
-            return Collections.singletonList(builder.remainingCount() > 0 && eocBound == Bound.END
-                    ? prefix.end()
-                    : prefix);
-        }
-
-        List<ByteBuffer> vals = slice.componentBounds(firstComponentBound, options);
-
-        ByteBuffer v = vals.get(firstName.position());
-        if (v == null)
-            throw new InvalidRequestException("Invalid null value in condition for column " + firstName.name);
-        builder.add(v);
-
-        while (iter.hasNext())
-        {
-            ColumnDefinition def = iter.next();
-            if (def.position() >= vals.size())
-                break;
-
-            v = vals.get(def.position());
-            if (v == null)
-                throw new InvalidRequestException("Invalid null value in condition for column " + def.name);
-            builder.add(v);
-        }
-        Operator relType = slice.getRelation(eocBound, firstComponentBound);
-        return Collections.singletonList(builder.build().withEOC(eocForRelation(relType)));
-    }
-
-    private static List<Composite> buildMultiColumnInBound(Bound bound,
-                                                           List<ColumnDefinition> defs,
-                                                           MultiColumnRestriction.IN restriction,
-                                                           boolean isReversed,
-                                                           CBuilder builder,
-                                                           CType type,
-                                                           QueryOptions options) throws InvalidRequestException
-    {
-        List<List<ByteBuffer>> splitInValues = restriction.splitValues(options);
-        Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
-
-        // The IN query might not have listed the values in comparator order, so we need to re-sort
-        // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
-        TreeSet<Composite> inValues = new TreeSet<>(isReversed ? type.reverseComparator() : type);
-        for (List<ByteBuffer> components : splitInValues)
-        {
-            for (int i = 0; i < components.size(); i++)
-                if (components.get(i) == null)
-                    throw new InvalidRequestException("Invalid null value in condition for column " + defs.get(i));
-
-            Composite prefix = builder.buildWith(components);
-            inValues.add(eocBound == Bound.END && builder.remainingCount() - components.size() > 0
-                         ? prefix.end()
-                         : prefix);
-        }
-        return new ArrayList<>(inValues);
-    }
-
-    private static List<Composite> buildMultiColumnEQBound(Bound bound,
-                                                           List<ColumnDefinition> defs,
-                                                           MultiColumnRestriction.EQ restriction,
-                                                           boolean isReversed,
-                                                           CBuilder builder,
-                                                           QueryOptions options) throws InvalidRequestException
-    {
-        Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
-        List<ByteBuffer> values = restriction.values(options);
-        for (int i = 0; i < values.size(); i++)
-        {
-            ByteBuffer component = values.get(i);
-            if (component == null)
-                throw new InvalidRequestException("Invalid null value in condition for column " + defs.get(i));
-            builder.add(component);
-        }
-
-        Composite prefix = builder.build();
-        return Collections.singletonList(builder.remainingCount() > 0 && eocBound == Bound.END
-                                         ? prefix.end()
-                                         : prefix);
-    }
-
-    private static boolean isNullRestriction(Restriction r, Bound b)
-    {
-        return r == null || (r.isSlice() && !((Restriction.Slice)r).hasBound(b));
-    }
-
-    private static ByteBuffer getSliceValue(Restriction r, Bound b, QueryOptions options) throws InvalidRequestException
-    {
-        Restriction.Slice slice = (Restriction.Slice)r;
-        assert slice.hasBound(b);
-        ByteBuffer val = slice.bound(b, options);
-        if (val == null)
-            throw new InvalidRequestException(String.format("Invalid null clustering key part %s", r));
-        return val;
-    }
-
-    private List<Composite> getRequestedBound(Bound b, QueryOptions options) throws InvalidRequestException
-    {
-        assert isColumnRange();
-        return buildBound(b, cfm.clusteringColumns(), columnRestrictions, isReversed, cfm.comparator, options);
-    }
-
     public List<IndexExpression> getValidatedIndexExpressions(QueryOptions options) throws InvalidRequestException
     {
-        if (!usesSecondaryIndexing || restrictedColumns.isEmpty())
+        if (!restrictions.usesSecondaryIndexing())
             return Collections.emptyList();
 
-        List<IndexExpression> expressions = new ArrayList<IndexExpression>();
-        for (ColumnDefinition def : restrictedColumns)
-        {
-            Restriction restriction;
-            switch (def.kind)
-            {
-                case PARTITION_KEY:
-                    restriction = keyRestrictions[def.position()];
-                    break;
-                case CLUSTERING_COLUMN:
-                    restriction = columnRestrictions[def.position()];
-                    break;
-                case REGULAR:
-                case STATIC:
-                    restriction = metadataRestrictions.get(def.name);
-                    break;
-                default:
-                    // We don't allow restricting a COMPACT_VALUE for now in prepare.
-                    throw new AssertionError();
-            }
-
-            if (restriction.isSlice())
-            {
-                Restriction.Slice slice = (Restriction.Slice)restriction;
-                for (Bound b : Bound.values())
-                {
-                    if (slice.hasBound(b))
-                    {
-                        ByteBuffer value = validateIndexedValue(def, slice.bound(b, options));
-                        Operator op = slice.getIndexOperator(b);
-                        // If the underlying comparator for name is reversed, we need to reverse the IndexOperator: user operation
-                        // always refer to the "forward" sorting even if the clustering order is reversed, but the 2ndary code does
-                        // use the underlying comparator as is.
-                        if (def.type instanceof ReversedType)
-                            op = reverse(op);
-                        expressions.add(new IndexExpression(def.name.bytes, op, value));
-                    }
-                }
-            }
-            else if (restriction.isContains())
-            {
-                SingleColumnRestriction.Contains contains = (SingleColumnRestriction.Contains)restriction;
-                for (ByteBuffer value : contains.values(options))
-                {
-                    validateIndexedValue(def, value);
-                    expressions.add(new IndexExpression(def.name.bytes, Operator.CONTAINS, value));
-                }
-                for (ByteBuffer key : contains.keys(options))
-                {
-                    validateIndexedValue(def, key);
-                    expressions.add(new IndexExpression(def.name.bytes, Operator.CONTAINS_KEY, key));
-                }
-            }
-            else
-            {
-                List<ByteBuffer> values = restriction.values(options);
-
-                if (values.size() != 1)
-                    throw new InvalidRequestException("IN restrictions are not supported on indexed columns");
+        List<IndexExpression> expressions = restrictions.getIndexExpressions(options);
 
-                ByteBuffer value = validateIndexedValue(def, values.get(0));
-                expressions.add(new IndexExpression(def.name.bytes, Operator.EQ, value));
-            }
-        }
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
+        SecondaryIndexManager secondaryIndexManager = cfs.indexManager;
+        secondaryIndexManager.validateIndexSearchersForQuery(expressions);
 
-        if (usesSecondaryIndexing)
-        {
-            ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
-            SecondaryIndexManager secondaryIndexManager = cfs.indexManager;
-            secondaryIndexManager.validateIndexSearchersForQuery(expressions);
-        }
-        
         return expressions;
     }
 
-    private static ByteBuffer validateIndexedValue(ColumnDefinition def, ByteBuffer value) throws InvalidRequestException
-    {
-        if (value == null)
-            throw new InvalidRequestException(String.format("Unsupported null value for indexed column %s", def.name));
-        if (value.remaining() > 0xFFFF)
-            throw new InvalidRequestException("Index expression values may not be larger than 64K");
-        return value;
-    }
-
     private CellName makeExclusiveSliceBound(Bound bound, CellNameType type, QueryOptions options) throws InvalidRequestException
     {
-        if (sliceRestriction.isInclusive(bound))
+        if (restrictions.areRequestedBoundsInclusive(bound))
             return null;
 
-        if (sliceRestriction.isMultiColumn())
-            return type.makeCellName(((MultiColumnRestriction.Slice) sliceRestriction).componentBounds(bound, options).toArray());
-        else
-            return type.makeCellName(sliceRestriction.bound(bound, options));
+       return type.makeCellName(restrictions.getClusteringColumnsBounds(bound, options).get(0));
     }
 
     private Iterator<Cell> applySliceRestriction(final Iterator<Cell> cells, final QueryOptions options) throws InvalidRequestException
     {
-        assert sliceRestriction != null;
-
         final CellNameType type = cfm.comparator;
+
         final CellName excludedStart = makeExclusiveSliceBound(Bound.START, type, options);
         final CellName excludedEnd = makeExclusiveSliceBound(Bound.END, type, options);
 
-        return new AbstractIterator<Cell>()
+        return Iterators.filter(cells, new Predicate<Cell>()
         {
-            protected Cell computeNext()
+            public boolean apply(Cell c)
             {
-                while (cells.hasNext())
-                {
-                    Cell c = cells.next();
-
-                    // For dynamic CF, the column could be out of the requested bounds (because we don't support strict bounds internally (unless
-                    // the comparator is composite that is)), filter here
-                    if ( (excludedStart != null && type.compare(c.name(), excludedStart) == 0)
-                      || (excludedEnd != null && type.compare(c.name(), excludedEnd) == 0) )
-                        continue;
-
-                    return c;
-                }
-                return endOfData();
+                // For dynamic CF, the column could be out of the requested bounds (because we don't support strict bounds internally (unless
+                // the comparator is composite that is)), filter here
+                return !((excludedStart != null && type.compare(c.name(), excludedStart) == 0)
+                            || (excludedEnd != null && type.compare(c.name(), excludedEnd) == 0));
             }
-        };
-    }
-
-    private static Operator reverse(Operator op)
-    {
-        switch (op)
-        {
-            case LT:  return Operator.GT;
-            case LTE: return Operator.GTE;
-            case GT:  return Operator.LT;
-            case GTE: return Operator.LTE;
-            default: return op;
-        }
+        });
     }
 
     private ResultSet process(List<Row> rows, QueryOptions options, int limit, long now) throws InvalidRequestException
@@ -1178,7 +609,7 @@ public class SelectStatement implements CQLStatement
         }
 
         Iterator<Cell> cells = cf.getSortedColumns().iterator();
-        if (sliceRestriction != null)
+        if (restrictions.isNonCompositeSliceWithExclusiveBounds())
             cells = applySliceRestriction(cells, options);
 
         CQL3Row.RowIterator iter = cfm.comparator.CQL3RowBuilder(cfm, now).group(cells);
@@ -1187,7 +618,7 @@ public class SelectStatement implements CQLStatement
         // 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())
+        if (staticRow != null && !iter.hasNext() && !restrictions.usesSecondaryIndexing() && restrictions.hasNoClusteringColumnsRestriction())
         {
             result.newRow(options.getProtocolVersion());
             for (ColumnDefinition def : selection.getColumns())
@@ -1259,18 +690,10 @@ public class SelectStatement implements CQLStatement
         result.add(row.getColumn(def.name));
     }
 
-    private boolean hasNoClusteringColumnsRestriction()
-    {
-        for (int i = 0; i < columnRestrictions.length; i++)
-            if (columnRestrictions[i] != null)
-                return false;
-        return true;
-    }
-
     private boolean needsPostQueryOrdering()
     {
         // We need post-query ordering only for queries with IN on the partition key and an ORDER BY.
-        return keyIsInRelation && !parameters.orderings.isEmpty();
+        return restrictions.keyIsInRelation() && !parameters.orderings.isEmpty();
     }
 
     /**
@@ -1281,83 +704,7 @@ public class SelectStatement implements CQLStatement
         if (cqlRows.size() == 0 || !needsPostQueryOrdering())
             return;
 
-        assert orderingIndexes != null;
-
-        List<Integer> idToSort = new ArrayList<Integer>();
-        List<Comparator<ByteBuffer>> sorters = new ArrayList<Comparator<ByteBuffer>>();
-
-        for (ColumnIdentifier.Raw identifier : parameters.orderings.keySet())
-        {
-            ColumnDefinition orderingColumn = cfm.getColumnDefinition(identifier.prepare(cfm));
-            idToSort.add(orderingIndexes.get(orderingColumn.name));
-            sorters.add(orderingColumn.type);
-        }
-
-        Comparator<List<ByteBuffer>> comparator = idToSort.size() == 1
-                                                ? new SingleColumnComparator(idToSort.get(0), sorters.get(0))
-                                                : new CompositeComparator(sorters, idToSort);
-        Collections.sort(cqlRows.rows, comparator);
-    }
-
-    private static boolean isReversedType(ColumnDefinition def)
-    {
-        return def.type instanceof ReversedType;
-    }
-
-    private boolean columnFilterIsIdentity()
-    {
-        for (Restriction r : columnRestrictions)
-        {
-            if (r != null)
-                return false;
-        }
-        return true;
-    }
-
-    private boolean hasClusteringColumnsRestriction()
-    {
-        for (int i = 0; i < columnRestrictions.length; i++)
-            if (columnRestrictions[i] != null)
-                return true;
-        return false;
-    }
-
-    private void validateDistinctSelection()
-    throws InvalidRequestException
-    {
-        Collection<ColumnDefinition> requestedColumns = selection.getColumns();
-        for (ColumnDefinition def : requestedColumns)
-            if (def.kind != ColumnDefinition.Kind.PARTITION_KEY && def.kind != ColumnDefinition.Kind.STATIC)
-                throw new InvalidRequestException(String.format("SELECT DISTINCT queries must only request partition key columns and/or static columns (not %s)", def.name));
-
-        // If it's a key range, we require that all partition key columns are selected so we don't have to bother with post-query grouping.
-        if (!isKeyRange)
-            return;
-
-        for (ColumnDefinition def : cfm.partitionKeyColumns())
-            if (!requestedColumns.contains(def))
-                throw new InvalidRequestException(String.format("SELECT DISTINCT queries must request all the partition key columns (missing %s)", def.name));
-    }
-
-    /**
-     * Checks if the specified column is restricted by multiple contains or contains key.
-     *
-     * @param columnDef the definition of the column to check
-     * @return <code>true</code> the specified column is restricted by multiple contains or contains key,
-     * <code>false</code> otherwise
-     */
-    private boolean isRestrictedByMultipleContains(ColumnDefinition columnDef)
-    {
-        if (!columnDef.type.isCollection())
-            return false;
-
-        Restriction restriction = metadataRestrictions.get(columnDef.name);
-
-        if (!(restriction instanceof Contains))
-            return false;
-
-        Contains contains = (Contains) restriction;
-        return (contains.numberOfValues() + contains.numberOfKeys()) > 1;
+        Collections.sort(cqlRows.rows, orderingComparator);
     }
 
     public static class RawStatement extends CFStatement
@@ -1385,136 +732,65 @@ public class SelectStatement implements CQLStatement
                                 ? Selection.wildcard(cfm)
                                 : Selection.fromSelectors(cfm, selectClause);
 
-            SelectStatement stmt = new SelectStatement(cfm, boundNames.size(), parameters, selection, prepareLimit(boundNames));
-
-            /*
-             * WHERE clause. For a given entity, rules are:
-             *   - EQ relation conflicts with anything else (including a 2nd EQ)
-             *   - Can't have more than one LT(E) relation (resp. GT(E) relation)
-             *   - IN relation are restricted to row keys (for now) and conflicts with anything else
-             *     (we could allow two IN for the same entity but that doesn't seem very useful)
-             *   - The value_alias cannot be restricted in any way (we don't support wide rows with indexed value in CQL so far)
-             */
-            boolean hasQueriableIndex = false;
-            boolean hasQueriableClusteringColumnIndex = false;
-            boolean hasSingleColumnRelations = false;
-            boolean hasMultiColumnRelations = false;
-
-            ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
-            SecondaryIndexManager indexManager = cfs.indexManager;
-
-            for (Relation relation : whereClause)
-            {
-                if (relation.isMultiColumn())
-                {
-                    MultiColumnRelation rel = (MultiColumnRelation) relation;
-                    List<ColumnDefinition> names = new ArrayList<>(rel.getEntities().size());
-                    for (ColumnIdentifier.Raw rawEntity : rel.getEntities())
-                    {
-                        ColumnIdentifier entity = rawEntity.prepare(cfm);
-                        ColumnDefinition def = cfm.getColumnDefinition(entity);
-                        boolean[] queriable = processRelationEntity(stmt, indexManager, relation, entity, def);
-                        hasQueriableIndex |= queriable[0];
-                        hasQueriableClusteringColumnIndex |= queriable[1];
-                        names.add(def);
-                        hasMultiColumnRelations |= ColumnDefinition.Kind.CLUSTERING_COLUMN == def.kind;
-                    }
-                    updateRestrictionsForRelation(stmt, names, rel, boundNames);
-                }
-                else
-                {
-                    SingleColumnRelation rel = (SingleColumnRelation) relation;
-                    ColumnIdentifier entity = rel.getEntity().prepare(cfm);
-                    ColumnDefinition def = cfm.getColumnDefinition(entity);
-                    boolean[] queriable = processRelationEntity(stmt, indexManager, relation, entity, def);
-                    hasQueriableIndex |= queriable[0];
-                    hasQueriableClusteringColumnIndex |= queriable[1];
-                    hasSingleColumnRelations |= ColumnDefinition.Kind.CLUSTERING_COLUMN == def.kind;
-                    updateRestrictionsForRelation(stmt, def, rel, boundNames);
-                }
-            }
-            if (hasSingleColumnRelations && hasMultiColumnRelations)
-                throw new InvalidRequestException("Mixing single column relations and multi column relations on clustering columns is not allowed");
+            StatementRestrictions restrictions = prepareRestrictions(cfm, boundNames, selection);
 
-             // At this point, the select statement if fully constructed, but we still have a few things to validate
-            processPartitionKeyRestrictions(stmt, hasQueriableIndex, cfm);
-
-            // All (or none) of the partition key columns have been specified;
-            // hence there is no need to turn these restrictions into index expressions.
-            if (!stmt.usesSecondaryIndexing)
-                stmt.restrictedColumns.removeAll(cfm.partitionKeyColumns());
-
-            if (stmt.selectsOnlyStaticColumns && stmt.hasClusteringColumnsRestriction())
-                throw new InvalidRequestException("Cannot restrict clustering columns when selecting only static columns");
-
-            processColumnRestrictions(stmt, hasQueriableIndex, cfm);
+            if (parameters.isDistinct)
+                validateDistinctSelection(cfm, selection, restrictions);
 
-            // Covers indexes on the first clustering column (among others).
-            if (stmt.isKeyRange && hasQueriableClusteringColumnIndex)
-                stmt.usesSecondaryIndexing = true;
+            Comparator<List<ByteBuffer>> orderingComparator = null;
+            boolean isReversed = false;
 
-            if (!stmt.usesSecondaryIndexing)
+            if (!parameters.orderings.isEmpty())
             {
-                for (ColumnDefinition def : cfm.clusteringColumns())
-                {
-                    // Remove clustering column restrictions that can be handled by slices; the remainder will be
-                    // handled by filters (which may require a secondary index).
-                    Restriction restriction = stmt.columnRestrictions[def.position()];
-                    if (restriction != null)
-                    {
-                        if (restriction.canEvaluateWithSlices())
-                            stmt.restrictedColumns.remove(def);
-                        else
-                            stmt.usesSecondaryIndexing = true;
-                    }
-                }
+                verifyOrderingIsAllowed(restrictions);
+                orderingComparator = getOrderingComparator(cfm, selection, restrictions);
+                isReversed = isReversed(cfm);
             }
 
-            // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
-            // there are restrictions not covered by the PK.
-            if (!stmt.metadataRestrictions.isEmpty())
-                stmt.usesSecondaryIndexing = true;
-
-            if (stmt.usesSecondaryIndexing)
-                validateSecondaryIndexSelections(stmt);
-
-            if (!stmt.parameters.orderings.isEmpty())
-                processOrderingClause(stmt, cfm);
+            if (isReversed)
+                restrictions.reverse();
 
-            checkNeedsFiltering(stmt);
+            checkNeedsFiltering(restrictions);
 
-            if (parameters.isDistinct)
-                stmt.validateDistinctSelection();
+            SelectStatement stmt = new SelectStatement(cfm,
+                                                        boundNames.size(),
+                                                        parameters,
+                                                        selection,
+                                                        restrictions,
+                                                        isReversed,
+                                                        orderingComparator,
+                                                        prepareLimit(boundNames));
 
             return new ParsedStatement.Prepared(stmt, boundNames);
         }
 
-        /** Returns a pair of (hasQueriableIndex, hasQueriableClusteringColumnIndex) */
-        private boolean[] processRelationEntity(SelectStatement stmt,
-                                                SecondaryIndexManager indexManager,
-                                                Relation relation,
-                                                ColumnIdentifier entity,
-                                                ColumnDefinition def) throws InvalidRequestException
-        {
-            if (def == null)
-                handleUnrecognizedEntity(entity, relation);
-
-            stmt.restrictedColumns.add(def);
-
-            SecondaryIndex index = indexManager.getIndexForColumn(def.name.bytes);
-            if (index != null && index.supportsOperator(relation.operator()))
-                return new boolean[]{true, def.kind == ColumnDefinition.Kind.CLUSTERING_COLUMN};
-
-            return new boolean[]{false, false};
-        }
-
-        /** Throws an InvalidRequestException for an unrecognized identifier in the WHERE clause */
-        private void handleUnrecognizedEntity(ColumnIdentifier entity, Relation relation) throws InvalidRequestException
+        /**
+         * Prepares the restrictions.
+         *
+         * @param cfm the column family meta data
+         * @param boundNames the variable specifications
+         * @param selection the selection
+         * @return the restrictions
+         * @throws InvalidRequestException if a problem occurs while building the restrictions
+         */
+        private StatementRestrictions prepareRestrictions(CFMetaData cfm,
+                                                          VariableSpecifications boundNames,
+                                                          Selection selection) throws InvalidRequestException
         {
-            if (containsAlias(entity))
-                throw new InvalidRequestException(String.format("Aliases aren't allowed in the where clause ('%s')", relation));
-            else
-                throw new InvalidRequestException(String.format("Undefined name %s in where clause ('%s')", entity, relation));
+            try
+            {
+                return new StatementRestrictions(cfm,
+                                                 whereClause,
+                                                 boundNames,
+                                                 selection.containsOnlyStaticColumns(),
+                                                 selection.containsACollection());
+            }
+            catch (UnrecognizedEntityException e)
+            {
+                if (containsAlias(e.entity))
+                    throw invalidRequest("Aliases aren't allowed in the where clause ('%s')", e.relation);
+                throw e;
+            }
         }
 
         /** Returns a Term for the limit or null if no limit is set */
@@ -1528,478 +804,89 @@ public class SelectStatement implements CQLStatement
             return prepLimit;
         }
 
-        private void updateRestrictionsForRelation(SelectStatement stmt, List<ColumnDefinition> defs, MultiColumnRelation relation, VariableSpecifications boundNames) throws InvalidRequestException
+        private static void verifyOrderingIsAllowed(StatementRestrictions restrictions) throws InvalidRequestException
         {
-            List<ColumnDefinition> restrictedColumns = new ArrayList<>();
-            Set<ColumnDefinition> seen = new HashSet<>(defs.size());
-
-            int previousPosition = -1;
-            for (ColumnDefinition def : defs)
-            {
-                // ensure multi-column restriction only applies to clustering columns
-                if (def.kind != ColumnDefinition.Kind.CLUSTERING_COLUMN)
-                    throw new InvalidRequestException(String.format("Multi-column relations can only be applied to clustering columns: %s", def));
-
-                if (seen.contains(def))
-                    throw new InvalidRequestException(String.format("Column \"%s\" appeared twice in a relation: %s", def, relation));
-                seen.add(def);
-
-                // check that no clustering columns were skipped
-                if (def.position() != previousPosition + 1)
-                {
-                    if (previousPosition == -1)
-                        throw new InvalidRequestException(String.format(
-                                "Clustering columns may not be skipped in multi-column relations. " +
-                                "They should appear in the PRIMARY KEY order. Got %s", relation));
-                    else
-                        throw new InvalidRequestException(String.format(
-                                "Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", relation));
-                }
-                previousPosition++;
-
-                Restriction existing = getExistingRestriction(stmt, def);
-                Operator operator = relation.operator();
-                if (existing != null)
-                {
-                    if (operator == Operator.EQ || operator == Operator.IN)
-                        throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by more than one relation if it is in an %s relation", def, relation.operator()));
-                    else if (!existing.isSlice())
-                        throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by an equality relation and an inequality relation", def));
-                }
-                restrictedColumns.add(def);
-            }
-
-            switch (relation.operator())
-            {
-                case EQ:
-                {
-                    Term t = relation.getValue().prepare(keyspace(), defs);
-                    t.collectMarkerSpecification(boundNames);
-                    Restriction restriction = new MultiColumnRestriction.EQ(t, false);
-                    for (ColumnDefinition def : restrictedColumns)
-                        stmt.columnRestrictions[def.position()] = restriction;
-                    break;
-                }
-                case IN:
-                {
-                    Restriction restriction;
-                    List<? extends Term.MultiColumnRaw> inValues = relation.getInValues();
-                    if (inValues != null)
-                    {
-                        // we have something like "(a, b, c) IN ((1, 2, 3), (4, 5, 6), ...) or
-                        // "(a, b, c) IN (?, ?, ?)
-                        List<Term> terms = new ArrayList<>(inValues.size());
-                        for (Term.MultiColumnRaw tuple : inValues)
-                        {
-                            Term t = tuple.prepare(keyspace(), defs);
-                            t.collectMarkerSpecification(boundNames);
-                            terms.add(t);
-                        }
-                         restriction = new MultiColumnRestriction.InWithValues(terms);
-                    }
-                    else
-                    {
-                        Tuples.INRaw rawMarker = relation.getInMarker();
-                        AbstractMarker t = rawMarker.prepare(keyspace(), defs);
-                        t.collectMarkerSpecification(boundNames);
-                        restriction = new MultiColumnRestriction.InWithMarker(t);
-                    }
-                    for (ColumnDefinition def : restrictedColumns)
-                        stmt.columnRestrictions[def.position()] = restriction;
-
-                    break;
-                }
-                case LT:
-                case LTE:
-                case GT:
-                case GTE:
-                {
-                    Term t = relation.getValue().prepare(keyspace(), defs);
-                    t.collectMarkerSpecification(boundNames);
-                    for (ColumnDefinition def : defs)
-                    {
-                        Restriction.Slice restriction = (Restriction.Slice)getExistingRestriction(stmt, def);
-                        if (restriction == null)
-                            restriction = new MultiColumnRestriction.Slice(false);
-                        else if (!restriction.isMultiColumn())
-                            throw new InvalidRequestException(String.format("Column \"%s\" cannot have both tuple-notation inequalities and single-column inequalities: %s", def.name, relation));
-                        restriction.setBound(def.name, relation.operator(), t);
-                        stmt.columnRestrictions[def.position()] = restriction;
-                    }
-                    break;
-                }
-                case NEQ:
-                    throw new InvalidRequestException(String.format("Unsupported \"!=\" relation: %s", relation));
-            }
+            checkFalse(restrictions.usesSecondaryIndexing(), "ORDER BY with 2ndary indexes is not supported.");
+            checkFalse(restrictions.isKeyRange(), "ORDER BY is only supported when the partition key is restricted by an EQ or an IN.");
         }
 
-        private Restriction getExistingRestriction(SelectStatement stmt, ColumnDefinition def)
+        private static void validateDistinctSelection(CFMetaData cfm,
+                                                      Selection selection,
+                                                      StatementRestrictions restrictions)
+                                                      throws InvalidRequestException
         {
-            switch (def.kind)
-            {
-                case PARTITION_KEY:
-                    return stmt.keyRestrictions[def.position()];
-                case CLUSTERING_COLUMN:
-                    return stmt.columnRestrictions[def.position()];
-                case REGULAR:
-                case STATIC:
-                    return stmt.metadataRestrictions.get(def.name);
-                default:
-                    throw new AssertionError();
-            }
-        }
+            Collection<ColumnDefinition> requestedColumns = selection.getColumns();
+            for (ColumnDefinition def : requestedColumns)
+                checkFalse(!def.isPartitionKey() && !def.isStatic(),
+                           "SELECT DISTINCT queries must only request partition key columns and/or static columns (not %s)",
+                           def.name);
 
-        private void updateRestrictionsForRelation(SelectStatement stmt, ColumnDefinition def, SingleColumnRelation relation, VariableSpecifications names) throws InvalidRequestException
-        {
-            switch (def.kind)
-            {
-                case PARTITION_KEY:
-                    stmt.keyRestrictions[def.position()] = updateSingleColumnRestriction(def, stmt.keyRestrictions[def.position()], relation, names);
-                    break;
-                case CLUSTERING_COLUMN:
-                    stmt.columnRestrictions[def.position()] = updateSingleColumnRestriction(def, stmt.columnRestrictions[def.position()], relation, names);
-                    break;
-                case COMPACT_VALUE:
-                    throw new InvalidRequestException(String.format("Predicates on the non-primary-key column (%s) of a COMPACT table are not yet supported", def.name));
-                case REGULAR:
-                case STATIC:
-                    // We only all IN on the row key and last clustering key so far, never on non-PK columns, and this even if there's an index
-                    Restriction r = updateSingleColumnRestriction(def, stmt.metadataRestrictions.get(def.name), relation, names);
-                    if (r.isIN() && !((Restriction.IN)r).canHaveOnlyOneValue())
-                        // Note: for backward compatibility reason, we conside a IN of 1 value the same as a EQ, so we let that slide.
-                        throw new InvalidRequestException(String.format("IN predicates on non-primary-key columns (%s) is not yet supported", def.name));
-                    stmt.metadataRestrictions.put(def.name, r);
-                    break;
-            }
-        }
-
-        Restriction updateSingleColumnRestriction(ColumnDefinition def, Restriction existingRestriction, SingleColumnRelation newRel, VariableSpecifications boundNames) throws InvalidRequestException
-        {
-            ColumnSpecification receiver = def;
-            if (newRel.onToken)
-            {
-                if (def.kind != ColumnDefinition.Kind.PARTITION_KEY)
-                    throw new InvalidRequestException(String.format("The token() function is only supported on the partition key, found on %s", def.name));
+            // If it's a key range, we require that all partition key columns are selected so we don't have to bother
+            // with post-query grouping.
+            if (!restrictions.isKeyRange())
+                return;
 
-                receiver = new ColumnSpecification(def.ksName,
-                                                   def.cfName,
-                                                   new ColumnIdentifier("partition key token", true),
-                                                   StorageService.getPartitioner().getTokenValidator());
-            }
-
-            // We don't support relations against entire collections (unless they're frozen), like "numbers = {1, 2, 3}"
-            if (receiver.type.isCollection() && receiver.type.isMultiCell() && !(newRel.operator() == Operator.CONTAINS_KEY || newRel.operator() == Operator.CONTAINS))
-            {
-                throw new InvalidRequestException(String.format("Collection column '%s' (%s) cannot be restricted by a '%s' relation",
-                                                                def.name, receiver.type.asCQL3Type(), newRel.operator()));
-            }
-
-            switch (newRel.operator())
-            {
-                case EQ:
-                {
-                    if (existingRestriction != null)
-                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", def.name));
-                    Term t = newRel.getValue().prepare(keyspace(), receiver);
-                    t.collectMarkerSpecification(boundNames);
-                    existingRestriction = new SingleColumnRestriction.EQ(t, newRel.onToken);
-                }
-                break;
-                case IN:
-                    if (existingRestriction != null)
-                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes a IN", def.name));
-
-                    if (newRel.getInValues() == null)
-                    {
-                        // Means we have a "SELECT ... IN ?"
-                        assert newRel.getValue() != null;
-                        Term t = newRel.getValue().prepare(keyspace(), receiver);
-                        t.collectMarkerSpecification(boundNames);
-                        existingRestriction = new SingleColumnRestriction.InWithMarker((Lists.Marker)t);
-                    }
-                    else
-                    {
-                        List<Term> inValues = new ArrayList<>(newRel.getInValues().size());
-                        for (Term.Raw raw : newRel.getInValues())
-                        {
-                            Term t = raw.prepare(keyspace(), receiver);
-                            t.collectMarkerSpecification(boundNames);
-                            inValues.add(t);
-                        }
-                        existingRestriction = new SingleColumnRestriction.InWithValues(inValues);
-                    }
-                    break;
-                case NEQ:
-                    throw new InvalidRequestException(String.format("Unsupported \"!=\" relation on column \"%s\"", def.name));
-                case GT:
-                case GTE:
-                case LT:
-                case LTE:
-                    {
-                        if (existingRestriction == null)
-                            existingRestriction = new SingleColumnRestriction.Slice(newRel.onToken);
-                        else if (!existingRestriction.isSlice())
-                            throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by both an equality and an inequality relation", def.name));
-                        else if (existingRestriction.isMultiColumn())
-                            throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by both a tuple notation inequality and a single column inequality (%s)", def.name, newRel));
-                        else if (existingRestriction.isOnToken() != newRel.onToken)
-                            // For partition keys, we shouldn't have slice restrictions without token(). And while this is rejected later by
-                            // processPartitionKeysRestrictions, we shouldn't update the existing restriction by the new one if the old one was using token()
-                            // and the new one isn't since that would bypass that later test.
-                            throw new InvalidRequestException("Only EQ and IN relation are supported on the partition key (unless you use the token() function)");
-
-                        Term t = newRel.getValue().prepare(keyspace(), receiver);
-                        t.collectMarkerSpecification(boundNames);
-                        ((SingleColumnRestriction.Slice)existingRestriction).setBound(def.name, newRel.operator(), t);
-                    }
-                    break;
-                case CONTAINS_KEY:
-                    if (!(receiver.type instanceof MapType))
-                        throw new InvalidRequestException(String.format("Cannot use CONTAINS KEY on non-map column %s", def.name));
-                    // Fallthrough on purpose
-                case CONTAINS:
-                {
-                    if (!receiver.type.isCollection())
-                        throw new InvalidRequestException(String.format("Cannot use %s relation on non collection column %s", newRel.operator(), def.name));
-
-                    if (existingRestriction == null)
-                        existingRestriction = new SingleColumnRestriction.Contains();
-                    else if (!existingRestriction.isContains())
-                        throw new InvalidRequestException(String.format("Collection column %s can only be restricted by CONTAINS or CONTAINS KEY", def.name));
-
-                    boolean isKey = newRel.operator() == Operator.CONTAINS_KEY;
-                    receiver = makeCollectionReceiver(receiver, isKey);
-                    Term t = newRel.getValue().prepare(keyspace(), receiver);
-                    t.collectMarkerSpecification(boundNames);
-                    ((SingleColumnRestriction.Contains)existingRestriction).add(t, isKey);
-                    break;
-                }
-            }
-            return existingRestriction;
+            for (ColumnDefinition def : cfm.partitionKeyColumns())
+                checkTrue(requestedColumns.contains(def),
+                          "SELECT DISTINCT queries must request all the partition key columns (missing %s)", def.name);
         }
 
-        private void processPartitionKeyRestrictions(SelectStatement stmt, boolean hasQueriableIndex, CFMetaData cfm) throws InvalidRequestException
+        private void handleUnrecognizedOrderingColumn(ColumnIdentifier column) throws InvalidRequestException
         {
-            // If there is a queriable index, no special condition are required on the other restrictions.
-            // But we still need to know 2 things:
-            //   - If we don't have a queriable index, is the query ok
-            //   - Is it queriable without 2ndary index, which is always more efficient
-            // If a component of the partition key is restricted by a relation, all preceding
-            // components must have a EQ. Only the last partition key component can be in IN relation.
-            boolean canRestrictFurtherComponents = true;
-            ColumnDefinition previous = null;
-            stmt.keyIsInRelation = false;
-            Iterator<ColumnDefinition> iter = cfm.partitionKeyColumns().iterator();
-            for (int i = 0; i < stmt.keyRestrictions.length; i++)
-            {
-                ColumnDefinition cdef = iter.next();
-                Restriction restriction = stmt.keyRestrictions[i];
-
-                if (restriction == null)
-                {
-                    if (stmt.onToken)
-                        throw new InvalidRequestException("The token() function must be applied to all partition key components or none of them");
-
-                    // The only time not restricting a key part is allowed is if none are restricted or an index is used.
-                    if (i > 0 && stmt.keyRestrictions[i - 1] != null)
-                    {
-                        if (hasQueriableIndex)
-                        {
-                            stmt.usesSecondaryIndexing = true;
-                            stmt.isKeyRange = true;
-                            break;
-                        }
-                        throw new InvalidRequestException(String.format("Partition key part %s must be restricted since preceding part is", cdef.name));
-                    }
-
-                    stmt.isKeyRange = true;
-                    canRestrictFurtherComponents = false;
-                }
-                else if (!canRestrictFurtherComponents)
-                {
-                    if (hasQueriableIndex)
-                    {
-                        stmt.usesSecondaryIndexing = true;
-                        break;
-                    }
-                    throw new InvalidRequestException(String.format(
-                            "Partitioning column \"%s\" cannot be restricted because the preceding column (\"%s\") is " +
-                            "either not restricted or is restricted by a non-EQ relation", cdef.name, previous));
-                }
-                else if (restriction.isOnToken())
-                {
-                    // If this is a query on tokens, it's necessarily a range query (there can be more than one key per token).
-                    stmt.isKeyRange = true;
-                    stmt.onToken = true;
-                }
-                else if (stmt.onToken)
-                {
-                    throw new InvalidRequestException(String.format("The token() function must be applied to all partition key components or none of them"));
-                }
-                else if (!restriction.isSlice())
-                {
-                    if (restriction.isIN())
-                    {
-                        // We only support IN for the last name so far
-                        if (i != stmt.keyRestrictions.length - 1)
-                            throw new InvalidRequestException(String.format("Partition KEY part %s cannot be restricted by IN relation (only the last part of the partition key can)", cdef.name));
-                        stmt.keyIsInRelation = true;
-                    }
-                }
-                else
-                {
-                    // Non EQ relation is not supported without token(), even if we have a 2ndary index (since even those are ordered by partitioner).
-                    // Note: In theory we could allow it for 2ndary index queries with ALLOW FILTERING, but that would probably require some special casing
-                    // Note bis: This is also why we don't bother handling the 'tuple' notation of #4851 for keys. If we lift the limitation for 2ndary
-                    // index with filtering, we'll need to handle it though.
-                    throw new InvalidRequestException("Only EQ and IN relation are supported on the partition key (unless you use the token() function)");
-                }
-                previous = cdef;
-            }
-
-            if (stmt.onToken)
-                checkTokenFunctionArgumentsOrder(cfm);
+            checkFalse(containsAlias(column), "Aliases are not allowed in order by clause ('%s')", column);
+            checkFalse(true, "Order by on unknown column %s", column);
         }
 
-        /**
-         * Checks that the column identifiers used as argument for the token function have been specified in the
-         * partition key order.
-         * @param cfm the Column Family MetaData
-         * @throws InvalidRequestException if the arguments have not been provided in the proper order.
-         */
-        private void checkTokenFunctionArgumentsOrder(CFMetaData cfm) throws InvalidRequestException
+        private Comparator<List<ByteBuffer>> getOrderingComparator(CFMetaData cfm,
+                                                                   Selection selection,
+                                                                   StatementRestrictions restrictions)
+                                                                   throws InvalidRequestException
         {
-            Iterator<ColumnDefinition> iter = Iterators.cycle(cfm.partitionKeyColumns());
-            for (Relation relation : whereClause)
-            {
-                SingleColumnRelation singleColumnRelation = (SingleColumnRelation) relation;
-                if (singleColumnRelation.onToken && !cfm.getColumnDefinition(singleColumnRelation.getEntity().prepare(cfm)).equals(iter.next()))
-                    throw new InvalidRequestException(String.format("The token function arguments must be in the partition key order: %s",
-                                                                    Joiner.on(',').join(cfm.partitionKeyColumns())));
-            }
-        }
+            if (!restrictions.keyIsInRelation())
+                return null;
 
-        private void processColumnRestrictions(SelectStatement stmt, boolean hasQueriableIndex, CFMetaData cfm) throws InvalidRequestException
-        {
-            // If a clustering key column is restricted by a non-EQ relation, all preceding
-            // columns must have a EQ, and all following must have no restriction. Unless
-            // the column is indexed that is.
-            boolean canRestrictFurtherComponents = true;
-            ColumnDefinition previous = null;
-            boolean previousIsSlice = false;
-            Iterator<ColumnDefinition> iter = cfm.clusteringColumns().iterator();
-            for (int i = 0; i < stmt.columnRestrictions.length; i++)
-            {
-                ColumnDefinition cdef = iter.next();
-                Restriction restriction = stmt.columnRestrictions[i];
+            Map<ColumnIdentifier, Integer> orderingIndexes = getOrderingIndex(cfm, selection);
 
-                if (restriction == null)
-                {
-                    canRestrictFurtherComponents = false;
-                    previousIsSlice = false;
-                }
-                else if (!canRestrictFurtherComponents)
-                {
-                    // We're here if the previous clustering column was either not restricted or was a slice.
-                    // We can't restrict the current column unless:
-                    //   1) we're in the special case of the 'tuple' notation from #4851 which we expand as multiple
-                    //      consecutive slices: in which case we're good with this restriction and we continue
-                    //   2) we have a 2ndary index, in which case we have to use it but can skip more validation
-                    if (!(previousIsSlice && restriction.isSlice() && restriction.isMultiColumn()))
-                    {
-                        if (hasQueriableIndex)
-                        {
-                            stmt.usesSecondaryIndexing = true; // handle gaps and non-keyrange cases.
-                            break;
-                        }
-                        throw new InvalidRequestException(String.format(
-                                "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is either not restricted or by a non-EQ relation)", cdef.name, previous));
-                    }
-                }
-                else if (restriction.isSlice())
-                {
-                    canRestrictFurtherComponents = false;
-                    previousIsSlice = true;
-                    Restriction.Slice slice = (Restriction.Slice)restriction;
-                    // For non-composite slices, we don't support internally the difference between exclusive and
-                    // inclusive bounds, so we deal with it manually.
-                    if (!cfm.comparator.isCompound() && (!slice.isInclusive(Bound.START) || !slice.isInclusive(Bound.END)))
-                        stmt.sliceRestriction = slice;
-                }
-                else if (restriction.isIN())
-                {
-                    if (stmt.selectACollection())
-                        throw new InvalidRequestException(String.format("Cannot restrict column \"%s\" by IN relation as a collection is selected by the query", cdef.name));
-                }
-                else if (restriction.isContains())
-                {
-                    if (!hasQueriableIndex)
-                        throw new InvalidRequestException(String.format("Cannot restrict column \"%s\" by a CONTAINS relation without a secondary index", cdef.name));
-                    stmt.usesSecondaryIndexing = true;
-                }
+            List<Integer> idToSort = new ArrayList<Integer>();
+            List<Comparator<ByteBuffer>> sorters = new ArrayList<Comparator<ByteBuffer>>();
 
-                previous = cdef;
+            for (ColumnIdentifier.Raw raw : parameters.orderings.keySet())
+            {
+                ColumnIdentifier identifier = raw.prepare(cfm);
+                ColumnDefinition orderingColumn = cfm.getColumnDefinition(identifier);
+                idToSort.add(orderingIndexes.get(orderingColumn.name));
+                sorters.add(orderingColumn.type);
             }
+            return idToSort.size() == 1 ? new SingleColumnComparator(idToSort.get(0), sorters.get(0))
+                    : new CompositeComparator(sorters, idToSort);
         }
 
-        private void validateSecondaryIndexSelections(SelectStatement stmt) throws InvalidRequestException
+        private Map<ColumnIdentifier, Integer> getOrderingIndex(CFMetaData cfm, Selection selection)
+                throws InvalidRequestException
         {
-            if (stmt.keyIsInRelation)
-                throw new InvalidRequestException("Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
-            // When the user only select static columns, the intent is that we don't query the whole partition but just
-            // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on static columns
-            // so far, 2i means that you've

<TRUNCATED>