cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject svn commit: r1163657 - in /cassandra/trunk: ./ conf/ src/java/org/apache/cassandra/config/ src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/db/compaction/ src/java/org/apache/cassandra/net/
Date Wed, 31 Aug 2011 15:02:24 GMT
Author: jbellis
Date: Wed Aug 31 15:02:23 2011
New Revision: 1163657

URL: http://svn.apache.org/viewvc?rev=1163657&view=rev
Log:
r/m compaction_thread_priority setting
patch by jbellis; reviewed by slebresne for CASSANDRA-3104

Modified:
    cassandra/trunk/CHANGES.txt
    cassandra/trunk/NEWS.txt
    cassandra/trunk/conf/cassandra.yaml
    cassandra/trunk/src/java/org/apache/cassandra/config/Config.java
    cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
    cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java
    cassandra/trunk/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
    cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java

Modified: cassandra/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/trunk/CHANGES.txt?rev=1163657&r1=1163656&r2=1163657&view=diff
==============================================================================
--- cassandra/trunk/CHANGES.txt (original)
+++ cassandra/trunk/CHANGES.txt Wed Aug 31 15:02:23 2011
@@ -44,13 +44,14 @@
    Thrift<->Avro conversion methods (CASSANDRA-3032)
  * Add timeouts to client request schedulers (CASSANDRA-3079)
  * Cli to use hashes rather than array of hashes for strategy options (CASSANDRA-3081)
- * LeveledCompactionStrategy (CASSANDRA-1608, 3085)
+ * LeveledCompactionStrategy (CASSANDRA-1608, 3085, 3110)
  * Improvements of the CLI `describe` command (CASSANDRA-2630)
  * reduce window where dropped CF sstables may not be deleted (CASSANDRA-2942)
  * Expose gossip/FD info to JMX (CASSANDRA-2806)
  * Fix streaming over SSL when compressed SSTable involved (CASSANDRA-3051)
  * Add support for pluggable secondary index implementations (CASSANDRA-3078)
- * Fix closing sstable iterators before using them (CASSANDRA-3110)
+ * remove compaction_thread_priority setting (CASSANDRA-3104)
+
 
 0.8.5
  * fix NPE when encryption_options is unspecified (CASSANDRA-3007)

Modified: cassandra/trunk/NEWS.txt
URL: http://svn.apache.org/viewvc/cassandra/trunk/NEWS.txt?rev=1163657&r1=1163656&r2=1163657&view=diff
==============================================================================
--- cassandra/trunk/NEWS.txt (original)
+++ cassandra/trunk/NEWS.txt Wed Aug 31 15:02:23 2011
@@ -5,6 +5,9 @@ Upgrading
 ---------
     - the BinaryMemtable bulk-load interface has been removed. Use the
       sstableloader tool instead.
+    - the compaction_thread_priority setting has been removed from 
+      cassandra.yaml (use compaction_throughput_mb_per_sec to throttle
+      compaction instead)
 
 Features
 --------

Modified: cassandra/trunk/conf/cassandra.yaml
URL: http://svn.apache.org/viewvc/cassandra/trunk/conf/cassandra.yaml?rev=1163657&r1=1163656&r2=1163657&view=diff
==============================================================================
--- cassandra/trunk/conf/cassandra.yaml (original)
+++ cassandra/trunk/conf/cassandra.yaml Wed Aug 31 15:02:23 2011
@@ -253,13 +253,6 @@ incremental_backups: false
 # is a data format change.
 snapshot_before_compaction: false
 
-# change this to increase the compaction thread's priority.  In java, 1 is the
-# lowest priority and that is our default.
-#
-# Under Linux, this cannot be higher than 5 ("normal" thread priority) without
-# running as root.
-# compaction_thread_priority: 1
-
 # Add column indexes to a row after its contents reach this size.
 # Increase if your column values are large, or if you have a very large
 # number of columns.  The competing causes are, Cassandra has to

