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/08/16 13:28:51 UTC

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

Branch: refs/heads/cassandra-2.2
Commit: 9583b6b312866a4861df1a795329695b3fc09328
Parents: 88b3cfc 527d189
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Aug 16 15:25:42 2016 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Aug 16 15:26:26 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  10 +
 .../AbstractPrimaryKeyRestrictions.java         |   5 +-
 .../cql3/restrictions/AbstractRestriction.java  |  10 +-
 .../ForwardingPrimaryKeyRestrictions.java       |  31 +-
 .../restrictions/MultiColumnRestriction.java    |  48 +-
 .../restrictions/PrimaryKeyRestrictionSet.java  |  36 +-
 .../restrictions/PrimaryKeyRestrictions.java    |   9 +-
 .../cql3/restrictions/Restriction.java          |  16 +-
 .../ReversedPrimaryKeyRestrictions.java         |   9 +-
 .../restrictions/SingleColumnRestriction.java   |  57 +-
 .../restrictions/StatementRestrictions.java     |  23 +-
 .../cql3/restrictions/TokenFilter.java          |  36 +-
 .../cql3/restrictions/TokenRestriction.java     |  13 +-
 .../cql3/statements/ModificationStatement.java  |   4 +-
 .../cql3/statements/SelectStatement.java        |  11 +-
 .../cql3/statements/UpdateStatement.java        |   2 +-
 .../cassandra/db/compaction/Scrubber.java       | 159 +++-
 .../cassandra/db/marshal/ReversedType.java      |  10 -
 .../PrimaryKeyRestrictionSetTest.java           | 372 ++++-----
 .../validation/entities/SecondaryIndexTest.java | 142 ++++
 .../cql3/validation/operations/DeleteTest.java  |  42 +
 .../operations/SelectOrderByTest.java           | 758 +++++++++++--------
 .../cql3/validation/operations/SelectTest.java  | 372 ++++++++-
 .../cassandra/db/marshal/ReversedTypeTest.java  |   4 +-
 25 files changed, 1540 insertions(+), 640 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9583b6b3/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 394598a,dee669a..c421398
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,24 -1,5 +1,25 @@@
 -2.1.16
 - * Fix queries with empty ByteBuffer values in clustering column restrictions (CASSANDRA-12127) 
 +2.2.8
 + * Only set broadcast_rpc_address on Ec2MultiRegionSnitch if it's not set (CASSANDRA-11357)
 + * Update StorageProxy range metrics for timeouts, failures and unavailables (CASSANDRA-9507)
 + * Add Sigar to classes included in clientutil.jar (CASSANDRA-11635)
 + * Add decay to histograms and timers used for metrics (CASSANDRA-11752)
 + * Fix hanging stream session (CASSANDRA-10992)
 + * Add byteman support for testing (CASSANDRA-12377)
 + * Fix INSERT JSON, fromJson() support of smallint, tinyint types (CASSANDRA-12371)
 + * Restore JVM metric export for metric reporters (CASSANDRA-12312)
 + * Release sstables of failed stream sessions only when outgoing transfers are finished (CASSANDRA-11345)
 + * Revert CASSANDRA-11427 (CASSANDRA-12351)
 + * Wait for tracing events before returning response and query at same consistency level client side (CASSANDRA-11465)
 + * cqlsh copyutil should get host metadata by connected address (CASSANDRA-11979)
 + * Fixed cqlshlib.test.remove_test_db (CASSANDRA-12214)
 + * Synchronize ThriftServer::stop() (CASSANDRA-12105)
 + * Use dedicated thread for JMX notifications (CASSANDRA-12146)
 + * NPE when trying to remove purgable tombstones from result (CASSANDRA-12143)
 + * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
 + * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
 + * Don't write shadowed range tombstone (CASSANDRA-12030)
 +Merged from 2.1:
