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 2016/07/04 12:32:28 UTC

[2/3] 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/e06dae81
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e06dae81
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e06dae81

Branch: refs/heads/cassandra-3.0
Commit: e06dae81fb08870ef6a6596b1557b88fc7762302
Parents: 65f8bb6 c857919
Author: Benjamin Lerer <b....@gmail.com>
Authored: Mon Jul 4 14:20:34 2016 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Mon Jul 4 14:22:55 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../restrictions/MultiColumnRestriction.java    |  2 +-
 .../restrictions/PrimaryKeyRestrictionSet.java  | 81 ++++----------------
 .../restrictions/StatementRestrictions.java     | 42 ++++++++--
 .../SelectMultiColumnRelationTest.java          | 32 +++++++-
 .../cql3/validation/operations/SelectTest.java  | 61 +++++++++++++++
 6 files changed, 142 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e06dae81/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 13a1c4f,0967ce4..451575c
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,41 -1,9 +1,42 @@@
 -2.1.16
 +2.2.8
 + * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
 +Merged from 2.1:
+  * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
 - * Reduce contention getting instances of CompositeType (CASSANDRA-10433)
 -
 -2.1.15
   * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
 +
 +
 +2.2.7
 + * Allow nodetool info to run with readonly JMX access (CASSANDRA-11755)
 + * Validate bloom_filter_fp_chance against lowest supported
 +   value when the table is created (CASSANDRA-11920)
 + * RandomAccessReader: call isEOF() only when rebuffering, not for every read operation (CASSANDRA-12013)
 + * Don't send erroneous NEW_NODE notifications on restart (CASSANDRA-11038)
 + * StorageService shutdown hook should use a volatile variable (CASSANDRA-11984)
 + * Persist local metadata earlier in startup sequence (CASSANDRA-11742)
 + * Run CommitLog tests with different compression settings (CASSANDRA-9039)
 + * cqlsh: fix tab completion for case-sensitive identifiers (CASSANDRA-11664)
 + * Avoid showing estimated key as -1 in tablestats (CASSANDRA-11587)
 + * Fix possible race condition in CommitLog.recover (CASSANDRA-11743)
 + * Enable client encryption in sstableloader with cli options (CASSANDRA-11708)
 + * Possible memory leak in NIODataInputStream (CASSANDRA-11867)
 + * Fix commit log replay after out-of-order flush completion (CASSANDRA-9669)
 + * Add seconds to cqlsh tracing session duration (CASSANDRA-11753)
 + * Prohibit Reverse Counter type as part of the PK (CASSANDRA-9395)
 + * cqlsh: correctly handle non-ascii chars in error messages (CASSANDRA-11626)
 + * Exit JVM if JMX server fails to startup (CASSANDRA-11540)
 + * Produce a heap dump when exiting on OOM (CASSANDRA-9861)
 + * Avoid read repairing purgeable tombstones on range slices (CASSANDRA-11427)
 + * Restore ability to filter on clustering columns when using a 2i (CASSANDRA-11510)
 + * JSON datetime formatting needs timezone (CASSANDRA-11137)
 + * Fix is_dense recalculation for Thrift-updated tables (CASSANDRA-11502)
 + * Remove unnescessary file existence check during anticompaction (CASSANDRA-11660)
 + * Add missing files to debian packages (CASSANDRA-11642)
 + * Avoid calling Iterables::concat in loops during ModificationStatement::getFunctions (CASSANDRA-11621)
 + * cqlsh: COPY FROM should use regular inserts for single statement batches and
 +   report errors correctly if workers processes crash on initialization (CASSANDRA-11474)
 + * Always close cluster with connection in CqlRecordWriter (CASSANDRA-11553)
 + * Fix slice queries on ordered COMPACT tables (CASSANDRA-10988)
 +Merged from 2.1:
   * Avoid stalling paxos when the paxos state expires (CASSANDRA-12043)
   * Remove finished incoming streaming connections from MessagingService (CASSANDRA-11854)
   * Don't try to get sstables for non-repairing column families (CASSANDRA-12077)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e06dae81/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
index 96e6f2b,0000000..51e2ce4
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,515 -1,0 +1,515 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.cql3.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.*;
 +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.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 List<ColumnDefinition> getColumnDefs()
 +    {
 +        return columnDefs;
 +    }
 +
 +    @Override
 +    public final Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
 +    {
 +        // 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));
 +        }
 +
 +        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)
 +        {
 +            boolean reversed = getFirstColumn().isReversedType();
 +
 +            EnumMap<Bound, List<ByteBuffer>> componentBounds = new EnumMap<Bound, List<ByteBuffer>>(Bound.class);
 +            componentBounds.put(Bound.START, componentBounds(Bound.START, options));
 +            componentBounds.put(Bound.END, componentBounds(Bound.END, options));
 +
 +            List<List<ByteBuffer>> toAdd = new ArrayList<>();
 +            List<ByteBuffer> values = new ArrayList<>();
 +
 +            for (int i = 0, m = columnDefs.size(); i < m; i++)
 +            {
 +                ColumnDefinition column = columnDefs.get(i);
 +                Bound b = reverseBoundIfNeeded(column, bound);
 +
 +                // For mixed order columns, we need to create additional slices when 2 columns are in reverse order
 +                if (reversed != column.isReversedType())
 +                {
 +                    reversed = column.isReversedType();
 +                    // As we are switching direction we need to add the current composite
 +                    toAdd.add(values);
 +
 +                    // The new bound side has no value for this component.  just stop
 +                    if (!hasComponent(b, i, componentBounds))
 +                        continue;
 +
 +                    // The other side has still some components. We need to end the slice that we have just open.
 +                    if (hasComponent(b.reverse(), i, componentBounds))
 +                        toAdd.add(values);
 +
 +                    // We need to rebuild where we are in this bound side
 +                    values = new ArrayList<ByteBuffer>();
 +
 +                    List<ByteBuffer> vals = componentBounds.get(b);
 +
 +                    int n = Math.min(i, vals.size());
 +                    for (int j = 0; j < n; j++)
 +                    {
 +                        ByteBuffer v = checkNotNull(vals.get(j),
 +                                                    "Invalid null value in condition for column %s",
 +                                                    columnDefs.get(j).name);
 +                        values.add(v);
 +                    }
 +                }
 +
 +                if (!hasComponent(b, i, componentBounds))
 +                    continue;
 +
 +                ByteBuffer v = checkNotNull(componentBounds.get(b).get(i), "Invalid null value in condition for column %s", columnDefs.get(i).name);
 +                values.add(v);
 +            }
 +            toAdd.add(values);
 +
 +            if (bound.isEnd())
 +                Collections.reverse(toAdd);
 +
 +            return builder.addAllElementsToAll(toAdd);
 +        }
 +
 +        @Override
 +        protected boolean isSupportedBy(SecondaryIndex index)
 +        {
 +            return slice.isSupportedBy(index);
 +        }
 +
 +        @Override
 +        public boolean hasBound(Bound bound)
 +        {
 +            return slice.hasBound(bound);
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return slice.getFunctions();
 +        }
 +
 +        @Override
 +        public boolean isInclusive(Bound bound)
 +        {
 +            return slice.isInclusive(bound);
 +        }
 +
 +        @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));
 +
 +            if (!getFirstColumn().equals(otherRestriction.getFirstColumn()))
 +            {
 +                ColumnDefinition column = getFirstColumn().position() > otherRestriction.getFirstColumn().position()
 +                        ? getFirstColumn() : otherRestriction.getFirstColumn();
 +
 +                throw invalidRequest("Column \"%s\" cannot be restricted by two inequalities not starting with the same column",
 +                                     column.name);
 +            }
 +
 +            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) && 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");
