You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2015/11/06 11:54:24 UTC

cassandra git commit: Skip sstable base on clustering in names query

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 925336031 -> 1e4d44e18


Skip sstable base on clustering in names query

patch by slebresne; reviewed by iamaleksey for CASSANDRA-10571


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

Branch: refs/heads/cassandra-3.0
Commit: 1e4d44e18e4cc1fd3f7e4eaf00c89ebc798e142c
Parents: 9253360
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Oct 23 12:20:28 2015 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Nov 6 11:53:32 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/SinglePartitionReadCommand.java          | 35 +++++++++++++++++++-
 .../db/filter/ClusteringIndexNamesFilter.java   | 14 ++++++--
 3 files changed, 47 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e4d44e1/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 98c98ab..bb8b4d2 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0
+ * Skip sstable on clustering in names query (CASSANDRA-10571)
  * Remove value skipping as it breaks read-repair (CASSANDRA-10655)
  * Fix bootstrapping with MVs (CASSANDRA-10621)
  * Make sure EACH_QUORUM reads are using NTS (CASSANDRA-10584)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e4d44e1/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 4d7d93c..4d410a0 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -527,7 +527,7 @@ public class SinglePartitionReadCommand extends ReadCommand
                 if (sstable.getMaxTimestamp() < mostRecentPartitionTombstone)
                     break;
 
-                if (!filter.shouldInclude(sstable))
+                if (!shouldInclude(sstable))
                 {
                     nonIntersectingSSTables++;
                     // sstable contains no tombstone if maxLocalDeletionTime == Integer.MAX_VALUE, so we can safely skip those entirely
@@ -612,6 +612,17 @@ public class SinglePartitionReadCommand extends ReadCommand
         }
     }
 
+    private boolean shouldInclude(SSTableReader sstable)
+    {
+        // If some static columns are queried, we should always include the sstable: the clustering values stats of the sstable
+        // don't tell us if the sstable contains static values in particular.
+        // TODO: we could record if a sstable contains any static value at all.
+        if (!columnFilter().fetchedColumns().statics.isEmpty())
+            return true;
+
+        return clusteringIndexFilter().shouldInclude(sstable);
+    }
+
     private boolean queryNeitherCountersNorCollections()
     {
         for (ColumnDefinition column : columnFilter().fetchedColumns())
@@ -675,6 +686,28 @@ public class SinglePartitionReadCommand extends ReadCommand
             if (filter == null)
                 break;
 
+            if (!shouldInclude(sstable))
+            {
+                // This mean that nothing queried by the filter can be in the sstable. One exception is the top-level partition deletion
+                // however: if it is set, it impacts everything and must be included. Getting that top-level partition deletion costs us
+                // some seek in general however (unless the partition is indexed and is in the key cache), so we first check if the sstable
+                // has any tombstone at all as a shortcut.
+                if (sstable.getSSTableMetadata().maxLocalDeletionTime == Integer.MAX_VALUE)
+                    continue; // Means no tombstone at all, we can skip that sstable
+
+                // We need to get the partition deletion and include it if it's live. In any case though, we're done with that sstable.
+                sstable.incrementReadCount();
+                try (UnfilteredRowIterator iter = sstable.iterator(partitionKey(), columnFilter(), filter.isReversed(), isForThrift()))
+                {
+                    if (iter.partitionLevelDeletion().isLive())
+                    {
+                        sstablesIterated++;
+                        result = add(UnfilteredRowIterators.noRowsIterator(iter.metadata(), iter.partitionKey(), Rows.EMPTY_STATIC_ROW, iter.partitionLevelDeletion(), filter.isReversed()), result, filter, sstable.isRepaired());
+                    }
+                }
+                continue;
+            }
+
             Tracing.trace("Merging data from sstable {}", sstable.descriptor.generation);
             sstable.incrementReadCount();
             try (UnfilteredRowIterator iter = filter.filter(sstable.iterator(partitionKey(), columnFilter(), filter.isReversed(), isForThrift())))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e4d44e1/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
index 388cd50..a81a7a6 100644
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.db.filter;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.*;
 
 import org.apache.cassandra.config.CFMetaData;
@@ -201,8 +202,17 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
 
     public boolean shouldInclude(SSTableReader sstable)
     {
-        // TODO: we could actually exclude some sstables
-        return true;
+        ClusteringComparator comparator = sstable.metadata.comparator;
+        List<ByteBuffer> minClusteringValues = sstable.getSSTableMetadata().minClusteringValues;
+        List<ByteBuffer> maxClusteringValues = sstable.getSSTableMetadata().maxClusteringValues;
+
+        // If any of the requested clustering is within the bounds covered by the sstable, we need to include the sstable
+        for (Clustering clustering : clusterings)
+        {
+            if (Slice.make(clustering).intersects(comparator, minClusteringValues, maxClusteringValues))
+                return true;
+        }
+        return false;
     }
 
     public String toString(CFMetaData metadata)