I am forwarding the message Ben Slater wrote me personally and asked me to post 
it. He has some problems with the subscription to this mailing list with his 
email.

Very uncommon in my experience – my guess would be at most 2 to 3 cluster out 
of the few hundred that we manage.

Also picking up on one of your comments earlier in the thread – " like anyone 
running RF=3 in AWS us-west-1 (or any other region with only 2 accessible 
AZs)": in the situation with RF3 and two racks/AZs the current NTS behaviour is 
no worse than the best logical case of 2 replicas in 1 rack. This issue is 
really only a problem with RF5 and 3 AZs where you can end up with 3 replicas 
in one AZ and then lose quorum with failure of that AZ.

We currently work around this issue by ensuring we define 5 racks if targeting 
RF5 in a region with less than 5 AZs but having multiple logical racks point to 
the same AZs (we also do the same with RF3 and 2 AZs because it makes some 
management ops simpler to have consistency).

Paulo made a good point about the system_auth case although now I think about 
it, it probably doesn’t impact there because I think system_auth is queried at 
LOCAL_ONE and high RF on system auth was more about replicating to lots of 
nodes than distributing across racks

Cheers
Ben

________________________________________
From: Paulo Motta <pauloricard...@gmail.com>
Sent: Tuesday, March 7, 2023 21:43
To: dev@cassandra.apache.org
Subject: Re: Degradation of availability when using NTS and RF > number of racks

NetApp Security WARNING: This is an external email. Do not click links or open 
attachments unless you recognize the sender and know the content is safe.



I'm not sure if this recommendation is still valid (or ever was) but it's not 
uncommon to have higher RF on system_auth keyspaces, where it would be quite 
dramatic to hit this bug on the loss of a properly configured rack for RF=3.

On Tue, Mar 7, 2023 at 2:40 PM Jeff Jirsa 
<jji...@gmail.com<mailto:jji...@gmail.com>> wrote:
Anyone have stats on how many people use RF > 3 per dc? (I know what it looks 
like in my day job but I don’t want to pretend it’s representative of the 
larger community)

I’m a fan of fixing this but I do wonder how common this is in the wild.

On Mar 7, 2023, at 9:12 AM, Derek Chen-Becker 
<de...@chen-becker.org<mailto:de...@chen-becker.org>> wrote:


I think that the warning would only be thrown in the case where a potentially 
QUORUM-busting configuration is used. I think it would be a worse experience to 
not warn and let the user discover later when they can't write at QUORUM.

Cheers,

Derek

On Tue, Mar 7, 2023 at 9:32 AM Jeremiah D Jordan 
<jeremiah.jor...@gmail.com<mailto:jeremiah.jor...@gmail.com>> wrote:
I agree with Paulo, it would be nice if we could figure out some way to make 
new NTS work correctly, with a parameter to fall back to the “bad” behavior, so 
that people restoring backups to a new cluster can get the right behavior to 
match their backups.
The problem with only fixing this in a new strategy is we have a ton of 
tutorials and docs out there which tell people to use NTS, so it would be great 
if we could keep “use NTS” as the recommendation.  Throwing a warning when 
someone uses NTS is kind of user hostile.  If someone just read some tutorial 
or doc which told them “make your key space this way” and then when they do 
that the database yells at them telling them they did it wrong, it is not a 
great experience.

-Jeremiah

> On Mar 7, 2023, at 10:16 AM, Benedict 
> <bened...@apache.org<mailto:bened...@apache.org>> wrote:
>
> My view is that if this is a pretty serious bug. I wonder if transactional 
> metadata will make it possible to safely fix this for users without 
> rebuilding (only via opt-in, of course).
>
>> On 7 Mar 2023, at 15:54, Miklosovic, Stefan 
>> <stefan.mikloso...@netapp.com<mailto:stefan.mikloso...@netapp.com>> wrote:
>>
>> Thanks everybody for the feedback.
>>
>> I think that emitting a warning upon keyspace creation (and alteration) 
>> should be enough for starters. If somebody can not live without 100% bullet 
>> proof solution over time we might choose some approach from the offered 
>> ones. As the saying goes there is no silver bullet. If we decide to 
>> implement that new strategy, we would probably emit warnings anyway on NTS 
>> but it would be already done so just new strategy would be provided.
>>
>> ________________________________________
>> From: Paulo Motta <pauloricard...@gmail.com<mailto:pauloricard...@gmail.com>>
>> Sent: Monday, March 6, 2023 17:48
>> To: dev@cassandra.apache.org<mailto:dev@cassandra.apache.org>
>> Subject: Re: Degradation of availability when using NTS and RF > number of 
>> racks
>>
>> NetApp Security WARNING: This is an external email. Do not click links or 
>> open attachments unless you recognize the sender and know the content is 
>> safe.
>>
>>
>>
>> It's a bit unfortunate that NTS does not maintain the ability to lose a rack 
>> without loss of quorum for RF > #racks > 2, since this can be easily 
>> achieved by evenly placing replicas across all racks.
>>
>> Since RackAwareTopologyStrategy is a superset of NetworkTopologyStrategy, 
>> can't we just use the new correct placement logic for newly created 
>> keyspaces instead of having a new strategy?
>>
>> The placement logic would be backwards-compatible for RF <= #racks. On 
>> upgrade, we could mark existing keyspaces with RF > #racks with 
>> use_legacy_replica_placement=true to maintain backwards compatibility and 
>> log a warning that the rack loss guarantee is not maintained for keyspaces 
>> created before the fix. Old keyspaces with RF <=#racks would still work with 
>> the new replica placement. The downside is that we would need to keep the 
>> old NTS logic around, or we could eventually deprecate it and require users 
>> to migrate keyspaces using the legacy placement strategy.
>>
>> Alternatively we could have RackAwareTopologyStrategy and fail NTS keyspace 
>> creation for RF > #racks and indicate users to use RackAwareTopologyStrategy 
>> to maintain the quorum guarantee on rack loss or set an override flag 
>> "support_quorum_on_rack_loss=false". This feels a bit iffy though since it 
>> could potentially confuse users about when to use each strategy.
>>
>> On Mon, Mar 6, 2023 at 5:51 AM Miklosovic, Stefan 
>> <stefan.mikloso...@netapp.com<mailto:stefan.mikloso...@netapp.com><mailto:stefan.mikloso...@netapp.com<mailto: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  |
+---------------------------------------------------------------+

Reply via email to