++            throw invalidRequest("Multi-column slice restrictions cannot be used for filtering.");
 +        }
 +
 +        @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
 +        {
 +            if (!slice.hasBound(b))
 +                return Collections.emptyList();
 +
 +            Terminal terminal = slice.bound(b).bind(options);
 +
 +            if (terminal instanceof Tuples.Value)
 +            {
 +                return ((Tuples.Value) terminal).getElements();
 +            }
 +
 +            return Collections.singletonList(terminal.get(options.getProtocolVersion()));
 +        }
 +
 +        private boolean hasComponent(Bound b, int index, EnumMap<Bound, List<ByteBuffer>> componentBounds)
 +        {
 +            return componentBounds.get(b).size() > index;
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e06dae81/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
index 0c10f13,0000000..e1cbc29
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
@@@ -1,460 -1,0 +1,405 @@@
 +/*
 + * 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.Lists;
 +
- import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.QueryOptions;
 +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.Keyspace;
 +import org.apache.cassandra.db.composites.*;
 +import org.apache.cassandra.db.composites.Composite.EOC;
 +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.invalidRequest;
- 
 +/**
 + * A set of single column restrictions on a primary key part (partition key or clustering key).
 + */
- final class PrimaryKeyRestrictionSet extends AbstractPrimaryKeyRestrictions
++final class PrimaryKeyRestrictionSet extends AbstractPrimaryKeyRestrictions implements Iterable<Restriction>
 +{
 +    /**
 +     * The restrictions.
 +     */
 +    private final RestrictionSet restrictions;
 +
 +    /**
 +     * <code>true</code> if the restrictions are corresponding to an EQ, <code>false</code> otherwise.
 +     */
 +    private boolean eq;
 +
 +    /**
 +     * <code>true</code> if the restrictions are corresponding to an IN, <code>false</code> otherwise.
 +     */
 +    private boolean in;
 +
 +    /**
 +     * <code>true</code> if the restrictions are corresponding to a Slice, <code>false</code> otherwise.
 +     */
 +    private boolean slice;
 +
 +    /**
 +     * <code>true</code> if the restrictions are corresponding to a Contains, <code>false</code> otherwise.
 +     */
 +    private boolean contains;
 +
- 
-     /**
-      * If restrictions apply to clustering columns, we need to check whether they can be satisfied by an index lookup
-      * as this affects which other restrictions can legally be specified (if an index is present, we are more lenient
-      * about what additional filtering can be performed on the results of a lookup - see CASSANDRA-11510).
-      *
-      * We don't hold a reference to the SecondaryIndexManager itself as this is not strictly a singleton (although
-      * we often treat is as one), the field would also require annotation with @Unmetered to avoid blowing up the
-      * object size (used when calculating the size of prepared statements for caching). Instead, we refer to the
-      * CFMetaData and retrieve the index manager when necessary.
-      *
-      * There are a couple of scenarios where the CFM can be null (and we make sure and test for null when we use it):
-      *  * where an empty set of restrictions are created for use in processing query results - see
-      *    SelectStatement.forSelection
-      *  * where the restrictions apply to partition keys and not clustering columns e.g.
-      *    StatementRestrictions.partitionKeyRestrictions
-      *  * in unit tests (in particular PrimaryKeyRestrictionSetTest which is primarily concerned with the correct
-      *    generation of bounds when secondary indexes are not used).
-      */
-     private final CFMetaData cfm;
- 
 +    public PrimaryKeyRestrictionSet(CType ctype)
 +    {
-         this(ctype, null);
-     }
- 
-     public PrimaryKeyRestrictionSet(CType ctype, CFMetaData cfm)
-     {
 +        super(ctype);
-         this.cfm = cfm;
 +        this.restrictions = new RestrictionSet();
 +        this.eq = true;
 +    }
 +
 +    private PrimaryKeyRestrictionSet(PrimaryKeyRestrictionSet primaryKeyRestrictions,
 +                                     Restriction restriction) throws InvalidRequestException
 +    {
 +        super(primaryKeyRestrictions.ctype);
 +        this.restrictions = primaryKeyRestrictions.restrictions.addRestriction(restriction);
-         this.cfm = primaryKeyRestrictions.cfm;
- 
-         if (!primaryKeyRestrictions.isEmpty() && !hasSupportingIndex(restriction))
-         {
-             ColumnDefinition lastRestrictionStart = primaryKeyRestrictions.restrictions.lastRestriction().getFirstColumn();
-             ColumnDefinition newRestrictionStart = restriction.getFirstColumn();
- 
-             checkFalse(primaryKeyRestrictions.isSlice() && newRestrictionStart.position() > lastRestrictionStart.position(),
-                        "Clustering column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)",
-                        newRestrictionStart.name,
-                        lastRestrictionStart.name);
- 
-             if (newRestrictionStart.position() < lastRestrictionStart.position() && restriction.isSlice())
-                 throw invalidRequest("PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)",
-                                      restrictions.nextColumn(newRestrictionStart).name,
-                                      newRestrictionStart.name);
-         }
 +
 +        if (restriction.isSlice() || primaryKeyRestrictions.isSlice())
 +            this.slice = true;
 +        else if (restriction.isContains() || primaryKeyRestrictions.isContains())
 +            this.contains = true;
 +        else if (restriction.isIN() || primaryKeyRestrictions.isIN())
 +            this.in = true;
 +        else
 +            this.eq = true;
 +    }
 +
-     private boolean hasSupportingIndex(Restriction restriction)
-     {
-         return cfm != null
-                && restriction.hasSupportingIndex(Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfId).indexManager);
-     }
- 
 +    @Override
 +    public boolean isSlice()
 +    {
 +        return slice;
 +    }
 +
 +    @Override
 +    public boolean isEQ()
 +    {
 +        return eq;
 +    }
 +
 +    @Override
 +    public boolean isIN()
 +    {
 +        return in;
 +    }
 +
 +    @Override
 +    public boolean isOnToken()
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public boolean isContains()
 +    {
 +        return contains;
 +    }
 +
 +    @Override
 +    public boolean isMultiColumn()
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public Iterable<Function> getFunctions()
 +    {
 +        return restrictions.getFunctions();
 +    }
 +
 +    @Override
 +    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException
 +    {
 +        if (restriction.isOnToken())
 +        {
 +            if (isEmpty())
 +                return (PrimaryKeyRestrictions) restriction;
 +
 +            return new TokenFilter(this, (TokenRestriction) restriction);
 +        }
 +
 +        return new PrimaryKeyRestrictionSet(this, restriction);
 +    }
 +
 +    @Override
 +    public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
 +    {
 +        return filterAndSort(appendTo(new CompositesBuilder(ctype), options).build());
 +    }
 +
 +    @Override
 +    public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
 +    {
 +        for (Restriction r : restrictions)
 +        {
 +            r.appendTo(builder, options);
 +            if (builder.hasMissingElements())
 +                break;
 +        }
 +        return builder;
 +    }
 +
 +    @Override
 +    public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
 +    @Override
 +    public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
 +    {
 +        CompositesBuilder builder = new CompositesBuilder(ctype);
 +        // The end-of-component of composite doesn't depend on whether the
 +        // component type is reversed or not (i.e. the ReversedType is applied
 +        // to the component comparator but not to the end-of-component itself),
 +        // it only depends on whether the slice is reversed
 +        int keyPosition = 0;
 +        for (Restriction r : restrictions)
 +        {
 +            ColumnDefinition def = r.getFirstColumn();
 +
 +            if (keyPosition != def.position() || r.isContains())
 +                break;
 +
 +            if (r.isSlice())
 +            {
 +                r.appendBoundTo(builder, bound, options);
 +
 +                // Since CASSANDRA-7281, the composites might not end with the same components and it is possible
 +                // that one of the composites is an empty one. Unfortunatly, AbstractCType will always sort
 +                // Composites.EMPTY before all the other components due to its EOC, even if it is not the desired
 +                // behaviour in some cases. To avoid that problem the code will use normal composites for the empty
 +                // ones until the composites are properly sorted. They will then be replaced by Composites.EMPTY as
 +                // it is what is expected by the intra-node serialization.
 +                // It is clearly a hack but it does not make a lot of sense to refactor 2.2 for that as the problem is
 +                // already solved in 3.0.
 +                List<Composite> composites = filterAndSort(setEocs(r, bound, builder.build()));
 +                return Lists.transform(composites, new com.google.common.base.Function<Composite, Composite>()
 +                {
 +                    @Override
 +                    public Composite apply(Composite composite)
 +                    {
 +                        return composite.isEmpty() ? Composites.EMPTY: composite;
 +                    }
 +                });
 +            }
 +
 +            r.appendBoundTo(builder, bound, options);
 +
 +            if (builder.hasMissingElements())
 +                return Collections.emptyList();
 +
 +            keyPosition = r.getLastColumn().position() + 1;
 +        }
 +        // Means no relation at all or everything was an equal
 +        // Note: if the builder is "full", there is no need to use the end-of-component bit. For columns selection,
 +        // it would be harmless to do it. However, we use this method got the partition key too. And when a query
 +        // with 2ndary index is done, and with the the partition provided with an EQ, we'll end up here, and in that
 +        // case using the eoc would be bad, since for the random partitioner we have no guarantee that
 +        // prefix.end() will sort after prefix (see #5240).
 +        EOC eoc = !builder.hasRemaining() ? EOC.NONE : (bound.isEnd() ? EOC.END : EOC.START);
 +        return filterAndSort(builder.buildWithEOC(eoc));
 +    }
 +
 +    /**
 +     * Removes duplicates and sort the specified composites.
 +     *
 +     * @param composites the composites to filter and sort
 +     * @return the composites sorted and without duplicates
 +     */
 +    private List<Composite> filterAndSort(List<Composite> composites)
 +    {
 +        if (composites.size() <= 1)
 +            return composites;
 +
 +        TreeSet<Composite> set = new TreeSet<Composite>(ctype);
 +        set.addAll(composites);
 +
 +        return new ArrayList<>(set);
 +    }
 +
 +    /**
 +     * Sets EOCs for the composites returned by the specified slice restriction for the given bound.
 +     *
 +     * @param r the slice restriction
 +     * @param bound the bound
 +     * @param composites the composites
 +     * @return the composites with their EOCs properly set
 +     */
 +    private List<Composite> setEocs(Restriction r, Bound bound, List<Composite> composites)
 +    {
 +        List<Composite> list = new ArrayList<>(composites.size());
 +
 +        // The first column of the slice might not be the first clustering column (e.g. clustering_0 = ? AND (clustering_1, clustering_2) >= (?, ?)
 +        int offset = r.getFirstColumn().position();
 +
 +        for (int i = 0, m = composites.size(); i < m; i++)
 +        {
 +            Composite composite = composites.get(i);
 +
 +            // Handle the no bound case
 +            if (composite.size() == offset)
 +            {
 +                list.add(composite.withEOC(bound.isEnd() ? EOC.END : EOC.START));
 +                continue;
 +            }
 +
 +            // In the case of mixed order columns, we will have some extra slices where the columns change directions.
 +            // For example: if we have clustering_0 DESC and clustering_1 ASC a slice like (clustering_0, clustering_1) > (1, 2)
 +            // will produce 2 slices: [EMPTY, 1.START] and [1.2.END, 1.END]
 +            // So, the END bound will return 2 composite with the same values 1
 +            if (composite.size() <= r.getLastColumn().position() && i < m - 1 && composite.equals(composites.get(i + 1)))
 +            {
 +                list.add(composite.withEOC(EOC.START));
 +                list.add(composites.get(i++).withEOC(EOC.END));
 +                continue;
 +            }
 +
 +            // Handle the normal bounds
 +            ColumnDefinition column = r.getColumnDefs().get(composite.size() - 1 - offset);
 +            Bound b = reverseBoundIfNeeded(column, bound);
 +
 +            Composite.EOC eoc = eocFor(r, bound, b);
 +            list.add(composite.withEOC(eoc));
 +        }
 +
 +        return list;
 +    }
 +
 +    @Override
 +    public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
 +    {
 +        return Composites.toByteBuffers(valuesAsComposites(options));
 +    }
 +
 +    @Override
 +    public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
 +    {
 +        return Composites.toByteBuffers(boundsAsComposites(b, options));
 +    }
 +
 +    private static Composite.EOC eocFor(Restriction r, Bound eocBound, Bound inclusiveBound)
 +    {
 +        if (eocBound.isStart())
 +            return r.isInclusive(inclusiveBound) ? Composite.EOC.NONE : Composite.EOC.END;
 +
 +        return r.isInclusive(inclusiveBound) ? Composite.EOC.END : Composite.EOC.START;
 +    }
 +
 +    @Override
 +    public boolean hasBound(Bound b)
 +    {
 +        if (isEmpty())
 +            return false;
 +        return restrictions.lastRestriction().hasBound(b);
 +    }
 +
 +    @Override
 +    public boolean isInclusive(Bound b)
 +    {
 +        if (isEmpty())
 +            return false;
 +        return restrictions.lastRestriction().isInclusive(b);
 +    }
 +
 +    @Override
 +    public boolean hasSupportingIndex(SecondaryIndexManager indexManager)
 +    {
 +        return restrictions.hasSupportingIndex(indexManager);
 +    }
 +
 +    @Override
 +    public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                     SecondaryIndexManager indexManager,
 +                                     QueryOptions options) throws InvalidRequestException
 +    {
 +        Boolean clusteringColumns = null;
 +        int position = 0;
 +
 +        for (Restriction restriction : restrictions)
 +        {
 +            ColumnDefinition columnDef = restriction.getFirstColumn();
 +
 +            // PrimaryKeyRestrictionSet contains only one kind of column, either partition key or clustering columns.
 +            // Therefore we only need to check the column kind once. All the other columns will be of the same kind.
 +            if (clusteringColumns == null)
 +                clusteringColumns = columnDef.isClusteringColumn() ? Boolean.TRUE : Boolean.FALSE;
 +
 +            // We ignore all the clustering columns that can be handled by slices.
-             if (clusteringColumns && !restriction.isContains()&& position == columnDef.position())
++            if (!clusteringColumns || handleInFilter(restriction, position) || restriction.hasSupportingIndex(indexManager))
 +            {
-                 position = restriction.getLastColumn().position() + 1;
-                 if (!restriction.hasSupportingIndex(indexManager))
-                     continue;
++                restriction.addIndexExpressionTo(expressions, indexManager, options);
++                continue;
 +            }
-             restriction.addIndexExpressionTo(expressions, indexManager, options);
++
++            if (!restriction.isSlice())
++                position = restriction.getLastColumn().position() + 1;
 +        }
 +    }
 +
 +    @Override
 +    public List<ColumnDefinition> getColumnDefs()
 +    {
 +        return restrictions.getColumnDefs();
 +    }
 +
 +    @Override
 +    public ColumnDefinition getFirstColumn()
 +    {
 +        return restrictions.firstColumn();
 +    }
 +
 +    @Override
 +    public ColumnDefinition getLastColumn()
 +    {
 +        return restrictions.lastColumn();
 +    }
 +
 +    public final boolean needsFiltering()
 +    {
 +        // Backported from ClusteringColumnRestrictions from CASSANDRA-11310 for 3.6
 +        // As that suggests, this should only be called on clustering column
 +        // and not partition key restrictions.
 +        int position = 0;
-         Restriction slice = null;
 +        for (Restriction restriction : restrictions)
 +        {
 +            if (handleInFilter(restriction, position))
 +                return true;
 +
-             if (slice != null && !slice.getFirstColumn().equals(restriction.getFirstColumn()))
-                 return true;
- 
-             if (slice == null && restriction.isSlice())
-                 slice = restriction;
-             else
++            if (!restriction.isSlice())
 +                position = restriction.getLastColumn().position() + 1;
 +        }
 +
 +        return false;
 +    }
 +
 +    private boolean handleInFilter(Restriction restriction, int index)
 +    {
 +        return restriction.isContains() || index != restriction.getFirstColumn().position();
 +    }
