[ https://issues.apache.org/jira/browse/CASSANDRA-13911?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17779034#comment-17779034 ]
Szymon Miezal commented on CASSANDRA-13911: ------------------------------------------- [~aleksey] The reason I have asked is that I have found that for {{SELECT DISTINCT queries SRP kicks in even though all of the rows seem to be easily fitting into the requested }}LIMIT. The test that replicates the scenario I am referring to https://github.com/szymon-miezal/cassandra-dtest/commit/56c0b1a7d01550b140d07503368f69b254431c76 {code:java} 127.0.0.1 0:00:00.003267 Parsing SELECT DISTINCT pk FROM test.test Native-Transport-Requests-1 127.0.0.1 0:00:00.004578 Preparing statement Native-Transport-Requests-1 127.0.0.1 0:00:00.015112 Computing ranges to query Native-Transport-Requests-1 127.0.0.1 0:00:00.015321 Submitting range requests on 2 ranges with a concurrency of 1 (0.0 rows per range expected) Native-Transport-Requests-1 127.0.0.1 0:00:00.016695 Enqueuing request to /127.0.0.1 Native-Transport-Requests-1 127.0.0.1 0:00:00.016948 Executing single-partition query on peers Native-Transport-Requests-1 127.0.0.1 0:00:00.017021 Acquiring sstable references Native-Transport-Requests-1 127.0.0.1 0:00:00.017086 Merging memtable contents Native-Transport-Requests-1 127.0.0.1 0:00:00.017262 Read 0 live rows and 0 tombstone cells Native-Transport-Requests-1 127.0.0.1 0:00:00.018357 Enqueuing request to /127.0.0.2 Native-Transport-Requests-1 127.0.0.1 0:00:00.018547 Submitted 1 concurrent range requests Native-Transport-Requests-1 127.0.0.1 0:00:00.018663 Sending RANGE_SLICE message to /127.0.0.2 MessagingService-Outgoing-/127.0.0.2-Small 127.0.0.1 0:00:00.020153 Sending RANGE_SLICE message to /127.0.0.1 MessagingService-Outgoing-/127.0.0.1-Small 127.0.0.1 0:00:00.020760 RANGE_SLICE message received from /127.0.0.1 MessagingService-Incoming-/127.0.0.1 127.0.0.1 0:00:00.021036 Executing seq scan across 0 sstables for (min(-9223372036854775808), min(-9223372036854775808)] ReadStage-3 127.0.0.1 0:00:00.021805 Read 10 live rows and 0 tombstone cells ReadStage-3 127.0.0.2 0:00:00.001305 RANGE_SLICE message received from /127.0.0.1 MessagingService-Incoming-/127.0.0.2 127.0.0.1 0:00:00.021972 Enqueuing response to /127.0.0.1 ReadStage-3 127.0.0.1 0:00:00.022209 Sending REQUEST_RESPONSE message to /127.0.0.1 MessagingService-Outgoing-/127.0.0.1-Small 127.0.0.1 0:00:00.022540 REQUEST_RESPONSE message received from /127.0.0.1 MessagingService-Incoming-/127.0.0.1 127.0.0.1 0:00:00.022798 Processing response from /127.0.0.1 RequestResponseStage-3 127.0.0.2 0:00:00.003844 Executing seq scan across 0 sstables for (min(-9223372036854775808), min(-9223372036854775808)] ReadStage-2 127.0.0.2 0:00:00.004773 Read 10 live rows and 0 tombstone cells ReadStage-2 127.0.0.2 0:00:00.004917 Enqueuing response to /127.0.0.1 ReadStage-2 127.0.0.2 0:00:00.005219 Sending REQUEST_RESPONSE message to /127.0.0.1 MessagingService-Outgoing-/127.0.0.1-Small 127.0.0.1 0:00:00.026714 REQUEST_RESPONSE message received from /127.0.0.2 MessagingService-Incoming-/127.0.0.1 127.0.0.1 0:00:00.027008 Processing response from /127.0.0.2 RequestResponseStage-2 127.0.0.1 0:00:00.040391 Requesting 4990 extra rows from /127.0.0.2 for short read protection Native-Transport-Requests-1 127.0.0.1 0:00:00.040788 Sending RANGE_SLICE message to /127.0.0.2 MessagingService-Outgoing-/127.0.0.2-Small 127.0.0.2 0:00:00.000013 RANGE_SLICE message received from /127.0.0.1 MessagingService-Incoming-/127.0.0.2 127.0.0.2 0:00:00.000344 Executing seq scan across 0 sstables for (random_key_8, min(-9223372036854775808)] ReadStage-4 127.0.0.2 0:00:00.000567 Read 0 live rows and 0 tombstone cells ReadStage-4 127.0.0.2 0:00:00.000684 Enqueuing response to /127.0.0.1 ReadStage-4 127.0.0.1 0:00:00.042362 REQUEST_RESPONSE message received from /127.0.0.2 MessagingService-Incoming-/127.0.0.1 127.0.0.2 0:00:00.000872 Sending REQUEST_RESPONSE message to /127.0.0.1 MessagingService-Outgoing-/127.0.0.1-Small 127.0.0.1 0:00:00.042675 Processing response from /127.0.0.2 RequestResponseStage-5 127.0.0.1 0:00:00.043017 Requesting 4990 extra rows from /127.0.0.1 for short read protection Native-Transport-Requests-1 127.0.0.1 0:00:00.043534 Executing seq scan across 0 sstables for (random_key_8, min(-9223372036854775808)] ReadStage-3 127.0.0.1 0:00:00.043742 Read 0 live rows and 0 tombstone cells ReadStage-3 {code} The above trace shows additional tow requests happening: {code:java} 127.0.0.1 0:00:00.040391 Requesting 4990 extra rows from /127.0.0.2 for short read protection Native-Transport-Requests-1 ... 127.0.0.1 0:00:00.043017 Requesting 4990 extra rows from /127.0.0.1 for short read protection Native-Transport-Requests-1 {code} therefore I am thinking whether [https://github.com/apache/cassandra/blob/cassandra-3.11/src/java/org/apache/cassandra/service/DataResolver.java#L749] is not too conservative. > 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