Author: jbellis
Date: Fri Oct 2 21:31:39 2009
New Revision: 821177
URL: http://svn.apache.org/viewvc?rev=821177&view=rev
Log:
convert compaction executor to DTPE (from DSTPE) so exceptions can be logged.
patch by jbellis; tested by molinaro for CASSANDRA-466
Modified:
incubator/cassandra/branches/cassandra-0.4/src/java/org/apache/cassandra/db/MinorCompactionManager.java
Modified:
incubator/cassandra/branches/cassandra-0.4/src/java/org/apache/cassandra/db/MinorCompactionManager.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/branches/cassandra-0.4/src/java/org/apache/cassandra/db/MinorCompactionManager.java?rev=821177&r1=821176&r2=821177&view=diff
==============================================================================
---
incubator/cassandra/branches/cassandra-0.4/src/java/org/apache/cassandra/db/MinorCompactionManager.java
(original)
+++
incubator/cassandra/branches/cassandra-0.4/src/java/org/apache/cassandra/db/MinorCompactionManager.java
Fri Oct 2 21:31:39 2009
@@ -23,11 +23,13 @@
import java.util.concurrent.Callable;
import java.util.concurrent.Future;
import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ExecutorService;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
import org.apache.cassandra.concurrent.ThreadFactoryImpl;
+import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.net.EndPoint;
@@ -38,7 +40,6 @@
private static MinorCompactionManager instance_;
private static Lock lock_ = new ReentrantLock();
private static Logger logger_ =
Logger.getLogger(MinorCompactionManager.class);
- private static final long intervalInMins_ = 5;
static final int MINCOMPACTION_THRESHOLD = 4; // compact this many
sstables min at a time
static final int MAXCOMPACTION_THRESHOLD = 32; // compact this many
sstables max at a time
@@ -149,7 +150,7 @@
}
- private ScheduledExecutorService compactor_ = new
DebuggableScheduledThreadPoolExecutor(1, new
ThreadFactoryImpl("MINOR-COMPACTION-POOL"));
+ private ExecutorService compactor_ = new
DebuggableThreadPoolExecutor("MINOR-COMPACTION-POOL");
public Future<Integer> submit(final ColumnFamilyStore columnFamilyStore)
{