Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 184673C52 for ; Thu, 28 Apr 2011 16:17:02 +0000 (UTC) Received: (qmail 91404 invoked by uid 500); 28 Apr 2011 16:17:01 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 91379 invoked by uid 500); 28 Apr 2011 16:17:01 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 91371 invoked by uid 99); 28 Apr 2011 16:17:01 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 28 Apr 2011 16:17:01 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 28 Apr 2011 16:17:00 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 8B76C23889F7; Thu, 28 Apr 2011 16:16:40 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1097530 - in /cassandra/branches/cassandra-0.8: CHANGES.txt src/java/org/apache/cassandra/db/CompactionManager.java Date: Thu, 28 Apr 2011 16:16:40 -0000 To: commits@cassandra.apache.org From: jbellis@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20110428161640.8B76C23889F7@eris.apache.org> Author: jbellis Date: Thu Apr 28 16:16:40 2011 New Revision: 1097530 URL: http://svn.apache.org/viewvc?rev=1097530&view=rev Log: make forceUserDefinedCompaction always try to attempt what is requested patch by jbellis; reviewed by slebresne for CASSANDRA-2575 Modified: cassandra/branches/cassandra-0.8/CHANGES.txt cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/CompactionManager.java Modified: cassandra/branches/cassandra-0.8/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/CHANGES.txt?rev=1097530&r1=1097529&r2=1097530&view=diff ============================================================================== --- cassandra/branches/cassandra-0.8/CHANGES.txt (original) +++ cassandra/branches/cassandra-0.8/CHANGES.txt Thu Apr 28 16:16:40 2011 @@ -11,6 +11,9 @@ * fix incorrect use of NBHM.size in ReadCallback that could cause reads to time out even when responses were received (CASSAMDRA-2552) * trigger read repair correctly for LOCAL_QUORUM reads (CASSANDRA-2556) + * forceUserDefinedCompaction will attempt to compact what it is given + even if the pessimistic estimate is that there is not enough disk space; + automatic compactions will only compact 2 or more sstables (CASSANDRA-2575) 0.8.0-beta1 Modified: cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/CompactionManager.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/CompactionManager.java?rev=1097530&r1=1097529&r2=1097530&view=diff ============================================================================== --- cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/CompactionManager.java (original) +++ cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/db/CompactionManager.java Thu Apr 28 16:16:40 2011 @@ -414,10 +414,11 @@ public class CompactionManager implement // attempt to schedule the set else if ((sstables = cfs.getDataTracker().markCompacting(sstables, 1, Integer.MAX_VALUE)) != null) { + String location = cfs.table.getDataFileLocation(1); // success: perform the compaction try { - doCompaction(cfs, sstables, gcBefore); + doCompactionWithoutSizeEstimation(cfs, sstables, gcBefore, location); } finally { @@ -489,11 +490,32 @@ public class CompactionManager implement } } + int doCompaction(ColumnFamilyStore cfs, Collection sstables, int gcBefore) throws IOException + { + Table table = cfs.table; + + // If the compaction file path is null that means we have no space left for this compaction. + // try again w/o the largest one. + Set smallerSSTables = new HashSet(sstables); + while (smallerSSTables.size() > 1) + { + String compactionFileLocation = table.getDataFileLocation(cfs.getExpectedCompactedFileSize(smallerSSTables)); + if (compactionFileLocation != null) + return doCompactionWithoutSizeEstimation(cfs, smallerSSTables, gcBefore, compactionFileLocation); + + logger.warn("insufficient space to compact all requested files " + StringUtils.join(smallerSSTables, ", ")); + smallerSSTables.remove(cfs.getMaxSizeFile(smallerSSTables)); + } + + logger.error("insufficient space to compact even the two smallest files, aborting"); + return 0; + } + /** * For internal use and testing only. The rest of the system should go through the submit* methods, * which are properly serialized. */ - int doCompaction(ColumnFamilyStore cfs, Collection sstables, int gcBefore) throws IOException + int doCompactionWithoutSizeEstimation(ColumnFamilyStore cfs, Collection sstables, int gcBefore, String compactionFileLocation) throws IOException { // 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. @@ -507,23 +529,6 @@ public class CompactionManager implement for (SSTableReader sstable : sstables) assert sstable.descriptor.cfname.equals(cfs.columnFamily); - String compactionFileLocation = table.getDataFileLocation(cfs.getExpectedCompactedFileSize(sstables)); - // If the compaction file path is null that means we have no space left for this compaction. - // try again w/o the largest one. - List smallerSSTables = new ArrayList(sstables); - while (compactionFileLocation == null && smallerSSTables.size() > 1) - { - logger.warn("insufficient space to compact all requested files " + StringUtils.join(smallerSSTables, ", ")); - smallerSSTables.remove(cfs.getMaxSizeFile(smallerSSTables)); - compactionFileLocation = table.getDataFileLocation(cfs.getExpectedCompactedFileSize(smallerSSTables)); - } - if (compactionFileLocation == null) - { - logger.error("insufficient space to compact even the two smallest files, aborting"); - return 0; - } - sstables = smallerSSTables; - // 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)