Skip to content

Commit 5629ab2

Browse files
committed
Upgrade eetcd and gun
## Why? To introduce AMQP over WebSocket, we will add gun to the Erlang AMQP 1.0 client. We want to add the latest version of gun for this new feature. Since rabbitmq_peer_discovery_etcd depends on the outdated eetcd 0.3.6 which in turn depends on the outdated gun 1.3.3, this commit first upgrades eetcd and gun. ## How? See https://github.com/zhongwencool/eetcd?tab=readme-ov-file#migration-from-eetcd-03x-to-04x ## Breaking Changes This commit causes the following breaking change: `rabbitmq.conf` settings * `cluster_formation.etcd.ssl_options.fail_if_no_peer_cert` * `cluster_formation.etcd.ssl_options.dh` * `cluster_formation.etcd.ssl_options.dhfile` are unsupported because they are not valid `ssl:tls_client_option()`. See erlang/otp#7497 (comment)
1 parent d8ca61c commit 5629ab2

File tree

10 files changed

+74
-94
lines changed

10 files changed

+74
-94
lines changed

deps/rabbitmq_peer_discovery_etcd/Makefile

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -5,8 +5,8 @@ PROJECT_MOD = rabbitmq_peer_discovery_etcd_app
55
DEPS = rabbit_common rabbitmq_peer_discovery_common rabbit eetcd gun
66
TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers ct_helper meck
77
dep_ct_helper = git https://github.com/extend/ct_helper.git master
8-
dep_gun = hex 1.3.3
9-
dep_eetcd = hex 0.3.6
8+
dep_gun = hex 2.1.0
9+
dep_eetcd = hex 0.4.0
1010

1111
DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-plugin.mk
1212
DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk

deps/rabbitmq_peer_discovery_etcd/README.md

