You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2015/08/09 22:47:55 UTC

[4/5] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/cassandra-3.0
Commit: 6d0cf7dbd2221c13434958e9e0d5e418f58fe23f
Parents: 0b99d33 c8d163f
Author: blerer <be...@datastax.com>
Authored: Sun Aug 9 22:11:14 2015 +0200
Committer: blerer <be...@datastax.com>
Committed: Sun Aug 9 22:12:16 2015 +0200

----------------------------------------------------------------------
 .../restrictions/MultiColumnRestriction.java    | 37 +++++++----
 .../restrictions/SingleColumnRestriction.java   | 68 ++++++++++++++++++--
 .../SelectMultiColumnRelationTest.java          | 45 +++++++++----
 .../SelectSingleColumnRelationTest.java         | 28 ++++++++
 4 files changed, 149 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d0cf7db/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
index c4bce4c,0000000..84a3952
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
@@@ -1,448 -1,0 +1,461 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.cql3.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +
++import org.apache.cassandra.cql3.Term.Terminal;
++
++import org.apache.cassandra.cql3.Term;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.functions.Function;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.IndexExpression;
 +import org.apache.cassandra.db.composites.CompositesBuilder;
 +import org.apache.cassandra.db.index.SecondaryIndex;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
- 
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 +
 +public abstract class MultiColumnRestriction extends AbstractRestriction
 +{
 +    /**
 +     * The columns to which the restriction apply.
 +     */
 +    protected final List<ColumnDefinition> columnDefs;
 +
 +    public MultiColumnRestriction(List<ColumnDefinition> columnDefs)
 +    {
 +        this.columnDefs = columnDefs;
 +    }
 +
 +    @Override
 +    public boolean isMultiColumn()
 +    {
 +        return true;
 +    }
 +
 +    @Override
 +    public ColumnDefinition getFirstColumn()
 +    {
 +        return columnDefs.get(0);
 +    }
 +
 +    @Override
 +    public ColumnDefinition getLastColumn()
 +    {
 +        return columnDefs.get(columnDefs.size() - 1);
 +    }
 +
 +    @Override
 +    public Collection<ColumnDefinition> getColumnDefs()
 +    {
 +        return columnDefs;
 +    }
 +
 +    @Override
 +    public final Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
 +    {
-             checkTrue(otherRestriction.isMultiColumn(),
-                       "Mixing single column relations and multi column relations on clustering columns is not allowed");
-             return doMergeWith(otherRestriction);
++        // We want to allow query like: (b,c) > (?, ?) AND b < ?
++        if (!otherRestriction.isMultiColumn()
++                && ((SingleColumnRestriction) otherRestriction).canBeConvertedToMultiColumnRestriction())
++        {
++            return doMergeWith(((SingleColumnRestriction) otherRestriction).toMultiColumnRestriction());
++        }
++
++        return doMergeWith(otherRestriction);
 +    }
 +
 +    protected abstract Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException;
 +
 +    /**
 +     * Returns the names of the columns that are specified within this <code>Restrictions</code> and the other one
 +     * as a comma separated <code>String</code>.
 +     *
 +     * @param otherRestriction the other restrictions
 +     * @return the names of the columns that are specified within this <code>Restrictions</code> and the other one
 +     * as a comma separated <code>String</code>.
 +     */
 +    protected final String getColumnsInCommons(Restriction otherRestriction)
 +    {
 +        Set<ColumnDefinition> commons = new HashSet<>(getColumnDefs());
 +        commons.retainAll(otherRestriction.getColumnDefs());
 +        StringBuilder builder = new StringBuilder();
 +        for (ColumnDefinition columnDefinition : commons)
 +        {
 +            if (builder.length() != 0)
 +                builder.append(" ,");
 +            builder.append(columnDefinition.name);
 +        }
 +        return builder.toString();
 +    }
 +
 +    @Override
 +    public final boolean hasSupportingIndex(SecondaryIndexManager indexManager)
 +    {
 +        for (ColumnDefinition columnDef : columnDefs)
 +        {
 +            SecondaryIndex index = indexManager.getIndexForColumn(columnDef.name.bytes);
 +            if (index != null && isSupportedBy(index))
 +                return true;
 +        }
 +        return false;
 +    }
 +
 +    /**
 +     * Check if this type of restriction is supported for by the specified index.
 +     * @param index the Secondary index
 +     *
 +     * @return <code>true</code> this type of restriction is supported by the specified index,
 +     * <code>false</code> otherwise.
 +     */
 +    protected abstract boolean isSupportedBy(SecondaryIndex index);
 +
 +    public static class EQ  extends MultiColumnRestriction
 +    {
 +        protected final Term value;
 +
 +        public EQ(List<ColumnDefinition> columnDefs, Term value)
 +        {
 +            super(columnDefs);
 +            this.value = value;
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return value.getFunctions();
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("EQ(%s)", value);
 +        }
 +
 +        @Override
 +        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal",
 +                                 getColumnsInCommons(otherRestriction));
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return index.supportsOperator(Operator.EQ);
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            Tuples.Value t = ((Tuples.Value) value.bind(options));
 +            List<ByteBuffer> values = t.getElements();
 +            for (int i = 0, m = values.size(); i < m; i++)
 +            {
 +                builder.addElementToAll(values.get(i));
 +                checkFalse(builder.containsNull(), "Invalid null value for column %s", columnDefs.get(i).name);
 +            }
 +            return builder;
 +        }
 +
 +        @Override
 +        public final void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                               SecondaryIndexManager indexManager,
 +                                               QueryOptions options) throws InvalidRequestException
 +        {
 +            Tuples.Value t = ((Tuples.Value) value.bind(options));
 +            List<ByteBuffer> values = t.getElements();
 +
 +            for (int i = 0, m = columnDefs.size(); i < m; i++)
 +            {
 +                ColumnDefinition columnDef = columnDefs.get(i);
 +                ByteBuffer component = validateIndexedValue(columnDef, values.get(i));
 +                expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, component));
 +            }
 +        }
 +    }
 +
 +    public abstract static class IN extends MultiColumnRestriction
 +    {
 +        /**
 +         * {@inheritDoc}
 +         */
 +        @Override
 +        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            List<List<ByteBuffer>> splitInValues = splitValues(options);
 +            builder.addAllElementsToAll(splitInValues);
 +
 +            if (builder.containsNull())
 +                throw invalidRequest("Invalid null value in condition for columns: %s", ColumnDefinition.toIdentifiers(columnDefs));
 +            return builder;
 +        }
 +
 +        public IN(List<ColumnDefinition> columnDefs)
 +        {
 +            super(columnDefs);
 +        }
 +
 +        @Override
 +        public boolean isIN()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            throw invalidRequest("%s cannot be restricted by more than one relation if it includes a IN",
 +                                 getColumnsInCommons(otherRestriction));
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return index.supportsOperator(Operator.IN);
 +        }
 +
 +        @Override
 +        public final void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                               SecondaryIndexManager indexManager,
 +                                               QueryOptions options) throws InvalidRequestException
 +        {
 +            List<List<ByteBuffer>> splitInValues = splitValues(options);
 +            checkTrue(splitInValues.size() == 1, "IN restrictions are not supported on indexed columns");
 +            List<ByteBuffer> values = splitInValues.get(0);
 +
 +            for (int i = 0, m = columnDefs.size(); i < m; i++)
 +            {
 +                ColumnDefinition columnDef = columnDefs.get(i);
 +                ByteBuffer component = validateIndexedValue(columnDef, values.get(i));
 +                expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, component));
 +            }
 +        }
 +
 +        protected abstract List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException;
 +    }
 +
 +    /**
 +     * An IN restriction that has a set of terms for in values.
 +     * For example: "SELECT ... WHERE (a, b, c) IN ((1, 2, 3), (4, 5, 6))" or "WHERE (a, b, c) IN (?, ?)"
 +     */
 +    public static class InWithValues extends MultiColumnRestriction.IN
 +    {
 +        protected final List<Term> values;
 +
 +        public InWithValues(List<ColumnDefinition> columnDefs, List<Term> values)
 +        {
 +            super(columnDefs);
 +            this.values = values;
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return Terms.getFunctions(values);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("IN(%s)", values);
 +        }
 +
 +        @Override
 +        protected List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
 +        {
 +            List<List<ByteBuffer>> buffers = new ArrayList<>(values.size());
 +            for (Term value : values)
 +            {
 +                Term.MultiItemTerminal term = (Term.MultiItemTerminal) value.bind(options);
 +                buffers.add(term.getElements());
 +            }
 +            return buffers;
 +        }
 +    }
 +
 +    /**
 +     * An IN restriction that uses a single marker for a set of IN values that are tuples.
 +     * For example: "SELECT ... WHERE (a, b, c) IN ?"
 +     */
 +    public static class InWithMarker extends MultiColumnRestriction.IN
 +    {
 +        protected final AbstractMarker marker;
 +
 +        public InWithMarker(List<ColumnDefinition> columnDefs, AbstractMarker marker)
 +        {
 +            super(columnDefs);
 +            this.marker = marker;
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return Collections.emptySet();
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return "IN ?";
 +        }
 +
 +        @Override
 +        protected List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
 +        {
 +            Tuples.InMarker inMarker = (Tuples.InMarker) marker;
 +            Tuples.InValue inValue = inMarker.bind(options);
 +            checkNotNull(inValue, "Invalid null value for IN restriction");
 +            return inValue.getSplitValues();
 +        }
 +    }
 +
 +    public static class Slice extends MultiColumnRestriction
 +    {
 +        private final TermSlice slice;
 +
 +        public Slice(List<ColumnDefinition> columnDefs, Bound bound, boolean inclusive, Term term)
 +        {
 +            this(columnDefs, TermSlice.newInstance(bound, inclusive, term));
 +        }
 +
-         private Slice(List<ColumnDefinition> columnDefs, TermSlice slice)
++        Slice(List<ColumnDefinition> columnDefs, TermSlice slice)
 +        {
 +            super(columnDefs);
 +            this.slice = slice;
 +        }
 +
 +        @Override
 +        public boolean isSlice()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
 +        {
 +            List<ByteBuffer> vals = componentBounds(bound, options);
 +
 +            for (int i = 0, m = vals.size(); i < m; i++)
 +            {
 +                ByteBuffer v = checkNotNull(vals.get(i), "Invalid null value in condition for column %s", columnDefs.get(i).name);
 +                builder.addElementToAll(v);
 +            }
 +            return builder;
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return slice.isSupportedBy(index);
 +        }
 +
 +        @Override
 +        public boolean hasBound(Bound b)
 +        {
 +            return slice.hasBound(b);
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return slice.getFunctions();
 +        }
 +
 +        @Override
 +        public boolean isInclusive(Bound b)
 +        {
 +            return slice.isInclusive(b);
 +        }
 +
 +        @Override
 +        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            checkTrue(otherRestriction.isSlice(),
 +                      "Column \"%s\" cannot be restricted by both an equality and an inequality relation",
 +                      getColumnsInCommons(otherRestriction));
 +
-             Slice otherSlice = (Slice) otherRestriction;
- 
 +            if (!getFirstColumn().equals(otherRestriction.getFirstColumn()))
 +            {
 +                ColumnDefinition column = getFirstColumn().position() > otherRestriction.getFirstColumn().position()
 +                        ? getFirstColumn() : otherRestriction.getFirstColumn();
 +
-                 throw invalidRequest("Column \"%s\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
++                throw invalidRequest("Column \"%s\" cannot be restricted by two inequalities not starting with the same column",
 +                                     column.name);
 +            }
 +
-             checkFalse(hasBound(Bound.START) && otherSlice.hasBound(Bound.START),
++            checkFalse(hasBound(Bound.START) && otherRestriction.hasBound(Bound.START),
 +                       "More than one restriction was found for the start bound on %s",
 +                       getColumnsInCommons(otherRestriction));
-             checkFalse(hasBound(Bound.END) && otherSlice.hasBound(Bound.END),
++            checkFalse(hasBound(Bound.END) && otherRestriction.hasBound(Bound.END),
 +                       "More than one restriction was found for the end bound on %s",
 +                       getColumnsInCommons(otherRestriction));
 +
++            Slice otherSlice = (Slice) otherRestriction;
 +            List<ColumnDefinition> newColumnDefs = columnDefs.size() >= otherSlice.columnDefs.size() ?  columnDefs : otherSlice.columnDefs;
++
 +            return new Slice(newColumnDefs, slice.merge(otherSlice.slice));
 +        }
 +
 +        @Override
 +        public final void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                               SecondaryIndexManager indexManager,
 +                                               QueryOptions options) throws InvalidRequestException
 +        {
 +            throw invalidRequest("Slice restrictions are not supported on indexed columns");
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return "SLICE" + slice;
 +        }
 +
 +        /**
 +         * Similar to bounds(), but returns one ByteBuffer per-component in the bound instead of a single
 +         * ByteBuffer to represent the entire bound.
 +         * @param b the bound type
 +         * @param options the query options
 +         * @return one ByteBuffer per-component in the bound
 +         * @throws InvalidRequestException if the components cannot be retrieved
 +         */
 +        private List<ByteBuffer> componentBounds(Bound b, QueryOptions options) throws InvalidRequestException
 +        {
-             Tuples.Value value = (Tuples.Value) slice.bound(b).bind(options);
-             return value.getElements();
++            Terminal terminal = slice.bound(b).bind(options);
++
++            if (terminal instanceof Tuples.Value)
++            {
++                return ((Tuples.Value) terminal).getElements();
++            }
++
++            return Collections.singletonList(terminal.get(options.getProtocolVersion()));
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d0cf7db/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
index d32f585,0000000..afa0419
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
@@@ -1,565 -1,0 +1,621 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.cql3.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +
 +import com.google.common.collect.Iterables;
 +
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.Term.Terminal;
 +import org.apache.cassandra.cql3.functions.Function;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.IndexExpression;
 +import org.apache.cassandra.db.composites.CompositesBuilder;
 +import org.apache.cassandra.db.index.SecondaryIndex;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.db.marshal.CompositeType;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
- 
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkBindValueSet;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 +
 +public abstract class SingleColumnRestriction extends AbstractRestriction
 +{
 +    /**
 +     * The definition of the column to which apply the restriction.
 +     */
 +    protected final ColumnDefinition columnDef;
 +
 +    public SingleColumnRestriction(ColumnDefinition columnDef)
 +    {
 +        this.columnDef = columnDef;
 +    }
 +
 +    @Override
 +    public Collection<ColumnDefinition> getColumnDefs()
 +    {
 +        return Collections.singletonList(columnDef);
 +    }
 +
 +    @Override
 +    public ColumnDefinition getFirstColumn()
 +    {
 +        return columnDef;
 +    }
 +
 +    @Override
 +    public ColumnDefinition getLastColumn()
 +    {
 +        return columnDef;
 +    }
 +
 +    @Override
 +    public boolean hasSupportingIndex(SecondaryIndexManager indexManager)
 +    {
 +        SecondaryIndex index = indexManager.getIndexForColumn(columnDef.name.bytes);
 +        return index != null && isSupportedBy(index);
 +    }
 +
 +    @Override
 +    public final Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
 +    {
-             checkFalse(otherRestriction.isMultiColumn(),
-                        "Mixing single column relations and multi column relations on clustering columns is not allowed");
++        // We want to allow query like: b > ? AND (b,c) < (?, ?)
++        if (otherRestriction.isMultiColumn() && canBeConvertedToMultiColumnRestriction())
++        {
++            return toMultiColumnRestriction().mergeWith(otherRestriction);
++        }
 +
-             return doMergeWith(otherRestriction);
++        return doMergeWith(otherRestriction);
 +    }
 +
 +    protected abstract Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException;
 +
 +    /**
++     * Converts this <code>SingleColumnRestriction</code> into a {@link MultiColumnRestriction}
++     *
++     * @return the <code>MultiColumnRestriction</code> corresponding to this
++     */
++    abstract MultiColumnRestriction toMultiColumnRestriction();
++
++    /**
++     * Checks if this <code>Restriction</code> can be converted into a {@link MultiColumnRestriction}
++     *
++     * @return <code>true</code> if this <code>Restriction</code> can be converted into a
++     * {@link MultiColumnRestriction}, <code>false</code> otherwise.
++     */
++    boolean canBeConvertedToMultiColumnRestriction()
++    {
++        return true;
++    }
++
++    /**
 +     * Check if this type of restriction is supported by the specified index.
 +     *
 +     * @param index the Secondary index
 +     * @return <code>true</code> this type of restriction is supported by the specified index,
 +     * <code>false</code> otherwise.
 +     */
 +    protected abstract boolean isSupportedBy(SecondaryIndex index);
 +
 +    public static final class EQ extends SingleColumnRestriction
 +    {
 +        private final Term value;
 +
 +        public EQ(ColumnDefinition columnDef, Term value)
 +        {
 +            super(columnDef);
 +            this.value = value;
 +        }
 +
 +        @Override
-         public Iterable getFunctions()
++        public Iterable<Function> getFunctions()
 +        {
 +            return value.getFunctions();
 +        }
 +
 +        @Override
 +        public boolean isEQ()
 +        {
 +            return true;
 +        }
 +
 +        @Override
++        MultiColumnRestriction toMultiColumnRestriction()
++        {
++            return new MultiColumnRestriction.EQ(Collections.singletonList(columnDef), value);
++        }
++
++        @Override
 +        public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                         SecondaryIndexManager indexManager,
 +                                         QueryOptions options) throws InvalidRequestException
 +        {
 +            ByteBuffer buffer = validateIndexedValue(columnDef, value.bindAndGet(options));
 +            expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, buffer));
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            builder.addElementToAll(value.bindAndGet(options));
 +            checkFalse(builder.containsNull(), "Invalid null value in condition for column %s", columnDef.name);
 +            checkFalse(builder.containsUnset(), "Invalid unset value for column %s", columnDef.name);
 +            return builder;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("EQ(%s)", value);
 +        }
 +
 +        @Override
 +        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal", columnDef.name);
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return index.supportsOperator(Operator.EQ);
 +        }
 +    }
 +
 +    public static abstract class IN extends SingleColumnRestriction
 +    {
 +        public IN(ColumnDefinition columnDef)
 +        {
 +            super(columnDef);
 +        }
 +
 +        @Override
 +        public final boolean isIN()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public final Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            throw invalidRequest("%s cannot be restricted by more than one relation if it includes a IN", columnDef.name);
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            builder.addEachElementToAll(getValues(options));
 +            checkFalse(builder.containsNull(), "Invalid null value in condition for column %s", columnDef.name);
 +            checkFalse(builder.containsUnset(), "Invalid unset value for column %s", columnDef.name);
 +            return builder;
 +        }
 +
 +        @Override
 +        public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                         SecondaryIndexManager indexManager,
 +                                         QueryOptions options) throws InvalidRequestException
 +        {
 +            List<ByteBuffer> values = getValues(options);
 +            checkTrue(values.size() == 1, "IN restrictions are not supported on indexed columns");
 +
 +            ByteBuffer value = validateIndexedValue(columnDef, values.get(0));
 +            expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, value));
 +        }
 +
 +        @Override
 +        protected final boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return index.supportsOperator(Operator.IN);
 +        }
 +
 +        protected abstract List<ByteBuffer> getValues(QueryOptions options) throws InvalidRequestException;
 +    }
 +
 +    public static class InWithValues extends IN
 +    {
 +        protected final List<Term> values;
 +
 +        public InWithValues(ColumnDefinition columnDef, List<Term> values)
 +        {
 +            super(columnDef);
 +            this.values = values;
 +        }
 +
 +        @Override
++        MultiColumnRestriction toMultiColumnRestriction()
++        {
++            return new MultiColumnRestriction.InWithValues(Collections.singletonList(columnDef), values);
++        }
++
++        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return Terms.getFunctions(values);
 +        }
 +
 +        @Override
 +        protected List<ByteBuffer> getValues(QueryOptions options) throws InvalidRequestException
 +        {
 +            List<ByteBuffer> buffers = new ArrayList<>(values.size());
 +            for (Term value : values)
 +                buffers.add(value.bindAndGet(options));
 +            return buffers;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("IN(%s)", values);
 +        }
 +    }
 +
 +    public static class InWithMarker extends IN
 +    {
 +        protected final AbstractMarker marker;
 +
 +        public InWithMarker(ColumnDefinition columnDef, AbstractMarker marker)
 +        {
 +            super(columnDef);
 +            this.marker = marker;
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return Collections.emptySet();
 +        }
 +
 +        @Override
++        MultiColumnRestriction toMultiColumnRestriction()
++        {
++            return new MultiColumnRestriction.InWithMarker(Collections.singletonList(columnDef), marker);
++        }
++
++        @Override
 +        protected List<ByteBuffer> getValues(QueryOptions options) throws InvalidRequestException
 +        {
 +            Terminal term = marker.bind(options);
 +            checkNotNull(term, "Invalid null value for column %s", columnDef.name);
 +            checkFalse(term == Constants.UNSET_VALUE, "Invalid unset value for column %s", columnDef.name);
 +            Term.MultiItemTerminal lval = (Term.MultiItemTerminal) term;
 +            return lval.getElements();
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return "IN ?";
 +        }
 +    }
 +