++
++    public Iterator<Restriction> iterator()
++    {
++        return restrictions.iterator();
++    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e06dae81/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
index 6f03c40,0000000..1547210
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
@@@ -1,613 -1,0 +1,645 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.cql3.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.*;
 +
 +import com.google.common.base.Joiner;
 +import com.google.common.collect.Iterables;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
- import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.functions.Function;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.composites.Composite;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.dht.*;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.service.StorageService;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +
 +import static org.apache.cassandra.config.ColumnDefinition.toIdentifiers;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
- import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
 +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 +
 +/**
 + * The restrictions corresponding to the relations specified on the where-clause of CQL query.
 + */
 +public final class StatementRestrictions
 +{
 +    public static final String REQUIRES_ALLOW_FILTERING_MESSAGE =
 +            "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";
 +
 +    /**
 +     * The Column Family meta data
 +     */
 +    public final CFMetaData cfm;
 +
 +    /**
 +     * Restrictions on partitioning columns
 +     */
 +    private PrimaryKeyRestrictions partitionKeyRestrictions;
 +
 +    /**
 +     * Restrictions on clustering columns
 +     */
 +    private PrimaryKeyRestrictions clusteringColumnsRestrictions;
 +
 +    /**
 +     * Restriction on non-primary key columns (i.e. secondary index restrictions)
 +     */
 +    private RestrictionSet nonPrimaryKeyRestrictions;
 +
 +    /**
 +     * The restrictions used to build the index expressions
 +     */
 +    private final List<Restrictions> indexRestrictions = new ArrayList<>();
 +
 +    /**
 +     * <code>true</code> if the secondary index need to be queried, <code>false</code> otherwise
 +     */
 +    private boolean usesSecondaryIndexing;
 +
 +    /**
 +     * Specify if the query will return a range of partition keys.
 +     */
 +    private boolean isKeyRange;
 +
 +    /**
 +     * Creates a new empty <code>StatementRestrictions</code>.
 +     *
 +     * @param cfm the column family meta data
 +     * @return a new empty <code>StatementRestrictions</code>.
 +     */
 +    public static StatementRestrictions empty(CFMetaData cfm)
 +    {
 +        return new StatementRestrictions(cfm);
 +    }
 +
 +    private StatementRestrictions(CFMetaData cfm)
 +    {
 +        this.cfm = cfm;
 +        this.partitionKeyRestrictions = new PrimaryKeyRestrictionSet(cfm.getKeyValidatorAsCType());
 +        this.clusteringColumnsRestrictions = new PrimaryKeyRestrictionSet(cfm.comparator);
 +        this.nonPrimaryKeyRestrictions = new RestrictionSet();
 +    }
 +
 +    public StatementRestrictions(CFMetaData cfm,
 +                                 List<Relation> whereClause,
 +                                 VariableSpecifications boundNames,
 +                                 boolean selectsOnlyStaticColumns,
 +                                 boolean selectACollection,
-                                  boolean useFiltering)
++                                 boolean allowFiltering)
 +    {
 +        this.cfm = cfm;
 +        this.partitionKeyRestrictions = new PrimaryKeyRestrictionSet(cfm.getKeyValidatorAsCType());
-         this.clusteringColumnsRestrictions = new PrimaryKeyRestrictionSet(cfm.comparator, cfm);
++        this.clusteringColumnsRestrictions = new PrimaryKeyRestrictionSet(cfm.comparator);
 +        this.nonPrimaryKeyRestrictions = new RestrictionSet();
 +
 +        /*
 +         * WHERE clause. For a given entity, rules are: - EQ relation conflicts with anything else (including a 2nd EQ)
 +         * - Can't have more than one LT(E) relation (resp. GT(E) relation) - IN relation are restricted to row keys
 +         * (for now) and conflicts with anything else (we could allow two IN for the same entity but that doesn't seem
 +         * very useful) - The value_alias cannot be restricted in any way (we don't support wide rows with indexed value
 +         * in CQL so far)
 +         */
 +        for (Relation relation : whereClause)
 +            addRestriction(relation.toRestriction(cfm, boundNames));
 +
 +        SecondaryIndexManager secondaryIndexManager = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName).indexManager;
 +        boolean hasQueriableClusteringColumnIndex = clusteringColumnsRestrictions.hasSupportingIndex(secondaryIndexManager);
 +        boolean hasQueriableIndex = hasQueriableClusteringColumnIndex
 +                || partitionKeyRestrictions.hasSupportingIndex(secondaryIndexManager)
 +                || nonPrimaryKeyRestrictions.hasSupportingIndex(secondaryIndexManager);
 +
 +        // At this point, the select statement if fully constructed, but we still have a few things to validate
 +        processPartitionKeyRestrictions(hasQueriableIndex);
 +
 +        // Some but not all of the partition key columns have been specified;
 +        // hence we need turn these restrictions into index expressions.
 +        if (usesSecondaryIndexing)
 +            indexRestrictions.add(partitionKeyRestrictions);
 +
 +        checkFalse(selectsOnlyStaticColumns && hasClusteringColumnsRestriction(),
 +                   "Cannot restrict clustering columns when selecting only static columns");
 +
 +        processClusteringColumnsRestrictions(hasQueriableIndex, selectACollection);
 +
 +        // Covers indexes on the first clustering column (among others).
 +        if (isKeyRange && hasQueriableClusteringColumnIndex)
 +            usesSecondaryIndexing = true;
 +
 +        usesSecondaryIndexing = usesSecondaryIndexing || clusteringColumnsRestrictions.isContains();
 +
 +        if (usesSecondaryIndexing)
 +            indexRestrictions.add(clusteringColumnsRestrictions);
 +
 +        // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
 +        // there is restrictions not covered by the PK.
 +        if (!nonPrimaryKeyRestrictions.isEmpty())
 +        {
 +            if (!hasQueriableIndex)
 +            {
 +                // Filtering for non-index query is only supported for thrift static CFs
 +                if (cfm.comparator.isDense() ||  cfm.comparator.isCompound())
 +                    throw invalidRequest("Predicates on non-primary-key columns (%s) are not yet supported for non secondary index queries",
 +                                         Joiner.on(", ").join(toIdentifiers(nonPrimaryKeyRestrictions.getColumnDefs())));
 +
-                 if (!useFiltering)
++                if (!allowFiltering)
 +                    throw invalidRequest(REQUIRES_ALLOW_FILTERING_MESSAGE);
 +            }
 +            usesSecondaryIndexing = true;
 +            indexRestrictions.add(nonPrimaryKeyRestrictions);
 +        }
 +
 +        if (usesSecondaryIndexing)
 +            validateSecondaryIndexSelections(selectsOnlyStaticColumns);
 +    }
 +
 +    private void addRestriction(Restriction restriction) throws InvalidRequestException
 +    {
 +        if (restriction.isMultiColumn())
 +            clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction);
 +        else if (restriction.isOnToken())
 +            partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
 +        else
 +            addSingleColumnRestriction((SingleColumnRestriction) restriction);
 +    }
 +
 +    public Iterable<Function> getFunctions()
 +    {
 +        return Iterables.concat(partitionKeyRestrictions.getFunctions(),
 +                                clusteringColumnsRestrictions.getFunctions(),
 +                                nonPrimaryKeyRestrictions.getFunctions());
 +    }
 +
 +    private void addSingleColumnRestriction(SingleColumnRestriction restriction) throws InvalidRequestException
 +    {
 +        ColumnDefinition def = restriction.columnDef;
 +        if (def.isPartitionKey())
 +            partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
 +        else if (def.isClusteringColumn())
 +            clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction);
 +        else
 +            nonPrimaryKeyRestrictions = nonPrimaryKeyRestrictions.addRestriction(restriction);
 +    }
 +
 +    /**
 +     * Checks if the restrictions on the partition key is an IN restriction.
 +     *
 +     * @return <code>true</code> the restrictions on the partition key is an IN restriction, <code>false</code>
 +     * otherwise.
 +     */
 +    public boolean keyIsInRelation()
 +    {
 +        return partitionKeyRestrictions.isIN();
 +    }
 +
 +    /**
 +     * Checks if the query request a range of partition keys.
 +     *
 +     * @return <code>true</code> if the query request a range of partition keys, <code>false</code> otherwise.
 +     */
 +    public boolean isKeyRange()
 +    {
 +        return this.isKeyRange;
 +    }
 +
 +    /**
 +     * Checks if the secondary index need to be queried.
 +     *
 +     * @return <code>true</code> if the secondary index need to be queried, <code>false</code> otherwise.
 +     */
 +    public boolean usesSecondaryIndexing()
 +    {
 +        return this.usesSecondaryIndexing;
 +    }
 +
 +    private void processPartitionKeyRestrictions(boolean hasQueriableIndex) throws InvalidRequestException
 +    {
 +        // If there is a queriable index, no special condition are required on the other restrictions.
 +        // But we still need to know 2 things:
 +        // - If we don't have a queriable index, is the query ok
 +        // - Is it queriable without 2ndary index, which is always more efficient
 +        // If a component of the partition key is restricted by a relation, all preceding
 +        // components must have a EQ. Only the last partition key component can be in IN relation.
 +        if (partitionKeyRestrictions.isOnToken())
 +            isKeyRange = true;
 +
 +        if (hasPartitionKeyUnrestrictedComponents())
 +        {
 +            if (!partitionKeyRestrictions.isEmpty())
 +            {
 +                if (!hasQueriableIndex)
 +                    throw invalidRequest("Partition key parts: %s must be restricted as other parts are",
 +                                         Joiner.on(", ").join(getPartitionKeyUnrestrictedComponents()));
 +            }
 +
 +            isKeyRange = true;
 +            usesSecondaryIndexing = hasQueriableIndex;
 +        }
 +    }
 +
 +    /**
 +     * Checks if the partition key has some unrestricted components.
 +     * @return <code>true</code> if the partition key has some unrestricted components, <code>false</code> otherwise.
 +     */
 +    private boolean hasPartitionKeyUnrestrictedComponents()
 +    {
 +        return partitionKeyRestrictions.size() <  cfm.partitionKeyColumns().size();
 +    }
 +
 +    public boolean hasPartitionKeyRestrictions()
 +    {
 +        return !partitionKeyRestrictions.isEmpty();
 +    }
 +
 +    /**
 +     * Checks if the restrictions contain any non-primary key restrictions
 +     * @return <code>true</code> if the restrictions contain any non-primary key restrictions, <code>false</code> otherwise.
 +     */
 +    public boolean hasNonPrimaryKeyRestrictions()
 +    {
 +        return !nonPrimaryKeyRestrictions.isEmpty();
 +    }
 +
 +    /**
 +     * Returns the partition key components that are not restricted.
 +     * @return the partition key components that are not restricted.
 +     */
 +    private List<ColumnIdentifier> getPartitionKeyUnrestrictedComponents()
 +    {
 +        List<ColumnDefinition> list = new ArrayList<>(cfm.partitionKeyColumns());
 +        list.removeAll(partitionKeyRestrictions.getColumnDefs());
 +        return ColumnDefinition.toIdentifiers(list);
 +    }
 +
 +    /**
 +     * Processes the clustering column restrictions.
 +     *
 +     * @param hasQueriableIndex <code>true</code> if some of the queried data are indexed, <code>false</code> otherwise
 +     * @param selectACollection <code>true</code> if the query should return a collection column
 +     * @throws InvalidRequestException if the request is invalid
 +     */
 +    private void processClusteringColumnsRestrictions(boolean hasQueriableIndex,
 +                                                      boolean selectACollection) throws InvalidRequestException
 +    {
++        validateClusteringRestrictions(hasQueriableIndex);
++
 +        checkFalse(clusteringColumnsRestrictions.isIN() && selectACollection,
 +                   "Cannot restrict clustering columns by IN relations when a collection is selected by the query");
 +        checkFalse(clusteringColumnsRestrictions.isContains() && !hasQueriableIndex,
 +                   "Cannot restrict clustering columns by a CONTAINS relation without a secondary index");
 +
 +        if (hasClusteringColumnsRestriction() && clusteringRestrictionsNeedFiltering())
 +        {
 +            if (hasQueriableIndex)
 +            {
 +                usesSecondaryIndexing = true;
 +                return;
 +            }
 +
 +            List<ColumnDefinition> clusteringColumns = cfm.clusteringColumns();
 +            List<ColumnDefinition> restrictedColumns = new LinkedList<>(clusteringColumnsRestrictions.getColumnDefs());
 +
 +            for (int i = 0, m = restrictedColumns.size(); i < m; i++)
 +            {
 +                ColumnDefinition clusteringColumn = clusteringColumns.get(i);
 +                ColumnDefinition restrictedColumn = restrictedColumns.get(i);
 +
 +                if (!clusteringColumn.equals(restrictedColumn))
 +                {
 +                    throw invalidRequest(
 +                              "PRIMARY KEY column \"%s\" cannot be restricted as preceding column \"%s\" is not restricted",
 +                              restrictedColumn.name,
 +                              clusteringColumn.name);
 +                }
 +            }
 +        }
 +    }
 +
