This is an automated email from the ASF dual-hosted git repository. ifesdjeen pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/cassandra-accord.git
commit fd608bc7bc1003481f8bd9a4c7d3ebe6b55daa3c Author: Alex Petrov <[email protected]> AuthorDate: Fri Jun 6 10:48:36 2025 +0200 Make durability service listen to topology changes Patch by Alex Petrov; reviewed by Benedict Elliott Smith for CASSANDRA-20703 --- accord-core/src/main/java/accord/local/Node.java | 2 +- .../src/main/java/accord/local/durability/GlobalDurability.java | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/accord-core/src/main/java/accord/local/Node.java b/accord-core/src/main/java/accord/local/Node.java index 5bb92a5b..d1a859b9 100644 --- a/accord-core/src/main/java/accord/local/Node.java +++ b/accord-core/src/main/java/accord/local/Node.java @@ -213,6 +213,7 @@ public class Node implements ConfigurationService.Listener, NodeCommandStoreServ scheduler.recurring(() -> commandStores.forEachCommandStore(store -> store.progressLog.maybeNotify()), 1, SECONDS); scheduler.recurring(timeouts::maybeNotify, 100, MILLISECONDS); configService.registerListener(this); + configService.registerListener(durabilityService); } public Map<TxnId, AsyncResult<? extends Outcome>> coordinating() @@ -381,7 +382,6 @@ public class Node implements ConfigurationService.Listener, NodeCommandStoreServ public void onEpochRetired(Ranges ranges, long epoch) { topology.onEpochRetired(ranges, epoch); - durabilityService.onEpochRetired(ranges, epoch); } // TODO (required): audit error handling, as the refactor to provide epoch timeouts appears to have broken a number of coordination diff --git a/accord-core/src/main/java/accord/local/durability/GlobalDurability.java b/accord-core/src/main/java/accord/local/durability/GlobalDurability.java index f47c1ff3..81d48b5d 100644 --- a/accord-core/src/main/java/accord/local/durability/GlobalDurability.java +++ b/accord-core/src/main/java/accord/local/durability/GlobalDurability.java @@ -174,7 +174,9 @@ public class GlobalDurability implements Callback<Object> synchronized void updateTopology(Topology latestGlobal) { - Invariants.require(currentGlobalTopology == null || latestGlobal.epoch() > currentGlobalTopology.epoch()); + if (currentGlobalTopology != null && latestGlobal.epoch() <= currentGlobalTopology.epoch()) + return; + currentGlobalTopology = latestGlobal; List<Node.Id> ids = new ArrayList<>(latestGlobal.nodes()); Collections.sort(ids); --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
