Skip to content

Commit 78d9cc9

Browse files
Merge pull request #11106 from rabbitmq/mergify/bp/v3.13.x/pr-11077
Notify of permission and parameter deletions when deleting a vhost (backport #11077)
2 parents a902058 + 0ed0fb8 commit 78d9cc9

File tree

9 files changed

+365
-37
lines changed

9 files changed

+365
-37
lines changed

deps/rabbit/src/rabbit_auth_backend_internal.erl

Lines changed: 32 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,9 @@
1717
-export([add_user/3, add_user/4, add_user/5, delete_user/2, lookup_user/1, exists/1,
1818
change_password/3, clear_password/2,
1919
hash_password/2, change_password_hash/2, change_password_hash/3,
20-
set_tags/3, set_permissions/6, clear_permissions/3, clear_permissions_for_vhost/2, set_permissions_globally/5,
21-
set_topic_permissions/6, clear_topic_permissions/3, clear_topic_permissions/4, clear_topic_permissions_for_vhost/2,
20+
set_tags/3, set_permissions/6, clear_permissions/3, set_permissions_globally/5,
21+
set_topic_permissions/6, clear_topic_permissions/3, clear_topic_permissions/4,
22+
clear_all_permissions_for_vhost/2,
2223
add_user_sans_validation/3, put_user/2, put_user/3,
2324
update_user/5,
2425
update_user_with_hash/5,
@@ -540,8 +541,35 @@ clear_permissions(Username, VirtualHost, ActingUser) ->
540541
erlang:raise(Class, Error, Stacktrace)
541542
end.
542543

543-
clear_permissions_for_vhost(VirtualHost, _ActingUser) ->
544-
rabbit_db_user:clear_matching_user_permissions('_', VirtualHost).
544+
-spec clear_all_permissions_for_vhost(VirtualHost, ActingUser) -> Ret when
545+
VirtualHost :: rabbit_types:vhost(),
546+
ActingUser :: rabbit_types:username(),
547+
Ret :: ok | {error, Reason :: any()}.
548+
549+
clear_all_permissions_for_vhost(VirtualHost, ActingUser) ->
550+
case rabbit_db_user:clear_all_permissions_for_vhost(VirtualHost) of
551+
{ok, Deletions} ->
552+
lists:foreach(
553+
fun (#topic_permission{topic_permission_key =
554+
#topic_permission_key{user_vhost =
555+
#user_vhost{username = Username}}}) ->
556+
rabbit_event:notify(
557+
topic_permission_deleted,
558+
[{user, Username},
559+
{vhost, VirtualHost},
560+
{user_who_performed_action, ActingUser}]);
561+
(#user_permission{user_vhost =
562+
#user_vhost{username = Username}}) ->
563+
rabbit_event:notify(
564+
permission_deleted,
565+
[{user, Username},
566+
{vhost, VirtualHost},
567+
{user_who_performed_action, ActingUser}])
568+
end, Deletions),
569+
ok;
570+
{error, _} = Err ->
571+
Err
572+
end.
545573

546574
set_permissions_globally(Username, ConfigurePerm, WritePerm, ReadPerm, ActingUser) ->
547575
VirtualHosts = rabbit_vhost:list_names(),
@@ -642,9 +670,6 @@ clear_topic_permissions(Username, VirtualHost, Exchange, ActingUser) ->
642670
erlang:raise(Class, Error, Stacktrace)
643671
end.
644672

645-
clear_topic_permissions_for_vhost(VirtualHost, _ActingUser) ->
646-
rabbit_db_user:clear_matching_topic_permissions('_', VirtualHost, '_').
647-
648673
put_user(User, ActingUser) -> put_user(User, undefined, ActingUser).
649674

650675
put_user(User, Version, ActingUser) ->

deps/rabbit/src/rabbit_db_rtparams.erl

Lines changed: 51 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,8 @@
1414
get/1,
1515
get_or_set/2,
1616
get_all/0, get_all/2,
17-
delete/1, delete/3]).
17+
delete/1, delete/3,
18+
delete_vhost/1]).
1819

1920
-export([khepri_vhost_rp_path/3,
2021
khepri_global_rp_path/1,
@@ -340,6 +341,55 @@ delete_matching_in_khepri(VHostName, Comp, Name) ->
340341
Key = {?any(VHostName), ?any(Comp), ?any(Name)},
341342
delete_in_khepri(Key).
342343

344+
%% -------------------------------------------------------------------
345+
%% delete_vhost().
346+
%% -------------------------------------------------------------------
347+
348+
-spec delete_vhost(VHostName) -> Ret when
349+
VHostName :: vhost:name(),
350+
Ret :: {ok, Deletions} | {error, Reason :: any()},
351+
Deletions :: [#runtime_parameters{}].
352+
%% @doc Deletes all runtime parameters belonging to the given virtual host.
353+
%%
354+
%% @returns an OK tuple containing the deleted runtime parameters if
355+
%% successful, or an error tuple otherwise.
356+
%%
357+
%% @private
358+
359+
delete_vhost(VHostName) when is_binary(VHostName) ->
360+
rabbit_khepri:handle_fallback(
361+
#{mnesia => fun() -> delete_vhost_in_mnesia(VHostName) end,
362+
khepri => fun() -> delete_vhost_in_khepri(VHostName) end}).
363+
364+
delete_vhost_in_mnesia(VHostName) ->
365+
rabbit_mnesia:execute_mnesia_transaction(
366+
fun() ->
367+
Deletions = delete_vhost_in_mnesia_tx(VHostName),
368+
{ok, Deletions}
369+
end).
370+
371+
delete_vhost_in_mnesia_tx(VHostName) ->
372+
Match = #runtime_parameters{key = {VHostName, '_', '_'},
373+
_ = '_'},
374+
[begin
375+
mnesia:delete(?MNESIA_TABLE, Key, write),
376+
Record
377+
end
378+
|| #runtime_parameters{key = Key} = Record
379+
<- mnesia:match_object(?MNESIA_TABLE, Match, read)].
380+
381+
delete_vhost_in_khepri(VHostName) ->
382+
Path = khepri_vhost_rp_path(
383+
VHostName, ?KHEPRI_WILDCARD_STAR, ?KHEPRI_WILDCARD_STAR),
384+
case rabbit_khepri:adv_delete_many(Path) of
385+
{ok, Props} ->
386+
{ok, rabbit_khepri:collect_payloads(Props)};
387+
{error, _} = Err ->
388+
Err
389+
end.
390+
391+
%% -------------------------------------------------------------------
392+
343393
khepri_rp_path() ->
344394
[?MODULE].
345395

deps/rabbit/src/rabbit_db_user.erl

Lines changed: 53 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,8 @@
2828
set_topic_permissions/1,
2929
clear_topic_permissions/3,
3030
clear_matching_topic_permissions/3,
31-
delete/1]).
31+
delete/1,
32+
clear_all_permissions_for_vhost/1]).
3233

