Author: jbellis Date: Fri Oct 7 19:09:10 2011 New Revision: 1180159 URL: http://svn.apache.org/viewvc?rev=1180159&view=rev Log: clean up CompactionTask patch by jbellis; reviewed by bcoverston for CASSANDRA-3330
Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/CompactionTask.java cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/CompactionTask.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/CompactionTask.java?rev=1180159&r1=1180158&r2=1180159&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/CompactionTask.java (original) +++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/CompactionTask.java Fri Oct 7 19:09:10 2011 @@ -39,7 +39,7 @@ import org.apache.cassandra.utils.Closea public class CompactionTask extends AbstractCompactionTask { - private static final Logger logger = LoggerFactory.getLogger(CompactionTask.class); + protected static final Logger logger = LoggerFactory.getLogger(CompactionTask.class); protected String compactionFileLocation; protected final int gcBefore; protected boolean isUserDefined; @@ -70,20 +70,13 @@ public class CompactionTask extends Abst assert sstables != null; Set<SSTableReader> toCompact = new HashSet<SSTableReader>(sstables); - if (!isUserDefined) - { - if (!allowSingletonCompaction() && toCompact.size() < 2) - { - String msg = "Nothing to compact in " + cfs.getColumnFamilyName(); - if (cfs.getCompactionStrategy() instanceof SizeTieredCompactionStrategy) - msg += ". Use forceUserDefinedCompaction if you wish to force compaction of single sstables (e.g. for tombstone collection)"; - logger.info(msg); - return 0; - } - - if (compactionFileLocation == null) - compactionFileLocation = cfs.table.getDataFileLocation(cfs.getExpectedCompactedFileSize(toCompact)); + if (!isCompactionInteresting(toCompact)) + return 0; + if (compactionFileLocation == null) + compactionFileLocation = cfs.table.getDataFileLocation(cfs.getExpectedCompactedFileSize(toCompact)); + if (partialCompactionsAcceptable()) + { // If the compaction file path is null that means we have no space left for this compaction. // Try again w/o the largest one. if (compactionFileLocation == null) @@ -219,12 +212,17 @@ public class CompactionTask extends Abst builder.append("]"); double mbps = dTime > 0 ? (double)endsize/(1024*1024)/((double)dTime/1000) : 0; - logger.info(String.format("Compacted to %s. %,d to %,d (~%d%% of original) bytes for %,d keys at %fMBPS. Time: %,dms.", + logger.info(String.format("Compacted to %s. %,d to %,d (~%d%% of original) bytes for %,d keys at %fMB/s. Time: %,dms.", builder.toString(), startsize, endsize, (int) (ratio * 100), totalkeysWritten, mbps, dTime)); logger.debug(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize))); return toCompact.size(); } + protected boolean partialCompactionsAcceptable() + { + return !isUserDefined; + } + //extensibility point for other strategies that may want to limit the upper bounds of the sstable segment size protected boolean newSSTableSegmentThresholdReached(SSTableWriter writer, long position) { @@ -232,11 +230,15 @@ public class CompactionTask extends Abst } /** - * extend this if the overridden compaction strategy requires single files to be compacted to function properly - * @return boolean + * @return true if the proposed compaction is worth proceeding with. We allow leveled compaction to + * override this to allow "promoting" sstables from one level to another w/o rewriting them, if there is no overlapping. */ - protected boolean allowSingletonCompaction() + protected boolean isCompactionInteresting(Set<SSTableReader> toCompact) { + if (isUserDefined || toCompact.size() >= 2) + return true; + logger.info(String.format("Nothing to compact in %s. Use forceUserDefinedCompaction if you wish to force compaction of single sstables (e.g. for tombstone collection)", + cfs.getColumnFamilyName())); return false; } Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java?rev=1180159&r1=1180158&r2=1180159&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java (original) +++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java Fri Oct 7 19:09:10 2011 @@ -43,7 +43,7 @@ public class LeveledCompactionStrategy e { private static final Logger logger = LoggerFactory.getLogger(LeveledCompactionStrategy.class); - private LeveledManifest manifest; + private final LeveledManifest manifest; private final String SSTABLE_SIZE_OPTION = "sstable_size_in_mb"; private final int maxSSTableSizeInMB; private final AtomicReference<LeveledCompactionTask> task = new AtomicReference<LeveledCompactionTask>(); @@ -144,13 +144,11 @@ public class LeveledCompactionStrategy e { SSTableAddedNotification flushedNotification = (SSTableAddedNotification) notification; manifest.add(flushedNotification.added); - manifest.logDistribution(); } else if (notification instanceof SSTableListChangedNotification) { SSTableListChangedNotification listChangedNotification = (SSTableListChangedNotification) notification; manifest.promote(listChangedNotification.removed, listChangedNotification.added); - manifest.logDistribution(); } } Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java?rev=1180159&r1=1180158&r2=1180159&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java (original) +++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java Fri Oct 7 19:09:10 2011 @@ -21,11 +21,19 @@ package org.apache.cassandra.db.compacti */ +import java.io.File; import java.io.IOException; import java.util.Collection; +import java.util.Set; import java.util.concurrent.CountDownLatch; +import com.google.common.collect.Iterables; + +import org.apache.commons.lang.StringUtils; + +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Table; import org.apache.cassandra.io.sstable.SSTableReader; import org.apache.cassandra.io.sstable.SSTableWriter; @@ -61,8 +69,14 @@ public class LeveledCompactionTask exten } @Override - protected boolean allowSingletonCompaction() + protected boolean isCompactionInteresting(Set<SSTableReader> toCompact) { return true; } + + @Override + protected boolean partialCompactionsAcceptable() + { + return false; + } } Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java?rev=1180159&r1=1180158&r2=1180159&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java (original) +++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java Fri Oct 7 19:09:10 2011 @@ -129,6 +129,7 @@ public class LeveledManifest public synchronized void add(SSTableReader reader) { + logDistribution(); logger.debug("Adding {} to L0", reader); add(reader, 0); serialize(); @@ -150,6 +151,7 @@ public class LeveledManifest public synchronized void promote(Iterable<SSTableReader> removed, Iterable<SSTableReader> added) { + logDistribution(); if (logger.isDebugEnabled()) logger.debug((Iterables.isEmpty(added) ? "Removing [" : "Replacing [") + toString(removed) + "]"); @@ -206,8 +208,6 @@ public class LeveledManifest public synchronized Collection<SSTableReader> getCompactionCandidates() { - logDistribution(); - // LevelDB gives each level a score of how much data it contains vs its ideal amount, and // compacts the level with the highest score. But this falls apart spectacularly once you // get behind. Consider this set of levels: @@ -257,7 +257,7 @@ public class LeveledManifest return generations.length > i ? generations[i].size() : 0; } - public void logDistribution() + private void logDistribution() { for (int i = 0; i < generations.length; i++) {