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

Missing results on reading large frozen text map

    XMLWordPrintableJSON

    Details

    • Type: Bug
    • Status: Open
    • Priority: Normal
    • Resolution: Unresolved
    • Fix Version/s: None
    • Component/s: Local/SSTable
    • Labels:
      None
    • Bug Category:
      Correctness - Recoverable Corruption / Loss
    • Severity:
      Critical
    • Complexity:
      Challenging
    • Discovered By:
      User Report
    • Platform:
      All
    • Impacts:
      None

      Description

      Cassandra version: 2.2.15

      I have been running into a case where, when fetching the results from a table with a frozen<map<text, text>>, if the number of results is greater than the fetch size (default 5000), we can end up with missing data.

      Side note: The table schema comes from using KairosDB, but we've isolated this issue to Cassandra itself. But it looks like this can cause problems for users of KairosDB as well.

      Repro case. Tested against fresh install of Cassandra 2.2.15.

      1. Create table (csqlsh)

      CREATE KEYSPACE test
        WITH REPLICATION = { 
         'class' : 'SimpleStrategy', 
         'replication_factor' : 1 
        };
      
        CREATE TABLE test.test (
          name text,
          tags frozen<map<text, text>>,
          PRIMARY KEY (name, tags)
        ) WITH CLUSTERING ORDER BY (tags ASC);
      

      2. Insert data (python3)

      import time
      from cassandra.cluster import Cluster
      
      cluster = Cluster(['127.0.0.1'])
      session = cluster.connect('test')
      
      for i in range(0, 20000):
          session.execute(
              """
              INSERT INTO test (name, tags)  
              VALUES (%s, %s)
              """,
              ("test_name", {'id':str(i)})
          )
      

       

      3. Flush

       

      nodetools flush

       

       

      4. Fetch data (python3)

      import time
      from cassandra.cluster import Cluster
      
      cluster = Cluster(['127.0.0.1'], control_connection_timeout=5000)
      session = cluster.connect('test')
      session.default_fetch_size = 5000
      session.default_timeout = 120
      
      count = 0
      rows = session.execute("select tags from test where name='test_name'")
      for row in rows:
          count += 1
      
      print(count)
      

      Result: 10111 (expected 20000)

       

      Changing the page size changes the result count. Some quick samples:

       

      default_fetch_size count
      5000 10111
      1000 1830
      999 1840
      998 1850
      20000 20000
      100000 20000

       

       

      In short, I cannot guarantee I'll get all the results back unless the page size > number of rows.

      This seems to get worse with multiple SSTables (eg nodetool flush between some of the insert batches). When using replication, the issue can get disgustingly bad - potentially giving a different result on each query.

      Interesting, if we pad the values on the tag map ("id" in this repro case) so that the insertion is in lexicographical order, there is no issue. I believe the issue also does not repro if I do not call "nodetools flush" before querying.

        Attachments

          Activity

            People

            • Assignee:
              ifesdjeen Alex Petrov
              Reporter:
              spodi Tyler Codispoti
              Authors:
              Alex Petrov
            • Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

              • Created:
                Updated: