Benedict created CASSANDRA-7554:
-----------------------------------

             Summary: Make CommitLogSegment sync/close asynchronous wrt each 
other
                 Key: CASSANDRA-7554
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7554
             Project: Cassandra
          Issue Type: Improvement
          Components: Core
            Reporter: Benedict
            Assignee: Benedict
             Fix For: 2.1.1


There are a few minor issues with CLS I wanted to tidy up after working on 
nearby code a bit recently, namely:

1) We use synchronized() for sync() and for various minor accessors, meaning 
either can block on the other, which is bad since sync() is lengthy
2) Currently close() (and hence recycle()) must wait for a sync() to complete, 
which means even if we have room available in segments waiting to be recycled 
an ongoing sync might prevent us from reclaiming the space, prematurely 
bottlenecking on the disk here
3) recycle() currently depends on close(), which depends on sync(); if we've 
decided to recycle/close a file before it is synced, this means we do not care 
about the contents so can actually _avoid_ syncing to disk (which is great in 
cases where the flush writers get ahead of the CL sync)

To solve these problems I've introduced a new fairly simple concurrency 
primitive called AsyncLock, which only supports tryLock(), or tryLock(Runnable) 
- with the latter executing the provided runnable on the thread _currently 
owning the lock_ after it relinquishes it. I've used this to make close() take 
a Runnable to be executed _when the segment is actually ready to be disposed 
of_ - which is either immediately, or once any in progress sync has completed. 
This means the manager thread never blocks on a sync.

There is a knock on effect here, which is that we are even less inclined to 
obey the CL limit (which has always been a soft limit), so I will file a 
separate minor ticket to introduce a hard limit for CL size in case users want 
to control this.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to