Uploaded image for project: 'Apache Cassandra'
  1. Apache Cassandra
  2. CASSANDRA-14812

Multiget Thrift query returns null records after digest mismatch

Agile BoardAttach filesAttach ScreenshotBulk Copy AttachmentsBulk Move AttachmentsVotersWatch issueWatchersCreate sub-taskConvert to sub-taskMoveLinkCloneLabelsUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    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 related to CASSANDRA-9975 triggers incorrect behaviour.

      When concatenated iterator is returned from the StorageProxy.fetchRows(...), 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(...) execution. While consuming iterator for many partitions Cassandra calls BaseIterator.tryGetMoreContents(...) 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?

      Attachments

        1. small_repro_script.py
          1 kB
          Sivukhin Nikita
        2. repro_script.py
          5 kB
          Sivukhin Nikita
        3. requirements.txt
          0.1 kB
          Sivukhin Nikita
        4. small_repro_script_cql.py
          2 kB
          Michael Semb Wever

        Issue Links

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            benedict Benedict Elliott Smith Assign to me
            sivukhin.nikita Sivukhin Nikita
            Benedict Elliott Smith
            Michael Semb Wever
            Votes:
            3 Vote for this issue
            Watchers:
            11 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Slack

                Issue deployment