You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2013/09/18 17:59:24 UTC

[1/2] git commit: Support variadic parameters for IN clauses

Updated Branches:
  refs/heads/trunk 5102e8d74 -> 1c9858306


Support variadic parameters for IN clauses

patch by slebresne; reviewed by iamaleksey for CASSANDRA-4210


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

Branch: refs/heads/trunk
Commit: e93578b3bd9776b99d4068201b1f872f8f8689e2
Parents: 278f210
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Sep 18 17:51:17 2013 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Sep 18 17:51:17 2013 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 doc/cql3/CQL.textile                            |   3 +
 .../apache/cassandra/cql3/AbstractMarker.java   |  25 ++
 src/java/org/apache/cassandra/cql3/Cql.g        |   1 +
 .../org/apache/cassandra/cql3/Relation.java     |   2 +-
 .../apache/cassandra/cql3/statements/Bound.java |  35 ++
 .../cql3/statements/ModificationStatement.java  |  69 ++--
 .../cassandra/cql3/statements/Restriction.java  | 289 +++++++++++++++
 .../cql3/statements/SelectStatement.java        | 348 ++++++-------------
 9 files changed, 511 insertions(+), 262 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e93578b3/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c1ff3cb..98ea03f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -23,6 +23,7 @@
  * Migrate pre-2.0 key/value/column aliases to system.schema_columns
    (CASSANDRA-6009)
  * Paging filter empty rows too agressively (CASSANDRA-6040)
+ * Support variadic parameters for IN clauses (CASSANDRA-4210)
 Merged from 1.2:
  * Tuning knobs for dealing with large blobs and many CFs (CASSANDRA-5982)
  * (Hadoop) Fix CQLRW for thrift tables (CASSANDRA-6002)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e93578b3/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 178d940..6ced296 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -487,6 +487,7 @@ bc(syntax)..
 
 <relation> ::= <identifier> '=' <term>
              | <identifier> IN '(' ( <term> ( ',' <term> )* )? ')'
+             | <identifier> IN '?'
 
 <option> ::= TIMESTAMP <integer>
            | TTL <integer>
@@ -537,6 +538,7 @@ bc(syntax)..
 
 <relation> ::= <identifier> '=' <term>
              | <identifier> IN '(' ( <term> ( ',' <term> )* )? ')'
+             | <identifier> IN '?'
 p. 
 __Sample:__
 
@@ -1087,6 +1089,7 @@ The following describes the addition/changes brought for each version of CQL.
 h3. 3.1.1
 
 * @SELECT@ statement now allows listing the partition keys (using the @DISTINCT@ modifier). See "CASSANDRA-4536":https://issues.apache.org/jira/browse/CASSANDRA-4536.
+* The syntax @c IN ?@ is now supported in @WHERE@ clauses. In that case, the value expected for the bind variable will be a list of whatever type @c@ is.
 
 h3. 3.1.0
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e93578b3/src/java/org/apache/cassandra/cql3/AbstractMarker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/AbstractMarker.java b/src/java/org/apache/cassandra/cql3/AbstractMarker.java
index b4a4143..0de0153 100644
--- a/src/java/org/apache/cassandra/cql3/AbstractMarker.java
+++ b/src/java/org/apache/cassandra/cql3/AbstractMarker.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.cql3;
 
 import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
 
@@ -82,4 +83,28 @@ public abstract class AbstractMarker extends Term.NonTerminal
             return "?";
         }
     }
