diff --git a/deps/rabbit/Makefile b/deps/rabbit/Makefile index c57975f0cce9..f75f96bb0879 100644 --- a/deps/rabbit/Makefile +++ b/deps/rabbit/Makefile @@ -268,7 +268,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_prop 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_prop 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..c4b40e7b2c4b 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, @@ -520,13 +520,13 @@ 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()}. + #?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,11 +564,12 @@ 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); + monitors = Monitors1}, Reply, Effects1); apply(#{machine_version := MachineVersion} = Meta, {down, Pid, Reason} = Cmd, #?MODULE{streams = Streams0, monitors = Monitors0, @@ -628,10 +629,19 @@ apply(#{machine_version := MachineVersion} = Meta, {down, Pid, Reason} = Cmd, return(Meta, State#?MODULE{streams = Streams0, monitors = Monitors1}, ok, Effects0) end; - {sac, Monitors1} -> - {SacState1, Effects} = rabbit_stream_sac_coordinator:handle_connection_down(Pid, SacState0), + {sac, Monitors1} when MachineVersion < 5 orelse Reason =/= noconnection -> + %% A connection went down, v5+ treats noconnection differently but + %% v4- does not. + Mod = sac_module(Meta), + {SacState1, Effects} = Mod:handle_connection_down(Pid, SacState0), return(Meta, State#?MODULE{single_active_consumer = SacState1, - monitors = Monitors1}, ok, Effects); + monitors = Monitors1}, ok, [Effects0 ++ Effects]); + {sac, Monitors1} when Reason =:= noconnection -> + %% the node of a connection got disconnected + Mod = sac_module(Meta), + {SacState1, Effects} = Mod:handle_connection_node_disconnected(Pid, SacState0), + return(Meta, State#?MODULE{single_active_consumer = SacState1, + monitors = Monitors1}, ok, [Effects0 ++ Effects]); error -> return(Meta, State, ok, Effects0) end; @@ -685,9 +695,11 @@ apply(#{machine_version := MachineVersion} = Meta, _ -> return(Meta, State0, stream_not_found, []) end; -apply(Meta, {nodeup, Node} = Cmd, +apply(#{machine_version := MachineVersion} = 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 +713,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 MachineVersion > 4 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 +741,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:forget_connection(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]), @@ -747,6 +775,11 @@ state_enter(leader, #?MODULE{streams = Streams, state_enter(_S, _) -> []. +sac_module(#{machine_version := MachineVersion}) when MachineVersion =< 4 -> + rabbit_stream_sac_coordinator_v4; +sac_module(_) -> + rabbit_stream_sac_coordinator. + all_member_nodes(Streams) -> maps:keys( maps:fold( @@ -780,7 +813,7 @@ members() -> end end. -maybe_resize_coordinator_cluster() -> +maybe_resize_coordinator_cluster(LeaderPid, MachineVersion) -> spawn(fun() -> RabbitIsRunning = rabbit:is_running(), case members() of @@ -803,22 +836,44 @@ maybe_resize_coordinator_cluster() -> [?MODULE, New]), add_member(Members, New) end, - case MemberNodes -- RabbitNodes of - [] -> - ok; - [Old | _] -> - %% 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, " - "deleting: ~w", [?MODULE, Old]), - remove_member(Leader, Members, Old) - end; + _ = case MemberNodes -- RabbitNodes of + [] -> + ok; + [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, " + "deleting: ~w", [?MODULE, Old]), + remove_member(Leader, Members, Old) + end, + maybe_handle_stale_nodes(MemberNodes, RabbitNodes, + LeaderPid, + MachineVersion); _ -> ok end end). +maybe_handle_stale_nodes(MemberNodes, ExpectedNodes, + LeaderPid, MachineVersion) when MachineVersion > 4 -> + case MemberNodes -- ExpectedNodes of + [] -> + ok; + Stale when length(ExpectedNodes) > 0 -> + rabbit_log:debug("Stale nodes detected in stream SAC " + "coordinator: ~w. Purging state.", + [Stale]), + Mod = sac_module(MachineVersion), + ra:pipeline_command(LeaderPid, Mod:make_purge_nodes(Stale)), + ok; + _ -> + ok + end; +maybe_handle_stale_nodes(_, _, _, _) -> + ok. + add_member(Members, Node) -> MinMacVersion = erpc:call(Node, ?MODULE, version, []), Conf = make_ra_conf(Node, [N || {_, N} <- Members], MinMacVersion), @@ -892,65 +947,63 @@ 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), + Pid = maybe_resize_coordinator_cluster(Leader, 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 +1011,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 +1030,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 +1072,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 +1080,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}}]. @@ -2214,6 +2268,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]), 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..68a484269ad7 100644 --- a/deps/rabbit/src/rabbit_stream_sac_coordinator.erl +++ b/deps/rabbit/src/rabbit_stream_sac_coordinator.erl @@ -18,9 +18,12 @@ -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{}. + -opaque state() :: #?MODULE{}. -export_type([state/0, @@ -31,18 +34,42 @@ 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_node_disconnected/2, + handle_node_reconnected/3, + forget_connection/2, consumer_groups/3, group_consumers/5, - overview/1]). + overview/1, + import_state/2]). +-export([make_purge_nodes/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(FORGOTTTEN, forgotten). + +-define(CONN_ACT, {?CONNECTED, ?ACTIVE}). +-define(CONN_WAIT, {?CONNECTED, ?WAITING}). +-define(DISCONN_ACT, {?DISCONNECTED, ?ACTIVE}). +-define(FORG_ACT, {?FORGOTTTEN, ?ACTIVE}). + %% Single Active Consumer API -spec register_consumer(binary(), binary(), @@ -59,25 +86,13 @@ register_consumer(VirtualHost, ConnectionPid, Owner, SubscriptionId) -> - process_command({sac, - #command_register_consumer{vhost = - VirtualHost, - stream = - Stream, - partition_index - = - PartitionIndex, - consumer_name - = - ConsumerName, - connection_pid - = - ConnectionPid, - owner = - Owner, - subscription_id - = - SubscriptionId}}). + process_command(#command_register_consumer{vhost = VirtualHost, + stream = Stream, + partition_index = PartitionIndex, + consumer_name = ConsumerName, + connection_pid = ConnectionPid, + owner = Owner, + subscription_id = SubscriptionId}). -spec unregister_consumer(binary(), binary(), @@ -90,34 +105,24 @@ unregister_consumer(VirtualHost, ConsumerName, ConnectionPid, SubscriptionId) -> - process_command({sac, - #command_unregister_consumer{vhost = - VirtualHost, - stream = - Stream, - consumer_name - = - ConsumerName, - connection_pid - = - ConnectionPid, - subscription_id - = - SubscriptionId}}). + process_command(#command_unregister_consumer{vhost = VirtualHost, + stream = Stream, + consumer_name = ConsumerName, + connection_pid = ConnectionPid, + subscription_id = SubscriptionId}). -spec activate_consumer(binary(), binary(), binary()) -> ok. -activate_consumer(VirtualHost, Stream, ConsumerName) -> - process_command({sac, - #command_activate_consumer{vhost = - VirtualHost, - stream = - Stream, - consumer_name - = - ConsumerName}}). +activate_consumer(VH, Stream, Name) -> + process_command(#command_activate_consumer{vhost =VH, + stream = Stream, + consumer_name= Name}). + +-spec connection_reconnected(connection_pid()) -> ok. +connection_reconnected(Pid) -> + process_command(#command_connection_reconnected{pid = Pid}). process_command(Cmd) -> - case rabbit_stream_coordinator:process_command(Cmd) of + case rabbit_stream_coordinator:process_command(wrap_cmd(Cmd)) of {ok, Res, _} -> Res; {error, _} = Err -> @@ -126,6 +131,10 @@ 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 -spec consumer_groups(binary(), [atom()]) -> {ok, @@ -231,7 +240,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,15 +265,263 @@ apply(#command_activate_consumer{vhost = VirtualHost, "the group does not longer exist", [{VirtualHost, Stream, ConsumerName}]), {undefined, []}; - Group -> - #consumer{pid = Pid, subscription_id = SubId} = - evaluate_active_consumer(Group), - Group1 = update_consumer_state_in_group(Group, Pid, SubId, true), - {Group1, [notify_consumer_effect(Pid, SubId, Stream, ConsumerName, true)]} + G0 -> + %% keep track of the former active, if any + {ActPid, ActSubId} = + case lookup_active_consumer(G0) of + {value, #consumer{pid = ActivePid, + subscription_id = ActiveSubId}} -> + {ActivePid, ActiveSubId}; + _ -> + {-1, -1} + end, + G1 = update_connected_consumers(G0, ?CONN_WAIT), + case evaluate_active_consumer(G1) of + undefined -> + {G1, []}; + #consumer{status = {?DISCONNECTED, _}} -> + %% we keep it this way, the consumer may come back + {G1, []}; + #consumer{pid = Pid, subscription_id = SubId} -> + G2 = update_consumer_state_in_group(G1, Pid, + SubId, + ?CONN_ACT), + %% do we need effects or not? + Effects = + case {Pid, SubId} of + {ActPid, ActSubId} -> + %% it is the same active consumer as before + %% no need to notify it + []; + _ -> + %% new active consumer, need to notify it + [notify_consumer_effect(Pid, SubId, Stream, + ConsumerName, true)] + end, + {G2, Effects} + end end, - StreamGroups1 = - update_groups(VirtualHost, Stream, ConsumerName, G, StreamGroups0), - {State0#?MODULE{groups = StreamGroups1}, ok, Eff}. + StreamGroups1 = update_groups(VirtualHost, Stream, ConsumerName, + G, StreamGroups0), + {State0#?MODULE{groups = StreamGroups1}, ok, Eff}; +apply(#command_connection_reconnected{pid = Pid}, + #?MODULE{groups = Groups0} = State0) -> + {State1, Eff} = + maps:fold(fun(G, _, {St, Eff}) -> + handle_group_connection_reconnected(Pid, St, Eff, G) + end, {State0, []}, Groups0), + + {State1, ok, Eff}; +apply(#command_purge_nodes{nodes = Nodes}, State0) -> + {State1, Eff} = lists:foldl(fun(N, {S0, Eff0}) -> + {S1, Eff1} = purge_node(N, S0), + {S1, Eff1 ++ Eff0} + end, {State0, []}, Nodes), + {State1, ok, Eff}. + +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 -> + {[C#consumer{status = {?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 =:= ?FORG_ACT -> + {[C#consumer{status = ?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 =:= ?FORG_ACT -> + %% update forgotten active + %% tell it to step down + {[C#consumer{status = ?CONN_WAIT} | Cs], + [notify_consumer_effect(P, SID, S, + Name, false, true) | Eff]}; + (#consumer{status = {?FORGOTTTEN, _}, + pid = P} = C, {Cs, Eff}) + when P =:= Pid -> + %% update forgotten + {[C#consumer{status = ?CONN_WAIT} | Cs], Eff}; + (#consumer{status = ?CONN_ACT, + pid = P, + subscription_id = SID} = C, {Cs, Eff}) -> + %% update connected active + %% tell it to step down + {[C#consumer{status = ?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 = ?FORG_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. -spec consumer_groups(binary(), [atom()], state()) -> {ok, [term()]}. consumer_groups(VirtualHost, InfoKeys, #?MODULE{groups = Groups}) -> @@ -313,7 +572,7 @@ group_consumers(VirtualHost, #{GroupId := #group{consumers = Consumers}} -> Cs = lists:foldr(fun(#consumer{subscription_id = SubId, owner = Owner, - active = Active}, + status = Status}, Acc) -> Record = lists:foldr(fun (subscription_id, RecAcc) -> @@ -324,12 +583,8 @@ group_consumers(VirtualHost, [{connection_name, Owner} | RecAcc]; - (state, RecAcc) - when Active -> - [{state, active} - | RecAcc]; (state, RecAcc) -> - [{state, inactive} + [{state, cli_consumer_status_label(Status)} | RecAcc]; (Unknown, RecAcc) -> [{Unknown, @@ -345,6 +600,13 @@ group_consumers(VirtualHost, {error, not_found} end. +cli_consumer_status_label({?FORGOTTTEN, _}) -> + inactive; +cli_consumer_status_label({_, ?ACTIVE}) -> + active; +cli_consumer_status_label(_) -> + inactive. + -spec ensure_monitors(command(), state(), map(), @@ -404,6 +666,29 @@ ensure_monitors(#command_unregister_consumer{vhost = VirtualHost, maps:put(Pid, PidGroup1, PidsGroups0)}, 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}. @@ -416,14 +701,88 @@ handle_connection_down(Pid, {State0, []}; {Groups, PidsGroups1} -> State1 = State0#?MODULE{pids_groups = PidsGroups1}, - maps:fold(fun(G, _, Acc) -> - handle_group_after_connection_down(Pid, Acc, G) - end, {State1, []}, Groups) + 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_node_disconnected( + ConnPid, Acc, G) + end, State1, Groups), + %% TODO configure timeout to forget connection from disconnected node + T = 60_000, + {State2, [{timer, {sac, node_disconnected, + #{connection_pid => 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 forget_connection(connection_pid(), state()) -> + {state(), ra_machine:effects()}. +forget_connection(Pid, #?MODULE{groups = Groups} = State0) -> + {State1, Eff} = + maps:fold(fun(G, _, {St, Eff}) -> + handle_group_forget_connection(Pid, St, Eff, G) + end, {State0, []}, Groups), + {State1, Eff}. + +handle_group_forget_connection(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 = {_, St}} = C, {L, _}) + when P == Pid -> + {[C#consumer{status = {?FORGOTTTEN, 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 +793,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 +818,82 @@ 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 = C0#consumer{status = {?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)}. + +- spec make_purge_nodes([node()]) -> {sac, command()}. +make_purge_nodes(Nodes) -> + wrap_cmd(#command_purge_nodes{nodes = Nodes}). + +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}) -> + #consumer{pid = Pid, + subscription_id = SubId, + owner = Owner, + status = active_to_status(Active)}. + +active_to_status(true) -> + {?CONNECTED, ?ACTIVE}; +active_to_status(false) -> + {?CONNECTED, ?WAITING}. + +is_active({?FORGOTTTEN, _}) -> + false; +is_active({_, ?ACTIVE}) -> + true; +is_active({_, ?DEACTIVATING}) -> + true; +is_active(_) -> + false. + do_register_consumer(VirtualHost, Stream, -1 = _PartitionIndex, @@ -464,8 +902,7 @@ 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 @@ -473,32 +910,29 @@ do_register_consumer(VirtualHost, #consumer{pid = ConnectionPid, owner = Owner, subscription_id = SubscriptionId, - active = false}; + status = {?CONNECTED, ?WAITING}}; false -> #consumer{pid = ConnectionPid, subscription_id = SubscriptionId, owner = Owner, - active = true} + status = {?CONNECTED, ?ACTIVE}} 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,8 +941,7 @@ 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 @@ -518,56 +951,25 @@ do_register_consumer(VirtualHost, #consumer{pid = ConnectionPid, owner = Owner, subscription_id = SubscriptionId, - active = true}, + status = {?CONNECTED, ?ACTIVE}}, 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 = #consumer{pid = ConnectionPid, + owner = Owner, + subscription_id = SubscriptionId, + status = {?CONNECTED, ?WAITING}}, 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 +993,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 +1006,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 +1014,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 +1030,9 @@ handle_consumer_removal(Group0, Stream, ConsumerName, ActiveRemoved) -> end end. +notify_connection_effect(Pid) -> + mod_call_effect(Pid, {sac, check_connection, #{}}). + notify_consumer_effect(Pid, SubId, Stream, Name, Active) -> notify_consumer_effect(Pid, SubId, Stream, Name, Active, false). @@ -675,29 +1085,75 @@ has_consumers_from_pid(#group{consumers = Consumers}, Pid) -> end, Consumers). -compute_active_consumer(#group{consumers = Crs, - partition_index = -1} = - Group) +compute_active_consumer(#group{partition_index = -1, + consumers = Crs} = 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}. + 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 -> + ok; + #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 + [C#consumer{status = ?CONN_ACT} | L]; + (#consumer{status = {?CONNECTED, _}} = C, L) -> + %% other connected consumers are set to "waiting" + [C#consumer{status = ?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. -evaluate_active_consumer(#group{partition_index = PartitionIndex, +do_evaluate_active_consumer(#group{partition_index = -1, 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 +1162,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, @@ -732,17 +1188,25 @@ update_groups(VirtualHost, 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}; + C0#consumer{status = 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) -> + C#consumer{status = 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 +1214,33 @@ 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). + diff --git a/deps/rabbit/src/rabbit_stream_sac_coordinator.hrl b/deps/rabbit/src/rabbit_stream_sac_coordinator.hrl index 7e1e7bf9c71d..6afeece42cfa 100644 --- a/deps/rabbit/src/rabbit_stream_sac_coordinator.hrl +++ b/deps/rabbit/src/rabbit_stream_sac_coordinator.hrl @@ -22,22 +22,28 @@ -type subscription_id() :: byte(). -type group_id() :: {vhost(), stream(), consumer_name()}. -type owner() :: binary(). +-type consumer_status() :: active | waiting | deactivating. +-type consumer_connectivity() :: connected | disconnected | forgotten. -record(consumer, {pid :: pid(), subscription_id :: subscription_id(), owner :: owner(), %% just a label - active :: boolean()}). + status :: {consumer_connectivity(), consumer_status()}}). -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(), %% future extensibility reserved_1, reserved_2}). + +-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 +62,7 @@ -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()]}). 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..380968d8f4ef --- /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_sac_coordinator_SUITE.erl b/deps/rabbit/test/rabbit_stream_sac_coordinator_SUITE.erl index 0a54ce4f05f6..21885a9d8291 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}]. @@ -69,62 +70,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), + ?assertEqual([consumer(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)], + ?assertEqual([consumer(ConnectionPid, 0, active), + consumer(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)], + ?assertEqual([consumer(ConnectionPid, 0, active), + consumer(ConnectionPid, 1, waiting), + consumer(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)], + State4, + ok, Effects4} = ?MOD:apply(Command3, State3), + ?assertEqual([consumer(ConnectionPid, 1, active), + consumer(ConnectionPid, 2, waiting)], Consumers4), - assertSendMessageEffect(ConnectionPid, 1, Stream, ConsumerName, true, Effects4), + 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), + ?assertEqual([consumer(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 +134,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), + ?assertEqual([consumer(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)], + ?assertEqual([consumer(ConnectionPid, 0, deactivating), + consumer(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)], + ?assertEqual([consumer(ConnectionPid, 0, waiting), + consumer(ConnectionPid, 1, active)], Consumers3), - assertSendMessageEffect(ConnectionPid, 1, Stream, ConsumerName, true, Effects3), + 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)], + ?assertEqual([consumer(ConnectionPid, 0, waiting), + consumer(ConnectionPid, 1, active), + consumer(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)], + ?assertEqual([consumer(ConnectionPid, 1, deactivating), + consumer(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)], + ?assertEqual([consumer(ConnectionPid, 1, waiting), + consumer(ConnectionPid, 2, active)], Consumers6), - assertSendMessageEffect(ConnectionPid, 2, Stream, ConsumerName, true, Effects6), + 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), + ?assertEqual([consumer(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,31 +220,29 @@ 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 = 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} = - 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), @@ -266,17 +251,13 @@ ensure_monitors_test(_) -> Group2 = cgroup([consumer(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 +265,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 +296,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 = cgroup([consumer(Pid0, 0, active), + consumer(Pid1, 1, waiting), + consumer(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, 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, cgroup([consumer(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, State1), assertEmpty(PidsGroups2), assertEmpty(Effects2), assertEmpty(Groups2), @@ -346,21 +321,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 = cgroup([consumer(Pid1, 0, active), + consumer(Pid0, 1, waiting), + consumer(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, State), assertSize(1, PidsGroups), assertSize(1, maps:get(Pid1, PidsGroups)), assertEmpty(Effects), - assertHasGroup(GroupId, cgroup([consumer(Pid1, 0, true)]), Groups), + assertHasGroup(GroupId, cgroup([consumer(Pid1, 0, active)]), Groups), ok. handle_connection_down_sac_no_more_consumers_test(_) -> @@ -368,14 +340,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 = cgroup([consumer(Pid0, 0, active), + consumer(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, State), assertEmpty(PidsGroups), assertEmpty(Groups), assertEmpty(Effects), @@ -386,21 +356,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 = cgroup([consumer(Pid1, 0, active), + consumer(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, 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, + cgroup([consumer(Pid1, 0, active), consumer(Pid1, 1, waiting)]), Groups), ok. @@ -409,22 +379,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 = cgroup(1, [consumer(Pid0, 0, waiting), + consumer(Pid0, 1, active), + consumer(Pid1, 2, waiting), + consumer(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, 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, + cgroup(1, [consumer(Pid0, 0, waiting), + consumer(Pid0, 1, active)]), Groups), ok. @@ -433,22 +402,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 = cgroup(1, [consumer(Pid0, 0, waiting), + consumer(Pid1, 1, active), + consumer(Pid0, 2, waiting), + consumer(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, 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, + cgroup(1, [consumer(Pid1, 1, deactivating), + consumer(Pid1, 3, waiting)]), Groups), ok. @@ -457,22 +426,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 = cgroup(1, [consumer(Pid0, 0, waiting), + consumer(Pid0, 1, active), + consumer(Pid1, 2, waiting), + consumer(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, 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, cgroup(1, [consumer(Pid1, 2, waiting), + consumer(Pid1, 3, active)]), Groups), ok. @@ -481,25 +448,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 = cgroup(1, [consumer(Pid0, 0, waiting), + consumer(Pid0, 1, waiting), + consumer(Pid1, 2, waiting), + consumer(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, 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, cgroup(1, [consumer(Pid1, 2, waiting), + consumer(Pid1, 3, waiting)]), Groups), ok. @@ -517,6 +482,987 @@ 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 = cgroup(1, [consumer(Pid0, 0, waiting), + consumer(Pid1, 1, active), + consumer(Pid2, 2, waiting)]), + State0 = state(#{GroupId => Group}), + + {#?STATE{pids_groups = PidsGroups1, groups = Groups1} = State1, + Effects1} = + ?MOD:handle_connection_down(Pid0, State0), + assertSize(2, PidsGroups1), + assertSize(1, maps:get(Pid1, PidsGroups1)), + assertSize(1, maps:get(Pid2, PidsGroups1)), + assertSendMessageSteppingDownEffect(Pid1, 1, Stream, ConsumerName, Effects1), + assertHasGroup(GroupId, + cgroup(1, [consumer(Pid1, 1, deactivating), + consumer(Pid2, 2, waiting)]), + Groups1), + + {#?STATE{pids_groups = PidsGroups2, groups = Groups2}, + Effects2} = ?MOD:handle_connection_down(Pid1, State1), + assertSize(1, PidsGroups2), + assertSize(1, maps:get(Pid2, PidsGroups2)), + assertSendMessageActivateEffect(Pid2, 2, Stream, ConsumerName, true, Effects2), + assertHasGroup(GroupId, + cgroup(1, [consumer(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 = #{}}, + ?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}, + cgroup(-1, [consumer(Pid0, 0, active), + consumer(Pid1, 1, waiting), + consumer(Pid2, 2, waiting)]), + Groups), + + assertHasGroup({<<"/">>, P, App1}, + cgroup(1, [consumer(Pid0, 0, waiting), + consumer(Pid1, 1, active), + consumer(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 = cgroup(1, [consumer(Pid0, 0, waiting), + consumer(Pid1, 1, active), + consumer(Pid2, 2, waiting)]), + State0 = state(#{GroupId => Group}), + + {#?STATE{pids_groups = PidsGroups1, groups = Groups1} = _State1, + [Effect1]} = + ?MOD:handle_connection_node_disconnected(Pid1, 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, + cgroup(1, [consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 1, {disconnected, active}), + consumer(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 = cgroup(0, [consumer(N0Pid0, 0, {connected, active}), + consumer(N1Pid0, 1, {disconnected, waiting}), + consumer(N0Pid1, 2, {connected, waiting})]), + + Group1 = cgroup(1, [consumer(N0Pid0, 0, {connected, waiting}), + consumer(N1Pid0, 1, {disconnected, active}), + consumer(N0Pid1, 2, {connected, waiting})]), + + Group2 = cgroup(1, [consumer(N0Pid0, 0, {connected, waiting}), + consumer(N1Pid0, 1, {disconnected, waiting}), + consumer(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 = cgroup([consumer(Pid0, 0, {disconnected, active}), + consumer(Pid1, 1, {connected, waiting}), + consumer(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, cgroup([consumer(Pid0, 0, {connected, active}), + consumer(Pid1, 1, {connected, waiting}), + consumer(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 = cgroup([consumer(Pid0, 0, {disconnected, waiting}), + consumer(Pid1, 1, {connected, active}), + consumer(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, cgroup([consumer(Pid1, 1, {connected, active}), + consumer(Pid0, 0, {connected, waiting}), + consumer(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 = cgroup(1, [consumer(Pid0, 0, {disconnected, waiting}), + consumer(Pid1, 1, {connected, waiting}), + consumer(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, cgroup(1, [consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 1, {connected, waiting}), + consumer(Pid2, 2, {connected, deactivating})]), + Groups1), + + assertSendMessageSteppingDownEffect(Pid2, 2, stream(), name(), Eff), + ok. + +forget_connection_simple_disconnected_becomes_forgotten_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = cgroup([consumer(Pid0, 0, {disconnected, active}), + consumer(Pid1, 1, {connected, waiting}), + consumer(Pid2, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + + {#?STATE{groups = Groups1}, Eff} = ?MOD:forget_connection(Pid0, State0), + + assertHasGroup(GId, cgroup([consumer(Pid0, 0, {forgotten, active}), + consumer(Pid1, 1, {connected, active}), + consumer(Pid2, 2, {connected, waiting})]), + Groups1), + assertSendMessageActivateEffect(Pid1, 1, stream(), name(), true, Eff), + ok. + +forget_connection_super_stream_disconnected_becomes_forgotten_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = cgroup(1, [consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 1, {disconnected, active}), + consumer(Pid2, 2, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + + {#?STATE{groups = Groups1}, Eff} = ?MOD:forget_connection(Pid1, State0), + + assertHasGroup(GId, cgroup(1, [consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 1, {forgotten, active}), + consumer(Pid2, 2, {connected, active})]), + Groups1), + + assertSendMessageActivateEffect(Pid2, 2, stream(), name(), true, Eff), + ok. + +register_consumer_simple_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + GId = group_id(), + Group = cgroup([consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 1, {disconnected, active}), + consumer(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, cgroup([consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 1, {disconnected, active}), + consumer(Pid0, 2, {connected, waiting}), + consumer(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 = cgroup(1, [consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 1, {disconnected, active}), + consumer(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, cgroup(1, [consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 1, {disconnected, active}), + consumer(Pid0, 2, {connected, waiting}), + consumer(Pid0, 3, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +unregister_consumer_simple_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + GId = group_id(), + Group = cgroup([consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 1, {disconnected, active}), + consumer(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, cgroup([consumer(Pid0, 0, {connected, waiting}), + consumer(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 = cgroup(1, [consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 1, {disconnected, active}), + consumer(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, cgroup(1, [consumer(Pid1, 1, {disconnected, active}), + consumer(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 = cgroup([consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 1, {disconnected, active}), + consumer(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, cgroup([consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 1, {disconnected, active}), + consumer(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 = cgroup(1, [consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 1, {disconnected, active}), + consumer(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, cgroup(1, [consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 1, {disconnected, active}), + consumer(Pid0, 2, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +handle_connection_down_simple_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = cgroup([consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 0, {disconnected, active}), + consumer(Pid2, 0, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + {#?STATE{groups = Groups1}, Eff} = ?MOD:handle_connection_down(Pid2, State0), + assertHasGroup(GId, cgroup([consumer(Pid0, 0, {connected, waiting}), + consumer(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 = cgroup(1, [consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 0, {disconnected, active}), + consumer(Pid2, 0, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + {#?STATE{groups = Groups1}, Eff} = ?MOD:handle_connection_down(Pid0, State0), + assertHasGroup(GId, cgroup(1, [consumer(Pid1, 0, {disconnected, active}), + consumer(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 = cgroup([consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 0, {disconnected, active}), + consumer(Pid2, 0, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + {#?STATE{groups = Groups1}, Eff} = + ?MOD:handle_connection_node_disconnected(Pid2, State0), + assertHasGroup(GId, cgroup([consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 0, {disconnected, active}), + consumer(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 = cgroup(1, [consumer(Pid0, 0, {connected, waiting}), + consumer(Pid1, 0, {disconnected, active}), + consumer(Pid2, 0, {connected, waiting})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + {#?STATE{groups = Groups1}, Eff} = + ?MOD:handle_connection_node_disconnected(Pid0, State0), + assertHasGroup(GId, cgroup(1, [consumer(Pid0, 0, {disconnected, waiting}), + consumer(Pid1, 0, {disconnected, active}), + consumer(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 = cgroup([consumer(Pid0, 0, {disconnected, waiting}), + consumer(Pid1, 0, {disconnected, active}), + consumer(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, cgroup([consumer(Pid1, 0, {disconnected, active}), + consumer(Pid0, 0, {connected, waiting}), + consumer(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 = cgroup([consumer(P0, 0, {forgotten, active}), + consumer(P1, 0, {disconnected, active}), + consumer(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, cgroup([consumer(P0, 0, {connected, waiting}), + consumer(P1, 0, {disconnected, active}), + consumer(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 = cgroup([consumer(P0, {forgotten, active}), + consumer(P1, {connected, active}), + consumer(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, cgroup([consumer(P0, {connected, waiting}), + consumer(P1, {connected, waiting}), + consumer(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, cgroup([consumer(P0, {connected, active}), + consumer(P1, {connected, waiting}), + consumer(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, cgroup([consumer(P0, {connected, active}), + consumer(P1, {connected, waiting}), + consumer(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 = cgroup(1, [consumer(Pid0, 0, {disconnected, active}), + consumer(Pid1, 0, {disconnected, waiting}), + consumer(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, cgroup(1, [consumer(Pid0, 0, {disconnected, active}), + consumer(Pid1, 0, {connected, waiting}), + consumer(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 = cgroup(1, [consumer(P0, {forgotten, active}), + consumer(P1, {disconnected, active}), + consumer(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, cgroup(1, [consumer(P0, {connected, waiting}), + consumer(P1, {disconnected, active}), + consumer(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 = cgroup(1, [consumer(P0, {forgotten, active}), + consumer(P1, {connected, waiting}), + consumer(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, cgroup(1, [consumer(P0, {connected, waiting}), + consumer(P1, {connected, waiting}), + consumer(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, cgroup(1, [consumer(P0, {connected, waiting}), + consumer(P1, {connected, active}), + consumer(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, cgroup(1, [consumer(P0, {connected, waiting}), + consumer(P1, {connected, active}), + consumer(P2, {connected, waiting})]), + Groups3), + assertEmpty(Eff3), + + ok. + +forget_connection_simple_disconn_active_blocks_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = cgroup([consumer(Pid0, {disconnected, waiting}), + consumer(Pid1, {connected, waiting}), + consumer(Pid2, {disconnected, active})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + + {#?STATE{groups = Groups1}, Eff} = ?MOD:forget_connection(Pid0, State0), + + assertHasGroup(GId, cgroup([consumer(Pid2, {disconnected, active}), + consumer(Pid0, {forgotten, waiting}), + consumer(Pid1, {connected, waiting})]), + Groups1), + assertEmpty(Eff), + ok. + +forget_connection_super_stream_disconn_active_block_rebalancing_test(_) -> + Pid0 = new_process(), + Pid1 = new_process(), + Pid2 = new_process(), + GId = group_id(), + Group = cgroup(1, [consumer(Pid0, {disconnected, waiting}), + consumer(Pid1, {connected, waiting}), + consumer(Pid2, {disconnected, active})]), + + Groups0 = #{GId => Group}, + State0 = state(Groups0), + + {#?STATE{groups = Groups1}, Eff} = ?MOD:forget_connection(Pid0, State0), + + assertHasGroup(GId, cgroup(1, [consumer(Pid0, {forgotten, waiting}), + consumer(Pid1, {connected, waiting}), + consumer(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 = cgroup([consumer(N1P0, {disconnected, active}), + consumer(N0P1, {connected, waiting}), + consumer(N0P2, {connected, waiting})]), + + Group1 = cgroup(1, [consumer(N1P1, {disconnected, waiting}), + consumer(N1P2, {disconnected, active}), + consumer(N0P0, {connected, waiting})]), + + Group2 = cgroup([consumer(N0P0, {connected, active}), + consumer(N0P1, {connected, waiting}), + consumer(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, cgroup([consumer(N0P1, {connected, active}), + consumer(N0P2, {connected, waiting})]), + Groups1), + assertHasGroup(GId1, cgroup(1, [consumer(N0P0, {connected, active})]), + Groups1), + assertHasGroup(GId2, cgroup([consumer(N0P0, {connected, active}), + consumer(N0P1, {connected, waiting}), + consumer(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 = cgroup([consumer(N0P0, {connected, active}), + consumer(N1P0, {connected, waiting}), + consumer(N0P1, {connected, waiting})]), + + G1 = cgroup(1, [consumer(N1P1, {connected, waiting}), + consumer(N0P2, {connected, active}), + consumer(N1P2, {connected, waiting})]), + + G2 = cgroup([consumer(N0P0, {connected, active}), + consumer(N1P1, {connected, waiting}), + consumer(N0P2, {connected, waiting})]), + + State0 = state(#{GId0 => G0, GId1 => G1, GId2 => G2}), + + {State1, Eff1} = ?MOD:handle_connection_node_disconnected(N1P0, State0), + {State2, Eff2} = ?MOD:handle_connection_node_disconnected(N1P1, State1), + {State3, Eff3} = ?MOD:handle_connection_node_disconnected(N1P2, State2), + + assertNodeDisconnectedTimerEffect(N1P0, Eff1), + assertNodeDisconnectedTimerEffect(N1P1, Eff2), + assertNodeDisconnectedTimerEffect(N1P2, Eff3), + + assertHasGroup(GId0, + cgroup([consumer(N0P0, {connected, active}), + consumer(N1P0, {disconnected, waiting}), + consumer(N0P1, {connected, waiting})]), + State3#?STATE.groups), + + assertHasGroup(GId1, + cgroup(1, [consumer(N1P1, {disconnected, waiting}), + consumer(N0P2, {connected, active}), + consumer(N1P2, {disconnected, waiting})]), + State3#?STATE.groups), + + assertHasGroup(GId2, + cgroup([consumer(N0P0, {connected, active}), + consumer(N1P1, {disconnected, waiting}), + consumer(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, + cgroup([consumer(N0P0, {connected, active}), + consumer(N1P0, {connected, waiting}), + consumer(N0P1, {connected, waiting})]), + Groups5), + + assertHasGroup(GId1, + cgroup(1, [consumer(N1P1, {disconnected, waiting}), + consumer(N0P2, {connected, active}), + consumer(N1P2, {disconnected, waiting})]), + Groups5), + + assertHasGroup(GId2, + cgroup([consumer(N0P0, {connected, active}), + consumer(N1P1, {disconnected, waiting}), + consumer(N0P2, {connected, waiting})]), + Groups5), + + assertEmpty(Eff5), + + Cmd5 = connection_reconnected_command(N1P1), + {#?STATE{groups = Groups6} = State6, ok, Eff6} = ?MOD:apply(Cmd5, State5), + + assertHasGroup(GId0, + cgroup([consumer(N0P0, {connected, active}), + consumer(N1P0, {connected, waiting}), + consumer(N0P1, {connected, waiting})]), + Groups6), + + assertHasGroup(GId1, + cgroup(1, [consumer(N1P1, {connected, waiting}), + consumer(N0P2, {connected, active}), + consumer(N1P2, {disconnected, waiting})]), + Groups6), + + assertHasGroup(GId2, + cgroup([consumer(N0P0, {connected, active}), + consumer(N1P1, {connected, waiting}), + consumer(N0P2, {connected, waiting})]), + Groups6), + + assertEmpty(Eff6), + + %% last connection does not come back for some reason + {#?STATE{groups = Groups7}, Eff7} = ?MOD:forget_connection(N1P2, State6), + + assertHasGroup(GId0, + cgroup([consumer(N0P0, {connected, active}), + consumer(N1P0, {connected, waiting}), + consumer(N0P1, {connected, waiting})]), + Groups7), + + assertHasGroup(GId1, + cgroup(1, [consumer(N1P1, {connected, waiting}), + consumer(N0P2, {connected, active}), + consumer(N1P2, {forgotten, waiting})]), + Groups7), + + assertHasGroup(GId2, + cgroup([consumer(N0P0, {connected, active}), + consumer(N1P1, {connected, waiting}), + consumer(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 = cgroup(1, [consumer(P0, {connected, waiting}), + consumer(P1, {connected, active}), + consumer(P2, {connected, waiting})]), + S0 = state(#{GId => G0}), + + {#?STATE{groups = G1} = S1, Eff1} = + ?MOD:handle_connection_node_disconnected(P1, S0), + + assertHasGroup(GId, + cgroup(1, [consumer(P0, {connected, waiting}), + consumer(P1, {disconnected, active}), + consumer(P2, {connected, waiting})]), + G1), + + assertNodeDisconnectedTimerEffect(P1, Eff1), + + {#?STATE{groups = G2} = S2, Eff2} = + ?MOD:handle_node_reconnected(N1, S1, []), + + assertHasGroup(GId, + cgroup(1, [consumer(P0, {connected, waiting}), + consumer(P1, {disconnected, active}), + consumer(P2, {connected, waiting})]), + G2), + + assertContainsCheckConnectionEffect(P1, Eff2), + + {#?STATE{groups = G3}, Eff3} = ?MOD:handle_connection_down(P1, S2), + + assertHasGroup(GId, + cgroup(1, [consumer(P0, {connected, waiting}), + consumer(P2, {connected, active})]), + G3), + + assertContainsSendMessageEffect(P2, stream(), true, Eff3), + + stop_node(N1Pid), + stop_node(N2Pid), + ok. + +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) -> @@ -528,13 +1474,19 @@ assertEmpty(Data) -> assertSize(0, Data). assertHasGroup(GroupId, Group, Groups) -> - ?assertEqual(#{GroupId => Group}, Groups). + G = maps:get(GroupId, Groups), + ?assertEqual(Group, G). -consumer(Pid, SubId, Active) -> +consumer(Pid, Status) -> + consumer(Pid, sub_id(), Status). + +consumer(Pid, SubId, {Connectivity, Status}) -> #consumer{pid = Pid, subscription_id = SubId, owner = <<"owning connection label">>, - active = Active}. + status = {Connectivity, Status}}; +consumer(Pid, SubId, Status) -> + consumer(Pid, SubId, {connected, Status}). cgroup(Consumers) -> cgroup(-1, Consumers). @@ -546,7 +1498,7 @@ state() -> state(#{}). state(Groups) -> - state(Groups, #{}). + state(Groups, ?MOD:compute_pid_group_dependencies(Groups)). state(Groups, PidsGroups) -> #?STATE{groups = Groups, pids_groups = PidsGroups}. @@ -579,28 +1531,82 @@ activate_consumer_command(Stream, ConsumerName) -> stream = Stream, consumer_name = ConsumerName}. -assertSendMessageEffect(Pid, SubId, Stream, ConsumerName, Active, [Effect]) -> +connection_reconnected_command(Pid) -> + #command_connection_reconnected{pid = Pid}. + +purge_nodes_command(Nodes) -> + #command_purge_nodes{nodes = Nodes}. + +assertContainsCheckConnectionEffect(Pid, Effects) -> + assertContainsSendMessageEffect(Pid, {sac, check_connection, #{}}, Effects). + +assertContainsSendMessageEffect(Pid, Stream, Active, Effects) -> + assertContainsSendMessageEffect(Pid, 0, Stream, name(), Active, Effects). + +assertContainsActivateMessage(Pid, Effects) -> + assertContainsSendMessageEffect(Pid, sub_id(), stream(), name(), + true, Effects). + +assertContainsSendMessageEffect(Pid, SubId, Stream, ConsumerName, Active, + Effects) -> + assertContainsSendMessageEffect(Pid, {sac, + #{subscription_id => SubId, + stream => Stream, + consumer_name => ConsumerName, + active => Active}}, + Effects). + +assertContainsSendMessageSteppingDownEffect(Pid, Effects) -> + assertContainsSendMessageSteppingDownEffect(Pid, sub_id(), stream(), + name(), Effects). + +assertContainsSendMessageSteppingDownEffect(Pid, SubId, Stream, ConsumerName, + Effects) -> + assertContainsSendMessageEffect(Pid, {sac, + #{subscription_id => SubId, + stream => Stream, + consumer_name => ConsumerName, + active => false, + stepping_down => true}}, Effects). + +assertContainsSendMessageEffect(Pid, Msg, Effects) -> + assertContainsEffect({mod_call, + rabbit_stream_sac_coordinator, + send_message, + [Pid, Msg]}, Effects). + +assertContainsMonitorProcessEffect(Pid, Effects) -> + assertContainsEffect({monitor, process, Pid}, Effects). + +assertContainsEffect(Effect, Effects) -> + Contains = lists:any(fun(Eff) -> Eff =:= Effect end, Effects), + ?assert(Contains, "List does not contain the expected effect"). + +assertSendMessageActivateEffect(Pid, SubId, Stream, ConsumerName, Active, Effects) -> + assertSendMessageEffect(Pid, {sac, + #{subscription_id => SubId, + stream => Stream, + consumer_name => ConsumerName, + active => Active} + }, Effects). + +assertSendMessageSteppingDownEffect(Pid, SubId, Stream, ConsumerName, Effects) -> + assertSendMessageEffect(Pid, {sac, + #{subscription_id => SubId, + stream => Stream, + consumer_name => ConsumerName, + active => false, + stepping_down => true}}, Effects). + +assertSendMessageEffect(Pid, Msg, [Effect]) -> ?assertEqual({mod_call, rabbit_stream_sac_coordinator, send_message, - [Pid, - {sac, - #{subscription_id => SubId, - stream => Stream, - consumer_name => ConsumerName, - active => Active} - }]}, + [Pid, Msg]}, Effect). -assertSendMessageSteppingDownEffect(Pid, SubId, Stream, ConsumerName, [Effect]) -> - ?assertEqual({mod_call, - rabbit_stream_sac_coordinator, - send_message, - [Pid, - {sac, - #{subscription_id => SubId, - stream => Stream, - consumer_name => ConsumerName, - active => false, - stepping_down => true}}]}, +assertNodeDisconnectedTimerEffect(Pid, [Effect]) -> + ?assertMatch({timer, + {sac, node_disconnected, #{connection_pid := Pid}}, + _}, Effect). diff --git a/deps/rabbit/test/rabbit_stream_sac_coordinator_v4_SUITE.erl b/deps/rabbit/test/rabbit_stream_sac_coordinator_v4_SUITE.erl new file mode 100644 index 000000000000..7426e8def751 --- /dev/null +++ b/deps/rabbit/test/rabbit_stream_sac_coordinator_v4_SUITE.erl @@ -0,0 +1,593 @@ +%% The contents of this file are subject to the Mozilla Public License +%% Version 2.0 (the "License"); you may not use this file except in +%% compliance with the License. You may obtain a copy of the License +%% at https://www.mozilla.org/en-US/MPL/2.0/ +%% +%% Software distributed under the License is distributed on an "AS IS" +%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See +%% the License for the specific language governing rights and +%% limitations under the License. +%% +%% The Original Code is RabbitMQ. +%% +%% The Initial Developer of the Original Code is Pivotal Software, Inc. +%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_stream_sac_coordinator_v4_SUITE). + +-compile(export_all). + +-include_lib("eunit/include/eunit.hrl"). +-include_lib("rabbit/src/rabbit_stream_sac_coordinator_v4.hrl"). + +%%%=================================================================== +%%% Common Test callbacks +%%%=================================================================== + +-define(STATE, rabbit_stream_sac_coordinator). +-define(MOD, rabbit_stream_sac_coordinator_v4). + +all() -> + [{group, tests}]. + +%% replicate eunit like test resolution +all_tests() -> + [F + || {F, _} <- ?MODULE:module_info(functions), + re:run(atom_to_list(F), "_test$") /= nomatch]. + +groups() -> + [{tests, [], all_tests()}]. + +init_per_suite(Config) -> + Config. + +end_per_suite(_Config) -> + ok. + +init_per_group(_Group, Config) -> + Config. + +end_per_group(_Group, _Config) -> + ok. + +init_per_testcase(_TestCase, Config) -> + ok = meck:new(rabbit_feature_flags), + meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> true end), + Config. + +end_per_testcase(_TestCase, _Config) -> + meck:unload(), + ok. + +simple_sac_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + ConnectionPid = self(), + GroupId = {<<"/">>, Stream, ConsumerName}, + Command0 = + register_consumer_command(Stream, -1, ConsumerName, ConnectionPid, 0), + State0 = state(), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers1}}} = + State1, + {ok, Active1}, Effects1} = + ?MOD:apply(Command0, State0), + ?assert(Active1), + ?assertEqual([consumer(ConnectionPid, 0, true)], Consumers1), + assertSendMessageEffect(ConnectionPid, 0, Stream, ConsumerName, true, Effects1), + + Command1 = + register_consumer_command(Stream, -1, ConsumerName, ConnectionPid, 1), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers2}}} = + State2, + {ok, Active2}, Effects2} = + ?MOD:apply(Command1, State1), + ?assertNot(Active2), + ?assertEqual([consumer(ConnectionPid, 0, true), + consumer(ConnectionPid, 1, false)], + Consumers2), + assertEmpty(Effects2), + + Command2 = + register_consumer_command(Stream, -1, ConsumerName, ConnectionPid, 2), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers3}}} = + State3, + {ok, Active3}, Effects3} = + ?MOD:apply(Command2, State2), + ?assertNot(Active3), + ?assertEqual([consumer(ConnectionPid, 0, true), + consumer(ConnectionPid, 1, false), + consumer(ConnectionPid, 2, false)], + Consumers3), + assertEmpty(Effects3), + + Command3 = + unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 0), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers4}}} = + State4, + ok, Effects4} = + ?MOD:apply(Command3, State3), + ?assertEqual([consumer(ConnectionPid, 1, true), + consumer(ConnectionPid, 2, false)], + Consumers4), + assertSendMessageEffect(ConnectionPid, 1, Stream, ConsumerName, true, Effects4), + + Command4 = + unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 1), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers5}}} = + State5, + ok, Effects5} = + ?MOD:apply(Command4, State4), + ?assertEqual([consumer(ConnectionPid, 2, true)], Consumers5), + assertSendMessageEffect(ConnectionPid, 2, Stream, ConsumerName, true, Effects5), + + Command5 = + unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 2), + {#?STATE{groups = Groups6}, ok, Effects6} = + ?MOD:apply(Command5, State5), + assertEmpty(Groups6), + assertEmpty(Effects6), + + ok. + +super_stream_partition_sac_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + ConnectionPid = self(), + GroupId = {<<"/">>, Stream, ConsumerName}, + Command0 = + register_consumer_command(Stream, 1, ConsumerName, ConnectionPid, 0), + State0 = state(), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers1}}} = + State1, + {ok, Active1}, Effects1} = + ?MOD:apply(Command0, State0), + ?assert(Active1), + ?assertEqual([consumer(ConnectionPid, 0, true)], Consumers1), + assertSendMessageEffect(ConnectionPid, 0, Stream, ConsumerName, true, Effects1), + + Command1 = + register_consumer_command(Stream, 1, ConsumerName, ConnectionPid, 1), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers2}}} = + State2, + {ok, Active2}, Effects2} = + ?MOD:apply(Command1, State1), + %% never active on registration + ?assertNot(Active2), + %% all consumers inactive, until the former active one steps down and activates the new consumer + ?assertEqual([consumer(ConnectionPid, 0, false), + consumer(ConnectionPid, 1, false)], + Consumers2), + assertSendMessageSteppingDownEffect(ConnectionPid, 0, Stream, ConsumerName, Effects2), + + Command2 = activate_consumer_command(Stream, ConsumerName), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers3}}} = + State3, + ok, Effects3} = + ?MOD:apply(Command2, State2), + + %% 1 (partition index) % 2 (consumer count) = 1 (active consumer index) + ?assertEqual([consumer(ConnectionPid, 0, false), + consumer(ConnectionPid, 1, true)], + Consumers3), + assertSendMessageEffect(ConnectionPid, 1, Stream, ConsumerName, true, Effects3), + + Command3 = + register_consumer_command(Stream, 1, ConsumerName, ConnectionPid, 2), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers4}}} = + State4, + {ok, Active4}, Effects4} = + ?MOD:apply(Command3, State3), + %% never active on registration + ?assertNot(Active4), + %% 1 (partition index) % 3 (consumer count) = 1 (active consumer index) + %% the active consumer stays the same + ?assertEqual([consumer(ConnectionPid, 0, false), + consumer(ConnectionPid, 1, true), + consumer(ConnectionPid, 2, false)], + Consumers4), + assertEmpty(Effects4), + + Command4 = + unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 0), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers5}}} = + State5, + ok, Effects5} = + ?MOD:apply(Command4, State4), + %% 1 (partition index) % 2 (consumer count) = 1 (active consumer index) + %% the active consumer will move from sub 1 to sub 2 + ?assertEqual([consumer(ConnectionPid, 1, false), + consumer(ConnectionPid, 2, false)], + Consumers5), + + assertSendMessageSteppingDownEffect(ConnectionPid, 1, Stream, ConsumerName, Effects5), + + Command5 = activate_consumer_command(Stream, ConsumerName), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers6}}} = + State6, + ok, Effects6} = + ?MOD:apply(Command5, State5), + + ?assertEqual([consumer(ConnectionPid, 1, false), + consumer(ConnectionPid, 2, true)], + Consumers6), + assertSendMessageEffect(ConnectionPid, 2, Stream, ConsumerName, true, Effects6), + + Command6 = + unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 1), + {#?STATE{groups = #{GroupId := #group{consumers = Consumers7}}} = + State7, + ok, Effects7} = + ?MOD:apply(Command6, State6), + ?assertEqual([consumer(ConnectionPid, 2, true)], Consumers7), + assertEmpty(Effects7), + + Command7 = + unregister_consumer_command(Stream, ConsumerName, ConnectionPid, 2), + {#?STATE{groups = Groups8}, ok, Effects8} = + ?MOD:apply(Command7, State7), + assertEmpty(Groups8), + assertEmpty(Effects8), + + ok. + +ensure_monitors_test(_) -> + GroupId = {<<"/">>, <<"stream">>, <<"app">>}, + Group = + cgroup([consumer(self(), 0, true), consumer(self(), 1, false)]), + State0 = state(#{GroupId => Group}), + Monitors0 = #{}, + Command0 = + register_consumer_command(<<"stream">>, -1, <<"app">>, self(), 0), + {#?STATE{pids_groups = PidsGroups1} = State1, Monitors1, Effects1} = + ?MOD:ensure_monitors(Command0, + State0, + Monitors0, + []), + assertSize(1, PidsGroups1), + assertSize(1, maps:get(self(), PidsGroups1)), + ?assertEqual(#{self() => sac}, Monitors1), + ?assertEqual([{monitor, process, self()}, {monitor, node, node()}], + Effects1), + + Command1 = + register_consumer_command(<<"stream">>, -1, <<"app">>, self(), 1), + + {#?STATE{pids_groups = PidsGroups2} = State2, Monitors2, Effects2} = + ?MOD:ensure_monitors(Command1, + State1, + Monitors1, + []), + assertSize(1, PidsGroups2), + assertSize(1, maps:get(self(), PidsGroups2)), + ?assertEqual(#{self() => sac}, Monitors2), + ?assertEqual([{monitor, process, self()}, {monitor, node, node()}], + Effects2), + + Group2 = cgroup([consumer(self(), 1, true)]), + + Command2 = + unregister_consumer_command(<<"stream">>, <<"app">>, self(), 0), + + {#?STATE{pids_groups = PidsGroups3} = State3, Monitors3, Effects3} = + ?MOD:ensure_monitors(Command2, + State2#?STATE{groups = + #{GroupId + => + Group2}}, + Monitors2, + []), + assertSize(1, PidsGroups3), + assertSize(1, maps:get(self(), PidsGroups3)), + ?assertEqual(#{self() => sac}, Monitors3), + ?assertEqual([], Effects3), + + %% trying with an unknown connection PID + %% the function should not change anything + UnknownConnectionPid = spawn(fun() -> ok end), + PassthroughCommand = + unregister_consumer_command(<<"stream">>, + <<"app">>, + UnknownConnectionPid, + 0), + + {State3, Monitors3, Effects3} = + ?MOD:ensure_monitors(PassthroughCommand, + State3, + Monitors3, + []), + + Command3 = + unregister_consumer_command(<<"stream">>, <<"app">>, self(), 1), + + {#?STATE{pids_groups = PidsGroups4} = _State4, Monitors4, Effects4} = + ?MOD:ensure_monitors(Command3, + State3#?STATE{groups = + #{}}, + Monitors3, + []), + assertEmpty(PidsGroups4), + assertEmpty(Monitors4), + ?assertEqual([{demonitor, process, self()}], Effects4), + + ok. + +handle_connection_down_sac_should_get_activated_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = spawn(fun() -> ok end), + Group = cgroup([consumer(Pid0, 0, true), + consumer(Pid1, 1, false), + consumer(Pid0, 2, false)]), + State0 = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}]), + Pid1 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups1, groups = Groups1} = State1, + Effects1} = + ?MOD:handle_connection_down(Pid0, State0), + assertSize(1, PidsGroups1), + assertSize(1, maps:get(Pid1, PidsGroups1)), + assertSendMessageEffect(Pid1, 1, Stream, ConsumerName, true, Effects1), + assertHasGroup(GroupId, cgroup([consumer(Pid1, 1, true)]), Groups1), + {#?STATE{pids_groups = PidsGroups2, groups = Groups2}, + Effects2} = + ?MOD:handle_connection_down(Pid1, State1), + assertEmpty(PidsGroups2), + assertEmpty(Effects2), + assertEmpty(Groups2), + + ok. + +handle_connection_down_sac_active_does_not_change_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = spawn(fun() -> ok end), + Group = cgroup([consumer(Pid1, 0, true), + consumer(Pid0, 1, false), + consumer(Pid0, 2, false)]), + State = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}]), + Pid1 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups, groups = Groups}, + Effects} = + ?MOD:handle_connection_down(Pid0, State), + assertSize(1, PidsGroups), + assertSize(1, maps:get(Pid1, PidsGroups)), + assertEmpty(Effects), + assertHasGroup(GroupId, cgroup([consumer(Pid1, 0, true)]), Groups), + ok. + +handle_connection_down_sac_no_more_consumers_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Group = cgroup([consumer(Pid0, 0, true), + consumer(Pid0, 1, false)]), + State = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups, groups = Groups}, + Effects} = + ?MOD:handle_connection_down(Pid0, State), + assertEmpty(PidsGroups), + assertEmpty(Groups), + assertEmpty(Effects), + ok. + +handle_connection_down_sac_no_consumers_in_down_connection_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = spawn(fun() -> ok end), + Group = cgroup([consumer(Pid1, 0, true), + consumer(Pid1, 1, false)]), + State = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}]), %% should not be there + Pid1 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups, groups = Groups}, + Effects} = + ?MOD:handle_connection_down(Pid0, State), + + assertSize(1, PidsGroups), + assertSize(1, maps:get(Pid1, PidsGroups)), + assertEmpty(Effects), + assertHasGroup(GroupId, cgroup([consumer(Pid1, 0, true), consumer(Pid1, 1, false)]), + Groups), + ok. + +handle_connection_down_super_stream_active_stays_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = spawn(fun() -> ok end), + Group = cgroup(1, [consumer(Pid0, 0, false), + consumer(Pid0, 1, true), + consumer(Pid1, 2, false), + consumer(Pid1, 3, false)]), + State = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}]), + Pid1 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups, groups = Groups}, + Effects} = + ?MOD:handle_connection_down(Pid1, State), + assertSize(1, PidsGroups), + assertSize(1, maps:get(Pid0, PidsGroups)), + assertEmpty(Effects), + assertHasGroup(GroupId, cgroup(1, [consumer(Pid0, 0, false), consumer(Pid0, 1, true)]), + Groups), + ok. + +handle_connection_down_super_stream_active_changes_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = spawn(fun() -> ok end), + Group = cgroup(1, [consumer(Pid0, 0, false), + consumer(Pid1, 1, true), + consumer(Pid0, 2, false), + consumer(Pid1, 3, false)]), + State = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}]), + Pid1 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups, groups = Groups}, + Effects} = + ?MOD:handle_connection_down(Pid0, State), + assertSize(1, PidsGroups), + assertSize(1, maps:get(Pid1, PidsGroups)), + assertSendMessageSteppingDownEffect(Pid1, 1, Stream, ConsumerName, Effects), + assertHasGroup(GroupId, cgroup(1, [consumer(Pid1, 1, false), consumer(Pid1, 3, false)]), + Groups), + ok. + +handle_connection_down_super_stream_activate_in_remaining_connection_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = spawn(fun() -> ok end), + Group = cgroup(1, [consumer(Pid0, 0, false), + consumer(Pid0, 1, true), + consumer(Pid1, 2, false), + consumer(Pid1, 3, false)]), + State = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}]), + Pid1 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups, groups = Groups}, + Effects} = + ?MOD:handle_connection_down(Pid0, State), + assertSize(1, PidsGroups), + assertSize(1, maps:get(Pid1, PidsGroups)), + assertSendMessageEffect(Pid1, 3, Stream, ConsumerName, true, Effects), + assertHasGroup(GroupId, cgroup(1, [consumer(Pid1, 2, false), consumer(Pid1, 3, true)]), + Groups), + ok. + +handle_connection_down_super_stream_no_active_removed_or_present_test(_) -> + Stream = <<"stream">>, + ConsumerName = <<"app">>, + GroupId = {<<"/">>, Stream, ConsumerName}, + Pid0 = self(), + Pid1 = spawn(fun() -> ok end), + %% this is a weird case that should not happen in the wild, + %% we test the logic in the code nevertheless. + %% No active consumer in the group + Group = cgroup(1, [consumer(Pid0, 0, false), + consumer(Pid0, 1, false), + consumer(Pid1, 2, false), + consumer(Pid1, 3, false)]), + State = state(#{GroupId => Group}, + #{Pid0 => maps:from_list([{GroupId, true}]), + Pid1 => maps:from_list([{GroupId, true}])}), + + {#?STATE{pids_groups = PidsGroups, groups = Groups}, + Effects} = + ?MOD:handle_connection_down(Pid0, State), + assertSize(1, PidsGroups), + assertSize(1, maps:get(Pid1, PidsGroups)), + assertEmpty(Effects), + assertHasGroup(GroupId, cgroup(1, [consumer(Pid1, 2, false), consumer(Pid1, 3, false)]), + Groups), + ok. + +assertSize(Expected, []) -> + ?assertEqual(Expected, 0); +assertSize(Expected, Map) when is_map(Map) -> + ?assertEqual(Expected, maps:size(Map)); +assertSize(Expected, List) when is_list(List) -> + ?assertEqual(Expected, length(List)). + +assertEmpty(Data) -> + assertSize(0, Data). + +assertHasGroup(GroupId, Group, Groups) -> + ?assertEqual(#{GroupId => Group}, Groups). + +consumer(Pid, SubId, Active) -> + #consumer{pid = Pid, + subscription_id = SubId, + owner = <<"owning connection label">>, + active = Active}. + +cgroup(Consumers) -> + cgroup(-1, Consumers). + +cgroup(PartitionIndex, Consumers) -> + #group{partition_index = PartitionIndex, consumers = Consumers}. + +state() -> + state(#{}). + +state(Groups) -> + state(Groups, #{}). + +state(Groups, PidsGroups) -> + #?STATE{groups = Groups, pids_groups = PidsGroups}. + +register_consumer_command(Stream, + PartitionIndex, + ConsumerName, + ConnectionPid, + SubId) -> + #command_register_consumer{vhost = <<"/">>, + stream = Stream, + partition_index = PartitionIndex, + consumer_name = ConsumerName, + connection_pid = ConnectionPid, + owner = <<"owning connection label">>, + subscription_id = SubId}. + +unregister_consumer_command(Stream, + ConsumerName, + ConnectionPid, + SubId) -> + #command_unregister_consumer{vhost = <<"/">>, + stream = Stream, + consumer_name = ConsumerName, + connection_pid = ConnectionPid, + subscription_id = SubId}. + +activate_consumer_command(Stream, ConsumerName) -> + #command_activate_consumer{vhost = <<"/">>, + stream = Stream, + consumer_name = ConsumerName}. + +assertSendMessageEffect(Pid, SubId, Stream, ConsumerName, Active, [Effect]) -> + ?assertEqual({mod_call, + rabbit_stream_sac_coordinator, + send_message, + [Pid, + {sac, + #{subscription_id => SubId, + stream => Stream, + consumer_name => ConsumerName, + active => Active} + }]}, + Effect). + +assertSendMessageSteppingDownEffect(Pid, SubId, Stream, ConsumerName, [Effect]) -> + ?assertEqual({mod_call, + rabbit_stream_sac_coordinator, + send_message, + [Pid, + {sac, + #{subscription_id => SubId, + stream => Stream, + consumer_name => ConsumerName, + active => false, + stepping_down => true}}]}, + Effect). diff --git a/deps/rabbitmq_stream/docs/stream_coordinator.md b/deps/rabbitmq_stream/docs/stream_coordinator.md new file mode 100644 index 000000000000..e3165a9ff37f --- /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 + C1->>C1: consumer update to client + deactivate C2 +``` + +### SAC with Super Stream Partition + +```mermaid +sequenceDiagram + participant C as Coordinator + participant C1 as Connection 1 + participant C2 as Connection 2 + participant C3 as Connection 3 + Note over C,C3: Super Stream SAC (partition = 1) + C1->>C: register sub 1 + C-)C1: {sac, sub 1, active = true} + activate C1 + C2->>C: register sub 2 + C-)C1: {sac, sub 1, active = false, step down = true} + deactivate C1 + C1->>C1: consumer update to client + C1->>C: activate consumer in group + C-)C2: {sac, sub 2, active = true} + activate C2 + C2->>C2: consumer update to client + C3->>C: register sub 3 + Note over C, C3: active consumer stays the same (partition % consumers = 1 % 3 = 1) + deactivate C2 +``` + +### `noconnection` management + +```mermaid +flowchart TB + A(monitor) --noconnection--> B(status = disconnected, set up timer) + B -. timeout .-> C(status = forgotten) + B -. nodeup .-> D(reissue monitors, send msg to connections) + D -. down .-> E(handle connection down) + D -. connection response .-> F(evaluate impacted groups) +``` + +* composite status for consumers: `{connected, active}`, `{disconnected,active}`, etc. +* `disconnected` status can prevent rebalancing in a group, e.g. `{disconnected, active}` (it is impossible to tell the active consumer to step down) +* consumers in `forgotten` status are ignored during rebalancing +* it may be necessary to reconcile a group if a `{forgotten, active}` consumer comes back in a group ("evaluate impacted groups" box above). +This is unlikely though. + +### Stale Node Detection + +```mermaid +flowchart TB + A(RA) -- tick --> B(stale nodes = RA known nodes - cluster nodes) + B -. no stale nodes .-> C(nothing to do) + B -. stale nodes .-> D(remove connections from state) +``` diff --git a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl index e5931ce041e3..046697274616 100644 --- a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl +++ b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl @@ -720,6 +720,9 @@ open(info, {OK, S, Data}, StatemData#statem_data{connection = Connection1, connection_state = State2}} end; +open(info, {sac, check_connection, _}, State) -> + rabbit_stream_sac_coordinator:connection_reconnected(self()), + {keep_state, State}; open(info, {sac, #{subscription_id := SubId, active := Active} = Msg},