Merge pull request #13946 from rabbitmq/ik-qq-reconiliation
Trigger a 4.2.x alpha release build / trigger_alpha_build (push) Waiting to run
Details
Test (make) / Build and Xref (1.17, 26) (push) Waiting to run
Details
Test (make) / Build and Xref (1.17, 27) (push) Waiting to run
Details
Test (make) / Test (1.17, 27, khepri) (push) Waiting to run
Details
Test (make) / Test (1.17, 27, mnesia) (push) Waiting to run
Details
Test (make) / Test mixed clusters (1.17, 27, khepri) (push) Waiting to run
Details
Test (make) / Test mixed clusters (1.17, 27, mnesia) (push) Waiting to run
Details
Test (make) / Type check (1.17, 27) (push) Waiting to run
Details
Trigger a 4.2.x alpha release build / trigger_alpha_build (push) Waiting to run
Details
Test (make) / Build and Xref (1.17, 26) (push) Waiting to run
Details
Test (make) / Build and Xref (1.17, 27) (push) Waiting to run
Details
Test (make) / Test (1.17, 27, khepri) (push) Waiting to run
Details
Test (make) / Test (1.17, 27, mnesia) (push) Waiting to run
Details
Test (make) / Test mixed clusters (1.17, 27, khepri) (push) Waiting to run
Details
Test (make) / Test mixed clusters (1.17, 27, mnesia) (push) Waiting to run
Details
Test (make) / Type check (1.17, 27) (push) Waiting to run
Details
QQ member reconciliation: switch to gen_event events instead of hardcoded triggers
This commit is contained in:
commit
7289487702
|
@ -857,7 +857,6 @@ handle_dead_rabbit(Node, State) ->
|
|||
%% statements on *one* node, rather than all of them.
|
||||
ok = rabbit_amqqueue:on_node_down(Node),
|
||||
ok = rabbit_alarm:on_node_down(Node),
|
||||
ok = rabbit_quorum_queue_periodic_membership_reconciliation:on_node_down(Node),
|
||||
State1 = case rabbit_khepri:is_enabled() of
|
||||
true -> State;
|
||||
false -> on_node_down_using_mnesia(Node, State)
|
||||
|
@ -898,8 +897,7 @@ handle_live_rabbit(Node) ->
|
|||
true -> ok;
|
||||
false -> on_node_up_using_mnesia(Node)
|
||||
end,
|
||||
ok = rabbit_vhosts:on_node_up(Node),
|
||||
ok = rabbit_quorum_queue_periodic_membership_reconciliation:on_node_up(Node).
|
||||
ok = rabbit_vhosts:on_node_up(Node).
|
||||
|
||||
on_node_up_using_mnesia(Node) ->
|
||||
ok = rabbit_mnesia:on_node_up(Node).
|
||||
|
|
|
@ -378,13 +378,11 @@ validate(_VHost, <<"operator_policy">>, Name, Term, _User) ->
|
|||
notify(VHost, <<"policy">>, Name, Term0, ActingUser) ->
|
||||
Term = rabbit_data_coercion:atomize_keys(Term0),
|
||||
update_matched_objects(VHost, Term, ActingUser),
|
||||
rabbit_quorum_queue_periodic_membership_reconciliation:policy_set(),
|
||||
rabbit_event:notify(policy_set, [{name, Name}, {vhost, VHost},
|
||||
{user_who_performed_action, ActingUser} | Term]);
|
||||
notify(VHost, <<"operator_policy">>, Name, Term0, ActingUser) ->
|
||||
Term = rabbit_data_coercion:atomize_keys(Term0),
|
||||
update_matched_objects(VHost, Term, ActingUser),
|
||||
rabbit_quorum_queue_periodic_membership_reconciliation:policy_set(),
|
||||
rabbit_event:notify(policy_set, [{name, Name}, {vhost, VHost},
|
||||
{user_who_performed_action, ActingUser} | Term]).
|
||||
|
||||
|
|
|
@ -0,0 +1,51 @@
|
|||
%% This Source Code Form is subject to the terms of the Mozilla Public
|
||||
%% License, v. 2.0. If a copy of the MPL was not distributed with this
|
||||
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
|
||||
%%
|
||||
%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
|
||||
%%
|
||||
|
||||
-module(rabbit_quorum_event_subscriber).
|
||||
|
||||
-behaviour(gen_event).
|
||||
|
||||
-export([init/1, handle_event/2, handle_call/2]).
|
||||
-export([register/0, unregister/0]).
|
||||
|
||||
-include_lib("rabbit_common/include/rabbit.hrl").
|
||||
|
||||
-rabbit_boot_step({rabbit_quorum_event_subscriber,
|
||||
[{description, "quorum queue event subscriber"},
|
||||
{mfa, {?MODULE, register, []}},
|
||||
{cleanup, {?MODULE, unregister, []}},
|
||||
{requires, rabbit_event},
|
||||
{enables, recovery}]}).
|
||||
|
||||
register() ->
|
||||
gen_event:add_handler(rabbit_alarm, ?MODULE, []),
|
||||
gen_event:add_handler(rabbit_event, ?MODULE, []).
|
||||
|
||||
unregister() ->
|
||||
gen_event:delete_handler(rabbit_alarm, ?MODULE, []),
|
||||
gen_event:delete_handler(rabbit_event, ?MODULE, []).
|
||||
|
||||
init([]) ->
|
||||
{ok, []}.
|
||||
|
||||
handle_call( _, State) ->
|
||||
{ok, ok, State}.
|
||||
|
||||
handle_event({node_up, Node}, State) ->
|
||||
rabbit_quorum_queue_periodic_membership_reconciliation:on_node_up(Node),
|
||||
{ok, State};
|
||||
handle_event({node_down, Node}, State) ->
|
||||
rabbit_quorum_queue_periodic_membership_reconciliation:on_node_down(Node),
|
||||
{ok, State};
|
||||
handle_event(#event{type = policy_set}, State) ->
|
||||
rabbit_quorum_queue_periodic_membership_reconciliation:policy_set(),
|
||||
{ok, State};
|
||||
handle_event(#event{type = operator_policy_set}, State) ->
|
||||
rabbit_quorum_queue_periodic_membership_reconciliation:policy_set(),
|
||||
{ok, State};
|
||||
handle_event(_, State) ->
|
||||
{ok, State}.
|
|
@ -19,6 +19,8 @@
|
|||
-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,
|
||||
code_change/3]).
|
||||
|
||||
-include_lib("kernel/include/logger.hrl").
|
||||
|
||||
-define(SERVER, ?MODULE).
|
||||
-define(DEFAULT_INTERVAL, 60_000*60).
|
||||
-define(DEFAULT_TRIGGER_INTERVAL, 10_000).
|
||||
|
@ -91,8 +93,7 @@ handle_cast({membership_reconciliation_trigger, _Reason}, #state{enabled = false
|
|||
{noreply, State, hibernate};
|
||||
handle_cast({membership_reconciliation_trigger, Reason}, #state{timer_ref = OldRef,
|
||||
trigger_interval = Time} = State) ->
|
||||
rabbit_log:debug("Quorum Queue membership reconciliation triggered: ~p",
|
||||
[Reason]),
|
||||
?LOG_DEBUG("Quorum Queue membership reconciliation scheduled: ~p", [Reason]),
|
||||
_ = erlang:cancel_timer(OldRef),
|
||||
Ref = erlang:send_after(Time, self(), ?EVAL_MSG),
|
||||
{noreply, State#state{timer_ref = Ref}};
|
||||
|
@ -158,7 +159,7 @@ reconciliate_quorum_members(ExpectedNodes, Running, [Q | LocalLeaders],
|
|||
end
|
||||
else
|
||||
{timeout, Reason} ->
|
||||
rabbit_log:debug("Find leader timeout: ~p", [Reason]),
|
||||
?LOG_DEBUG("Find leader timeout: ~p", [Reason]),
|
||||
ok;
|
||||
_ ->
|
||||
noop
|
||||
|
@ -184,13 +185,13 @@ maybe_add_member(Q, Running, MemberNodes, TargetSize) ->
|
|||
QName = amqqueue:get_name(Q),
|
||||
case rabbit_quorum_queue:add_member(Q, Node) of
|
||||
ok ->
|
||||
rabbit_log:debug(
|
||||
?LOG_DEBUG(
|
||||
"Added node ~ts as a member to ~ts as "
|
||||
"the queues target group size(#~w) is not met and "
|
||||
"there are enough new nodes(#~w) in the cluster",
|
||||
[Node, rabbit_misc:rs(QName), TargetSize, length(New)]);
|
||||
{error, Err} ->
|
||||
rabbit_log:warning(
|
||||
?LOG_WARNING(
|
||||
"~ts: failed to add member (replica) on node ~w, error: ~w",
|
||||
[rabbit_misc:rs(QName), Node, Err])
|
||||
end,
|
||||
|
@ -235,12 +236,12 @@ remove_members(Q, [Node | Nodes]) ->
|
|||
case rabbit_quorum_queue:delete_member(Q, Node) of
|
||||
ok ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
rabbit_log:debug("~ts: Successfully removed member (replica) on node ~w",
|
||||
?LOG_DEBUG("~ts: Successfully removed member (replica) on node ~w",
|
||||
[rabbit_misc:rs(QName), Node]),
|
||||
ok;
|
||||
{error, Err} ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
rabbit_log:warning("~ts: failed to remove member (replica) on node "
|
||||
?LOG_DEBUG("~ts: failed to remove member (replica) on node "
|
||||
"~w, error: ~w",
|
||||
[rabbit_misc:rs(QName), Node, Err])
|
||||
end,
|
||||
|
|
|
@ -12,17 +12,33 @@
|
|||
-include_lib("amqp_client/include/amqp_client.hrl").
|
||||
-compile([nowarn_export_all, export_all]).
|
||||
|
||||
%% The reconciler has two modes of triggering itself
|
||||
%% - timer based
|
||||
%% - event based
|
||||
%% The default config of this test has Interval very short - 5 second which is lower than
|
||||
%% wait_until timeout. Meaninig that even if all domain triggers (node_up/down, policy_set, etc)
|
||||
%% are disconnected tests would be still green.
|
||||
%% So to test triggers it is essential to set Interval high enough (the very default value of 60 minutes is perfect)
|
||||
%%
|
||||
%% TODO: test `policy_set` trigger
|
||||
|
||||
all() ->
|
||||
[
|
||||
{group, unclustered}
|
||||
{group, unclustered},
|
||||
{group, unclustered_triggers}
|
||||
].
|
||||
|
||||
groups() ->
|
||||
[
|
||||
{unclustered, [],
|
||||
{unclustered, [], %% low interval, even if triggers do not work all tests should pass
|
||||
[
|
||||
{quorum_queue_3, [], [auto_grow, auto_grow_drained_node, auto_shrink]}
|
||||
]},
|
||||
%% uses an interval longer than `wait_until` (30s by default)
|
||||
{unclustered_triggers, [],
|
||||
[
|
||||
%% see also `auto_grow_drained_node`
|
||||
{quorum_queue_3, [], [auto_grow, auto_shrink]}
|
||||
]}
|
||||
].
|
||||
|
||||
|
@ -30,8 +46,14 @@ groups() ->
|
|||
%% Testsuite setup/teardown.
|
||||
%% -------------------------------------------------------------------
|
||||
|
||||
init_per_suite(Config0) ->
|
||||
init_per_suite(Config) ->
|
||||
rabbit_ct_helpers:log_environment(),
|
||||
rabbit_ct_helpers:run_setup_steps(Config, []).
|
||||
|
||||
end_per_suite(Config) ->
|
||||
rabbit_ct_helpers:run_teardown_steps(Config).
|
||||
|
||||
init_per_group(unclustered, Config0) ->
|
||||
Config1 = rabbit_ct_helpers:merge_app_env(
|
||||
Config0, {rabbit, [{quorum_tick_interval, 1000},
|
||||
{quorum_membership_reconciliation_enabled, true},
|
||||
|
@ -39,12 +61,22 @@ init_per_suite(Config0) ->
|
|||
{quorum_membership_reconciliation_interval, 5000},
|
||||
{quorum_membership_reconciliation_trigger_interval, 2000},
|
||||
{quorum_membership_reconciliation_target_group_size, 3}]}),
|
||||
rabbit_ct_helpers:run_setup_steps(Config1, []).
|
||||
|
||||
end_per_suite(Config) ->
|
||||
rabbit_ct_helpers:run_teardown_steps(Config).
|
||||
init_per_group(unclustered, Config) ->
|
||||
rabbit_ct_helpers:set_config(Config, [{rmq_nodes_clustered, false}]);
|
||||
rabbit_ct_helpers:set_config(Config1, [{rmq_nodes_clustered, false}]);
|
||||
init_per_group(unclustered_triggers, Config0) ->
|
||||
Config1 = rabbit_ct_helpers:merge_app_env(
|
||||
Config0, {rabbit, [{quorum_tick_interval, 1000},
|
||||
{quorum_membership_reconciliation_enabled, true},
|
||||
{quorum_membership_reconciliation_auto_remove, true},
|
||||
{quorum_membership_reconciliation_interval, 50000},
|
||||
{quorum_membership_reconciliation_trigger_interval, 2000},
|
||||
{quorum_membership_reconciliation_target_group_size, 3}]}),
|
||||
%% shrink timeout is set here because without it, when a node stopped right after a queue was created,
|
||||
%% the test will pass without any triggers because cluster change will likely happen before the trigger_interval,
|
||||
%% scheduled in response to queue_created event.
|
||||
%% See also a comment in `auto_shrink/1`.
|
||||
rabbit_ct_helpers:set_config(Config1, [{rmq_nodes_clustered, false},
|
||||
{quorum_membership_reconciliation_interval, 50000},
|
||||
{shrink_timeout, 2000}]);
|
||||
init_per_group(Group, Config) ->
|
||||
ClusterSize = 3,
|
||||
Config1 = rabbit_ct_helpers:set_config(Config,
|
||||
|
@ -57,6 +89,8 @@ init_per_group(Group, Config) ->
|
|||
|
||||
end_per_group(unclustered, Config) ->
|
||||
Config;
|
||||
end_per_group(unclustered_triggers, Config) ->
|
||||
Config;
|
||||
end_per_group(_, Config) ->
|
||||
rabbit_ct_helpers:run_steps(Config,
|
||||
rabbit_ct_broker_helpers:teardown_steps()).
|
||||
|
@ -72,34 +106,17 @@ init_per_testcase(Testcase, Config) ->
|
|||
]),
|
||||
rabbit_ct_helpers:run_steps(Config2, rabbit_ct_client_helpers:setup_steps()).
|
||||
|
||||
merge_app_env(Config) ->
|
||||
rabbit_ct_helpers:merge_app_env(
|
||||
rabbit_ct_helpers:merge_app_env(Config,
|
||||
{rabbit, [{core_metrics_gc_interval, 100}]}),
|
||||
{ra, [{min_wal_roll_over_interval, 30000}]}).
|
||||
|
||||
end_per_testcase(Testcase, Config) ->
|
||||
[Server0, Server1, Server2] =
|
||||
rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
|
||||
Ch = rabbit_ct_client_helpers:open_channel(Config, Server1),
|
||||
amqp_channel:call(Ch, #'queue.delete'{queue = rabbit_data_coercion:to_binary(Testcase)}),
|
||||
reset_nodes([Server2, Server0], Server1),
|
||||
Config1 = rabbit_ct_helpers:run_steps(
|
||||
Config,
|
||||
rabbit_ct_client_helpers:teardown_steps()),
|
||||
rabbit_ct_helpers:testcase_finished(Config1, Testcase).
|
||||
|
||||
reset_nodes([], _Leader) ->
|
||||
ok;
|
||||
reset_nodes([Node| Nodes], Leader) ->
|
||||
ok = rabbit_control_helper:command(stop_app, Node),
|
||||
case rabbit_control_helper:command(forget_cluster_node, Leader, [atom_to_list(Node)]) of
|
||||
ok -> ok;
|
||||
{error, _, <<"Error:\n{:not_a_cluster_node, ~c\"The node selected is not in the cluster.\"}">>} -> ok
|
||||
end,
|
||||
ok = rabbit_control_helper:command(reset, Node),
|
||||
ok = rabbit_control_helper:command(start_app, Node),
|
||||
reset_nodes(Nodes, Leader).
|
||||
|
||||
|
||||
%% -------------------------------------------------------------------
|
||||
%% Testcases.
|
||||
%% -------------------------------------------------------------------
|
||||
|
@ -134,6 +151,10 @@ auto_grow(Config) ->
|
|||
end).
|
||||
|
||||
auto_grow_drained_node(Config) ->
|
||||
%% NOTE: with large Interval (larger than wait_until) test will fail.
|
||||
%% the reason is that entering/exiting drain state does not emit events
|
||||
%% and even if they did via gen_event, they going to be only local to that node.
|
||||
%% so reconciliator has no choice but to wait full Interval
|
||||
[Server0, Server1, Server2] =
|
||||
rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
|
||||
Ch = rabbit_ct_client_helpers:open_channel(Config, Server1),
|
||||
|
@ -169,7 +190,6 @@ auto_grow_drained_node(Config) ->
|
|||
3 =:= length(M)
|
||||
end).
|
||||
|
||||
|
||||
auto_shrink(Config) ->
|
||||
[Server0, Server1, Server2] =
|
||||
rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
|
||||
|
@ -186,6 +206,18 @@ auto_shrink(Config) ->
|
|||
Server1}),
|
||||
3 =:= length(M)
|
||||
end),
|
||||
|
||||
%% QQ member reconciliation does not act immediately but rather after a scheduled delay.
|
||||
%% So if this test wants to test that the reconciliator reacts to, say, node_down or a similar event,
|
||||
%% it has to wait at least a trigger_interval ms to pass before removing node. Otherwise
|
||||
%% the shrink effect would come from the previous trigger.
|
||||
%%
|
||||
%% When a `queue_created` trigger set up a timer to fire after a trigger_interval, the queue has 3 members
|
||||
%% and stop_app executes much quicker than the trigger_interval. Therefore the number of members
|
||||
%% will be updated even without a node_down event.
|
||||
|
||||
timer:sleep(rabbit_ct_helpers:get_config(Config, shrink_timeout, 0)),
|
||||
|
||||
ok = rabbit_control_helper:command(stop_app, Server2),
|
||||
ok = rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_db_cluster, forget_member,
|
||||
[Server2, false]),
|
||||
|
@ -196,7 +228,27 @@ auto_shrink(Config) ->
|
|||
2 =:= length(M)
|
||||
end).
|
||||
|
||||
%% -------------------------------------------------------------------
|
||||
%% Helpers.
|
||||
%% -------------------------------------------------------------------
|
||||
|
||||
merge_app_env(Config) ->
|
||||
rabbit_ct_helpers:merge_app_env(
|
||||
rabbit_ct_helpers:merge_app_env(Config,
|
||||
{rabbit, [{core_metrics_gc_interval, 100}]}),
|
||||
{ra, [{min_wal_roll_over_interval, 30000}]}).
|
||||
|
||||
reset_nodes([], _Leader) ->
|
||||
ok;
|
||||
reset_nodes([Node| Nodes], Leader) ->
|
||||
ok = rabbit_control_helper:command(stop_app, Node),
|
||||
case rabbit_control_helper:command(forget_cluster_node, Leader, [atom_to_list(Node)]) of
|
||||
ok -> ok;
|
||||
{error, _, <<"Error:\n{:not_a_cluster_node, ~c\"The node selected is not in the cluster.\"}">>} -> ok
|
||||
end,
|
||||
ok = rabbit_control_helper:command(reset, Node),
|
||||
ok = rabbit_control_helper:command(start_app, Node),
|
||||
reset_nodes(Nodes, Leader).
|
||||
|
||||
add_server_to_cluster(Server, Leader) ->
|
||||
ok = rabbit_control_helper:command(stop_app, Server),
|
||||
|
|
Loading…
Reference in New Issue