Repository: cassandra Updated Branches: refs/heads/trunk 7fc0e9efe -> 5f37e9fa5
Improve compaction logging Patch by Mihai Suteu; reviewed by marcuse for CASSANDRA-7818 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/5f37e9fa Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/5f37e9fa Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/5f37e9fa Branch: refs/heads/trunk Commit: 5f37e9fa57189b6f9fd30f1c5f1894686d076b34 Parents: 7fc0e9e Author: Marcus Eriksson <[email protected]> Authored: Tue Sep 16 08:36:07 2014 +0200 Committer: Marcus Eriksson <[email protected]> Committed: Tue Sep 16 08:37:19 2014 +0200 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../cassandra/db/compaction/CompactionTask.java | 14 +++++++++++--- 2 files changed, 12 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f37e9fa/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index 9a43511..ed141ef 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0 + * Improve compaction logging (CASSANDRA-7818) * Remove YamlFileNetworkTopologySnitch (CASSANDRA-7917) * Support Java source code for user-defined functions (CASSANDRA-7562) * Require arg types to disambiguate UDF drops (CASSANDRA-7812) http://git-wip-us.apache.org/repos/asf/cassandra/blob/5f37e9fa/src/java/org/apache/cassandra/db/compaction/CompactionTask.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java index 349caf3..fa0cffc 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java @@ -44,6 +44,7 @@ import org.apache.cassandra.io.sstable.SSTableWriter; import org.apache.cassandra.io.sstable.metadata.MetadataCollector; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.utils.CloseableIterator; +import org.apache.cassandra.utils.UUIDGen; public class CompactionTask extends AbstractCompactionTask { @@ -132,7 +133,14 @@ public class CompactionTask extends AbstractCompactionTask // 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) - logger.info("Compacting {}", sstables); + StringBuilder ssTableLoggerMsg = new StringBuilder("["); + for (SSTableReader sstr : sstables) + { + ssTableLoggerMsg.append(String.format("SSTableReader(path=%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; @@ -246,8 +254,8 @@ public class CompactionTask extends AbstractCompactionTask } SystemKeyspace.updateCompactionHistory(cfs.keyspace.getName(), cfs.name, System.currentTimeMillis(), startsize, endsize, mergedRows); - logger.info(String.format("Compacted %d sstables to [%s]. %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s. %,d total partitions merged to %,d. Partition merge counts were {%s}", - oldSStables.size(), newSSTableNames.toString(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary.toString())); + 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, oldSStables.size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary.toString())); logger.debug(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize))); logger.debug("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedTotalKeys, ((double)(totalKeysWritten - estimatedTotalKeys)/totalKeysWritten)); }
