Skip to content

Commit d8fcfc6

Browse files
committed
Notify of permission deletions when deleting a vhost
1 parent 57f9aec commit d8fcfc6

File tree

5 files changed

+247
-11
lines changed

5 files changed

+247
-11
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_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: 46 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -170,6 +170,10 @@
170170

171171
-export([force_shrink_member_to_current_member/0]).
172172

173+
%% Helpers for working with the Khepri API / types.
174+
-export([collect_payloads/1,
175+
collect_payloads/2]).
176+
173177
-ifdef(TEST).
174178
-export([force_metadata_store/1,
175179
clear_forced_metadata_store/0]).
@@ -983,6 +987,48 @@ info() ->
983987
handle_async_ret(RaEvent) ->
984988
khepri:handle_async_ret(?STORE_ID, RaEvent).
985989

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

deps/rabbit/src/rabbit_vhost.erl

Lines changed: 1 addition & 2 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

deps/rabbit/test/vhost_SUITE.erl

Lines changed: 115 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@ groups() ->
3333
vhost_failure_forces_connection_closure,
3434
vhost_creation_idempotency,
3535
vhost_update_idempotency,
36+
vhost_deletion,
3637
parse_tags
3738
],
3839
ClusterSize2Tests = [
@@ -41,7 +42,8 @@ groups() ->
4142
vhost_failure_forces_connection_closure_on_failure_node,
4243
node_starts_with_dead_vhosts,
4344
node_starts_with_dead_vhosts_with_mirrors,
44-
vhost_creation_idempotency
45+
vhost_creation_idempotency,
46+
vhost_deletion
4547
],
4648
[
4749
{cluster_size_1_network, [], ClusterSize1Tests},
@@ -375,6 +377,118 @@ vhost_update_idempotency(Config) ->
375377
rabbit_ct_broker_helpers:delete_vhost(Config, VHost)
376378
end.
377379

380+
vhost_deletion(Config) ->
381+
VHost = <<"deletion-vhost">>,
382+
ActingUser = <<"acting-user">>,
383+
Node = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
384+
385+
set_up_vhost(Config, VHost),
386+
387+
Conn = rabbit_ct_client_helpers:open_unmanaged_connection(Config, 0, VHost),
388+
{ok, Chan} = amqp_connection:open_channel(Conn),
389+
390+
%% Declare some resources under the vhost. These should be deleted when the
391+
%% vhost is deleted.
392+
QName = <<"vhost-deletion-queue">>,
393+
#'queue.declare_ok'{} = amqp_channel:call(
394+
Chan, #'queue.declare'{queue = QName, durable = true}),
395+
XName = <<"vhost-deletion-exchange">>,
396+
#'exchange.declare_ok'{} = amqp_channel:call(
397+
Chan,
398+
#'exchange.declare'{exchange = XName,
399+
durable = true,
400+
type = <<"direct">>}),
401+
RoutingKey = QName,
402+
#'queue.bind_ok'{} = amqp_channel:call(
403+
Chan,
404+
#'queue.bind'{exchange = XName,
405+
queue = QName,
406+
routing_key = RoutingKey}),
407+
PolicyName = <<"ttl-policy">>,
408+
rabbit_ct_broker_helpers:set_policy_in_vhost(
409+
Config, Node, VHost,
410+
PolicyName, <<"policy_ttl-queue">>, <<"all">>, [{<<"message-ttl">>, 20}],
411+
ActingUser),
412+
413+
% Load the dummy event handler module on the node.
414+
ok = rabbit_ct_broker_helpers:rpc(Config, Node, test_rabbit_event_handler, okay, []),
415+
ok = rabbit_ct_broker_helpers:rpc(Config, Node, gen_event, add_handler,
416+
[rabbit_event, test_rabbit_event_handler, []]),
417+
try
418+
rabbit_ct_broker_helpers:delete_vhost(Config, VHost),
419+
420+
Events0 = rabbit_ct_broker_helpers:rpc(Config, Node,
421+
gen_event, call,
422+
[rabbit_event, test_rabbit_event_handler, events, 1000]),
423+
ct:pal(
424+
?LOW_IMPORTANCE,
425+
"Events emitted during deletion: ~p", [lists:reverse(Events0)]),
426+
427+
%% Reorganize the event props into maps for easier matching.
428+
Events = [{Type, maps:from_list(Props)} ||
429+
#event{type = Type, props = Props} <- Events0],
430+
431+
?assertMatch(#{user := <<"guest">>, vhost := VHost},
432+
proplists:get_value(permission_deleted, Events)),
433+
434+
?assertMatch(#{source_name := XName,
435+
source_kind := exchange,
436+
destination_name := QName,
437+
destination_kind := queue,
438+
routing_key := RoutingKey,
439+
vhost := VHost},
440+
proplists:get_value(binding_deleted, Events)),
441+
442+
?assertMatch(#{name := #resource{name = QName,
443+
kind = queue,
444+
virtual_host = VHost}},
445+
proplists:get_value(queue_deleted, Events)),
446+
447+
?assertEqual(
448+
lists:sort([<<>>, <<"amq.direct">>, <<"amq.fanout">>, <<"amq.headers">>,
449+
<<"amq.match">>, <<"amq.rabbitmq.trace">>, <<"amq.topic">>,
450+
<<"vhost-deletion-exchange">>]),
451+
lists:sort(lists:filtermap(
452+
fun ({exchange_deleted,
453+
#{name := #resource{name = Name}}}) ->
454+
{true, Name};
455+
(_Event) ->
456+
false
457+
end, Events))),
458+
459+
%% TODO: parameter_cleared and vhost_limits_cleared
460+
%% ?assertMatch(#{name := PolicyName, vhost := VHost},
461+
%% proplists:get_value(policy_cleared, Events)),
462+
463+
?assertMatch(#{name := VHost,
464+
user_who_performed_action := ActingUser},
465+
proplists:get_value(vhost_deleted, Events)),
466+
?assertMatch(#{name := VHost,
467+
node := Node,
468+
user_who_performed_action := ?INTERNAL_USER},
469+
proplists:get_value(vhost_down, Events)),
470+
471+
?assert(proplists:is_defined(channel_closed, Events)),
472+
?assert(proplists:is_defined(connection_closed, Events)),
473+
474+
%% VHost deletion is not idempotent - we return an error - but deleting
475+
%% the same vhost again should not cause any more resources to be
476+
%% deleted. So we should see no new events in the `rabbit_event'
477+
%% handler.
478+
?assertEqual(
479+
{error, {no_such_vhost, VHost}},
480+
rabbit_ct_broker_helpers:delete_vhost(Config, VHost)),
481+
?assertEqual(
482+
Events0,
483+
rabbit_ct_broker_helpers:rpc(
484+
Config, Node,
485+
gen_event, call,
486+
[rabbit_event, test_rabbit_event_handler, events, 1000]))
487+
after
488+
rabbit_ct_broker_helpers:rpc(Config, Node,
489+
gen_event, delete_handler, [rabbit_event, test_rabbit_event_handler, []])
490+
end.
491+
378492
vhost_is_created_with_default_limits(Config) ->
379493
VHost = <<"vhost1">>,
380494
Limits = [{<<"max-connections">>, 10}, {<<"max-queues">>, 1}],

0 commit comments

Comments
 (0)