Skip to content

Commit f85d9df

Browse files
By @Ayanda-D: new CLI health check that detects QQs without an elected reachable leader #13433 (#13487)
* Implement rabbitmq-queues leader_health_check command for quorum queues (cherry picked from commit c26edbe) * Tests for rabbitmq-queues leader_health_check command (cherry picked from commit 6cc03b0) * Ensure calling ParentPID in leader health check execution and reuse and extend formatting API, with amqqueue:to_printable/2 (cherry picked from commit 76d66a1) * Extend core leader health check tests and update badrpc error handling in cli tests (cherry picked from commit 857e2a7) * Refactor leader_health_check command validators and ignore vhost arg (cherry picked from commit 6cf9339) * Update leader_health_check_command description and banner (cherry picked from commit 96b8bce) * Improve output formatting for healthy leaders and support silent mode in rabbitmq-queues leader_health_check command (cherry picked from commit 239a69b) * Support global flag to run leader health check for all queues in all vhosts on local node (cherry picked from commit 48ba3e1) * Return immediately for leader health checks on empty vhosts (cherry picked from commit 7873737) * Rename leader health check timeout refs (cherry picked from commit b7dec89) * Update banner message for global leader health check (cherry picked from commit c7da4d5) * QQ leader-health-check: check_process_limit_safety before spawning leader checks (cherry picked from commit 1736845) * Log leader health check result in broker logs (if any leaderless queues) (cherry picked from commit 1084179) * Ensure check_passed result for leader health internal calls) (cherry picked from commit 68739a6) * Extend CLI format output to process check_passed payload (cherry picked from commit 5f5e992) * Format leader healthcheck result log and function exports (cherry picked from commit ebffd7d) * Change leader_health_check command scope from queues to diagnostics (cherry picked from commit 663fc98) * Update (c) line year (cherry picked from commit df82f12) * Rename command to check_for_quorum_queues_without_an_elected_leader and use across_all_vhosts option for global checks (cherry picked from commit b2acbae) * Use rabbit_db_queue for qq leader health check lookups and introduce rabbit_db_queue:get_all_by_type_and_vhost/2. Update leader health check timeout to 5s and process limit threshold to 20% of node's process_limit. (cherry picked from commit 7a8e166) * Update tests: quorum_queue_SUITE and rabbit_db_queue_SUITE (cherry picked from commit 9bdb81f) * Fix typo (cli test module) (cherry picked from commit 6158568) * Small refactor - simpler final leader health check result return on function head match (cherry picked from commit ea07938) * Clear dialyzer warning & fix type spec (cherry picked from commit a45aa81) * Ignore result without strict match to avoid diayzer warning (cherry picked from commit bb43c0b) * 'rabbitmq-diagnostics check_for_quorum_queues_without_an_elected_leader' documentation edits (cherry picked from commit 845230b) * 'rabbitmq-diagnostics check_for_quorum_queues_without_an_elected_leader' output copywriting (cherry picked from commit 235f43b) * diagnostics check_for_quorum_queues_without_an_elected_leader: behave like a health check w.r.t. error reporting (cherry picked from commit db73767) * check_for_quorum_queues_without_an_elected_leader: handle --quiet and --silent plus simplify function heads. References #13433. (cherry picked from commit 7b39231) --------- Co-authored-by: Ayanda Dube <[email protected]>
1 parent 0ae28af commit f85d9df

File tree

8 files changed

+434
-1
lines changed

8 files changed

+434
-1
lines changed

deps/rabbit/src/amqqueue.erl

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -66,10 +66,12 @@
6666
pattern_match_on_type/1,
6767
pattern_match_on_durable/1,
6868
pattern_match_on_type_and_durable/2,
69+
pattern_match_on_type_and_vhost/2,
6970
reset_decorators/1,
7071
set_immutable/1,
7172
qnode/1,
7273
to_printable/1,
74+
to_printable/2,
7375
macros/0]).
7476

7577
-define(record_version, amqqueue_v2).
@@ -531,6 +533,12 @@ pattern_match_on_durable(IsDurable) ->
531533
pattern_match_on_type_and_durable(Type, IsDurable) ->
532534
#amqqueue{type = Type, durable = IsDurable, _ = '_'}.
533535

536+
-spec pattern_match_on_type_and_vhost(atom(), binary()) ->
537+
amqqueue_pattern().
538+
539+
pattern_match_on_type_and_vhost(Type, VHost) ->
540+
#amqqueue{type = Type, vhost = VHost, _ = '_'}.
541+
534542
-spec reset_decorators(amqqueue()) -> amqqueue().
535543

