Skip to content

Commit 132d594

Browse files
authored
Merge pull request #7289 from rabbitmq/mergify/bp/v3.10.x/pr-7282
rabbit_vhost:set_tags/2 avoids notifying if tags are unchanged (backport #7280) (backport #7282)
2 parents 169d5c7 + 53597fb commit 132d594

File tree

5 files changed

+138
-11
lines changed

5 files changed

+138
-11
lines changed

deps/rabbit/BUILD.bazel

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1033,6 +1033,9 @@ rabbitmq_integration_suite(
10331033
name = "vhost_SUITE",
10341034
size = "medium",
10351035
flaky = True,
1036+
additional_srcs = [
1037+
"test/test_rabbit_event_handler.erl",
1038+
],
10361039
)
10371040

10381041
rabbitmq_suite(

deps/rabbit/src/rabbit_vhost.erl

Lines changed: 31 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -501,33 +501,54 @@ update_metadata(VHostName, Fun) ->
501501
vhost:set_metadata(Record, Meta)
502502
end).
503503

504-
-spec update_tags(vhost:name(), [vhost_tag()], rabbit_types:username()) -> vhost:vhost() | rabbit_types:ok_or_error(any()).
504+
are_different0([], []) ->
505+
false;
506+
are_different0([], [_ | _]) ->
507+
true;
508+
are_different0([_ | _], []) ->
509+
true;
510+
are_different0([E], [E]) ->
511+
false;
512+
are_different0([E | R1], [E | R2]) ->
513+
are_different0(R1, R2);
514+
are_different0(_, _) ->
515+
true.
516+
517+
are_different(L1, L2) ->
518+
are_different0(lists:usort(L1), lists:usort(L2)).
519+
520+
-spec update_tags(vhost:name(), [vhost_tag()], rabbit_types:username()) -> vhost:vhost().
505521
update_tags(VHostName, Tags, ActingUser) ->
506-
ConvertedTags = [rabbit_data_coercion:to_atom(I) || I <- Tags],
522+
CurrentTags = case mnesia:dirty_read({rabbit_vhost, VHostName}) of
523+
[V] when ?is_vhost(V) -> vhost:get_tags(V);
524+
[] -> []
525+
end,
526+
ConvertedTags = lists:usort([rabbit_data_coercion:to_atom(I) || I <- Tags]),
507527
try
508528
R = rabbit_misc:execute_mnesia_transaction(fun() ->
509529
update_tags(VHostName, ConvertedTags)
510530
end),
511-
rabbit_log:info("Successfully set tags for virtual host '~s' to ~p", [VHostName, ConvertedTags]),
512-
rabbit_event:notify(vhost_tags_set, [{name, VHostName},
513-
{tags, ConvertedTags},
514-
{user_who_performed_action, ActingUser}]),
531+
rabbit_log:info("Successfully set tags for virtual host '~ts' to ~tp", [VHostName, ConvertedTags]),
532+
rabbit_event:notify_if(are_different(CurrentTags, ConvertedTags),
533+
vhost_tags_set, [{name, VHostName},
534+
{tags, ConvertedTags},
535+
{user_who_performed_action, ActingUser}]),
515536
R
516537
catch
517538
throw:{error, {no_such_vhost, _}} = Error ->
518-
rabbit_log:warning("Failed to set tags for virtual host '~s': the virtual host does not exist", [VHostName]),
539+
rabbit_log:warning("Failed to set tags for virtual host '~ts': the virtual host does not exist", [VHostName]),
519540
throw(Error);
520541
throw:Error ->
521-
rabbit_log:warning("Failed to set tags for virtual host '~s': ~p", [VHostName, Error]),
542+
rabbit_log:warning("Failed to set tags for virtual host '~ts': ~tp", [VHostName, Error]),
522543
throw(Error);
523544
exit:Error ->
524-
rabbit_log:warning("Failed to set tags for virtual host '~s': ~p", [VHostName, Error]),
545+
rabbit_log:warning("Failed to set tags for virtual host '~ts': ~tp", [VHostName, Error]),
525546
exit(Error)
526547
end.
527548

528549
-spec update_tags(vhost:name(), [vhost_tag()]) -> vhost:vhost() | rabbit_types:ok_or_error(any()).
529550
update_tags(VHostName, Tags) ->
530-
ConvertedTags = [rabbit_data_coercion:to_atom(I) || I <- Tags],
551+
ConvertedTags = lists:usort([rabbit_data_coercion:to_atom(I) || I <- Tags]),
531552
update(VHostName, fun(Record) ->
532553
Meta0 = vhost:get_metadata(Record),
533554
Meta = maps:put(tags, ConvertedTags, Meta0),
Lines changed: 33 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,33 @@
1+
-module(test_rabbit_event_handler).
2+
3+
-behaviour(gen_event).
4+
5+
-export([okay/0]).
6+
-export([init/1, handle_call/2, handle_event/2, handle_info/2,
7+
terminate/2, code_change/3]).
8+
9+
-include_lib("rabbit_common/include/rabbit.hrl").
10+
11+
% an exported callable func, used to allow rabbit_ct_broker_helpers
12+
% to load this code when rpc'd
13+
okay() -> ok.
14+
15+
init([]) ->
16+
{ok, #{events => []}}.
17+
18+
handle_event(#event{} = Event, #{events := Events} = State) ->
19+
{ok, State#{events => [Event | Events]}};
20+
handle_event(_, State) ->
21+
{ok, State}.
22+
23+
handle_call(events, #{events := Events} = State) ->
24+
{ok, Events, State}.
25+
26+
handle_info(_Info, State) ->
27+
{ok, State}.
28+
29+
terminate(_Arg, _State) ->
30+
ok.
31+
32+
code_change(_OldVsn, State, _Extra) ->
33+
{ok, State}.

deps/rabbit/test/vhost_SUITE.erl

Lines changed: 70 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ groups() ->
2626
single_node_vhost_deletion_forces_connection_closure,
2727
vhost_failure_forces_connection_closure,
2828
vhost_creation_idempotency,
29+
vhost_update_idempotency,
2930
parse_tags
3031
],
3132
ClusterSize2Tests = [
@@ -92,6 +93,13 @@ end_per_group(_Group, Config) ->
9293
rabbit_ct_client_helpers:teardown_steps() ++
9394
rabbit_ct_broker_helpers:teardown_steps()).
9495

96+
init_per_testcase(vhost_update_idempotency = Testcase, Config) ->
97+
rabbit_ct_helpers:testcase_started(Config, Testcase),
98+
clear_all_connection_tracking_tables(Config),
99+
case rabbit_ct_broker_helpers:enable_feature_flag(Config, virtual_host_metadata) of
100+
ok -> Config;
101+
Skip -> Skip
102+
end;
95103
init_per_testcase(Testcase, Config) ->
96104
rabbit_ct_helpers:testcase_started(Config, Testcase),
97105
clear_all_connection_tracking_tables(Config),
@@ -319,6 +327,68 @@ vhost_creation_idempotency(Config) ->
319327
rabbit_ct_broker_helpers:delete_vhost(Config, VHost)
320328
end.
321329

330+
vhost_update_idempotency(Config) ->
331+
VHost = <<"update-idempotency-test">>,
332+
ActingUser = <<"acting-user">>,
333+
try
334+
% load the dummy event handler on the node
335+
ok = rabbit_ct_broker_helpers:rpc(Config, 0, test_rabbit_event_handler, okay, []),
336+
337+
ok = rabbit_ct_broker_helpers:rpc(Config, 0, gen_event, add_handler,
338+
[rabbit_event, test_rabbit_event_handler, []]),
339+
340+
?assertEqual(ok, rabbit_ct_broker_helpers:add_vhost(Config, VHost)),
341+
342+
?assertMatch({vhost,VHost, _, #{tags := [private,replicate]}},
343+
rabbit_ct_broker_helpers:rpc(Config, 0,
344+
rabbit_vhost, update_tags,
345+
[VHost, [private, replicate], ActingUser])),
346+
?assertMatch({vhost,VHost, _, #{tags := [private,replicate]}},
347+
rabbit_ct_broker_helpers:rpc(Config, 0,
348+
rabbit_vhost, update_tags,
349+
[VHost, [replicate, private], ActingUser])),
350+
351+
Events = rabbit_ct_broker_helpers:rpc(Config, 0,
352+
gen_event, call,
353+
[rabbit_event, test_rabbit_event_handler, events, 100]),
354+
ct:pal(?LOW_IMPORTANCE, "Events: ~p", [lists:reverse(Events)]),
355+
TagsSetEvents = lists:filter(fun
356+
(#event{type = vhost_tags_set}) -> true;
357+
(_) -> false
358+
end, Events),
359+
?assertMatch([#event{type = vhost_tags_set,
360+
props = [{name, VHost},
361+
{tags, [private, replicate]},
362+
{user_who_performed_action, ActingUser}]}],
363+
TagsSetEvents)
364+
after
365+
rabbit_ct_broker_helpers:rpc(Config, 0,
366+
gen_event, delete_handler, [rabbit_event, test_rabbit_event_handler, []]),
367+
rabbit_ct_broker_helpers:delete_vhost(Config, VHost)
368+
end.
369+
370+
vhost_is_created_with_default_limits(Config) ->
371+
VHost = <<"vhost1">>,
372+
Limits = [{<<"max-connections">>, 10}, {<<"max-queues">>, 1}],
373+
Pattern = [{<<"pattern">>, ".*"}],
374+
Env = [{vhosts, [{<<"id">>, Limits++Pattern}]}],
375+
?assertEqual(ok, rabbit_ct_broker_helpers:rpc(Config, 0,
376+
application, set_env, [rabbit, default_limits, Env])),
377+
?assertEqual(ok, rabbit_ct_broker_helpers:add_vhost(Config, VHost)),
378+
?assertEqual(Limits, rabbit_ct_broker_helpers:rpc(Config, 0,
379+
rabbit_vhost_limit, list, [VHost])).
380+
381+
vhost_is_created_with_operator_policies(Config) ->
382+
VHost = <<"vhost1">>,
383+
PolicyName = <<"default-operator-policy">>,
384+
Definition = [{<<"expires">>, 10}],
385+
Env = [{operator, [{PolicyName, Definition}]}],
386+
?assertEqual(ok, rabbit_ct_broker_helpers:rpc(Config, 0,
387+
application, set_env, [rabbit, default_policies, Env])),
388+
?assertEqual(ok, rabbit_ct_broker_helpers:add_vhost(Config, VHost)),
389+
?assertNotEqual(not_found, rabbit_ct_broker_helpers:rpc(Config, 0,
390+
rabbit_policy, lookup_op, [VHost, PolicyName])).
391+
322392
parse_tags(Config) ->
323393
rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, parse_tags1, [Config]).
324394

deps/rabbitmq_cli/test/ctl/set_vhost_tags_command_test.exs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -77,7 +77,7 @@ defmodule SetVhostTagsCommandTest do
7777
fn record -> record[:vhost] == context[:vhost] end
7878
)
7979

80-
assert result[:tags] == context[:tags]
80+
assert Enum.sort(result[:tags]) == Enum.sort(context[:tags])
8181
end
8282

8383
@tag user: @vhost, tags: [:qa]

0 commit comments

Comments
 (0)