[
https://issues.apache.org/jira/browse/CASSANDRA-14812?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sivukhin Nikita updated CASSANDRA-14812:
----------------------------------------
Description:
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?
was:
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?
> Multiget Thrift query processor skips records in case of digest mismatch
> ------------------------------------------------------------------------
>
> Key: CASSANDRA-14812
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14812
> Project: Cassandra
> Issue Type: Bug
> Reporter: Sivukhin Nikita
> Priority: Major
> Labels: bug
> Attachments: repro_script.py, requirements.txt, small_repro_script.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: [email protected]
For additional commands, e-mail: [email protected]