A huge number of people use this legal and unsafe combination - like anyone running RF=3 in AWS us-west-1 (or any other region with only 2 accessible AZs), and no patch is going to suddenly make that safe, and banning it hurts users a lot.
If we're really going to ship a less-bad version of this, then that less-bad version probably wants to reject invalid configs (like RF=3 with 2 racks), but again, it'll be approximately impossible for anyone to document what it takes to move from the maybe-unsafe version to the definitely-safe version without rewriting all of the data into the cluster, so most people won't be able to use it anyway? On Mon, Mar 6, 2023 at 8:31 AM Derek Chen-Becker <de...@chen-becker.org> wrote: > 1) It does seem a like a big footgun. I think it violates the principle of > least surprise if someone has configured NTS thinking that they are > improving availability > 2) I don't know that we want to ban it outright, since maybe there's a > case for someone to be using a different CL that would be OK with the loss > of a majority of replicas (e.g. ONE). For example, we don't fail if someone > uses ALL or EACH_QUORUM with a problematic setup, do we? Would we warn on > keyspace creation with RF > racks or are you suggesting that the warning > would be at query time? > 3) agreed, this doesn't seem like an enhancement as much as it is > identifying legal but likely incorrect configuration > > Cheers, > > Derek > > On Mon, Mar 6, 2023 at 3:52 AM Miklosovic, Stefan < > stefan.mikloso...@netapp.com> wrote: > >> Hi all, >> >> some time ago we identified an issue with NetworkTopologyStrategy. The >> problem is that when RF > number of racks, it may happen that NTS places >> replicas in such a way that when whole rack is lost, we lose QUORUM and >> data are not available anymore if QUORUM CL is used. >> >> To illustrate this problem, lets have this setup: >> >> 9 nodes in 1 DC, 3 racks, 3 nodes per rack. RF = 5. Then, NTS could place >> replicas like this: 3 replicas in rack1, 1 replica in rack2, 1 replica in >> rack3. Hence, when rack1 is lost, we do not have QUORUM. >> >> It seems to us that there is already some logic around this scenario (1) >> but the implementation is not entirely correct. This solution is not >> computing the replica placement correctly so the above problem would be >> addressed. >> >> We created a draft here (2, 3) which fixes it. >> >> There is also a test which simulates this scenario. When I assign 256 >> tokens to each node randomly (by same mean as generatetokens command uses) >> and I try to compute natural replicas for 1 billion random tokens and I >> compute how many cases there will be when 3 replicas out of 5 are inserted >> in the same rack (so by losing it we would lose quorum), for above setup I >> get around 6%. >> >> For 12 nodes, 3 racks, 4 nodes per rack, rf = 5, this happens in 10% >> cases. >> >> To interpret this number, it basically means that with such topology, RF >> and CL, when a random rack fails completely, when doing a random read, >> there is 6% chance that data will not be available (or 10%, respectively). >> >> One caveat here is that NTS is not compatible with this new strategy >> anymore because it will place replicas differently. So I guess that fixing >> this in NTS will not be possible because of upgrades. I think people would >> need to setup completely new keyspace and somehow migrate data if they wish >> or they just start from scratch with this strategy. >> >> Questions: >> >> 1) do you think this is meaningful to fix and it might end up in trunk? >> >> 2) should not we just ban this scenario entirely? It might be possible to >> check the configuration upon keyspace creation (rf > num of racks) and if >> we see this is problematic we would just fail that query? Guardrail maybe? >> >> 3) people in the ticket mention writing "CEP" for this but I do not see >> any reason to do so. It is just a strategy as any other. What would that >> CEP would even be about? Is this necessary? >> >> Regards >> >> (1) >> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java#L126-L128 >> (2) https://github.com/apache/cassandra/pull/2191 >> (3) https://issues.apache.org/jira/browse/CASSANDRA-16203 > > > > -- > +---------------------------------------------------------------+ > | Derek Chen-Becker | > | GPG Key available at https://keybase.io/dchenbecker and | > | https://pgp.mit.edu/pks/lookup?search=derek%40chen-becker.org | > | Fngrprnt: EB8A 6480 F0A3 C8EB C1E7 7F42 AFC5 AFEE 96E4 6ACC | > +---------------------------------------------------------------+ > >