Skip to content

Commit e47690e

Browse files
SimonUngemergify[bot]
authored andcommitted
See #7593. Use connection_max to stop connections in rabbitmq
(cherry picked from commit b42e99a)
1 parent 3d54107 commit e47690e

File tree

7 files changed

+143
-10
lines changed

7 files changed

+143
-10
lines changed

deps/rabbit/BUILD.bazel

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,7 @@ _APP_ENV = """[
3939
{frame_max, 131072},
4040
%% see rabbitmq-server#1593
4141
{channel_max, 2047},
42-
{connection_max, infinity},
42+
{ranch_connection_max, infinity},
4343
{heartbeat, 60},
4444
{msg_store_file_size_limit, 16777216},
4545
{msg_store_shutdown_timeout, 600000},
@@ -507,6 +507,11 @@ rabbitmq_integration_suite(
507507
size = "medium",
508508
)
509509

510+
rabbitmq_integration_suite(
511+
name = "per_node_limit_SUITE",
512+
size = "medium",
513+
)
514+
510515
rabbitmq_integration_suite(
511516
name = "metrics_SUITE",
512517
size = "medium",

deps/rabbit/Makefile

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@ define PROJECT_ENV
2626
{frame_max, 131072},
2727
%% see rabbitmq-server#1593
2828
{channel_max, 2047},
29-
{connection_max, infinity},
29+
{ranch_connection_max, infinity},
3030
{heartbeat, 60},
3131
{msg_store_file_size_limit, 16777216},
3232
{msg_store_shutdown_timeout, 600000},

deps/rabbit/priv/schema/rabbit.schema

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -883,6 +883,20 @@ end}.
883883
end
884884
}.
885885

886+
{mapping, "ranch_connection_max", "rabbit.ranch_connection_max",
887+
[{datatype, [{atom, infinity}, integer]}]}.
888+
889+
{translation, "rabbit.ranch_connection_max",
890+
fun(Conf) ->
891+
case cuttlefish:conf_get("ranch_connection_max", Conf, undefined) of
892+
undefined -> cuttlefish:unset();
893+
infinity -> infinity;
894+
Val when is_integer(Val) -> Val;
895+
_ -> cuttlefish:invalid("should be a non-negative integer")
896+
end
897+
end
898+
}.
899+
886900

887901
{mapping, "max_message_size", "rabbit.max_message_size",
888902
[{datatype, integer}, {validators, ["max_message_size"]}]}.

deps/rabbit/src/rabbit_reader.erl

Lines changed: 16 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1212,12 +1212,13 @@ handle_method0(#'connection.open'{virtual_host = VHost},
12121212
State = #v1{connection_state = opening,
12131213
connection = Connection = #connection{
12141214
log_name = ConnName,
1215+
port = Port,
12151216
user = User = #user{username = Username},
12161217
protocol = Protocol},
12171218
helper_sup = SupPid,
12181219
sock = Sock,
12191220
throttle = Throttle}) ->
1220-
1221+
ok = is_over_node_connection_limit(Port),
12211222
ok = is_over_vhost_connection_limit(VHost, User),
12221223
ok = is_over_user_connection_limit(User),
12231224
ok = rabbit_access_control:check_vhost_access(User, VHost, {socket, Sock}, #{}),
@@ -1318,6 +1319,20 @@ is_vhost_alive(VHostPath, User) ->
13181319
[VHostPath, User#user.username, VHostPath])
13191320
end.
13201321

1322+
is_over_node_connection_limit(Port) ->
1323+
{Addr, _, _} = hd(rabbit_networking:tcp_listener_addresses(Port)),
1324+
Ref = rabbit_networking:ranch_ref(Addr, Port),
1325+
#{active_connections := ActiveConns} = ranch:info(Ref),
1326+
Limit = rabbit_misc:get_env(rabbit, connection_max, infinity),
1327+
case ActiveConns > Limit of
1328+
false -> ok;
1329+
true ->
1330+
rabbit_misc:protocol_error(not_allowed,
1331+
"connection refused: "
1332+
"node connection limit (~tp) is reached",
1333+
[Limit])
1334+
end.
1335+
13211336
is_over_vhost_connection_limit(VHostPath, User) ->
13221337
try rabbit_vhost_limit:is_over_connection_limit(VHostPath) of
13231338
false -> ok;

deps/rabbit/src/tcp_listener_sup.erl

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ start_link(IPAddress, Port, Transport, SocketOpts, ProtoSup, ProtoOpts, OnStartu
3535
init({IPAddress, Port, Transport, SocketOpts, ProtoSup, ProtoOpts, OnStartup, OnShutdown,
3636
ConcurrentAcceptorCount, ConcurrentConnsSups, ConnectionType, Label}) ->
3737
{ok, AckTimeout} = application:get_env(rabbit, ssl_handshake_timeout),
38-
MaxConnections = max_conn(rabbit_misc:get_env(rabbit, connection_max, infinity),
38+
MaxConnections = max_conn(rabbit_misc:get_env(rabbit, ranch_connection_max, infinity),
3939
ConcurrentConnsSups),
4040
RanchListenerOpts = #{
4141
num_acceptors => ConcurrentAcceptorCount,

deps/rabbit/test/config_schema_SUITE_data/rabbit.snippets

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -287,13 +287,13 @@ tcp_listen_options.exit_on_close = false",
287287
"total_memory_available_override_value = 1024MB",
288288
[{rabbit,[{total_memory_available_override_value, "1024MB"}]}],
289289
[]},
290-
{connection_max,
291-
"connection_max = 999",
292-
[{rabbit,[{connection_max, 999}]}],
290+
{ranch_connection_max,
291+
"ranch_connection_max = 999",
292+
[{rabbit,[{ranch_connection_max, 999}]}],
293293
[]},
294-
{connection_max,
295-
"connection_max = infinity",
296-
[{rabbit,[{connection_max, infinity}]}],
294+
{ranch_connection_max,
295+
"ranch_connection_max = infinity",
296+
[{rabbit,[{ranch_connection_max, infinity}]}],
297297
[]},
298298
{channel_max,
299299
"channel_max = 16",
Lines changed: 99 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,99 @@
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) 2011-2023 VMware, Inc. or its affiliates. All rights reserved.
6+
%%
7+
8+
-module(per_node_limit_SUITE).
9+
10+
-include_lib("common_test/include/ct.hrl").
11+
-include_lib("amqp_client/include/amqp_client.hrl").
12+
-include_lib("eunit/include/eunit.hrl").
13+
14+
-compile(export_all).
15+
16+
all() ->
17+
[
18+
{group, parallel_tests}
19+
].
20+
21+
groups() ->
22+
[
23+
{parallel_tests, [parallel], [
24+
node_connection_limit
25+
]}
26+
].
27+
28+
suite() ->
29+
[
30+
{timetrap, {minutes, 3}}
31+
].
32+
33+
%% -------------------------------------------------------------------
34+
%% Testsuite setup/teardown.
35+
%% -------------------------------------------------------------------
36+
37+
init_per_suite(Config) ->
38+
rabbit_ct_helpers:log_environment(),
39+
rabbit_ct_helpers:run_setup_steps(Config).
40+
41+
end_per_suite(Config) ->
42+
rabbit_ct_helpers:run_teardown_steps(Config).
43+
44+
init_per_group(Group, Config) ->
45+
Config1 = rabbit_ct_helpers:set_config(Config, [
46+
{rmq_nodename_suffix, Group},
47+
{rmq_nodes_count, 1}
48+
]),
49+
rabbit_ct_helpers:run_steps(Config1,
50+
rabbit_ct_broker_helpers:setup_steps() ++
51+
rabbit_ct_client_helpers:setup_steps()).
52+
53+
end_per_group(_Group, Config) ->
54+
rabbit_ct_helpers:run_steps(Config,
55+
rabbit_ct_client_helpers:teardown_steps() ++
56+
rabbit_ct_broker_helpers:teardown_steps()).
57+
58+
init_per_testcase(Testcase, Config) ->
59+
rabbit_ct_helpers:testcase_started(Config, Testcase).
60+
61+
end_per_testcase(Testcase, Config) ->
62+
rabbit_ct_helpers:testcase_finished(Config, Testcase).
63+
64+
%% -------------------------------------------------------------------
65+
%% Test cases
66+
%% -------------------------------------------------------------------
67+
68+
node_connection_limit(Config) ->
69+
%% Set limit to 0, don't accept any connections
70+
set_node_limit(Config, 0),
71+
{error, not_allowed} = rabbit_ct_client_helpers:open_unmanaged_connection(Config, 0),
72+
73+
%% Set limit to 5, accept 5 connections
74+
Connections = open_connections_to_limit(Config, 5),
75+
%% But no more than 5
76+
{error, not_allowed} = rabbit_ct_client_helpers:open_unmanaged_connection(Config, 0),
77+
close_all_connections(Connections),
78+
79+
set_node_limit(Config, infinity),
80+
C = rabbit_ct_client_helpers:open_unmanaged_connection(Config, 0),
81+
true = is_pid(C).
82+
83+
%% -------------------------------------------------------------------
84+
%% Implementation
85+
%% -------------------------------------------------------------------
86+
87+
open_connections_to_limit(Config, Limit) ->
88+
set_node_limit(Config, Limit),
89+
Connections = [rabbit_ct_client_helpers:open_unmanaged_connection(Config, 0) || _ <- lists:seq(1,Limit)],
90+
true = lists:all(fun(E) -> is_pid(E) end, Connections),
91+
Connections.
92+
93+
close_all_connections(Connections) ->
94+
[rabbit_ct_client_helpers:close_connection(C) || C <- Connections].
95+
96+
set_node_limit(Config, Limit) ->
97+
rabbit_ct_broker_helpers:rpc(Config, 0,
98+
application,
99+
set_env, [rabbit, connection_max, Limit]).

0 commit comments

Comments
 (0)