-     public static class Slice extends SingleColumnRestriction
++    public static final class Slice extends SingleColumnRestriction
 +    {
 +        private final TermSlice slice;
 +
 +        public Slice(ColumnDefinition columnDef, Bound bound, boolean inclusive, Term term)
 +        {
 +            super(columnDef);
 +            slice = TermSlice.newInstance(bound, inclusive, term);
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return slice.getFunctions();
 +        }
 +
 +        @Override
++        MultiColumnRestriction toMultiColumnRestriction()
++        {
++            return new MultiColumnRestriction.Slice(Collections.singletonList(columnDef), slice);
++        }
++
++        @Override
 +        public boolean isSlice()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
 +        public boolean hasBound(Bound b)
 +        {
 +            return slice.hasBound(b);
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
 +        {
 +            ByteBuffer value = slice.bound(bound).bindAndGet(options);
 +            checkBindValueSet(value, "Invalid unset value for column %s", columnDef.name);
 +            return builder.addElementToAll(value);
 +
 +        }
 +
 +        @Override
 +        public boolean isInclusive(Bound b)
 +        {
 +            return slice.isInclusive(b);
 +        }
 +
 +        @Override
 +        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            checkTrue(otherRestriction.isSlice(),
 +                      "Column \"%s\" cannot be restricted by both an equality and an inequality relation",
 +                      columnDef.name);
 +
 +            SingleColumnRestriction.Slice otherSlice = (SingleColumnRestriction.Slice) otherRestriction;
 +
 +            checkFalse(hasBound(Bound.START) && otherSlice.hasBound(Bound.START),
 +                       "More than one restriction was found for the start bound on %s", columnDef.name);
 +
 +            checkFalse(hasBound(Bound.END) && otherSlice.hasBound(Bound.END),
 +                       "More than one restriction was found for the end bound on %s", columnDef.name);
 +
 +            return new Slice(columnDef,  slice.merge(otherSlice.slice));
 +        }
 +
 +        @Override
 +        public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                         SecondaryIndexManager indexManager,
 +                                         QueryOptions options) throws InvalidRequestException
 +        {
 +            for (Bound b : Bound.values())
 +            {
 +                if (hasBound(b))
 +                {
 +                    ByteBuffer value = validateIndexedValue(columnDef, slice.bound(b).bindAndGet(options));
 +                    Operator op = slice.getIndexOperator(b);
 +                    // If the underlying comparator for name is reversed, we need to reverse the IndexOperator: user operation
 +                    // always refer to the "forward" sorting even if the clustering order is reversed, but the 2ndary code does
 +                    // use the underlying comparator as is.
 +                    op = columnDef.isReversedType() ? op.reverse() : op;
 +                    expressions.add(new IndexExpression(columnDef.name.bytes, op, value));
 +                }
 +            }
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return slice.isSupportedBy(index);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("SLICE%s", slice);
 +        }
 +
 +        private Slice(ColumnDefinition columnDef, TermSlice slice)
 +        {
 +            super(columnDef);
 +            this.slice = slice;
 +        }
 +    }
 +
 +    // This holds CONTAINS, CONTAINS_KEY, and map[key] = value restrictions because we might want to have any combination of them.
 +    public static final class Contains extends SingleColumnRestriction
 +    {
 +        private List<Term> values = new ArrayList<>(); // for CONTAINS
 +        private List<Term> keys = new ArrayList<>(); // for CONTAINS_KEY
 +        private List<Term> entryKeys = new ArrayList<>(); // for map[key] = value
 +        private List<Term> entryValues = new ArrayList<>(); // for map[key] = value
 +
 +        public Contains(ColumnDefinition columnDef, Term t, boolean isKey)
 +        {
 +            super(columnDef);
 +            if (isKey)
 +                keys.add(t);
 +            else
 +                values.add(t);
 +        }
 +
 +        public Contains(ColumnDefinition columnDef, Term mapKey, Term mapValue)
 +        {
 +            super(columnDef);
 +            entryKeys.add(mapKey);
 +            entryValues.add(mapValue);
 +        }
 +
 +        @Override
++        MultiColumnRestriction toMultiColumnRestriction()
++        {
++            throw new UnsupportedOperationException();
++        }
++
++        @Override
++        boolean canBeConvertedToMultiColumnRestriction()
++        {
++            return false;
++        }
++
++        @Override
 +        public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
 +        public boolean isContains()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
 +        {
 +            checkTrue(otherRestriction.isContains(),
 +                      "Collection column %s can only be restricted by CONTAINS, CONTAINS KEY, or map-entry equality",
 +                      columnDef.name);
 +
 +            SingleColumnRestriction.Contains newContains = new Contains(columnDef);
 +
 +            copyKeysAndValues(this, newContains);
 +            copyKeysAndValues((Contains) otherRestriction, newContains);
 +
 +            return newContains;
 +        }
 +
 +        @Override
 +        public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                         SecondaryIndexManager indexManager,
 +                                         QueryOptions options)
 +                                         throws InvalidRequestException
 +        {
 +            addExpressionsFor(expressions, bindAndGet(values, options), Operator.CONTAINS);
 +            addExpressionsFor(expressions, bindAndGet(keys, options), Operator.CONTAINS_KEY);
 +            addExpressionsFor(expressions, entries(options), Operator.EQ);
 +        }
 +
 +        private void addExpressionsFor(List<IndexExpression> target, List<ByteBuffer> values,
 +                                       Operator op) throws InvalidRequestException
 +        {
 +            for (ByteBuffer value : values)
 +            {
 +                validateIndexedValue(columnDef, value);
 +                target.add(new IndexExpression(columnDef.name.bytes, op, value));
 +            }
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            boolean supported = false;
 +
 +            if (numberOfValues() > 0)
 +                supported |= index.supportsOperator(Operator.CONTAINS);
 +
 +            if (numberOfKeys() > 0)
 +                supported |= index.supportsOperator(Operator.CONTAINS_KEY);
 +
 +            if (numberOfEntries() > 0)
 +                supported |= index.supportsOperator(Operator.EQ);
 +
 +            return supported;
 +        }
 +
 +        public int numberOfValues()
 +        {
 +            return values.size();
 +        }
 +
 +        public int numberOfKeys()
 +        {
 +            return keys.size();
 +        }
 +
 +        public int numberOfEntries()
 +        {
 +            return entryKeys.size();
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return Iterables.concat(Terms.getFunctions(values),
 +                                    Terms.getFunctions(keys),
 +                                    Terms.getFunctions(entryKeys),
 +                                    Terms.getFunctions(entryValues));
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("CONTAINS(values=%s, keys=%s, entryKeys=%s, entryValues=%s)", values, keys, entryKeys, entryValues);
 +        }
 +
 +        @Override
 +        public boolean hasBound(Bound b)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
 +        public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
 +        public boolean isInclusive(Bound b)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        private List<ByteBuffer> entries(QueryOptions options) throws InvalidRequestException
 +        {
 +            List<ByteBuffer> entryBuffers = new ArrayList<>(entryKeys.size());
 +            List<ByteBuffer> keyBuffers = bindAndGet(entryKeys, options);
 +            List<ByteBuffer> valueBuffers = bindAndGet(entryValues, options);
 +            for (int i = 0; i < entryKeys.size(); i++)
 +            {
 +                if (valueBuffers.get(i) == null)
 +                    throw new InvalidRequestException("Unsupported null value for map-entry equality");
 +                entryBuffers.add(CompositeType.build(keyBuffers.get(i), valueBuffers.get(i)));
 +            }
 +            return entryBuffers;
 +        }
 +
 +        /**
 +         * Binds the query options to the specified terms and returns the resulting values.
 +         *
 +         * @param terms the terms
 +         * @param options the query options
 +         * @return the value resulting from binding the query options to the specified terms
 +         * @throws InvalidRequestException if a problem occurs while binding the query options
 +         */
 +        private static List<ByteBuffer> bindAndGet(List<Term> terms, QueryOptions options) throws InvalidRequestException
 +        {
 +            List<ByteBuffer> buffers = new ArrayList<>(terms.size());
 +            for (Term value : terms)
 +                buffers.add(value.bindAndGet(options));
 +            return buffers;
 +        }
 +
 +        /**
 +         * Copies the keys and value from the first <code>Contains</code> to the second one.
 +         *
 +         * @param from the <code>Contains</code> to copy from
 +         * @param to the <code>Contains</code> to copy to
 +         */
 +        private static void copyKeysAndValues(Contains from, Contains to)
 +        {
 +            to.values.addAll(from.values);
 +            to.keys.addAll(from.keys);
 +            to.entryKeys.addAll(from.entryKeys);
 +            to.entryValues.addAll(from.entryValues);
 +        }
 +
 +        private Contains(ColumnDefinition columnDef)
 +        {
 +            super(columnDef);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d0cf7db/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
index 39412b9,98dda26..84343a7
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
@@@ -21,8 -22,8 +21,6 @@@ import org.junit.Test
  
  import org.apache.cassandra.cql3.CQLTester;
  
--import static org.junit.Assert.assertEquals;
--
  public class SelectMultiColumnRelationTest extends CQLTester
  {
      @Test
@@@ -37,7 -38,9 +35,9 @@@
                                   "SELECT * FROM %s WHERE a = 0 AND (b) = (?) AND (b) > (?)", 0, 0);
              assertInvalidMessage("More than one restriction was found for the start bound on b",
                                   "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND (b) > (?)", 0, 1);
+             assertInvalidMessage("More than one restriction was found for the start bound on b",
+                                  "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND b > ?", 0, 1);
 -            assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
 +            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
                                   "SELECT * FROM %s WHERE (a, b) = (?, ?)", 0, 0);
          }
      }
