@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(); }
protected Throwable doCommit(Throwable accumulate) { sstableOld.markObsolete(tidier); sstableOld.selfRef().release(); LogTransaction.waitForDeletions(); Throwable ret = txnLogs.commit(accumulate); sstableNew.selfRef().release(); return ret; }
@Test public void testCommitOnlyOld() 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); LogTransaction.SSTableTidier tidier = log.obsoleted(sstable); assertNotNull(tidier); log.finish(); sstable.markObsolete(tidier); sstable.selfRef().release(); assertFiles(dataFolder.getPath(), new HashSet<>()); }
@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); }