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 2012/02/07 11:21:41 UTC

[2/2] git commit: Support query by names for compact CF (CQL3)

Support query by names for compact CF (CQL3)

patch by slebresne; reviewed by urandom for CASSANDRA-3791


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

Branch: refs/heads/trunk
Commit: 3459c38d7b0183752e2dcd1cb497d1e502096838
Parents: abd4555
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Thu Jan 26 17:17:59 2012 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Feb 7 11:11:48 2012 +0100

----------------------------------------------------------------------
 .../org/apache/cassandra/cql3/CFDefinition.java    |    4 +-
 .../cassandra/cql3/statements/SelectStatement.java |  435 +++++++++------
 2 files changed, 270 insertions(+), 169 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/3459c38d/src/java/org/apache/cassandra/cql3/CFDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CFDefinition.java b/src/java/org/apache/cassandra/cql3/CFDefinition.java
index ab1a29c..a99f6ee 100644
--- a/src/java/org/apache/cassandra/cql3/CFDefinition.java
+++ b/src/java/org/apache/cassandra/cql3/CFDefinition.java
@@ -222,13 +222,13 @@ public class CFDefinition implements Iterable<CFDefinition.Name>
         {
             this.kind = kind;
             this.name = name;
-            this.compositePosition = position;
+            this.position = position;
             this.type = type;
         }
 
         public final Kind kind;
         public final ColumnIdentifier name;
-        public final int compositePosition; // only make sense for COLUMN_ALIAS if CFDefinition.isComposite()
+        public final int position; // only make sense for COLUMN_ALIAS
         public final AbstractType<?> type;
 
         @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3459c38d/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 4eaf72a..c68a6b6 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -23,6 +23,8 @@ import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.TimeoutException;
 
+import com.google.common.base.Objects;
+import com.google.common.collect.AbstractIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -31,6 +33,7 @@ import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.IColumn;
 import org.apache.cassandra.db.CounterColumn;
+import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.RangeSliceCommand;
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.db.Row;
@@ -85,13 +88,27 @@ public class SelectStatement implements CQLStatement
     public final CFDefinition cfDef;
     public final Parameters parameters;
     private final List<Pair<CFDefinition.Name, ColumnIdentifier>> selectedNames = new ArrayList<Pair<CFDefinition.Name, ColumnIdentifier>>(); // empty => wildcard
-    private final Map<ColumnIdentifier, Restriction> restrictions = new HashMap<ColumnIdentifier, Restriction>();
-    private boolean hasIndexedExpression;
+
+    private Restriction keyRestriction;
+    private final Restriction[] columnRestrictions;
+    private final Map<CFDefinition.Name, Restriction> metadataRestrictions = new HashMap<CFDefinition.Name, Restriction>();
+
+    private static enum Bound
+    {
+        START(0), END(1);
+
+        public final int idx;
+        Bound(int idx)
+        {
+            this.idx = idx;
+        }
+    };
 
     public SelectStatement(CFDefinition cfDef, int boundTerms, Parameters parameters)
     {
         this.cfDef = cfDef;
         this.boundTerms = boundTerms;
+        this.columnRestrictions = new Restriction[cfDef.columns.size()];
         this.parameters = parameters;
     }
 
@@ -143,7 +160,7 @@ public class SelectStatement implements CQLStatement
                     new HashMap<ByteBuffer, String>(),
                     TypeParser.getShortName(cfDef.cfm.comparator),
                     TypeParser.getShortName(cfDef.cfm.getDefaultValidator()));
-            result.rows = process(rows, result.schema);
+            result.rows = process(rows, result.schema, variables);
             return result;
         }
     }
