@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)); }
@Test public void testAbortMultipleFolders() 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]); Arrays.stream(tidiers).forEach(LogTransaction.SSTableTidier::abort); log.abort(); Arrays.stream(sstables).forEach(s -> s.selfRef().release()); LogTransaction.waitForDeletions(); assertFiles(dataFolder1.getPath(), new HashSet<>(sstables[0].getAllFilePaths())); assertFiles(dataFolder2.getPath(), new HashSet<>(sstables[2].getAllFilePaths())); }
static Set<File> listFiles(File folder, Directories.FileType... types) { Collection<Directories.FileType> match = Arrays.asList(types); return new LogAwareFileLister( folder.toPath(), (file, type) -> match.contains(type), Directories.OnTxnErr.IGNORE) .list() .stream() .map( f -> { try { return f.getCanonicalFile(); } catch (IOException e) { throw new IOError(e); } }) .collect(Collectors.toSet()); }
@Test public void testGetTemporaryFilesMultipleFolders() throws IOException { 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) }; // they should all have the same number of files since they are created in the same way int numSStableFiles = sstables[0].getAllFilePaths().size(); LogTransaction log = new LogTransaction(OperationType.COMPACTION); assertNotNull(log); for (File dataFolder : new File[] {dataFolder1, dataFolder2}) { Set<File> tmpFiles = getTemporaryFiles(dataFolder); assertNotNull(tmpFiles); assertEquals(0, tmpFiles.size()); } LogTransaction.SSTableTidier[] tidiers = { log.obsoleted(sstables[0]), log.obsoleted(sstables[2]) }; log.trackNew(sstables[1]); log.trackNew(sstables[3]); for (File dataFolder : new File[] {dataFolder1, dataFolder2}) { Set<File> tmpFiles = getTemporaryFiles(dataFolder); assertNotNull(tmpFiles); assertEquals(numSStableFiles, tmpFiles.size()); } log.finish(); for (File dataFolder : new File[] {dataFolder1, dataFolder2}) { Set<File> tmpFiles = getTemporaryFiles(dataFolder); assertNotNull(tmpFiles); assertEquals(numSStableFiles, tmpFiles.size()); } sstables[0].markObsolete(tidiers[0]); sstables[2].markObsolete(tidiers[1]); Arrays.stream(sstables).forEach(s -> s.selfRef().release()); LogTransaction.waitForDeletions(); for (File dataFolder : new File[] {dataFolder1, dataFolder2}) { Set<File> tmpFiles = getTemporaryFiles(dataFolder); assertNotNull(tmpFiles); assertEquals(0, tmpFiles.size()); } }
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)); }