From d1aab61566bd7394323956f35ba9bc0b9ffc29ee Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Arnaud=20Cogolu=C3=A8gnes?= <514737+acogoluegnes@users.noreply.github.com> Date: Tue, 10 Jun 2025 12:01:18 +0200 Subject: [PATCH 1/4] Prevent blocked groups in stream SAC with fine-grained status A boolean status in the stream SAC coordinator is not enough to follow the evolution of a consumer. For example a former active consumer that is stepping down can go down before another consumer in the group is activated, letting the coordinator expect an activation request that will never arrive, leaving the group without any active consumer. This commit introduces 3 status: active (formerly "true"), waiting (formerly "false"), and deactivating. The coordinator will now know when a deactivating consumer goes down and will trigger a rebalancing to avoid a stuck group. This commit also introduces a status related to the connectivity state of a consumer. The possible values are: connected, disconnected, and presumed_down. Consumers are by default connected, they can become disconnected if the coordinator receives a down event with a noconnection reason, meaning the node of the consumer has been disconnected from the other nodes. Consumers can become connected again when their node joins the other nodes again. Disconnected consumers are still considered part of a group, as they are expected to come back at some point. For example there is no rebalancing in a group if the active consumer got disconnected. The coordinator sets a timer when a disconnection occurs. When the timer expires, corresponding disconnected consumers pass into the "presumed down" state. At this point they are no longer considered part of their respective group and are excluded from rebalancing decision. They are expected to get removed from the group by the appropriate down event of a monitor. So the consumer status is now a tuple, e.g. {connected, active}. Note this is an implementation detail: only the stream SAC coordinator deals with the status of stream SAC consumers. 2 new configuration entries are introduced: * rabbit.stream_sac_disconnected_timeout: this is the duration in ms of the disconnected-to-forgotten timer. * rabbit.stream_cmd_timeout: this is the timeout in ms to apply RA commands in the coordinator. It used to be a fixed value of 30 seconds. The default value is still the same. The setting has been introduced to make integration tests faster. Fixes #14070 --- deps/rabbit/Makefile | 2 +- deps/rabbit/ct.test.spec | 1 + deps/rabbit/src/rabbit_stream_coordinator.erl | 275 ++- deps/rabbit/src/rabbit_stream_coordinator.hrl | 1 + .../src/rabbit_stream_sac_coordinator.erl | 1068 ++++++++--- .../src/rabbit_stream_sac_coordinator.hrl | 28 +- .../src/rabbit_stream_sac_coordinator_v4.erl | 774 ++++++++ .../src/rabbit_stream_sac_coordinator_v4.hrl | 58 + .../test/rabbit_stream_coordinator_SUITE.erl | 2 +- .../rabbit_stream_sac_coordinator_SUITE.erl | 1634 ++++++++++++++--- ...rabbit_stream_sac_coordinator_v4_SUITE.erl | 593 ++++++ .../src/stream_test_utils.erl | 11 +- .../docs/stream_coordinator.md | 77 + .../src/rabbit_stream_reader.erl | 3 + deps/rabbitmq_stream/test/commands_SUITE.erl | 40 +- .../test/rabbit_stream_SUITE.erl | 45 +- .../test/rabbit_stream_partitions_SUITE.erl | 786 ++++++++ 17 files changed, 4805 insertions(+), 593 deletions(-) create mode 100644 deps/rabbit/src/rabbit_stream_sac_coordinator_v4.erl create mode 100644 deps/rabbit/src/rabbit_stream_sac_coordinator_v4.hrl create mode 100644 deps/rabbit/test/rabbit_stream_sac_coordinator_v4_SUITE.erl create mode 100644 deps/rabbitmq_stream/docs/stream_coordinator.md create mode 100644 deps/rabbitmq_stream/test/rabbit_stream_partitions_SUITE.erl diff --git a/deps/rabbit/Makefile b/deps/rabbit/Makefile index dec23f4b1f5c..0a786304751c 100644 --- a/deps/rabbit/Makefile +++ b/deps/rabbit/Makefile @@ -267,7 +267,7 @@ PARALLEL_CT_SET_2_B = clustering_recovery crashing_queues deprecated_features di PARALLEL_CT_SET_2_C = disk_monitor dynamic_qq unit_disk_monitor unit_file_handle_cache unit_log_management unit_operator_policy PARALLEL_CT_SET_2_D = queue_length_limits queue_parallel quorum_queue_member_reconciliation rabbit_fifo rabbit_fifo_dlx rabbit_stream_coordinator -PARALLEL_CT_SET_3_A = definition_import per_user_connection_channel_limit_partitions per_vhost_connection_limit_partitions policy priority_queue_recovery rabbit_fifo_v0 rabbit_stream_sac_coordinator unit_credit_flow unit_queue_consumers unit_queue_location unit_quorum_queue +PARALLEL_CT_SET_3_A = definition_import per_user_connection_channel_limit_partitions per_vhost_connection_limit_partitions policy priority_queue_recovery rabbit_fifo_v0 rabbit_stream_sac_coordinator_v4 rabbit_stream_sac_coordinator unit_credit_flow unit_queue_consumers unit_queue_location unit_quorum_queue PARALLEL_CT_SET_3_B = cluster_upgrade list_consumers_sanity_check list_queues_online_and_offline logging lqueue maintenance_mode rabbit_fifo_q PARALLEL_CT_SET_3_C = cli_forget_cluster_node feature_flags_v2 mc_unit message_containers_deaths_v2 message_size_limit metadata_store_migration PARALLEL_CT_SET_3_D = metadata_store_phase1 metrics mirrored_supervisor peer_discovery_classic_config proxy_protocol runtime_parameters unit_stats_and_metrics unit_supervisor2 unit_vm_memory_monitor diff --git a/deps/rabbit/ct.test.spec b/deps/rabbit/ct.test.spec index 104f7f40bfda..1056fa164051 100644 --- a/deps/rabbit/ct.test.spec +++ b/deps/rabbit/ct.test.spec @@ -117,6 +117,7 @@ , rabbit_local_random_exchange_SUITE , rabbit_msg_interceptor_SUITE , rabbit_stream_coordinator_SUITE +, rabbit_stream_sac_coordinator_v4_SUITE , rabbit_stream_sac_coordinator_SUITE , rabbitmq_4_0_deprecations_SUITE , rabbitmq_queues_cli_integration_SUITE diff --git a/deps/rabbit/src/rabbit_stream_coordinator.erl b/deps/rabbit/src/rabbit_stream_coordinator.erl index d601918c4a4d..f7d26d014ba6 100644 --- a/deps/rabbit/src/rabbit_stream_coordinator.erl +++ b/deps/rabbit/src/rabbit_stream_coordinator.erl @@ -15,7 +15,7 @@ apply/3, state_enter/2, init_aux/1, - handle_aux/6, + handle_aux/5, tick/2, version/0, which_module/1, @@ -31,8 +31,7 @@ transfer_leadership/1, forget_node/1, status/0, - member_overview/0 - ]). + member_overview/0]). %% stream API -export([new_stream/2, @@ -42,8 +41,7 @@ add_replica/2, delete_replica/2, register_listener/1, - register_local_member_listener/1 - ]). + register_local_member_listener/1]). -export([local_pid/1, writer_pid/1, @@ -57,10 +55,8 @@ query_stream_overview/2, ra_local_query/1]). - -export([log_overview/1, - key_metrics_rpc/1 - ]). + key_metrics_rpc/1]). %% for SAC coordinator -export([sac_state/1]). @@ -68,11 +64,10 @@ %% for testing and debugging -export([eval_listeners/3, replay/1, - state/0]). + state/0, + sac_state/0]). --import(rabbit_queue_type_util, [ - erpc_call/5 - ]). +-import(rabbit_queue_type_util, [erpc_call/5]). -rabbit_boot_step({?MODULE, [{description, "Restart stream coordinator"}, @@ -90,6 +85,10 @@ -include("amqqueue.hrl"). -define(REPLICA_FRESHNESS_LIMIT_MS, 10 * 1000). %% 10s +-define(V2_OR_MORE(Vsn), Vsn >= 2). +-define(V5_OR_MORE(Vsn), Vsn >= 5). +-define(SAC_V4, rabbit_stream_sac_coordinator_v4). +-define(SAC_CURRENT, rabbit_stream_sac_coordinator). -type state() :: #?MODULE{}. -type args() :: #{index := ra:index(), @@ -119,7 +118,8 @@ {retention_updated, stream_id(), args()} | {mnesia_updated, stream_id(), args()} | {sac, rabbit_stream_sac_coordinator:command()} | - ra_machine:effect(). + {machine_version, ra_machine:version(), ra_machine:version()} | + ra_machine:builtin_command(). -export_type([command/0]). @@ -278,6 +278,16 @@ state() -> Any end. +%% for debugging +sac_state() -> + case state() of + S when is_record(S, ?MODULE) -> + sac_state(S); + R -> + R + end. + + writer_pid(StreamId) when is_list(StreamId) -> MFA = {?MODULE, query_writer_pid, [StreamId]}, query_pid(StreamId, MFA). @@ -426,10 +436,16 @@ process_command(Cmd) -> process_command([], _Cmd) -> {error, coordinator_unavailable}; process_command([Server | Servers], Cmd) -> - case ra:process_command(Server, Cmd, ?CMD_TIMEOUT) of + case ra:process_command(Server, Cmd, cmd_timeout()) of {timeout, _} -> + CmdLabel = case Cmd of + {sac, SacCmd} -> + element(1, SacCmd); + _ -> + element(1, Cmd) + end, rabbit_log:warning("Coordinator timeout on server ~w when processing command ~W", - [element(2, Server), element(1, Cmd), 10]), + [element(2, Server), CmdLabel, 10]), process_command(Servers, Cmd); {error, noproc} -> process_command(Servers, Cmd); @@ -439,6 +455,9 @@ process_command([Server | Servers], Cmd) -> Reply end. +cmd_timeout() -> + application:get_env(rabbit, stream_cmd_timeout, ?CMD_TIMEOUT). + ensure_coordinator_started() -> Local = {?MODULE, node()}, ExpectedMembers = expected_coord_members(), @@ -520,13 +539,16 @@ reachable_coord_members() -> Nodes = rabbit_nodes:list_reachable(), [{?MODULE, Node} || Node <- Nodes]. -version() -> 4. +version() -> 5. which_module(_) -> ?MODULE. -init(_Conf) -> - #?MODULE{single_active_consumer = rabbit_stream_sac_coordinator:init_state()}. +init(#{machine_version := Vsn}) when ?V5_OR_MORE(Vsn) -> + #?MODULE{single_active_consumer = + rabbit_stream_sac_coordinator:init_state()}; +init(_) -> + #?MODULE{single_active_consumer = rabbit_stream_sac_coordinator_v4:init_state()}. -spec apply(ra_machine:command_meta_data(), command(), state()) -> {state(), term(), ra_machine:effects()}. @@ -564,12 +586,13 @@ apply(#{index := _Idx, machine_version := MachineVersion} = Meta0, end; apply(Meta, {sac, SacCommand}, #?MODULE{single_active_consumer = SacState0, monitors = Monitors0} = State0) -> - {SacState1, Reply, Effects0} = rabbit_stream_sac_coordinator:apply(SacCommand, SacState0), + Mod = sac_module(Meta), + {SacState1, Reply, Effects0} = Mod:apply(SacCommand, SacState0), {SacState2, Monitors1, Effects1} = - rabbit_stream_sac_coordinator:ensure_monitors(SacCommand, SacState1, Monitors0, Effects0), + Mod:ensure_monitors(SacCommand, SacState1, Monitors0, Effects0), return(Meta, State0#?MODULE{single_active_consumer = SacState2, - monitors = Monitors1}, Reply, Effects1); -apply(#{machine_version := MachineVersion} = Meta, {down, Pid, Reason} = Cmd, + monitors = Monitors1}, Reply, Effects1); +apply(#{machine_version := Vsn} = Meta, {down, Pid, Reason} = Cmd, #?MODULE{streams = Streams0, monitors = Monitors0, listeners = StateListeners0, @@ -581,7 +604,7 @@ apply(#{machine_version := MachineVersion} = Meta, {down, Pid, Reason} = Cmd, [] end, case maps:take(Pid, Monitors0) of - {{StreamId, listener}, Monitors} when MachineVersion < 2 -> + {{StreamId, listener}, Monitors} when Vsn < 2 -> Listeners = case maps:take(StreamId, StateListeners0) of error -> StateListeners0; @@ -595,7 +618,7 @@ apply(#{machine_version := MachineVersion} = Meta, {down, Pid, Reason} = Cmd, end, return(Meta, State#?MODULE{listeners = Listeners, monitors = Monitors}, ok, Effects0); - {{PidStreams, listener}, Monitors} when MachineVersion >= 2 -> + {{PidStreams, listener}, Monitors} when ?V2_OR_MORE(Vsn) -> Streams = maps:fold( fun(StreamId, _, Acc) -> case Acc of @@ -629,9 +652,11 @@ apply(#{machine_version := MachineVersion} = Meta, {down, Pid, Reason} = Cmd, monitors = Monitors1}, ok, Effects0) end; {sac, Monitors1} -> - {SacState1, Effects} = rabbit_stream_sac_coordinator:handle_connection_down(Pid, SacState0), + {SacState1, SacEffects} = sac_handle_connection_down(SacState0, Pid, + Reason, Vsn), return(Meta, State#?MODULE{single_active_consumer = SacState1, - monitors = Monitors1}, ok, Effects); + monitors = Monitors1}, + ok, [Effects0 ++ SacEffects]); error -> return(Meta, State, ok, Effects0) end; @@ -657,11 +682,11 @@ apply(#{machine_version := MachineVersion} = Meta, return(Meta, State0, stream_not_found, []) end; -apply(#{machine_version := MachineVersion} = Meta, +apply(#{machine_version := Vsn} = Meta, {register_listener, #{pid := Pid, stream_id := StreamId} = Args}, #?MODULE{streams = Streams, - monitors = Monitors0} = State0) when MachineVersion >= 2 -> + monitors = Monitors0} = State0) when ?V2_OR_MORE(Vsn) -> Node = maps:get(node, Args, node(Pid)), Type = maps:get(type, Args, leader), @@ -685,9 +710,11 @@ apply(#{machine_version := MachineVersion} = Meta, _ -> return(Meta, State0, stream_not_found, []) end; -apply(Meta, {nodeup, Node} = Cmd, +apply(#{machine_version := Vsn} = Meta, + {nodeup, Node} = Cmd, #?MODULE{monitors = Monitors0, - streams = Streams0} = State) -> + streams = Streams0, + single_active_consumer = Sac0} = State) -> %% reissue monitors for all disconnected members {Effects0, Monitors} = maps:fold( @@ -701,14 +728,24 @@ apply(Meta, {nodeup, Node} = Cmd, {Acc, Mon} end end, {[], Monitors0}, Streams0), - {Streams, Effects} = + {Streams, Effects1} = maps:fold(fun (Id, S0, {Ss, E0}) -> S1 = update_stream(Meta, Cmd, S0), {S, E} = evaluate_stream(Meta, S1, E0), {Ss#{Id => S}, E} end, {Streams0, Effects0}, Streams0), + + {Sac1, Effects2} = case ?V5_OR_MORE(Vsn) of + true -> + SacMod = sac_module(Meta), + SacMod:handle_node_reconnected(Node, + Sac0, Effects1); + false -> + {Sac0, Effects1} + end, return(Meta, State#?MODULE{monitors = Monitors, - streams = Streams}, ok, Effects); + streams = Streams, + single_active_consumer = Sac1}, ok, Effects2); apply(Meta, {machine_version, From, To}, State0) -> rabbit_log:info("Stream coordinator machine version changes from ~tp to ~tp, " ++ "applying incremental upgrade.", [From, To]), @@ -719,6 +756,12 @@ apply(Meta, {machine_version, From, To}, State0) -> {S1, Eff0 ++ Eff1} end, {State0, []}, lists:seq(From, To - 1)), return(Meta, State1, ok, Effects); +apply(Meta, {timeout, {sac, node_disconnected, #{connection_pid := Pid}}}, + #?MODULE{single_active_consumer = SacState0} = State0) -> + Mod = sac_module(Meta), + {SacState1, Effects} = Mod:presume_connection_down(Pid, SacState0), + return(Meta, State0#?MODULE{single_active_consumer = SacState1}, ok, + Effects); apply(Meta, UnkCmd, State) -> rabbit_log:debug("~ts: unknown command ~W", [?MODULE, UnkCmd, 10]), @@ -737,16 +780,23 @@ state_enter(recover, _) -> put('$rabbit_vm_category', ?MODULE), []; state_enter(leader, #?MODULE{streams = Streams, - monitors = Monitors}) -> + monitors = Monitors, + single_active_consumer = SacState}) -> Pids = maps:keys(Monitors), %% monitor all the known nodes Nodes = all_member_nodes(Streams), NodeMons = [{monitor, node, N} || N <- Nodes], - NodeMons ++ [{aux, fail_active_actions} | - [{monitor, process, P} || P <- Pids]]; + SacEffects = ?SAC_CURRENT:state_enter(leader, SacState), + SacEffects ++ NodeMons ++ [{aux, fail_active_actions} | + [{monitor, process, P} || P <- Pids]]; state_enter(_S, _) -> []. +sac_module(#{machine_version := Vsn}) when ?V5_OR_MORE(Vsn) -> + ?SAC_CURRENT; +sac_module(_) -> + ?SAC_V4. + all_member_nodes(Streams) -> maps:keys( maps:fold( @@ -754,8 +804,9 @@ all_member_nodes(Streams) -> maps:merge(Acc, M) end, #{}, Streams)). -tick(_Ts, _State) -> - [{aux, maybe_resize_coordinator_cluster}]. +tick(_Ts, #?MODULE{single_active_consumer = SacState}) -> + [{aux, maybe_resize_coordinator_cluster} | + maybe_update_sac_configuration(SacState)]. members() -> %% TODO: this can be replaced with a ra_leaderboard @@ -780,7 +831,7 @@ members() -> end end. -maybe_resize_coordinator_cluster() -> +maybe_resize_coordinator_cluster(LeaderPid, SacNodes, MachineVersion) -> spawn(fun() -> RabbitIsRunning = rabbit:is_running(), case members() of @@ -806,19 +857,49 @@ maybe_resize_coordinator_cluster() -> case MemberNodes -- RabbitNodes of [] -> ok; - [Old | _] -> + [Old | _] when length(RabbitNodes) > 0 -> %% this ought to be rather rare as the stream %% coordinator member is now removed as part %% of the forget_cluster_node command - rabbit_log:info("~ts: Rabbit node(s) removed from the cluster, " + rabbit_log:info("~ts: Rabbit node(s) removed " + "from the cluster, " "deleting: ~w", [?MODULE, Old]), - remove_member(Leader, Members, Old) - end; + _ = remove_member(Leader, Members, Old), + ok + end, + maybe_handle_stale_nodes(SacNodes, RabbitNodes, + LeaderPid, + MachineVersion); _ -> ok end end). +maybe_handle_stale_nodes(SacNodes, BrokerNodes, + LeaderPid, Vsn) when ?V5_OR_MORE(Vsn) -> + case SacNodes -- BrokerNodes of + [] -> + ok; + Stale when length(BrokerNodes) > 0 -> + rabbit_log:debug("Stale nodes detected in stream SAC " + "coordinator: ~w. Purging state.", + [Stale]), + ra:pipeline_command(LeaderPid, sac_make_purge_nodes(Stale)), + ok; + _ -> + ok + end; +maybe_handle_stale_nodes(_, _, _, _) -> + ok. + +maybe_update_sac_configuration(SacState) -> + case sac_check_conf_change(SacState) of + {new, UpdatedConf} -> + [{append, sac_make_update_conf(UpdatedConf), noreply}]; + _ -> + [] + end. + add_member(Members, Node) -> MinMacVersion = erpc:call(Node, ?MODULE, version, []), Conf = make_ra_conf(Node, [N || {_, N} <- Members], MinMacVersion), @@ -892,65 +973,64 @@ init_aux(_Name) -> %% TODO ensure the dead writer is restarted as a replica at some point in time, increasing timeout? handle_aux(leader, _, maybe_resize_coordinator_cluster, - #aux{resizer = undefined} = Aux, LogState, _) -> - Pid = maybe_resize_coordinator_cluster(), - {no_reply, Aux#aux{resizer = Pid}, LogState, [{monitor, process, aux, Pid}]}; + #aux{resizer = undefined} = Aux, RaAux) -> + Leader = ra_aux:leader_id(RaAux), + MachineVersion = ra_aux:effective_machine_version(RaAux), + SacNodes = sac_list_nodes(ra_aux:machine_state(RaAux), MachineVersion), + Pid = maybe_resize_coordinator_cluster(Leader, SacNodes, MachineVersion), + {no_reply, Aux#aux{resizer = Pid}, RaAux, [{monitor, process, aux, Pid}]}; handle_aux(leader, _, maybe_resize_coordinator_cluster, - AuxState, LogState, _) -> + AuxState, RaAux) -> %% Coordinator resizing is still happening, let's ignore this tick event - {no_reply, AuxState, LogState}; + {no_reply, AuxState, RaAux}; handle_aux(leader, _, {down, Pid, _}, - #aux{resizer = Pid} = Aux, LogState, _) -> + #aux{resizer = Pid} = Aux, RaAux) -> %% Coordinator resizing has finished - {no_reply, Aux#aux{resizer = undefined}, LogState}; + {no_reply, Aux#aux{resizer = undefined}, RaAux}; handle_aux(leader, _, {start_writer, StreamId, #{epoch := Epoch, node := Node} = Args, Conf}, - Aux, LogState, _) -> + Aux, RaAux) -> rabbit_log:debug("~ts: running action: 'start_writer'" " for ~ts on node ~w in epoch ~b", [?MODULE, StreamId, Node, Epoch]), ActionFun = phase_start_writer(StreamId, Args, Conf), - run_action(starting, StreamId, Args, ActionFun, Aux, LogState); + run_action(starting, StreamId, Args, ActionFun, Aux, RaAux); handle_aux(leader, _, {start_replica, StreamId, #{epoch := Epoch, node := Node} = Args, Conf}, - Aux, LogState, _) -> + Aux, RaAux) -> rabbit_log:debug("~ts: running action: 'start_replica'" " for ~ts on node ~w in epoch ~b", [?MODULE, StreamId, Node, Epoch]), ActionFun = phase_start_replica(StreamId, Args, Conf), - run_action(starting, StreamId, Args, ActionFun, Aux, LogState); + run_action(starting, StreamId, Args, ActionFun, Aux, RaAux); handle_aux(leader, _, {stop, StreamId, #{node := Node, epoch := Epoch} = Args, Conf}, - Aux, LogState, _) -> + Aux, RaAux) -> rabbit_log:debug("~ts: running action: 'stop'" " for ~ts on node ~w in epoch ~b", [?MODULE, StreamId, Node, Epoch]), ActionFun = phase_stop_member(StreamId, Args, Conf), - run_action(stopping, StreamId, Args, ActionFun, Aux, LogState); + run_action(stopping, StreamId, Args, ActionFun, Aux, RaAux); handle_aux(leader, _, {update_mnesia, StreamId, Args, Conf}, - #aux{actions = _Monitors} = Aux, LogState, - #?MODULE{streams = _Streams}) -> + #aux{actions = _Monitors} = Aux, RaAux) -> rabbit_log:debug("~ts: running action: 'update_mnesia'" " for ~ts", [?MODULE, StreamId]), ActionFun = phase_update_mnesia(StreamId, Args, Conf), - run_action(updating_mnesia, StreamId, Args, ActionFun, Aux, LogState); + run_action(updating_mnesia, StreamId, Args, ActionFun, Aux, RaAux); handle_aux(leader, _, {update_retention, StreamId, Args, _Conf}, - #aux{actions = _Monitors} = Aux, LogState, - #?MODULE{streams = _Streams}) -> + #aux{actions = _Monitors} = Aux, RaAux) -> rabbit_log:debug("~ts: running action: 'update_retention'" " for ~ts", [?MODULE, StreamId]), ActionFun = phase_update_retention(StreamId, Args), - run_action(update_retention, StreamId, Args, ActionFun, Aux, LogState); + run_action(update_retention, StreamId, Args, ActionFun, Aux, RaAux); handle_aux(leader, _, {delete_member, StreamId, #{node := Node} = Args, Conf}, - #aux{actions = _Monitors} = Aux, LogState, - #?MODULE{streams = _Streams}) -> + #aux{actions = _Monitors} = Aux, RaAux) -> rabbit_log:debug("~ts: running action: 'delete_member'" " for ~ts ~ts", [?MODULE, StreamId, Node]), ActionFun = phase_delete_member(StreamId, Args, Conf), - run_action(delete_member, StreamId, Args, ActionFun, Aux, LogState); + run_action(delete_member, StreamId, Args, ActionFun, Aux, RaAux); handle_aux(leader, _, fail_active_actions, - #aux{actions = Actions} = Aux, LogState, - #?MODULE{streams = Streams}) -> + #aux{actions = Actions} = Aux, RaAux) -> %% this bit of code just creates an exclude map of currently running %% tasks to avoid failing them, this could only really happen during %% a leader flipflap @@ -958,14 +1038,15 @@ handle_aux(leader, _, fail_active_actions, || {P, {S, _, _}} <- maps_to_list(Actions), is_process_alive(P)]), rabbit_log:debug("~ts: failing actions: ~w", [?MODULE, Exclude]), + #?MODULE{streams = Streams} = ra_aux:machine_state(RaAux), fail_active_actions(Streams, Exclude), - {no_reply, Aux, LogState, []}; + {no_reply, Aux, RaAux, []}; handle_aux(leader, _, {down, Pid, normal}, - #aux{actions = Monitors} = Aux, LogState, _) -> + #aux{actions = Monitors} = Aux, RaAux) -> %% action process finished normally, just remove from actions map - {no_reply, Aux#aux{actions = maps:remove(Pid, Monitors)}, LogState, []}; + {no_reply, Aux#aux{actions = maps:remove(Pid, Monitors)}, RaAux, []}; handle_aux(leader, _, {down, Pid, Reason}, - #aux{actions = Monitors0} = Aux, LogState, _) -> + #aux{actions = Monitors0} = Aux, RaAux) -> %% An action has failed - report back to the state machine case maps:get(Pid, Monitors0, undefined) of {StreamId, Action, #{node := Node, epoch := Epoch} = Args} -> @@ -976,13 +1057,13 @@ handle_aux(leader, _, {down, Pid, Reason}, Cmd = {action_failed, StreamId, Args#{action => Action}}, send_self_command(Cmd), {no_reply, Aux#aux{actions = maps:remove(Pid, Monitors)}, - LogState, []}; + RaAux, []}; undefined -> %% should this ever happen? - {no_reply, Aux, LogState, []} + {no_reply, Aux, RaAux, []} end; -handle_aux(_, _, _, AuxState, LogState, _) -> - {no_reply, AuxState, LogState}. +handle_aux(_, _, _, AuxState, RaAux) -> + {no_reply, AuxState, RaAux}. overview(#?MODULE{streams = Streams, monitors = Monitors, @@ -1018,7 +1099,7 @@ stream_overview0(#stream{epoch = Epoch, run_action(Action, StreamId, #{node := _Node, epoch := _Epoch} = Args, - ActionFun, #aux{actions = Actions0} = Aux, Log) -> + ActionFun, #aux{actions = Actions0} = Aux, RaAux) -> Coordinator = self(), Pid = spawn_link(fun() -> ActionFun(), @@ -1026,7 +1107,7 @@ run_action(Action, StreamId, #{node := _Node, end), Effects = [{monitor, process, aux, Pid}], Actions = Actions0#{Pid => {StreamId, Action, Args}}, - {no_reply, Aux#aux{actions = Actions}, Log, Effects}. + {no_reply, Aux#aux{actions = Actions}, RaAux, Effects}. wrap_reply(From, Reply) -> [{reply, From, {wrap_reply, Reply}}]. @@ -1641,20 +1722,20 @@ update_stream0(_Meta, {update_config, _StreamId, Conf}, update_stream0(_Meta, _Cmd, undefined) -> undefined. -inform_listeners_eol(MachineVersion, +inform_listeners_eol(Vsn, #stream{target = deleted, listeners = Listeners, queue_ref = QRef}) - when MachineVersion =< 1 -> + when Vsn =< 1 -> lists:map(fun(Pid) -> {send_msg, Pid, {queue_event, QRef, eol}, cast} end, maps:keys(Listeners)); -inform_listeners_eol(MachineVersion, +inform_listeners_eol(Vsn, #stream{target = deleted, listeners = Listeners, - queue_ref = QRef}) when MachineVersion >= 2 -> + queue_ref = QRef}) when ?V2_OR_MORE(Vsn) -> LPidsMap = maps:fold(fun({P, _}, _V, Acc) -> Acc#{P => ok} end, #{}, Listeners), @@ -1702,9 +1783,9 @@ eval_listeners(MachineVersion, #stream{listeners = Listeners0, _ -> {Stream, Effects0} end; -eval_listeners(MachineVersion, #stream{listeners = Listeners0} = Stream0, +eval_listeners(Vsn, #stream{listeners = Listeners0} = Stream0, _OldStream, Effects0) - when MachineVersion >= 2 -> + when ?V2_OR_MORE(Vsn) -> %% Iterating over stream listeners. %% Returning the new map of listeners and the effects (notification of changes) {Listeners1, Effects1} = @@ -2199,8 +2280,10 @@ machine_version(1, 2, State = #?MODULE{streams = Streams0, monitors = Monitors2, listeners = undefined}, Effects}; machine_version(2, 3, State) -> - rabbit_log:info("Stream coordinator machine version changes from 2 to 3, updating state."), - {State#?MODULE{single_active_consumer = rabbit_stream_sac_coordinator:init_state()}, + rabbit_log:info("Stream coordinator machine version changes from 2 to 3, " + "updating state."), + SacState = rabbit_stream_sac_coordinator_v4:init_state(), + {State#?MODULE{single_active_consumer = SacState}, []}; machine_version(3, 4, #?MODULE{streams = Streams0} = State) -> rabbit_log:info("Stream coordinator machine version changes from 3 to 4, updating state."), @@ -2214,6 +2297,11 @@ machine_version(3, 4, #?MODULE{streams = Streams0} = State) -> end, Members)} end, Streams0), {State#?MODULE{streams = Streams}, []}; +machine_version(4 = From, 5, #?MODULE{single_active_consumer = Sac0} = State) -> + rabbit_log:info("Stream coordinator machine version changes from 4 to 5, updating state."), + SacExport = rabbit_stream_sac_coordinator_v4:state_to_map(Sac0), + Sac1 = rabbit_stream_sac_coordinator:import_state(From, SacExport), + {State#?MODULE{single_active_consumer = Sac1}, []}; machine_version(From, To, State) -> rabbit_log:info("Stream coordinator machine version changes from ~tp to ~tp, no state changes required.", [From, To]), @@ -2350,3 +2438,22 @@ maps_to_list(M) -> ra_local_query(QueryFun) -> ra:local_query({?MODULE, node()}, QueryFun, infinity). + +sac_handle_connection_down(SacState, Pid, Reason, Vsn) when ?V5_OR_MORE(Vsn) -> + ?SAC_CURRENT:handle_connection_down(Pid, Reason, SacState); +sac_handle_connection_down(SacState, Pid, _Reason, _Vsn) -> + ?SAC_V4:handle_connection_down(Pid, SacState). + +sac_make_purge_nodes(Nodes) -> + rabbit_stream_sac_coordinator:make_purge_nodes(Nodes). + +sac_make_update_conf(Conf) -> + rabbit_stream_sac_coordinator:make_update_conf(Conf). + +sac_check_conf_change(SacState) -> + rabbit_stream_sac_coordinator:check_conf_change(SacState). + +sac_list_nodes(State, Vsn) when ?V5_OR_MORE(Vsn) -> + rabbit_stream_sac_coordinator:list_nodes(sac_state(State)); +sac_list_nodes(_, _) -> + []. diff --git a/deps/rabbit/src/rabbit_stream_coordinator.hrl b/deps/rabbit/src/rabbit_stream_coordinator.hrl index 630a95e1290e..3603be485835 100644 --- a/deps/rabbit/src/rabbit_stream_coordinator.hrl +++ b/deps/rabbit/src/rabbit_stream_coordinator.hrl @@ -68,6 +68,7 @@ listeners = #{} :: undefined | #{stream_id() => #{pid() := queue_ref()}}, single_active_consumer = undefined :: undefined | + rabbit_stream_sac_coordinator_v4:state() | rabbit_stream_sac_coordinator:state(), %% future extensibility reserved_2}). diff --git a/deps/rabbit/src/rabbit_stream_sac_coordinator.erl b/deps/rabbit/src/rabbit_stream_sac_coordinator.erl index 9975cebb485b..b29b4d8fe00f 100644 --- a/deps/rabbit/src/rabbit_stream_sac_coordinator.erl +++ b/deps/rabbit/src/rabbit_stream_sac_coordinator.erl @@ -18,9 +18,13 @@ -include("rabbit_stream_sac_coordinator.hrl"). --opaque command() :: - #command_register_consumer{} | #command_unregister_consumer{} | - #command_activate_consumer{}. +-opaque command() :: #command_register_consumer{} | + #command_unregister_consumer{} | + #command_activate_consumer{} | + #command_connection_reconnected{} | + #command_purge_nodes{} | + #command_update_conf{}. + -opaque state() :: #?MODULE{}. -export_type([state/0, @@ -31,18 +35,52 @@ unregister_consumer/5, activate_consumer/3, consumer_groups/2, - group_consumers/4]). + group_consumers/4, + connection_reconnected/1]). -export([apply/2, init_state/0, send_message/2, ensure_monitors/4, - handle_connection_down/2, + handle_connection_down/3, + handle_node_reconnected/3, + presume_connection_down/2, consumer_groups/3, group_consumers/5, - overview/1]). + overview/1, + import_state/2, + check_conf_change/1, + list_nodes/1, + state_enter/2 + ]). +-export([make_purge_nodes/1, + make_update_conf/1]). + +%% exported for unit tests only +-ifdef(TEST). +-export([compute_pid_group_dependencies/1]). +-endif. -import(rabbit_stream_coordinator, [ra_local_query/1]). +-define(ACTIVE, active). +-define(WAITING, waiting). +-define(DEACTIVATING, deactivating). + +-define(CONNECTED, connected). +-define(DISCONNECTED, disconnected). +-define(PDOWN, presumed_down). + +-define(CONN_ACT, {?CONNECTED, ?ACTIVE}). +-define(CONN_WAIT, {?CONNECTED, ?WAITING}). +-define(DISCONN_ACT, {?DISCONNECTED, ?ACTIVE}). +-define(PDOWN_ACT, {?PDOWN, ?ACTIVE}). + +-define(DISCONNECTED_TIMEOUT_APP_KEY, stream_sac_disconnected_timeout). +-define(DISCONNECTED_TIMEOUT_CONF_KEY, disconnected_timeout). +-define(DISCONNECTED_TIMEOUT_MS, 60_000). +-define(SAC_ERRORS, [partition_index_conflict, not_found]). +-define(IS_STATE_REC(T), is_record(T, ?MODULE)). + %% Single Active Consumer API -spec register_consumer(binary(), binary(), @@ -59,25 +97,13 @@ register_consumer(VirtualHost, ConnectionPid, Owner, SubscriptionId) -> - process_command({sac, - #command_register_consumer{vhost = - VirtualHost, - stream = - Stream, - partition_index - = - PartitionIndex, - consumer_name - = - ConsumerName, - connection_pid - = - ConnectionPid, - owner = - Owner, - subscription_id - = - SubscriptionId}}). + process_command(#command_register_consumer{vhost = VirtualHost, + stream = Stream, + partition_index = PartitionIndex, + consumer_name = ConsumerName, + connection_pid = ConnectionPid, + owner = Owner, + subscription_id = SubscriptionId}). -spec unregister_consumer(binary(), binary(), @@ -90,34 +116,24 @@ unregister_consumer(VirtualHost, ConsumerName, ConnectionPid, SubscriptionId) -> - process_command({sac, - #command_unregister_consumer{vhost = - VirtualHost, - stream = - Stream, - consumer_name - = - ConsumerName, - connection_pid - = - ConnectionPid, - subscription_id - = - SubscriptionId}}). + process_command(#command_unregister_consumer{vhost = VirtualHost, + stream = Stream, + consumer_name = ConsumerName, + connection_pid = ConnectionPid, + subscription_id = SubscriptionId}). -spec activate_consumer(binary(), binary(), binary()) -> ok. -activate_consumer(VirtualHost, Stream, ConsumerName) -> - process_command({sac, - #command_activate_consumer{vhost = - VirtualHost, - stream = - Stream, - consumer_name - = - ConsumerName}}). +activate_consumer(VH, Stream, Name) -> + process_command(#command_activate_consumer{vhost =VH, + stream = Stream, + consumer_name= Name}). + +-spec connection_reconnected(connection_pid()) -> ok. +connection_reconnected(Pid) -> + process_command(#command_connection_reconnected{pid = Pid}). process_command(Cmd) -> - case rabbit_stream_coordinator:process_command(Cmd) of + case rabbit_stream_coordinator:process_command(wrap_cmd(Cmd)) of {ok, Res, _} -> Res; {error, _} = Err -> @@ -126,7 +142,12 @@ process_command(Cmd) -> Err end. +-spec wrap_cmd(command()) -> {sac, command()}. +wrap_cmd(Cmd) -> + {sac, Cmd}. + %% return the current groups for a given virtual host +%% (CLI command) -spec consumer_groups(binary(), [atom()]) -> {ok, [term()] | {error, atom()}}. @@ -148,6 +169,7 @@ consumer_groups(VirtualHost, InfoKeys) -> end. %% get the consumers of a given group in a given virtual host +%% (CLI command) -spec group_consumers(binary(), binary(), binary(), [atom()]) -> {ok, [term()]} | {error, atom()}. @@ -171,7 +193,7 @@ group_consumers(VirtualHost, Stream, Reference, InfoKeys) -> {timeout, _} -> {error, timeout} end. --spec overview(state()) -> map(). +-spec overview(state() | undefined) -> map() | undefined. overview(undefined) -> undefined; overview(#?MODULE{groups = Groups}) -> @@ -186,7 +208,9 @@ overview(#?MODULE{groups = Groups}) -> -spec init_state() -> state(). init_state() -> - #?MODULE{groups = #{}, pids_groups = #{}}. + DisconTimeout = ?DISCONNECTED_TIMEOUT_MS, + #?MODULE{groups = #{}, pids_groups = #{}, + conf = #{?DISCONNECTED_TIMEOUT_CONF_KEY => DisconTimeout}}. -spec apply(command(), state()) -> {state(), term(), ra_machine:effects()}. @@ -231,7 +255,9 @@ apply(#command_unregister_consumer{vhost = VirtualHost, of {value, Consumer} -> G1 = remove_from_group(Consumer, Group0), - handle_consumer_removal(G1, Stream, ConsumerName, Consumer#consumer.active); + handle_consumer_removal( + G1, Stream, ConsumerName, + is_active(Consumer#consumer.status)); false -> {Group0, []} end, @@ -254,48 +280,303 @@ apply(#command_activate_consumer{vhost = VirtualHost, "the group does not longer exist", [{VirtualHost, Stream, ConsumerName}]), {undefined, []}; - Group -> - #consumer{pid = Pid, subscription_id = SubId} = - evaluate_active_consumer(Group), - Group1 = update_consumer_state_in_group(Group, Pid, SubId, true), - {Group1, [notify_consumer_effect(Pid, SubId, Stream, ConsumerName, true)]} + G0 -> + %% keep track of the former active, if any + {ActPid, ActSubId} = + case lookup_active_consumer(G0) of + {value, #consumer{pid = ActivePid, + subscription_id = ActiveSubId}} -> + {ActivePid, ActiveSubId}; + _ -> + {-1, -1} + end, + G1 = update_connected_consumers(G0, ?CONN_WAIT), + case evaluate_active_consumer(G1) of + undefined -> + {G1, []}; + #consumer{status = {?DISCONNECTED, _}} -> + %% we keep it this way, the consumer may come back + {G1, []}; + #consumer{pid = Pid, subscription_id = SubId} -> + G2 = update_consumer_state_in_group(G1, Pid, + SubId, + ?CONN_ACT), + %% do we need effects or not? + Effects = + case {Pid, SubId} of + {ActPid, ActSubId} -> + %% it is the same active consumer as before + %% no need to notify it + []; + _ -> + %% new active consumer, need to notify it + [notify_consumer_effect(Pid, SubId, Stream, + ConsumerName, true)] + end, + {G2, Effects} + end end, - StreamGroups1 = - update_groups(VirtualHost, Stream, ConsumerName, G, StreamGroups0), - {State0#?MODULE{groups = StreamGroups1}, ok, Eff}. + StreamGroups1 = update_groups(VirtualHost, Stream, ConsumerName, + G, StreamGroups0), + {State0#?MODULE{groups = StreamGroups1}, ok, Eff}; +apply(#command_connection_reconnected{pid = Pid}, + #?MODULE{groups = Groups0} = State0) -> + {State1, Eff} = + maps:fold(fun(G, _, {St, Eff}) -> + handle_group_connection_reconnected(Pid, St, Eff, G) + end, {State0, []}, Groups0), + + {State1, ok, Eff}; +apply(#command_purge_nodes{nodes = Nodes}, State0) -> + {State1, Eff} = lists:foldl(fun(N, {S0, Eff0}) -> + {S1, Eff1} = purge_node(N, S0), + {S1, Eff1 ++ Eff0} + end, {State0, []}, Nodes), + {State1, ok, Eff}; +apply(#command_update_conf{conf = NewConf}, State) -> + {State#?MODULE{conf = NewConf}, ok, []}; +apply(UnkCmd, State) -> + rabbit_log:debug("~ts: unknown SAC command ~W", [?MODULE, UnkCmd, 10]), + {State, {error, unknown_command}, []}. + +purge_node(Node, #?MODULE{groups = Groups0} = State0) -> + PidsGroups = compute_node_pid_group_dependencies(Node, Groups0), + maps:fold(fun(Pid, Groups, {S0, Eff0}) -> + {S1, Eff1} = handle_connection_down0(Pid, S0, Groups), + {S1, Eff1 ++ Eff0} + end, {State0, []}, PidsGroups). +handle_group_connection_reconnected(Pid, #?MODULE{groups = Groups0} = S0, + Eff0, {VH, S, Name} = K) -> + case lookup_group(VH, S, Name, Groups0) of + undefined -> + {S0, Eff0}; + Group -> + case has_forgotten_active(Group, Pid) of + true -> + %% a forgotten active is coming in the connection + %% we need to reconcile the group, + %% as there may have been 2 active consumers at a time + handle_forgotten_active_reconnected(Pid, S0, Eff0, K); + false -> + do_handle_group_connection_reconnected(Pid, S0, Eff0, K) + end + end. + +do_handle_group_connection_reconnected(Pid, #?MODULE{groups = Groups0} = S0, + Eff0, {VH, S, Name} = K) -> + G0 = #group{consumers = Consumers0} = lookup_group(VH, S, Name, Groups0), + {Consumers1, Updated} = + lists:foldr( + fun(#consumer{pid = P, status = {_, St}} = C, {L, _}) + when P == Pid -> + {[csr_status(C, {?CONNECTED, St}) | L], true}; + (C, {L, UpdatedFlag}) -> + {[C | L], UpdatedFlag or false} + end, {[], false}, Consumers0), + + case Updated of + true -> + G1 = G0#group{consumers = Consumers1}, + {G2, Eff} = maybe_rebalance_group(G1, K), + Groups1 = update_groups(VH, S, Name, G2, Groups0), + {S0#?MODULE{groups = Groups1}, Eff ++ Eff0}; + false -> + {S0, Eff0} + end. + +handle_forgotten_active_reconnected(Pid, + #?MODULE{groups = Groups0} = S0, + Eff0, {VH, S, Name}) -> + G0 = #group{consumers = Consumers0} = lookup_group(VH, S, Name, Groups0), + {Consumers1, Eff1} = + case has_disconnected_active(G0) of + true -> + %% disconnected active consumer in the group, no rebalancing possible + %% we update the disconnected active consumers + %% and tell them to step down + lists:foldr(fun(#consumer{status = St, + pid = P, + subscription_id = SID} = C, {Cs, Eff}) + when P =:= Pid andalso St =:= ?PDOWN_ACT -> + {[csr_status(C, ?CONN_WAIT) | Cs], + [notify_consumer_effect(Pid, SID, S, + Name, false, true) | Eff]}; + (C, {Cs, Eff}) -> + {[C | Cs], Eff} + end, {[], Eff0}, Consumers0); + false -> + lists:foldr(fun(#consumer{status = St, + pid = P, + subscription_id = SID} = C, {Cs, Eff}) + when P =:= Pid andalso St =:= ?PDOWN_ACT -> + %% update forgotten active + %% tell it to step down + {[csr_status(C, ?CONN_WAIT) | Cs], + [notify_consumer_effect(P, SID, S, + Name, false, true) | Eff]}; + (#consumer{status = {?PDOWN, _}, + pid = P} = C, {Cs, Eff}) + when P =:= Pid -> + %% update forgotten + {[csr_status(C, ?CONN_WAIT) | Cs], Eff}; + (#consumer{status = ?CONN_ACT, + pid = P, + subscription_id = SID} = C, {Cs, Eff}) -> + %% update connected active + %% tell it to step down + {[csr_status(C, ?CONN_WAIT) | Cs], + [notify_consumer_effect(P, SID, S, + Name, false, true) | Eff]}; + (C, {Cs, Eff}) -> + {[C | Cs], Eff} + end, {[], Eff0}, Consumers0) + end, + G1 = G0#group{consumers = Consumers1}, + Groups1 = update_groups(VH, S, Name, G1, Groups0), + {S0#?MODULE{groups = Groups1}, Eff1}. + +has_forgotten_active(#group{consumers = Consumers}, Pid) -> + case lists:search(fun(#consumer{status = ?PDOWN_ACT, + pid = P}) when P =:= Pid -> + true; + (_) -> false + end, Consumers) of + false -> + false; + _ -> + true + end. + +has_disconnected_active(Group) -> + has_consumer_with_status(Group, ?DISCONN_ACT). + +has_consumer_with_status(#group{consumers = Consumers}, Status) -> + case lists:search(fun(#consumer{status = S}) when S =:= Status -> + true; + (_) -> false + end, Consumers) of + false -> + false; + _ -> + true + end. + +maybe_rebalance_group(#group{partition_index = -1, consumers = Consumers0} = G0, + {_VH, S, Name}) -> + case lookup_active_consumer(G0) of + {value, ActiveConsumer} -> + %% there is already an active consumer, we just re-arrange + %% the group to make sure the active consumer is the first in the array + Consumers1 = lists:filter(fun(C) -> + not same_consumer(C, ActiveConsumer) + end, Consumers0), + G1 = G0#group{consumers = [ActiveConsumer | Consumers1]}, + {G1, []}; + _ -> + %% no active consumer + G1 = compute_active_consumer(G0), + case lookup_active_consumer(G1) of + {value, #consumer{pid = Pid, subscription_id = SubId}} -> + %% creating the side effect to notify the new active consumer + {G1, [notify_consumer_effect(Pid, SubId, S, Name, true)]}; + _ -> + %% no active consumer found in the group, nothing to do + {G1, []} + end + end; +maybe_rebalance_group(#group{partition_index = _, consumers = Consumers} = G, + {_VH, S, Name}) -> + case lookup_active_consumer(G) of + {value, #consumer{pid = ActPid, + subscription_id = ActSubId} = CurrentActive} -> + case evaluate_active_consumer(G) of + undefined -> + %% no-one to select + {G, []}; + CurrentActive -> + %% the current active stays the same + {G, []}; + _ -> + %% there's a change, telling the active it's not longer active + {update_consumer_state_in_group(G, + ActPid, + ActSubId, + {?CONNECTED, ?DEACTIVATING}), + [notify_consumer_effect(ActPid, + ActSubId, + S, + Name, + false, + true)]} + end; + false -> + %% no active consumer in the (non-empty) group, + case lists:search(fun(#consumer{status = Status}) -> + Status =:= {?CONNECTED, ?DEACTIVATING} + end, Consumers) of + {value, _Deactivating} -> + %% we are waiting for the reply of a former active + %% nothing to do + {G, []}; + _ -> + %% nothing going on in the group + %% a {disconnected, active} may have become {forgotten, active} + %% we must select a new active + case evaluate_active_consumer(G) of + undefined -> + %% no-one to select + {G, []}; + #consumer{pid = ActPid, subscription_id = ActSubId} -> + {update_consumer_state_in_group(G, + ActPid, + ActSubId, + {?CONNECTED, ?ACTIVE}), + [notify_consumer_effect(ActPid, + ActSubId, + S, + Name, + true)]} + end + end + end. + +%% used by CLI -spec consumer_groups(binary(), [atom()], state()) -> {ok, [term()]}. -consumer_groups(VirtualHost, InfoKeys, #?MODULE{groups = Groups}) -> +consumer_groups(VirtualHost, InfoKeys, #?MODULE{groups = Groups} = S) + when ?IS_STATE_REC(S) -> Res = maps:fold(fun ({VH, Stream, Reference}, #group{consumers = Consumers, partition_index = PartitionIndex}, Acc) - when VH == VirtualHost -> + when VH == VirtualHost -> Record = - lists:foldr(fun (stream, RecAcc) -> - [{stream, Stream} | RecAcc]; - (reference, RecAcc) -> - [{reference, Reference} - | RecAcc]; - (partition_index, RecAcc) -> - [{partition_index, - PartitionIndex} - | RecAcc]; - (consumers, RecAcc) -> - [{consumers, - length(Consumers)} - | RecAcc]; - (Unknown, RecAcc) -> - [{Unknown, unknown_field} - | RecAcc] - end, - [], InfoKeys), + lists:foldr(fun (stream, RecAcc) -> + [{stream, Stream} | RecAcc]; + (reference, RecAcc) -> + [{reference, Reference} + | RecAcc]; + (partition_index, RecAcc) -> + [{partition_index, + PartitionIndex} + | RecAcc]; + (consumers, RecAcc) -> + [{consumers, + length(Consumers)} + | RecAcc]; + (Unknown, RecAcc) -> + [{Unknown, unknown_field} + | RecAcc] + end, + [], InfoKeys), [Record | Acc]; (_GroupId, _Group, Acc) -> Acc end, [], Groups), - {ok, lists:reverse(Res)}. + {ok, lists:reverse(Res)}; +consumer_groups(VirtualHost, InfoKeys, S) -> + rabbit_stream_sac_coordinator_v4:consumer_groups(VirtualHost, InfoKeys, S). -spec group_consumers(binary(), binary(), @@ -303,47 +584,45 @@ consumer_groups(VirtualHost, InfoKeys, #?MODULE{groups = Groups}) -> [atom()], state()) -> {ok, [term()]} | {error, not_found}. -group_consumers(VirtualHost, - Stream, - Reference, - InfoKeys, - #?MODULE{groups = Groups}) -> - GroupId = {VirtualHost, Stream, Reference}, +group_consumers(VH, St, Ref, InfoKeys, + #?MODULE{groups = Groups} = S) + when ?IS_STATE_REC(S) -> + GroupId = {VH, St, Ref}, case Groups of #{GroupId := #group{consumers = Consumers}} -> - Cs = lists:foldr(fun(#consumer{subscription_id = SubId, - owner = Owner, - active = Active}, - Acc) -> - Record = - lists:foldr(fun (subscription_id, RecAcc) -> - [{subscription_id, - SubId} - | RecAcc]; - (connection_name, RecAcc) -> - [{connection_name, - Owner} - | RecAcc]; - (state, RecAcc) - when Active -> - [{state, active} - | RecAcc]; - (state, RecAcc) -> - [{state, inactive} - | RecAcc]; - (Unknown, RecAcc) -> - [{Unknown, - unknown_field} - | RecAcc] - end, - [], InfoKeys), - [Record | Acc] + Cs = lists:foldr(fun(C, Acc) -> + [csr_cli_record(C, InfoKeys) | Acc] end, [], Consumers), {ok, Cs}; _ -> {error, not_found} - end. + end; +group_consumers(VirtualHost, Stream, Reference, InfoKeys, S) -> + rabbit_stream_sac_coordinator_v4:group_consumers(VirtualHost, Stream, + Reference, InfoKeys, S). + +csr_cli_record(#consumer{subscription_id = SubId, owner = Owner, + status = Status}, InfoKeys) -> + lists:foldr(fun (subscription_id, Acc) -> + [{subscription_id, SubId} | Acc]; + (connection_name, Acc) -> + [{connection_name, Owner} | Acc]; + (state, Acc) -> + [{state, cli_csr_status_label(Status)} | Acc]; + (Unknown, Acc) -> + [{Unknown, unknown_field} | Acc] + end, + [], InfoKeys). + + +cli_csr_status_label({Cnty, Acty}) -> + rabbit_data_coercion:to_list(Acty) ++ " (" ++ connectivity_label(Cnty) ++ ")". + +connectivity_label(?PDOWN) -> + "presumed down"; +connectivity_label(Cnty) -> + rabbit_data_coercion:to_list(Cnty). -spec ensure_monitors(command(), state(), @@ -358,17 +637,20 @@ ensure_monitors(#command_register_consumer{vhost = VirtualHost, Monitors0, Effects) -> GroupId = {VirtualHost, Stream, ConsumerName}, + %% get the group IDs that depend on the PID Groups0 = maps:get(Pid, PidsGroups0, #{}), - PidsGroups1 = - maps:put(Pid, maps:put(GroupId, true, Groups0), PidsGroups0), + %% add the group ID + Groups1 = Groups0#{GroupId => true}, + %% update the PID-to-group map + PidsGroups1 = PidsGroups0#{Pid => Groups1}, {State0#?MODULE{pids_groups = PidsGroups1}, Monitors0#{Pid => sac}, [{monitor, process, Pid}, {monitor, node, node(Pid)} | Effects]}; ensure_monitors(#command_unregister_consumer{vhost = VirtualHost, stream = Stream, consumer_name = ConsumerName, connection_pid = Pid}, - #?MODULE{groups = StreamGroups0, pids_groups = PidsGroups0} = - State0, + #?MODULE{groups = StreamGroups0, + pids_groups = PidsGroups0} = State0, Monitors, Effects) when is_map_key(Pid, PidsGroups0) -> @@ -400,30 +682,126 @@ ensure_monitors(#command_unregister_consumer{vhost = VirtualHost, maps:remove(Pid, Monitors), [{demonitor, process, Pid} | Effects]}; false -> %% one or more groups still depend on the PID - {State0#?MODULE{pids_groups = - maps:put(Pid, PidGroup1, PidsGroups0)}, + {State0#?MODULE{pids_groups = PidsGroups0#{Pid => PidGroup1}}, Monitors, Effects} end; +ensure_monitors(#command_connection_reconnected{pid = Pid}, + #?MODULE{pids_groups = PidsGroups, + groups = Groups} = State, + Monitors, + Effects) + when not is_map_key(Pid, Monitors) orelse + not is_map_key(Pid, PidsGroups) -> + %% the connection PID should be monitored + %% the inconsistency can happen when a forgotten connection comes back, + %% we must re-compute the connection PID / group dependency mapping + %% and re-issue the monitor + AllPidsGroups = compute_pid_group_dependencies(Groups), + {State#?MODULE{pids_groups = AllPidsGroups}, + Monitors#{Pid => sac}, + [{monitor, process, Pid}, {monitor, node, node(Pid)} | Effects]}; +ensure_monitors(#command_purge_nodes{}, + #?MODULE{groups = Groups} = State, + Monitors, + Effects) -> + AllPidsGroups = compute_pid_group_dependencies(Groups), + {State#?MODULE{pids_groups = AllPidsGroups}, + Monitors, + Effects}; ensure_monitors(_, #?MODULE{} = State0, Monitors, Effects) -> {State0, Monitors, Effects}. --spec handle_connection_down(connection_pid(), state()) -> - {state(), ra_machine:effects()}. -handle_connection_down(Pid, +-spec handle_connection_down(connection_pid(), term(), state()) -> + {state(), ra_machine:effects()}. +handle_connection_down(Pid, noconnection, State) -> + handle_connection_node_disconnected(Pid, State); +handle_connection_down(Pid, _Reason, #?MODULE{pids_groups = PidsGroups0} = State0) -> case maps:take(Pid, PidsGroups0) of error -> {State0, []}; {Groups, PidsGroups1} -> State1 = State0#?MODULE{pids_groups = PidsGroups1}, + handle_connection_down0(Pid, State1, Groups) + end. + +handle_connection_down0(Pid, State, Groups) -> + maps:fold(fun(G, _, Acc) -> + handle_group_after_connection_down(Pid, Acc, G) + end, {State, []}, Groups). + +-spec handle_connection_node_disconnected(connection_pid(), state()) -> + {state(), ra_machine:effects()}. +handle_connection_node_disconnected(ConnPid, + #?MODULE{pids_groups = PidsGroups0} = State0) -> + case maps:take(ConnPid, PidsGroups0) of + error -> + {State0, []}; + {Groups, PidsGroups1} -> + State1 = State0#?MODULE{pids_groups = PidsGroups1}, + State2 = maps:fold(fun(G, _, Acc) -> - handle_group_after_connection_down(Pid, Acc, G) - end, {State1, []}, Groups) + handle_group_after_connection_node_disconnected( + ConnPid, Acc, G) + end, State1, Groups), + T = disconnected_timeout(State2), + {State2, [node_disconnected_timer_effect(ConnPid, T)]} + end. + +-spec handle_node_reconnected(node(), state(), ra_machine:effects()) -> + {state(), ra_machine:effects()}. +handle_node_reconnected(Node, + #?MODULE{pids_groups = PidsGroups0, + groups = Groups0} = State0, + Effects0) -> + NodePidsGroups = compute_node_pid_group_dependencies(Node, Groups0), + PidsGroups1 = maps:merge(PidsGroups0, NodePidsGroups), + Effects1 = + lists:foldr(fun(P, Acc) -> + [notify_connection_effect(P), + {monitor, process, P} | Acc] + end, Effects0, maps:keys(NodePidsGroups)), + + {State0#?MODULE{pids_groups = PidsGroups1}, Effects1}. + +-spec presume_connection_down(connection_pid(), state()) -> + {state(), ra_machine:effects()}. +presume_connection_down(Pid, #?MODULE{groups = Groups} = State0) -> + {State1, Eff} = + maps:fold(fun(G, _, {St, Eff}) -> + handle_group_connection_presumed_down(Pid, St, Eff, G) + end, {State0, []}, Groups), + {State1, Eff}. + +handle_group_connection_presumed_down(Pid, #?MODULE{groups = Groups0} = S0, + Eff0, {VH, S, Name} = K) -> + case lookup_group(VH, S, Name, Groups0) of + undefined -> + {S0, Eff0}; + #group{consumers = Consumers0} = G0 -> + {Consumers1, Updated} = + lists:foldr( + fun(#consumer{pid = P, status = {?DISCONNECTED, St}} = C, {L, _}) + when P == Pid -> + {[csr_status(C, {?PDOWN, St}) | L], true}; + (C, {L, UpdatedFlag}) -> + {[C | L], UpdatedFlag or false} + end, {[], false}, Consumers0), + + case Updated of + true -> + G1 = G0#group{consumers = Consumers1}, + {G2, Eff} = maybe_rebalance_group(G1, K), + Groups1 = update_groups(VH, S, Name, G2, Groups0), + {S0#?MODULE{groups = Groups1}, Eff ++ Eff0}; + false -> + {S0, Eff0} + end end. handle_group_after_connection_down(Pid, - {#?MODULE{groups = Groups0} = S0, Eff0}, - {VirtualHost, Stream, ConsumerName}) -> + {#?MODULE{groups = Groups0} = S0, Eff0}, + {VirtualHost, Stream, ConsumerName}) -> case lookup_group(VirtualHost, Stream, ConsumerName, @@ -434,17 +812,20 @@ handle_group_after_connection_down(Pid, %% remove the connection consumers from the group state %% keep flags to know what happened {Consumers1, ActiveRemoved, AnyRemoved} = - lists:foldl( - fun(#consumer{pid = P, active = S}, {L, ActiveFlag, _}) when P == Pid -> - {L, S or ActiveFlag, true}; - (C, {L, ActiveFlag, AnyFlag}) -> - {L ++ [C], ActiveFlag, AnyFlag} - end, {[], false, false}, Consumers0), + lists:foldl( + fun(#consumer{pid = P, status = S}, {L, ActiveFlag, _}) + when P == Pid -> + {L, is_active(S) or ActiveFlag, true}; + (C, {L, ActiveFlag, AnyFlag}) -> + {L ++ [C], ActiveFlag, AnyFlag} + end, {[], false, false}, Consumers0), case AnyRemoved of true -> G1 = G0#group{consumers = Consumers1}, - {G2, Effects} = handle_consumer_removal(G1, Stream, ConsumerName, ActiveRemoved), + {G2, Effects} = handle_consumer_removal(G1, Stream, + ConsumerName, + ActiveRemoved), Groups1 = update_groups(VirtualHost, Stream, ConsumerName, @@ -456,6 +837,162 @@ handle_group_after_connection_down(Pid, end end. +handle_group_after_connection_node_disconnected(ConnPid, + #?MODULE{groups = Groups0} = S0, + {VirtualHost, Stream, ConsumerName}) -> + case lookup_group(VirtualHost, + Stream, + ConsumerName, + Groups0) of + undefined -> + S0; + #group{consumers = Cs0} = G0 -> + Cs1 = lists:foldr(fun(#consumer{status = {_, St}, + pid = Pid} = C0, + Acc) when Pid =:= ConnPid -> + C1 = csr_status(C0, {?DISCONNECTED, St}), + [C1 | Acc]; + (C, Acc) -> + [C | Acc] + end, [], Cs0), + G1 = G0#group{consumers = Cs1}, + Groups1 = update_groups(VirtualHost, + Stream, + ConsumerName, + G1, + Groups0), + S0#?MODULE{groups = Groups1} + end. + +-spec import_state(ra_machine:version(), map()) -> state(). +import_state(4, #{<<"groups">> := Groups, <<"pids_groups">> := PidsGroups}) -> + #?MODULE{groups = map_to_groups(Groups), + pids_groups = map_to_pids_groups(PidsGroups), + conf = #{disconnected_timeout => ?DISCONNECTED_TIMEOUT_MS}}. + +-spec check_conf_change(state() | term()) -> {new, conf()} | unchanged. +check_conf_change(State) when ?IS_STATE_REC(State) -> + #?MODULE{conf = Conf} = State, + DisconTimeout = lookup_disconnected_timeout(), + case Conf of + #{?DISCONNECTED_TIMEOUT_CONF_KEY := DT} + when DT /= DisconTimeout -> + {new, #{?DISCONNECTED_TIMEOUT_CONF_KEY => DisconTimeout}}; + C when is_map_key(?DISCONNECTED_TIMEOUT_CONF_KEY, C) == false -> + {new, #{?DISCONNECTED_TIMEOUT_CONF_KEY => DisconTimeout}}; + _ -> + unchanged + end; +check_conf_change(_) -> + unchanged. + +-spec list_nodes(state()) -> [node()]. +list_nodes(#?MODULE{groups = Groups}) -> + Nodes = maps:fold(fun(_, G, Acc) -> + GNodes = nodes_from_group(G), + maps:merge(Acc, GNodes) + end, #{}, Groups), + lists:sort(maps:keys(Nodes)). + +-spec state_enter(ra_server:ra_state(), state() | term()) -> + ra_machine:effects(). +state_enter(leader, #?MODULE{groups = Groups} = State) + when ?IS_STATE_REC(State) -> + %% iterate over groups + {Nodes, DisConns} = + maps:fold(fun(_, #group{consumers = Cs}, Acc) -> + %% iterage over group consumers + lists:foldl(fun(#consumer{pid = P, + status = {?DISCONNECTED, _}, + ts = Ts}, + {Nodes, DisConns}) -> + %% disconnected consumer, + %% store connection PID and node + {Nodes#{node(P) => true}, + DisConns#{P => Ts}}; + (#consumer{pid = P}, {Nodes, DisConns}) -> + %% store connection node + {Nodes#{node(P) => true}, DisConns} + end, Acc, Cs) + end, {#{}, #{}}, Groups), + DisTimeout = disconnected_timeout(State), + %% monitor involved nodes + %% reset a timer for disconnected connections + [{monitor, node, N} || N <- lists:sort(maps:keys(Nodes))] ++ + [begin + Time = case ts() - Ts of + T when T < 10_000 -> + %% 10 seconds is arbitrary, nothing specific about the value + 10_000; + T when T > DisTimeout -> + DisTimeout + end, + node_disconnected_timer_effect(P, Time) + end || P := Ts <- maps:iterator(DisConns, ordered)]; +state_enter(_, _) -> + []. + +nodes_from_group(#group{consumers = Cs}) when is_list(Cs) -> + lists:foldl(fun(#consumer{pid = Pid}, Acc) -> + Acc#{node(Pid) => true} + end, #{}, Cs); +nodes_from_group(_) -> + #{}. + +-spec make_purge_nodes([node()]) -> {sac, command()}. +make_purge_nodes(Nodes) -> + wrap_cmd(#command_purge_nodes{nodes = Nodes}). + +- spec make_update_conf(conf()) -> {sac, command()}. +make_update_conf(Conf) -> + wrap_cmd(#command_update_conf{conf = Conf}). + +lookup_disconnected_timeout() -> + application:get_env(rabbit, ?DISCONNECTED_TIMEOUT_APP_KEY, + ?DISCONNECTED_TIMEOUT_MS). + +disconnected_timeout(#?MODULE{conf = #{?DISCONNECTED_TIMEOUT_CONF_KEY := T}}) -> + T; +disconnected_timeout(_) -> + ?DISCONNECTED_TIMEOUT_MS. + +map_to_groups(Groups) when is_map(Groups) -> + maps:fold(fun(K, V, Acc) -> + Acc#{K => map_to_group(V)} + end, #{}, Groups); +map_to_groups(_) -> + #{}. + +map_to_pids_groups(PidsGroups) when is_map(PidsGroups) -> + PidsGroups; +map_to_pids_groups(_) -> + #{}. + +map_to_group(#{<<"consumers">> := Consumers, <<"partition_index">> := Index}) -> + C = lists:foldl(fun(V, Acc) -> + Acc ++ [map_to_consumer(V)] + end, [], Consumers), + #group{consumers = C, + partition_index = Index}. + +map_to_consumer(#{<<"pid">> := Pid, <<"subscription_id">> := SubId, + <<"owner">> := Owner, <<"active">> := Active}) -> + csr(Pid, SubId, Owner, active_to_status(Active)). + +active_to_status(true) -> + {?CONNECTED, ?ACTIVE}; +active_to_status(false) -> + {?CONNECTED, ?WAITING}. + +is_active({?PDOWN, _}) -> + false; +is_active({_, ?ACTIVE}) -> + true; +is_active({_, ?DEACTIVATING}) -> + true; +is_active(_) -> + false. + do_register_consumer(VirtualHost, Stream, -1 = _PartitionIndex, @@ -464,41 +1001,31 @@ do_register_consumer(VirtualHost, Owner, SubscriptionId, #?MODULE{groups = StreamGroups0} = State) -> - Group0 = - lookup_group(VirtualHost, Stream, ConsumerName, StreamGroups0), + Group0 = lookup_group(VirtualHost, Stream, ConsumerName, StreamGroups0), Consumer = case lookup_active_consumer(Group0) of {value, _} -> - #consumer{pid = ConnectionPid, - owner = Owner, - subscription_id = SubscriptionId, - active = false}; + csr(ConnectionPid, SubscriptionId, Owner, ?CONN_WAIT); false -> - #consumer{pid = ConnectionPid, - subscription_id = SubscriptionId, - owner = Owner, - active = true} + csr(ConnectionPid, SubscriptionId, Owner, ?CONN_ACT) end, Group1 = add_to_group(Consumer, Group0), - StreamGroups1 = - update_groups(VirtualHost, - Stream, - ConsumerName, - Group1, - StreamGroups0), + StreamGroups1 = update_groups(VirtualHost, Stream, ConsumerName, + Group1, + StreamGroups0), - #consumer{active = Active} = Consumer, + #consumer{status = Status} = Consumer, Effects = - case Active of - true -> + case Status of + {_, ?ACTIVE} -> [notify_consumer_effect(ConnectionPid, SubscriptionId, - Stream, ConsumerName, Active)]; + Stream, ConsumerName, is_active(Status))]; _ -> [] end, - {State#?MODULE{groups = StreamGroups1}, {ok, Active}, Effects}; + {State#?MODULE{groups = StreamGroups1}, {ok, is_active(Status)}, Effects}; do_register_consumer(VirtualHost, Stream, _PartitionIndex, @@ -507,67 +1034,28 @@ do_register_consumer(VirtualHost, Owner, SubscriptionId, #?MODULE{groups = StreamGroups0} = State) -> - Group0 = - lookup_group(VirtualHost, Stream, ConsumerName, StreamGroups0), + Group0 = lookup_group(VirtualHost, Stream, ConsumerName, StreamGroups0), {Group1, Effects} = case Group0 of #group{consumers = []} -> %% first consumer in the group, it's the active one - Consumer0 = - #consumer{pid = ConnectionPid, - owner = Owner, - subscription_id = SubscriptionId, - active = true}, + Consumer0 = csr(ConnectionPid, SubscriptionId, Owner, ?CONN_ACT), G1 = add_to_group(Consumer0, Group0), {G1, [notify_consumer_effect(ConnectionPid, SubscriptionId, Stream, ConsumerName, true)]}; _G -> - %% whatever the current state is, the newcomer will be passive - Consumer0 = - #consumer{pid = ConnectionPid, - owner = Owner, - subscription_id = SubscriptionId, - active = false}, + Consumer0 = csr(ConnectionPid, SubscriptionId, Owner, ?CONN_WAIT), G1 = add_to_group(Consumer0, Group0), - - case lookup_active_consumer(G1) of - {value, - #consumer{pid = ActPid, subscription_id = ActSubId} = - CurrentActive} -> - case evaluate_active_consumer(G1) of - CurrentActive -> - %% the current active stays the same - {G1, []}; - _ -> - %% there's a change, telling the active it's not longer active - {update_consumer_state_in_group(G1, - ActPid, - ActSubId, - false), - [notify_consumer_effect(ActPid, - ActSubId, - Stream, - ConsumerName, - false, - true)]} - end; - false -> - %% no active consumer in the (non-empty) group, - %% we are waiting for the reply of a former active - {G1, []} - end + maybe_rebalance_group(G1, {VirtualHost, Stream, ConsumerName}) end, - StreamGroups1 = - update_groups(VirtualHost, - Stream, - ConsumerName, - Group1, - StreamGroups0), - {value, #consumer{active = Active}} = + StreamGroups1 = update_groups(VirtualHost, Stream, ConsumerName, + Group1, + StreamGroups0), + {value, #consumer{status = Status}} = lookup_consumer(ConnectionPid, SubscriptionId, Group1), - {State#?MODULE{groups = StreamGroups1}, {ok, Active}, Effects}. + {State#?MODULE{groups = StreamGroups1}, {ok, is_active(Status)}, Effects}. handle_consumer_removal(#group{consumers = []} = G, _, _, _) -> {G, []}; @@ -591,10 +1079,11 @@ handle_consumer_removal(#group{partition_index = -1} = Group0, end; handle_consumer_removal(Group0, Stream, ConsumerName, ActiveRemoved) -> case lookup_active_consumer(Group0) of - {value, - #consumer{pid = ActPid, subscription_id = ActSubId} = - CurrentActive} -> + {value, #consumer{pid = ActPid, + subscription_id = ActSubId} = CurrentActive} -> case evaluate_active_consumer(Group0) of + undefined -> + {Group0, []}; CurrentActive -> %% the current active stays the same {Group0, []}; @@ -603,7 +1092,7 @@ handle_consumer_removal(Group0, Stream, ConsumerName, ActiveRemoved) -> {update_consumer_state_in_group(Group0, ActPid, ActSubId, - false), + {?CONNECTED, ?DEACTIVATING}), [notify_consumer_effect(ActPid, ActSubId, Stream, ConsumerName, false, true)]} end; @@ -611,11 +1100,15 @@ handle_consumer_removal(Group0, Stream, ConsumerName, ActiveRemoved) -> case ActiveRemoved of true -> %% the active one is going away, picking a new one - #consumer{pid = P, subscription_id = SID} = - evaluate_active_consumer(Group0), - {update_consumer_state_in_group(Group0, P, SID, true), - [notify_consumer_effect(P, SID, - Stream, ConsumerName, true)]}; + case evaluate_active_consumer(Group0) of + undefined -> + {Group0, []}; + #consumer{pid = P, subscription_id = SID} -> + {update_consumer_state_in_group(Group0, P, SID, + {?CONNECTED, ?ACTIVE}), + [notify_consumer_effect(P, SID, + Stream, ConsumerName, true)]} + end; false -> %% no active consumer in the (non-empty) group, %% we are waiting for the reply of a former active @@ -623,6 +1116,9 @@ handle_consumer_removal(Group0, Stream, ConsumerName, ActiveRemoved) -> end end. +notify_connection_effect(Pid) -> + mod_call_effect(Pid, {sac, check_connection, #{}}). + notify_consumer_effect(Pid, SubId, Stream, Name, Active) -> notify_consumer_effect(Pid, SubId, Stream, Name, Active, false). @@ -675,29 +1171,74 @@ has_consumers_from_pid(#group{consumers = Consumers}, Pid) -> end, Consumers). -compute_active_consumer(#group{consumers = Crs, - partition_index = -1} = - Group) - when length(Crs) == 0 -> - Group; compute_active_consumer(#group{partition_index = -1, - consumers = [Consumer0]} = - Group0) -> - Consumer1 = Consumer0#consumer{active = true}, - Group0#group{consumers = [Consumer1]}; + consumers = Crs} = Group) + when length(Crs) == 0 -> + Group; compute_active_consumer(#group{partition_index = -1, - consumers = [Consumer0 | T]} = - Group0) -> - Consumer1 = Consumer0#consumer{active = true}, - Consumers = lists:map(fun(C) -> C#consumer{active = false} end, T), - Group0#group{consumers = [Consumer1] ++ Consumers}. - -evaluate_active_consumer(#group{partition_index = PartitionIndex, - consumers = Consumers}) - when PartitionIndex >= 0 -> + consumers = Consumers} = G) -> + case lists:search(fun(#consumer{status = S}) -> + S =:= {?DISCONNECTED, ?ACTIVE} + end, Consumers) of + {value, _DisconnectedActive} -> + G; + false -> + case evaluate_active_consumer(G) of + undefined -> + G; + #consumer{pid = Pid, subscription_id = SubId} -> + Consumers1 = + lists:foldr( + fun(#consumer{pid = P, subscription_id = SID} = C, L) + when P =:= Pid andalso SID =:= SubId -> + %% change status of new active + [csr_status(C, ?CONN_ACT) | L]; + (#consumer{status = {?CONNECTED, _}} = C, L) -> + %% other connected consumers are set to "waiting" + [csr_status(C, ?CONN_WAIT) | L]; + (C, L) -> + %% other consumers stay the same + [C | L] + end, [], Consumers), + G#group{consumers = Consumers1} + end + end. + +evaluate_active_consumer(#group{consumers = Consumers}) + when length(Consumers) == 0 -> + undefined; +evaluate_active_consumer(#group{consumers = Consumers} = G) -> + case lists:search(fun(#consumer{status = S}) -> + S =:= ?DISCONN_ACT + end, Consumers) of + {value, C} -> + C; + _ -> + do_evaluate_active_consumer(G#group{consumers = eligible(Consumers)}) + end. + +do_evaluate_active_consumer(#group{consumers = Consumers}) + when length(Consumers) == 0 -> + undefined; +do_evaluate_active_consumer(#group{partition_index = -1, + consumers = [Consumer]}) -> + Consumer; +do_evaluate_active_consumer(#group{partition_index = -1, + consumers = [Consumer | _]}) -> + Consumer; +do_evaluate_active_consumer(#group{partition_index = PartitionIndex, + consumers = Consumers}) + when PartitionIndex >= 0 -> ActiveConsumerIndex = PartitionIndex rem length(Consumers), lists:nth(ActiveConsumerIndex + 1, Consumers). +eligible(Consumers) -> + lists:filter(fun(#consumer{status = {?CONNECTED, _}}) -> + true; + (_) -> + false + end, Consumers). + lookup_consumer(ConnectionPid, SubscriptionId, #group{consumers = Consumers}) -> lists:search(fun(#consumer{pid = ConnPid, subscription_id = SubId}) -> @@ -706,7 +1247,7 @@ lookup_consumer(ConnectionPid, SubscriptionId, Consumers). lookup_active_consumer(#group{consumers = Consumers}) -> - lists:search(fun(#consumer{active = Active}) -> Active end, + lists:search(fun(#consumer{status = Status}) -> is_active(Status) end, Consumers). update_groups(_VirtualHost, @@ -727,22 +1268,30 @@ update_groups(VirtualHost, ConsumerName, Group, StreamGroups) -> - maps:put({VirtualHost, Stream, ConsumerName}, Group, StreamGroups). + StreamGroups#{{VirtualHost, Stream, ConsumerName} => Group}. update_consumer_state_in_group(#group{consumers = Consumers0} = G, Pid, SubId, - NewState) -> + NewStatus) -> CS1 = lists:map(fun(C0) -> case C0 of #consumer{pid = Pid, subscription_id = SubId} -> - C0#consumer{active = NewState}; + csr_status(C0, NewStatus); C -> C end end, Consumers0), G#group{consumers = CS1}. +update_connected_consumers(#group{consumers = Consumers0} = G, NewStatus) -> + Consumers1 = lists:map(fun(#consumer{status = {?CONNECTED, _}} = C) -> + csr_status(C, NewStatus); + (C) -> + C + end, Consumers0), + G#group{consumers = Consumers1}. + mod_call_effect(Pid, Msg) -> {mod_call, rabbit_stream_sac_coordinator, send_message, [Pid, Msg]}. @@ -750,3 +1299,52 @@ mod_call_effect(Pid, Msg) -> send_message(ConnectionPid, Msg) -> ConnectionPid ! Msg, ok. + +same_consumer(#consumer{pid = Pid, subscription_id = SubId}, + #consumer{pid = Pid, subscription_id = SubId}) -> + true; +same_consumer(_, _) -> + false. + +-spec compute_pid_group_dependencies(groups()) -> pids_groups(). +compute_pid_group_dependencies(Groups) -> + maps:fold(fun(K, #group{consumers = Cs}, Acc) -> + lists:foldl(fun(#consumer{pid = Pid}, AccIn) -> + PG0 = maps:get(Pid, AccIn, #{}), + PG1 = PG0#{K => true}, + AccIn#{Pid => PG1} + end, Acc, Cs) + end, #{}, Groups). + +-spec compute_node_pid_group_dependencies(node(), groups()) -> pids_groups(). +compute_node_pid_group_dependencies(Node, Groups) -> + maps:fold(fun(K, #group{consumers = Consumers}, Acc) -> + lists:foldl(fun(#consumer{pid = Pid}, AccIn) + when node(Pid) =:= Node -> + PG0 = maps:get(Pid, AccIn, #{}), + PG1 = PG0#{K => true}, + AccIn#{Pid => PG1}; + (_, AccIn) -> + AccIn + end, Acc, Consumers) + end, #{}, Groups). + +-spec csr(pid(), subscription_id(), owner(), consumer_status()) -> + consumer(). +csr(Pid, Id, Owner, Status) -> + #consumer{pid = Pid, + subscription_id = Id, + owner = Owner, + status = Status, + ts = ts()}. + +-spec csr_status(consumer(), consumer_status()) -> consumer(). +csr_status(C, Status) -> + C#consumer{status = Status, ts = ts()}. + +node_disconnected_timer_effect(Pid, T) -> + {timer, {sac, node_disconnected, + #{connection_pid => Pid}}, T}. + +ts() -> + erlang:system_time(millisecond). diff --git a/deps/rabbit/src/rabbit_stream_sac_coordinator.hrl b/deps/rabbit/src/rabbit_stream_sac_coordinator.hrl index 7e1e7bf9c71d..e94ec1d92bc3 100644 --- a/deps/rabbit/src/rabbit_stream_sac_coordinator.hrl +++ b/deps/rabbit/src/rabbit_stream_sac_coordinator.hrl @@ -22,22 +22,34 @@ -type subscription_id() :: byte(). -type group_id() :: {vhost(), stream(), consumer_name()}. -type owner() :: binary(). +-type consumer_activity() :: active | waiting | deactivating. +-type consumer_connectivity() :: connected | disconnected | presumed_down. +-type consumer_status() :: {consumer_connectivity(), consumer_activity()}. +-type conf() :: map(). +-type timestamp() :: integer(). -record(consumer, {pid :: pid(), subscription_id :: subscription_id(), owner :: owner(), %% just a label - active :: boolean()}). + status :: consumer_status(), + ts :: timestamp()}). -record(group, {consumers :: [#consumer{}], partition_index :: integer()}). -record(rabbit_stream_sac_coordinator, - {groups :: #{group_id() => #group{}}, - pids_groups :: - #{connection_pid() => - #{group_id() => true}}, %% inner map acts as a set + {groups :: groups(), + pids_groups :: pids_groups(), + conf :: conf(), %% future extensibility reserved_1, reserved_2}). + +-type consumer() :: #consumer{}. +-type group() :: #group{}. +-type groups() :: #{group_id() => group()}. +%% inner map acts as a set +-type pids_groups() :: #{connection_pid() => #{group_id() => true}}. + %% commands -record(command_register_consumer, {vhost :: vhost(), @@ -56,3 +68,9 @@ -record(command_activate_consumer, {vhost :: vhost(), stream :: stream(), consumer_name :: consumer_name()}). +-record(command_connection_reconnected, + {pid :: connection_pid()}). +-record(command_purge_nodes, + {nodes :: [node()]}). +-record(command_update_conf, + {conf :: conf()}). diff --git a/deps/rabbit/src/rabbit_stream_sac_coordinator_v4.erl b/deps/rabbit/src/rabbit_stream_sac_coordinator_v4.erl new file mode 100644 index 000000000000..0244e4323dc7 --- /dev/null +++ b/deps/rabbit/src/rabbit_stream_sac_coordinator_v4.erl @@ -0,0 +1,774 @@ +%% The contents of this file are subject to the Mozilla Public License +%% Version 2.0 (the "License"); you may not use this file except in +%% compliance with the License. You may obtain a copy of the License +%% at https://www.mozilla.org/en-US/MPL/2.0/ +%% +%% Software distributed under the License is distributed on an "AS IS" +%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See +%% the License for the specific language governing rights and +%% limitations under the License. +%% +%% The Original Code is RabbitMQ. +%% +%% The Initial Developer of the Original Code is Pivotal Software, Inc. +%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_stream_sac_coordinator_v4). + +-include("rabbit_stream_sac_coordinator_v4.hrl"). + +-opaque command() :: + #command_register_consumer{} | #command_unregister_consumer{} | + #command_activate_consumer{}. +-opaque state() :: #rabbit_stream_sac_coordinator{}. + +-export_type([state/0, + command/0]). + +%% Single Active Consumer API +-export([register_consumer/7, + unregister_consumer/5, + activate_consumer/3, + consumer_groups/2, + group_consumers/4]). +-export([apply/2, + init_state/0, + send_message/2, + ensure_monitors/4, + handle_connection_down/2, + consumer_groups/3, + group_consumers/5, + overview/1, + state_to_map/1]). + +-import(rabbit_stream_coordinator, [ra_local_query/1]). + +-define(STATE, rabbit_stream_sac_coordinator). + +%% Single Active Consumer API +-spec register_consumer(binary(), + binary(), + integer(), + binary(), + pid(), + binary(), + integer()) -> + {ok, boolean()} | {error, term()}. +register_consumer(VirtualHost, + Stream, + PartitionIndex, + ConsumerName, + ConnectionPid, + Owner, + SubscriptionId) -> + process_command({sac, + #command_register_consumer{vhost = + VirtualHost, + stream = + Stream, + partition_index + = + PartitionIndex, + consumer_name + = + ConsumerName, + connection_pid + = + ConnectionPid, + owner = + Owner, + subscription_id + = + SubscriptionId}}). + +-spec unregister_consumer(binary(), + binary(), + binary(), + pid(), + integer()) -> + ok | {error, term()}. +unregister_consumer(VirtualHost, + Stream, + ConsumerName, + ConnectionPid, + SubscriptionId) -> + process_command({sac, + #command_unregister_consumer{vhost = + VirtualHost, + stream = + Stream, + consumer_name + = + ConsumerName, + connection_pid + = + ConnectionPid, + subscription_id + = + SubscriptionId}}). + +-spec activate_consumer(binary(), binary(), binary()) -> ok. +activate_consumer(VirtualHost, Stream, ConsumerName) -> + process_command({sac, + #command_activate_consumer{vhost = + VirtualHost, + stream = + Stream, + consumer_name + = + ConsumerName}}). + +process_command(Cmd) -> + case rabbit_stream_coordinator:process_command(Cmd) of + {ok, Res, _} -> + Res; + {error, _} = Err -> + rabbit_log:warning("SAC coordinator command ~tp returned error ~tp", + [Cmd, Err]), + Err + end. + +%% return the current groups for a given virtual host +-spec consumer_groups(binary(), [atom()]) -> + {ok, + [term()] | {error, atom()}}. +consumer_groups(VirtualHost, InfoKeys) -> + case ra_local_query(fun(State) -> + SacState = + rabbit_stream_coordinator:sac_state(State), + consumer_groups(VirtualHost, + InfoKeys, + SacState) + end) + of + {ok, {_, Result}, _} -> Result; + {error, noproc} -> + %% not started yet, so no groups + {ok, []}; + {error, _} = Err -> Err; + {timeout, _} -> {error, timeout} + end. + +%% get the consumers of a given group in a given virtual host +-spec group_consumers(binary(), binary(), binary(), [atom()]) -> + {ok, [term()]} | + {error, atom()}. +group_consumers(VirtualHost, Stream, Reference, InfoKeys) -> + case ra_local_query(fun(State) -> + SacState = + rabbit_stream_coordinator:sac_state(State), + group_consumers(VirtualHost, + Stream, + Reference, + InfoKeys, + SacState) + end) + of + {ok, {_, {ok, _} = Result}, _} -> Result; + {ok, {_, {error, _} = Err}, _} -> Err; + {error, noproc} -> + %% not started yet, so the group cannot exist + {error, not_found}; + {error, _} = Err -> Err; + {timeout, _} -> {error, timeout} + end. + +-spec overview(state()) -> map(). +overview(undefined) -> + undefined; +overview(#?STATE{groups = Groups}) -> + GroupsOverview = + maps:map(fun(_, + #group{consumers = Consumers, partition_index = Idx}) -> + #{num_consumers => length(Consumers), + partition_index => Idx} + end, + Groups), + #{num_groups => map_size(Groups), groups => GroupsOverview}. + +-spec init_state() -> state(). +init_state() -> + #?STATE{groups = #{}, pids_groups = #{}}. + +-spec apply(command(), state()) -> + {state(), term(), ra_machine:effects()}. +apply(#command_register_consumer{vhost = VirtualHost, + stream = Stream, + partition_index = PartitionIndex, + consumer_name = ConsumerName, + connection_pid = ConnectionPid, + owner = Owner, + subscription_id = SubscriptionId}, + #?STATE{groups = StreamGroups0} = State) -> + StreamGroups1 = + maybe_create_group(VirtualHost, + Stream, + PartitionIndex, + ConsumerName, + StreamGroups0), + + do_register_consumer(VirtualHost, + Stream, + PartitionIndex, + ConsumerName, + ConnectionPid, + Owner, + SubscriptionId, + State#?STATE{groups = StreamGroups1}); +apply(#command_unregister_consumer{vhost = VirtualHost, + stream = Stream, + consumer_name = ConsumerName, + connection_pid = ConnectionPid, + subscription_id = SubscriptionId}, + #?STATE{groups = StreamGroups0} = State0) -> + {State1, Effects1} = + case lookup_group(VirtualHost, Stream, ConsumerName, StreamGroups0) of + undefined -> + {State0, []}; + Group0 -> + {Group1, Effects} = + case lookup_consumer(ConnectionPid, SubscriptionId, Group0) + of + {value, Consumer} -> + G1 = remove_from_group(Consumer, Group0), + handle_consumer_removal(G1, Stream, ConsumerName, Consumer#consumer.active); + false -> + {Group0, []} + end, + SGS = update_groups(VirtualHost, + Stream, + ConsumerName, + Group1, + StreamGroups0), + {State0#?STATE{groups = SGS}, Effects} + end, + {State1, ok, Effects1}; +apply(#command_activate_consumer{vhost = VirtualHost, + stream = Stream, + consumer_name = ConsumerName}, + #?STATE{groups = StreamGroups0} = State0) -> + {G, Eff} = + case lookup_group(VirtualHost, Stream, ConsumerName, StreamGroups0) of + undefined -> + rabbit_log:warning("Trying to activate consumer in group ~tp, but " + "the group does not longer exist", + [{VirtualHost, Stream, ConsumerName}]), + {undefined, []}; + Group -> + #consumer{pid = Pid, subscription_id = SubId} = + evaluate_active_consumer(Group), + Group1 = update_consumer_state_in_group(Group, Pid, SubId, true), + {Group1, [notify_consumer_effect(Pid, SubId, Stream, ConsumerName, true)]} + end, + StreamGroups1 = + update_groups(VirtualHost, Stream, ConsumerName, G, StreamGroups0), + {State0#?STATE{groups = StreamGroups1}, ok, Eff}. + +-spec consumer_groups(binary(), [atom()], state()) -> {ok, [term()]}. +consumer_groups(VirtualHost, InfoKeys, #?STATE{groups = Groups}) -> + Res = maps:fold(fun ({VH, Stream, Reference}, + #group{consumers = Consumers, + partition_index = PartitionIndex}, + Acc) + when VH == VirtualHost -> + Record = + lists:foldr(fun (stream, RecAcc) -> + [{stream, Stream} | RecAcc]; + (reference, RecAcc) -> + [{reference, Reference} + | RecAcc]; + (partition_index, RecAcc) -> + [{partition_index, + PartitionIndex} + | RecAcc]; + (consumers, RecAcc) -> + [{consumers, + length(Consumers)} + | RecAcc]; + (Unknown, RecAcc) -> + [{Unknown, unknown_field} + | RecAcc] + end, + [], InfoKeys), + [Record | Acc]; + (_GroupId, _Group, Acc) -> + Acc + end, + [], Groups), + {ok, lists:reverse(Res)}. + +-spec group_consumers(binary(), + binary(), + binary(), + [atom()], + state()) -> + {ok, [term()]} | {error, not_found}. +group_consumers(VirtualHost, + Stream, + Reference, + InfoKeys, + #?STATE{groups = Groups}) -> + GroupId = {VirtualHost, Stream, Reference}, + case Groups of + #{GroupId := #group{consumers = Consumers}} -> + Cs = lists:foldr(fun(#consumer{subscription_id = SubId, + owner = Owner, + active = Active}, + Acc) -> + Record = + lists:foldr(fun (subscription_id, RecAcc) -> + [{subscription_id, + SubId} + | RecAcc]; + (connection_name, RecAcc) -> + [{connection_name, + Owner} + | RecAcc]; + (state, RecAcc) + when Active -> + [{state, active} + | RecAcc]; + (state, RecAcc) -> + [{state, inactive} + | RecAcc]; + (Unknown, RecAcc) -> + [{Unknown, + unknown_field} + | RecAcc] + end, + [], InfoKeys), + [Record | Acc] + end, + [], Consumers), + {ok, Cs}; + _ -> + {error, not_found} + end. + +-spec ensure_monitors(command(), + state(), + map(), + ra_machine:effects()) -> + {state(), map(), ra_machine:effects()}. +ensure_monitors(#command_register_consumer{vhost = VirtualHost, + stream = Stream, + consumer_name = ConsumerName, + connection_pid = Pid}, + #?STATE{pids_groups = PidsGroups0} = State0, + Monitors0, + Effects) -> + GroupId = {VirtualHost, Stream, ConsumerName}, + Groups0 = maps:get(Pid, PidsGroups0, #{}), + PidsGroups1 = + maps:put(Pid, maps:put(GroupId, true, Groups0), PidsGroups0), + {State0#?STATE{pids_groups = PidsGroups1}, Monitors0#{Pid => sac}, + [{monitor, process, Pid}, {monitor, node, node(Pid)} | Effects]}; +ensure_monitors(#command_unregister_consumer{vhost = VirtualHost, + stream = Stream, + consumer_name = ConsumerName, + connection_pid = Pid}, + #?STATE{groups = StreamGroups0, pids_groups = PidsGroups0} = + State0, + Monitors, + Effects) + when is_map_key(Pid, PidsGroups0) -> + GroupId = {VirtualHost, Stream, ConsumerName}, + #{Pid := PidGroup0} = PidsGroups0, + PidGroup1 = + case lookup_group(VirtualHost, Stream, ConsumerName, StreamGroups0) of + undefined -> + %% group is gone, can be removed from the PID map + maps:remove(GroupId, PidGroup0); + Group -> + %% group still exists, check if other consumers are from this PID + %% if yes, don't change the PID set + %% if no, remove group from PID set + case has_consumers_from_pid(Group, Pid) of + true -> + %% the group still depends on this PID, keep the group entry in the set + PidGroup0; + false -> + %% the group does not depend on the PID anymore, remove the group entry from the map + maps:remove(GroupId, PidGroup0) + end + end, + case maps:size(PidGroup1) == 0 of + true -> + %% no more groups depend on the PID + %% remove PID from data structure and demonitor it + {State0#?STATE{pids_groups = maps:remove(Pid, PidsGroups0)}, + maps:remove(Pid, Monitors), [{demonitor, process, Pid} | Effects]}; + false -> + %% one or more groups still depend on the PID + {State0#?STATE{pids_groups = + maps:put(Pid, PidGroup1, PidsGroups0)}, + Monitors, Effects} + end; +ensure_monitors(_, #?STATE{} = State0, Monitors, Effects) -> + {State0, Monitors, Effects}. + +-spec handle_connection_down(connection_pid(), state()) -> + {state(), ra_machine:effects()}. +handle_connection_down(Pid, + #?STATE{pids_groups = PidsGroups0} = State0) -> + case maps:take(Pid, PidsGroups0) of + error -> + {State0, []}; + {Groups, PidsGroups1} -> + State1 = State0#?STATE{pids_groups = PidsGroups1}, + maps:fold(fun(G, _, Acc) -> + handle_group_after_connection_down(Pid, Acc, G) + end, {State1, []}, Groups) + end. + +handle_group_after_connection_down(Pid, + {#?STATE{groups = Groups0} = S0, Eff0}, + {VirtualHost, Stream, ConsumerName}) -> + case lookup_group(VirtualHost, + Stream, + ConsumerName, + Groups0) of + undefined -> + {S0, Eff0}; + #group{consumers = Consumers0} = G0 -> + %% remove the connection consumers from the group state + %% keep flags to know what happened + {Consumers1, ActiveRemoved, AnyRemoved} = + lists:foldl( + fun(#consumer{pid = P, active = S}, {L, ActiveFlag, _}) when P == Pid -> + {L, S or ActiveFlag, true}; + (C, {L, ActiveFlag, AnyFlag}) -> + {L ++ [C], ActiveFlag, AnyFlag} + end, {[], false, false}, Consumers0), + + case AnyRemoved of + true -> + G1 = G0#group{consumers = Consumers1}, + {G2, Effects} = handle_consumer_removal(G1, Stream, ConsumerName, ActiveRemoved), + Groups1 = update_groups(VirtualHost, + Stream, + ConsumerName, + G2, + Groups0), + {S0#?STATE{groups = Groups1}, Effects ++ Eff0}; + false -> + {S0, Eff0} + end + end. + +-spec state_to_map(state()) -> map(). +state_to_map(#?STATE{groups = Groups, pids_groups = PidsGroups}) -> + #{<<"groups">> => groups_to_map(Groups), + <<"pids_groups">> => pids_groups_to_map(PidsGroups)}. + +groups_to_map(Groups) when is_map(Groups) -> + maps:fold(fun(K, V, Acc) -> + Acc#{K => group_to_map(V)} + end, #{}, Groups). + +pids_groups_to_map(PidsGroups) when is_map(PidsGroups) -> + PidsGroups. + +group_to_map(#group{consumers = Consumers, partition_index = Index}) -> + OutConsumers = lists:foldl(fun(C, Acc) -> + Acc ++ [consumer_to_map(C)] + end, [], Consumers), + #{<<"consumers">> => OutConsumers, <<"partition_index">> => Index}. + +consumer_to_map(#consumer{pid = Pid, subscription_id = SubId, + owner = Owner, active = Active}) -> + #{<<"pid">> => Pid, <<"subscription_id">> => SubId, + <<"owner">> => Owner, <<"active">> => Active}. + +do_register_consumer(VirtualHost, + Stream, + -1 = _PartitionIndex, + ConsumerName, + ConnectionPid, + Owner, + SubscriptionId, + #?STATE{groups = StreamGroups0} = State) -> + Group0 = + lookup_group(VirtualHost, Stream, ConsumerName, StreamGroups0), + + Consumer = + case lookup_active_consumer(Group0) of + {value, _} -> + #consumer{pid = ConnectionPid, + owner = Owner, + subscription_id = SubscriptionId, + active = false}; + false -> + #consumer{pid = ConnectionPid, + subscription_id = SubscriptionId, + owner = Owner, + active = true} + end, + Group1 = add_to_group(Consumer, Group0), + StreamGroups1 = + update_groups(VirtualHost, + Stream, + ConsumerName, + Group1, + StreamGroups0), + + #consumer{active = Active} = Consumer, + Effects = + case Active of + true -> + [notify_consumer_effect(ConnectionPid, SubscriptionId, + Stream, ConsumerName, Active)]; + _ -> + [] + end, + + {State#?STATE{groups = StreamGroups1}, {ok, Active}, Effects}; +do_register_consumer(VirtualHost, + Stream, + _PartitionIndex, + ConsumerName, + ConnectionPid, + Owner, + SubscriptionId, + #?STATE{groups = StreamGroups0} = State) -> + Group0 = + lookup_group(VirtualHost, Stream, ConsumerName, StreamGroups0), + + {Group1, Effects} = + case Group0 of + #group{consumers = []} -> + %% first consumer in the group, it's the active one + Consumer0 = + #consumer{pid = ConnectionPid, + owner = Owner, + subscription_id = SubscriptionId, + active = true}, + G1 = add_to_group(Consumer0, Group0), + {G1, + [notify_consumer_effect(ConnectionPid, SubscriptionId, + Stream, ConsumerName, true)]}; + _G -> + %% whatever the current state is, the newcomer will be passive + Consumer0 = + #consumer{pid = ConnectionPid, + owner = Owner, + subscription_id = SubscriptionId, + active = false}, + G1 = add_to_group(Consumer0, Group0), + + case lookup_active_consumer(G1) of + {value, + #consumer{pid = ActPid, subscription_id = ActSubId} = + CurrentActive} -> + case evaluate_active_consumer(G1) of + CurrentActive -> + %% the current active stays the same + {G1, []}; + _ -> + %% there's a change, telling the active it's not longer active + {update_consumer_state_in_group(G1, + ActPid, + ActSubId, + false), + [notify_consumer_effect(ActPid, + ActSubId, + Stream, + ConsumerName, + false, + true)]} + end; + false -> + %% no active consumer in the (non-empty) group, + %% we are waiting for the reply of a former active + {G1, []} + end + end, + StreamGroups1 = + update_groups(VirtualHost, + Stream, + ConsumerName, + Group1, + StreamGroups0), + {value, #consumer{active = Active}} = + lookup_consumer(ConnectionPid, SubscriptionId, Group1), + {State#?STATE{groups = StreamGroups1}, {ok, Active}, Effects}. + +handle_consumer_removal(#group{consumers = []} = G, _, _, _) -> + {G, []}; +handle_consumer_removal(#group{partition_index = -1} = Group0, + Stream, ConsumerName, ActiveRemoved) -> + case ActiveRemoved of + true -> + %% this is the active consumer we remove, computing the new one + Group1 = compute_active_consumer(Group0), + case lookup_active_consumer(Group1) of + {value, #consumer{pid = Pid, subscription_id = SubId}} -> + %% creating the side effect to notify the new active consumer + {Group1, [notify_consumer_effect(Pid, SubId, Stream, ConsumerName, true)]}; + _ -> + %% no active consumer found in the group, nothing to do + {Group1, []} + end; + false -> + %% not the active consumer, nothing to do. + {Group0, []} + end; +handle_consumer_removal(Group0, Stream, ConsumerName, ActiveRemoved) -> + case lookup_active_consumer(Group0) of + {value, + #consumer{pid = ActPid, subscription_id = ActSubId} = + CurrentActive} -> + case evaluate_active_consumer(Group0) of + CurrentActive -> + %% the current active stays the same + {Group0, []}; + _ -> + %% there's a change, telling the active it's not longer active + {update_consumer_state_in_group(Group0, + ActPid, + ActSubId, + false), + [notify_consumer_effect(ActPid, ActSubId, + Stream, ConsumerName, false, true)]} + end; + false -> + case ActiveRemoved of + true -> + %% the active one is going away, picking a new one + #consumer{pid = P, subscription_id = SID} = + evaluate_active_consumer(Group0), + {update_consumer_state_in_group(Group0, P, SID, true), + [notify_consumer_effect(P, SID, + Stream, ConsumerName, true)]}; + false -> + %% no active consumer in the (non-empty) group, + %% we are waiting for the reply of a former active + {Group0, []} + end + end. + +notify_consumer_effect(Pid, SubId, Stream, Name, Active) -> + notify_consumer_effect(Pid, SubId, Stream, Name, Active, false). + +notify_consumer_effect(Pid, SubId, Stream, Name, Active, false = _SteppingDown) -> + mod_call_effect(Pid, + {sac, #{subscription_id => SubId, + stream => Stream, + consumer_name => Name, + active => Active}}); +notify_consumer_effect(Pid, SubId, Stream, Name, Active, true = SteppingDown) -> + mod_call_effect(Pid, + {sac, #{subscription_id => SubId, + stream => Stream, + consumer_name => Name, + active => Active, + stepping_down => SteppingDown}}). + +maybe_create_group(VirtualHost, + Stream, + PartitionIndex, + ConsumerName, + StreamGroups) -> + case StreamGroups of + #{{VirtualHost, Stream, ConsumerName} := _Group} -> + StreamGroups; + SGS -> + maps:put({VirtualHost, Stream, ConsumerName}, + #group{consumers = [], partition_index = PartitionIndex}, + SGS) + end. + +lookup_group(VirtualHost, Stream, ConsumerName, StreamGroups) -> + maps:get({VirtualHost, Stream, ConsumerName}, StreamGroups, + undefined). + +add_to_group(Consumer, #group{consumers = Consumers} = Group) -> + Group#group{consumers = Consumers ++ [Consumer]}. + +remove_from_group(Consumer, #group{consumers = Consumers} = Group) -> + Group#group{consumers = lists:delete(Consumer, Consumers)}. + +has_consumers_from_pid(#group{consumers = Consumers}, Pid) -> + lists:any(fun (#consumer{pid = P}) when P == Pid -> + true; + (_) -> + false + end, + Consumers). + +compute_active_consumer(#group{consumers = Crs, + partition_index = -1} = + Group) + when length(Crs) == 0 -> + Group; +compute_active_consumer(#group{partition_index = -1, + consumers = [Consumer0]} = + Group0) -> + Consumer1 = Consumer0#consumer{active = true}, + Group0#group{consumers = [Consumer1]}; +compute_active_consumer(#group{partition_index = -1, + consumers = [Consumer0 | T]} = + Group0) -> + Consumer1 = Consumer0#consumer{active = true}, + Consumers = lists:map(fun(C) -> C#consumer{active = false} end, T), + Group0#group{consumers = [Consumer1] ++ Consumers}. + +evaluate_active_consumer(#group{partition_index = PartitionIndex, + consumers = Consumers}) + when PartitionIndex >= 0 -> + ActiveConsumerIndex = PartitionIndex rem length(Consumers), + lists:nth(ActiveConsumerIndex + 1, Consumers). + +lookup_consumer(ConnectionPid, SubscriptionId, + #group{consumers = Consumers}) -> + lists:search(fun(#consumer{pid = ConnPid, subscription_id = SubId}) -> + ConnPid == ConnectionPid andalso SubId == SubscriptionId + end, + Consumers). + +lookup_active_consumer(#group{consumers = Consumers}) -> + lists:search(fun(#consumer{active = Active}) -> Active end, + Consumers). + +update_groups(_VirtualHost, + _Stream, + _ConsumerName, + undefined, + StreamGroups) -> + StreamGroups; +update_groups(VirtualHost, + Stream, + ConsumerName, + #group{consumers = []}, + StreamGroups) -> + %% the group is now empty, removing the key + maps:remove({VirtualHost, Stream, ConsumerName}, StreamGroups); +update_groups(VirtualHost, + Stream, + ConsumerName, + Group, + StreamGroups) -> + maps:put({VirtualHost, Stream, ConsumerName}, Group, StreamGroups). + +update_consumer_state_in_group(#group{consumers = Consumers0} = G, + Pid, + SubId, + NewState) -> + CS1 = lists:map(fun(C0) -> + case C0 of + #consumer{pid = Pid, subscription_id = SubId} -> + C0#consumer{active = NewState}; + C -> C + end + end, + Consumers0), + G#group{consumers = CS1}. + +mod_call_effect(Pid, Msg) -> + {mod_call, rabbit_stream_sac_coordinator, send_message, [Pid, Msg]}. + +-spec send_message(pid(), term()) -> ok. +send_message(ConnectionPid, Msg) -> + ConnectionPid ! Msg, + ok. diff --git a/deps/rabbit/src/rabbit_stream_sac_coordinator_v4.hrl b/deps/rabbit/src/rabbit_stream_sac_coordinator_v4.hrl new file mode 100644 index 000000000000..7e1e7bf9c71d --- /dev/null +++ b/deps/rabbit/src/rabbit_stream_sac_coordinator_v4.hrl @@ -0,0 +1,58 @@ +%% The contents of this file are subject to the Mozilla Public License +%% Version 2.0 (the "License"); you may not use this file except in +%% compliance with the License. You may obtain a copy of the License +%% at https://www.mozilla.org/en-US/MPL/2.0/ +%% +%% Software distributed under the License is distributed on an "AS IS" +%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See +%% the License for the specific language governing rights and +%% limitations under the License. +%% +%% The Original Code is RabbitMQ. +%% +%% The Initial Developer of the Original Code is Pivotal Software, Inc. +%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-type vhost() :: binary(). +-type partition_index() :: integer(). +-type stream() :: binary(). +-type consumer_name() :: binary(). +-type connection_pid() :: pid(). +-type subscription_id() :: byte(). +-type group_id() :: {vhost(), stream(), consumer_name()}. +-type owner() :: binary(). + +-record(consumer, + {pid :: pid(), + subscription_id :: subscription_id(), + owner :: owner(), %% just a label + active :: boolean()}). +-record(group, + {consumers :: [#consumer{}], partition_index :: integer()}). +-record(rabbit_stream_sac_coordinator, + {groups :: #{group_id() => #group{}}, + pids_groups :: + #{connection_pid() => + #{group_id() => true}}, %% inner map acts as a set + %% future extensibility + reserved_1, + reserved_2}). +%% commands +-record(command_register_consumer, + {vhost :: vhost(), + stream :: stream(), + partition_index :: partition_index(), + consumer_name :: consumer_name(), + connection_pid :: connection_pid(), + owner :: owner(), + subscription_id :: subscription_id()}). +-record(command_unregister_consumer, + {vhost :: vhost(), + stream :: stream(), + consumer_name :: consumer_name(), + connection_pid :: connection_pid(), + subscription_id :: subscription_id()}). +-record(command_activate_consumer, + {vhost :: vhost(), stream :: stream(), + consumer_name :: consumer_name()}). diff --git a/deps/rabbit/test/rabbit_stream_coordinator_SUITE.erl b/deps/rabbit/test/rabbit_stream_coordinator_SUITE.erl index b965ad167b63..6e12c8c313c2 100644 --- a/deps/rabbit/test/rabbit_stream_coordinator_SUITE.erl +++ b/deps/rabbit/test/rabbit_stream_coordinator_SUITE.erl @@ -1363,7 +1363,7 @@ delete_replica_leader(_) -> ok. overview(_Config) -> - S0 = rabbit_stream_coordinator:init(undefined), + S0 = rabbit_stream_coordinator:init(#{machine_version => 5}), O0 = rabbit_stream_coordinator:overview(S0), ?assertMatch(#{num_monitors := 0, num_streams := 0, diff --git a/deps/rabbit/test/rabbit_stream_sac_coordinator_SUITE.erl b/deps/rabbit/test/rabbit_stream_sac_coordinator_SUITE.erl index 0a54ce4f05f6..59d4e64a8082 100644 --- a/deps/rabbit/test/rabbit_stream_sac_coordinator_SUITE.erl +++ b/deps/rabbit/test/rabbit_stream_sac_coordinator_SUITE.erl @@ -26,6 +26,7 @@ %%%=================================================================== -define(STATE, rabbit_stream_sac_coordinator). +-define(MOD, rabbit_stream_sac_coordinator). all() -> [{group, tests}]. @@ -60,6 +61,19 @@ end_per_testcase(_TestCase, _Config) -> meck:unload(), ok. +check_conf_test(_) -> + K = disconnected_timeout, + Def = 60_000, + ?assertMatch({new, #{K := Def}}, + ?MOD:check_conf_change(state_with_conf(#{}))), + ?assertMatch({new, #{K := Def}}, + ?MOD:check_conf_change(state_with_conf(#{K => 42}))), + ?assertMatch(unchanged, + ?MOD:check_conf_change(state_with_conf(#{K => Def}))), + ?assertMatch(unchanged, + ?MOD:check_conf_change(#{K => Def})), + ok. + simple_sac_test(_) -> Stream = <<"stream">>, ConsumerName = <<"app">>, @@ -69,62 +83,56 @@ simple_sac_test(_) -> register_consumer_command(Stream, -1, ConsumerName, ConnectionPid, 0), State0 = state(), {#?STATE{groups = #{GroupId := #group{consumers = Consumers1}}} = - State1, - {ok, Active1}, Effects1} = - rabbit_stream_sac_coordinator:apply(Command0, State0), + State1, + {ok, Active1}, Effects1} = ?MOD:apply(Command0, State0), ?assert(Active1), - ?assertEqual([consumer(ConnectionPid, 0, true)], Consumers1), - assertSendMessageEffect(ConnectionPid, 0, Stream, ConsumerName, true, Effects1), + assertCsrsEqual([csr(ConnectionPid, 0, active)], Consumers1), + assertSendMessageActivateEffect(ConnectionPid, 0, Stream, ConsumerName, true, Effects1), Command1 = register_consumer_command(Stream, -1, ConsumerName, ConnectionPid, 1), {#?STATE{groups = #{GroupId := #group{consumers = Consumers2}}} = - State2, - {ok, Active2}, Effects2} = - rabbit_stream_sac_coordinator:apply(Command1, State1), + State2, + {ok, Active2}, Effects2} = ?MOD:apply(Command1, State1), ?assertNot(Active2), - ?assertEqual([consumer(ConnectionPid, 0, true), - consumer(ConnectionPid, 1, false)], - Consumers2), + assertCsrsEqual([csr(ConnectionPid, 0, active), + csr(ConnectionPid, 1, waiting)], + Consumers2), assertEmpty(Effects2), Command2 = register_consumer_command(Stream, -1, ConsumerName, ConnectionPid, 2), {#?STATE{groups = #{GroupId := #group{consumers = Consumers3}}} = - State3, - {ok, Active3}, Effects3} = - rabbit_stream_sac_coordinator:apply(Command2, State2), + State3, + {ok, Active3}, Effects3} = ?MOD:apply(Command2, State2), ?assertNot(Active3), - ?assertEqual([consumer(ConnectionPid, 0, true), - consumer(ConnectionPid, 1, false), - consumer(ConnectionPid, 2, false)], - Consumers3), + assertCsrsEqual([csr(ConnectionPid, 0, active), + csr(ConnectionPid, 1, waiting), + csr(ConnectionPid, 2, waiting)], + Consumers3), assertEmpty(Effects3), Command3 = unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 0), {#?STATE{groups = #{GroupId := #group{consumers = Consumers4}}} = - State4, - ok, Effects4} = - rabbit_stream_sac_coordinator:apply(Command3, State3), - ?assertEqual([consumer(ConnectionPid, 1, true), - consumer(ConnectionPid, 2, false)], - Consumers4), - assertSendMessageEffect(ConnectionPid, 1, Stream, ConsumerName, true, Effects4), + State4, + ok, Effects4} = ?MOD:apply(Command3, State3), + assertCsrsEqual([csr(ConnectionPid, 1, active), + csr(ConnectionPid, 2, waiting)], + Consumers4), + assertSendMessageActivateEffect(ConnectionPid, 1, Stream, ConsumerName, true, Effects4), Command4 = unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 1), {#?STATE{groups = #{GroupId := #group{consumers = Consumers5}}} = - State5, - ok, Effects5} = - rabbit_stream_sac_coordinator:apply(Command4, State4), - ?assertEqual([consumer(ConnectionPid, 2, true)], Consumers5), - assertSendMessageEffect(ConnectionPid, 2, Stream, ConsumerName, true, Effects5), + State5, + ok, Effects5} = ?MOD:apply(Command4, State4), + assertCsrsEqual([csr(ConnectionPid, 2, active)], Consumers5), + assertSendMessageActivateEffect(ConnectionPid, 2, Stream, ConsumerName, true, Effects5), Command5 = unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 2), - {#?STATE{groups = Groups6}, ok, Effects6} = - rabbit_stream_sac_coordinator:apply(Command5, State5), + {#?STATE{groups = Groups6}, ok, Effects6} = ?MOD:apply(Command5, State5), assertEmpty(Groups6), assertEmpty(Effects6), @@ -139,93 +147,85 @@ super_stream_partition_sac_test(_) -> register_consumer_command(Stream, 1, ConsumerName, ConnectionPid, 0), State0 = state(), {#?STATE{groups = #{GroupId := #group{consumers = Consumers1}}} = - State1, - {ok, Active1}, Effects1} = - rabbit_stream_sac_coordinator:apply(Command0, State0), + State1, + {ok, Active1}, Effects1} = ?MOD:apply(Command0, State0), ?assert(Active1), - ?assertEqual([consumer(ConnectionPid, 0, true)], Consumers1), - assertSendMessageEffect(ConnectionPid, 0, Stream, ConsumerName, true, Effects1), + assertCsrsEqual([csr(ConnectionPid, 0, active)], Consumers1), + assertSendMessageActivateEffect(ConnectionPid, 0, Stream, ConsumerName, true, Effects1), Command1 = register_consumer_command(Stream, 1, ConsumerName, ConnectionPid, 1), {#?STATE{groups = #{GroupId := #group{consumers = Consumers2}}} = - State2, - {ok, Active2}, Effects2} = - rabbit_stream_sac_coordinator:apply(Command1, State1), + State2, + {ok, Active2}, Effects2} = ?MOD:apply(Command1, State1), %% never active on registration ?assertNot(Active2), %% all consumers inactive, until the former active one steps down and activates the new consumer - ?assertEqual([consumer(ConnectionPid, 0, false), - consumer(ConnectionPid, 1, false)], - Consumers2), + assertCsrsEqual([csr(ConnectionPid, 0, deactivating), + csr(ConnectionPid, 1, waiting)], + Consumers2), assertSendMessageSteppingDownEffect(ConnectionPid, 0, Stream, ConsumerName, Effects2), Command2 = activate_consumer_command(Stream, ConsumerName), {#?STATE{groups = #{GroupId := #group{consumers = Consumers3}}} = - State3, - ok, Effects3} = - rabbit_stream_sac_coordinator:apply(Command2, State2), + State3, + ok, Effects3} = ?MOD:apply(Command2, State2), %% 1 (partition index) % 2 (consumer count) = 1 (active consumer index) - ?assertEqual([consumer(ConnectionPid, 0, false), - consumer(ConnectionPid, 1, true)], - Consumers3), - assertSendMessageEffect(ConnectionPid, 1, Stream, ConsumerName, true, Effects3), + assertCsrsEqual([csr(ConnectionPid, 0, waiting), + csr(ConnectionPid, 1, active)], + Consumers3), + assertSendMessageActivateEffect(ConnectionPid, 1, Stream, ConsumerName, true, Effects3), Command3 = register_consumer_command(Stream, 1, ConsumerName, ConnectionPid, 2), {#?STATE{groups = #{GroupId := #group{consumers = Consumers4}}} = - State4, - {ok, Active4}, Effects4} = - rabbit_stream_sac_coordinator:apply(Command3, State3), + State4, + {ok, Active4}, Effects4} = ?MOD:apply(Command3, State3), %% never active on registration ?assertNot(Active4), %% 1 (partition index) % 3 (consumer count) = 1 (active consumer index) %% the active consumer stays the same - ?assertEqual([consumer(ConnectionPid, 0, false), - consumer(ConnectionPid, 1, true), - consumer(ConnectionPid, 2, false)], - Consumers4), + assertCsrsEqual([csr(ConnectionPid, 0, waiting), + csr(ConnectionPid, 1, active), + csr(ConnectionPid, 2, waiting)], + Consumers4), assertEmpty(Effects4), Command4 = unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 0), {#?STATE{groups = #{GroupId := #group{consumers = Consumers5}}} = - State5, - ok, Effects5} = - rabbit_stream_sac_coordinator:apply(Command4, State4), + State5, + ok, Effects5} = ?MOD:apply(Command4, State4), %% 1 (partition index) % 2 (consumer count) = 1 (active consumer index) %% the active consumer will move from sub 1 to sub 2 - ?assertEqual([consumer(ConnectionPid, 1, false), - consumer(ConnectionPid, 2, false)], - Consumers5), + assertCsrsEqual([csr(ConnectionPid, 1, deactivating), + csr(ConnectionPid, 2, waiting)], + Consumers5), assertSendMessageSteppingDownEffect(ConnectionPid, 1, Stream, ConsumerName, Effects5), Command5 = activate_consumer_command(Stream, ConsumerName), {#?STATE{groups = #{GroupId := #group{consumers = Consumers6}}} = - State6, - ok, Effects6} = - rabbit_stream_sac_coordinator:apply(Command5, State5), + State6, + ok, Effects6} = ?MOD:apply(Command5, State5), - ?assertEqual([consumer(ConnectionPid, 1, false), - consumer(ConnectionPid, 2, true)], - Consumers6), - assertSendMessageEffect(ConnectionPid, 2, Stream, ConsumerName, true, Effects6), + assertCsrsEqual([csr(ConnectionPid, 1, waiting), + csr(ConnectionPid, 2, active)], + Consumers6), + assertSendMessageActivateEffect(ConnectionPid, 2, Stream, ConsumerName, true, Effects6), Command6 = unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 1), {#?STATE{groups = #{GroupId := #group{consumers = Consumers7}}} = - State7, - ok, Effects7} = - rabbit_stream_sac_coordinator:apply(Command6, State6), - ?assertEqual([consumer(ConnectionPid, 2, true)], Consumers7), + State7, + ok, Effects7} = ?MOD:apply(Command6, State6), + assertCsrsEqual([csr(ConnectionPid, 2, active)], Consumers7), assertEmpty(Effects7), Command7 = unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 2), - {#?STATE{groups = Groups8}, ok, Effects8} = - rabbit_stream_sac_coordinator:apply(Command7, State7), + {#?STATE{groups = Groups8}, ok, Effects8} = ?MOD:apply(Command7, State7), assertEmpty(Groups8), assertEmpty(Effects8), @@ -233,50 +233,44 @@ super_stream_partition_sac_test(_) -> ensure_monitors_test(_) -> GroupId = {<<"/">>, <<"stream">>, <<"app">>}, - Group = - cgroup([consumer(self(), 0, true), consumer(self(), 1, false)]), - State0 = state(#{GroupId => Group}), + Group = grp([csr(self(), 0, true), csr(self(), 1, false)]), + State0 = state(#{GroupId => Group}, #{}), Monitors0 = #{}, Command0 = register_consumer_command(<<"stream">>, -1, <<"app">>, self(), 0), {#?STATE{pids_groups = PidsGroups1} = State1, Monitors1, Effects1} = - rabbit_stream_sac_coordinator:ensure_monitors(Command0, - State0, - Monitors0, - []), + ?MOD:ensure_monitors(Command0, + State0, + Monitors0, + []), assertSize(1, PidsGroups1), assertSize(1, maps:get(self(), PidsGroups1)), ?assertEqual(#{self() => sac}, Monitors1), ?assertEqual([{monitor, process, self()}, {monitor, node, node()}], Effects1), - Command1 = - register_consumer_command(<<"stream">>, -1, <<"app">>, self(), 1), + Command1 = register_consumer_command(<<"stream">>, -1, <<"app">>, self(), 1), {#?STATE{pids_groups = PidsGroups2} = State2, Monitors2, Effects2} = - rabbit_stream_sac_coordinator:ensure_monitors(Command1, - State1, - Monitors1, - []), + ?MOD:ensure_monitors(Command1, + State1, + Monitors1, + []), assertSize(1, PidsGroups2), assertSize(1, maps:get(self(), PidsGroups2)), ?assertEqual(#{self() => sac}, Monitors2), ?assertEqual([{monitor, process, self()}, {monitor, node, node()}], Effects2), - Group2 = cgroup([consumer(self(), 1, true)]), + Group2 = grp([csr(self(), 1, true)]), - Command2 = - unregister_consumer_command(<<"stream">>, <<"app">>, self(), 0), + Command2 = unregister_consumer_command(<<"stream">>, <<"app">>, self(), 0), {#?STATE{pids_groups = PidsGroups3} = State3, Monitors3, Effects3} = - rabbit_stream_sac_coordinator:ensure_monitors(Command2, - State2#?STATE{groups = - #{GroupId - => - Group2}}, - Monitors2, - []), + ?MOD:ensure_monitors(Command2, + State2#?STATE{groups = #{GroupId => Group2}}, + Monitors2, + []), assertSize(1, PidsGroups3), assertSize(1, maps:get(self(), PidsGroups3)), ?assertEqual(#{self() => sac}, Monitors3), @@ -284,28 +278,26 @@ ensure_monitors_test(_) -> %% trying with an unknown connection PID %% the function should not change anything - UnknownConnectionPid = spawn(fun() -> ok end), - PassthroughCommand = - unregister_consumer_command(<<"stream">>, - <<"app">>, - UnknownConnectionPid, - 0), + UnknownConnectionPid = new_process(), + PassthroughCommand = unregister_consumer_command(<<"stream">>, + <<"app">>, + UnknownConnectionPid, + 0), {State3, Monitors3, Effects3} = - rabbit_stream_sac_coordinator:ensure_monitors(PassthroughCommand, - State3, - Monitors3, - []), + ?MOD:ensure_monitors(PassthroughCommand, + State3, + Monitors3, + []), Command3 = unregister_consumer_command(<<"stream">>, <<"app">>, self(), 1), {#?STATE{pids_groups = PidsGroups4} = _State4, Monitors4, Effects4} = - rabbit_stream_sac_coordinator:ensure_monitors(Command3, - State3#?STATE{groups = - #{}}, - Monitors3, - []), + ?MOD:ensure_monitors(Command3, + State3#?STATE{groups = #{}}, + Monitors3, + []), assertEmpty(PidsGroups4), assertEmpty(Monitors4), ?assertEqual([{demonitor, process, self()}], Effects4), @@ -317,24 +309,20 @@ handle_connection_down_sac_should_get_activated_test(_) -> ConsumerName = <<"app">>, GroupId = {<<"/">>, Stream, ConsumerName}, Pid0 = self(), - Pid1 = spawn(fun() -> ok end), - Group = cgroup([consumer(Pid0, 0, true), - consumer(Pid1, 1, false), - consumer(Pid0, 2, false)]), - State0 = state(#{GroupId => Group}, - #{Pid0 => maps:from_list([{GroupId, true}]), - Pid1 => maps:from_list([{GroupId, true}])}), + Pid1 = new_process(), + Group = grp([csr(Pid0, 0, active), + csr(Pid1, 1, waiting), + csr(Pid0, 2, waiting)]), + State0 = state(#{GroupId => Group}), {#?STATE{pids_groups = PidsGroups1, groups = Groups1} = State1, - Effects1} = - rabbit_stream_sac_coordinator:handle_connection_down(Pid0, State0), + Effects1} = ?MOD:handle_connection_down(Pid0, normal, State0), assertSize(1, PidsGroups1), assertSize(1, maps:get(Pid1, PidsGroups1)), - assertSendMessageEffect(Pid1, 1, Stream, ConsumerName, true, Effects1), - assertHasGroup(GroupId, cgroup([consumer(Pid1, 1, true)]), Groups1), + assertSendMessageActivateEffect(Pid1, 1, Stream, ConsumerName, true, Effects1), + assertHasGroup(GroupId, grp([csr(Pid1, 1, active)]), Groups1), {#?STATE{pids_groups = PidsGroups2, groups = Groups2}, - Effects2} = - rabbit_stream_sac_coordinator:handle_connection_down(Pid1, State1), + Effects2} = ?MOD:handle_connection_down(Pid1, normal, State1), assertEmpty(PidsGroups2), assertEmpty(Effects2), assertEmpty(Groups2), @@ -346,21 +334,18 @@ handle_connection_down_sac_active_does_not_change_test(_) -> ConsumerName = <<"app">>, GroupId = {<<"/">>, Stream, ConsumerName}, Pid0 = self(), - Pid1 = spawn(fun() -> ok end), - Group = cgroup([consumer(Pid1, 0, true), - consumer(Pid0, 1, false), - consumer(Pid0, 2, false)]), - State = state(#{GroupId => Group}, - #{Pid0 => maps:from_list([{GroupId, true}]), - Pid1 => maps:from_list([{GroupId, true}])}), + Pid1 = new_process(), + Group = grp([csr(Pid1, 0, active), + csr(Pid0, 1, waiting), + csr(Pid0, 2, waiting)]), + State = state(#{GroupId => Group}), {#?STATE{pids_groups = PidsGroups, groups = Groups}, - Effects} = - rabbit_stream_sac_coordinator:handle_connection_down(Pid0, State), + Effects} = ?MOD:handle_connection_down(Pid0, normal, State), assertSize(1, PidsGroups), assertSize(1, maps:get(Pid1, PidsGroups)), assertEmpty(Effects), - assertHasGroup(GroupId, cgroup([consumer(Pid1, 0, true)]), Groups), + assertHasGroup(GroupId, grp([csr(Pid1, 0, active)]), Groups), ok. handle_connection_down_sac_no_more_consumers_test(_) -> @@ -368,14 +353,12 @@ handle_connection_down_sac_no_more_consumers_test(_) -> ConsumerName = <<"app">>, GroupId = {<<"/">>, Stream, ConsumerName}, Pid0 = self(), - Group = cgroup([consumer(Pid0, 0, true), - consumer(Pid0, 1, false)]), - State = state(#{GroupId => Group}, - #{Pid0 => maps:from_list([{GroupId, true}])}), + Group = grp([csr(Pid0, 0, active), + csr(Pid0, 1, waiting)]), + State = state(#{GroupId => Group}), {#?STATE{pids_groups = PidsGroups, groups = Groups}, - Effects} = - rabbit_stream_sac_coordinator:handle_connection_down(Pid0, State), + Effects} = ?MOD:handle_connection_down(Pid0, normal, State), assertEmpty(PidsGroups), assertEmpty(Groups), assertEmpty(Effects), @@ -386,21 +369,21 @@ handle_connection_down_sac_no_consumers_in_down_connection_test(_) -> ConsumerName = <<"app">>, GroupId = {<<"/">>, Stream, ConsumerName}, Pid0 = self(), - Pid1 = spawn(fun() -> ok end), - Group = cgroup([consumer(Pid1, 0, true), - consumer(Pid1, 1, false)]), + Pid1 = new_process(), + Group = grp([csr(Pid1, 0, active), + csr(Pid1, 1, waiting)]), State = state(#{GroupId => Group}, #{Pid0 => maps:from_list([{GroupId, true}]), %% should not be there Pid1 => maps:from_list([{GroupId, true}])}), {#?STATE{pids_groups = PidsGroups, groups = Groups}, - Effects} = - rabbit_stream_sac_coordinator:handle_connection_down(Pid0, State), + Effects} = ?MOD:handle_connection_down(Pid0, normal, State), assertSize(1, PidsGroups), assertSize(1, maps:get(Pid1, PidsGroups)), assertEmpty(Effects), - assertHasGroup(GroupId, cgroup([consumer(Pid1, 0, true), consumer(Pid1, 1, false)]), + assertHasGroup(GroupId, + grp([csr(Pid1, 0, active), csr(Pid1, 1, waiting)]), Groups), ok. @@ -409,22 +392,21 @@ handle_connection_down_super_stream_active_stays_test(_) -> ConsumerName = <<"app">>, GroupId = {<<"/">>, Stream, ConsumerName}, Pid0 = self(), - Pid1 = spawn(fun() -> ok end), - Group = cgroup(1, [consumer(Pid0, 0, false), - consumer(Pid0, 1, true), - consumer(Pid1, 2, false), - consumer(Pid1, 3, false)]), - State = state(#{GroupId => Group}, - #{Pid0 => maps:from_list([{GroupId, true}]), - Pid1 => maps:from_list([{GroupId, true}])}), + Pid1 = new_process(), + Group = grp(1, [csr(Pid0, 0, waiting), + csr(Pid0, 1, active), + csr(Pid1, 2, waiting), + csr(Pid1, 3, waiting)]), + State = state(#{GroupId => Group}), {#?STATE{pids_groups = PidsGroups, groups = Groups}, - Effects} = - rabbit_stream_sac_coordinator:handle_connection_down(Pid1, State), + Effects} = ?MOD:handle_connection_down(Pid1, normal, State), assertSize(1, PidsGroups), assertSize(1, maps:get(Pid0, PidsGroups)), assertEmpty(Effects), - assertHasGroup(GroupId, cgroup(1, [consumer(Pid0, 0, false), consumer(Pid0, 1, true)]), + assertHasGroup(GroupId, + grp(1, [csr(Pid0, 0, waiting), + csr(Pid0, 1, active)]), Groups), ok. @@ -433,22 +415,22 @@ handle_connection_down_super_stream_active_changes_test(_) -> ConsumerName = <<"app">>, GroupId = {<<"/">>, Stream, ConsumerName}, Pid0 = self(), - Pid1 = spawn(fun() -> ok end), - Group = cgroup(1, [consumer(Pid0, 0, false), - consumer(Pid1, 1, true), - consumer(Pid0, 2, false), - consumer(Pid1, 3, false)]), - State = state(#{GroupId => Group}, - #{Pid0 => maps:from_list([{GroupId, true}]), - Pid1 => maps:from_list([{GroupId, true}])}), + Pid1 = new_process(), + Group = grp(1, [csr(Pid0, 0, waiting), + csr(Pid1, 1, active), + csr(Pid0, 2, waiting), + csr(Pid1, 3, waiting)]), + State = state(#{GroupId => Group}), {#?STATE{pids_groups = PidsGroups, groups = Groups}, Effects} = - rabbit_stream_sac_coordinator:handle_connection_down(Pid0, State), + ?MOD:handle_connection_down(Pid0, normal, State), assertSize(1, PidsGroups), assertSize(1, maps:get(Pid1, PidsGroups)), assertSendMessageSteppingDownEffect(Pid1, 1, Stream, ConsumerName, Effects), - assertHasGroup(GroupId, cgroup(1, [consumer(Pid1, 1, false), consumer(Pid1, 3, false)]), + assertHasGroup(GroupId, + grp(1, [csr(Pid1, 1, deactivating), + csr(Pid1, 3, waiting)]), Groups), ok. @@ -457,22 +439,20 @@ handle_connection_down_super_stream_activate_in_remaining_connection_test(_) -> ConsumerName = <<"app">>, GroupId = {<<"/">>, Stream, ConsumerName}, Pid0 = self(), - Pid1 = spawn(fun() -> ok end), - Group = cgroup(1, [consumer(Pid0, 0, false), - consumer(Pid0, 1, true), - consumer(Pid1, 2, false), - consumer(Pid1, 3, false)]), - State = state(#{GroupId => Group}, - #{Pid0 => maps:from_list([{GroupId, true}]), - Pid1 => maps:from_list([{GroupId, true}])}), + Pid1 = new_process(), + Group = grp(1, [csr(Pid0, 0, waiting), + csr(Pid0, 1, active), + csr(Pid1, 2, waiting), + csr(Pid1, 3, waiting)]), + State = state(#{GroupId => Group}), {#?STATE{pids_groups = PidsGroups, groups = Groups}, - Effects} = - rabbit_stream_sac_coordinator:handle_connection_down(Pid0, State), + Effects} = ?MOD:handle_connection_down(Pid0, normal, State), assertSize(1, PidsGroups), assertSize(1, maps:get(Pid1, PidsGroups)), - assertSendMessageEffect(Pid1, 3, Stream, ConsumerName, true, Effects), - assertHasGroup(GroupId, cgroup(1, [consumer(Pid1, 2, false), consumer(Pid1, 3, true)]), + assertSendMessageActivateEffect(Pid1, 3, Stream, ConsumerName, true, Effects), + assertHasGroup(GroupId, grp(1, [csr(Pid1, 2, waiting), + csr(Pid1, 3, active)]), Groups), ok. @@ -481,25 +461,23 @@ handle_connection_down_super_stream_no_active_removed_or_present_test(_) -> ConsumerName = <<"app">>, GroupId = {<<"/">>, Stream, ConsumerName}, Pid0 = self(), - Pid1 = spawn(fun() -> ok end), + Pid1 = new_process(), %% this is a weird case that should not happen in the wild, %% we test the logic in the code nevertheless. %% No active consumer in the group - Group = cgroup(1, [consumer(Pid0, 0, false), - consumer(Pid0, 1, false), - consumer(Pid1, 2, false), - consumer(Pid1, 3, false)]), - State = state(#{GroupId => Group}, - #{Pid0 => maps:from_list([{GroupId, true}]), - Pid1 => maps:from_list([{GroupId, true}])}), + Group = grp(1, [csr(Pid0, 0, waiting), + csr(Pid0, 1, waiting), + csr(Pid1, 2, waiting), + csr(Pid1, 3, waiting)]), + State = state(#{GroupId => Group}), {#?STATE{pids_groups = PidsGroups, groups = Groups}, - Effects} = - rabbit_stream_sac_coordinator:handle_connection_down(Pid0, State), + Effects} = ?MOD:handle_connection_down(Pid0, normal, State), assertSize(1, PidsGroups), assertSize(1, maps:get(Pid1, PidsGroups)), assertEmpty(Effects), - assertHasGroup(GroupId, cgroup(1, [consumer(Pid1, 2, false), consumer(Pid1, 3, false)]), + assertHasGroup(GroupId, grp(1, [csr(Pid1, 2, waiting), + csr(Pid1, 3, waiting)]), Groups), ok. @@ -517,6 +495,1137 @@ register_consumer_with_different_partition_index_should_return_error_test(_) -> {_, {error, partition_index_conflict}, []} = rabbit_stream_sac_coordinator:apply(Command1, State1). +handle_connection_down_consumers_from_dead_connection_should_be_filtered_out_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = new_process(), + Pid2 = new_process(), + Group = grp(1, [csr(Pid0, 0, waiting), + csr(Pid1, 1, active), + csr(Pid2, 2, waiting)]), + State0 = state(#{GroupId => Group}), + + {#?STATE{pids_groups = PidsGroups1, groups = Groups1} = State1, + Effects1} = + ?MOD:handle_connection_down(Pid0, normal, State0), + assertSize(2, PidsGroups1), + assertSize(1, maps:get(Pid1, PidsGroups1)), + assertSize(1, maps:get(Pid2, PidsGroups1)), + assertSendMessageSteppingDownEffect(Pid1, 1, Stream, ConsumerName, Effects1), + assertHasGroup(GroupId, + grp(1, [csr(Pid1, 1, deactivating), + csr(Pid2, 2, waiting)]), + Groups1), + + {#?STATE{pids_groups = PidsGroups2, groups = Groups2}, + Effects2} = ?MOD:handle_connection_down(Pid1, normal, State1), + assertSize(1, PidsGroups2), + assertSize(1, maps:get(Pid2, PidsGroups2)), + assertSendMessageActivateEffect(Pid2, 2, Stream, ConsumerName, true, Effects2), + assertHasGroup(GroupId, + grp(1, [csr(Pid2, 2, active)]), + Groups2), + + ok. + +import_state_v4_empty_test(_) -> + OldMod = rabbit_stream_sac_coordinator_v4, + OldState = OldMod:init_state(), + Export = OldMod:state_to_map(OldState), + ?assertEqual(#?STATE{groups = #{}, pids_groups = #{}, + conf = #{disconnected_timeout => 60_000}}, + ?MOD:import_state(4, Export)), + ok. + +import_state_v4_test(_) -> + OldMod = rabbit_stream_sac_coordinator_v4, + OldState0 = OldMod:init_state(), + Pid0 = self(), + Pid1 = new_process(), + Pid2 = new_process(), + S = <<"stream">>, + App0 = <<"app-0">>, + Cmd0 = register_consumer_command(S, -1, App0, Pid0, 0), + OldState1 = apply_ensure_monitors(OldMod, Cmd0, OldState0), + Cmd1 = register_consumer_command(S, -1, App0, Pid1, 1), + OldState2 = apply_ensure_monitors(OldMod, Cmd1, OldState1), + Cmd2 = register_consumer_command(S, -1, App0, Pid2, 2), + OldState3 = apply_ensure_monitors(OldMod, Cmd2, OldState2), + + P = <<"stream-1">>, + App1 = <<"app-1">>, + Cmd3 = register_consumer_command(P, 1, App1, Pid0, 0), + OldState4 = apply_ensure_monitors(OldMod, Cmd3, OldState3), + Cmd4 = register_consumer_command(P, 1, App1, Pid1, 1), + OldState5 = apply_ensure_monitors(OldMod, Cmd4, OldState4), + Cmd5 = register_consumer_command(P, 1, App1, Pid2, 2), + OldState6 = apply_ensure_monitors(OldMod, Cmd5, OldState5), + Cmd6 = activate_consumer_command(P, App1), + OldState7 = apply_ensure_monitors(OldMod, Cmd6, OldState6), + + Export = OldMod:state_to_map(OldState7), + #?STATE{groups = Groups, pids_groups = PidsGroups} = ?MOD:import_state(4, Export), + assertHasGroup({<<"/">>, S, App0}, + grp(-1, [csr(Pid0, 0, active), + csr(Pid1, 1, waiting), + csr(Pid2, 2, waiting)]), + Groups), + + assertHasGroup({<<"/">>, P, App1}, + grp(1, [csr(Pid0, 0, waiting), + csr(Pid1, 1, active), + csr(Pid2, 2, waiting)]), + Groups), + assertSize(3, PidsGroups), + assertSize(2, maps:get(Pid0, PidsGroups)), + assertSize(2, maps:get(Pid1, PidsGroups)), + assertSize(2, maps:get(Pid2, PidsGroups)), + + ok. + +handle_connection_node_disconnected_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = new_process(), + Pid2 = new_process(), + Group = grp(1, [csr(Pid0, 0, waiting), + csr(Pid1, 1, active), + csr(Pid2, 2, waiting)]), + State0 = state(#{GroupId => Group}), + + {#?STATE{pids_groups = PidsGroups1, groups = Groups1} = _State1, + [Effect1]} = + ?MOD:handle_connection_down(Pid1, noconnection, State0), + assertSize(2, PidsGroups1), + assertSize(1, maps:get(Pid0, PidsGroups1)), + assertSize(1, maps:get(Pid2, PidsGroups1)), + ?assertEqual({timer, {sac, node_disconnected, #{connection_pid => Pid1}}, + 60_000}, + Effect1), + assertHasGroup(GroupId, + grp(1, [csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {disconnected, active}), + csr(Pid2, 2, {connected, waiting})]), + Groups1), + ok. + +handle_node_reconnected_test(_) -> + N0 = node(), + {N1Pid, N1} = start_node(?FUNCTION_NAME), + N0Pid0 = new_process(N0), + N0Pid1 = new_process(N0), + N1Pid0 = new_process(N1), + + S0 = <<"s0">>, + S1 = <<"s1">>, + S2 = <<"s2">>, + + GId0 = group_id(S0), + GId1 = group_id(S1), + GId2 = group_id(S2), + + + Group0 = grp(0, [csr(N0Pid0, 0, {connected, active}), + csr(N1Pid0, 1, {disconnected, waiting}), + csr(N0Pid1, 2, {connected, waiting})]), + + Group1 = grp(1, [csr(N0Pid0, 0, {connected, waiting}), + csr(N1Pid0, 1, {disconnected, active}), + csr(N0Pid1, 2, {connected, waiting})]), + + Group2 = grp(1, [csr(N0Pid0, 0, {connected, waiting}), + csr(N1Pid0, 1, {disconnected, waiting}), + csr(N0Pid1, 2, {connected, active})]), + + Groups0 = #{GId0 => Group0, + GId1 => Group1, + GId2 => Group2}, + %% Pid2 is missing from PIDs to groups dependency mapping + State0 = state(Groups0, + #{N0Pid0 => #{GId0 => true, GId1 => true, GId2 => true}, + N0Pid1 => #{GId0 => true, GId1 => true, GId2 => true}}), + {#?STATE{pids_groups = PidsGroups1, groups = Groups1} = _State1, + Effects1} = + ?MOD:handle_node_reconnected(N1, State0, []), + + ?assertEqual(Groups0, Groups1), + ?assertEqual(#{N0Pid0 => #{GId0 => true, GId1 => true, GId2 => true}, + N1Pid0 => #{GId0 => true, GId1 => true, GId2 => true}, + N0Pid1 => #{GId0 => true, GId1 => true, GId2 => true}}, + PidsGroups1), + + assertSize(2, Effects1), + assertContainsCheckConnectionEffect(N1Pid0, Effects1), + assertContainsMonitorProcessEffect(N1Pid0, Effects1), + + stop_node(N1Pid), + ok. + +connection_reconnected_simple_disconnected_becomes_connected_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = grp([csr(Pid0, 0, {disconnected, active}), + csr(Pid1, 1, {connected, waiting}), + csr(Pid2, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + + Cmd = connection_reconnected_command(Pid0), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + + assertHasGroup(GId, grp([csr(Pid0, 0, {connected, active}), + csr(Pid1, 1, {connected, waiting}), + csr(Pid2, 2, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +connection_reconnected_simple_active_should_be_first_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + %% disconnected for a while, got first in consumer array + %% because consumers arrived and left + Group = grp([csr(Pid0, 0, {disconnected, waiting}), + csr(Pid1, 1, {connected, active}), + csr(Pid2, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + + Cmd = connection_reconnected_command(Pid0), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + + assertHasGroup(GId, grp([csr(Pid1, 1, {connected, active}), + csr(Pid0, 0, {connected, waiting}), + csr(Pid2, 2, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +connection_reconnected_super_disconnected_becomes_connected_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = grp(1, [csr(Pid0, 0, {disconnected, waiting}), + csr(Pid1, 1, {connected, waiting}), + csr(Pid2, 2, {connected, active})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + + Cmd = connection_reconnected_command(Pid0), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + + assertHasGroup(GId, grp(1, [csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {connected, waiting}), + csr(Pid2, 2, {connected, deactivating})]), + Groups1), + + assertSendMessageSteppingDownEffect(Pid2, 2, stream(), name(), Eff), + ok. + +presume_conn_down_simple_disconnected_becomes_presumed_down_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = grp([csr(Pid0, 0, {disconnected, active}), + csr(Pid1, 1, {connected, waiting}), + csr(Pid2, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + + {#?STATE{groups = Groups1}, Eff} = ?MOD:presume_connection_down(Pid0, State0), + + assertHasGroup(GId, grp([csr(Pid0, 0, {presumed_down, active}), + csr(Pid1, 1, {connected, active}), + csr(Pid2, 2, {connected, waiting})]), + Groups1), + assertSendMessageActivateEffect(Pid1, 1, stream(), name(), true, Eff), + ok. + +presume_conn_down_super_stream_disconnected_becomes_presumed_down_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = grp(1, [csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {disconnected, active}), + csr(Pid2, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + + {#?STATE{groups = Groups1}, Eff} = ?MOD:presume_connection_down(Pid1, State0), + + assertHasGroup(GId, grp(1, [csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {presumed_down, active}), + csr(Pid2, 2, {connected, active})]), + Groups1), + + assertSendMessageActivateEffect(Pid2, 2, stream(), name(), true, Eff), + ok. + +presume_conn_down_simple_connected_does_not_become_presumed_down_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = grp([csr(Pid0, 0, {connected, active}), + csr(Pid1, 1, {connected, waiting}), + csr(Pid2, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + + {#?STATE{groups = Groups1}, Eff} = ?MOD:presume_connection_down(Pid1, State0), + + assertHasGroup(GId, grp([csr(Pid0, 0, {connected, active}), + csr(Pid1, 1, {connected, waiting}), + csr(Pid2, 2, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +presume_conn_down_super_stream_connected_does_not_become_presumed_down_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = grp(1, [csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {connected, active}), + csr(Pid2, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + + {#?STATE{groups = Groups1}, Eff} = ?MOD:presume_connection_down(Pid1, State0), + + assertHasGroup(GId, grp(1, [csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {connected, active}), + csr(Pid2, 2, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + + +register_consumer_simple_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + GId = group_id(), + Group = grp([csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {disconnected, active}), + csr(Pid0, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = register_consumer_command(stream(), -1, name(), Pid0, 3), + {#?STATE{groups = Groups1}, {ok, false}, Eff} = ?MOD:apply(Cmd, State0), + assertHasGroup(GId, grp([csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {disconnected, active}), + csr(Pid0, 2, {connected, waiting}), + csr(Pid0, 3, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +register_consumer_super_stream_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + GId = group_id(), + Group = grp(1, [csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {disconnected, active}), + csr(Pid0, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = register_consumer_command(stream(), 1, name(), Pid0, 3), + {#?STATE{groups = Groups1}, {ok, false}, Eff} = ?MOD:apply(Cmd, State0), + assertHasGroup(GId, grp(1, [csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {disconnected, active}), + csr(Pid0, 2, {connected, waiting}), + csr(Pid0, 3, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +unregister_active_consumer_should_not_select_disconnected_consumer(_) -> + P = self(), + GId = group_id(), + Group = grp([csr(P, 0, {connected, active}), + csr(P, 1, {disconnected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = unregister_consumer_command(stream(), name(), P, 0), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + assertHasGroup(GId, grp([csr(P, 1, {disconnected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +unregister_consumer_simple_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + GId = group_id(), + Group = grp([csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {disconnected, active}), + csr(Pid0, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = unregister_consumer_command(stream(), name(), Pid0, 2), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + assertHasGroup(GId, grp([csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {disconnected, active})]), + Groups1), + assertEmpty(Eff), + ok. + +unregister_consumer_super_stream_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + GId = group_id(), + Group = grp(1, [csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {disconnected, active}), + csr(Pid0, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = unregister_consumer_command(stream(), name(), Pid0, 0), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + assertHasGroup(GId, grp(1, [csr(Pid1, 1, {disconnected, active}), + csr(Pid0, 2, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +activate_consumer_simple_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + GId = group_id(), + Group = grp([csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {disconnected, active}), + csr(Pid0, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = activate_consumer_command(stream(), name()), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + assertHasGroup(GId, grp([csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {disconnected, active}), + csr(Pid0, 2, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +active_consumer_super_stream_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + GId = group_id(), + Group = grp(1, [csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {disconnected, active}), + csr(Pid0, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = activate_consumer_command(stream(), name()), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + assertHasGroup(GId, grp(1, [csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 1, {disconnected, active}), + csr(Pid0, 2, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +handle_connection_down_simple_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = grp([csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 0, {disconnected, active}), + csr(Pid2, 0, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + {#?STATE{groups = Groups1}, Eff} = ?MOD:handle_connection_down(Pid2, normal, + State0), + assertHasGroup(GId, grp([csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 0, {disconnected, active})]), + Groups1), + assertEmpty(Eff), + ok. + +handle_connection_down_super_stream_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = grp(1, [csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 0, {disconnected, active}), + csr(Pid2, 0, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + {#?STATE{groups = Groups1}, Eff} = ?MOD:handle_connection_down(Pid0, normal, + State0), + assertHasGroup(GId, grp(1, [csr(Pid1, 0, {disconnected, active}), + csr(Pid2, 0, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +handle_connection_node_disconnected_simple_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = grp([csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 0, {disconnected, active}), + csr(Pid2, 0, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + {#?STATE{groups = Groups1}, Eff} = + ?MOD:handle_connection_down(Pid2, noconnection, State0), + assertHasGroup(GId, grp([csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 0, {disconnected, active}), + csr(Pid2, 0, {disconnected, waiting})]), + Groups1), + assertNodeDisconnectedTimerEffect(Pid2, Eff), + ok. + +handle_connection_node_disconnected_super_stream_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = grp(1, [csr(Pid0, 0, {connected, waiting}), + csr(Pid1, 0, {disconnected, active}), + csr(Pid2, 0, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + {#?STATE{groups = Groups1}, Eff} = + ?MOD:handle_connection_down(Pid0, noconnection, State0), + assertHasGroup(GId, grp(1, [csr(Pid0, 0, {disconnected, waiting}), + csr(Pid1, 0, {disconnected, active}), + csr(Pid2, 0, {connected, waiting})]), + Groups1), + assertNodeDisconnectedTimerEffect(Pid0, Eff), + ok. + +connection_reconnected_simple_disconn_active_blocks_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = grp([csr(Pid0, 0, {disconnected, waiting}), + csr(Pid1, 0, {disconnected, active}), + csr(Pid2, 0, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = connection_reconnected_command(Pid0), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + + assertHasGroup(GId, grp([csr(Pid1, 0, {disconnected, active}), + csr(Pid0, 0, {connected, waiting}), + csr(Pid2, 0, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +connection_reconnected_simple_forg_act_disconn_active_blocks_rebalancing_test(_) -> + P0 = new_process(), + P1 = new_process(), + P2 = new_process(), + GId = group_id(), + Group = grp([csr(P0, 0, {presumed_down, active}), + csr(P1, 0, {disconnected, active}), + csr(P2, 0, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = connection_reconnected_command(P0), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + + assertHasGroup(GId, grp([csr(P0, 0, {connected, waiting}), + csr(P1, 0, {disconnected, active}), + csr(P2, 0, {connected, waiting})]), + Groups1), + assertSize(1, Eff), + assertContainsSendMessageSteppingDownEffect(P0, Eff), + ok. + +connection_reconnected_simple_forg_act_should_trigger_rebalancing_test(_) -> + P0 = new_process(), + P1 = new_process(), + P2 = new_process(), + GId = group_id(), + Group = grp([csr(P0, {presumed_down, active}), + csr(P1, {connected, active}), + csr(P2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + S0 = state(Groups0), + Cmd0 = connection_reconnected_command(P0), + {#?STATE{groups = Groups1} = S1, ok, Eff1} = ?MOD:apply(Cmd0, S0), + + assertHasGroup(GId, grp([csr(P0, {connected, waiting}), + csr(P1, {connected, waiting}), + csr(P2, {connected, waiting})]), + Groups1), + assertSize(2, Eff1), + assertContainsSendMessageSteppingDownEffect(P0, 0, stream(), name(), Eff1), + assertContainsSendMessageSteppingDownEffect(P1, 0, stream(), name(), Eff1), + + %% activation from the first consumer stepping down + Cmd1 = activate_consumer_command(stream(), name()), + {#?STATE{groups = Groups2} = S2, ok, Eff2} = ?MOD:apply(Cmd1, S1), + assertHasGroup(GId, grp([csr(P0, {connected, active}), + csr(P1, {connected, waiting}), + csr(P2, {connected, waiting})]), + Groups2), + assertSize(1, Eff2), + assertContainsActivateMessage(P0, Eff2), + + %% activation from the second consumer stepping down + %% this is expected, but should not change the state + Cmd2 = activate_consumer_command(stream(), name()), + {#?STATE{groups = Groups3}, ok, Eff3} = ?MOD:apply(Cmd2, S2), + assertHasGroup(GId, grp([csr(P0, {connected, active}), + csr(P1, {connected, waiting}), + csr(P2, {connected, waiting})]), + Groups3), + assertEmpty(Eff3), + + ok. + +connection_reconnected_super_stream_disconn_active_blocks_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = grp(1, [csr(Pid0, 0, {disconnected, active}), + csr(Pid1, 0, {disconnected, waiting}), + csr(Pid2, 0, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = connection_reconnected_command(Pid1), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + + assertHasGroup(GId, grp(1, [csr(Pid0, 0, {disconnected, active}), + csr(Pid1, 0, {connected, waiting}), + csr(Pid2, 0, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +connection_reconnected_super_stream_forg_act_disconn_active_blocks_rebalancing_test(_) -> + P0 = new_process(), + P1 = new_process(), + P2 = new_process(), + GId = group_id(), + Group = grp(1, [csr(P0, {presumed_down, active}), + csr(P1, {disconnected, active}), + csr(P2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = connection_reconnected_command(P0), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + + assertHasGroup(GId, grp(1, [csr(P0, {connected, waiting}), + csr(P1, {disconnected, active}), + csr(P2, {connected, waiting})]), + Groups1), + assertSize(1, Eff), + assertContainsSendMessageSteppingDownEffect(P0, Eff), + ok. + +connection_reconnected_super_stream_forg_act_should_trigger_rebalancing_test(_) -> + P0 = new_process(), + P1 = new_process(), + P2 = new_process(), + GId = group_id(), + Group = grp(1, [csr(P0, {presumed_down, active}), + csr(P1, {connected, waiting}), + csr(P2, {connected, active})]), + + Groups0 = #{GId => Group}, + S0 = state(Groups0), + Cmd0 = connection_reconnected_command(P0), + {#?STATE{groups = Groups1} = S1, ok, Eff1} = ?MOD:apply(Cmd0, S0), + + assertHasGroup(GId, grp(1, [csr(P0, {connected, waiting}), + csr(P1, {connected, waiting}), + csr(P2, {connected, waiting})]), + Groups1), + assertSize(2, Eff1), + assertContainsSendMessageSteppingDownEffect(P0, 0, stream(), name(), Eff1), + assertContainsSendMessageSteppingDownEffect(P2, 0, stream(), name(), Eff1), + + %% activation from the first consumer stepping down + Cmd1 = activate_consumer_command(stream(), name()), + {#?STATE{groups = Groups2} = S2, ok, Eff2} = ?MOD:apply(Cmd1, S1), + assertHasGroup(GId, grp(1, [csr(P0, {connected, waiting}), + csr(P1, {connected, active}), + csr(P2, {connected, waiting})]), + Groups2), + assertSize(1, Eff2), + assertContainsActivateMessage(P1, Eff2), + + %% activation from the second consumer stepping down + %% this is expected, but should not change the state + Cmd2 = activate_consumer_command(stream(), name()), + {#?STATE{groups = Groups3}, ok, Eff3} = ?MOD:apply(Cmd2, S2), + assertHasGroup(GId, grp(1, [csr(P0, {connected, waiting}), + csr(P1, {connected, active}), + csr(P2, {connected, waiting})]), + Groups3), + assertEmpty(Eff3), + + ok. + +presume_conn_down_simple_disconn_active_blocks_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = grp([csr(Pid0, {disconnected, waiting}), + csr(Pid1, {connected, waiting}), + csr(Pid2, {disconnected, active})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + + {#?STATE{groups = Groups1}, Eff} = ?MOD:presume_connection_down(Pid0, State0), + + assertHasGroup(GId, grp([csr(Pid2, {disconnected, active}), + csr(Pid0, {presumed_down, waiting}), + csr(Pid1, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +presume_conn_down_super_stream_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = grp(1, [csr(Pid0, {disconnected, waiting}), + csr(Pid1, {connected, waiting}), + csr(Pid2, {disconnected, active})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + + {#?STATE{groups = Groups1}, Eff} = ?MOD:presume_connection_down(Pid0, State0), + + assertHasGroup(GId, grp(1, [csr(Pid0, {presumed_down, waiting}), + csr(Pid1, {connected, waiting}), + csr(Pid2, {disconnected, active})]), + Groups1), + assertEmpty(Eff), + ok. + +purge_nodes_test(_) -> + N0 = node(), + {N1Pid, N1} = start_node(?FUNCTION_NAME), + + N0P0 = new_process(N0), + N0P1 = new_process(N0), + N0P2 = new_process(N0), + N1P0 = new_process(N1), + N1P1 = new_process(N1), + N1P2 = new_process(N1), + + S0 = <<"s0">>, + S1 = <<"s1">>, + S2 = <<"s2">>, + + GId0 = group_id(S0), + GId1 = group_id(S1), + GId2 = group_id(S2), + + Group0 = grp([csr(N1P0, {disconnected, active}), + csr(N0P1, {connected, waiting}), + csr(N0P2, {connected, waiting})]), + + Group1 = grp(1, [csr(N1P1, {disconnected, waiting}), + csr(N1P2, {disconnected, active}), + csr(N0P0, {connected, waiting})]), + + Group2 = grp([csr(N0P0, {connected, active}), + csr(N0P1, {connected, waiting}), + csr(N0P2, {connected, waiting})]), + + State0 = state(#{GId0 => Group0, GId1 => Group1, GId2 => Group2}), + Cmd = purge_nodes_command([N1]), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + + assertSize(3, Groups1), + assertHasGroup(GId0, grp([csr(N0P1, {connected, active}), + csr(N0P2, {connected, waiting})]), + Groups1), + assertHasGroup(GId1, grp(1, [csr(N0P0, {connected, active})]), + Groups1), + assertHasGroup(GId2, grp([csr(N0P0, {connected, active}), + csr(N0P1, {connected, waiting}), + csr(N0P2, {connected, waiting})]), + Groups1), + + assertSize(2, Eff), + assertContainsSendMessageEffect(N0P1, S0, true, Eff), + assertContainsSendMessageEffect(N0P0, S1, true, Eff), + + stop_node(N1Pid), + ok. + +node_disconnected_and_reconnected_test(_) -> + N0 = node(), + {N1Pid, N1} = start_node(?FUNCTION_NAME), + + N0P0 = new_process(N0), + N0P1 = new_process(N0), + N0P2 = new_process(N0), + N1P0 = new_process(N1), + N1P1 = new_process(N1), + N1P2 = new_process(N1), + + N0Pids = [N0P0, N0P1, N0P2], + N1Pids = [N1P0, N1P1, N1P2], + + S0 = <<"s0">>, + S1 = <<"s1">>, + S2 = <<"s2">>, + + GId0 = group_id(S0), + GId1 = group_id(S1), + GId2 = group_id(S2), + + GIds = [GId0, GId1, GId2], + + G0 = grp([csr(N0P0, {connected, active}), + csr(N1P0, {connected, waiting}), + csr(N0P1, {connected, waiting})]), + + G1 = grp(1, [csr(N1P1, {connected, waiting}), + csr(N0P2, {connected, active}), + csr(N1P2, {connected, waiting})]), + + G2 = grp([csr(N0P0, {connected, active}), + csr(N1P1, {connected, waiting}), + csr(N0P2, {connected, waiting})]), + + State0 = state(#{GId0 => G0, GId1 => G1, GId2 => G2}), + + {State1, Eff1} = ?MOD:handle_connection_down(N1P0, noconnection, State0), + {State2, Eff2} = ?MOD:handle_connection_down(N1P1, noconnection, State1), + {State3, Eff3} = ?MOD:handle_connection_down(N1P2, noconnection, State2), + + assertNodeDisconnectedTimerEffect(N1P0, Eff1), + assertNodeDisconnectedTimerEffect(N1P1, Eff2), + assertNodeDisconnectedTimerEffect(N1P2, Eff3), + + assertHasGroup(GId0, + grp([csr(N0P0, {connected, active}), + csr(N1P0, {disconnected, waiting}), + csr(N0P1, {connected, waiting})]), + State3#?STATE.groups), + + assertHasGroup(GId1, + grp(1, [csr(N1P1, {disconnected, waiting}), + csr(N0P2, {connected, active}), + csr(N1P2, {disconnected, waiting})]), + State3#?STATE.groups), + + assertHasGroup(GId2, + grp([csr(N0P0, {connected, active}), + csr(N1P1, {disconnected, waiting}), + csr(N0P2, {connected, waiting})]), + State3#?STATE.groups), + + PidsGroups3 = State3#?STATE.pids_groups, + assertSize(3, PidsGroups3), + [ ?assert(maps:is_key(Pid, PidsGroups3)) || Pid <- N0Pids], + [ ?assertNot(maps:is_key(Pid, PidsGroups3)) || Pid <- N1Pids], + + {State4, Eff4} = ?MOD:handle_node_reconnected(N1, State3, []), + %% groups should not change + [?assertEqual(maps:get(GId, State3#?STATE.groups), + maps:get(GId, State4#?STATE.groups)) + || GId <- GIds], + + %% all connections should be checked and monitored + [begin + assertContainsCheckConnectionEffect(Pid, Eff4), + assertContainsMonitorProcessEffect(Pid, Eff4) + end || Pid <- N1Pids], + + Cmd4 = connection_reconnected_command(N1P0), + {#?STATE{groups = Groups5} = State5, ok, Eff5} = ?MOD:apply(Cmd4, State4), + + assertHasGroup(GId0, + grp([csr(N0P0, {connected, active}), + csr(N1P0, {connected, waiting}), + csr(N0P1, {connected, waiting})]), + Groups5), + + assertHasGroup(GId1, + grp(1, [csr(N1P1, {disconnected, waiting}), + csr(N0P2, {connected, active}), + csr(N1P2, {disconnected, waiting})]), + Groups5), + + assertHasGroup(GId2, + grp([csr(N0P0, {connected, active}), + csr(N1P1, {disconnected, waiting}), + csr(N0P2, {connected, waiting})]), + Groups5), + + assertEmpty(Eff5), + + Cmd5 = connection_reconnected_command(N1P1), + {#?STATE{groups = Groups6} = State6, ok, Eff6} = ?MOD:apply(Cmd5, State5), + + assertHasGroup(GId0, + grp([csr(N0P0, {connected, active}), + csr(N1P0, {connected, waiting}), + csr(N0P1, {connected, waiting})]), + Groups6), + + assertHasGroup(GId1, + grp(1, [csr(N1P1, {connected, waiting}), + csr(N0P2, {connected, active}), + csr(N1P2, {disconnected, waiting})]), + Groups6), + + assertHasGroup(GId2, + grp([csr(N0P0, {connected, active}), + csr(N1P1, {connected, waiting}), + csr(N0P2, {connected, waiting})]), + Groups6), + + assertEmpty(Eff6), + + %% last connection does not come back for some reason + {#?STATE{groups = Groups7}, Eff7} = ?MOD:presume_connection_down(N1P2, State6), + + assertHasGroup(GId0, + grp([csr(N0P0, {connected, active}), + csr(N1P0, {connected, waiting}), + csr(N0P1, {connected, waiting})]), + Groups7), + + assertHasGroup(GId1, + grp(1, [csr(N1P1, {connected, waiting}), + csr(N0P2, {connected, active}), + csr(N1P2, {presumed_down, waiting})]), + Groups7), + + assertHasGroup(GId2, + grp([csr(N0P0, {connected, active}), + csr(N1P1, {connected, waiting}), + csr(N0P2, {connected, waiting})]), + Groups7), + + assertEmpty(Eff7), + + stop_node(N1Pid), + ok. + +node_disconnected_reconnected_connection_down_test(_) -> + N0 = node(), + {N1Pid, N1} = start_node(list_to_atom(atom_to_list(?FUNCTION_NAME) ++ "1")), + {N2Pid, N2} = start_node(list_to_atom(atom_to_list(?FUNCTION_NAME) ++ "2")), + + P0 = new_process(N0), + P1 = new_process(N1), + P2 = new_process(N2), + + GId = group_id(), + + G0 = grp(1, [csr(P0, {connected, waiting}), + csr(P1, {connected, active}), + csr(P2, {connected, waiting})]), + S0 = state(#{GId => G0}), + + {#?STATE{groups = G1} = S1, Eff1} = + ?MOD:handle_connection_down(P1, noconnection, S0), + + assertHasGroup(GId, + grp(1, [csr(P0, {connected, waiting}), + csr(P1, {disconnected, active}), + csr(P2, {connected, waiting})]), + G1), + + assertNodeDisconnectedTimerEffect(P1, Eff1), + + {#?STATE{groups = G2} = S2, Eff2} = + ?MOD:handle_node_reconnected(N1, S1, []), + + assertHasGroup(GId, + grp(1, [csr(P0, {connected, waiting}), + csr(P1, {disconnected, active}), + csr(P2, {connected, waiting})]), + G2), + + assertContainsCheckConnectionEffect(P1, Eff2), + + {#?STATE{groups = G3}, Eff3} = ?MOD:handle_connection_down(P1, normal, S2), + + assertHasGroup(GId, + grp(1, [csr(P0, {connected, waiting}), + csr(P2, {connected, active})]), + G3), + + assertContainsSendMessageEffect(P2, stream(), true, Eff3), + + stop_node(N1Pid), + stop_node(N2Pid), + ok. + +list_nodes_test(_) -> + N0 = node(), + {N1Pid, N1} = start_node(list_to_atom(atom_to_list(?FUNCTION_NAME) ++ "1")), + {N2Pid, N2} = start_node(list_to_atom(atom_to_list(?FUNCTION_NAME) ++ "2")), + + P0 = new_process(N0), + P1 = new_process(N1), + P2 = new_process(N2), + + Id0 = group_id(<<"sO">>), + Id1 = group_id(<<"s1">>), + Id2 = group_id(<<"s2">>), + + ?assertEqual(lists:sort([N0, N1, N2]), + list_nodes(#{Id0 => grp([csr(P0), csr(P0), csr(P0)]), + Id1 => grp([csr(P1), csr(P1), csr(P1)]), + Id2 => grp([csr(P2), csr(P2), csr(P2)])})), + ?assertEqual(lists:sort([N0, N2]), + list_nodes(#{Id0 => grp([csr(P0), csr(P0), csr(P0)]), + Id2 => grp([csr(P2), csr(P2), csr(P2)])})), + ?assertEqual(lists:sort([N2]), + list_nodes(#{Id2 => grp([csr(P2), csr(P2), csr(P2)])})), + ?assertEqual(lists:sort([N1, N2]), + list_nodes(#{Id0 => grp([csr(P1), csr(P2), csr(P2)]), + Id1 => grp([csr(P1), csr(P1), csr(P2)]), + Id2 => grp([csr(P2), csr(P2), csr(P2)])})), + ?assertEqual(lists:sort([N0, N1, N2]), + list_nodes(#{Id0 => grp([csr(P0), csr(P1), csr(P2)])})), + assertEmpty(list_nodes(#{})), + + stop_node(N1Pid), + stop_node(N2Pid), + ok. + +state_enter_test(_) -> + N0 = node(), + {N1Pid, N1} = start_node(list_to_atom(atom_to_list(?FUNCTION_NAME) ++ "1")), + {N2Pid, N2} = start_node(list_to_atom(atom_to_list(?FUNCTION_NAME) ++ "2")), + + P0 = new_process(N0), + P1 = new_process(N1), + P2 = new_process(N2), + + Id0 = group_id(<<"sO">>), + Id1 = group_id(<<"s1">>), + Id2 = group_id(<<"s2">>), + + assertEmpty(?MOD:state_enter(follower, #{})), + + ?assertEqual(mon_node_eff([N0, N1, N2]), + state_enter_leader(#{Id0 => grp([csr(P0), csr(P0), csr(P0)]), + Id1 => grp([csr(P1), csr(P1), csr(P1)]), + Id2 => grp([csr(P2), csr(P2), csr(P2)])})), + + ?assertEqual(mon_node_eff([N0, N1]), + state_enter_leader(#{Id0 => grp([csr(P0), csr(P0), csr(P0)]), + Id1 => grp([csr(P1), csr(P1), csr(P1)]), + Id2 => grp([csr(P0), csr(P1), csr(P1)])})), + + ?assertEqual(lists:sort(mon_node_eff([N0, N1]) ++ [timer_eff(P1)]), + state_enter_leader(#{Id0 => grp([csr(P0), csr(P1, {disconnected, waiting})]), + Id2 => grp([csr(P0)])})), + + ?assertEqual(lists:sort(mon_node_eff([N0, N1, N2]) ++ timer_eff([P1, P2])), + state_enter_leader(#{Id0 => grp([csr(P0), csr(P1, {disconnected, waiting})]), + Id1 => grp([csr(P0), csr(P2, {disconnected, waiting})]), + Id2 => grp([csr(P0), csr(P1, {disconnected, waiting})])})), + + stop_node(N1Pid), + stop_node(N2Pid), + ok. + +mon_node_eff(Nodes) when is_list(Nodes) -> + lists:sort([mon_node_eff(N) || N <- Nodes]); +mon_node_eff(N) -> + {monitor, node, N}. + +timer_eff(Pids) when is_list(Pids) -> + lists:sort([timer_eff(Pid) || Pid <- Pids]); +timer_eff(Pid) -> + {timer, {sac, node_disconnected, + #{connection_pid => Pid}}, 10_000}. + +state_enter_leader(MapState) -> + lists:sort(?MOD:state_enter(leader, state(MapState))). + +list_nodes(MapState) -> + lists:sort(?MOD:list_nodes(state(MapState))). + +start_node(Name) -> + {ok, NodePid, Node} = peer:start(#{ + name => Name, + connection => standard_io, + shutdown => close + }), + {NodePid, Node}. + +stop_node(NodePid) -> + _ = peer:stop(NodePid). + +new_process() -> + new_process(node()). + +new_process(Node) -> + spawn(Node, fun() -> ok end). + +group_id() -> + group_id(stream()). + +group_id(S) -> + {<<"/">>, S, name()}. + +stream() -> + <<"sO">>. + +name() -> + <<"app">>. + +sub_id() -> + 0. + +apply_ensure_monitors(Mod, Cmd, State0) -> + {State1, _, _} = Mod:apply(Cmd, State0), + {State2, _, _} = Mod:ensure_monitors(Cmd, State1, #{}, []), + State2. + assertSize(Expected, []) -> ?assertEqual(Expected, 0); assertSize(Expected, Map) when is_map(Map) -> @@ -527,30 +1636,59 @@ assertSize(Expected, List) when is_list(List) -> assertEmpty(Data) -> assertSize(0, Data). -assertHasGroup(GroupId, Group, Groups) -> - ?assertEqual(#{GroupId => Group}, Groups). - -consumer(Pid, SubId, Active) -> +assertHasGroup(GroupId, + #group{partition_index = ExpectedPI, consumers = ExpectedCs}, + Groups) -> + #{GroupId := #group{partition_index = CurrentPI, consumers = CurrentCs}} = Groups, + ?assertEqual(ExpectedPI, CurrentPI), + assertCsrsEqual(ExpectedCs, CurrentCs). + +assertCsrsEqual([Expected], [Current]) -> + assertCsrEqual(Expected, Current); +assertCsrsEqual(ExpectedCs, CurrentCs) -> + assertSize(length(ExpectedCs), CurrentCs), + lists:foreach(fun(N) -> + Expected = lists:nth(N, ExpectedCs), + Current = lists:nth(N, CurrentCs), + assertCsrEqual(Expected, Current) + end, lists:seq(1, length(ExpectedCs))). + +assertCsrEqual(Expected, Current) -> + ?assertEqual(Expected#consumer{ts = 0}, Current#consumer{ts = 0}). + +csr(Pid) -> + csr(Pid, {connected, waiting}). + +csr(Pid, Status) -> + csr(Pid, sub_id(), Status). + +csr(Pid, SubId, {Connectivity, Status}) -> #consumer{pid = Pid, subscription_id = SubId, owner = <<"owning connection label">>, - active = Active}. + status = {Connectivity, Status}, + ts = erlang:system_time(millisecond)}; +csr(Pid, SubId, Status) -> + csr(Pid, SubId, {connected, Status}). -cgroup(Consumers) -> - cgroup(-1, Consumers). +grp(Consumers) -> + grp(-1, Consumers). -cgroup(PartitionIndex, Consumers) -> +grp(PartitionIndex, Consumers) -> #group{partition_index = PartitionIndex, consumers = Consumers}. state() -> state(#{}). state(Groups) -> - state(Groups, #{}). + state(Groups, ?MOD:compute_pid_group_dependencies(Groups)). state(Groups, PidsGroups) -> #?STATE{groups = Groups, pids_groups = PidsGroups}. +state_with_conf(Conf) -> + #?STATE{conf = Conf}. + register_consumer_command(Stream, PartitionIndex, ConsumerName, @@ -579,28 +1717,82 @@ activate_consumer_command(Stream, ConsumerName) -> stream = Stream, consumer_name = ConsumerName}. -assertSendMessageEffect(Pid, SubId, Stream, ConsumerName, Active, [Effect]) -> +connection_reconnected_command(Pid) -> + #command_connection_reconnected{pid = Pid}. + +purge_nodes_command(Nodes) -> + #command_purge_nodes{nodes = Nodes}. + +assertContainsCheckConnectionEffect(Pid, Effects) -> + assertContainsSendMessageEffect(Pid, {sac, check_connection, #{}}, Effects). + +assertContainsSendMessageEffect(Pid, Stream, Active, Effects) -> + assertContainsSendMessageEffect(Pid, 0, Stream, name(), Active, Effects). + +assertContainsActivateMessage(Pid, Effects) -> + assertContainsSendMessageEffect(Pid, sub_id(), stream(), name(), + true, Effects). + +assertContainsSendMessageEffect(Pid, SubId, Stream, ConsumerName, Active, + Effects) -> + assertContainsSendMessageEffect(Pid, {sac, + #{subscription_id => SubId, + stream => Stream, + consumer_name => ConsumerName, + active => Active}}, + Effects). + +assertContainsSendMessageSteppingDownEffect(Pid, Effects) -> + assertContainsSendMessageSteppingDownEffect(Pid, sub_id(), stream(), + name(), Effects). + +assertContainsSendMessageSteppingDownEffect(Pid, SubId, Stream, ConsumerName, + Effects) -> + assertContainsSendMessageEffect(Pid, {sac, + #{subscription_id => SubId, + stream => Stream, + consumer_name => ConsumerName, + active => false, + stepping_down => true}}, Effects). + +assertContainsSendMessageEffect(Pid, Msg, Effects) -> + assertContainsEffect({mod_call, + rabbit_stream_sac_coordinator, + send_message, + [Pid, Msg]}, Effects). + +assertContainsMonitorProcessEffect(Pid, Effects) -> + assertContainsEffect({monitor, process, Pid}, Effects). + +assertContainsEffect(Effect, Effects) -> + Contains = lists:any(fun(Eff) -> Eff =:= Effect end, Effects), + ?assert(Contains, "List does not contain the expected effect"). + +assertSendMessageActivateEffect(Pid, SubId, Stream, ConsumerName, Active, Effects) -> + assertSendMessageEffect(Pid, {sac, + #{subscription_id => SubId, + stream => Stream, + consumer_name => ConsumerName, + active => Active} + }, Effects). + +assertSendMessageSteppingDownEffect(Pid, SubId, Stream, ConsumerName, Effects) -> + assertSendMessageEffect(Pid, {sac, + #{subscription_id => SubId, + stream => Stream, + consumer_name => ConsumerName, + active => false, + stepping_down => true}}, Effects). + +assertSendMessageEffect(Pid, Msg, [Effect]) -> ?assertEqual({mod_call, rabbit_stream_sac_coordinator, send_message, - [Pid, - {sac, - #{subscription_id => SubId, - stream => Stream, - consumer_name => ConsumerName, - active => Active} - }]}, + [Pid, Msg]}, Effect). -assertSendMessageSteppingDownEffect(Pid, SubId, Stream, ConsumerName, [Effect]) -> - ?assertEqual({mod_call, - rabbit_stream_sac_coordinator, - send_message, - [Pid, - {sac, - #{subscription_id => SubId, - stream => Stream, - consumer_name => ConsumerName, - active => false, - stepping_down => true}}]}, +assertNodeDisconnectedTimerEffect(Pid, [Effect]) -> + ?assertMatch({timer, + {sac, node_disconnected, #{connection_pid := Pid}}, + _}, Effect). diff --git a/deps/rabbit/test/rabbit_stream_sac_coordinator_v4_SUITE.erl b/deps/rabbit/test/rabbit_stream_sac_coordinator_v4_SUITE.erl new file mode 100644 index 000000000000..7426e8def751 --- /dev/null +++ b/deps/rabbit/test/rabbit_stream_sac_coordinator_v4_SUITE.erl @@ -0,0 +1,593 @@ +%% The contents of this file are subject to the Mozilla Public License +%% Version 2.0 (the "License"); you may not use this file except in +%% compliance with the License. You may obtain a copy of the License +%% at https://www.mozilla.org/en-US/MPL/2.0/ +%% +%% Software distributed under the License is distributed on an "AS IS" +%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See +%% the License for the specific language governing rights and +%% limitations under the License. +%% +%% The Original Code is RabbitMQ. +%% +%% The Initial Developer of the Original Code is Pivotal Software, Inc. +%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_stream_sac_coordinator_v4_SUITE). + +-compile(export_all). + +-include_lib("eunit/include/eunit.hrl"). +-include_lib("rabbit/src/rabbit_stream_sac_coordinator_v4.hrl"). + +%%%=================================================================== +%%% Common Test callbacks +%%%=================================================================== + +-define(STATE, rabbit_stream_sac_coordinator). +-define(MOD, rabbit_stream_sac_coordinator_v4). + +all() -> + [{group, tests}]. + +%% replicate eunit like test resolution +all_tests() -> + [F + || {F, _} <- ?MODULE:module_info(functions), + re:run(atom_to_list(F), "_test$") /= nomatch]. + +groups() -> + [{tests, [], all_tests()}]. + +init_per_suite(Config) -> + Config. + +end_per_suite(_Config) -> + ok. + +init_per_group(_Group, Config) -> + Config. + +end_per_group(_Group, _Config) -> + ok. + +init_per_testcase(_TestCase, Config) -> + ok = meck:new(rabbit_feature_flags), + meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> true end), + Config. + +end_per_testcase(_TestCase, _Config) -> + meck:unload(), + ok. + +simple_sac_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + ConnectionPid = self(), + GroupId = {<<"/">>, Stream, ConsumerName}, + Command0 = + register_consumer_command(Stream, -1, ConsumerName, ConnectionPid, 0), + State0 = state(), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers1}}} = + State1, + {ok, Active1}, Effects1} = + ?MOD:apply(Command0, State0), + ?assert(Active1), + ?assertEqual([consumer(ConnectionPid, 0, true)], Consumers1), + assertSendMessageEffect(ConnectionPid, 0, Stream, ConsumerName, true, Effects1), + + Command1 = + register_consumer_command(Stream, -1, ConsumerName, ConnectionPid, 1), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers2}}} = + State2, + {ok, Active2}, Effects2} = + ?MOD:apply(Command1, State1), + ?assertNot(Active2), + ?assertEqual([consumer(ConnectionPid, 0, true), + consumer(ConnectionPid, 1, false)], + Consumers2), + assertEmpty(Effects2), + + Command2 = + register_consumer_command(Stream, -1, ConsumerName, ConnectionPid, 2), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers3}}} = + State3, + {ok, Active3}, Effects3} = + ?MOD:apply(Command2, State2), + ?assertNot(Active3), + ?assertEqual([consumer(ConnectionPid, 0, true), + consumer(ConnectionPid, 1, false), + consumer(ConnectionPid, 2, false)], + Consumers3), + assertEmpty(Effects3), + + Command3 = + unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 0), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers4}}} = + State4, + ok, Effects4} = + ?MOD:apply(Command3, State3), + ?assertEqual([consumer(ConnectionPid, 1, true), + consumer(ConnectionPid, 2, false)], + Consumers4), + assertSendMessageEffect(ConnectionPid, 1, Stream, ConsumerName, true, Effects4), + + Command4 = + unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 1), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers5}}} = + State5, + ok, Effects5} = + ?MOD:apply(Command4, State4), + ?assertEqual([consumer(ConnectionPid, 2, true)], Consumers5), + assertSendMessageEffect(ConnectionPid, 2, Stream, ConsumerName, true, Effects5), + + Command5 = + unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 2), + {#?STATE{groups = Groups6}, ok, Effects6} = + ?MOD:apply(Command5, State5), + assertEmpty(Groups6), + assertEmpty(Effects6), + + ok. + +super_stream_partition_sac_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + ConnectionPid = self(), + GroupId = {<<"/">>, Stream, ConsumerName}, + Command0 = + register_consumer_command(Stream, 1, ConsumerName, ConnectionPid, 0), + State0 = state(), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers1}}} = + State1, + {ok, Active1}, Effects1} = + ?MOD:apply(Command0, State0), + ?assert(Active1), + ?assertEqual([consumer(ConnectionPid, 0, true)], Consumers1), + assertSendMessageEffect(ConnectionPid, 0, Stream, ConsumerName, true, Effects1), + + Command1 = + register_consumer_command(Stream, 1, ConsumerName, ConnectionPid, 1), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers2}}} = + State2, + {ok, Active2}, Effects2} = + ?MOD:apply(Command1, State1), + %% never active on registration + ?assertNot(Active2), + %% all consumers inactive, until the former active one steps down and activates the new consumer + ?assertEqual([consumer(ConnectionPid, 0, false), + consumer(ConnectionPid, 1, false)], + Consumers2), + assertSendMessageSteppingDownEffect(ConnectionPid, 0, Stream, ConsumerName, Effects2), + + Command2 = activate_consumer_command(Stream, ConsumerName), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers3}}} = + State3, + ok, Effects3} = + ?MOD:apply(Command2, State2), + + %% 1 (partition index) % 2 (consumer count) = 1 (active consumer index) + ?assertEqual([consumer(ConnectionPid, 0, false), + consumer(ConnectionPid, 1, true)], + Consumers3), + assertSendMessageEffect(ConnectionPid, 1, Stream, ConsumerName, true, Effects3), + + Command3 = + register_consumer_command(Stream, 1, ConsumerName, ConnectionPid, 2), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers4}}} = + State4, + {ok, Active4}, Effects4} = + ?MOD:apply(Command3, State3), + %% never active on registration + ?assertNot(Active4), + %% 1 (partition index) % 3 (consumer count) = 1 (active consumer index) + %% the active consumer stays the same + ?assertEqual([consumer(ConnectionPid, 0, false), + consumer(ConnectionPid, 1, true), + consumer(ConnectionPid, 2, false)], + Consumers4), + assertEmpty(Effects4), + + Command4 = + unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 0), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers5}}} = + State5, + ok, Effects5} = + ?MOD:apply(Command4, State4), + %% 1 (partition index) % 2 (consumer count) = 1 (active consumer index) + %% the active consumer will move from sub 1 to sub 2 + ?assertEqual([consumer(ConnectionPid, 1, false), + consumer(ConnectionPid, 2, false)], + Consumers5), + + assertSendMessageSteppingDownEffect(ConnectionPid, 1, Stream, ConsumerName, Effects5), + + Command5 = activate_consumer_command(Stream, ConsumerName), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers6}}} = + State6, + ok, Effects6} = + ?MOD:apply(Command5, State5), + + ?assertEqual([consumer(ConnectionPid, 1, false), + consumer(ConnectionPid, 2, true)], + Consumers6), + assertSendMessageEffect(ConnectionPid, 2, Stream, ConsumerName, true, Effects6), + + Command6 = + unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 1), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers7}}} = + State7, + ok, Effects7} = + ?MOD:apply(Command6, State6), + ?assertEqual([consumer(ConnectionPid, 2, true)], Consumers7), + assertEmpty(Effects7), + + Command7 = + unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 2), + {#?STATE{groups = Groups8}, ok, Effects8} = + ?MOD:apply(Command7, State7), + assertEmpty(Groups8), + assertEmpty(Effects8), + + ok. + +ensure_monitors_test(_) -> + GroupId = {<<"/">>, <<"stream">>, <<"app">>}, + Group = + cgroup([consumer(self(), 0, true), consumer(self(), 1, false)]), + State0 = state(#{GroupId => Group}), + Monitors0 = #{}, + Command0 = + register_consumer_command(<<"stream">>, -1, <<"app">>, self(), 0), + {#?STATE{pids_groups = PidsGroups1} = State1, Monitors1, Effects1} = + ?MOD:ensure_monitors(Command0, + State0, + Monitors0, + []), + assertSize(1, PidsGroups1), + assertSize(1, maps:get(self(), PidsGroups1)), + ?assertEqual(#{self() => sac}, Monitors1), + ?assertEqual([{monitor, process, self()}, {monitor, node, node()}], + Effects1), + + Command1 = + register_consumer_command(<<"stream">>, -1, <<"app">>, self(), 1), + + {#?STATE{pids_groups = PidsGroups2} = State2, Monitors2, Effects2} = + ?MOD:ensure_monitors(Command1, + State1, + Monitors1, + []), + assertSize(1, PidsGroups2), + assertSize(1, maps:get(self(), PidsGroups2)), + ?assertEqual(#{self() => sac}, Monitors2), + ?assertEqual([{monitor, process, self()}, {monitor, node, node()}], + Effects2), + + Group2 = cgroup([consumer(self(), 1, true)]), + + Command2 = + unregister_consumer_command(<<"stream">>, <<"app">>, self(), 0), + + {#?STATE{pids_groups = PidsGroups3} = State3, Monitors3, Effects3} = + ?MOD:ensure_monitors(Command2, + State2#?STATE{groups = + #{GroupId + => + Group2}}, + Monitors2, + []), + assertSize(1, PidsGroups3), + assertSize(1, maps:get(self(), PidsGroups3)), + ?assertEqual(#{self() => sac}, Monitors3), + ?assertEqual([], Effects3), + + %% trying with an unknown connection PID + %% the function should not change anything + UnknownConnectionPid = spawn(fun() -> ok end), + PassthroughCommand = + unregister_consumer_command(<<"stream">>, + <<"app">>, + UnknownConnectionPid, + 0), + + {State3, Monitors3, Effects3} = + ?MOD:ensure_monitors(PassthroughCommand, + State3, + Monitors3, + []), + + Command3 = + unregister_consumer_command(<<"stream">>, <<"app">>, self(), 1), + + {#?STATE{pids_groups = PidsGroups4} = _State4, Monitors4, Effects4} = + ?MOD:ensure_monitors(Command3, + State3#?STATE{groups = + #{}}, + Monitors3, + []), + assertEmpty(PidsGroups4), + assertEmpty(Monitors4), + ?assertEqual([{demonitor, process, self()}], Effects4), + + ok. + +handle_connection_down_sac_should_get_activated_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = spawn(fun() -> ok end), + Group = cgroup([consumer(Pid0, 0, true), + consumer(Pid1, 1, false), + consumer(Pid0, 2, false)]), + State0 = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}]), + Pid1 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups1, groups = Groups1} = State1, + Effects1} = + ?MOD:handle_connection_down(Pid0, State0), + assertSize(1, PidsGroups1), + assertSize(1, maps:get(Pid1, PidsGroups1)), + assertSendMessageEffect(Pid1, 1, Stream, ConsumerName, true, Effects1), + assertHasGroup(GroupId, cgroup([consumer(Pid1, 1, true)]), Groups1), + {#?STATE{pids_groups = PidsGroups2, groups = Groups2}, + Effects2} = + ?MOD:handle_connection_down(Pid1, State1), + assertEmpty(PidsGroups2), + assertEmpty(Effects2), + assertEmpty(Groups2), + + ok. + +handle_connection_down_sac_active_does_not_change_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = spawn(fun() -> ok end), + Group = cgroup([consumer(Pid1, 0, true), + consumer(Pid0, 1, false), + consumer(Pid0, 2, false)]), + State = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}]), + Pid1 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups, groups = Groups}, + Effects} = + ?MOD:handle_connection_down(Pid0, State), + assertSize(1, PidsGroups), + assertSize(1, maps:get(Pid1, PidsGroups)), + assertEmpty(Effects), + assertHasGroup(GroupId, cgroup([consumer(Pid1, 0, true)]), Groups), + ok. + +handle_connection_down_sac_no_more_consumers_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Group = cgroup([consumer(Pid0, 0, true), + consumer(Pid0, 1, false)]), + State = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups, groups = Groups}, + Effects} = + ?MOD:handle_connection_down(Pid0, State), + assertEmpty(PidsGroups), + assertEmpty(Groups), + assertEmpty(Effects), + ok. + +handle_connection_down_sac_no_consumers_in_down_connection_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = spawn(fun() -> ok end), + Group = cgroup([consumer(Pid1, 0, true), + consumer(Pid1, 1, false)]), + State = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}]), %% should not be there + Pid1 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups, groups = Groups}, + Effects} = + ?MOD:handle_connection_down(Pid0, State), + + assertSize(1, PidsGroups), + assertSize(1, maps:get(Pid1, PidsGroups)), + assertEmpty(Effects), + assertHasGroup(GroupId, cgroup([consumer(Pid1, 0, true), consumer(Pid1, 1, false)]), + Groups), + ok. + +handle_connection_down_super_stream_active_stays_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = spawn(fun() -> ok end), + Group = cgroup(1, [consumer(Pid0, 0, false), + consumer(Pid0, 1, true), + consumer(Pid1, 2, false), + consumer(Pid1, 3, false)]), + State = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}]), + Pid1 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups, groups = Groups}, + Effects} = + ?MOD:handle_connection_down(Pid1, State), + assertSize(1, PidsGroups), + assertSize(1, maps:get(Pid0, PidsGroups)), + assertEmpty(Effects), + assertHasGroup(GroupId, cgroup(1, [consumer(Pid0, 0, false), consumer(Pid0, 1, true)]), + Groups), + ok. + +handle_connection_down_super_stream_active_changes_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = spawn(fun() -> ok end), + Group = cgroup(1, [consumer(Pid0, 0, false), + consumer(Pid1, 1, true), + consumer(Pid0, 2, false), + consumer(Pid1, 3, false)]), + State = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}]), + Pid1 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups, groups = Groups}, + Effects} = + ?MOD:handle_connection_down(Pid0, State), + assertSize(1, PidsGroups), + assertSize(1, maps:get(Pid1, PidsGroups)), + assertSendMessageSteppingDownEffect(Pid1, 1, Stream, ConsumerName, Effects), + assertHasGroup(GroupId, cgroup(1, [consumer(Pid1, 1, false), consumer(Pid1, 3, false)]), + Groups), + ok. + +handle_connection_down_super_stream_activate_in_remaining_connection_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = spawn(fun() -> ok end), + Group = cgroup(1, [consumer(Pid0, 0, false), + consumer(Pid0, 1, true), + consumer(Pid1, 2, false), + consumer(Pid1, 3, false)]), + State = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}]), + Pid1 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups, groups = Groups}, + Effects} = + ?MOD:handle_connection_down(Pid0, State), + assertSize(1, PidsGroups), + assertSize(1, maps:get(Pid1, PidsGroups)), + assertSendMessageEffect(Pid1, 3, Stream, ConsumerName, true, Effects), + assertHasGroup(GroupId, cgroup(1, [consumer(Pid1, 2, false), consumer(Pid1, 3, true)]), + Groups), + ok. + +handle_connection_down_super_stream_no_active_removed_or_present_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = spawn(fun() -> ok end), + %% this is a weird case that should not happen in the wild, + %% we test the logic in the code nevertheless. + %% No active consumer in the group + Group = cgroup(1, [consumer(Pid0, 0, false), + consumer(Pid0, 1, false), + consumer(Pid1, 2, false), + consumer(Pid1, 3, false)]), + State = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}]), + Pid1 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups, groups = Groups}, + Effects} = + ?MOD:handle_connection_down(Pid0, State), + assertSize(1, PidsGroups), + assertSize(1, maps:get(Pid1, PidsGroups)), + assertEmpty(Effects), + assertHasGroup(GroupId, cgroup(1, [consumer(Pid1, 2, false), consumer(Pid1, 3, false)]), + Groups), + ok. + +assertSize(Expected, []) -> + ?assertEqual(Expected, 0); +assertSize(Expected, Map) when is_map(Map) -> + ?assertEqual(Expected, maps:size(Map)); +assertSize(Expected, List) when is_list(List) -> + ?assertEqual(Expected, length(List)). + +assertEmpty(Data) -> + assertSize(0, Data). + +assertHasGroup(GroupId, Group, Groups) -> + ?assertEqual(#{GroupId => Group}, Groups). + +consumer(Pid, SubId, Active) -> + #consumer{pid = Pid, + subscription_id = SubId, + owner = <<"owning connection label">>, + active = Active}. + +cgroup(Consumers) -> + cgroup(-1, Consumers). + +cgroup(PartitionIndex, Consumers) -> + #group{partition_index = PartitionIndex, consumers = Consumers}. + +state() -> + state(#{}). + +state(Groups) -> + state(Groups, #{}). + +state(Groups, PidsGroups) -> + #?STATE{groups = Groups, pids_groups = PidsGroups}. + +register_consumer_command(Stream, + PartitionIndex, + ConsumerName, + ConnectionPid, + SubId) -> + #command_register_consumer{vhost = <<"/">>, + stream = Stream, + partition_index = PartitionIndex, + consumer_name = ConsumerName, + connection_pid = ConnectionPid, + owner = <<"owning connection label">>, + subscription_id = SubId}. + +unregister_consumer_command(Stream, + ConsumerName, + ConnectionPid, + SubId) -> + #command_unregister_consumer{vhost = <<"/">>, + stream = Stream, + consumer_name = ConsumerName, + connection_pid = ConnectionPid, + subscription_id = SubId}. + +activate_consumer_command(Stream, ConsumerName) -> + #command_activate_consumer{vhost = <<"/">>, + stream = Stream, + consumer_name = ConsumerName}. + +assertSendMessageEffect(Pid, SubId, Stream, ConsumerName, Active, [Effect]) -> + ?assertEqual({mod_call, + rabbit_stream_sac_coordinator, + send_message, + [Pid, + {sac, + #{subscription_id => SubId, + stream => Stream, + consumer_name => ConsumerName, + active => Active} + }]}, + Effect). + +assertSendMessageSteppingDownEffect(Pid, SubId, Stream, ConsumerName, [Effect]) -> + ?assertEqual({mod_call, + rabbit_stream_sac_coordinator, + send_message, + [Pid, + {sac, + #{subscription_id => SubId, + stream => Stream, + consumer_name => ConsumerName, + active => false, + stepping_down => true}}]}, + Effect). diff --git a/deps/rabbitmq_ct_helpers/src/stream_test_utils.erl b/deps/rabbitmq_ct_helpers/src/stream_test_utils.erl index 0c2f939ae17d..b6e1dbc4a24d 100644 --- a/deps/rabbitmq_ct_helpers/src/stream_test_utils.erl +++ b/deps/rabbitmq_ct_helpers/src/stream_test_utils.erl @@ -18,6 +18,9 @@ connect(Config, Node) -> StreamPort = rabbit_ct_broker_helpers:get_node_config(Config, Node, tcp_port_stream), + connect(StreamPort). + +connect(StreamPort) -> {ok, Sock} = gen_tcp:connect("localhost", StreamPort, [{active, false}, {mode, binary}]), C0 = rabbit_stream_core:init(0), @@ -71,8 +74,14 @@ delete_publisher(Sock, C0, PublisherId) -> {{response, 1, {delete_publisher, ?RESPONSE_CODE_OK}}, C1} = receive_stream_commands(Sock, C0), {ok, C1}. + subscribe(Sock, C0, Stream, SubscriptionId, InitialCredit) -> - SubscribeFrame = rabbit_stream_core:frame({request, 1, {subscribe, SubscriptionId, Stream, _OffsetSpec = first, InitialCredit, _Props = #{}}}), + subscribe(Sock, C0, Stream, SubscriptionId, InitialCredit, #{}). + +subscribe(Sock, C0, Stream, SubscriptionId, InitialCredit, Props) -> + Cmd = {subscribe, SubscriptionId, Stream, _OffsetSpec = first, + InitialCredit, Props}, + SubscribeFrame = rabbit_stream_core:frame({request, 1, Cmd}), ok = gen_tcp:send(Sock, SubscribeFrame), {{response, 1, {subscribe, ?RESPONSE_CODE_OK}}, C1} = receive_stream_commands(Sock, C0), {ok, C1}. diff --git a/deps/rabbitmq_stream/docs/stream_coordinator.md b/deps/rabbitmq_stream/docs/stream_coordinator.md new file mode 100644 index 000000000000..2904053d5760 --- /dev/null +++ b/deps/rabbitmq_stream/docs/stream_coordinator.md @@ -0,0 +1,77 @@ +# Stream Coordinator + +## Single Active Consumer + +### "Simple" SAC (Not Super Stream) + +```mermaid +sequenceDiagram + participant C as Coordinator + participant C1 as Connection 1 + participant C2 as Connection 2 + participant C3 as Connection 3 + Note over C,C3: Simple SAC (not super stream) + C1->>C: register sub 1 + C-)C1: {sac, sub 1, active = true} + activate C1 + C1->>C1: consumer update to client + C2->>C: register sub 2 + C3->>C: register sub 3 + C1->>C: unregister sub 1 + deactivate C1 + C-)C2: {sac, sub 2, active = true} + activate C2 + C2->>C2: consumer update to client + deactivate C2 +``` + +### SAC with Super Stream Partition + +```mermaid +sequenceDiagram + participant C as Coordinator + participant C1 as Connection 1 + participant C2 as Connection 2 + participant C3 as Connection 3 + Note over C,C3: Super Stream SAC (partition = 1) + C1->>C: register sub 1 + C-)C1: {sac, sub 1, active = true} + activate C1 + C2->>C: register sub 2 + C-)C1: {sac, sub 1, active = false, step down = true} + deactivate C1 + C1->>C1: consumer update to client + C1->>C: activate consumer in group + C-)C2: {sac, sub 2, active = true} + activate C2 + C2->>C2: consumer update to client + C3->>C: register sub 3 + Note over C, C3: active consumer stays the same (partition % consumers = 1 % 3 = 1) + deactivate C2 +``` + +### `noconnection` management + +```mermaid +flowchart TB + A(monitor) --noconnection--> B(status = disconnected, set up timer) + B -. timeout .-> C(status = forgotten) + B -. nodeup .-> D(reissue monitors, send msg to connections) + D -. down .-> E(handle connection down) + D -. connection response .-> F(evaluate impacted groups) +``` + +* composite status for consumers: `{connected, active}`, `{disconnected,active}`, etc. +* `disconnected` status can prevent rebalancing in a group, e.g. `{disconnected, active}` (it is impossible to tell the active consumer to step down) +* consumers in `forgotten` status are ignored during rebalancing +* it may be necessary to reconcile a group if a `{forgotten, active}` consumer comes back in a group ("evaluate impacted groups" box above). +This is unlikely though. + +### Stale Node Detection + +```mermaid +flowchart TB + A(RA) -- tick --> B(stale nodes = RA known nodes - cluster nodes) + B -. no stale nodes .-> C(nothing to do) + B -. stale nodes .-> D(remove connections from state) +``` diff --git a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl index f2f054bdd1e3..544700a53499 100644 --- a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl +++ b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl @@ -720,6 +720,9 @@ open(info, {OK, S, Data}, StatemData#statem_data{connection = Connection1, connection_state = State2}} end; +open(info, {sac, check_connection, _}, State) -> + rabbit_stream_sac_coordinator:connection_reconnected(self()), + {keep_state, State}; open(info, {sac, #{subscription_id := SubId, active := Active} = Msg}, diff --git a/deps/rabbitmq_stream/test/commands_SUITE.erl b/deps/rabbitmq_stream/test/commands_SUITE.erl index 711500518b3d..0942f9476522 100644 --- a/deps/rabbitmq_stream/test/commands_SUITE.erl +++ b/deps/rabbitmq_stream/test/commands_SUITE.erl @@ -378,7 +378,7 @@ list_consumer_groups_run(Config) -> {ok, []} = ?COMMAND_LIST_CONSUMER_GROUPS:run([], Opts), StreamPort = rabbit_stream_SUITE:get_stream_port(Config), - {S, C} = start_stream_connection(StreamPort), + {S, C0} = start_stream_connection(StreamPort), ?awaitMatch(1, connection_count(Config), ?WAIT), ConsumerReference = <<"foo">>, @@ -387,11 +387,11 @@ list_consumer_groups_run(Config) -> <<"name">> => ConsumerReference}, Stream1 = <<"list_consumer_groups_run_1">>, - create_stream(S, Stream1, C), - subscribe(S, 0, Stream1, SubProperties, C), - handle_consumer_update(S, C, 0), - subscribe(S, 1, Stream1, SubProperties, C), - subscribe(S, 2, Stream1, SubProperties, C), + C1 = create_stream(S, Stream1, C0), + C2 = subscribe(S, 0, Stream1, SubProperties, C1), + C3 = handle_consumer_update(S, C2, 0), + C4 = subscribe(S, 1, Stream1, SubProperties, C3), + C5 = subscribe(S, 2, Stream1, SubProperties, C4), ?awaitMatch(3, consumer_count(Config), ?WAIT), @@ -399,11 +399,11 @@ list_consumer_groups_run(Config) -> assertConsumerGroup(Stream1, ConsumerReference, -1, 3, CG1), Stream2 = <<"list_consumer_groups_run_2">>, - create_stream(S, Stream2, C), - subscribe(S, 3, Stream2, SubProperties, C), - handle_consumer_update(S, C, 3), - subscribe(S, 4, Stream2, SubProperties, C), - subscribe(S, 5, Stream2, SubProperties, C), + C6 = create_stream(S, Stream2, C5), + C7 = subscribe(S, 3, Stream2, SubProperties, C6), + C8 = handle_consumer_update(S, C7, 3), + C9 = subscribe(S, 4, Stream2, SubProperties, C8), + C10 = subscribe(S, 5, Stream2, SubProperties, C9), ?awaitMatch(3 + 3, consumer_count(Config), ?WAIT), @@ -411,10 +411,10 @@ list_consumer_groups_run(Config) -> assertConsumerGroup(Stream1, ConsumerReference, -1, 3, CG1), assertConsumerGroup(Stream2, ConsumerReference, -1, 3, CG2), - delete_stream(S, Stream1, C), - delete_stream(S, Stream2, C), + C11 = delete_stream(S, Stream1, C10), + C12 = delete_stream(S, Stream2, C11), - close(S, C), + close(S, C12), {ok, []} = ?COMMAND_LIST_CONSUMER_GROUPS:run([], Opts), ok. @@ -490,9 +490,9 @@ list_group_consumers_run(Config) -> {ok, Consumers1} = ?COMMAND_LIST_GROUP_CONSUMERS:run(Args, OptsGroup1), - ?assertEqual([[{subscription_id, 0}, {state, active}], - [{subscription_id, 1}, {state, inactive}], - [{subscription_id, 2}, {state, inactive}]], + ?assertEqual([[{subscription_id, 0}, {state, "active (connected)"}], + [{subscription_id, 1}, {state, "waiting (connected)"}], + [{subscription_id, 2}, {state, "waiting (connected)"}]], Consumers1), Stream2 = <<"list_group_consumers_run_2">>, @@ -510,9 +510,9 @@ list_group_consumers_run(Config) -> {ok, Consumers2} = ?COMMAND_LIST_GROUP_CONSUMERS:run(Args, OptsGroup2), - ?assertEqual([[{subscription_id, 3}, {state, active}], - [{subscription_id, 4}, {state, inactive}], - [{subscription_id, 5}, {state, inactive}]], + ?assertEqual([[{subscription_id, 3}, {state, "active (connected)"}], + [{subscription_id, 4}, {state, "waiting (connected)"}], + [{subscription_id, 5}, {state, "waiting (connected)"}]], Consumers2), delete_stream(S, Stream1, C), diff --git a/deps/rabbitmq_stream/test/rabbit_stream_SUITE.erl b/deps/rabbitmq_stream/test/rabbit_stream_SUITE.erl index 66a111cc3b11..5fdc48b61ab1 100644 --- a/deps/rabbitmq_stream/test/rabbit_stream_SUITE.erl +++ b/deps/rabbitmq_stream/test/rabbit_stream_SUITE.erl @@ -596,35 +596,23 @@ max_segment_size_bytes_validation(Config) -> ok. close_connection_on_consumer_update_timeout(Config) -> - Transport = gen_tcp, - Port = get_stream_port(Config), - {ok, S} = - Transport:connect("localhost", Port, - [{active, false}, {mode, binary}]), - C0 = rabbit_stream_core:init(0), - C1 = test_peer_properties(Transport, S, C0), - C2 = test_authenticate(Transport, S, C1), Stream = atom_to_binary(?FUNCTION_NAME, utf8), - C3 = test_create_stream(Transport, S, Stream, C2), + {ok, S, C0} = stream_test_utils:connect(Config, 0), + {ok, C1} = stream_test_utils:create_stream(S, C0, Stream), SubId = 42, - C4 = test_subscribe(Transport, S, SubId, Stream, - #{<<"single-active-consumer">> => <<"true">>, - <<"name">> => <<"foo">>}, - ?RESPONSE_CODE_OK, - C3), - {Cmd, _C5} = receive_commands(Transport, S, C4), + Props = #{<<"single-active-consumer">> => <<"true">>, + <<"name">> => <<"foo">>}, + {ok, C2} = stream_test_utils:subscribe(S, C1, Stream, SubId, 10, Props), + + {Cmd, _C3} = receive_commands(S, C2), ?assertMatch({request, _, {consumer_update, SubId, true}}, Cmd), - closed = wait_for_socket_close(Transport, S, 10), - {ok, Sb} = - Transport:connect("localhost", Port, - [{active, false}, {mode, binary}]), - Cb0 = rabbit_stream_core:init(0), - Cb1 = test_peer_properties(Transport, Sb, Cb0), - Cb2 = test_authenticate(Transport, Sb, Cb1), - Cb3 = test_delete_stream(Transport, Sb, Stream, Cb2, false), - _Cb4 = test_close(Transport, Sb, Cb3), - closed = wait_for_socket_close(Transport, Sb, 10), + closed = wait_for_socket_close(S, 10), + + {ok, Sb, Cb0} = stream_test_utils:connect(Config, 0), + {ok, Cb1} = stream_test_utils:delete_stream(Sb, Cb0, Stream), + stream_test_utils:close(Sb, Cb1), + closed = wait_for_socket_close(Sb, 10), ok. set_filter_size(Config) -> @@ -1606,6 +1594,9 @@ test_close(Transport, S, C0) -> receive_commands(Transport, S, C0), C. +wait_for_socket_close(S, Attempt) -> + wait_for_socket_close(gen_tcp, S, Attempt). + wait_for_socket_close(_Transport, _S, 0) -> not_closed; wait_for_socket_close(Transport, S, Attempt) -> @@ -1616,6 +1607,10 @@ wait_for_socket_close(Transport, S, Attempt) -> closed end. + +receive_commands(S, C) -> + receive_commands(gen_tcp, S, C). + receive_commands(Transport, S, C) -> stream_test_utils:receive_stream_commands(Transport, S, C). diff --git a/deps/rabbitmq_stream/test/rabbit_stream_partitions_SUITE.erl b/deps/rabbitmq_stream/test/rabbit_stream_partitions_SUITE.erl new file mode 100644 index 000000000000..e4d37696f81c --- /dev/null +++ b/deps/rabbitmq_stream/test/rabbit_stream_partitions_SUITE.erl @@ -0,0 +1,786 @@ +%% The contents of this file are subject to the Mozilla Public License +%% Version 2.0 (the "License"); you may not use this file except in +%% compliance with the License. You may obtain a copy of the License +%% at https://www.mozilla.org/en-US/MPL/2.0/ +%% +%% Software distributed under the License is distributed on an "AS IS" +%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See +%% the License for the specific language governing rights and +%% limitations under the License. +%% +%% The Original Code is RabbitMQ. +%% +%% The Initial Developer of the Original Code is Pivotal Software, Inc. +%% Copyright (c) 2025 Broadcom. All Rights Reserved. +%% The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_stream_partitions_SUITE). + +-include_lib("eunit/include/eunit.hrl"). +-include_lib("rabbitmq_stream_common/include/rabbit_stream.hrl"). +-include_lib("rabbit/src/rabbit_stream_sac_coordinator.hrl"). + +-compile(nowarn_export_all). +-compile(export_all). + +-define(NET_TICKTIME_S, 5). +-define(TRSPT, gen_tcp). +-define(CORR_ID, 1). +-define(SAC_STATE, rabbit_stream_sac_coordinator). + +-record(node, {name :: node(), stream_port :: pos_integer()}). + +all() -> + [{group, cluster}]. + +groups() -> + [{cluster, [], + [simple_sac_consumer_should_get_disconnected_on_network_partition, + simple_sac_consumer_should_get_disconnected_on_coord_leader_network_partition, + super_stream_sac_consumer_should_get_disconnected_on_network_partition, + super_stream_sac_consumer_should_get_disconnected_on_coord_leader_network_partition]} + ]. + +init_per_suite(Config) -> + case rabbit_ct_helpers:is_mixed_versions() of + true -> + {skip, "mixed version clusters are not supported"}; + _ -> + rabbit_ct_helpers:log_environment(), + Config + end. + +end_per_suite(Config) -> + Config. + +init_per_group(Group, Config) -> + Config1 = rabbit_ct_helpers:run_setup_steps( + Config, + [fun rabbit_ct_broker_helpers:configure_dist_proxy/1]), + rabbit_ct_helpers:set_config(Config1, + [{rmq_nodename_suffix, Group}, + {net_ticktime, ?NET_TICKTIME_S}]). +end_per_group(_, Config) -> + Config. + +init_per_testcase(TestCase, Config) -> + Config1 = rabbit_ct_helpers:testcase_started(Config, TestCase), + Config2 = rabbit_ct_helpers:set_config( + Config1, [{rmq_nodes_clustered, true}, + {rmq_nodes_count, 3}, + {tcp_ports_base} + ]), + rabbit_ct_helpers:run_setup_steps( + Config2, + [fun(StepConfig) -> + rabbit_ct_helpers:merge_app_env(StepConfig, + {aten, + [{poll_interval, + 1000}]}) + end, + fun(StepConfig) -> + rabbit_ct_helpers:merge_app_env(StepConfig, + {rabbit, + [{stream_cmd_timeout, 5000}, + {stream_sac_disconnected_timeout, + 2000}]}) + end] + ++ rabbit_ct_broker_helpers:setup_steps()). + +end_per_testcase(TestCase, Config) -> + Config1 = rabbit_ct_helpers:testcase_finished(Config, TestCase), + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_broker_helpers:teardown_steps()). + + +simple_sac_consumer_should_get_disconnected_on_network_partition(Config) -> + init_coordinator(Config), + CL = coordinator_leader(Config), + + S = rabbit_data_coercion:to_binary(?FUNCTION_NAME), + init_stream(Config, CL, S), + + [L, F1, F2] = topology(Config, S), + + %% the stream leader and the coordinator leader are on the same node + %% another node will be isolated + ?assertEqual(L#node.name, coordinator_leader(Config)), + + {ok, So0, C0_00} = stream_test_utils:connect(Config, 0), + {ok, So1, C1_00} = stream_test_utils:connect(Config, 1), + {ok, So2, C2_00} = stream_test_utils:connect(Config, 2), + + C0_01 = register_sac(So0, C0_00, S, 0), + C0_02 = receive_consumer_update(So0, C0_01), + + C1_01 = register_sac(So1, C1_00, S, 1), + C2_01 = register_sac(So2, C2_00, S, 2), + SubIdToState0 = #{0 => {So0, C0_02}, + 1 => {So1, C1_01}, + 2 => {So2, C2_01}}, + + Consumers1 = query_consumers(Config, S), + assertSize(3, Consumers1), + assertConsumersConnected(Consumers1), + + LN = L#node.name, + F1N = F1#node.name, + F2N = F2#node.name, + + Isolated = F1N, + {value, DisconnectedConsumer} = + lists:search(fun(#consumer{pid = ConnPid}) -> + rpc(Config, erlang, node, [ConnPid]) =:= Isolated + end, Consumers1), + #consumer{subscription_id = DiscSubId} = DisconnectedConsumer, + + rabbit_ct_broker_helpers:block_traffic_between(Isolated, LN), + rabbit_ct_broker_helpers:block_traffic_between(Isolated, F2N), + + wait_for_disconnected_consumer(Config, LN, S), + wait_for_presumed_down_consumer(Config, LN, S), + + rabbit_ct_broker_helpers:allow_traffic_between(Isolated, LN), + rabbit_ct_broker_helpers:allow_traffic_between(Isolated, F2N), + + wait_for_all_consumers_connected(Config, LN, S), + + Consumers2 = query_consumers(Config, LN, S), + %% the disconnected, then presumed down consumer is cancelled, + %% because the stream member on its node has been restarted + assertSize(2, Consumers2), + assertConsumersConnected(Consumers2), + ?assertMatch([DisconnectedConsumer], + Consumers1 -- Consumers2), + + %% assert the cancelled consumer received a metadata update frame + SubIdToState1 = + maps:fold(fun(K, {S0, C0}, Acc) when K == DiscSubId -> + C1 = receive_metadata_update(S0, C0), + Acc#{K => {S0, C1}}; + (K, {S0, C0}, Acc) -> + Acc#{K => {S0, C0}} + end, #{}, SubIdToState0), + + delete_stream(stream_port(Config, 0), S), + + %% online consumers should receive a metadata update frame (stream deleted) + %% we unqueue the this frame before closing the connection + %% directly closing the connection of the cancelled consumer + maps:foreach(fun(K, {S0, C0}) when K /= DiscSubId -> + {_, C1} = receive_commands(S0, C0), + {ok, _} = stream_test_utils:close(S0, C1); + (_, {S0, C0}) -> + {ok, _} = stream_test_utils:close(S0, C0) + end, SubIdToState1), + + ok. + +simple_sac_consumer_should_get_disconnected_on_coord_leader_network_partition(Config) -> + init_coordinator(Config), + CL = coordinator_leader(Config), + [CF1, CF2] = all_nodes(Config) -- [CL], + + S = rabbit_data_coercion:to_binary(?FUNCTION_NAME), + init_stream(Config, CF1, S), + [L, _F1, _F2] = topology(Config, S), + + %% the stream leader and the coordinator leader are not on the same node + %% the coordinator leader node will be isolated + ?assertNotEqual(L#node.name, CL), + + {ok, So0, C0_00} = stream_test_utils:connect(Config, CL), + {ok, So1, C1_00} = stream_test_utils:connect(Config, CF1), + {ok, So2, C2_00} = stream_test_utils:connect(Config, CF2), + + C0_01 = register_sac(So0, C0_00, S, 0), + C0_02 = receive_consumer_update(So0, C0_01), + + C1_01 = register_sac(So1, C1_00, S, 1), + C2_01 = register_sac(So2, C2_00, S, 2), + SubIdToState0 = #{0 => {So0, C0_02}, + 1 => {So1, C1_01}, + 2 => {So2, C2_01}}, + + Consumers1 = query_consumers(Config, S), + assertSize(3, Consumers1), + assertConsumersConnected(Consumers1), + + %% N1 is the coordinator leader + Isolated = CL, + NotIsolated = CF1, + {value, DisconnectedConsumer} = + lists:search(fun(#consumer{pid = ConnPid}) -> + rpc(Config, erlang, node, [ConnPid]) =:= Isolated + end, Consumers1), + #consumer{subscription_id = DiscSubId} = DisconnectedConsumer, + + rabbit_ct_broker_helpers:block_traffic_between(Isolated, CF1), + rabbit_ct_broker_helpers:block_traffic_between(Isolated, CF2), + + wait_for_disconnected_consumer(Config, NotIsolated, S), + wait_for_presumed_down_consumer(Config, NotIsolated, S), + + rabbit_ct_broker_helpers:allow_traffic_between(Isolated, CF1), + rabbit_ct_broker_helpers:allow_traffic_between(Isolated, CF2), + + wait_for_coordinator_ready(Config), + + wait_for_all_consumers_connected(Config, NotIsolated, S), + + Consumers2 = query_consumers(Config, NotIsolated, S), + + %% the disconnected, then presumed down consumer is cancelled, + %% because the stream member on its node has been restarted + assertSize(2, Consumers2), + assertConsumersConnected(Consumers2), + assertEmpty(lists:filter(fun(C) -> + same_consumer(DisconnectedConsumer, C) + end, Consumers2)), + + [#consumer{subscription_id = ActiveSubId}] = + lists:filter(fun(#consumer{status = St}) -> + St =:= {connected, active} + end, Consumers2), + + SubIdToState1 = + maps:fold(fun(K, {S0, C0}, Acc) when K == DiscSubId -> + %% cancelled consumer received a metadata update + C1 = receive_metadata_update(S0, C0), + Acc#{K => {S0, C1}}; + (K, {S0, C0}, Acc) when K == ActiveSubId -> + %% promoted consumer should have received consumer update + C1 = receive_consumer_update_and_respond(S0, C0), + Acc#{K => {S0, C1}}; + (K, {S0, C0}, Acc) -> + Acc#{K => {S0, C0}} + end, #{}, SubIdToState0), + + delete_stream(L#node.stream_port, S), + + %% online consumers should receive a metadata update frame (stream deleted) + %% we unqueue this frame before closing the connection + %% directly closing the connection of the cancelled consumer + maps:foreach(fun(K, {S0, C0}) when K /= DiscSubId -> + {_, C1} = receive_commands(S0, C0), + {ok, _} = stream_test_utils:close(S0, C1); + (_, {S0, C0}) -> + {ok, _} = stream_test_utils:close(S0, C0) + end, SubIdToState1), + + ok. + +super_stream_sac_consumer_should_get_disconnected_on_network_partition(Config) -> + init_coordinator(Config), + CL = coordinator_leader(Config), + + Ss = rabbit_data_coercion:to_binary(?FUNCTION_NAME), + + Partition = init_super_stream(Config, CL, Ss, 1, CL), + [L, F1, F2] = topology(Config, Partition), + + wait_for_coordinator_ready(Config), + + %% we expect the stream leader and the coordinator leader to be on the same node + %% another node will be isolated + ?assertEqual(L#node.name, CL), + + {ok, So0, C0_00} = stream_test_utils:connect(L#node.stream_port), + {ok, So1, C1_00} = stream_test_utils:connect(F1#node.stream_port), + {ok, So2, C2_00} = stream_test_utils:connect(F2#node.stream_port), + + C0_01 = register_sac(So0, C0_00, Partition, 0, Ss), + C0_02 = receive_consumer_update(So0, C0_01), + + C1_01 = register_sac(So1, C1_00, Partition, 1, Ss), + C2_01 = register_sac(So2, C2_00, Partition, 2, Ss), + SubIdToState0 = #{0 => {So0, C0_02}, + 1 => {So1, C1_01}, + 2 => {So2, C2_01}}, + + Consumers1 = query_consumers(Config, Partition), + assertSize(3, Consumers1), + assertConsumersConnected(Consumers1), + + LN = L#node.name, + F1N = F1#node.name, + F2N = F2#node.name, + + Isolated = F1N, + NotIsolated = F2N, + {value, DisconnectedConsumer} = + lists:search(fun(#consumer{pid = ConnPid}) -> + rpc(Config, erlang, node, [ConnPid]) =:= Isolated + end, Consumers1), + #consumer{subscription_id = DiscSubId} = DisconnectedConsumer, + + rabbit_ct_broker_helpers:block_traffic_between(Isolated, LN), + rabbit_ct_broker_helpers:block_traffic_between(Isolated, F2N), + + wait_for_disconnected_consumer(Config, NotIsolated, Partition), + wait_for_presumed_down_consumer(Config, NotIsolated, Partition), + + rabbit_ct_broker_helpers:allow_traffic_between(Isolated, LN), + rabbit_ct_broker_helpers:allow_traffic_between(Isolated, F2N), + + wait_for_coordinator_ready(Config), + + wait_for_all_consumers_connected(Config, NotIsolated, Partition), + + Consumers2 = query_consumers(Config, NotIsolated, Partition), + + %% the disconnected, then presumed down consumer is cancelled, + %% because the stream member on its node has been restarted + assertSize(2, Consumers2), + assertConsumersConnected(Consumers2), + assertEmpty(lists:filter(fun(C) -> + same_consumer(DisconnectedConsumer, C) + end, Consumers2)), + + SubIdToState1 = + maps:fold(fun(K, {S0, C0}, Acc) when K == DiscSubId -> + %% cancelled consumer received a metadata update + C1 = receive_metadata_update(S0, C0), + Acc#{K => {S0, C1}}; + (K, {S0, C0}, Acc) -> + Acc#{K => {S0, C0}} + end, #{}, SubIdToState0), + + delete_super_stream(L#node.stream_port, Ss), + + %% online consumers should receive a metadata update frame (stream deleted) + %% we unqueue this frame before closing the connection + %% directly closing the connection of the cancelled consumer + maps:foreach(fun(K, {S0, C0}) when K /= DiscSubId -> + {_, C1} = receive_commands(S0, C0), + {ok, _} = stream_test_utils:close(S0, C1); + (_, {S0, C0}) -> + {ok, _} = stream_test_utils:close(S0, C0) + end, SubIdToState1), + ok. + +super_stream_sac_consumer_should_get_disconnected_on_coord_leader_network_partition(Config) -> + init_coordinator(Config), + CL = coordinator_leader(Config), + [CF1, _] = all_nodes(Config) -- [CL], + Ss = rabbit_data_coercion:to_binary(?FUNCTION_NAME), + Partition = init_super_stream(Config, CL, Ss, 2, CF1), + [L, F1, F2] = topology(Config, Partition), + + wait_for_coordinator_ready(Config), + + %% check stream leader and coordinator are not on the same node + %% the coordinator leader node will be isolated + ?assertNotEqual(L#node.name, CL), + + {ok, So0, C0_00} = stream_test_utils:connect(L#node.stream_port), + {ok, So1, C1_00} = stream_test_utils:connect(F1#node.stream_port), + {ok, So2, C2_00} = stream_test_utils:connect(F2#node.stream_port), + + C0_01 = register_sac(So0, C0_00, Partition, 0, Ss), + C0_02 = receive_consumer_update(So0, C0_01), + + C1_01 = register_sac(So1, C1_00, Partition, 1, Ss), + + %% former active gets de-activated + C0_03 = receive_consumer_update_and_respond(So0, C0_02), + + %% gets activated + C1_02 = receive_consumer_update_and_respond(So1, C1_01), + + C2_01 = register_sac(So2, C2_00, Partition, 2, Ss), + SubIdToState0 = #{0 => {So0, C0_03}, + 1 => {So1, C1_02}, + 2 => {So2, C2_01}}, + + Consumers1 = query_consumers(Config, Partition), + assertSize(3, Consumers1), + assertConsumersConnected(Consumers1), + + LN = L#node.name, + F1N = F1#node.name, + F2N = F2#node.name, + + Isolated = F1N, + NotIsolated = F2N, + {value, DisconnectedConsumer} = + lists:search(fun(#consumer{pid = ConnPid}) -> + rpc(Config, erlang, node, [ConnPid]) =:= Isolated + end, Consumers1), + #consumer{subscription_id = DiscSubId} = DisconnectedConsumer, + + rabbit_ct_broker_helpers:block_traffic_between(Isolated, LN), + rabbit_ct_broker_helpers:block_traffic_between(Isolated, F2N), + + wait_for_disconnected_consumer(Config, NotIsolated, Partition), + wait_for_presumed_down_consumer(Config, NotIsolated, Partition), + + rabbit_ct_broker_helpers:allow_traffic_between(Isolated, LN), + rabbit_ct_broker_helpers:allow_traffic_between(Isolated, F2N), + + wait_for_coordinator_ready(Config), + + wait_for_all_consumers_connected(Config, NotIsolated, Partition), + + Consumers2 = query_consumers(Config, NotIsolated, Partition), + + %% the disconnected, then presumed down consumer is cancelled, + %% because the stream member on its node has been restarted + assertSize(2, Consumers2), + assertConsumersConnected(Consumers2), + assertEmpty(lists:filter(fun(C) -> + same_consumer(DisconnectedConsumer, C) + end, Consumers2)), + + [#consumer{subscription_id = ActiveSubId}] = + lists:filter(fun(#consumer{status = St}) -> + St =:= {connected, active} + end, Consumers2), + + SubIdToState1 = + maps:fold(fun(K, {S0, C0}, Acc) when K == DiscSubId -> + %% cancelled consumer received a metadata update + C1 = receive_metadata_update(S0, C0), + Acc#{K => {S0, C1}}; + (K, {S0, C0}, Acc) when K == ActiveSubId -> + %% promoted consumer should have received consumer update + C1 = receive_consumer_update_and_respond(S0, C0), + Acc#{K => {S0, C1}}; + (K, {S0, C0}, Acc) -> + Acc#{K => {S0, C0}} + end, #{}, SubIdToState0), + + delete_super_stream(L#node.stream_port, Ss), + + %% online consumers should receive a metadata update frame (stream deleted) + %% we unqueue this frame before closing the connection + %% directly closing the connection of the cancelled consumer + maps:foreach(fun(K, {S0, C0}) when K /= DiscSubId -> + {_, C1} = receive_commands(S0, C0), + {ok, _} = stream_test_utils:close(S0, C1); + (_, {S0, C0}) -> + {ok, _} = stream_test_utils:close(S0, C0) + end, SubIdToState1), + ok. + +same_consumer(#consumer{owner = P1, subscription_id = Id1}, + #consumer{owner = P2, subscription_id = Id2}) + when P1 == P2 andalso Id1 == Id2 -> + true; +same_consumer(_, _) -> + false. + +cluster_nodes(Config) -> + lists:map(fun(N) -> + #node{name = node_config(Config, N, nodename), + stream_port = stream_port(Config, N)} + end, lists:seq(0, node_count(Config) - 1)). + +node_count(Config) -> + test_server:lookup_config(rmq_nodes_count, Config). + +nodename(Config, N) -> + node_config(Config, N, nodename). + +stream_port(Config, N) -> + node_config(Config, N, tcp_port_stream). + +node_config(Config, N, K) -> + rabbit_ct_broker_helpers:get_node_config(Config, N, K). + +topology(Config, St) -> + Members = stream_members(Config, St), + LN = leader(Members), + Nodes = cluster_nodes(Config), + [L] = lists:filter(fun(#node{name = N}) -> + N =:= LN + end, Nodes), + [F1, F2] = lists:filter(fun(#node{name = N}) -> + N =/= LN + end, Nodes), + + [L, F1, F2]. + +leader(Members) -> + maps:fold(fun(Node, {_, writer}, _Acc) -> + Node; + (_, _, Acc) -> + Acc + end, undefined, Members). + +stream_members(Config, Stream) -> + {ok, Q} = rpc(Config, rabbit_amqqueue, lookup, [Stream, <<"/">>]), + #{name := StreamId} = amqqueue:get_type_state(Q), + State = rpc(Config, rabbit_stream_coordinator, state, []), + {ok, Members} = rpc(Config, rabbit_stream_coordinator, query_members, + [StreamId, State]), + Members. + +init_coordinator(Config) -> + %% to make sure the coordinator is initialized + init_stream(Config, 0, <<"dummy">>), + delete_stream(stream_port(Config, 0), <<"dummy">>), + wait_for_coordinator_ready(Config). + +init_stream(Config, N, St) -> + {ok, S, C0} = stream_test_utils:connect(stream_port(Config, N)), + {ok, C1} = stream_test_utils:create_stream(S, C0, St), + NC = node_count(Config), + wait_for_members(S, C1, St, NC), + {ok, _} = stream_test_utils:close(S, C1). + +delete_stream(Port, St) -> + {ok, S, C0} = stream_test_utils:connect(Port), + {ok, C1} = stream_test_utils:delete_stream(S, C0, St), + {ok, _} = stream_test_utils:close(S, C1). + +init_super_stream(Config, Node, Ss, PartitionIndex, ExpectedNode) -> + {ok, S, C0} = stream_test_utils:connect(Config, Node), + NC = node_count(Config), + Partitions = [unicode:characters_to_binary([Ss, <<"-">>, integer_to_binary(N)]) + || N <- lists:seq(0, NC - 1)], + Bks = [integer_to_binary(N) || N <- lists:seq(0, NC - 1)], + SsCreationFrame = request({create_super_stream, Ss, Partitions, Bks, #{}}), + ok = ?TRSPT:send(S, SsCreationFrame), + {Cmd1, C1} = receive_commands(S, C0), + ?assertMatch({response, ?CORR_ID, {create_super_stream, ?RESPONSE_CODE_OK}}, + Cmd1), + [wait_for_members(S, C1, P, NC) || P <- Partitions], + Partition = lists:nth(PartitionIndex, Partitions), + [#node{name = LN} | _] = topology(Config, Partition), + P = case LN of + ExpectedNode -> + Partition; + _ -> + enforce_stream_leader_on_node(Config, S, C1, + Partitions, Partition, + ExpectedNode, 10) + end, + {ok, _} = stream_test_utils:close(S, C1), + P. + + +enforce_stream_leader_on_node(_, _, _, _, _, _, 0) -> + ct:fail("could not create super stream partition on chosen node"); +enforce_stream_leader_on_node(Config, S, C, + Partitions, Partition, Node, Count) -> + CL = coordinator_leader(Config), + NC = node_count(Config), + [begin + case P of + Partition -> + restart_stream(Config, CL, P, Node); + _ -> + restart_stream(Config, CL, P, undefined) + end, + wait_for_members(S, C, P, NC) + end || P <- Partitions], + [#node{name = LN} | _] = topology(Config, Partition), + case LN of + Node -> + Partition; + _ -> + timer:sleep(500), + enforce_stream_leader_on_node(Config, S, C, + Partitions, Partition, Node, + Count - 1) + end. + +delete_super_stream(Port, Ss) -> + {ok, S, C0} = stream_test_utils:connect(Port), + SsDeletionFrame = request({delete_super_stream, Ss}), + ok = ?TRSPT:send(S, SsDeletionFrame), + {Cmd1, C1} = receive_commands(S, C0), + ?assertMatch({response, ?CORR_ID, {delete_super_stream, ?RESPONSE_CODE_OK}}, + Cmd1), + {ok, _} = stream_test_utils:close(S, C1). + +register_sac(S, C0, St, SubId, SuperStream) -> + register_sac0(S, C0, St, SubId, #{<<"super-stream">> => SuperStream}). + +register_sac(S, C0, St, SubId) -> + register_sac0(S, C0, St, SubId, #{}). + +register_sac0(S, C0, St, SubId, Args) -> + SacSubscribeFrame = request({subscribe, SubId, St, + first, 1, + Args#{<<"single-active-consumer">> => <<"true">>, + <<"name">> => name()}}), + ok = ?TRSPT:send(S, SacSubscribeFrame), + {Cmd1, C1} = receive_commands(S, C0), + ?assertMatch({response, ?CORR_ID, {subscribe, ?RESPONSE_CODE_OK}}, + Cmd1), + C1. + +receive_consumer_update(S, C0) -> + {Cmd, C1} = receive_commands(S, C0), + ?assertMatch({request, _CorrId, {consumer_update, _SubId, _Status}}, + Cmd), + C1. + +receive_consumer_update_and_respond(S, C0) -> + {Cmd, C1} = receive_commands(S, C0), + ?assertMatch({request, _CorrId, {consumer_update, _SubId, _Status}}, + Cmd), + {request, CorrId, {consumer_update, _SubId, _Status}} = Cmd, + Frame = response(CorrId, {consumer_update, ?RESPONSE_CODE_OK, first}), + ok = ?TRSPT:send(S, Frame), + C1. + +receive_metadata_update(S, C0) -> + {Cmd, C1} = receive_commands(S, C0), + ?assertMatch({metadata_update, _, ?RESPONSE_CODE_STREAM_NOT_AVAILABLE}, + Cmd), + C1. + +unsubscribe(S, C0) -> + {ok, C1} = stream_test_utils:unsubscribe(S, C0, sub_id()), + C1. + +query_consumers(Config, Stream) -> + query_consumers(Config, 0, Stream). + +query_consumers(Config, Node, Stream) -> + Key = group_key(Stream), + #?SAC_STATE{groups = #{Key := #group{consumers = Consumers}}} = + rpc(Config, Node, rabbit_stream_coordinator, sac_state, []), + Consumers. + + +all_nodes(Config) -> + lists:map(fun(N) -> + nodename(Config, N) + end, lists:seq(0, node_count(Config) - 1)). + +coordinator_status(Config) -> + rpc(Config, rabbit_stream_coordinator, status, []). + +coordinator_leader(Config) -> + Status = coordinator_status(Config), + case lists:search(fun(St) -> + RS = proplists:get_value(<<"Raft State">>, St, + undefined), + RS == leader + end, Status) of + {value, Leader} -> + proplists:get_value(<<"Node Name">>, Leader, undefined); + _ -> + undefined + end. + +restart_stream(Config, Node, S, undefined) -> + rpc(Config, Node, rabbit_stream_queue, restart_stream, [<<"/">>, S, #{}]); +restart_stream(Config, Node, S, Leader) -> + Opts = #{preferred_leader_node => Leader}, + rpc(Config, Node, rabbit_stream_queue, restart_stream, [<<"/">>, S, Opts]). + + +rpc(Config, M, F, A) -> + rpc(Config, 0, M, F, A). + +rpc(Config, Node, M, F, A) -> + rabbit_ct_broker_helpers:rpc(Config, Node, M, F, A). + +group_key(Stream) -> + {<<"/">>, Stream, name()}. + +request(Cmd) -> + request(?CORR_ID, Cmd). + +request(CorrId, Cmd) -> + rabbit_stream_core:frame({request, CorrId, Cmd}). + +response(CorrId, Cmd) -> + rabbit_stream_core:frame({response, CorrId, Cmd}). + +receive_commands(S, C) -> + receive_commands(?TRSPT, S, C). + +receive_commands(Transport, S, C) -> + stream_test_utils:receive_stream_commands(Transport, S, C). + +sub_id() -> + 0. + +name() -> + <<"app">>. + +wait_for_members(S, C, St, ExpectedCount) -> + T = ?TRSPT, + GetStreamNodes = + fun() -> + MetadataFrame = request({metadata, [St]}), + ok = gen_tcp:send(S, MetadataFrame), + {CmdMetadata, _} = receive_commands(T, S, C), + {response, 1, + {metadata, _Nodes, #{St := {Leader = {_H, _P}, Replicas}}}} = + CmdMetadata, + [Leader | Replicas] + end, + rabbit_ct_helpers:await_condition(fun() -> + length(GetStreamNodes()) == ExpectedCount + end). + +wait_for_disconnected_consumer(Config, Node, Stream) -> + rabbit_ct_helpers:await_condition( + fun() -> + Cs = query_consumers(Config, Node, Stream), + lists:any(fun(#consumer{status = {disconnected, _}}) -> + true; + (_) -> + false + end, Cs) + end). + +wait_for_presumed_down_consumer(Config, Node, Stream) -> + rabbit_ct_helpers:await_condition( + fun() -> + Cs = query_consumers(Config, Node, Stream), + lists:any(fun(#consumer{status = {presumed_down, _}}) -> + true; + (_) -> + false + end, Cs) + end). + +wait_for_all_consumers_connected(Config, Node, Stream) -> + rabbit_ct_helpers:await_condition( + fun() -> + Cs = query_consumers(Config, Node, Stream), + lists:all(fun(#consumer{status = {connected, _}}) -> + true; + (_) -> + false + end, Cs) + end, 30_000). + +wait_for_coordinator_ready(Config) -> + NC = node_count(Config), + rabbit_ct_helpers:await_condition( + fun() -> + Status = coordinator_status(Config), + lists:all(fun(St) -> + RS = proplists:get_value(<<"Raft State">>, St, + undefined), + RS == leader orelse RS == follower + end, Status) andalso length(Status) == NC + end). + +assertConsumersConnected(Consumers) when length(Consumers) > 0 -> + lists:foreach(fun(#consumer{status = St}) -> + ?assertMatch({connected, _}, St, + "Consumer should be connected") + end, Consumers); +assertConsumersConnected(_) -> + ?assert(false, "The consumer list is empty"). + +assertSize(Expected, []) -> + ?assertEqual(Expected, 0); +assertSize(Expected, Map) when is_map(Map) -> + ?assertEqual(Expected, maps:size(Map)); +assertSize(Expected, List) when is_list(List) -> + ?assertEqual(Expected, length(List)). + +assertEmpty(Data) -> + assertSize(0, Data). From a9cf04903081d58b3498af72a357b71818de8f67 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Arnaud=20Cogolu=C3=A8gnes?= <514737+acogoluegnes@users.noreply.github.com> Date: Mon, 2 Jun 2025 09:18:24 +0200 Subject: [PATCH 2/4] Remove only stream subscriptions affected by down stream member The clean-up of a stream connection state when a stream member goes down can remove subscriptions not affected by the member. The subscription state is removed from the connection, but the subscription is not removed from the SAC state (if the subscription is a SAC), because the subscription member PID does not match the down member PID. When the actual member of the subscription goes down, the subscription is no longer part of the state, so the clean-up does not find the subscription and does not remove it from the SAC state. This lets a ghost consumer in the corresponding SAC group. This commit makes sure only the affected subscriptions are removed from the state when a stream member goes down. Fixes #13961 --- .../src/rabbit_stream_reader.erl | 177 ++++++++++-------- .../test/rabbit_stream_reader_SUITE.erl | 38 ++++ 2 files changed, 138 insertions(+), 77 deletions(-) diff --git a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl index 544700a53499..0b1633b41709 100644 --- a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl +++ b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl @@ -106,7 +106,8 @@ close_sent/3]). -ifdef(TEST). -export([ensure_token_expiry_timer/2, - evaluate_state_after_secret_update/4]). + evaluate_state_after_secret_update/4, + clean_subscriptions/4]). -endif. callback_mode() -> @@ -3280,89 +3281,19 @@ clean_state_after_super_stream_deletion(Partitions, Connection, State, Transport clean_state_after_stream_deletion_or_failure(MemberPid, Stream, #stream_connection{ - user = #user{username = Username}, - virtual_host = VirtualHost, - stream_subscriptions = StreamSubscriptions, - publishers = Publishers, - publisher_to_ids = PublisherToIds, - stream_leaders = Leaders, - outstanding_requests = Requests0} = C0, - #stream_connection_state{consumers = Consumers} = S0) -> + stream_leaders = Leaders} = C0, + S0) -> {SubscriptionsCleaned, C1, S1} = case stream_has_subscriptions(Stream, C0) of true -> - #{Stream := SubscriptionIds} = StreamSubscriptions, - Requests1 = lists:foldl( - fun(SubId, Rqsts0) -> - #{SubId := Consumer} = Consumers, - case {MemberPid, Consumer} of - {undefined, _C} -> - rabbit_stream_metrics:consumer_cancelled(self(), - stream_r(Stream, - C0), - SubId, - Username), - maybe_unregister_consumer( - VirtualHost, Consumer, - single_active_consumer(Consumer), - Rqsts0); - {MemberPid, #consumer{configuration = - #consumer_configuration{member_pid = MemberPid}}} -> - rabbit_stream_metrics:consumer_cancelled(self(), - stream_r(Stream, - C0), - SubId, - Username), - maybe_unregister_consumer( - VirtualHost, Consumer, - single_active_consumer(Consumer), - Rqsts0); - _ -> - Rqsts0 - end - end, Requests0, SubscriptionIds), - {true, - C0#stream_connection{stream_subscriptions = - maps:remove(Stream, - StreamSubscriptions), - outstanding_requests = Requests1}, - S0#stream_connection_state{consumers = - maps:without(SubscriptionIds, - Consumers)}}; + clean_subscriptions(MemberPid, Stream, C0, S0); false -> {false, C0, S0} end, {PublishersCleaned, C2, S2} = case stream_has_publishers(Stream, C1) of true -> - {PurgedPubs, PurgedPubToIds} = - maps:fold(fun(PubId, - #publisher{stream = S, reference = Ref}, - {Pubs, PubToIds}) when S =:= Stream andalso MemberPid =:= undefined -> - rabbit_stream_metrics:publisher_deleted(self(), - stream_r(Stream, - C1), - PubId), - {maps:remove(PubId, Pubs), - maps:remove({Stream, Ref}, PubToIds)}; - (PubId, - #publisher{stream = S, reference = Ref, leader = MPid}, - {Pubs, PubToIds}) when S =:= Stream andalso MPid =:= MemberPid -> - rabbit_stream_metrics:publisher_deleted(self(), - stream_r(Stream, - C1), - PubId), - {maps:remove(PubId, Pubs), - maps:remove({Stream, Ref}, PubToIds)}; - - (_PubId, _Publisher, {Pubs, PubToIds}) -> - {Pubs, PubToIds} - end, - {Publishers, PublisherToIds}, Publishers), - {true, - C1#stream_connection{publishers = PurgedPubs, - publisher_to_ids = PurgedPubToIds}, - S1}; + clean_publishers(MemberPid, Stream, C1, S1); false -> {false, C1, S1} end, @@ -3384,6 +3315,98 @@ clean_state_after_stream_deletion_or_failure(MemberPid, Stream, {not_cleaned, C2#stream_connection{stream_leaders = Leaders1}, S2} end. +clean_subscriptions(MemberPid, Stream, + #stream_connection{user = #user{username = Username}, + virtual_host = VirtualHost, + stream_subscriptions = StreamSubs, + outstanding_requests = Requests0} = C0, + #stream_connection_state{consumers = Consumers} = S0) -> + #{Stream := SubIds} = StreamSubs, + {DelSubs1, Requests1} = + lists:foldl( + fun(SubId, {DelSubIds, Rqsts0}) -> + #{SubId := Consumer} = Consumers, + case {MemberPid, Consumer} of + {undefined, _C} -> + rabbit_stream_metrics:consumer_cancelled(self(), + stream_r(Stream, + C0), + SubId, + Username), + Rqsts1 = maybe_unregister_consumer( + VirtualHost, Consumer, + single_active_consumer(Consumer), + Rqsts0), + {[SubId | DelSubIds], Rqsts1}; + {MemberPid, + #consumer{configuration = + #consumer_configuration{member_pid = MemberPid}}} -> + rabbit_stream_metrics:consumer_cancelled(self(), + stream_r(Stream, + C0), + SubId, + Username), + Rqsts1 = maybe_unregister_consumer( + VirtualHost, Consumer, + single_active_consumer(Consumer), + Rqsts0), + {[SubId | DelSubIds], Rqsts1}; + _ -> + {DelSubIds, Rqsts0} + end + end, {[], Requests0}, SubIds), + case DelSubs1 of + [] -> + {false, C0, S0}; + _ -> + StreamSubs1 = case SubIds -- DelSubs1 of + [] -> + maps:remove(Stream, StreamSubs); + RemSubIds -> + StreamSubs#{Stream => RemSubIds} + end, + Consumers1 = maps:without(DelSubs1, Consumers), + {true, + C0#stream_connection{stream_subscriptions = StreamSubs1, + outstanding_requests = Requests1}, + S0#stream_connection_state{consumers = Consumers1}} + end. + +clean_publishers(MemberPid, Stream, + #stream_connection{ + publishers = Publishers, + publisher_to_ids = PublisherToIds} = C0, S0) -> + {Updated, PurgedPubs, PurgedPubToIds} = + maps:fold(fun(PubId, #publisher{stream = S, reference = Ref}, + {_, Pubs, PubToIds}) + when S =:= Stream andalso MemberPid =:= undefined -> + rabbit_stream_metrics:publisher_deleted(self(), + stream_r(Stream, + C0), + PubId), + {true, + maps:remove(PubId, Pubs), + maps:remove({Stream, Ref}, PubToIds)}; + (PubId, #publisher{stream = S, reference = Ref, leader = MPid}, + {_, Pubs, PubToIds}) + when S =:= Stream andalso MPid =:= MemberPid -> + rabbit_stream_metrics:publisher_deleted(self(), + stream_r(Stream, + C0), + PubId), + {true, + maps:remove(PubId, Pubs), + maps:remove({Stream, Ref}, PubToIds)}; + + (_PubId, _Publisher, {Updated, Pubs, PubToIds}) -> + {Updated, Pubs, PubToIds} + end, + {false, Publishers, PublisherToIds}, Publishers), + {Updated, + C0#stream_connection{publishers = PurgedPubs, + publisher_to_ids = PurgedPubToIds}, + S0}. + store_offset(Reference, _, _, C) when ?IS_INVALID_REF(Reference) -> rabbit_log:warning("Reference is too long to store offset: ~p", [byte_size(Reference)]), C; @@ -3401,8 +3424,7 @@ store_offset(Reference, Stream, Offset, Connection0) -> lookup_leader(Stream, #stream_connection{stream_leaders = StreamLeaders, - virtual_host = VirtualHost} = - Connection) -> + virtual_host = VirtualHost} = Connection) -> case maps:get(Stream, StreamLeaders, undefined) of undefined -> case lookup_leader_from_manager(VirtualHost, Stream) of @@ -3411,6 +3433,7 @@ lookup_leader(Stream, {ok, LeaderPid} -> Connection1 = maybe_monitor_stream(LeaderPid, Stream, Connection), + {LeaderPid, Connection1#stream_connection{stream_leaders = StreamLeaders#{Stream => diff --git a/deps/rabbitmq_stream/test/rabbit_stream_reader_SUITE.erl b/deps/rabbitmq_stream/test/rabbit_stream_reader_SUITE.erl index c32666706ca2..747cd3105e37 100644 --- a/deps/rabbitmq_stream/test/rabbit_stream_reader_SUITE.erl +++ b/deps/rabbitmq_stream/test/rabbit_stream_reader_SUITE.erl @@ -184,6 +184,44 @@ evaluate_state_after_secret_update_test(_) -> ?assert(is_integer(Cancel2)), ok. +clean_subscriptions_should_remove_only_affected_subscriptions_test(_) -> + Mod = rabbit_stream_reader, + meck:new(Mod, [passthrough]), + meck:new(rabbit_stream_metrics, [stub_all]), + meck:new(rabbit_stream_sac_coordinator, [stub_all]), + + S = <<"s1">>, + Pid1 = new_process(), + Pid2 = new_process(), + StreamSubs = #{S => [0, 1]}, + Consumers = #{0 => consumer(S, Pid1), + 1 => consumer(S, Pid2)}, + + C0 = #stream_connection{stream_subscriptions = StreamSubs, + user = #user{}}, + S0 = #stream_connection_state{consumers = Consumers}, + {Cleaned1, C1, S1} = Mod:clean_subscriptions(Pid1, S, C0, S0), + ?assert(Cleaned1), + ?assertEqual(#{S => [1]}, + C1#stream_connection.stream_subscriptions), + ?assertEqual(#{1 => consumer(S, Pid2)}, + S1#stream_connection_state.consumers), + + {Cleaned2, C2, S2} = Mod:clean_subscriptions(Pid2, S, C1, S1), + ?assert(Cleaned2), + ?assertEqual(#{}, C2#stream_connection.stream_subscriptions), + ?assertEqual(#{}, S2#stream_connection_state.consumers), + + ok. + +consumer(S, Pid) -> + #consumer{configuration = #consumer_configuration{stream = S, + member_pid = Pid}}. + consumer(S) -> #consumer{configuration = #consumer_configuration{stream = S}, log = osiris_log:init(#{})}. + +new_process() -> + spawn(node(), fun() -> ok end). + From 58f4e83c2242a87b627ccfa0d3c56fa42a464695 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Arnaud=20Cogolu=C3=A8gnes?= <514737+acogoluegnes@users.noreply.github.com> Date: Fri, 6 Jun 2025 11:17:51 +0200 Subject: [PATCH 3/4] Close stream connection in case of unexpected error from SAC coordinator Calls to the stream SAC coordinator can fail for various reason (e.g. a timeout because of a network partition). The stream reader does not take into account what the SAC coordinator returns and moves on even in case of errors. This can lead to inconsistent state for SAC groups. This commit changes this behavior by handling unexpected errors from the SAC coordinator and closing the connection. The client is expected to reconnect. This is safer than risking inconsistent state. Fixes #14040 --- .../src/rabbit_stream_sac_coordinator.erl | 23 ++-- .../src/rabbit_stream_reader.erl | 103 +++++++++++------- 2 files changed, 77 insertions(+), 49 deletions(-) diff --git a/deps/rabbit/src/rabbit_stream_sac_coordinator.erl b/deps/rabbit/src/rabbit_stream_sac_coordinator.erl index b29b4d8fe00f..0c078a4b1622 100644 --- a/deps/rabbit/src/rabbit_stream_sac_coordinator.erl +++ b/deps/rabbit/src/rabbit_stream_sac_coordinator.erl @@ -27,6 +27,8 @@ -opaque state() :: #?MODULE{}. +-type sac_error() :: partition_index_conflict | not_found. + -export_type([state/0, command/0]). @@ -50,7 +52,8 @@ import_state/2, check_conf_change/1, list_nodes/1, - state_enter/2 + state_enter/2, + is_sac_error/1 ]). -export([make_purge_nodes/1, make_update_conf/1]). @@ -89,7 +92,7 @@ pid(), binary(), integer()) -> - {ok, boolean()} | {error, term()}. + {ok, boolean()} | {error, sac_error() | term()}. register_consumer(VirtualHost, Stream, PartitionIndex, @@ -110,7 +113,7 @@ register_consumer(VirtualHost, binary(), pid(), integer()) -> - ok | {error, term()}. + ok | {error, sac_error() | term()}. unregister_consumer(VirtualHost, Stream, ConsumerName, @@ -122,13 +125,15 @@ unregister_consumer(VirtualHost, connection_pid = ConnectionPid, subscription_id = SubscriptionId}). --spec activate_consumer(binary(), binary(), binary()) -> ok. +-spec activate_consumer(binary(), binary(), binary()) -> + ok | {error, sac_error() | term()}. activate_consumer(VH, Stream, Name) -> process_command(#command_activate_consumer{vhost =VH, stream = Stream, consumer_name= Name}). --spec connection_reconnected(connection_pid()) -> ok. +-spec connection_reconnected(connection_pid()) -> + ok | {error, sac_error() | term()}. connection_reconnected(Pid) -> process_command(#command_connection_reconnected{pid = Pid}). @@ -150,7 +155,7 @@ wrap_cmd(Cmd) -> %% (CLI command) -spec consumer_groups(binary(), [atom()]) -> {ok, - [term()] | {error, atom()}}. + [term()]} | {error, sac_error() | term()}. consumer_groups(VirtualHost, InfoKeys) -> case ra_local_query(fun(State) -> SacState = @@ -172,7 +177,7 @@ consumer_groups(VirtualHost, InfoKeys) -> %% (CLI command) -spec group_consumers(binary(), binary(), binary(), [atom()]) -> {ok, [term()]} | - {error, atom()}. + {error, sac_error() | term()}. group_consumers(VirtualHost, Stream, Reference, InfoKeys) -> case ra_local_query(fun(State) -> SacState = @@ -932,6 +937,10 @@ state_enter(leader, #?MODULE{groups = Groups} = State) state_enter(_, _) -> []. +-spec is_sac_error(term()) -> boolean(). +is_sac_error(Reason) -> + lists:member(Reason, ?SAC_ERRORS). + nodes_from_group(#group{consumers = Cs}) when is_list(Cs) -> lists:foldl(fun(#consumer{pid = Pid}, Acc) -> Acc#{node(Pid) => true} diff --git a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl index 0b1633b41709..c7ef31b292c1 100644 --- a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl +++ b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl @@ -81,6 +81,7 @@ -define(UNKNOWN_FIELD, unknown_field). -define(SILENT_CLOSE_DELAY, 3_000). -define(IS_INVALID_REF(Ref), is_binary(Ref) andalso byte_size(Ref) > 255). +-define(SAC_MOD, rabbit_stream_sac_coordinator). -import(rabbit_stream_utils, [check_write_permitted/2, check_read_permitted/3]). @@ -722,7 +723,7 @@ open(info, {OK, S, Data}, connection_state = State2}} end; open(info, {sac, check_connection, _}, State) -> - rabbit_stream_sac_coordinator:connection_reconnected(self()), + _ = sac_connection_reconnected(self()), {keep_state, State}; open(info, {sac, #{subscription_id := SubId, @@ -794,17 +795,15 @@ open(info, rabbit_log:debug("Subscription ~tp on ~tp has been deleted.", [SubId, Stream]), rabbit_log:debug("Active ~tp, message ~tp", [Active, Msg]), - case {Active, Msg} of - {false, #{stepping_down := true, - stream := St, - consumer_name := ConsumerName}} -> - rabbit_log:debug("Former active consumer gone, activating consumer " ++ - "on stream ~tp, group ~tp", [St, ConsumerName]), - _ = rabbit_stream_sac_coordinator:activate_consumer(VirtualHost, - St, - ConsumerName); - _ -> - ok + _ = case {Active, Msg} of + {false, #{stepping_down := true, + stream := St, + consumer_name := ConsumerName}} -> + rabbit_log:debug("Former active consumer gone, activating consumer " ++ + "on stream ~tp, group ~tp", [St, ConsumerName]), + sac_activate_consumer(VirtualHost, St, ConsumerName); + _ -> + ok end, {Connection0, ConnState0} end, @@ -2554,9 +2553,8 @@ handle_frame_post_auth(Transport, rabbit_log:debug("Subscription ~tp on stream ~tp, group ~tp " ++ "has stepped down, activating consumer", [SubscriptionId, Stream, ConsumerName]), - _ = rabbit_stream_sac_coordinator:activate_consumer(VirtualHost, - Stream, - ConsumerName), + _ = sac_activate_consumer(VirtualHost, Stream, + ConsumerName), ok; _ -> ok @@ -3015,21 +3013,9 @@ handle_subscription(Transport,#stream_connection{ maybe_register_consumer(_, _, _, _, _, _, false = _Sac) -> {ok, true}; -maybe_register_consumer(VirtualHost, - Stream, - ConsumerName, - ConnectionName, - SubscriptionId, - Properties, - true) -> - PartitionIndex = partition_index(VirtualHost, Stream, Properties), - rabbit_stream_sac_coordinator:register_consumer(VirtualHost, - Stream, - PartitionIndex, - ConsumerName, - self(), - ConnectionName, - SubscriptionId). +maybe_register_consumer(VH, St, Name, ConnName, SubId, Properties, true) -> + PartitionIndex = partition_index(VH, St, Properties), + sac_register_consumer(VH, St, PartitionIndex, Name, self(), ConnName, SubId). maybe_send_consumer_update(Transport, Connection = #stream_connection{ @@ -3175,13 +3161,12 @@ maybe_unregister_consumer(VirtualHost, ConsumerName = consumer_name(Properties), Requests1 = maps:fold( - fun(_, #request{content = - #{active := false, - subscription_id := SubId, - stepping_down := true}}, Acc) when SubId =:= SubscriptionId -> - _ = rabbit_stream_sac_coordinator:activate_consumer(VirtualHost, - Stream, - ConsumerName), + fun(_, #request{content = #{active := false, + subscription_id := SubId, + stepping_down := true}}, Acc) + when SubId =:= SubscriptionId -> + _ = sac_activate_consumer(VirtualHost, Stream, + ConsumerName), rabbit_log:debug("Outstanding SAC activation request for stream '~tp', " ++ "group '~tp', sending activation.", [Stream, ConsumerName]), @@ -3190,11 +3175,8 @@ maybe_unregister_consumer(VirtualHost, Acc#{K => V} end, maps:new(), Requests), - _ = rabbit_stream_sac_coordinator:unregister_consumer(VirtualHost, - Stream, - ConsumerName, - self(), - SubscriptionId), + _ = sac_unregister_consumer(VirtualHost, Stream, ConsumerName, + self(), SubscriptionId), Requests1. partition_index(VirtualHost, Stream, Properties) -> @@ -4037,3 +4019,40 @@ stream_from_consumers(SubId, Consumers) -> %% for a bit so they can't DOS us with repeated failed logins etc. silent_close_delay() -> timer:sleep(?SILENT_CLOSE_DELAY). + +sac_connection_reconnected(Pid) -> + sac_call(fun() -> + ?SAC_MOD:connection_reconnected(Pid) + end). + +sac_activate_consumer(VH, St, Name) -> + sac_call(fun() -> + ?SAC_MOD:activate_consumer(VH, St, Name) + end). + +sac_register_consumer(VH, St, PartitionIndex, Name, Pid, ConnName, SubId) -> + sac_call(fun() -> + ?SAC_MOD:register_consumer(VH, St, PartitionIndex, + Name, Pid, ConnName, + SubId) + end). + +sac_unregister_consumer(VH, St, Name, Pid, SubId) -> + sac_call(fun() -> + ?SAC_MOD:unregister_consumer(VH, St, Name, Pid, SubId) + end). + +sac_call(Call) -> + case Call() of + {error, Reason} = Err -> + case ?SAC_MOD:is_sac_error(Reason) of + true -> + Err; + _ -> + rabbit_log:info("Stream SAC coordinator call failed with ~tp", + [Reason]), + throw({stop, {shutdown, stream_sac_coordinator_error}}) + end; + R -> + R + end. From 41acc117bdccb07796c80849f21094e2b98f21a4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Arnaud=20Cogolu=C3=A8gnes?= <514737+acogoluegnes@users.noreply.github.com> Date: Tue, 10 Jun 2025 16:51:08 +0200 Subject: [PATCH 4/4] Add activate_stream_consumer command New CLI command to trigger a rebalancing in a SAC group and activate a consumer. This is a last resort solution if all consumers in a group accidently end up in {connected, waiting} state. The command re-uses an existing function, which only picks the consumer that should be active. This means it does not try to "fix" the state (e.g. removing a disconnected consumer because its node is definitely gone from the cluster). Fixes #14055 --- .../src/rabbit_stream_sac_coordinator.erl | 15 ++- .../rabbit_stream_sac_coordinator_SUITE.erl | 80 +++++++++++++++ ...Commands.ActivateStreamConsumerCommand.erl | 99 +++++++++++++++++++ deps/rabbitmq_stream/test/commands_SUITE.erl | 68 +++++++++++++ 4 files changed, 257 insertions(+), 5 deletions(-) create mode 100644 deps/rabbitmq_stream/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ActivateStreamConsumerCommand.erl diff --git a/deps/rabbit/src/rabbit_stream_sac_coordinator.erl b/deps/rabbit/src/rabbit_stream_sac_coordinator.erl index 0c078a4b1622..00b7fb5dde3e 100644 --- a/deps/rabbit/src/rabbit_stream_sac_coordinator.erl +++ b/deps/rabbit/src/rabbit_stream_sac_coordinator.erl @@ -128,7 +128,7 @@ unregister_consumer(VirtualHost, -spec activate_consumer(binary(), binary(), binary()) -> ok | {error, sac_error() | term()}. activate_consumer(VH, Stream, Name) -> - process_command(#command_activate_consumer{vhost =VH, + process_command(#command_activate_consumer{vhost = VH, stream = Stream, consumer_name= Name}). @@ -323,7 +323,13 @@ apply(#command_activate_consumer{vhost = VirtualHost, end, StreamGroups1 = update_groups(VirtualHost, Stream, ConsumerName, G, StreamGroups0), - {State0#?MODULE{groups = StreamGroups1}, ok, Eff}; + R = case G of + undefined -> + {error, not_found}; + _ -> + ok + end, + {State0#?MODULE{groups = StreamGroups1}, R, Eff}; apply(#command_connection_reconnected{pid = Pid}, #?MODULE{groups = Groups0} = State0) -> {State1, Eff} = @@ -1157,9 +1163,8 @@ maybe_create_group(VirtualHost, #{{VirtualHost, Stream, ConsumerName} := _} -> {ok, StreamGroups}; SGS -> - {ok, maps:put({VirtualHost, Stream, ConsumerName}, - #group{consumers = [], partition_index = PartitionIndex}, - SGS)} + {ok, SGS#{{VirtualHost, Stream, ConsumerName} => + #group{consumers = [], partition_index = PartitionIndex}}} end. lookup_group(VirtualHost, Stream, ConsumerName, StreamGroups) -> diff --git a/deps/rabbit/test/rabbit_stream_sac_coordinator_SUITE.erl b/deps/rabbit/test/rabbit_stream_sac_coordinator_SUITE.erl index 59d4e64a8082..800ddb656ab6 100644 --- a/deps/rabbit/test/rabbit_stream_sac_coordinator_SUITE.erl +++ b/deps/rabbit/test/rabbit_stream_sac_coordinator_SUITE.erl @@ -949,6 +949,82 @@ active_consumer_super_stream_disconn_active_block_rebalancing_test(_) -> assertEmpty(Eff), ok. +activate_consumer_simple_unblock_all_waiting_test(_) -> + P = self(), + GId = group_id(), + Group = grp([csr(P, 0, {connected, waiting}), + csr(P, 1, {connected, waiting}), + csr(P, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = activate_consumer_command(stream(), name()), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + assertHasGroup(GId, grp([csr(P, 0, {connected, active}), + csr(P, 1, {connected, waiting}), + csr(P, 2, {connected, waiting})]), + Groups1), + assertContainsActivateMessage(P, 0, Eff), + ok. + +activate_consumer_simple_unblock_ignore_disconnected_test(_) -> + P = self(), + GId = group_id(), + Group = grp([csr(P, 0, {disconnected, waiting}), + csr(P, 1, {connected, waiting}), + csr(P, 2, {connected, waiting}), + csr(P, 3, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = activate_consumer_command(stream(), name()), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + assertHasGroup(GId, grp([csr(P, 0, {disconnected, waiting}), + csr(P, 1, {connected, active}), + csr(P, 2, {connected, waiting}), + csr(P, 3, {connected, waiting})]), + Groups1), + assertContainsActivateMessage(P, 1, Eff), + ok. + +activate_consumer_super_stream_unblock_all_waiting_test(_) -> + P = self(), + GId = group_id(), + Group = grp(1, [csr(P, 0, {connected, waiting}), + csr(P, 1, {connected, waiting}), + csr(P, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = activate_consumer_command(stream(), name()), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + assertHasGroup(GId, grp(1, [csr(P, 0, {connected, waiting}), + csr(P, 1, {connected, active}), + csr(P, 2, {connected, waiting})]), + Groups1), + assertContainsActivateMessage(P, 1, Eff), + ok. + +activate_consumer_super_stream_unblock_ignore_disconnected_test(_) -> + P = self(), + GId = group_id(), + Group = grp(1, [csr(P, 0, {disconnected, waiting}), + csr(P, 1, {connected, waiting}), + csr(P, 2, {connected, waiting}), + csr(P, 3, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + Cmd = activate_consumer_command(stream(), name()), + {#?STATE{groups = Groups1}, ok, Eff} = ?MOD:apply(Cmd, State0), + assertHasGroup(GId, grp(1, [csr(P, 0, {disconnected, waiting}), + csr(P, 1, {connected, waiting}), + csr(P, 2, {connected, active}), + csr(P, 3, {connected, waiting})]), + Groups1), + assertContainsActivateMessage(P, 2, Eff), + ok. + handle_connection_down_simple_disconn_active_block_rebalancing_test(_) -> Pid0 = new_process(), Pid1 = new_process(), @@ -1729,6 +1805,10 @@ assertContainsCheckConnectionEffect(Pid, Effects) -> assertContainsSendMessageEffect(Pid, Stream, Active, Effects) -> assertContainsSendMessageEffect(Pid, 0, Stream, name(), Active, Effects). +assertContainsActivateMessage(Pid, SubId, Effects) -> + assertContainsSendMessageEffect(Pid, SubId, stream(), name(), + true, Effects). + assertContainsActivateMessage(Pid, Effects) -> assertContainsSendMessageEffect(Pid, sub_id(), stream(), name(), true, Effects). diff --git a/deps/rabbitmq_stream/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ActivateStreamConsumerCommand.erl b/deps/rabbitmq_stream/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ActivateStreamConsumerCommand.erl new file mode 100644 index 000000000000..5910269e1002 --- /dev/null +++ b/deps/rabbitmq_stream/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ActivateStreamConsumerCommand.erl @@ -0,0 +1,99 @@ +%% The contents of this file are subject to the Mozilla Public License +%% Version 2.0 (the "License"); you may not use this file except in +%% compliance with the License. You may obtain a copy of the License +%% at https://www.mozilla.org/MPL/ +%% +%% Software distributed under the License is distributed on an "AS IS" +%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See +%% the License for the specific language governing rights and +%% limitations under the License. +%% +%% The Original Code is RabbitMQ. +%% +%% The Initial Developer of the Original Code is GoPivotal, Inc. +%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. + +-module('Elixir.RabbitMQ.CLI.Ctl.Commands.ActivateStreamConsumerCommand'). + +-include_lib("rabbitmq_stream_common/include/rabbit_stream.hrl"). + +-behaviour('Elixir.RabbitMQ.CLI.CommandBehaviour'). + +-export([formatter/0, + scopes/0, + switches/0, + aliases/0, + usage/0, + usage_additional/0, + usage_doc_guides/0, + banner/2, + validate/2, + merge_defaults/2, + run/2, + output/2, + description/0, + help_section/0]). + +formatter() -> + 'Elixir.RabbitMQ.CLI.Formatters.String'. + +scopes() -> + [ctl, streams]. + +switches() -> + [{stream, string}, {reference, string}]. + +aliases() -> + []. + +description() -> + <<"Trigger a rebalancing to activate a consumer in " + "a single active consumer group">>. + +help_section() -> + {plugin, stream}. + +validate([], #{stream := _, reference := _}) -> + ok; +validate(Args, _) when is_list(Args) andalso length(Args) > 0 -> + {validation_failure, too_many_args}; +validate(_, _) -> + {validation_failure, not_enough_args}. + +merge_defaults(_Args, Opts) -> + {[], maps:merge(#{vhost => <<"/">>}, Opts)}. + +usage() -> + <<"activate_stream_consumer --stream " + "--reference [--vhost ]">>. + +usage_additional() -> + <<"debugging command, use only when a group does not have " + "an active consumer">>. + +usage_doc_guides() -> + [?STREAMS_GUIDE_URL]. + +run(_, + #{node := NodeName, + vhost := VHost, + stream := Stream, + reference := Reference, + timeout := Timeout}) -> + rabbit_misc:rpc_call(NodeName, + rabbit_stream_sac_coordinator, + activate_consumer, + [VHost, Stream, Reference], + Timeout). + +banner(_, _) -> + <<"Activating a consumer in the group ...">>. + +output(ok, _Opts) -> + 'Elixir.RabbitMQ.CLI.DefaultOutput':output({ok, + <<"OK">>}); +output({error, not_found}, _Opts) -> + 'Elixir.RabbitMQ.CLI.DefaultOutput':output({error_string, + <<"The group does not exist">>}); +output(Result, _Opts) -> + 'Elixir.RabbitMQ.CLI.DefaultOutput':output(Result). diff --git a/deps/rabbitmq_stream/test/commands_SUITE.erl b/deps/rabbitmq_stream/test/commands_SUITE.erl index 0942f9476522..0928acd6b5a7 100644 --- a/deps/rabbitmq_stream/test/commands_SUITE.erl +++ b/deps/rabbitmq_stream/test/commands_SUITE.erl @@ -33,6 +33,9 @@ 'Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamGroupConsumersCommand'). -define(COMMAND_LIST_STREAM_TRACKING, 'Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamTrackingCommand'). +-define(COMMAND_ACTIVATE_STREAM_CONSUMER, + 'Elixir.RabbitMQ.CLI.Ctl.Commands.ActivateStreamConsumerCommand'). + all() -> [{group, list_connections}, @@ -40,6 +43,7 @@ all() -> {group, list_publishers}, {group, list_consumer_groups}, {group, list_group_consumers}, + {group, activate_consumer}, {group, list_stream_tracking}, {group, super_streams}]. @@ -57,6 +61,9 @@ groups() -> {list_group_consumers, [], [list_group_consumers_validate, list_group_consumers_merge_defaults, list_group_consumers_run]}, + {activate_consumer, [], + [activate_consumer_validate, activate_consumer_merge_defaults, + activate_consumer_run]}, {list_stream_tracking, [], [list_stream_tracking_validate, list_stream_tracking_merge_defaults, list_stream_tracking_run]}, @@ -524,6 +531,67 @@ list_group_consumers_run(Config) -> close(S, C), ok. +activate_consumer_validate(_) -> + Cmd = ?COMMAND_ACTIVATE_STREAM_CONSUMER, + ValidOpts = #{vhost => <<"/">>, + stream => <<"s1">>, + reference => <<"foo">>}, + ?assertMatch({validation_failure, not_enough_args}, + Cmd:validate([], #{})), + ?assertMatch({validation_failure, not_enough_args}, + Cmd:validate([], #{vhost => <<"test">>})), + ?assertMatch({validation_failure, too_many_args}, + Cmd:validate([<<"foo">>], ValidOpts)), + ?assertMatch(ok, Cmd:validate([], ValidOpts)). + +activate_consumer_merge_defaults(_Config) -> + Cmd = ?COMMAND_ACTIVATE_STREAM_CONSUMER, + Opts = #{vhost => <<"/">>, + stream => <<"s1">>, + reference => <<"foo">>}, + ?assertEqual({[], Opts}, + Cmd:merge_defaults([], maps:without([vhost], Opts))), + Merged = maps:merge(Opts, #{vhost => "vhost"}), + ?assertEqual({[], Merged}, + Cmd:merge_defaults([], Merged)). + +activate_consumer_run(Config) -> + Cmd = ?COMMAND_ACTIVATE_STREAM_CONSUMER, + Node = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + Opts =#{node => Node, + timeout => 10000, + vhost => <<"/">>}, + Args = [], + + St = atom_to_binary(?FUNCTION_NAME, utf8), + ConsumerReference = <<"foo">>, + OptsGroup = maps:merge(#{stream => St, reference => ConsumerReference}, + Opts), + + %% the group does not exist yet + ?assertEqual({error, not_found}, Cmd:run(Args, OptsGroup)), + + StreamPort = rabbit_stream_SUITE:get_stream_port(Config), + {S, C} = start_stream_connection(StreamPort), + ?awaitMatch(1, connection_count(Config), ?WAIT), + + SubProperties =#{<<"single-active-consumer">> => <<"true">>, + <<"name">> => ConsumerReference}, + + create_stream(S, St, C), + subscribe(S, 0, St, SubProperties, C), + handle_consumer_update(S, C, 0), + subscribe(S, 1, St, SubProperties, C), + subscribe(S, 2, St, SubProperties, C), + + ?awaitMatch(3, consumer_count(Config), ?WAIT), + + ?assertEqual(ok, Cmd:run(Args, OptsGroup)), + + delete_stream(S, St, C), + close(S, C), + ok. + handle_consumer_update(S, C0, SubId) -> {{request, CorrId, {consumer_update, SubId, true}}, C1} = rabbit_stream_SUITE:receive_commands(gen_tcp, S, C0),