+
+    // A raw that stands for multiple values, i.e. when we have 'IN ?'
+    public static class INRaw extends Raw
+    {
+        public INRaw(int bindIndex)
+        {
+            super(bindIndex);
+        }
+
+        private static ColumnSpecification makeInReceiver(ColumnSpecification receiver)
+        {
+            ColumnIdentifier inName = new ColumnIdentifier("in(" + receiver.name + ")", true);
+            return new ColumnSpecification(receiver.ksName, receiver.cfName, inName, ListType.getInstance(receiver.type));
+        }
+
+        @Override
+        public AbstractMarker prepare(ColumnSpecification receiver) throws InvalidRequestException
+        {
+            if (receiver.type instanceof CollectionType)
+                throw new InvalidRequestException("Invalid IN relation on collection column");
+
+            return new Lists.Marker(bindIndex, makeInReceiver(receiver));
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e93578b3/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 61bf3c8..5e11b39 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -853,6 +853,7 @@ relation[List<Relation> clauses]
             for (ColumnIdentifier id : l)
                 $clauses.add(new Relation(id, type, t, true));
         }
+    | name=cident K_IN QMARK { $clauses.add(new Relation(name, Relation.Type.IN, new AbstractMarker.INRaw(++currentBindMarkerIdx))); }
     | name=cident K_IN { Relation rel = Relation.createInRelation($name.id); }
        '(' ( f1=term { rel.addInValue(f1); } (',' fN=term { rel.addInValue(fN); } )* )? ')' { $clauses.add(rel); }
     ;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e93578b3/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 0217565..15ed540 100644
--- a/src/java/org/apache/cassandra/cql3/Relation.java
+++ b/src/java/org/apache/cassandra/cql3/Relation.java
@@ -81,7 +81,7 @@ public class Relation
 
     public Term.Raw getValue()
     {
-        assert relationType != Type.IN;
+        assert relationType != Type.IN || value == null || value instanceof AbstractMarker.INRaw;
         return value;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e93578b3/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
new file mode 100644
index 0000000..8aa3a37
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/Bound.java
@@ -0,0 +1,35 @@
+/*
+ * 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;
+
+public enum Bound
+{
+    START(0), END(1);
+
+    public final int idx;
+
+    Bound(int idx)
+    {
+        this.idx = idx;
+    }
+
+    public static Bound reverse(Bound b)
+    {
+        return b == START ? END : START;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e93578b3/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 0b55ed1..2f1785d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -50,7 +50,7 @@ public abstract class ModificationStatement implements CQLStatement
     public final CFMetaData cfm;
     private final Attributes attrs;
 
-    private final Map<ColumnIdentifier, List<Term>> processedKeys = new HashMap<ColumnIdentifier, List<Term>>();
+    private final Map<ColumnIdentifier, Restriction> processedKeys = new HashMap<ColumnIdentifier, Restriction>();
     private final List<Operation> columnOperations = new ArrayList<Operation>();
 
     private List<Operation> columnConditions;
@@ -135,7 +135,7 @@ public abstract class ModificationStatement implements CQLStatement
         ifNotExists = true;
     }
 
-    private void addKeyValues(ColumnIdentifier name, List<Term> values) throws InvalidRequestException
+    private void addKeyValues(ColumnIdentifier name, Restriction values) throws InvalidRequestException
     {
         if (processedKeys.put(name, values) != null)
             throw new InvalidRequestException(String.format("Multiple definitions found for PRIMARY KEY part %s", name));
@@ -143,7 +143,7 @@ public abstract class ModificationStatement implements CQLStatement
 
     public void addKeyValue(ColumnIdentifier name, Term value) throws InvalidRequestException
     {
-        addKeyValues(name, Collections.singletonList(value));
+        addKeyValues(name, new Restriction.EQ(value, false));
     }
 
     public void processWhereClause(List<Relation> whereClause, ColumnSpecification[] names) throws InvalidRequestException
@@ -159,22 +159,40 @@ public abstract class ModificationStatement implements CQLStatement
             {
                 case KEY_ALIAS:
                 case COLUMN_ALIAS:
-                    List<Term.Raw> rawValues;
+                    Restriction restriction;
+
                     if (rel.operator() == Relation.Type.EQ)
-                        rawValues = Collections.singletonList(rel.getValue());
+                    {
+                        Term t = rel.getValue().prepare(name);
+                        t.collectMarkerSpecification(names);
+                        restriction = new Restriction.EQ(t, false);
+                    }
                     else if (name.kind == CFDefinition.Name.Kind.KEY_ALIAS && rel.operator() == Relation.Type.IN)
-                        rawValues = rel.getInValues();
+                    {
+                        if (rel.getValue() != null)
+                        {
+                            Term t = rel.getValue().prepare(name);
+                            t.collectMarkerSpecification(names);
+                            restriction = Restriction.IN.create(t);
+                        }
+                        else
+                        {
+                            List<Term> values = new ArrayList<Term>(rel.getInValues().size());
+                            for (Term.Raw raw : rel.getInValues())
+                            {
+                                Term t = raw.prepare(name);
+                                t.collectMarkerSpecification(names);
+                                values.add(t);
+                            }
+                            restriction = Restriction.IN.create(values);
+                        }
+                    }
                     else
-                        throw new InvalidRequestException(String.format("Invalid operator %s for PRIMARY KEY part %s", rel.operator(), name));
-
-                    List<Term> values = new ArrayList<Term>(rawValues.size());
-                    for (Term.Raw raw : rawValues)
                     {
-                        Term t = raw.prepare(name);
-                        t.collectMarkerSpecification(names);
-                        values.add(t);
+                        throw new InvalidRequestException(String.format("Invalid operator %s for PRIMARY KEY part %s", rel.operator(), name));
                     }
-                    addKeyValues(name.name, values);
+
+                    addKeyValues(name.name, restriction);
                     break;
                 case VALUE_ALIAS:
                 case COLUMN_METADATA:
@@ -191,15 +209,16 @@ public abstract class ModificationStatement implements CQLStatement
         List<ByteBuffer> keys = new ArrayList<ByteBuffer>();
         for (CFDefinition.Name name : cfDef.keys.values())
         {
-            List<Term> values = processedKeys.get(name.name);
-            if (values == null)
+            Restriction r = processedKeys.get(name.name);
+            if (r == null)
                 throw new InvalidRequestException(String.format("Missing mandatory PRIMARY KEY part %s", name));
 
+            List<ByteBuffer> values = r.values(variables);
+
             if (keyBuilder.remainingCount() == 1)
             {
-                for (Term t : values)
+                for (ByteBuffer val : values)
                 {
-                    ByteBuffer val = t.bindAndGet(variables);
                     if (val == null)
                         throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", name));
                     keys.add(keyBuilder.copy().add(val).build());
@@ -207,9 +226,9 @@ public abstract class ModificationStatement implements CQLStatement
             }
             else
             {
-                if (values.isEmpty() || values.size() > 1)
+                if (values.size() != 1)
                     throw new InvalidRequestException("IN is only supported on the last column of the partition key");
-                ByteBuffer val = values.get(0).bindAndGet(variables);
+                ByteBuffer val = values.get(0);
                 if (val == null)
                     throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", name));
                 keyBuilder.add(val);
@@ -226,8 +245,8 @@ public abstract class ModificationStatement implements CQLStatement
         CFDefinition.Name firstEmptyKey = null;
         for (CFDefinition.Name name : cfDef.columns.values())
         {
-            List<Term> values = processedKeys.get(name.name);
-            if (values == null)
+            Restriction r = processedKeys.get(name.name);
+            if (r == null)
             {
                 firstEmptyKey = name;
                 if (requireFullClusteringKey() && cfDef.isComposite && !cfDef.isCompact)
@@ -239,8 +258,9 @@ public abstract class ModificationStatement implements CQLStatement
             }
             else
             {
+                List<ByteBuffer> values = r.values(variables);
                 assert values.size() == 1; // We only allow IN for row keys so far
-                ByteBuffer val = values.get(0).bindAndGet(variables);
+                ByteBuffer val = values.get(0);
                 if (val == null)
                     throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", name));
                 builder.add(val);
@@ -253,8 +273,7 @@ public abstract class ModificationStatement implements CQLStatement
     {
         for (CFDefinition.Name name : cfm.getCfDef().columns.values())
         {
-            List<Term> values = processedKeys.get(name.name);
-            if (values == null || values.isEmpty())
+            if (processedKeys.get(name.name) == null)
                 return name;
         }
         return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e93578b3/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
new file mode 100644
index 0000000..45cc607
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
@@ -0,0 +1,289 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.thrift.IndexOperator;
+import org.apache.cassandra.cql3.*;
+
+/**
+ * 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();
+
+    // Only supported for EQ and IN, but it's convenient to have here
+    public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException;
+
+    public static class EQ implements Restriction
+    {
+        private final Term value;
+        private final boolean onToken;
+
+        public EQ(Term value, boolean onToken)
+        {
+            this.value = value;
+            this.onToken = onToken;
+        }
+
+        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            return Collections.singletonList(value.bindAndGet(variables));
+        }
+
+        public boolean isSlice()
+        {
+            return false;
+        }
+
+        public boolean isEQ()
+        {
+            return true;
+        }
+
+        public boolean isIN()
+        {
+            return false;
+        }
+
+        public boolean isOnToken()
+        {
+            return onToken;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("EQ(%s)%s", value, onToken ? "*" : "");
+        }
+    }
+
+    public static abstract class IN implements Restriction
+    {
+        public static IN create(List<Term> values)
+        {
+            return new WithValues(values);
+        }
+
+        public static IN create(Term value) throws InvalidRequestException
+        {
+            assert value instanceof Lists.Marker; // we shouldn't have got there otherwise
+            return new WithMarker((Lists.Marker)value);
+        }
+
+        public boolean isSlice()
+        {
+            return false;
+        }
+
+        public boolean isEQ()
+        {
+            return false;
+        }
+
+        public boolean isIN()
+        {
+            return true;
+        }
+
+        public boolean isOnToken()
+        {
+            return false;
+        }
+
+        private static class WithValues extends IN
+        {
+            private final List<Term> values;
+
+            private WithValues(List<Term> values)
+            {
+                this.values = values;
+            }
+
+            public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
+            {
+                List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(values.size());
+                for (Term value : values)
+                    buffers.add(value.bindAndGet(variables));
+                return buffers;
+            }
+
+            @Override
+            public String toString()
+            {
+                return String.format("IN(%s)", values);
+            }
+        }
+
+        private static class WithMarker extends IN
+        {
+            private final Lists.Marker marker;
+
+            private WithMarker(Lists.Marker marker)
+            {
+                this.marker = marker;
+            }
+
+            public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
+            {
+                Lists.Value lval = marker.bind(variables);
+                if (lval == null)
+                    throw new InvalidRequestException("Invalid null value for IN restriction");
+                return lval.elements;
+            }
+
+            @Override
+            public String toString()
+            {
+                return "IN ?";
+            }
+        }
+    }
+
+    public static class Slice implements Restriction
+    {
+        private final Term[] bounds;
+        private final boolean[] boundInclusive;
+        private final boolean onToken;
+
+        public Slice(boolean onToken)
+        {
+            this.bounds = new Term[2];
+            this.boundInclusive = new boolean[2];
+            this.onToken = onToken;
+        }
+
+        public boolean isSlice()
+        {
+            return true;
+        }
+
+        public boolean isEQ()
+        {
+            return false;
+        }
+
+        public boolean isIN()
+        {
+            return false;
+        }
+
+        public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        public boolean isOnToken()
+        {
+            return onToken;
+        }
+
+        public boolean hasBound(Bound b)
+        {
+            return bounds[b.idx] != null;
+        }
+
+        public ByteBuffer bound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
+        {
+            return bounds[b.idx].bindAndGet(variables);
+        }
+
+        public boolean isInclusive(Bound b)
+        {
+            return bounds[b.idx] == null || boundInclusive[b.idx];
+        }
+
+        public Relation.Type getRelation(Bound eocBound, Bound inclusiveBound)
+        {
+            switch (eocBound)
+            {
+                case START:
+                    return boundInclusive[inclusiveBound.idx] ? Relation.Type.GTE : Relation.Type.GT;
+                case END:
+                    return boundInclusive[inclusiveBound.idx] ? Relation.Type.LTE : Relation.Type.LT;
+            }
+            throw new AssertionError();
+        }
+
+        public IndexOperator getIndexOperator(Bound b)
+        {
+            switch (b)
+            {
+                case START:
+                    return boundInclusive[b.idx] ? IndexOperator.GTE : IndexOperator.GT;
+                case END:
+                    return boundInclusive[b.idx] ? IndexOperator.LTE : IndexOperator.LT;
+            }
+            throw new AssertionError();
+        }
+
+        public void setBound(ColumnIdentifier name, Relation.Type type, Term t) throws InvalidRequestException
+        {
+            Bound b;
+            boolean inclusive;
+            switch (type)
+            {
+                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("Invalid restrictions found on %s", 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 ? "*" : "");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e93578b3/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 8e09fe5..c85864f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -70,7 +70,7 @@ public class SelectStatement implements CQLStatement
 
     // The name of all restricted names not covered by the key or index filter
     private final Set<CFDefinition.Name> restrictedNames = new HashSet<CFDefinition.Name>();
-    private Restriction sliceRestriction;
+    private Restriction.Slice sliceRestriction;
 
     private boolean isReversed;
     private boolean onToken;
@@ -83,23 +83,6 @@ public class SelectStatement implements CQLStatement
     // Used by forSelection below
     private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier, Boolean>emptyMap(), false, false, null, false);
 
-    private static enum Bound
-    {
-        START(0), END(1);
-
-        public final int idx;
-
-        Bound(int idx)
-        {
-            this.idx = idx;
-        }
-
-        public static Bound reverse(Bound b)
-        {
-            return b == START ? END : START;
-        }
-    }
-
     public SelectStatement(CFDefinition cfDef, int boundTerms, Parameters parameters, Selection selection, Term limit)
     {
         this.cfDef = cfDef;
@@ -458,12 +441,14 @@ public class SelectStatement implements CQLStatement
         for (CFDefinition.Name name : cfDef.keys.values())
         {
             Restriction r = keyRestrictions[name.position];
-            assert r != null;
+            assert r != null && !r.isSlice();
+
+            List<ByteBuffer> values = r.values(variables);
+
             if (builder.remainingCount() == 1)
             {
-                for (Term t : r.eqValues)
+                for (ByteBuffer val : values)
                 {
-                    ByteBuffer val = t.bindAndGet(variables);
                     if (val == null)
                         throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", name));
                     keys.add(builder.copy().add(val).build());
@@ -471,9 +456,10 @@ public class SelectStatement implements CQLStatement
             }
             else
             {
-                if (r.isINRestriction())
+                // 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 = r.eqValues.get(0).bindAndGet(variables);
+                ByteBuffer val = values.get(0);
                 if (val == null)
                     throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", name));
                 builder.add(val);
@@ -499,14 +485,20 @@ public class SelectStatement implements CQLStatement
         assert onToken;
 
         Restriction keyRestriction = keyRestrictions[0];
-        Term t = keyRestriction.isEquality()
-               ? keyRestriction.eqValues.get(0)
-               : keyRestriction.bound(b);
+        ByteBuffer value;
+        if (keyRestriction.isEQ())
+        {
+            value = keyRestriction.values(variables).get(0);
+        }
+        else
+        {
+            Restriction.Slice slice = (Restriction.Slice)keyRestriction;
+            if (!slice.hasBound(b))
+                return p.getMinimumToken();
 
-        if (t == null)
-            return p.getMinimumToken();
+            value = slice.bound(b, variables);
+        }
 
-        ByteBuffer value = t.bindAndGet(variables);
         if (value == null)
             throw new InvalidRequestException("Invalid null token value");
         return p.getTokenFactory().fromByteArray(value);
@@ -518,8 +510,8 @@ public class SelectStatement implements CQLStatement
         {
             if (r == null)
                 return true;
-            else if (!r.isEquality())
-                return r.isInclusive(b);
+            else if (r.isSlice())
+                return ((Restriction.Slice)r).isInclusive(b);
         }
         // All equality
         return true;
@@ -536,7 +528,7 @@ public class SelectStatement implements CQLStatement
         // 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.isEquality())
+            if (r == null || r.isSlice())
                 return true;
         }
         return false;
@@ -551,21 +543,29 @@ public class SelectStatement implements CQLStatement
         for (Restriction r : columnRestrictions)
         {
             ColumnIdentifier id = idIter.next();
-            assert r != null && r.isEquality();
-            if (r.isINRestriction())
+            assert r != null && !r.isSlice();
+
+            List<ByteBuffer> values = r.values(variables);
+            if (values.size() == 1)
+            {
+                ByteBuffer val = values.get(0);
+                if (val == null)
+                    throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", id));
+                builder.add(val);
+            }
+            else
             {
                 // We have a IN, which we only support for the last column.
                 // If compact, just add all values and we're done. Otherwise,
                 // for each value of the IN, creates all the columns corresponding to the selection.
-                if (r.eqValues.isEmpty())
+                if (values.isEmpty())
                     return null;
                 SortedSet<ByteBuffer> columns = new TreeSet<ByteBuffer>(cfDef.cfm.comparator);
-                Iterator<Term> iter = r.eqValues.iterator();
+                Iterator<ByteBuffer> iter = values.iterator();
                 while (iter.hasNext())
                 {
-                    Term v = iter.next();
+                    ByteBuffer val = iter.next();
                     ColumnNameBuilder b = iter.hasNext() ? builder.copy() : builder;
-                    ByteBuffer val = v.bindAndGet(variables);
                     if (val == null)
                         throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", id));
                     b.add(val);
@@ -576,13 +576,6 @@ public class SelectStatement implements CQLStatement
                 }
                 return columns;
             }
-            else
-            {
-                ByteBuffer val = r.eqValues.get(0).bindAndGet(variables);
-                if (val == null)
-                    throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", id));
-                builder.add(val);
-            }
         }
 
         return addSelectedColumns(builder);
@@ -663,7 +656,7 @@ public class SelectStatement implements CQLStatement
             // But if the actual comparator itself is reversed, we must inversed the bounds too.
             Bound b = isReversed == isReversedType(name) ? bound : Bound.reverse(bound);
             Restriction r = restrictions[name.position];
-            if (r == null || (!r.isEquality() && r.bound(b) == null))
+            if (r == null || (r.isSlice() && !((Restriction.Slice)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
@@ -673,18 +666,27 @@ public class SelectStatement implements CQLStatement
                                                  : builder.build());
             }
 
-            if (r.isEquality())
+            if (r.isSlice())
             {
-                if (r.isINRestriction())
+                Restriction.Slice slice = (Restriction.Slice)r;
+                assert slice.hasBound(b);
+                ByteBuffer val = slice.bound(b, variables);
+                if (val == null)
+                    throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
+                return Collections.singletonList(builder.add(val, slice.getRelation(eocBound, b)).build());
+            }
+            else
+            {
+                List<ByteBuffer> values = r.values(variables);
+                if (values.size() != 1)
                 {
                     // IN query, we only support it on the clustering column
                     assert name.position == names.size() - 1;
                     // 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<ByteBuffer> s = new TreeSet<ByteBuffer>(isReversed ? cfDef.cfm.comparator.reverseComparator : cfDef.cfm.comparator);
-                    for (Term t : r.eqValues)
+                    for (ByteBuffer val : values)
                     {
-                        ByteBuffer val = t.bindAndGet(variables);
                         if (val == null)
                             throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
                         ColumnNameBuilder copy = builder.copy().add(val);
@@ -694,20 +696,11 @@ public class SelectStatement implements CQLStatement
                     return new ArrayList<ByteBuffer>(s);
                 }
 
-                ByteBuffer val = r.eqValues.get(0).bindAndGet(variables);
+                ByteBuffer val = values.get(0);
                 if (val == null)
                     throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
                 builder.add(val);
             }
-            else
-            {
-                Term t = r.bound(b);
-                assert t != null;
-                ByteBuffer val = t.bindAndGet(variables);
-                if (val == null)
-                    throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
-                return Collections.singletonList(builder.add(val, r.getRelation(eocBound, b)).build());
-            }
         }
         // 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,
@@ -749,31 +742,36 @@ public class SelectStatement implements CQLStatement
                     throw new AssertionError();
             }
 
-            if (restriction.isEquality())
-            {
-                assert restriction.eqValues.size() == 1; // IN is not supported for indexed columns.
-                ByteBuffer value = restriction.eqValues.get(0).bindAndGet(variables);
-                if (value == null)
-                    throw new InvalidRequestException(String.format("Unsupported null value for indexed column %s", name));
-                if (value.remaining() > 0xFFFF)
-                    throw new InvalidRequestException("Index expression values may not be larger than 64K");
-                expressions.add(new IndexExpression(name.name.key, IndexOperator.EQ, value));
-            }
-            else
+            if (restriction.isSlice())
             {
+                Restriction.Slice slice = (Restriction.Slice)restriction;
                 for (Bound b : Bound.values())
                 {
-                    if (restriction.bound(b) != null)
+                    if (slice.hasBound(b))
                     {
-                        ByteBuffer value = restriction.bound(b).bindAndGet(variables);
+                        ByteBuffer value = slice.bound(b, variables);
                         if (value == null)
                             throw new InvalidRequestException(String.format("Unsupported null value for indexed column %s", name));
                         if (value.remaining() > 0xFFFF)
                             throw new InvalidRequestException("Index expression values may not be larger than 64K");
-                        expressions.add(new IndexExpression(name.name.key, restriction.getIndexOperator(b), value));
+                        expressions.add(new IndexExpression(name.name.key, slice.getIndexOperator(b), value));
                     }
                 }
             }
+            else
+            {
+                List<ByteBuffer> values = restriction.values(variables);
+
+                if (values.size() != 1)
+                    throw new InvalidRequestException("IN restrictions are not supported on indexed columns");
+
+                ByteBuffer value = values.get(0);
+                if (value == null)
+                    throw new InvalidRequestException(String.format("Unsupported null value for indexed column %s", name));
+                if (value.remaining() > 0xFFFF)
+                    throw new InvalidRequestException("Index expression values may not be larger than 64K");
+                expressions.add(new IndexExpression(name.name.key, IndexOperator.EQ, value));
+            }
         }
         return expressions;
     }
@@ -787,20 +785,22 @@ public class SelectStatement implements CQLStatement
         // If the restriction for the last column alias is an IN, respect
         // requested order
         Restriction last = columnRestrictions[columnRestrictions.length - 1];
-        if (last == null || !last.isEquality())
+        if (last == null || last.isSlice())
             return cf.getSortedColumns();
 
         ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
         for (int i = 0; i < columnRestrictions.length - 1; i++)
-            builder.add(columnRestrictions[i].eqValues.get(0).bindAndGet(variables));
+            builder.add(columnRestrictions[i].values(variables).get(0));
+
 
-        final List<ByteBuffer> requested = new ArrayList<ByteBuffer>(last.eqValues.size());
-        Iterator<Term> iter = last.eqValues.iterator();
+        List<ByteBuffer> values = last.values(variables);
+        final List<ByteBuffer> requested = new ArrayList<ByteBuffer>(values.size());
+        Iterator<ByteBuffer> iter = values.iterator();
         while (iter.hasNext())
         {
-            Term t = iter.next();
+            ByteBuffer t = iter.next();
             ColumnNameBuilder b = iter.hasNext() ? builder.copy() : builder;
-            requested.add(b.add(t.bindAndGet(variables)).build());
+            requested.add(b.add(t).build());
         }
 
         return new Iterable<Column>()
@@ -881,9 +881,9 @@ public class SelectStatement implements CQLStatement
                 else if (sliceRestriction != null)
                 {
                     // For dynamic CF, the column could be out of the requested bounds, filter here
-                    if (!sliceRestriction.isInclusive(Bound.START) && c.name().equals(sliceRestriction.bound(Bound.START).bindAndGet(variables)))
+                    if (!sliceRestriction.isInclusive(Bound.START) && c.name().equals(sliceRestriction.bound(Bound.START, variables)))
                         continue;
-                    if (!sliceRestriction.isInclusive(Bound.END) && c.name().equals(sliceRestriction.bound(Bound.END).bindAndGet(variables)))
+                    if (!sliceRestriction.isInclusive(Bound.END) && c.name().equals(sliceRestriction.bound(Bound.END, variables)))
                         continue;
                 }
 
@@ -1177,7 +1177,7 @@ public class SelectStatement implements CQLStatement
                     }
                     throw new InvalidRequestException(String.format("partition key part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cname, previous));
                 }
-                else if (restriction.onToken)
+                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;
@@ -1187,9 +1187,9 @@ public class SelectStatement implements CQLStatement
                 {
                     throw new InvalidRequestException(String.format("The token() function must be applied to all partition key components or none of them"));
                 }
-                else if (restriction.isEquality())
+                else if (!restriction.isSlice())
                 {
-                    if (restriction.isINRestriction())
+                    if (restriction.isIN())
                     {
                         // We only support IN for the last name so far
                         if (i != stmt.keyRestrictions.length - 1)
@@ -1235,15 +1235,16 @@ public class SelectStatement implements CQLStatement
                     }
                     throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted (preceding part %s is either not restricted or by a non-EQ relation)", cname, previous));
                 }
-                else if (!restriction.isEquality())
+                else if (restriction.isSlice())
                 {
                     canRestrictFurtherComponents = false;
+                    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 (!cfDef.isComposite && (!restriction.isInclusive(Bound.START) || !restriction.isInclusive(Bound.END)))
-                        stmt.sliceRestriction = restriction;
+                    if (!cfDef.isComposite && (!slice.isInclusive(Bound.START) || !slice.isInclusive(Bound.END)))
+                        stmt.sliceRestriction = slice;
                 }
-                else if (restriction.isINRestriction())
+                else if (restriction.isIN())
                 {
                     // We only support IN for the last name and for compact storage so far
                     // TODO: #3885 allows us to extend to non compact as well, but that remains to be done
@@ -1439,21 +1440,32 @@ public class SelectStatement implements CQLStatement
                             throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", name));
                         Term t = newRel.getValue().prepare(receiver);
                         t.collectMarkerSpecification(boundNames);
-                        restriction = new Restriction(t, newRel.onToken);
+                        restriction = new Restriction.EQ(t, newRel.onToken);
                     }
                     break;
                 case IN:
                     if (restriction != null)
                         throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes a IN", name));
-                    List<Term> inValues = new ArrayList<Term>(newRel.getInValues().size());
-                    for (Term.Raw raw : newRel.getInValues())
+
+                    if (newRel.getInValues() == null)
                     {
-                        Term t = raw.prepare(receiver);
+                        // Means we have a "SELECT ... IN ?"
+                        assert newRel.getValue() != null;
+                        Term t = newRel.getValue().prepare(receiver);
                         t.collectMarkerSpecification(boundNames);
-                        inValues.add(t);
+                        restriction = Restriction.IN.create(t);
+                    }
+                    else
+                    {
+                        List<Term> inValues = new ArrayList<Term>(newRel.getInValues().size());
+                        for (Term.Raw raw : newRel.getInValues())
+                        {
+                            Term t = raw.prepare(receiver);
+                            t.collectMarkerSpecification(boundNames);
+                            inValues.add(t);
+                        }
+                        restriction = Restriction.IN.create(inValues);
                     }
-                    restriction = new Restriction(inValues);
-
                     break;
                 case GT:
                 case GTE:
@@ -1461,10 +1473,12 @@ public class SelectStatement implements CQLStatement
                 case LTE:
                     {
                         if (restriction == null)
-                            restriction = new Restriction(newRel.onToken);
+                            restriction = new Restriction.Slice(newRel.onToken);
+                        else if (!restriction.isSlice())
+                            throw new InvalidRequestException(String.format("%s cannot be restricted by both an equal and an inequal relation", name));
                         Term t = newRel.getValue().prepare(receiver);
                         t.collectMarkerSpecification(boundNames);
-                        restriction.setBound(name.name, newRel.operator(), t);
+                        ((Restriction.Slice)restriction).setBound(name.name, newRel.operator(), t);
                     }
                     break;
             }
@@ -1484,144 +1498,6 @@ public class SelectStatement implements CQLStatement
         }
     }
 
-    // A rather raw class that simplify validation and query for select
-    // Don't made public as this can be easily badly used
-    private static class Restriction
-    {
-        // for equality
-        List<Term> eqValues; // if null, it's a restriction by bounds
-
-        // for bounds
-        private final Term[] bounds;
-        private final boolean[] boundInclusive;
-
-        final boolean onToken;
-
-
-        Restriction(List<Term> values, boolean onToken)
-        {
-            this.eqValues = values;
-            this.bounds = null;
-            this.boundInclusive = null;
-            this.onToken = onToken;
-        }
-
-        Restriction(List<Term> values)
-        {
-            this(values, false);
-        }
-
-        Restriction(Term value, boolean onToken)
-        {
-            this(Collections.singletonList(value), onToken);
-        }
-
-        Restriction(boolean onToken)
-        {
-            this.eqValues = null;
-            this.bounds = new Term[2];
-            this.boundInclusive = new boolean[2];
-            this.onToken = onToken;
-        }
-
-        boolean isEquality()
-        {
-            return eqValues != null;
-        }
-
-        boolean isINRestriction()
-        {
-            return isEquality() && (eqValues.isEmpty() || eqValues.size() > 1);
-        }
-
-        public Term bound(Bound b)
-        {
-            return bounds[b.idx];
-        }
-
-        public boolean isInclusive(Bound b)
-        {
-            return bounds[b.idx] == null || boundInclusive[b.idx];
-        }
-
-        public Relation.Type getRelation(Bound eocBound, Bound inclusiveBound)
-        {
-            switch (eocBound)
-            {
-                case START:
-                    return boundInclusive[inclusiveBound.idx] ? Relation.Type.GTE : Relation.Type.GT;
-                case END:
-                    return boundInclusive[inclusiveBound.idx] ? Relation.Type.LTE : Relation.Type.LT;
-            }
-            throw new AssertionError();
-        }
-
-        public IndexOperator getIndexOperator(Bound b)
-        {
-            switch (b)
-            {
-                case START:
-                    return boundInclusive[b.idx] ? IndexOperator.GTE : IndexOperator.GT;
-                case END:
-                    return boundInclusive[b.idx] ? IndexOperator.LTE : IndexOperator.LT;
-            }
-            throw new AssertionError();
-        }
-
-        public void setBound(ColumnIdentifier name, Relation.Type type, Term t) throws InvalidRequestException
-        {
-            Bound b;
-            boolean inclusive;
-            switch (type)
-            {
-                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 == null)
-                throw new InvalidRequestException(String.format("%s cannot be restricted by both an equal and an inequal relation", name));
-
-            if (bounds[b.idx] != null)
-                throw new InvalidRequestException(String.format("Invalid restrictions found on %s", name));
-            bounds[b.idx] = t;
-            boundInclusive[b.idx] = inclusive;
-        }
-
-        @Override
-        public String toString()
-        {
-            String s;
-            if (eqValues == null)
-            {
-                s = String.format("SLICE(%s %s, %s %s)", boundInclusive[0] ? ">=" : ">",
-                                                            bounds[0],
-                                                            boundInclusive[1] ? "<=" : "<",
-                                                            bounds[1]);
-            }
-            else
-            {
-                s = String.format("EQ(%s)", eqValues);
-            }
-            return onToken ? s + "*" : s;
-        }
-    }
-
     public static class Parameters
     {
         private final Map<ColumnIdentifier, Boolean> orderings;


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

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

Conflicts:
	src/java/org/apache/cassandra/cql3/statements/SelectStatement.java


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

Branch: refs/heads/trunk
Commit: 1c9858306517ddecc5dd9d17b85d20a9724afe72
Parents: 5102e8d e93578b
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Sep 18 17:59:17 2013 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Sep 18 17:59:17 2013 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 doc/cql3/CQL.textile                            |   3 +
 .../apache/cassandra/cql3/AbstractMarker.java   |  25 ++
 src/java/org/apache/cassandra/cql3/Cql.g        |   1 +
 .../org/apache/cassandra/cql3/Relation.java     |   2 +-
 .../apache/cassandra/cql3/statements/Bound.java |  35 ++
 .../cql3/statements/ModificationStatement.java  |  69 ++--
 .../cassandra/cql3/statements/Restriction.java  | 289 +++++++++++++++
 .../cql3/statements/SelectStatement.java        | 348 ++++++-------------
 9 files changed, 511 insertions(+), 262 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1c985830/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1c985830/src/java/org/apache/cassandra/cql3/statements/Restriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/Restriction.java
index 0000000,45cc607..73444ad
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/cql3/statements/Restriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
@@@ -1,0 -1,289 +1,289 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.cassandra.cql3.statements;
+ 
+ import java.nio.ByteBuffer;
+ import java.util.ArrayList;
+ import java.util.Collections;
+ import java.util.List;
+ 
+ import org.apache.cassandra.exceptions.InvalidRequestException;
 -import org.apache.cassandra.thrift.IndexOperator;
++import org.apache.cassandra.db.IndexExpression;
+ import org.apache.cassandra.cql3.*;
+ 
+ /**
+  * 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();
+ 
+     // Only supported for EQ and IN, but it's convenient to have here
+     public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException;
+ 
+     public static class EQ implements Restriction
+     {
+         private final Term value;
+         private final boolean onToken;
+ 
+         public EQ(Term value, boolean onToken)
+         {
+             this.value = value;
+             this.onToken = onToken;
+         }
+ 
+         public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
+         {
+             return Collections.singletonList(value.bindAndGet(variables));
+         }
+ 
+         public boolean isSlice()
+         {
+             return false;
+         }
+ 
+         public boolean isEQ()
+         {
+             return true;
+         }
+ 
+         public boolean isIN()
+         {
+             return false;
+         }
+ 
+         public boolean isOnToken()
+         {
+             return onToken;
+         }
+ 
+         @Override
+         public String toString()
+         {
+             return String.format("EQ(%s)%s", value, onToken ? "*" : "");
+         }
+     }
+ 
+     public static abstract class IN implements Restriction
+     {
+         public static IN create(List<Term> values)
+         {
+             return new WithValues(values);
+         }
+ 
+         public static IN create(Term value) throws InvalidRequestException
+         {
+             assert value instanceof Lists.Marker; // we shouldn't have got there otherwise
+             return new WithMarker((Lists.Marker)value);
+         }
+ 
+         public boolean isSlice()
+         {
+             return false;
+         }
+ 
+         public boolean isEQ()
+         {
+             return false;
+         }
+ 
+         public boolean isIN()
+         {
+             return true;
+         }
+ 
+         public boolean isOnToken()
+         {
+             return false;
+         }
+ 
+         private static class WithValues extends IN
+         {
+             private final List<Term> values;
+ 
+             private WithValues(List<Term> values)
+             {
+                 this.values = values;
+             }
+ 
+             public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
+             {
+                 List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(values.size());
+                 for (Term value : values)
+                     buffers.add(value.bindAndGet(variables));
+                 return buffers;
+             }
+ 
+             @Override
+             public String toString()
+             {
+                 return String.format("IN(%s)", values);
+             }
+         }
+ 
+         private static class WithMarker extends IN
+         {
+             private final Lists.Marker marker;
+ 
+             private WithMarker(Lists.Marker marker)
+             {
+                 this.marker = marker;
+             }
+ 
+             public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
+             {
+                 Lists.Value lval = marker.bind(variables);
+                 if (lval == null)
+                     throw new InvalidRequestException("Invalid null value for IN restriction");
+                 return lval.elements;
+             }
+ 
+             @Override
+             public String toString()
+             {
+                 return "IN ?";
+             }
+         }
+     }
+ 
+     public static class Slice implements Restriction
+     {
+         private final Term[] bounds;
+         private final boolean[] boundInclusive;
+         private final boolean onToken;
+ 
+         public Slice(boolean onToken)
+         {
+             this.bounds = new Term[2];
+             this.boundInclusive = new boolean[2];
+             this.onToken = onToken;
+         }
+ 
+         public boolean isSlice()
+         {
+             return true;
+         }
+ 
+         public boolean isEQ()
+         {
+             return false;
+         }
+ 
+         public boolean isIN()
+         {
+             return false;
+         }
+ 
+         public List<ByteBuffer> values(List<ByteBuffer> variables) throws InvalidRequestException
+         {
+             throw new UnsupportedOperationException();
+         }
+ 
+         public boolean isOnToken()
+         {
+             return onToken;
+         }
+ 
+         public boolean hasBound(Bound b)
+         {
+             return bounds[b.idx] != null;
+         }
+ 
+         public ByteBuffer bound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
+         {
+             return bounds[b.idx].bindAndGet(variables);
+         }
+ 
+         public boolean isInclusive(Bound b)
+         {
+             return bounds[b.idx] == null || boundInclusive[b.idx];
+         }
+ 
+         public Relation.Type getRelation(Bound eocBound, Bound inclusiveBound)
+         {
+             switch (eocBound)
+             {
+                 case START:
+                     return boundInclusive[inclusiveBound.idx] ? Relation.Type.GTE : Relation.Type.GT;
+                 case END:
+                     return boundInclusive[inclusiveBound.idx] ? Relation.Type.LTE : Relation.Type.LT;
+             }
+             throw new AssertionError();
+         }
+ 
 -        public IndexOperator getIndexOperator(Bound b)
++        public IndexExpression.Operator getIndexOperator(Bound b)
+         {
+             switch (b)
+             {
+                 case START:
 -                    return boundInclusive[b.idx] ? IndexOperator.GTE : IndexOperator.GT;
++                    return boundInclusive[b.idx] ? IndexExpression.Operator.GTE : IndexExpression.Operator.GT;
+                 case END:
 -                    return boundInclusive[b.idx] ? IndexOperator.LTE : IndexOperator.LT;
++                    return boundInclusive[b.idx] ? IndexExpression.Operator.LTE : IndexExpression.Operator.LT;
+             }
+             throw new AssertionError();
+         }
+ 
+         public void setBound(ColumnIdentifier name, Relation.Type type, Term t) throws InvalidRequestException
+         {
+             Bound b;
+             boolean inclusive;
+             switch (type)
+             {
+                 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("Invalid restrictions found on %s", 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 ? "*" : "");
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1c985830/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 8412dd4,c85864f..9a3ffc3
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -772,6 -758,20 +756,20 @@@ public class SelectStatement implement
                      }
                  }
              }
+             else
+             {
+                 List<ByteBuffer> values = restriction.values(variables);
+ 
+                 if (values.size() != 1)
+                     throw new InvalidRequestException("IN restrictions are not supported on indexed columns");
+ 
+                 ByteBuffer value = values.get(0);
+                 if (value == null)
+                     throw new InvalidRequestException(String.format("Unsupported null value for indexed column %s", name));
+                 if (value.remaining() > 0xFFFF)
+                     throw new InvalidRequestException("Index expression values may not be larger than 64K");
 -                expressions.add(new IndexExpression(name.name.key, IndexOperator.EQ, value));
++                expressions.add(new IndexExpression(name.name.key, IndexExpression.Operator.EQ, value));
+             }
          }
          return expressions;
      }