Skip to content

Commit

Permalink
Merge pull request #13169 from rabbitmq/mergify/bp/v4.0.x/pr-13167
Browse files Browse the repository at this point in the history
New health checks for metadata store initialization (backport #13167)
  • Loading branch information
michaelklishin authored Jan 29, 2025
2 parents 8645d17 + 255034d commit 9409baf
Show file tree
Hide file tree
Showing 17 changed files with 442 additions and 30 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
## This Source Code Form is subject to the terms of the Mozilla Public
## License, v. 2.0. If a copy of the MPL was not distributed with this
## file, You can obtain one at https://mozilla.org/MPL/2.0/.
##
## Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.

defmodule RabbitMQ.CLI.Diagnostics.Commands.CheckIfMetadataStoreIsInitializedCommand do
@moduledoc """
Exits with a non-zero code if the node reports that its metadata store
has finished its initialization procedure.
This command is meant to be used in health checks.
"""

@behaviour RabbitMQ.CLI.CommandBehaviour

use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
use RabbitMQ.CLI.Core.MergesNoDefaults
use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments

def run([], %{node: node_name, timeout: timeout}) do
:rabbit_misc.rpc_call(node_name, :rabbit_db, :is_init_finished, [], timeout)
end


def output(true, %{silent: true}) do
{:ok, :check_passed}
end
def output(true, %{formatter: "json"}) do
{:ok, %{"result" => "ok"}}
end
def output(true, %{node: node_name} = _options) do
{:ok, "Metadata store on node #{node_name} has completed its initialization"}
end

def output(false, %{silent: true}) do
{:error, :check_failed}
end
def output(false, %{node: node_name, formatter: "json"}) do
{:error, :check_failed,
%{
"result" => "error",
"message" => "Metadata store on node #{node_name} reports to not yet have finished initialization"
}}
end
def output(false, %{node: node_name} = _options) do
{:error,
"Metadata store on node #{node_name} reports to not yet have finished initialization"}
end

use RabbitMQ.CLI.DefaultOutput

def help_section(), do: :observability_and_health_checks

def description(),
do:
"Health check that exits with a non-zero code if the metadata store on target node has not yet finished initializing"

def usage, do: "check_if_metadata_store_initialized"

def banner([], %{node: node_name}) do
"Checking if metadata store on node #{node_name} has finished initializing ..."
end
end
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
## This Source Code Form is subject to the terms of the Mozilla Public
## License, v. 2.0. If a copy of the MPL was not distributed with this
## file, You can obtain one at https://mozilla.org/MPL/2.0/.
##
## Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.

defmodule RabbitMQ.CLI.Diagnostics.Commands.CheckIfMetadataStoreIsInitializedWithDataCommand do
@moduledoc """
Exits with a non-zero code if the node reports that its metadata store
has finished its initialization procedure.
This command is meant to be used in health checks.
"""

@behaviour RabbitMQ.CLI.CommandBehaviour

use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
use RabbitMQ.CLI.Core.MergesNoDefaults
use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments

def run([], %{node: node_name, timeout: timeout}) do
case :rabbit_misc.rpc_call(node_name, :rabbit_db, :is_init_finished, [], timeout) do
{:badrpc, _} = err ->
err

{:error, _} = err ->
err

{:error, _, _} = err ->
err

false -> false

true ->
case :rabbit_misc.rpc_call(node_name, :rabbit_db_vhost, :count_all, [], timeout) do
{:error, _} = err ->
err

{:ok, n} ->
(n > 0)
end
end
end

def output(true, %{silent: true}) do
{:ok, :check_passed}
end
def output(true, %{formatter: "json"}) do
{:ok, %{"result" => "ok"}}
end
def output(true, %{node: node_name} = _options) do
{:ok, "Metadata store on node #{node_name} has completed its initialization"}
end

def output(false, %{silent: true}) do
{:error, :check_failed}
end
def output(false, %{node: node_name, formatter: "json"}) do
{:error, :check_failed,
%{
"result" => "error",
"message" => "Metadata store on node #{node_name} reports to not yet have finished initialization"
}}
end
def output(false, %{node: node_name} = _options) do
{:error,
"Metadata store on node #{node_name} reports to not yet have finished initialization"}
end

use RabbitMQ.CLI.DefaultOutput

def help_section(), do: :observability_and_health_checks

def description(),
do:
"Health check that exits with a non-zero code if the metadata store on target node has not yet finished initializing"

def usage, do: "check_if_metadata_store_initialized"

def banner([], %{node: node_name}) do
"Checking if metadata store on node #{node_name} has finished initializing ..."
end
end
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
## This Source Code Form is subject to the terms of the Mozilla Public
## License, v. 2.0. If a copy of the MPL was not distributed with this
## file, You can obtain one at https://mozilla.org/MPL/2.0/.
##
## Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.

defmodule CheckIfMetadataStoreIsInitializedCommandTest do
use ExUnit.Case, async: false
import TestHelper

@command RabbitMQ.CLI.Diagnostics.Commands.CheckIfMetadataStoreIsInitializedCommand

setup_all do
RabbitMQ.CLI.Core.Distribution.start()

start_rabbitmq_app()

on_exit([], fn ->
start_rabbitmq_app()
end)

:ok
end

setup context do
{:ok,
opts: %{
node: get_rabbit_hostname(),
timeout: context[:test_timeout] || 30000
}}
end

test "merge_defaults: nothing to do" do
assert @command.merge_defaults([], %{}) == {[], %{}}
end

test "validate: treats positional arguments as a failure" do
assert @command.validate(["extra-arg"], %{}) == {:validation_failure, :too_many_args}
end

test "validate: treats empty positional arguments and default switches as a success" do
assert @command.validate([], %{}) == :ok
end

@tag test_timeout: 3000
test "run: targeting an unreachable node throws a badrpc", context do
assert match?(
{:badrpc, _},
@command.run([], Map.merge(context[:opts], %{node: :jake@thedog}))
)
end

test "run: when the RabbitMQ app is booted and started, returns true", context do
await_rabbitmq_startup()

assert @command.run([], context[:opts])
end

test "output: when the result is true, returns successfully", context do
assert match?({:ok, _}, @command.output(true, context[:opts]))
end

# this is a check command
test "output: when the result is false, returns an error", context do
assert match?({:error, _}, @command.output(false, context[:opts]))
end
end
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
## This Source Code Form is subject to the terms of the Mozilla Public
## License, v. 2.0. If a copy of the MPL was not distributed with this
## file, You can obtain one at https://mozilla.org/MPL/2.0/.
##
## Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.

defmodule CheckIfMetadataStoreIsInitializedWithDataCommandTest do
use ExUnit.Case, async: false
import TestHelper

@command RabbitMQ.CLI.Diagnostics.Commands.CheckIfMetadataStoreIsInitializedWithDataCommand

setup_all do
RabbitMQ.CLI.Core.Distribution.start()

start_rabbitmq_app()

on_exit([], fn ->
start_rabbitmq_app()
end)

:ok
end

setup context do
{:ok,
opts: %{
node: get_rabbit_hostname(),
timeout: context[:test_timeout] || 30000
}}
end

test "merge_defaults: nothing to do" do
assert @command.merge_defaults([], %{}) == {[], %{}}
end

test "validate: treats positional arguments as a failure" do
assert @command.validate(["extra-arg"], %{}) == {:validation_failure, :too_many_args}
end

test "validate: treats empty positional arguments and default switches as a success" do
assert @command.validate([], %{}) == :ok
end

@tag test_timeout: 3000
test "run: targeting an unreachable node throws a badrpc", context do
assert match?(
{:badrpc, _},
@command.run([], Map.merge(context[:opts], %{node: :jake@thedog}))
)
end

test "run: when the RabbitMQ app is booted and started, returns true", context do
await_rabbitmq_startup()

assert @command.run([], context[:opts])
end

test "output: when the result is true, returns successfully", context do
assert match?({:ok, _}, @command.output(true, context[:opts]))
end

# this is a check command
test "output: when the result is false, returns an error", context do
assert match?({:error, _}, @command.output(false, context[:opts]))
end
end
2 changes: 2 additions & 0 deletions deps/rabbitmq_management/src/rabbit_mgmt_dispatcher.erl
Original file line number Diff line number Diff line change
Expand Up @@ -195,6 +195,8 @@ dispatcher() ->
%% modern generation of fine-grained health checks
{"/health/checks/alarms", rabbit_mgmt_wm_health_check_alarms, []},
{"/health/checks/local-alarms", rabbit_mgmt_wm_health_check_local_alarms, []},
{"/health/checks/metadata-store/initialized", rabbit_mgmt_wm_health_check_metadata_store_initialized, []},
{"/health/checks/metadata-store/initialized/with-data", rabbit_mgmt_wm_health_check_metadata_store_initialized_with_data, []},
{"/health/checks/certificate-expiration/:within/:unit", rabbit_mgmt_wm_health_check_certificate_expiration, []},
{"/health/checks/port-listener/:port", rabbit_mgmt_wm_health_check_port_listener, []},
{"/health/checks/protocol-listener/:protocol", rabbit_mgmt_wm_health_check_protocol_listener, []},
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ to_json(ReqData, Context) ->
rabbit_mgmt_util:vhost(ReqData),
ReqData,
Context,
fun(_Ch) -> rabbit_mgmt_util:reply([{status, ok}], ReqData, Context) end
fun(_Ch) -> rabbit_mgmt_util:reply(#{status => ok}, ReqData, Context) end
).

is_authorized(ReqData, Context) ->
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ to_json(ReqData, Context) ->
end,
case rabbit_alarm:get_alarms(Timeout) of
[] ->
rabbit_mgmt_util:reply([{status, ok}], ReqData, Context);
rabbit_mgmt_util:reply(#{status => ok}, ReqData, Context);
Xs when length(Xs) > 0 ->
Msg = "There are alarms in effect in the cluster",
failure(Msg, Xs, ReqData, Context)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,18 +50,20 @@ to_json(ReqData, Context) ->
end, [], Local),
case ExpiringListeners of
[] ->
rabbit_mgmt_util:reply([{status, ok}], ReqData, Context);
rabbit_mgmt_util:reply(#{status => ok}, ReqData, Context);
_ ->
Msg = <<"Certificates expiring">>,
failure(Msg, ExpiringListeners, ReqData, Context)
end
end.

failure(Message, Listeners, ReqData, Context) ->
{Response, ReqData1, Context1} = rabbit_mgmt_util:reply([{status, failed},
{reason, Message},
{expired, Listeners}],
ReqData, Context),
Body = #{
status => failed,
reason => Message,
expired => Listeners
},
{Response, ReqData1, Context1} = rabbit_mgmt_util:reply(Body, ReqData, Context),
{stop, cowboy_req:reply(503, #{}, Response, ReqData1), Context1}.

is_authorized(ReqData, Context) ->
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ to_json(ReqData, Context) ->
end,
case rabbit_alarm:get_local_alarms(Timeout) of
[] ->
rabbit_mgmt_util:reply([{status, ok}], ReqData, Context);
rabbit_mgmt_util:reply(#{status => ok}, ReqData, Context);
Xs when length(Xs) > 0 ->
Msg = "There are alarms in effect on the node",
failure(Msg, Xs, ReqData, Context)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
%%

%% An HTTP API counterpart of 'rabbitmq-dignoastics check_local_alarms'
-module(rabbit_mgmt_wm_health_check_metadata_store_initialized).

-export([init/2, to_json/2, content_types_provided/2, is_authorized/2]).
-export([resource_exists/2]).
-export([variances/2]).

-include("rabbit_mgmt.hrl").
-include_lib("rabbitmq_management_agent/include/rabbit_mgmt_records.hrl").

%%--------------------------------------------------------------------

init(Req, _State) ->
{cowboy_rest, rabbit_mgmt_headers:set_common_permission_headers(Req, ?MODULE), #context{}}.

variances(Req, Context) ->
{[<<"accept-encoding">>, <<"origin">>], Req, Context}.

content_types_provided(ReqData, Context) ->
{rabbit_mgmt_util:responder_map(to_json), ReqData, Context}.

resource_exists(ReqData, Context) ->
{true, ReqData, Context}.

to_json(ReqData, Context) ->
Result = rabbit_db:is_init_finished(),
case Result of
true ->
rabbit_mgmt_util:reply(#{status => ok}, ReqData, Context);
false ->
Msg = "Metadata store has not yet reported as initialized",
failure(Msg, ReqData, Context)
end.

failure(Message, ReqData, Context) ->
Body = #{
status => failed,
reason => rabbit_data_coercion:to_binary(Message)
},
{Response, ReqData1, Context1} = rabbit_mgmt_util:reply(Body, ReqData, Context),
{stop, cowboy_req:reply(?HEALTH_CHECK_FAILURE_STATUS, #{}, Response, ReqData1), Context1}.

is_authorized(ReqData, Context) ->
rabbit_mgmt_util:is_authorized(ReqData, Context).
Loading

0 comments on commit 9409baf

Please sign in to comment.