Repository: cassandra
Updated Branches:
  refs/heads/trunk 26bde8dfe -> 2dfa90e2b


Fix thrift range filtering w/out 2ary index lookups

Patch by Sam Tunnicliffe; review by Tyler Hobbs for CASSANDRA-7741


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

Branch: refs/heads/trunk
Commit: 59a8e1c6b6bcb93ee63b2f309bf379b8a97cdffa
Parents: 5172860
Author: beobal <s...@beobal.com>
Authored: Tue Aug 12 13:57:52 2014 -0500
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Tue Aug 12 13:58:49 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/service/StorageProxy.java  | 44 ++++++++++++--------
 2 files changed, 28 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/59a8e1c6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index de45aeb..be91bbc 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.0-rc6
+ * Fix Thrift range filtering without 2ary index lookups (CASSANDRA-7741)
  * Add tracing entries about concurrent range requests (CASSANDRA-7599)
  * (cqlsh) Fix DESCRIBE for NTS keyspaces (CASSANDRA-7729)
  * Remove netty buffer ref-counting (CASSANDRA-7735)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/59a8e1c6/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java 
b/src/java/org/apache/cassandra/service/StorageProxy.java
index d963ee0..d37ac0a 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -1413,14 +1413,19 @@ public class StorageProxy implements StorageProxyMBean
         if (command.rowFilter != null && !command.rowFilter.isEmpty())
         {
             List<SecondaryIndexSearcher> searchers = 
cfs.indexManager.getIndexSearchersForQuery(command.rowFilter);
-            assert !searchers.isEmpty() : "Got row filter with no matching 
SecondaryIndexSearchers";
-
-            // Secondary index query (cql3 or otherwise).  Estimate result 
rows based on most selective 2ary index.
-            for (SecondaryIndexSearcher searcher : searchers)
+            if (searchers.isEmpty())
             {
-                // use our own mean column count as our estimate for how many 
matching rows each node will have
-                SecondaryIndex highestSelectivityIndex = 
searcher.highestSelectivityIndex(command.rowFilter);
-                resultRowsPerRange = Math.min(resultRowsPerRange, 
highestSelectivityIndex.estimateResultRows());
+                resultRowsPerRange = 
calculateResultRowsUsingEstimatedKeys(cfs);
+            }
+            else
+            {
+                // Secondary index query (cql3 or otherwise).  Estimate result 
rows based on most selective 2ary index.
+                for (SecondaryIndexSearcher searcher : searchers)
+                {
+                    // use our own mean column count as our estimate for how 
many matching rows each node will have
+                    SecondaryIndex highestSelectivityIndex = 
searcher.highestSelectivityIndex(command.rowFilter);
+                    resultRowsPerRange = Math.min(resultRowsPerRange, 
highestSelectivityIndex.estimateResultRows());
+                }
             }
         }
         else if (!command.countCQL3Rows())
@@ -1430,22 +1435,27 @@ public class StorageProxy implements StorageProxyMBean
         }
         else
         {
-            if (cfs.metadata.comparator.isDense())
-            {
-                // one storage row per result row, so use key estimate directly
-                resultRowsPerRange = cfs.estimateKeys();
-            }
-            else
-            {
-                float resultRowsPerStorageRow = ((float) cfs.getMeanColumns()) 
/ cfs.metadata.regularColumns().size();
-                resultRowsPerRange = resultRowsPerStorageRow * 
(cfs.estimateKeys());
-            }
+            resultRowsPerRange = calculateResultRowsUsingEstimatedKeys(cfs);
         }
 
         // adjust resultRowsPerRange by the number of tokens this node has and 
the replication factor for this ks
         return (resultRowsPerRange / DatabaseDescriptor.getNumTokens()) / 
keyspace.getReplicationStrategy().getReplicationFactor();
     }
 
+    private static float 
calculateResultRowsUsingEstimatedKeys(ColumnFamilyStore cfs)
+    {
+        if (cfs.metadata.comparator.isDense())
+        {
+            // one storage row per result row, so use key estimate directly
+            return cfs.estimateKeys();
+        }
+        else
+        {
+            float resultRowsPerStorageRow = ((float) cfs.getMeanColumns()) / 
cfs.metadata.regularColumns().size();
+            return resultRowsPerStorageRow * (cfs.estimateKeys());
+        }
+    }
+
     public static List<Row> getRangeSlice(AbstractRangeCommand command, 
ConsistencyLevel consistency_level)
     throws UnavailableException, ReadTimeoutException
     {

Reply via email to