Author: jbellis Date: Thu Feb 3 20:01:04 2011 New Revision: 1066930 URL: http://svn.apache.org/viewvc?rev=1066930&view=rev Log: revert #2061
Added: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/concurrent/RetryingScheduledThreadPoolExecutor.java - copied unchanged from r1066889, cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/concurrent/RetryingScheduledThreadPoolExecutor.java Removed: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/concurrent/DebuggableScheduledThreadPoolExecutor.java Modified: cassandra/branches/cassandra-0.7/CHANGES.txt cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java Modified: cassandra/branches/cassandra-0.7/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/CHANGES.txt?rev=1066930&r1=1066929&r2=1066930&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/CHANGES.txt (original) +++ cassandra/branches/cassandra-0.7/CHANGES.txt Thu Feb 3 20:01:04 2011 @@ -51,8 +51,6 @@ * avoid blocking gossip while deleting handoff hints (CASSANDRA-2073) * ignore messages from newer versions, keep track of nodes in gossip regardless of version (CASSANDRA-1970) - * log exceptions on scheduled executors (CASSANDRA-2061) - 0.7.0-final * fix offsets to ByteBuffer.get (CASSANDRA-1939) Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java?rev=1066930&r1=1066929&r2=1066930&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java Thu Feb 3 20:01:04 2011 @@ -75,15 +75,10 @@ public class DebuggableThreadPoolExecuto } } - @Override public void afterExecute(Runnable r, Throwable t) { super.afterExecute(r,t); - logTaskException(r, t); - } - static void logTaskException(Runnable r, Throwable t) - { // exceptions wrapped by FutureTask if (r instanceof FutureTask) { @@ -97,9 +92,7 @@ public class DebuggableThreadPoolExecuto } catch (ExecutionException e) { - if (Thread.getDefaultUncaughtExceptionHandler() == null) - logger.error("Error in executor task", e.getCause()); - else + if (Thread.getDefaultUncaughtExceptionHandler() != null) Thread.getDefaultUncaughtExceptionHandler().uncaughtException(Thread.currentThread(), e.getCause()); } } @@ -107,10 +100,7 @@ public class DebuggableThreadPoolExecuto // exceptions for non-FutureTask runnables [i.e., added via execute() instead of submit()] if (t != null) { - if (Thread.getDefaultUncaughtExceptionHandler() == null) - logger.error("Error in executor task", t); - else - Thread.getDefaultUncaughtExceptionHandler().uncaughtException(Thread.currentThread(), t); + logger.error("Error in ThreadPoolExecutor", t); } } } Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=1066930&r1=1066929&r2=1066930&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Thu Feb 3 20:01:04 2011 @@ -34,9 +34,9 @@ import org.apache.commons.collections.It import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor; import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor; import org.apache.cassandra.concurrent.NamedThreadFactory; +import org.apache.cassandra.concurrent.RetryingScheduledThreadPoolExecutor; import org.apache.cassandra.concurrent.StageManager; import org.apache.cassandra.config.CFMetaData; import org.apache.cassandra.config.ColumnDefinition; @@ -62,7 +62,7 @@ public class ColumnFamilyStore implement private static Logger logger = LoggerFactory.getLogger(ColumnFamilyStore.class); private static final ScheduledThreadPoolExecutor cacheSavingExecutor = - new DebuggableScheduledThreadPoolExecutor("CACHE-SAVER", Thread.MIN_PRIORITY); + new RetryingScheduledThreadPoolExecutor("CACHE-SAVER", Thread.MIN_PRIORITY); /* * submitFlush first puts [Binary]Memtable.getSortedContents on the flushSorter executor, Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java?rev=1066930&r1=1066929&r2=1066930&view=diff ============================================================================== --- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java (original) +++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/StorageService.java Thu Feb 3 20:01:04 2011 @@ -32,17 +32,17 @@ import javax.management.ObjectName; import com.google.common.base.Charsets; import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimap; + +import org.apache.cassandra.db.commitlog.CommitLog; +import org.apache.cassandra.locator.*; import org.apache.log4j.Level; import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor; -import org.apache.cassandra.concurrent.Stage; -import org.apache.cassandra.concurrent.StageManager; +import org.apache.cassandra.concurrent.*; import org.apache.cassandra.config.*; import org.apache.cassandra.db.*; -import org.apache.cassandra.db.commitlog.CommitLog; import org.apache.cassandra.db.migration.AddKeyspace; import org.apache.cassandra.db.migration.Migration; import org.apache.cassandra.dht.BootStrapper; @@ -52,10 +52,6 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.gms.*; import org.apache.cassandra.io.DeletionService; import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.locator.AbstractReplicationStrategy; -import org.apache.cassandra.locator.DynamicEndpointSnitch; -import org.apache.cassandra.locator.IEndpointSnitch; -import org.apache.cassandra.locator.TokenMetadata; import org.apache.cassandra.net.IAsyncResult; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; @@ -150,7 +146,7 @@ public class StorageService implements I }}; - public static final DebuggableScheduledThreadPoolExecutor scheduledTasks = new DebuggableScheduledThreadPoolExecutor("ScheduledTasks"); + public static final RetryingScheduledThreadPoolExecutor scheduledTasks = new RetryingScheduledThreadPoolExecutor("ScheduledTasks"); /* This abstraction maintains the token/endpoint metadata information */ private TokenMetadata tokenMetadata_ = new TokenMetadata();