Hello,

I found possible deadlock in AbstractCommitLogSegmentManager. The root
cause is incorrect use of LockSupport.park/unpark pair. Unpark should be
invoked only if caller is sure that thread was parked in appropriate place.
Otherwice permission given by calling unpark can be consumed by other
structures (for example - inside ReadWriteLock).

Jira: https://issues.apache.org/jira/browse/CASSANDRA-13652

I suggest simplest solution: change LockSupport to Semaphore.
PR: https://github.com/apache/cassandra/pull/127

Also I suggest another solution with SynchronousQueue-like structure to
move available segment from Manager Thread to consumers. With theese
changes code became more clear and
straightforward.

PR https://github.com/apache/cassandra/pull/129

We can not use j.u.c.SynchronousQueue because we need to support shutdown
and there is only way to terminate SynchronousQueue.put is to call
Thread.interrupt(). But C* uses nio and it does not
expect ClosedByInterruptException during IO operations. Thus we can not
interrupt Manager Thread.
I implemented o.a.c.u.c.Transferer that supports shutdown and restart
(needed for tests).
https://github.com/Fuud/cassandra/blob/e1a695874dc24e532ae21ef627e852bf999a75f3/src/java/org/apache/cassandra/utils/concurrent/Transferer.java

Also I modified o.a.c.d.c.SimpleCachedBufferPool to support waiting for
free space.

Please feel free to ask any questions.

Thank you.

Feodor Bobin
fuudtorrent...@gmail.com

Reply via email to