This is an automated email from the ASF dual-hosted git repository.
asf-gitbox-commits pushed a commit to branch cassandra-6.0
in repository https://gitbox.apache.org/repos/asf/cassandra.git
The following commit(s) were added to refs/heads/cassandra-6.0 by this push:
new 7927ac4395 Bring back the ability to Optionally avoid hint transfer
during decommission (CASSANDRA-17808) lost due to merge of ae0842372f
7927ac4395 is described below
commit 7927ac4395cb1b68794a163baaa3e9c5553166c9
Author: Matt Byrd <[email protected]>
AuthorDate: Thu May 14 13:56:26 2026 +0100
Bring back the ability to Optionally avoid hint transfer during
decommission (CASSANDRA-17808)
lost due to merge of ae0842372f
patch by Matt Byrd; reviewed by Stefan Miklosovic and Sam Tunnicliffe for
CASSANDRA-21341
---
CHANGES.txt | 1 +
.../tcm/sequences/UnbootstrapStreams.java | 18 ++++++++++++++++--
.../test/HintedHandoffAddRemoveNodesTest.java | 22 ++++++++++++----------
3 files changed, 29 insertions(+), 12 deletions(-)
diff --git a/CHANGES.txt b/CHANGES.txt
index 5af655765b..dce5f7f77f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
6.0-alpha2
+ * Restore option to avoid hint transfer during decommission (CASSANDRA-21341)
* Add an offline cluster metadata tool (CASSANDRA-19151)
* Accord: Tail Latency Improvements (CASSANDRA-21361)
* Artificial Latency Injection (CASSANDRA-17024)
diff --git
a/src/java/org/apache/cassandra/tcm/sequences/UnbootstrapStreams.java
b/src/java/org/apache/cassandra/tcm/sequences/UnbootstrapStreams.java
index e62cf77dfa..4a38530753 100644
--- a/src/java/org/apache/cassandra/tcm/sequences/UnbootstrapStreams.java
+++ b/src/java/org/apache/cassandra/tcm/sequences/UnbootstrapStreams.java
@@ -30,9 +30,11 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.batchlog.BatchlogManager;
+import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.hints.HintsService;
import org.apache.cassandra.locator.EndpointsByReplica;
import org.apache.cassandra.locator.InetAddressAndPort;
import org.apache.cassandra.locator.RangesAtEndpoint;
@@ -50,6 +52,7 @@ import org.apache.cassandra.tcm.membership.NodeId;
import org.apache.cassandra.tcm.ownership.MovementMap;
import org.apache.cassandra.tcm.ownership.PlacementDeltas;
import org.apache.cassandra.utils.concurrent.Future;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
public class UnbootstrapStreams implements LeaveStreams
{
@@ -125,9 +128,20 @@ public class UnbootstrapStreams implements LeaveStreams
logger.debug("waiting for batch log processing.");
batchlogReplay.get();
- logger.info("streaming hints to other nodes");
+ Future<?> hintsSuccess = ImmediateFuture.success(null);
- Future<?> hintsSuccess = StorageService.instance.streamHints();
+ if (DatabaseDescriptor.getTransferHintsOnDecommission())
+ {
+ logger.info("streaming hints to other nodes");
+ hintsSuccess = StorageService.instance.streamHints();
+ }
+ else
+ {
+ logger.info("pausing dispatch and deleting hints");
+ DatabaseDescriptor.setHintedHandoffEnabled(false);
+ HintsService.instance.pauseDispatch();
+ HintsService.instance.deleteAllHints();
+ }
// wait for the transfer runnables to signal the latch.
logger.debug("waiting for stream acks.");
diff --git
a/test/distributed/org/apache/cassandra/distributed/test/HintedHandoffAddRemoveNodesTest.java
b/test/distributed/org/apache/cassandra/distributed/test/HintedHandoffAddRemoveNodesTest.java
index e62fb4af1a..aa08dcb329 100644
---
a/test/distributed/org/apache/cassandra/distributed/test/HintedHandoffAddRemoveNodesTest.java
+++
b/test/distributed/org/apache/cassandra/distributed/test/HintedHandoffAddRemoveNodesTest.java
@@ -71,25 +71,27 @@ public class HintedHandoffAddRemoveNodesTest extends
TestBaseImpl
{
cluster.schemaChange(withKeyspace("CREATE TABLE
%s.decom_no_hints_test (key int PRIMARY KEY, value int)"));
- cluster.coordinator(1).execute(withKeyspace("INSERT INTO
%s.decom_no_hints_test (key, value) VALUES (?, ?)"), ALL, 0, 0);
- long hintsBeforeShutdown = countTotalHints(cluster.get(1));
+ int secondNode = 2;
+ cluster.coordinator(secondNode).execute(withKeyspace("INSERT INTO
%s.decom_no_hints_test (key, value) VALUES (?, ?)"), ALL, 0, 0);
+ long hintsBeforeShutdown =
countTotalHints(cluster.get(secondNode));
assertThat(hintsBeforeShutdown).isEqualTo(0);
- long hintsDelivered = countHintsDelivered(cluster.get(1));
+ long hintsDelivered = countHintsDelivered(cluster.get(secondNode));
assertThat(hintsDelivered).isEqualTo(0);
// Shutdown node 3 so hints can be written against it.
cluster.get(3).shutdown().get();
- cluster.coordinator(1).execute(withKeyspace("INSERT INTO
%s.decom_no_hints_test (key, value) VALUES (?, ?)"), TWO, 0, 0);
- Awaitility.await().until(() -> countTotalHints(cluster.get(1)) >
0);
- long hintsAfterShutdown = countTotalHints(cluster.get(1));
+ cluster.coordinator(secondNode).execute(withKeyspace("INSERT INTO
%s.decom_no_hints_test (key, value) VALUES (?, ?)"), TWO, 0, 0);
+ Awaitility.await().until(() ->
countTotalHints(cluster.get(secondNode)) > 0);
+ long hintsAfterShutdown = countTotalHints(cluster.get(secondNode));
assertThat(hintsAfterShutdown).isEqualTo(1);
- cluster.get(2).runOnInstance(() ->
setProgressBarrierMinConsistencyLevel(org.apache.cassandra.db.ConsistencyLevel.ONE));
+ cluster.get(secondNode).runOnInstance(() ->
setProgressBarrierMinConsistencyLevel(org.apache.cassandra.db.ConsistencyLevel.ONE));
+
ClusterUtils.waitForCMSToQuiesce(cluster, cluster.get(1), 3);
- cluster.get(2).nodetoolResult("decommission",
"--force").asserts().success();
- long hintsDeliveredByDecom = countHintsDelivered(cluster.get(2));
- String mode = cluster.get(2).callOnInstance(() ->
StorageService.instance.getOperationMode());
+ cluster.get(secondNode).nodetoolResult("decommission",
"--force").asserts().success();
+ long hintsDeliveredByDecom =
countHintsDelivered(cluster.get(secondNode));
+ String mode = cluster.get(secondNode).callOnInstance(() ->
StorageService.instance.getOperationMode());
assertEquals(StorageService.Mode.DECOMMISSIONED.toString(), mode);
assertThat(hintsDeliveredByDecom).isEqualTo(0);
}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]