From 97183e8ab55c1ec0fdbdfa679e61f779d3fa89c7 Mon Sep 17 00:00:00 2001 From: Michael Klishin Date: Thu, 2 Jan 2025 16:50:51 -0500 Subject: [PATCH] Introduce a way to protect a virtual host from deletion Accidental "fat finger" virtual deletion accidents would be easier to avoid if there was a protection mechanism that would apply equally even to CLI tools and external applications that do not use confirmations for deletion operations. This introduce the following changes: * Virtual host metadata now supports a new queue, 'protected_from_deletion', which, when set, will be considered by key virtual host deletion function(s) * DELETE /api/vhosts/{name} was adapted to handle such blocked deletion attempts to respond with a 412 Precondition Failed status * 'rabbitmqctl list_vhosts' and 'rabbitmqctl delete_vhost' were adapted accordingly * DELETE /api/vhosts/{name}/deletion/protection is a new endpoint that can be used to remove the protective seal (the metadata key) * POST /api/vhosts/{name}/deletion/protection marks the virtual host as protected In the case of the HTTP API, all operations on virtual host metadata require administrative privileges from the target user. Other considerations: * When a virtual host does not exist, the behavior remains the same: the original, protection-unaware code path is used to preserve backwards compatibility References #12772. (cherry picked from commit f62d46c286735146bd8289e12803af9e7883180d) --- deps/rabbit/src/rabbit_db_vhost.erl | 32 +++++ deps/rabbit/src/rabbit_vhost.erl | 70 ++++++--- deps/rabbit/src/vhost.erl | 25 ++++ .../cli/ctl/commands/delete_vhost_command.ex | 10 +- .../cli/ctl/commands/list_vhosts_command.ex | 2 +- .../update_vhost_metadata_command_test.exs | 28 ++-- .../src/rabbit_ct_broker_helpers.erl | 14 ++ .../src/rabbit_mgmt_test_util.erl | 4 + .../priv/www/api/index.html | 8 ++ .../src/rabbit_mgmt_dispatcher.erl | 1 + .../src/rabbit_mgmt_util.erl | 5 +- .../src/rabbit_mgmt_wm_vhost.erl | 9 +- ...bbit_mgmt_wm_vhost_deletion_protection.erl | 88 ++++++++++++ .../test/clustering_prop_SUITE.erl | 2 +- .../test/rabbit_mgmt_http_SUITE.erl | 7 +- .../rabbit_mgmt_http_health_checks_SUITE.erl | 4 +- ...t_http_vhost_deletion_protection_SUITE.erl | 134 ++++++++++++++++++ 17 files changed, 400 insertions(+), 43 deletions(-) create mode 100644 deps/rabbitmq_management/src/rabbit_mgmt_wm_vhost_deletion_protection.erl create mode 100644 deps/rabbitmq_management/test/rabbit_mgmt_http_vhost_deletion_protection_SUITE.erl diff --git a/deps/rabbit/src/rabbit_db_vhost.erl b/deps/rabbit/src/rabbit_db_vhost.erl index 1b2a1d6f3c5e..9c925fcb0255 100644 --- a/deps/rabbit/src/rabbit_db_vhost.erl +++ b/deps/rabbit/src/rabbit_db_vhost.erl @@ -23,6 +23,8 @@ count_all/0, list/0, update/2, + enable_protection_from_deletion/1, + disable_protection_from_deletion/1, with_fun_in_mnesia_tx/2, with_fun_in_khepri_tx/2, delete/1, @@ -224,6 +226,36 @@ set_tags_in_khepri(VHostName, Tags) -> UpdateFun = fun(VHost) -> do_set_tags(VHost, Tags) end, update_in_khepri(VHostName, UpdateFun). +%% ------------------------------------------------------------------- +%% Deletion protection +%% ------------------------------------------------------------------- + +-spec enable_protection_from_deletion(VHostName) -> Ret when + VHostName :: vhost:name(), + Ret :: {ok, VHost} | + {error, {no_such_vhost, VHostName}} | + rabbit_khepri:timeout_error(), + VHost :: vhost:vhost(). +enable_protection_from_deletion(VHostName) -> + MetadataPatch = #{ + protected_from_deletion => true + }, + rabbit_log:info("Enabling deletion protection for virtual host '~ts'", [VHostName]), + merge_metadata(VHostName, MetadataPatch). + +-spec disable_protection_from_deletion(VHostName) -> Ret when + VHostName :: vhost:name(), + Ret :: {ok, VHost} | + {error, {no_such_vhost, VHostName}} | + rabbit_khepri:timeout_error(), + VHost :: vhost:vhost(). +disable_protection_from_deletion(VHostName) -> + MetadataPatch = #{ + protected_from_deletion => false + }, + rabbit_log:info("Disabling deletion protection for virtual host '~ts'", [VHostName]), + merge_metadata(VHostName, MetadataPatch). + %% ------------------------------------------------------------------- %% exists(). %% ------------------------------------------------------------------- diff --git a/deps/rabbit/src/rabbit_vhost.erl b/deps/rabbit/src/rabbit_vhost.erl index 26c25187da26..5e5b867f9f84 100644 --- a/deps/rabbit/src/rabbit_vhost.erl +++ b/deps/rabbit/src/rabbit_vhost.erl @@ -11,11 +11,11 @@ -include("vhost.hrl"). -export([recover/0, recover/1, read_config/1]). --export([add/2, add/3, add/4, delete/2, exists/1, assert/1, +-export([add/2, add/3, add/4, delete/2, delete_ignoring_protection/2, exists/1, assert/1, set_limits/2, vhost_cluster_state/1, is_running_on_all_nodes/1, await_running_on_all_nodes/2, list/0, count/0, list_names/0, all/0, all_tagged_with/1]). -export([parse_tags/1, update_tags/3]). --export([update_metadata/3]). +-export([update_metadata/3, enable_protection_from_deletion/1, disable_protection_from_deletion/1]). -export([lookup/1, default_name/0]). -export([info/1, info/2, info_all/0, info_all/1, info_all/2, info_all/3]). -export([dir/1, msg_store_dir_path/1, msg_store_dir_wildcard/0, msg_store_dir_base/0, config_file_path/1, ensure_config_file/1]). @@ -254,19 +254,22 @@ declare_default_exchanges(VHostName, ActingUser) -> -spec update_metadata(vhost:name(), vhost:metadata(), rabbit_types:username()) -> rabbit_types:ok_or_error(any()). update_metadata(Name, Metadata0, ActingUser) -> - Metadata = maps:with([description, tags, default_queue_type], Metadata0), + KnownKeys = [description, tags, default_queue_type, protected_from_deletion], + Metadata = maps:with(KnownKeys, Metadata0), case rabbit_db_vhost:merge_metadata(Name, Metadata) of {ok, VHost} -> Description = vhost:get_description(VHost), Tags = vhost:get_tags(VHost), DefaultQueueType = vhost:get_default_queue_type(VHost), + IsProtected = vhost:is_protected_from_deletion(VHost), rabbit_event:notify( vhost_updated, info(VHost) ++ [{user_who_performed_action, ActingUser}, {description, Description}, {tags, Tags}, - {default_queue_type, DefaultQueueType}]), + {default_queue_type, DefaultQueueType}, + {deletion_protection, IsProtected}]), ok; {error, _} = Error -> Error @@ -278,45 +281,61 @@ update(Name, Description, Tags, DefaultQueueType, ActingUser) -> update_metadata(Name, Metadata, ActingUser). -spec delete(vhost:name(), rabbit_types:username()) -> rabbit_types:ok_or_error(any()). -delete(VHost, ActingUser) -> +delete(Name, ActingUser) -> + case rabbit_db_vhost:get(Name) of + %% preserve the original behavior for backwards compatibility + undefined -> delete_ignoring_protection(Name, ActingUser); + VHost -> + case vhost:is_protected_from_deletion(VHost) of + true -> + Msg = "Refusing to delete virtual host '~ts' because it is protected from deletion", + rabbit_log:debug(Msg, [Name]), + {error, protected_from_deletion}; + false -> + delete_ignoring_protection(Name, ActingUser) + end + end. + +-spec delete_ignoring_protection(vhost:name(), rabbit_types:username()) -> rabbit_types:ok_or_error(any()). +delete_ignoring_protection(Name, ActingUser) -> %% FIXME: We are forced to delete the queues and exchanges outside %% the TX below. Queue deletion involves sending messages to the queue %% process, which in turn results in further database actions and %% eventually the termination of that process. Exchange deletion causes %% notifications which must be sent outside the TX - rabbit_log:info("Deleting vhost '~ts'", [VHost]), + rabbit_log:info("Deleting vhost '~ts'", [Name]), %% TODO: This code does a lot of "list resources, walk through the list to %% delete each resource". This feature should be provided by each called %% modules, like `rabbit_amqqueue:delete_all_for_vhost(VHost)'. These new %% calls would be responsible for the atomicity, not this code. %% Clear the permissions first to prohibit new incoming connections when deleting a vhost - rabbit_log:info("Clearing permissions in vhost '~ts' because it's being deleted", [VHost]), - ok = rabbit_auth_backend_internal:clear_all_permissions_for_vhost(VHost, ActingUser), - rabbit_log:info("Deleting queues in vhost '~ts' because it's being deleted", [VHost]), + rabbit_log:info("Clearing permissions in vhost '~ts' because it's being deleted", [Name]), + ok = rabbit_auth_backend_internal:clear_all_permissions_for_vhost(Name, ActingUser), + rabbit_log:info("Deleting queues in vhost '~ts' because it's being deleted", [Name]), QDelFun = fun (Q) -> rabbit_amqqueue:delete(Q, false, false, ActingUser) end, [begin - Name = amqqueue:get_name(Q), - assert_benign(rabbit_amqqueue:with(Name, QDelFun), ActingUser) - end || Q <- rabbit_amqqueue:list(VHost)], - rabbit_log:info("Deleting exchanges in vhost '~ts' because it's being deleted", [VHost]), - ok = rabbit_exchange:delete_all(VHost, ActingUser), - rabbit_log:info("Clearing policies and runtime parameters in vhost '~ts' because it's being deleted", [VHost]), - _ = rabbit_runtime_parameters:clear_vhost(VHost, ActingUser), - rabbit_log:debug("Removing vhost '~ts' from the metadata storage because it's being deleted", [VHost]), - Ret = case rabbit_db_vhost:delete(VHost) of + QName = amqqueue:get_name(Q), + assert_benign(rabbit_amqqueue:with(QName, QDelFun), ActingUser) + end || Q <- rabbit_amqqueue:list(Name)], + rabbit_log:info("Deleting exchanges in vhost '~ts' because it's being deleted", [Name]), + ok = rabbit_exchange:delete_all(Name, ActingUser), + rabbit_log:info("Clearing policies and runtime parameters in vhost '~ts' because it's being deleted", [Name]), + _ = rabbit_runtime_parameters:clear_vhost(Name, ActingUser), + rabbit_log:debug("Removing vhost '~ts' from the metadata storage because it's being deleted", [Name]), + Ret = case rabbit_db_vhost:delete(Name) of true -> ok = rabbit_event:notify( vhost_deleted, - [{name, VHost}, + [{name, Name}, {user_who_performed_action, ActingUser}]); false -> - {error, {no_such_vhost, VHost}}; + {error, {no_such_vhost, Name}}; {error, _} = Err -> Err end, %% After vhost was deleted from the database, we try to stop vhost %% supervisors on all the nodes. - rabbit_vhost_sup_sup:delete_on_all_nodes(VHost), + rabbit_vhost_sup_sup:delete_on_all_nodes(Name), Ret. -spec put_vhost(vhost:name(), @@ -530,6 +549,14 @@ lookup(VHostName) -> VHost -> VHost end. +-spec enable_protection_from_deletion(vhost:name()) -> vhost:vhost() | rabbit_types:ok_or_error(any()). +enable_protection_from_deletion(VHostName) -> + rabbit_db_vhost:enable_protection_from_deletion(VHostName). + +-spec disable_protection_from_deletion(vhost:name()) -> vhost:vhost() | rabbit_types:ok_or_error(any()). +disable_protection_from_deletion(VHostName) -> + rabbit_db_vhost:disable_protection_from_deletion(VHostName). + -spec assert(vhost:name()) -> 'ok'. assert(VHostName) -> case exists(VHostName) of @@ -624,6 +651,7 @@ i(cluster_state, VHost) -> vhost_cluster_state(vhost:get_name(VHost)); i(description, VHost) -> vhost:get_description(VHost); i(tags, VHost) -> vhost:get_tags(VHost); i(default_queue_type, VHost) -> rabbit_queue_type:short_alias_of(default_queue_type(vhost:get_name(VHost))); +i(protected_from_deletion, VHost) -> vhost:is_protected_from_deletion(VHost); i(metadata, VHost) -> DQT = rabbit_queue_type:short_alias_of(default_queue_type(vhost:get_name(VHost))), case vhost:get_metadata(VHost) of diff --git a/deps/rabbit/src/vhost.erl b/deps/rabbit/src/vhost.erl index f28607f7cdfa..a16116a3a99e 100644 --- a/deps/rabbit/src/vhost.erl +++ b/deps/rabbit/src/vhost.erl @@ -30,6 +30,11 @@ set_limits/2, set_metadata/2, merge_metadata/2, + + is_protected_from_deletion/1, + enable_protection_from_deletion/1, + disable_protection_from_deletion/1, + new_metadata/3, is_tagged_with/2, @@ -89,6 +94,8 @@ vhost_pattern/0, vhost_v2_pattern/0]). +-define(DELETION_PROTECTION_KEY, protected_from_deletion). + -spec new(name(), limits()) -> vhost(). new(Name, Limits) -> #vhost{virtual_host = Name, limits = Limits}. @@ -123,6 +130,7 @@ info_keys() -> description, tags, default_queue_type, + protected_from_deletion, metadata, tracing, cluster_state]. @@ -187,6 +195,23 @@ metadata_merger(default_queue_type, _, NewVHostDefaultQueueType) -> metadata_merger(_, _, NewMetadataValue) -> NewMetadataValue. +-spec is_protected_from_deletion(vhost()) -> boolean(). +is_protected_from_deletion(VHost) -> + case get_metadata(VHost) of + Map when map_size(Map) =:= 0 -> false; + #{?DELETION_PROTECTION_KEY := true} -> true; + #{?DELETION_PROTECTION_KEY := false} -> false; + _ -> false + end. + +-spec enable_protection_from_deletion(vhost()) -> vhost(). +enable_protection_from_deletion(VHost) -> + merge_metadata(VHost, #{?DELETION_PROTECTION_KEY => true}). + +-spec disable_protection_from_deletion(vhost()) -> vhost(). +disable_protection_from_deletion(VHost) -> + merge_metadata(VHost, #{?DELETION_PROTECTION_KEY => false}). + -spec new_metadata(binary(), [atom()], rabbit_queue_type:queue_type() | 'undefined') -> metadata(). new_metadata(Description, Tags, undefined) -> #{description => Description, diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/delete_vhost_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/delete_vhost_command.ex index 02f741b62d0c..273234b02648 100644 --- a/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/delete_vhost_command.ex +++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/delete_vhost_command.ex @@ -5,10 +5,12 @@ ## Copyright (c) 2007-2023 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. defmodule RabbitMQ.CLI.Ctl.Commands.DeleteVhostCommand do - alias RabbitMQ.CLI.Core.{DocGuide, Helpers} + alias RabbitMQ.CLI.Core.{DocGuide, Helpers, ExitCodes} @behaviour RabbitMQ.CLI.CommandBehaviour + @protected_from_deletion_err "Cannot delete this virtual host: it is protected from deletion. To lift the protection, inspect and update its metadata" + use RabbitMQ.CLI.Core.MergesNoDefaults use RabbitMQ.CLI.Core.AcceptsOnePositionalArgument use RabbitMQ.CLI.Core.RequiresRabbitAppRunning @@ -17,6 +19,12 @@ defmodule RabbitMQ.CLI.Ctl.Commands.DeleteVhostCommand do :rabbit_misc.rpc_call(node_name, :rabbit_vhost, :delete, [arg, Helpers.cli_acting_user()]) end + def output({:error, :protected_from_deletion}, %{formatter: "json", node: node_name}) do + {:error, %{"result" => "error", "node" => node_name, "reason" => @protected_from_deletion_err}} + end + def output({:error, :protected_from_deletion}, _opts) do + {:error, ExitCodes.exit_dataerr(), @protected_from_deletion_err} + end use RabbitMQ.CLI.DefaultOutput def usage, do: "delete_vhost " diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/list_vhosts_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/list_vhosts_command.ex index 6ad98132c3e3..078bd1bc7cba 100644 --- a/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/list_vhosts_command.ex +++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/list_vhosts_command.ex @@ -11,7 +11,7 @@ defmodule RabbitMQ.CLI.Ctl.Commands.ListVhostsCommand do @behaviour RabbitMQ.CLI.CommandBehaviour use RabbitMQ.CLI.DefaultOutput - @info_keys ~w(name description tags default_queue_type tracing cluster_state)a + @info_keys ~w(name description tags default_queue_type protected_from_deletion tracing cluster_state metadata)a def info_keys(), do: @info_keys diff --git a/deps/rabbitmq_cli/test/ctl/update_vhost_metadata_command_test.exs b/deps/rabbitmq_cli/test/ctl/update_vhost_metadata_command_test.exs index dd44f709bb04..5b92ca499342 100644 --- a/deps/rabbitmq_cli/test/ctl/update_vhost_metadata_command_test.exs +++ b/deps/rabbitmq_cli/test/ctl/update_vhost_metadata_command_test.exs @@ -2,7 +2,7 @@ ## 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-2023 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +## Copyright (c) 2007-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. defmodule UpdateVhostMetadataCommandTest do use ExUnit.Case, async: false @@ -81,6 +81,24 @@ defmodule UpdateVhostMetadataCommandTest do assert vh[:tags] == [:a1, :b2, :c3] end + test "run: enabling and disabling deletion protection succeeds", context do + add_vhost(@vhost) + + opts = Map.merge(context[:opts], %{description: "Protected from deletion", protected_from_deletion: true}) + assert @command.run([@vhost], opts) == :ok + vh = find_vhost(@vhost) + assert vh[:tags] == [:my_tag] + end + + test "run: vhost tags are coerced to a list", context do + add_vhost(@vhost) + + opts = Map.merge(context[:opts], %{description: "My vhost", tags: "my_tag"}) + assert @command.run([@vhost], opts) == :ok + vh = find_vhost(@vhost) + assert vh[:tags] == [:my_tag] + end + test "run: attempt to use a non-existent virtual host fails", context do vh = "a-non-existent-3882-vhost" @@ -95,14 +113,6 @@ defmodule UpdateVhostMetadataCommandTest do assert match?({:badrpc, _}, @command.run(["na"], opts)) end - test "run: vhost tags are coerced to a list", context do - add_vhost(@vhost) - - opts = Map.merge(context[:opts], %{description: "My vhost", tags: "my_tag"}) - assert @command.run([@vhost], opts) == :ok - vh = find_vhost(@vhost) - assert vh[:tags] == [:my_tag] - end test "banner", context do assert @command.banner([@vhost], context[:opts]) =~ diff --git a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl index 6137fcf2fcbf..d68881ec1246 100644 --- a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl +++ b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl @@ -112,6 +112,8 @@ add_vhost/3, add_vhost/4, update_vhost_metadata/3, + enable_vhost_protection_from_deletion/2, + disable_vhost_protection_from_deletion/2, delete_vhost/2, delete_vhost/3, delete_vhost/4, @@ -1606,6 +1608,18 @@ add_vhost(Config, Node, VHost, Username) -> update_vhost_metadata(Config, VHost, Meta) -> catch rpc(Config, 0, rabbit_vhost, update_metadata, [VHost, Meta, <<"acting-user">>]). +enable_vhost_protection_from_deletion(Config, VHost) -> + MetadataPatch = #{ + protected_from_deletion => true + }, + update_vhost_metadata(Config, VHost, MetadataPatch). + +disable_vhost_protection_from_deletion(Config, VHost) -> + MetadataPatch = #{ + protected_from_deletion => false + }, + update_vhost_metadata(Config, VHost, MetadataPatch). + delete_vhost(Config, VHost) -> delete_vhost(Config, 0, VHost). diff --git a/deps/rabbitmq_ct_helpers/src/rabbit_mgmt_test_util.erl b/deps/rabbitmq_ct_helpers/src/rabbit_mgmt_test_util.erl index 2d454d5e85aa..c1fc1437fcd9 100644 --- a/deps/rabbitmq_ct_helpers/src/rabbit_mgmt_test_util.erl +++ b/deps/rabbitmq_ct_helpers/src/rabbit_mgmt_test_util.erl @@ -68,6 +68,10 @@ http_post(Config, Path, List, CodeExp) -> http_post(Config, Path, List, User, Pass, CodeExp) -> http_post_raw(Config, Path, format_for_upload(List), User, Pass, CodeExp). +http_post_json(Config, Path, Body, Assertion) -> + http_upload_raw(Config, post, Path, Body, "guest", "guest", + Assertion, [{"content-type", "application/json"}]). + http_post_accept_json(Config, Path, List, CodeExp) -> http_post_accept_json(Config, Path, List, "guest", "guest", CodeExp). diff --git a/deps/rabbitmq_management/priv/www/api/index.html b/deps/rabbitmq_management/priv/www/api/index.html index 14bcaeb36a22..d7e4a4a5214d 100644 --- a/deps/rabbitmq_management/priv/www/api/index.html +++ b/deps/rabbitmq_management/priv/www/api/index.html @@ -723,6 +723,14 @@