++ * Fix queries with empty ByteBuffer values in clustering column restrictions (CASSANDRA-12127)
   * Disable passing control to post-flush after flush failure to prevent data loss (CASSANDRA-11828)
   * Allow STCS-in-L0 compactions to reduce scope with LCS (CASSANDRA-12040)
   * cannot use cql since upgrading python to 2.7.11+ (CASSANDRA-11850)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9583b6b3/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index f0712eb,6a70adc..0a3ab36
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -13,13 -13,18 +13,23 @@@ restore snapshots created with the prev
  'sstableloader' tool. You can upgrade the file format of your snapshots
  using the provided 'sstableupgrade' tool.
  
 -2.1.16
 -======
++2.2.8
++=====
+ 
+ Upgrading
+ ---------
+     - The ReversedType behaviour has been corrected for clustering columns of
+       BYTES type containing empty value. Scrub should be run on the existing
+       SSTables containing a descending clustering column of BYTES type to correct
 -      their ordering. See CASSANDRA-12127 for details.
++      their ordering. See CASSANDRA-12127 for more details.
+ 
 -2.1.15
 -======
 +2.2.7
 +=====
 +
 +New features
 +------------
 +    - JSON timestamps are now in UTC and contain the timezone information, see
 +      CASSANDRA-11137 for more details.
  
  Upgrading
  ---------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9583b6b3/src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java
index 2eaa386,0000000..51c3e26
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java
@@@ -1,60 -1,0 +1,61 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.cql3.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.List;
 +
