Skip to content

Commit bb5a37f

Browse files
committed
Expand health checks for readiness to serve clients
This is a squash of the commits from `main` which add new API health checks in connection to <https://redirect.github.com/rabbitmq/rabbitmq-server/issues/13782>. GET /api/health/checks/is-in-service GET /api/health/checks/below-node-connection-limit GET /api/health/checks/ready-to-serve-clients Additionally GET /api/health/checks/protocol-listener/{protocol} Now accepts multiple protocol names, comma-separated. When listeners are available for all requested protocols then the endpoint returns 200 OK. Squashed commits: 0d692fa Prefer node-local listeners helper in protocol-listener health check This is a minor change that avoids a cluster-wide query for active listeners. The old code called `rabbit_networking:active_listeners/0` and then filtered the results by ones available on the local node. This caused an RPC and concatenation of all other cluster members' listeners and then in the next line filtered down to local nodes. Equivalently we can use `rabbit_networking:node_listeners(node())` which dumps a local ETS table. This is not a very impactful change but it's nice to keep the latency of the health-check handlers low and reduce some unnecessary cluster noise. 5d319be Accept multiple protocols in protocol listener health check This updates the health check for protocol listeners to accept a set of protocols, comma-separated. The check only returns 200 OK when all requested protocols have active listeners. 07fe630 Add an 'is-in-service' health check wrapping `rabbit:is_serving/0` This is useful for a load balancer, for example, to be able to avoid sending new connections to a node which is running and has listeners bound to TCP ports but is being drained for maintenance. 028b692 Add a health check for testing the node connection limit 67bdc01 Add a health check for testing readiness to serve clients
1 parent f415470 commit bb5a37f

7 files changed

+353
-23
lines changed

deps/rabbitmq_management/priv/www/api/index.html

Lines changed: 53 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1054,10 +1054,12 @@ <h2>Reference</h2>
10541054
<td></td>
10551055
<td></td>
10561056
<td></td>
1057-
<td class="path">/api/health/checks/protocol-listener/<i>protocol</i></td>
1057+
<td class="path">/api/health/checks/protocol-listener/<i>protocols</i></td>
10581058
<td>
1059-
Responds a 200 OK if there is an active listener for the given protocol,
1060-
otherwise responds with a 503 Service Unavailable. Valid protocol names are: amqp091, amqp10, mqtt, stomp, web-mqtt, web-stomp.
1059+
Responds a 200 OK if all given protocols have active listeners,
1060+
otherwise responds with a 503 Service Unavailable. Multiple protocols
1061+
may be provided by separating the names with commas. Valid protocol
1062+
names are: amqp091, amqp10, mqtt, stomp, web-mqtt, web-stomp.
10611063
</td>
10621064
</tr>
10631065
<tr>
@@ -1097,6 +1099,54 @@ <h2>Reference</h2>
10971099
otherwise responds with a 503 Service Unavailable.
10981100
</td>
10991101
</tr>
1102+
<tr>
1103+
<td>X</td>
1104+
<td></td>
1105+
<td></td>
1106+
<td></td>
1107+
<td class="path">/api/health/checks/is-in-service</td>
1108+
<td>
1109+
Responds a 200 OK if the target node is booted, running, and ready to
1110+
serve clients, otherwise responds with a 503 Service Unavailable. If the
1111+
target node is being drained for maintenance then this check returns 503
1112+
Service Unavailable.
1113+
</td>
1114+
</tr>
1115+
<tr>
1116+
<td>X</td>
1117+
<td></td>
1118+
<td></td>
1119+
<td></td>
1120+
<td class="path">/api/health/checks/below-node-connection-limit</td>
1121+
<td>
1122+
Responds a 200 OK if the target node has fewer connections to the AMQP
1123+
and AMQPS ports than the configured maximum, otherwise responds with a
1124+
503 Service Unavailable.
1125+
</td>
1126+
</tr>
1127+
<tr>
1128+
<td>X</td>
1129+
<td></td>
1130+
<td></td>
1131+
<td></td>
1132+
<td class="path">/api/health/checks/ready-to-serve-clients</td>
1133+
<td>
1134+
<p>
1135+
Responds a 200 OK if the target node is ready to serve clients, otherwise
1136+
responds with a 503 Service Unavailable. This check combines:
1137+
</p>
1138+
<ol>
1139+
<li>/api/health/checks/is-in-service</li>
1140+
<li>/api/health/checks/protocol-listener/amqp or /api/health/checks/protocol-listener/amqps</li>
1141+
<li>/api/health/checks/below-node-connection-limit</li>
1142+
</ol>
1143+
<p>
1144+
So this check will only return 200 OK if the target node is in service,
1145+
an AMQP or AMQPS listener is available and the target node has fewer active
1146+
AMQP and AMQPS connections that its configured limit.
1147+
</p>
1148+
</td>
1149+
</tr>
11001150
<tr>
11011151
<td>X</td>
11021152
<td></td>