@@ -166,8 +183,8 @@ public class SelectStatement implements CQLStatement
         // ...a range (slice) of column names
         if (isColumnRange())
         {
-            ByteBuffer start =getRequestedBound(true, variables);
-            ByteBuffer finish = getRequestedBound(false, variables);
+            ByteBuffer start = getRequestedBound(Bound.START, variables);
+            ByteBuffer finish = getRequestedBound(Bound.END, variables);
 
             // Note that we use the total limit for every key. This is
             // potentially inefficient, but then again, IN + LIMIT is not a
@@ -217,8 +234,8 @@ public class SelectStatement implements CQLStatement
         List<Row> rows;
         IPartitioner<?> p = StorageService.getPartitioner();
 
-        ByteBuffer startKeyBytes = getKeyStart(variables);
-        ByteBuffer finishKeyBytes = getKeyFinish(variables);
+        ByteBuffer startKeyBytes = getKeyBound(Bound.START, variables);
+        ByteBuffer finishKeyBytes = getKeyBound(Bound.END, variables);
 
         RowPosition startKey = RowPosition.forKey(startKeyBytes, p);
         RowPosition finishKey = RowPosition.forKey(finishKeyBytes, p);
@@ -230,15 +247,15 @@ public class SelectStatement implements CQLStatement
                 throw new InvalidRequestException("Start key must sort before (or equal to) finish key in your partitioner!");
         }
         AbstractBounds<RowPosition> bounds;
-        if (includeStartKey())
+        if (includeKeyBound(Bound.START))
         {
-            bounds = includeFinishKey()
+            bounds = includeKeyBound(Bound.END)
                    ? new Bounds<RowPosition>(startKey, finishKey)
                    : new IncludingExcludingBounds<RowPosition>(startKey, finishKey);
         }
         else
         {
-            bounds = includeFinishKey()
+            bounds = includeKeyBound(Bound.END)
                    ? new Range<RowPosition>(startKey, finishKey)
                    : new ExcludingBounds<RowPosition>(startKey, finishKey);
         }
