You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Benjamin Roth (JIRA)" <ji...@apache.org> on 2016/09/30 08:05:21 UTC

[jira] [Commented] (CASSANDRA-12730) Thousands of empty SSTables created during repair - TMOF death

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

Benjamin Roth commented on CASSANDRA-12730:
-------------------------------------------

Kind of these logs are very frequent during repair:
DEBUG [STREAM-IN-/10.23.71.6:40687] 2016-09-30 07:43:14,472 ColumnFamilyStore.java:895 - Enqueuing flush of contact_like_in: 27.883MiB (1%) on-heap, 14.681MiB (0%) off-heap
DEBUG [PerDiskMemtableFlushWriter_0:3707] 2016-09-30 07:43:14,482 Memtable.java:465 - Writing Memtable-contact_like_in_sorted_mv@731070338(28.241MiB serialized bytes, 924784 ops, 2%/0% of on/off-heap limit), flushed range = (min(-9223372036854775808), max(9223372036854775807)]
DEBUG [PerDiskMemtableFlushWriter_0:3702] 2016-09-30 07:43:14,581 Memtable.java:465 - Writing Memtable-contact_like_in@737051553(14.832MiB serialized bytes, 183233 ops, 1%/0% of on/off-heap limit), flushed range = (min(-9223372036854775808), max(9223372036854775807)]
DEBUG [PerDiskMemtableFlushWriter_0:3702] 2016-09-30 07:43:15,799 Memtable.java:494 - Completed flushing /var/lib/cassandra/data/likes/contact_like_in-ecc5d0406ebd11e69a8bb79d91f44e40/mc-61473-big-Data.db (5.633MiB) for commitlog position CommitLogPosition(segmentId=1474982873663, position=26189216)
DEBUG [StreamReceiveTask:1302] 2016-09-30 07:43:15,844 ColumnFamilyStore.java:895 - Enqueuing flush of dislike_out: 7.670MiB (0%) on-heap, 2.796MiB (0%) off-heap
DEBUG [PerDiskMemtableFlushWriter_0:3706] 2016-09-30 07:43:16,263 Memtable.java:465 - Writing Memtable-dislike_out@322768716(3.909MiB serialized bytes, 88762 ops, 0%/0% of on/off-heap limit), flushed range = (min(-9223372036854775808), max(9223372036854775807)]
DEBUG [PerDiskMemtableFlushWriter_0:3706] 2016-09-30 07:43:16,629 Memtable.java:494 - Completed flushing /var/lib/cassandra/data/likes/dislike_out-00bb3c416eb711e69c111f9c71eb235f/mc-53978-big-Data.db (1.872MiB) for commitlog position CommitLogPosition(segmentId=1474982873663, position=26292737)
DEBUG [PerDiskMemtableFlushWriter_0:3707] 2016-09-30 07:43:17,412 Memtable.java:494 - Completed flushing /var/lib/cassandra/data/likes/contact_like_in_sorted_mv-eecc27906ebd11e6af14431d2ea5454d/mc-111097-big-Data.db (19.623MiB) for commitlog position CommitLogPosition(segmentId=1474982873663, position=26182562)
DEBUG [AntiEntropyStage:1] 2016-09-30 07:43:17,510 ColumnFamilyStore.java:895 - Enqueuing flush of contact_like_in: 6.598MiB (0%) on-heap, 3.512MiB (0%) off-heap
DEBUG [StreamReceiveTask:1303] 2016-09-30 07:43:17,522 ColumnFamilyStore.java:895 - Enqueuing flush of contact_like_in: 2.451KiB (0%) on-heap, 0.806KiB (0%) off-heap
DEBUG [PerDiskMemtableFlushWriter_0:3700] 2016-09-30 07:43:17,525 Memtable.java:465 - Writing Memtable-contact_like_in@778919401(3.494MiB serialized bytes, 42235 ops, 0%/0% of on/off-heap limit), flushed range = (min(-9223372036854775808), max(9223372036854775807)]
DEBUG [PerDiskMemtableFlushWriter_0:3704] 2016-09-30 07:43:17,545 Memtable.java:465 - Writing Memtable-contact_like_in@1722999255(0.942KiB serialized bytes, 13 ops, 0%/0% of on/off-heap limit), flushed range = (min(-9223372036854775808), max(9223372036854775807)]
DEBUG [PerDiskMemtableFlushWriter_0:3704] 2016-09-30 07:43:17,545 Memtable.java:494 - Completed flushing /var/lib/cassandra/data/likes/contact_like_in-ecc5d0406ebd11e69a8bb79d91f44e40/mc-61478-big-Data.db (0.477KiB) for commitlog position CommitLogPosition(segmentId=1474982873663, position=26412231)

I guess, there is normally nothing bad about them, but you can see that VERY small SSTables are written. Most probably this is triggered by a stream during repair. I am not that deep into the code but from the outside (and thats also was StreamSession.prepare says) it looks like memtables are flushed for every little stream. Is this REALLY necessary?
When repairing a big table with a lot of small ranges and partitions, this of course will create tons of memtables and a lot of compaction.

If the node has enough time for compaction during repair, these tables are compacted away again, but under heavy load, this induced several severe issues:
1. FDs just may run out, happened to me
2. More resources needed to compact all the tiny SSTables
3. Even more load on the node for reads as it is impossibly efficient to read from hundrets, thousands or even ten thousands of TINY sstables

=> If a tipping point is reached, this may trigger a death spiral for the node (which actually happened to me).

One could argue that it is not good to put so much repair pressure on a node that it tends to be overloaded, but I think it is a much better solution to choose a safer design (and to take care of the node's load).

> Thousands of empty SSTables created during repair - TMOF death
> --------------------------------------------------------------
>
>                 Key: CASSANDRA-12730
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-12730
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Local Write-Read Paths
>            Reporter: Benjamin Roth
>            Priority: Critical
>
> Last night I ran a repair on a keyspace with 7 tables and 4 MVs each containing a few hundret million records. After a few hours a node died because of "too many open files".
> Normally one would just raise the limit, but: We already set this to 100k. The problem was that the repair created roughly over 100k SSTables for a certain MV. The strange thing is that these SSTables had almost no data (like 53bytes, 90bytes, ...). Some of them (<5%) had a few 100 KB, very few (<1% had normal sizes like >= few MB). I could understand, that SSTables queue up as they are flushed and not compacted in time but then they should have at least a few MB (depending on config and avail mem), right?
> Of course then the node runs out of FDs and I guess it is not a good idea to raise the limit even higher as I expect that this would just create even more empty SSTables before dying at last.
> Only 1 CF (MV) was affected. All other CFs (also MVs) behave sanely. Empty SSTables have been created equally over time. 100-150 every minute. Among the empty SSTables there are also Tables that look normal like having few MBs.
> I didn't see any errors or exceptions in the logs until TMOF occured. Just tons of streams due to the repair (which I actually run over cs-reaper as subrange, full repairs).
> After having restarted that node (and no more repair running), the number of SSTables went down again as they are compacted away slowly.
> According to [~zznate] this issue may relate to CASSANDRA-10342 + CASSANDRA-8641



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)