536544
reset_decorators(#amqqueue{} = Queue) ->
@@ -564,6 +572,14 @@ to_printable(#amqqueue{name = QName = #resource{name = Name},
564572
<<"virtual_host">> => VHost,
565573
<<"type">> => Type}.
566574

575+
-spec to_printable(rabbit_types:r(queue), atom() | binary()) -> #{binary() => any()}.
576+
to_printable(QName = #resource{name = Name, virtual_host = VHost}, Type) ->
577+
_ = rabbit_queue_type:discover(Type),
578+
#{<<"readable_name">> => rabbit_data_coercion:to_binary(rabbit_misc:rs(QName)),
579+
<<"name">> => Name,
580+
<<"virtual_host">> => VHost,
581+
<<"type">> => Type}.
582+
567583
% private
568584

569585
macros() ->

deps/rabbit/src/rabbit_db_queue.erl

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
get_all/0,
2222
get_all/1,
2323
get_all_by_type/1,
24+
get_all_by_type_and_vhost/2,
2425
get_all_by_type_and_node/3,
2526
list/0,
2627
count/0,
@@ -829,6 +830,28 @@ get_all_by_type(Type) ->
829830
khepri => fun() -> get_all_by_pattern_in_khepri(Pattern) end
830831
}).
831832

833+
%% -------------------------------------------------------------------
834+
%% get_all_by_type_and_vhost().
835+
%% -------------------------------------------------------------------
836+
837+
-spec get_all_by_type_and_vhost(Type, VHost) -> [Queue] when
838+
Type :: atom(),
839+
VHost :: binary(),
840+
Queue :: amqqueue:amqqueue().
841+
842+
%% @doc Gets all queues belonging to the given type and vhost
843+
%%
844+
%% @returns a list of queue records.
845+
%%
846+
%% @private
847+
848+
get_all_by_type_and_vhost(Type, VHost) ->
849+
Pattern = amqqueue:pattern_match_on_type_and_vhost(Type, VHost),
850+
rabbit_khepri:handle_fallback(
851+
#{mnesia => fun() -> get_all_by_pattern_in_mnesia(Pattern) end,
852+
khepri => fun() -> get_all_by_pattern_in_khepri(Pattern) end
853+
}).
854+
832855
get_all_by_pattern_in_mnesia(Pattern) ->
833856
rabbit_db:list_in_mnesia(?MNESIA_TABLE, Pattern).
834857

deps/rabbit/src/rabbit_quorum_queue.erl

Lines changed: 78 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -84,6 +84,9 @@
8484
file_handle_other_reservation/0,
8585
file_handle_release_reservation/0]).
8686

87+
-export([leader_health_check/2,
88+
run_leader_health_check/4]).
89+
8790
-ifdef(TEST).
8891
-export([filter_promotable/2,
8992
ra_machine_config/1]).
@@ -147,6 +150,8 @@
147150
-define(SNAPSHOT_INTERVAL, 8192). %% the ra default is 4096
148151
% -define(UNLIMITED_PREFETCH_COUNT, 2000). %% something large for ra
149152
-define(MIN_CHECKPOINT_INTERVAL, 8192). %% the ra default is 16384
153+
-define(LEADER_HEALTH_CHECK_TIMEOUT, 5_000).
154+
-define(GLOBAL_LEADER_HEALTH_CHECK_TIMEOUT, 60_000).
150155

151156
%%----------- QQ policies ---------------------------------------------------
152157

@@ -2179,3 +2184,76 @@ file_handle_other_reservation() ->
21792184

