[ 
https://issues.apache.org/jira/browse/CASSANDRA-1930?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jonathan Ellis resolved CASSANDRA-1930.
---------------------------------------

       Resolution: Fixed
    Fix Version/s: 0.7.1
         Reviewer: jbellis

bq. The reason we originally switched to a fair policy was, without it there is 
no guarantee that the writer (the flushing thread) will ever get the lock, and 
we were seeing OOMing from the memtable exhausting memory instead of getting 
switched out. 

This reasoning is obsolete; as Stu points out, we try to flush whenever we do a 
write that violates the memtable threshold.  So you should see at most 
concurrent-writes extra updates before the flush.

Committed, with comment on RRWL instantiation.

(Still open to another ticket for a new approach entirely, but this fix seems 
better than the status quo.)

> 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.7.1, 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