++    /**
++     * Validates whether or not restrictions are allowed for execution when secondary index is not used.
++     */
++    public final void validateClusteringRestrictions(boolean hasQueriableIndex)
++    {
++        assert clusteringColumnsRestrictions instanceof PrimaryKeyRestrictionSet;
++
++        // If there's a queriable index, filtering will take care of clustering restrictions
++        if (hasQueriableIndex)
++            return;
++
++        Iterator<Restriction> iter = ((PrimaryKeyRestrictionSet)clusteringColumnsRestrictions).iterator();
++        Restriction previousRestriction = null;
++
++        while (iter.hasNext())
++        {
++            Restriction restriction = iter.next();
++
++            if (previousRestriction != null)
++            {
++                ColumnDefinition lastRestrictionStart = previousRestriction.getFirstColumn();
++                ColumnDefinition newRestrictionStart = restriction.getFirstColumn();
++
++                if (previousRestriction.isSlice() && newRestrictionStart.position() > lastRestrictionStart.position())
++                    throw invalidRequest("Clustering column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)",
++                                         newRestrictionStart.name,
++                                         lastRestrictionStart.name);
++            }
++            previousRestriction = restriction;
++        }
++    }
++
 +    public final boolean clusteringRestrictionsNeedFiltering()
 +    {
 +        assert clusteringColumnsRestrictions instanceof PrimaryKeyRestrictionSet;
 +        return ((PrimaryKeyRestrictionSet) clusteringColumnsRestrictions).needsFiltering();
 +    }
 +
 +    public List<IndexExpression> getIndexExpressions(SecondaryIndexManager indexManager,
 +                                                     QueryOptions options) throws InvalidRequestException
 +    {
 +        if (!usesSecondaryIndexing || indexRestrictions.isEmpty())
 +            return Collections.emptyList();
 +
 +        List<IndexExpression> expressions = new ArrayList<>();
 +        for (Restrictions restrictions : indexRestrictions)
 +            restrictions.addIndexExpressionTo(expressions, indexManager, options);
 +
 +        return expressions;
 +    }
 +
 +    /**
 +     * Returns the partition keys for which the data is requested.
 +     *
 +     * @param options the query options
 +     * @return the partition keys for which the data is requested.
 +     * @throws InvalidRequestException if the partition keys cannot be retrieved
 +     */
 +    public Collection<ByteBuffer> getPartitionKeys(final QueryOptions options) throws InvalidRequestException
 +    {
 +        return partitionKeyRestrictions.values(options);
 +    }
 +
 +    /**
 +     * Returns the specified bound of the partition key.
 +     *
 +     * @param b the boundary type
 +     * @param options the query options
 +     * @return the specified bound of the partition key
 +     * @throws InvalidRequestException if the boundary cannot be retrieved
 +     */
 +    private ByteBuffer getPartitionKeyBound(Bound b, QueryOptions options) throws InvalidRequestException
 +    {
 +        // Deal with unrestricted partition key components (special-casing is required to deal with 2i queries on the
 +        // first
 +        // component of a composite partition key).
 +        if (hasPartitionKeyUnrestrictedComponents())
 +            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +
 +        // We deal with IN queries for keys in other places, so we know buildBound will return only one result
 +        return partitionKeyRestrictions.bounds(b, options).get(0);
 +    }
 +
 +    /**
 +     * Returns the partition key bounds.
 +     *
 +     * @param options the query options
 +     * @return the partition key bounds
 +     * @throws InvalidRequestException if the query is invalid
 +     */
 +    public AbstractBounds<RowPosition> getPartitionKeyBounds(QueryOptions options) throws InvalidRequestException
 +    {
 +        IPartitioner p = StorageService.getPartitioner();
 +
 +        if (partitionKeyRestrictions.isOnToken())
 +        {
 +            return getPartitionKeyBoundsForTokenRestrictions(p, options);
 +        }
 +
 +        return getPartitionKeyBounds(p, options);
 +    }
 +
 +    private AbstractBounds<RowPosition> getPartitionKeyBounds(IPartitioner p,
 +                                                              QueryOptions options) throws InvalidRequestException
 +    {
 +        ByteBuffer startKeyBytes = getPartitionKeyBound(Bound.START, options);
 +        ByteBuffer finishKeyBytes = getPartitionKeyBound(Bound.END, options);
 +
 +        RowPosition startKey = RowPosition.ForKey.get(startKeyBytes, p);
 +        RowPosition finishKey = RowPosition.ForKey.get(finishKeyBytes, p);
 +
 +        if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum())
 +            return null;
 +
 +        if (partitionKeyRestrictions.isInclusive(Bound.START))
 +        {
 +            return partitionKeyRestrictions.isInclusive(Bound.END)
 +                    ? new Bounds<>(startKey, finishKey)
 +                    : new IncludingExcludingBounds<>(startKey, finishKey);
 +        }
 +
 +        return partitionKeyRestrictions.isInclusive(Bound.END)
 +                ? new Range<>(startKey, finishKey)
 +                : new ExcludingBounds<>(startKey, finishKey);
 +    }
 +
 +    private AbstractBounds<RowPosition> getPartitionKeyBoundsForTokenRestrictions(IPartitioner p,
 +                                                                                  QueryOptions options)
 +                                                                                          throws InvalidRequestException
 +    {
 +        Token startToken = getTokenBound(Bound.START, options, p);
 +        Token endToken = getTokenBound(Bound.END, options, p);
 +
 +        boolean includeStart = partitionKeyRestrictions.isInclusive(Bound.START);
 +        boolean includeEnd = partitionKeyRestrictions.isInclusive(Bound.END);
 +
 +        /*
 +         * If we ask SP.getRangeSlice() for (token(200), token(200)], it will happily return the whole ring.
 +         * However, wrapping range doesn't really make sense for CQL, and we want to return an empty result in that
 +         * case (CASSANDRA-5573). So special case to create a range that is guaranteed to be empty.
 +         *
 +         * In practice, we want to return an empty result set if either startToken > endToken, or both are equal but
 +         * one of the bound is excluded (since [a, a] can contains something, but not (a, a], [a, a) or (a, a)).
 +         * Note though that in the case where startToken or endToken is the minimum token, then this special case
 +         * rule should not apply.
 +         */
 +        int cmp = startToken.compareTo(endToken);
 +        if (!startToken.isMinimum() && !endToken.isMinimum()
 +                && (cmp > 0 || (cmp == 0 && (!includeStart || !includeEnd))))
 +            return null;
 +
 +        RowPosition start = includeStart ? startToken.minKeyBound() : startToken.maxKeyBound();
 +        RowPosition end = includeEnd ? endToken.maxKeyBound() : endToken.minKeyBound();
 +
 +        return new Range<>(start, end);
 +    }
 +
 +    private Token getTokenBound(Bound b, QueryOptions options, IPartitioner p) throws InvalidRequestException
 +    {
 +        if (!partitionKeyRestrictions.hasBound(b))
 +            return p.getMinimumToken();
 +
 +        ByteBuffer value = partitionKeyRestrictions.bounds(b, options).get(0);
 +        checkNotNull(value, "Invalid null token value");
 +        return p.getTokenFactory().fromByteArray(value);
 +    }
 +
 +    /**
 +     * Checks if the query does not contains any restriction on the clustering columns.
 +     *
 +     * @return <code>true</code> if the query does not contains any restriction on the clustering columns,
 +     * <code>false</code> otherwise.
 +     */
 +    public boolean hasNoClusteringColumnsRestriction()
 +    {
 +        return clusteringColumnsRestrictions.isEmpty();
 +    }
 +
 +    /**
 +     * Checks if the query has some restrictions on the clustering columns.
 +     *
 +     * @return <code>true</code> if the query has some restrictions on the clustering columns,
 +     * <code>false</code> otherwise.
 +     */
 +    public boolean hasClusteringColumnsRestriction()
 +    {
 +        return !clusteringColumnsRestrictions.isEmpty();
 +    }
 +
 +    // For non-composite slices, we don't support internally the difference between exclusive and
 +    // inclusive bounds, so we deal with it manually.
 +    public boolean isNonCompositeSliceWithExclusiveBounds()
 +    {
 +        return !cfm.comparator.isCompound()
 +                && clusteringColumnsRestrictions.isSlice()
 +                && (!clusteringColumnsRestrictions.isInclusive(Bound.START) || !clusteringColumnsRestrictions.isInclusive(Bound.END));
 +    }
 +
 +    /**
 +     * Returns the requested clustering columns as <code>Composite</code>s.
 +     *
 +     * @param options the query options
 +     * @return the requested clustering columns as <code>Composite</code>s
 +     * @throws InvalidRequestException if the query is not valid
 +     */
 +    public List<Composite> getClusteringColumnsAsComposites(QueryOptions options) throws InvalidRequestException
 +    {
 +        return clusteringColumnsRestrictions.valuesAsComposites(options);
 +    }
 +
 +    /**
 +     * Returns the bounds (start or end) of the clustering columns as <code>Composites</code>.
 +     *
 +     * @param b the bound type
 +     * @param options the query options
 +     * @return the bounds (start or end) of the clustering columns as <code>Composites</code>
 +     * @throws InvalidRequestException if the request is not valid
 +     */
 +    public List<Composite> getClusteringColumnsBoundsAsComposites(Bound b,
 +                                                                  QueryOptions options) throws InvalidRequestException
 +    {
 +        List<Composite> bounds = clusteringColumnsRestrictions.boundsAsComposites(b, options);
 +        for (Composite c : bounds) {
 +            if (!c.isEmpty())
 +                QueryProcessor.validateComposite(c, cfm.comparator);
 +        }
 +        return bounds;
 +    }
 +
 +    /**
 +     * Returns the bounds (start or end) of the clustering columns.
 +     *
 +     * @param b the bound type
 +     * @param options the query options
 +     * @return the bounds (start or end) of the clustering columns
 +     * @throws InvalidRequestException if the request is not valid
 +     */
 +    public List<ByteBuffer> getClusteringColumnsBounds(Bound b, QueryOptions options) throws InvalidRequestException
 +    {
 +        return clusteringColumnsRestrictions.bounds(b, options);
 +    }
 +
 +    /**
 +     * Checks if the bounds (start or end) of the clustering columns are inclusive.
 +     *
 +     * @param bound the bound type
 +     * @return <code>true</code> if the bounds (start or end) of the clustering columns are inclusive,
 +     * <code>false</code> otherwise
 +     */
 +    public boolean areRequestedBoundsInclusive(Bound bound)
 +    {
 +        return clusteringColumnsRestrictions.isInclusive(bound);
 +    }
 +
 +    /**
 +     * Checks if the query returns a range of columns.
 +     *
 +     * @return <code>true</code> if the query returns a range of columns, <code>false</code> otherwise.
 +     */
 +    public boolean isColumnRange()
 +    {
 +        // Due to CASSANDRA-5762, we always do a slice for CQL3 tables (not dense, composite).
 +        // Static CF (non dense but non composite) never entails a column slice however
 +        if (!cfm.comparator.isDense())
 +            return cfm.comparator.isCompound();
 +
 +        // Otherwise (i.e. for compact table where we don't have a row marker anyway and thus don't care about
 +        // CASSANDRA-5762),
 +        // it is a range query if it has at least one the column alias for which no relation is defined or is not EQ.
 +        return clusteringColumnsRestrictions.size() < cfm.clusteringColumns().size() || clusteringColumnsRestrictions.isSlice();
 +    }
 +
 +    /**
 +     * Checks if the query need to use filtering.
 +     * @return <code>true</code> if the query need to use filtering, <code>false</code> otherwise.
 +     */
 +    public boolean needFiltering()
 +    {
 +        int numberOfRestrictedColumns = 0;
 +        for (Restrictions restrictions : indexRestrictions)
 +            numberOfRestrictedColumns += restrictions.size();
 +
 +        return numberOfRestrictedColumns > 1
 +                || (numberOfRestrictedColumns == 0 && !clusteringColumnsRestrictions.isEmpty())
 +                || (numberOfRestrictedColumns != 0
 +                        && nonPrimaryKeyRestrictions.hasMultipleContains());
 +    }
 +
 +    private void validateSecondaryIndexSelections(boolean selectsOnlyStaticColumns) throws InvalidRequestException
 +    {
 +        checkFalse(keyIsInRelation(),
 +                   "Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
 +        // When the user only select static columns, the intent is that we don't query the whole partition but just
 +        // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on
 +        // static columns
 +        // so far, 2i means that you've restricted a non static column, so the query is somewhat non-sensical.
 +        checkFalse(selectsOnlyStaticColumns, "Queries using 2ndary indexes don't support selecting only static columns");
 +    }
 +
 +    public void reverse()
 +    {
 +        clusteringColumnsRestrictions = new ReversedPrimaryKeyRestrictions(clusteringColumnsRestrictions);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e06dae81/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
index a1542ce,98dda26..0975662
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
@@@ -79,39 -81,43 +79,39 @@@ public class SelectMultiColumnRelationT
                                   "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("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("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)",
++            assertInvalidMessage("Clustering 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 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 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("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
 -            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 (b) < (?) AND (c) < (?) AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
-             assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
 -            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 (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: (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);
          }
      }
  
@@@ -862,16 -786,12 +862,42 @@@
                     row(0, 0, 1, 1, 2),
                     row(0, 1, 1, 1, 2));
  
 -        assertRows(execute("SELECT * FROM %s WHERE (b) >= (?) AND e = ?", 1, 2),
 +        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
 +                             "SELECT * FROM %s WHERE (b) >= (?) AND e = ?", 1, 2);
 +        assertRows(execute("SELECT * FROM %s WHERE (b) >= (?) AND e = ? ALLOW FILTERING", 1, 2),
                     row(0, 1, 1, 1, 2));
  
 -        assertRows(execute("SELECT * FROM %s WHERE (b, c) >= (?, ?) AND e = ?", 1, 1, 2),
 +        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
 +                             "SELECT * FROM %s WHERE (b, c) >= (?, ?) AND e = ?", 1, 1, 2);
 +        assertRows(execute("SELECT * FROM %s WHERE (b, c) >= (?, ?) AND e = ? ALLOW FILTERING", 1, 1, 2),
                     row(0, 1, 1, 1, 2));
++    }
+ 
++    @Test
++    public void testMultiColumnRestrictionsWithIndex() throws Throwable
++    {
++        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, v int, PRIMARY KEY (a, b, c, d, e))");
++        createIndex("CREATE INDEX ON %s (v)");
++        for (int i = 1; i <= 5; i++)
++        {
++            execute("INSERT INTO %s (a,b,c,d,e,v) VALUES (?,?,?,?,?,?)", 0, i, 0, 0, 0, 0);
++            execute("INSERT INTO %s (a,b,c,d,e,v) VALUES (?,?,?,?,?,?)", 0, i, i, 0, 0, 0);
++            execute("INSERT INTO %s (a,b,c,d,e,v) VALUES (?,?,?,?,?,?)", 0, i, i, i, 0, 0);
++            execute("INSERT INTO %s (a,b,c,d,e,v) VALUES (?,?,?,?,?,?)", 0, i, i, i, i, 0);
++            execute("INSERT INTO %s (a,b,c,d,e,v) VALUES (?,?,?,?,?,?)", 0, i, i, i, i, i);
++        }
 +
++        String errorMsg = "Multi-column slice restrictions cannot be used for filtering.";
++        assertInvalidMessage(errorMsg,
++                             "SELECT * FROM %s WHERE a = 0 AND (c,d) < (2,2) AND v = 0 ALLOW FILTERING");
++        assertInvalidMessage(errorMsg,
++                             "SELECT * FROM %s WHERE a = 0 AND (d,e) < (2,2) AND b = 1 AND v = 0 ALLOW FILTERING");
++        assertInvalidMessage(errorMsg,
++                             "SELECT * FROM %s WHERE a = 0 AND b = 1 AND (d,e) < (2,2) AND v = 0 ALLOW FILTERING");
++        assertInvalidMessage(errorMsg,
++                             "SELECT * FROM %s WHERE a = 0 AND b > 1 AND (d,e) < (2,2) AND v = 0 ALLOW FILTERING");
++        assertInvalidMessage(errorMsg,
++                             "SELECT * FROM %s WHERE a = 0 AND (b,c) > (1,0) AND (d,e) < (2,2) AND v = 0 ALLOW FILTERING");
      }
  
      @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e06dae81/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