Reference

/api/vhosts/name/topic-permissions A list of all topic permissions for a given virtual host. + + + + X + X + /api/vhosts/name/deletion/protection + Enables (when used with POST) or disabled (with DELETE) deletion protection for the virtual host. + diff --git a/deps/rabbitmq_management/src/rabbit_mgmt_dispatcher.erl b/deps/rabbitmq_management/src/rabbit_mgmt_dispatcher.erl index ad6ef0414ae9..dd9d61ba5905 100644 --- a/deps/rabbitmq_management/src/rabbit_mgmt_dispatcher.erl +++ b/deps/rabbitmq_management/src/rabbit_mgmt_dispatcher.erl @@ -161,6 +161,7 @@ dispatcher() -> {"/bindings/:vhost/e/:source/:dtype/:destination/:props", rabbit_mgmt_wm_binding, []}, {"/vhosts", rabbit_mgmt_wm_vhosts, []}, {"/vhosts/:vhost", rabbit_mgmt_wm_vhost, []}, + {"/vhosts/:vhost/deletion/protection", rabbit_mgmt_wm_vhost_deletion_protection, []}, {"/vhosts/:vhost/start/:node", rabbit_mgmt_wm_vhost_restart, []}, {"/vhosts/:vhost/permissions", rabbit_mgmt_wm_permissions_vhost, []}, {"/vhosts/:vhost/topic-permissions", rabbit_mgmt_wm_topic_permissions_vhost, []}, diff --git a/deps/rabbitmq_management/src/rabbit_mgmt_util.erl b/deps/rabbitmq_management/src/rabbit_mgmt_util.erl index 5e737bcaf2ea..557ac0433835 100644 --- a/deps/rabbitmq_management/src/rabbit_mgmt_util.erl +++ b/deps/rabbitmq_management/src/rabbit_mgmt_util.erl @@ -19,7 +19,7 @@ is_authorized_global_parameters/2]). -export([user/1]). -export([bad_request/3, service_unavailable/3, bad_request_exception/4, - internal_server_error/3, internal_server_error/4, + internal_server_error/3, internal_server_error/4, precondition_failed/3, id/2, parse_bool/1, parse_int/1, redirect_to_home/3]). -export([with_decode/4, not_found/3]). -export([with_channel/4, with_channel/5]). @@ -675,6 +675,9 @@ not_found(Reason, ReqData, Context) -> method_not_allowed(Reason, ReqData, Context) -> halt_response(405, method_not_allowed, Reason, ReqData, Context). +precondition_failed(Reason, ReqData, Context) -> + halt_response(412, precondition_failed, Reason, ReqData, Context). + internal_server_error(Reason, ReqData, Context) -> internal_server_error(internal_server_error, Reason, ReqData, Context). diff --git a/deps/rabbitmq_management/src/rabbit_mgmt_wm_vhost.erl b/deps/rabbitmq_management/src/rabbit_mgmt_wm_vhost.erl index 3a2b98499c48..cec419c96af2 100644 --- a/deps/rabbitmq_management/src/rabbit_mgmt_wm_vhost.erl +++ b/deps/rabbitmq_management/src/rabbit_mgmt_wm_vhost.erl @@ -95,15 +95,20 @@ delete_resource(ReqData, Context = #context{user = #user{username = Username}}) case rabbit_vhost:delete(VHost, Username) of ok -> {true, ReqData, Context}; + {error, protected_from_deletion} -> + Msg = "Refusing to delete virtual host '~ts' because it is protected from deletion", + Reason = iolist_to_binary(io_lib:format(Msg, [VHost])), + rabbit_log:error(Msg, [VHost]), + rabbit_mgmt_util:precondition_failed(Reason, ReqData, Context); {error, timeout} -> rabbit_mgmt_util:internal_server_error( timeout, - "Timed out waiting for the vhost to be deleted", + "Timed out waiting for the virtual host to be deleted", ReqData, Context); {error, E} -> Reason = iolist_to_binary( io_lib:format( - "Error occurred while deleting vhost: ~tp", + "Error occurred while deleting virtual host '~tp'", [E])), rabbit_mgmt_util:internal_server_error( Reason, ReqData, Context) diff --git a/deps/rabbitmq_management/src/rabbit_mgmt_wm_vhost_deletion_protection.erl b/deps/rabbitmq_management/src/rabbit_mgmt_wm_vhost_deletion_protection.erl new file mode 100644 index 000000000000..c391f671ef72 --- /dev/null +++ b/deps/rabbitmq_management/src/rabbit_mgmt_wm_vhost_deletion_protection.erl @@ -0,0 +1,88 @@ +%% 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. +%% + +-module(rabbit_mgmt_wm_vhost_deletion_protection). + +-export([init/2, resource_exists/2, + content_types_accepted/2, + is_authorized/2, allowed_methods/2, accept_content/2, + delete_resource/2, id/1]). +-export([variances/2]). + +-import(rabbit_misc, [pget/2]). + +-include_lib("rabbitmq_management_agent/include/rabbit_mgmt_records.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +-dialyzer({nowarn_function, accept_content/2}). + +%%-------------------------------------------------------------------- + +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_accepted(ReqData, Context) -> + {[{'*', accept_content}], ReqData, Context}. + +allowed_methods(ReqData, Context) -> + {[<<"GET">>, <<"POST">>, <<"DELETE">>, <<"OPTIONS">>], ReqData, Context}. + +resource_exists(ReqData, Context) -> + {rabbit_db_vhost:exists(id(ReqData)), ReqData, Context}. + +accept_content(ReqData, Context) -> + Name = id(ReqData), + case rabbit_db_vhost:enable_protection_from_deletion(Name) of + {ok, _NewRecord} -> + {true, ReqData, Context}; + {error, {no_such_vhost, _}} -> + Msg = "Cannot enable deletion protection of virtual host '~ts' because it does not exist", + Reason = iolist_to_binary(io_lib:format(Msg, [Name])), + rabbit_log:error(Msg, [Name]), + rabbit_mgmt_util:not_found( + Reason, ReqData, Context); + {error, E} -> + Msg = "Cannot enable deletion protection of virtual host '~ts': ~tp", + Reason = iolist_to_binary(io_lib:format(Msg, [Name, E])), + rabbit_log:error(Msg, [Name]), + rabbit_mgmt_util:internal_server_error( + Reason, ReqData, Context) + end. + +delete_resource(ReqData, Context) -> + Name = id(ReqData), + case rabbit_db_vhost:disable_protection_from_deletion(Name) of + {ok, _NewRecord} -> + {true, ReqData, Context}; + {error, {no_such_vhost, _}} -> + Msg = "Cannot disable deletion protection of virtual host '~ts' because it does not exist", + Reason = iolist_to_binary(io_lib:format(Msg, [Name])), + rabbit_log:error(Msg, [Name]), + rabbit_mgmt_util:not_found( + Reason, ReqData, Context); + {error, E} -> + Msg = "Cannot disable deletion protection of virtual host '~ts': ~tp", + Reason = iolist_to_binary(io_lib:format(Msg, [Name, E])), + rabbit_log:error(Msg, [Name]), + rabbit_mgmt_util:internal_server_error( + Reason, ReqData, Context) + end. + +is_authorized(ReqData, Context) -> + rabbit_mgmt_util:is_authorized_admin(ReqData, Context). + +%%-------------------------------------------------------------------- + +id(ReqData) -> + case rabbit_mgmt_util:id(vhost, ReqData) of + [Value] -> Value; + Value -> Value + end. + diff --git a/deps/rabbitmq_management/test/clustering_prop_SUITE.erl b/deps/rabbitmq_management/test/clustering_prop_SUITE.erl index 56f9d5e80b66..2c335b6fd8b3 100644 --- a/deps/rabbitmq_management/test/clustering_prop_SUITE.erl +++ b/deps/rabbitmq_management/test/clustering_prop_SUITE.erl @@ -282,7 +282,7 @@ dump_table(Config, Table) -> Data0 = rabbit_ct_broker_helpers:rpc(Config, 1, ets, tab2list, [Table]), ct:pal(?LOW_IMPORTANCE, "Node 1: Dump of table ~tp:~n~tp~n", [Table, Data0]). -retry_for(Fun, 0) -> +retry_for(_Fun, 0) -> false; retry_for(Fun, Retries) -> case Fun() of diff --git a/deps/rabbitmq_management/test/rabbit_mgmt_http_SUITE.erl b/deps/rabbitmq_management/test/rabbit_mgmt_http_SUITE.erl index 00bac6dd2299..8926e0f9fd2e 100644 --- a/deps/rabbitmq_management/test/rabbit_mgmt_http_SUITE.erl +++ b/deps/rabbitmq_management/test/rabbit_mgmt_http_SUITE.erl @@ -27,6 +27,7 @@ http_get_no_decode/5, http_put/4, http_put/6, http_post/4, http_post/6, + http_post_json/4, http_upload_raw/8, http_delete/3, http_delete/4, http_delete/5, http_put_raw/4, http_post_accept_json/4, @@ -3463,7 +3464,7 @@ check_cors_all_endpoints(Config) -> Endpoints = get_all_http_endpoints(), [begin - ct:pal("Options for ~tp~n", [EP]), + ct:pal("Verifying CORS for module ~tp using an OPTIONS request~n", [EP]), {ok, {{_, 200, _}, _, _}} = req(Config, options, EP, [{"origin", "https://rabbitmq.com"}]) end || EP <- Endpoints]. @@ -4208,10 +4209,6 @@ publish(Ch) -> publish(Ch) end. -http_post_json(Config, Path, Body, Assertion) -> - http_upload_raw(Config, post, Path, Body, "guest", "guest", - Assertion, [{"content-type", "application/json"}]). - %% @doc encode fields and file for HTTP post multipart/form-data. %% @reference Inspired by Python implementation. format_multipart_filedata(Boundary, Files) -> diff --git a/deps/rabbitmq_management/test/rabbit_mgmt_http_health_checks_SUITE.erl b/deps/rabbitmq_management/test/rabbit_mgmt_http_health_checks_SUITE.erl index 4e12d7696cda..e891cf032b17 100644 --- a/deps/rabbitmq_management/test/rabbit_mgmt_http_health_checks_SUITE.erl +++ b/deps/rabbitmq_management/test/rabbit_mgmt_http_health_checks_SUITE.erl @@ -47,7 +47,7 @@ all_tests() -> [ ]. %% ------------------------------------------------------------------- -%% Testsuite setup/teardown. +%% Test suite setup/teardown %% ------------------------------------------------------------------- init_per_group(Group, Config0) -> @@ -90,7 +90,7 @@ end_per_testcase(Testcase, Config) -> rabbit_ct_helpers:testcase_finished(Config, Testcase). %% ------------------------------------------------------------------- -%% Testcases. +%% Test cases %% ------------------------------------------------------------------- health_checks_test(Config) -> diff --git a/deps/rabbitmq_management/test/rabbit_mgmt_http_vhost_deletion_protection_SUITE.erl b/deps/rabbitmq_management/test/rabbit_mgmt_http_vhost_deletion_protection_SUITE.erl new file mode 100644 index 000000000000..40597c4ad6f2 --- /dev/null +++ b/deps/rabbitmq_management/test/rabbit_mgmt_http_vhost_deletion_protection_SUITE.erl @@ -0,0 +1,134 @@ +%% 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. +%% + +-module(rabbit_mgmt_http_vhost_deletion_protection_SUITE). + +-include("rabbit_mgmt.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("rabbitmq_ct_helpers/include/rabbit_mgmt_test.hrl"). + +-import(rabbit_mgmt_test_util, [http_get/3, http_delete/3, http_post_json/4]). + +-compile(nowarn_export_all). +-compile(export_all). + +all() -> + [ + {group, cluster_size_3}, + {group, single_node} + ]. + +groups() -> + [ + {cluster_size_3, [], all_tests()}, + {single_node, [], all_tests()} + ]. + +all_tests() -> [ + protected_virtual_host_cannot_be_deleted, + virtual_host_can_be_deleted_after_protection_removal + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_group(Group, Config0) -> + rabbit_ct_helpers:log_environment(), + inets:start(), + ClusterSize = case Group of + cluster_size_3 -> 3; + cluster_size_5 -> 5; + single_node -> 1 + end, + NodeConf = [{rmq_nodename_suffix, Group}, + {rmq_nodes_count, ClusterSize}, + {tcp_ports_base}], + Config1 = rabbit_ct_helpers:set_config(Config0, NodeConf), + rabbit_ct_helpers:run_setup_steps( + Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_group(_, Config) -> + inets:stop(), + Teardown0 = rabbit_ct_client_helpers:teardown_steps(), + Teardown1 = rabbit_ct_broker_helpers:teardown_steps(), + Steps = Teardown0 ++ Teardown1, + rabbit_ct_helpers:run_teardown_steps(Config, Steps). + +init_per_testcase(Testcase, Config) -> + case rabbit_ct_helpers:is_mixed_versions() of + true -> + {skip, "not mixed versions compatible"}; + _ -> + rabbit_ct_helpers:testcase_started(Config, Testcase) + end. + +end_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% ------------------------------------------------------------------- +%% Test cases +%% ------------------------------------------------------------------- + +-define(DELETION_PROTECTION_KEY, protected_from_deletion). + +protected_virtual_host_cannot_be_deleted(Config) -> + VH = rabbit_data_coercion:to_binary(?FUNCTION_NAME), + + MetaWhenLocked = #{ + ?DELETION_PROTECTION_KEY => true, + tags => [VH, "locked"] + }, + + MetaWhenUnlocked = #{ + ?DELETION_PROTECTION_KEY => false, + tags => [VH] + }, + + %% extra care needs to be taken to a delete a potentially protected virtual host + rabbit_ct_broker_helpers:update_vhost_metadata(Config, VH, MetaWhenUnlocked), + rabbit_ct_broker_helpers:delete_vhost(Config, VH), + rabbit_ct_broker_helpers:add_vhost(Config, VH), + rabbit_ct_broker_helpers:set_full_permissions(Config, VH), + + %% protect the virtual host from deletion + rabbit_ct_broker_helpers:update_vhost_metadata(Config, VH, MetaWhenLocked), + + %% deletion fails with 412 Precondition Failed + Path = io_lib:format("/vhosts/~ts", [VH]), + http_delete(Config, Path, 412), + + rabbit_ct_broker_helpers:update_vhost_metadata(Config, VH, MetaWhenUnlocked), + http_delete(Config, Path, {group, '2xx'}), + + passed. + +virtual_host_can_be_deleted_after_protection_removal(Config) -> + VH = rabbit_data_coercion:to_binary(?FUNCTION_NAME), + + rabbit_ct_broker_helpers:disable_vhost_protection_from_deletion(Config, VH), + rabbit_ct_broker_helpers:delete_vhost(Config, VH), + rabbit_ct_broker_helpers:add_vhost(Config, VH), + rabbit_ct_broker_helpers:set_full_permissions(Config, VH), + + rabbit_ct_broker_helpers:enable_vhost_protection_from_deletion(Config, VH), + + %% deletion fails with 412 Precondition Failed + Path = io_lib:format("/vhosts/~ts", [VH]), + http_delete(Config, Path, 412), + + %% lift the protection + rabbit_ct_broker_helpers:disable_vhost_protection_from_deletion(Config, VH), + %% deletion succeeds + http_delete(Config, Path, {group, '2xx'}), + %% subsequent deletion responds with a 404 Not Found + http_delete(Config, Path, ?NOT_FOUND), + + passed. \ No newline at end of file