[
https://issues.apache.org/jira/browse/CASSANDRA-7949?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14208154#comment-14208154
]
Nikolai Grigoriev commented on CASSANDRA-7949:
----------------------------------------------
I had to rebuild one of the nodes in that test cluster. After bootstrapping it
I have checked the results - I had over 6,5K pending compactions and many large
sstables (between few Gb and 40-60Gb). I knew that under traffic this will
*never* return to reasonable number of pending compactions.
I have decided to give it another try, enable the option from CASSANDRA-6621
and re-bootstrap. This time I did not end up with huge sstables but, I think,
it will also never recover. This is, essentially, what the node does most of
the time:
{code}
pending tasks: 7217
compaction type keyspace table completed
total unit progress
Compaction myks mytable1 5434997373
10667184206 bytes 50.95%
Compaction myks mytable2 1080506914
7466286503 bytes 14.47%
Active compaction remaining time : 0h00m09s
{code}
while:
{code}
# nodetool cfstats myks.mytable1
Keyspace: myks
Read Count: 49783
Read Latency: 38.612470602414476 ms.
Write Count: 521971
Write Latency: 1.3617571608384373 ms.
Pending Tasks: 0
Table: mytable1
SSTable count: 7893
SSTables in each level: [7828/4, 10, 56, 0, 0, 0, 0, 0, 0]
Space used (live), bytes: 1181508730955
Space used (total), bytes: 1181509085659
SSTable Compression Ratio: 0.3068450302663634
Number of keys (estimate): 28180352
Memtable cell count: 153554
Memtable data size, bytes: 41190431
Memtable switch count: 178
Local read count: 49826
Local read latency: 38.886 ms
Local write count: 522464
Local write latency: 1.392 ms
Pending tasks: 0
Bloom filter false positives: 11802553
Bloom filter false ratio: 0.98767
Bloom filter space used, bytes: 17686928
Compacted partition minimum bytes: 104
Compacted partition maximum bytes: 3379391
Compacted partition mean bytes: 142171
Average live cells per slice (last five minutes): 537.5
Average tombstones per slice (last five minutes): 0.0
{code}
By the way, this is the picture from another node that functions normally:
{code}
# nodetool cfstats myks.mytable1
Keyspace: myks
Read Count: 4638154
Read Latency: 20.784106776316612 ms.
Write Count: 15067667
Write Latency: 1.7291775639188205 ms.
Pending Tasks: 0
Table: mytable1
SSTable count: 4561
SSTables in each level: [37/4, 15/10, 106/100, 1053/1000, 3350,
0, 0, 0, 0]
Space used (live), bytes: 1129716897255
Space used (total), bytes: 1129752918759
SSTable Compression Ratio: 0.33488717551698993
Number of keys (estimate): 25036672
Memtable cell count: 334212
Memtable data size, bytes: 115610737
Memtable switch count: 4476
Local read count: 4638155
Local read latency: 20.784 ms
Local write count: 15067679
Local write latency: 1.729 ms
Pending tasks: 0
Bloom filter false positives: 104377
Bloom filter false ratio: 0.59542
Bloom filter space used, bytes: 20319608
Compacted partition minimum bytes: 104
Compacted partition maximum bytes: 3379391
Compacted partition mean bytes: 152368
Average live cells per slice (last five minutes): 529.5
Average tombstones per slice (last five minutes): 0.0
{code}
So, not only the streaming has created an excessive amount of sstables, the
compactions are not advancing at all. In fact, the number of pending
compactions grows slowly on that (first) node. New L0 sstables get added
because the write activity is taking place.
Just a simple math. If I take the compaction throughput of the node when it
uses only one thread and compare it to my write rate I think the latter is like
4x the former. Under this conditions this node will never recover - while
having plenty of resources and very fast I/O.
> LCS compaction low performance, many pending compactions, nodes are almost
> idle
> -------------------------------------------------------------------------------
>
> Key: CASSANDRA-7949
> URL: https://issues.apache.org/jira/browse/CASSANDRA-7949
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Environment: DSE 4.5.1-1, Cassandra 2.0.8
> Reporter: Nikolai Grigoriev
> Attachments: iostats.txt, nodetool_compactionstats.txt,
> nodetool_tpstats.txt, pending compactions 2day.png, system.log.gz, vmstat.txt
>
>
> I've been evaluating new cluster of 15 nodes (32 core, 6x800Gb SSD disks +
> 2x600Gb SAS, 128Gb RAM, OEL 6.5) and I've built a simulator that creates the
> load similar to the load in our future product. Before running the simulator
> I had to pre-generate enough data. This was done using Java code and DataStax
> Java driver. To avoid going deep into details, two tables have been
> generated. Each table currently has about 55M rows and between few dozens and
> few thousands of columns in each row.
> This data generation process was generating massive amount of non-overlapping
> data. Thus, the activity was write-only and highly parallel. This is not the
> type of the traffic that the system will have ultimately to deal with, it
> will be mix of reads and updates to the existing data in the future. This is
> just to explain the choice of LCS, not mentioning the expensive SSD disk
> space.
> At some point while generating the data I have noticed that the compactions
> started to pile up. I knew that I was overloading the cluster but I still
> wanted the genration test to complete. I was expecting to give the cluster
> enough time to finish the pending compactions and get ready for real traffic.
> However, after the storm of write requests have been stopped I have noticed
> that the number of pending compactions remained constant (and even climbed up
> a little bit) on all nodes. After trying to tune some parameters (like
> setting the compaction bandwidth cap to 0) I have noticed a strange pattern:
> the nodes were compacting one of the CFs in a single stream using virtually
> no CPU and no disk I/O. This process was taking hours. After that it would be
> followed by a short burst of few dozens of compactions running in parallel
> (CPU at 2000%, some disk I/O - up to 10-20%) and then getting stuck again for
> many hours doing one compaction at time. So it looks like this:
> # nodetool compactionstats
> pending tasks: 3351
> compaction type keyspace table completed
> total unit progress
> Compaction myks table_list1 66499295588
> 1910515889913 bytes 3.48%
> Active compaction remaining time : n/a
> # df -h
> ...
> /dev/sdb 1.5T 637G 854G 43% /cassandra-data/disk1
> /dev/sdc 1.5T 425G 1.1T 29% /cassandra-data/disk2
> /dev/sdd 1.5T 429G 1.1T 29% /cassandra-data/disk3
> # find . -name **table_list1**Data** | grep -v snapshot | wc -l
> 1310
> Among these files I see:
> 1043 files of 161Mb (my sstable size is 160Mb)
> 9 large files - 3 between 1 and 2Gb, 3 of 5-8Gb, 55Gb, 70Gb and 370Gb
> 263 files of various sized - between few dozens of Kb and 160Mb
> I've been running the heavy load for about 1,5days and it's been close to 3
> days after that and the number of pending compactions does not go down.
> I have applied one of the not-so-obvious recommendations to disable
> multithreaded compactions and that seems to be helping a bit - I see some
> nodes started to have fewer pending compactions. About half of the cluster,
> in fact. But even there I see they are sitting idle most of the time lazily
> compacting in one stream with CPU at ~140% and occasionally doing the bursts
> of compaction work for few minutes.
> I am wondering if this is really a bug or something in the LCS logic that
> would manifest itself only in such an edge case scenario where I have loaded
> lots of unique data quickly.
> By the way, I see this pattern only for one of two tables - the one that has
> about 4 times more data than another (space-wise, number of rows is the
> same). Looks like all these pending compactions are really only for that
> larger table.
> I'll be attaching the relevant logs shortly.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)