index 9b10d0e,68cf6f8..c8df4c3
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
@@@ -2278,4 -1264,68 +2278,65 @@@ public class SelectTest extends CQLTest
                             "SELECT * FROM %s WHERE a = 'foo' AND b= 'bar' AND c IN (?, ?)",
                             new String(TOO_BIG.array()), new String(TOO_BIG.array()));
      }
+ 
+     @Test
+     public void testFilteringWithSecondaryIndex() throws Throwable
+     {
+         createTable("CREATE TABLE %s (pk int, " +
+                     "c1 int, " +
+                     "c2 int, " +
+                     "c3 int, " +
+                     "v int, " +
+                     "PRIMARY KEY (pk, c1, c2, c3))");
+         createIndex("CREATE INDEX v_idx_1 ON %s (v);");
+ 
+         for (int i = 1; i <= 5; i++)
+         {
+             execute("INSERT INTO %s (pk, c1, c2, c3, v) VALUES (?, ?, ?, ?, ?)", 1, 1, 1, 1, i);
+             execute("INSERT INTO %s (pk, c1, c2, c3, v) VALUES (?, ?, ?, ?, ?)", 1, 1, 1, i, i);
+             execute("INSERT INTO %s (pk, c1, c2, c3, v) VALUES (?, ?, ?, ?, ?)", 1, 1, i, i, i);
+             execute("INSERT INTO %s (pk, c1, c2, c3, v) VALUES (?, ?, ?, ?, ?)", 1, i, i, i, i);
+         }
+ 
+         assertRows(execute("SELECT * FROM %s WHERE pk = 1 AND  c1 > 0 AND c1 < 5 AND c2 = 1 AND v = 3 ALLOW FILTERING;"),
+                    row(1, 1, 1, 3, 3));
+ 
+         assertEmpty(execute("SELECT * FROM %s WHERE pk = 1 AND  c1 > 1 AND c1 < 5 AND c2 = 1 AND v = 3 ALLOW FILTERING;"));
+ 
+         assertRows(execute("SELECT * FROM %s WHERE pk = 1 AND  c1 > 1 AND c2 > 2 AND c3 > 2 AND v = 3 ALLOW FILTERING;"),
+                    row(1, 3, 3, 3, 3));
+ 
+         assertRows(execute("SELECT * FROM %s WHERE pk = 1 AND  c1 > 1 AND c2 > 2 AND c3 = 3 AND v = 3 ALLOW FILTERING;"),
+                    row(1, 3, 3, 3, 3));
+ 
 -        assertRows(execute("SELECT * FROM %s WHERE pk = 1 AND  (c1, c2) > (1, 3) AND c3 = 3 AND v = 3 ALLOW FILTERING;"),
 -                   row(1, 3, 3, 3, 3));
 -
 -        assertInvalidMessage("Clustering column \"c1\" cannot be restricted by an IN relation",
 -                             "SELECT * FROM %s WHERE pk = 1 AND  c1 IN(0,1,2) AND c2 = 1 AND v = 3 ALLOW FILTERING;");
++        assertRows(execute("SELECT * FROM %s WHERE pk = 1 AND  c1 IN(0,1,2) AND c2 = 1 AND v = 3 ALLOW FILTERING;"),
++                   row(1, 1, 1, 3, 3));
+ 
 -        assertInvalidMessage("Clustering column \"c1\" cannot be restricted by an IN relation",
 -                             "SELECT * FROM %s WHERE pk = 1 AND  c1 IN(0,1,2) AND c2 = 1 AND v = 3");
++        assertRows(execute("SELECT * FROM %s WHERE pk = 1 AND  c1 IN(0,1,2) AND c2 = 1 AND v = 3"),
++                   row(1, 1, 1, 3, 3));
+ 
 -        assertInvalidMessage("PRIMARY KEY column \"c2\" cannot be restricted (preceding column \"c1\" is restricted by a non-EQ relation)",
++        assertInvalidMessage("Clustering column \"c2\" cannot be restricted (preceding column \"c1\" is restricted by a non-EQ relation)",
+                              "SELECT * FROM %s WHERE pk = 1 AND  c1 > 0 AND c1 < 5 AND c2 = 1 ALLOW FILTERING;");
+ 
 -        assertInvalidMessage("PRIMARY KEY column \"c2\" cannot be restricted (preceding column \"c1\" is not restricted)",
++        assertInvalidMessage("PRIMARY KEY column \"c2\" cannot be restricted as preceding column \"c1\" is not restricted",
+                              "SELECT * FROM %s WHERE pk = 1 AND  c2 = 1 ALLOW FILTERING;");
+     }
+ 
+     @Test
+     public void testIndexQueryWithCompositePartitionKey() throws Throwable
+     {
+         createTable("CREATE TABLE %s (p1 int, p2 int, v int, PRIMARY KEY ((p1, p2)))");
 -        assertInvalidMessage("Partition key part p2 must be restricted since preceding part is",
++        assertInvalidMessage("Partition key parts: p2 must be restricted as other parts are",
+                              "SELECT * FROM %s WHERE p1 = 1 AND v = 3 ALLOW FILTERING");
+ 
+         createIndex("CREATE INDEX ON %s(v)");
+ 
+         execute("INSERT INTO %s(p1, p2, v) values (?, ?, ?)", 1, 1, 3);
+         execute("INSERT INTO %s(p1, p2, v) values (?, ?, ?)", 1, 2, 3);
+         execute("INSERT INTO %s(p1, p2, v) values (?, ?, ?)", 2, 1, 3);
+ 
+         assertRows(execute("SELECT * FROM %s WHERE p1 = 1 AND v = 3 ALLOW FILTERING"),
+                    row(1, 2, 3),
+                    row(1, 1, 3));
+     }
  }