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 2012/01/27 16:03:39 UTC

git commit: Avoids over-paging during get_count

Updated Branches:
  refs/heads/trunk 006137294 -> a5b3abb66


Avoids over-paging during get_count

patch by slebresne; reviewed by jbellis for CASSANDRA-3798


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

Branch: refs/heads/trunk
Commit: a5b3abb667928e231b9dc9cfcb9184024f23f8d5
Parents: 0061372
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Jan 27 16:02:43 2012 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Jan 27 16:02:43 2012 +0100

----------------------------------------------------------------------
 CHANGES.txt                                        |    1 +
 .../apache/cassandra/thrift/CassandraServer.java   |   37 ++++++++-------
 2 files changed, 21 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a5b3abb6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b1f3061..8f62ab3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -59,6 +59,7 @@
  * Add initial code for CQL 3.0-beta (CASSANDRA-3781, 3753)
  * Add wide row support for ColumnFamilyInputFormat (CASSANDRA-3264)
  * Allow extending CompositeType comparator (CASSANDRA-3657)
+ * Avoids over-paging during get_count (CASSANDRA-3798)
 
 
 1.0.8

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a5b3abb6/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 8aacea1..03e6728 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -434,8 +434,9 @@ public class CassandraServer implements Cassandra.Iface
                                                    false,
                                                    Integer.MAX_VALUE);
         }
-        
+
         int requestedCount = predicate.slice_range.count;
+        int pages = 0;
         while (true)
         {
             predicate.slice_range.count = Math.min(pageSize, requestedCount);
@@ -443,29 +444,31 @@ public class CassandraServer implements Cassandra.Iface
             if (columns.isEmpty())
                 break;
 
-            totalCount += columns.size();
-            requestedCount -= columns.size();
-            ColumnOrSuperColumn lastColumn = columns.get(columns.size() - 1);
-            ByteBuffer lastName =
-                    lastColumn.isSetSuper_column() ? lastColumn.super_column.name :
-                        (lastColumn.isSetColumn() ? lastColumn.column.name :
-                            (lastColumn.isSetCounter_column() ? lastColumn.counter_column.name : lastColumn.counter_super_column.name));
-            if ((requestedCount == 0) || ((columns.size() == 1) && (lastName.equals(predicate.slice_range.start))))
-            {
+            ColumnOrSuperColumn firstColumn = columns.get(columns.size() - 1);
+            ByteBuffer firstName = getName(columns.get(0));
+            int newColumns = pages == 0 || !firstName.equals(predicate.slice_range.start) ? columns.size() : columns.size() - 1;
+            totalCount += newColumns;
+            requestedCount -= newColumns;
+            pages++;
+            // We're done if either:
+            //   - We've querying the number of columns requested by the user
+            //   - The last page wasn't full
+            if (requestedCount == 0 || columns.size() < predicate.slice_range.count)
                 break;
-            }
             else
-            {
-                predicate.slice_range.start = lastName;
-                // remove the count for the column that starts the next slice
-                totalCount--;
-                requestedCount++;
-            }
+                predicate.slice_range.start = getName(columns.get(columns.size() - 1));
         }
 
         return totalCount;
     }
 
+    private static ByteBuffer getName(ColumnOrSuperColumn cosc)
+    {
+        return cosc.isSetSuper_column() ? cosc.super_column.name :
+                   (cosc.isSetColumn() ? cosc.column.name :
+                       (cosc.isSetCounter_column() ? cosc.counter_column.name : cosc.counter_super_column.name));
+    }
+
     public Map<ByteBuffer, Integer> multiget_count(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {