You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "driftx (via GitHub)" <gi...@apache.org> on 2023/04/12 15:29:48 UTC

[GitHub] [cassandra] driftx commented on a diff in pull request #2261: CASSANDRA-18430: When decommissioning should set Severity to limit traffic

driftx commented on code in PR #2261:
URL: https://github.com/apache/cassandra/pull/2261#discussion_r1164290606


##########
src/java/org/apache/cassandra/gms/Gossiper.java:
##########
@@ -2141,6 +2141,11 @@ public void addLocalApplicationState(ApplicationState applicationState, Versione
         addLocalApplicationStates(Arrays.asList(Pair.create(applicationState, value)));
     }
 
+    public void addLocalApplicationState(ApplicationState state1, VersionedValue value1, ApplicationState state2, VersionedValue value2)

Review Comment:
   was this an actual problem? STATUS is deprecated in 4.0 and not supposed to be used in 5.0 (that probably needs its own ticket though)



##########
src/java/org/apache/cassandra/service/StorageProxy.java:
##########
@@ -1617,10 +1618,19 @@ private static void sendMessagesToNonlocalDC(Message<? extends IMutation> messag
             target = targets.get(0);
         }
 
+        Tracing.trace("Sending mutation to remote replica {}", target);
         MessagingService.instance().sendWriteWithCallback(message, target, handler);
         logger.trace("Sending message to {}@{}", message.id(), target);
     }
 
+    private static Replica pickReplica(EndpointsForToken targets)
+    {
+        EndpointsForToken healthy = targets.filter(r -> DynamicEndpointSnitch.getSeverity(r.endpoint()) == 0);

Review Comment:
   I think that should be fast enough without caching.



##########
src/java/org/apache/cassandra/service/StorageService.java:
##########
@@ -5083,8 +5083,11 @@ private List<DecoratedKey> keySamples(Iterable<ColumnFamilyStore> cfses, Range<T
      */
     private void startLeaving()
     {
-        Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS_WITH_PORT, valueFactory.leaving(getLocalTokens()));
-        Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.leaving(getLocalTokens()));
+        DatabaseDescriptor.getSeverityDuringDecommission()
+                          .ifPresent(severity ->
+                                     Gossiper.instance.addLocalApplicationState(ApplicationState.SEVERITY, valueFactory.severity(severity)));

Review Comment:
   Why not call DES.setSeverity()?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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