Author: goffinet
Date: Wed Oct  7 21:47:38 2009
New Revision: 822931

URL: http://svn.apache.org/viewvc?rev=822931&view=rev
Log:
Added the ability to set maxthreshold to zero in nodeprobe, and added a special 
case to CF.doCompaction to not run compaction when its disabled. patch by 
goffinet; reviewed by jbellis for CASSANDRA-465

Modified:
    
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=822931&r1=822930&r2=822931&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
 (original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
 Wed Oct  7 21:47:38 2009
@@ -611,20 +611,27 @@
      */
     int doCompaction(int minThreshold, int maxThreshold) throws IOException
     {
-        logger_.debug("Checking to see if compaction of " + columnFamily_ + " 
would be useful");
         int filesCompacted = 0;
-        for (List<SSTableReader> sstables : getCompactionBuckets(ssTables_, 
50L * 1024L * 1024L))
+        if (minThreshold > 0 && maxThreshold > 0)
         {
-            if (sstables.size() < minThreshold)
+            logger_.debug("Checking to see if compaction of " + columnFamily_ 
+ " would be useful");
+            for (List<SSTableReader> sstables : 
getCompactionBuckets(ssTables_, 50L * 1024L * 1024L))
             {
-                continue;
+                if (sstables.size() < minThreshold)
+                {
+                    continue;
+                }
+                // if we have too many to compact all at once, compact older 
ones first -- this avoids
+                // re-compacting files we just created.
+                Collections.sort(sstables);
+                filesCompacted += doFileCompaction(sstables.subList(0, 
Math.min(sstables.size(), maxThreshold)));
             }
-            // if we have too many to compact all at once, compact older ones 
first -- this avoids
-            // re-compacting files we just created.
-            Collections.sort(sstables);
-            filesCompacted += doFileCompaction(sstables.subList(0, 
Math.min(sstables.size(), maxThreshold)));
+            logger_.debug(filesCompacted + " files compacted");
+        }
+        else
+        {
+            logger_.debug("Compaction is currently disabled.");
         }
-        logger_.debug(filesCompacted + " files compacted");
         return filesCompacted;
     }
 

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java?rev=822931&r1=822930&r2=822931&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java 
(original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java 
Wed Oct  7 21:47:38 2009
@@ -429,7 +429,7 @@
     public void setCompactionThreshold(int minimumCompactionThreshold, int 
maximumCompactionThreshold)
     {
         mcmProxy.setMinimumCompactionThreshold(minimumCompactionThreshold);
-        if (maximumCompactionThreshold > 0)
+        if (maximumCompactionThreshold >= 0)
         {
              
mcmProxy.setMaximumCompactionThreshold(maximumCompactionThreshold);
         }
@@ -563,11 +563,25 @@
                 System.exit(1);
             }
             int minthreshold = Integer.parseInt(arguments[1]);
-            int maxthreshold = 0;
+            int maxthreshold = 
CompactionManager.instance().getMaximumCompactionThreshold();
             if (arguments.length > 2)
             {
                 maxthreshold = Integer.parseInt(arguments[2]);
             }
+
+            if (minthreshold > maxthreshold)
+            {
+                System.err.println("Min threshold can't be greater than Max 
threshold");
+                NodeProbe.printUsage();
+                System.exit(1);
+            }
+
+            if (minthreshold < 2 && maxthreshold != 0)
+            {
+                System.err.println("Min threshold must be at least 2");
+                NodeProbe.printUsage();
+                System.exit(1);
+            }
             probe.setCompactionThreshold(minthreshold, maxthreshold);
         }
         else


Reply via email to