Skip to content

Commit 6424c86

Browse files
dumbbellmergify[bot]
authored andcommitted
rabbitmq_peer_discovery_etcd: Add clustering testcases
[Why] The existing testsuite tried if the communication with an etcd node would work, but didn't test an actual cluster formation. [How] The new testcases try to create a cluster using the local etcd 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 etcd source code added as a Git submodule in a previous commit to compile etcd locally just for the testsuite. (cherry picked from commit 50b4901)
1 parent 8cb7f36 commit 6424c86

File tree

3 files changed

+269
-31
lines changed

3 files changed

+269
-31
lines changed

deps/rabbitmq_peer_discovery_etcd/BUILD.bazel

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -99,7 +99,7 @@ rabbitmq_integration_suite(
9999

100100
rabbitmq_integration_suite(
101101
name = "system_SUITE",
102-
size = "medium",
102+
size = "large",
103103
)
104104

105105
rabbitmq_suite(

deps/rabbitmq_peer_discovery_etcd/test/system_SUITE.erl

Lines changed: 267 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -9,19 +9,33 @@
99

1010
-module(system_SUITE).
1111

12-
-compile(export_all).
13-
1412
-include_lib("common_test/include/ct.hrl").
1513
-include_lib("eunit/include/eunit.hrl").
1614

1715
-include("rabbit_peer_discovery_etcd.hrl").
1816

19-
-import(rabbit_data_coercion, [to_binary/1, to_integer/1]).
17+
-define(ETCD_GIT_REPO, "https://github.com/etcd-io/etcd.git").
18+
-define(ETCD_GIT_REF, "v3.5.13").
19+
20+
-export([all/0,
21+
groups/0,
22+
init_per_suite/1,
23+
end_per_suite/1,
24+
init_per_group/2,
25+
end_per_group/2,
26+
init_per_testcase/2,
27+
end_per_testcase/2,
2028

29+
etcd_connection_sanity_check_test/1,
30+
init_opens_a_connection_test/1,
31+
registration_with_locking_test/1,
32+
start_one_member_at_a_time/1,
33+
start_members_concurrently/1]).
2134

2235
all() ->
2336
[
24-
{group, v3_client}
37+
{group, v3_client},
38+
{group, clustering}
2539
].
2640

2741
groups() ->
@@ -30,47 +44,178 @@ groups() ->
3044
etcd_connection_sanity_check_test,
3145
init_opens_a_connection_test,
3246
registration_with_locking_test
33-
]}
47+
]},
48+
{clustering, [], [start_one_member_at_a_time,
49+
start_members_concurrently]}
3450
].
3551

3652
init_per_suite(Config) ->
3753
rabbit_ct_helpers:log_environment(),
38-
rabbit_ct_helpers:run_setup_steps(Config, [fun init_etcd/1]).
54+
rabbit_ct_helpers:run_setup_steps(
55+
Config,
56+
[fun clone_etcd/1,
57+
fun compile_etcd/1,
58+
fun start_etcd/1]).
3959

4060
end_per_suite(Config) ->
4161
rabbit_ct_helpers:run_teardown_steps(Config, [fun stop_etcd/1]).
4262