@@ -280,8 +297,8 @@ public class SelectStatement implements CQLStatement
         if (isColumnRange())
         {
             SliceRange sliceRange = new SliceRange();
-            sliceRange.start = getRequestedBound(true, variables);
-            sliceRange.finish = getRequestedBound(false, variables);
+            sliceRange.start = getRequestedBound(Bound.START, variables);
+            sliceRange.finish = getRequestedBound(Bound.END, variables);
             sliceRange.reversed = parameters.isColumnsReversed;
             sliceRange.count = -1; // We use this for range slices, where the count is ignored in favor of the global column count
             thriftSlicePredicate.slice_range = sliceRange;
@@ -303,77 +320,47 @@ public class SelectStatement implements CQLStatement
 
     private boolean isKeyRange()
     {
-        if (hasIndexedExpression)
+        // If indexed columns, they always use getRangeSlices
+        if (!metadataRestrictions.isEmpty())
             return true;
 
-        Restriction r = restrictions.get(cfDef.key.name);
-        return r == null || !r.isEquality();
+        return keyRestriction == null || !keyRestriction.isEquality();
     }
 
     private Collection<ByteBuffer> getKeys(final List<ByteBuffer> variables) throws InvalidRequestException
     {
-        final Restriction r = restrictions.get(cfDef.key.name);
-        if (r == null || !r.isEquality())
-            throw new IllegalStateException();
+        assert keyRestriction != null || keyRestriction.isEquality();
 
-        List<ByteBuffer> keys = new ArrayList<ByteBuffer>(r.eqValues.size());
-        for (Term t : r.eqValues)
+        List<ByteBuffer> keys = new ArrayList<ByteBuffer>(keyRestriction.eqValues.size());
+        for (Term t : keyRestriction.eqValues)
             keys.add(t.getByteBuffer(cfDef.key.type, variables));
         return keys;
     }
 
-    private ByteBuffer getKeyStart(List<ByteBuffer> variables) throws InvalidRequestException
+    private ByteBuffer getKeyBound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
     {
-        Restriction r = restrictions.get(cfDef.key.name);
-        if (r == null)
+        if (keyRestriction == null)
         {
-            return null;
+            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
         }
-        else if (r.isEquality())
+        else if (keyRestriction.isEquality())
         {
-            assert r.eqValues.size() == 1;
-            return r.eqValues.get(0).getByteBuffer(cfDef.key.type, variables);
+            assert keyRestriction.eqValues.size() == 1;
+            return keyRestriction.eqValues.get(0).getByteBuffer(cfDef.key.type, variables);
         }
         else
         {
-            return r.start == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : r.start.getByteBuffer(cfDef.key.type, variables);
+            Term bound = keyRestriction.bound(b);
+            return bound == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : bound.getByteBuffer(cfDef.key.type, variables);
         }
     }
 
-    private boolean includeStartKey()
+    private boolean includeKeyBound(Bound b)
     {
-        Restriction r = restrictions.get(cfDef.key.name);
-        if (r == null || r.isEquality())
+        if (keyRestriction == null || keyRestriction.isEquality())
             return true;
         else
-            return r.startInclusive;
-    }
-
-    private ByteBuffer getKeyFinish(List<ByteBuffer> variables) throws InvalidRequestException
-    {
-        Restriction r = restrictions.get(cfDef.key.name);
-        if (r == null)
-        {
-            return null;
-        }
-        else if (r.isEquality())
-        {
-            assert r.eqValues.size() == 1;
-            return r.eqValues.get(0).getByteBuffer(cfDef.key.type, variables);
-        }
-        else
-        {
-            return r.end == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : r.end.getByteBuffer(cfDef.key.type, variables);
-        }
-    }
-
-    private boolean includeFinishKey()
-    {
-        Restriction r = restrictions.get(cfDef.key.name);
-        if (r == null || r.isEquality())
-            return true;
-        else
-            return r.endInclusive;
+            return keyRestriction.isInclusive(b);
     }
 
     private boolean isColumnRange()
@@ -384,9 +371,8 @@ public class SelectStatement implements CQLStatement
 
         // Otherwise, 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 (CFDefinition.Name name : cfDef.columns.values())
+        for (Restriction r : columnRestrictions)
         {
-            Restriction r = restrictions.get(name.name);
             if (r == null || !r.isEquality())
                 return true;
         }
@@ -403,11 +389,27 @@ public class SelectStatement implements CQLStatement
         assert !isColumnRange();
 
         ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
-        for (CFDefinition.Name name : cfDef.columns.values())
+        for (Restriction r : columnRestrictions)
         {
-            Restriction r = restrictions.get(name.name);
-            assert r != null && r.isEquality() && r.eqValues.size() == 1;
-            builder.add(r.eqValues.get(0), Relation.Type.EQ, variables);
+            assert r != null && r.isEquality();
+            if (r.eqValues.size() > 1)
+            {
+                // We have a IN. We only support this for the last column, so just create all columns and return.
+                List<ByteBuffer> columns = new ArrayList<ByteBuffer>(r.eqValues.size());
+                Iterator<Term> iter = r.eqValues.iterator();
+                while (iter.hasNext())
+                {
+                    Term v = iter.next();
+                    ColumnNameBuilder b = iter.hasNext() ? builder.copy() : builder;
+                    ByteBuffer cname = b.add(v, Relation.Type.EQ, variables).build();
+                    columns.add(cname);
+                }
+                return columns;
+            }
+            else
+            {
+                builder.add(r.eqValues.get(0), Relation.Type.EQ, variables);
+            }
         }
 
         if (cfDef.isCompact)
@@ -433,20 +435,19 @@ public class SelectStatement implements CQLStatement
         }
     }
 