++import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.composites.CType;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +/**
 + * Base class for <code>PrimaryKeyRestrictions</code>.
 + */
 +abstract class AbstractPrimaryKeyRestrictions extends AbstractRestriction implements PrimaryKeyRestrictions
 +{
 +    /**
 +     * The composite type.
 +     */
 +    protected final CType ctype;
 +
 +    public AbstractPrimaryKeyRestrictions(CType ctype)
 +    {
 +        this.ctype = ctype;
 +    }
 +
 +    @Override
-     public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
++    public List<ByteBuffer> bounds(CFMetaData cfm, Bound b, QueryOptions options) throws InvalidRequestException
 +    {
-         return values(options);
++        return values(cfm, options);
 +    }
 +
 +    @Override
 +    public final boolean isEmpty()
 +    {
 +        return getColumnDefs().isEmpty();
 +    }
 +
 +    @Override
 +    public final int size()
 +    {
 +        return getColumnDefs().size();
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9583b6b3/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
index 385bfbc,0000000..4093780
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
@@@ -1,113 -1,0 +1,119 @@@
 +/*
 + * 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 org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +
 +import org.apache.cassandra.cql3.ColumnSpecification;
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.composites.CompositesBuilder;
 +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;
 +
 +/**
 + * Base class for <code>Restriction</code>s
 + */
 +abstract class AbstractRestriction  implements Restriction
 +{
 +    @Override
 +    public  boolean isOnToken()
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public boolean isMultiColumn()
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public boolean isSlice()
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public boolean isEQ()
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public boolean isIN()
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public boolean isContains()
 +    {
 +        return false;
 +    }
 +
 +    @Override
 +    public boolean hasBound(Bound b)
 +    {
 +        return true;
 +    }
 +
 +    @Override
-     public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
++    public CompositesBuilder appendBoundTo(CFMetaData cfm, CompositesBuilder builder, Bound bound, QueryOptions options)
 +    {
-         return appendTo(builder, options);
++        return appendTo(cfm, builder, options);
 +    }
 +
 +    @Override
 +    public boolean isInclusive(Bound b)
 +    {
 +        return true;
++    }
++
++    public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options)
++    {
++        return false;
 +    }
 +
 +    protected static ByteBuffer validateIndexedValue(ColumnSpecification columnSpec,
 +                                                     ByteBuffer value)
 +                                                     throws InvalidRequestException
 +    {
 +        checkNotNull(value, "Unsupported null value for column %s", columnSpec.name);
 +        checkBindValueSet(value, "Unsupported unset value for column %s", columnSpec.name);
 +        checkFalse(value.remaining() > 0xFFFF, "Index expression values may not be larger than 64K");
 +        return value;
 +    }
 +
 +    /**
 +     * Reverses the specified bound if the column type is a reversed one.
 +     *
 +     * @param columnDefinition the column definition
 +     * @param bound the bound
 +     * @return the bound reversed if the column type was a reversed one or the original bound
 +     */
 +    protected static Bound reverseBoundIfNeeded(ColumnDefinition columnDefinition, Bound bound)
 +    {
 +        return columnDefinition.isReversedType() ? bound.reverse() : bound;
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9583b6b3/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
index 71855a0,0000000..76d0233
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
@@@ -1,185 -1,0 +1,192 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.cql3.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.List;
 +
++import org.apache.cassandra.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.composites.Composite;
 +import org.apache.cassandra.db.composites.CompositesBuilder;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +/**
 + * A <code>PrimaryKeyRestrictions</code> which forwards all its method calls to another 
 + * <code>PrimaryKeyRestrictions</code>. Subclasses should override one or more methods to modify the behavior 
 + * of the backing <code>PrimaryKeyRestrictions</code> as desired per the decorator pattern. 
 + */
 +abstract class ForwardingPrimaryKeyRestrictions implements PrimaryKeyRestrictions
 +{
 +    /**
 +     * Returns the backing delegate instance that methods are forwarded to.
 +     * @return the backing delegate instance that methods are forwarded to.
 +     */
 +    protected abstract PrimaryKeyRestrictions getDelegate();
 +
 +    @Override
 +    public Iterable<Function> getFunctions()
 +    {
 +        return getDelegate().getFunctions();
 +    }
 +
 +    @Override
 +    public List<ColumnDefinition> getColumnDefs()
 +    {
 +        return getDelegate().getColumnDefs();
 +    }
 +
 +    @Override
 +    public ColumnDefinition getFirstColumn()
 +    {
 +        return getDelegate().getFirstColumn();
 +    }
 +
 +    @Override
 +    public ColumnDefinition getLastColumn()
 +    {
 +        return getDelegate().getLastColumn();
 +    }
 +
 +    @Override
 +    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException
 +    {
 +        return getDelegate().mergeWith(restriction);
 +    }
 +
 +    @Override
 +    public boolean hasSupportingIndex(SecondaryIndexManager secondaryIndexManager)
 +    {
 +        return getDelegate().hasSupportingIndex(secondaryIndexManager);
 +    }
 +
 +    @Override
-     public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
++    public List<ByteBuffer> values(CFMetaData cfm, QueryOptions options) throws InvalidRequestException
 +    {
-         return getDelegate().values(options);
++        return getDelegate().values(cfm, options);
 +    }
 +
 +    @Override
-     public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
++    public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options)
 +    {
-         return getDelegate().appendTo(builder, options);
++        return getDelegate().appendTo(cfm, builder, options);
 +    }
 +
 +    @Override
-     public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
++    public List<Composite> valuesAsComposites(CFMetaData cfm, QueryOptions options) throws InvalidRequestException
 +    {
-         return getDelegate().valuesAsComposites(options);
++        return getDelegate().valuesAsComposites(cfm, options);
 +    }
 +
 +    @Override
-     public List<ByteBuffer> bounds(Bound bound, QueryOptions options) throws InvalidRequestException
++    public List<ByteBuffer> bounds(CFMetaData cfm, Bound bound, QueryOptions options) throws InvalidRequestException
 +    {
-         return getDelegate().bounds(bound, options);
++        return getDelegate().bounds(cfm, bound, options);
 +    }
 +
 +    @Override
-     public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
++    public List<Composite> boundsAsComposites(CFMetaData cfm, Bound bound, QueryOptions options) throws InvalidRequestException
 +    {
-         return getDelegate().boundsAsComposites(bound, options);
++        return getDelegate().boundsAsComposites(cfm, bound, options);
 +    }
 +
 +    @Override
-     public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
++    public CompositesBuilder appendBoundTo(CFMetaData cfm, CompositesBuilder builder, Bound bound, QueryOptions options)
 +    {
-         return getDelegate().appendBoundTo(builder, bound, options);
++        return getDelegate().appendBoundTo(cfm, builder, bound, options);
 +    }
 +
 +    @Override
 +    public boolean isInclusive(Bound bound)
 +    {
 +        return getDelegate().isInclusive(bound.reverse());
 +    }
 +
 +    @Override
 +    public boolean isEmpty()
 +    {
 +        return getDelegate().isEmpty();
 +    }
 +
 +    @Override
 +    public int size()
 +    {
 +        return getDelegate().size();
 +    }
 +
 +    @Override
 +    public boolean isOnToken()
 +    {
 +        return getDelegate().isOnToken();
 +    }
 +
 +    @Override
 +    public boolean isSlice()
 +    {
 +        return getDelegate().isSlice();
 +    }
 +
 +    @Override
 +    public boolean isEQ()
 +    {
 +        return getDelegate().isEQ();
 +    }
 +
 +    @Override
 +    public boolean isIN()
 +    {
 +        return getDelegate().isIN();
 +    }
 +
 +    @Override
 +    public boolean isContains()
 +    {
 +        return getDelegate().isContains();
 +    }
 +
 +    @Override
 +    public boolean isMultiColumn()
 +    {
 +        return getDelegate().isMultiColumn();
 +    }
 +
 +    @Override
 +    public boolean hasBound(Bound b)
 +    {
 +        return getDelegate().hasBound(b);
 +    }
 +
 +    @Override
 +    public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                     SecondaryIndexManager indexManager,
 +                                     QueryOptions options) throws InvalidRequestException
 +    {
 +        getDelegate().addIndexExpressionTo(expressions, indexManager, options);
 +    }
++
++    @Override
++    public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options)
++    {
++        return getDelegate().isNotReturningAnyRows(cfm, options);
++    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9583b6b3/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
index 51e2ce4,0000000..44f25ec
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,553 @@@
 +/*
 + * 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.CFMetaData;
 +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)
++        public CompositesBuilder appendTo(CFMetaData cfm, 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));
 +            }
 +        }
++
++        @Override
++        public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options)
++        {
++            // Dense non-compound tables do not accept empty ByteBuffers. By consequence, we know that
++            // any query with an EQ restriction containing an empty value will not return any results.
++            return !cfm.comparator.isCompound()
++                    && !((Tuples.Value) value.bind(options)).getElements().get(0).hasRemaining();
++        }
 +    }
 +
 +    public abstract static class IN extends MultiColumnRestriction
 +    {
 +        /**
 +         * {@inheritDoc}
 +         */
 +        @Override
-         public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
++        public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options)
 +        {
-             List<List<ByteBuffer>> splitInValues = splitValues(options);
++            List<List<ByteBuffer>> splitInValues = filterValuesIfNeeded(cfm, splitValues(options));
 +            builder.addAllElementsToAll(splitInValues);
 +
 +            if (builder.containsNull())
 +                throw invalidRequest("Invalid null value in condition for columns: %s", ColumnDefinition.toIdentifiers(columnDefs));
 +            return builder;
 +        }
 +
++        private List<List<ByteBuffer>> filterValuesIfNeeded(CFMetaData cfm, List<List<ByteBuffer>> splitInValues)
++        {
++            if (cfm.comparator.isCompound())
++                return splitInValues;
++
++            // Dense non-compound tables do not accept empty ByteBuffers. By consequence, we know that we can
++            // ignore any IN value which is an empty byte buffer an which otherwise will trigger an error.
++
++            // As some List implementations do not support remove, we copy the list to be on the safe side.
++            List<List<ByteBuffer>> filteredValues = new ArrayList<>(splitInValues.size());
++            for (List<ByteBuffer> values : splitInValues)
++            {
++                if (values.get(0).hasRemaining())
++                    filteredValues.add(values);
++            }
++            return filteredValues;
++        }
++
 +        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)
