I don't say reaper is the problem. I don't want to do wrong to Reaper but
in the end it is "just" an instrumentation for CS's built in repairs that
slices and schedules, right?
The problem I see is that the built in repairs are rather inefficient (for
many, maybe not all use cases) due to many reasons. To name some of them:

- Overstreaming as only whole partitions are repaired, not single mutations
- Race conditions in merkle tree calculation on nodes taking part in a
repair session
- Every stream creates a SSTable, needing to be compacted
- Possible SSTable creation floods can even kill a node due to "too many
open files" - yes we had that
- Incremental repairs have issues

Today we had a super simple case where I first ran 'nodetool repair' on a
super small system keyspace and then ran a 'scrape-repair':
- nodetool took 4 minutes on a single node
- scraping took 1 sec repairing all nodes together

In the beginning I was twisting my brain how this could be optimized in CS
- in the end going with scraping solved every problem we had.

2018-04-05 20:32 GMT+02:00 Jonathan Haddad <j...@jonhaddad.com>:

> To be fair, reaper in 2016 only worked with 2.0 and was just sitting
> around, more or less.
>
> Since then we've had 401 commits changing tens of thousands of lines of
> code, dealing with fault tolerance, repair retries, scalability, etc.
> We've had 1 reaper node managing repairs across dozens of clusters and
> thousands of nodes.  It's a totally different situation today.
>
>
> On Thu, Apr 5, 2018 at 11:17 AM benjamin roth <brs...@gmail.com> wrote:
>
> > That would be totally awesome!
> >
> > Not sure if it helps here but for completeness:
> > We completely "dumped" regular repairs - no matter if 'nodetool repair'
> or
> > reaper - and run our own tool that does simply CL_ALL scraping over the
> > whole cluster.
> > It runs now for over a year in production and the only problem we
> > encountered was that we got timeouts when scraping (too) large /
> tombstoned
> > partitions. It turned out that the large partitions weren't even readable
> > with CQL / cqlsh / DevCenter. So that wasn't a problem of the repair. It
> > was rather a design problem. Storing data that can't be read doesn't make
> > sense anyway.
> >
> > What I can tell from our experience:
> > - It works much more reliable than what we had before - also more
> reliable
> > than reaper (state of 2016)
> > - It runs totally smooth and much faster than regular repairs as it only
> > streams what needs to be streamed
> > - It's easily manageable, interruptible, resumable on a very fine-grained
> > level. The only thing you need to do is to store state (KS/CF/Last Token)
> > in a simple storage like redis
> > - It works even pretty well when populating a empty node e.g. when
> changing
> > RFs / bootstrapping DCs
> > - You can easily control the cluster-load by tuning the concurrency of
> the
> > scrape process
> >
> > I don't see a reason for us to ever go back to built-in repairs if they
> > don't improve immensely. In many cases (especially with MVs) they are
> true
> > resource killers.
> >
> > Just my 2 cent and experience.
> >
> > 2018-04-04 17:00 GMT+02:00 Ben Bromhead <b...@instaclustr.com>:
> >
> > > +1 to including the implementation in Cassandra itself. Makes managed
> > > repair a first-class citizen, it nicely rounds out Cassandra's
> > consistency
> > > story and makes it 1000x more likely that repairs will get run.
> > >
> > >
> > >
> > >
> > > On Wed, Apr 4, 2018 at 10:45 AM Jon Haddad <j...@jonhaddad.com> wrote:
> > >
> > > > Implementation details aside, I’m firmly in the “it would be nice of
> C*
> > > > could take care of it” camp.  Reaper is pretty damn easy to use and
> > > people
> > > > *still* don’t put it in prod.
> > > >
> > > >
> > > > > On Apr 4, 2018, at 4:16 AM, Rahul Singh <
> > rahul.xavier.si...@gmail.com>
> > > > wrote:
> > > > >
> > > > > I understand the merits of both approaches. In working with other
> DBs
> > > In
> > > > the “old country” of SQL, we often had to write indexing sequences
> > > manually
> > > > for important tables. It was “built into the product” but in order to
> > > > leverage the maximum benefits of indices we had to have different
> > indices
> > > > other than the clustered (physical index). The process still sucked.
> > It’s
> > > > never perfect.
> > > > >
> > > > > The JVM is already fraught with GC issues and putting another
> process
> > > > being managed in the same heapspace is what I’m worried about.
> > > Technically
> > > > the process could be in the same binary but started as a side Car or
> in
> > > the
> > > > same main process.
> > > > >
> > > > > Consider a process called “cassandra-agent” that’s sitting around
> > with
> > > a
> > > > scheduler based on config or a Cassandra table. Distributed in the
> same
> > > > release. Shell / service scripts would start it. The end user knows
> it
> > > only
> > > > by examining the .sh files. This opens possibilities of including a
> GUI
> > > > hosted in the same process without cluttering the core coolness of
> > > > Cassandra.
> > > > >
> > > > > Best,
> > > > >
> > > > > --
> > > > > Rahul Singh
> > > > > rahul.si...@anant.us
> > > > >
> > > > > Anant Corporation
> > > > >
> > > > > On Apr 4, 2018, 2:50 AM -0400, Dor Laor <d...@scylladb.com>, wrote:
> > > > >> We at Scylla, implemented repair in a similar way to the Cassandra
> > > > reaper.
> > > > >> We do
> > > > >> that using an external application, written in go that manages
> > repair
> > > > for
> > > > >> multiple clusters
> > > > >> and saves the data in an external Scylla cluster. The logic
> > resembles
> > > > the
> > > > >> reaper one with
> > > > >> some specific internal sharding optimizations and uses the Scylla
> > rest
> > > > api.
> > > > >>
> > > > >> However, I have doubts it's the ideal way. After playing a bit
> with
> > > > >> CockroachDB, I realized
> > > > >> it's super nice to have a single binary that repairs itself,
> > provides
> > > a
> > > > GUI
> > > > >> and is the core DB.
> > > > >>
> > > > >> Even while distributed, you can elect a leader node to manage the
> > > > repair in
> > > > >> a consistent
> > > > >> way so the complexity can be reduced to a minimum. Repair can
> write
> > > its
> > > > >> status to the
> > > > >> system tables and to provide an api for progress, rate control,
> etc.
> > > > >>
> > > > >> The big advantage for repair to embedded in the core is that there
> > is
> > > no
> > > > >> need to expose
> > > > >> internal state to the repair logic. So an external program doesn't
> > > need
> > > > to
> > > > >> deal with different
> > > > >> version of Cassandra, different repair capabilities of the core
> > (such
> > > as
> > > > >> incremental on/off)
> > > > >> and so forth. A good database should schedule its own repair, it
> > knows
> > > > >> whether the shreshold
> > > > >> of hintedhandoff was cross or not, it knows whether nodes where
> > > > replaced,
> > > > >> etc,
> > > > >>
> > > > >> My 2 cents. Dor
> > > > >>
> > > > >> On Tue, Apr 3, 2018 at 11:13 PM, Dinesh Joshi <
> > > > >> dinesh.jo...@yahoo.com.invalid> wrote:
> > > > >>
> > > > >>> Simon,
> > > > >>> You could still do load aware repair outside of the main process
> by
> > > > >>> reading Cassandra's metrics.
> > > > >>> In general, I don't think the maintenance tasks necessarily need
> to
> > > > live
> > > > >>> in the main process. They could negatively impact the read /
> write
> > > > path.
> > > > >>> Unless strictly required by the serving path, it could live in a
> > > > sidecar
> > > > >>> process. There are multiple benefits including isolation, faster
> > > > iteration,
> > > > >>> loose coupling. For example - this would mean that the
> maintenance
> > > > tasks
> > > > >>> can have a different gc profile than the main process and it
> would
> > be
> > > > ok.
> > > > >>> Today that is not the case.
> > > > >>> The only issue I see is that the project does not provide an
> > official
> > > > >>> sidecar. Perhaps there should be one. We probably would've not
> had
> > to
> > > > have
> > > > >>> this discussion ;)
> > > > >>> Dinesh
> > > > >>>
> > > > >>> On Tuesday, April 3, 2018, 10:12:56 PM PDT, Qingcun Zhou <
> > > > >>> zhouqing...@gmail.com> wrote:
> > > > >>>
> > > > >>> Repair has been a problem for us at Uber. In general I'm in favor
> > of
> > > > >>> including the scheduling logic in Cassandra daemon. It has the
> > > benefit
> > > > of
> > > > >>> introducing something like load-aware repair, eg, only schedule
> > > repair
> > > > >>> while no ongoing compaction or traffic is low, etc. As proposed
> by
> > > > others,
> > > > >>> we can expose keyspace/table-level configurations so that users
> can
> > > > opt-in.
> > > > >>> Regarding the risk, yes there will be problems at the beginning
> but
> > > in
> > > > the
> > > > >>> long run, users will appreciate that repair works out of the box,
> > > just
> > > > like
> > > > >>> compaction. We have large Cassandra deployments and can work with
> > > > Netflix
> > > > >>> folks for intensive testing to boost user confidence.
> > > > >>>
> > > > >>> On the other hand, have we looked into how other NoSQL databases
> do
> > > > repair?
> > > > >>> Is there a side car process?
> > > > >>>
> > > > >>>
> > > > >>> On Tue, Apr 3, 2018 at 9:21 PM, sankalp kohli <
> > > kohlisank...@gmail.com
> > > > >>> wrote:
> > > > >>>
> > > > >>>> Repair is critical for running C* and I agree with Roopa that it
> > > > needs to
> > > > >>>> be part of the offering. I think we should make it easy for new
> > > users
> > > > to
> > > > >>>> run C*.
> > > > >>>>
> > > > >>>> Can we have a side car process which we can add to Apache
> > Cassandra
> > > > >>>> offering and we can put this repair their? I am also fine
> putting
> > it
> > > > in
> > > > >>> C*
> > > > >>>> if side car is more long term.
> > > > >>>>
> > > > >>>> On Tue, Apr 3, 2018 at 6:20 PM, Roopa Tangirala <
> > > > >>>> rtangir...@netflix.com.invalid> wrote:
> > > > >>>>
> > > > >>>>> In seeing so many companies grapple with running repairs
> > > successfully
> > > > >>> in
> > > > >>>>> production, and seeing the success of distributed scheduled
> > repair
> > > > here
> > > > >>>> at
> > > > >>>>> Netflix, I strongly believe that adding this to Cassandra would
> > be
> > > a
> > > > >>>> great
> > > > >>>>> addition to the database. I am hoping, we as a community will
> > make
> > > it
> > > > >>>> easy
> > > > >>>>> for teams to operate and run Cassandra by enhancing the core
> > > product,
> > > > >>> and
> > > > >>>>> making the maintenances like repairs and compactions part of
> the
> > > > >>> database
> > > > >>>>> without external tooling. We can have an experimental flag for
> > the
> > > > >>>> feature
> > > > >>>>> and only teams who are confident with the service can enable
> > them,
> > > > >>> while
> > > > >>>>> others can fall back to default repairs.
> > > > >>>>>
> > > > >>>>>
> > > > >>>>> *Regards,*
> > > > >>>>>
> > > > >>>>> *Roopa Tangirala*
> > > > >>>>>
> > > > >>>>> Engineering Manager CDE
> > > > >>>>>
> > > > >>>>> *(408) 438-3156 - mobile*
> > > > >>>>>
> > > > >>>>>
> > > > >>>>>
> > > > >>>>>
> > > > >>>>>
> > > > >>>>> On Tue, Apr 3, 2018 at 4:19 PM, Kenneth Brotman <
> > > > >>>>> kenbrot...@yahoo.com.invalid> wrote:
> > > > >>>>>
> > > > >>>>>> Why not make it configurable?
> > > > >>>>>> auto_manage_repair_consistancy: true (default: false)
> > > > >>>>>>
> > > > >>>>>> Then users can use the built in auto repair function that
> would
> > be
> > > > >>>>> created
> > > > >>>>>> or continue to handle it as now. Default behavior would be
> > "false"
> > > > >>> so
> > > > >>>>>> nothing changes on its own. Just wondering why not have that
> > > option?
> > > > >>>> It
> > > > >>>>>> might accelerate progress as others have already suggested.
> > > > >>>>>>
> > > > >>>>>> Kenneth Brotman
> > > > >>>>>>
> > > > >>>>>> -----Original Message-----
> > > > >>>>>> From: Nate McCall [mailto:zznat...@gmail.com]
> > > > >>>>>> Sent: Tuesday, April 03, 2018 1:37 PM
> > > > >>>>>> To: dev
> > > > >>>>>> Subject: Re: Repair scheduling tools
> > > > >>>>>>
> > > > >>>>>> This document does a really good job of listing out some of
> the
> > > > >>> issues
> > > > >>>> of
> > > > >>>>>> coordinating scheduling repair. Regardless of which camp you
> > fall
> > > > >>> into,
> > > > >>>>> it
> > > > >>>>>> is certainly worth a read.
> > > > >>>>>>
> > > > >>>>>> On Wed, Apr 4, 2018 at 8:10 AM, Joseph Lynch <
> > > joe.e.ly...@gmail.com
> > > > >>>>>> wrote:
> > > > >>>>>>> I just want to say I think it would be great for our users if
> > we
> > > > >>>> moved
> > > > >>>>>>> repair scheduling into Cassandra itself. The team here at
> > Netflix
> > > > >>> has
> > > > >>>>>>> opened the ticket
> > > > >>>>>>> <https://issues.apache.org/jira/browse/CASSANDRA-14346
> > > > >>>>>>> and have written a detailed design document
> > > > >>>>>>> <https://docs.google.com/document/d/1RV4rOrG1gwlD5IljmrIq_
> > > > >>>> t45rz7H3xs9G
> > > > >>>>>>> bFSEyGzEtM/edit#heading=h.iasguic42ger
> > > > >>>>>>> that includes problem discussion and prior art if anyone
> wants
> > to
> > > > >>>>>>> contribute to that. We tried to fairly discuss existing
> > > solutions,
> > > > >>>>>>> what their drawbacks are, and a proposed solution.
> > > > >>>>>>>
> > > > >>>>>>> If we were to put this as part of the main Cassandra daemon,
> I
> > > > >>> think
> > > > >>>>>>> it should probably be marked experimental and of course be
> > > > >>> something
> > > > >>>>>>> that users opt into (table by table or cluster by cluster)
> with
> > > the
> > > > >>>>>>> understanding that it might not fully work out of the box the
> > > first
> > > > >>>>>>> time we ship it. We have to be willing to take risks but we
> > also
> > > > >>> have
> > > > >>>>>>> to be honest with our users. It may help build confidence if
> a
> > > few
> > > > >>>>>>> major deployments use it (such as Netflix) and we are happy
> of
> > > > >>> course
> > > > >>>>>>> to provide that QA as best we can.
> > > > >>>>>>>
> > > > >>>>>>> -Joey
> > > > >>>>>>>
> > > > >>>>>>> On Tue, Apr 3, 2018 at 10:48 AM, Blake Eggleston
> > > > >>>>>>> <beggles...@apple.com
> > > > >>>>>>> wrote:
> > > > >>>>>>>
> > > > >>>>>>>> Hi dev@,
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>> The question of the best way to schedule repairs came up on
> > > > >>>>>>>> CASSANDRA-14346, and I thought it would be good to bring up
> > the
> > > > >>> idea
> > > > >>>>>>>> of an external tool on the dev list.
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>> Cassandra lacks any sort of tools for automating routine
> tasks
> > > > >>> that
> > > > >>>>>>>> are required for running clusters, specifically repair.
> > Regular
> > > > >>>>>>>> repair is a must for most clusters, like compaction. This
> > means
> > > > >>>> that,
> > > > >>>>>>>> especially as far as eventual consistency is concerned,
> > > Cassandra
> > > > >>>>>>>> isn’t totally functional out of the box. Operators either
> need
> > > to
> > > > >>>>>>>> find a 3rd party solution or implement one themselves.
> Adding
> > > this
> > > > >>>> to
> > > > >>>>>>>> Cassandra would make it easier to use.
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>> Is this something we should be doing? If so, what should it
> > look
> > > > >>>> like?
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>> Personally, I feel like this is a pretty big gap in the
> > project
> > > > >>> and
> > > > >>>>>>>> would like to see an out of process tool offered. Ideally,
> > > > >>> Cassandra
> > > > >>>>>>>> would just take care of itself, but writing a distributed
> > repair
> > > > >>>>>>>> scheduler that you trust to run in production is a lot
> harder
> > > than
> > > > >>>>>>>> writing a single process management application that can
> > > failover.
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>> Any thoughts on this?
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>> Thanks,
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>> Blake
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>
> > > > >>>>>> ------------------------------------------------------------
> > > > >>> ---------
> > > > >>>>>> To unsubscribe, e-mail: dev-unsubscr...@cassandra.apache.org
> > > > >>>>>> For additional commands, e-mail:
> dev-h...@cassandra.apache.org
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>> ------------------------------------------------------------
> > > > >>> ---------
> > > > >>>>>> To unsubscribe, e-mail: dev-unsubscr...@cassandra.apache.org
> > > > >>>>>> For additional commands, e-mail:
> dev-h...@cassandra.apache.org
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>>>
> > > > >>>
> > > > >>>
> > > > >>>
> > > > >>> --
> > > > >>> Thank you & Best Regards,
> > > > >>> --Simon (Qingcun) Zhou
> > > > >>>
> > > >
> > > >
> > > > ------------------------------------------------------------
> ---------
> > > > To unsubscribe, e-mail: dev-unsubscr...@cassandra.apache.org
> > > > For additional commands, e-mail: dev-h...@cassandra.apache.org
> > > >
> > > > --
> > > Ben Bromhead
> > > CTO | Instaclustr <https://www.instaclustr.com/>
> > > +1 650 284 9692 <(650)%20284-9692>
> > > Reliability at Scale
> > > Cassandra, Spark, Elasticsearch on AWS, Azure, GCP and Softlayer
> > >
> >
>

Reply via email to