Skip to content

Commit f6a025b

Browse files
dumbbellmergify[bot]
authored andcommitted
rabbit_networking: Support systems with distinct listeners for IPv4 and IPv6
[Why] If the system uses distinct listeners for IPv4 and IPv6, the functions listing or stopping listeners would crash because they assumed that the system would listen implicitly to IPv4 when starting an IPv6 listener. This is a common Linuxism. [How] The code now understands that a given protocol can have several associated listeners. The name and return value of several exported functions changed to reflect that they work on several elements or return a list of elements. Therefore it is a breaking change for plugins relying on these functions. (cherry picked from commit 578c3a6)
1 parent 8e118d7 commit f6a025b

File tree

7 files changed

+45
-46
lines changed

7 files changed

+45
-46
lines changed

deps/rabbit/src/rabbit_networking.erl

Lines changed: 22 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -33,8 +33,8 @@
3333
close_all_user_connections/2,
3434
force_connection_event_refresh/1, force_non_amqp_connection_event_refresh/1,
3535
handshake/2, handshake/3, tcp_host/1,
36-
ranch_ref/1, ranch_ref/2, ranch_ref_of_protocol/1, ranch_ref_to_protocol/1,
37-
listener_of_protocol/1, stop_ranch_listener_of_protocol/1,
36+
ranch_ref/1, ranch_ref/2, ranch_refs_of_protocol/1, ranch_ref_to_protocol/1,
37+
listeners_of_protocol/1, stop_ranch_listeners_of_protocol/1,
3838
list_local_connections_of_protocol/1]).
3939

4040
%% Used by TCP-based transports, e.g. STOMP adapter
@@ -229,9 +229,9 @@ ranch_ref(undefined) ->
229229
ranch_ref(IPAddress, Port) ->
230230
{acceptor, IPAddress, Port}.
231231

232-
-spec ranch_ref_of_protocol(atom()) -> ranch:ref() | undefined.
233-
ranch_ref_of_protocol(Protocol) ->
234-
ranch_ref(listener_of_protocol(Protocol)).
232+
-spec ranch_refs_of_protocol(atom()) -> [ranch:ref()].
233+
ranch_refs_of_protocol(Protocol) ->
234+
[ranch_ref(Listener) || Listener <- listeners_of_protocol(Protocol)].
235235

236236
-spec ranch_ref_to_protocol(ranch:ref()) -> atom() | undefined.
237237
ranch_ref_to_protocol({acceptor, IPAddress, Port}) ->
@@ -248,32 +248,32 @@ ranch_ref_to_protocol({acceptor, IPAddress, Port}) ->
248248
ranch_ref_to_protocol(_) ->
249249
undefined.
250250

251-
-spec listener_of_protocol(atom()) -> #listener{}.
252-
listener_of_protocol(Protocol) ->
251+
-spec listeners_of_protocol(atom()) -> [#listener{}].
252+
listeners_of_protocol(Protocol) ->
253253
MatchSpec = #listener{
254254
protocol = Protocol,
255255
_ = '_'
256256
},
257-
case ets:match_object(?ETS_TABLE, MatchSpec) of
258-
[] -> undefined;
259-
[Row] -> Row
260-
end.
257+
ets:match_object(?ETS_TABLE, MatchSpec).
261258

262-
-spec stop_ranch_listener_of_protocol(atom()) -> ok | {error, not_found}.
263-
stop_ranch_listener_of_protocol(Protocol) ->
264-
case ranch_ref_of_protocol(Protocol) of
265-
undefined -> ok;
266-
Ref ->
267-
?LOG_DEBUG("Stopping Ranch listener for protocol ~ts", [Protocol]),
268-
ranch:stop_listener(Ref)
259+
-spec stop_ranch_listeners_of_protocol(atom()) -> ok.
260+
stop_ranch_listeners_of_protocol(Protocol) ->
261+
case ranch_refs_of_protocol(Protocol) of
262+
[] ->
263+
ok;
264+
Refs ->
265+
?LOG_DEBUG("Stopping Ranch listeners for protocol ~ts", [Protocol]),
266+
lists:foreach(fun ranch:stop_listener/1, Refs)
269267
end.
270268

271269
-spec list_local_connections_of_protocol(atom()) -> [pid()].
272270
list_local_connections_of_protocol(Protocol) ->
273-
case ranch_ref_of_protocol(Protocol) of
274-
undefined -> [];
275-
AcceptorRef -> ranch:procs(AcceptorRef, connections)
276-
end.
271+
Refs = ranch_refs_of_protocol(Protocol),
272+
lists:flatten(
273+
lists:map(
274+
fun(Ref) ->
275+
ranch:procs(Ref, connections)
276+
end, Refs)).
277277

278278
-spec start_tcp_listener(
279279
listener_config(), integer()) -> 'ok' | {'error', term()}.

deps/rabbitmq_management/src/rabbit_mgmt_wm_health_check_below_node_connection_limit.erl

Lines changed: 6 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -55,10 +55,9 @@ to_json(ReqData, Context) ->
5555
end.
5656

