@Test public void testCommitSameDesc() throws Throwable { ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE); File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables(); SSTableReader sstableOld1 = sstable(dataFolder, cfs, 0, 128); SSTableReader sstableOld2 = sstable(dataFolder, cfs, 0, 256); SSTableReader sstableNew = sstable(dataFolder, cfs, 1, 128); LogTransaction log = new LogTransaction(OperationType.COMPACTION); assertNotNull(log); log.trackNew(sstableNew); sstableOld1.setReplaced(); LogTransaction.SSTableTidier tidier = log.obsoleted(sstableOld2); assertNotNull(tidier); log.finish(); sstableOld2.markObsolete(tidier); sstableOld1.selfRef().release(); sstableOld2.selfRef().release(); assertFiles(dataFolder.getPath(), new HashSet<>(sstableNew.getAllFilePaths())); sstableNew.selfRef().release(); }
@Test public void testGetTemporaryFilesThrowsIfCompletingAfterObsoletion() throws Throwable { ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE); File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables(); SSTableReader sstable = sstable(dataFolder, cfs, 0, 128); LogTransaction logs = new LogTransaction(OperationType.COMPACTION); assertNotNull(logs); LogTransaction.SSTableTidier tidier = logs.obsoleted(sstable); sstable.markObsolete(tidier); sstable.selfRef().release(); LogTransaction.waitForDeletions(); try { // This should race with the asynchronous deletion of txn log files // it should throw because we are violating the requirement that a transaction must // finish before deleting files (i.e. releasing sstables) getTemporaryFiles(dataFolder); fail("Expected runtime exception"); } catch (RuntimeException e) { // pass as long as the cause is not an assertion assertFalse(e.getCause() instanceof AssertionError); } logs.finish(); }
@Test public void testCommitOnlyNew() throws Throwable { ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE); File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables(); SSTableReader sstable = sstable(dataFolder, cfs, 0, 128); LogTransaction log = new LogTransaction(OperationType.COMPACTION); assertNotNull(log); log.trackNew(sstable); log.finish(); assertFiles(dataFolder.getPath(), new HashSet<>(sstable.getAllFilePaths())); sstable.selfRef().release(); }
@Test public void testUntrack() throws Throwable { ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE); File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables(); SSTableReader sstableNew = sstable(dataFolder, cfs, 1, 128); // complete a transaction without keep the new files since they were untracked LogTransaction log = new LogTransaction(OperationType.COMPACTION); assertNotNull(log); log.trackNew(sstableNew); log.untrackNew(sstableNew); log.finish(); sstableNew.selfRef().release(); Thread.sleep(1); LogTransaction.waitForDeletions(); assertFiles(dataFolder.getPath(), Collections.<String>emptySet()); }
@Test public void testGetTemporaryFilesSafeAfterObsoletion() throws Throwable { ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE); File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables(); SSTableReader sstable = sstable(dataFolder, cfs, 0, 128); LogTransaction logs = new LogTransaction(OperationType.COMPACTION); assertNotNull(logs); LogTransaction.SSTableTidier tidier = logs.obsoleted(sstable); logs.finish(); sstable.markObsolete(tidier); sstable.selfRef().release(); // This should race with the asynchronous deletion of txn log files // It doesn't matter what it returns but it should not throw because the txn // was completed before deleting files (i.e. releasing sstables) for (int i = 0; i < 200; i++) getTemporaryFiles(dataFolder); }
@Test public void testCommitMultipleFolders() 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]); log.finish(); sstables[0].markObsolete(tidiers[0]); sstables[2].markObsolete(tidiers[1]); Arrays.stream(sstables).forEach(s -> s.selfRef().release()); LogTransaction.waitForDeletions(); assertFiles(dataFolder1.getPath(), new HashSet<>(sstables[1].getAllFilePaths())); assertFiles(dataFolder2.getPath(), new HashSet<>(sstables[3].getAllFilePaths())); }
@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()); } }
@Test public void testGetTemporaryFiles() throws IOException { ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE); File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables(); SSTableReader sstable1 = sstable(dataFolder, cfs, 0, 128); Set<File> tmpFiles = getTemporaryFiles(dataFolder); assertNotNull(tmpFiles); assertEquals(0, tmpFiles.size()); try (LogTransaction log = new LogTransaction(OperationType.WRITE)) { Directories directories = new Directories(cfs.metadata); File[] beforeSecondSSTable = dataFolder.listFiles(pathname -> !pathname.isDirectory()); SSTableReader sstable2 = sstable(dataFolder, cfs, 1, 128); log.trackNew(sstable2); Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list(); assertEquals(2, sstables.size()); // this should contain sstable1, sstable2 and the transaction log file File[] afterSecondSSTable = dataFolder.listFiles(pathname -> !pathname.isDirectory()); int numNewFiles = afterSecondSSTable.length - beforeSecondSSTable.length; assertEquals( numNewFiles - 1, sstable2.getAllFilePaths().size()); // new files except for transaction log file tmpFiles = getTemporaryFiles(dataFolder); assertNotNull(tmpFiles); assertEquals(numNewFiles - 1, tmpFiles.size()); File ssTable2DataFile = new File(sstable2.descriptor.filenameFor(Component.DATA)); File ssTable2IndexFile = new File(sstable2.descriptor.filenameFor(Component.PRIMARY_INDEX)); assertTrue(tmpFiles.contains(ssTable2DataFile)); assertTrue(tmpFiles.contains(ssTable2IndexFile)); List<File> files = directories.sstableLister(Directories.OnTxnErr.THROW).listFiles(); List<File> filesNoTmp = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true).listFiles(); assertNotNull(files); assertNotNull(filesNoTmp); assertTrue(files.contains(ssTable2DataFile)); assertTrue(files.contains(ssTable2IndexFile)); assertFalse(filesNoTmp.contains(ssTable2DataFile)); assertFalse(filesNoTmp.contains(ssTable2IndexFile)); log.finish(); // Now it should be empty since the transaction has finished tmpFiles = getTemporaryFiles(dataFolder); assertNotNull(tmpFiles); assertEquals(0, tmpFiles.size()); filesNoTmp = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true).listFiles(); assertNotNull(filesNoTmp); assertTrue(filesNoTmp.contains(ssTable2DataFile)); assertTrue(filesNoTmp.contains(ssTable2IndexFile)); sstable1.selfRef().release(); sstable2.selfRef().release(); } }