Caleb William Rackliffe created CASSANDRA-11062:
---------------------------------------------------
Summary: BatchlogManager May Attempt to Flush Hints After
HintsService is Shutdown
Key: CASSANDRA-11062
URL: https://issues.apache.org/jira/browse/CASSANDRA-11062
Project: Cassandra
Issue Type: Bug
Reporter: Caleb William Rackliffe
Priority: Minor
Fix For: 3.0.x
{{ScheduledThreadPoolExecutor}}'s default behavior is to keep running delayed
tasks after shutdown, so I think that means {{BatchlogManager}} is trying to
call {{replayFailedBatches()}} after drain has instructed both it and the
{{HintsService}} to shut down. When this happens, we get an exception when that
tries to submit a task to the executor in {{HintsWriteExecutor}}:
{noformat}
ERROR [BatchlogTasks:1] 2016-01-22 17:01:38,936 CassandraDaemon.java:195 -
Exception in thread Thread[BatchlogTasks:1,5,main]
java.util.concurrent.RejectedExecutionException: ThreadPoolExecutor has shut
down
at
org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$1.rejectedExecution(DebuggableThreadPoolExecutor.java:61)
~[cassandra-all-3.0.1.816.jar:3.0.1.816]
at
java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:823)
[na:1.8.0_65]
at
java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1369)
[na:1.8.0_65]
at
org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.execute(DebuggableThreadPoolExecutor.java:165)
~[cassandra-all-3.0.1.816.jar:3.0.1.816]
at
java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:112)
~[na:1.8.0_65]
at
org.apache.cassandra.hints.HintsWriteExecutor.flushBufferPool(HintsWriteExecutor.java:89)
~[cassandra-all-3.0.1.816.jar:3.0.1.816]
at
org.apache.cassandra.hints.HintsService.flushAndFsyncBlockingly(HintsService.java:177)
~[cassandra-all-3.0.1.816.jar:3.0.1.816]
at
org.apache.cassandra.batchlog.BatchlogManager.processBatchlogEntries(BatchlogManager.java:259)
~[cassandra-all-3.0.1.816.jar:3.0.1.816]
at
org.apache.cassandra.batchlog.BatchlogManager.replayFailedBatches(BatchlogManager.java:200)
~[cassandra-all-3.0.1.816.jar:3.0.1.816]
at
org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118)
~[cassandra-all-3.0.1.816.jar:3.0.1.816]
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
[na:1.8.0_65]
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
[na:1.8.0_65]
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
[na:1.8.0_65]
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
[na:1.8.0_65]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
[na:1.8.0_65]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
[na:1.8.0_65]
{noformat}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)