Skip to content

Commit c78aec7

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 forced-reset 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 scenario 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. This is noted in the rabbitmqctl.8 manpage.
1 parent 211fc5b commit c78aec7

File tree

5 files changed

+19
-44
lines changed

5 files changed

+19
-44
lines changed

deps/rabbit/docs/rabbitmqctl.8

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -346,7 +346,7 @@ next time it is started:
346346
.sp
347347
.Dl rabbitmqctl force_boot
348348
.\" ------------------------------------------------------------------
349-
.It Cm force_reset
349+
.It Cm force_reset Em (deprecated)
350350
.Pp
351351
Forcefully returns a RabbitMQ node to its virgin state.
352352
.Pp
@@ -359,6 +359,13 @@ management database state and cluster configuration.
359359
It should only be used as a last resort if the database or cluster
360360
configuration has been corrupted.
361361
.Pp
362+
The
363+
.Cm force_reset
364+
command is
365+
.Sy deprecated .
366+
It remains available when the Mnesia metadata store is used.
367+
It is unsupported with the Khepri metadata store.
368+
.Pp
362369
For
363370
.Cm reset
364371
and

deps/rabbit/src/rabbit_db.erl

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -163,11 +163,13 @@ force_reset_using_mnesia() ->
163163
#{domain => ?RMQLOG_DOMAIN_DB}),
164164
rabbit_mnesia:force_reset().
165165

166+
-spec force_reset_using_khepri() -> no_return().
167+
166168
force_reset_using_khepri() ->
167-
?LOG_DEBUG(
168-
"DB: resetting node forcefully (using Khepri)",
169+
?LOG_ERROR(
170+
"DB: resetting node forcefully is unsupported with Khepri",
169171
#{domain => ?RMQLOG_DOMAIN_DB}),
170-
rabbit_khepri:force_reset().
172+
throw({error, "Forced reset is unsupported with Khepri"}).
171173

172174
-spec force_load_on_next_boot() -> Ret when
173175
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,

deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -55,7 +55,6 @@
5555
kill_node_after/3,
5656

5757
reset_node/2,
58-
force_reset_node/2,
5958

6059
forget_cluster_node/3,
6160
forget_cluster_node/4,
@@ -2159,10 +2158,6 @@ reset_node(Config, Node) ->
21592158
Name = get_node_config(Config, Node, nodename),
21602159
rabbit_control_helper:command(reset, Name).
21612160

2162-
force_reset_node(Config, Node) ->
2163-
Name = get_node_config(Config, Node, nodename),
2164-
rabbit_control_helper:command(force_reset, Name).
2165-
21662161
forget_cluster_node(Config, Node, NodeToForget) ->
21672162
forget_cluster_node(Config, Node, NodeToForget, []).
21682163
forget_cluster_node(Config, Node, NodeToForget, Opts) ->

0 commit comments

Comments
 (0)