Skip to content

Commit 4bc9132

Browse files
authored
CNDB-13563: Fix ReducingKeyIteratorTest#testTotalAndReadBytesManySSTables (#1762)
### What is the issue This test doesn't disable compaction and doesn't retain a reference to the sstables, so it can run with an unexpected amount of sstables and also race with the removal of the sstables backing the ReducingKeyIterator, which causes a variety of memory safety issues. This test fails approximately 1/30 times when multiplexed in CI. ### What does this PR fix and why was it fixed This fixes several issues. First, it disables compaction after the schema is created. Second, it fulfills the contract of ReducingKeyIteratorTest by taking references to the sstables, which should be superfluous with compactions disabled, but I prefer testing the contract. Third, it verifies that nothing is changing the number of sstables in the system (which would have caught the previous compaction misconfiguration).
1 parent d2dda61 commit 4bc9132

File tree

1 file changed

+14
-11
lines changed

1 file changed

+14
-11
lines changed

test/unit/org/apache/cassandra/io/sstable/ReducingKeyIteratorTest.java

Lines changed: 14 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -18,21 +18,20 @@
1818
package org.apache.cassandra.io.sstable;
1919

2020
import java.io.IOException;
21-
import java.util.Set;
2221

2322
import org.junit.After;
2423
import org.junit.Assert;
2524
import org.junit.BeforeClass;
2625
import org.junit.Test;
27-
2826
import org.slf4j.LoggerFactory;
2927

3028
import org.apache.cassandra.SchemaLoader;
3129
import org.apache.cassandra.db.ColumnFamilyStore;
3230
import org.apache.cassandra.db.Keyspace;
3331
import org.apache.cassandra.db.RowUpdateBuilder;
3432
import org.apache.cassandra.db.compaction.CompactionManager;
35-
import org.apache.cassandra.io.sstable.format.SSTableReader;
33+
import org.apache.cassandra.db.lifecycle.SSTableSet;
34+
import org.apache.cassandra.db.lifecycle.View;
3635
import org.apache.cassandra.schema.KeyspaceParams;
3736
import org.apache.cassandra.utils.ByteBufferUtil;
3837

@@ -47,11 +46,12 @@ public class ReducingKeyIteratorTest
4746
public static void setup() throws Exception
4847
{
4948
SchemaLoader.prepareServer();
50-
CompactionManager.instance.disableAutoCompaction();
5149

5250
SchemaLoader.createKeyspace(KEYSPACE1,
5351
KeyspaceParams.simple(1),
5452
SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
53+
// schema must exist before we can disable compaction on it
54+
CompactionManager.instance.disableAutoCompaction();
5555
}
5656

5757
@After
@@ -92,14 +92,17 @@ public void testTotalAndReadBytes(int tableCount, int rowCount) throws IOExcepti
9292
store.forceBlockingFlush(UNIT_TESTS);
9393
}
9494

95-
Set<SSTableReader> sstables = store.getLiveSSTables();
96-
ReducingKeyIterator reducingIterator = new ReducingKeyIterator(sstables);
97-
98-
while (reducingIterator.hasNext())
95+
try (ColumnFamilyStore.RefViewFragment viewFragment = store.selectAndReference(View.selectFunction(SSTableSet.LIVE));
96+
ReducingKeyIterator reducingIterator = new ReducingKeyIterator(viewFragment.sstables))
9997
{
100-
Assert.assertTrue(reducingIterator.getTotalBytes() >= reducingIterator.getBytesRead());
101-
reducingIterator.next();
98+
// verify we have the expected number of sstables
99+
Assert.assertEquals(tableCount, viewFragment.sstables.size());
100+
while (reducingIterator.hasNext())
101+
{
102+
Assert.assertTrue(reducingIterator.getTotalBytes() >= reducingIterator.getBytesRead());
103+
reducingIterator.next();
104+
}
105+
Assert.assertEquals(reducingIterator.getTotalBytes(), reducingIterator.getBytesRead());
102106
}
103-
Assert.assertEquals(reducingIterator.getTotalBytes(), reducingIterator.getBytesRead());
104107
}
105108
}

0 commit comments

Comments
 (0)