Skip to content

Commit 2e9400a

Browse files
Merge pull request #9782 from rabbitmq/join_cluster-handles-stop-start-automatically
rabbit_db: `join/2` now takes care of stopping/starting RabbitMQ
2 parents 9950188 + a26f95d commit 2e9400a

File tree

6 files changed

+90
-20
lines changed

6 files changed

+90
-20
lines changed

deps/rabbit/src/rabbit_db_cluster.erl

Lines changed: 43 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -96,6 +96,28 @@ join(RemoteNode, NodeType)
9696
when is_atom(RemoteNode) andalso ?IS_NODE_TYPE(NodeType) ->
9797
case can_join(RemoteNode) of
9898
{ok, ClusterNodes} when is_list(ClusterNodes) ->
99+
%% RabbitMQ and Mnesia must be stopped to modify the cluster. In
100+
%% particular, we stop Mnesia regardless of the remotely selected
101+
%% database because we might change it during the join.
102+
RestartMnesia = rabbit_mnesia:is_running(),
103+
RestartFFCtl = rabbit_ff_controller:is_running(),
104+
RestartRabbit = rabbit:is_running(),
105+
case RestartRabbit of
106+
true ->
107+
rabbit:stop();
108+
false ->
109+
case RestartFFCtl of
110+
true ->
111+
ok = rabbit_ff_controller:wait_for_task_and_stop();
112+
false ->
113+
ok
114+
end,
115+
case RestartMnesia of
116+
true -> rabbit_mnesia:stop_mnesia();
117+
false -> ok
118+
end
119+
end,
120+
99121
ok = rabbit_db:reset(),
100122
rabbit_feature_flags:copy_feature_states_after_reset(RemoteNode),
101123

@@ -106,6 +128,27 @@ join(RemoteNode, NodeType)
106128
true -> join_using_khepri(ClusterNodes, NodeType);
107129
false -> join_using_mnesia(ClusterNodes, NodeType)
108130
end,
131+
132+
%% Restart RabbitMQ afterwards, if it was running before the join.
133+
%% Likewise for the Feature flags controller and Mnesia (if we
134+
%% still need it).
135+
case RestartRabbit of
136+
true ->
137+
rabbit:start();
138+
false ->
139+
case RestartFFCtl of
140+
true ->
141+
ok = rabbit_sup:start_child(rabbit_ff_controller);
142+
false ->
143+
ok
144+
end,
145+
NeedMnesia = not rabbit_khepri:is_enabled(),
146+
case RestartMnesia andalso NeedMnesia of
147+
true -> rabbit_mnesia:start_mnesia(false);
148+
false -> ok
149+
end
150+
end,
151+
109152
case Ret of
110153
ok ->
111154
rabbit_node_monitor:notify_joined_cluster(),

deps/rabbit/src/rabbit_ff_controller.erl

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -49,7 +49,8 @@
4949
running_nodes/0,
5050
collect_inventory_on_nodes/1, collect_inventory_on_nodes/2,
5151
mark_as_enabled_on_nodes/4,
52-
wait_for_task_and_stop/0]).
52+
wait_for_task_and_stop/0,
53+
is_running/0]).
5354

