You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2017/08/30 16:38:01 UTC

[01/10] cassandra git commit: Fix race condition in read command serialization

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 d03c046ac -> 7f297bcf8
  refs/heads/cassandra-3.11 6d6081e3c -> 7ad1945ee
  refs/heads/trunk 76efcc699 -> 3e4d000c9


Fix race condition in read command serialization

patch by Aleksey Yeschenko; reviewed by Sam Tunnicliffe for
CASSANDRA-13363


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

Branch: refs/heads/cassandra-3.0
Commit: 7f297bcf8aced983cbc9c4103d0ebefc1789f0dd
Parents: d03c046
Author: Aleksey Yeschenko <al...@yeschenko.com>
Authored: Mon Aug 14 16:43:06 2017 +0100
Committer: Aleksey Yeschenko <al...@yeschenko.com>
Committed: Wed Aug 30 16:16:46 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +-
 .../cql3/statements/SelectStatement.java        |  16 +-
 .../db/AbstractReadCommandBuilder.java          |   2 +-
 .../cassandra/db/PartitionRangeReadCommand.java | 133 +++++++++++---
 .../org/apache/cassandra/db/ReadCommand.java    | 149 ++++++++-------
 .../db/SinglePartitionReadCommand.java          | 180 ++++++++++++++++---
 .../cassandra/index/SecondaryIndexManager.java  |   9 +-
 .../internal/composites/CompositesSearcher.java |   6 +-
 .../index/internal/keys/KeysSearcher.java       |   3 +-
 .../cassandra/service/AbstractReadExecutor.java |   4 +-
 .../service/pager/PartitionRangeQueryPager.java |   8 +-
 .../cassandra/thrift/CassandraServer.java       |  69 ++++---
 test/unit/org/apache/cassandra/Util.java        |  26 +--
 .../apache/cassandra/db/SecondaryIndexTest.java |  10 +-
 .../db/SinglePartitionSliceCommandTest.java     |  45 ++---
 .../cassandra/io/sstable/SSTableReaderTest.java |   2 +-
 16 files changed, 427 insertions(+), 238 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 452dc9b..aca9e1f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
 3.0.15
- * enable segement creation before recovering commitlogs (CASSANDRA-13587)
+ * Fix race condition in read command serialization (CASSANDRA-13363)
+ * Enable segement creation before recovering commitlogs (CASSANDRA-13587)
  * Fix AssertionError in short read protection (CASSANDRA-13747)
  * Don't skip corrupted sstables on startup (CASSANDRA-13620)
  * Fix the merging of cells with different user type versions (CASSANDRA-13776)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index bd377f4..3882a23 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -540,18 +540,10 @@ public class SelectStatement implements CQLStatement
         if (keyBounds == null)
             return ReadQuery.EMPTY;
 
-        PartitionRangeReadCommand command = new PartitionRangeReadCommand(cfm,
-                                                                          nowInSec,
-                                                                          queriedColumns,
-                                                                          rowFilter,
-                                                                          limit,
-                                                                          new DataRange(keyBounds, clusteringIndexFilter),
-                                                                          Optional.empty());
-        // If there's a secondary index that the command can use, have it validate
-        // the request parameters. Note that as a side effect, if a viable Index is
-        // identified by the CFS's index manager, it will be cached in the command
-        // and serialized during distribution to replicas in order to avoid performing
-        // further lookups.
+        PartitionRangeReadCommand command =
+            PartitionRangeReadCommand.create(false, cfm, nowInSec, queriedColumns, rowFilter, limit, new DataRange(keyBounds, clusteringIndexFilter));
+
+        // If there's a secondary index that the command can use, have it validate the request parameters.
         command.maybeValidateIndex();
 
         return command;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java b/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
index afbab74..d219816 100644
--- a/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
+++ b/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
@@ -336,7 +336,7 @@ public abstract class AbstractReadCommandBuilder
             else
                 bounds = new ExcludingBounds<>(start, end);
 
-            return new PartitionRangeReadCommand(cfs.metadata, nowInSeconds, makeColumnFilter(), filter, makeLimits(), new DataRange(bounds, makeFilter()), Optional.empty());
+            return PartitionRangeReadCommand.create(false, cfs.metadata, nowInSeconds, makeColumnFilter(), filter, makeLimits(), new DataRange(bounds, makeFilter()));
         }
 
         static DecoratedKey makeKey(CFMetaData metadata, Object... partitionKey)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index fb2dd0d..9e557e0 100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@ -20,8 +20,8 @@ package org.apache.cassandra.db;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Optional;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.config.CFMetaData;
@@ -59,31 +59,39 @@ public class PartitionRangeReadCommand extends ReadCommand
     private final DataRange dataRange;
     private int oldestUnrepairedTombstone = Integer.MAX_VALUE;
 
-    public PartitionRangeReadCommand(boolean isDigest,
-                                     int digestVersion,
-                                     boolean isForThrift,
-                                     CFMetaData metadata,
-                                     int nowInSec,
-                                     ColumnFilter columnFilter,
-                                     RowFilter rowFilter,
-                                     DataLimits limits,
-                                     DataRange dataRange,
-                                     Optional<IndexMetadata> index)
+    private PartitionRangeReadCommand(boolean isDigest,
+                                      int digestVersion,
+                                      boolean isForThrift,
+                                      CFMetaData metadata,
+                                      int nowInSec,
+                                      ColumnFilter columnFilter,
+                                      RowFilter rowFilter,
+                                      DataLimits limits,
+                                      DataRange dataRange,
+                                      IndexMetadata index)
     {
-        super(Kind.PARTITION_RANGE, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits);
+        super(Kind.PARTITION_RANGE, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
         this.dataRange = dataRange;
-        this.index = index;
     }
 
-    public PartitionRangeReadCommand(CFMetaData metadata,
-                                     int nowInSec,
-                                     ColumnFilter columnFilter,
-                                     RowFilter rowFilter,
-                                     DataLimits limits,
-                                     DataRange dataRange,
-                                     Optional<IndexMetadata> index)
+    public static PartitionRangeReadCommand create(boolean isForThrift,
+                                                   CFMetaData metadata,
+                                                   int nowInSec,
+                                                   ColumnFilter columnFilter,
+                                                   RowFilter rowFilter,
+                                                   DataLimits limits,
+                                                   DataRange dataRange)
     {
-        this(false, 0, false, metadata, nowInSec, columnFilter, rowFilter, limits, dataRange, index);
+        return new PartitionRangeReadCommand(false,
+                                             0,
+                                             isForThrift,
+                                             metadata,
+                                             nowInSec,
+                                             columnFilter,
+                                             rowFilter,
+                                             limits,
+                                             dataRange,
+                                             findIndex(metadata, rowFilter));
     }
 
     /**
@@ -96,13 +104,14 @@ public class PartitionRangeReadCommand extends ReadCommand
      */
     public static PartitionRangeReadCommand allDataRead(CFMetaData metadata, int nowInSec)
     {
-        return new PartitionRangeReadCommand(metadata,
+        return new PartitionRangeReadCommand(false, 0, false,
+                                             metadata,
                                              nowInSec,
                                              ColumnFilter.all(metadata),
                                              RowFilter.NONE,
                                              DataLimits.NONE,
                                              DataRange.allData(metadata.partitioner),
-                                             Optional.empty());
+                                             null);
     }
 
     public DataRange dataRange()
@@ -122,17 +131,72 @@ public class PartitionRangeReadCommand extends ReadCommand
 
     public PartitionRangeReadCommand forSubRange(AbstractBounds<PartitionPosition> range)
     {
-        return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange().forSubRange(range), index);
+        return new PartitionRangeReadCommand(isDigestQuery(),
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             limits(),
+                                             dataRange().forSubRange(range),
+                                             indexMetadata());
     }
 
     public PartitionRangeReadCommand copy()
     {
-        return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange(), index);
+        return new PartitionRangeReadCommand(isDigestQuery(),
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             limits(),
+                                             dataRange(),
+                                             indexMetadata());
+    }
+
+    public PartitionRangeReadCommand copyAsDigestQuery()
+    {
+        return new PartitionRangeReadCommand(true,
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             limits(),
+                                             dataRange(),
+                                             indexMetadata());
+    }
+
+    public PartitionRangeReadCommand withUpdatedDataRange(DataRange newDataRange)
+    {
+        return new PartitionRangeReadCommand(isDigestQuery(),
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             limits(),
+                                             newDataRange,
+                                             indexMetadata());
     }
 
-    public PartitionRangeReadCommand withUpdatedLimit(DataLimits newLimits)
+    public PartitionRangeReadCommand withUpdatedLimitsAndDataRange(DataLimits newLimits, DataRange newDataRange)
     {
-        return new PartitionRangeReadCommand(metadata(), nowInSec(), columnFilter(), rowFilter(), newLimits, dataRange(), index);
+        return new PartitionRangeReadCommand(isDigestQuery(),
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             newLimits,
+                                             newDataRange,
+                                             indexMetadata());
     }
 
     public long getTimeout()
@@ -173,7 +237,8 @@ public class PartitionRangeReadCommand extends ReadCommand
         metric.rangeLatency.addNano(latencyNanos);
     }
 
-    protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadOrderGroup orderGroup)
+    @VisibleForTesting
+    public UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadOrderGroup orderGroup)
     {
         ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange()));
         Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().getKeyValidator()));
@@ -337,7 +402,17 @@ public class PartitionRangeReadCommand extends ReadCommand
 
     private static class Deserializer extends SelectionDeserializer
     {
-        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
+        public ReadCommand deserialize(DataInputPlus in,
+                                       int version,
+                                       boolean isDigest,
+                                       int digestVersion,
+                                       boolean isForThrift,
+                                       CFMetaData metadata,
+                                       int nowInSec,
+                                       ColumnFilter columnFilter,
+                                       RowFilter rowFilter,
+                                       DataLimits limits,
+                                       IndexMetadata index)
         throws IOException
         {
             DataRange range = DataRange.serializer.deserialize(in, version, metadata);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index 76180cc..66985b6 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -22,6 +22,8 @@ import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.function.Predicate;
 
+import javax.annotation.Nullable;
+
 import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -106,24 +108,27 @@ public abstract class ReadCommand implements ReadQuery
     private final RowFilter rowFilter;
     private final DataLimits limits;
 
-    // SecondaryIndexManager will attempt to provide the most selective of any available indexes
-    // during execution. Here we also store an the results of that lookup to repeating it over
-    // the lifetime of the command.
-    protected Optional<IndexMetadata> index = Optional.empty();
-
-    // Flag to indicate whether the index manager has been queried to select an index for this
-    // command. This is necessary as the result of that lookup may be null, in which case we
-    // still don't want to repeat it.
-    private boolean indexManagerQueried = false;
-
-    private boolean isDigestQuery;
+    private final boolean isDigestQuery;
     // if a digest query, the version for which the digest is expected. Ignored if not a digest.
     private int digestVersion;
     private final boolean isForThrift;
 
+    @Nullable
+    private final IndexMetadata index;
+
     protected static abstract class SelectionDeserializer
     {
-        public abstract ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index) throws IOException;
+        public abstract ReadCommand deserialize(DataInputPlus in,
+                                                int version,
+                                                boolean isDigest,
+                                                int digestVersion,
+                                                boolean isForThrift,
+                                                CFMetaData metadata,
+                                                int nowInSec,
+                                                ColumnFilter columnFilter,
+                                                RowFilter rowFilter,
+                                                DataLimits limits,
+                                                IndexMetadata index) throws IOException;
     }
 
     protected enum Kind
@@ -147,7 +152,8 @@ public abstract class ReadCommand implements ReadQuery
                           int nowInSec,
                           ColumnFilter columnFilter,
                           RowFilter rowFilter,
-                          DataLimits limits)
+                          DataLimits limits,
+                          IndexMetadata index)
     {
         this.kind = kind;
         this.isDigestQuery = isDigestQuery;
@@ -158,6 +164,7 @@ public abstract class ReadCommand implements ReadQuery
         this.columnFilter = columnFilter;
         this.rowFilter = rowFilter;
         this.limits = limits;
+        this.index = index;
     }
 
     protected abstract void serializeSelection(DataOutputPlus out, int version) throws IOException;
@@ -253,18 +260,6 @@ public abstract class ReadCommand implements ReadQuery
     }
 
     /**
-     * Sets whether this command should be a digest one or not.
-     *
-     * @param isDigestQuery whether the command should be set as a digest one or not.
-     * @return this read command.
-     */
-    public ReadCommand setIsDigestQuery(boolean isDigestQuery)
-    {
-        this.isDigestQuery = isDigestQuery;
-        return this;
-    }
-
-    /**
      * Sets the digest version, for when digest for that command is requested.
      * <p>
      * Note that we allow setting this independently of setting the command as a digest query as
@@ -291,6 +286,30 @@ public abstract class ReadCommand implements ReadQuery
     }
 
     /**
+     * Index (metadata) chosen for this query. Can be null.
+     *
+     * @return index (metadata) chosen for this query
+     */
+    @Nullable
+    public IndexMetadata indexMetadata()
+    {
+        return index;
+    }
+
+    /**
+     *  Index instance chosen for this query. Can be null.
+     *
+     * @return Index instance chosen for this query. Can be null.
+     */
+    @Nullable
+    public Index index()
+    {
+        return null == index
+             ? null
+             : Keyspace.openAndGetStore(metadata).indexManager.getIndex(index);
+    }
+
+    /**
      * The clustering index filter this command to use for the provided key.
      * <p>
      * Note that that method should only be called on a key actually queried by this command
@@ -310,6 +329,11 @@ public abstract class ReadCommand implements ReadQuery
      */
     public abstract ReadCommand copy();
 
+    /**
+     * Returns a copy of this command with isDigestQuery set to true.
+     */
+    public abstract ReadCommand copyAsDigestQuery();
+
     protected abstract UnfilteredPartitionIterator queryStorage(ColumnFamilyStore cfs, ReadOrderGroup orderGroup);
 
     protected abstract int oldestUnrepairedTombstone();
@@ -321,35 +345,32 @@ public abstract class ReadCommand implements ReadQuery
              : ReadResponse.createDataResponse(iterator, this);
     }
 
-    public long indexSerializedSize(int version)
+    long indexSerializedSize(int version)
     {
-        if (index.isPresent())
-            return IndexMetadata.serializer.serializedSize(index.get(), version);
-        else
-            return 0;
+        return null != index
+             ? IndexMetadata.serializer.serializedSize(index, version)
+             : 0;
     }
 
     public Index getIndex(ColumnFamilyStore cfs)
     {
-        // if we've already consulted the index manager, and it returned a valid index
-        // the result should be cached here.
-        if(index.isPresent())
-            return cfs.indexManager.getIndex(index.get());
-
-        // if no cached index is present, but we've already consulted the index manager
-        // then no registered index is suitable for this command, so just return null.
-        if (indexManagerQueried)
+        return null != index
+             ? cfs.indexManager.getIndex(index)
+             : null;
+    }
+
+    static IndexMetadata findIndex(CFMetaData table, RowFilter rowFilter)
+    {
+        if (table.getIndexes().isEmpty() || rowFilter.isEmpty())
             return null;
 
-        // do the lookup, set the flag to indicate so and cache the result if not null
-        Index selected = cfs.indexManager.getBestIndexFor(this);
-        indexManagerQueried = true;
+        ColumnFamilyStore cfs = Keyspace.openAndGetStore(table);
 
-        if (selected == null)
-            return null;
+        Index index = cfs.indexManager.getBestIndexFor(rowFilter);
 
-        index = Optional.of(selected.getIndexMetadata());
-        return selected;
+        return null != index
+             ? index.getIndexMetadata()
+             : null;
     }
 
     /**
@@ -602,7 +623,7 @@ public abstract class ReadCommand implements ReadQuery
             assert version >= MessagingService.VERSION_30;
 
             out.writeByte(command.kind.ordinal());
-            out.writeByte(digestFlag(command.isDigestQuery()) | thriftFlag(command.isForThrift()) | indexFlag(command.index.isPresent()));
+            out.writeByte(digestFlag(command.isDigestQuery()) | thriftFlag(command.isForThrift()) | indexFlag(null != command.index));
             if (command.isDigestQuery())
                 out.writeUnsignedVInt(command.digestVersion());
             CFMetaData.serializer.serialize(command.metadata(), out, version);
@@ -610,8 +631,8 @@ public abstract class ReadCommand implements ReadQuery
             ColumnFilter.serializer.serialize(command.columnFilter(), out, version);
             RowFilter.serializer.serialize(command.rowFilter(), out, version);
             DataLimits.serializer.serialize(command.limits(), out, version);
-            if (command.index.isPresent())
-                IndexMetadata.serializer.serialize(command.index.get(), out, version);
+            if (null != command.index)
+                IndexMetadata.serializer.serialize(command.index, out, version);
 
             command.serializeSelection(out, version);
         }
@@ -631,18 +652,16 @@ public abstract class ReadCommand implements ReadQuery
             ColumnFilter columnFilter = ColumnFilter.serializer.deserialize(in, version, metadata);
             RowFilter rowFilter = RowFilter.serializer.deserialize(in, version, metadata);
             DataLimits limits = DataLimits.serializer.deserialize(in, version);
-            Optional<IndexMetadata> index = hasIndex
-                                            ? deserializeIndexMetadata(in, version, metadata)
-                                            : Optional.empty();
+            IndexMetadata index = hasIndex ? deserializeIndexMetadata(in, version, metadata) : null;
 
             return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
         }
 
-        private Optional<IndexMetadata> deserializeIndexMetadata(DataInputPlus in, int version, CFMetaData cfm) throws IOException
+        private IndexMetadata deserializeIndexMetadata(DataInputPlus in, int version, CFMetaData cfm) throws IOException
         {
             try
             {
-                return Optional.of(IndexMetadata.serializer.deserialize(in, version, cfm));
+                return IndexMetadata.serializer.deserialize(in, version, cfm);
             }
             catch (UnknownIndexException e)
             {
@@ -652,7 +671,7 @@ public abstract class ReadCommand implements ReadQuery
                                                "index. Please wait for schema agreement after index creation.",
                                                cfm.ksName, cfm.cfName, e.indexId.toString());
                 logger.info(message);
-                return Optional.empty();
+                return null;
             }
         }
 
@@ -830,7 +849,7 @@ public abstract class ReadCommand implements ReadQuery
             else
                 limits = DataLimits.cqlLimits(maxResults);
 
-            return new PartitionRangeReadCommand(false, 0, true, metadata, nowInSec, selection, rowFilter, limits, new DataRange(keyRange, filter), Optional.empty());
+            return PartitionRangeReadCommand.create(true, metadata, nowInSec, selection, rowFilter, limits, new DataRange(keyRange, filter));
         }
 
         static void serializeRowFilter(DataOutputPlus out, RowFilter rowFilter) throws IOException
@@ -939,9 +958,8 @@ public abstract class ReadCommand implements ReadQuery
             ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter) command.dataRange().clusteringIndexFilter;
             ClusteringIndexSliceFilter sliceFilter = LegacyReadCommandSerializer.convertNamesFilterToSliceFilter(filter, metadata);
             DataRange newRange = new DataRange(command.dataRange().keyRange(), sliceFilter);
-            return new PartitionRangeReadCommand(
-                    command.isDigestQuery(), command.digestVersion(), command.isForThrift(), metadata, command.nowInSec(),
-                    command.columnFilter(), command.rowFilter(), command.limits(), newRange, Optional.empty());
+
+            return command.withUpdatedDataRange(newRange);
         }
 
         static ColumnFilter getColumnSelectionForSlice(boolean selectsStatics, int compositesToGroup, CFMetaData metadata)
@@ -1096,7 +1114,7 @@ public abstract class ReadCommand implements ReadQuery
                 // missing without any problems, so we can safely always set "inclusive" to false in the data range
                 dataRange = dataRange.forPaging(keyRange, metadata.comparator, startBound.getAsClustering(metadata), false);
             }
-            return new PartitionRangeReadCommand(false, 0, true, metadata, nowInSec, selection, rowFilter, limits, dataRange, Optional.empty());
+            return PartitionRangeReadCommand.create(true, metadata, nowInSec, selection, rowFilter, limits, dataRange);
         }
 
         public long serializedSize(ReadCommand command, int version)
@@ -1290,10 +1308,7 @@ public abstract class ReadCommand implements ReadQuery
         {
             Pair<ColumnFilter, ClusteringIndexNamesFilter> selectionAndFilter = deserializeNamesSelectionAndFilter(in, metadata);
 
-            // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
-            return new SinglePartitionReadCommand(
-                    isDigest, version, true, metadata, nowInSeconds, selectionAndFilter.left, RowFilter.NONE, DataLimits.NONE,
-                    key, selectionAndFilter.right);
+            return SinglePartitionReadCommand.legacyNamesCommand(isDigest, version, metadata, nowInSeconds, selectionAndFilter.left, key, selectionAndFilter.right);
         }
 
         static Pair<ColumnFilter, ClusteringIndexNamesFilter> deserializeNamesSelectionAndFilter(DataInputPlus in, CFMetaData metadata) throws IOException
@@ -1422,8 +1437,7 @@ public abstract class ReadCommand implements ReadQuery
             else
                 limits = DataLimits.cqlLimits(count);
 
-            // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
-            return new SinglePartitionReadCommand(isDigest, version, true, metadata, nowInSeconds, columnFilter, RowFilter.NONE, limits, key, filter);
+            return SinglePartitionReadCommand.legacySliceCommand(isDigest, version, metadata, nowInSeconds, columnFilter, limits, key, filter);
         }
 
         private long serializedSliceCommandSize(SinglePartitionReadCommand command)
@@ -1605,9 +1619,8 @@ public abstract class ReadCommand implements ReadQuery
 
             ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter)command.clusteringIndexFilter();
             ClusteringIndexSliceFilter sliceFilter = convertNamesFilterToSliceFilter(filter, metadata);
-            return new SinglePartitionReadCommand(
-                    command.isDigestQuery(), command.digestVersion(), command.isForThrift(), metadata, command.nowInSec(),
-                    command.columnFilter(), command.rowFilter(), command.limits(), command.partitionKey(), sliceFilter);
+
+            return command.withUpdatedClusteringIndexFilter(sliceFilter);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 686ec35..00464ca 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -70,18 +70,19 @@ public class SinglePartitionReadCommand extends ReadCommand
 
     private int oldestUnrepairedTombstone = Integer.MAX_VALUE;
 
-    public SinglePartitionReadCommand(boolean isDigest,
-                                      int digestVersion,
-                                      boolean isForThrift,
-                                      CFMetaData metadata,
-                                      int nowInSec,
-                                      ColumnFilter columnFilter,
-                                      RowFilter rowFilter,
-                                      DataLimits limits,
-                                      DecoratedKey partitionKey,
-                                      ClusteringIndexFilter clusteringIndexFilter)
+    private SinglePartitionReadCommand(boolean isDigest,
+                                       int digestVersion,
+                                       boolean isForThrift,
+                                       CFMetaData metadata,
+                                       int nowInSec,
+                                       ColumnFilter columnFilter,
+                                       RowFilter rowFilter,
+                                       DataLimits limits,
+                                       DecoratedKey partitionKey,
+                                       ClusteringIndexFilter clusteringIndexFilter,
+                                       IndexMetadata index)
     {
-        super(Kind.SINGLE_PARTITION, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits);
+        super(Kind.SINGLE_PARTITION, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
         assert partitionKey.getPartitioner() == metadata.partitioner;
         this.partitionKey = partitionKey;
         this.clusteringIndexFilter = clusteringIndexFilter;
@@ -90,6 +91,44 @@ public class SinglePartitionReadCommand extends ReadCommand
     /**
      * Creates a new read command on a single partition.
      *
+     * @param isForThrift whether the query is for thrift or not.
+     * @param metadata the table to query.
+     * @param nowInSec the time in seconds to use are "now" for this query.
+     * @param columnFilter the column filter to use for the query.
+     * @param rowFilter the row filter to use for the query.
+     * @param limits the limits to use for the query.
+     * @param partitionKey the partition key for the partition to query.
+     * @param clusteringIndexFilter the clustering index filter to use for the query.
+     * @param indexMetadata explicitly specified index to use for the query
+     *
+     * @return a newly created read command.
+     */
+    public static SinglePartitionReadCommand create(boolean isForThrift,
+                                                    CFMetaData metadata,
+                                                    int nowInSec,
+                                                    ColumnFilter columnFilter,
+                                                    RowFilter rowFilter,
+                                                    DataLimits limits,
+                                                    DecoratedKey partitionKey,
+                                                    ClusteringIndexFilter clusteringIndexFilter,
+                                                    IndexMetadata indexMetadata)
+    {
+        return new SinglePartitionReadCommand(false,
+                                              0,
+                                              isForThrift,
+                                              metadata,
+                                              nowInSec,
+                                              columnFilter,
+                                              rowFilter,
+                                              limits,
+                                              partitionKey,
+                                              clusteringIndexFilter,
+                                              indexMetadata);
+    }
+
+    /**
+     * Creates a new read command on a single partition.
+     *
      * @param metadata the table to query.
      * @param nowInSec the time in seconds to use are "now" for this query.
      * @param columnFilter the column filter to use for the query.
@@ -112,7 +151,7 @@ public class SinglePartitionReadCommand extends ReadCommand
     }
 
     /**
-     * Creates a new read command on a single partition for thrift.
+     * Creates a new read command on a single partition.
      *
      * @param isForThrift whether the query is for thrift or not.
      * @param metadata the table to query.
@@ -134,7 +173,15 @@ public class SinglePartitionReadCommand extends ReadCommand
                                                     DecoratedKey partitionKey,
                                                     ClusteringIndexFilter clusteringIndexFilter)
     {
-        return new SinglePartitionReadCommand(false, 0, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, partitionKey, clusteringIndexFilter);
+        return create(isForThrift,
+                      metadata,
+                      nowInSec,
+                      columnFilter,
+                      rowFilter,
+                      limits,
+                      partitionKey,
+                      clusteringIndexFilter,
+                      findIndex(metadata, rowFilter));
     }
 
     /**
@@ -148,7 +195,11 @@ public class SinglePartitionReadCommand extends ReadCommand
      *
      * @return a newly created read command. The returned command will use no row filter and have no limits.
      */
-    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, ColumnFilter columnFilter, ClusteringIndexFilter filter)
+    public static SinglePartitionReadCommand create(CFMetaData metadata,
+                                                    int nowInSec,
+                                                    DecoratedKey key,
+                                                    ColumnFilter columnFilter,
+                                                    ClusteringIndexFilter filter)
     {
         return create(metadata, nowInSec, columnFilter, RowFilter.NONE, DataLimits.NONE, key, filter);
     }
@@ -164,7 +215,7 @@ public class SinglePartitionReadCommand extends ReadCommand
      */
     public static SinglePartitionReadCommand fullPartitionRead(CFMetaData metadata, int nowInSec, DecoratedKey key)
     {
-        return SinglePartitionReadCommand.create(metadata, nowInSec, key, Slices.ALL);
+        return create(metadata, nowInSec, key, Slices.ALL);
     }
 
     /**
@@ -178,7 +229,7 @@ public class SinglePartitionReadCommand extends ReadCommand
      */
     public static SinglePartitionReadCommand fullPartitionRead(CFMetaData metadata, int nowInSec, ByteBuffer key)
     {
-        return SinglePartitionReadCommand.create(metadata, nowInSec, metadata.decorateKey(key), Slices.ALL);
+        return create(metadata, nowInSec, metadata.decorateKey(key), Slices.ALL);
     }
 
     /**
@@ -211,7 +262,7 @@ public class SinglePartitionReadCommand extends ReadCommand
     public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, Slices slices)
     {
         ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(slices, false);
-        return SinglePartitionReadCommand.create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
+        return create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
     }
 
     /**
@@ -244,7 +295,7 @@ public class SinglePartitionReadCommand extends ReadCommand
     public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, NavigableSet<Clustering> names)
     {
         ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(names, false);
-        return SinglePartitionReadCommand.create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
+        return create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
     }
 
     /**
@@ -265,7 +316,82 @@ public class SinglePartitionReadCommand extends ReadCommand
 
     public SinglePartitionReadCommand copy()
     {
-        return new SinglePartitionReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), partitionKey(), clusteringIndexFilter());
+        return new SinglePartitionReadCommand(isDigestQuery(),
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              partitionKey(),
+                                              clusteringIndexFilter(),
+                                              indexMetadata());
+    }
+
+    public SinglePartitionReadCommand copyAsDigestQuery()
+    {
+        return new SinglePartitionReadCommand(true,
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              partitionKey(),
+                                              clusteringIndexFilter(),
+                                              indexMetadata());
+    }
+
+    public SinglePartitionReadCommand withUpdatedClusteringIndexFilter(ClusteringIndexFilter filter)
+    {
+        return new SinglePartitionReadCommand(isDigestQuery(),
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              partitionKey(),
+                                              filter,
+                                              indexMetadata());
+    }
+
+    static SinglePartitionReadCommand legacySliceCommand(boolean isDigest,
+                                                         int digestVersion,
+                                                         CFMetaData metadata,
+                                                         int nowInSec,
+                                                         ColumnFilter columnFilter,
+                                                         DataLimits limits,
+                                                         DecoratedKey partitionKey,
+                                                         ClusteringIndexSliceFilter filter)
+    {
+        // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
+        return new SinglePartitionReadCommand(isDigest,
+                                              digestVersion,
+                                              true,
+                                              metadata,
+                                              nowInSec,
+                                              columnFilter,
+                                              RowFilter.NONE,
+                                              limits,
+                                              partitionKey,
+                                              filter,
+                                              null);
+    }
+
+    static SinglePartitionReadCommand legacyNamesCommand(boolean isDigest,
+                                                         int digestVersion,
+                                                         CFMetaData metadata,
+                                                         int nowInSec,
+                                                         ColumnFilter columnFilter,
+                                                         DecoratedKey partitionKey,
+                                                         ClusteringIndexNamesFilter filter)
+    {
+        // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
+        return new SinglePartitionReadCommand(isDigest, digestVersion, true, metadata, nowInSec, columnFilter, RowFilter.NONE, DataLimits.NONE, partitionKey, filter,null);
     }
 
     public DecoratedKey partitionKey()
@@ -432,7 +558,7 @@ public class SinglePartitionReadCommand extends ReadCommand
                 final int rowsToCache = metadata().params.caching.rowsPerPartitionToCache();
 
                 @SuppressWarnings("resource") // we close on exception or upon closing the result of this method
-                UnfilteredRowIterator iter = SinglePartitionReadCommand.fullPartitionRead(metadata(), nowInSec(), partitionKey()).queryMemtableAndDisk(cfs, readOp);
+                UnfilteredRowIterator iter = fullPartitionRead(metadata(), nowInSec(), partitionKey()).queryMemtableAndDisk(cfs, readOp);
                 try
                 {
                     // Use a custom iterator instead of DataLimits to avoid stopping the original iterator
@@ -1068,12 +1194,22 @@ public class SinglePartitionReadCommand extends ReadCommand
 
     private static class Deserializer extends SelectionDeserializer
     {
-        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
+        public ReadCommand deserialize(DataInputPlus in,
+                                       int version,
+                                       boolean isDigest,
+                                       int digestVersion,
+                                       boolean isForThrift,
+                                       CFMetaData metadata,
+                                       int nowInSec,
+                                       ColumnFilter columnFilter,
+                                       RowFilter rowFilter,
+                                       DataLimits limits,
+                                       IndexMetadata index)
         throws IOException
         {
             DecoratedKey key = metadata.decorateKey(metadata.getKeyValidator().readValue(in, DatabaseDescriptor.getMaxValueSize()));
             ClusteringIndexFilter filter = ClusteringIndexFilter.serializer.deserialize(in, version, metadata);
-            return new SinglePartitionReadCommand(isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter);
+            return new SinglePartitionReadCommand(isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter, index);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index dd6dde4..5976ddf 100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@ -40,6 +40,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.concurrent.StageManager;
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.statements.IndexTarget;
@@ -697,17 +698,17 @@ public class SecondaryIndexManager implements IndexRegistry
      * cached for future use when obtaining a Searcher, getting the index's underlying CFS for
      * ReadOrderGroup, or an estimate of the result size from an average index query.
      *
-     * @param command ReadCommand to be executed
+     * @param rowFilter RowFilter of the command to be executed
      * @return an Index instance, ready to use during execution of the command, or null if none
      * of the registered indexes can support the command.
      */
-    public Index getBestIndexFor(ReadCommand command)
+    public Index getBestIndexFor(RowFilter rowFilter)
     {
-        if (indexes.isEmpty() || command.rowFilter().isEmpty())
+        if (indexes.isEmpty() || rowFilter.isEmpty())
             return null;
 
         Set<Index> searchableIndexes = new HashSet<>();
-        for (RowFilter.Expression expression : command.rowFilter())
+        for (RowFilter.Expression expression : rowFilter)
         {
             if (expression.isCustom())
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
index 135839b..f8a7c66 100644
--- a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
@@ -136,13 +136,15 @@ public class CompositesSearcher extends CassandraIndexSearcher
 
                     // Query the gathered index hits. We still need to filter stale hits from the resulting query.
                     ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(clusterings.build(), false);
-                    SinglePartitionReadCommand dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata,
+                    SinglePartitionReadCommand dataCmd = SinglePartitionReadCommand.create(isForThrift(),
+                                                                                           index.baseCfs.metadata,
                                                                                            command.nowInSec(),
                                                                                            command.columnFilter(),
                                                                                            command.rowFilter(),
                                                                                            DataLimits.NONE,
                                                                                            partitionKey,
-                                                                                           filter);
+                                                                                           filter,
+                                                                                           null);
                     @SuppressWarnings("resource") // We close right away if empty, and if it's assign to next it will be called either
                     // by the next caller of next, or through closing this iterator is this come before.
                     UnfilteredRowIterator dataIter =

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
index 189b652..c14c5a7 100644
--- a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
@@ -99,7 +99,8 @@ public class KeysSearcher extends CassandraIndexSearcher
                                                                                            command.rowFilter(),
                                                                                            DataLimits.NONE,
                                                                                            key,
-                                                                                           command.clusteringIndexFilter(key));
+                                                                                           command.clusteringIndexFilter(key),
+                                                                                           null);
 
                     @SuppressWarnings("resource") // filterIfStale closes it's iterator if either it materialize it or if it returns null.
                                                   // Otherwise, we close right away if empty, and if it's assigned to next it will be called either

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
index cae1f1a..177fdb2 100644
--- a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
+++ b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
@@ -88,7 +88,7 @@ public abstract class AbstractReadExecutor
 
     protected void makeDigestRequests(Iterable<InetAddress> endpoints)
     {
-        makeRequests(command.copy().setIsDigestQuery(true), endpoints);
+        makeRequests(command.copyAsDigestQuery(), endpoints);
     }
 
     private void makeRequests(ReadCommand readCommand, Iterable<InetAddress> endpoints)
@@ -284,7 +284,7 @@ public abstract class AbstractReadExecutor
                 // Could be waiting on the data, or on enough digests.
                 ReadCommand retryCommand = command;
                 if (handler.resolver.isDataPresent())
-                    retryCommand = command.copy().setIsDigestQuery(true);
+                    retryCommand = command.copyAsDigestQuery();
 
                 InetAddress extraReplica = Iterables.getLast(targetReplicas);
                 if (traceState != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java b/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
index 9c216e3..ea79017 100644
--- a/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.service.pager;
 
-import java.util.Optional;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -27,8 +25,6 @@ import org.apache.cassandra.db.filter.DataLimits;
 import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.index.Index;
-import org.apache.cassandra.schema.IndexMetadata;
 
 /**
  * Pages a PartitionRangeReadCommand.
@@ -90,9 +86,7 @@ public class PartitionRangeQueryPager extends AbstractQueryPager
             }
         }
 
-        Index index = command.getIndex(Keyspace.openAndGetStore(command.metadata()));
-        Optional<IndexMetadata> indexMetadata = index != null ? Optional.of(index.getIndexMetadata()) : Optional.empty();
-        return new PartitionRangeReadCommand(command.metadata(), command.nowInSec(), command.columnFilter(), command.rowFilter(), limits, pageRange, indexMetadata);
+        return ((PartitionRangeReadCommand) command).withUpdatedLimitsAndDataRange(limits, pageRange);
     }
 
     protected void recordLast(DecoratedKey key, Row last)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 86caac3..cb74b15 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -48,7 +48,6 @@ import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.index.Index;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.locator.DynamicEndpointSnitch;
 import org.apache.cassandra.metrics.ClientMetrics;
@@ -1520,16 +1519,16 @@ public class CassandraServer implements Cassandra.Iface
                 ColumnFilter columns = makeColumnFilter(metadata, column_parent, predicate);
                 ClusteringIndexFilter filter = toInternalFilter(metadata, column_parent, predicate);
                 DataLimits limits = getLimits(range.count, metadata.isSuper() && !column_parent.isSetSuper_column(), predicate);
-                PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                              0,
-                                                                              true,
-                                                                              metadata,
-                                                                              nowInSec,
-                                                                              columns,
-                                                                              ThriftConversion.rowFilterFromThrift(metadata, range.row_filter),
-                                                                              limits,
-                                                                              new DataRange(bounds, filter),
-                                                                              Optional.empty());
+
+                PartitionRangeReadCommand cmd =
+                    PartitionRangeReadCommand.create(true,
+                                                     metadata,
+                                                     nowInSec,
+                                                     columns,
+                                                     ThriftConversion.rowFilterFromThrift(metadata, range.row_filter),
+                                                     limits,
+                                                     new DataRange(bounds, filter));
+
                 try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
                 {
                     assert results != null;
@@ -1614,16 +1613,16 @@ public class CassandraServer implements Cassandra.Iface
                 Clustering pageFrom = metadata.isSuper()
                                     ? new Clustering(start_column)
                                     : LegacyLayout.decodeCellName(metadata, start_column).clustering;
-                PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                              0,
-                                                                              true,
-                                                                              metadata,
-                                                                              nowInSec,
-                                                                              ColumnFilter.all(metadata),
-                                                                              RowFilter.NONE,
-                                                                              limits,
-                                                                              new DataRange(bounds, filter).forPaging(bounds, metadata.comparator, pageFrom, true),
-                                                                              Optional.empty());
+
+                PartitionRangeReadCommand cmd =
+                    PartitionRangeReadCommand.create(true,
+                                                     metadata,
+                                                     nowInSec,
+                                                     ColumnFilter.all(metadata),
+                                                     RowFilter.NONE,
+                                                     limits,
+                                                     new DataRange(bounds, filter).forPaging(bounds, metadata.comparator, pageFrom, true));
+
                 try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
                 {
                     return thriftifyKeySlices(results, new ColumnParent(column_family), limits.perPartitionCount());
@@ -1706,21 +1705,17 @@ public class CassandraServer implements Cassandra.Iface
             ColumnFilter columns = makeColumnFilter(metadata, column_parent, column_predicate);
             ClusteringIndexFilter filter = toInternalFilter(metadata, column_parent, column_predicate);
             DataLimits limits = getLimits(index_clause.count, metadata.isSuper() && !column_parent.isSetSuper_column(), column_predicate);
-            PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                          0,
-                                                                          true,
-                                                                          metadata,
-                                                                          nowInSec,
-                                                                          columns,
-                                                                          ThriftConversion.rowFilterFromThrift(metadata, index_clause.expressions),
-                                                                          limits,
-                                                                          new DataRange(bounds, filter),
-                                                                          Optional.empty());
-            // If there's a secondary index that the command can use, have it validate
-            // the request parameters. Note that as a side effect, if a viable Index is
-            // identified by the CFS's index manager, it will be cached in the command
-            // and serialized during distribution to replicas in order to avoid performing
-            // further lookups.
+
+            PartitionRangeReadCommand cmd =
+                PartitionRangeReadCommand.create(true,
+                                                 metadata,
+                                                 nowInSec,
+                                                 columns,
+                                                 ThriftConversion.rowFilterFromThrift(metadata, index_clause.expressions),
+                                                 limits,
+                                                 new DataRange(bounds, filter));
+
+            // If there's a secondary index that the command can use, have it validate the request parameters.
             cmd.maybeValidateIndex();
 
             try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
@@ -2533,7 +2528,7 @@ public class CassandraServer implements Cassandra.Iface
                 // We want to know if the partition exists, so just fetch a single cell.
                 ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(Slices.ALL, false);
                 DataLimits limits = DataLimits.thriftLimits(1, 1);
-                return new SinglePartitionReadCommand(false, 0, true, metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, limits, key, filter);
+                return SinglePartitionReadCommand.create(true, metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, limits, key, filter);
             }
 
             // Gather the clustering for the expected values and query those.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index e8b42bc..d758efe 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -629,31 +629,7 @@ public class Util
                                                              ColumnFamilyStore cfs,
                                                              ReadOrderGroup orderGroup)
     {
-        return new InternalPartitionRangeReadCommand(command).queryStorageInternal(cfs, orderGroup);
-    }
-
-    private static final class InternalPartitionRangeReadCommand extends PartitionRangeReadCommand
-    {
-
-        private InternalPartitionRangeReadCommand(PartitionRangeReadCommand original)
-        {
-            super(original.isDigestQuery(),
-                  original.digestVersion(),
-                  original.isForThrift(),
-                  original.metadata(),
-                  original.nowInSec(),
-                  original.columnFilter(),
-                  original.rowFilter(),
-                  original.limits(),
-                  original.dataRange(),
-                  Optional.empty());
-        }
-
-        private UnfilteredPartitionIterator queryStorageInternal(ColumnFamilyStore cfs,
-                                                                 ReadOrderGroup orderGroup)
-        {
-            return queryStorage(cfs, orderGroup);
-        }
+        return command.queryStorage(cfs, orderGroup);
     }
 
     public static Closeable markDirectoriesUnwriteable(ColumnFamilyStore cfs)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
index bbccc48..2457c4a 100644
--- a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
@@ -118,7 +118,7 @@ public class SecondaryIndexTest
                                       .filterOn("birthdate", Operator.EQ, 1L)
                                       .build();
 
-        Index.Searcher searcher = cfs.indexManager.getBestIndexFor(rc).searcherFor(rc);
+        Index.Searcher searcher = rc.index().searcherFor(rc);
         try (ReadOrderGroup orderGroup = rc.startOrderGroup(); UnfilteredPartitionIterator pi = searcher.search(orderGroup))
         {
             assertTrue(pi.hasNext());
@@ -204,7 +204,7 @@ public class SecondaryIndexTest
 
         // verify that it's not being indexed under any other value either
         ReadCommand rc = Util.cmd(cfs).build();
-        assertNull(cfs.indexManager.getBestIndexFor(rc));
+        assertNull(rc.index());
 
         // resurrect w/ a newer timestamp
         new RowUpdateBuilder(cfs.metadata, 2, "k1").clustering("c").add("birthdate", 1L).build().apply();;
@@ -222,13 +222,13 @@ public class SecondaryIndexTest
         // todo - checking the # of index searchers for the command is probably not the best thing to test here
         RowUpdateBuilder.deleteRow(cfs.metadata, 3, "k1", "c").applyUnsafe();
         rc = Util.cmd(cfs).build();
-        assertNull(cfs.indexManager.getBestIndexFor(rc));
+        assertNull(rc.index());
 
         // make sure obsolete mutations don't generate an index entry
         // todo - checking the # of index searchers for the command is probably not the best thing to test here
         new RowUpdateBuilder(cfs.metadata, 3, "k1").clustering("c").add("birthdate", 1L).build().apply();;
         rc = Util.cmd(cfs).build();
-        assertNull(cfs.indexManager.getBestIndexFor(rc));
+        assertNull(rc.index());
     }
 
     @Test
@@ -504,7 +504,7 @@ public class SecondaryIndexTest
         ColumnDefinition cdef = cfs.metadata.getColumnDefinition(col);
 
         ReadCommand rc = Util.cmd(cfs).filterOn(cdef.name.toString(), Operator.EQ, ((AbstractType) cdef.cellValueType()).decompose(val)).build();
-        Index.Searcher searcher = cfs.indexManager.getBestIndexFor(rc).searcherFor(rc);
+        Index.Searcher searcher = rc.index().searcherFor(rc);
         if (count != 0)
             assertNotNull(searcher);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
index 7f59e2f..02b642e 100644
--- a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
+++ b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
@@ -116,13 +116,14 @@ public class SinglePartitionSliceCommandTest
         ByteBuffer zero = ByteBufferUtil.bytes(0);
         Slices slices = Slices.with(cfm.comparator, Slice.make(Slice.Bound.inclusiveStartOf(zero), Slice.Bound.inclusiveEndOf(zero)));
         ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(slices, false);
-        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                          FBUtilities.nowInSeconds(),
-                                                          columnFilter,
-                                                          RowFilter.NONE,
-                                                          DataLimits.NONE,
-                                                          key,
-                                                          sliceFilter);
+        ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                            cfm,
+                                                            FBUtilities.nowInSeconds(),
+                                                            columnFilter,
+                                                            RowFilter.NONE,
+                                                            DataLimits.NONE,
+                                                            key,
+                                                            sliceFilter);
 
         DataOutputBuffer out = new DataOutputBuffer((int) ReadCommand.legacyReadCommandSerializer.serializedSize(cmd, MessagingService.VERSION_21));
         ReadCommand.legacyReadCommandSerializer.serialize(cmd, out, MessagingService.VERSION_21);
@@ -166,13 +167,14 @@ public class SinglePartitionSliceCommandTest
 
         ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
         ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.NONE, false);
-        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                         FBUtilities.nowInSeconds(),
-                                                         columnFilter,
-                                                         RowFilter.NONE,
-                                                         DataLimits.NONE,
-                                                         key,
-                                                         sliceFilter);
+        ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                            cfm,
+                                                            FBUtilities.nowInSeconds(),
+                                                            columnFilter,
+                                                            RowFilter.NONE,
+                                                            DataLimits.NONE,
+                                                            key,
+                                                            sliceFilter);
 
         // check raw iterator for static cell
         try (ReadOrderGroup orderGroup = cmd.startOrderGroup(); UnfilteredPartitionIterator pi = cmd.executeLocally(orderGroup))
@@ -224,13 +226,14 @@ public class SinglePartitionSliceCommandTest
         ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
         Slice slice = Slice.make(Slice.Bound.BOTTOM, Slice.Bound.inclusiveEndOf(ByteBufferUtil.bytes("i1")));
         ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.with(cfm.comparator, slice), false);
-        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                         FBUtilities.nowInSeconds(),
-                                                         columnFilter,
-                                                         RowFilter.NONE,
-                                                         DataLimits.NONE,
-                                                         key,
-                                                         sliceFilter);
+        ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                            cfm,
+                                                            FBUtilities.nowInSeconds(),
+                                                            columnFilter,
+                                                            RowFilter.NONE,
+                                                            DataLimits.NONE,
+                                                            key,
+                                                            sliceFilter);
 
         String ret = cmd.toCQLString();
         Assert.assertNotNull(ret);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 640b68b..c2598ec 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -604,7 +604,7 @@ public class SSTableReaderTest
                                              .columns("birthdate")
                                              .filterOn("birthdate", Operator.EQ, 1L)
                                              .build();
-        Index.Searcher searcher = indexedCFS.indexManager.getBestIndexFor(rc).searcherFor(rc);
+        Index.Searcher searcher = rc.index().searcherFor(rc);
         assertNotNull(searcher);
         try (ReadOrderGroup orderGroup = ReadOrderGroup.forCommand(rc))
         {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[05/10] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
index e9051b4,0000000..03d89e1
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
@@@ -1,2566 -1,0 +1,2568 @@@
 +/*
 + * 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.index.sasi;
 +
 +import java.io.FileWriter;
 +import java.io.Writer;
 +import java.nio.ByteBuffer;
 +import java.nio.file.FileSystems;
 +import java.nio.file.Files;
 +import java.nio.file.Path;
 +import java.nio.file.attribute.BasicFileAttributes;
 +import java.util.*;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.Executors;
 +import java.util.concurrent.ThreadLocalRandom;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.index.Index;
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.Term;
 +import org.apache.cassandra.cql3.statements.IndexTarget;
 +import org.apache.cassandra.cql3.statements.SelectStatement;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.filter.ColumnFilter;
 +import org.apache.cassandra.db.filter.DataLimits;
 +import org.apache.cassandra.db.filter.RowFilter;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.db.marshal.*;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.dht.IPartitioner;
 +import org.apache.cassandra.dht.Murmur3Partitioner;
 +import org.apache.cassandra.dht.Range;
 +import org.apache.cassandra.exceptions.ConfigurationException;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.index.sasi.conf.ColumnIndex;
 +import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
 +import org.apache.cassandra.index.sasi.exceptions.TimeQuotaExceededException;
 +import org.apache.cassandra.index.sasi.memory.IndexMemtable;
 +import org.apache.cassandra.index.sasi.plan.QueryController;
 +import org.apache.cassandra.index.sasi.plan.QueryPlan;
 +import org.apache.cassandra.io.sstable.SSTable;
 +import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.schema.KeyspaceMetadata;
 +import org.apache.cassandra.schema.KeyspaceParams;
 +import org.apache.cassandra.schema.Tables;
 +import org.apache.cassandra.serializers.MarshalException;
 +import org.apache.cassandra.serializers.TypeSerializer;
 +import org.apache.cassandra.service.MigrationManager;
 +import org.apache.cassandra.service.QueryState;
 +import org.apache.cassandra.thrift.CqlRow;
 +import org.apache.cassandra.transport.messages.ResultMessage;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.Pair;
 +
 +import com.google.common.collect.Lists;
 +import com.google.common.util.concurrent.Uninterruptibles;
 +
 +import junit.framework.Assert;
 +
 +import org.junit.*;
 +
 +public class SASIIndexTest
 +{
 +    private static final IPartitioner PARTITIONER;
 +
 +    static {
 +        System.setProperty("cassandra.config", "cassandra-murmur.yaml");
 +        PARTITIONER = Murmur3Partitioner.instance;
 +    }
 +
 +    private static final String KS_NAME = "sasi";
 +    private static final String CF_NAME = "test_cf";
 +    private static final String CLUSTERING_CF_NAME_1 = "clustering_test_cf_1";
 +    private static final String CLUSTERING_CF_NAME_2 = "clustering_test_cf_2";
 +    private static final String STATIC_CF_NAME = "static_sasi_test_cf";
 +    private static final String FTS_CF_NAME = "full_text_search_sasi_test_cf";
 +
 +    @BeforeClass
 +    public static void loadSchema() throws ConfigurationException
 +    {
 +        SchemaLoader.loadSchema();
 +        MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(KS_NAME,
 +                                                                     KeyspaceParams.simpleTransient(1),
 +                                                                     Tables.of(SchemaLoader.sasiCFMD(KS_NAME, CF_NAME),
 +                                                                               SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME_1),
 +                                                                               SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME_2, "location"),
 +                                                                               SchemaLoader.staticSASICFMD(KS_NAME, STATIC_CF_NAME),
 +                                                                               SchemaLoader.fullTextSearchSASICFMD(KS_NAME, FTS_CF_NAME))));
 +    }
 +
 +    @Before
 +    public void cleanUp()
 +    {
 +        Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME).truncateBlocking();
 +    }
 +
 +    @Test
 +    public void testSingleExpressionQueries() throws Exception
 +    {
 +        testSingleExpressionQueries(false);
 +        cleanupData();
 +        testSingleExpressionQueries(true);
 +    }
 +
 +    private void testSingleExpressionQueries(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +            put("key1", Pair.create("Pavel", 14));
 +            put("key2", Pair.create("Pavel", 26));
 +            put("key3", Pair.create("Pavel", 27));
 +            put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("av")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("as")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("aw")));
 +        Assert.assertEquals(rows.toString(), 0, rows.size());
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("avel")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("n")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(27)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key3", "key4"}, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(13)));
 +        Assert.assertEquals(rows.toString(), 0, rows.size());
 +    }
 +
 +    @Test
 +    public void testEmptyTokenizedResults() throws Exception
 +    {
 +        testEmptyTokenizedResults(false);
 +        cleanupData();
 +        testEmptyTokenizedResults(true);
 +    }
 +
 +    private void testEmptyTokenizedResults(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("  ", 14));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data, forceFlush);
 +
 +        Set<String> rows= getIndexed(store, 10, buildExpression(UTF8Type.instance.decompose("first_name"), Operator.LIKE_MATCHES, UTF8Type.instance.decompose("doesntmatter")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{}, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testMultiExpressionQueries() throws Exception
 +    {
 +        testMultiExpressionQueries(false);
 +        cleanupData();
 +        testMultiExpressionQueries(true);
 +    }
 +
 +    public void testMultiExpressionQueries(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 26));
 +                put("key3", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows;
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(14)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(27)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key1", "key2"}, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GT, Int32Type.instance.decompose(14)),
 +                         buildExpression(age, Operator.LT, Int32Type.instance.decompose(27)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GT, Int32Type.instance.decompose(12)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GTE, Int32Type.instance.decompose(13)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GTE, Int32Type.instance.decompose(16)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.LT, Int32Type.instance.decompose(30)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.LTE, Int32Type.instance.decompose(29)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.LTE, Int32Type.instance.decompose(25)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("avel")),
 +                                     buildExpression(age, Operator.LTE, Int32Type.instance.decompose(25)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("n")),
 +                                     buildExpression(age, Operator.LTE, Int32Type.instance.decompose(25)));
 +        Assert.assertTrue(rows.isEmpty());
 +
 +    }
 +
 +    @Test
 +    public void testCrossSSTableQueries() throws Exception
 +    {
 +        testCrossSSTableQueries(false);
 +        cleanupData();
 +        testCrossSSTableQueries(true);
 +
 +    }
 +
 +    private void testCrossSSTableQueries(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("Maxie", 43));
 +                put("key1", Pair.create("Chelsie", 33));
 +                put("key2", Pair.create("Josephine", 43));
 +                put("key3", Pair.create("Shanna", 27));
 +                put("key4", Pair.create("Amiya", 36));
 +            }};
 +
 +        loadData(part1, forceFlush); // first sstable
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key5", Pair.create("Americo", 20));
 +                put("key6", Pair.create("Fiona", 39));
 +                put("key7", Pair.create("Francis", 41));
 +                put("key8", Pair.create("Charley", 21));
 +                put("key9", Pair.create("Amely", 40));
 +            }};
 +
 +        loadData(part2, forceFlush);
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key10", Pair.create("Eddie", 42));
 +                put("key11", Pair.create("Oswaldo", 35));
 +                put("key12", Pair.create("Susana", 35));
 +                put("key13", Pair.create("Alivia", 42));
 +                put("key14", Pair.create("Demario", 28));
 +            }};
 +
 +        ColumnFamilyStore store = loadData(part3, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows;
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("Fiona")),
 +                                     buildExpression(age, Operator.LT, Int32Type.instance.decompose(40)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key6" }, rows.toArray(new String[rows.size()])));
 +
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key14",
 +                                                                        "key3", "key4", "key6", "key7", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 5,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertEquals(rows.toString(), 5, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GTE, Int32Type.instance.decompose(35)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key4", "key6", "key7" },
 +                                                         rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key3", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(27)),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(50)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ie")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(43)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key10" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key12", "key13", "key3", "key4", "key6" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(33)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testQueriesThatShouldBeTokenized() throws Exception
 +    {
 +        testQueriesThatShouldBeTokenized(false);
 +        cleanupData();
 +        testQueriesThatShouldBeTokenized(true);
 +    }
 +
 +    private void testQueriesThatShouldBeTokenized(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("If you can dream it, you can do it.", 43));
 +                put("key1", Pair.create("What you get by achieving your goals is not " +
 +                        "as important as what you become by achieving your goals, do it.", 33));
 +                put("key2", Pair.create("Keep your face always toward the sunshine " +
 +                        "- and shadows will fall behind you.", 43));
 +                put("key3", Pair.create("We can't help everyone, but everyone can " +
 +                        "help someone.", 27));
 +            }};
 +
 +        ColumnFamilyStore store = loadData(part1, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows = getIndexed(store, 10,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS,
 +                        UTF8Type.instance.decompose("What you get by achieving your goals")),
 +                buildExpression(age, Operator.GT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertEquals(rows.toString(), Collections.singleton("key1"), rows);
 +
 +        rows = getIndexed(store, 10,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("do it.")));
 +
 +        Assert.assertEquals(rows.toString(), Arrays.asList("key0", "key1"), Lists.newArrayList(rows));
 +    }
 +
 +    @Test
 +    public void testPrefixSearchWithContainsMode() throws Exception
 +    {
 +        testPrefixSearchWithContainsMode(false);
 +        cleanupData();
 +        testPrefixSearchWithContainsMode(true);
 +    }
 +
 +    private void testPrefixSearchWithContainsMode(boolean forceFlush) throws Exception
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(FTS_CF_NAME);
 +
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("1a4abbcd-b5de-4c69-a578-31231e01ff09"), "Poker Face", "Lady Gaga");
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("9472a394-359b-4a06-b1d5-b6afce590598"), "Forgetting the Way Home", "Our Lady of Bells");
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("4f8dc18e-54e6-4e16-b507-c5324b61523b"), "Zamki na piasku", "Lady Pank");
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("eaf294fa-bad5-49d4-8f08-35ba3636a706"), "Koncertowa", "Lady Pank");
 +
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        final UntypedResultSet results = executeCQL(FTS_CF_NAME, "SELECT * FROM %s.%s WHERE artist LIKE 'lady%%'");
 +        Assert.assertNotNull(results);
 +        Assert.assertEquals(3, results.size());
 +    }
 +
 +    @Test
 +    public void testMultiExpressionQueriesWhereRowSplitBetweenSSTables() throws Exception
 +    {
 +        testMultiExpressionQueriesWhereRowSplitBetweenSSTables(false);
 +        cleanupData();
 +        testMultiExpressionQueriesWhereRowSplitBetweenSSTables(true);
 +    }
 +
 +    private void testMultiExpressionQueriesWhereRowSplitBetweenSSTables(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("Maxie", -1));
 +                put("key1", Pair.create("Chelsie", 33));
 +                put("key2", Pair.create((String)null, 43));
 +                put("key3", Pair.create("Shanna", 27));
 +                put("key4", Pair.create("Amiya", 36));
 +        }};
 +
 +        loadData(part1, forceFlush); // first sstable
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key5", Pair.create("Americo", 20));
 +                put("key6", Pair.create("Fiona", 39));
 +                put("key7", Pair.create("Francis", 41));
 +                put("key8", Pair.create("Charley", 21));
 +                put("key9", Pair.create("Amely", 40));
 +                put("key14", Pair.create((String)null, 28));
 +        }};
 +
 +        loadData(part2, forceFlush);
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create((String)null, 43));
 +                put("key10", Pair.create("Eddie", 42));
 +                put("key11", Pair.create("Oswaldo", 35));
 +                put("key12", Pair.create("Susana", 35));
 +                put("key13", Pair.create("Alivia", 42));
 +                put("key14", Pair.create("Demario", -1));
 +                put("key2", Pair.create("Josephine", -1));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part3, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows = getIndexed(store, 10,
 +                                      buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("Fiona")),
 +                                      buildExpression(age, Operator.LT, Int32Type.instance.decompose(40)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key6" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key14",
 +                                                                        "key3", "key4", "key6", "key7", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 5,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertEquals(rows.toString(), 5, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GTE, Int32Type.instance.decompose(35)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key4", "key6", "key7" },
 +                                                         rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key3", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(27)),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
 +
 +        Map<String, Pair<String, Integer>> part4 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key12", Pair.create((String)null, 12));
 +                put("key14", Pair.create("Demario", 42));
 +                put("key2", Pair.create("Frank", -1));
 +        }};
 +
 +        store = loadData(part4, forceFlush);
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("Susana")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(13)),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key12" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("Demario")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(30)));
 +        Assert.assertTrue(rows.toString(), rows.size() == 0);
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("Josephine")));
 +        Assert.assertTrue(rows.toString(), rows.size() == 0);
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(50)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ie")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(43)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key1", "key10" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testPagination() throws Exception
 +    {
 +        testPagination(false);
 +        cleanupData();
 +        testPagination(true);
 +    }
 +
 +    private void testPagination(boolean forceFlush) throws Exception
 +    {
 +        // split data into 3 distinct SSTables to test paging with overlapping token intervals.
 +
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key01", Pair.create("Ali", 33));
 +                put("key02", Pair.create("Jeremy", 41));
 +                put("key03", Pair.create("Elvera", 22));
 +                put("key04", Pair.create("Bailey", 45));
 +                put("key05", Pair.create("Emerson", 32));
 +                put("key06", Pair.create("Kadin", 38));
 +                put("key07", Pair.create("Maggie", 36));
 +                put("key08", Pair.create("Kailey", 36));
 +                put("key09", Pair.create("Armand", 21));
 +                put("key10", Pair.create("Arnold", 35));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key11", Pair.create("Ken", 38));
 +                put("key12", Pair.create("Penelope", 43));
 +                put("key13", Pair.create("Wyatt", 34));
 +                put("key14", Pair.create("Johnpaul", 34));
 +                put("key15", Pair.create("Trycia", 43));
 +                put("key16", Pair.create("Aida", 21));
 +                put("key17", Pair.create("Devon", 42));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key18", Pair.create("Christina", 20));
 +                put("key19", Pair.create("Rick", 19));
 +                put("key20", Pair.create("Fannie", 22));
 +                put("key21", Pair.create("Keegan", 29));
 +                put("key22", Pair.create("Ignatius", 36));
 +                put("key23", Pair.create("Ellis", 26));
 +                put("key24", Pair.create("Annamarie", 29));
 +                put("key25", Pair.create("Tianna", 31));
 +                put("key26", Pair.create("Dennis", 32));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part1, forceFlush);
 +
 +        loadData(part2, forceFlush);
 +        loadData(part3, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<DecoratedKey> uniqueKeys = getPaged(store, 4,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                buildExpression(age, Operator.GTE, Int32Type.instance.decompose(21)));
 +
 +
 +        List<String> expected = new ArrayList<String>()
 +        {{
 +                add("key25");
 +                add("key20");
 +                add("key13");
 +                add("key22");
 +                add("key09");
 +                add("key14");
 +                add("key16");
 +                add("key24");
 +                add("key03");
 +                add("key04");
 +                add("key08");
 +                add("key07");
 +                add("key15");
 +                add("key06");
 +                add("key21");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // now let's test a single equals condition
 +
 +        uniqueKeys = getPaged(store, 4, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        expected = new ArrayList<String>()
 +        {{
 +                add("key25");
 +                add("key20");
 +                add("key13");
 +                add("key22");
 +                add("key09");
 +                add("key14");
 +                add("key16");
 +                add("key24");
 +                add("key03");
 +                add("key04");
 +                add("key18");
 +                add("key08");
 +                add("key07");
 +                add("key15");
 +                add("key06");
 +                add("key21");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // now let's test something which is smaller than a single page
 +        uniqueKeys = getPaged(store, 4,
 +                              buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                              buildExpression(age, Operator.EQ, Int32Type.instance.decompose(36)));
 +
 +        expected = new ArrayList<String>()
 +        {{
 +                add("key22");
 +                add("key08");
 +                add("key07");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // the same but with the page size of 2 to test minimal pagination windows
 +
 +        uniqueKeys = getPaged(store, 2,
 +                              buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                              buildExpression(age, Operator.EQ, Int32Type.instance.decompose(36)));
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // and last but not least, test age range query with pagination
 +        uniqueKeys = getPaged(store, 4,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                buildExpression(age, Operator.GT, Int32Type.instance.decompose(20)),
 +                buildExpression(age, Operator.LTE, Int32Type.instance.decompose(36)));
 +
 +        expected = new ArrayList<String>()
 +        {{
 +                add("key25");
 +                add("key20");
 +                add("key13");
 +                add("key22");
 +                add("key09");
 +                add("key14");
 +                add("key16");
 +                add("key24");
 +                add("key03");
 +                add("key08");
 +                add("key07");
 +                add("key21");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        Set<String> rows;
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' limit 10 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key03", "key04", "key09", "key13", "key14", "key16", "key20", "key22", "key24", "key25" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' and token(id) >= token('key14') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key03", "key04", "key14", "key16", "key24" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' and token(id) >= token('key14') and token(id) <= token('key24') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key16", "key24" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' and age > 30 and token(id) >= token('key14') and token(id) <= token('key24') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name like '%%ie' limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key07", "key20", "key24" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name like '%%ie' AND token(id) > token('key24') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key07", "key24" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testColumnNamesWithSlashes() throws Exception
 +    {
 +        testColumnNamesWithSlashes(false);
 +        cleanupData();
 +        testColumnNamesWithSlashes(true);
 +    }
 +
 +    private void testColumnNamesWithSlashes(boolean forceFlush) throws Exception
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        Mutation rm1 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key1")));
 +        rm1.add(PartitionUpdate.singleRowUpdate(store.metadata,
 +                                                rm1.key(),
 +                                                buildRow(buildCell(store.metadata,
 +                                                                   UTF8Type.instance.decompose("/data/output/id"),
 +                                                                   AsciiType.instance.decompose("jason"),
 +                                                                   System.currentTimeMillis()))));
 +
 +        Mutation rm2 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key2")));
 +        rm2.add(PartitionUpdate.singleRowUpdate(store.metadata,
 +                                                rm2.key(),
 +                                                buildRow(buildCell(store.metadata,
 +                                                                   UTF8Type.instance.decompose("/data/output/id"),
 +                                                                   AsciiType.instance.decompose("pavel"),
 +                                                                   System.currentTimeMillis()))));
 +
 +        Mutation rm3 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key3")));
 +        rm3.add(PartitionUpdate.singleRowUpdate(store.metadata,
 +                                                rm3.key(),
 +                                                buildRow(buildCell(store.metadata,
 +                                                                   UTF8Type.instance.decompose("/data/output/id"),
 +                                                                   AsciiType.instance.decompose("Aleksey"),
 +                                                                   System.currentTimeMillis()))));
 +
 +        rm1.apply();
 +        rm2.apply();
 +        rm3.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        final ByteBuffer dataOutputId = UTF8Type.instance.decompose("/data/output/id");
 +
 +        Set<String> rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("A")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        // doesn't really make sense to rebuild index for in-memory data
 +        if (!forceFlush)
 +            return;
 +
 +        store.indexManager.invalidateAllIndexesBlocking();
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("A")));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +        // now let's trigger index rebuild and check if we got the data back
 +        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("data_output_id"));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        // also let's try to build an index for column which has no data to make sure that doesn't fail
 +        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("first_name"));
 +        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("data_output_id"));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("el")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testInvalidate() throws Exception
 +    {
 +        testInvalidate(false);
 +        cleanupData();
 +        testInvalidate(true);
 +    }
 +
 +    private void testInvalidate(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("Maxie", -1));
 +                put("key1", Pair.create("Chelsie", 33));
 +                put("key2", Pair.create((String) null, 43));
 +                put("key3", Pair.create("Shanna", 27));
 +                put("key4", Pair.create("Amiya", 36));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part1, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key0", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(33)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        store.indexManager.invalidateAllIndexesBlocking();
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(33)));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key5", Pair.create("Americo", 20));
 +                put("key6", Pair.create("Fiona", 39));
 +                put("key7", Pair.create("Francis", 41));
 +                put("key8", Pair.create("Fred", 21));
 +                put("key9", Pair.create("Amely", 40));
 +                put("key14", Pair.create("Dino", 28));
 +        }};
 +
 +        loadData(part2, forceFlush);
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key6", "key7" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(40)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key9" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testTruncate()
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key01", Pair.create("Ali", 33));
 +                put("key02", Pair.create("Jeremy", 41));
 +                put("key03", Pair.create("Elvera", 22));
 +                put("key04", Pair.create("Bailey", 45));
 +                put("key05", Pair.create("Emerson", 32));
 +                put("key06", Pair.create("Kadin", 38));
 +                put("key07", Pair.create("Maggie", 36));
 +                put("key08", Pair.create("Kailey", 36));
 +                put("key09", Pair.create("Armand", 21));
 +                put("key10", Pair.create("Arnold", 35));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key11", Pair.create("Ken", 38));
 +                put("key12", Pair.create("Penelope", 43));
 +                put("key13", Pair.create("Wyatt", 34));
 +                put("key14", Pair.create("Johnpaul", 34));
 +                put("key15", Pair.create("Trycia", 43));
 +                put("key16", Pair.create("Aida", 21));
 +                put("key17", Pair.create("Devon", 42));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key18", Pair.create("Christina", 20));
 +                put("key19", Pair.create("Rick", 19));
 +                put("key20", Pair.create("Fannie", 22));
 +                put("key21", Pair.create("Keegan", 29));
 +                put("key22", Pair.create("Ignatius", 36));
 +                put("key23", Pair.create("Ellis", 26));
 +                put("key24", Pair.create("Annamarie", 29));
 +                put("key25", Pair.create("Tianna", 31));
 +                put("key26", Pair.create("Dennis", 32));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part1, 1000, true);
 +
 +        loadData(part2, 2000, true);
 +        loadData(part3, 3000, true);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +
 +        Set<String> rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 16, rows.size());
 +
 +        // make sure we don't prematurely delete anything
 +        store.indexManager.truncateAllIndexesBlocking(500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 16, rows.size());
 +
 +        store.indexManager.truncateAllIndexesBlocking(1500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        store.indexManager.truncateAllIndexesBlocking(2500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 6, rows.size());
 +
 +        store.indexManager.truncateAllIndexesBlocking(3500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 0, rows.size());
 +
 +        // add back in some data just to make sure it all still works
 +        Map<String, Pair<String, Integer>> part4 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key40", Pair.create("Tianna", 31));
 +                put("key41", Pair.create("Dennis", 32));
 +        }};
 +
 +        loadData(part4, 4000, true);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 1, rows.size());
 +    }
 +
 +
 +    @Test
 +    public void testConcurrentMemtableReadsAndWrites() throws Exception
 +    {
 +        final ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        ExecutorService scheduler = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
 +
 +        final int writeCount = 10000;
 +        final AtomicInteger updates = new AtomicInteger(0);
 +
 +        for (int i = 0; i < writeCount; i++)
 +        {
 +            final String key = "key" + i;
 +            final String firstName = "first_name#" + i;
 +            final String lastName = "last_name#" + i;
 +
 +            scheduler.submit((Runnable) () -> {
 +                try
 +                {
 +                    newMutation(key, firstName, lastName, 26, System.currentTimeMillis()).apply();
 +                    Uninterruptibles.sleepUninterruptibly(5, TimeUnit.MILLISECONDS); // back up a bit to do more reads
 +                }
 +                finally
 +                {
 +                    updates.incrementAndGet();
 +                }
 +            });
 +        }
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        int previousCount = 0;
 +
 +        do
 +        {
 +            // this loop figures out if number of search results monotonically increasing
 +            // to make sure that concurrent updates don't interfere with reads, uses first_name and age
 +            // indexes to test correctness of both Trie and SkipList ColumnIndex implementations.
 +
 +            Set<DecoratedKey> rows = getPaged(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                                          buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
 +
 +            Assert.assertTrue(previousCount <= rows.size());
 +            previousCount = rows.size();
 +        }
 +        while (updates.get() < writeCount);
 +
 +        // to make sure that after all of the right are done we can read all "count" worth of rows
 +        Set<DecoratedKey> rows = getPaged(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
 +
 +        Assert.assertEquals(writeCount, rows.size());
 +    }
 +
 +    @Test
 +    public void testSameKeyInMemtableAndSSTables()
 +    {
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Map<String, Pair<String, Integer>> data1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 26));
 +                put("key3", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data1, true);
 +
 +        Map<String, Pair<String, Integer>> data2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 28));
 +        }};
 +
 +        loadData(data2, true);
 +
 +        Map<String, Pair<String, Integer>> data3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 15));
 +                put("key4", Pair.create("Jason", 29));
 +        }};
 +
 +        loadData(data3, false);
 +
 +        Set<String> rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(15)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(29)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(27)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key2", "key3"}, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testInsertingIncorrectValuesIntoAgeIndex()
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key1")));
 +        update(rm, new ArrayList<Cell>()
 +        {{
 +            add(buildCell(age, LongType.instance.decompose(26L), System.currentTimeMillis()));
 +            add(buildCell(firstName, AsciiType.instance.decompose("pavel"), System.currentTimeMillis()));
 +        }});
 +        rm.apply();
 +
 +        store.forceBlockingFlush();
 +
 +        Set<String> rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
 +                                                 buildExpression(age, Operator.GTE, Int32Type.instance.decompose(26)));
 +
 +        // index is expected to have 0 results because age value was of wrong type
 +        Assert.assertEquals(0, rows.size());
 +    }
 +
 +
 +    @Test
 +    public void testUnicodeSupport()
 +    {
 +        testUnicodeSupport(false);
 +        cleanupData();
 +        testUnicodeSupport(true);
 +    }
 +
 +    private void testUnicodeSupport(boolean forceFlush)
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("comment");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, comment, UTF8Type.instance.decompose("ⓈⓅⒺⒸⒾⒶⓁ ⒞⒣⒜⒭⒮ and normal ones"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, comment, UTF8Type.instance.decompose("龍馭鬱"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, comment, UTF8Type.instance.decompose("インディアナ"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, comment, UTF8Type.instance.decompose("レストラン"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key5"));
 +        update(rm, comment, UTF8Type.instance.decompose("ベンジャミン ウエスト"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ⓈⓅⒺⒸⒾ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("normal")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ベンジャミン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key5" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("レストラ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("インディ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ベンジャミ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key5" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4", "key5" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("レストラン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testUnicodeSuffixModeNoSplits()
 +    {
 +        testUnicodeSuffixModeNoSplits(false);
 +        cleanupData();
 +        testUnicodeSuffixModeNoSplits(true);
 +    }
 +
 +    private void testUnicodeSuffixModeNoSplits(boolean forceFlush)
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("comment_suffix_split");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, comment, UTF8Type.instance.decompose("龍馭鬱"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, comment, UTF8Type.instance.decompose("インディアナ"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, comment, UTF8Type.instance.decompose("レストラン"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, comment, UTF8Type.instance.decompose("ベンジャミン ウエスト"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ベンジャミン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("トラン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ディア")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ジャミン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("ベンジャミン ウエスト")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testThatTooBigValueIsRejected()
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("comment_suffix_split");
 +
 +        for (int i = 0; i < 10; i++)
 +        {
 +            byte[] randomBytes = new byte[ThreadLocalRandom.current().nextInt(OnDiskIndexBuilder.MAX_TERM_SIZE, 5 * OnDiskIndexBuilder.MAX_TERM_SIZE)];
 +            ThreadLocalRandom.current().nextBytes(randomBytes);
 +
 +            final ByteBuffer bigValue = UTF8Type.instance.decompose(new String(randomBytes));
 +
 +            Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +            update(rm, comment, bigValue, System.currentTimeMillis());
 +            rm.apply();
 +
 +            Set<String> rows;
 +
 +            rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, bigValue.duplicate()));
 +            Assert.assertEquals(0, rows.size());
 +
 +            store.forceBlockingFlush();
 +
 +            rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, bigValue.duplicate()));
 +            Assert.assertEquals(0, rows.size());
 +        }
 +    }
 +
 +    @Test
 +    public void testSearchTimeouts() throws Exception
 +    {
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +
 +        Map<String, Pair<String, Integer>> data1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 26));
 +                put("key3", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data1, true);
 +
 +        RowFilter filter = RowFilter.create();
 +        filter.add(store.metadata.getColumnDefinition(firstName), Operator.LIKE_CONTAINS, AsciiType.instance.fromString("a"));
 +
-         ReadCommand command = new PartitionRangeReadCommand(store.metadata,
-                                                             FBUtilities.nowInSeconds(),
-                                                             ColumnFilter.all(store.metadata),
-                                                             filter,
-                                                             DataLimits.NONE,
-                                                             DataRange.allData(store.metadata.partitioner),
-                                                             Optional.empty());
- 
++        ReadCommand command =
++            PartitionRangeReadCommand.create(false,
++                                             store.metadata,
++                                             FBUtilities.nowInSeconds(),
++                                             ColumnFilter.all(store.metadata),
++                                             filter,
++                                             DataLimits.NONE,
++                                             DataRange.allData(store.metadata.partitioner));
 +        try
 +        {
 +            new QueryPlan(store, command, 0).execute(ReadExecutionController.empty());
 +            Assert.fail();
 +        }
 +        catch (TimeQuotaExceededException e)
 +        {
 +            // correct behavior
 +        }
 +        catch (Exception e)
 +        {
 +            Assert.fail();
 +            e.printStackTrace();
 +        }
 +
 +        // to make sure that query doesn't fail in normal conditions
 +
 +        try (ReadExecutionController controller = command.executionController())
 +        {
 +            Set<String> rows = getKeys(new QueryPlan(store, command, DatabaseDescriptor.getRangeRpcTimeout()).execute(controller));
 +            Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +        }
 +    }
 +
 +    @Test
 +    public void testLowerCaseAnalyzer()
 +    {
 +        testLowerCaseAnalyzer(false);
 +        cleanupData();
 +        testLowerCaseAnalyzer(true);
 +    }
 +
 +    @Test
 +    public void testChinesePrefixSearch()
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer fullName = UTF8Type.instance.decompose("/output/full-name/");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, fullName, UTF8Type.instance.decompose("美加 八田"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, fullName, UTF8Type.instance.decompose("仁美 瀧澤"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, fullName, UTF8Type.instance.decompose("晃宏 高須"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, fullName, UTF8Type.instance.decompose("弘孝 大竹"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key5"));
 +        update(rm, fullName, UTF8Type.instance.decompose("満枝 榎本"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key6"));
 +        update(rm, fullName, UTF8Type.instance.decompose("飛鳥 上原"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key7"));
 +        update(rm, fullName, UTF8Type.instance.decompose("大輝 鎌田"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key8"));
 +        update(rm, fullName, UTF8Type.instance.decompose("利久 寺地"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        store.forceBlockingFlush();
 +
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.EQ, UTF8Type.instance.decompose("美加 八田")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("美加")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.EQ, UTF8Type.instance.decompose("晃宏 高須")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("大輝")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key7" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    public void testLowerCaseAnalyzer(boolean forceFlush)
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("address");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, comment, UTF8Type.instance.decompose("577 Rogahn Valleys Apt. 178"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, comment, UTF8Type.instance.decompose("89809 Beverly Course Suite 089"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, comment, UTF8Type.instance.decompose("165 clydie oval apt. 399"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 Rogahn Valleys")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 ROgAhn VallEYs")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 rogahn valleys")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 rogahn")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("57")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 Beverly Course")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 BEVERly COURSE")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 beverly course")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 Beverly")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("8980")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 ClYdie OvAl APT. 399")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 Clydie Oval Apt. 399")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 clydie oval apt. 399")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 ClYdie OvA")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 ClYdi")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testPrefixSSTableLookup()
 +    {
 +        // This test coverts particular case which interval lookup can return invalid results
 +        // when queried on the prefix e.g. "j".
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer name = UTF8Type.instance.decompose("first_name_prefix");
 +
 +        Mutation rm;
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, name, UTF8Type.instance.decompose("Pavel"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, name, UTF8Type.instance.decompose("Jordan"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, name, UTF8Type.instance.decompose("Mikhail"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, name, UTF8Type.instance.decompose("Michael"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key5"));
 +        update(rm, name, UTF8Type.instance.decompose("Johnny"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        // first flush would make interval for name - 'johnny' -> 'pavel'
 +        store.forceBlockingFlush();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key6"));
 +        update(rm, name, UTF8Type.instance.decompose("Jason"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key7"));
 +        update(rm, name, UTF8Type.instance.decompose("Vijay"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key8")); // this name is going to be tokenized
 +        update(rm, name, UTF8Type.instance.decompose("Jean-Claude"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        // this flush is going to produce range - 'jason' -> 'vijay'
 +        store.forceBlockingFlush();
 +
 +        // make sure that overlap of the prefixes is properly handled across sstables
 +        // since simple interval tree lookup is not going to cover it, prefix lookup actually required.
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("J")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key5", "key6", "key8"}, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("j")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key5", "key6", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("m")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("v")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key7" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("p")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("j")),
 +                                     buildExpression(name, Operator.NEQ, UTF8Type.instance.decompose("joh")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key6", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("pavel")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Pave")));
 +        Assert.assertTrue(rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Pavel")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("JeAn")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("claUde")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Jean")));
 +        Assert.assertTrue(rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Jean-Claude")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key8" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testSettingIsLiteralOption()
 +    {
 +
 +        // special type which is UTF-8 but is only on the inside
 +        AbstractType<?> stringType = new AbstractType<String>(AbstractType.ComparisonType.CUSTOM)
 +        {
 +            public ByteBuffer fromString(String source) throws MarshalException
 +            {
 +                return UTF8Type.instance.fromString(source);
 +            }
 +
 +            public Term fromJSONObject(Object parsed) throws MarshalException
 +            {
 +                throw new UnsupportedOperationException();
 +            }
 +
 +            public TypeSerializer<String> getSerializer()
 +            {
 +                return UTF8Type.instance.getSerializer();
 +            }
 +
 +            public int compareCustom(ByteBuffer a, ByteBuffer b)
 +            {
 +                return UTF8Type.instance.compare(a, b);
 +            }
 +        };
 +
 +        // first let's check that we get 'false' for 'isLiteral' if we don't set the option with special comparator
 +        ColumnDefinition columnA = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-A", stringType);
 +
 +        ColumnIndex indexA = new ColumnIndex(UTF8Type.instance, columnA, IndexMetadata.fromSchemaMetadata("special-index-A", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(true,  indexA.isIndexed());
 +        Assert.assertEquals(false, indexA.isLiteral());
 +
 +        // now let's double-check that we do get 'true' when we set it
 +        ColumnDefinition columnB = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-B", stringType);
 +
 +        ColumnIndex indexB = new ColumnIndex(UTF8Type.instance, columnB, IndexMetadata.fromSchemaMetadata("special-index-B", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +            put("is_literal", "true");
 +        }}));
 +
 +        Assert.assertEquals(true, indexB.isIndexed());
 +        Assert.assertEquals(true, indexB.isLiteral());
 +
 +        // and finally we should also get a 'true' if it's built-in UTF-8/ASCII comparator
 +        ColumnDefinition columnC = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-C", UTF8Type.instance);
 +
 +        ColumnIndex indexC = new ColumnIndex(UTF8Type.instance, columnC, IndexMetadata.fromSchemaMetadata("special-index-C", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(true, indexC.isIndexed());
 +        Assert.assertEquals(true, indexC.isLiteral());
 +
 +        ColumnDefinition columnD = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-D", AsciiType.instance);
 +
 +        ColumnIndex indexD = new ColumnIndex(UTF8Type.instance, columnD, IndexMetadata.fromSchemaMetadata("special-index-D", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(true, indexD.isIndexed());
 +        Assert.assertEquals(true, indexD.isLiteral());
 +
 +        // and option should supersedes the comparator type
 +        ColumnDefinition columnE = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-E", UTF8Type.instance);
 +
 +        ColumnIndex indexE = new ColumnIndex(UTF8Type.instance, columnE, IndexMetadata.fromSchemaMetadata("special-index-E", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +            put("is_literal", "false");
 +        }}));
 +
 +        Assert.assertEquals(true,  indexE.isIndexed());
 +        Assert.assertEquals(false, indexE.isLiteral());
 +
 +        // test frozen-collection
 +        ColumnDefinition columnF = ColumnDefinition.regularDef(KS_NAME,
 +                                                               CF_NAME,
 +                                                               "special-F",
 +                                                               ListType.getInstance(UTF8Type.instance, false));
 +
 +        ColumnIndex indexF = new ColumnIndex(UTF8Type.instance, columnF, IndexMetadata.fromSchemaMetadata("special-index-F", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[03/10] cassandra git commit: Fix race condition in read command serialization

Posted by al...@apache.org.
Fix race condition in read command serialization

patch by Aleksey Yeschenko; reviewed by Sam Tunnicliffe for
CASSANDRA-13363


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

Branch: refs/heads/trunk
Commit: 7f297bcf8aced983cbc9c4103d0ebefc1789f0dd
Parents: d03c046
Author: Aleksey Yeschenko <al...@yeschenko.com>
Authored: Mon Aug 14 16:43:06 2017 +0100
Committer: Aleksey Yeschenko <al...@yeschenko.com>
Committed: Wed Aug 30 16:16:46 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +-
 .../cql3/statements/SelectStatement.java        |  16 +-
 .../db/AbstractReadCommandBuilder.java          |   2 +-
 .../cassandra/db/PartitionRangeReadCommand.java | 133 +++++++++++---
 .../org/apache/cassandra/db/ReadCommand.java    | 149 ++++++++-------
 .../db/SinglePartitionReadCommand.java          | 180 ++++++++++++++++---
 .../cassandra/index/SecondaryIndexManager.java  |   9 +-
 .../internal/composites/CompositesSearcher.java |   6 +-
 .../index/internal/keys/KeysSearcher.java       |   3 +-
 .../cassandra/service/AbstractReadExecutor.java |   4 +-
 .../service/pager/PartitionRangeQueryPager.java |   8 +-
 .../cassandra/thrift/CassandraServer.java       |  69 ++++---
 test/unit/org/apache/cassandra/Util.java        |  26 +--
 .../apache/cassandra/db/SecondaryIndexTest.java |  10 +-
 .../db/SinglePartitionSliceCommandTest.java     |  45 ++---
 .../cassandra/io/sstable/SSTableReaderTest.java |   2 +-
 16 files changed, 427 insertions(+), 238 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 452dc9b..aca9e1f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
 3.0.15
- * enable segement creation before recovering commitlogs (CASSANDRA-13587)
+ * Fix race condition in read command serialization (CASSANDRA-13363)
+ * Enable segement creation before recovering commitlogs (CASSANDRA-13587)
  * Fix AssertionError in short read protection (CASSANDRA-13747)
  * Don't skip corrupted sstables on startup (CASSANDRA-13620)
  * Fix the merging of cells with different user type versions (CASSANDRA-13776)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index bd377f4..3882a23 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -540,18 +540,10 @@ public class SelectStatement implements CQLStatement
         if (keyBounds == null)
             return ReadQuery.EMPTY;
 
-        PartitionRangeReadCommand command = new PartitionRangeReadCommand(cfm,
-                                                                          nowInSec,
-                                                                          queriedColumns,
-                                                                          rowFilter,
-                                                                          limit,
-                                                                          new DataRange(keyBounds, clusteringIndexFilter),
-                                                                          Optional.empty());
-        // If there's a secondary index that the command can use, have it validate
-        // the request parameters. Note that as a side effect, if a viable Index is
-        // identified by the CFS's index manager, it will be cached in the command
-        // and serialized during distribution to replicas in order to avoid performing
-        // further lookups.
+        PartitionRangeReadCommand command =
+            PartitionRangeReadCommand.create(false, cfm, nowInSec, queriedColumns, rowFilter, limit, new DataRange(keyBounds, clusteringIndexFilter));
+
+        // If there's a secondary index that the command can use, have it validate the request parameters.
         command.maybeValidateIndex();
 
         return command;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java b/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
index afbab74..d219816 100644
--- a/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
+++ b/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
@@ -336,7 +336,7 @@ public abstract class AbstractReadCommandBuilder
             else
                 bounds = new ExcludingBounds<>(start, end);
 
-            return new PartitionRangeReadCommand(cfs.metadata, nowInSeconds, makeColumnFilter(), filter, makeLimits(), new DataRange(bounds, makeFilter()), Optional.empty());
+            return PartitionRangeReadCommand.create(false, cfs.metadata, nowInSeconds, makeColumnFilter(), filter, makeLimits(), new DataRange(bounds, makeFilter()));
         }
 
         static DecoratedKey makeKey(CFMetaData metadata, Object... partitionKey)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index fb2dd0d..9e557e0 100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@ -20,8 +20,8 @@ package org.apache.cassandra.db;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Optional;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.config.CFMetaData;
@@ -59,31 +59,39 @@ public class PartitionRangeReadCommand extends ReadCommand
     private final DataRange dataRange;
     private int oldestUnrepairedTombstone = Integer.MAX_VALUE;
 
-    public PartitionRangeReadCommand(boolean isDigest,
-                                     int digestVersion,
-                                     boolean isForThrift,
-                                     CFMetaData metadata,
-                                     int nowInSec,
-                                     ColumnFilter columnFilter,
-                                     RowFilter rowFilter,
-                                     DataLimits limits,
-                                     DataRange dataRange,
-                                     Optional<IndexMetadata> index)
+    private PartitionRangeReadCommand(boolean isDigest,
+                                      int digestVersion,
+                                      boolean isForThrift,
+                                      CFMetaData metadata,
+                                      int nowInSec,
+                                      ColumnFilter columnFilter,
+                                      RowFilter rowFilter,
+                                      DataLimits limits,
+                                      DataRange dataRange,
+                                      IndexMetadata index)
     {
-        super(Kind.PARTITION_RANGE, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits);
+        super(Kind.PARTITION_RANGE, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
         this.dataRange = dataRange;
-        this.index = index;
     }
 
-    public PartitionRangeReadCommand(CFMetaData metadata,
-                                     int nowInSec,
-                                     ColumnFilter columnFilter,
-                                     RowFilter rowFilter,
-                                     DataLimits limits,
-                                     DataRange dataRange,
-                                     Optional<IndexMetadata> index)
+    public static PartitionRangeReadCommand create(boolean isForThrift,
+                                                   CFMetaData metadata,
+                                                   int nowInSec,
+                                                   ColumnFilter columnFilter,
+                                                   RowFilter rowFilter,
+                                                   DataLimits limits,
+                                                   DataRange dataRange)
     {
-        this(false, 0, false, metadata, nowInSec, columnFilter, rowFilter, limits, dataRange, index);
+        return new PartitionRangeReadCommand(false,
+                                             0,
+                                             isForThrift,
+                                             metadata,
+                                             nowInSec,
+                                             columnFilter,
+                                             rowFilter,
+                                             limits,
+                                             dataRange,
+                                             findIndex(metadata, rowFilter));
     }
 
     /**
@@ -96,13 +104,14 @@ public class PartitionRangeReadCommand extends ReadCommand
      */
     public static PartitionRangeReadCommand allDataRead(CFMetaData metadata, int nowInSec)
     {
-        return new PartitionRangeReadCommand(metadata,
+        return new PartitionRangeReadCommand(false, 0, false,
+                                             metadata,
                                              nowInSec,
                                              ColumnFilter.all(metadata),
                                              RowFilter.NONE,
                                              DataLimits.NONE,
                                              DataRange.allData(metadata.partitioner),
-                                             Optional.empty());
+                                             null);
     }
 
     public DataRange dataRange()
@@ -122,17 +131,72 @@ public class PartitionRangeReadCommand extends ReadCommand
 
     public PartitionRangeReadCommand forSubRange(AbstractBounds<PartitionPosition> range)
     {
-        return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange().forSubRange(range), index);
+        return new PartitionRangeReadCommand(isDigestQuery(),
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             limits(),
+                                             dataRange().forSubRange(range),
+                                             indexMetadata());
     }
 
     public PartitionRangeReadCommand copy()
     {
-        return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange(), index);
+        return new PartitionRangeReadCommand(isDigestQuery(),
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             limits(),
+                                             dataRange(),
+                                             indexMetadata());
+    }
+
+    public PartitionRangeReadCommand copyAsDigestQuery()
+    {
+        return new PartitionRangeReadCommand(true,
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             limits(),
+                                             dataRange(),
+                                             indexMetadata());
+    }
+
+    public PartitionRangeReadCommand withUpdatedDataRange(DataRange newDataRange)
+    {
+        return new PartitionRangeReadCommand(isDigestQuery(),
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             limits(),
+                                             newDataRange,
+                                             indexMetadata());
     }
 
-    public PartitionRangeReadCommand withUpdatedLimit(DataLimits newLimits)
+    public PartitionRangeReadCommand withUpdatedLimitsAndDataRange(DataLimits newLimits, DataRange newDataRange)
     {
-        return new PartitionRangeReadCommand(metadata(), nowInSec(), columnFilter(), rowFilter(), newLimits, dataRange(), index);
+        return new PartitionRangeReadCommand(isDigestQuery(),
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             newLimits,
+                                             newDataRange,
+                                             indexMetadata());
     }
 
     public long getTimeout()
@@ -173,7 +237,8 @@ public class PartitionRangeReadCommand extends ReadCommand
         metric.rangeLatency.addNano(latencyNanos);
     }
 
-    protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadOrderGroup orderGroup)
+    @VisibleForTesting
+    public UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadOrderGroup orderGroup)
     {
         ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange()));
         Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().getKeyValidator()));
@@ -337,7 +402,17 @@ public class PartitionRangeReadCommand extends ReadCommand
 
     private static class Deserializer extends SelectionDeserializer
     {
-        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
+        public ReadCommand deserialize(DataInputPlus in,
+                                       int version,
+                                       boolean isDigest,
+                                       int digestVersion,
+                                       boolean isForThrift,
+                                       CFMetaData metadata,
+                                       int nowInSec,
+                                       ColumnFilter columnFilter,
+                                       RowFilter rowFilter,
+                                       DataLimits limits,
+                                       IndexMetadata index)
         throws IOException
         {
             DataRange range = DataRange.serializer.deserialize(in, version, metadata);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index 76180cc..66985b6 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -22,6 +22,8 @@ import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.function.Predicate;
 
+import javax.annotation.Nullable;
+
 import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -106,24 +108,27 @@ public abstract class ReadCommand implements ReadQuery
     private final RowFilter rowFilter;
     private final DataLimits limits;
 
-    // SecondaryIndexManager will attempt to provide the most selective of any available indexes
-    // during execution. Here we also store an the results of that lookup to repeating it over
-    // the lifetime of the command.
-    protected Optional<IndexMetadata> index = Optional.empty();
-
-    // Flag to indicate whether the index manager has been queried to select an index for this
-    // command. This is necessary as the result of that lookup may be null, in which case we
-    // still don't want to repeat it.
-    private boolean indexManagerQueried = false;
-
-    private boolean isDigestQuery;
+    private final boolean isDigestQuery;
     // if a digest query, the version for which the digest is expected. Ignored if not a digest.
     private int digestVersion;
     private final boolean isForThrift;
 
+    @Nullable
+    private final IndexMetadata index;
+
     protected static abstract class SelectionDeserializer
     {
-        public abstract ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index) throws IOException;
+        public abstract ReadCommand deserialize(DataInputPlus in,
+                                                int version,
+                                                boolean isDigest,
+                                                int digestVersion,
+                                                boolean isForThrift,
+                                                CFMetaData metadata,
+                                                int nowInSec,
+                                                ColumnFilter columnFilter,
+                                                RowFilter rowFilter,
+                                                DataLimits limits,
+                                                IndexMetadata index) throws IOException;
     }
 
     protected enum Kind
@@ -147,7 +152,8 @@ public abstract class ReadCommand implements ReadQuery
                           int nowInSec,
                           ColumnFilter columnFilter,
                           RowFilter rowFilter,
-                          DataLimits limits)
+                          DataLimits limits,
+                          IndexMetadata index)
     {
         this.kind = kind;
         this.isDigestQuery = isDigestQuery;
@@ -158,6 +164,7 @@ public abstract class ReadCommand implements ReadQuery
         this.columnFilter = columnFilter;
         this.rowFilter = rowFilter;
         this.limits = limits;
+        this.index = index;
     }
 
     protected abstract void serializeSelection(DataOutputPlus out, int version) throws IOException;
@@ -253,18 +260,6 @@ public abstract class ReadCommand implements ReadQuery
     }
 
     /**
-     * Sets whether this command should be a digest one or not.
-     *
-     * @param isDigestQuery whether the command should be set as a digest one or not.
-     * @return this read command.
-     */
-    public ReadCommand setIsDigestQuery(boolean isDigestQuery)
-    {
-        this.isDigestQuery = isDigestQuery;
-        return this;
-    }
-
-    /**
      * Sets the digest version, for when digest for that command is requested.
      * <p>
      * Note that we allow setting this independently of setting the command as a digest query as
@@ -291,6 +286,30 @@ public abstract class ReadCommand implements ReadQuery
     }
 
     /**
+     * Index (metadata) chosen for this query. Can be null.
+     *
+     * @return index (metadata) chosen for this query
+     */
+    @Nullable
+    public IndexMetadata indexMetadata()
+    {
+        return index;
+    }
+
+    /**
+     *  Index instance chosen for this query. Can be null.
+     *
+     * @return Index instance chosen for this query. Can be null.
+     */
+    @Nullable
+    public Index index()
+    {
+        return null == index
+             ? null
+             : Keyspace.openAndGetStore(metadata).indexManager.getIndex(index);
+    }
+
+    /**
      * The clustering index filter this command to use for the provided key.
      * <p>
      * Note that that method should only be called on a key actually queried by this command
@@ -310,6 +329,11 @@ public abstract class ReadCommand implements ReadQuery
      */
     public abstract ReadCommand copy();
 
+    /**
+     * Returns a copy of this command with isDigestQuery set to true.
+     */
+    public abstract ReadCommand copyAsDigestQuery();
+
     protected abstract UnfilteredPartitionIterator queryStorage(ColumnFamilyStore cfs, ReadOrderGroup orderGroup);
 
     protected abstract int oldestUnrepairedTombstone();
@@ -321,35 +345,32 @@ public abstract class ReadCommand implements ReadQuery
              : ReadResponse.createDataResponse(iterator, this);
     }
 
-    public long indexSerializedSize(int version)
+    long indexSerializedSize(int version)
     {
-        if (index.isPresent())
-            return IndexMetadata.serializer.serializedSize(index.get(), version);
-        else
-            return 0;
+        return null != index
+             ? IndexMetadata.serializer.serializedSize(index, version)
+             : 0;
     }
 
     public Index getIndex(ColumnFamilyStore cfs)
     {
-        // if we've already consulted the index manager, and it returned a valid index
-        // the result should be cached here.
-        if(index.isPresent())
-            return cfs.indexManager.getIndex(index.get());
-
-        // if no cached index is present, but we've already consulted the index manager
-        // then no registered index is suitable for this command, so just return null.
-        if (indexManagerQueried)
+        return null != index
+             ? cfs.indexManager.getIndex(index)
+             : null;
+    }
+
+    static IndexMetadata findIndex(CFMetaData table, RowFilter rowFilter)
+    {
+        if (table.getIndexes().isEmpty() || rowFilter.isEmpty())
             return null;
 
-        // do the lookup, set the flag to indicate so and cache the result if not null
-        Index selected = cfs.indexManager.getBestIndexFor(this);
-        indexManagerQueried = true;
+        ColumnFamilyStore cfs = Keyspace.openAndGetStore(table);
 
-        if (selected == null)
-            return null;
+        Index index = cfs.indexManager.getBestIndexFor(rowFilter);
 
-        index = Optional.of(selected.getIndexMetadata());
-        return selected;
+        return null != index
+             ? index.getIndexMetadata()
+             : null;
     }
 
     /**
@@ -602,7 +623,7 @@ public abstract class ReadCommand implements ReadQuery
             assert version >= MessagingService.VERSION_30;
 
             out.writeByte(command.kind.ordinal());
-            out.writeByte(digestFlag(command.isDigestQuery()) | thriftFlag(command.isForThrift()) | indexFlag(command.index.isPresent()));
+            out.writeByte(digestFlag(command.isDigestQuery()) | thriftFlag(command.isForThrift()) | indexFlag(null != command.index));
             if (command.isDigestQuery())
                 out.writeUnsignedVInt(command.digestVersion());
             CFMetaData.serializer.serialize(command.metadata(), out, version);
@@ -610,8 +631,8 @@ public abstract class ReadCommand implements ReadQuery
             ColumnFilter.serializer.serialize(command.columnFilter(), out, version);
             RowFilter.serializer.serialize(command.rowFilter(), out, version);
             DataLimits.serializer.serialize(command.limits(), out, version);
-            if (command.index.isPresent())
-                IndexMetadata.serializer.serialize(command.index.get(), out, version);
+            if (null != command.index)
+                IndexMetadata.serializer.serialize(command.index, out, version);
 
             command.serializeSelection(out, version);
         }
@@ -631,18 +652,16 @@ public abstract class ReadCommand implements ReadQuery
             ColumnFilter columnFilter = ColumnFilter.serializer.deserialize(in, version, metadata);
             RowFilter rowFilter = RowFilter.serializer.deserialize(in, version, metadata);
             DataLimits limits = DataLimits.serializer.deserialize(in, version);
-            Optional<IndexMetadata> index = hasIndex
-                                            ? deserializeIndexMetadata(in, version, metadata)
-                                            : Optional.empty();
+            IndexMetadata index = hasIndex ? deserializeIndexMetadata(in, version, metadata) : null;
 
             return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
         }
 
-        private Optional<IndexMetadata> deserializeIndexMetadata(DataInputPlus in, int version, CFMetaData cfm) throws IOException
+        private IndexMetadata deserializeIndexMetadata(DataInputPlus in, int version, CFMetaData cfm) throws IOException
         {
             try
             {
-                return Optional.of(IndexMetadata.serializer.deserialize(in, version, cfm));
+                return IndexMetadata.serializer.deserialize(in, version, cfm);
             }
             catch (UnknownIndexException e)
             {
@@ -652,7 +671,7 @@ public abstract class ReadCommand implements ReadQuery
                                                "index. Please wait for schema agreement after index creation.",
                                                cfm.ksName, cfm.cfName, e.indexId.toString());
                 logger.info(message);
-                return Optional.empty();
+                return null;
             }
         }
 
@@ -830,7 +849,7 @@ public abstract class ReadCommand implements ReadQuery
             else
                 limits = DataLimits.cqlLimits(maxResults);
 
-            return new PartitionRangeReadCommand(false, 0, true, metadata, nowInSec, selection, rowFilter, limits, new DataRange(keyRange, filter), Optional.empty());
+            return PartitionRangeReadCommand.create(true, metadata, nowInSec, selection, rowFilter, limits, new DataRange(keyRange, filter));
         }
 
         static void serializeRowFilter(DataOutputPlus out, RowFilter rowFilter) throws IOException
@@ -939,9 +958,8 @@ public abstract class ReadCommand implements ReadQuery
             ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter) command.dataRange().clusteringIndexFilter;
             ClusteringIndexSliceFilter sliceFilter = LegacyReadCommandSerializer.convertNamesFilterToSliceFilter(filter, metadata);
             DataRange newRange = new DataRange(command.dataRange().keyRange(), sliceFilter);
-            return new PartitionRangeReadCommand(
-                    command.isDigestQuery(), command.digestVersion(), command.isForThrift(), metadata, command.nowInSec(),
-                    command.columnFilter(), command.rowFilter(), command.limits(), newRange, Optional.empty());
+
+            return command.withUpdatedDataRange(newRange);
         }
 
         static ColumnFilter getColumnSelectionForSlice(boolean selectsStatics, int compositesToGroup, CFMetaData metadata)
@@ -1096,7 +1114,7 @@ public abstract class ReadCommand implements ReadQuery
                 // missing without any problems, so we can safely always set "inclusive" to false in the data range
                 dataRange = dataRange.forPaging(keyRange, metadata.comparator, startBound.getAsClustering(metadata), false);
             }
-            return new PartitionRangeReadCommand(false, 0, true, metadata, nowInSec, selection, rowFilter, limits, dataRange, Optional.empty());
+            return PartitionRangeReadCommand.create(true, metadata, nowInSec, selection, rowFilter, limits, dataRange);
         }
 
         public long serializedSize(ReadCommand command, int version)
@@ -1290,10 +1308,7 @@ public abstract class ReadCommand implements ReadQuery
         {
             Pair<ColumnFilter, ClusteringIndexNamesFilter> selectionAndFilter = deserializeNamesSelectionAndFilter(in, metadata);
 
-            // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
-            return new SinglePartitionReadCommand(
-                    isDigest, version, true, metadata, nowInSeconds, selectionAndFilter.left, RowFilter.NONE, DataLimits.NONE,
-                    key, selectionAndFilter.right);
+            return SinglePartitionReadCommand.legacyNamesCommand(isDigest, version, metadata, nowInSeconds, selectionAndFilter.left, key, selectionAndFilter.right);
         }
 
         static Pair<ColumnFilter, ClusteringIndexNamesFilter> deserializeNamesSelectionAndFilter(DataInputPlus in, CFMetaData metadata) throws IOException
@@ -1422,8 +1437,7 @@ public abstract class ReadCommand implements ReadQuery
             else
                 limits = DataLimits.cqlLimits(count);
 
-            // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
-            return new SinglePartitionReadCommand(isDigest, version, true, metadata, nowInSeconds, columnFilter, RowFilter.NONE, limits, key, filter);
+            return SinglePartitionReadCommand.legacySliceCommand(isDigest, version, metadata, nowInSeconds, columnFilter, limits, key, filter);
         }
 
         private long serializedSliceCommandSize(SinglePartitionReadCommand command)
@@ -1605,9 +1619,8 @@ public abstract class ReadCommand implements ReadQuery
 
             ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter)command.clusteringIndexFilter();
             ClusteringIndexSliceFilter sliceFilter = convertNamesFilterToSliceFilter(filter, metadata);
-            return new SinglePartitionReadCommand(
-                    command.isDigestQuery(), command.digestVersion(), command.isForThrift(), metadata, command.nowInSec(),
-                    command.columnFilter(), command.rowFilter(), command.limits(), command.partitionKey(), sliceFilter);
+
+            return command.withUpdatedClusteringIndexFilter(sliceFilter);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 686ec35..00464ca 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -70,18 +70,19 @@ public class SinglePartitionReadCommand extends ReadCommand
 
     private int oldestUnrepairedTombstone = Integer.MAX_VALUE;
 
-    public SinglePartitionReadCommand(boolean isDigest,
-                                      int digestVersion,
-                                      boolean isForThrift,
-                                      CFMetaData metadata,
-                                      int nowInSec,
-                                      ColumnFilter columnFilter,
-                                      RowFilter rowFilter,
-                                      DataLimits limits,
-                                      DecoratedKey partitionKey,
-                                      ClusteringIndexFilter clusteringIndexFilter)
+    private SinglePartitionReadCommand(boolean isDigest,
+                                       int digestVersion,
+                                       boolean isForThrift,
+                                       CFMetaData metadata,
+                                       int nowInSec,
+                                       ColumnFilter columnFilter,
+                                       RowFilter rowFilter,
+                                       DataLimits limits,
+                                       DecoratedKey partitionKey,
+                                       ClusteringIndexFilter clusteringIndexFilter,
+                                       IndexMetadata index)
     {
-        super(Kind.SINGLE_PARTITION, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits);
+        super(Kind.SINGLE_PARTITION, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
         assert partitionKey.getPartitioner() == metadata.partitioner;
         this.partitionKey = partitionKey;
         this.clusteringIndexFilter = clusteringIndexFilter;
@@ -90,6 +91,44 @@ public class SinglePartitionReadCommand extends ReadCommand
     /**
      * Creates a new read command on a single partition.
      *
+     * @param isForThrift whether the query is for thrift or not.
+     * @param metadata the table to query.
+     * @param nowInSec the time in seconds to use are "now" for this query.
+     * @param columnFilter the column filter to use for the query.
+     * @param rowFilter the row filter to use for the query.
+     * @param limits the limits to use for the query.
+     * @param partitionKey the partition key for the partition to query.
+     * @param clusteringIndexFilter the clustering index filter to use for the query.
+     * @param indexMetadata explicitly specified index to use for the query
+     *
+     * @return a newly created read command.
+     */
+    public static SinglePartitionReadCommand create(boolean isForThrift,
+                                                    CFMetaData metadata,
+                                                    int nowInSec,
+                                                    ColumnFilter columnFilter,
+                                                    RowFilter rowFilter,
+                                                    DataLimits limits,
+                                                    DecoratedKey partitionKey,
+                                                    ClusteringIndexFilter clusteringIndexFilter,
+                                                    IndexMetadata indexMetadata)
+    {
+        return new SinglePartitionReadCommand(false,
+                                              0,
+                                              isForThrift,
+                                              metadata,
+                                              nowInSec,
+                                              columnFilter,
+                                              rowFilter,
+                                              limits,
+                                              partitionKey,
+                                              clusteringIndexFilter,
+                                              indexMetadata);
+    }
+
+    /**
+     * Creates a new read command on a single partition.
+     *
      * @param metadata the table to query.
      * @param nowInSec the time in seconds to use are "now" for this query.
      * @param columnFilter the column filter to use for the query.
@@ -112,7 +151,7 @@ public class SinglePartitionReadCommand extends ReadCommand
     }
 
     /**
-     * Creates a new read command on a single partition for thrift.
+     * Creates a new read command on a single partition.
      *
      * @param isForThrift whether the query is for thrift or not.
      * @param metadata the table to query.
@@ -134,7 +173,15 @@ public class SinglePartitionReadCommand extends ReadCommand
                                                     DecoratedKey partitionKey,
                                                     ClusteringIndexFilter clusteringIndexFilter)
     {
-        return new SinglePartitionReadCommand(false, 0, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, partitionKey, clusteringIndexFilter);
+        return create(isForThrift,
+                      metadata,
+                      nowInSec,
+                      columnFilter,
+                      rowFilter,
+                      limits,
+                      partitionKey,
+                      clusteringIndexFilter,
+                      findIndex(metadata, rowFilter));
     }
 
     /**
@@ -148,7 +195,11 @@ public class SinglePartitionReadCommand extends ReadCommand
      *
      * @return a newly created read command. The returned command will use no row filter and have no limits.
      */
-    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, ColumnFilter columnFilter, ClusteringIndexFilter filter)
+    public static SinglePartitionReadCommand create(CFMetaData metadata,
+                                                    int nowInSec,
+                                                    DecoratedKey key,
+                                                    ColumnFilter columnFilter,
+                                                    ClusteringIndexFilter filter)
     {
         return create(metadata, nowInSec, columnFilter, RowFilter.NONE, DataLimits.NONE, key, filter);
     }
@@ -164,7 +215,7 @@ public class SinglePartitionReadCommand extends ReadCommand
      */
     public static SinglePartitionReadCommand fullPartitionRead(CFMetaData metadata, int nowInSec, DecoratedKey key)
     {
-        return SinglePartitionReadCommand.create(metadata, nowInSec, key, Slices.ALL);
+        return create(metadata, nowInSec, key, Slices.ALL);
     }
 
     /**
@@ -178,7 +229,7 @@ public class SinglePartitionReadCommand extends ReadCommand
      */
     public static SinglePartitionReadCommand fullPartitionRead(CFMetaData metadata, int nowInSec, ByteBuffer key)
     {
-        return SinglePartitionReadCommand.create(metadata, nowInSec, metadata.decorateKey(key), Slices.ALL);
+        return create(metadata, nowInSec, metadata.decorateKey(key), Slices.ALL);
     }
 
     /**
@@ -211,7 +262,7 @@ public class SinglePartitionReadCommand extends ReadCommand
     public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, Slices slices)
     {
         ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(slices, false);
-        return SinglePartitionReadCommand.create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
+        return create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
     }
 
     /**
@@ -244,7 +295,7 @@ public class SinglePartitionReadCommand extends ReadCommand
     public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, NavigableSet<Clustering> names)
     {
         ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(names, false);
-        return SinglePartitionReadCommand.create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
+        return create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
     }
 
     /**
@@ -265,7 +316,82 @@ public class SinglePartitionReadCommand extends ReadCommand
 
     public SinglePartitionReadCommand copy()
     {
-        return new SinglePartitionReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), partitionKey(), clusteringIndexFilter());
+        return new SinglePartitionReadCommand(isDigestQuery(),
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              partitionKey(),
+                                              clusteringIndexFilter(),
+                                              indexMetadata());
+    }
+
+    public SinglePartitionReadCommand copyAsDigestQuery()
+    {
+        return new SinglePartitionReadCommand(true,
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              partitionKey(),
+                                              clusteringIndexFilter(),
+                                              indexMetadata());
+    }
+
+    public SinglePartitionReadCommand withUpdatedClusteringIndexFilter(ClusteringIndexFilter filter)
+    {
+        return new SinglePartitionReadCommand(isDigestQuery(),
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              partitionKey(),
+                                              filter,
+                                              indexMetadata());
+    }
+
+    static SinglePartitionReadCommand legacySliceCommand(boolean isDigest,
+                                                         int digestVersion,
+                                                         CFMetaData metadata,
+                                                         int nowInSec,
+                                                         ColumnFilter columnFilter,
+                                                         DataLimits limits,
+                                                         DecoratedKey partitionKey,
+                                                         ClusteringIndexSliceFilter filter)
+    {
+        // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
+        return new SinglePartitionReadCommand(isDigest,
+                                              digestVersion,
+                                              true,
+                                              metadata,
+                                              nowInSec,
+                                              columnFilter,
+                                              RowFilter.NONE,
+                                              limits,
+                                              partitionKey,
+                                              filter,
+                                              null);
+    }
+
+    static SinglePartitionReadCommand legacyNamesCommand(boolean isDigest,
+                                                         int digestVersion,
+                                                         CFMetaData metadata,
+                                                         int nowInSec,
+                                                         ColumnFilter columnFilter,
+                                                         DecoratedKey partitionKey,
+                                                         ClusteringIndexNamesFilter filter)
+    {
+        // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
+        return new SinglePartitionReadCommand(isDigest, digestVersion, true, metadata, nowInSec, columnFilter, RowFilter.NONE, DataLimits.NONE, partitionKey, filter,null);
     }
 
     public DecoratedKey partitionKey()
@@ -432,7 +558,7 @@ public class SinglePartitionReadCommand extends ReadCommand
                 final int rowsToCache = metadata().params.caching.rowsPerPartitionToCache();
 
                 @SuppressWarnings("resource") // we close on exception or upon closing the result of this method
-                UnfilteredRowIterator iter = SinglePartitionReadCommand.fullPartitionRead(metadata(), nowInSec(), partitionKey()).queryMemtableAndDisk(cfs, readOp);
+                UnfilteredRowIterator iter = fullPartitionRead(metadata(), nowInSec(), partitionKey()).queryMemtableAndDisk(cfs, readOp);
                 try
                 {
                     // Use a custom iterator instead of DataLimits to avoid stopping the original iterator
@@ -1068,12 +1194,22 @@ public class SinglePartitionReadCommand extends ReadCommand
 
     private static class Deserializer extends SelectionDeserializer
     {
-        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
+        public ReadCommand deserialize(DataInputPlus in,
+                                       int version,
+                                       boolean isDigest,
+                                       int digestVersion,
+                                       boolean isForThrift,
+                                       CFMetaData metadata,
+                                       int nowInSec,
+                                       ColumnFilter columnFilter,
+                                       RowFilter rowFilter,
+                                       DataLimits limits,
+                                       IndexMetadata index)
         throws IOException
         {
             DecoratedKey key = metadata.decorateKey(metadata.getKeyValidator().readValue(in, DatabaseDescriptor.getMaxValueSize()));
             ClusteringIndexFilter filter = ClusteringIndexFilter.serializer.deserialize(in, version, metadata);
-            return new SinglePartitionReadCommand(isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter);
+            return new SinglePartitionReadCommand(isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter, index);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index dd6dde4..5976ddf 100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@ -40,6 +40,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.concurrent.StageManager;
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.statements.IndexTarget;
@@ -697,17 +698,17 @@ public class SecondaryIndexManager implements IndexRegistry
      * cached for future use when obtaining a Searcher, getting the index's underlying CFS for
      * ReadOrderGroup, or an estimate of the result size from an average index query.
      *
-     * @param command ReadCommand to be executed
+     * @param rowFilter RowFilter of the command to be executed
      * @return an Index instance, ready to use during execution of the command, or null if none
      * of the registered indexes can support the command.
      */
-    public Index getBestIndexFor(ReadCommand command)
+    public Index getBestIndexFor(RowFilter rowFilter)
     {
-        if (indexes.isEmpty() || command.rowFilter().isEmpty())
+        if (indexes.isEmpty() || rowFilter.isEmpty())
             return null;
 
         Set<Index> searchableIndexes = new HashSet<>();
-        for (RowFilter.Expression expression : command.rowFilter())
+        for (RowFilter.Expression expression : rowFilter)
         {
             if (expression.isCustom())
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
index 135839b..f8a7c66 100644
--- a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
@@ -136,13 +136,15 @@ public class CompositesSearcher extends CassandraIndexSearcher
 
                     // Query the gathered index hits. We still need to filter stale hits from the resulting query.
                     ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(clusterings.build(), false);
-                    SinglePartitionReadCommand dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata,
+                    SinglePartitionReadCommand dataCmd = SinglePartitionReadCommand.create(isForThrift(),
+                                                                                           index.baseCfs.metadata,
                                                                                            command.nowInSec(),
                                                                                            command.columnFilter(),
                                                                                            command.rowFilter(),
                                                                                            DataLimits.NONE,
                                                                                            partitionKey,
-                                                                                           filter);
+                                                                                           filter,
+                                                                                           null);
                     @SuppressWarnings("resource") // We close right away if empty, and if it's assign to next it will be called either
                     // by the next caller of next, or through closing this iterator is this come before.
                     UnfilteredRowIterator dataIter =

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
index 189b652..c14c5a7 100644
--- a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
@@ -99,7 +99,8 @@ public class KeysSearcher extends CassandraIndexSearcher
                                                                                            command.rowFilter(),
                                                                                            DataLimits.NONE,
                                                                                            key,
-                                                                                           command.clusteringIndexFilter(key));
+                                                                                           command.clusteringIndexFilter(key),
+                                                                                           null);
 
                     @SuppressWarnings("resource") // filterIfStale closes it's iterator if either it materialize it or if it returns null.
                                                   // Otherwise, we close right away if empty, and if it's assigned to next it will be called either

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
index cae1f1a..177fdb2 100644
--- a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
+++ b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
@@ -88,7 +88,7 @@ public abstract class AbstractReadExecutor
 
     protected void makeDigestRequests(Iterable<InetAddress> endpoints)
     {
-        makeRequests(command.copy().setIsDigestQuery(true), endpoints);
+        makeRequests(command.copyAsDigestQuery(), endpoints);
     }
 
     private void makeRequests(ReadCommand readCommand, Iterable<InetAddress> endpoints)
@@ -284,7 +284,7 @@ public abstract class AbstractReadExecutor
                 // Could be waiting on the data, or on enough digests.
                 ReadCommand retryCommand = command;
                 if (handler.resolver.isDataPresent())
-                    retryCommand = command.copy().setIsDigestQuery(true);
+                    retryCommand = command.copyAsDigestQuery();
 
                 InetAddress extraReplica = Iterables.getLast(targetReplicas);
                 if (traceState != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java b/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
index 9c216e3..ea79017 100644
--- a/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.service.pager;
 
-import java.util.Optional;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -27,8 +25,6 @@ import org.apache.cassandra.db.filter.DataLimits;
 import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.index.Index;
-import org.apache.cassandra.schema.IndexMetadata;
 
 /**
  * Pages a PartitionRangeReadCommand.
@@ -90,9 +86,7 @@ public class PartitionRangeQueryPager extends AbstractQueryPager
             }
         }
 
-        Index index = command.getIndex(Keyspace.openAndGetStore(command.metadata()));
-        Optional<IndexMetadata> indexMetadata = index != null ? Optional.of(index.getIndexMetadata()) : Optional.empty();
-        return new PartitionRangeReadCommand(command.metadata(), command.nowInSec(), command.columnFilter(), command.rowFilter(), limits, pageRange, indexMetadata);
+        return ((PartitionRangeReadCommand) command).withUpdatedLimitsAndDataRange(limits, pageRange);
     }
 
     protected void recordLast(DecoratedKey key, Row last)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 86caac3..cb74b15 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -48,7 +48,6 @@ import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.index.Index;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.locator.DynamicEndpointSnitch;
 import org.apache.cassandra.metrics.ClientMetrics;
@@ -1520,16 +1519,16 @@ public class CassandraServer implements Cassandra.Iface
                 ColumnFilter columns = makeColumnFilter(metadata, column_parent, predicate);
                 ClusteringIndexFilter filter = toInternalFilter(metadata, column_parent, predicate);
                 DataLimits limits = getLimits(range.count, metadata.isSuper() && !column_parent.isSetSuper_column(), predicate);
-                PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                              0,
-                                                                              true,
-                                                                              metadata,
-                                                                              nowInSec,
-                                                                              columns,
-                                                                              ThriftConversion.rowFilterFromThrift(metadata, range.row_filter),
-                                                                              limits,
-                                                                              new DataRange(bounds, filter),
-                                                                              Optional.empty());
+
+                PartitionRangeReadCommand cmd =
+                    PartitionRangeReadCommand.create(true,
+                                                     metadata,
+                                                     nowInSec,
+                                                     columns,
+                                                     ThriftConversion.rowFilterFromThrift(metadata, range.row_filter),
+                                                     limits,
+                                                     new DataRange(bounds, filter));
+
                 try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
                 {
                     assert results != null;
@@ -1614,16 +1613,16 @@ public class CassandraServer implements Cassandra.Iface
                 Clustering pageFrom = metadata.isSuper()
                                     ? new Clustering(start_column)
                                     : LegacyLayout.decodeCellName(metadata, start_column).clustering;
-                PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                              0,
-                                                                              true,
-                                                                              metadata,
-                                                                              nowInSec,
-                                                                              ColumnFilter.all(metadata),
-                                                                              RowFilter.NONE,
-                                                                              limits,
-                                                                              new DataRange(bounds, filter).forPaging(bounds, metadata.comparator, pageFrom, true),
-                                                                              Optional.empty());
+
+                PartitionRangeReadCommand cmd =
+                    PartitionRangeReadCommand.create(true,
+                                                     metadata,
+                                                     nowInSec,
+                                                     ColumnFilter.all(metadata),
+                                                     RowFilter.NONE,
+                                                     limits,
+                                                     new DataRange(bounds, filter).forPaging(bounds, metadata.comparator, pageFrom, true));
+
                 try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
                 {
                     return thriftifyKeySlices(results, new ColumnParent(column_family), limits.perPartitionCount());
@@ -1706,21 +1705,17 @@ public class CassandraServer implements Cassandra.Iface
             ColumnFilter columns = makeColumnFilter(metadata, column_parent, column_predicate);
             ClusteringIndexFilter filter = toInternalFilter(metadata, column_parent, column_predicate);
             DataLimits limits = getLimits(index_clause.count, metadata.isSuper() && !column_parent.isSetSuper_column(), column_predicate);
-            PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                          0,
-                                                                          true,
-                                                                          metadata,
-                                                                          nowInSec,
-                                                                          columns,
-                                                                          ThriftConversion.rowFilterFromThrift(metadata, index_clause.expressions),
-                                                                          limits,
-                                                                          new DataRange(bounds, filter),
-                                                                          Optional.empty());
-            // If there's a secondary index that the command can use, have it validate
-            // the request parameters. Note that as a side effect, if a viable Index is
-            // identified by the CFS's index manager, it will be cached in the command
-            // and serialized during distribution to replicas in order to avoid performing
-            // further lookups.
+
+            PartitionRangeReadCommand cmd =
+                PartitionRangeReadCommand.create(true,
+                                                 metadata,
+                                                 nowInSec,
+                                                 columns,
+                                                 ThriftConversion.rowFilterFromThrift(metadata, index_clause.expressions),
+                                                 limits,
+                                                 new DataRange(bounds, filter));
+
+            // If there's a secondary index that the command can use, have it validate the request parameters.
             cmd.maybeValidateIndex();
 
             try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
@@ -2533,7 +2528,7 @@ public class CassandraServer implements Cassandra.Iface
                 // We want to know if the partition exists, so just fetch a single cell.
                 ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(Slices.ALL, false);
                 DataLimits limits = DataLimits.thriftLimits(1, 1);
-                return new SinglePartitionReadCommand(false, 0, true, metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, limits, key, filter);
+                return SinglePartitionReadCommand.create(true, metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, limits, key, filter);
             }
 
             // Gather the clustering for the expected values and query those.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index e8b42bc..d758efe 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -629,31 +629,7 @@ public class Util
                                                              ColumnFamilyStore cfs,
                                                              ReadOrderGroup orderGroup)
     {
-        return new InternalPartitionRangeReadCommand(command).queryStorageInternal(cfs, orderGroup);
-    }
-
-    private static final class InternalPartitionRangeReadCommand extends PartitionRangeReadCommand
-    {
-
-        private InternalPartitionRangeReadCommand(PartitionRangeReadCommand original)
-        {
-            super(original.isDigestQuery(),
-                  original.digestVersion(),
-                  original.isForThrift(),
-                  original.metadata(),
-                  original.nowInSec(),
-                  original.columnFilter(),
-                  original.rowFilter(),
-                  original.limits(),
-                  original.dataRange(),
-                  Optional.empty());
-        }
-
-        private UnfilteredPartitionIterator queryStorageInternal(ColumnFamilyStore cfs,
-                                                                 ReadOrderGroup orderGroup)
-        {
-            return queryStorage(cfs, orderGroup);
-        }
+        return command.queryStorage(cfs, orderGroup);
     }
 
     public static Closeable markDirectoriesUnwriteable(ColumnFamilyStore cfs)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
index bbccc48..2457c4a 100644
--- a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
@@ -118,7 +118,7 @@ public class SecondaryIndexTest
                                       .filterOn("birthdate", Operator.EQ, 1L)
                                       .build();
 
-        Index.Searcher searcher = cfs.indexManager.getBestIndexFor(rc).searcherFor(rc);
+        Index.Searcher searcher = rc.index().searcherFor(rc);
         try (ReadOrderGroup orderGroup = rc.startOrderGroup(); UnfilteredPartitionIterator pi = searcher.search(orderGroup))
         {
             assertTrue(pi.hasNext());
@@ -204,7 +204,7 @@ public class SecondaryIndexTest
 
         // verify that it's not being indexed under any other value either
         ReadCommand rc = Util.cmd(cfs).build();
-        assertNull(cfs.indexManager.getBestIndexFor(rc));
+        assertNull(rc.index());
 
         // resurrect w/ a newer timestamp
         new RowUpdateBuilder(cfs.metadata, 2, "k1").clustering("c").add("birthdate", 1L).build().apply();;
@@ -222,13 +222,13 @@ public class SecondaryIndexTest
         // todo - checking the # of index searchers for the command is probably not the best thing to test here
         RowUpdateBuilder.deleteRow(cfs.metadata, 3, "k1", "c").applyUnsafe();
         rc = Util.cmd(cfs).build();
-        assertNull(cfs.indexManager.getBestIndexFor(rc));
+        assertNull(rc.index());
 
         // make sure obsolete mutations don't generate an index entry
         // todo - checking the # of index searchers for the command is probably not the best thing to test here
         new RowUpdateBuilder(cfs.metadata, 3, "k1").clustering("c").add("birthdate", 1L).build().apply();;
         rc = Util.cmd(cfs).build();
-        assertNull(cfs.indexManager.getBestIndexFor(rc));
+        assertNull(rc.index());
     }
 
     @Test
@@ -504,7 +504,7 @@ public class SecondaryIndexTest
         ColumnDefinition cdef = cfs.metadata.getColumnDefinition(col);
 
         ReadCommand rc = Util.cmd(cfs).filterOn(cdef.name.toString(), Operator.EQ, ((AbstractType) cdef.cellValueType()).decompose(val)).build();
-        Index.Searcher searcher = cfs.indexManager.getBestIndexFor(rc).searcherFor(rc);
+        Index.Searcher searcher = rc.index().searcherFor(rc);
         if (count != 0)
             assertNotNull(searcher);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
index 7f59e2f..02b642e 100644
--- a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
+++ b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
@@ -116,13 +116,14 @@ public class SinglePartitionSliceCommandTest
         ByteBuffer zero = ByteBufferUtil.bytes(0);
         Slices slices = Slices.with(cfm.comparator, Slice.make(Slice.Bound.inclusiveStartOf(zero), Slice.Bound.inclusiveEndOf(zero)));
         ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(slices, false);
-        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                          FBUtilities.nowInSeconds(),
-                                                          columnFilter,
-                                                          RowFilter.NONE,
-                                                          DataLimits.NONE,
-                                                          key,
-                                                          sliceFilter);
+        ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                            cfm,
+                                                            FBUtilities.nowInSeconds(),
+                                                            columnFilter,
+                                                            RowFilter.NONE,
+                                                            DataLimits.NONE,
+                                                            key,
+                                                            sliceFilter);
 
         DataOutputBuffer out = new DataOutputBuffer((int) ReadCommand.legacyReadCommandSerializer.serializedSize(cmd, MessagingService.VERSION_21));
         ReadCommand.legacyReadCommandSerializer.serialize(cmd, out, MessagingService.VERSION_21);
@@ -166,13 +167,14 @@ public class SinglePartitionSliceCommandTest
 
         ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
         ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.NONE, false);
-        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                         FBUtilities.nowInSeconds(),
-                                                         columnFilter,
-                                                         RowFilter.NONE,
-                                                         DataLimits.NONE,
-                                                         key,
-                                                         sliceFilter);
+        ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                            cfm,
+                                                            FBUtilities.nowInSeconds(),
+                                                            columnFilter,
+                                                            RowFilter.NONE,
+                                                            DataLimits.NONE,
+                                                            key,
+                                                            sliceFilter);
 
         // check raw iterator for static cell
         try (ReadOrderGroup orderGroup = cmd.startOrderGroup(); UnfilteredPartitionIterator pi = cmd.executeLocally(orderGroup))
@@ -224,13 +226,14 @@ public class SinglePartitionSliceCommandTest
         ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
         Slice slice = Slice.make(Slice.Bound.BOTTOM, Slice.Bound.inclusiveEndOf(ByteBufferUtil.bytes("i1")));
         ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.with(cfm.comparator, slice), false);
-        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                         FBUtilities.nowInSeconds(),
-                                                         columnFilter,
-                                                         RowFilter.NONE,
-                                                         DataLimits.NONE,
-                                                         key,
-                                                         sliceFilter);
+        ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                            cfm,
+                                                            FBUtilities.nowInSeconds(),
+                                                            columnFilter,
+                                                            RowFilter.NONE,
+                                                            DataLimits.NONE,
+                                                            key,
+                                                            sliceFilter);
 
         String ret = cmd.toCQLString();
         Assert.assertNotNull(ret);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 640b68b..c2598ec 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -604,7 +604,7 @@ public class SSTableReaderTest
                                              .columns("birthdate")
                                              .filterOn("birthdate", Operator.EQ, 1L)
                                              .build();
-        Index.Searcher searcher = indexedCFS.indexManager.getBestIndexFor(rc).searcherFor(rc);
+        Index.Searcher searcher = rc.index().searcherFor(rc);
         assertNotNull(searcher);
         try (ReadOrderGroup orderGroup = ReadOrderGroup.forCommand(rc))
         {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[10/10] cassandra git commit: Merge branch 'cassandra-3.11' into trunk

Posted by al...@apache.org.
Merge branch 'cassandra-3.11' into trunk


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

Branch: refs/heads/trunk
Commit: 3e4d000c9e3ffa2df88c32d78c866e0598898dd4
Parents: 76efcc6 7ad1945
Author: Aleksey Yeschenko <al...@yeschenko.com>
Authored: Wed Aug 30 17:32:09 2017 +0100
Committer: Aleksey Yeschenko <al...@yeschenko.com>
Committed: Wed Aug 30 17:37:41 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cql3/statements/SelectStatement.java        |  16 +-
 .../db/AbstractReadCommandBuilder.java          |   2 +-
 .../cassandra/db/PartitionRangeReadCommand.java | 112 +++++++++++---
 .../org/apache/cassandra/db/ReadCommand.java    | 131 +++++++++-------
 .../db/SinglePartitionReadCommand.java          | 148 ++++++++++++++-----
 .../cassandra/index/SecondaryIndexManager.java  |   8 +-
 .../internal/composites/CompositesSearcher.java |   3 +-
 .../index/internal/keys/KeysSearcher.java       |   3 +-
 .../cassandra/service/AbstractReadExecutor.java |   4 +-
 .../service/pager/PartitionRangeQueryPager.java |   8 +-
 test/unit/org/apache/cassandra/Util.java        |  25 +---
 .../apache/cassandra/db/SecondaryIndexTest.java |  10 +-
 .../db/SinglePartitionSliceCommandTest.java     |  29 ++--
 .../cassandra/index/sasi/SASIIndexTest.java     |  43 +++---
 .../cassandra/io/sstable/SSTableReaderTest.java |   2 +-
 .../cassandra/service/ReadExecutorTest.java     |   2 +-
 17 files changed, 338 insertions(+), 209 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 77eebcf,85efafb..84fef5e
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -569,18 -552,10 +569,10 @@@ public class SelectStatement implement
          if (keyBounds == null)
              return ReadQuery.EMPTY;
  
-         PartitionRangeReadCommand command = new PartitionRangeReadCommand(table,
-                                                                           nowInSec,
-                                                                           columnFilter,
-                                                                           rowFilter,
-                                                                           limit,
-                                                                           new DataRange(keyBounds, clusteringIndexFilter),
-                                                                           Optional.empty());
-         // If there's a secondary index that the command can use, have it validate
-         // the request parameters. Note that as a side effect, if a viable Index is
-         // identified by the CFS's index manager, it will be cached in the command
-         // and serialized during distribution to replicas in order to avoid performing
-         // further lookups.
+         PartitionRangeReadCommand command =
 -            PartitionRangeReadCommand.create(false, cfm, nowInSec, queriedColumns, rowFilter, limit, new DataRange(keyBounds, clusteringIndexFilter));
++            PartitionRangeReadCommand.create(table, nowInSec, columnFilter, rowFilter, limit, new DataRange(keyBounds, clusteringIndexFilter));
+ 
+         // If there's a secondary index that the command can use, have it validate the request parameters.
          command.maybeValidateIndex();
  
          return command;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
index 8ced1c7,1c69813..481e906
--- a/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
+++ b/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
@@@ -336,10 -336,10 +336,10 @@@ public abstract class AbstractReadComma
              else
                  bounds = new ExcludingBounds<>(start, end);
  
-             return new PartitionRangeReadCommand(cfs.metadata(), nowInSeconds, makeColumnFilter(), filter, makeLimits(), new DataRange(bounds, makeFilter()), Optional.empty());
 -            return PartitionRangeReadCommand.create(false, cfs.metadata, nowInSeconds, makeColumnFilter(), filter, makeLimits(), new DataRange(bounds, makeFilter()));
++            return PartitionRangeReadCommand.create(cfs.metadata(), nowInSeconds, makeColumnFilter(), filter, makeLimits(), new DataRange(bounds, makeFilter()));
          }
  
 -        static DecoratedKey makeKey(CFMetaData metadata, Object... partitionKey)
 +        static DecoratedKey makeKey(TableMetadata metadata, Object... partitionKey)
          {
              if (partitionKey.length == 1 && partitionKey[0] instanceof DecoratedKey)
                  return (DecoratedKey)partitionKey[0];

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index da7daa7,f7b6660..e88f7fb
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@@ -20,11 -20,11 +20,11 @@@ package org.apache.cassandra.db
  import java.io.IOException;
  import java.util.ArrayList;
  import java.util.List;
- import java.util.Optional;
  
+ import com.google.common.annotations.VisibleForTesting;
  import com.google.common.collect.Iterables;
  
 -import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.schema.TableMetadata;
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.db.filter.*;
  import org.apache.cassandra.db.lifecycle.View;
@@@ -59,30 -60,39 +59,36 @@@ public class PartitionRangeReadCommand 
      private final DataRange dataRange;
      private int oldestUnrepairedTombstone = Integer.MAX_VALUE;
  
-     public PartitionRangeReadCommand(boolean isDigest,
+     private PartitionRangeReadCommand(boolean isDigest,
 -                                      int digestVersion,
 -                                      boolean isForThrift,
 -                                      CFMetaData metadata,
 -                                      int nowInSec,
 -                                      ColumnFilter columnFilter,
 -                                      RowFilter rowFilter,
 -                                      DataLimits limits,
 -                                      DataRange dataRange,
 -                                      IndexMetadata index)
 +                                     int digestVersion,
 +                                     TableMetadata metadata,
 +                                     int nowInSec,
 +                                     ColumnFilter columnFilter,
 +                                     RowFilter rowFilter,
 +                                     DataLimits limits,
 +                                     DataRange dataRange,
-                                      Optional<IndexMetadata> index)
++                                     IndexMetadata index)
      {
-         super(Kind.PARTITION_RANGE, isDigest, digestVersion, metadata, nowInSec, columnFilter, rowFilter, limits);
 -        super(Kind.PARTITION_RANGE, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
++        super(Kind.PARTITION_RANGE, isDigest, digestVersion, metadata, nowInSec, columnFilter, rowFilter, limits, index);
          this.dataRange = dataRange;
-         this.index = index;
      }
  
-     public PartitionRangeReadCommand(TableMetadata metadata,
-                                      int nowInSec,
-                                      ColumnFilter columnFilter,
-                                      RowFilter rowFilter,
-                                      DataLimits limits,
-                                      DataRange dataRange,
-                                      Optional<IndexMetadata> index)
 -    public static PartitionRangeReadCommand create(boolean isForThrift,
 -                                                   CFMetaData metadata,
++    public static PartitionRangeReadCommand create(TableMetadata metadata,
+                                                    int nowInSec,
+                                                    ColumnFilter columnFilter,
+                                                    RowFilter rowFilter,
+                                                    DataLimits limits,
+                                                    DataRange dataRange)
      {
-         this(false, 0, metadata, nowInSec, columnFilter, rowFilter, limits, dataRange, index);
+         return new PartitionRangeReadCommand(false,
+                                              0,
 -                                             isForThrift,
+                                              metadata,
+                                              nowInSec,
+                                              columnFilter,
+                                              rowFilter,
+                                              limits,
+                                              dataRange,
+                                              findIndex(metadata, rowFilter));
      }
  
      /**
@@@ -93,9 -103,10 +99,11 @@@
       *
       * @return a newly created read command that queries everything in the table.
       */
 -    public static PartitionRangeReadCommand allDataRead(CFMetaData metadata, int nowInSec)
 +    public static PartitionRangeReadCommand allDataRead(TableMetadata metadata, int nowInSec)
      {
-         return new PartitionRangeReadCommand(metadata,
 -        return new PartitionRangeReadCommand(false, 0, false,
++        return new PartitionRangeReadCommand(false,
++                                             0,
+                                              metadata,
                                               nowInSec,
                                               ColumnFilter.all(metadata),
                                               RowFilter.NONE,
@@@ -142,18 -152,86 +149,67 @@@
          // DataLimits.CQLGroupByLimits.GroupByAwareCounter assumes that if GroupingState.hasClustering(), then we're in
          // the middle of a group, but we can't make that assumption if we query and range "in advance" of where we are
          // on the ring.
-         DataLimits newLimits = isRangeContinuation ? limits() : limits().withoutState();
-         return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), metadata(), nowInSec(), columnFilter(), rowFilter(), newLimits, newRange, index);
+         return new PartitionRangeReadCommand(isDigestQuery(),
+                                              digestVersion(),
 -                                             isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              isRangeContinuation ? limits() : limits().withoutState(),
+                                              dataRange().forSubRange(range),
+                                              indexMetadata());
      }
  
      public PartitionRangeReadCommand copy()
      {
-         return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange(), index);
+         return new PartitionRangeReadCommand(isDigestQuery(),
+                                              digestVersion(),
 -                                             isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              dataRange(),
+                                              indexMetadata());
+     }
+ 
+     public PartitionRangeReadCommand copyAsDigestQuery()
+     {
+         return new PartitionRangeReadCommand(true,
+                                              digestVersion(),
 -                                             isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              dataRange(),
+                                              indexMetadata());
+     }
+ 
+     public ReadCommand withUpdatedLimit(DataLimits newLimits)
+     {
+         return new PartitionRangeReadCommand(isDigestQuery(),
+                                              digestVersion(),
 -                                             isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              newLimits,
+                                              dataRange(),
+                                              indexMetadata());
      }
  
-     public PartitionRangeReadCommand withUpdatedLimit(DataLimits newLimits)
 -    public PartitionRangeReadCommand withUpdatedDataRange(DataRange newDataRange)
 -    {
 -        return new PartitionRangeReadCommand(isDigestQuery(),
 -                                             digestVersion(),
 -                                             isForThrift(),
 -                                             metadata(),
 -                                             nowInSec(),
 -                                             columnFilter(),
 -                                             rowFilter(),
 -                                             limits(),
 -                                             newDataRange,
 -                                             indexMetadata());
 -    }
 -
+     public PartitionRangeReadCommand withUpdatedLimitsAndDataRange(DataLimits newLimits, DataRange newDataRange)
      {
-         return new PartitionRangeReadCommand(metadata(), nowInSec(), columnFilter(), rowFilter(), newLimits, dataRange(), index);
+         return new PartitionRangeReadCommand(isDigestQuery(),
+                                              digestVersion(),
 -                                             isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              newLimits,
+                                              newDataRange,
+                                              indexMetadata());
      }
  
      public long getTimeout()
@@@ -194,10 -272,11 +250,11 @@@
          metric.rangeLatency.addNano(latencyNanos);
      }
  
-     protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadExecutionController executionController)
+     @VisibleForTesting
+     public UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadExecutionController executionController)
      {
          ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange()));
 -        Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().getKeyValidator()));
 +        Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().partitionKeyType));
  
          // fetch data from current memtable, historical memtables, and SSTables in the correct order.
          final List<UnfilteredPartitionIterator> iterators = new ArrayList<>(Iterables.size(view.memtables) + view.sstables.size());
@@@ -356,7 -438,17 +413,16 @@@
  
      private static class Deserializer extends SelectionDeserializer
      {
-         public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, TableMetadata metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
+         public ReadCommand deserialize(DataInputPlus in,
+                                        int version,
+                                        boolean isDigest,
+                                        int digestVersion,
 -                                       boolean isForThrift,
 -                                       CFMetaData metadata,
++                                       TableMetadata metadata,
+                                        int nowInSec,
+                                        ColumnFilter columnFilter,
+                                        RowFilter rowFilter,
+                                        DataLimits limits,
+                                        IndexMetadata index)
          throws IOException
          {
              DataRange range = DataRange.serializer.deserialize(in, version, metadata);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ReadCommand.java
index 08224bf,54389f0..e135902
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@@ -18,9 -18,13 +18,10 @@@
  package org.apache.cassandra.db;
  
  import java.io.IOException;
 -import java.nio.ByteBuffer;
--import java.util.*;
  import java.util.function.Predicate;
  
+ import javax.annotation.Nullable;
+ 
 -import com.google.common.collect.Lists;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
@@@ -32,22 -37,23 +33,22 @@@ import org.apache.cassandra.db.partitio
  import org.apache.cassandra.db.rows.*;
  import org.apache.cassandra.db.transform.StoppingTransformation;
  import org.apache.cassandra.db.transform.Transformation;
 -import org.apache.cassandra.dht.AbstractBounds;
++import org.apache.cassandra.exceptions.UnknownIndexException;
  import org.apache.cassandra.index.Index;
  import org.apache.cassandra.index.IndexNotAvailableException;
 -import org.apache.cassandra.io.ForwardingVersionedSerializer;
  import org.apache.cassandra.io.IVersionedSerializer;
  import org.apache.cassandra.io.util.DataInputPlus;
  import org.apache.cassandra.io.util.DataOutputPlus;
  import org.apache.cassandra.metrics.TableMetrics;
  import org.apache.cassandra.net.MessageOut;
 -import org.apache.cassandra.net.MessagingService;
  import org.apache.cassandra.schema.IndexMetadata;
 -import org.apache.cassandra.schema.UnknownIndexException;
 +import org.apache.cassandra.schema.Schema;
 +import org.apache.cassandra.schema.SchemaConstants;
 +import org.apache.cassandra.schema.TableId;
 +import org.apache.cassandra.schema.TableMetadata;
- import org.apache.cassandra.exceptions.UnknownIndexException;
  import org.apache.cassandra.service.ClientWarn;
  import org.apache.cassandra.tracing.Tracing;
 -import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.FBUtilities;
 -import org.apache.cassandra.utils.Pair;
  
  /**
   * General interface for storage-engine read commands (common to both range and
@@@ -69,23 -112,27 +70,25 @@@ public abstract class ReadCommand exten
      private final RowFilter rowFilter;
      private final DataLimits limits;
  
-     // SecondaryIndexManager will attempt to provide the most selective of any available indexes
-     // during execution. Here we also store an the results of that lookup to repeating it over
-     // the lifetime of the command.
-     protected Optional<IndexMetadata> index = Optional.empty();
- 
-     // Flag to indicate whether the index manager has been queried to select an index for this
-     // command. This is necessary as the result of that lookup may be null, in which case we
-     // still don't want to repeat it.
-     private boolean indexManagerQueried = false;
- 
-     private boolean isDigestQuery;
+     private final boolean isDigestQuery;
      // if a digest query, the version for which the digest is expected. Ignored if not a digest.
      private int digestVersion;
 -    private final boolean isForThrift;
  
+     @Nullable
+     private final IndexMetadata index;
+ 
      protected static abstract class SelectionDeserializer
      {
-         public abstract ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, TableMetadata metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index) throws IOException;
+         public abstract ReadCommand deserialize(DataInputPlus in,
+                                                 int version,
+                                                 boolean isDigest,
+                                                 int digestVersion,
 -                                                boolean isForThrift,
 -                                                CFMetaData metadata,
++                                                TableMetadata metadata,
+                                                 int nowInSec,
+                                                 ColumnFilter columnFilter,
+                                                 RowFilter rowFilter,
+                                                 DataLimits limits,
+                                                 IndexMetadata index) throws IOException;
      }
  
      protected enum Kind
@@@ -249,6 -288,40 +242,30 @@@
      }
  
      /**
 -     * Whether this query is for thrift or not.
 -     *
 -     * @return whether this query is for thrift.
 -     */
 -    public boolean isForThrift()
 -    {
 -        return isForThrift;
 -    }
 -
 -    /**
+      * Index (metadata) chosen for this query. Can be null.
+      *
+      * @return index (metadata) chosen for this query
+      */
+     @Nullable
+     public IndexMetadata indexMetadata()
+     {
+         return index;
+     }
+ 
+     /**
+      *  Index instance chosen for this query. Can be null.
+      *
+      * @return Index instance chosen for this query. Can be null.
+      */
+     @Nullable
+     public Index index()
+     {
+         return null == index
+              ? null
+              : Keyspace.openAndGetStore(metadata).indexManager.getIndex(index);
+     }
+ 
+     /**
       * The clustering index filter this command to use for the provided key.
       * <p>
       * Note that that method should only be called on a key actually queried by this command
@@@ -289,25 -366,23 +310,23 @@@
  
      public Index getIndex(ColumnFamilyStore cfs)
      {
-         // if we've already consulted the index manager, and it returned a valid index
-         // the result should be cached here.
-         if(index.isPresent())
-             return cfs.indexManager.getIndex(index.get());
- 
-         // if no cached index is present, but we've already consulted the index manager
-         // then no registered index is suitable for this command, so just return null.
-         if (indexManagerQueried)
+         return null != index
+              ? cfs.indexManager.getIndex(index)
+              : null;
+     }
+ 
 -    static IndexMetadata findIndex(CFMetaData table, RowFilter rowFilter)
++    static IndexMetadata findIndex(TableMetadata table, RowFilter rowFilter)
+     {
 -        if (table.getIndexes().isEmpty() || rowFilter.isEmpty())
++        if (table.indexes.isEmpty() || rowFilter.isEmpty())
              return null;
  
-         // do the lookup, set the flag to indicate so and cache the result if not null
-         Index selected = cfs.indexManager.getBestIndexFor(this);
-         indexManagerQueried = true;
+         ColumnFamilyStore cfs = Keyspace.openAndGetStore(table);
  
-         if (selected == null)
-             return null;
+         Index index = cfs.indexManager.getBestIndexFor(rowFilter);
  
-         index = Optional.of(selected.getIndexMetadata());
-         return selected;
+         return null != index
+              ? index.getIndexMetadata()
+              : null;
      }
  
      /**
@@@ -619,11 -695,13 +638,11 @@@
  
          public void serialize(ReadCommand command, DataOutputPlus out, int version) throws IOException
          {
 -            assert version >= MessagingService.VERSION_30;
 -
              out.writeByte(command.kind.ordinal());
-             out.writeByte(digestFlag(command.isDigestQuery()) | indexFlag(command.index.isPresent()));
 -            out.writeByte(digestFlag(command.isDigestQuery()) | thriftFlag(command.isForThrift()) | indexFlag(null != command.index));
++            out.writeByte(digestFlag(command.isDigestQuery()) | indexFlag(null != command.indexMetadata()));
              if (command.isDigestQuery())
                  out.writeUnsignedVInt(command.digestVersion());
 -            CFMetaData.serializer.serialize(command.metadata(), out, version);
 +            command.metadata.id.serialize(out);
              out.writeInt(command.nowInSec());
              ColumnFilter.serializer.serialize(command.columnFilter(), out, version);
              RowFilter.serializer.serialize(command.rowFilter(), out, version);
@@@ -653,19 -726,17 +672,17 @@@
              int nowInSec = in.readInt();
              ColumnFilter columnFilter = ColumnFilter.serializer.deserialize(in, version, metadata);
              RowFilter rowFilter = RowFilter.serializer.deserialize(in, version, metadata);
-             DataLimits limits = DataLimits.serializer.deserialize(in, version, metadata.comparator);
-             Optional<IndexMetadata> index = hasIndex
-                                           ? deserializeIndexMetadata(in, version, metadata)
-                                           : Optional.empty();
+             DataLimits limits = DataLimits.serializer.deserialize(in, version,  metadata.comparator);
+             IndexMetadata index = hasIndex ? deserializeIndexMetadata(in, version, metadata) : null;
  
 -            return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
 +            return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, metadata, nowInSec, columnFilter, rowFilter, limits, index);
          }
  
-         private Optional<IndexMetadata> deserializeIndexMetadata(DataInputPlus in, int version, TableMetadata metadata) throws IOException
 -        private IndexMetadata deserializeIndexMetadata(DataInputPlus in, int version, CFMetaData cfm) throws IOException
++        private IndexMetadata deserializeIndexMetadata(DataInputPlus in, int version, TableMetadata metadata) throws IOException
          {
              try
              {
-                 return Optional.of(IndexMetadata.serializer.deserialize(in, version, metadata));
 -                return IndexMetadata.serializer.deserialize(in, version, cfm);
++                return IndexMetadata.serializer.deserialize(in, version, metadata);
              }
              catch (UnknownIndexException e)
              {
@@@ -673,8 -744,8 +690,8 @@@
                              "If an index was just created, this is likely due to the schema not " +
                              "being fully propagated. Local read will proceed without using the " +
                              "index. Please wait for schema agreement after index creation.",
 -                            cfm.ksName, cfm.cfName, e.indexId);
 +                            metadata.keyspace, metadata.name, e.indexId);
-                 return Optional.empty();
+                 return null;
              }
          }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index bd65535,c7080e7..f4f36d8
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@@ -22,6 -22,6 +22,7 @@@ import java.nio.ByteBuffer
  import java.util.*;
  import java.util.stream.Collectors;
  
++import com.google.common.annotations.VisibleForTesting;
  import com.google.common.collect.Iterables;
  import com.google.common.collect.Sets;
  
@@@ -71,17 -73,19 +72,19 @@@ public class SinglePartitionReadComman
  
      private int oldestUnrepairedTombstone = Integer.MAX_VALUE;
  
-     public SinglePartitionReadCommand(boolean isDigest,
-                                       int digestVersion,
-                                       TableMetadata metadata,
-                                       int nowInSec,
-                                       ColumnFilter columnFilter,
-                                       RowFilter rowFilter,
-                                       DataLimits limits,
-                                       DecoratedKey partitionKey,
-                                       ClusteringIndexFilter clusteringIndexFilter)
 -    private SinglePartitionReadCommand(boolean isDigest,
 -                                       int digestVersion,
 -                                       boolean isForThrift,
 -                                       CFMetaData metadata,
 -                                       int nowInSec,
 -                                       ColumnFilter columnFilter,
 -                                       RowFilter rowFilter,
 -                                       DataLimits limits,
 -                                       DecoratedKey partitionKey,
 -                                       ClusteringIndexFilter clusteringIndexFilter,
 -                                       IndexMetadata index)
++    @VisibleForTesting
++    protected SinglePartitionReadCommand(boolean isDigest,
++                                         int digestVersion,
++                                         TableMetadata metadata,
++                                         int nowInSec,
++                                         ColumnFilter columnFilter,
++                                         RowFilter rowFilter,
++                                         DataLimits limits,
++                                         DecoratedKey partitionKey,
++                                         ClusteringIndexFilter clusteringIndexFilter,
++                                         IndexMetadata index)
      {
-         super(Kind.SINGLE_PARTITION, isDigest, digestVersion, metadata, nowInSec, columnFilter, rowFilter, limits);
 -        super(Kind.SINGLE_PARTITION, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
++        super(Kind.SINGLE_PARTITION, isDigest, digestVersion, metadata, nowInSec, columnFilter, rowFilter, limits, index);
          assert partitionKey.getPartitioner() == metadata.partitioner;
          this.partitionKey = partitionKey;
          this.clusteringIndexFilter = clusteringIndexFilter;
@@@ -97,6 -102,43 +100,41 @@@
       * @param limits the limits to use for the query.
       * @param partitionKey the partition key for the partition to query.
       * @param clusteringIndexFilter the clustering index filter to use for the query.
+      * @param indexMetadata explicitly specified index to use for the query
+      *
+      * @return a newly created read command.
+      */
 -    public static SinglePartitionReadCommand create(boolean isForThrift,
 -                                                    CFMetaData metadata,
++    public static SinglePartitionReadCommand create(TableMetadata metadata,
+                                                     int nowInSec,
+                                                     ColumnFilter columnFilter,
+                                                     RowFilter rowFilter,
+                                                     DataLimits limits,
+                                                     DecoratedKey partitionKey,
+                                                     ClusteringIndexFilter clusteringIndexFilter,
+                                                     IndexMetadata indexMetadata)
+     {
+         return new SinglePartitionReadCommand(false,
+                                               0,
 -                                              isForThrift,
+                                               metadata,
+                                               nowInSec,
+                                               columnFilter,
+                                               rowFilter,
+                                               limits,
+                                               partitionKey,
+                                               clusteringIndexFilter,
+                                               indexMetadata);
+     }
+ 
+     /**
+      * Creates a new read command on a single partition.
+      *
+      * @param metadata the table to query.
+      * @param nowInSec the time in seconds to use are "now" for this query.
+      * @param columnFilter the column filter to use for the query.
+      * @param rowFilter the row filter to use for the query.
+      * @param limits the limits to use for the query.
+      * @param partitionKey the partition key for the partition to query.
+      * @param clusteringIndexFilter the clustering index filter to use for the query.
       *
       * @return a newly created read command.
       */
@@@ -108,7 -176,15 +146,14 @@@
                                                      DecoratedKey partitionKey,
                                                      ClusteringIndexFilter clusteringIndexFilter)
      {
-         return new SinglePartitionReadCommand(false, 0, metadata, nowInSec, columnFilter, rowFilter, limits, partitionKey, clusteringIndexFilter);
 -        return create(isForThrift,
 -                      metadata,
++        return create(metadata,
+                       nowInSec,
+                       columnFilter,
+                       rowFilter,
+                       limits,
+                       partitionKey,
+                       clusteringIndexFilter,
+                       findIndex(metadata, rowFilter));
      }
  
      /**
@@@ -122,7 -198,11 +167,11 @@@
       *
       * @return a newly created read command. The returned command will use no row filter and have no limits.
       */
-     public static SinglePartitionReadCommand create(TableMetadata metadata, int nowInSec, DecoratedKey key, ColumnFilter columnFilter, ClusteringIndexFilter filter)
 -    public static SinglePartitionReadCommand create(CFMetaData metadata,
++    public static SinglePartitionReadCommand create(TableMetadata metadata,
+                                                     int nowInSec,
+                                                     DecoratedKey key,
+                                                     ColumnFilter columnFilter,
+                                                     ClusteringIndexFilter filter)
      {
          return create(metadata, nowInSec, columnFilter, RowFilter.NONE, DataLimits.NONE, key, filter);
      }
@@@ -136,9 -216,9 +185,9 @@@
       *
       * @return a newly created read command that queries all the rows of {@code key}.
       */
 -    public static SinglePartitionReadCommand fullPartitionRead(CFMetaData metadata, int nowInSec, DecoratedKey key)
 +    public static SinglePartitionReadCommand fullPartitionRead(TableMetadata metadata, int nowInSec, DecoratedKey key)
      {
-         return SinglePartitionReadCommand.create(metadata, nowInSec, key, Slices.ALL);
+         return create(metadata, nowInSec, key, Slices.ALL);
      }
  
      /**
@@@ -150,9 -230,9 +199,9 @@@
       *
       * @return a newly created read command that queries all the rows of {@code key}.
       */
 -    public static SinglePartitionReadCommand fullPartitionRead(CFMetaData metadata, int nowInSec, ByteBuffer key)
 +    public static SinglePartitionReadCommand fullPartitionRead(TableMetadata metadata, int nowInSec, ByteBuffer key)
      {
-         return SinglePartitionReadCommand.create(metadata, nowInSec, metadata.partitioner.decorateKey(key), Slices.ALL);
 -        return create(metadata, nowInSec, metadata.decorateKey(key), Slices.ALL);
++        return create(metadata, nowInSec, metadata.partitioner.decorateKey(key), Slices.ALL);
      }
  
      /**
@@@ -182,10 -262,10 +231,10 @@@
       * @return a newly created read command that queries the {@code slices} in {@code key}. The returned query will
       * query every columns for the table (without limit or row filtering) and be in forward order.
       */
 -    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, Slices slices)
 +    public static SinglePartitionReadCommand create(TableMetadata metadata, int nowInSec, DecoratedKey key, Slices slices)
      {
          ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(slices, false);
-         return SinglePartitionReadCommand.create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
+         return create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
      }
  
      /**
@@@ -215,10 -295,10 +264,10 @@@
       * @return a newly created read command that queries the {@code names} in {@code key}. The returned query will
       * query every columns (without limit or row filtering) and be in forward order.
       */
 -    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, NavigableSet<Clustering> names)
 +    public static SinglePartitionReadCommand create(TableMetadata metadata, int nowInSec, DecoratedKey key, NavigableSet<Clustering> names)
      {
          ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(names, false);
-         return SinglePartitionReadCommand.create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
+         return create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
      }
  
      /**
@@@ -239,9 -319,99 +288,46 @@@
  
      public SinglePartitionReadCommand copy()
      {
-         return new SinglePartitionReadCommand(isDigestQuery(), digestVersion(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), partitionKey(), clusteringIndexFilter());
+         return new SinglePartitionReadCommand(isDigestQuery(),
+                                               digestVersion(),
 -                                              isForThrift(),
+                                               metadata(),
+                                               nowInSec(),
+                                               columnFilter(),
+                                               rowFilter(),
+                                               limits(),
+                                               partitionKey(),
+                                               clusteringIndexFilter(),
+                                               indexMetadata());
+     }
+ 
+     public SinglePartitionReadCommand copyAsDigestQuery()
+     {
+         return new SinglePartitionReadCommand(true,
+                                               digestVersion(),
 -                                              isForThrift(),
+                                               metadata(),
+                                               nowInSec(),
+                                               columnFilter(),
+                                               rowFilter(),
+                                               limits(),
+                                               partitionKey(),
+                                               clusteringIndexFilter(),
+                                               indexMetadata());
+     }
+ 
+     public SinglePartitionReadCommand withUpdatedLimit(DataLimits newLimits)
+     {
+         return new SinglePartitionReadCommand(isDigestQuery(),
+                                               digestVersion(),
 -                                              isForThrift(),
+                                               metadata(),
+                                               nowInSec(),
+                                               columnFilter(),
+                                               rowFilter(),
+                                               newLimits,
+                                               partitionKey(),
+                                               clusteringIndexFilter(),
+                                               indexMetadata());
      }
  
 -    public SinglePartitionReadCommand withUpdatedClusteringIndexFilter(ClusteringIndexFilter filter)
 -    {
 -        return new SinglePartitionReadCommand(isDigestQuery(),
 -                                              digestVersion(),
 -                                              isForThrift(),
 -                                              metadata(),
 -                                              nowInSec(),
 -                                              columnFilter(),
 -                                              rowFilter(),
 -                                              limits(),
 -                                              partitionKey(),
 -                                              filter,
 -                                              indexMetadata());
 -    }
 -
 -    static SinglePartitionReadCommand legacySliceCommand(boolean isDigest,
 -                                                         int digestVersion,
 -                                                         CFMetaData metadata,
 -                                                         int nowInSec,
 -                                                         ColumnFilter columnFilter,
 -                                                         DataLimits limits,
 -                                                         DecoratedKey partitionKey,
 -                                                         ClusteringIndexSliceFilter filter)
 -    {
 -        // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
 -        return new SinglePartitionReadCommand(isDigest,
 -                                              digestVersion,
 -                                              true,
 -                                              metadata,
 -                                              nowInSec,
 -                                              columnFilter,
 -                                              RowFilter.NONE,
 -                                              limits,
 -                                              partitionKey,
 -                                              filter,
 -                                              null);
 -    }
 -
 -    static SinglePartitionReadCommand legacyNamesCommand(boolean isDigest,
 -                                                         int digestVersion,
 -                                                         CFMetaData metadata,
 -                                                         int nowInSec,
 -                                                         ColumnFilter columnFilter,
 -                                                         DecoratedKey partitionKey,
 -                                                         ClusteringIndexNamesFilter filter)
 -    {
 -        // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
 -        return new SinglePartitionReadCommand(isDigest, digestVersion, true, metadata, nowInSec, columnFilter, RowFilter.NONE, DataLimits.NONE, partitionKey, filter,null);
 -    }
 -
      public DecoratedKey partitionKey()
      {
          return partitionKey;
@@@ -1094,12 -1260,22 +1167,21 @@@
  
      private static class Deserializer extends SelectionDeserializer
      {
-         public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, TableMetadata metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
+         public ReadCommand deserialize(DataInputPlus in,
+                                        int version,
+                                        boolean isDigest,
+                                        int digestVersion,
 -                                       boolean isForThrift,
 -                                       CFMetaData metadata,
++                                       TableMetadata metadata,
+                                        int nowInSec,
+                                        ColumnFilter columnFilter,
+                                        RowFilter rowFilter,
+                                        DataLimits limits,
+                                        IndexMetadata index)
          throws IOException
          {
 -            DecoratedKey key = metadata.decorateKey(metadata.getKeyValidator().readValue(in, DatabaseDescriptor.getMaxValueSize()));
 +            DecoratedKey key = metadata.partitioner.decorateKey(metadata.partitionKeyType.readValue(in, DatabaseDescriptor.getMaxValueSize()));
              ClusteringIndexFilter filter = ClusteringIndexFilter.serializer.deserialize(in, version, metadata);
-             return new SinglePartitionReadCommand(isDigest, digestVersion, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter);
 -            return new SinglePartitionReadCommand(isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter, index);
++            return new SinglePartitionReadCommand(isDigest, digestVersion, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter, index);
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
index eecb55e,f2100db..df5e7ce
--- a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
@@@ -204,10 -202,10 +204,10 @@@ public class SecondaryIndexTes
  
          // verify that it's not being indexed under any other value either
          ReadCommand rc = Util.cmd(cfs).build();
-         assertNull(cfs.indexManager.getBestIndexFor(rc));
+         assertNull(rc.index());
  
          // resurrect w/ a newer timestamp
 -        new RowUpdateBuilder(cfs.metadata, 2, "k1").clustering("c").add("birthdate", 1L).build().apply();;
 +        new RowUpdateBuilder(cfs.metadata(), 2, "k1").clustering("c").add("birthdate", 1L).build().apply();;
          assertIndexedOne(cfs, col, 1L);
  
          // verify that row and delete w/ older timestamp does nothing
@@@ -220,15 -218,15 +220,15 @@@
  
          // delete the entire row (w/ newer timestamp this time)
          // todo - checking the # of index searchers for the command is probably not the best thing to test here
 -        RowUpdateBuilder.deleteRow(cfs.metadata, 3, "k1", "c").applyUnsafe();
 +        RowUpdateBuilder.deleteRow(cfs.metadata(), 3, "k1", "c").applyUnsafe();
          rc = Util.cmd(cfs).build();
-         assertNull(cfs.indexManager.getBestIndexFor(rc));
+         assertNull(rc.index());
  
          // make sure obsolete mutations don't generate an index entry
          // todo - checking the # of index searchers for the command is probably not the best thing to test here
 -        new RowUpdateBuilder(cfs.metadata, 3, "k1").clustering("c").add("birthdate", 1L).build().apply();;
 +        new RowUpdateBuilder(cfs.metadata(), 3, "k1").clustering("c").add("birthdate", 1L).build().apply();;
          rc = Util.cmd(cfs).build();
-         assertNull(cfs.indexManager.getBestIndexFor(rc));
+         assertNull(rc.index());
      }
  
      @Test
@@@ -533,10 -520,10 +533,10 @@@
      }
      private void assertIndexedCount(ColumnFamilyStore cfs, ByteBuffer col, Object val, int count)
      {
 -        ColumnDefinition cdef = cfs.metadata.getColumnDefinition(col);
 +        ColumnMetadata cdef = cfs.metadata().getColumn(col);
  
          ReadCommand rc = Util.cmd(cfs).filterOn(cdef.name.toString(), Operator.EQ, ((AbstractType) cdef.cellValueType()).decompose(val)).build();
-         Index.Searcher searcher = cfs.indexManager.getBestIndexFor(rc).searcherFor(rc);
+         Index.Searcher searcher = rc.index().searcherFor(rc);
          if (count != 0)
              assertNotNull(searcher);
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
index 1d4bdb6,b056da1..f79066b
--- a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
+++ b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
@@@ -115,15 -174,16 +115,15 @@@ public class SinglePartitionSliceComman
          QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, s) VALUES ('k1', 's')");
          Assert.assertFalse(QueryProcessor.executeInternal("SELECT s FROM ks.tbl WHERE k='k1'").isEmpty());
  
 -        ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
 +        ColumnFilter columnFilter = ColumnFilter.selection(RegularAndStaticColumns.of(s));
          ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.NONE, false);
-         ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, metadata,
-                                                          FBUtilities.nowInSeconds(),
-                                                          columnFilter,
-                                                          RowFilter.NONE,
-                                                          DataLimits.NONE,
-                                                          key,
-                                                          sliceFilter);
 -        ReadCommand cmd = SinglePartitionReadCommand.create(true,
 -                                                            cfm,
++        ReadCommand cmd = SinglePartitionReadCommand.create(metadata,
+                                                             FBUtilities.nowInSeconds(),
+                                                             columnFilter,
+                                                             RowFilter.NONE,
+                                                             DataLimits.NONE,
+                                                             key,
+                                                             sliceFilter);
  
          // check raw iterator for static cell
          try (ReadExecutionController executionController = cmd.executionController(); UnfilteredPartitionIterator pi = cmd.executeLocally(executionController))
@@@ -170,19 -230,20 +170,18 @@@
      @Test
      public void toCQLStringIsSafeToCall() throws IOException
      {
 -        DecoratedKey key = cfm.decorateKey(ByteBufferUtil.bytes("k1"));
 +        DecoratedKey key = metadata.partitioner.decorateKey(ByteBufferUtil.bytes("k1"));
  
 -        ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
 +        ColumnFilter columnFilter = ColumnFilter.selection(RegularAndStaticColumns.of(s));
          Slice slice = Slice.make(ClusteringBound.BOTTOM, ClusteringBound.inclusiveEndOf(ByteBufferUtil.bytes("i1")));
 -        ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.with(cfm.comparator, slice), false);
 -        ReadCommand cmd = SinglePartitionReadCommand.create(true,
 -                                                            cfm,
 +        ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.with(metadata.comparator, slice), false);
-         ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, metadata,
-                                                          FBUtilities.nowInSeconds(),
-                                                          columnFilter,
-                                                          RowFilter.NONE,
-                                                          DataLimits.NONE,
-                                                          key,
-                                                          sliceFilter);
- 
++        ReadCommand cmd = SinglePartitionReadCommand.create(metadata,
+                                                             FBUtilities.nowInSeconds(),
+                                                             columnFilter,
+                                                             RowFilter.NONE,
+                                                             DataLimits.NONE,
+                                                             key,
+                                                             sliceFilter);
 -
          String ret = cmd.toCQLString();
          Assert.assertNotNull(ret);
          Assert.assertFalse(ret.isEmpty());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
index 986e604,03d89e1..406832a
--- a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
@@@ -1304,16 -1307,16 +1304,15 @@@ public class SASIIndexTes
          ColumnFamilyStore store = loadData(data1, true);
  
          RowFilter filter = RowFilter.create();
 -        filter.add(store.metadata.getColumnDefinition(firstName), Operator.LIKE_CONTAINS, AsciiType.instance.fromString("a"));
 +        filter.add(store.metadata().getColumn(firstName), Operator.LIKE_CONTAINS, AsciiType.instance.fromString("a"));
  
-         ReadCommand command = new PartitionRangeReadCommand(store.metadata(),
-                                                             FBUtilities.nowInSeconds(),
-                                                             ColumnFilter.all(store.metadata()),
-                                                             filter,
-                                                             DataLimits.NONE,
-                                                             DataRange.allData(store.metadata().partitioner),
-                                                             Optional.empty());
- 
+         ReadCommand command =
 -            PartitionRangeReadCommand.create(false,
 -                                             store.metadata,
++            PartitionRangeReadCommand.create(store.metadata(),
+                                              FBUtilities.nowInSeconds(),
 -                                             ColumnFilter.all(store.metadata),
++                                             ColumnFilter.all(store.metadata()),
+                                              filter,
+                                              DataLimits.NONE,
 -                                             DataRange.allData(store.metadata.partitioner));
++                                             DataRange.allData(store.metadata().partitioner));
          try
          {
              new QueryPlan(store, command, 0).execute(ReadExecutionController.empty());
@@@ -2267,16 -2270,17 +2266,16 @@@
              put("key1", Pair.create("Pavel", 14));
          }}, false);
  
 -        ColumnIndex index = ((SASIIndex) store.indexManager.getIndexByName("first_name")).getIndex();
 +        ColumnIndex index = ((SASIIndex) store.indexManager.getIndexByName(store.name + "_first_name")).getIndex();
          IndexMemtable beforeFlushMemtable = index.getCurrentMemtable();
  
-         PartitionRangeReadCommand command = new PartitionRangeReadCommand(store.metadata(),
-                                                                           FBUtilities.nowInSeconds(),
-                                                                           ColumnFilter.all(store.metadata()),
-                                                                           RowFilter.NONE,
-                                                                           DataLimits.NONE,
-                                                                           DataRange.allData(store.getPartitioner()),
-                                                                           Optional.empty());
+         PartitionRangeReadCommand command =
 -            PartitionRangeReadCommand.create(false,
 -                                             store.metadata,
++            PartitionRangeReadCommand.create(store.metadata(),
+                                              FBUtilities.nowInSeconds(),
 -                                             ColumnFilter.all(store.metadata),
++                                             ColumnFilter.all(store.metadata()),
+                                              RowFilter.NONE,
+                                              DataLimits.NONE,
+                                              DataRange.allData(store.getPartitioner()));
  
          QueryController controller = new QueryController(store, command, Integer.MAX_VALUE);
          org.apache.cassandra.index.sasi.plan.Expression expression =
@@@ -2408,15 -2412,16 +2407,15 @@@
  
          RowFilter filter = RowFilter.create();
          for (Expression e : expressions)
 -            filter.add(store.metadata.getColumnDefinition(e.name), e.op, e.value);
 +            filter.add(store.metadata().getColumn(e.name), e.op, e.value);
  
-         ReadCommand command = new PartitionRangeReadCommand(store.metadata(),
-                                                             FBUtilities.nowInSeconds(),
-                                                             columnFilter,
-                                                             filter,
-                                                             DataLimits.cqlLimits(maxResults),
-                                                             range,
-                                                             Optional.empty());
+         ReadCommand command =
 -            PartitionRangeReadCommand.create(false,
 -                                             store.metadata,
++            PartitionRangeReadCommand.create(store.metadata(),
+                                              FBUtilities.nowInSeconds(),
+                                              columnFilter,
+                                              filter,
 -                                             DataLimits.thriftLimits(maxResults, DataLimits.NO_LIMIT),
++                                             DataLimits.cqlLimits(maxResults),
+                                              range);
  
          return command.executeLocally(command.executionController());
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e4d000c/test/unit/org/apache/cassandra/service/ReadExecutorTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/service/ReadExecutorTest.java
index fca8eca,0000000..7630cc6
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/service/ReadExecutorTest.java
+++ b/test/unit/org/apache/cassandra/service/ReadExecutorTest.java
@@@ -1,215 -1,0 +1,215 @@@
 +/*
 + * 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.service;
 +
 +import java.net.InetAddress;
 +import java.util.List;
 +import java.util.concurrent.TimeUnit;
 +
 +import com.google.common.collect.ImmutableList;
 +import org.junit.Before;
 +import org.junit.BeforeClass;
 +import org.junit.Test;
 +
 +import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.Util;
 +import org.apache.cassandra.db.ColumnFamilyStore;
 +import org.apache.cassandra.db.ConsistencyLevel;
 +import org.apache.cassandra.db.Keyspace;
 +import org.apache.cassandra.db.SinglePartitionReadCommand;
 +import org.apache.cassandra.exceptions.ReadFailureException;
 +import org.apache.cassandra.exceptions.ReadTimeoutException;
 +import org.apache.cassandra.exceptions.RequestFailureReason;
 +import org.apache.cassandra.net.MessageOut;
 +import org.apache.cassandra.net.MessagingService;
 +import org.apache.cassandra.schema.KeyspaceParams;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.fail;
 +
 +public class ReadExecutorTest
 +{
 +    static Keyspace ks;
 +    static ColumnFamilyStore cfs;
 +    static List<InetAddress> targets;
 +
 +    @BeforeClass
 +    public static void setUpClass() throws Throwable
 +    {
 +        SchemaLoader.loadSchema();
 +        SchemaLoader.createKeyspace("Foo", KeyspaceParams.simple(3), SchemaLoader.standardCFMD("Foo", "Bar"));
 +        ks = Keyspace.open("Foo");
 +        cfs = ks.getColumnFamilyStore("Bar");
 +        targets = ImmutableList.of(InetAddress.getByName("127.0.0.255"), InetAddress.getByName("127.0.0.254"), InetAddress.getByName("127.0.0.253"));
 +        cfs.sampleLatencyNanos = 0;
 +    }
 +
 +    @Before
 +    public void resetCounters() throws Throwable
 +    {
 +        cfs.metric.speculativeInsufficientReplicas.dec(cfs.metric.speculativeInsufficientReplicas.getCount());
 +        cfs.metric.speculativeRetries.dec(cfs.metric.speculativeRetries.getCount());
 +        cfs.metric.speculativeFailedRetries.dec(cfs.metric.speculativeFailedRetries.getCount());
 +    }
 +
 +    /**
 +     * If speculation would have been beneficial but could not be attempted due to lack of replicas
 +     * count that it occured
 +     */
 +    @Test
 +    public void testUnableToSpeculate() throws Throwable
 +    {
 +        assertEquals(0, cfs.metric.speculativeInsufficientReplicas.getCount());
 +        assertEquals(0, ks.metric.speculativeInsufficientReplicas.getCount());
 +        AbstractReadExecutor executor = new AbstractReadExecutor.NeverSpeculatingReadExecutor(ks, cfs, new MockSinglePartitionReadCommand(), ConsistencyLevel.LOCAL_QUORUM, targets, System.nanoTime(), true);
 +        executor.maybeTryAdditionalReplicas();
 +        try
 +        {
 +            executor.get();
 +            fail();
 +        }
 +        catch (ReadTimeoutException e)
 +        {
 +            //expected
 +        }
 +        assertEquals(1, cfs.metric.speculativeInsufficientReplicas.getCount());
 +        assertEquals(1, ks.metric.speculativeInsufficientReplicas.getCount());
 +
 +        //Shouldn't increment
 +        executor = new AbstractReadExecutor.NeverSpeculatingReadExecutor(ks, cfs, new MockSinglePartitionReadCommand(), ConsistencyLevel.LOCAL_QUORUM, targets, System.nanoTime(), false);
 +        executor.maybeTryAdditionalReplicas();
 +        try
 +        {
 +            executor.get();
 +            fail();
 +        }
 +        catch (ReadTimeoutException e)
 +        {
 +            //expected
 +        }
 +        assertEquals(1, cfs.metric.speculativeInsufficientReplicas.getCount());
 +        assertEquals(1, ks.metric.speculativeInsufficientReplicas.getCount());
 +    }
 +
 +    /**
 +     *  Test that speculation when it is attempted is countedc, and when it succeed
 +     *  no failure is counted.
 +     */
 +    @Test
 +    public void testSpeculateSucceeded() throws Throwable
 +    {
 +        assertEquals(0, cfs.metric.speculativeRetries.getCount());
 +        assertEquals(0, cfs.metric.speculativeFailedRetries.getCount());
 +        assertEquals(0, ks.metric.speculativeRetries.getCount());
 +        assertEquals(0, ks.metric.speculativeFailedRetries.getCount());
 +        AbstractReadExecutor executor = new AbstractReadExecutor.SpeculatingReadExecutor(ks, cfs, new MockSinglePartitionReadCommand(TimeUnit.DAYS.toMillis(365)), ConsistencyLevel.LOCAL_QUORUM, targets, System.nanoTime());
 +        executor.maybeTryAdditionalReplicas();
 +        new Thread()
 +        {
 +            @Override
 +            public void run()
 +            {
 +                //Failures end the read promptly but don't require mock data to be suppleid
 +                executor.handler.onFailure(targets.get(0), RequestFailureReason.READ_TOO_MANY_TOMBSTONES);
 +                executor.handler.onFailure(targets.get(1), RequestFailureReason.READ_TOO_MANY_TOMBSTONES);
 +                executor.handler.condition.signalAll();
 +            }
 +        }.start();
 +
 +        try
 +        {
 +            executor.get();
 +            fail();
 +        }
 +        catch (ReadFailureException e)
 +        {
 +            //expected
 +        }
 +        assertEquals(1, cfs.metric.speculativeRetries.getCount());
 +        assertEquals(0, cfs.metric.speculativeFailedRetries.getCount());
 +        assertEquals(1, ks.metric.speculativeRetries.getCount());
 +        assertEquals(0, ks.metric.speculativeFailedRetries.getCount());
 +
 +    }
 +
 +    /**
 +     * Test that speculation failure statistics are incremented if speculation occurs
 +     * and the read still times out.
 +     */
 +    @Test
 +    public void testSpeculateFailed() throws Throwable
 +    {
 +        assertEquals(0, cfs.metric.speculativeRetries.getCount());
 +        assertEquals(0, cfs.metric.speculativeFailedRetries.getCount());
 +        assertEquals(0, ks.metric.speculativeRetries.getCount());
 +        assertEquals(0, ks.metric.speculativeFailedRetries.getCount());
 +        AbstractReadExecutor executor = new AbstractReadExecutor.SpeculatingReadExecutor(ks, cfs, new MockSinglePartitionReadCommand(), ConsistencyLevel.LOCAL_QUORUM, targets, System.nanoTime());
 +        executor.maybeTryAdditionalReplicas();
 +        try
 +        {
 +            executor.get();
 +            fail();
 +        }
 +        catch (ReadTimeoutException e)
 +        {
 +            //expected
 +        }
 +        assertEquals(1, cfs.metric.speculativeRetries.getCount());
 +        assertEquals(1, cfs.metric.speculativeFailedRetries.getCount());
 +        assertEquals(1, ks.metric.speculativeRetries.getCount());
 +        assertEquals(1, ks.metric.speculativeFailedRetries.getCount());
 +    }
 +
 +    public static class MockSinglePartitionReadCommand extends SinglePartitionReadCommand
 +    {
 +        private final long timeout;
 +
 +        MockSinglePartitionReadCommand()
 +        {
 +            this(0);
 +        }
 +
 +        MockSinglePartitionReadCommand(long timeout)
 +        {
-             super(false, 0, cfs.metadata(), 0, null, null, null, Util.dk("ry@n_luvs_teh_y@nk33z"), null);
++            super(false, 0, cfs.metadata(), 0, null, null, null, Util.dk("ry@n_luvs_teh_y@nk33z"), null, null);
 +            this.timeout = timeout;
 +        }
 +
 +        @Override
 +        public long getTimeout()
 +        {
 +            return timeout;
 +        }
 +
 +        @Override
 +        public MessageOut createMessage()
 +        {
 +            return new MessageOut(MessagingService.Verb.BATCH_REMOVE)
 +            {
 +                @Override
 +                public int serializedSize(int version)
 +                {
 +                    return 0;
 +                }
 +            };
 +        }
 +
 +    }
 +
 +}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[09/10] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


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

Branch: refs/heads/cassandra-3.11
Commit: 7ad1945ee7592990027bee4fe6bbfcac72940954
Parents: 6d6081e 7f297bc
Author: Aleksey Yeschenko <al...@yeschenko.com>
Authored: Wed Aug 30 17:04:49 2017 +0100
Committer: Aleksey Yeschenko <al...@yeschenko.com>
Committed: Wed Aug 30 17:08:53 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cql3/statements/SelectStatement.java        |  16 +-
 .../db/AbstractReadCommandBuilder.java          |   2 +-
 .../cassandra/db/PartitionRangeReadCommand.java | 149 ++++++++++---
 .../org/apache/cassandra/db/ReadCommand.java    | 149 +++++++------
 .../db/SinglePartitionReadCommand.java          | 209 +++++++++++++++----
 .../cassandra/index/SecondaryIndexManager.java  |   9 +-
 .../internal/composites/CompositesSearcher.java |   6 +-
 .../index/internal/keys/KeysSearcher.java       |   3 +-
 .../cassandra/service/AbstractReadExecutor.java |   4 +-
 .../service/pager/PartitionRangeQueryPager.java |   8 +-
 .../cassandra/thrift/CassandraServer.java       |  68 +++---
 test/unit/org/apache/cassandra/Util.java        |  26 +--
 .../apache/cassandra/db/SecondaryIndexTest.java |  10 +-
 .../db/SinglePartitionSliceCommandTest.java     |  45 ++--
 .../cassandra/index/sasi/SASIIndexTest.java     |  46 ++--
 .../cassandra/io/sstable/SSTableReaderTest.java |   2 +-
 17 files changed, 479 insertions(+), 274 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index c4aee3a,aca9e1f..d848eff
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,11 -1,6 +1,12 @@@
 -3.0.15
 +3.11.1
 + * Fix cassandra-stress hang issues when an error during cluster connection happens (CASSANDRA-12938)
 + * Better bootstrap failure message when blocked by (potential) range movement (CASSANDRA-13744)
 + * "ignore" option is ignored in sstableloader (CASSANDRA-13721)
 + * Deadlock in AbstractCommitLogSegmentManager (CASSANDRA-13652)
 + * Duplicate the buffer before passing it to analyser in SASI operation (CASSANDRA-13512)
 + * Properly evict pstmts from prepared statements cache (CASSANDRA-13641)
 +Merged from 3.0:
+  * Fix race condition in read command serialization (CASSANDRA-13363)
 - * Enable segement creation before recovering commitlogs (CASSANDRA-13587)
   * Fix AssertionError in short read protection (CASSANDRA-13747)
   * Don't skip corrupted sstables on startup (CASSANDRA-13620)
   * Fix the merging of cells with different user type versions (CASSANDRA-13776)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index d0487a3,9e557e0..f7b6660
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@@ -121,41 -129,74 +130,108 @@@ public class PartitionRangeReadCommand 
          return dataRange.isNamesQuery();
      }
  
 -    public PartitionRangeReadCommand forSubRange(AbstractBounds<PartitionPosition> range)
 +    /**
 +     * Returns an equivalent command but that only queries data within the provided range.
 +     *
 +     * @param range the sub-range to restrict the command to. This method <b>assumes</b> that this is a proper sub-range
 +     * of the command this is applied to.
 +     * @param isRangeContinuation whether {@code range} is a direct continuation of whatever previous range we have
 +     * queried. This matters for the {@code DataLimits} that may contain states when we do paging and in the context of
 +     * parallel queries: that state only make sense if the range queried is indeed the follow-up of whatever range we've
 +     * previously query (that yield said state). In practice this means that ranges for which {@code isRangeContinuation}
 +     * is false may have to be slightly pessimistic when counting data and may include a little bit than necessary, and
 +     * this should be dealt with post-query (in the case of {@code StorageProxy.getRangeSlice()}, which uses this method
 +     * for replica queries, this is dealt with by re-counting results on the coordinator). Note that if this is the
 +     * first range we queried, then the {@code DataLimits} will have not state and the value of this parameter doesn't
 +     * matter.
 +     */
 +    public PartitionRangeReadCommand forSubRange(AbstractBounds<PartitionPosition> range, boolean isRangeContinuation)
      {
-         DataRange newRange = dataRange().forSubRange(range);
 +        // If we're not a continuation of whatever range we've previously queried, we should ignore the states of the
 +        // DataLimits as it's either useless, or misleading. This is particularly important for GROUP BY queries, where
 +        // DataLimits.CQLGroupByLimits.GroupByAwareCounter assumes that if GroupingState.hasClustering(), then we're in
 +        // the middle of a group, but we can't make that assumption if we query and range "in advance" of where we are
 +        // on the ring.
-         DataLimits newLimits = isRangeContinuation ? limits() : limits().withoutState();
-         return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), newLimits, newRange, index);
+         return new PartitionRangeReadCommand(isDigestQuery(),
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
 -                                             limits(),
++                                             isRangeContinuation ? limits() : limits().withoutState(),
+                                              dataRange().forSubRange(range),
+                                              indexMetadata());
      }
  
      public PartitionRangeReadCommand copy()
      {
-         return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange(), index);
+         return new PartitionRangeReadCommand(isDigestQuery(),
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              dataRange(),
+                                              indexMetadata());
+     }
+ 
+     public PartitionRangeReadCommand copyAsDigestQuery()
+     {
+         return new PartitionRangeReadCommand(true,
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              dataRange(),
+                                              indexMetadata());
+     }
+ 
++    public ReadCommand withUpdatedLimit(DataLimits newLimits)
++    {
++        return new PartitionRangeReadCommand(isDigestQuery(),
++                                             digestVersion(),
++                                             isForThrift(),
++                                             metadata(),
++                                             nowInSec(),
++                                             columnFilter(),
++                                             rowFilter(),
++                                             newLimits,
++                                             dataRange(),
++                                             indexMetadata());
++    }
++
+     public PartitionRangeReadCommand withUpdatedDataRange(DataRange newDataRange)
+     {
+         return new PartitionRangeReadCommand(isDigestQuery(),
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              newDataRange,
+                                              indexMetadata());
      }
  
-     public PartitionRangeReadCommand withUpdatedLimit(DataLimits newLimits)
+     public PartitionRangeReadCommand withUpdatedLimitsAndDataRange(DataLimits newLimits, DataRange newDataRange)
      {
-         return new PartitionRangeReadCommand(metadata(), nowInSec(), columnFilter(), rowFilter(), newLimits, dataRange(), index);
+         return new PartitionRangeReadCommand(isDigestQuery(),
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              newLimits,
+                                              newDataRange,
+                                              indexMetadata());
      }
  
      public long getTimeout()
@@@ -196,7 -237,8 +272,8 @@@
          metric.rangeLatency.addNano(latencyNanos);
      }
  
-     protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadExecutionController executionController)
+     @VisibleForTesting
 -    public UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadOrderGroup orderGroup)
++    public UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadExecutionController executionController)
      {
          ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange()));
          Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().getKeyValidator()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ReadCommand.java
index 050546c,66985b6..54389f0
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@@ -322,7 -329,12 +341,12 @@@ public abstract class ReadCommand exten
       */
      public abstract ReadCommand copy();
  
+     /**
+      * Returns a copy of this command with isDigestQuery set to true.
+      */
+     public abstract ReadCommand copyAsDigestQuery();
+ 
 -    protected abstract UnfilteredPartitionIterator queryStorage(ColumnFamilyStore cfs, ReadOrderGroup orderGroup);
 +    protected abstract UnfilteredPartitionIterator queryStorage(ColumnFamilyStore cfs, ReadExecutionController executionController);
  
      protected abstract int oldestUnrepairedTombstone();
  
@@@ -684,9 -630,9 +705,9 @@@
              out.writeInt(command.nowInSec());
              ColumnFilter.serializer.serialize(command.columnFilter(), out, version);
              RowFilter.serializer.serialize(command.rowFilter(), out, version);
 -            DataLimits.serializer.serialize(command.limits(), out, version);
 +            DataLimits.serializer.serialize(command.limits(), out, version, command.metadata.comparator);
-             if (command.index.isPresent())
-                 IndexMetadata.serializer.serialize(command.index.get(), out, version);
+             if (null != command.index)
+                 IndexMetadata.serializer.serialize(command.index, out, version);
  
              command.serializeSelection(out, version);
          }
@@@ -705,10 -651,8 +726,8 @@@
              int nowInSec = in.readInt();
              ColumnFilter columnFilter = ColumnFilter.serializer.deserialize(in, version, metadata);
              RowFilter rowFilter = RowFilter.serializer.deserialize(in, version, metadata);
 -            DataLimits limits = DataLimits.serializer.deserialize(in, version);
 +            DataLimits limits = DataLimits.serializer.deserialize(in, version,  metadata.comparator);
-             Optional<IndexMetadata> index = hasIndex
-                                             ? deserializeIndexMetadata(in, version, metadata)
-                                             : Optional.empty();
+             IndexMetadata index = hasIndex ? deserializeIndexMetadata(in, version, metadata) : null;
  
              return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
          }
@@@ -721,12 -665,13 +740,12 @@@
              }
              catch (UnknownIndexException e)
              {
 -                String message = String.format("Couldn't find a defined index on %s.%s with the id %s. " +
 -                                               "If an index was just created, this is likely due to the schema not " +
 -                                               "being fully propagated. Local read will proceed without using the " +
 -                                               "index. Please wait for schema agreement after index creation.",
 -                                               cfm.ksName, cfm.cfName, e.indexId.toString());
 -                logger.info(message);
 +                logger.info("Couldn't find a defined index on {}.{} with the id {}. " +
 +                            "If an index was just created, this is likely due to the schema not " +
 +                            "being fully propagated. Local read will proceed without using the " +
 +                            "index. Please wait for schema agreement after index creation.",
 +                            cfm.ksName, cfm.cfName, e.indexId);
-                 return Optional.empty();
+                 return null;
              }
          }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 5d93c65,00464ca..c7080e7
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@@ -268,7 -316,82 +319,97 @@@ public class SinglePartitionReadComman
  
      public SinglePartitionReadCommand copy()
      {
-         return new SinglePartitionReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), partitionKey(), clusteringIndexFilter());
+         return new SinglePartitionReadCommand(isDigestQuery(),
+                                               digestVersion(),
+                                               isForThrift(),
+                                               metadata(),
+                                               nowInSec(),
+                                               columnFilter(),
+                                               rowFilter(),
+                                               limits(),
+                                               partitionKey(),
+                                               clusteringIndexFilter(),
+                                               indexMetadata());
+     }
+ 
+     public SinglePartitionReadCommand copyAsDigestQuery()
+     {
+         return new SinglePartitionReadCommand(true,
+                                               digestVersion(),
+                                               isForThrift(),
+                                               metadata(),
+                                               nowInSec(),
+                                               columnFilter(),
+                                               rowFilter(),
+                                               limits(),
+                                               partitionKey(),
+                                               clusteringIndexFilter(),
+                                               indexMetadata());
+     }
+ 
++    public SinglePartitionReadCommand withUpdatedLimit(DataLimits newLimits)
++    {
++        return new SinglePartitionReadCommand(isDigestQuery(),
++                                              digestVersion(),
++                                              isForThrift(),
++                                              metadata(),
++                                              nowInSec(),
++                                              columnFilter(),
++                                              rowFilter(),
++                                              newLimits,
++                                              partitionKey(),
++                                              clusteringIndexFilter(),
++                                              indexMetadata());
++    }
++
+     public SinglePartitionReadCommand withUpdatedClusteringIndexFilter(ClusteringIndexFilter filter)
+     {
+         return new SinglePartitionReadCommand(isDigestQuery(),
+                                               digestVersion(),
+                                               isForThrift(),
+                                               metadata(),
+                                               nowInSec(),
+                                               columnFilter(),
+                                               rowFilter(),
+                                               limits(),
+                                               partitionKey(),
+                                               filter,
+                                               indexMetadata());
+     }
+ 
+     static SinglePartitionReadCommand legacySliceCommand(boolean isDigest,
+                                                          int digestVersion,
+                                                          CFMetaData metadata,
+                                                          int nowInSec,
+                                                          ColumnFilter columnFilter,
+                                                          DataLimits limits,
+                                                          DecoratedKey partitionKey,
+                                                          ClusteringIndexSliceFilter filter)
+     {
+         // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
+         return new SinglePartitionReadCommand(isDigest,
+                                               digestVersion,
+                                               true,
+                                               metadata,
+                                               nowInSec,
+                                               columnFilter,
+                                               RowFilter.NONE,
+                                               limits,
+                                               partitionKey,
+                                               filter,
+                                               null);
+     }
+ 
+     static SinglePartitionReadCommand legacyNamesCommand(boolean isDigest,
+                                                          int digestVersion,
+                                                          CFMetaData metadata,
+                                                          int nowInSec,
+                                                          ColumnFilter columnFilter,
+                                                          DecoratedKey partitionKey,
+                                                          ClusteringIndexNamesFilter filter)
+     {
+         // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
+         return new SinglePartitionReadCommand(isDigest, digestVersion, true, metadata, nowInSec, columnFilter, RowFilter.NONE, DataLimits.NONE, partitionKey, filter,null);
      }
  
      public DecoratedKey partitionKey()
@@@ -334,26 -457,12 +475,12 @@@
                        lastReturned == null ? clusteringIndexFilter() : clusteringIndexFilter.forPaging(metadata().comparator, lastReturned, false));
      }
  
-     public SinglePartitionReadCommand withUpdatedLimit(DataLimits newLimits)
-     {
-         return new SinglePartitionReadCommand(isDigestQuery(),
-                                               digestVersion(),
-                                               isForThrift(),
-                                               metadata(),
-                                               nowInSec(),
-                                               columnFilter(),
-                                               rowFilter(),
-                                               newLimits,
-                                               partitionKey,
-                                               clusteringIndexFilter);
-     }
- 
 -    public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState) throws RequestExecutionException
 +    public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState, long queryStartNanoTime) throws RequestExecutionException
      {
 -        return StorageProxy.read(Group.one(this), consistency, clientState);
 +        return StorageProxy.read(Group.one(this), consistency, clientState, queryStartNanoTime);
      }
  
 -    public SinglePartitionPager getPager(PagingState pagingState, int protocolVersion)
 +    public SinglePartitionPager getPager(PagingState pagingState, ProtocolVersion protocolVersion)
      {
          return getPager(this, pagingState, protocolVersion);
      }
@@@ -449,7 -558,7 +576,7 @@@
                  final int rowsToCache = metadata().params.caching.rowsPerPartitionToCache();
  
                  @SuppressWarnings("resource") // we close on exception or upon closing the result of this method
-                 UnfilteredRowIterator iter = SinglePartitionReadCommand.fullPartitionRead(metadata(), nowInSec(), partitionKey()).queryMemtableAndDisk(cfs, executionController);
 -                UnfilteredRowIterator iter = fullPartitionRead(metadata(), nowInSec(), partitionKey()).queryMemtableAndDisk(cfs, readOp);
++                UnfilteredRowIterator iter = fullPartitionRead(metadata(), nowInSec(), partitionKey()).queryMemtableAndDisk(cfs, executionController);
                  try
                  {
                      // Use a custom iterator instead of DataLimits to avoid stopping the original iterator

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
index e777620,f8a7c66..2007800
--- a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
@@@ -109,65 -108,43 +109,67 @@@ public class CompositesSearcher extend
                          nextEntry = index.decodeEntry(indexKey, indexHits.next());
                      }
  
 -                    // Gather all index hits belonging to the same partition and query the data for those hits.
 -                    // TODO: it's much more efficient to do 1 read for all hits to the same partition than doing
 -                    // 1 read per index hit. However, this basically mean materializing all hits for a partition
 -                    // in memory so we should consider adding some paging mechanism. However, index hits should
 -                    // be relatively small so it's much better than the previous code that was materializing all
 -                    // *data* for a given partition.
 -                    BTreeSet.Builder<Clustering> clusterings = BTreeSet.builder(index.baseCfs.getComparator());
 -                    List<IndexEntry> entries = new ArrayList<>();
 +                    SinglePartitionReadCommand dataCmd;
                      DecoratedKey partitionKey = index.baseCfs.decorateKey(nextEntry.indexedKey);
 -
 -                    while (nextEntry != null && partitionKey.getKey().equals(nextEntry.indexedKey))
 +                    List<IndexEntry> entries = new ArrayList<>();
 +                    if (isStaticColumn())
                      {
 -                        // We're queried a slice of the index, but some hits may not match some of the clustering column constraints
 -                        if (isMatchingEntry(partitionKey, nextEntry, command))
 -                        {
 -                            clusterings.add(nextEntry.indexedEntryClustering);
 -                            entries.add(nextEntry);
 +                        // The index hit may not match the commad key constraint
 +                        if (!isMatchingEntry(partitionKey, nextEntry, command)) {
 +                            nextEntry = indexHits.hasNext() ? index.decodeEntry(indexKey, indexHits.next()) : null;
 +                            continue;
                          }
  
 +                        // If the index is on a static column, we just need to do a full read on the partition.
 +                        // Note that we want to re-use the command.columnFilter() in case of future change.
 +                        dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata,
 +                                                                    command.nowInSec(),
 +                                                                    command.columnFilter(),
 +                                                                    RowFilter.NONE,
 +                                                                    DataLimits.NONE,
 +                                                                    partitionKey,
 +                                                                    new ClusteringIndexSliceFilter(Slices.ALL, false));
 +                        entries.add(nextEntry);
                          nextEntry = indexHits.hasNext() ? index.decodeEntry(indexKey, indexHits.next()) : null;
                      }
 +                    else
 +                    {
 +                        // Gather all index hits belonging to the same partition and query the data for those hits.
 +                        // TODO: it's much more efficient to do 1 read for all hits to the same partition than doing
 +                        // 1 read per index hit. However, this basically mean materializing all hits for a partition
 +                        // in memory so we should consider adding some paging mechanism. However, index hits should
 +                        // be relatively small so it's much better than the previous code that was materializing all
 +                        // *data* for a given partition.
 +                        BTreeSet.Builder<Clustering> clusterings = BTreeSet.builder(index.baseCfs.getComparator());
 +                        while (nextEntry != null && partitionKey.getKey().equals(nextEntry.indexedKey))
 +                        {
 +                            // We're queried a slice of the index, but some hits may not match some of the clustering column constraints
 +                            if (isMatchingEntry(partitionKey, nextEntry, command))
 +                            {
 +                                clusterings.add(nextEntry.indexedEntryClustering);
 +                                entries.add(nextEntry);
 +                            }
 +
 +                            nextEntry = indexHits.hasNext() ? index.decodeEntry(indexKey, indexHits.next()) : null;
 +                        }
  
 -                    // Because we've eliminated entries that don't match the clustering columns, it's possible we added nothing
 -                    if (clusterings.isEmpty())
 -                        continue;
 +                        // Because we've eliminated entries that don't match the clustering columns, it's possible we added nothing
 +                        if (clusterings.isEmpty())
 +                            continue;
 +
 +                        // Query the gathered index hits. We still need to filter stale hits from the resulting query.
 +                        ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(clusterings.build(), false);
-                         dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata,
++                        dataCmd = SinglePartitionReadCommand.create(isForThrift(),
++                                                                    index.baseCfs.metadata,
 +                                                                    command.nowInSec(),
 +                                                                    command.columnFilter(),
 +                                                                    command.rowFilter(),
 +                                                                    DataLimits.NONE,
 +                                                                    partitionKey,
-                                                                     filter);
++                                                                    filter,
++                                                                    null);
 +                    }
  
 -                    // Query the gathered index hits. We still need to filter stale hits from the resulting query.
 -                    ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(clusterings.build(), false);
 -                    SinglePartitionReadCommand dataCmd = SinglePartitionReadCommand.create(isForThrift(),
 -                                                                                           index.baseCfs.metadata,
 -                                                                                           command.nowInSec(),
 -                                                                                           command.columnFilter(),
 -                                                                                           command.rowFilter(),
 -                                                                                           DataLimits.NONE,
 -                                                                                           partitionKey,
 -                                                                                           filter,
 -                                                                                           null);
                      @SuppressWarnings("resource") // We close right away if empty, and if it's assign to next it will be called either
                      // by the next caller of next, or through closing this iterator is this come before.
                      UnfilteredRowIterator dataIter =

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
index 5ba13a4,ea79017..e6ad3d4
--- a/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
@@@ -17,16 -17,14 +17,12 @@@
   */
  package org.apache.cassandra.service.pager;
  
- import java.util.Optional;
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
--
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.filter.DataLimits;
  import org.apache.cassandra.db.rows.Row;
  import org.apache.cassandra.dht.*;
  import org.apache.cassandra.exceptions.RequestExecutionException;
- import org.apache.cassandra.index.Index;
- import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.transport.ProtocolVersion;
  
  /**
   * Pages a PartitionRangeReadCommand.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/CassandraServer.java
index e71f512,cb74b15..f43b7a4
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@@ -1547,17 -1519,17 +1547,17 @@@ public class CassandraServer implement
                  ColumnFilter columns = makeColumnFilter(metadata, column_parent, predicate);
                  ClusteringIndexFilter filter = toInternalFilter(metadata, column_parent, predicate);
                  DataLimits limits = getLimits(range.count, metadata.isSuper() && !column_parent.isSetSuper_column(), predicate);
-                 PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                               0,
-                                                                               true,
-                                                                               metadata,
-                                                                               nowInSec,
-                                                                               columns,
-                                                                               ThriftConversion.rowFilterFromThrift(metadata, range.row_filter),
-                                                                               limits,
-                                                                               new DataRange(bounds, filter),
-                                                                               Optional.empty());
+ 
+                 PartitionRangeReadCommand cmd =
+                     PartitionRangeReadCommand.create(true,
+                                                      metadata,
+                                                      nowInSec,
+                                                      columns,
+                                                      ThriftConversion.rowFilterFromThrift(metadata, range.row_filter),
+                                                      limits,
+                                                      new DataRange(bounds, filter));
+ 
 -                try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
 +                try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel, queryStartNanoTime))
                  {
                      assert results != null;
                      return thriftifyKeySlices(results, column_parent, limits.perPartitionCount());
@@@ -1640,19 -1611,19 +1640,19 @@@
                  ClusteringIndexFilter filter = new ClusteringIndexSliceFilter(Slices.ALL, false);
                  DataLimits limits = getLimits(range.count, true, Integer.MAX_VALUE);
                  Clustering pageFrom = metadata.isSuper()
 -                                    ? new Clustering(start_column)
 +                                    ? Clustering.make(start_column)
                                      : LegacyLayout.decodeCellName(metadata, start_column).clustering;
-                 PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                               0,
-                                                                               true,
-                                                                               metadata,
-                                                                               nowInSec,
-                                                                               ColumnFilter.all(metadata),
-                                                                               RowFilter.NONE,
-                                                                               limits,
-                                                                               new DataRange(bounds, filter).forPaging(bounds, metadata.comparator, pageFrom, true),
-                                                                               Optional.empty());
+ 
+                 PartitionRangeReadCommand cmd =
+                     PartitionRangeReadCommand.create(true,
+                                                      metadata,
+                                                      nowInSec,
+                                                      ColumnFilter.all(metadata),
+                                                      RowFilter.NONE,
+                                                      limits,
+                                                      new DataRange(bounds, filter).forPaging(bounds, metadata.comparator, pageFrom, true));
+ 
 -                try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
 +                try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel, queryStartNanoTime))
                  {
                      return thriftifyKeySlices(results, new ColumnParent(column_family), limits.perPartitionCount());
                  }
@@@ -1735,24 -1705,20 +1735,20 @@@
              ColumnFilter columns = makeColumnFilter(metadata, column_parent, column_predicate);
              ClusteringIndexFilter filter = toInternalFilter(metadata, column_parent, column_predicate);
              DataLimits limits = getLimits(index_clause.count, metadata.isSuper() && !column_parent.isSetSuper_column(), column_predicate);
-             PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                           0,
-                                                                           true,
-                                                                           metadata,
-                                                                           nowInSec,
-                                                                           columns,
-                                                                           ThriftConversion.rowFilterFromThrift(metadata, index_clause.expressions),
-                                                                           limits,
-                                                                           new DataRange(bounds, filter),
-                                                                           Optional.empty());
-             // If there's a secondary index that the command can use, have it validate
-             // the request parameters. Note that as a side effect, if a viable Index is
-             // identified by the CFS's index manager, it will be cached in the command
-             // and serialized during distribution to replicas in order to avoid performing
-             // further lookups.
+ 
+             PartitionRangeReadCommand cmd =
+                 PartitionRangeReadCommand.create(true,
+                                                  metadata,
+                                                  nowInSec,
+                                                  columns,
+                                                  ThriftConversion.rowFilterFromThrift(metadata, index_clause.expressions),
+                                                  limits,
+                                                  new DataRange(bounds, filter));
+ 
+             // If there's a secondary index that the command can use, have it validate the request parameters.
              cmd.maybeValidateIndex();
  
 -            try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
 +            try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel, queryStartNanoTime))
              {
                  return thriftifyKeySlices(results, column_parent, limits.perPartitionCount());
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/Util.java
index 3fa24d7,d758efe..a3ad653
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@@ -668,33 -627,9 +668,9 @@@ public class Uti
  
      public static UnfilteredPartitionIterator executeLocally(PartitionRangeReadCommand command,
                                                               ColumnFamilyStore cfs,
 -                                                             ReadOrderGroup orderGroup)
 +                                                             ReadExecutionController controller)
      {
-         return new InternalPartitionRangeReadCommand(command).queryStorageInternal(cfs, controller);
-     }
- 
-     private static final class InternalPartitionRangeReadCommand extends PartitionRangeReadCommand
-     {
- 
-         private InternalPartitionRangeReadCommand(PartitionRangeReadCommand original)
-         {
-             super(original.isDigestQuery(),
-                   original.digestVersion(),
-                   original.isForThrift(),
-                   original.metadata(),
-                   original.nowInSec(),
-                   original.columnFilter(),
-                   original.rowFilter(),
-                   original.limits(),
-                   original.dataRange(),
-                   Optional.empty());
-         }
- 
-         private UnfilteredPartitionIterator queryStorageInternal(ColumnFamilyStore cfs,
-                                                                  ReadExecutionController controller)
-         {
-             return queryStorage(cfs, controller);
-         }
 -        return command.queryStorage(cfs, orderGroup);
++        return command.queryStorage(cfs, controller);
      }
  
      public static Closeable markDirectoriesUnwriteable(ColumnFamilyStore cfs)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
index a037d90,2457c4a..f2100db
--- a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
@@@ -115,9 -118,8 +115,9 @@@ public class SecondaryIndexTes
                                        .filterOn("birthdate", Operator.EQ, 1L)
                                        .build();
  
-         Index.Searcher searcher = cfs.indexManager.getBestIndexFor(rc).searcherFor(rc);
+         Index.Searcher searcher = rc.index().searcherFor(rc);
 -        try (ReadOrderGroup orderGroup = rc.startOrderGroup(); UnfilteredPartitionIterator pi = searcher.search(orderGroup))
 +        try (ReadExecutionController executionController = rc.executionController();
 +             UnfilteredPartitionIterator pi = searcher.search(executionController))
          {
              assertTrue(pi.hasNext());
              pi.next().close();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
index 3c09c93,02b642e..b056da1
--- a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
+++ b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
@@@ -117,15 -114,16 +117,16 @@@ public class SinglePartitionSliceComman
  
          ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(v));
          ByteBuffer zero = ByteBufferUtil.bytes(0);
 -        Slices slices = Slices.with(cfm.comparator, Slice.make(Slice.Bound.inclusiveStartOf(zero), Slice.Bound.inclusiveEndOf(zero)));
 +        Slices slices = Slices.with(cfm.comparator, Slice.make(ClusteringBound.inclusiveStartOf(zero), ClusteringBound.inclusiveEndOf(zero)));
          ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(slices, false);
-         ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                           FBUtilities.nowInSeconds(),
-                                                           columnFilter,
-                                                           RowFilter.NONE,
-                                                           DataLimits.NONE,
-                                                           key,
-                                                           sliceFilter);
+         ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                             cfm,
+                                                             FBUtilities.nowInSeconds(),
+                                                             columnFilter,
+                                                             RowFilter.NONE,
+                                                             DataLimits.NONE,
+                                                             key,
+                                                             sliceFilter);
  
          DataOutputBuffer out = new DataOutputBuffer((int) ReadCommand.legacyReadCommandSerializer.serializedSize(cmd, MessagingService.VERSION_21));
          ReadCommand.legacyReadCommandSerializer.serialize(cmd, out, MessagingService.VERSION_21);
@@@ -175,16 -167,17 +176,17 @@@
  
          ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
          ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.NONE, false);
-         ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                          FBUtilities.nowInSeconds(),
-                                                          columnFilter,
-                                                          RowFilter.NONE,
-                                                          DataLimits.NONE,
-                                                          key,
-                                                          sliceFilter);
+         ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                             cfm,
+                                                             FBUtilities.nowInSeconds(),
+                                                             columnFilter,
+                                                             RowFilter.NONE,
+                                                             DataLimits.NONE,
+                                                             key,
+                                                             sliceFilter);
  
          // check raw iterator for static cell
 -        try (ReadOrderGroup orderGroup = cmd.startOrderGroup(); UnfilteredPartitionIterator pi = cmd.executeLocally(orderGroup))
 +        try (ReadExecutionController executionController = cmd.executionController(); UnfilteredPartitionIterator pi = cmd.executeLocally(executionController))
          {
              checkForS(pi);
          }
@@@ -231,15 -224,16 +233,16 @@@
          DecoratedKey key = cfm.decorateKey(ByteBufferUtil.bytes("k1"));
  
          ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
 -        Slice slice = Slice.make(Slice.Bound.BOTTOM, Slice.Bound.inclusiveEndOf(ByteBufferUtil.bytes("i1")));
 +        Slice slice = Slice.make(ClusteringBound.BOTTOM, ClusteringBound.inclusiveEndOf(ByteBufferUtil.bytes("i1")));
          ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.with(cfm.comparator, slice), false);
-         ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                          FBUtilities.nowInSeconds(),
-                                                          columnFilter,
-                                                          RowFilter.NONE,
-                                                          DataLimits.NONE,
-                                                          key,
-                                                          sliceFilter);
+         ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                             cfm,
+                                                             FBUtilities.nowInSeconds(),
+                                                             columnFilter,
+                                                             RowFilter.NONE,
+                                                             DataLimits.NONE,
+                                                             key,
+                                                             sliceFilter);
  
          String ret = cmd.toCQLString();
          Assert.assertNotNull(ret);


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[07/10] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 0928ad4,c2598ec..5a8cf67
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@@ -595,11 -604,11 +595,11 @@@ public class SSTableReaderTes
                                               .columns("birthdate")
                                               .filterOn("birthdate", Operator.EQ, 1L)
                                               .build();
-         Index.Searcher searcher = indexedCFS.indexManager.getBestIndexFor(rc).searcherFor(rc);
+         Index.Searcher searcher = rc.index().searcherFor(rc);
          assertNotNull(searcher);
 -        try (ReadOrderGroup orderGroup = ReadOrderGroup.forCommand(rc))
 +        try (ReadExecutionController executionController = rc.executionController())
          {
 -            assertEquals(1, Util.size(UnfilteredPartitionIterators.filter(searcher.search(orderGroup), rc.nowInSec())));
 +            assertEquals(1, Util.size(UnfilteredPartitionIterators.filter(searcher.search(executionController), rc.nowInSec())));
          }
      }
  


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[04/10] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 0928ad4,c2598ec..5a8cf67
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@@ -595,11 -604,11 +595,11 @@@ public class SSTableReaderTes
                                               .columns("birthdate")
                                               .filterOn("birthdate", Operator.EQ, 1L)
                                               .build();
-         Index.Searcher searcher = indexedCFS.indexManager.getBestIndexFor(rc).searcherFor(rc);
+         Index.Searcher searcher = rc.index().searcherFor(rc);
          assertNotNull(searcher);
 -        try (ReadOrderGroup orderGroup = ReadOrderGroup.forCommand(rc))
 +        try (ReadExecutionController executionController = rc.executionController())
          {
 -            assertEquals(1, Util.size(UnfilteredPartitionIterators.filter(searcher.search(orderGroup), rc.nowInSec())));
 +            assertEquals(1, Util.size(UnfilteredPartitionIterators.filter(searcher.search(executionController), rc.nowInSec())));
          }
      }
  


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[06/10] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


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

Branch: refs/heads/trunk
Commit: 7ad1945ee7592990027bee4fe6bbfcac72940954
Parents: 6d6081e 7f297bc
Author: Aleksey Yeschenko <al...@yeschenko.com>
Authored: Wed Aug 30 17:04:49 2017 +0100
Committer: Aleksey Yeschenko <al...@yeschenko.com>
Committed: Wed Aug 30 17:08:53 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cql3/statements/SelectStatement.java        |  16 +-
 .../db/AbstractReadCommandBuilder.java          |   2 +-
 .../cassandra/db/PartitionRangeReadCommand.java | 149 ++++++++++---
 .../org/apache/cassandra/db/ReadCommand.java    | 149 +++++++------
 .../db/SinglePartitionReadCommand.java          | 209 +++++++++++++++----
 .../cassandra/index/SecondaryIndexManager.java  |   9 +-
 .../internal/composites/CompositesSearcher.java |   6 +-
 .../index/internal/keys/KeysSearcher.java       |   3 +-
 .../cassandra/service/AbstractReadExecutor.java |   4 +-
 .../service/pager/PartitionRangeQueryPager.java |   8 +-
 .../cassandra/thrift/CassandraServer.java       |  68 +++---
 test/unit/org/apache/cassandra/Util.java        |  26 +--
 .../apache/cassandra/db/SecondaryIndexTest.java |  10 +-
 .../db/SinglePartitionSliceCommandTest.java     |  45 ++--
 .../cassandra/index/sasi/SASIIndexTest.java     |  46 ++--
 .../cassandra/io/sstable/SSTableReaderTest.java |   2 +-
 17 files changed, 479 insertions(+), 274 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index c4aee3a,aca9e1f..d848eff
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,11 -1,6 +1,12 @@@
 -3.0.15
 +3.11.1
 + * Fix cassandra-stress hang issues when an error during cluster connection happens (CASSANDRA-12938)
 + * Better bootstrap failure message when blocked by (potential) range movement (CASSANDRA-13744)
 + * "ignore" option is ignored in sstableloader (CASSANDRA-13721)
 + * Deadlock in AbstractCommitLogSegmentManager (CASSANDRA-13652)
 + * Duplicate the buffer before passing it to analyser in SASI operation (CASSANDRA-13512)
 + * Properly evict pstmts from prepared statements cache (CASSANDRA-13641)
 +Merged from 3.0:
+  * Fix race condition in read command serialization (CASSANDRA-13363)
 - * Enable segement creation before recovering commitlogs (CASSANDRA-13587)
   * Fix AssertionError in short read protection (CASSANDRA-13747)
   * Don't skip corrupted sstables on startup (CASSANDRA-13620)
   * Fix the merging of cells with different user type versions (CASSANDRA-13776)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index d0487a3,9e557e0..f7b6660
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@@ -121,41 -129,74 +130,108 @@@ public class PartitionRangeReadCommand 
          return dataRange.isNamesQuery();
      }
  
 -    public PartitionRangeReadCommand forSubRange(AbstractBounds<PartitionPosition> range)
 +    /**
 +     * Returns an equivalent command but that only queries data within the provided range.
 +     *
 +     * @param range the sub-range to restrict the command to. This method <b>assumes</b> that this is a proper sub-range
 +     * of the command this is applied to.
 +     * @param isRangeContinuation whether {@code range} is a direct continuation of whatever previous range we have
 +     * queried. This matters for the {@code DataLimits} that may contain states when we do paging and in the context of
 +     * parallel queries: that state only make sense if the range queried is indeed the follow-up of whatever range we've
 +     * previously query (that yield said state). In practice this means that ranges for which {@code isRangeContinuation}
 +     * is false may have to be slightly pessimistic when counting data and may include a little bit than necessary, and
 +     * this should be dealt with post-query (in the case of {@code StorageProxy.getRangeSlice()}, which uses this method
 +     * for replica queries, this is dealt with by re-counting results on the coordinator). Note that if this is the
 +     * first range we queried, then the {@code DataLimits} will have not state and the value of this parameter doesn't
 +     * matter.
 +     */
 +    public PartitionRangeReadCommand forSubRange(AbstractBounds<PartitionPosition> range, boolean isRangeContinuation)
      {
-         DataRange newRange = dataRange().forSubRange(range);
 +        // If we're not a continuation of whatever range we've previously queried, we should ignore the states of the
 +        // DataLimits as it's either useless, or misleading. This is particularly important for GROUP BY queries, where
 +        // DataLimits.CQLGroupByLimits.GroupByAwareCounter assumes that if GroupingState.hasClustering(), then we're in
 +        // the middle of a group, but we can't make that assumption if we query and range "in advance" of where we are
 +        // on the ring.
-         DataLimits newLimits = isRangeContinuation ? limits() : limits().withoutState();
-         return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), newLimits, newRange, index);
+         return new PartitionRangeReadCommand(isDigestQuery(),
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
 -                                             limits(),
++                                             isRangeContinuation ? limits() : limits().withoutState(),
+                                              dataRange().forSubRange(range),
+                                              indexMetadata());
      }
  
      public PartitionRangeReadCommand copy()
      {
-         return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange(), index);
+         return new PartitionRangeReadCommand(isDigestQuery(),
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              dataRange(),
+                                              indexMetadata());
+     }
+ 
+     public PartitionRangeReadCommand copyAsDigestQuery()
+     {
+         return new PartitionRangeReadCommand(true,
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              dataRange(),
+                                              indexMetadata());
+     }
+ 
++    public ReadCommand withUpdatedLimit(DataLimits newLimits)
++    {
++        return new PartitionRangeReadCommand(isDigestQuery(),
++                                             digestVersion(),
++                                             isForThrift(),
++                                             metadata(),
++                                             nowInSec(),
++                                             columnFilter(),
++                                             rowFilter(),
++                                             newLimits,
++                                             dataRange(),
++                                             indexMetadata());
++    }
++
+     public PartitionRangeReadCommand withUpdatedDataRange(DataRange newDataRange)
+     {
+         return new PartitionRangeReadCommand(isDigestQuery(),
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              newDataRange,
+                                              indexMetadata());
      }
  
-     public PartitionRangeReadCommand withUpdatedLimit(DataLimits newLimits)
+     public PartitionRangeReadCommand withUpdatedLimitsAndDataRange(DataLimits newLimits, DataRange newDataRange)
      {
-         return new PartitionRangeReadCommand(metadata(), nowInSec(), columnFilter(), rowFilter(), newLimits, dataRange(), index);
+         return new PartitionRangeReadCommand(isDigestQuery(),
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              newLimits,
+                                              newDataRange,
+                                              indexMetadata());
      }
  
      public long getTimeout()
@@@ -196,7 -237,8 +272,8 @@@
          metric.rangeLatency.addNano(latencyNanos);
      }
  
-     protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadExecutionController executionController)
+     @VisibleForTesting
 -    public UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadOrderGroup orderGroup)
++    public UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadExecutionController executionController)
      {
          ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange()));
          Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().getKeyValidator()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ReadCommand.java
index 050546c,66985b6..54389f0
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@@ -322,7 -329,12 +341,12 @@@ public abstract class ReadCommand exten
       */
      public abstract ReadCommand copy();
  
+     /**
+      * Returns a copy of this command with isDigestQuery set to true.
+      */
+     public abstract ReadCommand copyAsDigestQuery();
+ 
 -    protected abstract UnfilteredPartitionIterator queryStorage(ColumnFamilyStore cfs, ReadOrderGroup orderGroup);
 +    protected abstract UnfilteredPartitionIterator queryStorage(ColumnFamilyStore cfs, ReadExecutionController executionController);
  
      protected abstract int oldestUnrepairedTombstone();
  
@@@ -684,9 -630,9 +705,9 @@@
              out.writeInt(command.nowInSec());
              ColumnFilter.serializer.serialize(command.columnFilter(), out, version);
              RowFilter.serializer.serialize(command.rowFilter(), out, version);
 -            DataLimits.serializer.serialize(command.limits(), out, version);
 +            DataLimits.serializer.serialize(command.limits(), out, version, command.metadata.comparator);
-             if (command.index.isPresent())
-                 IndexMetadata.serializer.serialize(command.index.get(), out, version);
+             if (null != command.index)
+                 IndexMetadata.serializer.serialize(command.index, out, version);
  
              command.serializeSelection(out, version);
          }
@@@ -705,10 -651,8 +726,8 @@@
              int nowInSec = in.readInt();
              ColumnFilter columnFilter = ColumnFilter.serializer.deserialize(in, version, metadata);
              RowFilter rowFilter = RowFilter.serializer.deserialize(in, version, metadata);
 -            DataLimits limits = DataLimits.serializer.deserialize(in, version);
 +            DataLimits limits = DataLimits.serializer.deserialize(in, version,  metadata.comparator);
-             Optional<IndexMetadata> index = hasIndex
-                                             ? deserializeIndexMetadata(in, version, metadata)
-                                             : Optional.empty();
+             IndexMetadata index = hasIndex ? deserializeIndexMetadata(in, version, metadata) : null;
  
              return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
          }
@@@ -721,12 -665,13 +740,12 @@@
              }
              catch (UnknownIndexException e)
              {
 -                String message = String.format("Couldn't find a defined index on %s.%s with the id %s. " +
 -                                               "If an index was just created, this is likely due to the schema not " +
 -                                               "being fully propagated. Local read will proceed without using the " +
 -                                               "index. Please wait for schema agreement after index creation.",
 -                                               cfm.ksName, cfm.cfName, e.indexId.toString());
 -                logger.info(message);
 +                logger.info("Couldn't find a defined index on {}.{} with the id {}. " +
 +                            "If an index was just created, this is likely due to the schema not " +
 +                            "being fully propagated. Local read will proceed without using the " +
 +                            "index. Please wait for schema agreement after index creation.",
 +                            cfm.ksName, cfm.cfName, e.indexId);
-                 return Optional.empty();
+                 return null;
              }
          }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 5d93c65,00464ca..c7080e7
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@@ -268,7 -316,82 +319,97 @@@ public class SinglePartitionReadComman
  
      public SinglePartitionReadCommand copy()
      {
-         return new SinglePartitionReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), partitionKey(), clusteringIndexFilter());
+         return new SinglePartitionReadCommand(isDigestQuery(),
+                                               digestVersion(),
+                                               isForThrift(),
+                                               metadata(),
+                                               nowInSec(),
+                                               columnFilter(),
+                                               rowFilter(),
+                                               limits(),
+                                               partitionKey(),
+                                               clusteringIndexFilter(),
+                                               indexMetadata());
+     }
+ 
+     public SinglePartitionReadCommand copyAsDigestQuery()
+     {
+         return new SinglePartitionReadCommand(true,
+                                               digestVersion(),
+                                               isForThrift(),
+                                               metadata(),
+                                               nowInSec(),
+                                               columnFilter(),
+                                               rowFilter(),
+                                               limits(),
+                                               partitionKey(),
+                                               clusteringIndexFilter(),
+                                               indexMetadata());
+     }
+ 
++    public SinglePartitionReadCommand withUpdatedLimit(DataLimits newLimits)
++    {
++        return new SinglePartitionReadCommand(isDigestQuery(),
++                                              digestVersion(),
++                                              isForThrift(),
++                                              metadata(),
++                                              nowInSec(),
++                                              columnFilter(),
++                                              rowFilter(),
++                                              newLimits,
++                                              partitionKey(),
++                                              clusteringIndexFilter(),
++                                              indexMetadata());
++    }
++
+     public SinglePartitionReadCommand withUpdatedClusteringIndexFilter(ClusteringIndexFilter filter)
+     {
+         return new SinglePartitionReadCommand(isDigestQuery(),
+                                               digestVersion(),
+                                               isForThrift(),
+                                               metadata(),
+                                               nowInSec(),
+                                               columnFilter(),
+                                               rowFilter(),
+                                               limits(),
+                                               partitionKey(),
+                                               filter,
+                                               indexMetadata());
+     }
+ 
+     static SinglePartitionReadCommand legacySliceCommand(boolean isDigest,
+                                                          int digestVersion,
+                                                          CFMetaData metadata,
+                                                          int nowInSec,
+                                                          ColumnFilter columnFilter,
+                                                          DataLimits limits,
+                                                          DecoratedKey partitionKey,
+                                                          ClusteringIndexSliceFilter filter)
+     {
+         // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
+         return new SinglePartitionReadCommand(isDigest,
+                                               digestVersion,
+                                               true,
+                                               metadata,
+                                               nowInSec,
+                                               columnFilter,
+                                               RowFilter.NONE,
+                                               limits,
+                                               partitionKey,
+                                               filter,
+                                               null);
+     }
+ 
+     static SinglePartitionReadCommand legacyNamesCommand(boolean isDigest,
+                                                          int digestVersion,
+                                                          CFMetaData metadata,
+                                                          int nowInSec,
+                                                          ColumnFilter columnFilter,
+                                                          DecoratedKey partitionKey,
+                                                          ClusteringIndexNamesFilter filter)
+     {
+         // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
+         return new SinglePartitionReadCommand(isDigest, digestVersion, true, metadata, nowInSec, columnFilter, RowFilter.NONE, DataLimits.NONE, partitionKey, filter,null);
      }
  
      public DecoratedKey partitionKey()
@@@ -334,26 -457,12 +475,12 @@@
                        lastReturned == null ? clusteringIndexFilter() : clusteringIndexFilter.forPaging(metadata().comparator, lastReturned, false));
      }
  
-     public SinglePartitionReadCommand withUpdatedLimit(DataLimits newLimits)
-     {
-         return new SinglePartitionReadCommand(isDigestQuery(),
-                                               digestVersion(),
-                                               isForThrift(),
-                                               metadata(),
-                                               nowInSec(),
-                                               columnFilter(),
-                                               rowFilter(),
-                                               newLimits,
-                                               partitionKey,
-                                               clusteringIndexFilter);
-     }
- 
 -    public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState) throws RequestExecutionException
 +    public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState, long queryStartNanoTime) throws RequestExecutionException
      {
 -        return StorageProxy.read(Group.one(this), consistency, clientState);
 +        return StorageProxy.read(Group.one(this), consistency, clientState, queryStartNanoTime);
      }
  
 -    public SinglePartitionPager getPager(PagingState pagingState, int protocolVersion)
 +    public SinglePartitionPager getPager(PagingState pagingState, ProtocolVersion protocolVersion)
      {
          return getPager(this, pagingState, protocolVersion);
      }
@@@ -449,7 -558,7 +576,7 @@@
                  final int rowsToCache = metadata().params.caching.rowsPerPartitionToCache();
  
                  @SuppressWarnings("resource") // we close on exception or upon closing the result of this method
-                 UnfilteredRowIterator iter = SinglePartitionReadCommand.fullPartitionRead(metadata(), nowInSec(), partitionKey()).queryMemtableAndDisk(cfs, executionController);
 -                UnfilteredRowIterator iter = fullPartitionRead(metadata(), nowInSec(), partitionKey()).queryMemtableAndDisk(cfs, readOp);
++                UnfilteredRowIterator iter = fullPartitionRead(metadata(), nowInSec(), partitionKey()).queryMemtableAndDisk(cfs, executionController);
                  try
                  {
                      // Use a custom iterator instead of DataLimits to avoid stopping the original iterator

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
index e777620,f8a7c66..2007800
--- a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
@@@ -109,65 -108,43 +109,67 @@@ public class CompositesSearcher extend
                          nextEntry = index.decodeEntry(indexKey, indexHits.next());
                      }
  
 -                    // Gather all index hits belonging to the same partition and query the data for those hits.
 -                    // TODO: it's much more efficient to do 1 read for all hits to the same partition than doing
 -                    // 1 read per index hit. However, this basically mean materializing all hits for a partition
 -                    // in memory so we should consider adding some paging mechanism. However, index hits should
 -                    // be relatively small so it's much better than the previous code that was materializing all
 -                    // *data* for a given partition.
 -                    BTreeSet.Builder<Clustering> clusterings = BTreeSet.builder(index.baseCfs.getComparator());
 -                    List<IndexEntry> entries = new ArrayList<>();
 +                    SinglePartitionReadCommand dataCmd;
                      DecoratedKey partitionKey = index.baseCfs.decorateKey(nextEntry.indexedKey);
 -
 -                    while (nextEntry != null && partitionKey.getKey().equals(nextEntry.indexedKey))
 +                    List<IndexEntry> entries = new ArrayList<>();
 +                    if (isStaticColumn())
                      {
 -                        // We're queried a slice of the index, but some hits may not match some of the clustering column constraints
 -                        if (isMatchingEntry(partitionKey, nextEntry, command))
 -                        {
 -                            clusterings.add(nextEntry.indexedEntryClustering);
 -                            entries.add(nextEntry);
 +                        // The index hit may not match the commad key constraint
 +                        if (!isMatchingEntry(partitionKey, nextEntry, command)) {
 +                            nextEntry = indexHits.hasNext() ? index.decodeEntry(indexKey, indexHits.next()) : null;
 +                            continue;
                          }
  
 +                        // If the index is on a static column, we just need to do a full read on the partition.
 +                        // Note that we want to re-use the command.columnFilter() in case of future change.
 +                        dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata,
 +                                                                    command.nowInSec(),
 +                                                                    command.columnFilter(),
 +                                                                    RowFilter.NONE,
 +                                                                    DataLimits.NONE,
 +                                                                    partitionKey,
 +                                                                    new ClusteringIndexSliceFilter(Slices.ALL, false));
 +                        entries.add(nextEntry);
                          nextEntry = indexHits.hasNext() ? index.decodeEntry(indexKey, indexHits.next()) : null;
                      }
 +                    else
 +                    {
 +                        // Gather all index hits belonging to the same partition and query the data for those hits.
 +                        // TODO: it's much more efficient to do 1 read for all hits to the same partition than doing
 +                        // 1 read per index hit. However, this basically mean materializing all hits for a partition
 +                        // in memory so we should consider adding some paging mechanism. However, index hits should
 +                        // be relatively small so it's much better than the previous code that was materializing all
 +                        // *data* for a given partition.
 +                        BTreeSet.Builder<Clustering> clusterings = BTreeSet.builder(index.baseCfs.getComparator());
 +                        while (nextEntry != null && partitionKey.getKey().equals(nextEntry.indexedKey))
 +                        {
 +                            // We're queried a slice of the index, but some hits may not match some of the clustering column constraints
 +                            if (isMatchingEntry(partitionKey, nextEntry, command))
 +                            {
 +                                clusterings.add(nextEntry.indexedEntryClustering);
 +                                entries.add(nextEntry);
 +                            }
 +
 +                            nextEntry = indexHits.hasNext() ? index.decodeEntry(indexKey, indexHits.next()) : null;
 +                        }
  
 -                    // Because we've eliminated entries that don't match the clustering columns, it's possible we added nothing
 -                    if (clusterings.isEmpty())
 -                        continue;
 +                        // Because we've eliminated entries that don't match the clustering columns, it's possible we added nothing
 +                        if (clusterings.isEmpty())
 +                            continue;
 +
 +                        // Query the gathered index hits. We still need to filter stale hits from the resulting query.
 +                        ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(clusterings.build(), false);
-                         dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata,
++                        dataCmd = SinglePartitionReadCommand.create(isForThrift(),
++                                                                    index.baseCfs.metadata,
 +                                                                    command.nowInSec(),
 +                                                                    command.columnFilter(),
 +                                                                    command.rowFilter(),
 +                                                                    DataLimits.NONE,
 +                                                                    partitionKey,
-                                                                     filter);
++                                                                    filter,
++                                                                    null);
 +                    }
  
 -                    // Query the gathered index hits. We still need to filter stale hits from the resulting query.
 -                    ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(clusterings.build(), false);
 -                    SinglePartitionReadCommand dataCmd = SinglePartitionReadCommand.create(isForThrift(),
 -                                                                                           index.baseCfs.metadata,
 -                                                                                           command.nowInSec(),
 -                                                                                           command.columnFilter(),
 -                                                                                           command.rowFilter(),
 -                                                                                           DataLimits.NONE,
 -                                                                                           partitionKey,
 -                                                                                           filter,
 -                                                                                           null);
                      @SuppressWarnings("resource") // We close right away if empty, and if it's assign to next it will be called either
                      // by the next caller of next, or through closing this iterator is this come before.
                      UnfilteredRowIterator dataIter =

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
index 5ba13a4,ea79017..e6ad3d4
--- a/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
@@@ -17,16 -17,14 +17,12 @@@
   */
  package org.apache.cassandra.service.pager;
  
- import java.util.Optional;
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
--
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.filter.DataLimits;
  import org.apache.cassandra.db.rows.Row;
  import org.apache.cassandra.dht.*;
  import org.apache.cassandra.exceptions.RequestExecutionException;
- import org.apache.cassandra.index.Index;
- import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.transport.ProtocolVersion;
  
  /**
   * Pages a PartitionRangeReadCommand.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/CassandraServer.java
index e71f512,cb74b15..f43b7a4
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@@ -1547,17 -1519,17 +1547,17 @@@ public class CassandraServer implement
                  ColumnFilter columns = makeColumnFilter(metadata, column_parent, predicate);
                  ClusteringIndexFilter filter = toInternalFilter(metadata, column_parent, predicate);
                  DataLimits limits = getLimits(range.count, metadata.isSuper() && !column_parent.isSetSuper_column(), predicate);
-                 PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                               0,
-                                                                               true,
-                                                                               metadata,
-                                                                               nowInSec,
-                                                                               columns,
-                                                                               ThriftConversion.rowFilterFromThrift(metadata, range.row_filter),
-                                                                               limits,
-                                                                               new DataRange(bounds, filter),
-                                                                               Optional.empty());
+ 
+                 PartitionRangeReadCommand cmd =
+                     PartitionRangeReadCommand.create(true,
+                                                      metadata,
+                                                      nowInSec,
+                                                      columns,
+                                                      ThriftConversion.rowFilterFromThrift(metadata, range.row_filter),
+                                                      limits,
+                                                      new DataRange(bounds, filter));
+ 
 -                try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
 +                try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel, queryStartNanoTime))
                  {
                      assert results != null;
                      return thriftifyKeySlices(results, column_parent, limits.perPartitionCount());
@@@ -1640,19 -1611,19 +1640,19 @@@
                  ClusteringIndexFilter filter = new ClusteringIndexSliceFilter(Slices.ALL, false);
                  DataLimits limits = getLimits(range.count, true, Integer.MAX_VALUE);
                  Clustering pageFrom = metadata.isSuper()
 -                                    ? new Clustering(start_column)
 +                                    ? Clustering.make(start_column)
                                      : LegacyLayout.decodeCellName(metadata, start_column).clustering;
-                 PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                               0,
-                                                                               true,
-                                                                               metadata,
-                                                                               nowInSec,
-                                                                               ColumnFilter.all(metadata),
-                                                                               RowFilter.NONE,
-                                                                               limits,
-                                                                               new DataRange(bounds, filter).forPaging(bounds, metadata.comparator, pageFrom, true),
-                                                                               Optional.empty());
+ 
+                 PartitionRangeReadCommand cmd =
+                     PartitionRangeReadCommand.create(true,
+                                                      metadata,
+                                                      nowInSec,
+                                                      ColumnFilter.all(metadata),
+                                                      RowFilter.NONE,
+                                                      limits,
+                                                      new DataRange(bounds, filter).forPaging(bounds, metadata.comparator, pageFrom, true));
+ 
 -                try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
 +                try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel, queryStartNanoTime))
                  {
                      return thriftifyKeySlices(results, new ColumnParent(column_family), limits.perPartitionCount());
                  }
@@@ -1735,24 -1705,20 +1735,20 @@@
              ColumnFilter columns = makeColumnFilter(metadata, column_parent, column_predicate);
              ClusteringIndexFilter filter = toInternalFilter(metadata, column_parent, column_predicate);
              DataLimits limits = getLimits(index_clause.count, metadata.isSuper() && !column_parent.isSetSuper_column(), column_predicate);
-             PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                           0,
-                                                                           true,
-                                                                           metadata,
-                                                                           nowInSec,
-                                                                           columns,
-                                                                           ThriftConversion.rowFilterFromThrift(metadata, index_clause.expressions),
-                                                                           limits,
-                                                                           new DataRange(bounds, filter),
-                                                                           Optional.empty());
-             // If there's a secondary index that the command can use, have it validate
-             // the request parameters. Note that as a side effect, if a viable Index is
-             // identified by the CFS's index manager, it will be cached in the command
-             // and serialized during distribution to replicas in order to avoid performing
-             // further lookups.
+ 
+             PartitionRangeReadCommand cmd =
+                 PartitionRangeReadCommand.create(true,
+                                                  metadata,
+                                                  nowInSec,
+                                                  columns,
+                                                  ThriftConversion.rowFilterFromThrift(metadata, index_clause.expressions),
+                                                  limits,
+                                                  new DataRange(bounds, filter));
+ 
+             // If there's a secondary index that the command can use, have it validate the request parameters.
              cmd.maybeValidateIndex();
  
 -            try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
 +            try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel, queryStartNanoTime))
              {
                  return thriftifyKeySlices(results, column_parent, limits.perPartitionCount());
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/Util.java
index 3fa24d7,d758efe..a3ad653
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@@ -668,33 -627,9 +668,9 @@@ public class Uti
  
      public static UnfilteredPartitionIterator executeLocally(PartitionRangeReadCommand command,
                                                               ColumnFamilyStore cfs,
 -                                                             ReadOrderGroup orderGroup)
 +                                                             ReadExecutionController controller)
      {
-         return new InternalPartitionRangeReadCommand(command).queryStorageInternal(cfs, controller);
-     }
- 
-     private static final class InternalPartitionRangeReadCommand extends PartitionRangeReadCommand
-     {
- 
-         private InternalPartitionRangeReadCommand(PartitionRangeReadCommand original)
-         {
-             super(original.isDigestQuery(),
-                   original.digestVersion(),
-                   original.isForThrift(),
-                   original.metadata(),
-                   original.nowInSec(),
-                   original.columnFilter(),
-                   original.rowFilter(),
-                   original.limits(),
-                   original.dataRange(),
-                   Optional.empty());
-         }
- 
-         private UnfilteredPartitionIterator queryStorageInternal(ColumnFamilyStore cfs,
-                                                                  ReadExecutionController controller)
-         {
-             return queryStorage(cfs, controller);
-         }
 -        return command.queryStorage(cfs, orderGroup);
++        return command.queryStorage(cfs, controller);
      }
  
      public static Closeable markDirectoriesUnwriteable(ColumnFamilyStore cfs)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
index a037d90,2457c4a..f2100db
--- a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
@@@ -115,9 -118,8 +115,9 @@@ public class SecondaryIndexTes
                                        .filterOn("birthdate", Operator.EQ, 1L)
                                        .build();
  
-         Index.Searcher searcher = cfs.indexManager.getBestIndexFor(rc).searcherFor(rc);
+         Index.Searcher searcher = rc.index().searcherFor(rc);
 -        try (ReadOrderGroup orderGroup = rc.startOrderGroup(); UnfilteredPartitionIterator pi = searcher.search(orderGroup))
 +        try (ReadExecutionController executionController = rc.executionController();
 +             UnfilteredPartitionIterator pi = searcher.search(executionController))
          {
              assertTrue(pi.hasNext());
              pi.next().close();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
index 3c09c93,02b642e..b056da1
--- a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
+++ b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
@@@ -117,15 -114,16 +117,16 @@@ public class SinglePartitionSliceComman
  
          ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(v));
          ByteBuffer zero = ByteBufferUtil.bytes(0);
 -        Slices slices = Slices.with(cfm.comparator, Slice.make(Slice.Bound.inclusiveStartOf(zero), Slice.Bound.inclusiveEndOf(zero)));
 +        Slices slices = Slices.with(cfm.comparator, Slice.make(ClusteringBound.inclusiveStartOf(zero), ClusteringBound.inclusiveEndOf(zero)));
          ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(slices, false);
-         ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                           FBUtilities.nowInSeconds(),
-                                                           columnFilter,
-                                                           RowFilter.NONE,
-                                                           DataLimits.NONE,
-                                                           key,
-                                                           sliceFilter);
+         ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                             cfm,
+                                                             FBUtilities.nowInSeconds(),
+                                                             columnFilter,
+                                                             RowFilter.NONE,
+                                                             DataLimits.NONE,
+                                                             key,
+                                                             sliceFilter);
  
          DataOutputBuffer out = new DataOutputBuffer((int) ReadCommand.legacyReadCommandSerializer.serializedSize(cmd, MessagingService.VERSION_21));
          ReadCommand.legacyReadCommandSerializer.serialize(cmd, out, MessagingService.VERSION_21);
@@@ -175,16 -167,17 +176,17 @@@
  
          ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
          ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.NONE, false);
-         ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                          FBUtilities.nowInSeconds(),
-                                                          columnFilter,
-                                                          RowFilter.NONE,
-                                                          DataLimits.NONE,
-                                                          key,
-                                                          sliceFilter);
+         ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                             cfm,
+                                                             FBUtilities.nowInSeconds(),
+                                                             columnFilter,
+                                                             RowFilter.NONE,
+                                                             DataLimits.NONE,
+                                                             key,
+                                                             sliceFilter);
  
          // check raw iterator for static cell
 -        try (ReadOrderGroup orderGroup = cmd.startOrderGroup(); UnfilteredPartitionIterator pi = cmd.executeLocally(orderGroup))
 +        try (ReadExecutionController executionController = cmd.executionController(); UnfilteredPartitionIterator pi = cmd.executeLocally(executionController))
          {
              checkForS(pi);
          }
@@@ -231,15 -224,16 +233,16 @@@
          DecoratedKey key = cfm.decorateKey(ByteBufferUtil.bytes("k1"));
  
          ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
 -        Slice slice = Slice.make(Slice.Bound.BOTTOM, Slice.Bound.inclusiveEndOf(ByteBufferUtil.bytes("i1")));
 +        Slice slice = Slice.make(ClusteringBound.BOTTOM, ClusteringBound.inclusiveEndOf(ByteBufferUtil.bytes("i1")));
          ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.with(cfm.comparator, slice), false);
-         ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                          FBUtilities.nowInSeconds(),
-                                                          columnFilter,
-                                                          RowFilter.NONE,
-                                                          DataLimits.NONE,
-                                                          key,
-                                                          sliceFilter);
+         ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                             cfm,
+                                                             FBUtilities.nowInSeconds(),
+                                                             columnFilter,
+                                                             RowFilter.NONE,
+                                                             DataLimits.NONE,
+                                                             key,
+                                                             sliceFilter);
  
          String ret = cmd.toCQLString();
          Assert.assertNotNull(ret);


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[02/10] cassandra git commit: Fix race condition in read command serialization

Posted by al...@apache.org.
Fix race condition in read command serialization

patch by Aleksey Yeschenko; reviewed by Sam Tunnicliffe for
CASSANDRA-13363


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

Branch: refs/heads/cassandra-3.11
Commit: 7f297bcf8aced983cbc9c4103d0ebefc1789f0dd
Parents: d03c046
Author: Aleksey Yeschenko <al...@yeschenko.com>
Authored: Mon Aug 14 16:43:06 2017 +0100
Committer: Aleksey Yeschenko <al...@yeschenko.com>
Committed: Wed Aug 30 16:16:46 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +-
 .../cql3/statements/SelectStatement.java        |  16 +-
 .../db/AbstractReadCommandBuilder.java          |   2 +-
 .../cassandra/db/PartitionRangeReadCommand.java | 133 +++++++++++---
 .../org/apache/cassandra/db/ReadCommand.java    | 149 ++++++++-------
 .../db/SinglePartitionReadCommand.java          | 180 ++++++++++++++++---
 .../cassandra/index/SecondaryIndexManager.java  |   9 +-
 .../internal/composites/CompositesSearcher.java |   6 +-
 .../index/internal/keys/KeysSearcher.java       |   3 +-
 .../cassandra/service/AbstractReadExecutor.java |   4 +-
 .../service/pager/PartitionRangeQueryPager.java |   8 +-
 .../cassandra/thrift/CassandraServer.java       |  69 ++++---
 test/unit/org/apache/cassandra/Util.java        |  26 +--
 .../apache/cassandra/db/SecondaryIndexTest.java |  10 +-
 .../db/SinglePartitionSliceCommandTest.java     |  45 ++---
 .../cassandra/io/sstable/SSTableReaderTest.java |   2 +-
 16 files changed, 427 insertions(+), 238 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 452dc9b..aca9e1f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
 3.0.15
- * enable segement creation before recovering commitlogs (CASSANDRA-13587)
+ * Fix race condition in read command serialization (CASSANDRA-13363)
+ * Enable segement creation before recovering commitlogs (CASSANDRA-13587)
  * Fix AssertionError in short read protection (CASSANDRA-13747)
  * Don't skip corrupted sstables on startup (CASSANDRA-13620)
  * Fix the merging of cells with different user type versions (CASSANDRA-13776)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index bd377f4..3882a23 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -540,18 +540,10 @@ public class SelectStatement implements CQLStatement
         if (keyBounds == null)
             return ReadQuery.EMPTY;
 
-        PartitionRangeReadCommand command = new PartitionRangeReadCommand(cfm,
-                                                                          nowInSec,
-                                                                          queriedColumns,
-                                                                          rowFilter,
-                                                                          limit,
-                                                                          new DataRange(keyBounds, clusteringIndexFilter),
-                                                                          Optional.empty());
-        // If there's a secondary index that the command can use, have it validate
-        // the request parameters. Note that as a side effect, if a viable Index is
-        // identified by the CFS's index manager, it will be cached in the command
-        // and serialized during distribution to replicas in order to avoid performing
-        // further lookups.
+        PartitionRangeReadCommand command =
+            PartitionRangeReadCommand.create(false, cfm, nowInSec, queriedColumns, rowFilter, limit, new DataRange(keyBounds, clusteringIndexFilter));
+
+        // If there's a secondary index that the command can use, have it validate the request parameters.
         command.maybeValidateIndex();
 
         return command;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java b/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
index afbab74..d219816 100644
--- a/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
+++ b/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
@@ -336,7 +336,7 @@ public abstract class AbstractReadCommandBuilder
             else
                 bounds = new ExcludingBounds<>(start, end);
 
-            return new PartitionRangeReadCommand(cfs.metadata, nowInSeconds, makeColumnFilter(), filter, makeLimits(), new DataRange(bounds, makeFilter()), Optional.empty());
+            return PartitionRangeReadCommand.create(false, cfs.metadata, nowInSeconds, makeColumnFilter(), filter, makeLimits(), new DataRange(bounds, makeFilter()));
         }
 
         static DecoratedKey makeKey(CFMetaData metadata, Object... partitionKey)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index fb2dd0d..9e557e0 100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@ -20,8 +20,8 @@ package org.apache.cassandra.db;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Optional;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.config.CFMetaData;
@@ -59,31 +59,39 @@ public class PartitionRangeReadCommand extends ReadCommand
     private final DataRange dataRange;
     private int oldestUnrepairedTombstone = Integer.MAX_VALUE;
 
-    public PartitionRangeReadCommand(boolean isDigest,
-                                     int digestVersion,
-                                     boolean isForThrift,
-                                     CFMetaData metadata,
-                                     int nowInSec,
-                                     ColumnFilter columnFilter,
-                                     RowFilter rowFilter,
-                                     DataLimits limits,
-                                     DataRange dataRange,
-                                     Optional<IndexMetadata> index)
+    private PartitionRangeReadCommand(boolean isDigest,
+                                      int digestVersion,
+                                      boolean isForThrift,
+                                      CFMetaData metadata,
+                                      int nowInSec,
+                                      ColumnFilter columnFilter,
+                                      RowFilter rowFilter,
+                                      DataLimits limits,
+                                      DataRange dataRange,
+                                      IndexMetadata index)
     {
-        super(Kind.PARTITION_RANGE, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits);
+        super(Kind.PARTITION_RANGE, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
         this.dataRange = dataRange;
-        this.index = index;
     }
 
-    public PartitionRangeReadCommand(CFMetaData metadata,
-                                     int nowInSec,
-                                     ColumnFilter columnFilter,
-                                     RowFilter rowFilter,
-                                     DataLimits limits,
-                                     DataRange dataRange,
-                                     Optional<IndexMetadata> index)
+    public static PartitionRangeReadCommand create(boolean isForThrift,
+                                                   CFMetaData metadata,
+                                                   int nowInSec,
+                                                   ColumnFilter columnFilter,
+                                                   RowFilter rowFilter,
+                                                   DataLimits limits,
+                                                   DataRange dataRange)
     {
-        this(false, 0, false, metadata, nowInSec, columnFilter, rowFilter, limits, dataRange, index);
+        return new PartitionRangeReadCommand(false,
+                                             0,
+                                             isForThrift,
+                                             metadata,
+                                             nowInSec,
+                                             columnFilter,
+                                             rowFilter,
+                                             limits,
+                                             dataRange,
+                                             findIndex(metadata, rowFilter));
     }
 
     /**
@@ -96,13 +104,14 @@ public class PartitionRangeReadCommand extends ReadCommand
      */
     public static PartitionRangeReadCommand allDataRead(CFMetaData metadata, int nowInSec)
     {
-        return new PartitionRangeReadCommand(metadata,
+        return new PartitionRangeReadCommand(false, 0, false,
+                                             metadata,
                                              nowInSec,
                                              ColumnFilter.all(metadata),
                                              RowFilter.NONE,
                                              DataLimits.NONE,
                                              DataRange.allData(metadata.partitioner),
-                                             Optional.empty());
+                                             null);
     }
 
     public DataRange dataRange()
@@ -122,17 +131,72 @@ public class PartitionRangeReadCommand extends ReadCommand
 
     public PartitionRangeReadCommand forSubRange(AbstractBounds<PartitionPosition> range)
     {
-        return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange().forSubRange(range), index);
+        return new PartitionRangeReadCommand(isDigestQuery(),
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             limits(),
+                                             dataRange().forSubRange(range),
+                                             indexMetadata());
     }
 
     public PartitionRangeReadCommand copy()
     {
-        return new PartitionRangeReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange(), index);
+        return new PartitionRangeReadCommand(isDigestQuery(),
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             limits(),
+                                             dataRange(),
+                                             indexMetadata());
+    }
+
+    public PartitionRangeReadCommand copyAsDigestQuery()
+    {
+        return new PartitionRangeReadCommand(true,
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             limits(),
+                                             dataRange(),
+                                             indexMetadata());
+    }
+
+    public PartitionRangeReadCommand withUpdatedDataRange(DataRange newDataRange)
+    {
+        return new PartitionRangeReadCommand(isDigestQuery(),
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             limits(),
+                                             newDataRange,
+                                             indexMetadata());
     }
 
-    public PartitionRangeReadCommand withUpdatedLimit(DataLimits newLimits)
+    public PartitionRangeReadCommand withUpdatedLimitsAndDataRange(DataLimits newLimits, DataRange newDataRange)
     {
-        return new PartitionRangeReadCommand(metadata(), nowInSec(), columnFilter(), rowFilter(), newLimits, dataRange(), index);
+        return new PartitionRangeReadCommand(isDigestQuery(),
+                                             digestVersion(),
+                                             isForThrift(),
+                                             metadata(),
+                                             nowInSec(),
+                                             columnFilter(),
+                                             rowFilter(),
+                                             newLimits,
+                                             newDataRange,
+                                             indexMetadata());
     }
 
     public long getTimeout()
@@ -173,7 +237,8 @@ public class PartitionRangeReadCommand extends ReadCommand
         metric.rangeLatency.addNano(latencyNanos);
     }
 
-    protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadOrderGroup orderGroup)
+    @VisibleForTesting
+    public UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadOrderGroup orderGroup)
     {
         ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange()));
         Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().getKeyValidator()));
@@ -337,7 +402,17 @@ public class PartitionRangeReadCommand extends ReadCommand
 
     private static class Deserializer extends SelectionDeserializer
     {
-        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
+        public ReadCommand deserialize(DataInputPlus in,
+                                       int version,
+                                       boolean isDigest,
+                                       int digestVersion,
+                                       boolean isForThrift,
+                                       CFMetaData metadata,
+                                       int nowInSec,
+                                       ColumnFilter columnFilter,
+                                       RowFilter rowFilter,
+                                       DataLimits limits,
+                                       IndexMetadata index)
         throws IOException
         {
             DataRange range = DataRange.serializer.deserialize(in, version, metadata);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index 76180cc..66985b6 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -22,6 +22,8 @@ import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.function.Predicate;
 
+import javax.annotation.Nullable;
+
 import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -106,24 +108,27 @@ public abstract class ReadCommand implements ReadQuery
     private final RowFilter rowFilter;
     private final DataLimits limits;
 
-    // SecondaryIndexManager will attempt to provide the most selective of any available indexes
-    // during execution. Here we also store an the results of that lookup to repeating it over
-    // the lifetime of the command.
-    protected Optional<IndexMetadata> index = Optional.empty();
-
-    // Flag to indicate whether the index manager has been queried to select an index for this
-    // command. This is necessary as the result of that lookup may be null, in which case we
-    // still don't want to repeat it.
-    private boolean indexManagerQueried = false;
-
-    private boolean isDigestQuery;
+    private final boolean isDigestQuery;
     // if a digest query, the version for which the digest is expected. Ignored if not a digest.
     private int digestVersion;
     private final boolean isForThrift;
 
+    @Nullable
+    private final IndexMetadata index;
+
     protected static abstract class SelectionDeserializer
     {
-        public abstract ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index) throws IOException;
+        public abstract ReadCommand deserialize(DataInputPlus in,
+                                                int version,
+                                                boolean isDigest,
+                                                int digestVersion,
+                                                boolean isForThrift,
+                                                CFMetaData metadata,
+                                                int nowInSec,
+                                                ColumnFilter columnFilter,
+                                                RowFilter rowFilter,
+                                                DataLimits limits,
+                                                IndexMetadata index) throws IOException;
     }
 
     protected enum Kind
@@ -147,7 +152,8 @@ public abstract class ReadCommand implements ReadQuery
                           int nowInSec,
                           ColumnFilter columnFilter,
                           RowFilter rowFilter,
-                          DataLimits limits)
+                          DataLimits limits,
+                          IndexMetadata index)
     {
         this.kind = kind;
         this.isDigestQuery = isDigestQuery;
@@ -158,6 +164,7 @@ public abstract class ReadCommand implements ReadQuery
         this.columnFilter = columnFilter;
         this.rowFilter = rowFilter;
         this.limits = limits;
+        this.index = index;
     }
 
     protected abstract void serializeSelection(DataOutputPlus out, int version) throws IOException;
@@ -253,18 +260,6 @@ public abstract class ReadCommand implements ReadQuery
     }
 
     /**
-     * Sets whether this command should be a digest one or not.
-     *
-     * @param isDigestQuery whether the command should be set as a digest one or not.
-     * @return this read command.
-     */
-    public ReadCommand setIsDigestQuery(boolean isDigestQuery)
-    {
-        this.isDigestQuery = isDigestQuery;
-        return this;
-    }
-
-    /**
      * Sets the digest version, for when digest for that command is requested.
      * <p>
      * Note that we allow setting this independently of setting the command as a digest query as
@@ -291,6 +286,30 @@ public abstract class ReadCommand implements ReadQuery
     }
 
     /**
+     * Index (metadata) chosen for this query. Can be null.
+     *
+     * @return index (metadata) chosen for this query
+     */
+    @Nullable
+    public IndexMetadata indexMetadata()
+    {
+        return index;
+    }
+
+    /**
+     *  Index instance chosen for this query. Can be null.
+     *
+     * @return Index instance chosen for this query. Can be null.
+     */
+    @Nullable
+    public Index index()
+    {
+        return null == index
+             ? null
+             : Keyspace.openAndGetStore(metadata).indexManager.getIndex(index);
+    }
+
+    /**
      * The clustering index filter this command to use for the provided key.
      * <p>
      * Note that that method should only be called on a key actually queried by this command
@@ -310,6 +329,11 @@ public abstract class ReadCommand implements ReadQuery
      */
     public abstract ReadCommand copy();
 
+    /**
+     * Returns a copy of this command with isDigestQuery set to true.
+     */
+    public abstract ReadCommand copyAsDigestQuery();
+
     protected abstract UnfilteredPartitionIterator queryStorage(ColumnFamilyStore cfs, ReadOrderGroup orderGroup);
 
     protected abstract int oldestUnrepairedTombstone();
@@ -321,35 +345,32 @@ public abstract class ReadCommand implements ReadQuery
              : ReadResponse.createDataResponse(iterator, this);
     }
 
-    public long indexSerializedSize(int version)
+    long indexSerializedSize(int version)
     {
-        if (index.isPresent())
-            return IndexMetadata.serializer.serializedSize(index.get(), version);
-        else
-            return 0;
+        return null != index
+             ? IndexMetadata.serializer.serializedSize(index, version)
+             : 0;
     }
 
     public Index getIndex(ColumnFamilyStore cfs)
     {
-        // if we've already consulted the index manager, and it returned a valid index
-        // the result should be cached here.
-        if(index.isPresent())
-            return cfs.indexManager.getIndex(index.get());
-
-        // if no cached index is present, but we've already consulted the index manager
-        // then no registered index is suitable for this command, so just return null.
-        if (indexManagerQueried)
+        return null != index
+             ? cfs.indexManager.getIndex(index)
+             : null;
+    }
+
+    static IndexMetadata findIndex(CFMetaData table, RowFilter rowFilter)
+    {
+        if (table.getIndexes().isEmpty() || rowFilter.isEmpty())
             return null;
 
-        // do the lookup, set the flag to indicate so and cache the result if not null
-        Index selected = cfs.indexManager.getBestIndexFor(this);
-        indexManagerQueried = true;
+        ColumnFamilyStore cfs = Keyspace.openAndGetStore(table);
 
-        if (selected == null)
-            return null;
+        Index index = cfs.indexManager.getBestIndexFor(rowFilter);
 
-        index = Optional.of(selected.getIndexMetadata());
-        return selected;
+        return null != index
+             ? index.getIndexMetadata()
+             : null;
     }
 
     /**
@@ -602,7 +623,7 @@ public abstract class ReadCommand implements ReadQuery
             assert version >= MessagingService.VERSION_30;
 
             out.writeByte(command.kind.ordinal());
-            out.writeByte(digestFlag(command.isDigestQuery()) | thriftFlag(command.isForThrift()) | indexFlag(command.index.isPresent()));
+            out.writeByte(digestFlag(command.isDigestQuery()) | thriftFlag(command.isForThrift()) | indexFlag(null != command.index));
             if (command.isDigestQuery())
                 out.writeUnsignedVInt(command.digestVersion());
             CFMetaData.serializer.serialize(command.metadata(), out, version);
@@ -610,8 +631,8 @@ public abstract class ReadCommand implements ReadQuery
             ColumnFilter.serializer.serialize(command.columnFilter(), out, version);
             RowFilter.serializer.serialize(command.rowFilter(), out, version);
             DataLimits.serializer.serialize(command.limits(), out, version);
-            if (command.index.isPresent())
-                IndexMetadata.serializer.serialize(command.index.get(), out, version);
+            if (null != command.index)
+                IndexMetadata.serializer.serialize(command.index, out, version);
 
             command.serializeSelection(out, version);
         }
@@ -631,18 +652,16 @@ public abstract class ReadCommand implements ReadQuery
             ColumnFilter columnFilter = ColumnFilter.serializer.deserialize(in, version, metadata);
             RowFilter rowFilter = RowFilter.serializer.deserialize(in, version, metadata);
             DataLimits limits = DataLimits.serializer.deserialize(in, version);
-            Optional<IndexMetadata> index = hasIndex
-                                            ? deserializeIndexMetadata(in, version, metadata)
-                                            : Optional.empty();
+            IndexMetadata index = hasIndex ? deserializeIndexMetadata(in, version, metadata) : null;
 
             return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
         }
 
-        private Optional<IndexMetadata> deserializeIndexMetadata(DataInputPlus in, int version, CFMetaData cfm) throws IOException
+        private IndexMetadata deserializeIndexMetadata(DataInputPlus in, int version, CFMetaData cfm) throws IOException
         {
             try
             {
-                return Optional.of(IndexMetadata.serializer.deserialize(in, version, cfm));
+                return IndexMetadata.serializer.deserialize(in, version, cfm);
             }
             catch (UnknownIndexException e)
             {
@@ -652,7 +671,7 @@ public abstract class ReadCommand implements ReadQuery
                                                "index. Please wait for schema agreement after index creation.",
                                                cfm.ksName, cfm.cfName, e.indexId.toString());
                 logger.info(message);
-                return Optional.empty();
+                return null;
             }
         }
 
@@ -830,7 +849,7 @@ public abstract class ReadCommand implements ReadQuery
             else
                 limits = DataLimits.cqlLimits(maxResults);
 
-            return new PartitionRangeReadCommand(false, 0, true, metadata, nowInSec, selection, rowFilter, limits, new DataRange(keyRange, filter), Optional.empty());
+            return PartitionRangeReadCommand.create(true, metadata, nowInSec, selection, rowFilter, limits, new DataRange(keyRange, filter));
         }
 
         static void serializeRowFilter(DataOutputPlus out, RowFilter rowFilter) throws IOException
@@ -939,9 +958,8 @@ public abstract class ReadCommand implements ReadQuery
             ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter) command.dataRange().clusteringIndexFilter;
             ClusteringIndexSliceFilter sliceFilter = LegacyReadCommandSerializer.convertNamesFilterToSliceFilter(filter, metadata);
             DataRange newRange = new DataRange(command.dataRange().keyRange(), sliceFilter);
-            return new PartitionRangeReadCommand(
-                    command.isDigestQuery(), command.digestVersion(), command.isForThrift(), metadata, command.nowInSec(),
-                    command.columnFilter(), command.rowFilter(), command.limits(), newRange, Optional.empty());
+
+            return command.withUpdatedDataRange(newRange);
         }
 
         static ColumnFilter getColumnSelectionForSlice(boolean selectsStatics, int compositesToGroup, CFMetaData metadata)
@@ -1096,7 +1114,7 @@ public abstract class ReadCommand implements ReadQuery
                 // missing without any problems, so we can safely always set "inclusive" to false in the data range
                 dataRange = dataRange.forPaging(keyRange, metadata.comparator, startBound.getAsClustering(metadata), false);
             }
-            return new PartitionRangeReadCommand(false, 0, true, metadata, nowInSec, selection, rowFilter, limits, dataRange, Optional.empty());
+            return PartitionRangeReadCommand.create(true, metadata, nowInSec, selection, rowFilter, limits, dataRange);
         }
 
         public long serializedSize(ReadCommand command, int version)
@@ -1290,10 +1308,7 @@ public abstract class ReadCommand implements ReadQuery
         {
             Pair<ColumnFilter, ClusteringIndexNamesFilter> selectionAndFilter = deserializeNamesSelectionAndFilter(in, metadata);
 
-            // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
-            return new SinglePartitionReadCommand(
-                    isDigest, version, true, metadata, nowInSeconds, selectionAndFilter.left, RowFilter.NONE, DataLimits.NONE,
-                    key, selectionAndFilter.right);
+            return SinglePartitionReadCommand.legacyNamesCommand(isDigest, version, metadata, nowInSeconds, selectionAndFilter.left, key, selectionAndFilter.right);
         }
 
         static Pair<ColumnFilter, ClusteringIndexNamesFilter> deserializeNamesSelectionAndFilter(DataInputPlus in, CFMetaData metadata) throws IOException
@@ -1422,8 +1437,7 @@ public abstract class ReadCommand implements ReadQuery
             else
                 limits = DataLimits.cqlLimits(count);
 
-            // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
-            return new SinglePartitionReadCommand(isDigest, version, true, metadata, nowInSeconds, columnFilter, RowFilter.NONE, limits, key, filter);
+            return SinglePartitionReadCommand.legacySliceCommand(isDigest, version, metadata, nowInSeconds, columnFilter, limits, key, filter);
         }
 
         private long serializedSliceCommandSize(SinglePartitionReadCommand command)
@@ -1605,9 +1619,8 @@ public abstract class ReadCommand implements ReadQuery
 
             ClusteringIndexNamesFilter filter = (ClusteringIndexNamesFilter)command.clusteringIndexFilter();
             ClusteringIndexSliceFilter sliceFilter = convertNamesFilterToSliceFilter(filter, metadata);
-            return new SinglePartitionReadCommand(
-                    command.isDigestQuery(), command.digestVersion(), command.isForThrift(), metadata, command.nowInSec(),
-                    command.columnFilter(), command.rowFilter(), command.limits(), command.partitionKey(), sliceFilter);
+
+            return command.withUpdatedClusteringIndexFilter(sliceFilter);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 686ec35..00464ca 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -70,18 +70,19 @@ public class SinglePartitionReadCommand extends ReadCommand
 
     private int oldestUnrepairedTombstone = Integer.MAX_VALUE;
 
-    public SinglePartitionReadCommand(boolean isDigest,
-                                      int digestVersion,
-                                      boolean isForThrift,
-                                      CFMetaData metadata,
-                                      int nowInSec,
-                                      ColumnFilter columnFilter,
-                                      RowFilter rowFilter,
-                                      DataLimits limits,
-                                      DecoratedKey partitionKey,
-                                      ClusteringIndexFilter clusteringIndexFilter)
+    private SinglePartitionReadCommand(boolean isDigest,
+                                       int digestVersion,
+                                       boolean isForThrift,
+                                       CFMetaData metadata,
+                                       int nowInSec,
+                                       ColumnFilter columnFilter,
+                                       RowFilter rowFilter,
+                                       DataLimits limits,
+                                       DecoratedKey partitionKey,
+                                       ClusteringIndexFilter clusteringIndexFilter,
+                                       IndexMetadata index)
     {
-        super(Kind.SINGLE_PARTITION, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits);
+        super(Kind.SINGLE_PARTITION, isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, index);
         assert partitionKey.getPartitioner() == metadata.partitioner;
         this.partitionKey = partitionKey;
         this.clusteringIndexFilter = clusteringIndexFilter;
@@ -90,6 +91,44 @@ public class SinglePartitionReadCommand extends ReadCommand
     /**
      * Creates a new read command on a single partition.
      *
+     * @param isForThrift whether the query is for thrift or not.
+     * @param metadata the table to query.
+     * @param nowInSec the time in seconds to use are "now" for this query.
+     * @param columnFilter the column filter to use for the query.
+     * @param rowFilter the row filter to use for the query.
+     * @param limits the limits to use for the query.
+     * @param partitionKey the partition key for the partition to query.
+     * @param clusteringIndexFilter the clustering index filter to use for the query.
+     * @param indexMetadata explicitly specified index to use for the query
+     *
+     * @return a newly created read command.
+     */
+    public static SinglePartitionReadCommand create(boolean isForThrift,
+                                                    CFMetaData metadata,
+                                                    int nowInSec,
+                                                    ColumnFilter columnFilter,
+                                                    RowFilter rowFilter,
+                                                    DataLimits limits,
+                                                    DecoratedKey partitionKey,
+                                                    ClusteringIndexFilter clusteringIndexFilter,
+                                                    IndexMetadata indexMetadata)
+    {
+        return new SinglePartitionReadCommand(false,
+                                              0,
+                                              isForThrift,
+                                              metadata,
+                                              nowInSec,
+                                              columnFilter,
+                                              rowFilter,
+                                              limits,
+                                              partitionKey,
+                                              clusteringIndexFilter,
+                                              indexMetadata);
+    }
+
+    /**
+     * Creates a new read command on a single partition.
+     *
      * @param metadata the table to query.
      * @param nowInSec the time in seconds to use are "now" for this query.
      * @param columnFilter the column filter to use for the query.
@@ -112,7 +151,7 @@ public class SinglePartitionReadCommand extends ReadCommand
     }
 
     /**
-     * Creates a new read command on a single partition for thrift.
+     * Creates a new read command on a single partition.
      *
      * @param isForThrift whether the query is for thrift or not.
      * @param metadata the table to query.
@@ -134,7 +173,15 @@ public class SinglePartitionReadCommand extends ReadCommand
                                                     DecoratedKey partitionKey,
                                                     ClusteringIndexFilter clusteringIndexFilter)
     {
-        return new SinglePartitionReadCommand(false, 0, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, partitionKey, clusteringIndexFilter);
+        return create(isForThrift,
+                      metadata,
+                      nowInSec,
+                      columnFilter,
+                      rowFilter,
+                      limits,
+                      partitionKey,
+                      clusteringIndexFilter,
+                      findIndex(metadata, rowFilter));
     }
 
     /**
@@ -148,7 +195,11 @@ public class SinglePartitionReadCommand extends ReadCommand
      *
      * @return a newly created read command. The returned command will use no row filter and have no limits.
      */
-    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, ColumnFilter columnFilter, ClusteringIndexFilter filter)
+    public static SinglePartitionReadCommand create(CFMetaData metadata,
+                                                    int nowInSec,
+                                                    DecoratedKey key,
+                                                    ColumnFilter columnFilter,
+                                                    ClusteringIndexFilter filter)
     {
         return create(metadata, nowInSec, columnFilter, RowFilter.NONE, DataLimits.NONE, key, filter);
     }
@@ -164,7 +215,7 @@ public class SinglePartitionReadCommand extends ReadCommand
      */
     public static SinglePartitionReadCommand fullPartitionRead(CFMetaData metadata, int nowInSec, DecoratedKey key)
     {
-        return SinglePartitionReadCommand.create(metadata, nowInSec, key, Slices.ALL);
+        return create(metadata, nowInSec, key, Slices.ALL);
     }
 
     /**
@@ -178,7 +229,7 @@ public class SinglePartitionReadCommand extends ReadCommand
      */
     public static SinglePartitionReadCommand fullPartitionRead(CFMetaData metadata, int nowInSec, ByteBuffer key)
     {
-        return SinglePartitionReadCommand.create(metadata, nowInSec, metadata.decorateKey(key), Slices.ALL);
+        return create(metadata, nowInSec, metadata.decorateKey(key), Slices.ALL);
     }
 
     /**
@@ -211,7 +262,7 @@ public class SinglePartitionReadCommand extends ReadCommand
     public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, Slices slices)
     {
         ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(slices, false);
-        return SinglePartitionReadCommand.create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
+        return create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
     }
 
     /**
@@ -244,7 +295,7 @@ public class SinglePartitionReadCommand extends ReadCommand
     public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, NavigableSet<Clustering> names)
     {
         ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(names, false);
-        return SinglePartitionReadCommand.create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
+        return create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
     }
 
     /**
@@ -265,7 +316,82 @@ public class SinglePartitionReadCommand extends ReadCommand
 
     public SinglePartitionReadCommand copy()
     {
-        return new SinglePartitionReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), partitionKey(), clusteringIndexFilter());
+        return new SinglePartitionReadCommand(isDigestQuery(),
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              partitionKey(),
+                                              clusteringIndexFilter(),
+                                              indexMetadata());
+    }
+
+    public SinglePartitionReadCommand copyAsDigestQuery()
+    {
+        return new SinglePartitionReadCommand(true,
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              partitionKey(),
+                                              clusteringIndexFilter(),
+                                              indexMetadata());
+    }
+
+    public SinglePartitionReadCommand withUpdatedClusteringIndexFilter(ClusteringIndexFilter filter)
+    {
+        return new SinglePartitionReadCommand(isDigestQuery(),
+                                              digestVersion(),
+                                              isForThrift(),
+                                              metadata(),
+                                              nowInSec(),
+                                              columnFilter(),
+                                              rowFilter(),
+                                              limits(),
+                                              partitionKey(),
+                                              filter,
+                                              indexMetadata());
+    }
+
+    static SinglePartitionReadCommand legacySliceCommand(boolean isDigest,
+                                                         int digestVersion,
+                                                         CFMetaData metadata,
+                                                         int nowInSec,
+                                                         ColumnFilter columnFilter,
+                                                         DataLimits limits,
+                                                         DecoratedKey partitionKey,
+                                                         ClusteringIndexSliceFilter filter)
+    {
+        // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
+        return new SinglePartitionReadCommand(isDigest,
+                                              digestVersion,
+                                              true,
+                                              metadata,
+                                              nowInSec,
+                                              columnFilter,
+                                              RowFilter.NONE,
+                                              limits,
+                                              partitionKey,
+                                              filter,
+                                              null);
+    }
+
+    static SinglePartitionReadCommand legacyNamesCommand(boolean isDigest,
+                                                         int digestVersion,
+                                                         CFMetaData metadata,
+                                                         int nowInSec,
+                                                         ColumnFilter columnFilter,
+                                                         DecoratedKey partitionKey,
+                                                         ClusteringIndexNamesFilter filter)
+    {
+        // messages from old nodes will expect the thrift format, so always use 'true' for isForThrift
+        return new SinglePartitionReadCommand(isDigest, digestVersion, true, metadata, nowInSec, columnFilter, RowFilter.NONE, DataLimits.NONE, partitionKey, filter,null);
     }
 
     public DecoratedKey partitionKey()
@@ -432,7 +558,7 @@ public class SinglePartitionReadCommand extends ReadCommand
                 final int rowsToCache = metadata().params.caching.rowsPerPartitionToCache();
 
                 @SuppressWarnings("resource") // we close on exception or upon closing the result of this method
-                UnfilteredRowIterator iter = SinglePartitionReadCommand.fullPartitionRead(metadata(), nowInSec(), partitionKey()).queryMemtableAndDisk(cfs, readOp);
+                UnfilteredRowIterator iter = fullPartitionRead(metadata(), nowInSec(), partitionKey()).queryMemtableAndDisk(cfs, readOp);
                 try
                 {
                     // Use a custom iterator instead of DataLimits to avoid stopping the original iterator
@@ -1068,12 +1194,22 @@ public class SinglePartitionReadCommand extends ReadCommand
 
     private static class Deserializer extends SelectionDeserializer
     {
-        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, boolean isForThrift, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
+        public ReadCommand deserialize(DataInputPlus in,
+                                       int version,
+                                       boolean isDigest,
+                                       int digestVersion,
+                                       boolean isForThrift,
+                                       CFMetaData metadata,
+                                       int nowInSec,
+                                       ColumnFilter columnFilter,
+                                       RowFilter rowFilter,
+                                       DataLimits limits,
+                                       IndexMetadata index)
         throws IOException
         {
             DecoratedKey key = metadata.decorateKey(metadata.getKeyValidator().readValue(in, DatabaseDescriptor.getMaxValueSize()));
             ClusteringIndexFilter filter = ClusteringIndexFilter.serializer.deserialize(in, version, metadata);
-            return new SinglePartitionReadCommand(isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter);
+            return new SinglePartitionReadCommand(isDigest, digestVersion, isForThrift, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter, index);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index dd6dde4..5976ddf 100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@ -40,6 +40,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.concurrent.StageManager;
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.statements.IndexTarget;
@@ -697,17 +698,17 @@ public class SecondaryIndexManager implements IndexRegistry
      * cached for future use when obtaining a Searcher, getting the index's underlying CFS for
      * ReadOrderGroup, or an estimate of the result size from an average index query.
      *
-     * @param command ReadCommand to be executed
+     * @param rowFilter RowFilter of the command to be executed
      * @return an Index instance, ready to use during execution of the command, or null if none
      * of the registered indexes can support the command.
      */
-    public Index getBestIndexFor(ReadCommand command)
+    public Index getBestIndexFor(RowFilter rowFilter)
     {
-        if (indexes.isEmpty() || command.rowFilter().isEmpty())
+        if (indexes.isEmpty() || rowFilter.isEmpty())
             return null;
 
         Set<Index> searchableIndexes = new HashSet<>();
-        for (RowFilter.Expression expression : command.rowFilter())
+        for (RowFilter.Expression expression : rowFilter)
         {
             if (expression.isCustom())
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
index 135839b..f8a7c66 100644
--- a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
@@ -136,13 +136,15 @@ public class CompositesSearcher extends CassandraIndexSearcher
 
                     // Query the gathered index hits. We still need to filter stale hits from the resulting query.
                     ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(clusterings.build(), false);
-                    SinglePartitionReadCommand dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata,
+                    SinglePartitionReadCommand dataCmd = SinglePartitionReadCommand.create(isForThrift(),
+                                                                                           index.baseCfs.metadata,
                                                                                            command.nowInSec(),
                                                                                            command.columnFilter(),
                                                                                            command.rowFilter(),
                                                                                            DataLimits.NONE,
                                                                                            partitionKey,
-                                                                                           filter);
+                                                                                           filter,
+                                                                                           null);
                     @SuppressWarnings("resource") // We close right away if empty, and if it's assign to next it will be called either
                     // by the next caller of next, or through closing this iterator is this come before.
                     UnfilteredRowIterator dataIter =

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
index 189b652..c14c5a7 100644
--- a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
@@ -99,7 +99,8 @@ public class KeysSearcher extends CassandraIndexSearcher
                                                                                            command.rowFilter(),
                                                                                            DataLimits.NONE,
                                                                                            key,
-                                                                                           command.clusteringIndexFilter(key));
+                                                                                           command.clusteringIndexFilter(key),
+                                                                                           null);
 
                     @SuppressWarnings("resource") // filterIfStale closes it's iterator if either it materialize it or if it returns null.
                                                   // Otherwise, we close right away if empty, and if it's assigned to next it will be called either

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
index cae1f1a..177fdb2 100644
--- a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
+++ b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
@@ -88,7 +88,7 @@ public abstract class AbstractReadExecutor
 
     protected void makeDigestRequests(Iterable<InetAddress> endpoints)
     {
-        makeRequests(command.copy().setIsDigestQuery(true), endpoints);
+        makeRequests(command.copyAsDigestQuery(), endpoints);
     }
 
     private void makeRequests(ReadCommand readCommand, Iterable<InetAddress> endpoints)
@@ -284,7 +284,7 @@ public abstract class AbstractReadExecutor
                 // Could be waiting on the data, or on enough digests.
                 ReadCommand retryCommand = command;
                 if (handler.resolver.isDataPresent())
-                    retryCommand = command.copy().setIsDigestQuery(true);
+                    retryCommand = command.copyAsDigestQuery();
 
                 InetAddress extraReplica = Iterables.getLast(targetReplicas);
                 if (traceState != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java b/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
index 9c216e3..ea79017 100644
--- a/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.service.pager;
 
-import java.util.Optional;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -27,8 +25,6 @@ import org.apache.cassandra.db.filter.DataLimits;
 import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.index.Index;
-import org.apache.cassandra.schema.IndexMetadata;
 
 /**
  * Pages a PartitionRangeReadCommand.
@@ -90,9 +86,7 @@ public class PartitionRangeQueryPager extends AbstractQueryPager
             }
         }
 
-        Index index = command.getIndex(Keyspace.openAndGetStore(command.metadata()));
-        Optional<IndexMetadata> indexMetadata = index != null ? Optional.of(index.getIndexMetadata()) : Optional.empty();
-        return new PartitionRangeReadCommand(command.metadata(), command.nowInSec(), command.columnFilter(), command.rowFilter(), limits, pageRange, indexMetadata);
+        return ((PartitionRangeReadCommand) command).withUpdatedLimitsAndDataRange(limits, pageRange);
     }
 
     protected void recordLast(DecoratedKey key, Row last)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 86caac3..cb74b15 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -48,7 +48,6 @@ import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.index.Index;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.locator.DynamicEndpointSnitch;
 import org.apache.cassandra.metrics.ClientMetrics;
@@ -1520,16 +1519,16 @@ public class CassandraServer implements Cassandra.Iface
                 ColumnFilter columns = makeColumnFilter(metadata, column_parent, predicate);
                 ClusteringIndexFilter filter = toInternalFilter(metadata, column_parent, predicate);
                 DataLimits limits = getLimits(range.count, metadata.isSuper() && !column_parent.isSetSuper_column(), predicate);
-                PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                              0,
-                                                                              true,
-                                                                              metadata,
-                                                                              nowInSec,
-                                                                              columns,
-                                                                              ThriftConversion.rowFilterFromThrift(metadata, range.row_filter),
-                                                                              limits,
-                                                                              new DataRange(bounds, filter),
-                                                                              Optional.empty());
+
+                PartitionRangeReadCommand cmd =
+                    PartitionRangeReadCommand.create(true,
+                                                     metadata,
+                                                     nowInSec,
+                                                     columns,
+                                                     ThriftConversion.rowFilterFromThrift(metadata, range.row_filter),
+                                                     limits,
+                                                     new DataRange(bounds, filter));
+
                 try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
                 {
                     assert results != null;
@@ -1614,16 +1613,16 @@ public class CassandraServer implements Cassandra.Iface
                 Clustering pageFrom = metadata.isSuper()
                                     ? new Clustering(start_column)
                                     : LegacyLayout.decodeCellName(metadata, start_column).clustering;
-                PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                              0,
-                                                                              true,
-                                                                              metadata,
-                                                                              nowInSec,
-                                                                              ColumnFilter.all(metadata),
-                                                                              RowFilter.NONE,
-                                                                              limits,
-                                                                              new DataRange(bounds, filter).forPaging(bounds, metadata.comparator, pageFrom, true),
-                                                                              Optional.empty());
+
+                PartitionRangeReadCommand cmd =
+                    PartitionRangeReadCommand.create(true,
+                                                     metadata,
+                                                     nowInSec,
+                                                     ColumnFilter.all(metadata),
+                                                     RowFilter.NONE,
+                                                     limits,
+                                                     new DataRange(bounds, filter).forPaging(bounds, metadata.comparator, pageFrom, true));
+
                 try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
                 {
                     return thriftifyKeySlices(results, new ColumnParent(column_family), limits.perPartitionCount());
@@ -1706,21 +1705,17 @@ public class CassandraServer implements Cassandra.Iface
             ColumnFilter columns = makeColumnFilter(metadata, column_parent, column_predicate);
             ClusteringIndexFilter filter = toInternalFilter(metadata, column_parent, column_predicate);
             DataLimits limits = getLimits(index_clause.count, metadata.isSuper() && !column_parent.isSetSuper_column(), column_predicate);
-            PartitionRangeReadCommand cmd = new PartitionRangeReadCommand(false,
-                                                                          0,
-                                                                          true,
-                                                                          metadata,
-                                                                          nowInSec,
-                                                                          columns,
-                                                                          ThriftConversion.rowFilterFromThrift(metadata, index_clause.expressions),
-                                                                          limits,
-                                                                          new DataRange(bounds, filter),
-                                                                          Optional.empty());
-            // If there's a secondary index that the command can use, have it validate
-            // the request parameters. Note that as a side effect, if a viable Index is
-            // identified by the CFS's index manager, it will be cached in the command
-            // and serialized during distribution to replicas in order to avoid performing
-            // further lookups.
+
+            PartitionRangeReadCommand cmd =
+                PartitionRangeReadCommand.create(true,
+                                                 metadata,
+                                                 nowInSec,
+                                                 columns,
+                                                 ThriftConversion.rowFilterFromThrift(metadata, index_clause.expressions),
+                                                 limits,
+                                                 new DataRange(bounds, filter));
+
+            // If there's a secondary index that the command can use, have it validate the request parameters.
             cmd.maybeValidateIndex();
 
             try (PartitionIterator results = StorageProxy.getRangeSlice(cmd, consistencyLevel))
@@ -2533,7 +2528,7 @@ public class CassandraServer implements Cassandra.Iface
                 // We want to know if the partition exists, so just fetch a single cell.
                 ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(Slices.ALL, false);
                 DataLimits limits = DataLimits.thriftLimits(1, 1);
-                return new SinglePartitionReadCommand(false, 0, true, metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, limits, key, filter);
+                return SinglePartitionReadCommand.create(true, metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, limits, key, filter);
             }
 
             // Gather the clustering for the expected values and query those.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index e8b42bc..d758efe 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -629,31 +629,7 @@ public class Util
                                                              ColumnFamilyStore cfs,
                                                              ReadOrderGroup orderGroup)
     {
-        return new InternalPartitionRangeReadCommand(command).queryStorageInternal(cfs, orderGroup);
-    }
-
-    private static final class InternalPartitionRangeReadCommand extends PartitionRangeReadCommand
-    {
-
-        private InternalPartitionRangeReadCommand(PartitionRangeReadCommand original)
-        {
-            super(original.isDigestQuery(),
-                  original.digestVersion(),
-                  original.isForThrift(),
-                  original.metadata(),
-                  original.nowInSec(),
-                  original.columnFilter(),
-                  original.rowFilter(),
-                  original.limits(),
-                  original.dataRange(),
-                  Optional.empty());
-        }
-
-        private UnfilteredPartitionIterator queryStorageInternal(ColumnFamilyStore cfs,
-                                                                 ReadOrderGroup orderGroup)
-        {
-            return queryStorage(cfs, orderGroup);
-        }
+        return command.queryStorage(cfs, orderGroup);
     }
 
     public static Closeable markDirectoriesUnwriteable(ColumnFamilyStore cfs)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
index bbccc48..2457c4a 100644
--- a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
@@ -118,7 +118,7 @@ public class SecondaryIndexTest
                                       .filterOn("birthdate", Operator.EQ, 1L)
                                       .build();
 
-        Index.Searcher searcher = cfs.indexManager.getBestIndexFor(rc).searcherFor(rc);
+        Index.Searcher searcher = rc.index().searcherFor(rc);
         try (ReadOrderGroup orderGroup = rc.startOrderGroup(); UnfilteredPartitionIterator pi = searcher.search(orderGroup))
         {
             assertTrue(pi.hasNext());
@@ -204,7 +204,7 @@ public class SecondaryIndexTest
 
         // verify that it's not being indexed under any other value either
         ReadCommand rc = Util.cmd(cfs).build();
-        assertNull(cfs.indexManager.getBestIndexFor(rc));
+        assertNull(rc.index());
 
         // resurrect w/ a newer timestamp
         new RowUpdateBuilder(cfs.metadata, 2, "k1").clustering("c").add("birthdate", 1L).build().apply();;
@@ -222,13 +222,13 @@ public class SecondaryIndexTest
         // todo - checking the # of index searchers for the command is probably not the best thing to test here
         RowUpdateBuilder.deleteRow(cfs.metadata, 3, "k1", "c").applyUnsafe();
         rc = Util.cmd(cfs).build();
-        assertNull(cfs.indexManager.getBestIndexFor(rc));
+        assertNull(rc.index());
 
         // make sure obsolete mutations don't generate an index entry
         // todo - checking the # of index searchers for the command is probably not the best thing to test here
         new RowUpdateBuilder(cfs.metadata, 3, "k1").clustering("c").add("birthdate", 1L).build().apply();;
         rc = Util.cmd(cfs).build();
-        assertNull(cfs.indexManager.getBestIndexFor(rc));
+        assertNull(rc.index());
     }
 
     @Test
@@ -504,7 +504,7 @@ public class SecondaryIndexTest
         ColumnDefinition cdef = cfs.metadata.getColumnDefinition(col);
 
         ReadCommand rc = Util.cmd(cfs).filterOn(cdef.name.toString(), Operator.EQ, ((AbstractType) cdef.cellValueType()).decompose(val)).build();
-        Index.Searcher searcher = cfs.indexManager.getBestIndexFor(rc).searcherFor(rc);
+        Index.Searcher searcher = rc.index().searcherFor(rc);
         if (count != 0)
             assertNotNull(searcher);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
index 7f59e2f..02b642e 100644
--- a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
+++ b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
@@ -116,13 +116,14 @@ public class SinglePartitionSliceCommandTest
         ByteBuffer zero = ByteBufferUtil.bytes(0);
         Slices slices = Slices.with(cfm.comparator, Slice.make(Slice.Bound.inclusiveStartOf(zero), Slice.Bound.inclusiveEndOf(zero)));
         ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(slices, false);
-        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                          FBUtilities.nowInSeconds(),
-                                                          columnFilter,
-                                                          RowFilter.NONE,
-                                                          DataLimits.NONE,
-                                                          key,
-                                                          sliceFilter);
+        ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                            cfm,
+                                                            FBUtilities.nowInSeconds(),
+                                                            columnFilter,
+                                                            RowFilter.NONE,
+                                                            DataLimits.NONE,
+                                                            key,
+                                                            sliceFilter);
 
         DataOutputBuffer out = new DataOutputBuffer((int) ReadCommand.legacyReadCommandSerializer.serializedSize(cmd, MessagingService.VERSION_21));
         ReadCommand.legacyReadCommandSerializer.serialize(cmd, out, MessagingService.VERSION_21);
@@ -166,13 +167,14 @@ public class SinglePartitionSliceCommandTest
 
         ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
         ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.NONE, false);
-        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                         FBUtilities.nowInSeconds(),
-                                                         columnFilter,
-                                                         RowFilter.NONE,
-                                                         DataLimits.NONE,
-                                                         key,
-                                                         sliceFilter);
+        ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                            cfm,
+                                                            FBUtilities.nowInSeconds(),
+                                                            columnFilter,
+                                                            RowFilter.NONE,
+                                                            DataLimits.NONE,
+                                                            key,
+                                                            sliceFilter);
 
         // check raw iterator for static cell
         try (ReadOrderGroup orderGroup = cmd.startOrderGroup(); UnfilteredPartitionIterator pi = cmd.executeLocally(orderGroup))
@@ -224,13 +226,14 @@ public class SinglePartitionSliceCommandTest
         ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
         Slice slice = Slice.make(Slice.Bound.BOTTOM, Slice.Bound.inclusiveEndOf(ByteBufferUtil.bytes("i1")));
         ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.with(cfm.comparator, slice), false);
-        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, true, cfm,
-                                                         FBUtilities.nowInSeconds(),
-                                                         columnFilter,
-                                                         RowFilter.NONE,
-                                                         DataLimits.NONE,
-                                                         key,
-                                                         sliceFilter);
+        ReadCommand cmd = SinglePartitionReadCommand.create(true,
+                                                            cfm,
+                                                            FBUtilities.nowInSeconds(),
+                                                            columnFilter,
+                                                            RowFilter.NONE,
+                                                            DataLimits.NONE,
+                                                            key,
+                                                            sliceFilter);
 
         String ret = cmd.toCQLString();
         Assert.assertNotNull(ret);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f297bcf/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 640b68b..c2598ec 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -604,7 +604,7 @@ public class SSTableReaderTest
                                              .columns("birthdate")
                                              .filterOn("birthdate", Operator.EQ, 1L)
                                              .build();
-        Index.Searcher searcher = indexedCFS.indexManager.getBestIndexFor(rc).searcherFor(rc);
+        Index.Searcher searcher = rc.index().searcherFor(rc);
         assertNotNull(searcher);
         try (ReadOrderGroup orderGroup = ReadOrderGroup.forCommand(rc))
         {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[08/10] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7ad1945e/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
index e9051b4,0000000..03d89e1
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
@@@ -1,2566 -1,0 +1,2568 @@@
 +/*
 + * 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.index.sasi;
 +
 +import java.io.FileWriter;
 +import java.io.Writer;
 +import java.nio.ByteBuffer;
 +import java.nio.file.FileSystems;
 +import java.nio.file.Files;
 +import java.nio.file.Path;
 +import java.nio.file.attribute.BasicFileAttributes;
 +import java.util.*;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.Executors;
 +import java.util.concurrent.ThreadLocalRandom;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import org.apache.cassandra.SchemaLoader;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.index.Index;
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.Term;
 +import org.apache.cassandra.cql3.statements.IndexTarget;
 +import org.apache.cassandra.cql3.statements.SelectStatement;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.filter.ColumnFilter;
 +import org.apache.cassandra.db.filter.DataLimits;
 +import org.apache.cassandra.db.filter.RowFilter;
 +import org.apache.cassandra.db.lifecycle.SSTableSet;
 +import org.apache.cassandra.db.marshal.*;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.dht.IPartitioner;
 +import org.apache.cassandra.dht.Murmur3Partitioner;
 +import org.apache.cassandra.dht.Range;
 +import org.apache.cassandra.exceptions.ConfigurationException;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.index.sasi.conf.ColumnIndex;
 +import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
 +import org.apache.cassandra.index.sasi.exceptions.TimeQuotaExceededException;
 +import org.apache.cassandra.index.sasi.memory.IndexMemtable;
 +import org.apache.cassandra.index.sasi.plan.QueryController;
 +import org.apache.cassandra.index.sasi.plan.QueryPlan;
 +import org.apache.cassandra.io.sstable.SSTable;
 +import org.apache.cassandra.schema.IndexMetadata;
 +import org.apache.cassandra.schema.KeyspaceMetadata;
 +import org.apache.cassandra.schema.KeyspaceParams;
 +import org.apache.cassandra.schema.Tables;
 +import org.apache.cassandra.serializers.MarshalException;
 +import org.apache.cassandra.serializers.TypeSerializer;
 +import org.apache.cassandra.service.MigrationManager;
 +import org.apache.cassandra.service.QueryState;
 +import org.apache.cassandra.thrift.CqlRow;
 +import org.apache.cassandra.transport.messages.ResultMessage;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.Pair;
 +
 +import com.google.common.collect.Lists;
 +import com.google.common.util.concurrent.Uninterruptibles;
 +
 +import junit.framework.Assert;
 +
 +import org.junit.*;
 +
 +public class SASIIndexTest
 +{
 +    private static final IPartitioner PARTITIONER;
 +
 +    static {
 +        System.setProperty("cassandra.config", "cassandra-murmur.yaml");
 +        PARTITIONER = Murmur3Partitioner.instance;
 +    }
 +
 +    private static final String KS_NAME = "sasi";
 +    private static final String CF_NAME = "test_cf";
 +    private static final String CLUSTERING_CF_NAME_1 = "clustering_test_cf_1";
 +    private static final String CLUSTERING_CF_NAME_2 = "clustering_test_cf_2";
 +    private static final String STATIC_CF_NAME = "static_sasi_test_cf";
 +    private static final String FTS_CF_NAME = "full_text_search_sasi_test_cf";
 +
 +    @BeforeClass
 +    public static void loadSchema() throws ConfigurationException
 +    {
 +        SchemaLoader.loadSchema();
 +        MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(KS_NAME,
 +                                                                     KeyspaceParams.simpleTransient(1),
 +                                                                     Tables.of(SchemaLoader.sasiCFMD(KS_NAME, CF_NAME),
 +                                                                               SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME_1),
 +                                                                               SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME_2, "location"),
 +                                                                               SchemaLoader.staticSASICFMD(KS_NAME, STATIC_CF_NAME),
 +                                                                               SchemaLoader.fullTextSearchSASICFMD(KS_NAME, FTS_CF_NAME))));
 +    }
 +
 +    @Before
 +    public void cleanUp()
 +    {
 +        Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME).truncateBlocking();
 +    }
 +
 +    @Test
 +    public void testSingleExpressionQueries() throws Exception
 +    {
 +        testSingleExpressionQueries(false);
 +        cleanupData();
 +        testSingleExpressionQueries(true);
 +    }
 +
 +    private void testSingleExpressionQueries(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +            put("key1", Pair.create("Pavel", 14));
 +            put("key2", Pair.create("Pavel", 26));
 +            put("key3", Pair.create("Pavel", 27));
 +            put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("av")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("as")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("aw")));
 +        Assert.assertEquals(rows.toString(), 0, rows.size());
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("avel")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("n")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(27)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key3", "key4"}, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(13)));
 +        Assert.assertEquals(rows.toString(), 0, rows.size());
 +    }
 +
 +    @Test
 +    public void testEmptyTokenizedResults() throws Exception
 +    {
 +        testEmptyTokenizedResults(false);
 +        cleanupData();
 +        testEmptyTokenizedResults(true);
 +    }
 +
 +    private void testEmptyTokenizedResults(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("  ", 14));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data, forceFlush);
 +
 +        Set<String> rows= getIndexed(store, 10, buildExpression(UTF8Type.instance.decompose("first_name"), Operator.LIKE_MATCHES, UTF8Type.instance.decompose("doesntmatter")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{}, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testMultiExpressionQueries() throws Exception
 +    {
 +        testMultiExpressionQueries(false);
 +        cleanupData();
 +        testMultiExpressionQueries(true);
 +    }
 +
 +    public void testMultiExpressionQueries(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> data = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 26));
 +                put("key3", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows;
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(14)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(27)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key1", "key2"}, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GT, Int32Type.instance.decompose(14)),
 +                         buildExpression(age, Operator.LT, Int32Type.instance.decompose(27)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GT, Int32Type.instance.decompose(12)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GTE, Int32Type.instance.decompose(13)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.GTE, Int32Type.instance.decompose(16)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.LT, Int32Type.instance.decompose(30)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.LTE, Int32Type.instance.decompose(29)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                         buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                         buildExpression(age, Operator.LTE, Int32Type.instance.decompose(25)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("avel")),
 +                                     buildExpression(age, Operator.LTE, Int32Type.instance.decompose(25)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("n")),
 +                                     buildExpression(age, Operator.LTE, Int32Type.instance.decompose(25)));
 +        Assert.assertTrue(rows.isEmpty());
 +
 +    }
 +
 +    @Test
 +    public void testCrossSSTableQueries() throws Exception
 +    {
 +        testCrossSSTableQueries(false);
 +        cleanupData();
 +        testCrossSSTableQueries(true);
 +
 +    }
 +
 +    private void testCrossSSTableQueries(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("Maxie", 43));
 +                put("key1", Pair.create("Chelsie", 33));
 +                put("key2", Pair.create("Josephine", 43));
 +                put("key3", Pair.create("Shanna", 27));
 +                put("key4", Pair.create("Amiya", 36));
 +            }};
 +
 +        loadData(part1, forceFlush); // first sstable
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key5", Pair.create("Americo", 20));
 +                put("key6", Pair.create("Fiona", 39));
 +                put("key7", Pair.create("Francis", 41));
 +                put("key8", Pair.create("Charley", 21));
 +                put("key9", Pair.create("Amely", 40));
 +            }};
 +
 +        loadData(part2, forceFlush);
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key10", Pair.create("Eddie", 42));
 +                put("key11", Pair.create("Oswaldo", 35));
 +                put("key12", Pair.create("Susana", 35));
 +                put("key13", Pair.create("Alivia", 42));
 +                put("key14", Pair.create("Demario", 28));
 +            }};
 +
 +        ColumnFamilyStore store = loadData(part3, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows;
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("Fiona")),
 +                                     buildExpression(age, Operator.LT, Int32Type.instance.decompose(40)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key6" }, rows.toArray(new String[rows.size()])));
 +
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key14",
 +                                                                        "key3", "key4", "key6", "key7", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 5,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertEquals(rows.toString(), 5, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GTE, Int32Type.instance.decompose(35)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key4", "key6", "key7" },
 +                                                         rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key3", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(27)),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(50)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ie")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(43)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key10" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key12", "key13", "key3", "key4", "key6" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(33)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testQueriesThatShouldBeTokenized() throws Exception
 +    {
 +        testQueriesThatShouldBeTokenized(false);
 +        cleanupData();
 +        testQueriesThatShouldBeTokenized(true);
 +    }
 +
 +    private void testQueriesThatShouldBeTokenized(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("If you can dream it, you can do it.", 43));
 +                put("key1", Pair.create("What you get by achieving your goals is not " +
 +                        "as important as what you become by achieving your goals, do it.", 33));
 +                put("key2", Pair.create("Keep your face always toward the sunshine " +
 +                        "- and shadows will fall behind you.", 43));
 +                put("key3", Pair.create("We can't help everyone, but everyone can " +
 +                        "help someone.", 27));
 +            }};
 +
 +        ColumnFamilyStore store = loadData(part1, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows = getIndexed(store, 10,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS,
 +                        UTF8Type.instance.decompose("What you get by achieving your goals")),
 +                buildExpression(age, Operator.GT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertEquals(rows.toString(), Collections.singleton("key1"), rows);
 +
 +        rows = getIndexed(store, 10,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("do it.")));
 +
 +        Assert.assertEquals(rows.toString(), Arrays.asList("key0", "key1"), Lists.newArrayList(rows));
 +    }
 +
 +    @Test
 +    public void testPrefixSearchWithContainsMode() throws Exception
 +    {
 +        testPrefixSearchWithContainsMode(false);
 +        cleanupData();
 +        testPrefixSearchWithContainsMode(true);
 +    }
 +
 +    private void testPrefixSearchWithContainsMode(boolean forceFlush) throws Exception
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(FTS_CF_NAME);
 +
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("1a4abbcd-b5de-4c69-a578-31231e01ff09"), "Poker Face", "Lady Gaga");
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("9472a394-359b-4a06-b1d5-b6afce590598"), "Forgetting the Way Home", "Our Lady of Bells");
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("4f8dc18e-54e6-4e16-b507-c5324b61523b"), "Zamki na piasku", "Lady Pank");
 +        executeCQL(FTS_CF_NAME, "INSERT INTO %s.%s (song_id, title, artist) VALUES(?, ?, ?)", UUID.fromString("eaf294fa-bad5-49d4-8f08-35ba3636a706"), "Koncertowa", "Lady Pank");
 +
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        final UntypedResultSet results = executeCQL(FTS_CF_NAME, "SELECT * FROM %s.%s WHERE artist LIKE 'lady%%'");
 +        Assert.assertNotNull(results);
 +        Assert.assertEquals(3, results.size());
 +    }
 +
 +    @Test
 +    public void testMultiExpressionQueriesWhereRowSplitBetweenSSTables() throws Exception
 +    {
 +        testMultiExpressionQueriesWhereRowSplitBetweenSSTables(false);
 +        cleanupData();
 +        testMultiExpressionQueriesWhereRowSplitBetweenSSTables(true);
 +    }
 +
 +    private void testMultiExpressionQueriesWhereRowSplitBetweenSSTables(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("Maxie", -1));
 +                put("key1", Pair.create("Chelsie", 33));
 +                put("key2", Pair.create((String)null, 43));
 +                put("key3", Pair.create("Shanna", 27));
 +                put("key4", Pair.create("Amiya", 36));
 +        }};
 +
 +        loadData(part1, forceFlush); // first sstable
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key5", Pair.create("Americo", 20));
 +                put("key6", Pair.create("Fiona", 39));
 +                put("key7", Pair.create("Francis", 41));
 +                put("key8", Pair.create("Charley", 21));
 +                put("key9", Pair.create("Amely", 40));
 +                put("key14", Pair.create((String)null, 28));
 +        }};
 +
 +        loadData(part2, forceFlush);
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create((String)null, 43));
 +                put("key10", Pair.create("Eddie", 42));
 +                put("key11", Pair.create("Oswaldo", 35));
 +                put("key12", Pair.create("Susana", 35));
 +                put("key13", Pair.create("Alivia", 42));
 +                put("key14", Pair.create("Demario", -1));
 +                put("key2", Pair.create("Josephine", -1));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part3, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows = getIndexed(store, 10,
 +                                      buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("Fiona")),
 +                                      buildExpression(age, Operator.LT, Int32Type.instance.decompose(40)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key6" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key14",
 +                                                                        "key3", "key4", "key6", "key7", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 5,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        Assert.assertEquals(rows.toString(), 5, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GTE, Int32Type.instance.decompose(35)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key11", "key12", "key13", "key4", "key6", "key7" },
 +                                                         rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key3", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(27)),
 +                          buildExpression(age, Operator.LT, Int32Type.instance.decompose(32)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
 +
 +        Map<String, Pair<String, Integer>> part4 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key12", Pair.create((String)null, 12));
 +                put("key14", Pair.create("Demario", 42));
 +                put("key2", Pair.create("Frank", -1));
 +        }};
 +
 +        store = loadData(part4, forceFlush);
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("Susana")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(13)),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key12" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("Demario")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(30)));
 +        Assert.assertTrue(rows.toString(), rows.size() == 0);
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("Josephine")));
 +        Assert.assertTrue(rows.toString(), rows.size() == 0);
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.GT, Int32Type.instance.decompose(10)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(50)));
 +
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        rows = getIndexed(store, 10,
 +                          buildExpression(firstName, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ie")),
 +                          buildExpression(age, Operator.LTE, Int32Type.instance.decompose(43)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key0", "key1", "key10" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testPagination() throws Exception
 +    {
 +        testPagination(false);
 +        cleanupData();
 +        testPagination(true);
 +    }
 +
 +    private void testPagination(boolean forceFlush) throws Exception
 +    {
 +        // split data into 3 distinct SSTables to test paging with overlapping token intervals.
 +
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key01", Pair.create("Ali", 33));
 +                put("key02", Pair.create("Jeremy", 41));
 +                put("key03", Pair.create("Elvera", 22));
 +                put("key04", Pair.create("Bailey", 45));
 +                put("key05", Pair.create("Emerson", 32));
 +                put("key06", Pair.create("Kadin", 38));
 +                put("key07", Pair.create("Maggie", 36));
 +                put("key08", Pair.create("Kailey", 36));
 +                put("key09", Pair.create("Armand", 21));
 +                put("key10", Pair.create("Arnold", 35));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key11", Pair.create("Ken", 38));
 +                put("key12", Pair.create("Penelope", 43));
 +                put("key13", Pair.create("Wyatt", 34));
 +                put("key14", Pair.create("Johnpaul", 34));
 +                put("key15", Pair.create("Trycia", 43));
 +                put("key16", Pair.create("Aida", 21));
 +                put("key17", Pair.create("Devon", 42));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key18", Pair.create("Christina", 20));
 +                put("key19", Pair.create("Rick", 19));
 +                put("key20", Pair.create("Fannie", 22));
 +                put("key21", Pair.create("Keegan", 29));
 +                put("key22", Pair.create("Ignatius", 36));
 +                put("key23", Pair.create("Ellis", 26));
 +                put("key24", Pair.create("Annamarie", 29));
 +                put("key25", Pair.create("Tianna", 31));
 +                put("key26", Pair.create("Dennis", 32));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part1, forceFlush);
 +
 +        loadData(part2, forceFlush);
 +        loadData(part3, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<DecoratedKey> uniqueKeys = getPaged(store, 4,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                buildExpression(age, Operator.GTE, Int32Type.instance.decompose(21)));
 +
 +
 +        List<String> expected = new ArrayList<String>()
 +        {{
 +                add("key25");
 +                add("key20");
 +                add("key13");
 +                add("key22");
 +                add("key09");
 +                add("key14");
 +                add("key16");
 +                add("key24");
 +                add("key03");
 +                add("key04");
 +                add("key08");
 +                add("key07");
 +                add("key15");
 +                add("key06");
 +                add("key21");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // now let's test a single equals condition
 +
 +        uniqueKeys = getPaged(store, 4, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +
 +        expected = new ArrayList<String>()
 +        {{
 +                add("key25");
 +                add("key20");
 +                add("key13");
 +                add("key22");
 +                add("key09");
 +                add("key14");
 +                add("key16");
 +                add("key24");
 +                add("key03");
 +                add("key04");
 +                add("key18");
 +                add("key08");
 +                add("key07");
 +                add("key15");
 +                add("key06");
 +                add("key21");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // now let's test something which is smaller than a single page
 +        uniqueKeys = getPaged(store, 4,
 +                              buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                              buildExpression(age, Operator.EQ, Int32Type.instance.decompose(36)));
 +
 +        expected = new ArrayList<String>()
 +        {{
 +                add("key22");
 +                add("key08");
 +                add("key07");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // the same but with the page size of 2 to test minimal pagination windows
 +
 +        uniqueKeys = getPaged(store, 2,
 +                              buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                              buildExpression(age, Operator.EQ, Int32Type.instance.decompose(36)));
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        // and last but not least, test age range query with pagination
 +        uniqueKeys = getPaged(store, 4,
 +                buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                buildExpression(age, Operator.GT, Int32Type.instance.decompose(20)),
 +                buildExpression(age, Operator.LTE, Int32Type.instance.decompose(36)));
 +
 +        expected = new ArrayList<String>()
 +        {{
 +                add("key25");
 +                add("key20");
 +                add("key13");
 +                add("key22");
 +                add("key09");
 +                add("key14");
 +                add("key16");
 +                add("key24");
 +                add("key03");
 +                add("key08");
 +                add("key07");
 +                add("key21");
 +        }};
 +
 +        Assert.assertEquals(expected, convert(uniqueKeys));
 +
 +        Set<String> rows;
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' limit 10 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key03", "key04", "key09", "key13", "key14", "key16", "key20", "key22", "key24", "key25" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' and token(id) >= token('key14') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key03", "key04", "key14", "key16", "key24" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' and token(id) >= token('key14') and token(id) <= token('key24') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14", "key16", "key24" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name LIKE '%%a%%' and age > 30 and token(id) >= token('key14') and token(id) <= token('key24') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key14" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name like '%%ie' limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key07", "key20", "key24" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = executeCQLWithKeys(String.format("SELECT * FROM %s.%s WHERE first_name like '%%ie' AND token(id) > token('key24') limit 5 ALLOW FILTERING;", KS_NAME, CF_NAME));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key07", "key24" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testColumnNamesWithSlashes() throws Exception
 +    {
 +        testColumnNamesWithSlashes(false);
 +        cleanupData();
 +        testColumnNamesWithSlashes(true);
 +    }
 +
 +    private void testColumnNamesWithSlashes(boolean forceFlush) throws Exception
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        Mutation rm1 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key1")));
 +        rm1.add(PartitionUpdate.singleRowUpdate(store.metadata,
 +                                                rm1.key(),
 +                                                buildRow(buildCell(store.metadata,
 +                                                                   UTF8Type.instance.decompose("/data/output/id"),
 +                                                                   AsciiType.instance.decompose("jason"),
 +                                                                   System.currentTimeMillis()))));
 +
 +        Mutation rm2 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key2")));
 +        rm2.add(PartitionUpdate.singleRowUpdate(store.metadata,
 +                                                rm2.key(),
 +                                                buildRow(buildCell(store.metadata,
 +                                                                   UTF8Type.instance.decompose("/data/output/id"),
 +                                                                   AsciiType.instance.decompose("pavel"),
 +                                                                   System.currentTimeMillis()))));
 +
 +        Mutation rm3 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key3")));
 +        rm3.add(PartitionUpdate.singleRowUpdate(store.metadata,
 +                                                rm3.key(),
 +                                                buildRow(buildCell(store.metadata,
 +                                                                   UTF8Type.instance.decompose("/data/output/id"),
 +                                                                   AsciiType.instance.decompose("Aleksey"),
 +                                                                   System.currentTimeMillis()))));
 +
 +        rm1.apply();
 +        rm2.apply();
 +        rm3.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        final ByteBuffer dataOutputId = UTF8Type.instance.decompose("/data/output/id");
 +
 +        Set<String> rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("A")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        // doesn't really make sense to rebuild index for in-memory data
 +        if (!forceFlush)
 +            return;
 +
 +        store.indexManager.invalidateAllIndexesBlocking();
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("A")));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +        // now let's trigger index rebuild and check if we got the data back
 +        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("data_output_id"));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        // also let's try to build an index for column which has no data to make sure that doesn't fail
 +        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("first_name"));
 +        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("data_output_id"));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("el")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testInvalidate() throws Exception
 +    {
 +        testInvalidate(false);
 +        cleanupData();
 +        testInvalidate(true);
 +    }
 +
 +    private void testInvalidate(boolean forceFlush) throws Exception
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key0", Pair.create("Maxie", -1));
 +                put("key1", Pair.create("Chelsie", 33));
 +                put("key2", Pair.create((String) null, 43));
 +                put("key3", Pair.create("Shanna", 27));
 +                put("key4", Pair.create("Amiya", 36));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part1, forceFlush);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Set<String> rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key0", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(33)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        store.indexManager.invalidateAllIndexesBlocking();
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(33)));
 +        Assert.assertTrue(rows.toString(), rows.isEmpty());
 +
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key5", Pair.create("Americo", 20));
 +                put("key6", Pair.create("Fiona", 39));
 +                put("key7", Pair.create("Francis", 41));
 +                put("key8", Pair.create("Fred", 21));
 +                put("key9", Pair.create("Amely", 40));
 +                put("key14", Pair.create("Dino", 28));
 +        }};
 +
 +        loadData(part2, forceFlush);
 +
 +        rows = getIndexed(store, 10, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key6", "key7" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(age, Operator.EQ, Int32Type.instance.decompose(40)));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{ "key9" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testTruncate()
 +    {
 +        Map<String, Pair<String, Integer>> part1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key01", Pair.create("Ali", 33));
 +                put("key02", Pair.create("Jeremy", 41));
 +                put("key03", Pair.create("Elvera", 22));
 +                put("key04", Pair.create("Bailey", 45));
 +                put("key05", Pair.create("Emerson", 32));
 +                put("key06", Pair.create("Kadin", 38));
 +                put("key07", Pair.create("Maggie", 36));
 +                put("key08", Pair.create("Kailey", 36));
 +                put("key09", Pair.create("Armand", 21));
 +                put("key10", Pair.create("Arnold", 35));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key11", Pair.create("Ken", 38));
 +                put("key12", Pair.create("Penelope", 43));
 +                put("key13", Pair.create("Wyatt", 34));
 +                put("key14", Pair.create("Johnpaul", 34));
 +                put("key15", Pair.create("Trycia", 43));
 +                put("key16", Pair.create("Aida", 21));
 +                put("key17", Pair.create("Devon", 42));
 +        }};
 +
 +        Map<String, Pair<String, Integer>> part3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key18", Pair.create("Christina", 20));
 +                put("key19", Pair.create("Rick", 19));
 +                put("key20", Pair.create("Fannie", 22));
 +                put("key21", Pair.create("Keegan", 29));
 +                put("key22", Pair.create("Ignatius", 36));
 +                put("key23", Pair.create("Ellis", 26));
 +                put("key24", Pair.create("Annamarie", 29));
 +                put("key25", Pair.create("Tianna", 31));
 +                put("key26", Pair.create("Dennis", 32));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(part1, 1000, true);
 +
 +        loadData(part2, 2000, true);
 +        loadData(part3, 3000, true);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +
 +        Set<String> rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 16, rows.size());
 +
 +        // make sure we don't prematurely delete anything
 +        store.indexManager.truncateAllIndexesBlocking(500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 16, rows.size());
 +
 +        store.indexManager.truncateAllIndexesBlocking(1500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 10, rows.size());
 +
 +        store.indexManager.truncateAllIndexesBlocking(2500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 6, rows.size());
 +
 +        store.indexManager.truncateAllIndexesBlocking(3500);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 0, rows.size());
 +
 +        // add back in some data just to make sure it all still works
 +        Map<String, Pair<String, Integer>> part4 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key40", Pair.create("Tianna", 31));
 +                put("key41", Pair.create("Dennis", 32));
 +        }};
 +
 +        loadData(part4, 4000, true);
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertEquals(rows.toString(), 1, rows.size());
 +    }
 +
 +
 +    @Test
 +    public void testConcurrentMemtableReadsAndWrites() throws Exception
 +    {
 +        final ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        ExecutorService scheduler = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
 +
 +        final int writeCount = 10000;
 +        final AtomicInteger updates = new AtomicInteger(0);
 +
 +        for (int i = 0; i < writeCount; i++)
 +        {
 +            final String key = "key" + i;
 +            final String firstName = "first_name#" + i;
 +            final String lastName = "last_name#" + i;
 +
 +            scheduler.submit((Runnable) () -> {
 +                try
 +                {
 +                    newMutation(key, firstName, lastName, 26, System.currentTimeMillis()).apply();
 +                    Uninterruptibles.sleepUninterruptibly(5, TimeUnit.MILLISECONDS); // back up a bit to do more reads
 +                }
 +                finally
 +                {
 +                    updates.incrementAndGet();
 +                }
 +            });
 +        }
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        int previousCount = 0;
 +
 +        do
 +        {
 +            // this loop figures out if number of search results monotonically increasing
 +            // to make sure that concurrent updates don't interfere with reads, uses first_name and age
 +            // indexes to test correctness of both Trie and SkipList ColumnIndex implementations.
 +
 +            Set<DecoratedKey> rows = getPaged(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                                          buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
 +
 +            Assert.assertTrue(previousCount <= rows.size());
 +            previousCount = rows.size();
 +        }
 +        while (updates.get() < writeCount);
 +
 +        // to make sure that after all of the right are done we can read all "count" worth of rows
 +        Set<DecoratedKey> rows = getPaged(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(26)));
 +
 +        Assert.assertEquals(writeCount, rows.size());
 +    }
 +
 +    @Test
 +    public void testSameKeyInMemtableAndSSTables()
 +    {
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Map<String, Pair<String, Integer>> data1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 26));
 +                put("key3", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data1, true);
 +
 +        Map<String, Pair<String, Integer>> data2 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 28));
 +        }};
 +
 +        loadData(data2, true);
 +
 +        Map<String, Pair<String, Integer>> data3 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 15));
 +                put("key4", Pair.create("Jason", 29));
 +        }};
 +
 +        loadData(data3, false);
 +
 +        Set<String> rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(15)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(29)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")),
 +                                      buildExpression(age, Operator.EQ, Int32Type.instance.decompose(27)));
 +
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[]{"key2", "key3"}, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testInsertingIncorrectValuesIntoAgeIndex()
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +        final ByteBuffer age = UTF8Type.instance.decompose("age");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key1")));
 +        update(rm, new ArrayList<Cell>()
 +        {{
 +            add(buildCell(age, LongType.instance.decompose(26L), System.currentTimeMillis()));
 +            add(buildCell(firstName, AsciiType.instance.decompose("pavel"), System.currentTimeMillis()));
 +        }});
 +        rm.apply();
 +
 +        store.forceBlockingFlush();
 +
 +        Set<String> rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")),
 +                                                 buildExpression(age, Operator.GTE, Int32Type.instance.decompose(26)));
 +
 +        // index is expected to have 0 results because age value was of wrong type
 +        Assert.assertEquals(0, rows.size());
 +    }
 +
 +
 +    @Test
 +    public void testUnicodeSupport()
 +    {
 +        testUnicodeSupport(false);
 +        cleanupData();
 +        testUnicodeSupport(true);
 +    }
 +
 +    private void testUnicodeSupport(boolean forceFlush)
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("comment");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, comment, UTF8Type.instance.decompose("ⓈⓅⒺⒸⒾⒶⓁ ⒞⒣⒜⒭⒮ and normal ones"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, comment, UTF8Type.instance.decompose("龍馭鬱"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, comment, UTF8Type.instance.decompose("インディアナ"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, comment, UTF8Type.instance.decompose("レストラン"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key5"));
 +        update(rm, comment, UTF8Type.instance.decompose("ベンジャミン ウエスト"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ⓈⓅⒺⒸⒾ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("normal")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ベンジャミン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key5" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("レストラ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("インディ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ベンジャミ")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key5" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4", "key5" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("レストラン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testUnicodeSuffixModeNoSplits()
 +    {
 +        testUnicodeSuffixModeNoSplits(false);
 +        cleanupData();
 +        testUnicodeSuffixModeNoSplits(true);
 +    }
 +
 +    private void testUnicodeSuffixModeNoSplits(boolean forceFlush)
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("comment_suffix_split");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, comment, UTF8Type.instance.decompose("龍馭鬱"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, comment, UTF8Type.instance.decompose("インディアナ"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, comment, UTF8Type.instance.decompose("レストラン"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, comment, UTF8Type.instance.decompose("ベンジャミン ウエスト"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("龍馭鬱")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ベンジャミン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("トラン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ディア")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ジャミン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("ン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_SUFFIX, UTF8Type.instance.decompose("ン")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("ベンジャミン ウエスト")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key4" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testThatTooBigValueIsRejected()
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("comment_suffix_split");
 +
 +        for (int i = 0; i < 10; i++)
 +        {
 +            byte[] randomBytes = new byte[ThreadLocalRandom.current().nextInt(OnDiskIndexBuilder.MAX_TERM_SIZE, 5 * OnDiskIndexBuilder.MAX_TERM_SIZE)];
 +            ThreadLocalRandom.current().nextBytes(randomBytes);
 +
 +            final ByteBuffer bigValue = UTF8Type.instance.decompose(new String(randomBytes));
 +
 +            Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +            update(rm, comment, bigValue, System.currentTimeMillis());
 +            rm.apply();
 +
 +            Set<String> rows;
 +
 +            rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, bigValue.duplicate()));
 +            Assert.assertEquals(0, rows.size());
 +
 +            store.forceBlockingFlush();
 +
 +            rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, bigValue.duplicate()));
 +            Assert.assertEquals(0, rows.size());
 +        }
 +    }
 +
 +    @Test
 +    public void testSearchTimeouts() throws Exception
 +    {
 +        final ByteBuffer firstName = UTF8Type.instance.decompose("first_name");
 +
 +        Map<String, Pair<String, Integer>> data1 = new HashMap<String, Pair<String, Integer>>()
 +        {{
 +                put("key1", Pair.create("Pavel", 14));
 +                put("key2", Pair.create("Pavel", 26));
 +                put("key3", Pair.create("Pavel", 27));
 +                put("key4", Pair.create("Jason", 27));
 +        }};
 +
 +        ColumnFamilyStore store = loadData(data1, true);
 +
 +        RowFilter filter = RowFilter.create();
 +        filter.add(store.metadata.getColumnDefinition(firstName), Operator.LIKE_CONTAINS, AsciiType.instance.fromString("a"));
 +
-         ReadCommand command = new PartitionRangeReadCommand(store.metadata,
-                                                             FBUtilities.nowInSeconds(),
-                                                             ColumnFilter.all(store.metadata),
-                                                             filter,
-                                                             DataLimits.NONE,
-                                                             DataRange.allData(store.metadata.partitioner),
-                                                             Optional.empty());
- 
++        ReadCommand command =
++            PartitionRangeReadCommand.create(false,
++                                             store.metadata,
++                                             FBUtilities.nowInSeconds(),
++                                             ColumnFilter.all(store.metadata),
++                                             filter,
++                                             DataLimits.NONE,
++                                             DataRange.allData(store.metadata.partitioner));
 +        try
 +        {
 +            new QueryPlan(store, command, 0).execute(ReadExecutionController.empty());
 +            Assert.fail();
 +        }
 +        catch (TimeQuotaExceededException e)
 +        {
 +            // correct behavior
 +        }
 +        catch (Exception e)
 +        {
 +            Assert.fail();
 +            e.printStackTrace();
 +        }
 +
 +        // to make sure that query doesn't fail in normal conditions
 +
 +        try (ReadExecutionController controller = command.executionController())
 +        {
 +            Set<String> rows = getKeys(new QueryPlan(store, command, DatabaseDescriptor.getRangeRpcTimeout()).execute(controller));
 +            Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +        }
 +    }
 +
 +    @Test
 +    public void testLowerCaseAnalyzer()
 +    {
 +        testLowerCaseAnalyzer(false);
 +        cleanupData();
 +        testLowerCaseAnalyzer(true);
 +    }
 +
 +    @Test
 +    public void testChinesePrefixSearch()
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer fullName = UTF8Type.instance.decompose("/output/full-name/");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, fullName, UTF8Type.instance.decompose("美加 八田"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, fullName, UTF8Type.instance.decompose("仁美 瀧澤"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, fullName, UTF8Type.instance.decompose("晃宏 高須"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, fullName, UTF8Type.instance.decompose("弘孝 大竹"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key5"));
 +        update(rm, fullName, UTF8Type.instance.decompose("満枝 榎本"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key6"));
 +        update(rm, fullName, UTF8Type.instance.decompose("飛鳥 上原"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key7"));
 +        update(rm, fullName, UTF8Type.instance.decompose("大輝 鎌田"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key8"));
 +        update(rm, fullName, UTF8Type.instance.decompose("利久 寺地"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        store.forceBlockingFlush();
 +
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.EQ, UTF8Type.instance.decompose("美加 八田")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("美加")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.EQ, UTF8Type.instance.decompose("晃宏 高須")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(fullName, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("大輝")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key7" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    public void testLowerCaseAnalyzer(boolean forceFlush)
 +    {
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer comment = UTF8Type.instance.decompose("address");
 +
 +        Mutation rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, comment, UTF8Type.instance.decompose("577 Rogahn Valleys Apt. 178"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, comment, UTF8Type.instance.decompose("89809 Beverly Course Suite 089"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, comment, UTF8Type.instance.decompose("165 clydie oval apt. 399"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        if (forceFlush)
 +            store.forceBlockingFlush();
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 Rogahn Valleys")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 ROgAhn VallEYs")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 rogahn valleys")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("577 rogahn")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("57")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 Beverly Course")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 BEVERly COURSE")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 beverly course")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("89809 Beverly")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("8980")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 ClYdie OvAl APT. 399")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 Clydie Oval Apt. 399")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 clydie oval apt. 399")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 ClYdie OvA")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165 ClYdi")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("165")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testPrefixSSTableLookup()
 +    {
 +        // This test coverts particular case which interval lookup can return invalid results
 +        // when queried on the prefix e.g. "j".
 +        ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 +
 +        final ByteBuffer name = UTF8Type.instance.decompose("first_name_prefix");
 +
 +        Mutation rm;
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key1"));
 +        update(rm, name, UTF8Type.instance.decompose("Pavel"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key2"));
 +        update(rm, name, UTF8Type.instance.decompose("Jordan"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key3"));
 +        update(rm, name, UTF8Type.instance.decompose("Mikhail"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key4"));
 +        update(rm, name, UTF8Type.instance.decompose("Michael"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key5"));
 +        update(rm, name, UTF8Type.instance.decompose("Johnny"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        // first flush would make interval for name - 'johnny' -> 'pavel'
 +        store.forceBlockingFlush();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key6"));
 +        update(rm, name, UTF8Type.instance.decompose("Jason"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key7"));
 +        update(rm, name, UTF8Type.instance.decompose("Vijay"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        rm = new Mutation(KS_NAME, decoratedKey("key8")); // this name is going to be tokenized
 +        update(rm, name, UTF8Type.instance.decompose("Jean-Claude"), System.currentTimeMillis());
 +        rm.apply();
 +
 +        // this flush is going to produce range - 'jason' -> 'vijay'
 +        store.forceBlockingFlush();
 +
 +        // make sure that overlap of the prefixes is properly handled across sstables
 +        // since simple interval tree lookup is not going to cover it, prefix lookup actually required.
 +
 +        Set<String> rows;
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("J")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key5", "key6", "key8"}, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("j")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key5", "key6", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("m")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key3", "key4" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("v")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key7" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("p")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_PREFIX, UTF8Type.instance.decompose("j")),
 +                                     buildExpression(name, Operator.NEQ, UTF8Type.instance.decompose("joh")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key2", "key6", "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("pavel")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Pave")));
 +        Assert.assertTrue(rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Pavel")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("JeAn")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("claUde")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key8" }, rows.toArray(new String[rows.size()])));
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Jean")));
 +        Assert.assertTrue(rows.isEmpty());
 +
 +        rows = getIndexed(store, 10, buildExpression(name, Operator.EQ, UTF8Type.instance.decompose("Jean-Claude")));
 +        Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key8" }, rows.toArray(new String[rows.size()])));
 +    }
 +
 +    @Test
 +    public void testSettingIsLiteralOption()
 +    {
 +
 +        // special type which is UTF-8 but is only on the inside
 +        AbstractType<?> stringType = new AbstractType<String>(AbstractType.ComparisonType.CUSTOM)
 +        {
 +            public ByteBuffer fromString(String source) throws MarshalException
 +            {
 +                return UTF8Type.instance.fromString(source);
 +            }
 +
 +            public Term fromJSONObject(Object parsed) throws MarshalException
 +            {
 +                throw new UnsupportedOperationException();
 +            }
 +
 +            public TypeSerializer<String> getSerializer()
 +            {
 +                return UTF8Type.instance.getSerializer();
 +            }
 +
 +            public int compareCustom(ByteBuffer a, ByteBuffer b)
 +            {
 +                return UTF8Type.instance.compare(a, b);
 +            }
 +        };
 +
 +        // first let's check that we get 'false' for 'isLiteral' if we don't set the option with special comparator
 +        ColumnDefinition columnA = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-A", stringType);
 +
 +        ColumnIndex indexA = new ColumnIndex(UTF8Type.instance, columnA, IndexMetadata.fromSchemaMetadata("special-index-A", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(true,  indexA.isIndexed());
 +        Assert.assertEquals(false, indexA.isLiteral());
 +
 +        // now let's double-check that we do get 'true' when we set it
 +        ColumnDefinition columnB = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-B", stringType);
 +
 +        ColumnIndex indexB = new ColumnIndex(UTF8Type.instance, columnB, IndexMetadata.fromSchemaMetadata("special-index-B", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +            put("is_literal", "true");
 +        }}));
 +
 +        Assert.assertEquals(true, indexB.isIndexed());
 +        Assert.assertEquals(true, indexB.isLiteral());
 +
 +        // and finally we should also get a 'true' if it's built-in UTF-8/ASCII comparator
 +        ColumnDefinition columnC = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-C", UTF8Type.instance);
 +
 +        ColumnIndex indexC = new ColumnIndex(UTF8Type.instance, columnC, IndexMetadata.fromSchemaMetadata("special-index-C", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(true, indexC.isIndexed());
 +        Assert.assertEquals(true, indexC.isLiteral());
 +
 +        ColumnDefinition columnD = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-D", AsciiType.instance);
 +
 +        ColumnIndex indexD = new ColumnIndex(UTF8Type.instance, columnD, IndexMetadata.fromSchemaMetadata("special-index-D", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(true, indexD.isIndexed());
 +        Assert.assertEquals(true, indexD.isLiteral());
 +
 +        // and option should supersedes the comparator type
 +        ColumnDefinition columnE = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-E", UTF8Type.instance);
 +
 +        ColumnIndex indexE = new ColumnIndex(UTF8Type.instance, columnE, IndexMetadata.fromSchemaMetadata("special-index-E", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +            put("is_literal", "false");
 +        }}));
 +
 +        Assert.assertEquals(true,  indexE.isIndexed());
 +        Assert.assertEquals(false, indexE.isLiteral());
 +
 +        // test frozen-collection
 +        ColumnDefinition columnF = ColumnDefinition.regularDef(KS_NAME,
 +                                                               CF_NAME,
 +                                                               "special-F",
 +                                                               ListType.getInstance(UTF8Type.instance, false));
 +
 +        ColumnIndex indexF = new ColumnIndex(UTF8Type.instance, columnF, IndexMetadata.fromSchemaMetadata("special-index-F", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
 +        {{
 +            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
 +        }}));
 +
 +        Assert.assertEquals(

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org