[ https://issues.apache.org/jira/browse/CASSANDRA-13911?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17776257#comment-17776257 ]
Szymon Miezal commented on CASSANDRA-13911: ------------------------------------------- Does anyone remember what was the motivation for this particular condition change [https://github.com/apache/cassandra/commit/1efdf330e291a41cd8051e0c1195f75b5d352370#diff-212cd446e1b8c2aeb27818def0ba8c67370a1e00c5f657c85e42f0d43adfe05bR559?] I have found that [https://github.com/apache/cassandra-dtest/commit/51ad68ec45c7a40de1c51b31651632f2e87ceaa4] passes without it as well which is suspicious. The motivation for the questions is that I have found that in case of DISTINCT queries we are making an additional SRP call for every call of https://github.com/apache/cassandra/blob/cassandra-3.11/src/java/org/apache/cassandra/service/DataResolver.java#L729 due to exactly that condition. > 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: Legacy/Coordination > Reporter: Aleksey Yeschenko > Assignee: Aleksey Yeschenko > Priority: Normal > 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 (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org