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

Dipietro Salvatore commented on CASSANDRA-19429:
------------------------------------------------

My initial patch and test are based on this patch 
(https://github.com/salvatoredipietro/cassandra/commit/45f770dd8e06e490a4cd0f222e1d4a3060a45a38):


{code:java}
>From 45f770dd8e06e490a4cd0f222e1d4a3060a45a38 Mon Sep 17 00:00:00 2001
From: Salvatore Dipietro <dipi...@amazon.com>
Date: Wed, 21 Feb 2024 16:29:15 -0800
Subject: [PATCH] Remove lock contention generated by getCapacity function in
 SSTableReader

---
 CHANGES.txt                                                   | 1 +
 .../org/apache/cassandra/io/sstable/format/SSTableReader.java | 4 ++--
 2 files changed, 3 insertions(+), 2 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 21084b8c721f..4ab8ccf21b3e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.1.5
+ * Remove lock contention generated by getCapacity function in SSTableReader
 Merged from 4.0:
  * Remove bashisms for mx4j tool in cassandra-env.sh (CASSANDRA-19416)
 Merged from 3.11:
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java 
b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index f26cf65c93e0..3b703911984d 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -520,7 +520,7 @@ public static SSTableReader open(Descriptor descriptor,
                 sstable.validate();
 
             if (sstable.getKeyCache() != null)
-                logger.trace("key cache contains {}/{} keys", 
sstable.getKeyCache().size(), sstable.getKeyCache().getCapacity());
+                logger.trace("key cache contains {} keys", 
sstable.getKeyCache().size());
 
             return sstable;
         }
@@ -717,7 +717,7 @@ public void setupOnline()
         // e.g. by BulkLoader, which does not initialize the cache.  As a 
kludge, we set up the cache
         // here when we know we're being wired into the rest of the server 
infrastructure.
         InstrumentingCache<KeyCacheKey, RowIndexEntry> maybeKeyCache = 
CacheService.instance.keyCache;
-        if (maybeKeyCache.getCapacity() > 0)
+        if (maybeKeyCache.size() > 0)
             keyCache = maybeKeyCache;
 
         final ColumnFamilyStore cfs = 
Schema.instance.getColumnFamilyStoreInstance(metadata().id);
 {code}

However, it seem that the `keyCache` variable remains set to `null`. 

Based on my experiments, I can see that `maybeKeyCache.getCapacity()` always 
returns a fixed long number for all its calls and it doesn't change over time.
Can someone please advice under which circumstances it is necessary to check 
that `maybeKeyCache.getCapacity() > 0` or if it is always possible to set it 
like ` keyCache = CacheService.instance.keyCache;` ?

> 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
>
>
> 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