From cfbbb424d717b5d0c7134b0ba878fc6a9c2aacf9 Mon Sep 17 00:00:00 2001 From: Will Dey Date: Thu, 7 Nov 2024 22:40:45 -0500 Subject: [PATCH] Panic when not able to delete compaction waste (#573) --- .../db/compaction/CompactionTask.java | 25 ++++++++---- .../db/compaction/CompactionsTest.java | 39 +++++-------------- 2 files changed, 28 insertions(+), 36 deletions(-) diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java index 89ea4d36ee..a6081f7ef2 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java @@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; @@ -166,7 +167,6 @@ public boolean apply(SSTableReader sstable) // SSTableScanners need to be closed before markCompactedSSTablesReplaced call as scanners contain references // to both ifile and dfile and SSTR will throw deletion errors on Windows if it tries to delete before scanner is closed. // See CASSANDRA-8019 and CASSANDRA-8399 - boolean abortFailed = false; UUID taskId = null; String taskIdLoggerMsg; List newSStables; @@ -216,11 +216,9 @@ public boolean apply(SSTableReader sstable) CompactionException exception = new CompactionException(taskIdLoggerMsg, ssTableLoggerMsg.toString(), e); if (readyToFinish && e.getSuppressed() != null && e.getSuppressed().length != 0) { - abortFailed = true; - logger.warn("CompactionAwareWriter failed to close correctly for {}/{}. This compaction won't be removed from " + - "system.compactions_in_progress to ensure sstable cleanup on startup proceeds correctly in case some " + - "compaction-product sstables are marked final while others remain tmp", + logger.error("CompactionAwareWriter failed to close correctly for {}/{}. Continuing to compact now can cause resurrection. Exiting", cfs.keyspace.getName(), cfs.name, e); + panic(); } throw exception; } @@ -229,14 +227,23 @@ public boolean apply(SSTableReader sstable) finally { Directories.removeExpectedSpaceUsedByCompaction(expectedWriteSize, CONSIDER_CONCURRENT_COMPACTIONS); - if (taskId != null && (!abortFailed)) + if (taskId != null) SystemKeyspace.finishCompaction(taskId); if (collector != null && ci != null) collector.finishCompaction(ci); } - ColumnFamilyStoreManager.instance.markForDeletion(cfs.metadata, transaction.logged.obsoleteDescriptors()); + try + { + ColumnFamilyStoreManager.instance.markForDeletion(cfs.metadata, transaction.logged.obsoleteDescriptors()); + } + catch (Exception e) + { + logger.error("Failed to write to the write-ahead log for {}/{}. Continuing to compact now can cause resurrection. Exiting", + cfs.keyspace.getName(), cfs.name, e); + panic(); + } // log a bunch of statistics about the result and save to system table compaction_history long dTime = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); @@ -269,6 +276,10 @@ public boolean apply(SSTableReader sstable) } } + protected void panic() { + System.exit(1); + } + @Override public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction transaction, Set nonExpiredSSTables) { diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java index 2b4b2032a8..23a15babef 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java @@ -570,7 +570,7 @@ public void incompletedCompactionAbortNotRemovedFromCompactionsInProgress() thro Set compacting = Sets.newHashSet(s, s2); LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN); - CompactionTask compaction = spy(new FailedAbortCompactionTask(cfs, txn, 0, CompactionManager.NO_GC, 1024 * 1024, true)); + FailedAbortCompactionTask compaction = spy(new FailedAbortCompactionTask(cfs, txn, 0, CompactionManager.NO_GC, 1024 * 1024, true)); try { compaction.runMayThrow(); @@ -582,34 +582,7 @@ public void incompletedCompactionAbortNotRemovedFromCompactionsInProgress() thro assertTrue(e.getCause().getMessage().contains("Exception thrown while some sstables in finish")); assertTrue(e.getCause().getSuppressed()[0].getMessage().contains("Failed to do anything for abort")); } - - Collection sstablesAfter = cfs.getSSTables(); - assertEquals(50, sstablesAfter.size()); - Set nonTmp = ImmutableSet.of(1, 2, 3, 4, 5); - Set actualNonTmp = new Directories(cfs.metadata).sstableLister().skipTemporary(true).list().keySet() - .stream().map(desc -> desc.generation).collect(Collectors.toSet()); - assertEquals(nonTmp, actualNonTmp); - - Map, Map> compactionLogs = SystemKeyspace.getUnfinishedCompactions(); - Pair pair = Pair.create(KEYSPACE1, cfName); - assertTrue(compactionLogs.containsKey(pair)); - - // Copy to a new CF in case in-memory tracking affects testing - File src = new Directories(cfs.metadata).getCFDirectories().get(0); - File dst = Arrays.stream(src.getParentFile().listFiles()) - .filter(file -> file.getName().contains(CF_STANDARD6)).findFirst().orElseThrow(() -> new SafeIllegalStateException("No Standard6 CF found")); - FileUtils.copyDirectory(src, dst); - CFMetaData cf2Metadata = Schema.instance.getKSMetaData(keyspace.getName()).cfMetaData().get(CF_STANDARD6); - // removes incomplete compaction product and tmp files - ColumnFamilyStore.removeUnusedSstables(cf2Metadata, compactionLogs.getOrDefault(pair, ImmutableMap.of())); - ColumnFamilyStore.scrubDataDirectories(cf2Metadata); - - Set allGenerations = new HashSet<>(); - for (Descriptor desc : new Directories(cf2Metadata).sstableLister().list().keySet()) - allGenerations.add(desc.generation); - // When we don't retain the compaction log, the ancestors 2 and 3 are deleted and products 4 and 5 are retained, despite 40+ tmp files in the unfinished - // product not having been committed! - assertEquals(ImmutableSet.of(1, 2, 3), allGenerations); + assertTrue(compaction.panicked); } private static class FailedAbortCompactionWriter extends MaxSSTableSizeWriter @@ -636,6 +609,8 @@ protected Throwable doAbort(Throwable _accumulate) { private static class FailedAbortCompactionTask extends LeveledCompactionTask { + private boolean panicked; + public FailedAbortCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int level, int gcBefore, long maxSSTableBytes, boolean majorCompaction) { super(cfs, txn, level, gcBefore, maxSSTableBytes, majorCompaction); @@ -646,6 +621,12 @@ public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Lif { return new FailedAbortCompactionWriter(cfs, txn, nonExpiredSSTables, 1024 * 1024, 0, false, compactionType); } + + @Override + protected void panic() + { + panicked = true; + } } private static Range rangeFor(int start, int end)