Author: jbellis Date: Sat Jul 10 03:57:15 2010 New Revision: 962753 URL: http://svn.apache.org/viewvc?rev=962753&view=rev Log: extend option to lower compaction priority to hinted handoff as well. patch by Tyler Hobbs; reviewed by jbellis for CASSANDRA-1260
Modified: cassandra/branches/cassandra-0.6/CHANGES.txt cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutor.java cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/HintedHandOffManager.java Modified: cassandra/branches/cassandra-0.6/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/CHANGES.txt?rev=962753&r1=962752&r2=962753&view=diff ============================================================================== --- cassandra/branches/cassandra-0.6/CHANGES.txt (original) +++ cassandra/branches/cassandra-0.6/CHANGES.txt Sat Jul 10 03:57:15 2010 @@ -3,6 +3,9 @@ (CASSANDRA-1229) * better performance for and stricter checking of UTF8 column names (CASSANDRA-1232) + * extend option to lower compaction priority to hinted handoff + as well (CASSANDRA-1260) + 0.6.3 * retry to make streaming connections up to 8 times. (CASSANDRA-1019) Modified: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutor.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutor.java?rev=962753&r1=962752&r2=962753&view=diff ============================================================================== --- cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutor.java (original) +++ cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutor.java Sat Jul 10 03:57:15 2010 @@ -40,6 +40,11 @@ public class JMXEnabledThreadPoolExecuto this(1, 1, Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), new NamedThreadFactory(threadPoolName)); } + public JMXEnabledThreadPoolExecutor(String threadPoolName, int priority) + { + this(1, 1, Integer.MAX_VALUE, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), new NamedThreadFactory(threadPoolName, priority)); + } + public JMXEnabledThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, Modified: cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/HintedHandOffManager.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/HintedHandOffManager.java?rev=962753&r1=962752&r2=962753&view=diff ============================================================================== --- cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/HintedHandOffManager.java (original) +++ cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/HintedHandOffManager.java Sat Jul 10 03:57:15 2010 @@ -90,7 +90,15 @@ public class HintedHandOffManager private final NonBlockingHashSet<InetAddress> queuedDeliveries = new NonBlockingHashSet<InetAddress>(); - private final ExecutorService executor_ = new JMXEnabledThreadPoolExecutor("HINTED-HANDOFF-POOL"); + private final ExecutorService executor_; + + public HintedHandOffManager() + { + int hhPriority = System.getProperty("cassandra.compaction.priority") == null + ? Thread.NORM_PRIORITY + : Integer.parseInt(System.getProperty("cassandra.compaction.priority")); + executor_ = new JMXEnabledThreadPoolExecutor("HINTED-HANDOFF-POOL", hhPriority); + } private static boolean sendMessage(InetAddress endPoint, String tableName, String key) throws IOException {