Modified: cassandra/trunk/src/java/org/apache/cassandra/config/Config.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/config/Config.java?rev=1163657&r1=1163656&r2=1163657&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/config/Config.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/config/Config.java Wed Aug 31 15:02:23 2011
@@ -77,8 +77,7 @@ public class Config
     public Integer thrift_max_message_length_in_mb = 16;
     public Integer thrift_framed_transport_size_in_mb = 15;
     public Boolean snapshot_before_compaction = false;
-    public Integer compaction_thread_priority = Thread.MIN_PRIORITY;
-    
+
     /* if the size of columns or super-columns are more than this, indexing will kick in
*/
     public Integer column_index_size_in_kb = 64;
     public Integer in_memory_compaction_limit_in_mb = 256;

Modified: cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java?rev=1163657&r1=1163656&r2=1163657&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java Wed Aug 31
15:02:23 2011
@@ -293,12 +293,6 @@ public class DatabaseDescriptor
                 throw new ConfigurationException("thrift_max_message_length_in_mb must be
greater than thrift_framed_transport_size_in_mb when using TFramedTransport");
             }
 
-            /* compaction thread priority */
-            if (conf.compaction_thread_priority < Thread.MIN_PRIORITY || conf.compaction_thread_priority
> Thread.NORM_PRIORITY)
-            {
-                throw new ConfigurationException("compaction_thread_priority must be between
1 and 5");
-            }
-            
             /* end point snitch */
             if (conf.endpoint_snitch == null)
             {
@@ -850,11 +844,6 @@ public class DatabaseDescriptor
         return conf.sliced_buffer_size_in_kb;
     }
 
-    public static int getCompactionThreadPriority()
-    {
-        return conf.compaction_thread_priority;
-    }
-
     public static boolean isSnapshotBeforeCompaction()
     {
         return conf.snapshot_before_compaction;

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java?rev=1163657&r1=1163656&r2=1163657&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java Wed Aug 31
15:02:23 2011
@@ -89,7 +89,7 @@ public class HintedHandOffManager implem
 
     private final NonBlockingHashSet<InetAddress> queuedDeliveries = new NonBlockingHashSet<InetAddress>();
 
-    private final ExecutorService executor_ = new JMXEnabledThreadPoolExecutor("HintedHandoff",
DatabaseDescriptor.getCompactionThreadPriority());
+    private final ExecutorService executor_ = new JMXEnabledThreadPoolExecutor("HintedHandoff");
 
     public HintedHandOffManager()
     {

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/compaction/CompactionManager.java?rev=1163657&r1=1163656&r2=1163657&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/compaction/CompactionManager.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/compaction/CompactionManager.java Wed
Aug 31 15:02:23 2011
@@ -957,12 +957,7 @@ public class CompactionManager implement
 
         protected CompactionExecutor(int minThreads, int maxThreads, String name, BlockingQueue<Runnable>
queue)
         {
-            super(minThreads,
-                  maxThreads,
-                  60,
-                  TimeUnit.SECONDS,
-                  queue,
-                  new NamedThreadFactory(name, DatabaseDescriptor.getCompactionThreadPriority()));
+            super(minThreads, maxThreads, 60, TimeUnit.SECONDS, queue, new NamedThreadFactory(name));
         }
 
         private CompactionExecutor(int threadCount, String name)

Modified: cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java?rev=1163657&r1=1163656&r2=1163657&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java Wed Aug 31 15:02:23
2011
@@ -129,7 +129,7 @@ public final class MessagingService impl
 
         listenGate = new SimpleCondition();
         verbHandlers_ = new EnumMap<StorageService.Verb, IVerbHandler>(StorageService.Verb.class);
-        streamExecutor_ = new DebuggableThreadPoolExecutor("Streaming", DatabaseDescriptor.getCompactionThreadPriority());
+        streamExecutor_ = new DebuggableThreadPoolExecutor("Streaming", Thread.MIN_PRIORITY);
         Runnable logDropped = new Runnable()
         {
             public void run()



Mime
View raw message