[Public] Hi all,
(My previous mail is not appearing in mailing list and resending again after 2 days) Myself Amit and working at AMD Bangalore, India. I am new to Cassandra and need to do Cassandra testing on large core systems. Usually should test on multi-nodes Cassandra but started with Single node testing to understand how Cassandra scales with increasing core counts. Test details: Operation: Insert > 90% (insert heavy) Operation: Scan < 10% Cassandra: 3.11.10 and trunk Benchmark: TPCx-IOT (similar to YCSB) Results shows scaling is poor beyond 16 cores and it is almost linear. Following settings are the common settings helped to get the better scores. 1. Memtable heap allocation: offheap_objects 2. memtable_flush_writers > 4 3. Java heap: 8-32GB with survivor ratio tuning 4. Separate storage space for Commitlog and Data. Many online blogs suggest to add new Cassandra node when unable to take high writes. But with large systems, high writes should be easily taken due to many cores. Need was to improve the scaling with more cores so this suggestion didn't help. After many rounds of testing it was observed that current implementation uses single thread for Commitlog syncing activity. Commitlog files are mapped using mmap system call and changes are written with msync. Periodic syncing with JVisualvm tool shows 1. thread is not 100% busy with Ramdisk usage for Commitlog storage and scaling improved on large systems. Ramdisk scores > 2 X NVME score. 2. thread becomes 100% busy with NVME usage for Commiglog and score does not improve much beyond 16 cores. Linux kernel uses 4K pages for mapped memory with mmap system call. So, to understand this further, disk I/O testing was done using FIO tool and results shows 1. NVME 4K random R/W throughput is very less with single thread and it improves with multi-threaded. 2. Ramdisk 4K random R/W throughput is good with single thread only and also better with multi-threaded Based on the FIO test results following two ideas were tested for Commitlog files with Cassandra-3.1.10 sources. 1. Enable Direct IO feature for Commitlog files (similar to [CASSANDRA-14466] Enable Direct I/O - ASF JIRA (apache.org)<https://issues.apache.org/jira/browse/CASSANDRA-14466> ) 2. Enable Multi-threaded syncing for Commitlog files. First one need to retest. Interestingly second one helped to improve the score with "NVME" disk. NVME disk configuration score is almost within 80-90% of ramdisk and 2 times of single threaded implementation. Multithreading enabled by adding new thread pool in "AbstractCommitLogSegmentManager" class and changed syncing thread as manager thread for this new thread pool to take care synchronization. Only tested with Cassandra-3.11.10 and needs complete testing but this change is working in my test environment. Tried these few experiments so that I could discuss here and seek your valuable suggestions to identify the right fix for insert heavy workloads. 1. Is it good idea to convert single threaded syncing to multi-threading implementation to improve the disk IO? 2. Direct I/O throughput is high with single thread and best fit for Commitlog case due to file size. This will improve writes on small to large systems. Good to bring this support for Commitlog files? Please suggest. Thanks, Amit Pawar