Lines changed: 0 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -14,11 +14,6 @@ it is not. Cluster provisioning and most of Day 2 operations such as [proper mon
1414
are not in scope for this plugin.
1515

1616

17-
## Supported RabbitMQ Versions
18-
19-
This plugin requires RabbitMQ 3.7.0 or later.
20-
21-
2217
## Supported etcd Versions
2318

2419
The plugin supports etcd 3.4 or later and uses the current stable v3 gRPC API.
@@ -36,17 +31,11 @@ before first node boot:
3631
rabbitmq-plugins --offline enable rabbitmq_peer_discovery_etcd
3732
```
3833

39-
4034
## Documentation
4135

4236
See [RabbitMQ Cluster Formation guide](https://www.rabbitmq.com/cluster-formation.html).
4337

4438

45-
## Contributing
46-
47-
See [CONTRIBUTING.md](./CONTRIBUTING.md) and our [development process overview](http://www.rabbitmq.com/github.html).
48-
49-
5039
## License
5140

5241
[Licensed under the MPL](LICENSE-MPL-RabbitMQ), same as RabbitMQ server.

deps/rabbitmq_peer_discovery_etcd/priv/schema/rabbitmq_peer_discovery_etcd.schema

Lines changed: 0 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -182,9 +182,6 @@ end}.
182182
{mapping, "cluster_formation.etcd.ssl_options.verify", "rabbit.cluster_formation.peer_discovery_etcd.ssl_options.verify", [
183183
{datatype, {enum, [verify_peer, verify_none]}}]}.
184184

185-
{mapping, "cluster_formation.etcd.ssl_options.fail_if_no_peer_cert", "rabbit.cluster_formation.peer_discovery_etcd.ssl_options.fail_if_no_peer_cert", [
186-
{datatype, {enum, [true, false]}}]}.
187-
188185
{mapping, "cluster_formation.etcd.ssl_options.cacertfile", "rabbit.cluster_formation.peer_discovery_etcd.ssl_options.cacertfile",
189186
[{datatype, string}, {validators, ["file_accessible"]}]}.
190187

@@ -214,17 +211,6 @@ end}.
214211
{mapping, "cluster_formation.etcd.ssl_options.depth", "rabbit.cluster_formation.peer_discovery_etcd.ssl_options.depth",
215212
[{datatype, integer}, {validators, ["byte"]}]}.
216213

217-
{mapping, "cluster_formation.etcd.ssl_options.dh", "rabbit.cluster_formation.peer_discovery_etcd.ssl_options.dh",
218-
[{datatype, string}]}.
219-
220-
{translation, "rabbit.cluster_formation.peer_discovery_etcd.ssl_options.dh",
221-
fun(Conf) ->
222-
list_to_binary(cuttlefish:conf_get("cluster_formation.etcd.ssl_options.dh", Conf))
223-
end}.
224-
225-
{mapping, "cluster_formation.etcd.ssl_options.dhfile", "rabbit.cluster_formation.peer_discovery_etcd.ssl_options.dhfile",
226-
[{datatype, string}, {validators, ["file_accessible"]}]}.
227-
228214
{mapping, "cluster_formation.etcd.ssl_options.key.RSAPrivateKey", "rabbit.cluster_formation.peer_discovery_etcd.ssl_options.key",
229215
[{datatype, string}]}.
230216

deps/rabbitmq_peer_discovery_etcd/src/rabbit_peer_discovery_etcd.erl

Lines changed: 7 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -16,8 +16,6 @@
1616
-export([init/0, list_nodes/0, supports_registration/0, register/0, unregister/0,
1717
post_registration/0, lock/1, unlock/1]).
1818

19-
-define(ETCD_CLIENT, rabbitmq_peer_discovery_etcd_v3_client).
20-
2119
%%
2220
%% API
2321
%%
@@ -36,10 +34,10 @@ init() ->
3634
_ = application:ensure_all_started(eetcd),
3735
Formation = application:get_env(rabbit, cluster_formation, []),
3836
Opts = maps:from_list(proplists:get_value(peer_discovery_etcd, Formation, [])),
39-
{ok, Pid} = rabbitmq_peer_discovery_etcd_v3_client:start_link(Opts),
37+
{ok, Pid} = rabbitmq_peer_discovery_etcd_client:start_link(Opts),
4038
%% unlink so that this supervisor's lifecycle does not affect RabbitMQ core
4139
unlink(Pid),
42-
rabbit_log:debug("etcd peer discovery: v3 client pid: ~tp", [whereis(rabbitmq_peer_discovery_etcd_v3_client)])
40+
rabbit_log:debug("etcd peer discovery client pid: ~tp", [whereis(rabbitmq_peer_discovery_etcd_client)])
4341
end,
4442
rabbit_peer_discovery_util:maybe_backend_configured(?BACKEND_CONFIG_KEY, NoOp, NoOp, Run),
4543

@@ -64,7 +62,7 @@ list_nodes() ->
6462
%% The node with the lowest create_revision is thus selected
6563
%% based on the assumption that the create_revision remains
6664
%% consistent throughout the lifetime of the etcd key.
67-
[{_, Node} | _] = rabbitmq_peer_discovery_etcd_v3_client:list_nodes(),
65+
[{_, Node} | _] = rabbitmq_peer_discovery_etcd_client:list_nodes(),
6866
{ok, {Node, disc}}
6967
end,
7068
rabbit_peer_discovery_util:maybe_backend_configured(?BACKEND_CONFIG_KEY, Fun0, Fun1, Fun2).
@@ -79,14 +77,14 @@ supports_registration() ->
7977
-spec register() -> ok | {error, string()}.
8078

8179
register() ->
82-
Result = ?ETCD_CLIENT:register(),
80+
Result = rabbitmq_peer_discovery_etcd_client:register(),
8381
rabbit_log:info("Registered node with etcd"),
8482
Result.
8583

8684

8785
-spec unregister() -> ok | {error, string()}.
8886
unregister() ->
89-
%% This backend unregisters on plugin (etcd v3 client) deactivation
87+
%% This backend unregisters on plugin (etcd client) deactivation
9088
%% because by the time unregistration happens, the plugin and thus the client
9189
%% it provides are already gone. MK.
9290
ok.
@@ -101,7 +99,7 @@ post_registration() ->
10199

102100
lock(Nodes) when is_list(Nodes) ->
103101
Node = node(),
104-
case rabbitmq_peer_discovery_etcd_v3_client:lock(Node) of
102+
case rabbitmq_peer_discovery_etcd_client:lock(Node) of
105103
{ok, GeneratedKey} -> {ok, GeneratedKey};
106104
{error, _} = Error -> Error
107105
end.
@@ -110,4 +108,4 @@ lock(Nodes) when is_list(Nodes) ->
110108
-spec unlock(Data :: term()) -> ok.
111109

112110
unlock(GeneratedKey) ->
113-
rabbitmq_peer_discovery_etcd_v3_client:unlock(GeneratedKey).
111+
rabbitmq_peer_discovery_etcd_client:unlock(GeneratedKey).

deps/rabbitmq_peer_discovery_etcd/src/rabbitmq_peer_discovery_etcd_app.erl

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ start(_Type, _StartArgs) ->
2020

2121
prep_stop(_State) ->
2222
try
23-
rabbitmq_peer_discovery_etcd_v3_client:unregister()
23+
rabbitmq_peer_discovery_etcd_client:unregister()
2424
catch
2525
_:_ -> ok
2626
end.

deps/rabbitmq_peer_discovery_etcd/src/rabbitmq_peer_discovery_etcd_v3_client.erl renamed to deps/rabbitmq_peer_discovery_etcd/src/rabbitmq_peer_discovery_etcd_client.erl

Lines changed: 36 additions & 35 deletions
Original file line numberDiff line numberDiff line change
@@ -5,7 +5,7 @@
55
%% Copyright (c) 2007-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
66
%%
77

8-
-module(rabbitmq_peer_discovery_etcd_v3_client).
8+
-module(rabbitmq_peer_discovery_etcd_client).
99

1010
%% API
1111
-export([]).
@@ -85,11 +85,11 @@ init(Args) ->
8585
callback_mode() -> [state_functions, state_enter].
8686

8787
terminate(Reason, State, Data) ->
88-
rabbit_log:debug("etcd v3 API client will terminate in state ~tp, reason: ~tp",
88+
rabbit_log:debug("etcd client will terminate in state ~tp, reason: ~tp",
8989
[State, Reason]),
9090
_ = disconnect(?ETCD_CONN_NAME, Data),
91-
rabbit_log:debug("etcd v3 API client has disconnected"),
92-
rabbit_log:debug("etcd v3 API client: total number of connections to etcd is ~tp", [length(eetcd_conn_sup:info())]),
91+
rabbit_log:debug("etcd client has disconnected"),
92+
rabbit_log:debug("etcd client: total number of connections to etcd is ~tp", [length(eetcd_conn_sup:info())]),
9393
ok.
9494

9595
register() ->
@@ -133,25 +133,19 @@ unlock(ServerRef, LockKey) ->
133133
%%
134134

135135
recover(enter, _PrevState, #statem_data{endpoints = Endpoints}) ->
136-
rabbit_log:debug("etcd v3 API client has entered recovery state, endpoints: ~ts",
136+
rabbit_log:debug("etcd client has entered recovery state, endpoints: ~ts",
137137
[string:join(Endpoints, ",")]),
138138
keep_state_and_data;
139139
recover(internal, start, Data = #statem_data{endpoints = Endpoints, connection_monitor = Ref}) ->
140-
rabbit_log:debug("etcd v3 API client will attempt to connect, endpoints: ~ts",
140+
rabbit_log:debug("etcd client will attempt to connect, endpoints: ~ts",
141141
[string:join(Endpoints, ",")]),
142142
maybe_demonitor(Ref),
143-
{Transport, TransportOpts} = pick_transport(Data),
144-
case Transport of
145-
tcp -> rabbit_log:info("etcd v3 API client is configured to connect over plain TCP, without using TLS");
146-
tls -> rabbit_log:info("etcd v3 API client is configured to use TLS")
147-
end,
148-
ConnName = ?ETCD_CONN_NAME,
149-
case connect(ConnName, Endpoints, Transport, TransportOpts, Data) of
143+
case connect(?ETCD_CONN_NAME, Endpoints, Data) of
150144
{ok, Pid} ->
151-
rabbit_log:debug("etcd v3 API client connection: ~tp", [Pid]),
152-
rabbit_log:debug("etcd v3 API client: total number of connections to etcd is ~tp", [length(eetcd_conn_sup:info())]),
145+
rabbit_log:debug("etcd client connection: ~tp", [Pid]),
146+
rabbit_log:debug("etcd client: total number of connections to etcd is ~tp", [length(eetcd_conn_sup:info())]),
153147
{next_state, connected, Data#statem_data{
154-
connection_name = ConnName,
148+
connection_name = ?ETCD_CONN_NAME,
155149
connection_pid = Pid,
156150
connection_monitor = monitor(process, Pid)
157151
}};
@@ -166,7 +160,7 @@ recover(state_timeout, _PrevState, Data) ->
166160
_ = ensure_disconnected(?ETCD_CONN_NAME, Data),
167161
{next_state, recover, reset_statem_data(Data)};
168162
recover({call, From}, Req, _Data) ->
169-
rabbit_log:error("etcd v3 API: client received a call ~tp while not connected, will do nothing", [Req]),
163+
rabbit_log:error("etcd client received a call ~tp while not connected, will do nothing", [Req]),
170164
gen_statem:reply(From, {error, not_connected}),
171165
keep_state_and_data.
172166

@@ -320,20 +314,21 @@ error_is_already_started({_Endpoint, already_started}) ->
320314
error_is_already_started({_Endpoint, _}) ->
321315
false.
322316

323-
connect(Name, Endpoints, Transport, TransportOpts, Data) ->
317+
connect(Name, Endpoints, Data) ->
324318
case eetcd_conn:lookup(Name) of
325319
{ok, Pid} when is_pid(Pid) ->
326320
{ok, Pid};
327321
{error, eetcd_conn_unavailable} ->
328-
do_connect(Name, Endpoints, Transport, TransportOpts, Data)
322+
do_connect(Name, Endpoints, Data)
329323
end.
330324

331-
do_connect(Name, Endpoints, Transport, TransportOpts, Data = #statem_data{username = Username}) ->
325+
do_connect(Name, Endpoints, Data = #statem_data{username = Username}) ->
326+
Opts = connection_options(Data),
332327
case Username of
333328
undefined -> rabbit_log:info("etcd peer discovery: will connect to etcd without authentication (no credentials configured)");
334329
_ -> rabbit_log:info("etcd peer discovery: will connect to etcd as user '~ts'", [Username])
335330
end,
336-
case eetcd:open(Name, Endpoints, connection_options(Data), Transport, TransportOpts) of
331+
case eetcd:open(Name, Endpoints, Opts) of
337332
{ok, Pid} -> {ok, Pid};
338333
{error, Errors0} ->
339334
Errors = case is_list(Errors0) of
@@ -354,16 +349,6 @@ do_connect(Name, Endpoints, Transport, TransportOpts, Data = #statem_data{userna
354349
end
355350
end.
356351

357-
connection_options(#statem_data{username = Username, obfuscated_password = Password}) ->
358-
SharedOpts = [{mode, random}],
359-
case {Username, Password} of
360-
{undefined, _} -> SharedOpts;
361-
{_, undefined} -> SharedOpts;
362-
{UVal, PVal} ->
363-
[{name, UVal}, {password, to_list(deobfuscate(PVal))}] ++ SharedOpts
364-
end.
365-
366-
367352
obfuscate(undefined) -> undefined;
368353
obfuscate(Password) ->
369354
credentials_obfuscation:encrypt(to_binary(Password)).
@@ -429,7 +414,23 @@ normalize_settings(Map) when is_map(Map) ->
429414
maps:merge(maps:without([etcd_prefix, lock_wait_time], Map),
430415
#{endpoints => AllEndpoints}).
431416

432-
pick_transport(#statem_data{tls_options = []}) ->
433-
{tcp, []};
434-
pick_transport(#statem_data{tls_options = Opts}) ->
435-
{tls, Opts}.
417+
connection_options(#statem_data{tls_options = TlsOpts,
418+
username = Username,
419+
obfuscated_password = Password}) ->
420+
Opts0 = case TlsOpts of
421+
[] ->
422+
rabbit_log:info("etcd client is configured to use plain TCP (without TLS)"),
423+
[{transport, tcp}];
424+
_ ->
425+
rabbit_log:info("etcd client is configured to use TLS"),
426+
[{transport, tls} | TlsOpts]
427+
end,
428+
Opts = [{mode, random} | Opts0],
429+
case Username =:= undefined orelse
430+
Password =:= undefined of
431+
true ->
432+
Opts;
433+
false ->
434+
[{name, Username},
435+
{password, to_list(deobfuscate(Password))}] ++ Opts
436+
end.

deps/rabbitmq_peer_discovery_etcd/src/rabbitmq_peer_discovery_etcd_sup.erl

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -24,16 +24,16 @@ init([]) ->
2424
Fun1 = fun() -> {ok, {Flags, []}} end,
2525
Fun2 = fun(_) ->
2626
%% we stop the previously started client and "re-attach" it. MK.
27-
rabbitmq_peer_discovery_etcd_v3_client:stop(),
27+
rabbitmq_peer_discovery_etcd_client:stop(),
2828
Formation = application:get_env(rabbit, cluster_formation, []),
2929
Opts = maps:from_list(proplists:get_value(peer_discovery_etcd, Formation, [])),
3030
EtcdClientFSM = #{
31-
id => rabbitmq_peer_discovery_etcd_v3_client,
32-
start => {rabbitmq_peer_discovery_etcd_v3_client, start_link, [Opts]},
31+
id => rabbitmq_peer_discovery_etcd_client,
32+
start => {rabbitmq_peer_discovery_etcd_client, start_link, [Opts]},
3333
restart => permanent,
3434
shutdown => ?SUPERVISOR_WAIT,
3535
type => worker,
36-
modules => [rabbitmq_peer_discovery_etcd_v3_client]
36+
modules => [rabbitmq_peer_discovery_etcd_client]
3737
},
3838
Specs = [
3939
EtcdClientFSM

deps/rabbitmq_peer_discovery_etcd/test/system_SUITE.erl

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -34,17 +34,17 @@
3434

3535
all() ->
3636
[
37-
{group, v3_client},
37+
{group, client},
3838
{group, clustering}
3939
].
4040

4141
groups() ->
4242
[
43-
{v3_client, [], [
44-
etcd_connection_sanity_check_test,
45-
init_opens_a_connection_test,
46-
registration_with_locking_test
47-
]},
43+
{client, [], [
44+
etcd_connection_sanity_check_test,
45+
init_opens_a_connection_test,
46+
registration_with_locking_test
47+
]},
4848
{clustering, [], [start_one_member_at_a_time,
4949
start_members_concurrently]}
5050
].
@@ -266,12 +266,12 @@ registration_with_locking_test(Config) ->
266266
end,
267267
rabbit_ct_helpers:await_condition(Condition1, 90000),
268268

269-
{ok, LockOwnerKey} = rabbitmq_peer_discovery_etcd_v3_client:lock(Pid, node()),
270-
rabbitmq_peer_discovery_etcd_v3_client:register(Pid),
271-
?assertEqual(ok, rabbitmq_peer_discovery_etcd_v3_client:unlock(Pid, LockOwnerKey)),
269+
{ok, LockOwnerKey} = rabbitmq_peer_discovery_etcd_client:lock(Pid, node()),
270+
rabbitmq_peer_discovery_etcd_client:register(Pid),
271+
?assertEqual(ok, rabbitmq_peer_discovery_etcd_client:unlock(Pid, LockOwnerKey)),
272272

273273
Condition2 = fun() ->
274-
case rabbitmq_peer_discovery_etcd_v3_client:list_nodes(Pid) of
274+
case rabbitmq_peer_discovery_etcd_client:list_nodes(Pid) of
275275
[{_, N}] when N =:= node() ->
276276
true;
277277
_ ->
@@ -381,7 +381,7 @@ assert_full_cluster(Config) ->
381381
%%
382382

383383
start_client(Endpoints) ->
384-
case rabbitmq_peer_discovery_etcd_v3_client:start(#{endpoints => Endpoints}) of
384+
case rabbitmq_peer_discovery_etcd_client:start(#{endpoints => Endpoints}) of
385385
{ok, Pid} ->
386386
{ok, Pid};
387387
{error, {already_started, Pid}} ->

0 commit comments

Comments
 (0)