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

Peter Schuller edited comment on CASSANDRA-4032 at 3/9/12 4:05 PM:
-------------------------------------------------------------------

{quote}
Are we sure that what we want is a SynchronousQueue with task rejected? After 
all, there is only on global memoryMeter, so we could end up failing to 
updateLiveRatio just based on a race, even if calculations are fast. I'd 
suggest instead a bounded queue (but maybe not infinite and we could indeed 
just skip task if that queue gets full).
{quote}

I agree it's fishy, though I'd suggest a separate ticket. This patch is 
intended to make the code behave the way the original commit intended.

This (from the code, not my patch) seems legit though:

{code}
    // we're careful to only allow one count to run at a time because counting 
is slow
    // (can be minutes, for a large memtable and a busy server), so we could 
keep memtables
    // alive after they're flushed and would otherwise be GC'd.
{code}

We could have one queue per unique CF and have a consumer that iterates over 
the set of queues, guaranteeing that each CF gets processed once per "cycle". A 
simpler solution is probably preferable though if we can think of one.

                
      was (Author: scode):
    {code}
Are we sure that what we want is a SynchronousQueue with task rejected? After 
all, there is only on global memoryMeter, so we could end up failing to 
updateLiveRatio just based on a race, even if calculations are fast. I'd 
suggest instead a bounded queue (but maybe not infinite and we could indeed 
just skip task if that queue gets full).
{code}

I agree it's fishy, though I'd suggest a separate ticket. This patch is 
intended to make the code behave the way the original commit intended.

This (from the code, not my patch) seems legit though:

{code}
    // we're careful to only allow one count to run at a time because counting 
is slow
    // (can be minutes, for a large memtable and a busy server), so we could 
keep memtables
    // alive after they're flushed and would otherwise be GC'd.
{code}

We could have one queue per unique CF and have a consumer that iterates over 
the set of queues, guaranteeing that each CF gets processed once per "cycle". A 
simpler solution is probably preferable though if we can think of one.

                  
> memtable.updateLiveRatio() is blocking, causing insane latencies for writes
> ---------------------------------------------------------------------------
>
>                 Key: CASSANDRA-4032
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-4032
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>            Reporter: Peter Schuller
>            Assignee: Peter Schuller
>             Fix For: 1.1.0
>
>         Attachments: CASSANDRA-4032-1.1.0-v1.txt
>
>
> Reproduce by just starting a fresh cassandra with a heap large enough for 
> live ratio calculation (which is {{O(n)}}) to be insanely slow, and then 
> running {{./bin/stress -d host -n100000000 -t10}}. With a large enough heap 
> and default flushing behavior this is bad enough that stress gets timeouts.
> Example ("blocked for" is my debug log added around submit()):
> {code}
>  INFO [MemoryMeter:1] 2012-03-09 15:07:30,857 Memtable.java (line 198) 
> CFS(Keyspace='Keyspace1', ColumnFamily='Standard1') liveRatio is 
> 8.89014894083727 (just-counted was 8.89014894083727).  calculation took 
> 28273ms for 1320245 columns
>  WARN [MutationStage:8] 2012-03-09 15:07:30,857 Memtable.java (line 209) 
> submit() blocked for: 231135
> {code}
> The calling code was written assuming a RejectedExecutionException is thrown, 
> but it's not because {{DebuggableThreadPoolExecutor}} installs a blocking 
> rejection handler.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to