Skip to content

Commit d536939

Browse files
committed
Add cluster_upgrade_SUITE to check mixed-version upgrades
This suite uses the mixed version secondary umbrella as a starting version for a cluster and then has a helper to upgrade the cluster to the current code. This is meant to ensure that we can upgrade from the previous minor. (cherry picked from commit 543bf76) (cherry picked from commit 98d0438) # Conflicts: # deps/rabbit/BUILD.bazel # deps/rabbit/app.bzl
1 parent 04e3564 commit d536939

File tree

4 files changed

+183
-1
lines changed

4 files changed

+183
-1
lines changed

deps/rabbit/BUILD.bazel

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1262,6 +1262,14 @@ rabbitmq_integration_suite(
12621262
],
12631263
)
12641264

1265+
rabbitmq_integration_suite(
1266+
name = "cluster_upgrade_SUITE",
1267+
size = "medium",
1268+
additional_beam = [
1269+
":test_queue_utils_beam",
1270+
],
1271+
)
1272+
12651273
rabbitmq_integration_suite(
12661274
name = "amqpl_direct_reply_to_SUITE",
12671275
)

deps/rabbit/app.bzl

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2184,3 +2184,12 @@ def test_suite_beam_files(name = "test_suite_beam_files"):
21842184
erlc_opts = "//:test_erlc_opts",
21852185
deps = ["//deps/amqp_client:erlang_app"],
21862186
)
2187+
erlang_bytecode(
2188+
name = "cluster_upgrade_SUITE_beam_files",
2189+
testonly = True,
2190+
srcs = ["test/cluster_upgrade_SUITE.erl"],
2191+
outs = ["test/cluster_upgrade_SUITE.beam"],
2192+
app_name = "rabbit",
2193+
erlc_opts = "//:test_erlc_opts",
2194+
deps = ["//deps/amqp_client:erlang_app", "//deps/rabbitmq_ct_helpers:erlang_app"],
2195+
)
Lines changed: 158 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,158 @@
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) 2007-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
6+
%%
7+
8+
-module(cluster_upgrade_SUITE).
9+
10+
-include_lib("eunit/include/eunit.hrl").
11+
-include_lib("amqp_client/include/amqp_client.hrl").
12+
-include_lib("common_test/include/ct.hrl").
13+
14+
-compile([export_all, nowarn_export_all]).
15+
16+
all() ->
17+
[
18+
{group, all_tests}
19+
].
20+
21+
groups() ->
22+
[
23+
{all_tests, [], all_tests()}
24+
].
25+
26+
all_tests() ->
27+
[
28+
queue_upgrade
29+
].
30+
31+
%% -------------------------------------------------------------------
32+
%% Test suite setup/teardown.
33+
%% -------------------------------------------------------------------
34+
35+
init_per_suite(Config) ->
36+
case rabbit_ct_helpers:is_mixed_versions() of
37+
true ->
38+
rabbit_ct_helpers:log_environment(),
39+
rabbit_ct_helpers:run_setup_steps(Config);
40+
false ->
41+
{skip, "cluster upgrade tests must be run in mixed versions "
42+
"testing only"}
43+
end.
44+
45+
end_per_suite(Config) ->
46+
rabbit_ct_helpers:run_teardown_steps(Config).
47+
48+
init_per_group(_Group, Config) ->
49+
Config.
50+
51+
end_per_group(_Group, _Config) ->
52+
ok.
53+
54+
init_per_testcase(Testcase, Config) ->
55+
Config1 = rabbit_ct_helpers:set_config(Config, [
56+
{rmq_nodename_suffix, Testcase},
57+
{rmq_nodes_count, 3},
58+
{force_secondary_umbrella, true}
59+
]),
60+
Config2 = rabbit_ct_helpers:run_steps(Config1,
61+
rabbit_ct_broker_helpers:setup_steps() ++
62+
rabbit_ct_client_helpers:setup_steps()),
63+
rabbit_ct_helpers:testcase_started(Config2, Testcase).
64+
65+
end_per_testcase(Testcase, Config) ->
66+
Config1 = rabbit_ct_helpers:run_steps(Config,
67+
rabbit_ct_client_helpers:teardown_steps() ++
68+
rabbit_ct_broker_helpers:teardown_steps()),
69+
rabbit_ct_helpers:testcase_finished(Config1, Testcase).
70+
71+
%% ---------------------------------------------------------------------------
72+
%% Test Cases
73+
%% ---------------------------------------------------------------------------
74+
75+
queue_upgrade(Config) ->
76+
ok = print_cluster_versions(Config),
77+
78+
%% Declare some resources before upgrading.
79+
{Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config),
80+
ClassicQName = <<"classic-q">>,
81+
QQName = <<"quorum-q">>,
82+
StreamQName = <<"stream-q">>,
83+
declare(Ch, ClassicQName, [{<<"x-queue-type">>, longstr, <<"classic">>}]),
84+
declare(Ch, QQName, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
85+
declare(Ch, StreamQName, [{<<"x-queue-type">>, longstr, <<"stream">>}]),
86+
[begin
87+
#'queue.bind_ok'{} = amqp_channel:call(
88+
Ch,
89+
#'queue.bind'{queue = Name,
90+
exchange = <<"amq.fanout">>,
91+
routing_key = Name})
92+
end || Name <- [ClassicQName, QQName, StreamQName]],
93+
Msgs = [<<"msg">>, <<"msg">>, <<"msg">>],
94+
publish_confirm(Ch, <<"amq.fanout">>, <<>>, Msgs),
95+
ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn, Ch),
96+
97+
%% Restart the servers
98+
Config1 = upgrade_cluster(Config),
99+
ok = print_cluster_versions(Config1),
100+
101+
%% Check that the resources are still there
102+
queue_utils:wait_for_messages(Config, [[ClassicQName, <<"3">>, <<"3">>, <<"0">>],
103+
[QQName, <<"3">>, <<"3">>, <<"0">>],
104+
[StreamQName, <<"3">>, <<"3">>, <<"0">>]]),
105+
106+
ok.
107+
108+
%% ----------------------------------------------------------------------------
109+
%% Internal utils
110+
%% ----------------------------------------------------------------------------
111+
112+
declare(Ch, Q, Args) ->
113+
#'queue.declare_ok'{} = amqp_channel:call(
114+
Ch, #'queue.declare'{queue = Q,
115+
durable = true,
116+
auto_delete = false,
117+
arguments = Args}).
118+
119+
publish(Ch, X, RK, Msg) ->
120+
ok = amqp_channel:cast(Ch,
121+
#'basic.publish'{exchange = X,
122+
routing_key = RK},
123+
#amqp_msg{props = #'P_basic'{delivery_mode = 2},
124+
payload = Msg}).
125+
126+
publish_confirm(Ch, X, RK, Msgs) ->
127+
#'confirm.select_ok'{} = amqp_channel:call(Ch, #'confirm.select'{}),
128+
amqp_channel:register_confirm_handler(Ch, self()),
129+
[publish(Ch, X, RK, Msg) || Msg <- Msgs],
130+
amqp_channel:wait_for_confirms(Ch, 5).
131+
132+
cluster_members(Config) ->
133+
rabbit_ct_broker_helpers:get_node_configs(Config, nodename).
134+
135+
upgrade_cluster(Config) ->
136+
Cluster = cluster_members(Config),
137+
ct:pal(?LOW_IMPORTANCE, "Stopping cluster ~p", [Cluster]),
138+
[ok = rabbit_ct_broker_helpers:stop_node(Config, N)
139+
|| N <- Cluster],
140+
ct:pal(?LOW_IMPORTANCE, "Restarting cluster ~p", [Cluster]),
141+
Config1 = rabbit_ct_helpers:set_config(
142+
Config, {force_secondary_umbrella, false}),
143+
[ok = rabbit_ct_broker_helpers:async_start_node(Config1, N)
144+
|| N <- Cluster],
145+
[ok = rabbit_ct_broker_helpers:wait_for_async_start_node(N)
146+
|| N <- Cluster],
147+
Config1.
148+
149+
print_cluster_versions(Config) ->
150+
Cluster = cluster_members(Config),
151+
Versions = [begin
152+
Version = rabbit_ct_broker_helpers:rpc(
153+
Config, N,
154+
rabbit, product_version, []),
155+
{N, Version}
156+
end || N <- Cluster],
157+
ct:pal("Cluster versions: ~p", [Versions]),
158+
ok.

deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -629,7 +629,14 @@ do_start_rabbitmq_node(Config, NodeConfig, I) ->
629629
true -> lists:nth(I + 1, WithPlugins0);
630630
false -> WithPlugins0
631631
end,
632-
CanUseSecondary = (I + 1) rem 2 =:= 0,
632+
ForceUseSecondary = rabbit_ct_helpers:get_config(
633+
Config, force_secondary_umbrella, undefined),
634+
CanUseSecondary = case ForceUseSecondary of
635+
undefined ->
636+
(I + 1) rem 2 =:= 0;
637+
Override when is_boolean(Override) ->
638+
Override
639+
end,
633640
UseSecondaryUmbrella = case ?config(secondary_umbrella, Config) of
634641
false -> false;
635642
_ -> CanUseSecondary

0 commit comments

Comments
 (0)