++        public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options)
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        @Override
-         public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
++        public CompositesBuilder appendBoundTo(CFMetaData cfm, 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("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()));
 +        }
 +
++        @Override
++        public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options)
++        {
++            // Dense non-compound tables do not accept empty ByteBuffers. By consequence, we know that
++            // any query with a slice restriction with an empty value for the END bound will not return any results.
++            return !cfm.comparator.isCompound()
++                    && hasBound(Bound.END)
++                    && !componentBounds(Bound.END, options).get(0).hasRemaining();
++        }
++
 +        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/9583b6b3/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
index e1cbc29,0000000..2549bdf
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,405 -1,0 +1,417 @@@
 +/*
 + * 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.composites.*;
 +import org.apache.cassandra.db.composites.Composite.EOC;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +/**
 + * A set of single column restrictions on a primary key part (partition key or clustering key).
 + */
 +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;
 +
 +    public PrimaryKeyRestrictionSet(CType ctype)
 +    {
 +        super(ctype);
 +        this.restrictions = new RestrictionSet();
 +        this.eq = true;
 +    }
 +
 +    private PrimaryKeyRestrictionSet(PrimaryKeyRestrictionSet primaryKeyRestrictions,
 +                                     Restriction restriction) throws InvalidRequestException
 +    {
 +        super(primaryKeyRestrictions.ctype);
 +        this.restrictions = primaryKeyRestrictions.restrictions.addRestriction(restriction);
 +
 +        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;
 +    }
 +
 +    @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
