{
logger.info("Writing {}", Memtable.this.toString());
SSTableReader ssTable;
// errors when creating the writer that may leave empty temp files.
SSTableWriter writer = createFlushWriter(cfs.getTempSSTablePath(sstableDirectory));
try
{
boolean trackContention = logger.isDebugEnabled();
int heavilyContendedRowCount = 0;
// (we can't clear out the map as-we-go to free up memory,
// since the memtable is being used for queries in the "pending flush" category)
for (Map.Entry<RowPosition, AtomicBTreeColumns> entry : rows.entrySet())
{
AtomicBTreeColumns cf = entry.getValue();
if (cf.isMarkedForDelete() && cf.hasColumns())
{
// When every node is up, there's no reason to write batchlog data out to sstables
// (which in turn incurs cost like compaction) since the BL write + delete cancel each other out,
// and BL data is strictly local, so we don't need to preserve tombstones for repair.
// If we have a data row + row level tombstone, then writing it is effectively an expensive no-op so we skip it.
// See CASSANDRA-4667.
if (cfs.name.equals(SystemKeyspace.BATCHLOG_TABLE) && cfs.keyspace.getName().equals(SystemKeyspace.NAME))
continue;
}
if (trackContention && cf.usePessimisticLocking())
heavilyContendedRowCount++;
if (!cf.isEmpty())
writer.append((DecoratedKey)entry.getKey(), cf);
}
if (writer.getFilePointer() > 0)
{
writer.isolateReferences();
// temp sstables should contain non-repaired data.
ssTable = writer.closeAndOpenReader();
logger.info(String.format("Completed flushing %s (%d bytes) for commitlog position %s",
ssTable.getFilename(), new File(ssTable.getFilename()).length(), context));
}
else
{
writer.abort();
ssTable = null;
logger.info("Completed flushing; nothing needed to be retained. Commitlog position was {}",
context);
}
if (heavilyContendedRowCount > 0)
logger.debug(String.format("High update contention in %d/%d partitions of %s ", heavilyContendedRowCount, rows.size(), Memtable.this.toString()));
return ssTable;
}
catch (Throwable e)
{
writer.abort();
throw Throwables.propagate(e);
}
}