5757
protocol_connection_count(Protocol) ->
58-
case rabbit_networking:ranch_ref_of_protocol(Protocol) of
59-
undefined ->
60-
0;
61-
RanchRef ->
62-
#{active_connections := Count} = ranch:info(RanchRef),
63-
Count
64-
end.
58+
Refs = rabbit_networking:ranch_refs_of_protocol(Protocol),
59+
lists:foldl(
60+
fun(Ref, Acc) ->
61+
#{active_connections := Count} = ranch:info(Ref),
62+
Acc + Count
63+
end, 0, Refs).

deps/rabbitmq_management/src/rabbit_mgmt_wm_health_check_ready_to_serve_clients.erl

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -49,10 +49,9 @@ to_json(ReqData, Context) ->
4949
check() ->
5050
case rabbit:is_serving() of
5151
true ->
52-
RanchRefs0 = [
53-
rabbit_networking:ranch_ref_of_protocol(amqp),
54-
rabbit_networking:ranch_ref_of_protocol('amqp/ssl')
55-
],
52+
RanchRefs0 = (
53+
rabbit_networking:ranch_refs_of_protocol(amqp) ++
54+
rabbit_networking:ranch_refs_of_protocol('amqp/ssl')),
5655
RanchRefs = [R || R <- RanchRefs0, R =/= undefined],
5756
case RanchRefs of
5857
[_ | _] ->

deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -63,8 +63,8 @@ init([{Listeners, SslListeners0}]) ->
6363

6464
-spec stop_listeners() -> ok.
6565
stop_listeners() ->
66-
_ = rabbit_networking:stop_ranch_listener_of_protocol(?MQTT_TCP_PROTOCOL),
67-
_ = rabbit_networking:stop_ranch_listener_of_protocol(?MQTT_TLS_PROTOCOL),
66+
rabbit_networking:stop_ranch_listeners_of_protocol(?MQTT_TCP_PROTOCOL),
67+
rabbit_networking:stop_ranch_listeners_of_protocol(?MQTT_TLS_PROTOCOL),
6868
ok.
6969

7070
%%

deps/rabbitmq_stomp/src/rabbit_stomp_sup.erl

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -42,8 +42,8 @@ init([{Listeners, SslListeners0}, Configuration]) ->
4242
SslListeners)}}.
4343

4444
stop_listeners() ->
45-
_ = rabbit_networking:stop_ranch_listener_of_protocol(?TCP_PROTOCOL),
46-
_ = rabbit_networking:stop_ranch_listener_of_protocol(?TLS_PROTOCOL),
45+
rabbit_networking:stop_ranch_listeners_of_protocol(?TCP_PROTOCOL),
46+
rabbit_networking:stop_ranch_listeners_of_protocol(?TLS_PROTOCOL),
4747
ok.
4848

4949
%%

deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -45,8 +45,8 @@ prep_stop(State) ->
4545

4646
-spec stop(_) -> ok.
4747
stop(_State) ->
48-
_ = rabbit_networking:stop_ranch_listener_of_protocol(?TCP_PROTOCOL),
49-
_ = rabbit_networking:stop_ranch_listener_of_protocol(?TLS_PROTOCOL),
48+
rabbit_networking:stop_ranch_listeners_of_protocol(?TCP_PROTOCOL),
49+
rabbit_networking:stop_ranch_listeners_of_protocol(?TLS_PROTOCOL),
5050
ok.
5151

5252
init([]) -> {ok, {{one_for_one, 1, 5}, []}}.

deps/rabbitmq_web_stomp/src/rabbit_web_stomp_listener.erl

Lines changed: 8 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -54,8 +54,8 @@ init() ->
5454
ok.
5555

5656
stop(State) ->
57-
_ = rabbit_networking:stop_ranch_listener_of_protocol(?TCP_PROTOCOL),
58-
_ = rabbit_networking:stop_ranch_listener_of_protocol(?TLS_PROTOCOL),
57+
rabbit_networking:stop_ranch_listeners_of_protocol(?TCP_PROTOCOL),
58+
rabbit_networking:stop_ranch_listeners_of_protocol(?TLS_PROTOCOL),
5959
State.
6060

6161
-spec list_connections() -> [pid()].
@@ -77,11 +77,12 @@ close_all_client_connections(Reason) ->
7777
%%
7878

7979
connection_pids_of_protocol(Protocol) ->
80-
case rabbit_networking:ranch_ref_of_protocol(Protocol) of
81-
undefined -> [];
82-
AcceptorRef ->
83-
lists:map(fun cowboy_ws_connection_pid/1, ranch:procs(AcceptorRef, connections))
84-
end.
80+
Refs = rabbit_networking:ranch_refs_of_protocol(Protocol),
81+
lists:foldl(
82+
fun(Ref, Acc) ->
83+
Procs = lists:map(fun cowboy_ws_connection_pid/1, ranch:procs(Ref, connections)),
84+
Acc ++ Procs
85+
end, [], Refs).
8586

8687
-spec cowboy_ws_connection_pid(pid()) -> pid().
8788
cowboy_ws_connection_pid(RanchConnPid) ->

0 commit comments

Comments
 (0)