@Test public void testRemoveUnfinishedLeftovers_abort_multipleFolders() throws Throwable { ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE); File origiFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables(); File dataFolder1 = new File(origiFolder, "1"); File dataFolder2 = new File(origiFolder, "2"); Files.createDirectories(dataFolder1.toPath()); Files.createDirectories(dataFolder2.toPath()); SSTableReader[] sstables = { sstable(dataFolder1, cfs, 0, 128), sstable(dataFolder1, cfs, 1, 128), sstable(dataFolder2, cfs, 2, 128), sstable(dataFolder2, cfs, 3, 128) }; LogTransaction log = new LogTransaction(OperationType.COMPACTION); assertNotNull(log); LogTransaction.SSTableTidier[] tidiers = { log.obsoleted(sstables[0]), log.obsoleted(sstables[2]) }; log.trackNew(sstables[1]); log.trackNew(sstables[3]); Collection<File> logFiles = log.logFiles(); Assert.assertEquals(2, logFiles.size()); // fake an abort log.txnFile().abort(); Arrays.stream(sstables).forEach(s -> s.selfRef().release()); // test listing Assert.assertEquals( sstables[1].getAllFilePaths().stream().map(File::new).collect(Collectors.toSet()), getTemporaryFiles(dataFolder1)); Assert.assertEquals( sstables[3].getAllFilePaths().stream().map(File::new).collect(Collectors.toSet()), getTemporaryFiles(dataFolder2)); // normally called at startup LogTransaction.removeUnfinishedLeftovers(Arrays.asList(dataFolder1, dataFolder2)); // old tables should be only table left assertFiles(dataFolder1.getPath(), new HashSet<>(sstables[0].getAllFilePaths())); assertFiles(dataFolder2.getPath(), new HashSet<>(sstables[2].getAllFilePaths())); // complete the transaction to avoid LEAK errors Arrays.stream(tidiers).forEach(LogTransaction.SSTableTidier::run); assertNull(log.complete(null)); }
private static void testObsoletedFilesChanged(Consumer<SSTableReader> modifier) throws IOException { ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE); File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables(); SSTableReader sstableOld = sstable(dataFolder, cfs, 0, 128); SSTableReader sstableNew = sstable(dataFolder, cfs, 1, 128); // simulate tracking sstables with a committed transaction except the checksum will be wrong LogTransaction log = new LogTransaction(OperationType.COMPACTION); assertNotNull(log); log.trackNew(sstableNew); /*TransactionLog.SSTableTidier tidier =*/ log.obsoleted(sstableOld); // modify the old sstable files modifier.accept(sstableOld); // Fake a commit log.txnFile().commit(); // This should not remove the old files LogTransaction.removeUnfinishedLeftovers(cfs.metadata); assertFiles( dataFolder.getPath(), Sets.newHashSet( Iterables.concat( sstableNew.getAllFilePaths(), sstableOld.getAllFilePaths(), log.logFilePaths()))); sstableOld.selfRef().release(); sstableNew.selfRef().release(); // complete the transaction to avoid LEAK errors assertNull(log.complete(null)); assertFiles( dataFolder.getPath(), Sets.newHashSet( Iterables.concat( sstableNew.getAllFilePaths(), sstableOld.getAllFilePaths(), log.logFilePaths()))); }
@Test public void testRemoveUnfinishedLeftovers_commit() throws Throwable { ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE); File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables(); SSTableReader sstableOld = sstable(dataFolder, cfs, 0, 128); SSTableReader sstableNew = sstable(dataFolder, cfs, 1, 128); // simulate tracking sstables with a committed transaction (new log file deleted) LogTransaction log = new LogTransaction(OperationType.COMPACTION); assertNotNull(log); log.trackNew(sstableNew); LogTransaction.SSTableTidier tidier = log.obsoleted(sstableOld); // Fake a commit log.txnFile().commit(); Set<File> tmpFiles = sstableOld.getAllFilePaths().stream().map(File::new).collect(Collectors.toSet()); sstableNew.selfRef().release(); sstableOld.selfRef().release(); Assert.assertEquals(tmpFiles, getTemporaryFiles(sstableOld.descriptor.directory)); // normally called at startup LogTransaction.removeUnfinishedLeftovers(cfs.metadata); // sstableNew should be only table left Directories directories = new Directories(cfs.metadata); Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list(); assertEquals(1, sstables.size()); assertFiles(dataFolder.getPath(), new HashSet<>(sstableNew.getAllFilePaths())); // complete the transaction to avoid LEAK errors tidier.run(); assertNull(log.complete(null)); }
private static void testRemoveUnfinishedLeftovers_multipleFolders_errorConditions( Consumer<LogTransaction> modifier, boolean shouldCommit) throws Throwable { ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE); File origiFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables(); File dataFolder1 = new File(origiFolder, "1"); File dataFolder2 = new File(origiFolder, "2"); Files.createDirectories(dataFolder1.toPath()); Files.createDirectories(dataFolder2.toPath()); SSTableReader[] sstables = { sstable(dataFolder1, cfs, 0, 128), sstable(dataFolder1, cfs, 1, 128), sstable(dataFolder2, cfs, 2, 128), sstable(dataFolder2, cfs, 3, 128) }; LogTransaction log = new LogTransaction(OperationType.COMPACTION); assertNotNull(log); LogTransaction.SSTableTidier[] tidiers = { log.obsoleted(sstables[0]), log.obsoleted(sstables[2]) }; log.trackNew(sstables[1]); log.trackNew(sstables[3]); // fake some error condition on the txn logs modifier.accept(log); Arrays.stream(sstables).forEach(s -> s.selfRef().release()); LogTransaction.removeUnfinishedLeftovers(Arrays.asList(dataFolder1, dataFolder2)); LogTransaction.waitForDeletions(); if (shouldCommit) { // only new sstables should still be there assertFiles(dataFolder1.getPath(), new HashSet<>(sstables[1].getAllFilePaths())); assertFiles(dataFolder2.getPath(), new HashSet<>(sstables[3].getAllFilePaths())); } else { // all files should still be there assertFiles( dataFolder1.getPath(), Sets.newHashSet( Iterables.concat( sstables[0].getAllFilePaths(), sstables[1].getAllFilePaths(), Collections.singleton(log.logFilePaths().get(0))))); assertFiles( dataFolder2.getPath(), Sets.newHashSet( Iterables.concat( sstables[2].getAllFilePaths(), sstables[3].getAllFilePaths(), Collections.singleton(log.logFilePaths().get(1))))); } // complete the transaction to avoid LEAK errors Arrays.stream(tidiers).forEach(LogTransaction.SSTableTidier::run); log.txnFile().commit(); // just anything to make sure transaction tidier will finish assertNull(log.complete(null)); }