43-
init_etcd(Config) ->
63+
init_per_group(clustering, Config) ->
64+
rabbit_ct_helpers:set_config(
65+
Config,
66+
[{rmq_nodes_count, 3},
67+
{rmq_nodes_clustered, false}]);
68+
init_per_group(_Group, Config) ->
69+
Config.
70+
71+
end_per_group(_Group, Config) ->
72+
Config.
73+
74+
init_per_testcase(Testcase, Config)
75+
when Testcase =:= start_one_member_at_a_time orelse
76+
Testcase =:= start_members_concurrently ->
77+
rabbit_ct_helpers:testcase_started(Config, Testcase),
78+
TestNumber = rabbit_ct_helpers:testcase_number(Config, ?MODULE, Testcase),
79+
ClusterSize = ?config(rmq_nodes_count, Config),
80+
Config1 = rabbit_ct_helpers:set_config(
81+
Config,
82+
[{rmq_nodename_suffix, Testcase},
83+
{tcp_ports_base, {skip_n_nodes,
84+
TestNumber * ClusterSize}}
85+
]),
86+
Config2 = rabbit_ct_helpers:merge_app_env(
87+
Config1, {rabbit, [{log, [{file, [{level, debug}]}]}]}),
88+
Config3 = rabbit_ct_helpers:run_steps(
89+
Config2,
90+
rabbit_ct_broker_helpers:setup_steps() ++
91+
rabbit_ct_client_helpers:setup_steps()),
92+
try
93+
_ = rabbit_ct_broker_helpers:rpc_all(
94+
Config3, rabbit_peer_discovery_backend, api_version, []),
95+
Config3
96+
catch
97+
error:{exception, undef,
98+
[{rabbit_peer_discovery_backend, api_version, _, _} | _]} ->
99+
Config4 = rabbit_ct_helpers:run_steps(
100+
Config3,
101+
rabbit_ct_client_helpers:teardown_steps() ++
102+
rabbit_ct_broker_helpers:teardown_steps()),
103+
rabbit_ct_helpers:testcase_finished(Config4, Testcase),
104+
{skip,
105+
"Some nodes use the old discover->register order; "
106+
"the testcase would likely fail"}
107+
end;
108+
init_per_testcase(_Testcase, Config) ->
109+
Config.
110+
111+
end_per_testcase(Testcase, Config)
112+
when Testcase =:= start_one_member_at_a_time orelse
113+
Testcase =:= start_members_concurrently ->
114+
Config1 = rabbit_ct_helpers:run_steps(
115+
Config,
116+
rabbit_ct_client_helpers:teardown_steps() ++
117+
rabbit_ct_broker_helpers:teardown_steps()),
118+
rabbit_ct_helpers:testcase_finished(Config1, Testcase);
119+
end_per_testcase(_Testcase, Config) ->
120+
Config.
121+
122+
clone_etcd(Config) ->
44123
DataDir = ?config(data_dir, Config),
45-
PrivDir = ?config(priv_dir, Config),
46-
TcpPort = 25389,
47-
EtcdDir = filename:join([PrivDir, "etcd"]),
48-
InitEtcd = filename:join([DataDir, "init-etcd.sh"]),
49-
Cmd = [InitEtcd, EtcdDir, {"~b", [TcpPort]}],
124+
EtcdSrcdir = filename:join(DataDir, "etcd"),
125+
Cmd = case filelib:is_dir(EtcdSrcdir) of
126+
true ->
127+
ct:pal(
128+
"Checking out etcd Git reference, ref = ~s",
129+
[?ETCD_GIT_REF]),
130+
["git", "-C", EtcdSrcdir,
131+
"checkout", ?ETCD_GIT_REF];
132+
false ->
133+
ct:pal(
134+
"Cloning etcd Git repository, ref = ~s",
135+
[?ETCD_GIT_REF]),
136+
["git", "clone",
137+
"--branch", ?ETCD_GIT_REF,
138+
?ETCD_GIT_REPO, EtcdSrcdir]
139+
end,
50140
case rabbit_ct_helpers:exec(Cmd) of
51-
{ok, Stdout} ->
52-
case re:run(Stdout, "^ETCD_PID=([0-9]+)$", [{capture, all_but_first, list}, multiline]) of
53-
{match, [EtcdPid]} ->
54-
ct:pal(?LOW_IMPORTANCE, "etcd PID: ~ts~netcd is listening on: ~b", [EtcdPid, TcpPort]),
55-
rabbit_ct_helpers:set_config(Config, [{etcd_pid, EtcdPid},
56-
{etcd_endpoints, [rabbit_misc:format("localhost:~tp", [TcpPort])]},
57-
{etcd_port, TcpPort}]);
58-
nomatch ->
59-
ct:pal(?HI_IMPORTANCE, "init-etcd.sh output did not match what's expected: ~tp", [Stdout])
60-
end;
61-
{error, Code, Reason} ->
62-
ct:pal(?HI_IMPORTANCE, "init-etcd.sh exited with code ~tp: ~tp", [Code, Reason]),
63-
_ = rabbit_ct_helpers:exec(["pkill", "-INT", "etcd"]),
64-
{skip, "Failed to initialize etcd"}
141+
{ok, _} ->
142+
rabbit_ct_helpers:set_config(Config, {etcd_srcdir, EtcdSrcdir});
143+
{error, _} ->
144+
{skip, "Failed to clone etcd"}
65145
end.
66146

147+
compile_etcd(Config) ->
148+
EtcdSrcdir = ?config(etcd_srcdir, Config),
149+
ct:pal("Compiling etcd in ~ts", [EtcdSrcdir]),
150+
Script0 = case os:type() of
151+
{win32, _} -> "build.bat";
152+
_ -> "build.sh"
153+
end,
154+
Script1 = filename:join(EtcdSrcdir, Script0),
155+
Cmd = [Script1],
156+
GOPATH = filename:join(EtcdSrcdir, "go"),
157+
GOFLAGS = "-modcacherw",
158+
Options = [{cd, EtcdSrcdir},
159+
{env, [{"BINDIR", false},
160+
{"GOPATH", GOPATH},
161+
{"GOFLAGS", GOFLAGS}]}],
162+
case rabbit_ct_helpers:exec(Cmd, Options) of
163+
{ok, _} ->
164+
EtcdExe = case os:type() of
165+
{win32, _} -> "etcd.exe";
166+
_ -> "etcd"
167+
end,
168+
EtcdBin = filename:join([EtcdSrcdir, "bin", EtcdExe]),
169+
?assert(filelib:is_regular(EtcdBin)),
170+
rabbit_ct_helpers:set_config(Config, {etcd_bin, EtcdBin});
171+
{error, _} ->
172+
{skip, "Failed to compile etcd"}
173+
end.
174+
175+
start_etcd(Config) ->
176+
ct:pal("Starting etcd daemon"),
177+
EtcdBin = ?config(etcd_bin, Config),
178+
PrivDir = ?config(priv_dir, Config),
179+
EtcdDataDir = filename:join(PrivDir, "data.etcd"),
180+
EtcdName = ?MODULE_STRING,
181+
EtcdHost = "localhost",
182+
EtcdClientPort = 2379,
183+
EtcdClientUrl = rabbit_misc:format(
184+
"http://~s:~b", [EtcdHost, EtcdClientPort]),
185+
EtcdAdvPort = 2380,
186+
EtcdAdvUrl = rabbit_misc:format(
187+
"http://~s:~b", [EtcdHost, EtcdAdvPort]),
188+
Cmd = [EtcdBin,
189+
"--data-dir", EtcdDataDir,
190+
"--name", EtcdName,
191+
"--initial-advertise-peer-urls", EtcdAdvUrl,
192+
"--listen-peer-urls", EtcdAdvUrl,
193+
"--advertise-client-urls", EtcdClientUrl,
194+
"--listen-client-urls", EtcdClientUrl,
195+
"--initial-cluster", EtcdName ++ "=" ++ EtcdAdvUrl,
196+
"--initial-cluster-state", "new",
197+
"--initial-cluster-token", "test-token",
198+
"--log-level", "debug", "--log-outputs", "stdout"],
199+
EtcdPid = spawn(fun() -> rabbit_ct_helpers:exec(Cmd) end),
200+
201+
EtcdEndpoint = rabbit_misc:format("~s:~b", [EtcdHost, EtcdClientPort]),
202+
rabbit_ct_helpers:set_config(
203+
Config,
204+
[{etcd_pid, EtcdPid},
205+
{etcd_endpoints, [EtcdEndpoint]}]).
206+
67207
stop_etcd(Config) ->
68-
EtcdPid = ?config(etcd_pid, Config),
69-
Cmd = ["kill", "-INT", EtcdPid],
70-
_ = rabbit_ct_helpers:exec(Cmd),
208+
case rabbit_ct_helpers:get_config(Config, etcd_pid) of
209+
EtcdPid when is_pid(EtcdPid) ->
210+
ct:pal(
211+
"Stopping etcd daemon by killing control process ~p",
212+
[EtcdPid]),
213+
erlang:exit(EtcdPid, kill);
214+
undefined ->
215+
ok
216+
end,
71217
Config.
72218