@@@ -78,40 -81,43 +78,39 @@@
                                   "SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?))", 1, 2, null);
  
              // Wrong type for 'd'
 -            assertInvalidMessage("Expected 4 or 0 byte int (6)",
 -                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
 -
 -            assertInvalidMessage("Invalid tuple type literal for b of type int",
 -                                 "SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
 +            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
 +            assertInvalid("SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
  
              // Mix single and tuple inequalities
-             assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
-                                  "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND b < ?", 0, 1, 0, 1);
-             assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
 -            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: c < ?",
++             assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                   "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND c < ?", 0, 1, 0, 1);
-             assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
-                                  "SELECT * FROM %s WHERE a = 0 AND b > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
-             assertInvalidMessage("Mixing single column relations and multi column relations on clustering columns is not allowed",
 -            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c, d) < (?, ?, ?)",
++            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                   "SELECT * FROM %s WHERE a = 0 AND c > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
  
 -            assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
 +            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
                                   "SELECT * FROM %s WHERE (a, b, c, d) IN ((?, ?, ?, ?))", 0, 1, 2, 3);
 -            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
 +            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
                                   "SELECT * FROM %s WHERE (c, d) IN ((?, ?))", 0, 1);
 -            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
 +
 +            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
                                   "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) IN ((?, ?))", 0, 0, 0, 0);
  
 -            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
 +            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
                                   "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) > (?, ?)", 0, 0, 0, 0);
              assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
                                   "SELECT * FROM %s WHERE a = ? AND (c, d) > (?, ?) AND b > ?  ", 0, 0, 0, 0);