-    private ByteBuffer getRequestedBound(boolean isStart, List<ByteBuffer> variables) throws InvalidRequestException
+    private ByteBuffer getRequestedBound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
     {
         assert isColumnRange();
 
         ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
-        for (CFDefinition.Name name : cfDef.columns.values())
+        for (Restriction r : columnRestrictions)
         {
-            Restriction r = restrictions.get(name.name);
             if (r == null)
             {
                 // 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.
-                if (builder.componentCount() > 0 && !isStart)
+                if (builder.componentCount() > 0 && b == Bound.END)
                     return builder.buildAsEndOfRange();
                 else
                     return builder.build();
@@ -459,14 +460,11 @@ public class SelectStatement implements CQLStatement
             }
             else
             {
-                Term t = isStart ? r.start : r.end;
-                Relation.Type op = isStart
-                    ? (r.startInclusive ? Relation.Type.GTE : Relation.Type.GT)
-                    : (r.endInclusive ? Relation.Type.LTE : Relation.Type.LT);
+                Term t = r.bound(b);
                 if (t == null)
                     return ByteBufferUtil.EMPTY_BYTE_BUFFER;
                 else
-                    return builder.add(t, op, variables).build();
+                    return builder.add(t, r.getRelation(b), variables).build();
             }
         }
         // Means no relation at all or everything was an equal
@@ -475,16 +473,14 @@ public class SelectStatement implements CQLStatement
 
     private List<IndexExpression> getIndexExpressions(List<ByteBuffer> variables) throws InvalidRequestException
     {
-        if (!hasIndexedExpression)
+        if (metadataRestrictions.isEmpty())
             return Collections.<IndexExpression>emptyList();
 
         List<IndexExpression> expressions = new ArrayList<IndexExpression>();
-        for (CFDefinition.Name name : cfDef.metadata.values())
+        for (Map.Entry<CFDefinition.Name, Restriction> entry : metadataRestrictions.entrySet())
         {
-            Restriction restriction = restrictions.get(name.name);
-            if (restriction == null)
-                continue;
-
+            CFDefinition.Name name = entry.getKey();
+            Restriction restriction = entry.getValue();
             if (restriction.isEquality())
             {
                 for (Term t : restriction.eqValues)
@@ -495,15 +491,10 @@ public class SelectStatement implements CQLStatement
             }
             else
             {
-                if (restriction.start != null)
+                for (Bound b : Bound.values())
                 {
-                    ByteBuffer value = restriction.start.getByteBuffer(name.type, variables);
-                    expressions.add(new IndexExpression(name.name.key, restriction.startInclusive ? IndexOperator.GTE : IndexOperator.GT, value));
-                }
-                if (restriction.end != null)
-                {
-                    ByteBuffer value = restriction.end.getByteBuffer(name.type, variables);
-                    expressions.add(new IndexExpression(name.name.key, restriction.endInclusive ? IndexOperator.GTE : IndexOperator.GT, value));
+                    ByteBuffer value = restriction.bound(b).getByteBuffer(name.type, variables);
+                    expressions.add(new IndexExpression(name.name.key, restriction.getIndexOperator(b), value));
                 }
             }
         }
@@ -539,7 +530,47 @@ public class SelectStatement implements CQLStatement
         schema.value_types.put(nameAsRequested, TypeParser.getShortName(p.left.type));
     }
 
-    private List<CqlRow> process(List<Row> rows, CqlMetadata schema)
+    private Iterable<IColumn> columnsInOrder(final ColumnFamily cf, final List<ByteBuffer> variables) throws InvalidRequestException
+    {
+        // 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())
+            return cf.getSortedColumns();
+
+        ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
+        for (int i = 0; i < columnRestrictions.length - 1; i++)
+            builder.add(columnRestrictions[i].eqValues.get(0), Relation.Type.EQ, variables);
+
+        final List<ByteBuffer> requested = new ArrayList<ByteBuffer>(last.eqValues.size());
+        Iterator<Term> iter = last.eqValues.iterator();
+        while (iter.hasNext())
+        {
+            Term t = iter.next();
+            ColumnNameBuilder b = iter.hasNext() ? builder.copy() : builder;
+            requested.add(b.add(t, Relation.Type.EQ, variables).build());
+        }
+
+        return new Iterable<IColumn>()
+        {
+            public Iterator<IColumn> iterator()
+            {
+                return new AbstractIterator<IColumn>()
+                {
+                    Iterator<ByteBuffer> iter = requested.iterator();
+                    public IColumn computeNext()
+                    {
+                        if (!iter.hasNext())
+                            return endOfData();
+                        IColumn column = cf.getColumn(iter.next());
+                        return column == null ? computeNext() : column;
+                    }
+                };
+            }
+        };
+    }
+
+    private List<CqlRow> process(List<Row> rows, CqlMetadata schema, List<ByteBuffer> variables) throws InvalidRequestException
     {
         List<CqlRow> cqlRows = new ArrayList<CqlRow>();
         List<Pair<CFDefinition.Name, ColumnIdentifier>> selection = getExpandedSelection();
@@ -553,7 +584,7 @@ public class SelectStatement implements CQLStatement
                 if (row.cf == null)
                     continue;
 
-                for (IColumn c : row.cf.getSortedColumns())
+                for (IColumn c : columnsInOrder(row.cf, variables))
                 {
                     if (c.isMarkedForDelete())
                         continue;
@@ -581,8 +612,8 @@ public class SelectStatement implements CQLStatement
                                 col.setTimestamp(c.timestamp());
                                 if (cfDef.isComposite)
                                 {
-                                    if (name.compositePosition < components.length)
-                                        col.setValue(components[name.compositePosition]);
+                                    if (name.position < components.length)
+                                        col.setValue(components[name.position]);
                                     else
                                         col.setValue(ByteBufferUtil.EMPTY_BYTE_BUFFER);
                                 }
@@ -715,7 +746,7 @@ public class SelectStatement implements CQLStatement
                     col.setValue(key).setTimestamp(-1L);
                     break;
                 case COLUMN_ALIAS:
-                    col.setValue(components[name.compositePosition]);
+                    col.setValue(components[name.position]);
                     col.setTimestamp(-1L);
                     break;
                 case VALUE_ALIAS:
@@ -793,9 +824,6 @@ public class SelectStatement implements CQLStatement
                 if (name == null)
                     throw new InvalidRequestException(String.format("Undefined name %s in where clause ('%s')", rel.getEntity(), rel));
 
-                if (name.kind == CFDefinition.Name.Kind.VALUE_ALIAS)
-                    throw new InvalidRequestException(String.format("Restricting the value of a compact CF (%s) is not supported", name.name));
-
                 if (rel.operator() == Relation.Type.IN)
                 {
                     for (Term value : rel.getInValues())
@@ -809,50 +837,20 @@ public class SelectStatement implements CQLStatement
                         types[value.bindIndex] = name.type;
                 }
 
-                Restriction restriction = stmt.restrictions.get(name.name);
-                switch (rel.operator())
+                switch (name.kind)
                 {
-                    case EQ:
-                        if (restriction != null)
-                            throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", name));
-                        stmt.restrictions.put(name.name, new Restriction(Collections.singletonList(rel.getValue())));
-                        break;
-                    case GT:
-                    case GTE:
-                        if (name.kind == CFDefinition.Name.Kind.KEY_ALIAS && !StorageService.getPartitioner().preservesOrder())
+                    case KEY_ALIAS:
+                        if (rel.operator() != Relation.Type.EQ && rel.operator() != Relation.Type.IN && !StorageService.getPartitioner().preservesOrder())
                             throw new InvalidRequestException("Only EQ and IN relation are supported on first component of the PRIMARY KEY for RandomPartitioner");
-                        if (restriction == null)
-                        {
-                            restriction = new Restriction();
-                            stmt.restrictions.put(name.name, restriction);
-                        }
-                        if (restriction.start != null)
-                            throw new InvalidRequestException(String.format("%s cannot be restricted by more than one Greater-Than relation", name));
-                        restriction.start = rel.getValue();
-                        if (rel.operator() == Relation.Type.GTE)
-                            restriction.startInclusive = true;
+                        stmt.keyRestriction = updateRestriction(name.name, stmt.keyRestriction, rel);
                         break;
-                    case LT:
-                    case LTE:
-                        if (name.kind == CFDefinition.Name.Kind.KEY_ALIAS && !StorageService.getPartitioner().preservesOrder())
-                            throw new InvalidRequestException("Only EQ and IN relation are supported on first component of the PRIMARY KEY for RandomPartitioner");
-                        if (restriction == null)
-                        {
-                            restriction = new Restriction();
-                            stmt.restrictions.put(name.name, restriction);
-                        }
-                        if (restriction.end != null)
-                            throw new InvalidRequestException(String.format("%s cannot be restricted by more than one Lesser-Than relation", name));
-                        restriction.end = rel.getValue();
-                        if (rel.operator() == Relation.Type.LTE)
-                            restriction.endInclusive = true;
+                    case COLUMN_ALIAS:
+                        stmt.columnRestrictions[name.position] = updateRestriction(name.name, stmt.columnRestrictions[name.position], rel);
                         break;
-                    case IN:
-                        if (restriction != null)
-                            throw new InvalidRequestException(String.format("%s cannot be restricted by more than one reation if it includes a IN", name));
-                        if (name.kind != CFDefinition.Name.Kind.KEY_ALIAS)
-                            throw new InvalidRequestException("IN relation can only be applied to the first component of the PRIMARY KEY");
-                        stmt.restrictions.put(name.name, new Restriction(rel.getInValues()));
+                    case VALUE_ALIAS:
+                        throw new InvalidRequestException(String.format("Restricting the value of a compact CF (%s) is not supported", name.name));
+                    case COLUMN_METADATA:
+                        stmt.metadataRestrictions.put(name, updateRestriction(name.name, stmt.metadataRestrictions.get(name), rel));
                         break;
                 }
             }
@@ -865,58 +863,55 @@ public class SelectStatement implements CQLStatement
             // components must have a EQ, and all following must have no restriction
             boolean shouldBeDone = false;
             CFDefinition.Name previous = null;
-            for (CFDefinition.Name cname : cfDef.columns.values())
+            Iterator<CFDefinition.Name> iter = cfDef.columns.values().iterator();
+            for (int i = 0; i < stmt.columnRestrictions.length; i++)
             {
-                Restriction restriction = stmt.restrictions.get(cname.name);
+                CFDefinition.Name cname = iter.next();
+                Restriction restriction = stmt.columnRestrictions[i];
                 if (restriction == null)
                     shouldBeDone = true;
                 else if (shouldBeDone)
                     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())
                     shouldBeDone = true;
-                // We could support IN for the last name, we don't yet
-                else if (restriction.eqValues.size() > 1)
-                    throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted by IN relation", cname));
+                // We only support IN for the last name so far
+                else if (restriction.eqValues.size() > 1 && i != stmt.columnRestrictions.length - 1)
+                    throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted by IN relation (only the first and last parts can)", cname));
 
                 previous = cname;
             }
 
             // Deal with indexed columns
-            if (!cfDef.metadata.values().isEmpty())
+            if (!stmt.metadataRestrictions.isEmpty())
             {
                 boolean hasEq = false;
                 Set<ByteBuffer> indexed = Table.open(keyspace()).getColumnFamilyStore(columnFamily()).indexManager.getIndexedColumns();
 
-                for (CFDefinition.Name name : cfDef.metadata.values())
+                for (Map.Entry<CFDefinition.Name, Restriction> entry : stmt.metadataRestrictions.entrySet())
                 {
-                    Restriction restriction = stmt.restrictions.get(name.name);
-                    if (restriction == null)
-                        continue;
-
-                    stmt.hasIndexedExpression = true;
-                    if (restriction.isEquality() && indexed.contains(name.name.key))
+                    if (entry.getValue().isEquality() && indexed.contains(entry.getKey().name.key))
                     {
                         hasEq = true;
                         break;
                     }
                 }
-
-                if (stmt.hasIndexedExpression && !hasEq)
+                if (!hasEq)
                     throw new InvalidRequestException("No indexed columns present in by-columns clause with Equal operator");
 
                 // If we have indexed columns and the key = X clause, we transform it into a key >= X AND key <= X clause.
                 // If it's a IN relation however, we reject it.
-                Restriction r = stmt.restrictions.get(cfDef.key.name);
-                if (r != null && r.isEquality())
+                if (stmt.keyRestriction != null && stmt.keyRestriction.isEquality())
                 {
-                    if (r.eqValues.size() > 1)
+                    if (stmt.keyRestriction.eqValues.size() > 1)
                         throw new InvalidRequestException("Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
 
-                    r.start = r.eqValues.get(0);
-                    r.startInclusive = true;
-                    r.end = r.eqValues.get(0);
-                    r.endInclusive = true;
-                    r.eqValues = null;
+                    Restriction newRestriction = new Restriction();
+                    for (Bound b : Bound.values())
+                    {
+                        newRestriction.setBound(b, stmt.keyRestriction.eqValues.get(0));
+                        newRestriction.setInclusive(b);
+                    }
+                    stmt.keyRestriction = newRestriction;
                 }
             }
 
@@ -924,14 +919,39 @@ public class SelectStatement implements CQLStatement
             // since we don't know how to reverse otherwise
             if (stmt.parameters.isColumnsReversed)
             {
-                Restriction r = stmt.restrictions.get(cfDef.key.name);
-                if (r == null || !r.isEquality())
+                if (stmt.keyRestriction == null || !stmt.keyRestriction.isEquality())
                     throw new InvalidRequestException("Descending order is only supported is the first part of the PRIMARY KEY is restricted by an Equal or a IN");
             }
 
             return new ParsedStatement.Prepared(stmt, Arrays.<AbstractType<?>>asList(types));
         }
 
+        Restriction updateRestriction(ColumnIdentifier name, Restriction restriction, Relation newRel) throws InvalidRequestException
+        {
+            switch (newRel.operator())
+            {
+                case EQ:
+                    if (restriction != null)
+                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", name));
+                    restriction = new Restriction(newRel.getValue());
+                    break;
+                case IN:
+                    if (restriction != null)
+                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one reation if it includes a IN", name));
+                    restriction = new Restriction(newRel.getInValues());
+                    break;
+                case GT:
+                case GTE:
+                case LT:
+                case LTE:
+                    if (restriction == null)
+                        restriction = new Restriction();
+                    restriction.setBound(name, newRel.operator(), newRel.getValue());
+                    break;
+            }
+            return restriction;
+        }
+
         @Override
         public String toString()
         {
@@ -952,26 +972,107 @@ public class SelectStatement implements CQLStatement
         // for equality
         List<Term> eqValues; // if null, it's a restriction by bounds
 
+        // for bounds
+        private final Term[] bounds;
+        private final boolean[] boundInclusive;
+
         Restriction(List<Term> values)
         {
             this.eqValues = values;
+            this.bounds = null;
+            this.boundInclusive = null;
         }
 
-        // for bounds
-        Term start;
-        boolean startInclusive;
-        Term end;
-        boolean endInclusive;
+        Restriction(Term value)
+        {
+            this(Collections.singletonList(value));
+        }
 
         Restriction()
         {
-            this(null);
+            this.eqValues = null;
+            this.bounds = new Term[2];
+            this.boundInclusive = new boolean[2];
         }
 
         boolean isEquality()
         {
             return eqValues != null;
         }
+
+        public void setBound(Bound b, Term t)
+        {
+            bounds[b.idx] = t;
+        }
+
+        public void setInclusive(Bound b)
+        {
+            boundInclusive[b.idx] = true;
+        }
+
+        public Term bound(Bound b)
+        {
+            return bounds[b.idx];
+        }
+
+        public boolean isInclusive(Bound b)
+        {
+            return boundInclusive[b.idx];
+        }
+
+        public Relation.Type getRelation(Bound b)
+        {
+            switch (b)
+            {
+                case START:
+                    return boundInclusive[b.idx] ? Relation.Type.GTE : Relation.Type.GT;
+                case END:
+                    return boundInclusive[b.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 = null;
+            boolean inclusive = false;
+            switch (type)
+            {
+                case GT:
+                    b = Bound.START;
+                    inclusive = false;
+                    break;
+                case GTE:
+                    b = Bound.START;
+                    inclusive = true;
+                    break;
+                case LTE:
+                    b = Bound.END;
+                    inclusive = true;
+                    break;
+                case LT:
+                    b = Bound.END;
+                    inclusive = true;
+                    break;
+            }
+
+            if (bounds[b.idx] != null)
+                throw new InvalidRequestException(String.format("Invalid restrictions found on %s", name));
+            bounds[b.idx] = t;
+            boundInclusive[b.idx] = inclusive;
+        }
     }
 
     public static class Parameters