5455
%% gen_statem callbacks.
5556
-export([callback_mode/0,
@@ -79,7 +80,13 @@ start_link() ->
7980
gen_statem:start_link({local, ?LOCAL_NAME}, ?MODULE, none, []).
8081

8182
wait_for_task_and_stop() ->
82-
gen_statem:stop(?LOCAL_NAME).
83+
case erlang:whereis(rabbit_sup) of
84+
undefined -> gen_statem:stop(?LOCAL_NAME);
85+
_ -> rabbit_sup:stop_child(?LOCAL_NAME)
86+
end.
87+
88+
is_running() ->
89+
is_pid(erlang:whereis(?LOCAL_NAME)).
8390

8491
is_supported(FeatureNames) ->
8592
is_supported(FeatureNames, ?TIMEOUT).
@@ -176,6 +183,9 @@ callback_mode() ->
176183
state_functions.
177184

178185
init(_Args) ->
186+
?LOG_DEBUG(
187+
"Feature flags: controller standing by",
188+
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
179189
{ok, standing_by, none}.
180190

181191
standing_by(

deps/rabbit/src/rabbit_mnesia.erl

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -177,7 +177,6 @@ create_cluster_callback(RemoteNode, NodeType) ->
177177
-> {ok, [node()]} | {ok, already_member} | {error, {inconsistent_cluster, string()}}.
178178

179179
can_join_cluster(DiscoveryNode) ->
180-
ensure_mnesia_not_running(),
181180
ensure_mnesia_dir(),
182181
case is_only_clustered_disc_node() of
183182
true -> e(clustering_only_disc_node);
@@ -225,6 +224,7 @@ join_cluster(ClusterNodes, NodeType) when is_list(ClusterNodes) ->
225224
join_cluster(DiscoveryNode, NodeType) when is_atom(DiscoveryNode) ->
226225
%% Code to remain compatible with `change_cluster_node_type/1' and older
227226
%% CLI.
227+
ensure_mnesia_not_running(),
228228
case can_join_cluster(DiscoveryNode) of
229229
{ok, ClusterNodes} when is_list(ClusterNodes) ->
230230
ok = reset_gracefully(),

deps/rabbit/test/clustering_management_SUITE.erl

Lines changed: 34 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -269,7 +269,30 @@ join_and_part_cluster(Config) ->
269269

270270
stop_reset_start(Config, Hare),
271271
assert_not_clustered(Hare),
272-
assert_not_clustered(Bunny).
272+
assert_not_clustered(Bunny),
273+
274+
%% Using `join_cluster` is allowed without stopping `rabbit` first since
275+
%% 3.13.0. It will only work if all nodes support it: check if they all
276+
%% expose `rabbit_ff_controller:is_running/0`.
277+
Supported = lists:all(
278+
fun(R) -> R end,
279+
rabbit_ct_broker_helpers:rpc_all(
280+
Config, erlang, function_exported,
281+
[rabbit_ff_controller, is_running, 0])),
282+
ct:pal(
283+
"Do all nodes support `join_cluster` without stopping `rabbit` "
284+
"first? ~p",
285+
[Supported]),
286+
case Supported of
287+
true ->
288+
?assertEqual(ok, join_cluster(Config, Rabbit, Bunny)),
289+
assert_clustered([Rabbit, Bunny]),
290+
291+
?assertEqual(ok, join_cluster(Config, Hare, Bunny)),
292+
assert_clustered([Rabbit, Bunny, Hare]);
293+
false ->
294+
ok
295+
end.
273296

274297
stop_start_cluster_node(Config) ->
275298
[Rabbit, Hare] = cluster_members(Config),
@@ -326,7 +349,15 @@ join_and_part_cluster_in_khepri(Config) ->
326349

327350
stop_reset_start(Config, Rabbit),
328351
assert_not_clustered(Rabbit),
329-
assert_not_clustered(Hare).
352+
assert_not_clustered(Hare),
353+
354+
%% Using `join_cluster` is allowed without stopping `rabbit` first since
355+
%% 3.13.0.
356+
?assertEqual(ok, join_cluster(Config, Rabbit, Bunny)),
357+
assert_clustered([Rabbit, Bunny]),
358+
359+
?assertEqual(ok, join_cluster(Config, Hare, Bunny)),
360+
assert_clustered([Rabbit, Bunny, Hare]).
330361

331362
join_cluster_bad_operations(Config) ->
332363
[Rabbit, Hare, Bunny] = cluster_members(Config),
@@ -342,10 +373,6 @@ join_cluster_bad_operations(Config) ->
342373
ok = start_app(Config, Rabbit),
343374
assert_not_clustered(Rabbit),
344375

345-
%% Trying to cluster with mnesia running
346-
assert_failure(fun () -> join_cluster(Config, Rabbit, Bunny) end),
347-
assert_not_clustered(Rabbit),
348-
349376
%% Trying to cluster the node with itself
350377
ok = stop_app(Config, Rabbit),
351378
assert_failure(fun () -> join_cluster(Config, Rabbit, Rabbit) end),
@@ -379,18 +406,14 @@ join_cluster_bad_operations(Config) ->
379406
ok.
380407

381408
join_cluster_bad_operations_in_khepri(Config) ->
382-
[Rabbit, _Hare, Bunny] = cluster_members(Config),
409+
[Rabbit, _Hare, _Bunny] = cluster_members(Config),
383410

384411
%% Nonexistent node
385412
ok = stop_app(Config, Rabbit),
386413
assert_failure(fun () -> join_cluster(Config, Rabbit, non@existent) end),
387414
ok = start_app(Config, Rabbit),
388415
assert_not_clustered(Rabbit),
389416

390-
%% Trying to cluster with mnesia running
391-
assert_failure(fun () -> join_cluster(Config, Rabbit, Bunny) end),
392-
assert_not_clustered(Rabbit),
393-
394417
%% Trying to cluster the node with itself
395418
ok = stop_app(Config, Rabbit),
396419
assert_failure(fun () -> join_cluster(Config, Rabbit, Rabbit) end),

deps/rabbit_common/mk/rabbitmq-run.mk

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -392,12 +392,8 @@ start-brokers start-cluster: $(DIST_TARGET)
392392
for n in $$(seq $(NODES)); do \
393393
nodename="rabbit-$$n@$(HOSTNAME)"; \
394394
if test '$@' = 'start-cluster' && test "$$nodename1"; then \
395-
ERL_LIBS="$(DIST_ERL_LIBS)" \
396-
$(RABBITMQCTL) -n "$$nodename" stop_app; \
397395
ERL_LIBS="$(DIST_ERL_LIBS)" \
398396
$(RABBITMQCTL) -n "$$nodename" join_cluster "$$nodename1"; \
399-
ERL_LIBS="$(DIST_ERL_LIBS)" \
400-
$(RABBITMQCTL) -n "$$nodename" start_app; \
401397
else \
402398
nodename1=$$nodename; \
403399
fi; \

deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/join_cluster_command.ex

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -28,8 +28,6 @@ defmodule RabbitMQ.CLI.Ctl.Commands.JoinClusterCommand do
2828
def validate([_], _), do: :ok
2929
def validate(_, _), do: {:validation_failure, :too_many_args}
3030

31-
use RabbitMQ.CLI.Core.RequiresRabbitAppStopped
32-
3331
def run([target_node], %{node: node_name, ram: ram, disc: disc} = opts) do
3432
node_type =
3533
case {ram, disc} do

0 commit comments

Comments
 (0)