You could use a load balancing policy at the driver level to do what you
want, mixed with the existing consistency levels as Jeff suggested.

On Wed, Mar 14, 2018 at 3:47 PM Carl Mueller <carl.muel...@smartthings.com>
wrote:

> But we COULD have CL2 write (for RF4)
>
> The extension to this idea is multiple backup/secondary replicas. So you
> have RF5 or RF6 or higher, but still are performing CL2 against the
> preferred first three for both read and write.
>
> You could also ascertain the general write health of affected ranges before
> taking a node down for maintenance from the primary, and then know the
> switchover is in good shape. Yes there are CAP limits and race conditions
> there, but you could get pretty good assurances (all repaired, low/zero
> queued hinted handoffs, etc).
>
> This is essentially like if you had two datacenters, but are doing
> local_quorum on the one datacenter. Well, except switchover is a bit more
> granular if you run out of replicas in the local.
>
>
>
> On Wed, Mar 14, 2018 at 5:17 PM, Jeff Jirsa <jji...@gmail.com> wrote:
>
> > Write at CL 3 and read at CL 2
> >
> > --
> > Jeff Jirsa
> >
> >
> > > On Mar 14, 2018, at 2:40 PM, Carl Mueller <
> carl.muel...@smartthings.com>
> > wrote:
> > >
> > > Currently there is little use for RF4. You're getting the requirements
> of
> > > QUORUM-3 but only one extra backup.
> > >
> > > I'd like to propose something that would make RF4 a sort of more
> heavily
> > > backed up RF3.
> > >
> > > A lot of this is probably achievable with strictly driver-level logic,
> so
> > > perhaps it would belong more there.
> > >
> > > Basically the idea is to have four replicas of the data, but only have
> to
> > > practically do QUORUM with three nodes. We consider the first three
> > > replicas the "primary replicas". On an ongoing basis for QUORUM reads
> and
> > > writes, we would rely on only those three replicas to satisfy
> > > two-out-of-three QUORUM. Writes are persisted to the fourth replica in
> > the
> > > normal manner of cassandra, it just doesn't count towards the QUORUM
> > write.
> > >
> > > On reads, with token and node health awareness by the driver, if the
> > > primaries are all healthy, two-of-three QUORUM is calculated from
> those.
> > >
> > > If however one of the three primaries is down, read QUORUM is a bit
> > > different:
> > > 1) if the first two replies come from the two remaining primaries and
> > > agree, the is returned
> > > 2) if the first two replies are a primary and the "hot spare" and those
> > > agree, that is returned
> > > 3) if the primary and hot spare disagree, wait for the next primary to
> > > return, and then take the agreement (hopefully) that results
> > >
> > > Then once the previous primary comes back online, the read quorum goes
> > back
> > > to preferring that set, with the assuming hinted handoff and repair
> will
> > > get it back up to snuff.
> > >
> > > There could also be some mechanism examining the hinted handoff status
> of
> > > the four to determine when to reactivate the primary that was down.
> > >
> > > For mutations, one could prefer a "QUORUM plus" that was a quorum of
> the
> > > primaries plus the hot spare.
> > >
> > > Of course one could do multiple hot spares, so RF5 could still be
> treated
> > > as RF3 + hot spares.
> > >
> > > The goal here is more data resiliency but not having to rely on as many
> > > nodes for resiliency.
> > >
> > > Since the data is ring-distributed, the fact there are primary owners
> of
> > > ranges should still be evenly distributed and no hot nodes should
> result
> >
> > ---------------------------------------------------------------------
> > To unsubscribe, e-mail: dev-unsubscr...@cassandra.apache.org
> > For additional commands, e-mail: dev-h...@cassandra.apache.org
> >
> >
>

Reply via email to