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

Stefan Miklosovic commented on CASSANDRA-19429:
-----------------------------------------------

I created this branch where I incorporated the idea above with more changes 
related to it (1). For example, we should no call getCapacity() in tracing log 
messages, we should firstly check if we are going to log on trace level and 
then construct log message. If we are not, then we call getCapacity() but we 
just throw it away. I think that in practice we are not logging on trace at all 
so this is just redundant.

When we check DatabaseDescriptor.getKeyCacheSizeInMiB(), if we change capacity 
of these caches via CacheServiceMBean, then it would have non-zero capacity but 
we never set it in DatabaseDescriptor. I covered this too, DatabaseDescriptor 
values are updated on CacheServiceMBean method calls too.

(1) https://github.com/apache/cassandra/pull/3133/files

[~dipiets] [~brandon.williams] does this make sense to you? [~dipiets] Could 
you please run your tests again with the (1) patch and check the numbers?

> Remove lock contention generated by getCapacity function in SSTableReader
> -------------------------------------------------------------------------
>
>                 Key: CASSANDRA-19429
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-19429
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Local/SSTable
>            Reporter: Dipietro Salvatore
>            Assignee: Dipietro Salvatore
>            Priority: Normal
>             Fix For: 4.0.x, 4.1.x
>
>         Attachments: asprof_cass4.1.3__lock_20240216052912lock.html
>
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> Profiling Cassandra 4.1.3 on large AWS instances, a high number of lock 
> acquires is measured in the `getCapacity` function from 
> `org/apache/cassandra/cache/InstrumentingCache` (1.9M lock acquires per 60 
> seconds). Based on our tests on r8g.24xlarge instances (using Ubuntu 22.04), 
> this limits the CPU utilization of the system to under 50% when testing at 
> full load and therefore limits the achieved throughput.
> Removing the lock contention from the SSTableReader.java file by replacing 
> the call to `getCapacity` with `size` achieves up to 2.95x increase in 
> throughput on r8g.24xlarge and 2x on r7i.24xlarge:
> |Instance type|Cass 4.1.3|Cass 4.1.3 patched|
> |r8g.24xlarge|168k ops|496k ops (2.95x)|
> |r7i.24xlarge|153k ops|304k ops (1.98x)|
>  
> Instructions to reproduce:
> {code:java}
> ## Requirements for Ubuntu 22.04
> sudo apt install -y ant git openjdk-11-jdk
> ## Build and run
> CASSANDRA_USE_JDK11=true ant realclean && CASSANDRA_USE_JDK11=true ant jar && 
> CASSANDRA_USE_JDK11=true ant stress-build  && rm -rf data && bin/cassandra -f 
> -R
> # Run
> bin/cqlsh -e 'drop table if exists keyspace1.standard1;' && \
> bin/cqlsh -e 'drop keyspace if exists keyspace1;' && \
> bin/nodetool clearsnapshot --all && tools/bin/cassandra-stress write 
> n=10000000 cl=ONE -rate threads=384 -node 127.0.0.1 -log file=cload.log 
> -graph file=cload.html && \
> bin/nodetool compact keyspace1   && sleep 30s && \
> tools/bin/cassandra-stress mixed ratio\(write=10,read=90\) duration=10m 
> cl=ONE -rate threads=406 -node localhost -log file=result.log -graph 
> file=graph.html
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to