From dd8486aafafc80846ce68a92dca16b52fa162659 Mon Sep 17 00:00:00 2001 From: Potato Date: Thu, 9 Jan 2025 01:17:10 +0800 Subject: [PATCH] RATIS-2236 Fixed bug where manual triggerSnapshot would never finish (#1207) --- .../java/org/apache/ratis/server/impl/StateMachineUpdater.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index c1db1fd3cc..a919ca732c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -216,6 +216,9 @@ private void waitForCommit() throws InterruptedException { // Thus it is possible to have applied > committed initially. final long applied = getLastAppliedIndex(); for(; applied >= raftLog.getLastCommittedIndex() && state == State.RUNNING && !shouldStop(); ) { + if (server.getSnapshotRequestHandler().shouldTriggerTakingSnapshot()) { + takeSnapshot(); + } if (awaitForSignal.await(100, TimeUnit.MILLISECONDS)) { return; }