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

ThrottledUnfilteredIterator failed on UnfilteredRowIterator with only partition level info

    XMLWordPrintableJSON

Details

    • Normal

    Description

      When repairing base table with MV, in order to avoid OOM, Cassandra-13299 added ThrottledUnfilteredIterator to split large partition into small chunks, but it didn't handle partition without unfiltered properly.

      repro
      // create cell tombstone, range tombstone, partition deletion
      createTable("CREATE TABLE %s (pk int, ck1 int, ck2 int, v1 int, v2 int, PRIMARY KEY (pk, ck1, ck2))");
      // partition deletion
      execute("DELETE FROM %s USING TIMESTAMP 160 WHERE pk=1");
      
      // flush and generate 1 sstable
      ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
      cfs.forceBlockingFlush();
      cfs.disableAutoCompaction();
      cfs.forceMajorCompaction();
      
      assertEquals(1, cfs.getLiveSSTables().size());
      SSTableReader reader = cfs.getLiveSSTables().iterator().next();
      
      try (ISSTableScanner scanner = reader.getScanner();
              CloseableIterator<UnfilteredRowIterator> throttled = ThrottledUnfilteredIterator.throttle(scanner, 100))
      {
          assertTrue(throttled.hasNext());
          UnfilteredRowIterator iterator = throttled.next();
          assertFalse(throttled.hasNext());
          assertFalse(iterator.hasNext());
          assertEquals(iterator.partitionLevelDeletion().markedForDeleteAt(), 160);
      }
      

      Attachments

        1. unit test.png
          204 kB
          Zhao Yang
        2. dtest.png
          233 kB
          Zhao Yang

        Activity

          People

            jasonstack Zhao Yang
            jasonstack Zhao Yang
            Zhao Yang
            Paulo Motta
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: