Notify user who performed the action in all system events
This commit is contained in:
parent
7b1f550c17
commit
7e3aae8c1a
|
|
@ -821,17 +821,20 @@ insert_default_data() ->
|
|||
DefaultWritePermBin = rabbit_data_coercion:to_binary(DefaultWritePerm),
|
||||
DefaultReadPermBin = rabbit_data_coercion:to_binary(DefaultReadPerm),
|
||||
|
||||
ok = rabbit_vhost:add(DefaultVHostBin),
|
||||
ok = rabbit_vhost:add(DefaultVHostBin, ?INTERNAL_USER),
|
||||
ok = rabbit_auth_backend_internal:add_user(
|
||||
DefaultUserBin,
|
||||
DefaultPassBin
|
||||
DefaultPassBin,
|
||||
?INTERNAL_USER
|
||||
),
|
||||
ok = rabbit_auth_backend_internal:set_tags(DefaultUserBin,DefaultTags),
|
||||
ok = rabbit_auth_backend_internal:set_tags(DefaultUserBin, DefaultTags,
|
||||
?INTERNAL_USER),
|
||||
ok = rabbit_auth_backend_internal:set_permissions(DefaultUserBin,
|
||||
DefaultVHostBin,
|
||||
DefaultConfigurePermBin,
|
||||
DefaultWritePermBin,
|
||||
DefaultReadPermBin),
|
||||
DefaultReadPermBin,
|
||||
?INTERNAL_USER),
|
||||
ok.
|
||||
|
||||
%%---------------------------------------------------------------------------
|
||||
|
|
|
|||
|
|
@ -131,7 +131,8 @@
|
|||
auto_delete,
|
||||
arguments,
|
||||
owner_pid,
|
||||
exclusive
|
||||
exclusive,
|
||||
user_who_performed_action
|
||||
]).
|
||||
|
||||
-define(INFO_KEYS, [pid | ?CREATION_EVENT_KEYS ++ ?STATISTICS_KEYS -- [name]]).
|
||||
|
|
@ -286,7 +287,9 @@ terminate(_Reason, State = #q{q = Q}) ->
|
|||
|
||||
terminate_delete(EmitStats, Reason,
|
||||
State = #q{q = #amqqueue{name = QName},
|
||||
backing_queue = BQ}) ->
|
||||
backing_queue = BQ,
|
||||
status = Status}) ->
|
||||
ActingUser = terminated_by(Status),
|
||||
fun (BQS) ->
|
||||
BQS1 = BQ:delete_and_terminate(Reason, BQS),
|
||||
if EmitStats -> rabbit_event:if_enabled(State, #q.stats_timer,
|
||||
|
|
@ -294,11 +297,17 @@ terminate_delete(EmitStats, Reason,
|
|||
true -> ok
|
||||
end,
|
||||
%% don't care if the internal delete doesn't return 'ok'.
|
||||
rabbit_amqqueue:internal_delete(QName),
|
||||
rabbit_amqqueue:internal_delete(QName, ActingUser),
|
||||
BQS1
|
||||
end.
|
||||
|
||||
terminate_shutdown(Fun, State) ->
|
||||
terminated_by({terminated_by, ActingUser}) ->
|
||||
ActingUser;
|
||||
terminated_by(_) ->
|
||||
?INTERNAL_USER.
|
||||
|
||||
terminate_shutdown(Fun, #q{status = Status} = State) ->
|
||||
ActingUser = terminated_by(Status),
|
||||
State1 = #q{backing_queue_state = BQS, consumers = Consumers} =
|
||||
lists:foldl(fun (F, S) -> F(S) end, State,
|
||||
[fun stop_sync_timer/1,
|
||||
|
|
@ -310,7 +319,7 @@ terminate_shutdown(Fun, State) ->
|
|||
_ -> ok = rabbit_memory_monitor:deregister(self()),
|
||||
QName = qname(State),
|
||||
notify_decorators(shutdown, State),
|
||||
[emit_consumer_deleted(Ch, CTag, QName) ||
|
||||
[emit_consumer_deleted(Ch, CTag, QName, ActingUser) ||
|
||||
{Ch, CTag, _, _, _} <-
|
||||
rabbit_queue_consumers:all(Consumers)],
|
||||
State1#q{backing_queue_state = Fun(BQS)}
|
||||
|
|
@ -731,7 +740,7 @@ handle_ch_down(DownPid, State = #q{consumers = Consumers,
|
|||
{ok, State1};
|
||||
{ChAckTags, ChCTags, Consumers1} ->
|
||||
QName = qname(State1),
|
||||
[emit_consumer_deleted(DownPid, CTag, QName) || CTag <- ChCTags],
|
||||
[emit_consumer_deleted(DownPid, CTag, QName, ?INTERNAL_USER) || CTag <- ChCTags],
|
||||
Holder1 = case Holder of
|
||||
{DownPid, _} -> none;
|
||||
Other -> Other
|
||||
|
|
@ -953,6 +962,8 @@ i(garbage_collection, _State) ->
|
|||
i(reductions, _State) ->
|
||||
{reductions, Reductions} = erlang:process_info(self(), reductions),
|
||||
Reductions;
|
||||
i(user_who_performed_action, #q{q = #amqqueue{options = Opts}}) ->
|
||||
maps:get(user, Opts);
|
||||
i(Item, #q{backing_queue_state = BQS, backing_queue = BQ}) ->
|
||||
BQ:info(Item, BQS).
|
||||
|
||||
|
|
@ -970,7 +981,7 @@ emit_stats(State, Extra) ->
|
|||
rabbit_event:notify(queue_stats, Extra ++ All).
|
||||
|
||||
emit_consumer_created(ChPid, CTag, Exclusive, AckRequired, QName,
|
||||
PrefetchCount, Args, Ref) ->
|
||||
PrefetchCount, Args, Ref, ActingUser) ->
|
||||
rabbit_event:notify(consumer_created,
|
||||
[{consumer_tag, CTag},
|
||||
{exclusive, Exclusive},
|
||||
|
|
@ -978,15 +989,17 @@ emit_consumer_created(ChPid, CTag, Exclusive, AckRequired, QName,
|
|||
{channel, ChPid},
|
||||
{queue, QName},
|
||||
{prefetch_count, PrefetchCount},
|
||||
{arguments, Args}],
|
||||
{arguments, Args},
|
||||
{user_who_performed_action, ActingUser}],
|
||||
Ref).
|
||||
|
||||
emit_consumer_deleted(ChPid, ConsumerTag, QName) ->
|
||||
emit_consumer_deleted(ChPid, ConsumerTag, QName, ActingUser) ->
|
||||
rabbit_core_metrics:consumer_deleted(ChPid, ConsumerTag, QName),
|
||||
rabbit_event:notify(consumer_deleted,
|
||||
[{consumer_tag, ConsumerTag},
|
||||
{channel, ChPid},
|
||||
{queue, QName}]).
|
||||
{queue, QName},
|
||||
{user_who_performed_action, ActingUser}]).
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
|
|
@ -996,7 +1009,7 @@ prioritise_call(Msg, _From, _Len, State) ->
|
|||
{info, _Items} -> 9;
|
||||
consumers -> 9;
|
||||
stat -> 7;
|
||||
{basic_consume, _, _, _, _, _, _, _, _, _} -> consumer_bias(State);
|
||||
{basic_consume, _, _, _, _, _, _, _, _, _, _} -> consumer_bias(State);
|
||||
{basic_cancel, _, _, _} -> consumer_bias(State);
|
||||
_ -> 0
|
||||
end.
|
||||
|
|
@ -1096,7 +1109,7 @@ handle_call({basic_get, ChPid, NoAck, LimiterPid}, _From,
|
|||
end;
|
||||
|
||||
handle_call({basic_consume, NoAck, ChPid, LimiterPid, LimiterActive,
|
||||
PrefetchCount, ConsumerTag, ExclusiveConsume, Args, OkMsg},
|
||||
PrefetchCount, ConsumerTag, ExclusiveConsume, Args, OkMsg, ActingUser},
|
||||
_From, State = #q{consumers = Consumers,
|
||||
exclusive_consumer = Holder}) ->
|
||||
case check_exclusive_access(Holder, ExclusiveConsume, State) of
|
||||
|
|
@ -1105,7 +1118,7 @@ handle_call({basic_consume, NoAck, ChPid, LimiterPid, LimiterActive,
|
|||
ChPid, ConsumerTag, NoAck,
|
||||
LimiterPid, LimiterActive,
|
||||
PrefetchCount, Args, is_empty(State),
|
||||
Consumers),
|
||||
ActingUser, Consumers),
|
||||
ExclusiveConsumer =
|
||||
if ExclusiveConsume -> {ChPid, ConsumerTag};
|
||||
true -> Holder
|
||||
|
|
@ -1121,12 +1134,12 @@ handle_call({basic_consume, NoAck, ChPid, LimiterPid, LimiterActive,
|
|||
PrefetchCount, Args),
|
||||
emit_consumer_created(ChPid, ConsumerTag, ExclusiveConsume,
|
||||
AckRequired, QName, PrefetchCount,
|
||||
Args, none),
|
||||
Args, none, ActingUser),
|
||||
notify_decorators(State1),
|
||||
reply(ok, run_message_queue(State1))
|
||||
end;
|
||||
|
||||
handle_call({basic_cancel, ChPid, ConsumerTag, OkMsg}, _From,
|
||||
handle_call({basic_cancel, ChPid, ConsumerTag, OkMsg, ActingUser}, _From,
|
||||
State = #q{consumers = Consumers,
|
||||
exclusive_consumer = Holder}) ->
|
||||
ok = maybe_send_reply(ChPid, OkMsg),
|
||||
|
|
@ -1140,7 +1153,7 @@ handle_call({basic_cancel, ChPid, ConsumerTag, OkMsg}, _From,
|
|||
end,
|
||||
State1 = State#q{consumers = Consumers1,
|
||||
exclusive_consumer = Holder1},
|
||||
emit_consumer_deleted(ChPid, ConsumerTag, qname(State1)),
|
||||
emit_consumer_deleted(ChPid, ConsumerTag, qname(State1), ActingUser),
|
||||
notify_decorators(State1),
|
||||
case should_auto_delete(State1) of
|
||||
false -> reply(ok, ensure_expiry_timer(State1));
|
||||
|
|
@ -1159,14 +1172,15 @@ handle_call(stat, _From, State) ->
|
|||
ensure_expiry_timer(State),
|
||||
reply({ok, BQ:len(BQS), rabbit_queue_consumers:count()}, State1);
|
||||
|
||||
handle_call({delete, IfUnused, IfEmpty}, _From,
|
||||
handle_call({delete, IfUnused, IfEmpty, ActingUser}, _From,
|
||||
State = #q{backing_queue_state = BQS, backing_queue = BQ}) ->
|
||||
IsEmpty = BQ:is_empty(BQS),
|
||||
IsUnused = is_unused(State),
|
||||
if
|
||||
IfEmpty and not(IsEmpty) -> reply({error, not_empty}, State);
|
||||
IfUnused and not(IsUnused) -> reply({error, in_use}, State);
|
||||
true -> stop({ok, BQ:len(BQS)}, State)
|
||||
true -> stop({ok, BQ:len(BQS)},
|
||||
State#q{status = {terminated_by, ActingUser}})
|
||||
end;
|
||||
|
||||
handle_call(purge, _From, State = #q{backing_queue = BQ,
|
||||
|
|
@ -1321,14 +1335,16 @@ handle_cast({force_event_refresh, Ref},
|
|||
QName = qname(State),
|
||||
AllConsumers = rabbit_queue_consumers:all(Consumers),
|
||||
case Exclusive of
|
||||
none -> [emit_consumer_created(
|
||||
Ch, CTag, false, AckRequired, QName, Prefetch,
|
||||
Args, Ref) ||
|
||||
{Ch, CTag, AckRequired, Prefetch, Args}
|
||||
<- AllConsumers];
|
||||
{Ch, CTag} -> [{Ch, CTag, AckRequired, Prefetch, Args}] = AllConsumers,
|
||||
emit_consumer_created(
|
||||
Ch, CTag, true, AckRequired, QName, Prefetch, Args, Ref)
|
||||
none ->
|
||||
[emit_consumer_created(
|
||||
Ch, CTag, false, AckRequired, QName, Prefetch,
|
||||
Args, Ref, ActingUser) ||
|
||||
{Ch, CTag, AckRequired, Prefetch, Args, ActingUser}
|
||||
<- AllConsumers];
|
||||
{Ch, CTag} ->
|
||||
[{Ch, CTag, AckRequired, Prefetch, Args, ActingUser}] = AllConsumers,
|
||||
emit_consumer_created(
|
||||
Ch, CTag, true, AckRequired, QName, Prefetch, Args, Ref, ActingUser)
|
||||
end,
|
||||
noreply(rabbit_event:init_stats_timer(State, #q.stats_timer));
|
||||
|
||||
|
|
|
|||
|
|
@ -17,11 +17,11 @@
|
|||
-module(rabbit_binding).
|
||||
-include("rabbit.hrl").
|
||||
|
||||
-export([recover/2, exists/1, add/1, add/2, remove/1, remove/2, list/1]).
|
||||
-export([recover/2, exists/1, add/2, add/3, remove/1, remove/3, list/1]).
|
||||
-export([list_for_source/1, list_for_destination/1,
|
||||
list_for_source_and_destination/2]).
|
||||
-export([new_deletions/0, combine_deletions/2, add_deletion/3,
|
||||
process_deletions/1]).
|
||||
process_deletions/2]).
|
||||
-export([info_keys/0, info/1, info/2, info_all/1, info_all/2, info_all/4]).
|
||||
%% these must all be run inside a mnesia tx
|
||||
-export([has_for_source/1, remove_for_source/1,
|
||||
|
|
@ -57,10 +57,10 @@
|
|||
-spec recover([rabbit_exchange:name()], [rabbit_amqqueue:name()]) ->
|
||||
'ok'.
|
||||
-spec exists(rabbit_types:binding()) -> boolean() | bind_errors().
|
||||
-spec add(rabbit_types:binding()) -> bind_res().
|
||||
-spec add(rabbit_types:binding(), inner_fun()) -> bind_res().
|
||||
-spec add(rabbit_types:binding(), rabbit_types:username()) -> bind_res().
|
||||
-spec add(rabbit_types:binding(), inner_fun(), rabbit_types:username()) -> bind_res().
|
||||
-spec remove(rabbit_types:binding()) -> bind_res().
|
||||
-spec remove(rabbit_types:binding(), inner_fun()) -> bind_res().
|
||||
-spec remove(rabbit_types:binding(), inner_fun(), rabbit_types:username()) -> bind_res().
|
||||
-spec list(rabbit_types:vhost()) -> bindings().
|
||||
-spec list_for_source
|
||||
(rabbit_types:binding_source()) -> bindings().
|
||||
|
|
@ -84,7 +84,7 @@
|
|||
(rabbit_types:binding_destination(), boolean()) -> deletions().
|
||||
-spec remove_transient_for_destination
|
||||
(rabbit_types:binding_destination()) -> deletions().
|
||||
-spec process_deletions(deletions()) -> rabbit_misc:thunk('ok').
|
||||
-spec process_deletions(deletions(), rabbit_types:username()) -> rabbit_misc:thunk('ok').
|
||||
-spec combine_deletions(deletions(), deletions()) -> deletions().
|
||||
-spec add_deletion
|
||||
(rabbit_exchange:name(),
|
||||
|
|
@ -158,9 +158,9 @@ exists(Binding) ->
|
|||
rabbit_misc:const(mnesia:read({rabbit_route, B}) /= [])
|
||||
end, fun not_found_or_absent_errs/1).
|
||||
|
||||
add(Binding) -> add(Binding, fun (_Src, _Dst) -> ok end).
|
||||
add(Binding, ActingUser) -> add(Binding, fun (_Src, _Dst) -> ok end, ActingUser).
|
||||
|
||||
add(Binding, InnerFun) ->
|
||||
add(Binding, InnerFun, ActingUser) ->
|
||||
binding_action(
|
||||
Binding,
|
||||
fun (Src, Dst, B) ->
|
||||
|
|
@ -172,7 +172,7 @@ add(Binding, InnerFun) ->
|
|||
case InnerFun(Src, Dst) of
|
||||
ok ->
|
||||
case mnesia:read({rabbit_route, B}) of
|
||||
[] -> add(Src, Dst, B);
|
||||
[] -> add(Src, Dst, B, ActingUser);
|
||||
[_] -> fun () -> ok end
|
||||
end;
|
||||
{error, _} = Err ->
|
||||
|
|
@ -183,7 +183,7 @@ add(Binding, InnerFun) ->
|
|||
end
|
||||
end, fun not_found_or_absent_errs/1).
|
||||
|
||||
add(Src, Dst, B) ->
|
||||
add(Src, Dst, B, ActingUser) ->
|
||||
[SrcDurable, DstDurable] = [durable(E) || E <- [Src, Dst]],
|
||||
case (SrcDurable andalso DstDurable andalso
|
||||
mnesia:read({rabbit_durable_route, B}) =/= []) of
|
||||
|
|
@ -193,14 +193,16 @@ add(Src, Dst, B) ->
|
|||
Serial = rabbit_exchange:serial(Src),
|
||||
fun () ->
|
||||
x_callback(Serial, Src, add_binding, B),
|
||||
ok = rabbit_event:notify(binding_created, info(B))
|
||||
ok = rabbit_event:notify(
|
||||
binding_created,
|
||||
info(B) ++ [{user_who_performed_action, ActingUser}])
|
||||
end;
|
||||
true -> rabbit_misc:const({error, binding_not_found})
|
||||
end.
|
||||
|
||||
remove(Binding) -> remove(Binding, fun (_Src, _Dst) -> ok end).
|
||||
remove(Binding) -> remove(Binding, fun (_Src, _Dst) -> ok end, ?INTERNAL_USER).
|
||||
|
||||
remove(Binding, InnerFun) ->
|
||||
remove(Binding, InnerFun, ActingUser) ->
|
||||
binding_action(
|
||||
Binding,
|
||||
fun (Src, Dst, B) ->
|
||||
|
|
@ -210,18 +212,18 @@ remove(Binding, InnerFun) ->
|
|||
_ -> rabbit_misc:const({error, binding_not_found})
|
||||
end;
|
||||
_ -> case InnerFun(Src, Dst) of
|
||||
ok -> remove(Src, Dst, B);
|
||||
ok -> remove(Src, Dst, B, ActingUser);
|
||||
{error, _} = Err -> rabbit_misc:const(Err)
|
||||
end
|
||||
end
|
||||
end, fun absent_errs_only/1).
|
||||
|
||||
remove(Src, Dst, B) ->
|
||||
remove(Src, Dst, B, ActingUser) ->
|
||||
ok = sync_route(#route{binding = B}, durable(Src), durable(Dst),
|
||||
fun mnesia:delete_object/3),
|
||||
Deletions = maybe_auto_delete(
|
||||
B#binding.source, [B], new_deletions(), false),
|
||||
process_deletions(Deletions).
|
||||
process_deletions(Deletions, ActingUser).
|
||||
|
||||
list(VHostPath) ->
|
||||
VHostResource = rabbit_misc:r(VHostPath, '_'),
|
||||
|
|
@ -539,7 +541,7 @@ merge_entry({X1, Deleted1, Bindings1}, {X2, Deleted2, Bindings2}) ->
|
|||
anything_but(not_deleted, Deleted1, Deleted2),
|
||||
[Bindings1 | Bindings2]}.
|
||||
|
||||
process_deletions(Deletions) ->
|
||||
process_deletions(Deletions, ActingUser) ->
|
||||
AugmentedDeletions =
|
||||
dict:map(fun (_XName, {X, deleted, Bindings}) ->
|
||||
Bs = lists:flatten(Bindings),
|
||||
|
|
@ -553,16 +555,21 @@ process_deletions(Deletions) ->
|
|||
fun() ->
|
||||
dict:fold(fun (XName, {X, deleted, Bs, Serial}, ok) ->
|
||||
ok = rabbit_event:notify(
|
||||
exchange_deleted, [{name, XName}]),
|
||||
del_notify(Bs),
|
||||
exchange_deleted,
|
||||
[{name, XName},
|
||||
{user_who_performed_action, ActingUser}]),
|
||||
del_notify(Bs, ActingUser),
|
||||
x_callback(Serial, X, delete, Bs);
|
||||
(_XName, {X, not_deleted, Bs, Serial}, ok) ->
|
||||
del_notify(Bs),
|
||||
del_notify(Bs, ActingUser),
|
||||
x_callback(Serial, X, remove_bindings, Bs)
|
||||
end, ok, AugmentedDeletions)
|
||||
end.
|
||||
|
||||
del_notify(Bs) -> [rabbit_event:notify(binding_deleted, info(B)) || B <- Bs].
|
||||
del_notify(Bs, ActingUser) -> [rabbit_event:notify(
|
||||
binding_deleted,
|
||||
info(B) ++ [{user_who_performed_action, ActingUser}])
|
||||
|| B <- Bs].
|
||||
|
||||
x_callback(Serial, X, F, Bs) ->
|
||||
ok = rabbit_exchange:callback(X, F, Serial, [X, Bs]).
|
||||
|
|
|
|||
|
|
@ -323,6 +323,7 @@ tracked_connection_from_connection_state(#connection{
|
|||
{node, node()},
|
||||
{vhost, VHost},
|
||||
{user, Username},
|
||||
{user_who_performed_action, Username},
|
||||
{connected_at, Ts},
|
||||
{pid, self()},
|
||||
{type, network},
|
||||
|
|
|
|||
|
|
@ -58,7 +58,7 @@ stop() ->
|
|||
init([DefaultVHost]) ->
|
||||
#exchange{} = rabbit_exchange:declare(
|
||||
rabbit_misc:r(DefaultVHost, exchange, ?LOG_EXCH_NAME),
|
||||
topic, true, false, true, []),
|
||||
topic, true, false, true, [], ?INTERNAL_USER),
|
||||
{ok, #resource{virtual_host = DefaultVHost,
|
||||
kind = exchange,
|
||||
name = ?LOG_EXCH_NAME}}.
|
||||
|
|
|
|||
|
|
@ -18,12 +18,12 @@
|
|||
-include("rabbit.hrl").
|
||||
-include("rabbit_framing.hrl").
|
||||
|
||||
-export([recover/0, policy_changed/2, callback/4, declare/6,
|
||||
-export([recover/0, policy_changed/2, callback/4, declare/7,
|
||||
assert_equivalence/6, assert_args_equivalence/2, check_type/1,
|
||||
lookup/1, lookup_or_die/1, list/0, list/1, lookup_scratch/2,
|
||||
update_scratch/3, update_decorators/1, immutable/1,
|
||||
info_keys/0, info/1, info/2, info_all/1, info_all/2, info_all/4,
|
||||
route/2, delete/2, validate_binding/2]).
|
||||
route/2, delete/3, validate_binding/2]).
|
||||
%% these must be run inside a mnesia tx
|
||||
-export([maybe_auto_delete/2, serial/1, peek_serial/1, update/2]).
|
||||
|
||||
|
|
@ -43,7 +43,7 @@
|
|||
(rabbit_types:exchange(), rabbit_types:exchange()) -> 'ok'.
|
||||
-spec declare
|
||||
(name(), type(), boolean(), boolean(), boolean(),
|
||||
rabbit_framing:amqp_table())
|
||||
rabbit_framing:amqp_table(), rabbit_types:username())
|
||||
-> rabbit_types:exchange().
|
||||
-spec check_type
|
||||
(binary()) -> atom() | rabbit_types:connection_exit().
|
||||
|
|
@ -86,8 +86,10 @@
|
|||
-spec route(rabbit_types:exchange(), rabbit_types:delivery())
|
||||
-> [rabbit_amqqueue:name()].
|
||||
-spec delete
|
||||
(name(), 'true') -> 'ok' | rabbit_types:error('not_found' | 'in_use');
|
||||
(name(), 'false') -> 'ok' | rabbit_types:error('not_found').
|
||||
(name(), 'true', rabbit_types:username()) ->
|
||||
'ok'| rabbit_types:error('not_found' | 'in_use');
|
||||
(name(), 'false', rabbit_types:username()) ->
|
||||
'ok' | rabbit_types:error('not_found').
|
||||
-spec validate_binding
|
||||
(rabbit_types:exchange(), rabbit_types:binding())
|
||||
-> rabbit_types:ok_or_error({'binding_invalid', string(), [any()]}).
|
||||
|
|
@ -101,7 +103,7 @@
|
|||
%%----------------------------------------------------------------------------
|
||||
|
||||
-define(INFO_KEYS, [name, type, durable, auto_delete, internal, arguments,
|
||||
policy]).
|
||||
policy, user_who_performed_action]).
|
||||
|
||||
recover() ->
|
||||
Xs = rabbit_misc:table_filter(
|
||||
|
|
@ -151,14 +153,15 @@ serial(#exchange{name = XName} = X) ->
|
|||
(false) -> none
|
||||
end.
|
||||
|
||||
declare(XName, Type, Durable, AutoDelete, Internal, Args) ->
|
||||
declare(XName, Type, Durable, AutoDelete, Internal, Args, Username) ->
|
||||
X = rabbit_exchange_decorator:set(
|
||||
rabbit_policy:set(#exchange{name = XName,
|
||||
type = Type,
|
||||
durable = Durable,
|
||||
auto_delete = AutoDelete,
|
||||
internal = Internal,
|
||||
arguments = Args})),
|
||||
arguments = Args,
|
||||
options = #{user => Username}})),
|
||||
XT = type_to_module(Type),
|
||||
%% We want to upset things if it isn't ok
|
||||
ok = XT:validate(X),
|
||||
|
|
@ -342,6 +345,8 @@ i(policy, X) -> case rabbit_policy:name(X) of
|
|||
none -> '';
|
||||
Policy -> Policy
|
||||
end;
|
||||
i(user_who_performed_action, #exchange{options = Opts}) ->
|
||||
maps:get(user, Opts);
|
||||
i(Item, #exchange{type = Type} = X) ->
|
||||
case (type_to_module(Type)):info(X, [Item]) of
|
||||
[{Item, I}] -> I;
|
||||
|
|
@ -437,7 +442,7 @@ call_with_exchange(XName, Fun) ->
|
|||
end
|
||||
end).
|
||||
|
||||
delete(XName, IfUnused) ->
|
||||
delete(XName, IfUnused, Username) ->
|
||||
Fun = case IfUnused of
|
||||
true -> fun conditional_delete/2;
|
||||
false -> fun unconditional_delete/2
|
||||
|
|
@ -449,7 +454,7 @@ delete(XName, IfUnused) ->
|
|||
%% see rabbitmq/rabbitmq-federation#7
|
||||
rabbit_runtime_parameters:set(XName#resource.virtual_host,
|
||||
?EXCHANGE_DELETE_IN_PROGRESS_COMPONENT,
|
||||
XName#resource.name, true, none),
|
||||
XName#resource.name, true, Username),
|
||||
call_with_exchange(
|
||||
XName,
|
||||
fun (X) ->
|
||||
|
|
@ -457,7 +462,7 @@ delete(XName, IfUnused) ->
|
|||
{deleted, X, Bs, Deletions} ->
|
||||
rabbit_binding:process_deletions(
|
||||
rabbit_binding:add_deletion(
|
||||
XName, {X, deleted, Bs}, Deletions));
|
||||
XName, {X, deleted, Bs}, Deletions), Username);
|
||||
{error, _InUseOrNotFound} = E ->
|
||||
rabbit_misc:const(E)
|
||||
end
|
||||
|
|
@ -465,7 +470,7 @@ delete(XName, IfUnused) ->
|
|||
after
|
||||
rabbit_runtime_parameters:clear(XName#resource.virtual_host,
|
||||
?EXCHANGE_DELETE_IN_PROGRESS_COMPONENT,
|
||||
XName#resource.name)
|
||||
XName#resource.name, Username)
|
||||
end.
|
||||
|
||||
validate_binding(X = #exchange{type = XType}, Binding) ->
|
||||
|
|
|
|||
|
|
@ -21,7 +21,7 @@
|
|||
-include("rabbit.hrl").
|
||||
|
||||
-export([register/0]).
|
||||
-export([validate/5, notify/4, notify_clear/3]).
|
||||
-export([validate/5, notify/5, notify_clear/4]).
|
||||
|
||||
-import(rabbit_misc, [pget/2]).
|
||||
|
||||
|
|
@ -42,8 +42,8 @@ register() ->
|
|||
validate(_VHost, ?EXCHANGE_DELETE_IN_PROGRESS_COMPONENT, _Name, _Term, _User) ->
|
||||
ok.
|
||||
|
||||
notify(_VHost, ?EXCHANGE_DELETE_IN_PROGRESS_COMPONENT, _Name, _Term) ->
|
||||
notify(_VHost, ?EXCHANGE_DELETE_IN_PROGRESS_COMPONENT, _Name, _Term, _Username) ->
|
||||
ok.
|
||||
|
||||
notify_clear(_VHost, ?EXCHANGE_DELETE_IN_PROGRESS_COMPONENT, _Name) ->
|
||||
notify_clear(_VHost, ?EXCHANGE_DELETE_IN_PROGRESS_COMPONENT, _Name, _Username) ->
|
||||
ok.
|
||||
|
|
|
|||
|
|
@ -43,10 +43,10 @@
|
|||
-export([register/0]).
|
||||
-export([invalidate/0, recover/0]).
|
||||
-export([name/1, name_op/1, effective_definition/1, get/2, get_arg/3, set/1]).
|
||||
-export([validate/5, notify/4, notify_clear/3]).
|
||||
-export([parse_set/6, set/6, delete/2, lookup/2, list/0, list/1,
|
||||
-export([validate/5, notify/5, notify_clear/4]).
|
||||
-export([parse_set/7, set/7, delete/3, lookup/2, list/0, list/1,
|
||||
list_formatted/1, list_formatted/3, info_keys/0]).
|
||||
-export([parse_set_op/6, set_op/6, delete_op/2, lookup_op/2, list_op/0, list_op/1,
|
||||
-export([parse_set_op/7, set_op/7, delete_op/3, lookup_op/2, list_op/0, list_op/1,
|
||||
list_formatted_op/1, list_formatted_op/3]).
|
||||
|
||||
-rabbit_boot_step({?MODULE,
|
||||
|
|
@ -198,38 +198,42 @@ invalid_file() ->
|
|||
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
parse_set_op(VHost, Name, Pattern, Definition, Priority, ApplyTo) ->
|
||||
parse_set(<<"operator_policy">>, VHost, Name, Pattern, Definition, Priority, ApplyTo).
|
||||
parse_set_op(VHost, Name, Pattern, Definition, Priority, ApplyTo, ActingUser) ->
|
||||
parse_set(<<"operator_policy">>, VHost, Name, Pattern, Definition, Priority,
|
||||
ApplyTo, ActingUser).
|
||||
|
||||
parse_set(VHost, Name, Pattern, Definition, Priority, ApplyTo) ->
|
||||
parse_set(<<"policy">>, VHost, Name, Pattern, Definition, Priority, ApplyTo).
|
||||
parse_set(VHost, Name, Pattern, Definition, Priority, ApplyTo, ActingUser) ->
|
||||
parse_set(<<"policy">>, VHost, Name, Pattern, Definition, Priority, ApplyTo,
|
||||
ActingUser).
|
||||
|
||||
parse_set(Type, VHost, Name, Pattern, Definition, Priority, ApplyTo) ->
|
||||
parse_set(Type, VHost, Name, Pattern, Definition, Priority, ApplyTo, ActingUser) ->
|
||||
try rabbit_data_coercion:to_integer(Priority) of
|
||||
Num -> parse_set0(Type, VHost, Name, Pattern, Definition, Num, ApplyTo)
|
||||
Num -> parse_set0(Type, VHost, Name, Pattern, Definition, Num, ApplyTo,
|
||||
ActingUser)
|
||||
catch
|
||||
error:badarg -> {error, "~p priority must be a number", [Priority]}
|
||||
end.
|
||||
|
||||
parse_set0(Type, VHost, Name, Pattern, Defn, Priority, ApplyTo) ->
|
||||
parse_set0(Type, VHost, Name, Pattern, Defn, Priority, ApplyTo, ActingUser) ->
|
||||
case rabbit_json:try_decode(Defn) of
|
||||
{ok, Term} ->
|
||||
set0(Type, VHost, Name,
|
||||
[{<<"pattern">>, Pattern},
|
||||
{<<"definition">>, maps:to_list(Term)},
|
||||
{<<"priority">>, Priority},
|
||||
{<<"apply-to">>, ApplyTo}]);
|
||||
{<<"apply-to">>, ApplyTo}],
|
||||
ActingUser);
|
||||
error ->
|
||||
{error_string, "JSON decoding error"}
|
||||
end.
|
||||
|
||||
set_op(VHost, Name, Pattern, Definition, Priority, ApplyTo) ->
|
||||
set(<<"operator_policy">>, VHost, Name, Pattern, Definition, Priority, ApplyTo).
|
||||
set_op(VHost, Name, Pattern, Definition, Priority, ApplyTo, ActingUser) ->
|
||||
set(<<"operator_policy">>, VHost, Name, Pattern, Definition, Priority, ApplyTo, ActingUser).
|
||||
|
||||
set(VHost, Name, Pattern, Definition, Priority, ApplyTo) ->
|
||||
set(<<"policy">>, VHost, Name, Pattern, Definition, Priority, ApplyTo).
|
||||
set(VHost, Name, Pattern, Definition, Priority, ApplyTo, ActingUser) ->
|
||||
set(<<"policy">>, VHost, Name, Pattern, Definition, Priority, ApplyTo, ActingUser).
|
||||
|
||||
set(Type, VHost, Name, Pattern, Definition, Priority, ApplyTo) ->
|
||||
set(Type, VHost, Name, Pattern, Definition, Priority, ApplyTo, ActingUser) ->
|
||||
PolicyProps = [{<<"pattern">>, Pattern},
|
||||
{<<"definition">>, Definition},
|
||||
{<<"priority">>, case Priority of
|
||||
|
|
@ -240,16 +244,16 @@ set(Type, VHost, Name, Pattern, Definition, Priority, ApplyTo) ->
|
|||
undefined -> <<"all">>;
|
||||
_ -> ApplyTo
|
||||
end}],
|
||||
set0(Type, VHost, Name, PolicyProps).
|
||||
set0(Type, VHost, Name, PolicyProps, ActingUser).
|
||||
|
||||
set0(Type, VHost, Name, Term) ->
|
||||
rabbit_runtime_parameters:set_any(VHost, Type, Name, Term, none).
|
||||
set0(Type, VHost, Name, Term, ActingUser) ->
|
||||
rabbit_runtime_parameters:set_any(VHost, Type, Name, Term, ActingUser).
|
||||
|
||||
delete_op(VHost, Name) ->
|
||||
rabbit_runtime_parameters:clear_any(VHost, <<"operator_policy">>, Name).
|
||||
delete_op(VHost, Name, ActingUser) ->
|
||||
rabbit_runtime_parameters:clear_any(VHost, <<"operator_policy">>, Name, ActingUser).
|
||||
|
||||
delete(VHost, Name) ->
|
||||
rabbit_runtime_parameters:clear_any(VHost, <<"policy">>, Name).
|
||||
delete(VHost, Name, ActingUser) ->
|
||||
rabbit_runtime_parameters:clear_any(VHost, <<"policy">>, Name, ActingUser).
|
||||
|
||||
lookup_op(VHost, Name) ->
|
||||
case rabbit_runtime_parameters:lookup(VHost, <<"operator_policy">>, Name) of
|
||||
|
|
@ -322,18 +326,23 @@ validate(_VHost, <<"operator_policy">>, Name, Term, _User) ->
|
|||
rabbit_parameter_validation:proplist(
|
||||
Name, operator_policy_validation(), Term).
|
||||
|
||||
notify(VHost, <<"policy">>, Name, Term) ->
|
||||
rabbit_event:notify(policy_set, [{name, Name}, {vhost, VHost} | Term]),
|
||||
notify(VHost, <<"policy">>, Name, Term, ActingUser) ->
|
||||
rabbit_event:notify(policy_set, [{name, Name}, {vhost, VHost},
|
||||
{user_who_performed_action, ActingUser} | Term]),
|
||||
update_policies(VHost);
|
||||
notify(VHost, <<"operator_policy">>, Name, Term) ->
|
||||
rabbit_event:notify(policy_set, [{name, Name}, {vhost, VHost} | Term]),
|
||||
notify(VHost, <<"operator_policy">>, Name, Term, ActingUser) ->
|
||||
rabbit_event:notify(policy_set, [{name, Name}, {vhost, VHost},
|
||||
{user_who_performed_action, ActingUser} | Term]),
|
||||
update_policies(VHost).
|
||||
|
||||
notify_clear(VHost, <<"policy">>, Name) ->
|
||||
rabbit_event:notify(policy_cleared, [{name, Name}, {vhost, VHost}]),
|
||||
notify_clear(VHost, <<"policy">>, Name, ActingUser) ->
|
||||
rabbit_event:notify(policy_cleared, [{name, Name}, {vhost, VHost},
|
||||
{user_who_performed_action, ActingUser}]),
|
||||
update_policies(VHost);
|
||||
notify_clear(VHost, <<"operator_policy">>, Name) ->
|
||||
rabbit_event:notify(operator_policy_cleared, [{name, Name}, {vhost, VHost}]),
|
||||
notify_clear(VHost, <<"operator_policy">>, Name, ActingUser) ->
|
||||
rabbit_event:notify(operator_policy_cleared,
|
||||
[{name, Name}, {vhost, VHost},
|
||||
{user_who_performed_action, ActingUser}]),
|
||||
update_policies(VHost).
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
|
|
|
|||
|
|
@ -51,13 +51,13 @@
|
|||
|
||||
-include("rabbit.hrl").
|
||||
|
||||
-export([parse_set/5, set/5, set_any/5, clear/3, clear_any/3, list/0, list/1,
|
||||
-export([parse_set/5, set/5, set_any/5, clear/4, clear_any/4, list/0, list/1,
|
||||
list_component/1, list/2, list_formatted/1, list_formatted/3,
|
||||
lookup/3, value/3, value/4, info_keys/0, clear_component/1]).
|
||||
|
||||
-export([parse_set_global/2, set_global/2, value_global/1, value_global/2,
|
||||
-export([parse_set_global/3, set_global/3, value_global/1, value_global/2,
|
||||
list_global/0, list_global_formatted/0, list_global_formatted/2,
|
||||
lookup_global/1, global_info_keys/0, clear_global/1]).
|
||||
lookup_global/1, global_info_keys/0, clear_global/2]).
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
|
|
@ -65,15 +65,18 @@
|
|||
-type ok_thunk_or_error_string() :: ok_or_error_string() | fun(() -> 'ok').
|
||||
|
||||
-spec parse_set(rabbit_types:vhost(), binary(), binary(), string(),
|
||||
rabbit_types:user() | 'none') -> ok_or_error_string().
|
||||
rabbit_types:user() | rabbit_types:username() | 'none')
|
||||
-> ok_or_error_string().
|
||||
-spec set(rabbit_types:vhost(), binary(), binary(), term(),
|
||||
rabbit_types:user() | 'none') -> ok_or_error_string().
|
||||
rabbit_types:user() | rabbit_types:username() | 'none')
|
||||
-> ok_or_error_string().
|
||||
-spec set_any(rabbit_types:vhost(), binary(), binary(), term(),
|
||||
rabbit_types:user() | 'none') -> ok_or_error_string().
|
||||
-spec set_global(atom(), term()) -> 'ok'.
|
||||
-spec clear(rabbit_types:vhost(), binary(), binary())
|
||||
-> ok_thunk_or_error_string().
|
||||
-spec clear_any(rabbit_types:vhost(), binary(), binary())
|
||||
rabbit_types:user() | rabbit_types:username() | 'none')
|
||||
-> ok_or_error_string().
|
||||
-spec set_global(atom(), term(), rabbit_types:username()) -> 'ok'.
|
||||
-spec clear(rabbit_types:vhost(), binary(), binary(), rabbit_types:username())
|
||||
-> ok_thunk_or_error_string().
|
||||
-spec clear_any(rabbit_types:vhost(), binary(), binary(), rabbit_types:username())
|
||||
-> ok_thunk_or_error_string().
|
||||
-spec list() -> [rabbit_types:infos()].
|
||||
-spec list(rabbit_types:vhost() | '_') -> [rabbit_types:infos()].
|
||||
|
|
@ -113,19 +116,20 @@ set(_, <<"policy">>, _, _, _) ->
|
|||
set(VHost, Component, Name, Term, User) ->
|
||||
set_any(VHost, Component, Name, Term, User).
|
||||
|
||||
parse_set_global(Name, String) ->
|
||||
parse_set_global(Name, String, ActingUser) ->
|
||||
Definition = rabbit_data_coercion:to_binary(String),
|
||||
case rabbit_json:try_decode(Definition) of
|
||||
{ok, Term} when is_map(Term) -> set_global(Name, maps:to_list(Term));
|
||||
{ok, Term} -> set_global(Name, Term);
|
||||
{ok, Term} when is_map(Term) -> set_global(Name, maps:to_list(Term), ActingUser);
|
||||
{ok, Term} -> set_global(Name, Term, ActingUser);
|
||||
error -> {error_string, "JSON decoding error"}
|
||||
end.
|
||||
|
||||
set_global(Name, Term) ->
|
||||
set_global(Name, Term, ActingUser) ->
|
||||
NameAsAtom = rabbit_data_coercion:to_atom(Name),
|
||||
mnesia_update(NameAsAtom, Term),
|
||||
event_notify(parameter_set, none, global, [{name, NameAsAtom},
|
||||
{value, Term}]),
|
||||
{value, Term},
|
||||
{user_who_performed_action, ActingUser}]),
|
||||
ok.
|
||||
|
||||
format_error(L) ->
|
||||
|
|
@ -141,15 +145,19 @@ set_any0(VHost, Component, Name, Term, User) ->
|
|||
case lookup_component(Component) of
|
||||
{ok, Mod} ->
|
||||
case flatten_errors(
|
||||
Mod:validate(VHost, Component, Name, Term, User)) of
|
||||
Mod:validate(VHost, Component, Name, Term, get_user(User))) of
|
||||
ok ->
|
||||
case mnesia_update(VHost, Component, Name, Term) of
|
||||
{old, Term} -> ok;
|
||||
_ -> event_notify(
|
||||
parameter_set, VHost, Component,
|
||||
[{name, Name},
|
||||
{value, Term}]),
|
||||
Mod:notify(VHost, Component, Name, Term)
|
||||
{old, Term} ->
|
||||
ok;
|
||||
_ ->
|
||||
ActingUser = get_username(User),
|
||||
event_notify(
|
||||
parameter_set, VHost, Component,
|
||||
[{name, Name},
|
||||
{value, Term},
|
||||
{user_who_performed_action, ActingUser}]),
|
||||
Mod:notify(VHost, Component, Name, Term, ActingUser)
|
||||
end,
|
||||
ok;
|
||||
E ->
|
||||
|
|
@ -159,6 +167,19 @@ set_any0(VHost, Component, Name, Term, User) ->
|
|||
E
|
||||
end.
|
||||
|
||||
%% Validate only an user record as expected by the API before #rabbitmq-event-exchange-10
|
||||
get_user(#user{} = User) ->
|
||||
User;
|
||||
get_user(_) ->
|
||||
none.
|
||||
|
||||
get_username(#user{username = Username}) ->
|
||||
Username;
|
||||
get_username(none) ->
|
||||
?INTERNAL_USER;
|
||||
get_username(Any) ->
|
||||
Any.
|
||||
|
||||
mnesia_update(Key, Term) ->
|
||||
rabbit_misc:execute_mnesia_transaction(mnesia_update_fun(Key, Term)).
|
||||
|
||||
|
|
@ -176,15 +197,17 @@ mnesia_update_fun(Key, Term) ->
|
|||
Res
|
||||
end.
|
||||
|
||||
clear(_, <<"policy">> , _) ->
|
||||
clear(_, <<"policy">> , _, _) ->
|
||||
{error_string, "policies may not be cleared using this method"};
|
||||
clear(VHost, Component, Name) ->
|
||||
clear_any(VHost, Component, Name).
|
||||
clear(VHost, Component, Name, ActingUser) ->
|
||||
clear_any(VHost, Component, Name, ActingUser).
|
||||
|
||||
clear_global(Key) ->
|
||||
clear_global(Key, ActingUser) ->
|
||||
KeyAsAtom = rabbit_data_coercion:to_atom(Key),
|
||||
Notify = fun() ->
|
||||
event_notify(parameter_set, none, global, [{name, KeyAsAtom}]),
|
||||
event_notify(parameter_set, none, global,
|
||||
[{name, KeyAsAtom},
|
||||
{user_who_performed_action, ActingUser}]),
|
||||
ok
|
||||
end,
|
||||
case value_global(KeyAsAtom) of
|
||||
|
|
@ -212,13 +235,14 @@ clear_component(Component) ->
|
|||
ok
|
||||
end.
|
||||
|
||||
clear_any(VHost, Component, Name) ->
|
||||
clear_any(VHost, Component, Name, ActingUser) ->
|
||||
Notify = fun () ->
|
||||
case lookup_component(Component) of
|
||||
{ok, Mod} -> event_notify(
|
||||
parameter_cleared, VHost, Component,
|
||||
[{name, Name}]),
|
||||
Mod:notify_clear(VHost, Component, Name);
|
||||
[{name, Name},
|
||||
{user_who_performed_action, ActingUser}]),
|
||||
Mod:notify_clear(VHost, Component, Name, ActingUser);
|
||||
_ -> ok
|
||||
end
|
||||
end,
|
||||
|
|
|
|||
|
|
@ -20,14 +20,14 @@
|
|||
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
-export([add/1, delete/1, exists/1, list/0, with/2, assert/1, update/2,
|
||||
-export([add/2, delete/2, exists/1, list/0, with/2, assert/1, update/2,
|
||||
set_limits/2, limits_of/1]).
|
||||
-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]).
|
||||
-export([purge_messages/1]).
|
||||
|
||||
-spec add(rabbit_types:vhost()) -> 'ok'.
|
||||
-spec delete(rabbit_types:vhost()) -> 'ok'.
|
||||
-spec add(rabbit_types:vhost(), rabbit_types:username()) -> 'ok'.
|
||||
-spec delete(rabbit_types:vhost(), rabbit_types:username()) -> 'ok'.
|
||||
-spec update(rabbit_types:vhost(), rabbit_misc:thunk(A)) -> A.
|
||||
-spec exists(rabbit_types:vhost()) -> boolean().
|
||||
-spec list() -> [rabbit_types:vhost()].
|
||||
|
|
@ -46,7 +46,7 @@
|
|||
|
||||
-define(INFO_KEYS, [name, tracing]).
|
||||
|
||||
add(VHostPath) ->
|
||||
add(VHostPath, ActingUser) ->
|
||||
rabbit_log:info("Adding vhost '~s'~n", [VHostPath]),
|
||||
R = rabbit_misc:execute_mnesia_transaction(
|
||||
fun () ->
|
||||
|
|
@ -62,7 +62,7 @@ add(VHostPath) ->
|
|||
(ok, false) ->
|
||||
[rabbit_exchange:declare(
|
||||
rabbit_misc:r(VHostPath, exchange, Name),
|
||||
Type, true, false, Internal, []) ||
|
||||
Type, true, false, Internal, [], ActingUser) ||
|
||||
{Name, Type, Internal} <-
|
||||
[{<<"">>, direct, false},
|
||||
{<<"amq.direct">>, direct, false},
|
||||
|
|
@ -75,24 +75,26 @@ add(VHostPath) ->
|
|||
{<<"amq.rabbitmq.trace">>, topic, true}]],
|
||||
ok
|
||||
end),
|
||||
rabbit_event:notify(vhost_created, info(VHostPath)),
|
||||
rabbit_event:notify(vhost_created, info(VHostPath)
|
||||
++ [{user_who_performed_action, ActingUser}]),
|
||||
R.
|
||||
|
||||
delete(VHostPath) ->
|
||||
delete(VHostPath, 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 mnesia actions and
|
||||
%% eventually the termination of that process. Exchange deletion causes
|
||||
%% notifications which must be sent outside the TX
|
||||
rabbit_log:info("Deleting vhost '~s'~n", [VHostPath]),
|
||||
QDelFun = fun (Q) -> rabbit_amqqueue:delete(Q, false, false) end,
|
||||
QDelFun = fun (Q) -> rabbit_amqqueue:delete(Q, false, false, ActingUser) end,
|
||||
[assert_benign(rabbit_amqqueue:with(Name, QDelFun)) ||
|
||||
#amqqueue{name = Name} <- rabbit_amqqueue:list(VHostPath)],
|
||||
[assert_benign(rabbit_exchange:delete(Name, false)) ||
|
||||
[assert_benign(rabbit_exchange:delete(Name, false, ActingUser)) ||
|
||||
#exchange{name = Name} <- rabbit_exchange:list(VHostPath)],
|
||||
Funs = rabbit_misc:execute_mnesia_transaction(
|
||||
with(VHostPath, fun () -> internal_delete(VHostPath) end)),
|
||||
ok = rabbit_event:notify(vhost_deleted, [{name, VHostPath}]),
|
||||
with(VHostPath, fun () -> internal_delete(VHostPath, ActingUser) end)),
|
||||
ok = rabbit_event:notify(vhost_deleted, [{name, VHostPath},
|
||||
{user_who_performed_action, ActingUser}]),
|
||||
[ok = Fun() || Fun <- Funs],
|
||||
ok.
|
||||
|
||||
|
|
@ -117,18 +119,19 @@ assert_benign({error, {absent, Q, _}}) ->
|
|||
{error, not_found} -> ok
|
||||
end.
|
||||
|
||||
internal_delete(VHostPath) ->
|
||||
internal_delete(VHostPath, ActingUser) ->
|
||||
[ok = rabbit_auth_backend_internal:clear_permissions(
|
||||
proplists:get_value(user, Info), VHostPath)
|
||||
proplists:get_value(user, Info), VHostPath, ActingUser)
|
||||
|| Info <- rabbit_auth_backend_internal:list_vhost_permissions(VHostPath)],
|
||||
TopicPermissions = rabbit_auth_backend_internal:list_vhost_topic_permissions(VHostPath),
|
||||
[ok = rabbit_auth_backend_internal:clear_topic_permissions(
|
||||
proplists:get_value(user, TopicPermission), VHostPath) || TopicPermission <- TopicPermissions],
|
||||
Fs1 = [rabbit_runtime_parameters:clear(VHostPath,
|
||||
proplists:get_value(component, Info),
|
||||
proplists:get_value(name, Info))
|
||||
proplists:get_value(name, Info),
|
||||
ActingUser)
|
||||
|| Info <- rabbit_runtime_parameters:list(VHostPath)],
|
||||
Fs2 = [rabbit_policy:delete(VHostPath, proplists:get_value(name, Info))
|
||||
Fs2 = [rabbit_policy:delete(VHostPath, proplists:get_value(name, Info), ActingUser)
|
||||
|| Info <- rabbit_policy:list(VHostPath)],
|
||||
ok = mnesia:delete({rabbit_vhost, VHostPath}),
|
||||
purge_messages(VHostPath),
|
||||
|
|
|
|||
|
|
@ -21,10 +21,10 @@
|
|||
-include("rabbit.hrl").
|
||||
|
||||
-export([register/0]).
|
||||
-export([parse_set/2, set/2, clear/1]).
|
||||
-export([parse_set/3, set/3, clear/2]).
|
||||
-export([list/0, list/1]).
|
||||
-export([update_limit/3, clear_limit/2, get_limit/2]).
|
||||
-export([validate/5, notify/4, notify_clear/3]).
|
||||
-export([update_limit/4, clear_limit/3, get_limit/2]).
|
||||
-export([validate/5, notify/5, notify_clear/4]).
|
||||
-export([connection_limit/1, queue_limit/1,
|
||||
is_over_queue_limit/1, is_over_connection_limit/1]).
|
||||
|
||||
|
|
@ -45,12 +45,15 @@ validate(_VHost, <<"vhost-limits">>, Name, Term, _User) ->
|
|||
rabbit_parameter_validation:proplist(
|
||||
Name, vhost_limit_validation(), Term).
|
||||
|
||||
notify(VHost, <<"vhost-limits">>, <<"limits">>, Limits) ->
|
||||
rabbit_event:notify(vhost_limits_set, [{name, <<"limits">>} | Limits]),
|
||||
notify(VHost, <<"vhost-limits">>, <<"limits">>, Limits, ActingUser) ->
|
||||
rabbit_event:notify(vhost_limits_set, [{name, <<"limits">>},
|
||||
{user_who_performed_action, ActingUser}
|
||||
| Limits]),
|
||||
update_vhost(VHost, Limits).
|
||||
|
||||
notify_clear(VHost, <<"vhost-limits">>, <<"limits">>) ->
|
||||
rabbit_event:notify(vhost_limits_cleared, [{name, <<"limits">>}]),
|
||||
notify_clear(VHost, <<"vhost-limits">>, <<"limits">>, ActingUser) ->
|
||||
rabbit_event:notify(vhost_limits_cleared, [{name, <<"limits">>},
|
||||
{user_who_performed_action, ActingUser}]),
|
||||
update_vhost(VHost, undefined).
|
||||
|
||||
connection_limit(VirtualHost) ->
|
||||
|
|
@ -128,38 +131,38 @@ is_over_queue_limit(VirtualHost) ->
|
|||
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
parse_set(VHost, Defn) ->
|
||||
parse_set(VHost, Defn, ActingUser) ->
|
||||
Definition = rabbit_data_coercion:to_binary(Defn),
|
||||
case rabbit_json:try_decode(Definition) of
|
||||
{ok, Term} ->
|
||||
set(VHost, maps:to_list(Term));
|
||||
set(VHost, maps:to_list(Term), ActingUser);
|
||||
error ->
|
||||
{error_string, "JSON decoding error"}
|
||||
end.
|
||||
|
||||
set(VHost, Defn) ->
|
||||
set(VHost, Defn, ActingUser) ->
|
||||
rabbit_runtime_parameters:set_any(VHost, <<"vhost-limits">>,
|
||||
<<"limits">>, Defn, none).
|
||||
<<"limits">>, Defn, ActingUser).
|
||||
|
||||
clear(VHost) ->
|
||||
clear(VHost, ActingUser) ->
|
||||
rabbit_runtime_parameters:clear_any(VHost, <<"vhost-limits">>,
|
||||
<<"limits">>).
|
||||
<<"limits">>, ActingUser).
|
||||
|
||||
update_limit(VHost, Name, Value) ->
|
||||
update_limit(VHost, Name, Value, ActingUser) ->
|
||||
OldDef = case rabbit_runtime_parameters:list(VHost, <<"vhost-limits">>) of
|
||||
[] -> [];
|
||||
[Param] -> pget(value, Param, [])
|
||||
end,
|
||||
NewDef = [{Name, Value} | lists:keydelete(Name, 1, OldDef)],
|
||||
set(VHost, NewDef).
|
||||
set(VHost, NewDef, ActingUser).
|
||||
|
||||
clear_limit(VHost, Name) ->
|
||||
clear_limit(VHost, Name, ActingUser) ->
|
||||
OldDef = case rabbit_runtime_parameters:list(VHost, <<"vhost-limits">>) of
|
||||
[] -> [];
|
||||
[Param] -> pget(value, Param, [])
|
||||
end,
|
||||
NewDef = lists:keydelete(Name, 1, OldDef),
|
||||
set(VHost, NewDef).
|
||||
set(VHost, NewDef, ActingUser).
|
||||
|
||||
vhost_limit_validation() ->
|
||||
[{<<"max-connections">>, fun rabbit_parameter_validation:integer/2, optional},
|
||||
|
|
|
|||
|
|
@ -20,7 +20,7 @@
|
|||
|
||||
-include("rabbit.hrl").
|
||||
|
||||
-export([validate/5, notify/4, notify_clear/3]).
|
||||
-export([validate/5, notify/5, notify_clear/4]).
|
||||
-export([register/0, unregister/0]).
|
||||
-export([validate_policy/1]).
|
||||
-export([register_policy_validator/0, unregister_policy_validator/0]).
|
||||
|
|
@ -43,8 +43,8 @@ validate(_, <<"test">>, <<"admin">>, _Term, User) ->
|
|||
end;
|
||||
validate(_, <<"test">>, _, _, _) -> {error, "meh", []}.
|
||||
|
||||
notify(_, _, _, _) -> ok.
|
||||
notify_clear(_, _, _) -> ok.
|
||||
notify(_, _, _, _, _) -> ok.
|
||||
notify_clear(_, _, _, _) -> ok.
|
||||
|
||||
%----------------------------------------------------------------------------
|
||||
|
||||
|
|
|
|||
|
|
@ -223,7 +223,7 @@ vhost_deletion(Config) ->
|
|||
rabbit_ct_broker_helpers:set_ha_policy_all(Config),
|
||||
ACh = rabbit_ct_client_helpers:open_channel(Config, A),
|
||||
amqp_channel:call(ACh, #'queue.declare'{queue = <<"vhost_deletion-q">>}),
|
||||
ok = rpc:call(A, rabbit_vhost, delete, [<<"/">>]),
|
||||
ok = rpc:call(A, rabbit_vhost, delete, [<<"/">>, <<"acting-user">>]),
|
||||
ok.
|
||||
|
||||
promote_on_shutdown(Config) ->
|
||||
|
|
|
|||
|
|
@ -245,7 +245,8 @@ unset_location_config(Config) ->
|
|||
declare(Config, QueueName, Durable, AutoDelete, Args, Owner) ->
|
||||
Node = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
|
||||
{new, Queue} = rpc:call(Node, rabbit_amqqueue, declare,
|
||||
[QueueName, Durable, AutoDelete, Args, Owner]),
|
||||
[QueueName, Durable, AutoDelete, Args, Owner,
|
||||
<<"acting-user">>]),
|
||||
Queue.
|
||||
|
||||
verify_min_master(Config, Q) ->
|
||||
|
|
|
|||
|
|
@ -70,13 +70,13 @@ topic_permission_database_access(Config) ->
|
|||
|
||||
topic_permission_database_access1(_Config) ->
|
||||
0 = length(ets:tab2list(rabbit_topic_permission)),
|
||||
rabbit_vhost:add(<<"/">>),
|
||||
rabbit_vhost:add(<<"other-vhost">>),
|
||||
rabbit_auth_backend_internal:add_user(<<"guest">>, <<"guest">>),
|
||||
rabbit_auth_backend_internal:add_user(<<"dummy">>, <<"dummy">>),
|
||||
rabbit_vhost:add(<<"/">>, <<"acting-user">>),
|
||||
rabbit_vhost:add(<<"other-vhost">>, <<"acting-user">>),
|
||||
rabbit_auth_backend_internal:add_user(<<"guest">>, <<"guest">>, <<"acting-user">>),
|
||||
rabbit_auth_backend_internal:add_user(<<"dummy">>, <<"dummy">>, <<"acting-user">>),
|
||||
|
||||
rabbit_auth_backend_internal:set_topic_permissions(
|
||||
<<"guest">>, <<"/">>, <<"amq.topic">>, "^a"
|
||||
<<"guest">>, <<"/">>, <<"amq.topic">>, "^a", <<"acting-user">>
|
||||
),
|
||||
1 = length(ets:tab2list(rabbit_topic_permission)),
|
||||
1 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"guest">>)),
|
||||
|
|
@ -88,7 +88,7 @@ topic_permission_database_access1(_Config) ->
|
|||
1 = length(rabbit_auth_backend_internal:list_topic_permissions()),
|
||||
|
||||
rabbit_auth_backend_internal:set_topic_permissions(
|
||||
<<"guest">>, <<"other-vhost">>, <<"amq.topic">>, ".*"
|
||||
<<"guest">>, <<"other-vhost">>, <<"amq.topic">>, ".*", <<"acting-user">>
|
||||
),
|
||||
2 = length(ets:tab2list(rabbit_topic_permission)),
|
||||
2 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"guest">>)),
|
||||
|
|
@ -100,10 +100,10 @@ topic_permission_database_access1(_Config) ->
|
|||
2 = length(rabbit_auth_backend_internal:list_topic_permissions()),
|
||||
|
||||
rabbit_auth_backend_internal:set_topic_permissions(
|
||||
<<"guest">>, <<"/">>, <<"topic1">>, "^a"
|
||||
<<"guest">>, <<"/">>, <<"topic1">>, "^a", <<"acting-user">>
|
||||
),
|
||||
rabbit_auth_backend_internal:set_topic_permissions(
|
||||
<<"guest">>, <<"/">>, <<"topic2">>, "^a"
|
||||
<<"guest">>, <<"/">>, <<"topic2">>, "^a", <<"acting-user">>
|
||||
),
|
||||
|
||||
4 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"guest">>)),
|
||||
|
|
@ -111,25 +111,29 @@ topic_permission_database_access1(_Config) ->
|
|||
1 = length(rabbit_auth_backend_internal:list_user_vhost_topic_permissions(<<"guest">>,<<"other-vhost">>)),
|
||||
4 = length(rabbit_auth_backend_internal:list_topic_permissions()),
|
||||
|
||||
rabbit_auth_backend_internal:clear_topic_permissions(<<"guest">>, <<"other-vhost">>),
|
||||
rabbit_auth_backend_internal:clear_topic_permissions(<<"guest">>, <<"other-vhost">>,
|
||||
<<"acting-user">>),
|
||||
0 = length(rabbit_auth_backend_internal:list_vhost_topic_permissions(<<"other-vhost">>)),
|
||||
3 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"guest">>)),
|
||||
rabbit_auth_backend_internal:clear_topic_permissions(<<"guest">>, <<"/">>, <<"topic1">>),
|
||||
rabbit_auth_backend_internal:clear_topic_permissions(<<"guest">>, <<"/">>, <<"topic1">>,
|
||||
<<"acting-user">>),
|
||||
2 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"guest">>)),
|
||||
rabbit_auth_backend_internal:clear_topic_permissions(<<"guest">>, <<"/">>),
|
||||
rabbit_auth_backend_internal:clear_topic_permissions(<<"guest">>, <<"/">>,
|
||||
<<"acting-user">>),
|
||||
0 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"guest">>)),
|
||||
|
||||
|
||||
{error, {no_such_user, _}} = (catch rabbit_auth_backend_internal:set_topic_permissions(
|
||||
<<"non-existing-user">>, <<"other-vhost">>, <<"amq.topic">>, ".*"
|
||||
<<"non-existing-user">>, <<"other-vhost">>, <<"amq.topic">>, ".*", <<"acting-user">>
|
||||
)),
|
||||
|
||||
{error, {no_such_vhost, _}} = (catch rabbit_auth_backend_internal:set_topic_permissions(
|
||||
<<"guest">>, <<"non-existing-vhost">>, <<"amq.topic">>, ".*"
|
||||
<<"guest">>, <<"non-existing-vhost">>, <<"amq.topic">>, ".*", <<"acting-user">>
|
||||
)),
|
||||
|
||||
{error, {no_such_user, _}} = (catch rabbit_auth_backend_internal:set_topic_permissions(
|
||||
<<"non-existing-user">>, <<"non-existing-vhost">>, <<"amq.topic">>, ".*"
|
||||
<<"non-existing-user">>, <<"non-existing-vhost">>, <<"amq.topic">>, ".*",
|
||||
<<"acting-user">>
|
||||
)),
|
||||
|
||||
{error, {no_such_user, _}} = (catch rabbit_auth_backend_internal:list_user_topic_permissions(
|
||||
|
|
@ -141,7 +145,7 @@ topic_permission_database_access1(_Config) ->
|
|||
)),
|
||||
|
||||
{error, {invalid_regexp, _, _}} = (catch rabbit_auth_backend_internal:set_topic_permissions(
|
||||
<<"guest">>, <<"/">>, <<"amq.topic">>, "["
|
||||
<<"guest">>, <<"/">>, <<"amq.topic">>, "[", <<"acting-user">>
|
||||
)),
|
||||
ok.
|
||||
|
||||
|
|
@ -159,11 +163,11 @@ topic_permission_checks1(_Config) ->
|
|||
#vhost{virtual_host = <<"other-vhost">>},
|
||||
write)
|
||||
end),
|
||||
rabbit_auth_backend_internal:add_user(<<"guest">>, <<"guest">>),
|
||||
rabbit_auth_backend_internal:add_user(<<"dummy">>, <<"dummy">>),
|
||||
rabbit_auth_backend_internal:add_user(<<"guest">>, <<"guest">>, <<"acting-user">>),
|
||||
rabbit_auth_backend_internal:add_user(<<"dummy">>, <<"dummy">>, <<"acting-user">>),
|
||||
|
||||
rabbit_auth_backend_internal:set_topic_permissions(
|
||||
<<"guest">>, <<"/">>, <<"amq.topic">>, "^a"
|
||||
<<"guest">>, <<"/">>, <<"amq.topic">>, "^a", <<"acting-user">>
|
||||
),
|
||||
1 = length(ets:tab2list(rabbit_topic_permission)),
|
||||
1 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"guest">>)),
|
||||
|
|
@ -172,7 +176,7 @@ topic_permission_checks1(_Config) ->
|
|||
0 = length(rabbit_auth_backend_internal:list_vhost_topic_permissions(<<"other-vhost">>)),
|
||||
|
||||
rabbit_auth_backend_internal:set_topic_permissions(
|
||||
<<"guest">>, <<"other-vhost">>, <<"amq.topic">>, ".*"
|
||||
<<"guest">>, <<"other-vhost">>, <<"amq.topic">>, ".*", <<"acting-user">>
|
||||
),
|
||||
2 = length(ets:tab2list(rabbit_topic_permission)),
|
||||
2 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"guest">>)),
|
||||
|
|
@ -214,4 +218,4 @@ topic_permission_checks1(_Config) ->
|
|||
write,
|
||||
Context
|
||||
),
|
||||
ok.
|
||||
ok.
|
||||
|
|
|
|||
|
|
@ -794,7 +794,7 @@ bq_variable_queue_delete_msg_store_files_callback1(Config) ->
|
|||
rabbit_amqqueue:declare(
|
||||
queue_name(Config,
|
||||
<<"bq_variable_queue_delete_msg_store_files_callback-q">>),
|
||||
true, false, [], none),
|
||||
true, false, [], none, <<"acting-user">>),
|
||||
Payload = <<0:8388608>>, %% 1MB
|
||||
Count = 30,
|
||||
publish_and_confirm(Q, Payload, Count),
|
||||
|
|
@ -811,7 +811,7 @@ bq_variable_queue_delete_msg_store_files_callback1(Config) ->
|
|||
%% give the queue a second to receive the close_fds callback msg
|
||||
timer:sleep(1000),
|
||||
|
||||
rabbit_amqqueue:delete(Q, false, false),
|
||||
rabbit_amqqueue:delete(Q, false, false, <<"acting-user">>),
|
||||
passed.
|
||||
|
||||
bq_queue_recover(Config) ->
|
||||
|
|
@ -822,7 +822,7 @@ bq_queue_recover1(Config) ->
|
|||
Count = 2 * rabbit_queue_index:next_segment_boundary(0),
|
||||
{new, #amqqueue { pid = QPid, name = QName } = Q} =
|
||||
rabbit_amqqueue:declare(queue_name(Config, <<"bq_queue_recover-q">>),
|
||||
true, false, [], none),
|
||||
true, false, [], none, <<"acting-user">>),
|
||||
publish_and_confirm(Q, <<>>, Count),
|
||||
|
||||
SupPid = rabbit_ct_broker_helpers:get_queue_sup_pid(QPid),
|
||||
|
|
@ -848,7 +848,7 @@ bq_queue_recover1(Config) ->
|
|||
rabbit_variable_queue:fetch(true, VQ1),
|
||||
CountMinusOne = rabbit_variable_queue:len(VQ2),
|
||||
_VQ3 = rabbit_variable_queue:delete_and_terminate(shutdown, VQ2),
|
||||
ok = rabbit_amqqueue:internal_delete(QName)
|
||||
ok = rabbit_amqqueue:internal_delete(QName, <<"acting-user">>)
|
||||
end),
|
||||
passed.
|
||||
|
||||
|
|
@ -2166,12 +2166,12 @@ change_password1(_Config) ->
|
|||
UserName = <<"test_user">>,
|
||||
Password = <<"test_password">>,
|
||||
case rabbit_auth_backend_internal:lookup_user(UserName) of
|
||||
{ok, _} -> rabbit_auth_backend_internal:delete_user(UserName);
|
||||
{ok, _} -> rabbit_auth_backend_internal:delete_user(UserName, <<"acting-user">>);
|
||||
_ -> ok
|
||||
end,
|
||||
ok = application:set_env(rabbit, password_hashing_module,
|
||||
rabbit_password_hashing_md5),
|
||||
ok = rabbit_auth_backend_internal:add_user(UserName, Password),
|
||||
ok = rabbit_auth_backend_internal:add_user(UserName, Password, <<"acting-user">>),
|
||||
{ok, #auth_user{username = UserName}} =
|
||||
rabbit_auth_backend_internal:user_login_authentication(
|
||||
UserName, [{password, Password}]),
|
||||
|
|
@ -2182,7 +2182,8 @@ change_password1(_Config) ->
|
|||
UserName, [{password, Password}]),
|
||||
|
||||
NewPassword = <<"test_password1">>,
|
||||
ok = rabbit_auth_backend_internal:change_password(UserName, NewPassword),
|
||||
ok = rabbit_auth_backend_internal:change_password(UserName, NewPassword,
|
||||
<<"acting-user">>),
|
||||
{ok, #auth_user{username = UserName}} =
|
||||
rabbit_auth_backend_internal:user_login_authentication(
|
||||
UserName, [{password, NewPassword}]),
|
||||
|
|
@ -3004,14 +3005,14 @@ declare_on_dead_queue1(_Config, SecondaryNode) ->
|
|||
fun () ->
|
||||
{new, #amqqueue{name = QueueName, pid = QPid}} =
|
||||
rabbit_amqqueue:declare(QueueName, false, false, [],
|
||||
none),
|
||||
none, <<"acting-user">>),
|
||||
exit(QPid, kill),
|
||||
Self ! {self(), killed, QPid}
|
||||
end),
|
||||
receive
|
||||
{Pid, killed, OldPid} ->
|
||||
Q = dead_queue_loop(QueueName, OldPid),
|
||||
{ok, 0} = rabbit_amqqueue:delete(Q, false, false),
|
||||
{ok, 0} = rabbit_amqqueue:delete(Q, false, false, <<"acting-user">>),
|
||||
passed
|
||||
after ?TIMEOUT -> throw(failed_to_create_and_kill_queue)
|
||||
end.
|
||||
|
|
@ -3038,9 +3039,9 @@ refresh_events1(Config, SecondaryNode) ->
|
|||
|
||||
{new, #amqqueue{name = QName} = Q} =
|
||||
rabbit_amqqueue:declare(queue_name(Config, <<"refresh_events-q">>),
|
||||
false, false, [], none),
|
||||
false, false, [], none, <<"acting-user">>),
|
||||
expect_events(name, QName, queue_created),
|
||||
rabbit_amqqueue:delete(Q, false, false),
|
||||
rabbit_amqqueue:delete(Q, false, false, <<"acting-user">>),
|
||||
|
||||
dummy_event_receiver:stop(),
|
||||
passed.
|
||||
|
|
@ -3074,7 +3075,8 @@ must_exit(Fun) ->
|
|||
end.
|
||||
|
||||
dead_queue_loop(QueueName, OldPid) ->
|
||||
{existing, Q} = rabbit_amqqueue:declare(QueueName, false, false, [], none),
|
||||
{existing, Q} = rabbit_amqqueue:declare(QueueName, false, false, [], none,
|
||||
<<"acting-user">>),
|
||||
case Q#amqqueue.pid of
|
||||
OldPid -> timer:sleep(25),
|
||||
dead_queue_loop(QueueName, OldPid);
|
||||
|
|
|
|||
Loading…
Reference in New Issue