public static String updateCompactionHistory( String keyspaceName, String columnFamilyName, AbstractCompactionIterable ci, long startSize, long endSize) { long[] counts = ci.getMergedRowCounts(); StringBuilder mergeSummary = new StringBuilder(counts.length * 10); Map<Integer, Long> mergedRows = new HashMap<>(); for (int i = 0; i < counts.length; i++) { long count = counts[i]; if (count == 0) continue; int rows = i + 1; mergeSummary.append(String.format("%d:%d, ", rows, count)); mergedRows.put(rows, count); } SystemKeyspace.updateCompactionHistory( keyspaceName, columnFamilyName, System.currentTimeMillis(), startSize, endSize, mergedRows); return mergeSummary.toString(); }
/** * For internal use and testing only. The rest of the system should go through the submit* * methods, which are properly serialized. Caller is in charge of marking/unmarking the sstables * as compacting. */ protected void runMayThrow() throws Exception { // The collection of sstables passed may be empty (but not null); even if // it is not empty, it may compact down to nothing if all rows are deleted. assert transaction != null; if (transaction.originals().isEmpty()) return; // Note that the current compaction strategy, is not necessarily the one this task was created // under. // This should be harmless; see comments to CFS.maybeReloadCompactionStrategy. AbstractCompactionStrategy strategy = cfs.getCompactionStrategy(); if (DatabaseDescriptor.isSnapshotBeforeCompaction()) cfs.snapshotWithoutFlush(System.currentTimeMillis() + "-compact-" + cfs.name); // note that we need to do a rough estimate early if we can fit the compaction on disk - this is // pessimistic, but // since we might remove sstables from the compaction in checkAvailableDiskSpace it needs to be // done here long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType); long earlySSTableEstimate = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes()); checkAvailableDiskSpace(earlySSTableEstimate, expectedWriteSize); // sanity check: all sstables must belong to the same cfs assert !Iterables.any( transaction.originals(), new Predicate<SSTableReader>() { @Override public boolean apply(SSTableReader sstable) { return !sstable.descriptor.cfname.equals(cfs.name); } }); UUID taskId = SystemKeyspace.startCompaction(cfs, transaction.originals()); // new sstables from flush can be added during a compaction, but only the compaction can remove // them, // so in our single-threaded compaction world this is a valid way of determining if we're // compacting // all the sstables (that existed when we started) StringBuilder ssTableLoggerMsg = new StringBuilder("["); for (SSTableReader sstr : transaction.originals()) { ssTableLoggerMsg.append( String.format("%s:level=%d, ", sstr.getFilename(), sstr.getSSTableLevel())); } ssTableLoggerMsg.append("]"); String taskIdLoggerMsg = taskId == null ? UUIDGen.getTimeUUID().toString() : taskId.toString(); logger.info("Compacting ({}) {}", taskIdLoggerMsg, ssTableLoggerMsg); long start = System.nanoTime(); long totalKeysWritten = 0; long estimatedKeys = 0; try (CompactionController controller = getCompactionController(transaction.originals())) { Set<SSTableReader> actuallyCompact = Sets.difference(transaction.originals(), controller.getFullyExpiredSSTables()); SSTableFormat.Type sstableFormat = getFormatType(transaction.originals()); List<SSTableReader> newSStables; AbstractCompactionIterable ci; // SSTableScanners need to be closed before markCompactedSSTablesReplaced call as scanners // contain references // to both ifile and dfile and SSTR will throw deletion errors on Windows if it tries to // delete before scanner is closed. // See CASSANDRA-8019 and CASSANDRA-8399 try (Refs<SSTableReader> refs = Refs.ref(actuallyCompact); AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(actuallyCompact)) { ci = new CompactionIterable( compactionType, scanners.scanners, controller, sstableFormat, taskId); try (CloseableIterator<AbstractCompactedRow> iter = ci.iterator()) { if (collector != null) collector.beginCompaction(ci); long lastCheckObsoletion = start; if (!controller.cfs.getCompactionStrategy().isActive) throw new CompactionInterruptedException(ci.getCompactionInfo()); try (CompactionAwareWriter writer = getCompactionAwareWriter(cfs, transaction, actuallyCompact)) { estimatedKeys = writer.estimatedKeys(); while (iter.hasNext()) { if (ci.isStopRequested()) throw new CompactionInterruptedException(ci.getCompactionInfo()); try (AbstractCompactedRow row = iter.next()) { if (writer.append(row)) totalKeysWritten++; if (System.nanoTime() - lastCheckObsoletion > TimeUnit.MINUTES.toNanos(1L)) { controller.maybeRefreshOverlaps(); lastCheckObsoletion = System.nanoTime(); } } } // don't replace old sstables yet, as we need to mark the compaction finished in the // system table newSStables = writer.finish(); } finally { // point of no return -- the new sstables are live on disk; next we'll start deleting // the old ones // (in replaceCompactedSSTables) if (taskId != null) SystemKeyspace.finishCompaction(taskId); if (collector != null) collector.finishCompaction(ci); } } } // log a bunch of statistics about the result and save to system table compaction_history long dTime = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); long startsize = SSTableReader.getTotalBytes(transaction.originals()); long endsize = SSTableReader.getTotalBytes(newSStables); double ratio = (double) endsize / (double) startsize; StringBuilder newSSTableNames = new StringBuilder(); for (SSTableReader reader : newSStables) newSSTableNames.append(reader.descriptor.baseFilename()).append(","); double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0; long totalSourceRows = 0; String mergeSummary = updateCompactionHistory( cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize); logger.info( String.format( "Compacted (%s) %d sstables to [%s] to level=%d. %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s. %,d total partitions merged to %,d. Partition merge counts were {%s}", taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary)); logger.debug( String.format( "CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize))); logger.debug( "Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double) (totalKeysWritten - estimatedKeys) / totalKeysWritten)); if (offline) Refs.release(Refs.selfRefs(newSStables)); } }