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..00b7fb5dde3e 100644 --- a/deps/rabbit/src/rabbit_stream_sac_coordinator.erl +++ b/deps/rabbit/src/rabbit_stream_sac_coordinator.erl @@ -18,11 +18,17 @@ -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{}. +-type sac_error() :: partition_index_conflict | not_found. + -export_type([state/0, command/0]). @@ -31,18 +37,53 @@ 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, + is_sac_error/1 + ]). +-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(), @@ -51,7 +92,7 @@ pid(), binary(), integer()) -> - {ok, boolean()} | {error, term()}. + {ok, boolean()} | {error, sac_error() | term()}. register_consumer(VirtualHost, Stream, PartitionIndex, @@ -59,65 +100,45 @@ 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(), binary(), pid(), integer()) -> - ok | {error, term()}. + ok | {error, sac_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(#command_unregister_consumer{vhost = VirtualHost, + stream = Stream, + consumer_name = ConsumerName, + connection_pid = ConnectionPid, + subscription_id = SubscriptionId}). + +-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 | {error, sac_error() | term()}. +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,10 +147,15 @@ 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()}}. + [term()]} | {error, sac_error() | term()}. consumer_groups(VirtualHost, InfoKeys) -> case ra_local_query(fun(State) -> SacState = @@ -148,9 +174,10 @@ 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()}. + {error, sac_error() | term()}. group_consumers(VirtualHost, Stream, Reference, InfoKeys) -> case ra_local_query(fun(State) -> SacState = @@ -171,7 +198,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 +213,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 +260,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 +285,309 @@ 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), + 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} = + 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 +595,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 +648,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 +693,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 +823,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 +848,166 @@ 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(_, _) -> + []. + +-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} + 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 +1016,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 +1049,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 +1094,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 +1107,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 +1115,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 +1131,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). @@ -652,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) -> @@ -675,29 +1185,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 +1261,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 +1282,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 +1313,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..800ddb656ab6 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,1213 @@ 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. + +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(), + 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 +1712,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 +1793,86 @@ 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, SubId, Effects) -> + assertContainsSendMessageEffect(Pid, SubId, stream(), name(), + true, 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/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/src/rabbit_stream_reader.erl b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl index f2f054bdd1e3..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]). @@ -106,7 +107,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() -> @@ -720,6 +722,9 @@ open(info, {OK, S, Data}, StatemData#statem_data{connection = Connection1, connection_state = State2}} end; +open(info, {sac, check_connection, _}, State) -> + _ = sac_connection_reconnected(self()), + {keep_state, State}; open(info, {sac, #{subscription_id := SubId, active := Active} = Msg}, @@ -790,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, @@ -2550,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 @@ -3011,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{ @@ -3171,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]), @@ -3186,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) -> @@ -3277,89 +3263,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, @@ -3381,6 +3297,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; @@ -3398,8 +3406,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 @@ -3408,6 +3415,7 @@ lookup_leader(Stream, {ok, LeaderPid} -> Connection1 = maybe_monitor_stream(LeaderPid, Stream, Connection), + {LeaderPid, Connection1#stream_connection{stream_leaders = StreamLeaders#{Stream => @@ -4011,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. diff --git a/deps/rabbitmq_stream/test/commands_SUITE.erl b/deps/rabbitmq_stream/test/commands_SUITE.erl index 711500518b3d..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]}, @@ -378,7 +385,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 +394,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 +406,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 +418,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 +497,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 +517,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), @@ -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), 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). 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). +