[prev in list] [next in list] [prev in thread] [next in thread] 

List:       cassandra-commits
Subject:    [jira] [Updated] (CASSANDRA-13911) IllegalStateException thrown by UPI.Serializer.hasNext() for some
From:       "Aleksey Yeschenko (JIRA)" <jira () apache ! org>
Date:       2017-09-30 10:48:03
Message-ID: JIRA.13105387.1506511049000.244848.1506768483081 () Atlassian ! JIRA
[Download RAW message or body]


     [ https://issues.apache.org/jira/browse/CASSANDRA-13911?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel \
]

Aleksey Yeschenko updated CASSANDRA-13911:
------------------------------------------
    Resolution: Fixed
        Status: Resolved  (was: Ready to Commit)

> 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-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[prev in list] [next in list] [prev in thread] [next in thread] 

Configure | About | News | Add a list | Sponsored by KoreLogic