private static SSTableReader sstable( File dataFolder, ColumnFamilyStore cfs, int generation, int size) throws IOException { Descriptor descriptor = new Descriptor(dataFolder, cfs.keyspace.getName(), cfs.getTableName(), generation); Set<Component> components = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.TOC); for (Component component : components) { File file = new File(descriptor.filenameFor(component)); if (!file.exists()) assertTrue(file.createNewFile()); try (RandomAccessFile raf = new RandomAccessFile(file, "rw")) { raf.setLength(size); } } SegmentedFile dFile = new BufferedSegmentedFile( new ChannelProxy(new File(descriptor.filenameFor(Component.DATA))), RandomAccessReader.DEFAULT_BUFFER_SIZE, 0); SegmentedFile iFile = new BufferedSegmentedFile( new ChannelProxy(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))), RandomAccessReader.DEFAULT_BUFFER_SIZE, 0); SerializationHeader header = SerializationHeader.make(cfs.metadata, Collections.emptyList()); StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata.comparator) .finalizeMetadata( cfs.metadata.partitioner.getClass().getCanonicalName(), 0.01f, -1, header) .get(MetadataType.STATS); SSTableReader reader = SSTableReader.internalOpen( descriptor, components, cfs.metadata, dFile, iFile, MockSchema.indexSummary.sharedCopy(), new AlwaysPresentFilter(), 1L, metadata, SSTableReader.OpenReason.NORMAL, header); reader.first = reader.last = MockSchema.readerBounds(generation); return reader; }
@Test public void testRemoveUnfinishedLeftovers_multipleFolders_partialNonFinalRecord_second() throws Throwable { testRemoveUnfinishedLeftovers_multipleFolders_errorConditions( txn -> { List<File> logFiles = txn.logFiles(); Assert.assertEquals(2, logFiles.size()); // insert a partial sstable record and a full commit record String sstableRecord = LogRecord.make(LogRecord.Type.ADD, Collections.emptyList(), 0, "abc").raw; int toChop = sstableRecord.length() / 2; FileUtils.append(logFiles.get(0), sstableRecord); FileUtils.append( logFiles.get(1), sstableRecord.substring(0, sstableRecord.length() - toChop)); String finalRecord = LogRecord.makeCommit(System.currentTimeMillis()).raw; FileUtils.append(logFiles.get(0), finalRecord); FileUtils.append(logFiles.get(1), finalRecord); }, false); }
@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()); }
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)); }