3334
-export([khepri_users_path/0,
3435
khepri_user_path/1,
@@ -548,6 +549,57 @@ clear_matching_user_permissions_in_khepri(Username, VHostName) ->
548549
any('_') -> ?KHEPRI_WILDCARD_STAR;
549550
any(Value) -> Value.
550551

552+
%% -------------------------------------------------------------------
553+
%% clear_all_permissions_for_vhost().
554+
%% -------------------------------------------------------------------
555+
556+
-spec clear_all_permissions_for_vhost(VHostName) -> Ret when
557+
VHostName :: vhost:name(),
558+
Ret :: {ok, DeletedPermissions} | {error, Reason :: any()},
559+
DeletedPermissions :: [#topic_permission{} | #user_permission{}].
560+
%% @doc Transactionally deletes all user and topic permissions for a virtual
561+
%% host, returning any permissions that were deleted.
562+
%%
563+
%% @returns an OK-tuple with the deleted permissions or an error tuple if the
564+
%% operation could not be completed.
565+
%%
566+
%% @private
567+
568+
clear_all_permissions_for_vhost(VHostName) when is_binary(VHostName) ->
569+
rabbit_khepri:handle_fallback(
570+
#{mnesia =>
571+
fun() -> clear_all_permissions_for_vhost_in_mnesia(VHostName) end,
572+
khepri =>
573+
fun() -> clear_all_permissions_for_vhost_in_khepri(VHostName) end}).
574+
575+
clear_all_permissions_for_vhost_in_mnesia(VHostName) ->
576+
rabbit_mnesia:execute_mnesia_transaction(
577+
fun() ->
578+
Deletions =
579+
clear_matching_topic_permissions_in_mnesia_tx(
580+
'_', VHostName, '_') ++
581+
clear_matching_user_permissions_in_mnesia_tx(
582+
'_', VHostName),
583+
{ok, Deletions}
584+
end).
585+
586+
clear_all_permissions_for_vhost_in_khepri(VHostName) ->
587+
rabbit_khepri:transaction(
588+
fun() ->
589+
UserPermissionsPath = khepri_user_permission_path(
590+
?KHEPRI_WILDCARD_STAR, VHostName),
591+
TopicPermissionsPath = khepri_topic_permission_path(
592+
?KHEPRI_WILDCARD_STAR, VHostName,
593+
?KHEPRI_WILDCARD_STAR),
594+
{ok, UserProps} = khepri_tx_adv:delete_many(UserPermissionsPath),
595+
{ok, TopicProps} = khepri_tx_adv:delete_many(
596+
TopicPermissionsPath),
597+
Deletions = rabbit_khepri:collect_payloads(
598+
TopicProps,
599+
rabbit_khepri:collect_payloads(UserProps)),
600+
{ok, Deletions}
601+
end, rw).
602+
551603
%% -------------------------------------------------------------------
552604
%% get_topic_permissions().
553605
%% -------------------------------------------------------------------

deps/rabbit/src/rabbit_khepri.erl

Lines changed: 50 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -133,6 +133,7 @@
133133
clear_payload/1,
134134
delete/1, delete/2,
135135
delete_or_fail/1,
136+
adv_delete_many/1,
136137

137138
transaction/1,
138139
transaction/2,
@@ -170,6 +171,10 @@
170171

171172
-export([force_shrink_member_to_current_member/0]).
172173

174+
%% Helpers for working with the Khepri API / types.
175+
-export([collect_payloads/1,
176+
collect_payloads/2]).
177+
173178
-ifdef(TEST).
174179
-export([force_metadata_store/1,
175180
clear_forced_metadata_store/0]).
@@ -946,6 +951,9 @@ delete_or_fail(Path) ->
946951
Error
947952
end.
948953

954+
adv_delete_many(Path) ->
955+
khepri_adv:delete_many(?STORE_ID, Path, ?DEFAULT_COMMAND_OPTIONS).
956+
949957
put(PathPattern, Data) ->
950958
khepri:put(
951959
?STORE_ID, PathPattern, Data, ?DEFAULT_COMMAND_OPTIONS).
@@ -983,6 +991,48 @@ info() ->
983991
handle_async_ret(RaEvent) ->
984992
khepri:handle_async_ret(?STORE_ID, RaEvent).
985993

994+
%% -------------------------------------------------------------------
995+
%% collect_payloads().
996+
%% -------------------------------------------------------------------
997+
998+
-spec collect_payloads(Props) -> Ret when
999+
Props :: khepri:node_props(),
1000+
Ret :: [Payload],
1001+
Payload :: term().
1002+
1003+
%% @doc Collects all payloads from a node props map.
1004+
%%
1005+
%% This is the same as calling `collect_payloads(Props, [])'.
1006+
%%
1007+
%% @private
1008+
1009+
collect_payloads(Props) when is_map(Props) ->
1010+
collect_payloads(Props, []).
1011+
1012+
-spec collect_payloads(Props, Acc0) -> Ret when
1013+
Props :: khepri:node_props(),
1014+
Acc0 :: [Payload],
1015+
Ret :: [Payload],
1016+
Payload :: term().
1017+
1018+
%% @doc Collects all payloads from a node props map into the accumulator list.
1019+
%%
1020+
%% This is meant to be used with the `khepri_adv' API to easily collect the
1021+
%% payloads from the return value of `khepri_adv:delete_many/4' for example.
1022+
%%
1023+
%% @returns all payloads in the node props map collected into a list, with
1024+
%% `Acc0' as the tail.
1025+
%%
1026+
%% @private
1027+
1028+
collect_payloads(Props, Acc0) when is_map(Props) andalso is_list(Acc0) ->
1029+
maps:fold(
1030+
fun (_Path, #{data := Payload}, Acc) ->
1031+
[Payload | Acc];
1032+
(_Path, _NoPayload, Acc) ->
1033+
Acc
1034+
end, Acc0, Props).
1035+
9861036
%% -------------------------------------------------------------------
9871037
%% if_has_data_wildcard().
9881038
%% -------------------------------------------------------------------

deps/rabbit/src/rabbit_runtime_parameters.erl

Lines changed: 22 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -203,8 +203,28 @@ clear_global(Key, ActingUser) ->
203203
{user_who_performed_action, ActingUser}])
204204
end.
205205

206-
clear_vhost(VHostName, _ActingUser) when is_binary(VHostName) ->
207-
ok = rabbit_db_rtparams:delete(VHostName, '_', '_').
206+
clear_vhost(VHostName, ActingUser) when is_binary(VHostName) ->
207+
case rabbit_db_rtparams:delete_vhost(VHostName) of
208+
{ok, DeletedParams} ->
209+
lists:foreach(
210+
fun(#runtime_parameters{key = {_VHost, Component, Name}}) ->
211+
case lookup_component(Component) of
212+
{ok, Mod} ->
213+
event_notify(
214+
parameter_cleared, VHostName, Component,
215+
[{name, Name},
216+
{user_who_performed_action, ActingUser}]),
217+
Mod:notify_clear(
218+
VHostName, Component, Name, ActingUser),
219+
ok;
220+
_ ->
221+
ok
222+
end
223+
end, DeletedParams),
224+
ok;
225+
{error, _} = Err ->
226+
Err
227+
end.
208228

209229
clear_component(<<"policy">>, _) ->
210230
{error_string, "policies may not be cleared using this method"};

deps/rabbit/src/rabbit_vhost.erl

Lines changed: 11 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -272,8 +272,7 @@ delete(VHost, ActingUser) ->
272272
%% calls would be responsible for the atomicity, not this code.
273273
%% Clear the permissions first to prohibit new incoming connections when deleting a vhost
274274
rabbit_log:info("Clearing permissions in vhost '~ts' because it's being deleted", [VHost]),
275-
_ = rabbit_auth_backend_internal:clear_permissions_for_vhost(VHost, ActingUser),
276-
_ = rabbit_auth_backend_internal:clear_topic_permissions_for_vhost(VHost, ActingUser),
275+
ok = rabbit_auth_backend_internal:clear_all_permissions_for_vhost(VHost, ActingUser),
277276
rabbit_log:info("Deleting queues in vhost '~ts' because it's being deleted", [VHost]),
278277
QDelFun = fun (Q) -> rabbit_amqqueue:delete(Q, false, false, ActingUser) end,
279278
[begin
@@ -285,22 +284,20 @@ delete(VHost, ActingUser) ->
285284
#exchange{name = Name} <- rabbit_exchange:list(VHost)],
286285
rabbit_log:info("Clearing policies and runtime parameters in vhost '~ts' because it's being deleted", [VHost]),
287286
_ = rabbit_runtime_parameters:clear_vhost(VHost, ActingUser),
288-
_ = [rabbit_policy:delete(VHost, proplists:get_value(name, Info), ActingUser)
289-
|| Info <- rabbit_policy:list(VHost)],
290287
rabbit_log:debug("Removing vhost '~ts' from the metadata storage because it's being deleted", [VHost]),
291-
case rabbit_db_vhost:delete(VHost) of
292-
true ->
293-
ok = rabbit_event:notify(
294-
vhost_deleted,
295-
[{name, VHost},
296-
{user_who_performed_action, ActingUser}]);
297-
false ->
298-
ok
299-
end,
288+
Ret = case rabbit_db_vhost:delete(VHost) of
289+
true ->
290+
ok = rabbit_event:notify(
291+
vhost_deleted,
292+
[{name, VHost},
293+
{user_who_performed_action, ActingUser}]);
294+
false ->
295+
{error, {no_such_vhost, VHost}}
296+
end,
300297
%% After vhost was deleted from the database, we try to stop vhost
301298
%% supervisors on all the nodes.
302299
rabbit_vhost_sup_sup:delete_on_all_nodes(VHost),
303-
ok.
300+
Ret.
304301

305302
-spec put_vhost(vhost:name(),
306303
binary(),

deps/rabbit/src/rabbit_vhost_limit.erl

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -39,12 +39,14 @@ validate(_VHost, <<"vhost-limits">>, Name, Term, _User) ->
3939

4040
notify(VHost, <<"vhost-limits">>, <<"limits">>, Limits, ActingUser) ->
4141
rabbit_event:notify(vhost_limits_set, [{name, <<"limits">>},
42+
{vhost, VHost},
4243
{user_who_performed_action, ActingUser}
4344
| Limits]),
4445
update_vhost(VHost, Limits).
4546

4647
notify_clear(VHost, <<"vhost-limits">>, <<"limits">>, ActingUser) ->
4748
rabbit_event:notify(vhost_limits_cleared, [{name, <<"limits">>},
49+
{vhost, VHost},
4850
{user_who_performed_action, ActingUser}]),
4951
%% If the function is called as a part of vhost deletion, the vhost can
5052
%% be already deleted.

0 commit comments

Comments
 (0)