-             assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
 -            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (c) < (?)",
++
++            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                   "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (b) < (?) AND (c) < (?)", 0, 0, 0, 0, 0);
-             assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
 -            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c) > (?, ?)",
++            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                   "SELECT * FROM %s WHERE a = ? AND (c) < (?) AND (b, c) > (?, ?) AND (b) < (?)", 0, 0, 0, 0, 0);
 -            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c) > (?, ?)",
 +            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
                                   "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND (c) < (?) AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
 -            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c) > (?, ?)",
++            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+                                  "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND c < ? AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
  
-             assertInvalidMessage("Column \"c\" cannot be restricted by two tuple-notation inequalities not starting with the same column",
 -            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (c) < (?)",
++            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                   "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (c) < (?)", 0, 0, 0, 0);
 -
 -            assertInvalidMessage("PRIMARY KEY column \"d\" cannot be restricted (preceding column \"c\" is restricted by an IN tuple notation)",
 -                                 "SELECT * FROM %s WHERE a = ? AND (b, c) in ((?, ?), (?, ?)) AND d > ?", 0, 0, 0, 0, 0, 0);
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d0cf7db/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
index 57b8a86,9896d0a..f1bce45
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
@@@ -514,50 -137,8 +514,78 @@@ public class SelectSingleColumnRelation
  
          assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? AND d >= ? AND f = ? ALLOW FILTERING", 0, 1, 1, 5),
                     row(0, 0, 1, 1, 1, 5));
 +    }
 +
 +    @Test
 +    public void testFunctionCallWithUnset() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (k int PRIMARY KEY, s text, i int)");
 +
 +        assertInvalidMessage("Invalid unset value for argument in call to function token",
 +                             "SELECT * FROM %s WHERE token(k) >= token(?)", unset());
 +        assertInvalidMessage("Invalid unset value for argument in call to function blobasint",
 +                             "SELECT * FROM %s WHERE k = blobAsInt(?)", unset());
 +    }
 +
 +    @Test
 +    public void testLimitWithUnset() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
 +        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
 +        execute("INSERT INTO %s (k, i) VALUES (2, 1)");
 +        assertRows(execute("SELECT k FROM %s LIMIT ?", unset()), // treat as 'unlimited'
 +                row(1),
 +                row(2)
 +        );
 +    }
 +
 +    @Test
 +    public void testWithUnsetValues() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (k int, i int, j int, s text, PRIMARY KEY(k,i,j))");
 +        createIndex("CREATE INDEX s_index ON %s (s)");
 +        // partition key
 +        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k = ?", unset());
 +        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN ?", unset());
 +        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN(?)", unset());
 +        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN(?,?)", 1, unset());
 +        // clustering column
 +        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i = ?", unset());
 +        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN ?", unset());
 +        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN(?)", unset());
 +        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN(?,?)", 1, unset());
 +        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE i = ? ALLOW FILTERING", unset());
 +        // indexed column
 +        assertInvalidMessage("Unsupported unset value for indexed column s", "SELECT * from %s WHERE s = ?", unset());
 +        // range
 +        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i > ?", unset());
 +    }
+ 
 -        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
 -                             "SELECT * FROM %s WHERE a = ? AND d >= ? AND f = ?", 0, 1, 5);
++    @Test
++    public void testInvalidSliceRestrictionOnPartitionKey() throws Throwable
++    {
++        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c text)");
++        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
++                             "SELECT * FROM %s WHERE a >= 1 and a < 4");
++        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
++                             "SELECT * FROM %s WHERE (a) >= (1) and (a) < (4)");
++    }
++
++    @Test
++    public void testInvalidMulticolumnSliceRestrictionOnPartitionKey() throws Throwable
++    {
++        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY ((a, b)))");
++        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
++                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and (a, b) < (4, 1)");
++        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
++                             "SELECT * FROM %s WHERE a >= 1 and (a, b) < (4, 1)");
++        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
++                             "SELECT * FROM %s WHERE b >= 1 and (a, b) < (4, 1)");
++        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
++                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and (b) < (4)");
++        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: b",
++                             "SELECT * FROM %s WHERE (b) < (4) and (a, b) >= (1, 1)");
++        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
++                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and a = 1");
+     }
  }