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); }