logger.info("Anticompacting {}", sstable);
Set<SSTableReader> sstableAsSet = new HashSet<>();
sstableAsSet.add(sstable);
File destination = cfs.directories.getDirectoryForCompactedSSTables();
SSTableRewriter repairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, sstable.maxDataAge, OperationType.ANTICOMPACTION, false);
SSTableRewriter unRepairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, sstable.maxDataAge, OperationType.ANTICOMPACTION, false);
AbstractCompactionStrategy strategy = cfs.getCompactionStrategy();
List<ICompactionScanner> scanners = strategy.getScanners(Arrays.asList(sstable));
try (CompactionController controller = new CompactionController(cfs, new HashSet<>(Collections.singleton(sstable)), CFMetaData.DEFAULT_GC_GRACE_SECONDS))
{
repairedSSTableWriter.switchWriter(CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, repairedAt, sstable));
unRepairedSSTableWriter.switchWriter(CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, ActiveRepairService.UNREPAIRED_SSTABLE, sstable));
CompactionIterable ci = new CompactionIterable(OperationType.ANTICOMPACTION, scanners, controller);
try (CloseableIterator<AbstractCompactedRow> iter = ci.iterator())
{
while(iter.hasNext())
{
AbstractCompactedRow row = iter.next();
// if current range from sstable is repaired, save it into the new repaired sstable
if (Range.isInRanges(row.key.getToken(), ranges))
{
repairedSSTableWriter.append(row);
repairedKeyCount++;
}
// otherwise save into the new 'non-repaired' table
else
{
unRepairedSSTableWriter.append(row);
unrepairedKeyCount++;
}
}
}
// we have the same readers being rewritten by both writers, so we ask the first one NOT to close them
// so that the second one can do so safely, without leaving us with references < 0 or any other ugliness
repairedSSTableWriter.finish(false, repairedAt);
unRepairedSSTableWriter.finish(ActiveRepairService.UNREPAIRED_SSTABLE);
// add repaired table with a non-null timestamp field to be saved in SSTableMetadata#repairedAt
anticompactedSSTables.addAll(repairedSSTableWriter.finished());
anticompactedSSTables.addAll(unRepairedSSTableWriter.finished());
}
catch (Throwable e)
{
logger.error("Error anticompacting " + sstable, e);
repairedSSTableWriter.abort();
unRepairedSSTableWriter.abort();
}
}
String format = "Repaired {} keys of {} for {}/{}";
logger.debug(format, repairedKeyCount, (repairedKeyCount + unrepairedKeyCount), cfs.keyspace, cfs.getColumnFamilyName());
String format2 = "Anticompaction completed successfully, anticompacted from {} to {} sstable(s).";