21802185
file_handle_release_reservation() ->
21812186
ok.
2187+
2188+
leader_health_check(QueueNameOrRegEx, VHost) ->
2189+
%% Set a process limit threshold to 20% of ErlangVM process limit, beyond which
2190+
%% we cannot spawn any new processes for executing QQ leader health checks.
2191+
ProcessLimitThreshold = round(0.2 * erlang:system_info(process_limit)),
2192+
2193+
leader_health_check(QueueNameOrRegEx, VHost, ProcessLimitThreshold).
2194+
2195+
leader_health_check(QueueNameOrRegEx, VHost, ProcessLimitThreshold) ->
2196+
Qs =
2197+
case VHost of
2198+
across_all_vhosts ->
2199+
rabbit_db_queue:get_all_by_type(?MODULE);
2200+
VHost when is_binary(VHost) ->
2201+
rabbit_db_queue:get_all_by_type_and_vhost(?MODULE, VHost)
2202+
end,
2203+
check_process_limit_safety(length(Qs), ProcessLimitThreshold),
2204+
ParentPID = self(),
2205+
HealthCheckRef = make_ref(),
2206+
HealthCheckPids =
2207+
lists:flatten(
2208+
[begin
2209+
{resource, _VHostN, queue, QueueName} = QResource = amqqueue:get_name(Q),
2210+
case re:run(QueueName, QueueNameOrRegEx, [{capture, none}]) of
2211+
match ->
2212+
{ClusterName, _} = rabbit_amqqueue:pid_of(Q),
2213+
_Pid = spawn(fun() -> run_leader_health_check(ClusterName, QResource, HealthCheckRef, ParentPID) end);
2214+
_ ->
2215+
[]
2216+
end
2217+
end || Q <- Qs, amqqueue:get_type(Q) == ?MODULE]),
2218+
Result = wait_for_leader_health_checks(HealthCheckRef, length(HealthCheckPids), []),
2219+
_ = spawn(fun() -> maybe_log_leader_health_check_result(Result) end),
2220+
Result.
2221+
2222+
run_leader_health_check(ClusterName, QResource, HealthCheckRef, From) ->
2223+
Leader = ra_leaderboard:lookup_leader(ClusterName),
2224+
2225+
%% Ignoring result here is required to clear a diayzer warning.
2226+
_ =
2227+
case ra_server_proc:ping(Leader, ?LEADER_HEALTH_CHECK_TIMEOUT) of
2228+
{pong,leader} ->
2229+
From ! {ok, HealthCheckRef, QResource};
2230+
_ ->
2231+
From ! {error, HealthCheckRef, QResource}
2232+
end,
2233+
ok.
2234+
2235+
wait_for_leader_health_checks(_Ref, 0, UnhealthyAcc) -> UnhealthyAcc;
2236+
wait_for_leader_health_checks(Ref, N, UnhealthyAcc) ->
2237+
receive
2238+
{ok, Ref, _QResource} ->
2239+
wait_for_leader_health_checks(Ref, N - 1, UnhealthyAcc);
2240+
{error, Ref, QResource} ->
2241+
wait_for_leader_health_checks(Ref, N - 1, [amqqueue:to_printable(QResource, ?MODULE) | UnhealthyAcc])
2242+
after
2243+
?GLOBAL_LEADER_HEALTH_CHECK_TIMEOUT ->
2244+
UnhealthyAcc
2245+
end.
2246+
2247+
check_process_limit_safety(QCount, ProcessLimitThreshold) ->
2248+
case (erlang:system_info(process_count) + QCount) >= ProcessLimitThreshold of
2249+
true ->
2250+
rabbit_log:warning("Leader health check not permitted, process limit threshold will be exceeded."),
2251+
throw({error, leader_health_check_process_limit_exceeded});
2252+
false ->
2253+
ok
2254+
end.
2255+
2256+
maybe_log_leader_health_check_result([]) -> ok;
2257+
maybe_log_leader_health_check_result(Result) ->
2258+
Qs = lists:map(fun(R) -> catch maps:get(<<"readable_name">>, R) end, Result),
2259+
rabbit_log:warning("Leader health check result (unhealthy leaders detected): ~tp", [Qs]).

deps/rabbit/test/quorum_queue_SUITE.erl

Lines changed: 130 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -195,7 +195,8 @@ all_tests() ->
195195
priority_queue_2_1_ratio,
196196
requeue_multiple_true,
197197
requeue_multiple_false,
198-
subscribe_from_each
198+
subscribe_from_each,
199+
leader_health_check
199200
].
200201

201202
memory_tests() ->
@@ -4205,6 +4206,129 @@ amqpl_headers(Config) ->
42054206
ok = amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DeliveryTag,
42064207
multiple = true}).
42074208

