[
https://issues.apache.org/jira/browse/CASSANDRA-12730?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15648092#comment-15648092
]
Paulo Motta commented on CASSANDRA-12730:
-----------------------------------------
bq. Once the mutations have been applied to memtable and flushed to disk, the
resulting sstables will not be flaged with a repairedAt timestamp. The next
repair process will pick up from there and "repair" the flushed sstables again
back to the other nodes, as the rows can't be found in the unrepaired set
there. This will go back and forth and each repair inconsistency found will
probably further aggravate the issue.
Good catch [[email protected]], I just noticed this yesterday while reviewing
CASSANDRA-10446 and was going to open a ticket but at the time I didn't think
it was so critical because I thought the source node would also not mark the
data as repaired so in the next repair round things would be fixed, but the
fact that at least one replica will mark as repaired while others will not will
make mismatches on MV tables bounce forever when running incremental repair.
Since it seems this is not what is causing the explosion of sstables here,
would you mind creating a ticket for that and posting your findings/repro
steps? Although we could probably reuse CASSANDRA-12489, it's unclear to me if
it's the same since that affects non-incremental subrange repair.
[~brstgt] It seems your fork does not contain CASSANDRA-12580 which should help
a lot with overstreaming, so could you try applying that and check if that will
at least mitigate this issue?
bq. Maybe this is offtopic to this issue but for my understanding it sounds
like doing incremental repairs with MVs always produced crap. In order to
guarantee a consistent "repairedAt" state, you probably need sth like a
sandboxed write path that is separated from the regular write path to be sure
that streaming mutations and regular mutations are completely separated. and
when streaming finished, you can flush all tables on all nodes and flag the
newly created SSTables as repaired. But that again sounds like a very complex
change.
Perhaps this is something that can be addressed by [~bdeggleston] on
CASSANDRA-10446, but it will probably be a bit more involved since to be
properly fixed it needs the memtable to distinguish between repaired and
non-repaired mutations and we don't have this infrastructure now. Perhaps a
simpler approach would be to skip anti-compaction altogether when there are
mismatches on MV table, but we should probably move this discussion to another
ticket since this is a different issue.
> 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)