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

Stu Hood commented on CASSANDRA-1930:
-------------------------------------

I think there is more going on here: while swapping the log entries around, we 
noticed that when a Memtable reaches its threshold, up to (all) N write threads 
will often notice, and race to acquire the writeLock in order to freeze the 
memtable. This means that we do way more writeLock acquisitions than we need to.

This is probably just more reason to replace this lock.

> db.Table flusherLock write lock fairness policy is sub-optimal
> --------------------------------------------------------------
>
>                 Key: CASSANDRA-1930
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1930
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Core
>            Reporter: Kelvin Kakugawa
>            Assignee: Kelvin Kakugawa
>             Fix For: 0.8
>
>         Attachments: CASSANDRA-1930.0001.patch
>
>
> h4. scenario:
> 1) high write throughput cluster
> 2) external services adding material cpu contention
> h4. symptoms:
> The row mutation stage falls *very* behind.
> h4. cause:
> ReentrantReadWriteLock's fair policy causes write lock acquisition / release 
> to require a material amount of CPU time.
> h4. summary:
> When there are other services contending for the CPU, the RRW lock's fair 
> policy causes write lock acquisition / release to take enough time to 
> eventually put threads waiting for read lock acquisition very behind.  We 
> repro'd this scenario by reducing the scope of the write lock to: 1 boolean 
> check, 1 boolean assignment, and 1 db.Memtable instantiation (itself, just: 2 
> variable assignments) w/ the same performance.  Modifying the fairness policy 
> to be the default policy allowed the row mutation stage to keep up.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to