++    public List<Composite> valuesAsComposites(CFMetaData cfm, QueryOptions options) throws InvalidRequestException
 +    {
-         return filterAndSort(appendTo(new CompositesBuilder(ctype), options).build());
++        return filterAndSort(appendTo(cfm, new CompositesBuilder(ctype), options).build());
 +    }
 +
 +    @Override
-     public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options)
++    public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options)
 +    {
 +        for (Restriction r : restrictions)
 +        {
-             r.appendTo(builder, options);
++            r.appendTo(cfm, builder, options);
 +            if (builder.hasMissingElements())
 +                break;
 +        }
 +        return builder;
 +    }
 +
 +    @Override
-     public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
++    public CompositesBuilder appendBoundTo(CFMetaData cfm, CompositesBuilder builder, Bound bound, QueryOptions options)
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
 +    @Override
-     public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
++    public List<Composite> boundsAsComposites(CFMetaData cfm, 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);
++                r.appendBoundTo(cfm, 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);
++            r.appendBoundTo(cfm, 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
++    public List<ByteBuffer> values(CFMetaData cfm, QueryOptions options) throws InvalidRequestException
 +    {
-         return Composites.toByteBuffers(valuesAsComposites(options));
++        return Composites.toByteBuffers(valuesAsComposites(cfm, options));
 +    }
 +
 +    @Override
-     public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException
++    public List<ByteBuffer> bounds(CFMetaData cfm, Bound b, QueryOptions options) throws InvalidRequestException
 +    {
-         return Composites.toByteBuffers(boundsAsComposites(b, options));
++        return Composites.toByteBuffers(boundsAsComposites(cfm, 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 || handleInFilter(restriction, position) || restriction.hasSupportingIndex(indexManager))
 +            {
 +                restriction.addIndexExpressionTo(expressions, indexManager, options);
 +                continue;
 +            }
 +
 +            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;
 +        for (Restriction restriction : restrictions)
 +        {
 +            if (handleInFilter(restriction, position))
 +                return true;
 +
 +            if (!restriction.isSlice())
 +                position = restriction.getLastColumn().position() + 1;
 +        }
 +
 +        return false;
 +    }
 +
++    @Override
++    public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options)
++    {
++        for (Restriction restriction : restrictions)
++        {
++            if (restriction.isNotReturningAnyRows(cfm, options))
++                return true;
++        }
++        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/9583b6b3/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java
index 7d7b492,0000000..5e5e3f5
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java
@@@ -1,44 -1,0 +1,45 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.cql3.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.List;
 +
++import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.composites.Composite;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +/**
 + * A set of restrictions on a primary key part (partition key or clustering key).
 + *
 + */
 +interface PrimaryKeyRestrictions extends Restriction, Restrictions
 +{
 +    @Override
 +    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException;
 +
-     public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException;
++    public List<ByteBuffer> values(CFMetaData cfm, QueryOptions options) throws InvalidRequestException;
 +
-     public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException;
++    public List<Composite> valuesAsComposites(CFMetaData cfm, QueryOptions options) throws InvalidRequestException;
 +
-     public List<ByteBuffer> bounds(Bound b, QueryOptions options) throws InvalidRequestException;
++    public List<ByteBuffer> bounds(CFMetaData cfm, Bound b, QueryOptions options) throws InvalidRequestException;
 +
-     public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException;
++    public List<Composite> boundsAsComposites(CFMetaData cfm, Bound bound, QueryOptions options) throws InvalidRequestException;
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9583b6b3/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
index f0ea6a7,0000000..9df100a
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
@@@ -1,138 -1,0 +1,150 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.cql3.restrictions;
 +
 +import java.util.List;
 +
++import org.apache.cassandra.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.composites.CompositesBuilder;
 +import org.apache.cassandra.db.index.SecondaryIndexManager;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +/**
 + * A restriction/clause on a column.
 + * The goal of this class being to group all conditions for a column in a SELECT.
 + *
 + * <p>Implementation of this class must be immutable. See {@link #mergeWith(Restriction)} for more explanation.</p>
 + */
 +public interface Restriction
 +{
 +    public boolean isOnToken();
 +    public boolean isSlice();
 +    public boolean isEQ();
 +    public boolean isIN();
 +    public boolean isContains();
 +    public boolean isMultiColumn();
 +
 +    /**
 +     * Returns the definition of the first column.
 +     * @return the definition of the first column.
 +     */
 +    public ColumnDefinition getFirstColumn();
 +
 +    /**
 +     * Returns the definition of the last column.
 +     * @return the definition of the last column.
 +     */
 +    public ColumnDefinition getLastColumn();
 +
 +    /**
 +     * Returns the column definitions in position order.
 +     * @return the column definitions in position order.
 +     */
 +    public List<ColumnDefinition> getColumnDefs();
 +
 +    /**
 +     * Return an Iterable over all of the functions (both native and user-defined) used by any component
 +     * of the restriction
 +     * @return functions all functions found (may contain duplicates)
 +     */
 +    public Iterable<Function> getFunctions();
 +
 +    /**
 +     * Checks if the specified bound is set or not.
 +     * @param b the bound type
 +     * @return <code>true</code> if the specified bound is set, <code>false</code> otherwise
 +     */
 +    public boolean hasBound(Bound b);
 +
 +    /**
 +     * Checks if the specified bound is inclusive or not.
 +     * @param b the bound type
 +     * @return <code>true</code> if the specified bound is inclusive, <code>false</code> otherwise
 +     */
 +    public boolean isInclusive(Bound b);
 +
 +    /**
 +     * Merges this restriction with the specified one.
 +     *
 +     * <p>Restriction are immutable. Therefore merging two restrictions result in a new one.
 +     * The reason behind this choice is that it allow a great flexibility in the way the merging can done while
 +     * preventing any side effect.</p>
 +     *
 +     * @param otherRestriction the restriction to merge into this one
 +     * @return the restriction resulting of the merge
 +     * @throws InvalidRequestException if the restrictions cannot be merged
 +     */
 +    public Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException;
 +
 +    /**
 +     * Check if the restriction is on indexed columns.
 +     *
 +     * @param indexManager the index manager
 +     * @return <code>true</code> if the restriction is on indexed columns, <code>false</code>
 +     */
 +    public boolean hasSupportingIndex(SecondaryIndexManager indexManager);
 +
 +    /**
 +     * Adds to the specified list the <code>IndexExpression</code>s corresponding to this <code>Restriction</code>.
 +     *
 +     * @param expressions the list to add the <code>IndexExpression</code>s to
 +     * @param indexManager the secondary index manager
 +     * @param options the query options
 +     * @throws InvalidRequestException if this <code>Restriction</code> cannot be converted into 
 +     * <code>IndexExpression</code>s
 +     */
 +    public void addIndexExpressionTo(List<IndexExpression> expressions,
 +                                     SecondaryIndexManager indexManager,
 +                                     QueryOptions options)
 +                                     throws InvalidRequestException;
 +
 +    /**
 +     * Appends the values of this <code>Restriction</code> to the specified builder.
 +     *
++     * @param cfm the table metadata
 +     * @param builder the <code>CompositesBuilder</code> to append to.
 +     * @param options the query options
 +     * @return the <code>CompositesBuilder</code>
 +     */
-     public CompositesBuilder appendTo(CompositesBuilder builder, QueryOptions options);
++    public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options);
 +
 +    /**
 +     * Appends the values of the <code>Restriction</code> for the specified bound to the specified builder.
 +     *
++     * @param cfm the table metadata
 +     * @param builder the <code>CompositesBuilder</code> to append to.
 +     * @param bound the bound
 +     * @param options the query options
 +     * @return the <code>CompositesBuilder</code>
 +     */
-     public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options);
++    public CompositesBuilder appendBoundTo(CFMetaData cfm, CompositesBuilder builder, Bound bound, QueryOptions options);
++
++    /**
++     * Checks if this restriction will prevent the query to return any rows.
++     *
++     * @param cfm the table metadata
++     * @param options the query options
++     * @return {@code true} if this restriction will prevent the query to return any rows, {@false} otherwise
++     */
++    public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options);
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9583b6b3/src/java/org/apache/cassandra/cql3/restrictions/ReversedPrimaryKeyRestrictions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/ReversedPrimaryKeyRestrictions.java
index 9b33161,0000000..9107acd
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/ReversedPrimaryKeyRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/ReversedPrimaryKeyRestrictions.java
@@@ -1,77 -1,0 +1,78 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.cql3.restrictions;
 +
 +import java.nio.ByteBuffer;
 +import java.util.Collections;
 +import java.util.List;
 +
++import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.cql3.QueryOptions;
 +import org.apache.cassandra.cql3.statements.Bound;
 +import org.apache.cassandra.db.composites.Composite;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +/**
 + * <code>PrimaryKeyRestrictions</code> decorator that reverse the slices.
 + */
 +final class ReversedPrimaryKeyRestrictions extends ForwardingPrimaryKeyRestrictions
 +{
 +    /**
 +     * The decorated restrictions.
 +     */
 +    private PrimaryKeyRestrictions restrictions;
 +
 +    public ReversedPrimaryKeyRestrictions(PrimaryKeyRestrictions restrictions)
 +    {
 +        this.restrictions = restrictions;
 +    }
 +
 +    @Override
 +    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException
 +    {
 +        return new ReversedPrimaryKeyRestrictions(this.restrictions.mergeWith(restriction));
 +    }
 +
 +    @Override
-     public List<ByteBuffer> bounds(Bound bound, QueryOptions options) throws InvalidRequestException
++    public List<ByteBuffer> bounds(CFMetaData cfm, Bound bound, QueryOptions options) throws InvalidRequestException
 +    {
-         List<ByteBuffer> buffers = restrictions.bounds(bound.reverse(), options);
++        List<ByteBuffer> buffers = restrictions.bounds(cfm, bound.reverse(), options);
 +        Collections.reverse(buffers);
 +        return buffers;
 +    }
 +
 +    @Override
-     public List<Composite> boundsAsComposites(Bound bound, QueryOptions options) throws InvalidRequestException
++    public List<Composite> boundsAsComposites(CFMetaData cfm, Bound bound, QueryOptions options) throws InvalidRequestException
 +    {
-         List<Composite> composites = restrictions.boundsAsComposites(bound.reverse(), options);
++        List<Composite> composites = restrictions.boundsAsComposites(cfm, bound.reverse(), options);
 +        Collections.reverse(composites);
 +        return composites;
 +    }
 +
 +    @Override
 +    public boolean isInclusive(Bound bound)
 +    {
 +        return this.restrictions.isInclusive(bound.reverse());
 +    }
 +
 +    @Override
 +    protected PrimaryKeyRestrictions getDelegate()
 +    {
 +        return this.restrictions;
 +    }
 +}