[ 
https://issues.apache.org/jira/browse/CASSANDRA-13911?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16187022#comment-16187022
 ] 

Aleksey Yeschenko commented on CASSANDRA-13911:
-----------------------------------------------

Thanks.

[3.0 utests|https://circleci.com/gh/iamaleksey/cassandra/43] have the usual 
unrelated failures (mostly MV schema races). [3.11 
utests|https://circleci.com/gh/iamaleksey/cassandra/44] have the common 
{{CommitLogSegmentManagerTest}} failure, and [4.0 
utests|https://circleci.com/gh/iamaleksey/cassandra/45] have an unrelated 
{{StreamTransferTaskTest}} failure + the usual {{ViewFilteringTest}} timeouts. 
Basically baseline.

[3.0 
dtests|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/347/]
 only have the usual suspects (also we need to do something about {{Could not 
do a git clone}} issue already).  [3.11 
dtests|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/348/]
 and [4.0 
dtests|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/349/]
 are a bit more noisy but nothing related fails for a good reason. The 
flakiness and timeouts on jenkins are pretty bad overall though.

Committed as 
[1efdf330e291a41cd8051e0c1195f75b5d352370|https://github.com/apache/cassandra/commit/1efdf330e291a41cd8051e0c1195f75b5d352370]
 to 3.0 and merged with 3.11 and trunk. Dtest committed as 
[51ad68ec45c7a40de1c51b31651632f2e87ceaa4|https://github.com/apache/cassandra-dtest/commit/51ad68ec45c7a40de1c51b31651632f2e87ceaa4].

> IllegalStateException thrown by UPI.Serializer.hasNext() for some SELECT 
> queries
> --------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-13911
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13911
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Coordination
>            Reporter: Aleksey Yeschenko
>            Assignee: Aleksey Yeschenko
>             Fix For: 3.0.15, 3.11.1
>
>
> Certain combinations of rows, in presence of per partition limit (set 
> explicitly in 3.6+ or implicitly to 1 via DISTINCT) cause 
> {{UnfilteredPartitionIterators.Serializer.hasNext()}} to throw 
> {{IllegalStateException}} .
> Relevant code snippet:
> {code}
> // We can't answer this until the previously returned iterator has been fully 
> consumed,
> // so complain if that's not the case.
> if (next != null && next.hasNext())
>     throw new IllegalStateException("Cannot call hasNext() until the previous 
> iterator has been fully consumed");
> {code}
> Since {{UnfilteredPartitionIterators.Serializer}} and 
> {{UnfilteredRowIteratorSerializer.serializer}} deserialize partitions/rows 
> lazily, it is required for correct operation of the partition iterator to 
> have the previous partition fully consumed, so that deserializing the next 
> one can start from the correct position in the byte buffer. However, that 
> condition won’t always be satisfied, as there are legitimate combinations of 
> rows that do not consume every row in every partition.
> For example, look at [this 
> dtest|https://github.com/iamaleksey/cassandra-dtest/commits/13911].
> In case we end up with a following pattern of rows:
> {code}
> node1, partition 0 | 0
> node2, partition 0 |   x x
> {code}
> , where {{x}} and {{x}} a row tombstones for rows 1 and 2, it’s sufficient 
> for {{MergeIterator}} to only look at row 0 in partition from node1 and at 
> row tombstone 1 from node2 to satisfy the per partition limit of 1. The 
> stopping merge result counter will stop iteration right there, leaving row 
> tombstone 2 from node2 unvisited and not deseiralized. Switching to the next 
> partition will in turn trigger the {{IllegalStateException}} because we 
> aren’t done yet.
> The stopping counter is behaving correctly, so is the {{MergeIterator}}. I’ll 
> note that simply removing that condition is not enough to fix the problem 
> properly - it’d just cause us to deseiralize garbage, trying to deserialize a 
> new partition from a position in the bytebuffer that precedes remaining rows 
> in the previous partition.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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

Reply via email to