You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "ZhaoYang (JIRA)" <ji...@apache.org> on 2018/03/15 07:58:00 UTC

[jira] [Created] (CASSANDRA-14315) ThrottledUnfilteredIterator failed on UnfilteredRowIterator with only partition level info

ZhaoYang created CASSANDRA-14315:
------------------------------------

             Summary: ThrottledUnfilteredIterator failed on UnfilteredRowIterator with only partition level info
                 Key: CASSANDRA-14315
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-14315
             Project: Cassandra
          Issue Type: Bug
          Components: Materialized Views
         Environment: | [trunk|https://github.com/jasonstack/cassandra/commits/throttle-trunk]|
| [dtest|https://github.com/jasonstack/cassandra-dtest/commits/throttle-mv] |

Changes:
1. when batch size is 0, don't throttle and return itself. (opt out)
2. when the original UnfilteredRowIterator has only partition level info, return itself once.
3. removed the assertion "UnfilteredPartitionIterator should not contain empty partitions"
            Reporter: ZhaoYang
            Assignee: ZhaoYang
             Fix For: 4.0


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.

{code:title=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);
}
{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org