Skip to content

Commit 4be4736

Browse files
committed
rabbit_db: force_reset command is unsupported with Khepri
[Why] The `force_reset` command simply removes local files on disk for the local node. In the case of Ra, this can't work because the rest of the cluster does not know about the forced-reset node. Therefore the leader will continue to send `append_entry` commands to the reset node. If that node restarts and receives these messages, it will either join the cluster again (because it's on an older Raft term) or it will hit an assertion and exit (because it's on the same Raft term). [How] Given we can't really support this scenarion and it has little value, the command will now return an error if someone attemps a `force_reset` with a node running Khepri. This also deprecates the command: once Mnesia support is removed, the command will be removed at the same time.
1 parent 3a17473 commit 4be4736

File tree

3 files changed

+9
-38
lines changed

3 files changed

+9
-38
lines changed

deps/rabbit/src/rabbit_db.erl

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -164,10 +164,10 @@ force_reset_using_mnesia() ->
164164
rabbit_mnesia:force_reset().
165165

166166
force_reset_using_khepri() ->
167-
?LOG_DEBUG(
168-
"DB: resetting node forcefully (using Khepri)",
167+
?LOG_ERROR(
168+
"DB: resetting node forcefully is unsupported with Khepri",
169169
#{domain => ?RMQLOG_DOMAIN_DB}),
170-
rabbit_khepri:force_reset().
170+
throw({error, "Forced reset is unsupported with Khepri"}).
171171

172172
-spec force_load_on_next_boot() -> Ret when
173173
Ret :: ok.

deps/rabbit/src/rabbit_khepri.erl

Lines changed: 1 addition & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -168,8 +168,7 @@
168168
-export([check_cluster_consistency/0,
169169
check_cluster_consistency/2,
170170
node_info/0]).
171-
-export([reset/0,
172-
force_reset/0]).
171+
-export([reset/0]).
173172
-export([cluster_status_from_khepri/0,
174173
cli_cluster_status/0]).
175174

@@ -601,23 +600,6 @@ reset() ->
601600

602601
%% @private
603602

604-
force_reset() ->
605-
case rabbit:is_running() of
606-
false ->
607-
ok = khepri:stop(?RA_CLUSTER_NAME),
608-
DataDir = maps:get(data_dir, ra_system:fetch(?RA_SYSTEM)),
609-
ok = rabbit_ra_systems:ensure_ra_system_stopped(?RA_SYSTEM),
610-
ok = rabbit_file:recursive_delete(
611-
filelib:wildcard(DataDir ++ "/*")),
612-
613-
_ = file:delete(rabbit_guid:filename()),
614-
ok;
615-
true ->
616-
throw({error, rabbitmq_unexpectedly_running})
617-
end.
618-
619-
%% @private
620-
621603
force_shrink_member_to_current_member() ->
622604
ok = ra_server_proc:force_shrink_members_to_current_member(
623605
{?RA_CLUSTER_NAME, node()}).

deps/rabbit/test/clustering_management_SUITE.erl

Lines changed: 5 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -953,22 +953,11 @@ force_reset_node_in_khepri(Config) ->
953953

954954
stop_join_start(Config, Rabbit, Hare),
955955
stop_app(Config, Rabbit),
956-
ok = force_reset(Config, Rabbit),
957-
assert_cluster_status({[Rabbit, Hare], [Rabbit, Hare], [Hare]}, [Hare]),
958-
%% Khepri is stopped, so it won't report anything.
959-
assert_status({[Rabbit], [], [Rabbit], [Rabbit], []}, [Rabbit]),
960-
%% Hare thinks that Rabbit is still clustered
961-
assert_cluster_status({[Rabbit, Hare], [Rabbit, Hare], [Hare]},
962-
[Hare]),
963-
ok = start_app(Config, Rabbit),
964-
assert_not_clustered(Rabbit),
965-
%% We can rejoin Rabbit and Hare. Unlike with Mnesia, we try to solve the
966-
%% inconsistency instead of returning an error.
967-
ok = stop_app(Config, Rabbit),
968-
?assertEqual(ok, join_cluster(Config, Rabbit, Hare, false)),
969-
ok = start_app(Config, Rabbit),
970-
assert_cluster_status({[Rabbit, Hare], [Rabbit, Hare], [Rabbit, Hare]},
971-
[Rabbit, Hare]).
956+
{error, 69, Msg} = force_reset(Config, Rabbit),
957+
?assertEqual(
958+
match,
959+
re:run(
960+
Msg, "Forced reset is unsupported with Khepri", [{capture, none}])).
972961

973962
status_with_alarm(Config) ->
974963
[Rabbit, Hare] = rabbit_ct_broker_helpers:get_node_configs(Config,

0 commit comments

Comments
 (0)