[ https://issues.apache.org/jira/browse/CASSANDRA-12888?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16012359#comment-16012359 ]
Paulo Motta commented on CASSANDRA-12888: ----------------------------------------- First of all thank you for your effort on this issue and sorry for the delay. I had an initial look at the patch and while I didn't spot any immediate flaws, it is not clear from looking at the code alone if this is not going to have any unintended repercussions in some other component (commit log archiving, memtable lifecyle/flushing, read/write path) since this is changing a core assumption that a given CF/table has a single active memtable (or at most 2, during flush) to having multiple active memtables. While in an ideal world correctness would be ensured by tests, we know well our test coverage is far from ideal so we need to make sure the approach is well discussed and all edge cases understood before jumping into actual code. One way to facilitate communication in changes like this is through a design doc, which is not generally required for small changes, but since we're modifying a core component it is important to spell out all the details in a design doc to make sure we're covering everything and to communicate the change to interested parties for feedback without exposing them to actual code. With this said, it would be great if you could prepare a short design doc with proposed changes and justification, how the memtable and flushing lifecycle is affected by this, impacted areas, API changes/additions, test plan before we proceed. Overall I think you are on the right path but your approach needs some adjustments. See comments below: * I think explicitly managing the repaired memtable lifecycle from the repair job is less prone to errors/leaks than let the memtable be created on demand when the mutation is being applied and removed organically by automatic flush. My suggestion would be to create the memtable at the start of the repair job when necessary, append mutations to it and then flush/remove the additional memtable at the end of the job. * In order for this to play along well with [CASSANDRA-9143|https://issues.apache.org/jira/browse/CASSANDRA-9143], the mutation would somehow need to include the repair session id which would be used to retrieve the repairedAt from {{ActiveRepairService}}. In this arrangement we would keep a memtable per active repair session to ensure mutations from the same job are flushed to the same sstable. {{Tracker.getMemtableFor}} would fetch the right memtable for that repair session id, and return the ordinary unrepaired memtable if there is not memtable created for that repair session id (and probably log a warning). * You modified flush to operate on all memtables of a given table, but we probably need to keep flush to a single memtable since we don't want to flush all memtables every time. For instance, if {{memtable_cleanup_threshold}} is reached you will want to flush only the largest memtable, not all memtables for a given table. Memtable will have different creation times, so you will only want to flush the expired memtable and not all memtables for a given table. For repair you will only want to flush unrepaired memtables. Sometimes you will need to flush all memtables of a given table (such as nodetool flush), but not every time. With this we keep the flush lifecyle pretty much the same it is today, there will be at most 2 active memtables for unrepaired data and at most 2 active memtables for each repair session (one being flushed and maybe its replacement), what will facilitate understanding and reduce surface to unintended side-effects. All of these changes need to be spelled out on the design doc as well as javadoc to make sure it's clearly communicated. [~krummas] [~bdeggleston] Does the above sound reasonable or do you have any other suggestions or remarks? Marcus, is this more or less how you planned to add repaired memtables on CASSANDRA-8911 or did you have something else in mind? > Incremental repairs broken for MVs and CDC > ------------------------------------------ > > Key: CASSANDRA-12888 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12888 > Project: Cassandra > Issue Type: Bug > Components: Streaming and Messaging > Reporter: Stefan Podkowinski > Assignee: Benjamin Roth > Priority: Critical > Fix For: 3.0.x, 3.11.x > > > SSTables streamed during the repair process will first be written locally and > afterwards either simply added to the pool of existing sstables or, in case > of existing MVs or active CDC, replayed on mutation basis: > As described in {{StreamReceiveTask.OnCompletionRunnable}}: > {quote} > We have a special path for views and for CDC. > For views, since the view requires cleaning up any pre-existing state, we > must put all partitions through the same write path as normal mutations. This > also ensures any 2is are also updated. > For CDC-enabled tables, we want to ensure that the mutations are run through > the CommitLog so they can be archived by the CDC process on discard. > {quote} > Using the regular write path turns out to be an issue for incremental > repairs, as we loose the {{repaired_at}} state in the process. Eventually the > streamed rows will end up in the unrepaired set, in contrast to the rows on > the sender site moved to the repaired set. The next repair run will stream > the same data back again, causing rows to bounce on and on between nodes on > each repair. > See linked dtest on steps to reproduce. An example for reproducing this > manually using ccm can be found > [here|https://gist.github.com/spodkowinski/2d8e0408516609c7ae701f2bf1e515e8] -- This message was sent by Atlassian JIRA (v6.3.15#6346) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org