pauloricardomg commented on code in PR #4547:
URL: https://github.com/apache/cassandra/pull/4547#discussion_r2699976273
##########
src/java/org/apache/cassandra/service/disk/usage/DiskUsageBroadcaster.java:
##########
@@ -131,6 +166,85 @@ private boolean computeHasStuffedOrFullNode()
return false;
}
+ /**
+ * Update the set of full nodes by datacenter based on the disk usage
state for the given endpoint.
+ * If the node is FULL, add it to the set for its datacenter. Otherwise,
remove it from the set.
+ * This method is idempotent - adding an already-present node or removing
an absent node has no effect.
+ *
+ * @param endpoint The endpoint whose state has changed.
+ * @param usageState The new disk usage state value.
+ */
+ private void computeUsageStateForEpDatacenter(InetAddressAndPort endpoint,
DiskUsageState usageState)
+ {
+ Location location = location(endpoint);
+ if (location.equals(Location.UNKNOWN))
+ {
+ noSpamLogger.warn("Unable to track disk usage by datacenter for
endpoint {} because we are unable to determine its location.",
+ endpoint);
+ return;
+ }
+
+ String datacenter = location.datacenter;
+ if (usageState.isFull())
+ {
+ // Add this node to the set of full nodes for its datacenter and
remove it from the stuffed nodes
+ // if it was there.
+ fullNodesByDatacenter.computeIfAbsent(datacenter, dc ->
ConcurrentHashMap.newKeySet())
+ .add(endpoint);
+ noSpamLogger.debug("Endpoint {} is FULL, added to full nodes set
for datacenter {}", endpoint, datacenter);
+ Set<InetAddressAndPort> stuffedNodes =
stuffedNodesByDatacenter.get(datacenter);
+ if (stuffedNodes != null && stuffedNodes.remove(endpoint))
+ {
+ noSpamLogger.debug("Endpoint {} is now FULL. Removed it from
the stuffed nodes set for datacenter {}",
+ endpoint, datacenter);
+ }
+ }
+ else if (usageState.isStuffed())
+ {
+ // Add this node to the set of stuffed nodes for its datacenter
and remove it from the full nodes
+ // if it was there.
+ stuffedNodesByDatacenter.computeIfAbsent(datacenter, dc ->
ConcurrentHashMap.newKeySet())
+ .add(endpoint);
+ noSpamLogger.debug("Endpoint {} is now STUFFED. Added it to the
stuffed nodes set for datacenter {}",
+ endpoint, datacenter);
+ Set<InetAddressAndPort> fullNodes =
fullNodesByDatacenter.get(datacenter);
+ if (fullNodes != null && fullNodes.remove(endpoint))
+ {
+ noSpamLogger.debug("Endpoint {} is now STUFFED. Removed it
from full nodes set for datacenter {}",
+ endpoint, datacenter);
+ }
+ }
+ else
+ {
+ // Remove this node from the set of full nodes and set of stuffed
nodes for its datacenter if it was there.
+ Set<InetAddressAndPort> fullNodes =
fullNodesByDatacenter.get(datacenter);
+ if (fullNodes != null && (fullNodes.remove(endpoint)))
Review Comment:
remove extra parenthesis
##########
src/java/org/apache/cassandra/service/disk/usage/DiskUsageBroadcaster.java:
##########
@@ -131,6 +166,85 @@ private boolean computeHasStuffedOrFullNode()
return false;
}
+ /**
+ * Update the set of full nodes by datacenter based on the disk usage
state for the given endpoint.
+ * If the node is FULL, add it to the set for its datacenter. Otherwise,
remove it from the set.
+ * This method is idempotent - adding an already-present node or removing
an absent node has no effect.
+ *
+ * @param endpoint The endpoint whose state has changed.
+ * @param usageState The new disk usage state value.
+ */
+ private void computeUsageStateForEpDatacenter(InetAddressAndPort endpoint,
DiskUsageState usageState)
+ {
+ Location location = location(endpoint);
+ if (location.equals(Location.UNKNOWN))
+ {
+ noSpamLogger.warn("Unable to track disk usage by datacenter for
endpoint {} because we are unable to determine its location.",
+ endpoint);
+ return;
+ }
+
+ String datacenter = location.datacenter;
+ if (usageState.isFull())
+ {
+ // Add this node to the set of full nodes for its datacenter and
remove it from the stuffed nodes
+ // if it was there.
+ fullNodesByDatacenter.computeIfAbsent(datacenter, dc ->
ConcurrentHashMap.newKeySet())
+ .add(endpoint);
+ noSpamLogger.debug("Endpoint {} is FULL, added to full nodes set
for datacenter {}", endpoint, datacenter);
+ Set<InetAddressAndPort> stuffedNodes =
stuffedNodesByDatacenter.get(datacenter);
+ if (stuffedNodes != null && stuffedNodes.remove(endpoint))
+ {
+ noSpamLogger.debug("Endpoint {} is now FULL. Removed it from
the stuffed nodes set for datacenter {}",
+ endpoint, datacenter);
+ }
+ }
+ else if (usageState.isStuffed())
+ {
+ // Add this node to the set of stuffed nodes for its datacenter
and remove it from the full nodes
+ // if it was there.
+ stuffedNodesByDatacenter.computeIfAbsent(datacenter, dc ->
ConcurrentHashMap.newKeySet())
+ .add(endpoint);
+ noSpamLogger.debug("Endpoint {} is now STUFFED. Added it to the
stuffed nodes set for datacenter {}",
+ endpoint, datacenter);
+ Set<InetAddressAndPort> fullNodes =
fullNodesByDatacenter.get(datacenter);
+ if (fullNodes != null && fullNodes.remove(endpoint))
+ {
+ noSpamLogger.debug("Endpoint {} is now STUFFED. Removed it
from full nodes set for datacenter {}",
+ endpoint, datacenter);
+ }
+ }
+ else
+ {
+ // Remove this node from the set of full nodes and set of stuffed
nodes for its datacenter if it was there.
+ Set<InetAddressAndPort> fullNodes =
fullNodesByDatacenter.get(datacenter);
+ if (fullNodes != null && (fullNodes.remove(endpoint)))
+ {
+ noSpamLogger.debug("Endpoint {} is no longer STUFFED or FULL,
removed from stuffed for datacenter {}",
+ endpoint, datacenter);
+ }
+ Set<InetAddressAndPort> stuffedNodes =
stuffedNodesByDatacenter.get(datacenter);
+ if (stuffedNodes != null && (stuffedNodes.remove(endpoint)))
+ {
+ noSpamLogger.debug("Endpoint {} is not longer STUFFED, removed
from the stuffed set for datacenter {}",
Review Comment:
typo: no longer
##########
src/java/org/apache/cassandra/db/guardrails/Guardrails.java:
##########
@@ -556,6 +556,19 @@ public final class Guardrails implements GuardrailsMBean
(isWarning, value) ->
isWarning ? "Replica disk usage exceeds warning threshold"
: "Write request failed because disk usage
exceeds failure threshold");
+ /**
+ * Guardrail on the data disk usage of replicas across a datacenter which
replicates a given keyspace.
+ * This is used at write time to verify the status of any node which might
replicate a given keyspace.
+ */
+ public static final Predicates<String> diskUsageKeyspaceWideProtection =
+ new Predicates<>("disk_usage_keyspace_wide_protection",
+ null,
+ state ->
DiskUsageBroadcaster.instance::isDatacenterStuffed,
+ state -> DiskUsageBroadcaster.instance::isDatacenterFull,
+ (isWarning, value) ->
+ isWarning ? "Disk usage in datacenter exceeds warning
threshold"
Review Comment:
I think the message should be "disk usage exceeds warning threshold in
keyspace datacenter", since it could be wrongly be interpreted that it's in the
coordinator's datacenter while it could be in another dc.
##########
src/java/org/apache/cassandra/service/disk/usage/DiskUsageBroadcaster.java:
##########
@@ -131,6 +166,85 @@ private boolean computeHasStuffedOrFullNode()
return false;
}
+ /**
+ * Update the set of full nodes by datacenter based on the disk usage
state for the given endpoint.
+ * If the node is FULL, add it to the set for its datacenter. Otherwise,
remove it from the set.
+ * This method is idempotent - adding an already-present node or removing
an absent node has no effect.
+ *
+ * @param endpoint The endpoint whose state has changed.
+ * @param usageState The new disk usage state value.
+ */
+ private void computeUsageStateForEpDatacenter(InetAddressAndPort endpoint,
DiskUsageState usageState)
+ {
+ Location location = location(endpoint);
+ if (location.equals(Location.UNKNOWN))
+ {
+ noSpamLogger.warn("Unable to track disk usage by datacenter for
endpoint {} because we are unable to determine its location.",
+ endpoint);
+ return;
+ }
+
+ String datacenter = location.datacenter;
+ if (usageState.isFull())
+ {
+ // Add this node to the set of full nodes for its datacenter and
remove it from the stuffed nodes
+ // if it was there.
+ fullNodesByDatacenter.computeIfAbsent(datacenter, dc ->
ConcurrentHashMap.newKeySet())
+ .add(endpoint);
+ noSpamLogger.debug("Endpoint {} is FULL, added to full nodes set
for datacenter {}", endpoint, datacenter);
+ Set<InetAddressAndPort> stuffedNodes =
stuffedNodesByDatacenter.get(datacenter);
+ if (stuffedNodes != null && stuffedNodes.remove(endpoint))
+ {
+ noSpamLogger.debug("Endpoint {} is now FULL. Removed it from
the stuffed nodes set for datacenter {}",
+ endpoint, datacenter);
+ }
+ }
+ else if (usageState.isStuffed())
+ {
+ // Add this node to the set of stuffed nodes for its datacenter
and remove it from the full nodes
+ // if it was there.
+ stuffedNodesByDatacenter.computeIfAbsent(datacenter, dc ->
ConcurrentHashMap.newKeySet())
+ .add(endpoint);
+ noSpamLogger.debug("Endpoint {} is now STUFFED. Added it to the
stuffed nodes set for datacenter {}",
+ endpoint, datacenter);
+ Set<InetAddressAndPort> fullNodes =
fullNodesByDatacenter.get(datacenter);
+ if (fullNodes != null && fullNodes.remove(endpoint))
+ {
+ noSpamLogger.debug("Endpoint {} is now STUFFED. Removed it
from full nodes set for datacenter {}",
+ endpoint, datacenter);
+ }
+ }
+ else
+ {
+ // Remove this node from the set of full nodes and set of stuffed
nodes for its datacenter if it was there.
+ Set<InetAddressAndPort> fullNodes =
fullNodesByDatacenter.get(datacenter);
+ if (fullNodes != null && (fullNodes.remove(endpoint)))
+ {
+ noSpamLogger.debug("Endpoint {} is no longer STUFFED or FULL,
removed from stuffed for datacenter {}",
+ endpoint, datacenter);
+ }
+ Set<InetAddressAndPort> stuffedNodes =
stuffedNodesByDatacenter.get(datacenter);
+ if (stuffedNodes != null && (stuffedNodes.remove(endpoint)))
Review Comment:
remove extra parenthesis
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]