[ https://issues.apache.org/jira/browse/CASSANDRA-14812?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16848873#comment-16848873 ]
mck edited comment on CASSANDRA-14812 at 5/27/19 12:00 PM: ----------------------------------------------------------- [~benedict], I have reviewed the patch and tested the python reproducible on 3.0.18 and 3.11.4, working with and failing without the patch applied. I'm not competent on this area, but I am jumping in to help as we too are seeing users unable to upgrade because of this fault. Review questions/points are: - is there a way to replicate the test for the CQL equivalent? While this bug does not impact CQL it is my understanding that CQL queries with `IN` clauses will still be going through this code path… I've attached the reproducible script rewritten for CQL, is it applicable? Should it be added as a dtest? (i don't think so but double-checking) - I understand overriding {{`filter(..)`}} for the NONE impl, although at first it is not intuitive that {{`DataLimits.NONE`}} is also used in thrift queries… - fyi the circleci results are here: https://circleci.com/workflow-run/3dd0d7f3-fa79-4118-80d8-247e85db40ea ; are these failures of concern? - {{"The branch I have uploaded also has a back port of CASSANDRA-14821"}}. I am confused… where is this? - a rebased commit for the 3.0 branch is here [mck/cassandra-3.0_14812|https://github.com/thelastpickle/cassandra/commits/mck/cassandra-3.0_14812] - the change in {{BasePartitions}} and the interactions from different {{StoppingTransformation}} subclasses is a bit harder to grok… It makes that the {{while}} loop does not need to continue in the situation where, {{stop}} has "leaked" and not been signalled, but where {{stopChild.isSignalled}} was. But not returning false in that same situation seems odd…? Do you want me to test the different cql interactions here (per partition, grouping, paging)? was (Author: michaelsembwever): [~benedict], I have reviewed the patch and tested the python reproducible on 3.0.18 and 3.11.4, working with and failing without the patch applied. I'm not competent on this area, but I am jumping in to help as we too are seeing users unable to upgrade because of this fault. Review questions/points are: - is there a way to replicate the test for the CQL equivalent? While this bug does not impact CQL it is my understanding that CQL queries with `IN` clauses will still be going through this code path… I've attached the reproducible script rewritten for CQL, is it applicable? Should it be added as a dtest? - I understand overriding {{`filter(..)`}} for the NONE impl, although at first it is not intuitive that {{`DataLimits.NONE`}} is also used in thrift queries… - fyi the circleci results are here: https://circleci.com/workflow-run/3dd0d7f3-fa79-4118-80d8-247e85db40ea ; are these failures of concern? - {{"The branch I have uploaded also has a back port of CASSANDRA-14821"}}. I am confused… where is this? - a rebased commit for the 3.0 branch is here [mck/cassandra-3.0_14812|https://github.com/thelastpickle/cassandra/commits/mck/cassandra-3.0_14812] - the change in {{BasePartitions}} and the interactions from different {{StoppingTransformation}} subclasses is a bit harder to grok… It makes that the {{while}} loop does not need to continue in the situation where, {{stop}} has "leaked" and not been signalled, but where {{stopChild.isSignalled}} was. But not returning false in that same situation seems odd…? Do you want me to test the different cql interactions here (per partition, grouping, paging)? > Multiget Thrift query returns null records after digest mismatch > ---------------------------------------------------------------- > > Key: CASSANDRA-14812 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14812 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Coordination, Messaging/Thrift > Reporter: Sivukhin Nikita > Assignee: Benedict > Priority: Urgent > Fix For: 3.0.x, 3.11.x > > Attachments: repro_script.py, requirements.txt, > small_repro_script.py, small_repro_script_cql.py > > > It seems that in Cassandra 3.0.0 a nasty bug was introduced in {{multiget}} > Thrift query processing logic. When one tries to read data from several > partitions with a single {{multiget}} query and {{DigestMismatch}} exception > is raised during this query processing, request coordinator prematurely > terminates response stream right at the point where the first > \{{DigestMismatch}} error is occurring. This leads to situation where clients > "do not see" some data contained in the database. > We managed to reproduce this bug in all versions of Cassandra starting with > v3.0.0. The pre-release version 3.0.0-rc2 works correctly. It looks like > [refactoring of iterator transformation > hierarchy|https://github.com/apache/cassandra/commit/609497471441273367013c09a1e0e1c990726ec7] > related to CASSANDRA-9975 triggers incorrect behaviour. > When concatenated iterator is returned from the > [StorageProxy.fetchRows(...)|https://github.com/apache/cassandra/blob/a05785d82c621c9cd04d8a064c38fd2012ef981c/src/java/org/apache/cassandra/service/StorageProxy.java#L1770], > Cassandra starts to consume this combined iterator. Because of > {{DigestMismatch}} exception some elements of this combined iterator contain > additional {{ThriftCounter}}, that was added during > [DataResolver.resolve(...)|https://github.com/apache/cassandra/blob/ee9e06b5a75c0be954694b191ea4170456015b98/src/java/org/apache/cassandra/service/reads/DataResolver.java#L120] > execution. While consuming iterator for many partitions Cassandra calls > [BaseIterator.tryGetMoreContents(...)|https://github.com/apache/cassandra/blob/a05785d82c621c9cd04d8a064c38fd2012ef981c/src/java/org/apache/cassandra/db/transform/BaseIterator.java#L115] > method that must switch from one partition iterator to another in case of > exhaustion of the former. In this case all Transformations contained in the > next iterator are applied to the combined BaseIterator that enumerates > partitions sequence which is wrong. This behaviour causes BaseIterator to > stop enumeration after it fully consumes partition with {{DigestMismatch}} > error, because this partition iterator has additional {{ThriftCounter}} data > limit. > The attachment contains the python2 script [^small_repro_script.py] that > reproduces this bug within 3-nodes ccmlib controlled cluster. Also, there is > an extended version of this script - [^repro_script.py] - that contains more > logging information and provides the ability to test behavior for many > Cassandra versions (to run all test cases from repro_script.py you can call > {{python -m unittest2 -v repro_script.ThriftMultigetTestCase}}). All the > necessary dependencies contained in the [^requirements.txt] > > This bug is critical in our production environment because we can't permit > any data skip. > Any ideas about a patch for this issue? -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org