Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 861B81872D for ; Wed, 5 Aug 2015 11:59:24 +0000 (UTC) Received: (qmail 84760 invoked by uid 500); 5 Aug 2015 11:59:18 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 84714 invoked by uid 500); 5 Aug 2015 11:59:18 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 84680 invoked by uid 99); 5 Aug 2015 11:59:18 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 05 Aug 2015 11:59:18 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 0550EDFC13; Wed, 5 Aug 2015 11:59:18 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: blerer@apache.org To: commits@cassandra.apache.org Date: Wed, 05 Aug 2015 11:59:18 -0000 Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: [1/4] cassandra git commit: Fix paging with static Repository: cassandra Updated Branches: refs/heads/trunk 3ab86d043 -> de49ed84c Fix paging with static patch by Sylvain Lebresne; reviewed by Benjamin Lerer for CASSANDRA-9775 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6aa83990 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6aa83990 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6aa83990 Branch: refs/heads/trunk Commit: 6aa83990530dbfe5e8a2c3a194c4dcbb3ffd4b59 Parents: e58b7df Author: Sylvain Lebresne Authored: Wed Aug 5 12:14:26 2015 +0200 Committer: blerer Committed: Wed Aug 5 12:18:12 2015 +0200 ---------------------------------------------------------------------- .../service/pager/AbstractQueryPager.java | 54 ++++++++++++++++---- .../service/pager/RangeNamesQueryPager.java | 6 +++ .../service/pager/RangeSliceQueryPager.java | 6 +++ .../service/pager/SinglePartitionPager.java | 6 +++ 4 files changed, 61 insertions(+), 11 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6aa83990/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java b/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java index 2c16ace..9991277 100644 --- a/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java +++ b/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java @@ -17,6 +17,8 @@ */ package org.apache.cassandra.service.pager; +import java.util.NoSuchElementException; + import org.apache.cassandra.config.CFMetaData; import org.apache.cassandra.db.*; import org.apache.cassandra.db.rows.*; @@ -79,6 +81,9 @@ abstract class AbstractQueryPager implements QueryPager private Row lastRow; + private boolean isFirstPartition = true; + private RowIterator nextPartition; + private PagerIterator(PartitionIterator iter, DataLimits pageLimits, int nowInSec) { super(iter, pageLimits, nowInSec); @@ -86,30 +91,56 @@ abstract class AbstractQueryPager implements QueryPager } @Override - @SuppressWarnings("resource") // iter is closed by closing the result - public RowIterator next() + @SuppressWarnings("resource") // iter is closed by closing the result or in close() + public boolean hasNext() { - RowIterator iter = super.next(); - try + while (nextPartition == null && super.hasNext()) { - DecoratedKey key = iter.partitionKey(); + if (nextPartition == null) + nextPartition = super.next(); + + DecoratedKey key = nextPartition.partitionKey(); if (lastKey == null || !lastKey.equals(key)) remainingInPartition = limits.perPartitionCount(); lastKey = key; - return new RowPagerIterator(iter); - } - catch (RuntimeException e) - { - iter.close(); - throw e; + + // If this is the first partition of this page, this could be the continuation of a partition we've started + // on the previous page. In which case, we could have the problem that the partition has no more "regular" + // rows (but the page size is such we didn't knew before) but it does has a static row. We should then skip + // the partition as returning it would means to the upper layer that the partition has "only" static columns, + // which is not the case (and we know the static results have been sent on the previous page). + if (isFirstPartition && isPreviouslyReturnedPartition(key) && !nextPartition.hasNext()) + { + nextPartition.close(); + nextPartition = null; + } + + isFirstPartition = false; } + return nextPartition != null; + } + + @Override + @SuppressWarnings("resource") // iter is closed by closing the result + public RowIterator next() + { + if (!hasNext()) + throw new NoSuchElementException(); + + RowIterator toReturn = nextPartition; + nextPartition = null; + + return new RowPagerIterator(toReturn); } @Override public void close() { super.close(); + if (nextPartition != null) + nextPartition.close(); + recordLast(lastKey, lastRow); int counted = counter.counted(); @@ -158,4 +189,5 @@ abstract class AbstractQueryPager implements QueryPager protected abstract ReadCommand nextPageReadCommand(int pageSize); protected abstract void recordLast(DecoratedKey key, Row row); + protected abstract boolean isPreviouslyReturnedPartition(DecoratedKey key); } http://git-wip-us.apache.org/repos/asf/cassandra/blob/6aa83990/src/java/org/apache/cassandra/service/pager/RangeNamesQueryPager.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/pager/RangeNamesQueryPager.java b/src/java/org/apache/cassandra/service/pager/RangeNamesQueryPager.java index 4171694..e085490 100644 --- a/src/java/org/apache/cassandra/service/pager/RangeNamesQueryPager.java +++ b/src/java/org/apache/cassandra/service/pager/RangeNamesQueryPager.java @@ -69,6 +69,12 @@ public class RangeNamesQueryPager extends AbstractQueryPager lastReturnedKey = key; } + protected boolean isPreviouslyReturnedPartition(DecoratedKey key) + { + // Note that lastReturnedKey can be null, but key cannot. + return key.equals(lastReturnedKey); + } + private AbstractBounds makeExcludingKeyBounds(PartitionPosition lastReturnedKey) { // We return a range that always exclude lastReturnedKey, since we've already http://git-wip-us.apache.org/repos/asf/cassandra/blob/6aa83990/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java b/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java index 0139d9c..169944b 100644 --- a/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java +++ b/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java @@ -99,6 +99,12 @@ public class RangeSliceQueryPager extends AbstractQueryPager } } + protected boolean isPreviouslyReturnedPartition(DecoratedKey key) + { + // Note that lastReturnedKey can be null, but key cannot. + return key.equals(lastReturnedKey); + } + private AbstractBounds makeKeyBounds(PartitionPosition lastReturnedKey, boolean includeLastKey) { AbstractBounds bounds = ((PartitionRangeReadCommand)command).dataRange().keyRange(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/6aa83990/src/java/org/apache/cassandra/service/pager/SinglePartitionPager.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/pager/SinglePartitionPager.java b/src/java/org/apache/cassandra/service/pager/SinglePartitionPager.java index 223c3fd..bb223b8 100644 --- a/src/java/org/apache/cassandra/service/pager/SinglePartitionPager.java +++ b/src/java/org/apache/cassandra/service/pager/SinglePartitionPager.java @@ -82,4 +82,10 @@ public class SinglePartitionPager extends AbstractQueryPager if (last != null) lastReturned = last.clustering(); } + + protected boolean isPreviouslyReturnedPartition(DecoratedKey key) + { + // We're querying a single partition, so if it's not the first page, it is the previously returned one. + return lastReturned != null; + } }