Skip to content

Commit 85286f4

Browse files
committed
rabbitmq_shovel: Add testcase for #9894
[Why] An upgrade scenario going from RabbitMQ 3.11.24 to the upcoming 3.12.8 was shared in issue #9894 to demonstrate that the change of child ID format broke rolling upgrades when there are existing dynamic shovels. [How] The testcase uses 4 nodes: * one reference node * one node to host source and target queues * one "old" node * one "new" node The reference node is using the new version to see what format it uses. The node hosting queues is using the old version but it is not relevant for this one? The testcase uses the old node to create the dynamic shovel, then the new node to simulate an upgrade by clustering it with the old node and stopping the old one.
1 parent 455a5a2 commit 85286f4

File tree

4 files changed

+320
-12
lines changed

4 files changed

+320
-12
lines changed

deps/rabbitmq_shovel/BUILD.bazel

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -164,6 +164,16 @@ rabbitmq_suite(
164164
],
165165
)
166166

167+
rabbitmq_integration_suite(
168+
name = "rolling_upgrade_SUITE",
169+
additional_beam = [
170+
"test/shovel_test_utils.beam",
171+
],
172+
deps = [
173+
"@khepri//:erlang_app",
174+
],
175+
)
176+
167177
rabbitmq_integration_suite(
168178
name = "shovel_status_command_SUITE",
169179
additional_beam = [

deps/rabbitmq_shovel/app.bzl

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -225,6 +225,15 @@ def test_suite_beam_files(name = "test_suite_beam_files"):
225225
erlc_opts = "//:test_erlc_opts",
226226
deps = ["//deps/rabbit_common:erlang_app"],
227227
)
228+
erlang_bytecode(
229+
name = "rolling_upgrade_SUITE_beam_files",
230+
testonly = True,
231+
srcs = ["test/rolling_upgrade_SUITE.erl"],
232+
outs = ["test/rolling_upgrade_SUITE.beam"],
233+
app_name = "rabbitmq_shovel",
234+
erlc_opts = "//:test_erlc_opts",
235+
deps = ["//deps/amqp_client:erlang_app", "@khepri//:erlang_app"],
236+
)
228237
erlang_bytecode(
229238
name = "shovel_status_command_SUITE_beam_files",
230239
testonly = True,
Lines changed: 268 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,268 @@
1+
%% This Source Code Form is subject to the terms of the Mozilla Public
2+
%% License, v. 2.0. If a copy of the MPL was not distributed with this
3+
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4+
%%
5+
%% Copyright (c) 2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
6+
%%
7+
8+
-module(rolling_upgrade_SUITE).
9+
10+
-include_lib("common_test/include/ct.hrl").
11+
-include_lib("eunit/include/eunit.hrl").
12+
-include_lib("amqp_client/include/amqp_client.hrl").
13+
-include_lib("khepri/include/khepri.hrl").
14+
15+
-export([suite/0,
16+
all/0,
17+
groups/0,
18+
init_per_suite/1,
19+
end_per_suite/1,
20+
init_per_group/2,
21+
end_per_group/2,
22+
init_per_testcase/2,
23+
end_per_testcase/2,
24+
25+
child_id_format/1]).
26+
27+
suite() ->
28+
[{timetrap, {minutes, 5}}].
29+
30+
all() ->
31+
[
32+
{group, mnesia_store},
33+
{group, khepri_store}
34+
].
35+
36+
groups() ->
37+
[{mnesia_store, [], [child_id_format]},
38+
{khepri_store, [], [child_id_format]}].
39+
40+
%% -------------------------------------------------------------------
41+
%% Testsuite setup/teardown.
42+
%% -------------------------------------------------------------------
43+
44+
init_per_suite(Config) ->
45+
rabbit_ct_helpers:log_environment(),
46+
rabbit_ct_helpers:run_setup_steps(Config).
47+
48+
end_per_suite(Config) ->
49+
rabbit_ct_helpers:run_teardown_steps(Config).
50+
51+
init_per_group(mnesia_store, Config) ->
52+
rabbit_ct_helpers:set_config(Config, [{metadata_store__manual, mnesia}]);
53+
init_per_group(khepri_store, Config) ->
54+
rabbit_ct_helpers:set_config(Config, [{metadata_store__manual, khepri}]).
55+
56+
end_per_group(_, Config) ->
57+
Config.
58+
59+
init_per_testcase(Testcase, Config) ->
60+
rabbit_ct_helpers:testcase_started(Config, Testcase),
61+
ClusterSize = 4,
62+
TestNumber = rabbit_ct_helpers:testcase_number(Config, ?MODULE, Testcase),
63+
Config1 = rabbit_ct_helpers:set_config(
64+
Config,
65+
[{rmq_nodes_count, ClusterSize},
66+
{rmq_nodes_clustered, false},
67+
{rmq_nodename_suffix, Testcase},
68+
{tcp_ports_base, {skip_n_nodes, TestNumber * ClusterSize}},
69+
{ignored_crashes,
70+
["process is stopped by supervisor",
71+
"broker forced connection closure with reason 'shutdown'"]}
72+
]),
73+
rabbit_ct_helpers:run_steps(
74+
Config1,
75+
rabbit_ct_broker_helpers:setup_steps() ++
76+
rabbit_ct_client_helpers:setup_steps()).
77+
78+
end_per_testcase(Testcase, Config) ->
79+
rabbit_ct_helpers:run_steps(
80+
Config,
81+
rabbit_ct_client_helpers:teardown_steps() ++
82+
rabbit_ct_broker_helpers:teardown_steps()),
83+
rabbit_ct_helpers:testcase_finished(Config, Testcase).
84+
85+
%% -------------------------------------------------------------------
86+
%% Testcases.
87+
%% -------------------------------------------------------------------
88+
89+
child_id_format(Config) ->
90+
[NewRefNode,
91+
OldNode,
92+
NewNode,
93+
NodeWithQueues] = rabbit_ct_broker_helpers:get_node_configs(
94+
Config, nodename),
95+
96+
%% We build this test on the assumption that `rabbit_ct_broker_helpers'
97+
%% starts nodes this way:
98+
%% Node 1: the primary copy of RabbitMQ the test is started from
99+
%% Node 2: the secondary umbrella (if any)
100+
%% Node 3: the primary copy
101+
%% Node 4: the secondary umbrella
102+
%% ...
103+
%%
104+
%% Therefore, `Pouet' will use the primary copy, `OldNode' the secondary
105+
%% umbrella, `NewRefNode' the primary copy, and `NodeWithQueues' the
106+
%% secondary umbrella.
107+
108+
%% Declare source and target queues on a node that won't run the shovel.
109+
ct:pal("Declaring queues on node ~s", [NodeWithQueues]),
110+
SourceQName = <<"source-queue">>,
111+
TargetQName = <<"target-queue">>,
112+
{Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(
113+
Config, NodeWithQueues),
114+
lists:foreach(
115+
fun(QName) ->
116+
?assertEqual(
117+
{'queue.declare_ok', QName, 0, 0},
118+
amqp_channel:call(
119+
Ch, #'queue.declare'{queue = QName, durable = true}))
120+
end, [SourceQName, TargetQName]),
121+
rabbit_ct_client_helpers:close_channel(Ch),
122+
rabbit_ct_client_helpers:close_connection(Conn),
123+
124+
%% Declare a dynamic shovel on the old node.
125+
ct:pal("Declaring queues on node ~s", [OldNode]),
126+
VHost = <<"/">>,
127+
ShovelName = <<"test-shovel">>,
128+
shovel_test_utils:set_param(
129+
Config,
130+
OldNode,
131+
NodeWithQueues,
132+
ShovelName,
133+
[{<<"src-queue">>, SourceQName},
134+
{<<"dest-queue">>, TargetQName}]),
135+
136+
%% We declare the same shovel on a new node that won't be clustered with
137+
%% the rest. It is only used as a reference node to determine which ID
138+
%% format the new version is using.
139+
ct:pal("Declaring queues on node ~s (as a reference)", [NewRefNode]),
140+
shovel_test_utils:set_param(
141+
Config,
142+
NewRefNode,
143+
NodeWithQueues,
144+
ShovelName,
145+
[{<<"src-queue">>, SourceQName},
146+
{<<"dest-queue">>, TargetQName}]),
147+
148+
%% Verify the format of the child ID. Some versions of RabbitMQ 3.11.x and
149+
%% 3.12.x use a temporary experimental format that was erroneously
150+
%% backported from a work-in-progress happening in the main branch.
151+
ct:pal("Checking mirrored_supervisor child ID formats"),
152+
[{Id0, _, _, _}] = rabbit_ct_broker_helpers:rpc(
153+
Config, NewRefNode,
154+
mirrored_supervisor, which_children,
155+
[rabbit_shovel_dyn_worker_sup_sup]),
156+
PrimaryIdType = case Id0 of
157+
{VHost, ShovelName} ->
158+
ct:pal(
159+
"The nodes from the primary umbrella are using "
160+
"the NORMAL mirrored_supervisor child ID format "
161+
"natively"),
162+
normal;
163+
{[VHost, ShovelName], {VHost, ShovelName}} ->
164+
ct:pal(
165+
"The nodes from the primary umbrella are using "
166+
"the TEMPORARY EXPERIMENTAL mirrored_supervisor "
167+
"child ID format natively"),
168+
temp_exp
169+
end,
170+
171+
[{Id1, _, _, _}] = rabbit_ct_broker_helpers:rpc(
172+
Config, OldNode,
173+
mirrored_supervisor, which_children,
174+
[rabbit_shovel_dyn_worker_sup_sup]),
175+
SecondaryIdType = case Id1 of
176+
{VHost, ShovelName} ->
177+
ct:pal(
178+
"The nodes from the secondary umbrella are "
179+
"using the NORMAL mirrored_supervisor child "
180+
"ID format natively"),
181+
normal;
182+
{[VHost, ShovelName], {VHost, ShovelName}} ->
183+
ct:pal(
184+
"The nodes from the secondary umbrella are "
185+
"using the TEMPORARY EXPERIMENTAL "
186+
"mirrored_supervisor child ID format "
187+
"natively"),
188+
temp_exp
189+
end,
190+
if
191+
PrimaryIdType =/= SecondaryIdType ->
192+
ct:pal(
193+
"The mirrored_supervisor child ID format is changing between "
194+
"the primary and the secondary umbrellas!");
195+
true ->
196+
ok
197+
end,
198+
199+
%% Verify that the supervisors exist on all nodes.
200+
ct:pal(
201+
"Checking running mirrored_supervisor children on old node ~s",
202+
[OldNode]),
203+
lists:foreach(
204+
fun(Node) ->
205+
?assertMatch(
206+
[{Id, _, _, _}]
207+
when (SecondaryIdType =:= normal andalso
208+
Id =:= {VHost, ShovelName}) orelse
209+
(SecondaryIdType =:= temp_exp andalso
210+
Id =:= {[VHost, ShovelName], {VHost, ShovelName}}),
211+
rabbit_ct_broker_helpers:rpc(
212+
Config, Node,
213+
mirrored_supervisor, which_children,
214+
[rabbit_shovel_dyn_worker_sup_sup]))
215+
end, [OldNode]),
216+
217+
%% Simulate a rolling upgrade by:
218+
%% 1. adding new nodes to the old cluster
219+
%% 2. stopping the old nodes
220+
%%
221+
%% After that, the supervisors run on the new code.
222+
ct:pal("Clustering nodes ~s and ~s", [OldNode, NewNode]),
223+
Config1 = rabbit_ct_broker_helpers:cluster_nodes(
224+
Config, [OldNode, NewNode]),
225+
ok = rabbit_ct_broker_helpers:stop_broker(Config1, OldNode),
226+
ok = rabbit_ct_broker_helpers:reset_node(Config1, OldNode),
227+
228+
shovel_test_utils:await_shovel(Config, NewNode, ShovelName),
229+
230+
case ?config(metadata_store__manual, Config) of
231+
mnesia ->
232+
ok;
233+
khepri ->
234+
ok = rabbit_ct_broker_helpers:enable_feature_flag(
235+
Config, [NewNode], khepri_db)
236+
end,
237+
238+
%% Verify that the supervisors still use the same IDs.
239+
ct:pal(
240+
"Checking running mirrored_supervisor children on new node ~s",
241+
[NewNode]),
242+
lists:foreach(
243+
fun(Node) ->
244+
?assertMatch(
245+
[{Id, _, _, _}]
246+
when (SecondaryIdType =:= normal andalso
247+
Id =:= {VHost, ShovelName}) orelse
248+
(SecondaryIdType =:= temp_exp andalso
249+
Id =:= {[VHost, ShovelName], {VHost, ShovelName}}),
250+
rabbit_ct_broker_helpers:rpc(
251+
Config1, Node,
252+
mirrored_supervisor, which_children,
253+
[rabbit_shovel_dyn_worker_sup_sup]))
254+
end, [NewNode]),
255+
256+
case ?config(metadata_store__manual, Config) of
257+
mnesia ->
258+
ok;
259+
khepri ->
260+
Path = rabbit_db_msup:khepri_mirrored_supervisor_path(),
261+
?assertMatch(
262+
{ok,
263+
#{[rabbit_db_msup, mirrored_supervisor_childspec,
264+
rabbit_shovel_dyn_worker_sup_sup, VHost, ShovelName] := _}},
265+
rabbit_ct_broker_helpers:rpc(
266+
Config, NewNode, rabbit_khepri, list,
267+
[Path ++ [?KHEPRI_WILDCARD_STAR_STAR]]))
268+
end.

deps/rabbitmq_shovel/test/shovel_test_utils.erl

Lines changed: 33 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -8,29 +8,44 @@
88
-module(shovel_test_utils).
99

1010
-include_lib("common_test/include/ct.hrl").
11-
-export([set_param/3, set_param_nowait/3, await_shovel/2, await_shovel1/2,
12-
shovels_from_status/0, get_shovel_status/2,
13-
await/1, await/2, clear_param/2]).
11+
-export([set_param/3, set_param/4, set_param/5, set_param_nowait/3,
12+
await_shovel/2, await_shovel/3, await_shovel1/2,
13+
shovels_from_status/0, get_shovel_status/2, get_shovel_status/3,
14+
await/1, await/2, clear_param/2, clear_param/3]).
1415

15-
make_uri(Config) ->
16+
make_uri(Config, Node) ->
1617
Hostname = ?config(rmq_hostname, Config),
17-
Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
18+
Port = rabbit_ct_broker_helpers:get_node_config(Config, Node, tcp_port_amqp),
1819
list_to_binary(lists:flatten(io_lib:format("amqp://~ts:~b",
1920
[Hostname, Port]))).
21+
2022
set_param(Config, Name, Value) ->
21-
set_param_nowait(Config, Name, Value),
22-
await_shovel(Config, Name).
23+
set_param_nowait(Config, 0, 0, Name, Value),
24+
await_shovel(Config, 0, Name).
25+
26+
set_param(Config, Node, Name, Value) ->
27+
set_param(Config, Node, Node, Name, Value).
28+
29+
set_param(Config, Node, QueueNode, Name, Value) ->
30+
set_param_nowait(Config, Node, QueueNode, Name, Value),
31+
await_shovel(Config, Node, Name).
2332

2433
set_param_nowait(Config, Name, Value) ->
25-
Uri = make_uri(Config),
26-
ok = rabbit_ct_broker_helpers:rpc(Config, 0,
34+
set_param_nowait(Config, 0, 0, Name, Value).
35+
36+
set_param_nowait(Config, Node, QueueNode, Name, Value) ->
37+
Uri = make_uri(Config, QueueNode),
38+
ok = rabbit_ct_broker_helpers:rpc(Config, Node,
2739
rabbit_runtime_parameters, set, [
2840
<<"/">>, <<"shovel">>, Name, [{<<"src-uri">>, Uri},
2941
{<<"dest-uri">>, [Uri]} |
3042
Value], none]).
3143

3244
await_shovel(Config, Name) ->
33-
rabbit_ct_broker_helpers:rpc(Config, 0,
45+
await_shovel(Config, 0, Name).
46+
47+
await_shovel(Config, Node, Name) ->
48+
rabbit_ct_broker_helpers:rpc(Config, Node,
3449
?MODULE, await_shovel1, [Config, Name]).
3550

3651
await_shovel1(_Config, Name) ->
@@ -41,8 +56,11 @@ shovels_from_status() ->
4156
[N || {{<<"/">>, N}, dynamic, {running, _}, _} <- S].
4257

4358
get_shovel_status(Config, Name) ->
59+
get_shovel_status(Config, 0, Name).
60+
61+
get_shovel_status(Config, Node, Name) ->
4462
S = rabbit_ct_broker_helpers:rpc(
45-
Config, 0, rabbit_shovel_status, lookup, [{<<"/">>, Name}]),
63+
Config, Node, rabbit_shovel_status, lookup, [{<<"/">>, Name}]),
4664
case S of
4765
not_found ->
4866
not_found;
@@ -70,5 +88,8 @@ await(Pred, Timeout) ->
7088
end.
7189

7290
clear_param(Config, Name) ->
73-
rabbit_ct_broker_helpers:rpc(Config, 0,
91+
clear_param(Config, 0, Name).
92+
93+
clear_param(Config, Node, Name) ->
94+
rabbit_ct_broker_helpers:rpc(Config, Node,
7495
rabbit_runtime_parameters, clear, [<<"/">>, <<"shovel">>, Name, <<"acting-user">>]).

0 commit comments

Comments
 (0)