Skip to content

Commit 211b67f

Browse files
committed
CNDB-14207: Don't mark the SAI non-queryable if flush fails
This commit changes handling SAI index flush failures. A flush does not force the index into the non-queryable state anymore. We can do this because after failure we rollback any partially flushed index components, and we abort the sstable flush as well. Therefore, both the failed-to-flush memtable and memtable indexes remain intact and can still serve queries. This change has several advantages: - the flush failure could be temporary and the flush may still succeed the next time, - even if the problem with flushing persists, reads will run fine - if this is a node-local problem, the other nodes have a chance to take over; a failure of one node does not propagate to the rest of the cluster
1 parent bce1f44 commit 211b67f

File tree

2 files changed

+42
-1
lines changed

2 files changed

+42
-1
lines changed

src/java/org/apache/cassandra/index/sai/disk/StorageAttachedIndexWriter.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -303,7 +303,7 @@ public void abort(Throwable accumulator, boolean fromIndex)
303303

304304
// For non-compaction, make any indexes involved in this transaction non-queryable, as they will likely not match the backing table.
305305
// For compaction: the compaction task should be aborted and new sstables will not be added to tracker
306-
if (fromIndex && opType != OperationType.COMPACTION)
306+
if (fromIndex && opType != OperationType.COMPACTION && opType != OperationType.FLUSH)
307307
indices.forEach(StorageAttachedIndex::makeIndexNonQueryable);
308308

309309
for (PerIndexWriter perIndexWriter : perIndexWriters)

test/unit/org/apache/cassandra/index/sai/functional/FlushingTest.java

Lines changed: 41 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,12 +23,19 @@
2323
import org.junit.Test;
2424

2525
import com.datastax.driver.core.ResultSet;
26+
import com.datastax.driver.core.exceptions.ReadFailureException;
2627
import org.apache.cassandra.db.marshal.Int32Type;
2728
import org.apache.cassandra.index.sai.IndexContext;
2829
import org.apache.cassandra.index.sai.SAITester;
2930
import org.apache.cassandra.index.sai.disk.v1.kdtree.NumericIndexWriter;
31+
import org.apache.cassandra.inject.ActionBuilder;
32+
import org.apache.cassandra.inject.Expression;
33+
import org.apache.cassandra.inject.Injection;
34+
import org.apache.cassandra.inject.Injections;
35+
import org.apache.cassandra.inject.InvokePointBuilder;
3036

3137
import static org.junit.Assert.assertEquals;
38+
import static org.junit.Assert.assertThrows;
3239

3340
public class FlushingTest extends SAITester
3441
{
@@ -84,4 +91,38 @@ public void testFlushingOverwriteDelete() throws Throwable
8491

8592
assertIndexFilesInToc(indexFiles());
8693
}
94+
95+
@Test
96+
public void testMemtableIndexFlushFailure() throws Throwable
97+
{
98+
Injection failMemtableComplete = Injections.newCustom("FailMemtableIndexWriterComplete")
99+
.add(InvokePointBuilder.newInvokePoint()
100+
.onClass("org.apache.cassandra.index.sai.disk.v1.MemtableIndexWriter")
101+
.onMethod("complete", "com.google.common.base.Stopwatch")
102+
)
103+
.add(ActionBuilder.newActionBuilder().actions()
104+
.doThrow(java.io.IOException.class, Expression.quote("Byteman-injected fault in MemtableIndexWriter.complete"))
105+
)
106+
.build();
107+
Injections.inject(failMemtableComplete);
108+
109+
createTable(CREATE_TABLE_TEMPLATE);
110+
createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
111+
112+
String pkValue = "key_bm_flush_fail";
113+
int indexedValue = 456;
114+
115+
execute("INSERT INTO %s (id1, v1) VALUES (?, ?)", pkValue, indexedValue);
116+
117+
// The Byteman rule will cause the SAI part of the flush to fail
118+
assertThrows(RuntimeException.class, this::flush);
119+
120+
// Assert that the index is still queryable for the inserted data despite the injected fault.
121+
ResultSet indexQueryResults = executeNet("SELECT id1 FROM %s WHERE v1 = ?", indexedValue);
122+
assertEquals("The index should be still usable despite flush failure", 1, indexQueryResults.all().size());
123+
124+
// Assert that the table is still queryable by primary key.
125+
ResultSet pkQueryResults = executeNet("SELECT v1 FROM %s WHERE id1 = ?", pkValue);
126+
assertEquals("Table should still be queryable by primary key", 1, pkQueryResults.all().size());
127+
}
87128
}

0 commit comments

Comments
 (0)