You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Branimir Lambov (Jira)" <ji...@apache.org> on 2022/12/05 15:36:00 UTC

[jira] [Comment Edited] (CASSANDRA-18093) `generatetokens` uses the wrong allocation strategy

    [ https://issues.apache.org/jira/browse/CASSANDRA-18093?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17643413#comment-17643413 ] 

Branimir Lambov edited comment on CASSANDRA-18093 at 12/5/22 3:35 PM:
----------------------------------------------------------------------

This is an example of the allocation {{generatetokens}} is currently constructing for the second nodes in each rack:
{code:java}
Replicated node load in rack=0 after allocating node 3: max 1.33 min 0.56 stddev 0.0370.
Replicated node load in rack=1 after allocating node 4: max 1.67 min 0.70 stddev 0.0288.
Replicated node load in rack=2 after allocating node 5: max 1.16 min 0.84 stddev 0.0064.
{code}
and this is what it should be
{code:java}
Replicated node load in rack=0 after allocating node 3: max 1.06 min 0.94 stddev 0.0052.
Replicated node load in rack=1 after allocating node 4: max 1.06 min 0.94 stddev 0.0052.
Replicated node load in rack=2 after allocating node 5: max 1.06 min 0.94 stddev 0.0052.
{code}
The second result was generated after applying the following patch:
{code:java}
diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
index 184332907b..84ccb95ff4 100644
--- a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
+++ b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
@@ -239,7 +239,8 @@ public class TokenAllocation
 
     private StrategyAdapter getOrCreateStrategy(String dc, String rack)
     {
-        return strategyByRackDc.computeIfAbsent(dc, k -> new HashMap<>()).computeIfAbsent(rack, k -> createStrategy(dc, rack));
+        return createStrategy(dc, rack);
+//        return strategyByRackDc.computeIfAbsent(dc, k -> new HashMap<>()).computeIfAbsent(rack, k -> createStrategy(dc, rack));
     }
 
     private StrategyAdapter createStrategy(String dc, String rack)
@@ -281,8 +282,11 @@ public class TokenAllocation
             // group by rack
             return createStrategy(strategy.snitch, dc, null, replicas, true);
         }
-        else if (racks == 1)
+        else if (racks == 1 || topology.getDatacenterEndpoints().get(dc).size() < replicas)
         {
+            // One rack, each node treated as separate.
+            // This is also used as a fallback when number of nodes is lower than the replication factor
+            // (where allocation is done randomly).
             return createStrategy(strategy.snitch, dc, null, replicas, false);
         }
{code}


was (Author: blambov):
This is an example of the allocation {{generatetokens}} is currently constructing for the second nodes in a rack:
{code}Replicated node load in rack=0 after allocating node 3: max 1.33 min 0.56 stddev 0.0370.
Replicated node load in rack=1 after allocating node 4: max 1.67 min 0.70 stddev 0.0288.
Replicated node load in rack=2 after allocating node 5: max 1.16 min 0.84 stddev 0.0064.
{code} and this is what it should be {code}Replicated node load in rack=0 after allocating node 3: max 1.06 min 0.94 stddev 0.0052.
Replicated node load in rack=1 after allocating node 4: max 1.06 min 0.94 stddev 0.0052.
Replicated node load in rack=2 after allocating node 5: max 1.06 min 0.94 stddev 0.0052.
{code}

The second result was generated after applying this patch:{code}diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
index 184332907b..84ccb95ff4 100644
--- a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
+++ b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java
@@ -239,7 +239,8 @@ public class TokenAllocation
 
     private StrategyAdapter getOrCreateStrategy(String dc, String rack)
     {
-        return strategyByRackDc.computeIfAbsent(dc, k -> new HashMap<>()).computeIfAbsent(rack, k -> createStrategy(dc, rack));
+        return createStrategy(dc, rack);
+//        return strategyByRackDc.computeIfAbsent(dc, k -> new HashMap<>()).computeIfAbsent(rack, k -> createStrategy(dc, rack));
     }
 
     private StrategyAdapter createStrategy(String dc, String rack)
@@ -281,8 +282,11 @@ public class TokenAllocation
             // group by rack
             return createStrategy(strategy.snitch, dc, null, replicas, true);
         }
-        else if (racks == 1)
+        else if (racks == 1 || topology.getDatacenterEndpoints().get(dc).size() < replicas)
         {
+            // One rack, each node treated as separate.
+            // This is also used as a fallback when number of nodes is lower than the replication factor
+            // (where allocation is done randomly).
             return createStrategy(strategy.snitch, dc, null, replicas, false);
         }
{code}

> `generatetokens` uses the wrong allocation strategy
> ---------------------------------------------------
>
>                 Key: CASSANDRA-18093
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-18093
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Branimir Lambov
>            Priority: Normal
>
> When the number of racks is the same as the replication factor, token allocation should use `NoReplicationTokenAllocator`. Because `generatetokens` caches the allocation strategy [here|https://github.com/apache/cassandra/commit/2346ed8241022882e77433e283ab8ce3004d12b0#diff-a93e5fa817f4b6d64484e0c28391b76e1760f51860de7f4f036470766ff5090cR225], it uses one that was created before all the racks are defined and does not switch to the correct one.
> This has the effect of constructing very poor allocations for the racks = RF case, especially for the first couple of nodes per rack.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org