73-
74219
%%
75220
%% Test cases
76221
%%
@@ -133,6 +278,98 @@ registration_with_locking_test(Config) ->
133278
gen_statem:stop(Pid)
134279
end.
135280

281+
start_one_member_at_a_time(Config) ->
282+
Config1 = configure_peer_discovery(Config),
283+
284+
Nodes = rabbit_ct_broker_helpers:get_node_configs(Config1, nodename),
285+
lists:foreach(
286+
fun(Node) ->
287+
?assertEqual(
288+
ok,
289+
rabbit_ct_broker_helpers:start_node(Config1, Node))
290+
end, Nodes),
291+
292+
assert_full_cluster(Config1).
293+
294+
start_members_concurrently(Config) ->
295+
Config1 = configure_peer_discovery(Config),
296+
297+
Nodes = rabbit_ct_broker_helpers:get_node_configs(Config1, nodename),
298+
Parent = self(),
299+
Pids = lists:map(
300+
fun(Node) ->
301+
spawn_link(
302+
fun() ->
303+
receive
304+
go ->
305+
?assertEqual(
306+
ok,
307+
rabbit_ct_broker_helpers:start_node(
308+
Config1, Node)),
309+
Parent ! started
310+
end
311+
end)
312+
end, Nodes),
313+
314+
lists:foreach(fun(Pid) -> Pid ! go end, Pids),
315+
lists:foreach(fun(_Pid) -> receive started -> ok end end, Pids),
316+
317+
assert_full_cluster(Config1).
318+
319+
configure_peer_discovery(Config) ->
320+
Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
321+
lists:foreach(
322+
fun(Node) ->
323+
Members = lists:sort(
324+
rabbit_ct_broker_helpers:cluster_members_online(
325+
Config, Node)),
326+
?assertEqual([Node], Members)
327+
end, Nodes),
328+
329+
lists:foreach(
330+
fun(Node) ->
331+
?assertEqual(
332+
ok,
333+
rabbit_ct_broker_helpers:stop_broker(Config, Node)),
334+
?assertEqual(
335+
ok,
336+
rabbit_ct_broker_helpers:reset_node(Config, Node)),
337+
?assertEqual(
338+
ok,
339+
rabbit_ct_broker_helpers:stop_node(Config, Node))
340+
end, Nodes),
341+
342+
Endpoints = ?config(etcd_endpoints, Config),
343+
Config1 = rabbit_ct_helpers:merge_app_env(
344+
Config,
345+
{rabbit,
346+
[{cluster_formation,
347+
[{peer_discovery_backend, rabbit_peer_discovery_etcd},
348+
{peer_discovery_etcd,
349+
[{endpoints, Endpoints},
350+
{etcd_prefix, "rabbitmq"},
351+
{cluster_name, atom_to_list(?FUNCTION_NAME)}]}]}]}),
352+
lists:foreach(
353+
fun(Node) ->
354+
?assertEqual(
355+
ok,
356+
rabbit_ct_broker_helpers:rewrite_node_config_file(
357+
Config1, Node))
358+
end, Nodes),
359+
360+
Config1.
361+
362+
assert_full_cluster(Config) ->
363+
Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
364+
ExpectedMembers = lists:sort(Nodes),
365+
lists:foreach(
366+
fun(Node) ->
367+
Members = lists:sort(
368+
rabbit_ct_broker_helpers:cluster_members_online(
369+
Config, Node)),
370+
?assertEqual(ExpectedMembers, Members)
371+
end, Nodes).
372+
136373
%%
137374
%% Helpers
138375
%%
Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1 @@
1+
/etcd/

0 commit comments

Comments
 (0)