Dmitry Konstantinov created CASSANDRA-20465:
-----------------------------------------------

             Summary: Reduce runtime overhead of 
org.apache.cassandra.schema.TableMetadataRef#get usage 
                 Key: CASSANDRA-20465
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-20465
             Project: Apache Cassandra
          Issue Type: Improvement
          Components: Cluster/Schema, Transactional Cluster Metadata
            Reporter: Dmitry Konstantinov
         Attachments: 5.1_cpu.html, image-2025-03-20-22-43-09-044.png, 
image-2025-03-20-22-46-34-571.png, image-2025-03-20-22-52-40-818.png, 
image-2025-03-20-22-53-25-001.png, image-2025-03-20-22-56-31-298.png, 
image-2025-03-20-22-58-00-837.png

Before TCM changes org.apache.cassandra.schema.TableMetadataRef#get invocation 
cheap (it just returned a field value), now it does a lookup from Schema every 
time:
!image-2025-03-20-22-43-09-044.png|width=300!

We have several places in code which uses TableMetadataRef#get assuming low 
cost.
So, currently we have about 0.93% of CPU spent for this operation in total, if 
we check only compaction + flush - 5.4% ( [^5.1_cpu.html] ).

Not sure if it is easy to reduce overheads in TableMetadataRef#get itself but 
we also can avoid them in many cases by a small adjustment of a logic on an 
invoker side:

1) org.apache.cassandra.db.ColumnFamilyStore#isRowCacheEnabled - by default row 
cache is fully disabled - probably it is better to check if it is enabled as a 
first condition:
!image-2025-03-20-22-46-34-571.png|width=300!

2) org.apache.cassandra.db.memtable.TrieMemtable#getFlushSet - we can lookup 
metadata once at the beginning of getFlushSet logic

!image-2025-03-20-22-52-40-818.png|width=300! 
!image-2025-03-20-22-53-25-001.png|width=300!

3) org.apache.cassandra.io.sstable.SSTableIdentityIterator.create - to think if 
we can keep TableMetadata during a compaction..
!image-2025-03-20-22-56-31-298.png|width=300!

4) org.apache.cassandra.io.sstable.keycache.KeyCacheSupport.getCacheKey - to 
think if we can retrieve only needed id/indexName fields once ..
!image-2025-03-20-22-58-00-837.png|width=300!



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