Skip to content

Commit 1eac1b4

Browse files
dumbbellmergify[bot]
authored andcommitted
rabbitmq_peer_discovery_consul: Add clustering testcases
[Why] Add a `system_SUITE` testsuite, copied from rabbitmq_peer_discovery_etcd, that attempts to start a RabbitMQ cluster where nodes use a Consul server to discover themselves. [How] The new testcases try to create a cluster using the local Consul node started by the testsuite. The first one starts one RabbitMQ node at a time. the second one starts all of them concurrently. While here, use the Consul source code added as a Git submodule in a previous commit to compile Consul locally just for the testsuite. (cherry picked from commit 27ed4d2)
1 parent bc19bfc commit 1eac1b4

File tree

5 files changed

+357
-0
lines changed

5 files changed

+357
-0
lines changed

deps/rabbitmq_peer_discovery_consul/BUILD.bazel

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -79,6 +79,11 @@ rabbitmq_integration_suite(
7979
name = "config_schema_SUITE",
8080
)
8181

82+
rabbitmq_integration_suite(
83+
name = "system_SUITE",
84+
size = "large",
85+
)
86+
8287
rabbitmq_suite(
8388
name = "rabbitmq_peer_discovery_consul_SUITE",
8489
size = "medium",

deps/rabbitmq_peer_discovery_consul/app.bzl

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -99,6 +99,15 @@ def test_suite_beam_files(name = "test_suite_beam_files"):
9999
app_name = "rabbitmq_peer_discovery_consul",
100100
erlc_opts = "//:test_erlc_opts",
101101
)
102+
erlang_bytecode(
103+
name = "system_SUITE_beam_files",
104+
testonly = True,
105+
srcs = ["test/system_SUITE.erl"],
106+
outs = ["test/system_SUITE.beam"],
107+
hdrs = [],
108+
app_name = "rabbitmq_peer_discovery_consul",
109+
erlc_opts = "//:test_erlc_opts",
110+
)
102111
erlang_bytecode(
103112
name = "rabbitmq_peer_discovery_consul_SUITE_beam_files",
104113
testonly = True,
Lines changed: 311 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,311 @@
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. The term “Broadcom” refers to Broadcom Inc.
6+
%% and/or its subsidiaries. All rights reserved. All rights reserved.
7+
%%
8+
9+
-module(system_SUITE).
10+
11+
-include_lib("common_test/include/ct.hrl").
12+
-include_lib("eunit/include/eunit.hrl").
13+
14+
-export([all/0,
15+
groups/0,
16+
init_per_suite/1,
17+
end_per_suite/1,
18+
init_per_group/2,
19+
end_per_group/2,
20+
init_per_testcase/2,
21+
end_per_testcase/2,
22+
23+
start_one_member_at_a_time/1,
24+
start_members_concurrently/1]).
25+
26+
-define(CONSUL_GIT_REPO, "https://github.com/hashicorp/consul.git").
27+
-define(CONSUL_GIT_REF, "v1.18.1").
28+
29+
all() ->
30+
[
31+
{group, clustering}
32+
].
33+
34+
groups() ->
35+
[
36+
{clustering, [], [start_one_member_at_a_time,
37+
start_members_concurrently]}
38+
].
39+
40+
init_per_suite(Config) ->
41+
rabbit_ct_helpers:log_environment(),
42+
rabbit_ct_helpers:run_setup_steps(
43+
Config,
44+
[fun clone_consul/1,
45+
fun compile_consul/1,
46+
fun config_consul/1,
47+
fun start_consul/1]).
48+
49+
end_per_suite(Config) ->
50+
rabbit_ct_helpers:run_teardown_steps(Config, [fun stop_consul/1]).
51+
52+
init_per_group(clustering, Config) ->
53+
rabbit_ct_helpers:set_config(
54+
Config,
55+
[{rmq_nodes_count, 3},
56+
{rmq_nodes_clustered, false}]);
57+
init_per_group(_Group, Config) ->
58+
Config.
59+
60+
end_per_group(_Group, Config) ->
61+
Config.
62+
63+
init_per_testcase(Testcase, Config)
64+
when Testcase =:= start_one_member_at_a_time orelse
65+
Testcase =:= start_members_concurrently ->
66+
rabbit_ct_helpers:testcase_started(Config, Testcase),
67+
TestNumber = rabbit_ct_helpers:testcase_number(Config, ?MODULE, Testcase),
68+
ClusterSize = ?config(rmq_nodes_count, Config),
69+
Config1 = rabbit_ct_helpers:set_config(
70+
Config,
71+
[{rmq_nodename_suffix, Testcase},
72+
{tcp_ports_base, {skip_n_nodes,
73+
TestNumber * ClusterSize}}
74+
]),
75+
Config2 = rabbit_ct_helpers:merge_app_env(
76+
Config1, {rabbit, [{log, [{file, [{level, debug}]}]}]}),
77+
Config3 = rabbit_ct_helpers:run_steps(
78+
Config2,
79+
rabbit_ct_broker_helpers:setup_steps() ++
80+
rabbit_ct_client_helpers:setup_steps()),
81+
try
82+
_ = rabbit_ct_broker_helpers:rpc_all(
83+
Config3, rabbit_peer_discovery_backend, api_version, []),
84+
Config3
85+
catch
86+
error:{exception, undef,
87+
[{rabbit_peer_discovery_backend, api_version, _, _} | _]} ->
88+
Config4 = rabbit_ct_helpers:run_steps(
89+
Config3,
90+
rabbit_ct_client_helpers:teardown_steps() ++
91+
rabbit_ct_broker_helpers:teardown_steps()),
92+
rabbit_ct_helpers:testcase_finished(Config4, Testcase),
93+
{skip,
94+
"Some nodes use the old discover->register order; "
95+
"the testcase would likely fail"}
96+
end;
97+
init_per_testcase(_Testcase, Config) ->
98+
Config.
99+
100+
end_per_testcase(Testcase, Config)
101+
when Testcase =:= start_one_member_at_a_time orelse
102+
Testcase =:= start_members_concurrently ->
103+
Config1 = rabbit_ct_helpers:run_steps(
104+
Config,
105+
rabbit_ct_client_helpers:teardown_steps() ++
106+
rabbit_ct_broker_helpers:teardown_steps()),
107+
rabbit_ct_helpers:testcase_finished(Config1, Testcase);
108+
end_per_testcase(_Testcase, Config) ->
109+
Config.
110+
111+
clone_consul(Config) ->
112+
DataDir = ?config(data_dir, Config),
113+
ConsulSrcdir = filename:join(DataDir, "consul"),
114+
Cmd = case filelib:is_dir(ConsulSrcdir) of
115+
true ->
116+
ct:pal(
117+
"Checking out Consul Git reference, ref = ~s",
118+
[?CONSUL_GIT_REF]),
119+
["git", "-C", ConsulSrcdir,
120+
"checkout", ?CONSUL_GIT_REF];
121+
false ->
122+
ct:pal(
123+
"Cloning Consul Git repository, ref = ~s",
124+
[?CONSUL_GIT_REF]),
125+
["git", "clone",
126+
"--branch", ?CONSUL_GIT_REF,
127+
?CONSUL_GIT_REPO, ConsulSrcdir]
128+
end,
129+
case rabbit_ct_helpers:exec(Cmd) of
130+
{ok, _} ->
131+
rabbit_ct_helpers:set_config(
132+
Config, {consul_srcdir, ConsulSrcdir});
133+
{error, _} ->
134+
{skip, "Failed to clone Consul"}
135+
end.
136+
137+
compile_consul(Config) ->
138+
ConsulSrcdir = ?config(consul_srcdir, Config),
139+
ct:pal("Compiling Consul in ~ts", [ConsulSrcdir]),
140+
Cmd = ["go", "install"],
141+
GOPATH = filename:join(ConsulSrcdir, "go"),
142+
GOFLAGS = "-modcacherw",
143+
Options = [{cd, ConsulSrcdir},
144+
{env, [{"BINDIR", false},
145+
{"GOPATH", GOPATH},
146+
{"GOFLAGS", GOFLAGS}]}],
147+
case rabbit_ct_helpers:exec(Cmd, Options) of
148+
{ok, _} ->
149+
ConsulExe = case os:type() of
150+
{win32, _} -> "consul.exe";
151+
_ -> "consul"
152+
end,
153+
ConsulBin = filename:join([GOPATH, "bin", ConsulExe]),
154+
?assert(filelib:is_regular(ConsulBin)),
155+
rabbit_ct_helpers:set_config(Config, {consul_bin, ConsulBin});
156+
{error, _} ->
157+
{skip, "Failed to compile Consul"}
158+
end.
159+
160+
config_consul(Config) ->
161+
DataDir = ?config(data_dir, Config),
162+
PrivDir = ?config(priv_dir, Config),
163+
ConsulConfDir = filename:join(PrivDir, "conf.consul"),
164+
ConsulDataDir = filename:join(PrivDir, "data.consul"),
165+
ConsulHost = "localhost",
166+
ConsulTcpPort = 8500,
167+
168+
ConsulConfTpl = filename:join(DataDir, "consul.hcl"),
169+
{ok, ConsulConf0} = file:read_file(ConsulConfTpl),
170+
ConsulConf1 = io_lib:format(
171+
"~ts~n"
172+
"node_name = \"~ts\"~n"
173+
"domain = \"~ts\"~n"
174+
"data_dir = \"~ts\"~n"
175+
"ports {~n"
176+
" http = ~b~n"
177+
" grpc = -1~n"
178+
"}~n",
179+
[ConsulConf0, ConsulHost, ConsulHost, ConsulDataDir,
180+
ConsulTcpPort]),
181+
ConsulConfFile = filename:join(ConsulConfDir, "consul.hcl"),
182+
ok = file:make_dir(ConsulConfDir),
183+
ok = file:write_file(ConsulConfFile, ConsulConf1),
184+
rabbit_ct_helpers:set_config(
185+
Config,
186+
[{consul_conf_dir, ConsulConfDir},
187+
{consul_host, ConsulHost},
188+
{consul_tcp_port, ConsulTcpPort}]).
189+
190+
start_consul(Config) ->
191+
ct:pal("Starting Consul daemon"),
192+
ConsulBin = ?config(consul_bin, Config),
193+
ConsulConfDir = ?config(consul_conf_dir, Config),
194+
Cmd = [ConsulBin, "agent", "-config-dir", ConsulConfDir],
195+
ConsulPid = spawn(fun() -> rabbit_ct_helpers:exec(Cmd) end),
196+
rabbit_ct_helpers:set_config(Config, {consul_pid, ConsulPid}).
197+
198+
stop_consul(Config) ->
199+
case rabbit_ct_helpers:get_config(Config, consul_pid) of
200+
ConsulPid when is_pid(ConsulPid) ->
201+
ct:pal(
202+
"Stopping Consul daemon by killing control process ~p",
203+
[ConsulPid]),
204+
erlang:exit(ConsulPid, kill),
205+
_ = case os:type() of
206+
{win32, _} -> ok;
207+
_ -> rabbit_ct_helpers:exec(["pkill", "consul"])
208+
end;
209+
undefined ->
210+
ok
211+
end,
212+
Config.
213+
214+
%%
215+
%% Test cases
216+
%%
217+
218+
start_one_member_at_a_time(Config) ->
219+
Config1 = configure_peer_discovery(Config),
220+
221+
Nodes = rabbit_ct_broker_helpers:get_node_configs(Config1, nodename),
222+
lists:foreach(
223+
fun(Node) ->
224+
?assertEqual(
225+
ok,
226+
rabbit_ct_broker_helpers:start_node(Config1, Node))
227+
end, Nodes),
228+
229+
assert_full_cluster(Config1).
230+
231+
start_members_concurrently(Config) ->
232+
Config1 = configure_peer_discovery(Config),
233+
234+
Nodes = rabbit_ct_broker_helpers:get_node_configs(Config1, nodename),
235+
Parent = self(),
236+
Pids = lists:map(
237+
fun(Node) ->
238+
spawn_link(
239+
fun() ->
240+
receive
241+
go ->
242+
?assertEqual(
243+
ok,
244+
rabbit_ct_broker_helpers:start_node(
245+
Config1, Node)),
246+
Parent ! started
247+
end
248+
end)
249+
end, Nodes),
250+
251+
lists:foreach(fun(Pid) -> Pid ! go end, Pids),
252+
lists:foreach(fun(_Pid) -> receive started -> ok end end, Pids),
253+
254+
assert_full_cluster(Config1).
255+
256+
configure_peer_discovery(Config) ->
257+
Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
258+
lists:foreach(
259+
fun(Node) ->
260+
Members = lists:sort(
261+
rabbit_ct_broker_helpers:cluster_members_online(
262+
Config, Node)),
263+
?assertEqual([Node], Members)
264+
end, Nodes),
265+
266+
lists:foreach(
267+
fun(Node) ->
268+
?assertEqual(
269+
ok,
270+
rabbit_ct_broker_helpers:stop_broker(Config, Node)),
271+
?assertEqual(
272+
ok,
273+
rabbit_ct_broker_helpers:reset_node(Config, Node)),
274+
?assertEqual(
275+
ok,
276+
rabbit_ct_broker_helpers:stop_node(Config, Node))
277+
end, Nodes),
278+
279+
ConsulHost = ?config(consul_host, Config),
280+
ConsulTcpPort = ?config(consul_tcp_port, Config),
281+
lists:foreach(
282+
fun(Node) ->
283+
Config1 = rabbit_ct_helpers:merge_app_env(
284+
Config,
285+
{rabbit,
286+
[{cluster_formation,
287+
[{peer_discovery_backend,
288+
rabbit_peer_discovery_consul},
289+
{peer_discovery_consul,
290+
[{consul_svc_id, atom_to_list(Node)},
291+
{consul_host, ConsulHost},
292+
{consul_port, ConsulTcpPort},
293+
{consul_scheme, "http"}]}]}]}),
294+
?assertEqual(
295+
ok,
296+
rabbit_ct_broker_helpers:rewrite_node_config_file(
297+
Config1, Node))
298+
end, Nodes),
299+
300+
Config.
301+
302+
assert_full_cluster(Config) ->
303+
Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
304+
ExpectedMembers = lists:sort(Nodes),
305+
lists:foreach(
306+
fun(Node) ->
307+
Members = lists:sort(
308+
rabbit_ct_broker_helpers:cluster_members_online(
309+
Config, Node)),
310+
?assertEqual(ExpectedMembers, Members)
311+
end, Nodes).
Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1 @@
1+
/consul/
Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,31 @@
1+
log_level = "DEBUG"
2+
enable_syslog = false
3+
enable_script_checks = false
4+
enable_local_script_checks = true
5+
6+
datacenter = "dc1"
7+
server = true
8+
bootstrap_expect = 1
9+
10+
## ACL configuration
11+
acl = {
12+
enabled = true
13+
default_policy = "allow"
14+
enable_token_persistence = true
15+
enable_token_replication = true
16+
down_policy = "extend-cache"
17+
}
18+
19+
# Enable service mesh
20+
connect {
21+
enabled = true
22+
}
23+
24+
# Addresses and ports
25+
client_addr = "0.0.0.0"
26+
bind_addr = "0.0.0.0"
27+
28+
addresses {
29+
grpc = "0.0.0.0"
30+
http = "0.0.0.0"
31+
}

0 commit comments

Comments
 (0)