[ 
https://issues.apache.org/jira/browse/CASSANDRA-13969?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16210966#comment-16210966
 ] 

Marcus Olsson commented on CASSANDRA-13969:
-------------------------------------------

When looking at the code it seems to be the thread pool initialized 
[here|https://github.com/apache/cassandra/blob/cassandra-2.2/src/java/org/apache/cassandra/repair/RepairRunnable.java#L220]
 (based on the name "Repair#XYZ"). This thread pool is shutdown using 
[executor.shutdownNow()|https://github.com/apache/cassandra/blob/cassandra-2.2/src/java/org/apache/cassandra/repair/RepairRunnable.java#L348]
 when the repair is finished, which is the only place I could find where the 
threads running in it would be interrupted.

Based on the exception it seems like the thread is interrupted after having 
completed it's task, in the afterExcecute()-method of the thread pool. The task 
running in this thread pool is an instance of RepairJob, which spawns several 
subtasks in a separate thread pool. These subtasks include sending snapshot 
requests, validation requests and sync requests. Each of these tasks wait for 
the previous one to complete while the RepairJob is only waiting for the 
validation requests to complete.

Based on the structure of things it seems like the thread pool where an 
exception is thrown is shutdown by the thread pool executing the subtasks, 
after all subtasks have been completed. I believe that a race condition could 
happen when the sync-task is basically a no-op and happens to trigger the chain 
of events that leads to the shutdownNow() before the afterExecute() of the 
RepairJob has a chance to finish.

---

I have created a simplified scenario as described above (outside of Cassandra) 
and changing the shutdownNow() to a shutdown() seems to fix that situation. I 
have not yet found a good/easy way to test this behaviour in unit tests, so if 
anyone has any idea that would be great.

One question here is if the shutdownNow() is necessary for i.e. making sure 
that threads are cleaned up properly in case of a failure? Looking at 
RepairSession it seems like the sub thread pool is using 
[shutdown()|https://github.com/apache/cassandra/blob/cassandra-2.2/src/java/org/apache/cassandra/repair/RepairSession.java#L270]
 rather than shutdownNow(), so this might not be a problem?

> InterruptedException while running repair
> -----------------------------------------
>
>                 Key: CASSANDRA-13969
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13969
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Repair
>         Environment: Cassandra 2.2.10, sub-range repairs
>            Reporter: Marcus Olsson
>            Priority: Minor
>
> In one of our test clusters we observed the following error in system.log:
> {noformat}
> 2017-10-12T15:55:25.617+0200 ERROR [Repair#34:1] CassandraDaemon.java:195 
> Exception in thread Thread[Repair#34:1,5,RMI Runtime]
> java.lang.AssertionError: java.lang.InterruptedException
>         at 
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.extractThrowable(DebuggableThreadPoolExecutor.java:265)
>  ~[apache-cassandra-2.2.10.jar:2.2.10]
>         at 
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.logExceptionsAfterExecute(DebuggableThreadPoolExecutor.java:225)
>  ~[apache-cassandra-2.2.10.jar:2.2.10]
>         at 
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.afterExecute(DebuggableThreadPoolExecutor.java:196)
>  ~[apache-cassandra-2.2.10.jar:2.2.10]
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1150)
>  ~[na:1.8.0_131]
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  ~[na:1.8.0_131]
>         at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_131]
> Caused by: java.lang.InterruptedException: null
>         at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1302)
>  ~[na:1.8.0_131]
>         at 
> com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:285)
>  ~[guava-16.0.jar:na]
>         at 
> com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:116) 
> ~[guava-16.0.jar:na]
>         at 
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.extractThrowable(DebuggableThreadPoolExecutor.java:261)
>  ~[apache-cassandra-2.2.10.jar:2.2.10]
>         ... 5 common frames omitted
> {noformat}
> Except for the exception there is no other odd outputs in system.log.
> From the repair history there is no indication of failed repairs and our 
> repair handler doesn't get any errors reported back through the progress 
> reporting either. One thing to note is that we utilize sub-range repairs and 
> repair one vnode at a time, which means that we effectively run several 
> hundreds of repair sessions for each table.
> From our repair handler the following is written in the logs:
> {noformat}
> 2017-10-12T15:55:25.611+0200 | INFO  | Repair of <keyspace>.<table> - 
> [(8922822608060820611,8928269034264081622]] completed successfully
> 2017-10-12T15:55:25.678+0200 | INFO  | Repair of <keyspace>.<table> - 
> [(-5406027845309604779,-5405899934869332173]] completed successfully
> 2017-10-12T15:55:25.744+0200 | INFO  | Repair of <keyspace>.<table> - 
> [(1498725784389153529,1509146082320230540]] completed successfully
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to