4209+
leader_health_check(Config) ->
4210+
VHost1 = <<"vhost1">>,
4211+
VHost2 = <<"vhost2">>,
4212+
4213+
set_up_vhost(Config, VHost1),
4214+
set_up_vhost(Config, VHost2),
4215+
4216+
%% check empty vhost
4217+
?assertEqual([],
4218+
rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4219+
[<<".*">>, VHost1])),
4220+
?assertEqual([],
4221+
rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4222+
[<<".*">>, across_all_vhosts])),
4223+
4224+
Conn1 = rabbit_ct_client_helpers:open_unmanaged_connection(Config, 0, VHost1),
4225+
{ok, Ch1} = amqp_connection:open_channel(Conn1),
4226+
4227+
Conn2 = rabbit_ct_client_helpers:open_unmanaged_connection(Config, 0, VHost2),
4228+
{ok, Ch2} = amqp_connection:open_channel(Conn2),
4229+
4230+
Qs1 = [<<"Q.1">>, <<"Q.2">>, <<"Q.3">>],
4231+
Qs2 = [<<"Q.4">>, <<"Q.5">>, <<"Q.6">>],
4232+
4233+
%% in vhost1
4234+
[?assertEqual({'queue.declare_ok', Q, 0, 0},
4235+
declare(Ch1, Q, [{<<"x-queue-type">>, longstr, <<"quorum">>}]))
4236+
|| Q <- Qs1],
4237+
4238+
%% in vhost2
4239+
[?assertEqual({'queue.declare_ok', Q, 0, 0},
4240+
declare(Ch2, Q, [{<<"x-queue-type">>, longstr, <<"quorum">>}]))
4241+
|| Q <- Qs2],
4242+
4243+
%% test sucessful health checks in vhost1, vhost2, across_all_vhosts
4244+
?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4245+
[<<".*">>, VHost1])),
4246+
?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4247+
[<<"Q.*">>, VHost1])),
4248+
[?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4249+
[Q, VHost1])) || Q <- Qs1],
4250+
4251+
?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4252+
[<<".*">>, VHost2])),
4253+
?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4254+
[<<"Q.*">>, VHost2])),
4255+
[?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4256+
[Q, VHost2])) || Q <- Qs2],
4257+
4258+
?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4259+
[<<".*">>, across_all_vhosts])),
4260+
?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4261+
[<<"Q.*">>, across_all_vhosts])),
4262+
4263+
%% clear leaderboard
4264+
Qs = rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_amqqueue, list, []),
4265+
4266+
[{_Q1_ClusterName, _Q1Res},
4267+
{_Q2_ClusterName, _Q2Res},
4268+
{_Q3_ClusterName, _Q3Res},
4269+
{_Q4_ClusterName, _Q4Res},
4270+
{_Q5_ClusterName, _Q5Res},
4271+
{_Q6_ClusterName, _Q6Res}] = QQ_Clusters =
4272+
lists:usort(
4273+
[begin
4274+
{ClusterName, _} = amqqueue:get_pid(Q),
4275+
{ClusterName, amqqueue:get_name(Q)}
4276+
end
4277+
|| Q <- Qs, amqqueue:get_type(Q) == rabbit_quorum_queue]),
4278+
4279+
[Q1Data, Q2Data, Q3Data, Q4Data, Q5Data, Q6Data] = QQ_Data =
4280+
[begin
4281+
rabbit_ct_broker_helpers:rpc(Config, 0, ra_leaderboard, clear, [Q_ClusterName]),
4282+
_QData = amqqueue:to_printable(Q_Res, rabbit_quorum_queue)
4283+
end
4284+
|| {Q_ClusterName, Q_Res} <- QQ_Clusters],
4285+
4286+
%% test failed health checks in vhost1, vhost2, across_all_vhosts
4287+
?assertEqual([Q1Data], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4288+
[<<"Q.1">>, VHost1])),
4289+
?assertEqual([Q2Data], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4290+
[<<"Q.2">>, VHost1])),
4291+
?assertEqual([Q3Data], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4292+
[<<"Q.3">>, VHost1])),
4293+
?assertEqual([Q1Data, Q2Data, Q3Data],
4294+
lists:usort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4295+
[<<".*">>, VHost1]))),
4296+
?assertEqual([Q1Data, Q2Data, Q3Data],
4297+
lists:usort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4298+
[<<"Q.*">>, VHost1]))),
4299+
4300+
?assertEqual([Q4Data], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4301+
[<<"Q.4">>, VHost2])),
4302+
?assertEqual([Q5Data], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4303+
[<<"Q.5">>, VHost2])),
4304+
?assertEqual([Q6Data], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4305+
[<<"Q.6">>, VHost2])),
4306+
?assertEqual([Q4Data, Q5Data, Q6Data],
4307+
lists:usort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4308+
[<<".*">>, VHost2]))),
4309+
?assertEqual([Q4Data, Q5Data, Q6Data],
4310+
lists:usort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4311+
[<<"Q.*">>, VHost2]))),
4312+
4313+
?assertEqual(QQ_Data,
4314+
lists:usort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4315+
[<<"Q.*">>, across_all_vhosts]))),
4316+
?assertEqual(QQ_Data,
4317+
lists:usort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue, leader_health_check,
4318+
[<<"Q.*">>, across_all_vhosts]))),
4319+
4320+
%% cleanup
4321+
[?assertMatch(#'queue.delete_ok'{},
4322+
amqp_channel:call(Ch1, #'queue.delete'{queue = Q}))
4323+
|| Q <- Qs1],
4324+
[?assertMatch(#'queue.delete_ok'{},
4325+
amqp_channel:call(Ch1, #'queue.delete'{queue = Q}))
4326+
|| Q <- Qs2],
4327+
4328+
amqp_connection:close(Conn1),
4329+
amqp_connection:close(Conn2).
4330+
4331+
42084332
leader_locator_client_local(Config) ->
42094333
[Server1 | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
42104334
Q = ?config(queue_name, Config),
@@ -4525,6 +4649,11 @@ declare_passive(Ch, Q, Args) ->
45254649
auto_delete = false,
45264650
passive = true,
45274651
arguments = Args}).
4652+
4653+
set_up_vhost(Config, VHost) ->
4654+
rabbit_ct_broker_helpers:add_vhost(Config, VHost),
4655+
rabbit_ct_broker_helpers:set_full_permissions(Config, <<"guest">>, VHost).
4656+
45284657
assert_queue_type(Server, Q, Expected) ->
45294658
assert_queue_type(Server, <<"/">>, Q, Expected).
45304659

deps/rabbit/test/rabbit_db_queue_SUITE.erl

Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ all_tests() ->
3535
get_all,
3636
get_all_by_vhost,
3737
get_all_by_type,
38+
get_all_by_type_and_vhost,
3839
get_all_by_type_and_node,
3940
list,
4041
count,
@@ -198,6 +199,30 @@ get_all_by_type1(_Config) ->
198199
?assertEqual([Q4], rabbit_db_queue:get_all_by_type(rabbit_stream_queue)),
199200
passed.
200201

202+
get_all_by_type_and_vhost(Config) ->
203+
passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_all_by_type_and_vhost1, [Config]).
204+
205+
get_all_by_type_and_vhost1(_Config) ->
206+
VHost1 = <<"carrots">>,
207+
VHost2 = <<"cabage">>,
208+
QName = rabbit_misc:r(VHost1, queue, <<"test-queue">>),
209+
QName2 = rabbit_misc:r(VHost2, queue, <<"test-queue2">>),
210+
QName3 = rabbit_misc:r(VHost2, queue, <<"test-queue3">>),
211+
QName4 = rabbit_misc:r(VHost1, queue, <<"test-queue4">>),
212+
Q = new_queue(QName, rabbit_classic_queue),
213+
Q2 = new_queue(QName2, rabbit_quorum_queue),
214+
Q3 = new_queue(QName3, rabbit_quorum_queue),
215+
Q4 = new_queue(QName4, rabbit_stream_queue),
216+
Quorum = lists:sort([Q2, Q3]),
217+
?assertEqual([], rabbit_db_queue:get_all_by_type_and_vhost(rabbit_classic_queue, VHost1)),
218+
?assertEqual([], lists:sort(rabbit_db_queue:get_all_by_type_and_vhost(rabbit_quorum_queue, VHost2))),
219+
?assertEqual([], rabbit_db_queue:get_all_by_type_and_vhost(rabbit_stream_queue, VHost1)),
220+
set_list([Q, Q2, Q3, Q4]),
221+
?assertEqual([Q], rabbit_db_queue:get_all_by_type_and_vhost(rabbit_classic_queue, VHost1)),
222+
?assertEqual(Quorum, lists:sort(rabbit_db_queue:get_all_by_type_and_vhost(rabbit_quorum_queue, VHost2))),
223+
?assertEqual([Q4], rabbit_db_queue:get_all_by_type_and_vhost(rabbit_stream_queue, VHost1)),
224+
passed.
225+
201226
get_all_by_type_and_node(Config) ->
202227
passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_all_by_type_and_node1, [Config]).
203228

deps/rabbitmq_cli/lib/rabbitmq/cli/core/output.ex

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,10 @@ defmodule RabbitMQ.CLI.Core.Output do
1818
:ok
1919
end
2020

21+
def format_output({:ok, :check_passed, output}, formatter, options) do
22+
{:ok, formatter.format_output(output, options)}
23+
end
24+
2125
def format_output({:ok, output}, formatter, options) do
2226
{:ok, formatter.format_output(output, options)}
2327
end

0 commit comments

Comments
 (0)