deps/rabbitmq_management/src/rabbit_mgmt_dispatcher.erl

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -187,10 +187,13 @@ dispatcher() ->
187187
{"/health/checks/local-alarms", rabbit_mgmt_wm_health_check_local_alarms, []},
188188
{"/health/checks/certificate-expiration/:within/:unit", rabbit_mgmt_wm_health_check_certificate_expiration, []},
189189
{"/health/checks/port-listener/:port", rabbit_mgmt_wm_health_check_port_listener, []},
190-
{"/health/checks/protocol-listener/:protocol", rabbit_mgmt_wm_health_check_protocol_listener, []},
190+
{"/health/checks/protocol-listener/:protocols", rabbit_mgmt_wm_health_check_protocol_listener, []},
191191
{"/health/checks/virtual-hosts", rabbit_mgmt_wm_health_check_virtual_hosts, []},
192192
{"/health/checks/node-is-mirror-sync-critical", rabbit_mgmt_wm_health_check_node_is_mirror_sync_critical, []},
193193
{"/health/checks/node-is-quorum-critical", rabbit_mgmt_wm_health_check_node_is_quorum_critical, []},
194+
{"/health/checks/is-in-service", rabbit_mgmt_wm_health_check_is_in_service, []},
195+
{"/health/checks/below-node-connection-limit", rabbit_mgmt_wm_health_check_below_node_connection_limit, []},
196+
{"/health/checks/ready-to-serve-clients", rabbit_mgmt_wm_health_check_ready_to_serve_clients, []},
194197
{"/reset", rabbit_mgmt_wm_reset, []},
195198
{"/reset/:node", rabbit_mgmt_wm_reset, []},
196199
{"/rebalance/queues", rabbit_mgmt_wm_rebalance_queues, [{queues, all}]},
Lines changed: 63 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,63 @@
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) 2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
6+
%%
7+
8+
-module(rabbit_mgmt_wm_health_check_below_node_connection_limit).
9+
10+
-export([init/2]).
11+
-export([to_json/2, content_types_provided/2]).
12+
-export([variances/2]).
13+
14+
-include("rabbit_mgmt.hrl").
15+
-include_lib("rabbitmq_management_agent/include/rabbit_mgmt_records.hrl").
16+
17+
init(Req, _State) ->
18+
Req1 = rabbit_mgmt_headers:set_no_cache_headers(
19+
rabbit_mgmt_headers:set_common_permission_headers(
20+
Req, ?MODULE), ?MODULE),
21+
{cowboy_rest, Req1, #context{}}.
22+
23+
variances(Req, Context) ->
24+
{[<<"accept-encoding">>, <<"origin">>], Req, Context}.
25+
26+
content_types_provided(ReqData, Context) ->
27+
{rabbit_mgmt_util:responder_map(to_json), ReqData, Context}.
28+
29+
to_json(ReqData, Context) ->
30+
ActiveConns = lists:foldl(
31+
fun(Protocol, Acc) ->
32+
Acc + protocol_connection_count(Protocol)
33+
end, 0, [amqp, 'amqp/ssl']),
34+
Limit = rabbit_misc:get_env(rabbit, connection_max, infinity),
35+
case ActiveConns < Limit of
36+
true ->
37+
rabbit_mgmt_util:reply(
38+
#{status => ok,
39+
limit => Limit,
40+
connections => ActiveConns}, ReqData, Context);
41+
false ->
42+
Body = #{
43+
status => failed,
44+
reason => <<"node connection limit is reached">>,
45+
limit => Limit,
46+
connections => ActiveConns
47+
},
48+
{Response, ReqData1, Context1} = rabbit_mgmt_util:reply(
49+
Body, ReqData, Context),
50+
{stop,
51+
cowboy_req:reply(
52+
?HEALTH_CHECK_FAILURE_STATUS, #{}, Response, ReqData1),
53+
Context1}
54+
end.
55+
56+
protocol_connection_count(Protocol) ->
57+
case rabbit_networking:ranch_ref_of_protocol(Protocol) of
58+
undefined ->
59+
0;
60+
RanchRef ->
61+
#{active_connections := Count} = ranch:info(RanchRef),
62+
Count
63+
end.
Lines changed: 44 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,44 @@
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) 2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
6+
%%
7+
8+
-module(rabbit_mgmt_wm_health_check_is_in_service).
9+
10+
-export([init/2]).
11+
-export([to_json/2, content_types_provided/2]).
12+
-export([variances/2]).
13+
14+
-include("rabbit_mgmt.hrl").
15+
-include_lib("rabbitmq_management_agent/include/rabbit_mgmt_records.hrl").
16+
17+
init(Req, _State) ->
18+
Req1 = rabbit_mgmt_headers:set_no_cache_headers(
19+
rabbit_mgmt_headers:set_common_permission_headers(
20+
Req, ?MODULE), ?MODULE),
21+
{cowboy_rest, Req1, #context{}}.
22+
23+
variances(Req, Context) ->
24+
{[<<"accept-encoding">>, <<"origin">>], Req, Context}.
25+
26+
content_types_provided(ReqData, Context) ->
27+
{rabbit_mgmt_util:responder_map(to_json), ReqData, Context}.
28+
29+
to_json(ReqData, Context) ->
30+
case rabbit:is_serving() of
31+
true ->
32+
rabbit_mgmt_util:reply(#{status => ok}, ReqData, Context);
33+
false ->
34+
Msg = "this rabbit node is not currently available to serve",
35+
failure(Msg, ReqData, Context)
36+
end.
37+
38+
failure(Message, ReqData, Context) ->
39+
Body = #{
40+
status => failed,
41+
reason => rabbit_data_coercion:to_binary(Message)
42+
},
43+
{Response, ReqData1, Context1} = rabbit_mgmt_util:reply(Body, ReqData, Context),
44+
{stop, cowboy_req:reply(?HEALTH_CHECK_FAILURE_STATUS, #{}, Response, ReqData1), Context1}.

deps/rabbitmq_management/src/rabbit_mgmt_wm_health_check_protocol_listener.erl

Lines changed: 25 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -27,38 +27,46 @@ content_types_provided(ReqData, Context) ->
2727
{rabbit_mgmt_util:responder_map(to_json), ReqData, Context}.
2828

2929
resource_exists(ReqData, Context) ->
30-
{case protocol(ReqData) of
30+
{case protocols(ReqData) of
3131
none -> false;
3232
_ -> true
3333
end, ReqData, Context}.
3434

3535
to_json(ReqData, Context) ->
36-
Protocol = normalize_protocol(protocol(ReqData)),
37-
Listeners = rabbit_networking:active_listeners(),
38-
Local = [L || #listener{node = N} = L <- Listeners, N == node()],
39-
ProtoListeners = [L || #listener{protocol = P} = L <- Local, atom_to_list(P) == Protocol],
40-
case ProtoListeners of
36+
Protocols = string:split(protocols(ReqData), ",", all),
37+
RequestedProtocols = sets:from_list(
38+
[normalize_protocol(P) || P <- Protocols],
39+
[{version, 2}]),
40+
Listeners = rabbit_networking:node_listeners(node()),
41+
ActiveProtocols = sets:from_list(
42+
[atom_to_list(P) || #listener{protocol = P} <- Listeners],
43+
[{version, 2}]),
44+
MissingProtocols = sets:to_list(sets:subtract(RequestedProtocols, ActiveProtocols)),
45+
case MissingProtocols of
4146
[] ->
42-
Msg = <<"No active listener">>,
43-
failure(Msg, Protocol, [P || #listener{protocol = P} <- Local], ReqData, Context);
47+
Body = #{status => ok,
48+
protocols => [list_to_binary(P) || P <- sets:to_list(ActiveProtocols)]},
49+
rabbit_mgmt_util:reply(Body, ReqData, Context);
4450
_ ->
45-
rabbit_mgmt_util:reply([{status, ok},
46-
{protocol, list_to_binary(Protocol)}], ReqData, Context)
51+
Msg = <<"No active listener">>,
52+
failure(Msg, MissingProtocols, sets:to_list(ActiveProtocols), ReqData, Context)
4753
end.
4854

4955
failure(Message, Missing, Protocols, ReqData, Context) ->
50-
{Response, ReqData1, Context1} = rabbit_mgmt_util:reply([{status, failed},
51-
{reason, Message},
52-
{missing, list_to_binary(Missing)},
53-
{protocols, Protocols}],
54-
ReqData, Context),
56+
Body = #{
57+
status => failed,
58+
reason => Message,
59+
missing => [list_to_binary(P) || P <- Missing],
60+
protocols => [list_to_binary(P) || P <- Protocols]
61+
},
62+
{Response, ReqData1, Context1} = rabbit_mgmt_util:reply(Body, ReqData, Context),
5563
{stop, cowboy_req:reply(503, #{}, Response, ReqData1), Context1}.
5664

5765
is_authorized(ReqData, Context) ->
5866
rabbit_mgmt_util:is_authorized(ReqData, Context).
5967

60-
protocol(ReqData) ->
61-
rabbit_mgmt_util:id(protocol, ReqData).
68+
protocols(ReqData) ->
69+
rabbit_mgmt_util:id(protocols, ReqData).
6270

6371
normalize_protocol(Protocol) ->
6472
case string:lowercase(binary_to_list(Protocol)) of
Lines changed: 81 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,81 @@
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) 2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
6+
%%
7+
8+
%% A composite health check that combines:
9+
%% * GET /api/health/checks/is-in-service
10+
%% * GET /api/health/checks/protocol-listener/amqp
11+
%% * GET /api/health/checks/below-node-connection-limit
12+
13+
-module(rabbit_mgmt_wm_health_check_ready_to_serve_clients).
14+
15+
-export([init/2]).
16+
-export([to_json/2, content_types_provided/2]).
17+
-export([variances/2]).
18+
19+
-include("rabbit_mgmt.hrl").
20+
-include_lib("rabbitmq_management_agent/include/rabbit_mgmt_records.hrl").
21+
22+
init(Req, _State) ->
23+
Req1 = rabbit_mgmt_headers:set_no_cache_headers(
24+
rabbit_mgmt_headers:set_common_permission_headers(
25+
Req, ?MODULE), ?MODULE),
26+
{cowboy_rest, Req1, #context{}}.
27+
28+
variances(Req, Context) ->
29+
{[<<"accept-encoding">>, <<"origin">>], Req, Context}.
30+
31+
content_types_provided(ReqData, Context) ->
32+
{rabbit_mgmt_util:responder_map(to_json), ReqData, Context}.
33+
34+
to_json(ReqData, Context) ->
35+
case check() of
36+
{ok, Body} ->
37+
rabbit_mgmt_util:reply(Body, ReqData, Context);
38+
{error, Body} ->
39+
{Response, ReqData1, Context1} = rabbit_mgmt_util:reply(
40+
Body, ReqData, Context),
41+
{stop,
42+
cowboy_req:reply(
43+
?HEALTH_CHECK_FAILURE_STATUS, #{}, Response, ReqData1),
44+
Context1}
45+
end.
46+
47+
check() ->
48+
case rabbit:is_serving() of
49+
true ->
50+
RanchRefs0 = [
51+
rabbit_networking:ranch_ref_of_protocol(amqp),
52+
rabbit_networking:ranch_ref_of_protocol('amqp/ssl')
53+
],
54+
RanchRefs = [R || R <- RanchRefs0, R =/= undefined],
55+
case RanchRefs of
56+
[_ | _] ->
57+
ActiveConns = lists:foldl(
58+
fun(RanchRef, Acc) ->
59+
#{active_connections := Count} = ranch:info(RanchRef),
60+
Acc + Count
61+
end, 0, RanchRefs),
62+
Limit = rabbit_misc:get_env(rabbit, connection_max, infinity),
63+
case ActiveConns < Limit of
64+
true ->
65+
{ok, #{status => ok,
66+
limit => Limit,
67+
connections => ActiveConns}};
68+
false ->
69+
{error, #{status => failed,
70+
reason => <<"node connection limit is reached">>,
71+
limit => Limit,
72+
connections => ActiveConns}}
73+
end;
74+
[] ->
75+
{error, #{status => failed,
76+
reason => <<"no active listeners for AMQP/AMQPS">>}}
77+
end;
78+
false ->
79+
{error, #{status => failed,
80+
reason => <<"the rabbit node is not currently available to serve">>}}
81+
end.

0 commit comments

Comments
 (0)