From 0312b19be7fc0f41cad7e01bd5b6c8050ad53635 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 12 May 2025 11:08:48 +0100 Subject: [PATCH 01/28] Ra v3 --- rabbitmq-components.mk | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rabbitmq-components.mk b/rabbitmq-components.mk index 7fc8aae4fbd5..fa6f101dd620 100644 --- a/rabbitmq-components.mk +++ b/rabbitmq-components.mk @@ -51,7 +51,7 @@ dep_khepri_mnesia_migration = hex 0.8.0 dep_meck = hex 1.0.0 dep_osiris = git https://github.com/rabbitmq/osiris v1.10.2 dep_prometheus = hex 5.1.1 -dep_ra = hex 2.17.1 +dep_ra = git https://github.com/rabbitmq/ra v3 dep_ranch = hex 2.2.0 dep_recon = hex 2.5.6 dep_redbug = hex 2.1.0 From 48b9407f69d14eeccf4108bed4dc43904e1b2ba6 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 9 Sep 2025 12:34:48 +0100 Subject: [PATCH 02/28] ra v3 wip --- deps/rabbit/src/rabbit_fifo.erl | 136 ++++++++++++++++++------ deps/rabbit/src/rabbit_fifo.hrl | 1 + deps/rabbit/src/rabbit_fifo_client.erl | 15 +-- deps/rabbit/src/rabbit_fifo_dlx.erl | 7 +- deps/rabbit/src/rabbit_fifo_index.erl | 5 + deps/rabbit/src/rabbit_fifo_q.erl | 8 ++ deps/rabbit/src/rabbit_quorum_queue.erl | 1 + 7 files changed, 132 insertions(+), 41 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index a43371cc816b..f889032e8113 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -42,6 +42,7 @@ %% ra_machine callbacks init/1, apply/3, + live_indexes/1, state_enter/2, tick/2, overview/1, @@ -687,6 +688,15 @@ apply(_Meta, Cmd, State) -> ?LOG_DEBUG("rabbit_fifo: unhandled command ~W", [Cmd, 10]), {State, ok, []}. +-spec live_indexes(state()) -> + [ra:index()]. +live_indexes(#?STATE{returns = Returns, + messages = Messages, + dlx = Dlx}) -> + DlxIndexes = rabbit_fifo_dlx:live_indexes(Dlx), + RtnIndexes = [I || ?MSG(I, _) <- lqueue:to_list(Returns)], + DlxIndexes ++ RtnIndexes ++ rabbit_fifo_q:indexes(Messages). + convert_v3_to_v4(#{} = _Meta, StateV3) -> %% TODO: consider emitting release cursors as checkpoints Messages0 = rabbit_fifo_v3:get_field(messages, StateV3), @@ -939,7 +949,7 @@ get_checked_out(CKey, From, To, #?STATE{consumers = Consumers}) -> end. -spec version() -> pos_integer(). -version() -> 7. +version() -> 8. which_module(0) -> rabbit_fifo_v0; which_module(1) -> rabbit_fifo_v1; @@ -948,7 +958,8 @@ which_module(3) -> rabbit_fifo_v3; which_module(4) -> ?MODULE; which_module(5) -> ?MODULE; which_module(6) -> ?MODULE; -which_module(7) -> ?MODULE. +which_module(7) -> ?MODULE; +which_module(8) -> ?MODULE. -define(AUX, aux_v3). @@ -958,6 +969,12 @@ which_module(7) -> ?MODULE. messages_total :: non_neg_integer(), indexes = ?CHECK_MIN_INDEXES :: non_neg_integer(), bytes_in = 0 :: non_neg_integer()}). +-record(snapshot, {index :: ra:index(), + timestamp :: milliseconds(), + % smallest_index :: undefined | ra:index(), + messages_total :: non_neg_integer(), + % indexes = ?CHECK_MIN_INDEXES :: non_neg_integer(), + bytes_out = 0 :: non_neg_integer()}). -record(aux_gc, {last_raft_idx = 0 :: ra:index()}). -record(aux, {name :: atom(), capacity :: term(), @@ -968,7 +985,7 @@ which_module(7) -> ?MODULE. gc = #aux_gc{} :: #aux_gc{}, tick_pid :: undefined | pid(), cache = #{} :: map(), - last_checkpoint :: #checkpoint{}, + last_checkpoint :: #checkpoint{} | #snapshot{}, bytes_in = 0 :: non_neg_integer(), bytes_out = 0 :: non_neg_integer()}). @@ -980,10 +997,10 @@ init_aux(Name) when is_atom(Name) -> Now = erlang:monotonic_time(microsecond), #?AUX{name = Name, capacity = {inactive, Now, 1, 1.0}, - last_checkpoint = #checkpoint{index = 0, - timestamp = erlang:system_time(millisecond), - messages_total = 0, - bytes_in = 0}}. + last_checkpoint = #snapshot{index = 0, + timestamp = erlang:system_time(millisecond), + messages_total = 0, + bytes_out = 0}}. handle_aux(RaftState, Tag, Cmd, #aux{name = Name, capacity = Cap, @@ -1001,13 +1018,16 @@ handle_aux(RaftState, Tag, Cmd, AuxV2, RaAux) handle_aux(leader, cast, eval, #?AUX{last_decorators_state = LastDec, bytes_in = BytesIn, + bytes_out = BytesOut, last_checkpoint = Check0} = Aux0, RaAux) -> #?STATE{cfg = #cfg{resource = QName}} = MacState = ra_aux:machine_state(RaAux), Ts = erlang:system_time(millisecond), - {Check, Effects0} = do_checkpoints(Ts, Check0, RaAux, BytesIn, false), + EffMacVer = ra_aux:effective_machine_version(RaAux), + {Check, Effects0} = do_checkpoints(EffMacVer, Ts, Check0, RaAux, + BytesIn, BytesOut, false), %% this is called after each batch of commands have been applied %% set timer for message expire @@ -1024,15 +1044,23 @@ handle_aux(leader, cast, eval, end; handle_aux(_RaftState, cast, eval, #?AUX{last_checkpoint = Check0, - bytes_in = BytesIn} = Aux0, + bytes_in = BytesIn, + bytes_out = BytesOut + } = Aux0, RaAux) -> Ts = erlang:system_time(millisecond), - {Check, Effects} = do_checkpoints(Ts, Check0, RaAux, BytesIn, false), + EffMacVer = ra_aux:effective_machine_version(RaAux), + {Check, Effects} = do_checkpoints(EffMacVer, Ts, Check0, RaAux, + BytesIn, BytesOut, false), {no_reply, Aux0#?AUX{last_checkpoint = Check}, RaAux, Effects}; handle_aux(_RaftState, cast, {bytes_in, {MetaSize, BodySize}}, #?AUX{bytes_in = Bytes} = Aux0, RaAux) -> {no_reply, Aux0#?AUX{bytes_in = Bytes + MetaSize + BodySize}, RaAux, []}; +handle_aux(_RaftState, cast, {bytes_out, BodySize}, + #?AUX{bytes_out = Bytes} = Aux0, + RaAux) -> + {no_reply, Aux0#?AUX{bytes_out = Bytes + BodySize}, RaAux, []}; handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, consumer_key = Key} = Ret, Corr, Pid}, Aux0, RaAux0) -> @@ -1163,12 +1191,15 @@ handle_aux(_, _, garbage_collection, Aux, RaAux) -> {no_reply, force_eval_gc(RaAux, Aux), RaAux}; handle_aux(_RaState, _, force_checkpoint, #?AUX{last_checkpoint = Check0, - bytes_in = BytesIn} = Aux, RaAux) -> + bytes_in = BytesIn, + bytes_out = BytesOut} = Aux, RaAux) -> Ts = erlang:system_time(millisecond), #?STATE{cfg = #cfg{resource = QR}} = ra_aux:machine_state(RaAux), ?LOG_DEBUG("~ts: rabbit_fifo: forcing checkpoint at ~b", [rabbit_misc:rs(QR), ra_aux:last_applied(RaAux)]), - {Check, Effects} = do_checkpoints(Ts, Check0, RaAux, BytesIn, true), + EffMacVer = ra_aux:effective_machine_version(RaAux), + {Check, Effects} = do_checkpoints(EffMacVer, Ts, Check0, RaAux, + BytesIn, BytesOut, true), {no_reply, Aux#?AUX{last_checkpoint = Check}, RaAux, Effects}; handle_aux(RaState, _, {dlx, _} = Cmd, Aux0, RaAux) -> #?STATE{dlx = DlxState, @@ -1797,7 +1828,8 @@ complete(Meta, ConsumerKey, [MsgId], #consumer{checked_out = Checked0} = Con0, #?STATE{ra_indexes = Indexes0, msg_bytes_checkout = BytesCheckout, - messages_total = Tot} = State0) -> + messages_total = Tot} = State0, + Effects) -> case maps:take(MsgId, Checked0) of {?MSG(Idx, Hdr), Checked} -> SettledSize = get_header(size, Hdr), @@ -1805,17 +1837,18 @@ complete(Meta, ConsumerKey, [MsgId], Con = Con0#consumer{checked_out = Checked, credit = increase_credit(Con0, 1)}, State1 = update_or_remove_con(Meta, ConsumerKey, Con, State0), - State1#?STATE{ra_indexes = Indexes, + {State1#?STATE{ra_indexes = Indexes, msg_bytes_checkout = BytesCheckout - SettledSize, - messages_total = Tot - 1}; + messages_total = Tot - 1}, + [{aux, {bytes_out, SettledSize}}, Effects]}; error -> - State0 + {State0, Effects} end; complete(Meta, ConsumerKey, MsgIds, #consumer{checked_out = Checked0} = Con0, #?STATE{ra_indexes = Indexes0, msg_bytes_checkout = BytesCheckout, - messages_total = Tot} = State0) -> + messages_total = Tot} = State0, Effects) -> {SettledSize, Checked, Indexes} = lists:foldl( fun (MsgId, {S0, Ch0, Idxs}) -> @@ -1831,9 +1864,10 @@ complete(Meta, ConsumerKey, MsgIds, Con = Con0#consumer{checked_out = Checked, credit = increase_credit(Con0, Len)}, State1 = update_or_remove_con(Meta, ConsumerKey, Con, State0), - State1#?STATE{ra_indexes = Indexes, + {State1#?STATE{ra_indexes = Indexes, msg_bytes_checkout = BytesCheckout - SettledSize, - messages_total = Tot - Len}. + messages_total = Tot - Len}, + [{aux, {bytes_out, SettledSize}}, Effects]}. increase_credit(#consumer{cfg = #consumer_cfg{lifetime = once}, credit = Credit}, _) -> @@ -1860,11 +1894,12 @@ increase_credit(#consumer{credit = Current}, Credit) -> complete_and_checkout(#{} = Meta, MsgIds, ConsumerKey, #consumer{} = Con0, Effects0, State0) -> - State1 = complete(Meta, ConsumerKey, MsgIds, Con0, State0), + {State1, Effects1} = complete(Meta, ConsumerKey, MsgIds, + Con0, State0, Effects0), %% a completion could have removed the active/quiescing consumer - Effects1 = add_active_effect(Con0, State1, Effects0), - {State2, Effects2} = activate_next_consumer(State1, Effects1), - checkout(Meta, State0, State2, Effects2). + Effects2 = add_active_effect(Con0, State1, Effects1), + {State2, Effects} = activate_next_consumer(State1, Effects2), + checkout(Meta, State0, State2, Effects). add_active_effect(#consumer{status = quiescing} = Consumer, #?STATE{cfg = #cfg{consumer_strategy = single_active}, @@ -1956,8 +1991,9 @@ return_one(Meta, MsgId, ?MSG(_, _) = Msg0, DelivFailed, Anns, {DlxState, DlxEffects} = rabbit_fifo_dlx:discard([Msg], delivery_limit, DLH, DlxState0), State1 = State0#?STATE{dlx = DlxState}, - State = complete(Meta, ConsumerKey, [MsgId], Con0, State1), - {State, DlxEffects ++ Effects0}; + {State, Effects} = complete(Meta, ConsumerKey, [MsgId], + Con0, State1, Effects0), + {State, DlxEffects ++ Effects}; _ -> Checked = maps:remove(MsgId, Checked0), Con = Con0#consumer{checked_out = Checked, @@ -2829,7 +2865,10 @@ convert(Meta, 5, To, State) -> convert(Meta, 6, To, State); convert(Meta, 6, To, State) -> %% no conversion needed, this version only includes a logic change - convert(Meta, 7, To, State). + convert(Meta, 7, To, State); +convert(Meta, 7, To, State) -> + %% no conversion needed, this version only includes a logic change + convert(Meta, 8, To, State). smallest_raft_index(#?STATE{messages = Messages, ra_indexes = Indexes, @@ -2999,12 +3038,43 @@ priority_tag(Msg) -> end. -do_checkpoints(Ts, #checkpoint{index = ChIdx, - timestamp = ChTime, - smallest_index = LastSmallest, - bytes_in = LastBytesIn, - indexes = MinIndexes} = Check0, - RaAux, BytesIn, Force) -> +do_checkpoints(MacVer, Ts, #checkpoint{index = _ChIdx, + timestamp = _SnapTime}, + RaAux, BytesIn, BytesOut, Force) when MacVer >= 8 -> + do_checkpoints(MacVer, Ts, #snapshot{}, RaAux, BytesIn, BytesOut, Force); +do_checkpoints(MacVer, Ts, #snapshot{index = _ChIdx, + timestamp = SnapTime, + bytes_out = LastBytesOut} = Snap0, + RaAux, _BytesIn, BytesOut, _Force) when MacVer >= 8 -> + LastAppliedIdx = ra_aux:last_applied(RaAux), + #?STATE{} = MacState = ra_aux:machine_state(RaAux), + TimeSince = Ts - SnapTime, + MsgsTot = messages_total(MacState), + ra_aux:overview(RaAux), + % MaxBytesFactor = max(1, MsgsTot / CheckMaxIndexes), + EnoughDataRemoved = BytesOut - LastBytesOut > ?SNAP_OUT_BYTES, + {CheckMinInterval, _CheckMinIndexes, _CheckMaxIndexes} = + persistent_term:get(quorum_queue_checkpoint_config, + {?CHECK_MIN_INTERVAL_MS, ?CHECK_MIN_INDEXES, + ?CHECK_MAX_INDEXES}), + EnoughTimeHasPassed = TimeSince > CheckMinInterval, + case (EnoughTimeHasPassed andalso + EnoughDataRemoved) of + true -> + {#snapshot{index = LastAppliedIdx, + timestamp = Ts, + messages_total = MsgsTot, + bytes_out = BytesOut}, + [{release_cursor, LastAppliedIdx, MacState}]}; + false -> + {Snap0, []} + end; +do_checkpoints(MacVer,Ts, #checkpoint{index = ChIdx, + timestamp = ChTime, + smallest_index = LastSmallest, + bytes_in = LastBytesIn, + indexes = MinIndexes} = Check0, + RaAux, BytesIn, _BytesOut, Force) when MacVer < 8 -> LastAppliedIdx = ra_aux:last_applied(RaAux), IndexesSince = LastAppliedIdx - ChIdx, #?STATE{} = MacState = ra_aux:machine_state(RaAux), @@ -3034,7 +3104,7 @@ do_checkpoints(Ts, #checkpoint{index = ChIdx, %% condition 1: enough indexes have been committed since the last %% checkpoint (IndexesSince > MinIndexes) orelse - %% condition 2: the queue is empty and _some_ commands + %% condition 2: the queue is empty and _some_ commands %% have been applied since the last checkpoint (MsgsTot == 0 andalso IndexesSince > 32) ) diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index b8b69bff7f45..71c30f4b05de 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -105,6 +105,7 @@ %% once these many bytes have been written since the last checkpoint %% we request a checkpoint irrespectively -define(CHECK_MAX_BYTES, 128_000_000). +-define(SNAP_OUT_BYTES, 64_000_000). -define(USE_AVG_HALF_LIFE, 10000.0). %% an average QQ without any message uses about 100KB so setting this limit diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index 4feacb113682..44e8e29d68d9 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -994,13 +994,14 @@ send_command(Server, Correlation, Command, Priority, #state{pending = Pending, next_seq = Seq, cfg = #cfg{soft_limit = SftLmt}} = State) -> - ok = case rabbit_fifo:is_return(Command) of - true -> - %% returns are sent to the aux machine for pre-evaluation - ra:cast_aux_command(Server, {Command, Seq, self()}); - _ -> - ra:pipeline_command(Server, Command, Seq, Priority) - end, + % ok = case rabbit_fifo:is_return(Command) of + % true -> + % %% returns are sent to the aux machine for pre-evaluation + % ra:cast_aux_command(Server, {Command, Seq, self()}); + % _ -> + % ra:pipeline_command(Server, Command, Seq, Priority) + % end, + ok = ra:pipeline_command(Server, Command, Seq, Priority), State#state{pending = Pending#{Seq => {Correlation, Command}}, next_seq = Seq + 1, slow = map_size(Pending) >= SftLmt}. diff --git a/deps/rabbit/src/rabbit_fifo_dlx.erl b/deps/rabbit/src/rabbit_fifo_dlx.erl index 53f404bb85cf..3d79f3ae84ae 100644 --- a/deps/rabbit/src/rabbit_fifo_dlx.erl +++ b/deps/rabbit/src/rabbit_fifo_dlx.erl @@ -26,7 +26,8 @@ dehydrate/1, stat/1, update_config/4, - smallest_raft_index/1 + smallest_raft_index/1, + live_indexes/1 ]). -record(checkout, {consumer :: pid(), @@ -365,5 +366,9 @@ dehydrate(State) -> smallest_raft_index(#?MODULE{ra_indexes = Indexes}) -> rabbit_fifo_index:smallest(Indexes). +-spec live_indexes(state()) -> [ra:index()]. +live_indexes(#?MODULE{ra_indexes = Indexes}) -> + rabbit_fifo_index:indexes(Indexes). + annotate_msg(H, Msg) -> rabbit_fifo:annotate_msg(H, Msg). diff --git a/deps/rabbit/src/rabbit_fifo_index.erl b/deps/rabbit/src/rabbit_fifo_index.erl index 559a1b171024..2acf3d5e05fc 100644 --- a/deps/rabbit/src/rabbit_fifo_index.erl +++ b/deps/rabbit/src/rabbit_fifo_index.erl @@ -13,6 +13,7 @@ delete/2, size/1, smallest/1, + indexes/1, map/2, to_list/1 ]). @@ -90,6 +91,10 @@ size(#?MODULE{data = Data}) -> smallest(#?MODULE{smallest = Smallest}) -> Smallest. +-spec indexes(state()) -> [ra:index()]. +indexes(#?MODULE{data = Data}) -> + maps:keys(Data). + -spec map(fun(), state()) -> state(). map(F, #?MODULE{data = Data} = State) -> State#?MODULE{data = maps:map(F, Data)}. diff --git a/deps/rabbit/src/rabbit_fifo_q.erl b/deps/rabbit/src/rabbit_fifo_q.erl index 3ddf165a03bc..c9c3a2fff700 100644 --- a/deps/rabbit/src/rabbit_fifo_q.erl +++ b/deps/rabbit/src/rabbit_fifo_q.erl @@ -8,6 +8,7 @@ get/1, len/1, from_lqueue/1, + indexes/1, get_lowest_index/1, overview/1 ]). @@ -81,6 +82,13 @@ from_lqueue(LQ) -> in(no, Item, Acc) end, new(), LQ). +indexes(#?MODULE{hi = {Hi1, Hi2}, + no = {No1, No2}}) -> + A = lists:map(fun (?MSG(I, _)) -> I end, Hi1), + B = lists:foldl(fun (?MSG(I, _), Acc) -> [I | Acc] end, A, Hi2), + C = lists:foldl(fun (?MSG(I, _), Acc) -> [I | Acc] end, B, No1), + lists:foldl(fun (?MSG(I, _), Acc) -> [I | Acc] end, C, No2). + -spec get_lowest_index(state()) -> undefined | ra:index(). get_lowest_index(#?MODULE{len = 0}) -> undefined; diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 9f0c04546abf..56b215986186 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -1993,6 +1993,7 @@ make_ra_conf(Q, ServerId, TickTimeout, FName = rabbit_misc:rs(QName), Formatter = {?MODULE, format_ra_event, [QName]}, LogCfg = #{uid => UId, + min_snapshot_interval => 0, snapshot_interval => SnapshotInterval, min_checkpoint_interval => CheckpointInterval, max_checkpoints => 3}, From ae4483b6b3a63fe4f0bc8c8a2de223671da9c28f Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 16 Sep 2025 11:05:52 +0100 Subject: [PATCH 03/28] QQ: implement snapshot_installed/4 callback --- deps/rabbit/src/rabbit_fifo.erl | 114 +++++++++++++++++------ deps/rabbit/src/rabbit_fifo_q.erl | 1 + deps/rabbit/src/rabbit_quorum_queue.erl | 3 +- deps/rabbit/test/quorum_queue_SUITE.erl | 94 +++++++++++++++++-- deps/rabbit/test/rabbit_fifo_SUITE.erl | 2 + deps/rabbit/test/rabbit_fifo_q_SUITE.erl | 2 + 6 files changed, 179 insertions(+), 37 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index f889032e8113..6ea5ef499512 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -43,6 +43,7 @@ init/1, apply/3, live_indexes/1, + snapshot_installed/4, state_enter/2, tick/2, overview/1, @@ -487,7 +488,8 @@ apply(#{index := Idx} = Meta, apply(#{index := Index}, #purge{}, #?STATE{messages_total = Total, returns = Returns, - ra_indexes = Indexes0 + ra_indexes = Indexes0, + msg_bytes_enqueue = MsgBytesEnqueue } = State0) -> NumReady = messages_ready(State0), Indexes = case Total of @@ -514,7 +516,9 @@ apply(#{index := Index}, #purge{}, returns = lqueue:new(), msg_bytes_enqueue = 0 }, - Effects0 = [{aux, force_checkpoint}, garbage_collection], + Effects0 = [{aux, {bytes_out, MsgBytesEnqueue}}, + {aux, force_checkpoint}, + garbage_collection], Reply = {purge, NumReady}, {State, Effects} = evaluate_limit(Index, State0, State1, Effects0), {State, Reply, Effects}; @@ -688,14 +692,61 @@ apply(_Meta, Cmd, State) -> ?LOG_DEBUG("rabbit_fifo: unhandled command ~W", [Cmd, 10]), {State, ok, []}. --spec live_indexes(state()) -> - [ra:index()]. -live_indexes(#?STATE{returns = Returns, +-spec live_indexes(state()) -> [ra:index()]. +live_indexes(#?STATE{cfg = #cfg{}, + returns = Returns, messages = Messages, + consumers = Consumers, dlx = Dlx}) -> DlxIndexes = rabbit_fifo_dlx:live_indexes(Dlx), RtnIndexes = [I || ?MSG(I, _) <- lqueue:to_list(Returns)], - DlxIndexes ++ RtnIndexes ++ rabbit_fifo_q:indexes(Messages). + CheckedIdxs = maps:fold( + fun (_Cid, #consumer{checked_out = Ch}, Acc0) -> + maps:fold( + fun (_MsgId, ?MSG(I, _), Acc) -> + [I | Acc] + end, Acc0, Ch) + end, RtnIndexes ++ DlxIndexes, Consumers), + + + CheckedIdxs ++ rabbit_fifo_q:indexes(Messages). + + +-spec snapshot_installed(Meta, State, OldMeta, OldState) -> + ra_machine:effects() when + Meta :: ra_snapshot:meta(), + State :: state(), + OldMeta :: ra_snapshot:meta(), + OldState :: state(). +snapshot_installed(_Meta, #?MODULE{cfg = #cfg{resource = QR}, + consumers = Consumers} = State, + _OldMeta, _OldState) -> + %% here we need to redliver all pending consumer messages + %% to local consumers + %% TODO: with some additional state (raft indexes assigned to consumer) + %% we could reduce the number of resends but it is questionable if this + %% complexity is worth the effort. rabbit_fifo_index will de-duplicate + %% deliveries anyway + SendAcc = maps:fold( + fun (_ConsumerKey, #consumer{cfg = #consumer_cfg{tag = Tag, + pid = Pid}, + checked_out = Checked}, + Acc) -> + case node(Pid) == node() of + true -> + Acc#{{Tag, Pid} => maps:to_list(Checked)}; + false -> + Acc + end + end, #{}, Consumers), + ?LOG_DEBUG("~ts: rabbit_fifo: install snapshot sending ~p", + [rabbit_misc:rs(QR), SendAcc]), + Effs = add_delivery_effects([], SendAcc, State), + ?LOG_DEBUG("~ts: rabbit_fifo: effs ~p", + [rabbit_misc:rs(QR), Effs]), + Effs. + + convert_v3_to_v4(#{} = _Meta, StateV3) -> %% TODO: consider emitting release cursors as checkpoints @@ -972,7 +1023,7 @@ which_module(8) -> ?MODULE. -record(snapshot, {index :: ra:index(), timestamp :: milliseconds(), % smallest_index :: undefined | ra:index(), - messages_total :: non_neg_integer(), + messages_total = 0 :: non_neg_integer(), % indexes = ?CHECK_MIN_INDEXES :: non_neg_integer(), bytes_out = 0 :: non_neg_integer()}). -record(aux_gc, {last_raft_idx = 0 :: ra:index()}). @@ -997,10 +1048,9 @@ init_aux(Name) when is_atom(Name) -> Now = erlang:monotonic_time(microsecond), #?AUX{name = Name, capacity = {inactive, Now, 1, 1.0}, - last_checkpoint = #snapshot{index = 0, - timestamp = erlang:system_time(millisecond), - messages_total = 0, - bytes_out = 0}}. + last_checkpoint = #checkpoint{index = 0, + timestamp = erlang:system_time(millisecond), + messages_total = 0}}. handle_aux(RaftState, Tag, Cmd, #aux{name = Name, capacity = Cap, @@ -1025,7 +1075,13 @@ handle_aux(leader, cast, eval, ra_aux:machine_state(RaAux), Ts = erlang:system_time(millisecond), - EffMacVer = ra_aux:effective_machine_version(RaAux), + EffMacVer = try ra_aux:effective_machine_version(RaAux) of + V -> V + catch _:_ -> + %% this function is not available in older aux states. + %% this is a guess + undefined + end, {Check, Effects0} = do_checkpoints(EffMacVer, Ts, Check0, RaAux, BytesIn, BytesOut, false), @@ -1840,7 +1896,7 @@ complete(Meta, ConsumerKey, [MsgId], {State1#?STATE{ra_indexes = Indexes, msg_bytes_checkout = BytesCheckout - SettledSize, messages_total = Tot - 1}, - [{aux, {bytes_out, SettledSize}}, Effects]}; + [{aux, {bytes_out, SettledSize}} | Effects]}; error -> {State0, Effects} end; @@ -1867,7 +1923,7 @@ complete(Meta, ConsumerKey, MsgIds, {State1#?STATE{ra_indexes = Indexes, msg_bytes_checkout = BytesCheckout - SettledSize, messages_total = Tot - Len}, - [{aux, {bytes_out, SettledSize}}, Effects]}. + [{aux, {bytes_out, SettledSize}} | Effects]}. increase_credit(#consumer{cfg = #consumer_cfg{lifetime = once}, credit = Credit}, _) -> @@ -3037,29 +3093,34 @@ priority_tag(Msg) -> no end. - -do_checkpoints(MacVer, Ts, #checkpoint{index = _ChIdx, - timestamp = _SnapTime}, - RaAux, BytesIn, BytesOut, Force) when MacVer >= 8 -> - do_checkpoints(MacVer, Ts, #snapshot{}, RaAux, BytesIn, BytesOut, Force); +do_checkpoints(MacVer, Ts, #checkpoint{timestamp = LastTs, + index = Idx}, + RaAux, BytesIn, BytesOut, Force) + when is_integer(MacVer) andalso MacVer >= 8 -> + do_checkpoints(MacVer, Ts, #snapshot{index = Idx, + timestamp = LastTs}, RaAux, BytesIn, + BytesOut, Force); do_checkpoints(MacVer, Ts, #snapshot{index = _ChIdx, - timestamp = SnapTime, - bytes_out = LastBytesOut} = Snap0, - RaAux, _BytesIn, BytesOut, _Force) when MacVer >= 8 -> + timestamp = SnapTime, + bytes_out = LastBytesOut} = Snap0, + RaAux, _BytesIn, BytesOut, Force) + when is_integer(MacVer) andalso MacVer >= 8 -> LastAppliedIdx = ra_aux:last_applied(RaAux), #?STATE{} = MacState = ra_aux:machine_state(RaAux), TimeSince = Ts - SnapTime, MsgsTot = messages_total(MacState), - ra_aux:overview(RaAux), + % ra_aux:overview(RaAux), % MaxBytesFactor = max(1, MsgsTot / CheckMaxIndexes), + % TODO: snapshots also need to be triggered by non settled commands + % that aren't enqueues EnoughDataRemoved = BytesOut - LastBytesOut > ?SNAP_OUT_BYTES, {CheckMinInterval, _CheckMinIndexes, _CheckMaxIndexes} = persistent_term:get(quorum_queue_checkpoint_config, {?CHECK_MIN_INTERVAL_MS, ?CHECK_MIN_INDEXES, ?CHECK_MAX_INDEXES}), EnoughTimeHasPassed = TimeSince > CheckMinInterval, - case (EnoughTimeHasPassed andalso - EnoughDataRemoved) of + case (EnoughTimeHasPassed andalso EnoughDataRemoved) orelse + Force of true -> {#snapshot{index = LastAppliedIdx, timestamp = Ts, @@ -3074,7 +3135,8 @@ do_checkpoints(MacVer,Ts, #checkpoint{index = ChIdx, smallest_index = LastSmallest, bytes_in = LastBytesIn, indexes = MinIndexes} = Check0, - RaAux, BytesIn, _BytesOut, Force) when MacVer < 8 -> + RaAux, BytesIn, _BytesOut, Force) + when not is_integer(MacVer) orelse MacVer < 8 -> LastAppliedIdx = ra_aux:last_applied(RaAux), IndexesSince = LastAppliedIdx - ChIdx, #?STATE{} = MacState = ra_aux:machine_state(RaAux), diff --git a/deps/rabbit/src/rabbit_fifo_q.erl b/deps/rabbit/src/rabbit_fifo_q.erl index c9c3a2fff700..5f9d2e194d40 100644 --- a/deps/rabbit/src/rabbit_fifo_q.erl +++ b/deps/rabbit/src/rabbit_fifo_q.erl @@ -82,6 +82,7 @@ from_lqueue(LQ) -> in(no, Item, Acc) end, new(), LQ). +-spec indexes(state()) -> [ra:index()]. indexes(#?MODULE{hi = {Hi1, Hi2}, no = {No1, No2}}) -> A = lists:map(fun (?MSG(I, _)) -> I end, Hi1), diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 56b215986186..91e43af781c7 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -1983,7 +1983,7 @@ make_ra_conf(Q, ServerId, Membership, MacVersion) Membership, MacVersion). make_ra_conf(Q, ServerId, TickTimeout, - SnapshotInterval, CheckpointInterval, + _SnapshotInterval, CheckpointInterval, Membership, MacVersion) -> QName = amqqueue:get_name(Q), #resource{name = QNameBin} = QName, @@ -1994,7 +1994,6 @@ make_ra_conf(Q, ServerId, TickTimeout, Formatter = {?MODULE, format_ra_event, [QName]}, LogCfg = #{uid => UId, min_snapshot_interval => 0, - snapshot_interval => SnapshotInterval, min_checkpoint_interval => CheckpointInterval, max_checkpoints => 3}, rabbit_misc:maps_put_truthy(membership, Membership, diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index 79eebd9c8b52..b5b920c1ecc2 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -105,7 +105,8 @@ groups() -> force_checkpoint, policy_repair, gh_12635, - replica_states + replica_states, + consumer_message_is_delevered_after_snapshot ] ++ all_tests()}, {cluster_size_5, [], [start_queue, @@ -1434,6 +1435,7 @@ force_vhost_queues_shrink_member_to_current_member(Config) -> end. force_checkpoint_on_queue(Config) -> + check_quorum_queues_v8_compat(Config), [Server0, Server1, Server2] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), Ch = rabbit_ct_client_helpers:open_channel(Config, Server0), @@ -1452,19 +1454,19 @@ force_checkpoint_on_queue(Config) -> rabbit_ct_helpers:await_condition( fun() -> {ok, State, _} = rpc:call(Server0, ra, member_overview, [{RaName, Server0}]), - #{log := #{latest_checkpoint_index := LCI}} = State, + #{log := #{snapshot_index := LCI}} = State, LCI =:= undefined end), rabbit_ct_helpers:await_condition( fun() -> {ok, State, _} = rpc:call(Server1, ra, member_overview, [{RaName, Server1}]), - #{log := #{latest_checkpoint_index := LCI}} = State, + #{log := #{snapshot_index := LCI}} = State, LCI =:= undefined end), rabbit_ct_helpers:await_condition( fun() -> {ok, State, _} = rpc:call(Server2, ra, member_overview, [{RaName, Server2}]), - #{log := #{latest_checkpoint_index := LCI}} = State, + #{log := #{snapshot_index := LCI}} = State, LCI =:= undefined end), @@ -1481,21 +1483,21 @@ force_checkpoint_on_queue(Config) -> fun() -> {ok, State, _} = rpc:call(Server0, ra, member_overview, [{RaName, Server0}]), ct:pal("Ra server state post forced checkpoint: ~tp~n", [State]), - #{log := #{latest_checkpoint_index := LCI}} = State, + #{log := #{snapshot_index := LCI}} = State, (LCI =/= undefined) andalso (LCI >= N) end), rabbit_ct_helpers:await_condition( fun() -> {ok, State, _} = rpc:call(Server1, ra, member_overview, [{RaName, Server1}]), ct:pal("Ra server state post forced checkpoint: ~tp~n", [State]), - #{log := #{latest_checkpoint_index := LCI}} = State, + #{log := #{snapshot_index := LCI}} = State, (LCI =/= undefined) andalso (LCI >= N) end), rabbit_ct_helpers:await_condition( fun() -> {ok, State, _} = rpc:call(Server2, ra, member_overview, [{RaName, Server2}]), ct:pal("Ra server state post forced checkpoint: ~tp~n", [State]), - #{log := #{latest_checkpoint_index := LCI}} = State, + #{log := #{snapshot_index := LCI}} = State, (LCI =/= undefined) andalso (LCI >= N) end). @@ -1767,13 +1769,77 @@ dont_leak_file_handles(Config) -> rabbit_ct_client_helpers:close_channel(C), ok. +consumer_message_is_delevered_after_snapshot(Config) -> + %% a consumer on a node that received a snapshot should have it's messages + %% delivered + [Server0, _Server1, Server2] = Nodes = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + ok = rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env, + [rabbit, quorum_min_checkpoint_interval, 1]), + + Ch0 = rabbit_ct_client_helpers:open_channel(Config, Server0), + #'confirm.select_ok'{} = amqp_channel:call(Ch0, #'confirm.select'{}), + QQ = ?config(queue_name, Config), + RaName = ra_name(QQ), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch0, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + %% stop server on a follower node + ok = rpc:call(Server2, ra, stop_server, [quorum_queues, {RaName, Server2}]), + Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server2), + %% create a consumer + qos(Ch2, 2, false), + subscribe(Ch2, QQ, false), + + %% publish some messages and make sure a snapshot has been taken + Msg = crypto:strong_rand_bytes(13_000), + + [publish(Ch0, QQ, Msg) || _ <- lists:seq(1, 5000)], + amqp_channel:wait_for_confirms(Ch0, 5), + %% need to sleep here a bit as QQs wont take + %% snapshots more often than once every second + timer:sleep(1100), + + %% then purge + #'queue.purge_ok'{} = amqp_channel:call(Ch0, #'queue.purge'{queue = QQ}), + + MacVer = lists:min([V || {ok, V} <- erpc:multicall(Nodes, rabbit_fifo, version, [])]), + ct:pal("machine version is ~b", [MacVer]), + + %% only await snapshot if all members have at least machine version 8 + if MacVer >= 8 -> + rabbit_ct_helpers:await_condition( + fun () -> + {ok, #{log := Log}, _} = rpc:call(Server0, ra, member_overview, + [{RaName, Server0}]), + undefined =/= maps:get(snapshot_index, Log) + end); + true -> + ok + end, + %% restart stopped member + ok = rpc:call(Server2, ra, restart_server, [quorum_queues, {RaName, Server2}]), + + %% messages should be delivered + receive + {#'basic.deliver'{delivery_tag = _DeliveryTag}, _} -> + ok + after 30000 -> + flush(1), + ct:fail("expected messages were not delivered") + end, + ok. + gh_12635(Config) -> + check_quorum_queues_v8_compat(Config), + % https://github.com/rabbitmq/rabbitmq-server/issues/12635 [Server0, _Server1, Server2] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), ok = rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env, - [rabbit, quorum_min_checkpoint_interval, 1]), + [rabbit, quorum_snapshot_interval, 1]), Ch0 = rabbit_ct_client_helpers:open_channel(Config, Server0), #'confirm.select_ok'{} = amqp_channel:call(Ch0, #'confirm.select'{}), @@ -1795,7 +1861,7 @@ gh_12635(Config) -> rabbit_ct_helpers:await_condition( fun () -> {ok, #{log := Log}, _} = rpc:call(Server0, ra, member_overview, [{RaName, Server0}]), - undefined =/= maps:get(latest_checkpoint_index, Log) + undefined =/= maps:get(snapshot_index, Log) end), %% publish 1 more message @@ -5274,6 +5340,16 @@ basic_get(Ch, Q, NoAck, Attempt) -> basic_get(Ch, Q, NoAck, Attempt - 1) end. +check_quorum_queues_v8_compat(Config) -> + Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + MacVer = lists:min([V || {ok, V} <- erpc:multicall(Nodes, rabbit_fifo, version, [])]), + case MacVer >= 8 of + true -> + ok; + false -> + throw({skip, "test will only work on QQ machine version > 8"}) + end. + lists_interleave([], _List) -> []; lists_interleave([Item | Items], List) diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 075065ea86ab..f5aed72ef3fe 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -2445,6 +2445,7 @@ aux_test(_) -> queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), single_active_consumer_on => false}), log => mock_log, + cfg => #cfg{}, last_applied => LastApplied}, ok = meck:new(ra_log, []), meck:expect(ra_log, last_index_term, fun (_) -> {0, 0} end), @@ -2476,6 +2477,7 @@ handle_aux_tick_test(Config) -> timer:sleep(10), persistent_term:put(quorum_queue_checkpoint_config, {1, 0, 1}), + meck:expect(ra_aux, effective_machine_version, fun (_) -> 1 end), {no_reply, _, _, [{checkpoint, 1, _}, {release_cursor, 0}]} = handle_aux(follower, cast, force_checkpoint, Aux0, State1), diff --git a/deps/rabbit/test/rabbit_fifo_q_SUITE.erl b/deps/rabbit/test/rabbit_fifo_q_SUITE.erl index 919aa40f0e44..8edc27e4a979 100644 --- a/deps/rabbit/test/rabbit_fifo_q_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_q_SUITE.erl @@ -77,6 +77,8 @@ basics(_Config) -> {no, ?MSG(4)}, {hi, ?MSG(5)} ]), + + ?assertEqual([1,2,3,4,5], lists:sort(rabbit_fifo_q:indexes(Q1))), {?MSG(1), Q2} = rabbit_fifo_q:out(Q1), {?MSG(3), Q3} = rabbit_fifo_q:out(Q2), {?MSG(2), Q4} = rabbit_fifo_q:out(Q3), From 8edccae0a5245926757ca4ed90fbcd850d97e17d Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 22 Sep 2025 11:38:16 +0100 Subject: [PATCH 04/28] QQ: add v8 modules And tidy up some backwards compatible code that is no longer needed. --- deps/rabbit/src/rabbit_fifo.erl | 494 +--- deps/rabbit/src/rabbit_fifo_client.erl | 37 +- deps/rabbit/src/rabbit_fifo_v7.erl | 3120 ++++++++++++++++++++ deps/rabbit/src/rabbit_fifo_v7.hrl | 232 ++ deps/rabbit/src/rabbit_quorum_queue.erl | 3 +- deps/rabbit/test/rabbit_fifo_SUITE.erl | 36 +- deps/rabbit/test/rabbit_fifo_int_SUITE.erl | 21 +- 7 files changed, 3515 insertions(+), 428 deletions(-) create mode 100644 deps/rabbit/src/rabbit_fifo_v7.erl create mode 100644 deps/rabbit/src/rabbit_fifo_v7.hrl diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 6ea5ef499512..6bf0e441a4aa 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -71,7 +71,6 @@ query_peek/2, query_notify_decorators_info/1, usage/1, - is_v4/0, %% misc get_msg_header/1, @@ -144,7 +143,6 @@ -record(purge_nodes, {nodes :: [node()]}). -record(update_config, {config :: config()}). -record(garbage_collection, {}). -% -record(eval_consumer_timeouts, {consumer_keys :: [consumer_key()]}). -opaque protocol() :: #enqueue{} | @@ -187,8 +185,6 @@ state/0, config/0]). -%% This function is never called since only rabbit_fifo_v0:init/1 is called. -%% See https://github.com/rabbitmq/ra/blob/e0d1e6315a45f5d3c19875d66f9d7bfaf83a46e3/src/ra_machine.erl#L258-L265 -spec init(config()) -> state(). init(#{name := Name, queue_resource := Resource} = Conf) -> @@ -232,7 +228,6 @@ update_config(Conf, State) -> last_active = LastActive}. % msg_ids are scoped per consumer -% ra_indexes holds all raft indexes for enqueues currently on queue -spec apply(ra_machine:command_meta_data(), command(), state()) -> {state(), ra_machine:reply(), ra_machine:effects() | ra_machine:effect()} | {state(), ra_machine:reply()}. @@ -242,6 +237,9 @@ apply(Meta, #enqueue{pid = From, seq = Seq, apply(#{reply_mode := {notify, _Corr, EnqPid}} = Meta, #?ENQ_V2{seq = Seq, msg = RawMsg, size = Size}, State00) -> apply_enqueue(Meta, EnqPid, Seq, RawMsg, Size, State00); +apply(Meta, #?ENQ_V2{seq = Seq, msg = RawMsg, size = Size}, State00) -> + %% untracked + apply_enqueue(Meta, undefined, Seq, RawMsg, Size, State00); apply(_Meta, #register_enqueuer{pid = Pid}, #?STATE{enqueuers = Enqueuers0, cfg = #cfg{overflow_strategy = Overflow}} = State0) -> @@ -271,18 +269,6 @@ apply(Meta, #settle{msg_ids = MsgIds, _ -> {State, ok} end; -apply(#{machine_version := 4} = Meta, - #discard{consumer_key = ConsumerKey, - msg_ids = MsgIds}, - #?STATE{consumers = Consumers } = State0) -> - %% buggy version that would have not found the consumer if the ConsumerKey - %% was a consumer_id() - case find_consumer(ConsumerKey, Consumers) of - {ConsumerKey, #consumer{} = Con} -> - discard(Meta, MsgIds, ConsumerKey, Con, true, #{}, State0); - _ -> - {State0, ok} - end; apply(Meta, #discard{consumer_key = ConsumerKey, msg_ids = MsgIds}, #?STATE{consumers = Consumers } = State0) -> @@ -320,8 +306,7 @@ apply(Meta, #modify{consumer_key = ConsumerKey, _ -> {State, ok} end; -apply(#{index := Idx, - machine_version := MacVer} = Meta, +apply(#{index := Idx} = Meta, #requeue{consumer_key = ConsumerKey, msg_id = MsgId, index = OldIdx, @@ -348,12 +333,7 @@ apply(#{index := Idx, Messages), enqueue_count = EnqCount + 1}, State2 = update_or_remove_con(Meta, ConsumerKey, Con, State1), - {State3, Effects} = case MacVer >= 7 of - true -> - activate_next_consumer({State2, []}); - false -> - {State2, []} - end, + {State3, Effects} = activate_next_consumer({State2, []}), checkout(Meta, State0, State3, Effects); _ -> {State00, ok, []} @@ -526,8 +506,7 @@ apply(#{index := _Idx}, #garbage_collection{}, State) -> {State, ok, [{aux, garbage_collection}]}; apply(Meta, {timeout, expire_msgs}, State) -> checkout(Meta, State, State, []); -apply(#{machine_version := Vsn, - system_time := Ts} = Meta, +apply(#{system_time := Ts} = Meta, {down, Pid, noconnection}, #?STATE{consumers = Cons0, cfg = #cfg{consumer_strategy = single_active}, @@ -537,7 +516,7 @@ apply(#{machine_version := Vsn, %% if the pid refers to an active or cancelled consumer, %% mark it as suspected and return it to the waiting queue {State1, Effects0} = - rabbit_fifo_maps:fold( + maps:fold( fun(CKey, ?CONSUMER_PID(P) = C0, {S0, E0}) when node(P) =:= Node -> %% the consumer should be returned to waiting @@ -559,7 +538,7 @@ apply(#{machine_version := Vsn, Effs1}; (_, _, S) -> S - end, {State0, []}, Cons0, Vsn), + end, {State0, []}, maps:iterator(Cons0, ordered)), WaitingConsumers = update_waiting_consumer_status(Node, State1, suspected_down), @@ -574,8 +553,7 @@ apply(#{machine_version := Vsn, end, Enqs0), Effects = [{monitor, node, Node} | Effects1], checkout(Meta, State0, State#?STATE{enqueuers = Enqs}, Effects); -apply(#{machine_version := Vsn, - system_time := Ts} = Meta, +apply(#{system_time := Ts} = Meta, {down, Pid, noconnection}, #?STATE{consumers = Cons0, enqueuers = Enqs0} = State0) -> @@ -590,7 +568,7 @@ apply(#{machine_version := Vsn, Node = node(Pid), {State, Effects1} = - rabbit_fifo_maps:fold( + maps:fold( fun(CKey, #consumer{cfg = #consumer_cfg{pid = P}, status = up} = C0, {St0, Eff}) when node(P) =:= Node -> @@ -601,7 +579,7 @@ apply(#{machine_version := Vsn, {St, Eff1}; (_, _, {St, Eff}) -> {St, Eff} - end, {State0, []}, Cons0, Vsn), + end, {State0, []}, maps:iterator(Cons0, ordered)), Enqs = maps:map(fun(P, E) when node(P) =:= Node -> E#enqueuer{status = suspected_down}; (_, E) -> E @@ -617,17 +595,14 @@ apply(#{machine_version := Vsn, apply(Meta, {down, Pid, _Info}, State0) -> {State1, Effects1} = activate_next_consumer(handle_down(Meta, Pid, State0)), checkout(Meta, State0, State1, Effects1); -apply(#{machine_version := Vsn} = Meta, - {nodeup, Node}, - #?STATE{consumers = Cons0, - enqueuers = Enqs0, - service_queue = _SQ0} = State0) -> +apply(Meta, {nodeup, Node}, #?STATE{consumers = Cons0, + enqueuers = Enqs0} = State0) -> %% A node we are monitoring has come back. %% If we have suspected any processes of being %% down we should now re-issue the monitors for them to detect if they're %% actually down or not Monitors = [{monitor, process, P} - || P <- suspected_pids_for(Node, Vsn, State0)], + || P <- suspected_pids_for(Node, State0)], Enqs1 = maps:map(fun(P, E) when node(P) =:= Node -> E#enqueuer{status = up}; @@ -644,7 +619,7 @@ apply(#{machine_version := Vsn} = Meta, ConsumerUpdateActiveFun = consumer_active_flag_update_function(State0), %% mark all consumers as up {State1, Effects1} = - rabbit_fifo_maps:fold( + maps:fold( fun(ConsumerKey, ?CONSUMER_PID(P) = C, {SAcc, EAcc}) when (node(P) =:= Node) and (C#consumer.status =/= cancelled) -> @@ -655,7 +630,7 @@ apply(#{machine_version := Vsn} = Meta, SAcc), EAcc1}; (_, _, Acc) -> Acc - end, {State0, Effects0}, Cons0, Vsn), + end, {State0, Effects0}, maps:iterator(Cons0, ordered)), Waiting = update_waiting_consumer_status(Node, State1, up), State2 = State1#?STATE{enqueuers = Enqs1, waiting_consumers = Waiting}, @@ -708,10 +683,8 @@ live_indexes(#?STATE{cfg = #cfg{}, end, Acc0, Ch) end, RtnIndexes ++ DlxIndexes, Consumers), - CheckedIdxs ++ rabbit_fifo_q:indexes(Messages). - -spec snapshot_installed(Meta, State, OldMeta, OldState) -> ra_machine:effects() when Meta :: ra_snapshot:meta(), @@ -748,58 +721,20 @@ snapshot_installed(_Meta, #?MODULE{cfg = #cfg{resource = QR}, -convert_v3_to_v4(#{} = _Meta, StateV3) -> - %% TODO: consider emitting release cursors as checkpoints - Messages0 = rabbit_fifo_v3:get_field(messages, StateV3), - Returns0 = lqueue:to_list(rabbit_fifo_v3:get_field(returns, StateV3)), - Consumers0 = rabbit_fifo_v3:get_field(consumers, StateV3), - Consumers = maps:map( - fun (_, #consumer{checked_out = Ch0} = C) -> - Ch = maps:map( - fun (_, ?MSG(I, #{delivery_count := DC} = H)) -> - ?MSG(I, H#{acquired_count => DC}); - (_, Msg) -> - Msg - end, Ch0), - C#consumer{checked_out = Ch} - end, Consumers0), - Returns = lqueue:from_list( - lists:map(fun (?MSG(I, #{delivery_count := DC} = H)) -> - ?MSG(I, H#{acquired_count => DC}); - (Msg) -> - Msg - end, Returns0)), - - Messages = rabbit_fifo_q:from_lqueue(Messages0), - Cfg = rabbit_fifo_v3:get_field(cfg, StateV3), - #?STATE{cfg = Cfg#cfg{unused_1 = ?NIL}, - messages = Messages, - messages_total = rabbit_fifo_v3:get_field(messages_total, StateV3), - returns = Returns, - enqueue_count = rabbit_fifo_v3:get_field(enqueue_count, StateV3), - enqueuers = rabbit_fifo_v3:get_field(enqueuers, StateV3), - ra_indexes = rabbit_fifo_v3:get_field(ra_indexes, StateV3), - consumers = Consumers, - service_queue = rabbit_fifo_v3:get_field(service_queue, StateV3), - dlx = rabbit_fifo_v3:get_field(dlx, StateV3), - msg_bytes_enqueue = rabbit_fifo_v3:get_field(msg_bytes_enqueue, StateV3), - msg_bytes_checkout = rabbit_fifo_v3:get_field(msg_bytes_checkout, StateV3), - waiting_consumers = rabbit_fifo_v3:get_field(waiting_consumers, StateV3), - last_active = rabbit_fifo_v3:get_field(last_active, StateV3), - msg_cache = rabbit_fifo_v3:get_field(msg_cache, StateV3), - unused_1 = []}. - -purge_node(#{machine_version := Vsn} = Meta, Node, State, Effects) -> +convert_v7_to_v8(#{} = _Meta, StateV7) -> + StateV8 = StateV7, + StateV8. + +purge_node(Meta, Node, State, Effects) -> lists:foldl(fun(Pid, {S0, E0}) -> {S, E} = handle_down(Meta, Pid, S0), {S, E0 ++ E} end, {State, Effects}, - all_pids_for(Node, Vsn, State)). + all_pids_for(Node, State)). %% any downs that are not noconnection -handle_down(#{machine_version := Vsn} = Meta, - Pid, #?STATE{consumers = Cons0, - enqueuers = Enqs0} = State0) -> +handle_down(Meta, Pid, #?STATE{consumers = Cons0, + enqueuers = Enqs0} = State0) -> % Remove any enqueuer for the down pid State1 = State0#?STATE{enqueuers = maps:remove(Pid, Enqs0)}, {Effects1, State2} = handle_waiting_consumer_down(Pid, State1), @@ -808,7 +743,7 @@ handle_down(#{machine_version := Vsn} = Meta, DownConsumers = maps:filter(fun(_CKey, ?CONSUMER_PID(P)) -> P =:= Pid end, Cons0), - DownConsumerKeys = rabbit_fifo_maps:keys(DownConsumers, Vsn), + DownConsumerKeys = maps_ordered_keys(DownConsumers), lists:foldl(fun(ConsumerKey, {S, E}) -> cancel_consumer(Meta, ConsumerKey, S, E, down) end, {State2, Effects1}, DownConsumerKeys). @@ -928,7 +863,7 @@ tick(Ts, #?STATE{cfg = #cfg{resource = QName}} = State) -> -spec overview(state()) -> map(). overview(#?STATE{consumers = Cons, enqueuers = Enqs, - enqueue_count = EnqCount, + % enqueue_count = EnqCount, msg_bytes_enqueue = EnqueueBytes, msg_bytes_checkout = CheckoutBytes, cfg = Cfg, @@ -973,15 +908,10 @@ overview(#?STATE{consumers = Cons, num_ready_messages_normal => MsgsNo, num_ready_messages_return => MsgsRet, num_messages => messages_total(State), - num_release_cursors => 0, %% backwards compat enqueue_message_bytes => EnqueueBytes, - checkout_message_bytes => CheckoutBytes, - release_cursors => [], %% backwards compat - in_memory_message_bytes => 0, %% backwards compat - num_in_memory_ready_messages => 0, %% backwards compat - release_cursor_enqueue_counter => EnqCount, - smallest_raft_index => smallest_raft_index(State), - smallest_raft_index_overview => smallest_raft_index_overview(State) + checkout_message_bytes => CheckoutBytes + % smallest_raft_index => smallest_raft_index(State), + % smallest_raft_index_overview => smallest_raft_index_overview(State) }, DlxOverview = rabbit_fifo_dlx:overview(DlxState), maps:merge(maps:merge(Overview, DlxOverview), SacOverview). @@ -1006,20 +936,14 @@ which_module(0) -> rabbit_fifo_v0; which_module(1) -> rabbit_fifo_v1; which_module(2) -> rabbit_fifo_v3; which_module(3) -> rabbit_fifo_v3; -which_module(4) -> ?MODULE; -which_module(5) -> ?MODULE; -which_module(6) -> ?MODULE; -which_module(7) -> ?MODULE; +which_module(4) -> rabbit_fifo_v7; +which_module(5) -> rabbit_fifo_v7; +which_module(6) -> rabbit_fifo_v7; +which_module(7) -> rabbit_fifo_v7; which_module(8) -> ?MODULE. --define(AUX, aux_v3). +-define(AUX, aux_v4). --record(checkpoint, {index :: ra:index(), - timestamp :: milliseconds(), - smallest_index :: undefined | ra:index(), - messages_total :: non_neg_integer(), - indexes = ?CHECK_MIN_INDEXES :: non_neg_integer(), - bytes_in = 0 :: non_neg_integer()}). -record(snapshot, {index :: ra:index(), timestamp :: milliseconds(), % smallest_index :: undefined | ra:index(), @@ -1027,16 +951,13 @@ which_module(8) -> ?MODULE. % indexes = ?CHECK_MIN_INDEXES :: non_neg_integer(), bytes_out = 0 :: non_neg_integer()}). -record(aux_gc, {last_raft_idx = 0 :: ra:index()}). --record(aux, {name :: atom(), - capacity :: term(), - gc = #aux_gc{} :: #aux_gc{}}). -record(?AUX, {name :: atom(), last_decorators_state :: term(), - capacity :: term(), + unused_1 :: term(), gc = #aux_gc{} :: #aux_gc{}, tick_pid :: undefined | pid(), cache = #{} :: map(), - last_checkpoint :: #checkpoint{} | #snapshot{}, + last_checkpoint :: tuple() | #snapshot{}, bytes_in = 0 :: non_neg_integer(), bytes_out = 0 :: non_neg_integer()}). @@ -1045,26 +966,30 @@ init_aux(Name) when is_atom(Name) -> ok = ra_machine_ets:create_table(rabbit_fifo_usage, [named_table, set, public, {write_concurrency, true}]), - Now = erlang:monotonic_time(microsecond), #?AUX{name = Name, - capacity = {inactive, Now, 1, 1.0}, - last_checkpoint = #checkpoint{index = 0, - timestamp = erlang:system_time(millisecond), - messages_total = 0}}. - -handle_aux(RaftState, Tag, Cmd, #aux{name = Name, - capacity = Cap, - gc = Gc}, RaAux) -> - %% convert aux state to new version - AuxV2 = init_aux(Name), - Aux = AuxV2#?AUX{capacity = Cap, - gc = Gc}, - handle_aux(RaftState, Tag, Cmd, Aux, RaAux); + last_checkpoint = #snapshot{index = 0, + timestamp = erlang:system_time(millisecond), + messages_total = 0}}. + handle_aux(RaftState, Tag, Cmd, AuxV2, RaAux) when element(1, AuxV2) == aux_v2 -> Name = element(2, AuxV2), AuxV3 = init_aux(Name), handle_aux(RaftState, Tag, Cmd, AuxV3, RaAux); +handle_aux(RaftState, Tag, Cmd, AuxV3, RaAux) + when element(1, AuxV3) == aux_v3 -> + Name = element(2, AuxV3), + AuxV3 = init_aux(Name), + AuxV4 = #?AUX{name = element(2, AuxV3), + last_decorators_state = element(3, AuxV3), + unused_1 = undefined, + gc = element(5, AuxV3), + tick_pid = element(6, AuxV3), + cache = element(7, AuxV3), + last_checkpoint = element(8, AuxV3), + bytes_in = element(9, AuxV3), + bytes_out = 0}, + handle_aux(RaftState, Tag, Cmd, AuxV4, RaAux); handle_aux(leader, cast, eval, #?AUX{last_decorators_state = LastDec, bytes_in = BytesIn, @@ -1082,7 +1007,7 @@ handle_aux(leader, cast, eval, %% this is a guess undefined end, - {Check, Effects0} = do_checkpoints(EffMacVer, Ts, Check0, RaAux, + {Check, Effects0} = do_snapshot(EffMacVer, Ts, Check0, RaAux, BytesIn, BytesOut, false), %% this is called after each batch of commands have been applied @@ -1106,7 +1031,7 @@ handle_aux(_RaftState, cast, eval, RaAux) -> Ts = erlang:system_time(millisecond), EffMacVer = ra_aux:effective_machine_version(RaAux), - {Check, Effects} = do_checkpoints(EffMacVer, Ts, Check0, RaAux, + {Check, Effects} = do_snapshot(EffMacVer, Ts, Check0, RaAux, BytesIn, BytesOut, false), {no_reply, Aux0#?AUX{last_checkpoint = Check}, RaAux, Effects}; handle_aux(_RaftState, cast, {bytes_in, {MetaSize, BodySize}}, @@ -1187,22 +1112,10 @@ handle_aux(_, _, {get_checked_out, ConsumerKey, MsgIds}, Aux0, RaAux0) -> _ -> {reply, {error, consumer_not_found}, Aux0, RaAux0} end; -handle_aux(_RaState, cast, Cmd, #?AUX{capacity = Use0} = Aux0, RaAux) - when Cmd == active orelse Cmd == inactive -> - {no_reply, Aux0#?AUX{capacity = update_use(Use0, Cmd)}, RaAux}; -handle_aux(_RaState, cast, tick, #?AUX{name = Name, - capacity = Use0} = State0, +handle_aux(_RaState, cast, tick, #?AUX{name = _Name} = State0, RaAux) -> - true = ets:insert(rabbit_fifo_usage, - {Name, capacity(Use0)}), Aux = eval_gc(RaAux, ra_aux:machine_state(RaAux), State0), - Effs = case smallest_raft_index(ra_aux:machine_state(RaAux)) of - undefined -> - [{release_cursor, ra_aux:last_applied(RaAux)}]; - Smallest -> - [{release_cursor, Smallest - 1}] - end, - {no_reply, Aux, RaAux, Effs}; + {no_reply, Aux, RaAux, []}; handle_aux(_RaState, cast, eol, #?AUX{name = Name} = Aux, RaAux) -> ets:delete(rabbit_fifo_usage, Name), {no_reply, Aux, RaAux}; @@ -1254,7 +1167,7 @@ handle_aux(_RaState, _, force_checkpoint, ?LOG_DEBUG("~ts: rabbit_fifo: forcing checkpoint at ~b", [rabbit_misc:rs(QR), ra_aux:last_applied(RaAux)]), EffMacVer = ra_aux:effective_machine_version(RaAux), - {Check, Effects} = do_checkpoints(EffMacVer, Ts, Check0, RaAux, + {Check, Effects} = do_snapshot(EffMacVer, Ts, Check0, RaAux, BytesIn, BytesOut, true), {no_reply, Aux#?AUX{last_checkpoint = Check}, RaAux, Effects}; handle_aux(RaState, _, {dlx, _} = Cmd, Aux0, RaAux) -> @@ -1462,11 +1375,6 @@ usage(Name) when is_atom(Name) -> [{_, Use}] -> Use end. --spec is_v4() -> boolean(). -is_v4() -> - %% Quorum queue v4 is introduced in RabbitMQ 4.0.0 - rabbit_feature_flags:is_enabled('rabbitmq_4.0.0'). - %%% Internal messages_ready(#?STATE{messages = M, @@ -1478,36 +1386,6 @@ messages_total(#?STATE{messages_total = Total, {DlxTotal, _} = rabbit_fifo_dlx:stat(DlxState), Total + DlxTotal. -update_use({inactive, _, _, _} = CUInfo, inactive) -> - CUInfo; -update_use({active, _, _} = CUInfo, active) -> - CUInfo; -update_use({active, Since, Avg}, inactive) -> - Now = erlang:monotonic_time(microsecond), - {inactive, Now, Now - Since, Avg}; -update_use({inactive, Since, Active, Avg}, active) -> - Now = erlang:monotonic_time(microsecond), - {active, Now, use_avg(Active, Now - Since, Avg)}. - -capacity({active, Since, Avg}) -> - use_avg(erlang:monotonic_time(microsecond) - Since, 0, Avg); -capacity({inactive, _, 1, 1.0}) -> - 1.0; -capacity({inactive, Since, Active, Avg}) -> - use_avg(Active, erlang:monotonic_time(microsecond) - Since, Avg). - -use_avg(0, 0, Avg) -> - Avg; -use_avg(Active, Inactive, Avg) -> - Time = Inactive + Active, - moving_average(Time, ?USE_AVG_HALF_LIFE, Active / Time, Avg). - -moving_average(_Time, _, Next, undefined) -> - Next; -moving_average(Time, HalfLife, Next, Current) -> - Weight = math:exp(Time * math:log(0.5) / HalfLife), - Next * (1 - Weight) + Current * Weight. - num_checked_out(#?STATE{consumers = Cons}) -> maps:fold(fun (_, #consumer{checked_out = C}, Acc) -> maps:size(C) + Acc @@ -1643,7 +1521,7 @@ activate_next_consumer(#?STATE{consumers = Cons0, waiting_consumers = Waiting}, Effects1 = consumer_update_active_effects(State, Active, false, waiting, - Effects0), + Effects0), Effects = consumer_update_active_effects(State, Consumer, true, single_active, Effects1), @@ -1850,7 +1728,7 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, {duplicate, State0, Effects0} end. -return(#{machine_version := MacVer} = Meta, ConsumerKey, +return(Meta, ConsumerKey, MsgIds, IncrDelCount, Anns, Checked, Effects0, State0) when is_map(Anns) -> %% We requeue in the same order as messages got returned by the client. @@ -1871,12 +1749,7 @@ return(#{machine_version := MacVer} = Meta, ConsumerKey, _ -> State1 end, - {State3, Effects2} = case MacVer >= 7 of - true -> - activate_next_consumer({State2, Effects1}); - false -> - {State2, Effects1} - end, + {State3, Effects2} = activate_next_consumer({State2, Effects1}), checkout(Meta, State0, State3, Effects2). % used to process messages that are finished @@ -2042,6 +1915,8 @@ return_one(Meta, MsgId, ?MSG(_, _) = Msg0, DelivFailed, Anns, #consumer{checked_out = Checked0} = Con0 = maps:get(ConsumerKey, Consumers), Msg = incr_msg(Msg0, DelivFailed, Anns), Header = get_msg_header(Msg), + %% TODO: do not use acquired count here as that includes all deliberate + %% returns, use delivery_count header instead case get_header(acquired_count, Header) of AcquiredCount when AcquiredCount > DeliveryLimit -> {DlxState, DlxEffects} = @@ -2694,17 +2569,15 @@ is_below(Val, Num) when is_integer(Val) andalso is_integer(Num) -> -spec make_enqueue(option(pid()), option(msg_seqno()), raw_msg()) -> protocol(). -make_enqueue(Pid, Seq, Msg) -> - case is_v4() of - true when is_pid(Pid) andalso - is_integer(Seq) -> - %% more compact format - #?ENQ_V2{seq = Seq, - msg = Msg, - size = ?SIZE(Msg)}; - _ -> - #enqueue{pid = Pid, seq = Seq, msg = Msg} - end. +make_enqueue(undefined, undefined, Msg) -> + %% need to keep this old version for untracked enqueues + #enqueue{msg = Msg}; +make_enqueue(Pid, Seq, Msg) + when is_pid(Pid) andalso + is_integer(Seq) -> + #?ENQ_V2{seq = Seq, + msg = Msg, + size = ?SIZE(Msg)}. -spec make_register_enqueuer(pid()) -> protocol(). make_register_enqueuer(Pid) -> @@ -2712,14 +2585,7 @@ make_register_enqueuer(Pid) -> -spec make_checkout(consumer_id(), checkout_spec(), consumer_meta()) -> protocol(). -make_checkout({_, _} = ConsumerId, Spec0, Meta) -> - Spec = case is_v4() of - false when Spec0 == remove -> - %% if v4 is not active, fall back to cancel spec - cancel; - _ -> - Spec0 - end, +make_checkout({_, _} = ConsumerId, Spec, Meta) -> #checkout{consumer_id = ConsumerId, spec = Spec, meta = Meta}. @@ -2754,18 +2620,11 @@ make_modify(ConsumerKey, MsgIds, DeliveryFailed, UndeliverableHere, Anns) is_boolean(DeliveryFailed) andalso is_boolean(UndeliverableHere) andalso is_map(Anns) -> - case is_v4() of - true -> - #modify{consumer_key = ConsumerKey, - msg_ids = MsgIds, - delivery_failed = DeliveryFailed, - undeliverable_here = UndeliverableHere, - annotations = Anns}; - false when UndeliverableHere -> - make_discard(ConsumerKey, MsgIds); - false -> - make_return(ConsumerKey, MsgIds) - end. + #modify{consumer_key = ConsumerKey, + msg_ids = MsgIds, + delivery_failed = DeliveryFailed, + undeliverable_here = UndeliverableHere, + annotations = Anns}. -spec make_purge() -> protocol(). @@ -2820,45 +2679,45 @@ all_nodes(#?STATE{consumers = Cons0, Acc#{node(P) => ok} end, Nodes1, WaitingConsumers0)). -all_pids_for(Node, Vsn, #?STATE{consumers = Cons0, - enqueuers = Enqs0, - waiting_consumers = WaitingConsumers0}) -> - Cons = rabbit_fifo_maps:fold(fun(_, ?CONSUMER_PID(P), Acc) - when node(P) =:= Node -> - [P | Acc]; - (_, _, Acc) -> - Acc - end, [], Cons0, Vsn), - Enqs = rabbit_fifo_maps:fold(fun(P, _, Acc) - when node(P) =:= Node -> - [P | Acc]; - (_, _, Acc) -> - Acc - end, Cons, Enqs0, Vsn), +all_pids_for(Node, #?STATE{consumers = Cons0, + enqueuers = Enqs0, + waiting_consumers = WaitingConsumers0}) -> + Cons = maps:fold(fun(_, ?CONSUMER_PID(P), Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, _, Acc) -> + Acc + end, [], maps:iterator(Cons0, ordered)), + Enqs = maps:fold(fun(P, _, Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, _, Acc) -> + Acc + end, Cons, maps:iterator(Enqs0, ordered)), lists:foldl(fun({_, ?CONSUMER_PID(P)}, Acc) when node(P) =:= Node -> [P | Acc]; (_, Acc) -> Acc end, Enqs, WaitingConsumers0). -suspected_pids_for(Node, Vsn, #?STATE{consumers = Cons0, - enqueuers = Enqs0, - waiting_consumers = WaitingConsumers0}) -> - Cons = rabbit_fifo_maps:fold(fun(_Key, - #consumer{cfg = #consumer_cfg{pid = P}, - status = suspected_down}, - Acc) +suspected_pids_for(Node, #?STATE{consumers = Cons0, + enqueuers = Enqs0, + waiting_consumers = WaitingConsumers0}) -> + Cons = maps:fold(fun(_Key, + #consumer{cfg = #consumer_cfg{pid = P}, + status = suspected_down}, + Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, _, Acc) -> + Acc + end, [], maps:iterator(Cons0, ordered)), + Enqs = maps:fold(fun(P, #enqueuer{status = suspected_down}, Acc) when node(P) =:= Node -> [P | Acc]; (_, _, Acc) -> Acc - end, [], Cons0, Vsn), - Enqs = rabbit_fifo_maps:fold(fun(P, #enqueuer{status = suspected_down}, Acc) - when node(P) =:= Node -> - [P | Acc]; - (_, _, Acc) -> - Acc - end, Cons, Enqs0, Vsn), + end, Cons, maps:iterator(Enqs0, ordered)), lists:foldl(fun({_Key, #consumer{cfg = #consumer_cfg{pid = P}, status = suspected_down}}, Acc) @@ -2912,7 +2771,7 @@ convert(Meta, 1, To, State) -> convert(Meta, 2, To, State) -> convert(Meta, 3, To, rabbit_fifo_v3:convert_v2_to_v3(State)); convert(Meta, 3, To, State) -> - convert(Meta, 4, To, convert_v3_to_v4(Meta, State)); + convert(Meta, 4, To, rabbit_fifo_v7:convert_v3_to_v4(Meta, State)); convert(Meta, 4, To, State) -> %% no conversion needed, this version only includes a logic change convert(Meta, 5, To, State); @@ -2923,23 +2782,32 @@ convert(Meta, 6, To, State) -> %% no conversion needed, this version only includes a logic change convert(Meta, 7, To, State); convert(Meta, 7, To, State) -> - %% no conversion needed, this version only includes a logic change - convert(Meta, 8, To, State). + convert(Meta, 8, To, convert_v7_to_v8(Meta, State)). smallest_raft_index(#?STATE{messages = Messages, - ra_indexes = Indexes, - dlx = DlxState}) -> + dlx = DlxState} = State) -> SmallestDlxRaIdx = rabbit_fifo_dlx:smallest_raft_index(DlxState), SmallestMsgsRaIdx = rabbit_fifo_q:get_lowest_index(Messages), - SmallestRaIdx = rabbit_fifo_index:smallest(Indexes), - min(min(SmallestDlxRaIdx, SmallestMsgsRaIdx), SmallestRaIdx). - -smallest_raft_index_overview(#?STATE{messages = Messages, - ra_indexes = Indexes, - dlx = DlxState}) -> - #{message => rabbit_fifo_q:get_lowest_index(Messages), - checked_out => rabbit_fifo_index:smallest(Indexes), - dlx => rabbit_fifo_dlx:smallest_raft_index(DlxState)}. + %% scan consumers and returns queue here instead + smallest_checked_out(State, min(SmallestDlxRaIdx, SmallestMsgsRaIdx)). + +smallest_checked_out(#?STATE{returns = Returns, + consumers = Consumers}, Min) -> + SmallestSoFar = lqueue:fold(fun (?MSG(I, _), Acc) -> + min(I, Acc) + end, Min, Returns), + maps:fold(fun (_Cid, #consumer{checked_out = Ch}, Acc0) -> + maps:fold( + fun (_MsgId, ?MSG(I, _), Acc) -> + min(I, Acc) + end, Acc0, Ch) + end, SmallestSoFar, Consumers). + +% smallest_raft_index_overview(#?STATE{messages = Messages, +% dlx = DlxState} = State) -> +% #{message => rabbit_fifo_q:get_lowest_index(Messages), +% checked_out => smallest_checked_out(State, undefined), +% dlx => rabbit_fifo_dlx:smallest_raft_index(DlxState)}. make_requeue(ConsumerKey, Notify, [{MsgId, Idx, Header, Msg}], Acc) -> lists:reverse([{append, @@ -3093,26 +2961,25 @@ priority_tag(Msg) -> no end. -do_checkpoints(MacVer, Ts, #checkpoint{timestamp = LastTs, - index = Idx}, +do_snapshot(MacVer, Ts, Ch, RaAux, BytesIn, BytesOut, Force) - when is_integer(MacVer) andalso MacVer >= 8 -> - do_checkpoints(MacVer, Ts, #snapshot{index = Idx, + when element(1, Ch) == checkpoint andalso + is_integer(MacVer) andalso + MacVer >= 8 -> + Idx = element(2, Ch), + LastTs = element(3, Ch), + do_snapshot(MacVer, Ts, #snapshot{index = Idx, timestamp = LastTs}, RaAux, BytesIn, BytesOut, Force); -do_checkpoints(MacVer, Ts, #snapshot{index = _ChIdx, - timestamp = SnapTime, - bytes_out = LastBytesOut} = Snap0, - RaAux, _BytesIn, BytesOut, Force) +do_snapshot(MacVer, Ts, #snapshot{index = _ChIdx, + timestamp = SnapTime, + bytes_out = LastBytesOut} = Snap0, + RaAux, _BytesIn, BytesOut, Force) when is_integer(MacVer) andalso MacVer >= 8 -> LastAppliedIdx = ra_aux:last_applied(RaAux), #?STATE{} = MacState = ra_aux:machine_state(RaAux), TimeSince = Ts - SnapTime, MsgsTot = messages_total(MacState), - % ra_aux:overview(RaAux), - % MaxBytesFactor = max(1, MsgsTot / CheckMaxIndexes), - % TODO: snapshots also need to be triggered by non settled commands - % that aren't enqueues EnoughDataRemoved = BytesOut - LastBytesOut > ?SNAP_OUT_BYTES, {CheckMinInterval, _CheckMinIndexes, _CheckMaxIndexes} = persistent_term:get(quorum_queue_checkpoint_config, @@ -3129,82 +2996,8 @@ do_checkpoints(MacVer, Ts, #snapshot{index = _ChIdx, [{release_cursor, LastAppliedIdx, MacState}]}; false -> {Snap0, []} - end; -do_checkpoints(MacVer,Ts, #checkpoint{index = ChIdx, - timestamp = ChTime, - smallest_index = LastSmallest, - bytes_in = LastBytesIn, - indexes = MinIndexes} = Check0, - RaAux, BytesIn, _BytesOut, Force) - when not is_integer(MacVer) orelse MacVer < 8 -> - LastAppliedIdx = ra_aux:last_applied(RaAux), - IndexesSince = LastAppliedIdx - ChIdx, - #?STATE{} = MacState = ra_aux:machine_state(RaAux), - TimeSince = Ts - ChTime, - NewSmallest = case smallest_raft_index(MacState) of - undefined -> - LastAppliedIdx; - Smallest -> - Smallest - end, - MsgsTot = messages_total(MacState), - %% more than 64MB (by default) of message data has been written to the log - %% best take a checkpoint - - {CheckMinInterval, CheckMinIndexes, CheckMaxIndexes} = - persistent_term:get(quorum_queue_checkpoint_config, - {?CHECK_MIN_INTERVAL_MS, ?CHECK_MIN_INDEXES, - ?CHECK_MAX_INDEXES}), - - %% scale the bytes limit as the backlog increases - MaxBytesFactor = max(1, MsgsTot / CheckMaxIndexes), - EnoughDataWritten = BytesIn - LastBytesIn > (?CHECK_MAX_BYTES * MaxBytesFactor), - EnoughTimeHasPassed = TimeSince > CheckMinInterval, - - case (EnoughTimeHasPassed andalso - ( - %% condition 1: enough indexes have been committed since the last - %% checkpoint - (IndexesSince > MinIndexes) orelse - %% condition 2: the queue is empty and _some_ commands - %% have been applied since the last checkpoint - (MsgsTot == 0 andalso IndexesSince > 32) - ) - ) orelse - %% condition 3: enough message data has been written to warrant a new - %% checkpoint, this ignores the time windowing - EnoughDataWritten orelse - %% force was requested, e.g. after a purge - Force - of - true -> - %% take fewer checkpoints the more messages there are on queue - NextIndexes = min(max(MsgsTot, CheckMinIndexes), CheckMaxIndexes), - %% take a checkpoint; - {#checkpoint{index = LastAppliedIdx, - timestamp = Ts, - smallest_index = NewSmallest, - messages_total = MsgsTot, - bytes_in = BytesIn, - indexes = NextIndexes}, - [{checkpoint, LastAppliedIdx, MacState} | - release_cursor(LastSmallest, NewSmallest)]}; - false -> - {Check0#checkpoint{smallest_index = NewSmallest}, - release_cursor(LastSmallest, NewSmallest)} end. -release_cursor(LastSmallest, Smallest) - when is_integer(LastSmallest) andalso - is_integer(Smallest) andalso - Smallest > LastSmallest -> - [{release_cursor, Smallest - 1}]; -release_cursor(undefined, Smallest) - when is_integer(Smallest) -> - [{release_cursor, Smallest - 1}]; -release_cursor(_, _) -> - []. - discard(Meta, MsgIds, ConsumerKey, #consumer{checked_out = Checked} = Con, DelFailed, Anns, @@ -3261,3 +3054,6 @@ exec_read(Flru0, ReadPlan, Msgs) -> %% retry without segment cache exec_read(undefined, ReadPlan, Msgs) end. + +maps_ordered_keys(Map) -> + lists:sort(maps:keys(Map)). diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index 44e8e29d68d9..922cdea09d14 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -373,24 +373,12 @@ checkout(ConsumerTag, CreditMode, #{} = Meta, is_tuple(CreditMode) -> Servers = sorted_servers(State0), ConsumerId = consumer_id(ConsumerTag), - Spec = case rabbit_fifo:is_v4() of - true -> - case CreditMode of - {simple_prefetch, 0} -> - {auto, {simple_prefetch, - ?UNLIMITED_PREFETCH_COUNT}}; - _ -> - {auto, CreditMode} - end; - false -> - case CreditMode of - {credited, _} -> - {auto, 0, credited}; - {simple_prefetch, 0} -> - {auto, ?UNLIMITED_PREFETCH_COUNT, simple_prefetch}; - {simple_prefetch, Num} -> - {auto, Num, simple_prefetch} - end + Spec = case CreditMode of + {simple_prefetch, 0} -> + {auto, {simple_prefetch, + ?UNLIMITED_PREFETCH_COUNT}}; + _ -> + {auto, CreditMode} end, Cmd = rabbit_fifo:make_checkout(ConsumerId, Spec, Meta), %% ??? @@ -415,13 +403,12 @@ checkout(ConsumerTag, CreditMode, #{} = Meta, end end, ConsumerKey = maps:get(key, Reply, ConsumerId), - SDels = maps:update_with( - ConsumerTag, - fun (C) -> C#consumer{ack = Ack} end, - #consumer{key = ConsumerKey, - last_msg_id = LastMsgId, - ack = Ack}, - CDels0), + SDels = maps:update_with(ConsumerTag, + fun (C) -> C#consumer{ack = Ack} end, + #consumer{key = ConsumerKey, + last_msg_id = LastMsgId, + ack = Ack}, + CDels0), {ok, Reply, State0#state{leader = Leader, consumers = SDels}}; Err -> diff --git a/deps/rabbit/src/rabbit_fifo_v7.erl b/deps/rabbit/src/rabbit_fifo_v7.erl new file mode 100644 index 000000000000..1c5a6ebc4112 --- /dev/null +++ b/deps/rabbit/src/rabbit_fifo_v7.erl @@ -0,0 +1,3120 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. + +-module(rabbit_fifo_v7). + +-behaviour(ra_machine). + +-compile(inline_list_funcs). +-compile(inline). +-compile({no_auto_import, [apply/3]}). +-dialyzer(no_improper_lists). + +-include("rabbit_fifo_v7.hrl"). +-include_lib("kernel/include/logger.hrl"). + +-define(STATE, rabbit_fifo). + +-define(CONSUMER_PID(Pid), #consumer{cfg = #consumer_cfg{pid = Pid}}). +-define(CONSUMER_PRIORITY(P), #consumer{cfg = #consumer_cfg{priority = P}}). +-define(CONSUMER_TAG_PID(Tag, Pid), + #consumer{cfg = #consumer_cfg{tag = Tag, + pid = Pid}}). + +-ifdef(TEST). +-define(SIZE(Msg), + case mc:is(Msg) of + true -> + mc:size(Msg); + false when is_binary(Msg) -> + {0, byte_size(Msg)}; + false -> + {0, erts_debug:size(Msg)} + end). +-else. +-define(SIZE(Msg), mc:size(Msg)). +-endif. + +-export([ + %% ra_machine callbacks + init/1, + apply/3, + state_enter/2, + tick/2, + overview/1, + + get_checked_out/4, + %% versioning + version/0, + which_module/1, + %% aux + init_aux/1, + handle_aux/5, + % queries + query_messages_ready/1, + query_messages_checked_out/1, + query_messages_total/1, + query_processes/1, + query_ra_indexes/1, + query_waiting_consumers/1, + query_consumer_count/1, + query_consumers/1, + query_stat/1, + query_stat_dlx/1, + query_single_active_consumer/1, + query_in_memory_usage/1, + query_peek/2, + query_notify_decorators_info/1, + usage/1, + is_v4/0, + + %% misc + get_msg_header/1, + get_header/2, + annotate_msg/2, + get_msg/1, + + %% protocol helpers + make_enqueue/3, + make_register_enqueuer/1, + make_checkout/3, + make_settle/2, + make_return/2, + is_return/1, + make_discard/2, + make_credit/4, + make_modify/5, + make_purge/0, + make_purge_nodes/1, + make_update_config/1, + make_garbage_collection/0, + + exec_read/3, + convert_v3_to_v4/2 + + ]). + +-ifdef(TEST). +-export([update_header/4, + chunk_disk_msgs/3, + smallest_raft_index/1, + make_requeue/4]). +-endif. + +-import(serial_number, [add/2, diff/2]). +-define(ENQ_V2, e). + +%% command records representing all the protocol actions that are supported +-record(enqueue, {pid :: option(pid()), + seq :: option(msg_seqno()), + msg :: raw_msg()}). +-record(?ENQ_V2, {seq :: option(msg_seqno()), + msg :: raw_msg(), + size :: {MetadataSize :: non_neg_integer(), + PayloadSize :: non_neg_integer()}}). +-record(requeue, {consumer_key :: consumer_key(), + msg_id :: msg_id(), + index :: ra:index(), + header :: msg_header(), + msg :: raw_msg()}). +-record(register_enqueuer, {pid :: pid()}). +-record(checkout, {consumer_id :: consumer_id(), + spec :: checkout_spec(), + meta :: consumer_meta()}). +-record(settle, {consumer_key :: consumer_key(), + msg_ids :: [msg_id()]}). +-record(return, {consumer_key :: consumer_key(), + msg_ids :: [msg_id()]}). +-record(discard, {consumer_key :: consumer_key(), + msg_ids :: [msg_id()]}). +-record(credit, {consumer_key :: consumer_key(), + credit :: non_neg_integer(), + delivery_count :: rabbit_queue_type:delivery_count(), + drain :: boolean()}). +-record(modify, {consumer_key :: consumer_key(), + msg_ids :: [msg_id()], + delivery_failed :: boolean(), + undeliverable_here :: boolean(), + annotations :: mc:annotations()}). +-record(purge, {}). +-record(purge_nodes, {nodes :: [node()]}). +-record(update_config, {config :: config()}). +-record(garbage_collection, {}). +% -record(eval_consumer_timeouts, {consumer_keys :: [consumer_key()]}). + +-opaque protocol() :: + #enqueue{} | + #?ENQ_V2{} | + #requeue{} | + #register_enqueuer{} | + #checkout{} | + #settle{} | + #return{} | + #discard{} | + #credit{} | + #modify{} | + #purge{} | + #purge_nodes{} | + #update_config{} | + #garbage_collection{}. + +-type command() :: protocol() | + rabbit_fifo_dlx:protocol() | + ra_machine:builtin_command(). +%% all the command types supported by ra fifo + +-type client_msg() :: delivery(). +%% the messages `rabbit_fifo' can send to consumers. + +-opaque state() :: #?STATE{}. + +-export_type([protocol/0, + delivery/0, + command/0, + credit_mode/0, + consumer_meta/0, + consumer_id/0, + consumer_key/0, + client_msg/0, + msg/0, + msg_id/0, + msg_seqno/0, + delivery_msg/0, + state/0, + config/0]). + +%% This function is never called since only rabbit_fifo_v0:init/1 is called. +%% See https://github.com/rabbitmq/ra/blob/e0d1e6315a45f5d3c19875d66f9d7bfaf83a46e3/src/ra_machine.erl#L258-L265 +-spec init(config()) -> state(). +init(#{name := Name, + queue_resource := Resource} = Conf) -> + update_config(Conf, #?STATE{cfg = #cfg{name = Name, + resource = Resource}}). + +update_config(Conf, State) -> + DLH = maps:get(dead_letter_handler, Conf, undefined), + BLH = maps:get(become_leader_handler, Conf, undefined), + Overflow = maps:get(overflow_strategy, Conf, drop_head), + MaxLength = maps:get(max_length, Conf, undefined), + MaxBytes = maps:get(max_bytes, Conf, undefined), + DeliveryLimit = case maps:get(delivery_limit, Conf, undefined) of + DL when is_number(DL) andalso + DL < 0 -> + undefined; + DL -> + DL + end, + + Expires = maps:get(expires, Conf, undefined), + MsgTTL = maps:get(msg_ttl, Conf, undefined), + ConsumerStrategy = case maps:get(single_active_consumer_on, Conf, false) of + true -> + single_active; + false -> + competing + end, + Cfg = State#?STATE.cfg, + + LastActive = maps:get(created, Conf, undefined), + State#?STATE{cfg = Cfg#cfg{dead_letter_handler = DLH, + become_leader_handler = BLH, + overflow_strategy = Overflow, + max_length = MaxLength, + max_bytes = MaxBytes, + consumer_strategy = ConsumerStrategy, + delivery_limit = DeliveryLimit, + expires = Expires, + msg_ttl = MsgTTL}, + last_active = LastActive}. + +% msg_ids are scoped per consumer +% ra_indexes holds all raft indexes for enqueues currently on queue +-spec apply(ra_machine:command_meta_data(), command(), state()) -> + {state(), ra_machine:reply(), ra_machine:effects() | ra_machine:effect()} | + {state(), ra_machine:reply()}. +apply(Meta, #enqueue{pid = From, seq = Seq, + msg = RawMsg}, State00) -> + apply_enqueue(Meta, From, Seq, RawMsg, message_size(RawMsg), State00); +apply(#{reply_mode := {notify, _Corr, EnqPid}} = Meta, + #?ENQ_V2{seq = Seq, msg = RawMsg, size = Size}, State00) -> + apply_enqueue(Meta, EnqPid, Seq, RawMsg, Size, State00); +apply(_Meta, #register_enqueuer{pid = Pid}, + #?STATE{enqueuers = Enqueuers0, + cfg = #cfg{overflow_strategy = Overflow}} = State0) -> + State = case maps:is_key(Pid, Enqueuers0) of + true -> + %% if the enqueuer exits just echo the overflow state + State0; + false -> + State0#?STATE{enqueuers = Enqueuers0#{Pid => #enqueuer{}}} + end, + Res = case is_over_limit(State) of + true when Overflow == reject_publish -> + reject_publish; + _ -> + ok + end, + {State, Res, [{monitor, process, Pid}]}; +apply(Meta, #settle{msg_ids = MsgIds, + consumer_key = Key}, + #?STATE{consumers = Consumers} = State) -> + case find_consumer(Key, Consumers) of + {ConsumerKey, Con0} -> + %% find_consumer/2 returns the actual consumer key even if + %% if id was passed instead for example + complete_and_checkout(Meta, MsgIds, ConsumerKey, + Con0, [], State); + _ -> + {State, ok} + end; +apply(#{machine_version := 4} = Meta, + #discard{consumer_key = ConsumerKey, + msg_ids = MsgIds}, + #?STATE{consumers = Consumers } = State0) -> + %% buggy version that would have not found the consumer if the ConsumerKey + %% was a consumer_id() + case find_consumer(ConsumerKey, Consumers) of + {ConsumerKey, #consumer{} = Con} -> + discard(Meta, MsgIds, ConsumerKey, Con, true, #{}, State0); + _ -> + {State0, ok} + end; +apply(Meta, #discard{consumer_key = ConsumerKey, + msg_ids = MsgIds}, + #?STATE{consumers = Consumers } = State0) -> + case find_consumer(ConsumerKey, Consumers) of + {ActualConsumerKey, #consumer{} = Con} -> + discard(Meta, MsgIds, ActualConsumerKey, Con, true, #{}, State0); + _ -> + {State0, ok} + end; +apply(Meta, #return{consumer_key = ConsumerKey, + msg_ids = MsgIds}, + #?STATE{consumers = Cons} = State) -> + case find_consumer(ConsumerKey, Cons) of + {ActualConsumerKey, #consumer{checked_out = Checked}} -> + return(Meta, ActualConsumerKey, MsgIds, false, + #{}, Checked, [], State); + _ -> + {State, ok} + end; +apply(Meta, #modify{consumer_key = ConsumerKey, + delivery_failed = DelFailed, + undeliverable_here = Undel, + annotations = Anns, + msg_ids = MsgIds}, + #?STATE{consumers = Cons} = State) -> + case find_consumer(ConsumerKey, Cons) of + {ActualConsumerKey, #consumer{checked_out = Checked}} + when Undel == false -> + return(Meta, ActualConsumerKey, MsgIds, DelFailed, + Anns, Checked, [], State); + {ActualConsumerKey, #consumer{} = Con} + when Undel == true -> + discard(Meta, MsgIds, ActualConsumerKey, + Con, DelFailed, Anns, State); + _ -> + {State, ok} + end; +apply(#{index := Idx, + machine_version := MacVer} = Meta, + #requeue{consumer_key = ConsumerKey, + msg_id = MsgId, + index = OldIdx, + header = Header0}, + #?STATE{consumers = Cons, + messages = Messages, + ra_indexes = Indexes0, + enqueue_count = EnqCount} = State00) -> + %% the actual consumer key was looked up in the aux handler so we + %% dont need to use find_consumer/2 here + case Cons of + #{ConsumerKey := #consumer{checked_out = Checked0} = Con0} + when is_map_key(MsgId, Checked0) -> + %% construct a message with the current raft index + %% and update acquired count before adding it to the message queue + Header = update_header(acquired_count, fun incr/1, 1, Header0), + State0 = add_bytes_return(Header, State00), + Con = Con0#consumer{checked_out = maps:remove(MsgId, Checked0), + credit = increase_credit(Con0, 1)}, + State1 = State0#?STATE{ra_indexes = rabbit_fifo_index:delete(OldIdx, + Indexes0), + messages = rabbit_fifo_q:in(no, + ?MSG(Idx, Header), + Messages), + enqueue_count = EnqCount + 1}, + State2 = update_or_remove_con(Meta, ConsumerKey, Con, State1), + {State3, Effects} = case MacVer >= 7 of + true -> + activate_next_consumer({State2, []}); + false -> + {State2, []} + end, + checkout(Meta, State0, State3, Effects); + _ -> + {State00, ok, []} + end; +apply(Meta, #credit{consumer_key = ConsumerKey} = Credit, + #?STATE{consumers = Cons} = State) -> + case Cons of + #{ConsumerKey := Con} -> + credit_active_consumer(Credit, Con, Meta, State); + _ -> + case lists:keytake(ConsumerKey, 1, State#?STATE.waiting_consumers) of + {value, {_, Con}, Waiting} -> + credit_inactive_consumer(Credit, Con, Waiting, State); + false -> + %% credit for unknown consumer - just ignore + {State, ok} + end + end; +apply(_, #checkout{spec = {dequeue, _}}, + #?STATE{cfg = #cfg{consumer_strategy = single_active}} = State0) -> + {State0, {error, {unsupported, single_active_consumer}}}; +apply(#{index := Index, + system_time := Ts, + from := From} = Meta, #checkout{spec = {dequeue, Settlement}, + meta = ConsumerMeta, + consumer_id = ConsumerId}, + #?STATE{consumers = Consumers} = State00) -> + %% dequeue always updates last_active + State0 = State00#?STATE{last_active = Ts}, + %% all dequeue operations result in keeping the queue from expiring + Exists = find_consumer(ConsumerId, Consumers) /= undefined, + case messages_ready(State0) of + 0 -> + {State0, {dequeue, empty}, []}; + _ when Exists -> + %% a dequeue using the same consumer_id isn't possible at this point + {State0, {dequeue, empty}}; + _ -> + {_, State1} = update_consumer(Meta, ConsumerId, ConsumerId, ConsumerMeta, + {once, {simple_prefetch, 1}}, 0, + State0), + case checkout_one(Meta, false, State1, []) of + {success, _, MsgId, + ?MSG(RaftIdx, Header), _ExpiredMsg, State2, Effects0} -> + {State4, Effects1} = + case Settlement of + unsettled -> + {_, Pid} = ConsumerId, + {State2, [{monitor, process, Pid} | Effects0]}; + settled -> + %% immediately settle the checkout + {State3, _, SettleEffects} = + apply(Meta, make_settle(ConsumerId, [MsgId]), + State2), + {State3, SettleEffects ++ Effects0} + end, + Effects2 = [reply_log_effect(RaftIdx, MsgId, Header, + messages_ready(State4), From) + | Effects1], + {State, _DroppedMsg, Effects} = + evaluate_limit(Index, false, State0, State4, Effects2), + {State, '$ra_no_reply', Effects}; + {nochange, _ExpiredMsg = true, State2, Effects0} -> + %% All ready messages expired. + State3 = State2#?STATE{consumers = + maps:remove(ConsumerId, + State2#?STATE.consumers)}, + {State, _, Effects} = evaluate_limit(Index, false, State0, + State3, Effects0), + {State, {dequeue, empty}, Effects} + end + end; +apply(#{index := _Idx} = Meta, + #checkout{spec = Spec, + consumer_id = ConsumerId}, State0) + when Spec == cancel orelse + Spec == remove -> + case consumer_key_from_id(ConsumerId, State0) of + {ok, ConsumerKey} -> + {State1, Effects1} = activate_next_consumer( + cancel_consumer(Meta, ConsumerKey, State0, [], + Spec)), + Reply = {ok, consumer_cancel_info(ConsumerKey, State1)}, + {State, _, Effects} = checkout(Meta, State0, State1, Effects1), + {State, Reply, Effects}; + error -> + {State0, {error, consumer_not_found}, []} + end; +apply(#{index := Idx} = Meta, + #checkout{spec = Spec0, + meta = ConsumerMeta, + consumer_id = {_, Pid} = ConsumerId}, State0) -> + %% might be better to check machine_version + IsV4 = tuple_size(Spec0) == 2, + %% normalise spec format + Spec = case Spec0 of + {_, _} -> + Spec0; + {Life, Prefetch, simple_prefetch} -> + {Life, {simple_prefetch, Prefetch}}; + {Life, _Credit, credited} -> + {Life, credited} + end, + Priority = get_priority(ConsumerMeta), + ConsumerKey = case consumer_key_from_id(ConsumerId, State0) of + {ok, K} -> + K; + error when IsV4 -> + %% if the consumer does not already exist use the + %% raft index as it's unique identifier in future + %% settle, credit, return and discard operations + Idx; + error -> + ConsumerId + end, + {Consumer, State1} = update_consumer(Meta, ConsumerKey, ConsumerId, + ConsumerMeta, Spec, Priority, State0), + {State2, Effs} = activate_next_consumer(State1, []), + #consumer{checked_out = Checked, + credit = Credit, + delivery_count = DeliveryCount, + next_msg_id = NextMsgId} = Consumer, + + %% reply with a consumer summary + Reply = {ok, #{next_msg_id => NextMsgId, + credit => Credit, + key => ConsumerKey, + delivery_count => DeliveryCount, + is_active => is_active(ConsumerKey, State2), + num_checked_out => map_size(Checked)}}, + checkout(Meta, State0, State2, [{monitor, process, Pid} | Effs], Reply); +apply(#{index := Index}, #purge{}, + #?STATE{messages_total = Total, + returns = Returns, + ra_indexes = Indexes0 + } = State0) -> + NumReady = messages_ready(State0), + Indexes = case Total of + NumReady -> + %% All messages are either in 'messages' queue or + %% 'returns' queue. + %% No message is awaiting acknowledgement. + %% Optimization: empty all 'ra_indexes'. + rabbit_fifo_index:empty(); + _ -> + %% Some messages are checked out to consumers + %% awaiting acknowledgement. + %% Therefore we cannot empty all 'ra_indexes'. + %% We only need to delete the indexes from the 'returns' + %% queue because messages of the 'messages' queue are + %% not part of the 'ra_indexes'. + lqueue:fold(fun(?MSG(I, _), Acc) -> + rabbit_fifo_index:delete(I, Acc) + end, Indexes0, Returns) + end, + State1 = State0#?STATE{ra_indexes = Indexes, + messages = rabbit_fifo_q:new(), + messages_total = Total - NumReady, + returns = lqueue:new(), + msg_bytes_enqueue = 0 + }, + Effects0 = [{aux, force_checkpoint}, garbage_collection], + Reply = {purge, NumReady}, + {State, _, Effects} = evaluate_limit(Index, false, State0, + State1, Effects0), + {State, Reply, Effects}; +apply(#{index := _Idx}, #garbage_collection{}, State) -> + {State, ok, [{aux, garbage_collection}]}; +apply(Meta, {timeout, expire_msgs}, State) -> + checkout(Meta, State, State, []); +apply(#{machine_version := Vsn, + system_time := Ts} = Meta, + {down, Pid, noconnection}, + #?STATE{consumers = Cons0, + cfg = #cfg{consumer_strategy = single_active}, + waiting_consumers = Waiting0, + enqueuers = Enqs0} = State0) -> + Node = node(Pid), + %% if the pid refers to an active or cancelled consumer, + %% mark it as suspected and return it to the waiting queue + {State1, Effects0} = + rabbit_fifo_maps:fold( + fun(CKey, ?CONSUMER_PID(P) = C0, {S0, E0}) + when node(P) =:= Node -> + %% the consumer should be returned to waiting + %% and checked out messages should be returned + Effs = consumer_update_active_effects( + S0, C0, false, suspected_down, E0), + {St, Effs1} = return_all(Meta, S0, Effs, CKey, C0, true), + %% if the consumer was cancelled there is a chance it got + %% removed when returning hence we need to be defensive here + Waiting = case St#?STATE.consumers of + #{CKey := C} -> + Waiting0 ++ [{CKey, C}]; + _ -> + Waiting0 + end, + {St#?STATE{consumers = maps:remove(CKey, St#?STATE.consumers), + waiting_consumers = Waiting, + last_active = Ts}, + Effs1}; + (_, _, S) -> + S + end, {State0, []}, Cons0, Vsn), + WaitingConsumers = update_waiting_consumer_status(Node, State1, + suspected_down), + + %% select a new consumer from the waiting queue and run a checkout + State2 = State1#?STATE{waiting_consumers = WaitingConsumers}, + {State, Effects1} = activate_next_consumer(State2, Effects0), + + %% mark any enquers as suspected + Enqs = maps:map(fun(P, E) when node(P) =:= Node -> + E#enqueuer{status = suspected_down}; + (_, E) -> E + end, Enqs0), + Effects = [{monitor, node, Node} | Effects1], + checkout(Meta, State0, State#?STATE{enqueuers = Enqs}, Effects); +apply(#{machine_version := Vsn, + system_time := Ts} = Meta, + {down, Pid, noconnection}, + #?STATE{consumers = Cons0, + enqueuers = Enqs0} = State0) -> + %% A node has been disconnected. This doesn't necessarily mean that + %% any processes on this node are down, they _may_ come back so here + %% we just mark them as suspected (effectively deactivated) + %% and return all checked out messages to the main queue for delivery to any + %% live consumers + %% + %% all pids for the disconnected node will be marked as suspected not just + %% the one we got the `down' command for + Node = node(Pid), + + {State, Effects1} = + rabbit_fifo_maps:fold( + fun(CKey, #consumer{cfg = #consumer_cfg{pid = P}, + status = up} = C0, + {St0, Eff}) when node(P) =:= Node -> + C = C0#consumer{status = suspected_down}, + {St, Eff0} = return_all(Meta, St0, Eff, CKey, C, true), + Eff1 = consumer_update_active_effects(St, C, false, + suspected_down, Eff0), + {St, Eff1}; + (_, _, {St, Eff}) -> + {St, Eff} + end, {State0, []}, Cons0, Vsn), + Enqs = maps:map(fun(P, E) when node(P) =:= Node -> + E#enqueuer{status = suspected_down}; + (_, E) -> E + end, Enqs0), + + % Monitor the node so that we can "unsuspect" these processes when the node + % comes back, then re-issue all monitors and discover the final fate of + % these processes + + Effects = [{monitor, node, Node} | Effects1], + checkout(Meta, State0, State#?STATE{enqueuers = Enqs, + last_active = Ts}, Effects); +apply(Meta, {down, Pid, _Info}, State0) -> + {State1, Effects1} = activate_next_consumer(handle_down(Meta, Pid, State0)), + checkout(Meta, State0, State1, Effects1); +apply(#{machine_version := Vsn} = Meta, + {nodeup, Node}, + #?STATE{consumers = Cons0, + enqueuers = Enqs0, + service_queue = _SQ0} = State0) -> + %% A node we are monitoring has come back. + %% If we have suspected any processes of being + %% down we should now re-issue the monitors for them to detect if they're + %% actually down or not + Monitors = [{monitor, process, P} + || P <- suspected_pids_for(Node, Vsn, State0)], + + Enqs1 = maps:map(fun(P, E) when node(P) =:= Node -> + E#enqueuer{status = up}; + (_, E) -> E + end, Enqs0), + ConsumerUpdateActiveFun = consumer_active_flag_update_function(State0), + %% mark all consumers as up + {State1, Effects1} = + rabbit_fifo_maps:fold( + fun(ConsumerKey, ?CONSUMER_PID(P) = C, {SAcc, EAcc}) + when (node(P) =:= Node) and + (C#consumer.status =/= cancelled) -> + EAcc1 = ConsumerUpdateActiveFun(SAcc, ConsumerKey, + C, true, up, EAcc), + {update_or_remove_con(Meta, ConsumerKey, + C#consumer{status = up}, + SAcc), EAcc1}; + (_, _, Acc) -> + Acc + end, {State0, Monitors}, Cons0, Vsn), + Waiting = update_waiting_consumer_status(Node, State1, up), + State2 = State1#?STATE{enqueuers = Enqs1, + waiting_consumers = Waiting}, + {State, Effects} = activate_next_consumer(State2, Effects1), + checkout(Meta, State0, State, Effects); +apply(_, {nodedown, _Node}, State) -> + {State, ok}; +apply(Meta, #purge_nodes{nodes = Nodes}, State0) -> + {State, Effects} = lists:foldl(fun(Node, {S, E}) -> + purge_node(Meta, Node, S, E) + end, {State0, []}, Nodes), + {State, ok, Effects}; +apply(Meta, + #update_config{config = #{} = Conf}, + #?STATE{cfg = #cfg{dead_letter_handler = OldDLH, + resource = QRes}, + dlx = DlxState0} = State0) -> + NewDLH = maps:get(dead_letter_handler, Conf, OldDLH), + {DlxState, Effects0} = rabbit_fifo_dlx:update_config(OldDLH, NewDLH, QRes, + DlxState0), + State1 = update_config(Conf, State0#?STATE{dlx = DlxState}), + checkout(Meta, State0, State1, Effects0); +apply(Meta, {machine_version, FromVersion, ToVersion}, V0State) -> + State = convert(Meta, FromVersion, ToVersion, V0State), + {State, ok, [{aux, {dlx, setup}}]}; +apply(Meta, {dlx, _} = Cmd, + #?STATE{cfg = #cfg{dead_letter_handler = DLH}, + dlx = DlxState0} = State0) -> + {DlxState, Effects0} = rabbit_fifo_dlx:apply(Meta, Cmd, DLH, DlxState0), + State1 = State0#?STATE{dlx = DlxState}, + checkout(Meta, State0, State1, Effects0); +apply(_Meta, Cmd, State) -> + %% handle unhandled commands gracefully + ?LOG_DEBUG("rabbit_fifo: unhandled command ~W", [Cmd, 10]), + {State, ok, []}. + +convert_v3_to_v4(#{} = _Meta, StateV3) -> + %% TODO: consider emitting release cursors as checkpoints + Messages0 = rabbit_fifo_v3:get_field(messages, StateV3), + Returns0 = lqueue:to_list(rabbit_fifo_v3:get_field(returns, StateV3)), + Consumers0 = rabbit_fifo_v3:get_field(consumers, StateV3), + Consumers = maps:map( + fun (_, #consumer{checked_out = Ch0} = C) -> + Ch = maps:map( + fun (_, ?MSG(I, #{delivery_count := DC} = H)) -> + ?MSG(I, H#{acquired_count => DC}); + (_, Msg) -> + Msg + end, Ch0), + C#consumer{checked_out = Ch} + end, Consumers0), + Returns = lqueue:from_list( + lists:map(fun (?MSG(I, #{delivery_count := DC} = H)) -> + ?MSG(I, H#{acquired_count => DC}); + (Msg) -> + Msg + end, Returns0)), + + Messages = rabbit_fifo_q:from_lqueue(Messages0), + Cfg = rabbit_fifo_v3:get_field(cfg, StateV3), + #?STATE{cfg = Cfg#cfg{unused_1 = ?NIL}, + messages = Messages, + messages_total = rabbit_fifo_v3:get_field(messages_total, StateV3), + returns = Returns, + enqueue_count = rabbit_fifo_v3:get_field(enqueue_count, StateV3), + enqueuers = rabbit_fifo_v3:get_field(enqueuers, StateV3), + ra_indexes = rabbit_fifo_v3:get_field(ra_indexes, StateV3), + consumers = Consumers, + service_queue = rabbit_fifo_v3:get_field(service_queue, StateV3), + dlx = rabbit_fifo_v3:get_field(dlx, StateV3), + msg_bytes_enqueue = rabbit_fifo_v3:get_field(msg_bytes_enqueue, StateV3), + msg_bytes_checkout = rabbit_fifo_v3:get_field(msg_bytes_checkout, StateV3), + waiting_consumers = rabbit_fifo_v3:get_field(waiting_consumers, StateV3), + last_active = rabbit_fifo_v3:get_field(last_active, StateV3), + msg_cache = rabbit_fifo_v3:get_field(msg_cache, StateV3), + unused_1 = []}. + +purge_node(#{machine_version := Vsn} = Meta, Node, State, Effects) -> + lists:foldl(fun(Pid, {S0, E0}) -> + {S, E} = handle_down(Meta, Pid, S0), + {S, E0 ++ E} + end, {State, Effects}, + all_pids_for(Node, Vsn, State)). + +%% any downs that are not noconnection +handle_down(#{machine_version := Vsn} = Meta, + Pid, #?STATE{consumers = Cons0, + enqueuers = Enqs0} = State0) -> + % Remove any enqueuer for the down pid + State1 = State0#?STATE{enqueuers = maps:remove(Pid, Enqs0)}, + {Effects1, State2} = handle_waiting_consumer_down(Pid, State1), + % return checked out messages to main queue + % Find the consumers for the down pid + DownConsumers = maps:filter(fun(_CKey, ?CONSUMER_PID(P)) -> + P =:= Pid + end, Cons0), + DownConsumerKeys = rabbit_fifo_maps:keys(DownConsumers, Vsn), + lists:foldl(fun(ConsumerKey, {S, E}) -> + cancel_consumer(Meta, ConsumerKey, S, E, down) + end, {State2, Effects1}, DownConsumerKeys). + +consumer_active_flag_update_function( + #?STATE{cfg = #cfg{consumer_strategy = competing}}) -> + fun(State, _ConsumerKey, Consumer, Active, ActivityStatus, Effects) -> + consumer_update_active_effects(State, Consumer, Active, + ActivityStatus, Effects) + end; +consumer_active_flag_update_function( + #?STATE{cfg = #cfg{consumer_strategy = single_active}}) -> + fun(_, _, _, _, _, Effects) -> + Effects + end. + +handle_waiting_consumer_down(_Pid, + #?STATE{cfg = #cfg{consumer_strategy = competing}} + = State) -> + {[], State}; +handle_waiting_consumer_down(_Pid, + #?STATE{cfg = #cfg{consumer_strategy = single_active}, + waiting_consumers = []} = State) -> + {[], State}; +handle_waiting_consumer_down(Pid, + #?STATE{cfg = #cfg{consumer_strategy = single_active}, + waiting_consumers = WaitingConsumers0} + = State0) -> + % get cancel effects for down waiting consumers + Down = lists:filter(fun({_, ?CONSUMER_PID(P)}) -> P =:= Pid end, + WaitingConsumers0), + Effects = lists:foldl(fun ({_ConsumerKey, Consumer}, Effects) -> + ConsumerId = consumer_id(Consumer), + cancel_consumer_effects(ConsumerId, State0, + Effects) + end, [], Down), + % update state to have only up waiting consumers + StillUp = lists:filter(fun({_CKey, ?CONSUMER_PID(P)}) -> + P =/= Pid + end, + WaitingConsumers0), + State = State0#?STATE{waiting_consumers = StillUp}, + {Effects, State}. + +update_waiting_consumer_status(Node, + #?STATE{waiting_consumers = WaitingConsumers}, + Status) -> + sort_waiting( + [case node(Pid) of + Node -> + {ConsumerKey, Consumer#consumer{status = Status}}; + _ -> + {ConsumerKey, Consumer} + end || {ConsumerKey, ?CONSUMER_PID(Pid) = Consumer} + <- WaitingConsumers, Consumer#consumer.status =/= cancelled]). + +-spec state_enter(ra_server:ra_state() | eol, state()) -> + ra_machine:effects(). +state_enter(RaState, #?STATE{cfg = #cfg{dead_letter_handler = DLH, + resource = QRes}, + dlx = DlxState} = State) -> + Effects = rabbit_fifo_dlx:state_enter(RaState, QRes, DLH, DlxState), + state_enter0(RaState, State, Effects). + +state_enter0(leader, #?STATE{consumers = Cons, + enqueuers = Enqs, + waiting_consumers = WaitingConsumers, + cfg = #cfg{name = Name, + resource = Resource, + become_leader_handler = BLH} + } = State, + Effects0) -> + TimerEffs = timer_effect(erlang:system_time(millisecond), State, Effects0), + % return effects to monitor all current consumers and enqueuers + Pids = lists:usort(maps:keys(Enqs) + ++ [P || ?CONSUMER_PID(P) <- maps:values(Cons)] + ++ [P || {_, ?CONSUMER_PID(P)} <- WaitingConsumers]), + Mons = [{monitor, process, P} || P <- Pids], + Nots = [{send_msg, P, leader_change, ra_event} || P <- Pids], + NodeMons = lists:usort([{monitor, node, node(P)} || P <- Pids]), + NotifyDecs = notify_decorators_startup(Resource), + Effects = TimerEffs ++ Mons ++ Nots ++ NodeMons ++ [NotifyDecs], + case BLH of + undefined -> + Effects; + {Mod, Fun, Args} -> + [{mod_call, Mod, Fun, Args ++ [Name]} | Effects] + end; +state_enter0(eol, #?STATE{enqueuers = Enqs, + consumers = Cons0, + waiting_consumers = WaitingConsumers0}, + Effects) -> + Custs = maps:fold(fun(_K, ?CONSUMER_PID(P) = V, S) -> + S#{P => V} + end, #{}, Cons0), + WaitingConsumers1 = lists:foldl(fun({_, ?CONSUMER_PID(P) = V}, Acc) -> + Acc#{P => V} + end, #{}, WaitingConsumers0), + AllConsumers = maps:merge(Custs, WaitingConsumers1), + [{send_msg, P, eol, ra_event} + || P <- maps:keys(maps:merge(Enqs, AllConsumers))] ++ + [{aux, eol} + | Effects]; +state_enter0(_, _, Effects) -> + %% catch all as not handling all states + Effects. + +-spec tick(non_neg_integer(), state()) -> ra_machine:effects(). +tick(Ts, #?STATE{cfg = #cfg{resource = QName}} = State) -> + case is_expired(Ts, State) of + true -> + [{mod_call, rabbit_quorum_queue, spawn_deleter, [QName]}]; + false -> + [{aux, {handle_tick, [QName, overview(State), all_nodes(State)]}}] + end. + +-spec overview(state()) -> map(). +overview(#?STATE{consumers = Cons, + enqueuers = Enqs, + enqueue_count = EnqCount, + msg_bytes_enqueue = EnqueueBytes, + msg_bytes_checkout = CheckoutBytes, + cfg = Cfg, + dlx = DlxState, + messages = Messages, + returns = Returns, + waiting_consumers = WaitingConsumers} = State) -> + Conf = #{name => Cfg#cfg.name, + resource => Cfg#cfg.resource, + dead_lettering_enabled => undefined =/= Cfg#cfg.dead_letter_handler, + dead_letter_handler => Cfg#cfg.dead_letter_handler, + overflow_strategy => Cfg#cfg.overflow_strategy, + max_length => Cfg#cfg.max_length, + max_bytes => Cfg#cfg.max_bytes, + consumer_strategy => Cfg#cfg.consumer_strategy, + expires => Cfg#cfg.expires, + msg_ttl => Cfg#cfg.msg_ttl, + delivery_limit => Cfg#cfg.delivery_limit + }, + SacOverview = case active_consumer(Cons) of + {SacConsumerKey, SacCon} -> + SacConsumerId = consumer_id(SacCon), + NumWaiting = length(WaitingConsumers), + #{single_active_consumer_id => SacConsumerId, + single_active_consumer_key => SacConsumerKey, + single_active_num_waiting_consumers => NumWaiting}; + _ -> + #{} + end, + MsgsRet = lqueue:len(Returns), + #{num_hi := MsgsHi, + num_no := MsgsNo} = rabbit_fifo_q:overview(Messages), + + Overview = #{type => ?STATE, + config => Conf, + num_consumers => map_size(Cons), + num_active_consumers => query_consumer_count(State), + num_checked_out => num_checked_out(State), + num_enqueuers => maps:size(Enqs), + num_ready_messages => messages_ready(State), + num_ready_messages_high => MsgsHi, + num_ready_messages_normal => MsgsNo, + num_ready_messages_return => MsgsRet, + num_messages => messages_total(State), + num_release_cursors => 0, %% backwards compat + enqueue_message_bytes => EnqueueBytes, + checkout_message_bytes => CheckoutBytes, + release_cursors => [], %% backwards compat + in_memory_message_bytes => 0, %% backwards compat + num_in_memory_ready_messages => 0, %% backwards compat + release_cursor_enqueue_counter => EnqCount, + smallest_raft_index => smallest_raft_index(State), + smallest_raft_index_overview => smallest_raft_index_overview(State) + }, + DlxOverview = rabbit_fifo_dlx:overview(DlxState), + maps:merge(maps:merge(Overview, DlxOverview), SacOverview). + +-spec get_checked_out(consumer_key(), msg_id(), msg_id(), state()) -> + [delivery_msg()]. +get_checked_out(CKey, From, To, #?STATE{consumers = Consumers}) -> + case find_consumer(CKey, Consumers) of + {_CKey, #consumer{checked_out = Checked}} -> + [begin + ?MSG(I, H) = maps:get(K, Checked), + {K, {I, H}} + end || K <- lists:seq(From, To), maps:is_key(K, Checked)]; + _ -> + [] + end. + +-spec version() -> pos_integer(). +version() -> 7. + +which_module(0) -> rabbit_fifo_v0; +which_module(1) -> rabbit_fifo_v1; +which_module(2) -> rabbit_fifo_v3; +which_module(3) -> rabbit_fifo_v3; +which_module(4) -> ?MODULE; +which_module(5) -> ?MODULE; +which_module(6) -> ?MODULE; +which_module(7) -> ?MODULE. + +-define(AUX, aux_v3). + +-record(checkpoint, {index :: ra:index(), + timestamp :: milliseconds(), + smallest_index :: undefined | ra:index(), + messages_total :: non_neg_integer(), + indexes = ?CHECK_MIN_INDEXES :: non_neg_integer(), + bytes_in = 0 :: non_neg_integer()}). +-record(aux_gc, {last_raft_idx = 0 :: ra:index()}). +-record(aux, {name :: atom(), + capacity :: term(), + gc = #aux_gc{} :: #aux_gc{}}). +-record(?AUX, {name :: atom(), + last_decorators_state :: term(), + capacity :: term(), + gc = #aux_gc{} :: #aux_gc{}, + tick_pid :: undefined | pid(), + cache = #{} :: map(), + last_checkpoint :: #checkpoint{}, + bytes_in = 0 :: non_neg_integer(), + bytes_out = 0 :: non_neg_integer()}). + +init_aux(Name) when is_atom(Name) -> + %% TODO: catch specific exception throw if table already exists + ok = ra_machine_ets:create_table(rabbit_fifo_usage, + [named_table, set, public, + {write_concurrency, true}]), + Now = erlang:monotonic_time(microsecond), + #?AUX{name = Name, + capacity = {inactive, Now, 1, 1.0}, + last_checkpoint = #checkpoint{index = 0, + timestamp = erlang:system_time(millisecond), + messages_total = 0, + bytes_in = 0}}. + +handle_aux(RaftState, Tag, Cmd, #aux{name = Name, + capacity = Cap, + gc = Gc}, RaAux) -> + %% convert aux state to new version + AuxV2 = init_aux(Name), + Aux = AuxV2#?AUX{capacity = Cap, + gc = Gc}, + handle_aux(RaftState, Tag, Cmd, Aux, RaAux); +handle_aux(RaftState, Tag, Cmd, AuxV2, RaAux) + when element(1, AuxV2) == aux_v2 -> + Name = element(2, AuxV2), + AuxV3 = init_aux(Name), + handle_aux(RaftState, Tag, Cmd, AuxV3, RaAux); +handle_aux(leader, cast, eval, + #?AUX{last_decorators_state = LastDec, + bytes_in = BytesIn, + last_checkpoint = Check0} = Aux0, + RaAux) -> + #?STATE{cfg = #cfg{resource = QName}} = MacState = + ra_aux:machine_state(RaAux), + + Ts = erlang:system_time(millisecond), + {Check, Effects0} = do_checkpoints(Ts, Check0, RaAux, BytesIn, false), + + %% this is called after each batch of commands have been applied + %% set timer for message expire + %% should really be the last applied index ts but this will have to do + Effects1 = timer_effect(Ts, MacState, Effects0), + case query_notify_decorators_info(MacState) of + LastDec -> + {no_reply, Aux0#?AUX{last_checkpoint = Check}, RaAux, Effects1}; + {MaxActivePriority, IsEmpty} = NewLast -> + Effects = [notify_decorators_effect(QName, MaxActivePriority, IsEmpty) + | Effects1], + {no_reply, Aux0#?AUX{last_checkpoint = Check, + last_decorators_state = NewLast}, RaAux, Effects} + end; +handle_aux(_RaftState, cast, eval, + #?AUX{last_checkpoint = Check0, + bytes_in = BytesIn} = Aux0, + RaAux) -> + Ts = erlang:system_time(millisecond), + {Check, Effects} = do_checkpoints(Ts, Check0, RaAux, BytesIn, false), + {no_reply, Aux0#?AUX{last_checkpoint = Check}, RaAux, Effects}; +handle_aux(_RaftState, cast, {bytes_in, {MetaSize, BodySize}}, + #?AUX{bytes_in = Bytes} = Aux0, + RaAux) -> + {no_reply, Aux0#?AUX{bytes_in = Bytes + MetaSize + BodySize}, RaAux, []}; +handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, + consumer_key = Key} = Ret, Corr, Pid}, + Aux0, RaAux0) -> + case ra_aux:machine_state(RaAux0) of + #?STATE{cfg = #cfg{delivery_limit = undefined}, + consumers = Consumers} -> + case find_consumer(Key, Consumers) of + {ConsumerKey, #consumer{checked_out = Checked}} -> + {RaAux, ToReturn} = + maps:fold( + fun (MsgId, ?MSG(Idx, Header), {RA0, Acc}) -> + %% it is possible this is not found if the consumer + %% crashed and the message got removed + case ra_aux:log_fetch(Idx, RA0) of + {{_Term, _Meta, Cmd}, RA} -> + Msg = get_msg(Cmd), + {RA, [{MsgId, Idx, Header, Msg} | Acc]}; + {undefined, RA} -> + {RA, Acc} + end + end, {RaAux0, []}, maps:with(MsgIds, Checked)), + + Appends = make_requeue(ConsumerKey, {notify, Corr, Pid}, + lists:sort(ToReturn), []), + {no_reply, Aux0, RaAux, Appends}; + _ -> + {no_reply, Aux0, RaAux0} + end; + _ -> + %% for returns with a delivery limit set we can just return as before + {no_reply, Aux0, RaAux0, [{append, Ret, {notify, Corr, Pid}}]} + end; +handle_aux(leader, _, {handle_tick, [QName, Overview0, Nodes]}, + #?AUX{tick_pid = Pid} = Aux, RaAux) -> + Overview = Overview0#{members_info => ra_aux:members_info(RaAux)}, + NewPid = + case process_is_alive(Pid) of + false -> + %% No active TICK pid + %% this function spawns and returns the tick process pid + rabbit_quorum_queue:handle_tick(QName, Overview, Nodes); + true -> + %% Active TICK pid, do nothing + Pid + end, + + %% TODO: check consumer timeouts + {no_reply, Aux#?AUX{tick_pid = NewPid}, RaAux, []}; +handle_aux(_, _, {get_checked_out, ConsumerKey, MsgIds}, Aux0, RaAux0) -> + #?STATE{cfg = #cfg{}, + consumers = Consumers} = ra_aux:machine_state(RaAux0), + case Consumers of + #{ConsumerKey := #consumer{checked_out = Checked}} -> + {RaState, IdMsgs} = + maps:fold( + fun (MsgId, ?MSG(Idx, Header), {S0, Acc}) -> + %% it is possible this is not found if the consumer + %% crashed and the message got removed + case ra_aux:log_fetch(Idx, S0) of + {{_Term, _Meta, Cmd}, S} -> + Msg = get_msg(Cmd), + {S, [{MsgId, {Header, Msg}} | Acc]}; + {undefined, S} -> + {S, Acc} + end + end, {RaAux0, []}, maps:with(MsgIds, Checked)), + {reply, {ok, IdMsgs}, Aux0, RaState}; + _ -> + {reply, {error, consumer_not_found}, Aux0, RaAux0} + end; +handle_aux(_RaState, cast, Cmd, #?AUX{capacity = Use0} = Aux0, RaAux) + when Cmd == active orelse Cmd == inactive -> + {no_reply, Aux0#?AUX{capacity = update_use(Use0, Cmd)}, RaAux}; +handle_aux(_RaState, cast, tick, #?AUX{name = Name, + capacity = Use0} = State0, + RaAux) -> + true = ets:insert(rabbit_fifo_usage, + {Name, capacity(Use0)}), + Aux = eval_gc(RaAux, ra_aux:machine_state(RaAux), State0), + Effs = case smallest_raft_index(ra_aux:machine_state(RaAux)) of + undefined -> + [{release_cursor, ra_aux:last_applied(RaAux)}]; + Smallest -> + [{release_cursor, Smallest - 1}] + end, + {no_reply, Aux, RaAux, Effs}; +handle_aux(_RaState, cast, eol, #?AUX{name = Name} = Aux, RaAux) -> + ets:delete(rabbit_fifo_usage, Name), + {no_reply, Aux, RaAux}; +handle_aux(_RaState, {call, _From}, oldest_entry_timestamp, + #?AUX{cache = Cache} = Aux0, RaAux0) -> + {CachedIdx, CachedTs} = maps:get(oldest_entry, Cache, + {undefined, undefined}), + case smallest_raft_index(ra_aux:machine_state(RaAux0)) of + %% if there are no entries, we return current timestamp + %% so that any previously obtained entries are considered + %% older than this + undefined -> + Aux1 = Aux0#?AUX{cache = maps:remove(oldest_entry, Cache)}, + {reply, {ok, erlang:system_time(millisecond)}, Aux1, RaAux0}; + CachedIdx -> + %% cache hit + {reply, {ok, CachedTs}, Aux0, RaAux0}; + Idx when is_integer(Idx) -> + case ra_aux:log_fetch(Idx, RaAux0) of + {{_Term, #{ts := Timestamp}, _Cmd}, RaAux} -> + Aux1 = Aux0#?AUX{cache = Cache#{oldest_entry => + {Idx, Timestamp}}}, + {reply, {ok, Timestamp}, Aux1, RaAux}; + {undefined, RaAux} -> + %% fetch failed + {reply, {error, failed_to_get_timestamp}, Aux0, RaAux} + end + end; +handle_aux(_RaState, {call, _From}, {peek, Pos}, Aux0, + RaAux0) -> + MacState = ra_aux:machine_state(RaAux0), + case query_peek(Pos, MacState) of + {ok, ?MSG(Idx, Header)} -> + %% need to re-hydrate from the log + {{_, _, Cmd}, RaAux} = ra_aux:log_fetch(Idx, RaAux0), + Msg = get_msg(Cmd), + {reply, {ok, {Header, Msg}}, Aux0, RaAux}; + Err -> + {reply, Err, Aux0, RaAux0} + end; +handle_aux(_, _, garbage_collection, Aux, RaAux) -> + {no_reply, force_eval_gc(RaAux, Aux), RaAux}; +handle_aux(_RaState, _, force_checkpoint, + #?AUX{last_checkpoint = Check0, + bytes_in = BytesIn} = Aux, RaAux) -> + Ts = erlang:system_time(millisecond), + #?STATE{cfg = #cfg{resource = QR}} = ra_aux:machine_state(RaAux), + ?LOG_DEBUG("~ts: rabbit_fifo: forcing checkpoint at ~b", + [rabbit_misc:rs(QR), ra_aux:last_applied(RaAux)]), + {Check, Effects} = do_checkpoints(Ts, Check0, RaAux, BytesIn, true), + {no_reply, Aux#?AUX{last_checkpoint = Check}, RaAux, Effects}; +handle_aux(RaState, _, {dlx, _} = Cmd, Aux0, RaAux) -> + #?STATE{dlx = DlxState, + cfg = #cfg{dead_letter_handler = DLH, + resource = QRes}} = ra_aux:machine_state(RaAux), + Aux = rabbit_fifo_dlx:handle_aux(RaState, Cmd, Aux0, QRes, DLH, DlxState), + {no_reply, Aux, RaAux}. + +eval_gc(RaAux, MacState, + #?AUX{gc = #aux_gc{last_raft_idx = LastGcIdx} = Gc} = AuxState) -> + {Idx, _} = ra_aux:log_last_index_term(RaAux), + #?STATE{cfg = #cfg{resource = QR}} = ra_aux:machine_state(RaAux), + {memory, Mem} = erlang:process_info(self(), memory), + case messages_total(MacState) of + 0 when Idx > LastGcIdx andalso + Mem > ?GC_MEM_LIMIT_B -> + garbage_collect(), + {memory, MemAfter} = erlang:process_info(self(), memory), + ?LOG_DEBUG("~ts: full GC sweep complete. " + "Process memory changed from ~.2fMB to ~.2fMB.", + [rabbit_misc:rs(QR), Mem/?MB, MemAfter/?MB]), + AuxState#?AUX{gc = Gc#aux_gc{last_raft_idx = Idx}}; + _ -> + AuxState + end. + +force_eval_gc(RaAux, + #?AUX{gc = #aux_gc{last_raft_idx = LastGcIdx} = Gc} = AuxState) -> + {Idx, _} = ra_aux:log_last_index_term(RaAux), + #?STATE{cfg = #cfg{resource = QR}} = ra_aux:machine_state(RaAux), + {memory, Mem} = erlang:process_info(self(), memory), + case Idx > LastGcIdx of + true -> + garbage_collect(), + {memory, MemAfter} = erlang:process_info(self(), memory), + ?LOG_DEBUG("~ts: full GC sweep complete. " + "Process memory changed from ~.2fMB to ~.2fMB.", + [rabbit_misc:rs(QR), Mem/?MB, MemAfter/?MB]), + AuxState#?AUX{gc = Gc#aux_gc{last_raft_idx = Idx}}; + false -> + AuxState + end. + +process_is_alive(Pid) when is_pid(Pid) -> + is_process_alive(Pid); +process_is_alive(_) -> + false. +%%% Queries + +query_messages_ready(State) -> + messages_ready(State). + +query_messages_checked_out(#?STATE{consumers = Consumers}) -> + maps:fold(fun (_, #consumer{checked_out = C}, S) -> + maps:size(C) + S + end, 0, Consumers). + +query_messages_total(State) -> + messages_total(State). + +query_processes(#?STATE{enqueuers = Enqs, consumers = Cons0}) -> + Cons = maps:fold(fun(_, ?CONSUMER_PID(P) = V, S) -> + S#{P => V} + end, #{}, Cons0), + maps:keys(maps:merge(Enqs, Cons)). + + +query_ra_indexes(#?STATE{ra_indexes = RaIndexes}) -> + RaIndexes. + +query_waiting_consumers(#?STATE{waiting_consumers = WaitingConsumers}) -> + WaitingConsumers. + +query_consumer_count(#?STATE{consumers = Consumers, + waiting_consumers = WaitingConsumers}) -> + Up = maps:filter(fun(_ConsumerKey, #consumer{status = Status}) -> + Status =/= suspected_down + end, Consumers), + maps:size(Up) + length(WaitingConsumers). + +query_consumers(#?STATE{consumers = Consumers, + waiting_consumers = WaitingConsumers, + cfg = #cfg{consumer_strategy = ConsumerStrategy}} + = State) -> + ActiveActivityStatusFun = + case ConsumerStrategy of + competing -> + fun(_ConsumerKey, #consumer{status = Status}) -> + case Status of + suspected_down -> + {false, Status}; + _ -> + {true, Status} + end + end; + single_active -> + SingleActiveConsumer = query_single_active_consumer(State), + fun(_, ?CONSUMER_TAG_PID(Tag, Pid)) -> + case SingleActiveConsumer of + {value, {Tag, Pid}} -> + {true, single_active}; + _ -> + {false, waiting} + end + end + end, + FromConsumers = + maps:fold(fun (_, #consumer{status = cancelled}, Acc) -> + Acc; + (Key, + #consumer{cfg = #consumer_cfg{tag = Tag, + pid = Pid, + meta = Meta}} = Consumer, + Acc) -> + {Active, ActivityStatus} = + ActiveActivityStatusFun(Key, Consumer), + maps:put(Key, + {Pid, Tag, + maps:get(ack, Meta, undefined), + maps:get(prefetch, Meta, undefined), + Active, + ActivityStatus, + maps:get(args, Meta, []), + maps:get(username, Meta, undefined)}, + Acc) + end, #{}, Consumers), + FromWaitingConsumers = + lists:foldl( + fun ({_, #consumer{status = cancelled}}, + Acc) -> + Acc; + ({Key, + #consumer{cfg = #consumer_cfg{tag = Tag, + pid = Pid, + meta = Meta}} = Consumer}, + Acc) -> + {Active, ActivityStatus} = + ActiveActivityStatusFun(Key, Consumer), + maps:put(Key, + {Pid, Tag, + maps:get(ack, Meta, undefined), + maps:get(prefetch, Meta, undefined), + Active, + ActivityStatus, + maps:get(args, Meta, []), + maps:get(username, Meta, undefined)}, + Acc) + end, #{}, WaitingConsumers), + maps:merge(FromConsumers, FromWaitingConsumers). + + +query_single_active_consumer(#?STATE{cfg = #cfg{consumer_strategy = single_active}, + consumers = Consumers}) -> + case active_consumer(Consumers) of + undefined -> + {error, no_value}; + {_CKey, ?CONSUMER_TAG_PID(Tag, Pid)} -> + {value, {Tag, Pid}} + end; +query_single_active_consumer(_) -> + disabled. + +query_stat(#?STATE{consumers = Consumers} = State) -> + {messages_ready(State), maps:size(Consumers)}. + +query_in_memory_usage(#?STATE{ }) -> + {0, 0}. + +query_stat_dlx(#?STATE{dlx = DlxState}) -> + rabbit_fifo_dlx:stat(DlxState). + +query_peek(Pos, State0) when Pos > 0 -> + case take_next_msg(State0) of + empty -> + {error, no_message_at_pos}; + {Msg, _State} + when Pos == 1 -> + {ok, Msg}; + {_Msg, State} -> + query_peek(Pos-1, State) + end. + +query_notify_decorators_info(#?STATE{consumers = Consumers} = State) -> + MaxActivePriority = maps:fold( + fun(_, #consumer{credit = C, + status = up, + cfg = #consumer_cfg{priority = P}}, + MaxP) when C > 0 -> + case MaxP of + empty -> P; + MaxP when MaxP > P -> MaxP; + _ -> P + end; + (_, _, MaxP) -> + MaxP + end, empty, Consumers), + IsEmpty = (messages_ready(State) == 0), + {MaxActivePriority, IsEmpty}. + +-spec usage(atom()) -> float(). +usage(Name) when is_atom(Name) -> + case ets:lookup(rabbit_fifo_usage, Name) of + [] -> 0.0; + [{_, Use}] -> Use + end. + +-spec is_v4() -> boolean(). +is_v4() -> + %% Quorum queue v4 is introduced in RabbitMQ 4.0.0 + rabbit_feature_flags:is_enabled('rabbitmq_4.0.0'). + +%%% Internal + +messages_ready(#?STATE{messages = M, + returns = R}) -> + rabbit_fifo_q:len(M) + lqueue:len(R). + +messages_total(#?STATE{messages_total = Total, + dlx = DlxState}) -> + {DlxTotal, _} = rabbit_fifo_dlx:stat(DlxState), + Total + DlxTotal. + +update_use({inactive, _, _, _} = CUInfo, inactive) -> + CUInfo; +update_use({active, _, _} = CUInfo, active) -> + CUInfo; +update_use({active, Since, Avg}, inactive) -> + Now = erlang:monotonic_time(microsecond), + {inactive, Now, Now - Since, Avg}; +update_use({inactive, Since, Active, Avg}, active) -> + Now = erlang:monotonic_time(microsecond), + {active, Now, use_avg(Active, Now - Since, Avg)}. + +capacity({active, Since, Avg}) -> + use_avg(erlang:monotonic_time(microsecond) - Since, 0, Avg); +capacity({inactive, _, 1, 1.0}) -> + 1.0; +capacity({inactive, Since, Active, Avg}) -> + use_avg(Active, erlang:monotonic_time(microsecond) - Since, Avg). + +use_avg(0, 0, Avg) -> + Avg; +use_avg(Active, Inactive, Avg) -> + Time = Inactive + Active, + moving_average(Time, ?USE_AVG_HALF_LIFE, Active / Time, Avg). + +moving_average(_Time, _, Next, undefined) -> + Next; +moving_average(Time, HalfLife, Next, Current) -> + Weight = math:exp(Time * math:log(0.5) / HalfLife), + Next * (1 - Weight) + Current * Weight. + +num_checked_out(#?STATE{consumers = Cons}) -> + maps:fold(fun (_, #consumer{checked_out = C}, Acc) -> + maps:size(C) + Acc + end, 0, Cons). + +cancel_consumer(Meta, ConsumerKey, + #?STATE{cfg = #cfg{consumer_strategy = competing}} = State, + Effects, Reason) -> + cancel_consumer0(Meta, ConsumerKey, State, Effects, Reason); +cancel_consumer(Meta, ConsumerKey, + #?STATE{cfg = #cfg{consumer_strategy = single_active}, + waiting_consumers = []} = State, + Effects, Reason) -> + %% single active consumer on, no consumers are waiting + cancel_consumer0(Meta, ConsumerKey, State, Effects, Reason); +cancel_consumer(Meta, ConsumerKey, + #?STATE{consumers = Cons0, + cfg = #cfg{consumer_strategy = single_active}, + waiting_consumers = Waiting0} = State0, + Effects0, Reason) -> + %% single active consumer on, consumers are waiting + case Cons0 of + #{ConsumerKey := #consumer{status = _}} -> + % The active consumer is to be removed + cancel_consumer0(Meta, ConsumerKey, State0, + Effects0, Reason); + _ -> + % The cancelled consumer is not active or cancelled + % Just remove it from idle_consumers + case lists:keyfind(ConsumerKey, 1, Waiting0) of + {_, ?CONSUMER_TAG_PID(T, P)} -> + Waiting = lists:keydelete(ConsumerKey, 1, Waiting0), + Effects = cancel_consumer_effects({T, P}, State0, Effects0), + % A waiting consumer isn't supposed to have any checked out messages, + % so nothing special to do here + {State0#?STATE{waiting_consumers = Waiting}, Effects}; + _ -> + {State0, Effects0} + end + end. + +consumer_update_active_effects(#?STATE{cfg = #cfg{resource = QName}}, + #consumer{cfg = #consumer_cfg{pid = CPid, + tag = CTag, + meta = Meta}}, + Active, ActivityStatus, + Effects) -> + Ack = maps:get(ack, Meta, undefined), + Prefetch = maps:get(prefetch, Meta, undefined), + Args = maps:get(args, Meta, []), + [{mod_call, rabbit_quorum_queue, update_consumer_handler, + [QName, {CTag, CPid}, false, Ack, Prefetch, Active, ActivityStatus, Args]} + | Effects]. + +cancel_consumer0(Meta, ConsumerKey, + #?STATE{consumers = C0} = S0, Effects0, Reason) -> + case C0 of + #{ConsumerKey := Consumer} -> + {S, Effects2} = maybe_return_all(Meta, ConsumerKey, Consumer, + S0, Effects0, Reason), + + %% The effects are emitted before the consumer is actually removed + %% if the consumer has unacked messages. This is a bit weird but + %% in line with what classic queues do (from an external point of + %% view) + Effects = cancel_consumer_effects(consumer_id(Consumer), S, Effects2), + {S, Effects}; + _ -> + %% already removed: do nothing + {S0, Effects0} + end. + +activate_next_consumer({State, Effects}) -> + activate_next_consumer(State, Effects). + +activate_next_consumer(#?STATE{cfg = #cfg{consumer_strategy = competing}} = State, + Effects) -> + {State, Effects}; +activate_next_consumer(#?STATE{consumers = Cons0, + waiting_consumers = Waiting0} = State0, + Effects0) -> + %% invariant, the waiting list always need to be sorted by consumers that are + %% up - then by priority + NextConsumer = + case Waiting0 of + [{_, #consumer{status = up}} = Next | _] -> + Next; + _ -> + undefined + end, + + case {active_consumer(Cons0), NextConsumer} of + {undefined, {NextCKey, #consumer{cfg = NextCCfg} = NextC}} -> + Remaining = tl(Waiting0), + %% TODO: can this happen? + Consumer = case maps:get(NextCKey, Cons0, undefined) of + undefined -> + NextC; + Existing -> + %% there was an exisiting non-active consumer + %% just update the existing cancelled consumer + %% with the new config + Existing#consumer{cfg = NextCCfg} + end, + #?STATE{service_queue = ServiceQueue} = State0, + ServiceQueue1 = maybe_queue_consumer(NextCKey, + Consumer, + ServiceQueue), + State = State0#?STATE{consumers = Cons0#{NextCKey => Consumer}, + service_queue = ServiceQueue1, + waiting_consumers = Remaining}, + Effects = consumer_update_active_effects(State, Consumer, + true, single_active, + Effects0), + {State, Effects}; + {{ActiveCKey, ?CONSUMER_PRIORITY(ActivePriority) = + #consumer{checked_out = ActiveChecked} = Active}, + {NextCKey, ?CONSUMER_PRIORITY(WaitingPriority) = Consumer}} + when WaitingPriority > ActivePriority andalso + map_size(ActiveChecked) == 0 -> + Remaining = tl(Waiting0), + %% the next consumer is a higher priority and should take over + %% and this consumer does not have any pending messages + #?STATE{service_queue = ServiceQueue} = State0, + ServiceQueue1 = maybe_queue_consumer(NextCKey, + Consumer, + ServiceQueue), + Cons1 = Cons0#{NextCKey => Consumer}, + Cons = maps:remove(ActiveCKey, Cons1), + Waiting = add_waiting({ActiveCKey, Active}, Remaining), + State = State0#?STATE{consumers = Cons, + service_queue = ServiceQueue1, + waiting_consumers = Waiting}, + Effects1 = consumer_update_active_effects(State, Active, + false, waiting, + Effects0), + Effects = consumer_update_active_effects(State, Consumer, + true, single_active, + Effects1), + {State, Effects}; + {{ActiveCKey, ?CONSUMER_PRIORITY(ActivePriority) = Active}, + {_NextCKey, ?CONSUMER_PRIORITY(WaitingPriority)}} + when WaitingPriority > ActivePriority -> + %% A higher priority consumer has attached but the current one has + %% pending messages + Cons = maps:update(ActiveCKey, + Active#consumer{status = quiescing}, + Cons0), + {State0#?STATE{consumers = Cons}, Effects0}; + _ -> + %% no activation + {State0, Effects0} + end. + +active_consumer({CKey, #consumer{status = Status} = Consumer, _I}) + when Status == up orelse Status == quiescing -> + {CKey, Consumer}; +active_consumer({_CKey, #consumer{status = _}, I}) -> + active_consumer(maps:next(I)); +active_consumer(none) -> + undefined; +active_consumer(M) when is_map(M) -> + I = maps:iterator(M), + active_consumer(maps:next(I)). + +is_active(_ConsumerKey, #?STATE{cfg = #cfg{consumer_strategy = competing}}) -> + %% all competing consumers are potentially active + true; +is_active(ConsumerKey, #?STATE{cfg = #cfg{consumer_strategy = single_active}, + consumers = Consumers}) -> + ConsumerKey == active_consumer(Consumers). + +maybe_return_all(#{system_time := Ts} = Meta, ConsumerKey, + #consumer{cfg = CCfg} = Consumer, S0, + Effects0, Reason) -> + case Reason of + cancel -> + {update_or_remove_con( + Meta, ConsumerKey, + Consumer#consumer{cfg = CCfg#consumer_cfg{lifetime = once}, + credit = 0, + status = cancelled}, + S0), Effects0}; + _ -> + {S1, Effects} = return_all(Meta, S0, Effects0, ConsumerKey, + Consumer, Reason == down), + {S1#?STATE{consumers = maps:remove(ConsumerKey, S1#?STATE.consumers), + last_active = Ts}, + Effects} + end. + +apply_enqueue(#{index := RaftIdx, + system_time := Ts} = Meta, From, + Seq, RawMsg, Size, State0) -> + Effects0 = [{aux, {bytes_in, Size}}], + case maybe_enqueue(RaftIdx, Ts, From, Seq, RawMsg, Size, + Effects0, State0) of + {ok, State1, Effects1} -> + checkout(Meta, State0, State1, Effects1); + {out_of_sequence, State, Effects} -> + {State, not_enqueued, Effects}; + {duplicate, State, Effects} -> + {State, ok, Effects} + end. + +decr_total(#?STATE{messages_total = Tot} = State) -> + State#?STATE{messages_total = Tot - 1}. + +drop_head(#?STATE{ra_indexes = Indexes0} = State0, Effects) -> + case take_next_msg(State0) of + {?MSG(Idx, Header) = Msg, State1} -> + Indexes = rabbit_fifo_index:delete(Idx, Indexes0), + State2 = State1#?STATE{ra_indexes = Indexes}, + State3 = decr_total(add_bytes_drop(Header, State2)), + #?STATE{cfg = #cfg{dead_letter_handler = DLH}, + dlx = DlxState} = State = State3, + {_, DlxEffects} = rabbit_fifo_dlx:discard([Msg], maxlen, DLH, DlxState), + {State, combine_effects(DlxEffects, Effects)}; + empty -> + {State0, Effects} + end. + +%% combine global counter update effects to avoid bulding a huge list of +%% effects if many messages are dropped at the same time as could happen +%% when the `max_length' is changed via a configuration update. +combine_effects([{mod_call, + rabbit_global_counters, + messages_dead_lettered, + [Reason, rabbit_quorum_queue, Type, NewLen]}], + [{mod_call, + rabbit_global_counters, + messages_dead_lettered, + [Reason, rabbit_quorum_queue, Type, PrevLen]} | Rem]) -> + [{mod_call, + rabbit_global_counters, + messages_dead_lettered, + [Reason, rabbit_quorum_queue, Type, PrevLen + NewLen]} | Rem]; +combine_effects(New, Old) -> + New ++ Old. + + +maybe_set_msg_ttl(Msg, RaCmdTs, Header, + #?STATE{cfg = #cfg{msg_ttl = MsgTTL}}) -> + case mc:is(Msg) of + true -> + TTL = min(MsgTTL, mc:ttl(Msg)), + update_expiry_header(RaCmdTs, TTL, Header); + false -> + Header + end. + +maybe_set_msg_delivery_count(Msg, Header) -> + case mc:is(Msg) of + true -> + case mc:get_annotation(delivery_count, Msg) of + undefined -> + Header; + DelCnt -> + update_header(delivery_count, fun (_) -> DelCnt end, + DelCnt, Header) + end; + false -> + Header + end. + +update_expiry_header(_, undefined, Header) -> + Header; +update_expiry_header(RaCmdTs, 0, Header) -> + %% We do not comply exactly with the "TTL=0 models AMQP immediate flag" semantics + %% as done for classic queues where the message is discarded if it cannot be + %% consumed immediately. + %% Instead, we discard the message if it cannot be consumed within the same millisecond + %% when it got enqueued. This behaviour should be good enough. + update_expiry_header(RaCmdTs + 1, Header); +update_expiry_header(RaCmdTs, TTL, Header) -> + update_expiry_header(RaCmdTs + TTL, Header). + +update_expiry_header(ExpiryTs, Header) -> + update_header(expiry, fun(Ts) -> Ts end, ExpiryTs, Header). + +maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, + {_MetaSize, BodySize}, + Effects, #?STATE{msg_bytes_enqueue = Enqueue, + enqueue_count = EnqCount, + messages = Messages, + messages_total = Total} = State0) -> + % direct enqueue without tracking + Size = BodySize, + Header0 = maybe_set_msg_ttl(RawMsg, Ts, BodySize, State0), + Header = maybe_set_msg_delivery_count(RawMsg, Header0), + Msg = ?MSG(RaftIdx, Header), + PTag = priority_tag(RawMsg), + State = State0#?STATE{msg_bytes_enqueue = Enqueue + Size, + enqueue_count = EnqCount + 1, + messages_total = Total + 1, + messages = rabbit_fifo_q:in(PTag, Msg, Messages) + }, + {ok, State, Effects}; +maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, + {_MetaSize, BodySize} = Size, + Effects0, #?STATE{msg_bytes_enqueue = Enqueue, + enqueue_count = EnqCount, + enqueuers = Enqueuers0, + messages = Messages, + messages_total = Total} = State0) -> + + case maps:get(From, Enqueuers0, undefined) of + undefined -> + State1 = State0#?STATE{enqueuers = Enqueuers0#{From => #enqueuer{}}}, + {Res, State, Effects} = maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, + RawMsg, Size, Effects0, + State1), + {Res, State, [{monitor, process, From} | Effects]}; + #enqueuer{next_seqno = MsgSeqNo} = Enq0 -> + % it is the next expected seqno + Header0 = maybe_set_msg_ttl(RawMsg, Ts, BodySize, State0), + Header = maybe_set_msg_delivery_count(RawMsg, Header0), + Msg = ?MSG(RaftIdx, Header), + Enq = Enq0#enqueuer{next_seqno = MsgSeqNo + 1}, + MsgCache = case can_immediately_deliver(State0) of + true -> + {RaftIdx, RawMsg}; + false -> + undefined + end, + PTag = priority_tag(RawMsg), + State = State0#?STATE{msg_bytes_enqueue = Enqueue + BodySize, + enqueue_count = EnqCount + 1, + messages_total = Total + 1, + messages = rabbit_fifo_q:in(PTag, Msg, Messages), + enqueuers = Enqueuers0#{From => Enq}, + msg_cache = MsgCache + }, + {ok, State, Effects0}; + #enqueuer{next_seqno = Next} + when MsgSeqNo > Next -> + %% TODO: when can this happen? + {out_of_sequence, State0, Effects0}; + #enqueuer{next_seqno = Next} when MsgSeqNo =< Next -> + % duplicate delivery + {duplicate, State0, Effects0} + end. + +return(#{machine_version := MacVer} = Meta, ConsumerKey, + MsgIds, IncrDelCount, Anns, Checked, Effects0, State0) + when is_map(Anns) -> + %% We requeue in the same order as messages got returned by the client. + {State1, Effects1} = + lists:foldl( + fun(MsgId, Acc = {S0, E0}) -> + case Checked of + #{MsgId := Msg} -> + return_one(Meta, MsgId, Msg, IncrDelCount, Anns, + S0, E0, ConsumerKey); + #{} -> + Acc + end + end, {State0, Effects0}, MsgIds), + State2 = case State1#?STATE.consumers of + #{ConsumerKey := Con} -> + update_or_remove_con(Meta, ConsumerKey, Con, State1); + _ -> + State1 + end, + {State3, Effects2} = case MacVer >= 7 of + true -> + activate_next_consumer({State2, Effects1}); + false -> + {State2, Effects1} + end, + checkout(Meta, State0, State3, Effects2). + +% used to process messages that are finished +complete(Meta, ConsumerKey, [MsgId], + #consumer{checked_out = Checked0} = Con0, + #?STATE{ra_indexes = Indexes0, + msg_bytes_checkout = BytesCheckout, + messages_total = Tot} = State0) -> + case maps:take(MsgId, Checked0) of + {?MSG(Idx, Hdr), Checked} -> + SettledSize = get_header(size, Hdr), + Indexes = rabbit_fifo_index:delete(Idx, Indexes0), + Con = Con0#consumer{checked_out = Checked, + credit = increase_credit(Con0, 1)}, + State1 = update_or_remove_con(Meta, ConsumerKey, Con, State0), + State1#?STATE{ra_indexes = Indexes, + msg_bytes_checkout = BytesCheckout - SettledSize, + messages_total = Tot - 1}; + error -> + State0 + end; +complete(Meta, ConsumerKey, MsgIds, + #consumer{checked_out = Checked0} = Con0, + #?STATE{ra_indexes = Indexes0, + msg_bytes_checkout = BytesCheckout, + messages_total = Tot} = State0) -> + {SettledSize, Checked, Indexes} + = lists:foldl( + fun (MsgId, {S0, Ch0, Idxs}) -> + case maps:take(MsgId, Ch0) of + {?MSG(Idx, Hdr), Ch} -> + S = get_header(size, Hdr) + S0, + {S, Ch, rabbit_fifo_index:delete(Idx, Idxs)}; + error -> + {S0, Ch0, Idxs} + end + end, {0, Checked0, Indexes0}, MsgIds), + Len = map_size(Checked0) - map_size(Checked), + Con = Con0#consumer{checked_out = Checked, + credit = increase_credit(Con0, Len)}, + State1 = update_or_remove_con(Meta, ConsumerKey, Con, State0), + State1#?STATE{ra_indexes = Indexes, + msg_bytes_checkout = BytesCheckout - SettledSize, + messages_total = Tot - Len}. + +increase_credit(#consumer{cfg = #consumer_cfg{lifetime = once}, + credit = Credit}, _) -> + %% once consumers cannot increment credit + Credit; +increase_credit(#consumer{cfg = #consumer_cfg{lifetime = auto, + credit_mode = credited}, + credit = Credit}, _) -> + %% credit_mode: `credited' also doesn't automatically increment credit + Credit; +increase_credit(#consumer{cfg = #consumer_cfg{lifetime = auto, + credit_mode = {credited, _}}, + credit = Credit}, _) -> + %% credit_mode: `credited' also doesn't automatically increment credit + Credit; +increase_credit(#consumer{cfg = #consumer_cfg{credit_mode = + {simple_prefetch, MaxCredit}}, + credit = Current}, Credit) + when MaxCredit > 0 -> + min(MaxCredit, Current + Credit); +increase_credit(#consumer{credit = Current}, Credit) -> + Current + Credit. + +complete_and_checkout(#{} = Meta, MsgIds, ConsumerKey, + #consumer{} = Con0, + Effects0, State0) -> + State1 = complete(Meta, ConsumerKey, MsgIds, Con0, State0), + %% a completion could have removed the active/quiescing consumer + Effects1 = add_active_effect(Con0, State1, Effects0), + {State2, Effects2} = activate_next_consumer(State1, Effects1), + checkout(Meta, State0, State2, Effects2). + +add_active_effect(#consumer{status = quiescing} = Consumer, + #?STATE{cfg = #cfg{consumer_strategy = single_active}, + consumers = Consumers} = State, + Effects) -> + case active_consumer(Consumers) of + undefined -> + consumer_update_active_effects(State, Consumer, false, waiting, Effects); + _ -> + Effects + end; +add_active_effect(_, _, Effects) -> + Effects. + +cancel_consumer_effects(ConsumerId, + #?STATE{cfg = #cfg{resource = QName}}, + Effects) when is_tuple(ConsumerId) -> + [{mod_call, rabbit_quorum_queue, + cancel_consumer_handler, [QName, ConsumerId]} | Effects]. + +update_msg_header(Key, Fun, Def, ?MSG(Idx, Header)) -> + ?MSG(Idx, update_header(Key, Fun, Def, Header)). + +update_header(expiry, _, Expiry, Size) + when is_integer(Size) -> + ?TUPLE(Size, Expiry); +update_header(Key, UpdateFun, Default, Size) + when is_integer(Size) -> + update_header(Key, UpdateFun, Default, #{size => Size}); +update_header(Key, UpdateFun, Default, ?TUPLE(Size, Expiry)) + when is_integer(Size) andalso + is_integer(Expiry) -> + update_header(Key, UpdateFun, Default, #{size => Size, + expiry => Expiry}); +update_header(Key, UpdateFun, Default, Header) + when is_map_key(size, Header) -> + maps:update_with(Key, UpdateFun, Default, Header). + +get_msg_header(?MSG(_Idx, Header)) -> + Header. + +get_header(size, Size) + when is_integer(Size) -> + Size; +get_header(_Key, Size) + when is_integer(Size) -> + undefined; +get_header(size, ?TUPLE(Size, Expiry)) + when is_integer(Size), is_integer(Expiry) -> + Size; +get_header(expiry, ?TUPLE(Size, Expiry)) + when is_integer(Size), is_integer(Expiry) -> + Expiry; +get_header(_Key, ?TUPLE(Size, Expiry)) + when is_integer(Size), is_integer(Expiry) -> + undefined; +get_header(Key, Header) + when is_map(Header) andalso is_map_key(size, Header) -> + maps:get(Key, Header, undefined). + +annotate_msg(Header, Msg0) -> + case mc:is(Msg0) of + true when is_map(Header) -> + Msg = maps:fold(fun (K, V, Acc) -> + mc:set_annotation(K, V, Acc) + end, Msg0, maps:get(anns, Header, #{})), + case Header of + #{delivery_count := DelCount} -> + mc:set_annotation(delivery_count, DelCount, Msg); + _ -> + Msg + end; + _ -> + Msg0 + end. + +return_one(Meta, MsgId, ?MSG(_, _) = Msg0, DelivFailed, Anns, + #?STATE{returns = Returns, + consumers = Consumers, + dlx = DlxState0, + cfg = #cfg{delivery_limit = DeliveryLimit, + dead_letter_handler = DLH}} = State0, + Effects0, ConsumerKey) -> + #consumer{checked_out = Checked0} = Con0 = maps:get(ConsumerKey, Consumers), + Msg = incr_msg(Msg0, DelivFailed, Anns), + Header = get_msg_header(Msg), + case get_header(acquired_count, Header) of + AcquiredCount when AcquiredCount > DeliveryLimit -> + {DlxState, DlxEffects} = + rabbit_fifo_dlx:discard([Msg], delivery_limit, DLH, DlxState0), + State1 = State0#?STATE{dlx = DlxState}, + State = complete(Meta, ConsumerKey, [MsgId], Con0, State1), + {State, DlxEffects ++ Effects0}; + _ -> + Checked = maps:remove(MsgId, Checked0), + Con = Con0#consumer{checked_out = Checked, + credit = increase_credit(Con0, 1)}, + {add_bytes_return( + Header, + State0#?STATE{consumers = Consumers#{ConsumerKey => Con}, + returns = lqueue:in(Msg, Returns)}), + Effects0} + end. + +return_all(Meta, #?STATE{consumers = Cons} = State0, Effects0, ConsumerKey, + #consumer{checked_out = Checked} = Con, DelivFailed) -> + State = State0#?STATE{consumers = Cons#{ConsumerKey => Con}}, + lists:foldl(fun ({MsgId, Msg}, {S, E}) -> + return_one(Meta, MsgId, Msg, DelivFailed, #{}, + S, E, ConsumerKey) + end, {State, Effects0}, lists:sort(maps:to_list(Checked))). + +checkout(Meta, OldState, State0, Effects0) -> + checkout(Meta, OldState, State0, Effects0, ok). + +checkout(#{index := Index} = Meta, + #?STATE{} = OldState, + State0, Effects0, Reply) -> + {#?STATE{cfg = #cfg{dead_letter_handler = DLH}, + dlx = DlxState0} = State1, _ExpiredMsg, Effects1} = + checkout0(Meta, checkout_one(Meta, false, State0, Effects0), #{}), + {DlxState, DlxDeliveryEffects} = rabbit_fifo_dlx:checkout(DLH, DlxState0), + %% TODO: only update dlx state if it has changed? + %% by this time the cache should be used + State2 = State1#?STATE{msg_cache = undefined, + dlx = DlxState}, + Effects2 = DlxDeliveryEffects ++ Effects1, + case evaluate_limit(Index, false, OldState, State2, Effects2) of + {State, _, Effects} -> + {State, Reply, Effects} + end. + +checkout0(Meta, {success, ConsumerKey, MsgId, + ?MSG(_, _) = Msg, ExpiredMsg, State, Effects}, + SendAcc0) -> + DelMsg = {MsgId, Msg}, + SendAcc = case maps:get(ConsumerKey, SendAcc0, undefined) of + undefined -> + SendAcc0#{ConsumerKey => [DelMsg]}; + LogMsgs -> + SendAcc0#{ConsumerKey => [DelMsg | LogMsgs]} + end, + checkout0(Meta, checkout_one(Meta, ExpiredMsg, State, Effects), SendAcc); +checkout0(_Meta, {_Activity, ExpiredMsg, State0, Effects0}, SendAcc) -> + Effects = add_delivery_effects(Effects0, SendAcc, State0), + {State0, ExpiredMsg, lists:reverse(Effects)}. + +evaluate_limit(_Index, Result, + #?STATE{cfg = #cfg{max_length = undefined, + max_bytes = undefined}}, + #?STATE{cfg = #cfg{max_length = undefined, + max_bytes = undefined}} = State, + Effects) -> + {State, Result, Effects}; +evaluate_limit(_Index, Result, _BeforeState, + #?STATE{cfg = #cfg{max_length = undefined, + max_bytes = undefined}, + enqueuers = Enqs0} = State0, + Effects0) -> + %% max_length and/or max_bytes policies have just been deleted + {Enqs, Effects} = unblock_enqueuers(Enqs0, Effects0), + {State0#?STATE{enqueuers = Enqs}, Result, Effects}; +evaluate_limit(Index, Result, BeforeState, + #?STATE{cfg = #cfg{overflow_strategy = Strategy}, + enqueuers = Enqs0} = State0, + Effects0) -> + case is_over_limit(State0) of + true when Strategy == drop_head -> + {State, Effects} = drop_head(State0, Effects0), + evaluate_limit(Index, true, BeforeState, State, Effects); + true when Strategy == reject_publish -> + %% generate send_msg effect for each enqueuer to let them know + %% they need to block + {Enqs, Effects} = + maps:fold( + fun (P, #enqueuer{blocked = undefined} = E0, {Enqs, Acc}) -> + E = E0#enqueuer{blocked = Index}, + {Enqs#{P => E}, + [{send_msg, P, {queue_status, reject_publish}, + [ra_event]} | Acc]}; + (_P, _E, Acc) -> + Acc + end, {Enqs0, Effects0}, Enqs0), + {State0#?STATE{enqueuers = Enqs}, Result, Effects}; + false when Strategy == reject_publish -> + %% TODO: optimise as this case gets called for every command + %% pretty much + Before = is_below_soft_limit(BeforeState), + case {Before, is_below_soft_limit(State0)} of + {false, true} -> + %% we have moved below the lower limit + {Enqs, Effects} = unblock_enqueuers(Enqs0, Effects0), + {State0#?STATE{enqueuers = Enqs}, Result, Effects}; + _ -> + {State0, Result, Effects0} + end; + false -> + {State0, Result, Effects0} + end. + +unblock_enqueuers(Enqs0, Effects0) -> + maps:fold( + fun (P, #enqueuer{} = E0, {Enqs, Acc}) -> + E = E0#enqueuer{blocked = undefined}, + {Enqs#{P => E}, + [{send_msg, P, {queue_status, go}, [ra_event]} + | Acc]}; + (_P, _E, Acc) -> + Acc + end, {Enqs0, Effects0}, Enqs0). + +%% [6,5,4,3,2,1] -> [[1,2],[3,4],[5,6]] +chunk_disk_msgs([], _Bytes, [[] | Chunks]) -> + Chunks; +chunk_disk_msgs([], _Bytes, Chunks) -> + Chunks; +chunk_disk_msgs([{_MsgId, ?MSG(_RaftIdx, Header)} = Msg | Rem], + Bytes, Chunks) + when Bytes >= ?DELIVERY_CHUNK_LIMIT_B -> + Size = get_header(size, Header), + chunk_disk_msgs(Rem, Size, [[Msg] | Chunks]); +chunk_disk_msgs([{_MsgId, ?MSG(_RaftIdx, Header)} = Msg | Rem], Bytes, + [CurChunk | Chunks]) -> + Size = get_header(size, Header), + chunk_disk_msgs(Rem, Bytes + Size, [[Msg | CurChunk] | Chunks]). + +add_delivery_effects(Effects0, AccMap, _State) + when map_size(AccMap) == 0 -> + %% does this ever happen? + Effects0; +add_delivery_effects(Effects0, AccMap, State) -> + maps:fold(fun (C, DiskMsgs, Efs) + when is_list(DiskMsgs) -> + lists:foldl( + fun (Msgs, E) -> + [delivery_effect(C, Msgs, State) | E] + end, Efs, chunk_disk_msgs(DiskMsgs, 0, [[]])) + end, Effects0, AccMap). + +take_next_msg(#?STATE{returns = Returns0, + messages = Messages0, + ra_indexes = Indexes0 + } = State) -> + case lqueue:out(Returns0) of + {{value, NextMsg}, Returns} -> + {NextMsg, State#?STATE{returns = Returns}}; + {empty, _} -> + case rabbit_fifo_q:out(Messages0) of + empty -> + empty; + {?MSG(RaftIdx, _) = Msg, Messages} -> + %% add index here + Indexes = rabbit_fifo_index:append(RaftIdx, Indexes0), + {Msg, State#?STATE{messages = Messages, + ra_indexes = Indexes}} + end + end. + +get_next_msg(#?STATE{returns = Returns0, + messages = Messages0}) -> + case lqueue:get(Returns0, empty) of + empty -> + rabbit_fifo_q:get(Messages0); + Msg -> + Msg + end. + +delivery_effect(ConsumerKey, [{MsgId, ?MSG(Idx, Header)}], + #?STATE{msg_cache = {Idx, RawMsg}} = State) -> + {CTag, CPid} = consumer_id(ConsumerKey, State), + {send_msg, CPid, {delivery, CTag, [{MsgId, {Header, RawMsg}}]}, + ?DELIVERY_SEND_MSG_OPTS}; +delivery_effect(ConsumerKey, Msgs, #?STATE{} = State) -> + {CTag, CPid} = consumer_id(ConsumerKey, State), + {RaftIdxs, _Num} = lists:foldr(fun ({_, ?MSG(I, _)}, {Acc, N}) -> + {[I | Acc], N+1} + end, {[], 0}, Msgs), + {log_ext, RaftIdxs, + fun (ReadPlan) -> + case node(CPid) == node() of + true -> + [{send_msg, CPid, {delivery, CTag, ReadPlan, Msgs}, + ?DELIVERY_SEND_MSG_OPTS}]; + false -> + %% if we got there we need to read the data on this node + %% and send it to the consumer pid as it isn't availble + %% locally + {DelMsgs, Flru} = exec_read(undefined, ReadPlan, Msgs), + %% we need to evict all cached items here + _ = ra_flru:evict_all(Flru), + [{send_msg, CPid, {delivery, CTag, DelMsgs}, + ?DELIVERY_SEND_MSG_OPTS}] + end + end, + {local, node(CPid)}}. + +reply_log_effect(RaftIdx, MsgId, Header, Ready, From) -> + {log, [RaftIdx], + fun ([]) -> + []; + ([Cmd]) -> + [{reply, From, {wrap_reply, + {dequeue, {MsgId, {Header, get_msg(Cmd)}}, Ready}}}] + end}. + +checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> + %% Before checking out any messsage to any consumer, + %% first remove all expired messages from the head of the queue. + {ExpiredMsg, #?STATE{service_queue = SQ0, + messages = Messages0, + msg_bytes_checkout = BytesCheckout, + msg_bytes_enqueue = BytesEnqueue, + consumers = Cons0} = InitState, Effects1} = + expire_msgs(Ts, ExpiredMsg0, InitState0, Effects0), + + case priority_queue:out(SQ0) of + {{value, ConsumerKey}, SQ1} + when is_map_key(ConsumerKey, Cons0) -> + case take_next_msg(InitState) of + {Msg, State0} -> + %% there are consumers waiting to be serviced + %% process consumer checkout + case maps:get(ConsumerKey, Cons0) of + #consumer{credit = Credit, + status = Status} + when Credit =:= 0 orelse + Status =/= up -> + %% not an active consumer but still in the consumers + %% map - this can happen when draining + %% or when higher priority single active consumers + %% take over, recurse without consumer in service + %% queue + checkout_one(Meta, ExpiredMsg, + InitState#?STATE{service_queue = SQ1}, + Effects1); + #consumer{checked_out = Checked0, + next_msg_id = Next, + credit = Credit, + delivery_count = DelCnt0, + cfg = Cfg} = Con0 -> + Checked = maps:put(Next, Msg, Checked0), + DelCnt = case credit_api_v2(Cfg) of + true -> add(DelCnt0, 1); + false -> DelCnt0 + 1 + end, + Con = Con0#consumer{checked_out = Checked, + next_msg_id = Next + 1, + credit = Credit - 1, + delivery_count = DelCnt}, + Size = get_header(size, get_msg_header(Msg)), + State1 = + State0#?STATE{service_queue = SQ1, + msg_bytes_checkout = BytesCheckout + Size, + msg_bytes_enqueue = BytesEnqueue - Size}, + State = update_or_remove_con( + Meta, ConsumerKey, Con, State1), + {success, ConsumerKey, Next, Msg, ExpiredMsg, + State, Effects1} + end; + empty -> + {nochange, ExpiredMsg, InitState, Effects1} + end; + {{value, _ConsumerId}, SQ1} -> + %% consumer was not active but was queued, recurse + checkout_one(Meta, ExpiredMsg, + InitState#?STATE{service_queue = SQ1}, Effects1); + {empty, _} -> + case rabbit_fifo_q:len(Messages0) of + 0 -> + {nochange, ExpiredMsg, InitState, Effects1}; + _ -> + {inactive, ExpiredMsg, InitState, Effects1} + end + end. + +%% dequeue all expired messages +expire_msgs(RaCmdTs, Result, State, Effects) -> + %% In the normal case, there are no expired messages. + %% Therefore, first lqueue:get/2 to check whether we need to lqueue:out/1 + %% because the latter can be much slower than the former. + case get_next_msg(State) of + ?MSG(_, ?TUPLE(Size, Expiry)) + when is_integer(Size), is_integer(Expiry), RaCmdTs >= Expiry -> + expire(RaCmdTs, State, Effects); + ?MSG(_, #{expiry := Expiry}) + when is_integer(Expiry), RaCmdTs >= Expiry -> + expire(RaCmdTs, State, Effects); + _ -> + {Result, State, Effects} + end. + +expire(RaCmdTs, State0, Effects) -> + {?MSG(Idx, Header) = Msg, + #?STATE{cfg = #cfg{dead_letter_handler = DLH}, + dlx = DlxState0, + ra_indexes = Indexes0, + messages_total = Tot, + msg_bytes_enqueue = MsgBytesEnqueue} = State1} = + take_next_msg(State0), + {DlxState, DlxEffects} = rabbit_fifo_dlx:discard([Msg], expired, + DLH, DlxState0), + Indexes = rabbit_fifo_index:delete(Idx, Indexes0), + State = State1#?STATE{dlx = DlxState, + ra_indexes = Indexes, + messages_total = Tot - 1, + msg_bytes_enqueue = + MsgBytesEnqueue - get_header(size, Header)}, + expire_msgs(RaCmdTs, true, State, DlxEffects ++ Effects). + +timer_effect(RaCmdTs, State, Effects) -> + T = case get_next_msg(State) of + ?MSG(_, ?TUPLE(Size, Expiry)) + when is_integer(Size) andalso + is_integer(Expiry) -> + %% Next message contains 'expiry' header. + %% (Re)set timer so that message will be dropped or + %% dead-lettered on time. + max(0, Expiry - RaCmdTs); + ?MSG(_, #{expiry := Expiry}) + when is_integer(Expiry) -> + max(0, Expiry - RaCmdTs); + _ -> + %% Next message does not contain 'expiry' header. + %% Therefore, do not set timer or cancel timer if it was set. + infinity + end, + [{timer, expire_msgs, T} | Effects]. + +update_or_remove_con(Meta, ConsumerKey, + #consumer{cfg = #consumer_cfg{lifetime = once}, + checked_out = Checked, + credit = 0} = Con, + #?STATE{consumers = Cons} = State) -> + case map_size(Checked) of + 0 -> + #{system_time := Ts} = Meta, + % we're done with this consumer + State#?STATE{consumers = maps:remove(ConsumerKey, Cons), + last_active = Ts}; + _ -> + % there are unsettled items so need to keep around + State#?STATE{consumers = maps:put(ConsumerKey, Con, Cons)} + end; +update_or_remove_con(_Meta, ConsumerKey, + #consumer{status = quiescing, + checked_out = Checked} = Con0, + #?STATE{consumers = Cons, + waiting_consumers = Waiting} = State) + when map_size(Checked) == 0 -> + Con = Con0#consumer{status = up}, + State#?STATE{consumers = maps:remove(ConsumerKey, Cons), + waiting_consumers = add_waiting({ConsumerKey, Con}, Waiting)}; +update_or_remove_con(_Meta, ConsumerKey, + #consumer{} = Con, + #?STATE{consumers = Cons, + service_queue = ServiceQueue} = State) -> + State#?STATE{consumers = maps:put(ConsumerKey, Con, Cons), + service_queue = maybe_queue_consumer(ConsumerKey, Con, + ServiceQueue)}. + +maybe_queue_consumer(Key, #consumer{credit = Credit, + status = up, + cfg = #consumer_cfg{priority = P}}, + ServiceQueue) + when Credit > 0 -> + % TODO: queue:member could surely be quite expensive, however the practical + % number of unique consumers may not be large enough for it to matter + case priority_queue:member(Key, ServiceQueue) of + true -> + ServiceQueue; + false -> + priority_queue:in(Key, P, ServiceQueue) + end; +maybe_queue_consumer(_Key, _Consumer, ServiceQueue) -> + ServiceQueue. + +update_consumer(Meta, ConsumerKey, {Tag, Pid}, ConsumerMeta, + {Life, Mode} = Spec, Priority, + #?STATE{cfg = #cfg{consumer_strategy = competing}, + consumers = Cons0} = State0) -> + Consumer = case Cons0 of + #{ConsumerKey := #consumer{} = Consumer0} -> + merge_consumer(Meta, Consumer0, ConsumerMeta, + Spec, Priority); + _ -> + Credit = included_credit(Mode), + DeliveryCount = initial_delivery_count(Mode), + #consumer{cfg = #consumer_cfg{tag = Tag, + pid = Pid, + lifetime = Life, + meta = ConsumerMeta, + priority = Priority, + credit_mode = Mode}, + credit = Credit, + delivery_count = DeliveryCount} + end, + {Consumer, update_or_remove_con(Meta, ConsumerKey, Consumer, State0)}; +update_consumer(Meta, ConsumerKey, {Tag, Pid}, ConsumerMeta, + {Life, Mode} = Spec, Priority, + #?STATE{cfg = #cfg{consumer_strategy = single_active}, + consumers = Cons0, + waiting_consumers = Waiting0, + service_queue = _ServiceQueue0} = State) -> + %% if it is the current active consumer, just update + %% if it is a cancelled active consumer, add to waiting unless it is the only + %% one, then merge + case active_consumer(Cons0) of + {ConsumerKey, #consumer{status = up} = Consumer0} -> + Consumer = merge_consumer(Meta, Consumer0, ConsumerMeta, + Spec, Priority), + {Consumer, update_or_remove_con(Meta, ConsumerKey, Consumer, State)}; + undefined when is_map_key(ConsumerKey, Cons0) -> + %% there is no active consumer and the current consumer is in the + %% consumers map and thus must be cancelled, in this case we can just + %% merge and effectively make this the current active one + Consumer0 = maps:get(ConsumerKey, Cons0), + Consumer = merge_consumer(Meta, Consumer0, ConsumerMeta, + Spec, Priority), + {Consumer, update_or_remove_con(Meta, ConsumerKey, Consumer, State)}; + _ -> + %% add as a new waiting consumer + Credit = included_credit(Mode), + DeliveryCount = initial_delivery_count(Mode), + Consumer = #consumer{cfg = #consumer_cfg{tag = Tag, + pid = Pid, + lifetime = Life, + meta = ConsumerMeta, + priority = Priority, + credit_mode = Mode}, + credit = Credit, + delivery_count = DeliveryCount}, + Waiting = add_waiting({ConsumerKey, Consumer}, Waiting0), + {Consumer, State#?STATE{waiting_consumers = Waiting}} + end. + +add_waiting({Key, _} = New, Waiting) -> + sort_waiting(lists:keystore(Key, 1, Waiting, New)). + +sort_waiting(Waiting) -> + lists:sort(fun + ({_, ?CONSUMER_PRIORITY(P1) = #consumer{status = up}}, + {_, ?CONSUMER_PRIORITY(P2) = #consumer{status = up}}) + when P1 =/= P2 -> + P2 =< P1; + ({C1, #consumer{status = up, + credit = Cr1}}, + {C2, #consumer{status = up, + credit = Cr2}}) -> + %% both are up, priority the same + if Cr1 == Cr2 -> + %% same credit + %% sort by key, first attached priority + C1 =< C2; + true -> + %% else sort by credit + Cr2 =< Cr1 + end; + (_, {_, #consumer{status = Status}}) -> + %% not up + Status /= up + end, Waiting). + +merge_consumer(_Meta, #consumer{cfg = CCfg, checked_out = Checked} = Consumer, + ConsumerMeta, {Life, Mode}, Priority) -> + Credit = included_credit(Mode), + NumChecked = map_size(Checked), + NewCredit = max(0, Credit - NumChecked), + Consumer#consumer{cfg = CCfg#consumer_cfg{priority = Priority, + meta = ConsumerMeta, + credit_mode = Mode, + lifetime = Life}, + status = up, + credit = NewCredit}. + +included_credit({simple_prefetch, Credit}) -> + Credit; +included_credit({credited, _}) -> + 0; +included_credit(credited) -> + 0. + +credit_active_consumer( + #credit{credit = LinkCreditRcv, + delivery_count = DeliveryCountRcv, + drain = Drain, + consumer_key = ConsumerKey}, + #consumer{delivery_count = DeliveryCountSnd, + cfg = Cfg} = Con0, + Meta, + #?STATE{consumers = Cons0, + service_queue = ServiceQueue0} = State0) -> + LinkCreditSnd = link_credit_snd(DeliveryCountRcv, LinkCreditRcv, + DeliveryCountSnd, Cfg), + %% grant the credit + Con1 = Con0#consumer{credit = LinkCreditSnd}, + ServiceQueue = maybe_queue_consumer(ConsumerKey, Con1, ServiceQueue0), + State1 = State0#?STATE{service_queue = ServiceQueue, + consumers = maps:update(ConsumerKey, Con1, Cons0)}, + {State2, ok, Effects} = checkout(Meta, State0, State1, []), + + #?STATE{consumers = Cons1 = #{ConsumerKey := Con2}} = State2, + #consumer{cfg = #consumer_cfg{pid = CPid, + tag = CTag}, + credit = PostCred, + delivery_count = PostDeliveryCount} = Con2, + Available = messages_ready(State2), + case credit_api_v2(Cfg) of + true -> + {Credit, DeliveryCount, State} = + case Drain andalso PostCred > 0 of + true -> + AdvancedDeliveryCount = add(PostDeliveryCount, PostCred), + ZeroCredit = 0, + Con = Con2#consumer{delivery_count = AdvancedDeliveryCount, + credit = ZeroCredit}, + Cons = maps:update(ConsumerKey, Con, Cons1), + State3 = State2#?STATE{consumers = Cons}, + {ZeroCredit, AdvancedDeliveryCount, State3}; + false -> + {PostCred, PostDeliveryCount, State2} + end, + %% We must send the delivery effects to the queue client + %% before credit_reply such that session process can send to + %% AMQP 1.0 client TRANSFERs before FLOW. + {State, ok, Effects ++ [{send_msg, CPid, + {credit_reply, CTag, DeliveryCount, + Credit, Available, Drain}, + ?DELIVERY_SEND_MSG_OPTS}]}; + false -> + %% We must always send a send_credit_reply because basic.credit + %% is synchronous. + %% Additionally, we keep the bug of credit API v1 that we + %% send to queue client the + %% send_drained reply before the delivery effects (resulting + %% in the wrong behaviour that the session process sends to + %% AMQP 1.0 client the FLOW before the TRANSFERs). + %% We have to keep this bug because old rabbit_fifo_client + %% implementations expect a send_drained Ra reply + %% (they can't handle such a Ra effect). + CreditReply = {send_credit_reply, Available}, + case Drain of + true -> + AdvancedDeliveryCount = PostDeliveryCount + PostCred, + Con = Con2#consumer{delivery_count = AdvancedDeliveryCount, + credit = 0}, + Cons = maps:update(ConsumerKey, Con, Cons1), + State = State2#?STATE{consumers = Cons}, + Reply = {multi, [CreditReply, + {send_drained, {CTag, PostCred}}]}, + {State, Reply, Effects}; + false -> + {State2, CreditReply, Effects} + end + end. + +credit_inactive_consumer( + #credit{credit = LinkCreditRcv, + delivery_count = DeliveryCountRcv, + drain = Drain, + consumer_key = ConsumerKey}, + #consumer{cfg = #consumer_cfg{pid = CPid, + tag = CTag} = Cfg, + delivery_count = DeliveryCountSnd} = Con0, + Waiting0, State0) -> + %% No messages are available for inactive consumers. + Available = 0, + LinkCreditSnd = link_credit_snd(DeliveryCountRcv, + LinkCreditRcv, + DeliveryCountSnd, + Cfg), + case credit_api_v2(Cfg) of + true -> + {Credit, DeliveryCount} = + case Drain of + true -> + %% By issuing drain=true, the client says "either send a transfer or a flow frame". + %% Since there are no messages to send to an inactive consumer, we advance the + %% delivery-count consuming all link-credit and send a credit_reply with drain=true + %% to the session which causes the session to send a flow frame to the client. + AdvancedDeliveryCount = add(DeliveryCountSnd, LinkCreditSnd), + {0, AdvancedDeliveryCount}; + false -> + {LinkCreditSnd, DeliveryCountSnd} + end, + %% Grant the credit. + Con = Con0#consumer{credit = Credit, + delivery_count = DeliveryCount}, + Waiting = add_waiting({ConsumerKey, Con}, Waiting0), + State = State0#?STATE{waiting_consumers = Waiting}, + {State, ok, + {send_msg, CPid, + {credit_reply, CTag, DeliveryCount, Credit, Available, Drain}, + ?DELIVERY_SEND_MSG_OPTS}}; + false -> + %% Credit API v1 doesn't support draining an inactive consumer. + %% Grant the credit. + Con = Con0#consumer{credit = LinkCreditSnd}, + Waiting = add_waiting({ConsumerKey, Con}, Waiting0), + State = State0#?STATE{waiting_consumers = Waiting}, + {State, {send_credit_reply, Available}} + end. + +is_over_limit(#?STATE{cfg = #cfg{max_length = undefined, + max_bytes = undefined}}) -> + false; +is_over_limit(#?STATE{cfg = #cfg{max_length = MaxLength, + max_bytes = MaxBytes}, + msg_bytes_enqueue = BytesEnq, + dlx = DlxState} = State) -> + {NumDlx, BytesDlx} = rabbit_fifo_dlx:stat(DlxState), + (messages_ready(State) + NumDlx > MaxLength) orelse + (BytesEnq + BytesDlx > MaxBytes). + +is_below_soft_limit(#?STATE{cfg = #cfg{max_length = undefined, + max_bytes = undefined}}) -> + false; +is_below_soft_limit(#?STATE{cfg = #cfg{max_length = MaxLength, + max_bytes = MaxBytes}, + msg_bytes_enqueue = BytesEnq, + dlx = DlxState} = State) -> + {NumDlx, BytesDlx} = rabbit_fifo_dlx:stat(DlxState), + is_below(MaxLength, messages_ready(State) + NumDlx) andalso + is_below(MaxBytes, BytesEnq + BytesDlx). + +is_below(undefined, _Num) -> + true; +is_below(Val, Num) when is_integer(Val) andalso is_integer(Num) -> + Num =< trunc(Val * ?LOW_LIMIT). + +-spec make_enqueue(option(pid()), option(msg_seqno()), raw_msg()) -> + protocol(). +make_enqueue(Pid, Seq, Msg) -> + case is_v4() of + true when is_pid(Pid) andalso + is_integer(Seq) -> + %% more compact format + #?ENQ_V2{seq = Seq, + msg = Msg, + size = ?SIZE(Msg)}; + _ -> + #enqueue{pid = Pid, seq = Seq, msg = Msg} + end. + +-spec make_register_enqueuer(pid()) -> protocol(). +make_register_enqueuer(Pid) -> + #register_enqueuer{pid = Pid}. + +-spec make_checkout(consumer_id(), checkout_spec(), consumer_meta()) -> + protocol(). +make_checkout({_, _} = ConsumerId, Spec0, Meta) -> + Spec = case is_v4() of + false when Spec0 == remove -> + %% if v4 is not active, fall back to cancel spec + cancel; + _ -> + Spec0 + end, + #checkout{consumer_id = ConsumerId, + spec = Spec, meta = Meta}. + +-spec make_settle(consumer_key(), [msg_id()]) -> protocol(). +make_settle(ConsumerKey, MsgIds) when is_list(MsgIds) -> + #settle{consumer_key = ConsumerKey, msg_ids = MsgIds}. + +-spec make_return(consumer_key(), [msg_id()]) -> protocol(). +make_return(ConsumerKey, MsgIds) -> + #return{consumer_key = ConsumerKey, msg_ids = MsgIds}. + +-spec is_return(protocol()) -> boolean(). +is_return(Command) -> + is_record(Command, return). + +-spec make_discard(consumer_key(), [msg_id()]) -> protocol(). +make_discard(ConsumerKey, MsgIds) -> + #discard{consumer_key = ConsumerKey, msg_ids = MsgIds}. + +-spec make_credit(consumer_key(), rabbit_queue_type:credit(), + non_neg_integer(), boolean()) -> protocol(). +make_credit(Key, Credit, DeliveryCount, Drain) -> + #credit{consumer_key = Key, + credit = Credit, + delivery_count = DeliveryCount, + drain = Drain}. + +-spec make_modify(consumer_key(), [msg_id()], + boolean(), boolean(), mc:annotations()) -> protocol(). +make_modify(ConsumerKey, MsgIds, DeliveryFailed, UndeliverableHere, Anns) + when is_list(MsgIds) andalso + is_boolean(DeliveryFailed) andalso + is_boolean(UndeliverableHere) andalso + is_map(Anns) -> + case is_v4() of + true -> + #modify{consumer_key = ConsumerKey, + msg_ids = MsgIds, + delivery_failed = DeliveryFailed, + undeliverable_here = UndeliverableHere, + annotations = Anns}; + false when UndeliverableHere -> + make_discard(ConsumerKey, MsgIds); + false -> + make_return(ConsumerKey, MsgIds) + end. + + +-spec make_purge() -> protocol(). +make_purge() -> #purge{}. + +-spec make_garbage_collection() -> protocol(). +make_garbage_collection() -> #garbage_collection{}. + +-spec make_purge_nodes([node()]) -> protocol(). +make_purge_nodes(Nodes) -> + #purge_nodes{nodes = Nodes}. + +-spec make_update_config(config()) -> protocol(). +make_update_config(Config) -> + #update_config{config = Config}. + +add_bytes_drop(Header, + #?STATE{msg_bytes_enqueue = Enqueue} = State) -> + Size = get_header(size, Header), + State#?STATE{msg_bytes_enqueue = Enqueue - Size}. + + +add_bytes_return(Header, + #?STATE{msg_bytes_checkout = Checkout, + msg_bytes_enqueue = Enqueue} = State) -> + Size = get_header(size, Header), + State#?STATE{msg_bytes_checkout = Checkout - Size, + msg_bytes_enqueue = Enqueue + Size}. + +message_size(B) when is_binary(B) -> + byte_size(B); +message_size(Msg) -> + case mc:is(Msg) of + true -> + mc:size(Msg); + false -> + %% probably only hit this for testing so ok to use erts_debug + {0, erts_debug:size(Msg)} + end. + +all_nodes(#?STATE{consumers = Cons0, + enqueuers = Enqs0, + waiting_consumers = WaitingConsumers0}) -> + Nodes0 = maps:fold(fun(_, ?CONSUMER_PID(P), Acc) -> + Acc#{node(P) => ok} + end, #{}, Cons0), + Nodes1 = maps:fold(fun(P, _, Acc) -> + Acc#{node(P) => ok} + end, Nodes0, Enqs0), + maps:keys( + lists:foldl(fun({_, ?CONSUMER_PID(P)}, Acc) -> + Acc#{node(P) => ok} + end, Nodes1, WaitingConsumers0)). + +all_pids_for(Node, Vsn, #?STATE{consumers = Cons0, + enqueuers = Enqs0, + waiting_consumers = WaitingConsumers0}) -> + Cons = rabbit_fifo_maps:fold(fun(_, ?CONSUMER_PID(P), Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, _, Acc) -> + Acc + end, [], Cons0, Vsn), + Enqs = rabbit_fifo_maps:fold(fun(P, _, Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, _, Acc) -> + Acc + end, Cons, Enqs0, Vsn), + lists:foldl(fun({_, ?CONSUMER_PID(P)}, Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, Acc) -> Acc + end, Enqs, WaitingConsumers0). + +suspected_pids_for(Node, Vsn, #?STATE{consumers = Cons0, + enqueuers = Enqs0, + waiting_consumers = WaitingConsumers0}) -> + Cons = rabbit_fifo_maps:fold(fun(_Key, + #consumer{cfg = #consumer_cfg{pid = P}, + status = suspected_down}, + Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, _, Acc) -> + Acc + end, [], Cons0, Vsn), + Enqs = rabbit_fifo_maps:fold(fun(P, #enqueuer{status = suspected_down}, Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, _, Acc) -> + Acc + end, Cons, Enqs0, Vsn), + lists:foldl(fun({_Key, + #consumer{cfg = #consumer_cfg{pid = P}, + status = suspected_down}}, Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, Acc) -> Acc + end, Enqs, WaitingConsumers0). + +is_expired(Ts, #?STATE{cfg = #cfg{expires = Expires}, + last_active = LastActive, + consumers = Consumers}) + when is_number(LastActive) andalso is_number(Expires) -> + %% TODO: should it be active consumers? + Active = maps:filter(fun (_, #consumer{status = suspected_down}) -> + false; + (_, _) -> + true + end, Consumers), + + Ts > (LastActive + Expires) andalso maps:size(Active) == 0; +is_expired(_Ts, _State) -> + false. + +get_priority(#{priority := Priority}) -> + Priority; +get_priority(#{args := Args}) -> + %% fallback, v3 option + case rabbit_misc:table_lookup(Args, <<"x-priority">>) of + {_Type, Value} -> + Value; + _ -> + 0 + end; +get_priority(_) -> + 0. + +notify_decorators_effect(QName, MaxActivePriority, IsEmpty) -> + {mod_call, rabbit_quorum_queue, spawn_notify_decorators, + [QName, consumer_state_changed, [MaxActivePriority, IsEmpty]]}. + +notify_decorators_startup(QName) -> + {mod_call, rabbit_quorum_queue, spawn_notify_decorators, + [QName, startup, []]}. + +convert(_Meta, To, To, State) -> + State; +convert(Meta, 0, To, State) -> + convert(Meta, 1, To, rabbit_fifo_v1:convert_v0_to_v1(State)); +convert(Meta, 1, To, State) -> + convert(Meta, 2, To, rabbit_fifo_v3:convert_v1_to_v2(State)); +convert(Meta, 2, To, State) -> + convert(Meta, 3, To, rabbit_fifo_v3:convert_v2_to_v3(State)); +convert(Meta, 3, To, State) -> + convert(Meta, 4, To, convert_v3_to_v4(Meta, State)); +convert(Meta, 4, To, State) -> + %% no conversion needed, this version only includes a logic change + convert(Meta, 5, To, State); +convert(Meta, 5, To, State) -> + %% no conversion needed, this version only includes a logic change + convert(Meta, 6, To, State); +convert(Meta, 6, To, State) -> + %% no conversion needed, this version only includes a logic change + convert(Meta, 7, To, State). + +smallest_raft_index(#?STATE{messages = Messages, + ra_indexes = Indexes, + dlx = DlxState}) -> + SmallestDlxRaIdx = rabbit_fifo_dlx:smallest_raft_index(DlxState), + SmallestMsgsRaIdx = rabbit_fifo_q:get_lowest_index(Messages), + SmallestRaIdx = rabbit_fifo_index:smallest(Indexes), + min(min(SmallestDlxRaIdx, SmallestMsgsRaIdx), SmallestRaIdx). + +smallest_raft_index_overview(#?STATE{messages = Messages, + ra_indexes = Indexes, + dlx = DlxState}) -> + #{message => rabbit_fifo_q:get_lowest_index(Messages), + checked_out => rabbit_fifo_index:smallest(Indexes), + dlx => rabbit_fifo_dlx:smallest_raft_index(DlxState)}. + +make_requeue(ConsumerKey, Notify, [{MsgId, Idx, Header, Msg}], Acc) -> + lists:reverse([{append, + #requeue{consumer_key = ConsumerKey, + index = Idx, + header = Header, + msg_id = MsgId, + msg = Msg}, + Notify} + | Acc]); +make_requeue(ConsumerKey, Notify, [{MsgId, Idx, Header, Msg} | Rem], Acc) -> + make_requeue(ConsumerKey, Notify, Rem, + [{append, + #requeue{consumer_key = ConsumerKey, + index = Idx, + header = Header, + msg_id = MsgId, + msg = Msg}, + noreply} + | Acc]); +make_requeue(_ConsumerId, _Notify, [], []) -> + []. + +can_immediately_deliver(#?STATE{service_queue = SQ, + consumers = Consumers} = State) -> + case messages_ready(State) of + 0 when map_size(Consumers) > 0 -> + %% TODO: is is probably good enough but to be 100% we'd need to + %% scan all consumers and ensure at least one has credit + priority_queue:is_empty(SQ) == false; + _ -> + false + end. + +incr(I) -> + I + 1. + +get_msg(#?ENQ_V2{msg = M}) -> + M; +get_msg(#enqueue{msg = M}) -> + M; +get_msg(#requeue{msg = M}) -> + M. + +initial_delivery_count({credited, Count}) -> + %% credit API v2 + Count; +initial_delivery_count(_) -> + %% credit API v1 + 0. + +credit_api_v2(#consumer_cfg{credit_mode = {credited, _}}) -> + true; +credit_api_v2(_) -> + false. + +link_credit_snd(DeliveryCountRcv, LinkCreditRcv, DeliveryCountSnd, ConsumerCfg) -> + case credit_api_v2(ConsumerCfg) of + true -> + amqp10_util:link_credit_snd(DeliveryCountRcv, LinkCreditRcv, DeliveryCountSnd); + false -> + C = DeliveryCountRcv + LinkCreditRcv - DeliveryCountSnd, + %% C can be negative when receiver decreases credits while messages are in flight. + max(0, C) + end. + +consumer_id(#consumer{cfg = Cfg}) -> + {Cfg#consumer_cfg.tag, Cfg#consumer_cfg.pid}. + +consumer_id(Key, #?STATE{consumers = Consumers}) + when is_integer(Key) -> + consumer_id(maps:get(Key, Consumers)); +consumer_id({_, _} = ConsumerId, _State) -> + ConsumerId. + + +consumer_key_from_id(ConsumerId, #?STATE{consumers = Consumers}) + when is_map_key(ConsumerId, Consumers) -> + {ok, ConsumerId}; +consumer_key_from_id(ConsumerId, #?STATE{consumers = Consumers, + waiting_consumers = Waiting}) -> + case consumer_key_from_id(ConsumerId, maps:next(maps:iterator(Consumers))) of + {ok, _} = Res -> + Res; + error -> + %% scan the waiting consumers + case lists:search(fun ({_K, ?CONSUMER_TAG_PID(T, P)}) -> + {T, P} == ConsumerId + end, Waiting) of + {value, {K, _}} -> + {ok, K}; + false -> + error + end + end; +consumer_key_from_id({CTag, CPid}, {Key, ?CONSUMER_TAG_PID(T, P), _I}) + when T == CTag andalso P == CPid -> + {ok, Key}; +consumer_key_from_id(ConsumerId, {_, _, I}) -> + consumer_key_from_id(ConsumerId, maps:next(I)); +consumer_key_from_id(_ConsumerId, none) -> + error. + +consumer_cancel_info(ConsumerKey, #?STATE{consumers = Consumers}) -> + case Consumers of + #{ConsumerKey := #consumer{checked_out = Checked}} -> + #{key => ConsumerKey, + num_checked_out => map_size(Checked)}; + _ -> + #{} + end. + +find_consumer(Key, Consumers) -> + case Consumers of + #{Key := Con} -> + {Key, Con}; + _ when is_tuple(Key) -> + %% sometimes rabbit_fifo_client may send a settle, return etc + %% by it's ConsumerId even if it was created with an integer key + %% as it may have lost it's state after a consumer cancel + maps_search(fun (_K, ?CONSUMER_TAG_PID(Tag, Pid)) -> + Key == {Tag, Pid} + end, Consumers); + _ -> + undefined + end. + +maps_search(_Pred, none) -> + undefined; +maps_search(Pred, {K, V, I}) -> + case Pred(K, V) of + true -> + {K, V}; + false -> + maps_search(Pred, maps:next(I)) + end; +maps_search(Pred, Map) when is_map(Map) -> + maps_search(Pred, maps:next(maps:iterator(Map))). + +priority_tag(Msg) -> + case mc:is(Msg) of + true -> + case mc:priority(Msg) of + P when is_integer(P) andalso + P > 4 -> + hi; + _ -> + no + end; + false -> + no + end. + + +do_checkpoints(Ts, #checkpoint{index = ChIdx, + timestamp = ChTime, + smallest_index = LastSmallest, + bytes_in = LastBytesIn, + indexes = MinIndexes} = Check0, + RaAux, BytesIn, Force) -> + LastAppliedIdx = ra_aux:last_applied(RaAux), + IndexesSince = LastAppliedIdx - ChIdx, + #?STATE{} = MacState = ra_aux:machine_state(RaAux), + TimeSince = Ts - ChTime, + NewSmallest = case smallest_raft_index(MacState) of + undefined -> + LastAppliedIdx; + Smallest -> + Smallest + end, + MsgsTot = messages_total(MacState), + %% more than 64MB (by default) of message data has been written to the log + %% best take a checkpoint + + {CheckMinInterval, CheckMinIndexes, CheckMaxIndexes} = + persistent_term:get(quorum_queue_checkpoint_config, + {?CHECK_MIN_INTERVAL_MS, ?CHECK_MIN_INDEXES, + ?CHECK_MAX_INDEXES}), + + %% scale the bytes limit as the backlog increases + MaxBytesFactor = max(1, MsgsTot / CheckMaxIndexes), + EnoughDataWritten = BytesIn - LastBytesIn > (?CHECK_MAX_BYTES * MaxBytesFactor), + EnoughTimeHasPassed = TimeSince > CheckMinInterval, + + case (EnoughTimeHasPassed andalso + ( + %% condition 1: enough indexes have been committed since the last + %% checkpoint + (IndexesSince > MinIndexes) orelse + %% condition 2: the queue is empty and _some_ commands + %% have been applied since the last checkpoint + (MsgsTot == 0 andalso IndexesSince > 32) + ) + ) orelse + %% condition 3: enough message data has been written to warrant a new + %% checkpoint, this ignores the time windowing + EnoughDataWritten orelse + %% force was requested, e.g. after a purge + Force + of + true -> + %% take fewer checkpoints the more messages there are on queue + NextIndexes = min(max(MsgsTot, CheckMinIndexes), CheckMaxIndexes), + %% take a checkpoint; + {#checkpoint{index = LastAppliedIdx, + timestamp = Ts, + smallest_index = NewSmallest, + messages_total = MsgsTot, + bytes_in = BytesIn, + indexes = NextIndexes}, + [{checkpoint, LastAppliedIdx, MacState} | + release_cursor(LastSmallest, NewSmallest)]}; + false -> + {Check0#checkpoint{smallest_index = NewSmallest}, + release_cursor(LastSmallest, NewSmallest)} + end. + +release_cursor(LastSmallest, Smallest) + when is_integer(LastSmallest) andalso + is_integer(Smallest) andalso + Smallest > LastSmallest -> + [{release_cursor, Smallest - 1}]; +release_cursor(undefined, Smallest) + when is_integer(Smallest) -> + [{release_cursor, Smallest - 1}]; +release_cursor(_, _) -> + []. + +discard(Meta, MsgIds, ConsumerKey, + #consumer{checked_out = Checked} = Con, + DelFailed, Anns, + #?STATE{cfg = #cfg{dead_letter_handler = DLH}, + dlx = DlxState0} = State0) -> + %% We publish to dead-letter exchange in the same order + %% as messages got rejected by the client. + DiscardMsgs = lists:filtermap( + fun(Id) -> + case maps:get(Id, Checked, undefined) of + undefined -> + false; + Msg0 -> + {true, incr_msg(Msg0, DelFailed, Anns)} + end + end, MsgIds), + {DlxState, Effects} = rabbit_fifo_dlx:discard(DiscardMsgs, rejected, + DLH, DlxState0), + State = State0#?STATE{dlx = DlxState}, + complete_and_checkout(Meta, MsgIds, ConsumerKey, Con, Effects, State). + +incr_msg(Msg0, DelFailed, Anns) -> + Msg1 = update_msg_header(acquired_count, fun incr/1, 1, Msg0), + Msg2 = case map_size(Anns) > 0 of + true -> + update_msg_header(anns, fun(A) -> + maps:merge(A, Anns) + end, Anns, + Msg1); + false -> + Msg1 + end, + case DelFailed of + true -> + update_msg_header(delivery_count, fun incr/1, 1, Msg2); + false -> + Msg2 + end. + +exec_read(Flru0, ReadPlan, Msgs) -> + try ra_log_read_plan:execute(ReadPlan, Flru0) of + {Entries, Flru} -> + %% return a list in original order + {lists:map(fun ({MsgId, ?MSG(Idx, Header)}) -> + Cmd = maps:get(Idx, Entries), + {MsgId, {Header, get_msg(Cmd)}} + end, Msgs), Flru} + catch exit:{missing_key, _} + when Flru0 =/= undefined -> + %% this segment has most likely been appended to but the + %% cached index doesn't know about new items and need to be + %% re-generated + _ = ra_flru:evict_all(Flru0), + %% retry without segment cache + exec_read(undefined, ReadPlan, Msgs) + end. diff --git a/deps/rabbit/src/rabbit_fifo_v7.hrl b/deps/rabbit/src/rabbit_fifo_v7.hrl new file mode 100644 index 000000000000..b8b69bff7f45 --- /dev/null +++ b/deps/rabbit/src/rabbit_fifo_v7.hrl @@ -0,0 +1,232 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. + +%% macros for memory optimised tuple structures +%% [A|B] saves 1 byte compared to {A,B} +-define(TUPLE(A, B), [A | B]). + +%% We only hold Raft index and message header in memory. +%% Raw message data is always stored on disk. +-define(MSG(Index, Header), ?TUPLE(Index, Header)). + +-define(NIL, []). + +-define(IS_HEADER(H), + (is_integer(H) andalso H >= 0) orelse + is_list(H) orelse + (is_map(H) andalso is_map_key(size, H))). + +-define(DELIVERY_SEND_MSG_OPTS, [local, ra_event]). + +-type optimised_tuple(A, B) :: nonempty_improper_list(A, B). + +-type option(T) :: undefined | T. + +-type raw_msg() :: term(). +%% The raw message. It is opaque to rabbit_fifo. + +-type msg_id() :: non_neg_integer(). +%% A consumer-scoped monotonically incrementing integer included with a +%% {@link delivery/0.}. Used to settle deliveries using +%% {@link rabbit_fifo_client:settle/3.} + +-type msg_seqno() :: non_neg_integer(). +%% A sender process scoped monotonically incrementing integer included +%% in enqueue messages. Used to ensure ordering of messages send from the +%% same process + +-type msg_header() :: msg_size() | + optimised_tuple(msg_size(), Expiry :: milliseconds()) | + #{size := msg_size(), + acquired_count => non_neg_integer(), + delivery_count => non_neg_integer(), + expiry => milliseconds()}. +%% The message header: +%% size: The size of the message payload in bytes. +%% delivery_count: The number of unsuccessful delivery attempts. +%% A non-zero value indicates a previous attempt. +%% return_count: The number of explicit returns. +%% expiry: Epoch time in ms when a message expires. Set during enqueue. +%% Value is determined by per-queue or per-message message TTL. +%% If it contains only the size it can be condensed to an integer. +%% If it contains only the size and expiry it can be condensed to an improper list. + +-type msg_size() :: non_neg_integer(). +%% the size in bytes of the msg payload + +-type msg() :: optimised_tuple(ra:index(), msg_header()). + +-type delivery_msg() :: {msg_id(), {msg_header(), raw_msg()}}. +%% A tuple consisting of the message id, and the headered message. + +-type delivery() :: {delivery, rabbit_types:ctag(), [delivery_msg()]}. +%% Represents the delivery of one or more rabbit_fifo messages. + +-type consumer_id() :: {rabbit_types:ctag(), pid()}. +%% The entity that receives messages. Uniquely identifies a consumer. + +-type consumer_idx() :: ra:index(). +%% v4 can reference consumers by the raft index they were added at. +%% The entity that receives messages. Uniquely identifies a consumer. +-type consumer_key() :: consumer_id() | consumer_idx(). + +-type credit_mode() :: + {credited, InitialDeliveryCount :: rabbit_queue_type:delivery_count()} | + %% machine_version 2 + {simple_prefetch, MaxCredit :: non_neg_integer()}. +%% determines how credit is replenished + +-type checkout_spec() :: {once | auto, + Num :: non_neg_integer(), + credited | simple_prefetch} | + + {dequeue, settled | unsettled} | + cancel | remove | + %% new v4 format + {once | auto, credit_mode()}. + +-type consumer_meta() :: #{ack => boolean(), + username => binary(), + prefetch => non_neg_integer(), + args => list(), + priority => non_neg_integer() + }. +%% static meta data associated with a consumer + +-type applied_mfa() :: {module(), atom(), list()}. +% represents a partially applied module call + +-define(CHECK_MIN_INTERVAL_MS, 1000). +-define(CHECK_MIN_INDEXES, 4096 * 2). +-define(CHECK_MAX_INDEXES, 666_667). +%% once these many bytes have been written since the last checkpoint +%% we request a checkpoint irrespectively +-define(CHECK_MAX_BYTES, 128_000_000). + +-define(USE_AVG_HALF_LIFE, 10000.0). +%% an average QQ without any message uses about 100KB so setting this limit +%% to ~10 times that should be relatively safe. +-define(GC_MEM_LIMIT_B, 2_000_000). + +-define(MB, 1_048_576). +-define(LOW_LIMIT, 0.8). +-define(DELIVERY_CHUNK_LIMIT_B, 128_000). + +-type milliseconds() :: non_neg_integer(). +-record(consumer_cfg, + {meta = #{} :: consumer_meta(), + pid :: pid(), + tag :: rabbit_types:ctag(), + %% the mode of how credit is incremented + %% simple_prefetch: credit is re-filled as deliveries are settled + %% or returned. + %% credited: credit can only be changed by receiving a consumer_credit + %% command: `{credit, ReceiverDeliveryCount, Credit}' + credit_mode :: credited | credit_mode(), + lifetime = once :: once | auto, + priority = 0 :: integer()}). + +-record(consumer, + {cfg = #consumer_cfg{}, + status = up :: up | suspected_down | cancelled | quiescing, + next_msg_id = 0 :: msg_id(), + checked_out = #{} :: #{msg_id() => msg()}, + %% max number of messages that can be sent + %% decremented for each delivery + credit = 0 :: non_neg_integer(), + %% AMQP 1.0 §2.6.7 + delivery_count :: rabbit_queue_type:delivery_count() + }). + +-type consumer() :: #consumer{}. + +-type consumer_strategy() :: competing | single_active. + +-type dead_letter_handler() :: option({at_most_once, applied_mfa()} | at_least_once). + +-record(enqueuer, + {next_seqno = 1 :: msg_seqno(), + % out of order enqueues - sorted list + unused = ?NIL, + status = up :: up | suspected_down, + %% it is useful to have a record of when this was blocked + %% so that we can retry sending the block effect if + %% the publisher did not receive the initial one + blocked :: option(ra:index()), + unused_1 = ?NIL, + unused_2 = ?NIL + }). + +-record(cfg, + {name :: atom(), + resource :: rabbit_types:r('queue'), + unused_1 = ?NIL, + dead_letter_handler :: dead_letter_handler(), + become_leader_handler :: option(applied_mfa()), + overflow_strategy = drop_head :: drop_head | reject_publish, + max_length :: option(non_neg_integer()), + max_bytes :: option(non_neg_integer()), + %% whether single active consumer is on or not for this queue + consumer_strategy = competing :: consumer_strategy(), + %% the maximum number of unsuccessful delivery attempts permitted + delivery_limit :: option(non_neg_integer()), + expires :: option(milliseconds()), + msg_ttl :: option(milliseconds()), + unused_2 = ?NIL, + unused_3 = ?NIL + }). + +-record(rabbit_fifo, + {cfg :: #cfg{}, + % unassigned messages + messages = rabbit_fifo_q:new() :: rabbit_fifo_q:state(), + messages_total = 0 :: non_neg_integer(), + % queue of returned msg_in_ids - when checking out it picks from + returns = lqueue:new() :: lqueue:lqueue(term()), + % a counter of enqueues - used to trigger shadow copy points + % reset to 0 when release_cursor gets stored + enqueue_count = 0 :: non_neg_integer(), + % a map containing all the live processes that have ever enqueued + % a message to this queue + enqueuers = #{} :: #{pid() => #enqueuer{}}, + % index of all messages that have been delivered at least once + % used to work out the smallest live raft index + % rabbit_fifo_index can be slow when calculating the smallest + % index when there are large gaps but should be faster than gb_trees + % for normal appending operations as it's backed by a map + ra_indexes = rabbit_fifo_index:empty() :: rabbit_fifo_index:state(), + unused_1 = ?NIL, + % consumers need to reflect consumer state at time of snapshot + consumers = #{} :: #{consumer_key() => consumer()}, + % consumers that require further service are queued here + service_queue = priority_queue:new() :: priority_queue:q(), + %% state for at-least-once dead-lettering + dlx = rabbit_fifo_dlx:init() :: rabbit_fifo_dlx:state(), + msg_bytes_enqueue = 0 :: non_neg_integer(), + msg_bytes_checkout = 0 :: non_neg_integer(), + %% one is picked if active consumer is cancelled or dies + %% used only when single active consumer is on + waiting_consumers = [] :: [{consumer_key(), consumer()}], + last_active :: option(non_neg_integer()), + msg_cache :: option({ra:index(), raw_msg()}), + unused_2 = ?NIL + }). + +-type config() :: #{name := atom(), + queue_resource := rabbit_types:r('queue'), + dead_letter_handler => dead_letter_handler(), + become_leader_handler => applied_mfa(), + checkpoint_min_indexes => non_neg_integer(), + checkpoint_max_indexes => non_neg_integer(), + max_length => non_neg_integer(), + max_bytes => non_neg_integer(), + overflow_strategy => drop_head | reject_publish, + single_active_consumer_on => boolean(), + delivery_limit => non_neg_integer() | -1, + expires => non_neg_integer(), + msg_ttl => non_neg_integer(), + created => non_neg_integer() + }. diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 91e43af781c7..5ffe1cd35bc9 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -588,8 +588,7 @@ handle_tick(QName, num_discarded := NumDiscarded, num_discard_checked_out := NumDiscardedCheckedOut, discard_message_bytes := DiscardBytes, - discard_checkout_message_bytes := DiscardCheckoutBytes, - smallest_raft_index := _} = Overview, + discard_checkout_message_bytes := DiscardCheckoutBytes} = Overview, Nodes) -> %% this makes calls to remote processes so cannot be run inside the %% ra server diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index f5aed72ef3fe..7e961294a691 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -367,6 +367,7 @@ untracked_enq_deq_test(Config) -> State0), {_State2, _, Effs} = apply(meta(Config, 3), make_checkout(Cid, {dequeue, settled}, #{}), State1), + ct:pal("Effs ~p", [State1]), ?ASSERT_EFF({log, [1], _}, Effs), ok. @@ -2438,7 +2439,7 @@ run_log(Module, Config, InitState, Entries, Invariant) -> aux_test(_) -> _ = ra_machine_ets:start_link(), - Aux0 = init_aux(aux_test), + Aux = init_aux(aux_test), LastApplied = 0, State0 = #{machine_state => init(#{name => ?FUNCTION_NAME, @@ -2449,41 +2450,10 @@ aux_test(_) -> last_applied => LastApplied}, ok = meck:new(ra_log, []), meck:expect(ra_log, last_index_term, fun (_) -> {0, 0} end), - {no_reply, Aux, State} = handle_aux(leader, cast, active, Aux0, State0), - {no_reply, _Aux, _, - [{release_cursor, LastApplied}]} = handle_aux(leader, cast, tick, Aux, State), - [X] = ets:lookup(rabbit_fifo_usage, aux_test), + {no_reply, _Aux, _, []} = handle_aux(leader, cast, tick, Aux, State0), meck:unload(), - ?assert(X > 0.0), ok. -handle_aux_tick_test(Config) -> - _ = ra_machine_ets:start_link(), - Aux0 = init_aux(aux_test), - LastApplied = 1, - MacState0 = init(#{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), - single_active_consumer_on => false}), - State0 = #{machine_state => MacState0, - log => mock_log, - last_applied => LastApplied}, - {MacState1, _} = enq(Config, 1, 1, first, MacState0), - State1 = State0#{machine_state => MacState1}, - meck:expect(ra_log, last_index_term, fun (_) -> {1, 0} end), - ?assertEqual(1, rabbit_fifo:smallest_raft_index(MacState1)), - %% the release cursor should be 1 lower than the smallest raft index - {no_reply, _, _, - [{release_cursor, 0}]} = handle_aux(leader, cast, tick, Aux0, State1), - timer:sleep(10), - - persistent_term:put(quorum_queue_checkpoint_config, {1, 0, 1}), - meck:expect(ra_aux, effective_machine_version, fun (_) -> 1 end), - {no_reply, _, _, - [{checkpoint, 1, _}, - {release_cursor, 0}]} = handle_aux(follower, cast, force_checkpoint, Aux0, State1), - ok. - - %% machine version conversion test machine_version_test(C) -> diff --git a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl index 9c02c32bab5c..a275ede639d5 100644 --- a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl @@ -43,8 +43,7 @@ all_tests() -> untracked_enqueue, flow, test_queries, - duplicate_delivery, - usage + duplicate_delivery ]. groups() -> @@ -280,23 +279,6 @@ duplicate_delivery(Config) -> rabbit_quorum_queue:stop_server(ServerId), ok. -usage(Config) -> - ClusterName = ?config(cluster_name, Config), - ServerId = ?config(node_id, Config), - ok = start_cluster(ClusterName, [ServerId]), - F0 = rabbit_fifo_client:init([ServerId]), - {ok, _, F1} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, #{}, F0), - {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, corr1, msg1, F1), - {ok, F3, []} = rabbit_fifo_client:enqueue(ClusterName, corr2, msg2, F2), - {_, _, _} = process_ra_events(receive_ra_events(2, 2), ClusterName, F3), - % force tick and usage stats emission - ServerId ! tick_timeout, - timer:sleep(50), - Use = rabbit_fifo:usage(element(1, ServerId)), - rabbit_quorum_queue:stop_server(ServerId), - ?assert(Use > 0.0), - ok. - resends_lost_command(Config) -> ClusterName = ?config(cluster_name, Config), ServerId = ?config(node_id, Config), @@ -523,6 +505,7 @@ discard(Config) -> uid => UId, log_init_args => #{data_dir => PrivDir, uid => UId}, initial_member => [], + initial_machine_version => rabbit_fifo:version(), machine => {module, rabbit_fifo, #{queue_resource => discard, dead_letter_handler => From 283d3182de671add3e044d4d4afe783f4c8dd673 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 24 Sep 2025 16:31:28 +0100 Subject: [PATCH 05/28] remove no longer needed test --- deps/rabbit/src/rabbit_fifo.erl | 2 -- 1 file changed, 2 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 6bf0e441a4aa..f959ea8ddeec 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -978,8 +978,6 @@ handle_aux(RaftState, Tag, Cmd, AuxV2, RaAux) handle_aux(RaftState, Tag, Cmd, AuxV3, RaAux); handle_aux(RaftState, Tag, Cmd, AuxV3, RaAux) when element(1, AuxV3) == aux_v3 -> - Name = element(2, AuxV3), - AuxV3 = init_aux(Name), AuxV4 = #?AUX{name = element(2, AuxV3), last_decorators_state = element(3, AuxV3), unused_1 = undefined, From 1ed1ff20284bc7a5a92928bf5a260ff2140f0dec Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 26 Sep 2025 09:14:16 +0100 Subject: [PATCH 06/28] QQ: remove use of rabbit_fifo_index From the main state machine (still used in the dlx module). This can be done as we no longer need to super efficiently query the smallest raft index. Removing it will reduce peak memory use somewhat as well as simplifying the code. --- deps/rabbit/src/rabbit_fifo.erl | 109 ++++++-------------- deps/rabbit/src/rabbit_fifo.hrl | 10 +- deps/rabbit/test/rabbit_fifo_prop_SUITE.erl | 57 ---------- 3 files changed, 36 insertions(+), 140 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index f959ea8ddeec..ad3e81912f47 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -60,7 +60,6 @@ query_messages_checked_out/1, query_messages_total/1, query_processes/1, - query_ra_indexes/1, query_waiting_consumers/1, query_consumer_count/1, query_consumers/1, @@ -309,12 +308,10 @@ apply(Meta, #modify{consumer_key = ConsumerKey, apply(#{index := Idx} = Meta, #requeue{consumer_key = ConsumerKey, msg_id = MsgId, - index = OldIdx, + index = _OldIdx, header = Header0}, #?STATE{consumers = Cons, - messages = Messages, - ra_indexes = Indexes0, - enqueue_count = EnqCount} = State00) -> + messages = Messages} = State00) -> %% the actual consumer key was looked up in the aux handler so we %% dont need to use find_consumer/2 here case Cons of @@ -326,12 +323,9 @@ apply(#{index := Idx} = Meta, State0 = add_bytes_return(Header, State00), Con = Con0#consumer{checked_out = maps:remove(MsgId, Checked0), credit = increase_credit(Con0, 1)}, - State1 = State0#?STATE{ra_indexes = rabbit_fifo_index:delete(OldIdx, - Indexes0), - messages = rabbit_fifo_q:in(no, + State1 = State0#?STATE{messages = rabbit_fifo_q:in(no, ?MSG(Idx, Header), - Messages), - enqueue_count = EnqCount + 1}, + Messages)}, State2 = update_or_remove_con(Meta, ConsumerKey, Con, State1), {State3, Effects} = activate_next_consumer({State2, []}), checkout(Meta, State0, State3, Effects); @@ -467,31 +461,10 @@ apply(#{index := Idx} = Meta, checkout(Meta, State0, State2, [{monitor, process, Pid} | Effs], Reply); apply(#{index := Index}, #purge{}, #?STATE{messages_total = Total, - returns = Returns, - ra_indexes = Indexes0, msg_bytes_enqueue = MsgBytesEnqueue } = State0) -> NumReady = messages_ready(State0), - Indexes = case Total of - NumReady -> - %% All messages are either in 'messages' queue or - %% 'returns' queue. - %% No message is awaiting acknowledgement. - %% Optimization: empty all 'ra_indexes'. - rabbit_fifo_index:empty(); - _ -> - %% Some messages are checked out to consumers - %% awaiting acknowledgement. - %% Therefore we cannot empty all 'ra_indexes'. - %% We only need to delete the indexes from the 'returns' - %% queue because messages of the 'messages' queue are - %% not part of the 'ra_indexes'. - lqueue:fold(fun(?MSG(I, _), Acc) -> - rabbit_fifo_index:delete(I, Acc) - end, Indexes0, Returns) - end, - State1 = State0#?STATE{ra_indexes = Indexes, - messages = rabbit_fifo_q:new(), + State1 = State0#?STATE{messages = rabbit_fifo_q:new(), messages_total = Total - NumReady, returns = lqueue:new(), msg_bytes_enqueue = 0 @@ -556,7 +529,7 @@ apply(#{system_time := Ts} = Meta, apply(#{system_time := Ts} = Meta, {down, Pid, noconnection}, #?STATE{consumers = Cons0, - enqueuers = Enqs0} = State0) -> + enqueuers = Enqs0} = State0) -> %% A node has been disconnected. This doesn't necessarily mean that %% any processes on this node are down, they _may_ come back so here %% we just mark them as suspected (effectively deactivated) @@ -723,7 +696,8 @@ snapshot_installed(_Meta, #?MODULE{cfg = #cfg{resource = QR}, convert_v7_to_v8(#{} = _Meta, StateV7) -> StateV8 = StateV7, - StateV8. + StateV8#?STATE{discarded_bytes = 0, + unused_0 = ?NIL}. purge_node(Meta, Node, State, Effects) -> lists:foldl(fun(Pid, {S0, E0}) -> @@ -863,7 +837,6 @@ tick(Ts, #?STATE{cfg = #cfg{resource = QName}} = State) -> -spec overview(state()) -> map(). overview(#?STATE{consumers = Cons, enqueuers = Enqs, - % enqueue_count = EnqCount, msg_bytes_enqueue = EnqueueBytes, msg_bytes_checkout = CheckoutBytes, cfg = Cfg, @@ -946,9 +919,7 @@ which_module(8) -> ?MODULE. -record(snapshot, {index :: ra:index(), timestamp :: milliseconds(), - % smallest_index :: undefined | ra:index(), messages_total = 0 :: non_neg_integer(), - % indexes = ?CHECK_MIN_INDEXES :: non_neg_integer(), bytes_out = 0 :: non_neg_integer()}). -record(aux_gc, {last_raft_idx = 0 :: ra:index()}). -record(?AUX, {name :: atom(), @@ -1234,9 +1205,6 @@ query_processes(#?STATE{enqueuers = Enqs, consumers = Cons0}) -> maps:keys(maps:merge(Enqs, Cons)). -query_ra_indexes(#?STATE{ra_indexes = RaIndexes}) -> - RaIndexes. - query_waiting_consumers(#?STATE{waiting_consumers = WaitingConsumers}) -> WaitingConsumers. @@ -1592,14 +1560,12 @@ apply_enqueue(#{index := RaftIdx, decr_total(#?STATE{messages_total = Tot} = State) -> State#?STATE{messages_total = Tot - 1}. -drop_head(#?STATE{ra_indexes = Indexes0} = State0, Effects) -> +drop_head(#?STATE{} = State0, Effects) -> case take_next_msg(State0) of - {?MSG(Idx, Header) = Msg, State1} -> - Indexes = rabbit_fifo_index:delete(Idx, Indexes0), - State2 = State1#?STATE{ra_indexes = Indexes}, - State3 = decr_total(add_bytes_drop(Header, State2)), + {?MSG(_Idx, Header) = Msg, State1} -> + State = decr_total(add_bytes_drop(Header, State1)), #?STATE{cfg = #cfg{dead_letter_handler = DLH}, - dlx = DlxState} = State = State3, + dlx = DlxState} = State, {_, DlxEffects} = rabbit_fifo_dlx:discard([Msg], maxlen, DLH, DlxState), {State, combine_effects(DlxEffects, Effects)}; empty -> @@ -1666,7 +1632,6 @@ update_expiry_header(ExpiryTs, Header) -> maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, {_MetaSize, BodySize}, Effects, #?STATE{msg_bytes_enqueue = Enqueue, - enqueue_count = EnqCount, messages = Messages, messages_total = Total} = State0) -> % direct enqueue without tracking @@ -1676,7 +1641,6 @@ maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, Msg = ?MSG(RaftIdx, Header), PTag = priority_tag(RawMsg), State = State0#?STATE{msg_bytes_enqueue = Enqueue + Size, - enqueue_count = EnqCount + 1, messages_total = Total + 1, messages = rabbit_fifo_q:in(PTag, Msg, Messages) }, @@ -1684,7 +1648,6 @@ maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, {_MetaSize, BodySize} = Size, Effects0, #?STATE{msg_bytes_enqueue = Enqueue, - enqueue_count = EnqCount, enqueuers = Enqueuers0, messages = Messages, messages_total = Total} = State0) -> @@ -1710,7 +1673,6 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, end, PTag = priority_tag(RawMsg), State = State0#?STATE{msg_bytes_enqueue = Enqueue + BodySize, - enqueue_count = EnqCount + 1, messages_total = Total + 1, messages = rabbit_fifo_q:in(PTag, Msg, Messages), enqueuers = Enqueuers0#{From => Enq}, @@ -1753,47 +1715,42 @@ return(Meta, ConsumerKey, % used to process messages that are finished complete(Meta, ConsumerKey, [MsgId], #consumer{checked_out = Checked0} = Con0, - #?STATE{ra_indexes = Indexes0, - msg_bytes_checkout = BytesCheckout, + #?STATE{msg_bytes_checkout = BytesCheckout, messages_total = Tot} = State0, Effects) -> case maps:take(MsgId, Checked0) of - {?MSG(Idx, Hdr), Checked} -> + {?MSG(_Idx, Hdr), Checked} -> SettledSize = get_header(size, Hdr), - Indexes = rabbit_fifo_index:delete(Idx, Indexes0), Con = Con0#consumer{checked_out = Checked, credit = increase_credit(Con0, 1)}, State1 = update_or_remove_con(Meta, ConsumerKey, Con, State0), - {State1#?STATE{ra_indexes = Indexes, - msg_bytes_checkout = BytesCheckout - SettledSize, - messages_total = Tot - 1}, + {State1#?STATE{msg_bytes_checkout = BytesCheckout - SettledSize, + messages_total = Tot - 1}, [{aux, {bytes_out, SettledSize}} | Effects]}; error -> {State0, Effects} end; complete(Meta, ConsumerKey, MsgIds, #consumer{checked_out = Checked0} = Con0, - #?STATE{ra_indexes = Indexes0, - msg_bytes_checkout = BytesCheckout, + #?STATE{msg_bytes_checkout = BytesCheckout, messages_total = Tot} = State0, Effects) -> - {SettledSize, Checked, Indexes} + {SettledSize, Checked} = lists:foldl( - fun (MsgId, {S0, Ch0, Idxs}) -> + fun (MsgId, {S0, Ch0}) -> case maps:take(MsgId, Ch0) of - {?MSG(Idx, Hdr), Ch} -> + {?MSG(_Idx, Hdr), Ch} -> S = get_header(size, Hdr) + S0, - {S, Ch, rabbit_fifo_index:delete(Idx, Idxs)}; + {S, Ch}; error -> - {S0, Ch0, Idxs} + {S0, Ch0} end - end, {0, Checked0, Indexes0}, MsgIds), + end, {0, Checked0}, MsgIds), Len = map_size(Checked0) - map_size(Checked), Con = Con0#consumer{checked_out = Checked, credit = increase_credit(Con0, Len)}, State1 = update_or_remove_con(Meta, ConsumerKey, Con, State0), - {State1#?STATE{ra_indexes = Indexes, - msg_bytes_checkout = BytesCheckout - SettledSize, - messages_total = Tot - Len}, + {State1#?STATE{msg_bytes_checkout = BytesCheckout - SettledSize, + messages_total = Tot - Len}, [{aux, {bytes_out, SettledSize}} | Effects]}. increase_credit(#consumer{cfg = #consumer_cfg{lifetime = once}, @@ -2076,9 +2033,7 @@ add_delivery_effects(Effects0, AccMap, State) -> end, Effects0, AccMap). take_next_msg(#?STATE{returns = Returns0, - messages = Messages0, - ra_indexes = Indexes0 - } = State) -> + messages = Messages0} = State) -> case lqueue:out(Returns0) of {{value, NextMsg}, Returns} -> {NextMsg, State#?STATE{returns = Returns}}; @@ -2086,11 +2041,8 @@ take_next_msg(#?STATE{returns = Returns0, case rabbit_fifo_q:out(Messages0) of empty -> empty; - {?MSG(RaftIdx, _) = Msg, Messages} -> - %% add index here - Indexes = rabbit_fifo_index:append(RaftIdx, Indexes0), - {Msg, State#?STATE{messages = Messages, - ra_indexes = Indexes}} + {?MSG(_RaftIdx, _) = Msg, Messages} -> + {Msg, State#?STATE{messages = Messages}} end end. @@ -2228,18 +2180,15 @@ expire_msgs(RaCmdTs, Result, State, Effects) -> end. expire(RaCmdTs, State0, Effects) -> - {?MSG(Idx, Header) = Msg, + {?MSG(_Idx, Header) = Msg, #?STATE{cfg = #cfg{dead_letter_handler = DLH}, dlx = DlxState0, - ra_indexes = Indexes0, messages_total = Tot, msg_bytes_enqueue = MsgBytesEnqueue} = State1} = take_next_msg(State0), {DlxState, DlxEffects} = rabbit_fifo_dlx:discard([Msg], expired, DLH, DlxState0), - Indexes = rabbit_fifo_index:delete(Idx, Indexes0), State = State1#?STATE{dlx = DlxState, - ra_indexes = Indexes, messages_total = Tot - 1, msg_bytes_enqueue = MsgBytesEnqueue - get_header(size, Header)}, diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index 71c30f4b05de..7c814545e360 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -187,9 +187,11 @@ messages_total = 0 :: non_neg_integer(), % queue of returned msg_in_ids - when checking out it picks from returns = lqueue:new() :: lqueue:lqueue(term()), - % a counter of enqueues - used to trigger shadow copy points + % discareded bytes - a counter that is incremented every time a command + % is procesesed that does not need to be kept (live indexes). + % Approximate, used for triggering snapshots % reset to 0 when release_cursor gets stored - enqueue_count = 0 :: non_neg_integer(), + discarded_bytes = 0, % a map containing all the live processes that have ever enqueued % a message to this queue enqueuers = #{} :: #{pid() => #enqueuer{}}, @@ -198,7 +200,7 @@ % rabbit_fifo_index can be slow when calculating the smallest % index when there are large gaps but should be faster than gb_trees % for normal appending operations as it's backed by a map - ra_indexes = rabbit_fifo_index:empty() :: rabbit_fifo_index:state(), + unused_0 = ?NIL, unused_1 = ?NIL, % consumers need to reflect consumer state at time of snapshot consumers = #{} :: #{consumer_key() => consumer()}, @@ -211,6 +213,8 @@ %% one is picked if active consumer is cancelled or dies %% used only when single active consumer is on waiting_consumers = [] :: [{consumer_key(), consumer()}], + %% records the timestamp whenever the queue was last considered + %% active in terms of consumer activity last_active :: option(non_neg_integer()), msg_cache :: option({ra:index(), raw_msg()}), unused_2 = ?NIL diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index e269a599ce23..e28b2ee07fae 100644 --- a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl @@ -61,7 +61,6 @@ all_tests() -> scenario32, upgrade, messages_total, - ra_indexes, simple_prefetch, simple_prefetch_without_checkout_cancel, simple_prefetch_01, @@ -910,30 +909,6 @@ messages_total(_Config) -> end) end, [], Size). -ra_indexes(_Config) -> - meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> false end), - Size = 256, - run_proper( - fun () -> - ?FORALL({Length, Bytes, DeliveryLimit, SingleActive}, - frequency([{5, {undefined, undefined, undefined, false}}, - {5, {oneof([range(1, 10), undefined]), - oneof([range(1, 1000), undefined]), - oneof([range(1, 3), undefined]), - oneof([true, false]) - }}]), - begin - Config = config(?FUNCTION_NAME, - Length, - Bytes, - SingleActive, - DeliveryLimit), - ?FORALL(O, ?LET(Ops, log_gen(Size), expand(Ops, Config)), - collect({log_size, length(O)}, - ra_indexes_prop(Config, O))) - end) - end, [], Size). - simple_prefetch(_Config) -> Size = 500, meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> true end), @@ -1592,38 +1567,6 @@ messages_total_invariant() -> end end. -ra_indexes_prop(Conf0, Commands) -> - Conf = Conf0#{release_cursor_interval => 100}, - Indexes = lists:seq(1, length(Commands)), - Entries = lists:zip(Indexes, Commands), - InitState = test_init(Conf), - run_log(InitState, Entries, ra_indexes_invariant()), - true. - -ra_indexes_invariant() -> - %% The raft indexes contained in the `ra_indexes` `rabbit_fifo_index` must - %% be the same as all indexes checked out by consumers plus those in the - %% `returns` queue. - fun(#rabbit_fifo{ra_indexes = Index, - consumers = C, - returns = R}) -> - RIdxs = lqueue:fold(fun(?MSG(I, _), Acc) -> [I | Acc] end, [], R), - CIdxs = maps:fold(fun(_, #consumer{checked_out = Ch}, Acc0) -> - maps:fold(fun(_, ?MSG(I, _), Acc) -> - [I | Acc] - end, Acc0, Ch) - end, [], C), - ActualIdxs = lists:sort(RIdxs ++ CIdxs), - IndexIdxs = lists:sort(rabbit_fifo_index:to_list(Index)), - case ActualIdxs == IndexIdxs of - true -> true; - false -> - ct:pal("ra_indexes invariant failed Expected ~b Got ~b", - [ActualIdxs, IndexIdxs]), - false - end - end. - simple_prefetch_prop(Conf0, Commands, WithCheckoutCancel) -> Conf = Conf0#{release_cursor_interval => 100}, Indexes = lists:seq(1, length(Commands)), From ddd32c61d73a567ee4ec55f063ec7d9ef9b03337 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 3 Oct 2025 13:09:06 +0100 Subject: [PATCH 07/28] QQ: track discarded bytes and take snapshots based on that. --- deps/rabbit/src/rabbit_fifo.erl | 693 ++++++++++++++---- deps/rabbit/src/rabbit_fifo.hrl | 27 +- deps/rabbit/src/rabbit_fifo_client.erl | 4 +- deps/rabbit/src/rabbit_fifo_dlx.erl | 2 +- deps/rabbit/src/rabbit_fifo_dlx.hrl | 3 +- deps/rabbit/src/rabbit_quorum_queue.erl | 18 +- deps/rabbit/test/quorum_queue_SUITE.erl | 10 +- deps/rabbit/test/rabbit_fifo_SUITE.erl | 204 +++++- .../rabbit_fifo_dlx_integration_SUITE.erl | 45 +- deps/rabbit/test/rabbit_fifo_prop_SUITE.erl | 5 +- 10 files changed, 803 insertions(+), 208 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index ad3e81912f47..1c50fc746eca 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -17,6 +17,7 @@ -include_lib("kernel/include/logger.hrl"). -define(STATE, ?MODULE). +-define(DLX, rabbit_fifo_dlx). -define(CONSUMER_PID(Pid), #consumer{cfg = #consumer_cfg{pid = Pid}}). -define(CONSUMER_PRIORITY(P), #consumer{cfg = #consumer_cfg{priority = P}}). @@ -24,6 +25,7 @@ #consumer{cfg = #consumer_cfg{tag = Tag, pid = Pid}}). +-define(ENQ_OVERHEAD, 256). -ifdef(TEST). -define(SIZE(Msg), case mc:is(Msg) of @@ -192,7 +194,7 @@ init(#{name := Name, update_config(Conf, State) -> DLH = maps:get(dead_letter_handler, Conf, undefined), - BLH = maps:get(become_leader_handler, Conf, undefined), + % BLH = maps:get(become_leader_handler, Conf, undefined), Overflow = maps:get(overflow_strategy, Conf, drop_head), MaxLength = maps:get(max_length, Conf, undefined), MaxBytes = maps:get(max_bytes, Conf, undefined), @@ -216,7 +218,6 @@ update_config(Conf, State) -> LastActive = maps:get(created, Conf, undefined), State#?STATE{cfg = Cfg#cfg{dead_letter_handler = DLH, - become_leader_handler = BLH, overflow_strategy = Overflow, max_length = MaxLength, max_bytes = MaxBytes, @@ -230,16 +231,30 @@ update_config(Conf, State) -> -spec apply(ra_machine:command_meta_data(), command(), state()) -> {state(), ra_machine:reply(), ra_machine:effects() | ra_machine:effect()} | {state(), ra_machine:reply()}. -apply(Meta, #enqueue{pid = From, seq = Seq, - msg = RawMsg}, State00) -> +apply(Meta, {machine_version, FromVersion, ToVersion}, VXState) -> + %% machine version upgrades cant be done in apply_ + State = convert(Meta, FromVersion, ToVersion, VXState), + %% TODO: force snapshot now? + {State, ok, [{aux, {dlx, setup}}]}; +apply(Meta, Cmd, #?STATE{discarded_bytes = DiscBytes} = State) -> + %% add estimated discared_bytes + %% TODO: optimise! + %% this is the simplest way to record the discarded bytes for most + %% commands but it is a bit mory garby as almost always creates a new + %% state copy before even processing the command + Bytes = estimate_discarded_size(Cmd), + apply_(Meta, Cmd, State#?STATE{discarded_bytes = DiscBytes + Bytes}). + +apply_(Meta, #enqueue{pid = From, seq = Seq, + msg = RawMsg}, State00) -> apply_enqueue(Meta, From, Seq, RawMsg, message_size(RawMsg), State00); -apply(#{reply_mode := {notify, _Corr, EnqPid}} = Meta, +apply_(#{reply_mode := {notify, _Corr, EnqPid}} = Meta, #?ENQ_V2{seq = Seq, msg = RawMsg, size = Size}, State00) -> apply_enqueue(Meta, EnqPid, Seq, RawMsg, Size, State00); -apply(Meta, #?ENQ_V2{seq = Seq, msg = RawMsg, size = Size}, State00) -> +apply_(Meta, #?ENQ_V2{seq = Seq, msg = RawMsg, size = Size}, State00) -> %% untracked apply_enqueue(Meta, undefined, Seq, RawMsg, Size, State00); -apply(_Meta, #register_enqueuer{pid = Pid}, +apply_(_Meta, #register_enqueuer{pid = Pid}, #?STATE{enqueuers = Enqueuers0, cfg = #cfg{overflow_strategy = Overflow}} = State0) -> State = case maps:is_key(Pid, Enqueuers0) of @@ -256,7 +271,7 @@ apply(_Meta, #register_enqueuer{pid = Pid}, ok end, {State, Res, [{monitor, process, Pid}]}; -apply(Meta, #settle{msg_ids = MsgIds, +apply_(Meta, #settle{msg_ids = MsgIds, consumer_key = Key}, #?STATE{consumers = Consumers} = State) -> case find_consumer(Key, Consumers) of @@ -268,7 +283,7 @@ apply(Meta, #settle{msg_ids = MsgIds, _ -> {State, ok} end; -apply(Meta, #discard{consumer_key = ConsumerKey, +apply_(Meta, #discard{consumer_key = ConsumerKey, msg_ids = MsgIds}, #?STATE{consumers = Consumers } = State0) -> case find_consumer(ConsumerKey, Consumers) of @@ -277,7 +292,7 @@ apply(Meta, #discard{consumer_key = ConsumerKey, _ -> {State0, ok} end; -apply(Meta, #return{consumer_key = ConsumerKey, +apply_(Meta, #return{consumer_key = ConsumerKey, msg_ids = MsgIds}, #?STATE{consumers = Cons} = State) -> case find_consumer(ConsumerKey, Cons) of @@ -287,7 +302,7 @@ apply(Meta, #return{consumer_key = ConsumerKey, _ -> {State, ok} end; -apply(Meta, #modify{consumer_key = ConsumerKey, +apply_(Meta, #modify{consumer_key = ConsumerKey, delivery_failed = DelFailed, undeliverable_here = Undel, annotations = Anns, @@ -305,7 +320,7 @@ apply(Meta, #modify{consumer_key = ConsumerKey, _ -> {State, ok} end; -apply(#{index := Idx} = Meta, +apply_(#{index := Idx} = Meta, #requeue{consumer_key = ConsumerKey, msg_id = MsgId, index = _OldIdx, @@ -332,7 +347,7 @@ apply(#{index := Idx} = Meta, _ -> {State00, ok, []} end; -apply(Meta, #credit{consumer_key = ConsumerKey} = Credit, +apply_(Meta, #credit{consumer_key = ConsumerKey} = Credit, #?STATE{consumers = Cons} = State) -> case Cons of #{ConsumerKey := Con} -> @@ -346,10 +361,10 @@ apply(Meta, #credit{consumer_key = ConsumerKey} = Credit, {State, ok} end end; -apply(_, #checkout{spec = {dequeue, _}}, +apply_(_, #checkout{spec = {dequeue, _}}, #?STATE{cfg = #cfg{consumer_strategy = single_active}} = State0) -> {State0, {error, {unsupported, single_active_consumer}}}; -apply(#{index := Index, +apply_(#{index := Index, system_time := Ts, from := From} = Meta, #checkout{spec = {dequeue, Settlement}, meta = ConsumerMeta, @@ -400,7 +415,7 @@ apply(#{index := Index, {State, {dequeue, empty}, Effects} end end; -apply(#{index := _Idx} = Meta, +apply_(#{index := _Idx} = Meta, #checkout{spec = Spec, consumer_id = ConsumerId}, State0) when Spec == cancel orelse @@ -416,7 +431,7 @@ apply(#{index := _Idx} = Meta, error -> {State0, {error, consumer_not_found}, []} end; -apply(#{index := Idx} = Meta, +apply_(#{index := Idx} = Meta, #checkout{spec = Spec0, meta = ConsumerMeta, consumer_id = {_, Pid} = ConsumerId}, State0) -> @@ -459,27 +474,23 @@ apply(#{index := Idx} = Meta, is_active => is_active(ConsumerKey, State2), num_checked_out => map_size(Checked)}}, checkout(Meta, State0, State2, [{monitor, process, Pid} | Effs], Reply); -apply(#{index := Index}, #purge{}, - #?STATE{messages_total = Total, - msg_bytes_enqueue = MsgBytesEnqueue - } = State0) -> +apply_(#{index := Index}, #purge{}, + #?STATE{messages_total = Total} = State0) -> NumReady = messages_ready(State0), State1 = State0#?STATE{messages = rabbit_fifo_q:new(), messages_total = Total - NumReady, returns = lqueue:new(), msg_bytes_enqueue = 0 }, - Effects0 = [{aux, {bytes_out, MsgBytesEnqueue}}, - {aux, force_checkpoint}, - garbage_collection], + Effects0 = [{aux, force_checkpoint}, garbage_collection], Reply = {purge, NumReady}, {State, Effects} = evaluate_limit(Index, State0, State1, Effects0), {State, Reply, Effects}; -apply(#{index := _Idx}, #garbage_collection{}, State) -> +apply_(#{index := _Idx}, #garbage_collection{}, State) -> {State, ok, [{aux, garbage_collection}]}; -apply(Meta, {timeout, expire_msgs}, State) -> +apply_(Meta, {timeout, expire_msgs}, State) -> checkout(Meta, State, State, []); -apply(#{system_time := Ts} = Meta, +apply_(#{system_time := Ts} = Meta, {down, Pid, noconnection}, #?STATE{consumers = Cons0, cfg = #cfg{consumer_strategy = single_active}, @@ -526,7 +537,7 @@ apply(#{system_time := Ts} = Meta, end, Enqs0), Effects = [{monitor, node, Node} | Effects1], checkout(Meta, State0, State#?STATE{enqueuers = Enqs}, Effects); -apply(#{system_time := Ts} = Meta, +apply_(#{system_time := Ts} = Meta, {down, Pid, noconnection}, #?STATE{consumers = Cons0, enqueuers = Enqs0} = State0) -> @@ -565,10 +576,10 @@ apply(#{system_time := Ts} = Meta, Effects = [{monitor, node, Node} | Effects1], checkout(Meta, State0, State#?STATE{enqueuers = Enqs, last_active = Ts}, Effects); -apply(Meta, {down, Pid, _Info}, State0) -> +apply_(Meta, {down, Pid, _Info}, State0) -> {State1, Effects1} = activate_next_consumer(handle_down(Meta, Pid, State0)), checkout(Meta, State0, State1, Effects1); -apply(Meta, {nodeup, Node}, #?STATE{consumers = Cons0, +apply_(Meta, {nodeup, Node}, #?STATE{consumers = Cons0, enqueuers = Enqs0} = State0) -> %% A node we are monitoring has come back. %% If we have suspected any processes of being @@ -609,33 +620,32 @@ apply(Meta, {nodeup, Node}, #?STATE{consumers = Cons0, waiting_consumers = Waiting}, {State, Effects} = activate_next_consumer(State2, Effects1), checkout(Meta, State0, State, Effects); -apply(_, {nodedown, _Node}, State) -> +apply_(_, {nodedown, _Node}, State) -> {State, ok}; -apply(Meta, #purge_nodes{nodes = Nodes}, State0) -> +apply_(Meta, #purge_nodes{nodes = Nodes}, State0) -> {State, Effects} = lists:foldl(fun(Node, {S, E}) -> purge_node(Meta, Node, S, E) end, {State0, []}, Nodes), {State, ok, Effects}; -apply(Meta, +apply_(Meta, #update_config{config = #{} = Conf}, #?STATE{cfg = #cfg{dead_letter_handler = OldDLH, resource = QRes}, dlx = DlxState0} = State0) -> NewDLH = maps:get(dead_letter_handler, Conf, OldDLH), - {DlxState, Effects0} = rabbit_fifo_dlx:update_config(OldDLH, NewDLH, QRes, - DlxState0), + {DlxState, Effects0} = update_config(OldDLH, NewDLH, QRes, + DlxState0), State1 = update_config(Conf, State0#?STATE{dlx = DlxState}), checkout(Meta, State0, State1, Effects0); -apply(Meta, {machine_version, FromVersion, ToVersion}, V0State) -> - State = convert(Meta, FromVersion, ToVersion, V0State), - {State, ok, [{aux, {dlx, setup}}]}; -apply(Meta, {dlx, _} = Cmd, - #?STATE{cfg = #cfg{dead_letter_handler = DLH}, +apply_(Meta, {dlx, _} = Cmd, + #?STATE{cfg = #cfg{dead_letter_handler = DLH}, + discarded_bytes = DiscardedBytes0, dlx = DlxState0} = State0) -> - {DlxState, Effects0} = rabbit_fifo_dlx:apply(Meta, Cmd, DLH, DlxState0), - State1 = State0#?STATE{dlx = DlxState}, + {DlxState, DiscardedBytes, Effects0} = dlx_apply(Meta, Cmd, DLH, DlxState0), + State1 = State0#?STATE{dlx = DlxState, + discarded_bytes = DiscardedBytes0 + DiscardedBytes}, checkout(Meta, State0, State1, Effects0); -apply(_Meta, Cmd, State) -> +apply_(_Meta, Cmd, State) -> %% handle unhandled commands gracefully ?LOG_DEBUG("rabbit_fifo: unhandled command ~W", [Cmd, 10]), {State, ok, []}. @@ -645,8 +655,10 @@ live_indexes(#?STATE{cfg = #cfg{}, returns = Returns, messages = Messages, consumers = Consumers, - dlx = Dlx}) -> - DlxIndexes = rabbit_fifo_dlx:live_indexes(Dlx), + dlx = #?DLX{discards = Discards}}) -> + DlxIndexes = lqueue:fold(fun (?TUPLE(_, ?MSG(I, _)), Acc) -> + [I | Acc] + end, [], Discards), RtnIndexes = [I || ?MSG(I, _) <- lqueue:to_list(Returns)], CheckedIdxs = maps:fold( fun (_Cid, #consumer{checked_out = Ch}, Acc0) -> @@ -664,7 +676,7 @@ live_indexes(#?STATE{cfg = #cfg{}, State :: state(), OldMeta :: ra_snapshot:meta(), OldState :: state(). -snapshot_installed(_Meta, #?MODULE{cfg = #cfg{resource = QR}, +snapshot_installed(_Meta, #?MODULE{cfg = #cfg{resource = _QR}, consumers = Consumers} = State, _OldMeta, _OldState) -> %% here we need to redliver all pending consumer messages @@ -685,15 +697,9 @@ snapshot_installed(_Meta, #?MODULE{cfg = #cfg{resource = QR}, Acc end end, #{}, Consumers), - ?LOG_DEBUG("~ts: rabbit_fifo: install snapshot sending ~p", - [rabbit_misc:rs(QR), SendAcc]), Effs = add_delivery_effects([], SendAcc, State), - ?LOG_DEBUG("~ts: rabbit_fifo: effs ~p", - [rabbit_misc:rs(QR), Effs]), Effs. - - convert_v7_to_v8(#{} = _Meta, StateV7) -> StateV8 = StateV7, StateV8#?STATE{discarded_bytes = 0, @@ -776,40 +782,34 @@ update_waiting_consumer_status(Node, -spec state_enter(ra_server:ra_state() | eol, state()) -> ra_machine:effects(). -state_enter(RaState, #?STATE{cfg = #cfg{dead_letter_handler = DLH, - resource = QRes}, - dlx = DlxState} = State) -> - Effects = rabbit_fifo_dlx:state_enter(RaState, QRes, DLH, DlxState), - state_enter0(RaState, State, Effects). - -state_enter0(leader, #?STATE{consumers = Cons, - enqueuers = Enqs, - waiting_consumers = WaitingConsumers, - cfg = #cfg{name = Name, - resource = Resource, - become_leader_handler = BLH} - } = State, - Effects0) -> - TimerEffs = timer_effect(erlang:system_time(millisecond), State, Effects0), +state_enter(leader, + #?STATE{consumers = Cons, + enqueuers = Enqs, + waiting_consumers = WaitingConsumers, + cfg = #cfg{resource = QRes, + dead_letter_handler = DLH}, + dlx = DlxState} = State) -> + TimerEffs = timer_effect(erlang:system_time(millisecond), State, []), % return effects to monitor all current consumers and enqueuers Pids = lists:usort(maps:keys(Enqs) - ++ [P || ?CONSUMER_PID(P) <- maps:values(Cons)] - ++ [P || {_, ?CONSUMER_PID(P)} <- WaitingConsumers]), + ++ [P || ?CONSUMER_PID(P) <- maps:values(Cons)] + ++ [P || {_, ?CONSUMER_PID(P)} <- WaitingConsumers]), Mons = [{monitor, process, P} || P <- Pids], Nots = [{send_msg, P, leader_change, ra_event} || P <- Pids], NodeMons = lists:usort([{monitor, node, node(P)} || P <- Pids]), - NotifyDecs = notify_decorators_startup(Resource), + NotifyDecs = notify_decorators_startup(QRes), Effects = TimerEffs ++ Mons ++ Nots ++ NodeMons ++ [NotifyDecs], - case BLH of - undefined -> - Effects; - {Mod, Fun, Args} -> - [{mod_call, Mod, Fun, Args ++ [Name]} | Effects] - end; -state_enter0(eol, #?STATE{enqueuers = Enqs, - consumers = Cons0, - waiting_consumers = WaitingConsumers0}, - Effects) -> + + case DLH of + at_least_once -> + ensure_worker_started(QRes, DlxState); + _ -> + ok + end, + Effects; +state_enter(eol, #?STATE{enqueuers = Enqs, + consumers = Cons0, + waiting_consumers = WaitingConsumers0}) -> Custs = maps:fold(fun(_K, ?CONSUMER_PID(P) = V, S) -> S#{P => V} end, #{}, Cons0), @@ -819,11 +819,18 @@ state_enter0(eol, #?STATE{enqueuers = Enqs, AllConsumers = maps:merge(Custs, WaitingConsumers1), [{send_msg, P, eol, ra_event} || P <- maps:keys(maps:merge(Enqs, AllConsumers))] ++ - [{aux, eol} - | Effects]; -state_enter0(_, _, Effects) -> + [{aux, eol}]; +state_enter(_, #?STATE{cfg = #cfg{dead_letter_handler = DLH, + resource = _QRes}, + dlx = DlxState}) -> + case DLH of + at_least_once -> + ensure_worker_terminated(DlxState); + _ -> + ok + end, %% catch all as not handling all states - Effects. + []. -spec tick(non_neg_integer(), state()) -> ra_machine:effects(). tick(Ts, #?STATE{cfg = #cfg{resource = QName}} = State) -> @@ -841,6 +848,7 @@ overview(#?STATE{consumers = Cons, msg_bytes_checkout = CheckoutBytes, cfg = Cfg, dlx = DlxState, + discarded_bytes = DiscardedBytes, messages = Messages, returns = Returns, waiting_consumers = WaitingConsumers} = State) -> @@ -854,8 +862,7 @@ overview(#?STATE{consumers = Cons, consumer_strategy => Cfg#cfg.consumer_strategy, expires => Cfg#cfg.expires, msg_ttl => Cfg#cfg.msg_ttl, - delivery_limit => Cfg#cfg.delivery_limit - }, + delivery_limit => Cfg#cfg.delivery_limit}, SacOverview = case active_consumer(Cons) of {SacConsumerKey, SacCon} -> SacConsumerId = consumer_id(SacCon), @@ -882,11 +889,10 @@ overview(#?STATE{consumers = Cons, num_ready_messages_return => MsgsRet, num_messages => messages_total(State), enqueue_message_bytes => EnqueueBytes, - checkout_message_bytes => CheckoutBytes - % smallest_raft_index => smallest_raft_index(State), - % smallest_raft_index_overview => smallest_raft_index_overview(State) + checkout_message_bytes => CheckoutBytes, + discarded_bytes => DiscardedBytes }, - DlxOverview = rabbit_fifo_dlx:overview(DlxState), + DlxOverview = dlx_overview(DlxState), maps:merge(maps:merge(Overview, DlxOverview), SacOverview). -spec get_checked_out(consumer_key(), msg_id(), msg_id(), state()) -> @@ -920,7 +926,7 @@ which_module(8) -> ?MODULE. -record(snapshot, {index :: ra:index(), timestamp :: milliseconds(), messages_total = 0 :: non_neg_integer(), - bytes_out = 0 :: non_neg_integer()}). + discarded_bytes = 0 :: non_neg_integer()}). -record(aux_gc, {last_raft_idx = 0 :: ra:index()}). -record(?AUX, {name :: atom(), last_decorators_state :: term(), @@ -961,11 +967,12 @@ handle_aux(RaftState, Tag, Cmd, AuxV3, RaAux) handle_aux(RaftState, Tag, Cmd, AuxV4, RaAux); handle_aux(leader, cast, eval, #?AUX{last_decorators_state = LastDec, - bytes_in = BytesIn, - bytes_out = BytesOut, + % bytes_in = BytesIn, + % bytes_out = BytesOut, last_checkpoint = Check0} = Aux0, RaAux) -> - #?STATE{cfg = #cfg{resource = QName}} = MacState = + #?STATE{cfg = #cfg{resource = QName}, + discarded_bytes = DiscardedBytes} = MacState = ra_aux:machine_state(RaAux), Ts = erlang:system_time(millisecond), @@ -977,7 +984,7 @@ handle_aux(leader, cast, eval, undefined end, {Check, Effects0} = do_snapshot(EffMacVer, Ts, Check0, RaAux, - BytesIn, BytesOut, false), + DiscardedBytes, false), %% this is called after each batch of commands have been applied %% set timer for message expire @@ -993,15 +1000,12 @@ handle_aux(leader, cast, eval, last_decorators_state = NewLast}, RaAux, Effects} end; handle_aux(_RaftState, cast, eval, - #?AUX{last_checkpoint = Check0, - bytes_in = BytesIn, - bytes_out = BytesOut - } = Aux0, - RaAux) -> + #?AUX{last_checkpoint = Check0} = Aux0, RaAux) -> Ts = erlang:system_time(millisecond), EffMacVer = ra_aux:effective_machine_version(RaAux), + #?STATE{discarded_bytes = DiscardedBytes} = ra_aux:machine_state(RaAux), {Check, Effects} = do_snapshot(EffMacVer, Ts, Check0, RaAux, - BytesIn, BytesOut, false), + DiscardedBytes, false), {no_reply, Aux0#?AUX{last_checkpoint = Check}, RaAux, Effects}; handle_aux(_RaftState, cast, {bytes_in, {MetaSize, BodySize}}, #?AUX{bytes_in = Bytes} = Aux0, @@ -1128,22 +1132,26 @@ handle_aux(_RaState, {call, _From}, {peek, Pos}, Aux0, handle_aux(_, _, garbage_collection, Aux, RaAux) -> {no_reply, force_eval_gc(RaAux, Aux), RaAux}; handle_aux(_RaState, _, force_checkpoint, - #?AUX{last_checkpoint = Check0, - bytes_in = BytesIn, - bytes_out = BytesOut} = Aux, RaAux) -> + #?AUX{last_checkpoint = Check0} = Aux, RaAux) -> Ts = erlang:system_time(millisecond), - #?STATE{cfg = #cfg{resource = QR}} = ra_aux:machine_state(RaAux), + #?STATE{cfg = #cfg{resource = QR}, + discarded_bytes = DiscardedBytes} = ra_aux:machine_state(RaAux), ?LOG_DEBUG("~ts: rabbit_fifo: forcing checkpoint at ~b", - [rabbit_misc:rs(QR), ra_aux:last_applied(RaAux)]), + [rabbit_misc:rs(QR), ra_aux:last_applied(RaAux)]), EffMacVer = ra_aux:effective_machine_version(RaAux), {Check, Effects} = do_snapshot(EffMacVer, Ts, Check0, RaAux, - BytesIn, BytesOut, true), + DiscardedBytes, true), {no_reply, Aux#?AUX{last_checkpoint = Check}, RaAux, Effects}; -handle_aux(RaState, _, {dlx, _} = Cmd, Aux0, RaAux) -> +handle_aux(leader, _, {dlx, setup}, Aux, RaAux) -> #?STATE{dlx = DlxState, cfg = #cfg{dead_letter_handler = DLH, resource = QRes}} = ra_aux:machine_state(RaAux), - Aux = rabbit_fifo_dlx:handle_aux(RaState, Cmd, Aux0, QRes, DLH, DlxState), + case DLH of + at_least_once -> + ensure_worker_started(QRes, DlxState); + _ -> + ok + end, {no_reply, Aux, RaAux}. eval_gc(RaAux, MacState, @@ -1304,7 +1312,7 @@ query_in_memory_usage(#?STATE{ }) -> {0, 0}. query_stat_dlx(#?STATE{dlx = DlxState}) -> - rabbit_fifo_dlx:stat(DlxState). + dlx_stat(DlxState). query_peek(Pos, State0) when Pos > 0 -> case take_next_msg(State0) of @@ -1349,7 +1357,7 @@ messages_ready(#?STATE{messages = M, messages_total(#?STATE{messages_total = Total, dlx = DlxState}) -> - {DlxTotal, _} = rabbit_fifo_dlx:stat(DlxState), + {DlxTotal, _} = dlx_stat(DlxState), Total + DlxTotal. num_checked_out(#?STATE{consumers = Cons}) -> @@ -1546,9 +1554,7 @@ maybe_return_all(#{system_time := Ts} = Meta, ConsumerKey, apply_enqueue(#{index := RaftIdx, system_time := Ts} = Meta, From, Seq, RawMsg, Size, State0) -> - Effects0 = [{aux, {bytes_in, Size}}], - case maybe_enqueue(RaftIdx, Ts, From, Seq, RawMsg, Size, - Effects0, State0) of + case maybe_enqueue(RaftIdx, Ts, From, Seq, RawMsg, Size, [], State0) of {ok, State1, Effects1} -> checkout(Meta, State0, State1, Effects1); {out_of_sequence, State, Effects} -> @@ -1560,14 +1566,17 @@ apply_enqueue(#{index := RaftIdx, decr_total(#?STATE{messages_total = Tot} = State) -> State#?STATE{messages_total = Tot - 1}. -drop_head(#?STATE{} = State0, Effects) -> +drop_head(#?STATE{discarded_bytes = DiscardedBytes0} = State0, Effects) -> case take_next_msg(State0) of {?MSG(_Idx, Header) = Msg, State1} -> State = decr_total(add_bytes_drop(Header, State1)), #?STATE{cfg = #cfg{dead_letter_handler = DLH}, dlx = DlxState} = State, - {_, DlxEffects} = rabbit_fifo_dlx:discard([Msg], maxlen, DLH, DlxState), - {State, combine_effects(DlxEffects, Effects)}; + {_, _RetainedBytes, DlxEffects} = + discard_or_dead_letter([Msg], maxlen, DLH, DlxState), + Size = get_header(size, Header), + {State#?STATE{discarded_bytes = DiscardedBytes0 + Size + ?ENQ_OVERHEAD}, + combine_effects(DlxEffects, Effects)}; empty -> {State0, Effects} end. @@ -1597,7 +1606,7 @@ maybe_set_msg_ttl(Msg, RaCmdTs, Header, TTL = min(MsgTTL, mc:ttl(Msg)), update_expiry_header(RaCmdTs, TTL, Header); false -> - Header + update_expiry_header(RaCmdTs, MsgTTL, Header) end. maybe_set_msg_delivery_count(Msg, Header) -> @@ -1630,13 +1639,13 @@ update_expiry_header(ExpiryTs, Header) -> update_header(expiry, fun(Ts) -> Ts end, ExpiryTs, Header). maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, - {_MetaSize, BodySize}, + {MetaSize, BodySize}, Effects, #?STATE{msg_bytes_enqueue = Enqueue, messages = Messages, messages_total = Total} = State0) -> % direct enqueue without tracking - Size = BodySize, - Header0 = maybe_set_msg_ttl(RawMsg, Ts, BodySize, State0), + Size = MetaSize + BodySize, + Header0 = maybe_set_msg_ttl(RawMsg, Ts, Size, State0), Header = maybe_set_msg_delivery_count(RawMsg, Header0), Msg = ?MSG(RaftIdx, Header), PTag = priority_tag(RawMsg), @@ -1646,22 +1655,23 @@ maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, }, {ok, State, Effects}; maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, - {_MetaSize, BodySize} = Size, - Effects0, #?STATE{msg_bytes_enqueue = Enqueue, + {MetaSize, BodySize} = MsgSize, + Effects0, #?STATE{msg_bytes_enqueue = BytesEnqueued, enqueuers = Enqueuers0, messages = Messages, + discarded_bytes = DiscardedBytes0, messages_total = Total} = State0) -> - + Size = MetaSize + BodySize, case maps:get(From, Enqueuers0, undefined) of undefined -> State1 = State0#?STATE{enqueuers = Enqueuers0#{From => #enqueuer{}}}, {Res, State, Effects} = maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, - RawMsg, Size, Effects0, + RawMsg, MsgSize, Effects0, State1), {Res, State, [{monitor, process, From} | Effects]}; #enqueuer{next_seqno = MsgSeqNo} = Enq0 -> % it is the next expected seqno - Header0 = maybe_set_msg_ttl(RawMsg, Ts, BodySize, State0), + Header0 = maybe_set_msg_ttl(RawMsg, Ts, Size, State0), Header = maybe_set_msg_delivery_count(RawMsg, Header0), Msg = ?MSG(RaftIdx, Header), Enq = Enq0#enqueuer{next_seqno = MsgSeqNo + 1}, @@ -1672,7 +1682,7 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, undefined end, PTag = priority_tag(RawMsg), - State = State0#?STATE{msg_bytes_enqueue = Enqueue + BodySize, + State = State0#?STATE{msg_bytes_enqueue = BytesEnqueued + Size, messages_total = Total + 1, messages = rabbit_fifo_q:in(PTag, Msg, Messages), enqueuers = Enqueuers0#{From => Enq}, @@ -1682,10 +1692,14 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, #enqueuer{next_seqno = Next} when MsgSeqNo > Next -> %% TODO: when can this happen? - {out_of_sequence, State0, Effects0}; + State = State0#?STATE{discarded_bytes = + DiscardedBytes0 + Size + ?ENQ_OVERHEAD}, + {out_of_sequence, State, Effects0}; #enqueuer{next_seqno = Next} when MsgSeqNo =< Next -> % duplicate delivery - {duplicate, State0, Effects0} + State = State0#?STATE{discarded_bytes = + DiscardedBytes0 + Size + ?ENQ_OVERHEAD}, + {duplicate, State, Effects0} end. return(Meta, ConsumerKey, @@ -1716,6 +1730,7 @@ return(Meta, ConsumerKey, complete(Meta, ConsumerKey, [MsgId], #consumer{checked_out = Checked0} = Con0, #?STATE{msg_bytes_checkout = BytesCheckout, + discarded_bytes = DiscBytes, messages_total = Tot} = State0, Effects) -> case maps:take(MsgId, Checked0) of @@ -1725,14 +1740,16 @@ complete(Meta, ConsumerKey, [MsgId], credit = increase_credit(Con0, 1)}, State1 = update_or_remove_con(Meta, ConsumerKey, Con, State0), {State1#?STATE{msg_bytes_checkout = BytesCheckout - SettledSize, + discarded_bytes = DiscBytes + SettledSize + ?ENQ_OVERHEAD, messages_total = Tot - 1}, - [{aux, {bytes_out, SettledSize}} | Effects]}; + Effects}; error -> {State0, Effects} end; complete(Meta, ConsumerKey, MsgIds, #consumer{checked_out = Checked0} = Con0, #?STATE{msg_bytes_checkout = BytesCheckout, + discarded_bytes = DiscBytes, messages_total = Tot} = State0, Effects) -> {SettledSize, Checked} = lists:foldl( @@ -1750,8 +1767,9 @@ complete(Meta, ConsumerKey, MsgIds, credit = increase_credit(Con0, Len)}, State1 = update_or_remove_con(Meta, ConsumerKey, Con, State0), {State1#?STATE{msg_bytes_checkout = BytesCheckout - SettledSize, + discarded_bytes = DiscBytes + SettledSize + (Len *?ENQ_OVERHEAD), messages_total = Tot - Len}, - [{aux, {bytes_out, SettledSize}} | Effects]}. + Effects}. increase_credit(#consumer{cfg = #consumer_cfg{lifetime = once}, credit = Credit}, _) -> @@ -1864,6 +1882,7 @@ return_one(Meta, MsgId, ?MSG(_, _) = Msg0, DelivFailed, Anns, #?STATE{returns = Returns, consumers = Consumers, dlx = DlxState0, + discarded_bytes = DiscardedBytes0, cfg = #cfg{delivery_limit = DeliveryLimit, dead_letter_handler = DLH}} = State0, Effects0, ConsumerKey) -> @@ -1874,9 +1893,13 @@ return_one(Meta, MsgId, ?MSG(_, _) = Msg0, DelivFailed, Anns, %% returns, use delivery_count header instead case get_header(acquired_count, Header) of AcquiredCount when AcquiredCount > DeliveryLimit -> - {DlxState, DlxEffects} = - rabbit_fifo_dlx:discard([Msg], delivery_limit, DLH, DlxState0), - State1 = State0#?STATE{dlx = DlxState}, + {DlxState, RetainedBytes, DlxEffects} = + discard_or_dead_letter([Msg], delivery_limit, DLH, DlxState0), + %% subtract retained bytes as complete/6 will add them on irrespective + %% of dead letter strategy, alt, consider adding a new argument to + %% indicate if message ids were retained + State1 = State0#?STATE{dlx = DlxState, + discarded_bytes = DiscardedBytes0 - RetainedBytes}, {State, Effects} = complete(Meta, ConsumerKey, [MsgId], Con0, State1, Effects0), {State, DlxEffects ++ Effects}; @@ -1908,7 +1931,7 @@ checkout(#{index := Index} = Meta, {#?STATE{cfg = #cfg{dead_letter_handler = DLH}, dlx = DlxState0} = State1, _ExpiredMsg, Effects1} = checkout0(Meta, checkout_one(Meta, false, State0, Effects0), #{}), - {DlxState, DlxDeliveryEffects} = rabbit_fifo_dlx:checkout(DLH, DlxState0), + {DlxState, DlxDeliveryEffects} = dlx_checkout(DLH, DlxState0), %% TODO: only update dlx state if it has changed? %% by this time the cache should be used State2 = State1#?STATE{msg_cache = undefined, @@ -2184,12 +2207,15 @@ expire(RaCmdTs, State0, Effects) -> #?STATE{cfg = #cfg{dead_letter_handler = DLH}, dlx = DlxState0, messages_total = Tot, + discarded_bytes = DiscardedBytes0, msg_bytes_enqueue = MsgBytesEnqueue} = State1} = take_next_msg(State0), - {DlxState, DlxEffects} = rabbit_fifo_dlx:discard([Msg], expired, - DLH, DlxState0), + {DlxState, _RetainedBytes, DlxEffects} = + discard_or_dead_letter([Msg], expired, DLH, DlxState0), + DiscardedSize = get_header(size, Header) + ?ENQ_OVERHEAD, State = State1#?STATE{dlx = DlxState, messages_total = Tot - 1, + discarded_bytes = DiscardedBytes0 + DiscardedSize, msg_bytes_enqueue = MsgBytesEnqueue - get_header(size, Header)}, expire_msgs(RaCmdTs, true, State, DlxEffects ++ Effects). @@ -2494,7 +2520,7 @@ is_over_limit(#?STATE{cfg = #cfg{max_length = MaxLength, max_bytes = MaxBytes}, msg_bytes_enqueue = BytesEnq, dlx = DlxState} = State) -> - {NumDlx, BytesDlx} = rabbit_fifo_dlx:stat(DlxState), + {NumDlx, BytesDlx} = dlx_stat(DlxState), (messages_ready(State) + NumDlx > MaxLength) orelse (BytesEnq + BytesDlx > MaxBytes). @@ -2505,7 +2531,7 @@ is_below_soft_limit(#?STATE{cfg = #cfg{max_length = MaxLength, max_bytes = MaxBytes}, msg_bytes_enqueue = BytesEnq, dlx = DlxState} = State) -> - {NumDlx, BytesDlx} = rabbit_fifo_dlx:stat(DlxState), + {NumDlx, BytesDlx} = dlx_stat(DlxState), is_below(MaxLength, messages_ready(State) + NumDlx) andalso is_below(MaxBytes, BytesEnq + BytesDlx). @@ -2732,8 +2758,10 @@ convert(Meta, 7, To, State) -> convert(Meta, 8, To, convert_v7_to_v8(Meta, State)). smallest_raft_index(#?STATE{messages = Messages, - dlx = DlxState} = State) -> - SmallestDlxRaIdx = rabbit_fifo_dlx:smallest_raft_index(DlxState), + dlx = #?DLX{discards = Discards}} = State) -> + SmallestDlxRaIdx = lqueue:fold(fun (?TUPLE(_, ?MSG(I, _)), Acc) -> + min(I, Acc) + end, undefined, Discards), SmallestMsgsRaIdx = rabbit_fifo_q:get_lowest_index(Messages), %% scan consumers and returns queue here instead smallest_checked_out(State, min(SmallestDlxRaIdx, SmallestMsgsRaIdx)). @@ -2750,12 +2778,6 @@ smallest_checked_out(#?STATE{returns = Returns, end, Acc0, Ch) end, SmallestSoFar, Consumers). -% smallest_raft_index_overview(#?STATE{messages = Messages, -% dlx = DlxState} = State) -> -% #{message => rabbit_fifo_q:get_lowest_index(Messages), -% checked_out => smallest_checked_out(State, undefined), -% dlx => rabbit_fifo_dlx:smallest_raft_index(DlxState)}. - make_requeue(ConsumerKey, Notify, [{MsgId, Idx, Header, Msg}], Acc) -> lists:reverse([{append, #requeue{consumer_key = ConsumerKey, @@ -2909,37 +2931,42 @@ priority_tag(Msg) -> end. do_snapshot(MacVer, Ts, Ch, - RaAux, BytesIn, BytesOut, Force) + RaAux, DiscardedBytes, Force) when element(1, Ch) == checkpoint andalso is_integer(MacVer) andalso MacVer >= 8 -> Idx = element(2, Ch), LastTs = element(3, Ch), - do_snapshot(MacVer, Ts, #snapshot{index = Idx, - timestamp = LastTs}, RaAux, BytesIn, - BytesOut, Force); + do_snapshot(MacVer, Ts, #snapshot{index = Idx, timestamp = LastTs}, + RaAux, DiscardedBytes, Force); do_snapshot(MacVer, Ts, #snapshot{index = _ChIdx, timestamp = SnapTime, - bytes_out = LastBytesOut} = Snap0, - RaAux, _BytesIn, BytesOut, Force) + discarded_bytes = LastDiscardedBytes} = Snap0, + RaAux, DiscardedBytes, Force) when is_integer(MacVer) andalso MacVer >= 8 -> LastAppliedIdx = ra_aux:last_applied(RaAux), #?STATE{} = MacState = ra_aux:machine_state(RaAux), TimeSince = Ts - SnapTime, MsgsTot = messages_total(MacState), - EnoughDataRemoved = BytesOut - LastBytesOut > ?SNAP_OUT_BYTES, + %% if the approximate snapshot size * 2 can be reclaimed it is worth + %% taking a snapshot + %% TODO: take number of enqueues and consumers into account + ApproxSnapSize = 4096 + (32 * MsgsTot), + EnoughDataRemoved = DiscardedBytes - LastDiscardedBytes > (ApproxSnapSize * 2), + {CheckMinInterval, _CheckMinIndexes, _CheckMaxIndexes} = persistent_term:get(quorum_queue_checkpoint_config, {?CHECK_MIN_INTERVAL_MS, ?CHECK_MIN_INDEXES, ?CHECK_MAX_INDEXES}), EnoughTimeHasPassed = TimeSince > CheckMinInterval, - case (EnoughTimeHasPassed andalso EnoughDataRemoved) orelse + case (EnoughTimeHasPassed andalso + EnoughDataRemoved) orelse Force of true -> {#snapshot{index = LastAppliedIdx, timestamp = Ts, messages_total = MsgsTot, - bytes_out = BytesOut}, + discarded_bytes = DiscardedBytes}, [{release_cursor, LastAppliedIdx, MacState}]}; false -> {Snap0, []} @@ -2949,6 +2976,7 @@ discard(Meta, MsgIds, ConsumerKey, #consumer{checked_out = Checked} = Con, DelFailed, Anns, #?STATE{cfg = #cfg{dead_letter_handler = DLH}, + discarded_bytes = DiscardedBytes0, dlx = DlxState0} = State0) -> %% We publish to dead-letter exchange in the same order %% as messages got rejected by the client. @@ -2961,9 +2989,10 @@ discard(Meta, MsgIds, ConsumerKey, {true, incr_msg(Msg0, DelFailed, Anns)} end end, MsgIds), - {DlxState, Effects} = rabbit_fifo_dlx:discard(DiscardMsgs, rejected, - DLH, DlxState0), - State = State0#?STATE{dlx = DlxState}, + {DlxState, RetainedBytes, Effects} = + discard_or_dead_letter(DiscardMsgs, rejected, DLH, DlxState0), + State = State0#?STATE{dlx = DlxState, + discarded_bytes = DiscardedBytes0 - RetainedBytes}, complete_and_checkout(Meta, MsgIds, ConsumerKey, Con, Effects, State). incr_msg(Msg0, DelFailed, Anns) -> @@ -3004,3 +3033,349 @@ exec_read(Flru0, ReadPlan, Msgs) -> maps_ordered_keys(Map) -> lists:sort(maps:keys(Map)). + +%% enqueue overhead: 256b + message size + +estimate_discarded_size(#?ENQ_V2{}) -> + 0; +estimate_discarded_size(Cmd) + when is_record(Cmd, settle) orelse + is_record(Cmd, return) orelse + is_record(Cmd, discard) orelse + is_record(Cmd, credit) -> + 128; +estimate_discarded_size(#checkout{}) -> + 300; +estimate_discarded_size(#register_enqueuer{}) -> + 200; +estimate_discarded_size(#modify{}) -> + 256; +estimate_discarded_size(#update_config{}) -> + 512; +estimate_discarded_size(#purge{}) -> + 64; +estimate_discarded_size(#purge_nodes{}) -> + 64; +estimate_discarded_size(#requeue{}) -> + 0; +estimate_discarded_size(#enqueue{}) -> + 0; +estimate_discarded_size({nodeup, _}) -> + 96; +estimate_discarded_size({down, _, _}) -> + 96; +estimate_discarded_size({dlx, _Cmd}) -> + 64; +estimate_discarded_size(_Cmd) -> + %% something is better than nothing + 64. + + +dlx_apply(_Meta, {dlx, {settle, MsgIds}}, at_least_once, + #?DLX{consumer = #dlx_consumer{checked_out = Checked0}} = State0) -> + Acked = maps:with(MsgIds, Checked0), + {DBytes, State} = + maps:fold(fun(MsgId, ?TUPLE(_Rsn, ?MSG(Idx, Hdr)), + {Sz, + #?DLX{consumer = #dlx_consumer{checked_out = Checked} = C, + msg_bytes_checkout = BytesCheckout, + ra_indexes = Indexes0} = S}) -> + Indexes = rabbit_fifo_index:delete(Idx, Indexes0), + Size = get_header(size, Hdr), + {Sz + Size + ?ENQ_OVERHEAD, + S#?DLX{consumer = C#dlx_consumer{checked_out = + maps:remove(MsgId, Checked)}, + msg_bytes_checkout = BytesCheckout - Size, + ra_indexes = Indexes}} + end, {0, State0}, Acked), + {State, DBytes, + [{mod_call, rabbit_global_counters, messages_dead_lettered_confirmed, + [rabbit_quorum_queue, at_least_once, maps:size(Acked)]}]}; +dlx_apply(_, {dlx, {checkout, Pid, Prefetch}}, + at_least_once, + #?DLX{consumer = undefined} = State0) -> + State = State0#?DLX{consumer = #dlx_consumer{pid = Pid, + prefetch = Prefetch}}, + {State, 0, []}; +dlx_apply(_, {dlx, {checkout, ConsumerPid, Prefetch}}, + at_least_once, + #?DLX{consumer = #dlx_consumer{pid = OldConsumerPid, + checked_out = CheckedOutOldConsumer}, + discards = Discards0, + msg_bytes = Bytes, + msg_bytes_checkout = BytesCheckout} = State0) -> + %% Since we allow only a single consumer, the new consumer replaces the old consumer. + case ConsumerPid of + OldConsumerPid -> + ok; + _ -> + ?LOG_DEBUG("Terminating ~p since ~p becomes active rabbit_fifo_dlx_worker", + [OldConsumerPid, ConsumerPid]), + %% turn into aux command + ensure_worker_terminated(State0) + end, + %% All checked out messages to the old consumer need to be returned to the discards queue + %% such that these messages will be re-delivered to the new consumer. + %% When inserting back into the discards queue, we respect the original order in which messages + %% were discarded. + Checked0 = maps:to_list(CheckedOutOldConsumer), + Checked1 = lists:keysort(1, Checked0), + {Discards, BytesMoved} = lists:foldr( + fun({_Id, ?TUPLE(_, Msg) = RsnMsg}, {D, B}) -> + Size = get_header(size, get_msg_header(Msg)), + {lqueue:in_r(RsnMsg, D), B + Size} + end, {Discards0, 0}, Checked1), + State = State0#?DLX{consumer = #dlx_consumer{pid = ConsumerPid, + prefetch = Prefetch}, + discards = Discards, + msg_bytes = Bytes + BytesMoved, + msg_bytes_checkout = BytesCheckout - BytesMoved}, + {State, 0, []}; +dlx_apply(_, Cmd, DLH, State) -> + ?LOG_DEBUG("Ignoring command ~tp for dead_letter_handler ~tp", [Cmd, DLH]), + {State, 0, []}. + +%% nodeup: 74 bytes +%% down: 90 bytes +%% enqueue overhead 210 + +% messages_get_next_msg(#messages{returns = Returns0, +% messages = Messages0}) -> +% case lqueue:get(Returns0, empty) of +% empty -> +% rabbit_fifo_q:get(Messages0); +% Msg -> +% Msg +% end. + +% messages_take_next_msg(#messages{returns = Returns0, +% messages = Messages0} = Msgs) -> +% case lqueue:out(Returns0) of +% {{value, NextMsg}, Returns} -> +% {NextMsg, Msgs#messages{returns = Returns}}; +% {empty, _} -> +% case rabbit_fifo_q:out(Messages0) of +% empty -> +% empty; +% {?MSG(_RaftIdx, _) = Msg, Messages} -> +% {Msg, Msgs#messages{messages = Messages}} +% end +% end. + +ensure_worker_started(QRef, #?DLX{consumer = undefined}) -> + start_worker(QRef); +ensure_worker_started(QRef, #?DLX{consumer = #dlx_consumer{pid = Pid}}) -> + case is_local_and_alive(Pid) of + true -> + ?LOG_DEBUG("rabbit_fifo_dlx_worker ~tp already started for ~ts", + [Pid, rabbit_misc:rs(QRef)]); + false -> + start_worker(QRef) + end. + +%% Ensure that starting the rabbit_fifo_dlx_worker succeeds. +%% Therefore, do not use an effect. +%% Also therefore, if starting the rabbit_fifo_dlx_worker fails, let the +%% Ra server process crash in which case another Ra node will become leader. +start_worker(QRef) -> + {ok, Pid} = supervisor:start_child(rabbit_fifo_dlx_sup, [QRef]), + ?LOG_DEBUG("started rabbit_fifo_dlx_worker ~tp for ~ts", + [Pid, rabbit_misc:rs(QRef)]). + +ensure_worker_terminated(#?DLX{consumer = undefined}) -> + ok; +ensure_worker_terminated(#?DLX{consumer = #dlx_consumer{pid = Pid}}) -> + case is_local_and_alive(Pid) of + true -> + %% Note that we can't return a mod_call effect here + %% because mod_call is executed on the leader only. + ok = supervisor:terminate_child(rabbit_fifo_dlx_sup, Pid), + ?LOG_DEBUG("terminated rabbit_fifo_dlx_worker ~tp", [Pid]); + false -> + ok + end. + +local_alive_consumer_pid(#?DLX{consumer = undefined}) -> + undefined; +local_alive_consumer_pid(#?DLX{consumer = #dlx_consumer{pid = Pid}}) -> + case is_local_and_alive(Pid) of + true -> + Pid; + false -> + undefined + end. + +is_local_and_alive(Pid) + when node(Pid) =:= node() -> + is_process_alive(Pid); +is_local_and_alive(_) -> + false. + +update_config(at_least_once, at_least_once, _, State) -> + case local_alive_consumer_pid(State) of + undefined -> + {State, []}; + Pid -> + %% Notify rabbit_fifo_dlx_worker about potentially updated policies. + {State, [{send_msg, Pid, {dlx_event, self(), lookup_topology}, cast}]} + end; +update_config(SameDLH, SameDLH, _, State) -> + {State, []}; +update_config(OldDLH, NewDLH, QRes, State0) -> + LogOnLeader = {mod_call, logger, debug, + ["Switching dead_letter_handler from ~tp to ~tp for ~ts", + [OldDLH, NewDLH, rabbit_misc:rs(QRes)]]}, + {State1, Effects0} = switch_from(OldDLH, QRes, State0), + {State, Effects} = switch_to(NewDLH, State1, Effects0), + {State, [LogOnLeader|Effects]}. + +switch_from(at_least_once, QRes, State) -> + %% Switch from at-least-once to some other strategy. + %% TODO: do worker teardown in aux handler + ensure_worker_terminated(State), + {Num, Bytes} = dlx_stat(State), + %% Log only on leader. + {#?DLX{}, [{mod_call, logger, info, + ["Deleted ~b dead-lettered messages (with total messages size of ~b bytes) in ~ts", + [Num, Bytes, rabbit_misc:rs(QRes)]]}]}; +switch_from(_, _, State) -> + {State, []}. + +switch_to(at_least_once, _, Effects) -> + %% Switch from some other strategy to at-least-once. + %% Dlx worker needs to be started on the leader. + %% The cleanest way to determine the Ra state of this node is delegation to handle_aux. + {#?DLX{}, [{aux, {dlx, setup}} | Effects]}; +switch_to(_, State, Effects) -> + {State, Effects}. + + +dlx_stat(#?DLX{consumer = Con, + discards = Discards, + msg_bytes = MsgBytes, + msg_bytes_checkout = MsgBytesCheckout}) -> + Num0 = lqueue:len(Discards), + Num = case Con of + undefined -> + Num0; + #dlx_consumer{checked_out = Checked} -> + %% O(1) because Erlang maps maintain their own size + Num0 + maps:size(Checked) + end, + Bytes = MsgBytes + MsgBytesCheckout, + {Num, Bytes}. + + +dlx_overview(#?DLX{consumer = undefined, + msg_bytes = MsgBytes, + msg_bytes_checkout = 0, + discards = Discards}) -> + dlx_overview0(Discards, #{}, MsgBytes, 0); +dlx_overview(#?DLX{consumer = #dlx_consumer{checked_out = Checked}, + msg_bytes = MsgBytes, + msg_bytes_checkout = MsgBytesCheckout, + discards = Discards}) -> + dlx_overview0(Discards, Checked, MsgBytes, MsgBytesCheckout). + +dlx_overview0(Discards, Checked, MsgBytes, MsgBytesCheckout) -> + #{num_discarded => lqueue:len(Discards), + num_discard_checked_out => maps:size(Checked), + discard_message_bytes => MsgBytes, + discard_checkout_message_bytes => MsgBytesCheckout}. + + +discard_or_dead_letter(Msgs, Reason, undefined, State) -> + {State, 0, + [{mod_call, rabbit_global_counters, messages_dead_lettered, + [Reason, rabbit_quorum_queue, disabled, length(Msgs)]}]}; +discard_or_dead_letter(Msgs0, Reason, {at_most_once, {Mod, Fun, Args}}, State) -> + Idxs = [I || ?MSG(I, _) <- Msgs0], + %% TODO: this could be turned into a log_ext effect instead to avoid + %% reading from disk inside the qq process + Effect = {log, Idxs, + fun (Log) -> + Lookup = maps:from_list(lists:zip(Idxs, Log)), + Msgs = [begin + Cmd = maps:get(Idx, Lookup), + %% ensure header delivery count + %% is copied to the message container + annotate_msg(H, rabbit_fifo:get_msg(Cmd)) + end || ?MSG(Idx, H) <- Msgs0], + [{mod_call, Mod, Fun, Args ++ [Reason, Msgs]}] + end}, + {State, 0, [Effect]}; +discard_or_dead_letter(Msgs, Reason, at_least_once, State0) + when Reason =/= maxlen -> + RetainedBytes = lists:foldl(fun (M, Acc) -> + Acc + size_in_bytes(M) + ?ENQ_OVERHEAD + end, 0, Msgs), + State = lists:foldl(fun(?MSG(Idx, _) = Msg, + #?DLX{discards = D0, + msg_bytes = B0, + ra_indexes = I0} = S0) -> + MsgSize = size_in_bytes(Msg), + D = lqueue:in(?TUPLE(Reason, Msg), D0), + B = B0 + MsgSize, + I = rabbit_fifo_index:append(Idx, I0), + S0#?DLX{discards = D, + msg_bytes = B, + ra_indexes = I} + end, State0, Msgs), + {State, RetainedBytes, + [{mod_call, rabbit_global_counters, messages_dead_lettered, + [Reason, rabbit_quorum_queue, at_least_once, length(Msgs)]}]}. + + +size_in_bytes(Msg) -> + Header = get_msg_header(Msg), + get_header(size, Header). + +dlx_checkout(at_least_once, #?DLX{consumer = #dlx_consumer{}} = State) -> + dlx_checkout0(dlx_checkout_one(State), []); +dlx_checkout(_, State) -> + {State, []}. + +dlx_checkout0({success, MsgId, ?TUPLE(Reason, ?MSG(Idx, H)), State}, SendAcc) -> + DelMsg = {Idx, {Reason, H, MsgId}}, + dlx_checkout0(dlx_checkout_one(State), [DelMsg | SendAcc]); +dlx_checkout0(#?DLX{consumer = #dlx_consumer{pid = Pid}} = State, SendAcc) -> + Effects = dlx_delivery_effects(Pid, SendAcc), + {State, Effects}. + +dlx_checkout_one(#?DLX{consumer = #dlx_consumer{checked_out = Checked, + prefetch = Prefetch}} = State) + when map_size(Checked) >= Prefetch -> + State; +dlx_checkout_one(#?DLX{discards = Discards0, + msg_bytes = Bytes, + msg_bytes_checkout = BytesCheckout, + consumer = #dlx_consumer{checked_out = Checked0, + next_msg_id = Next} = Con0} = State0) -> + case lqueue:out(Discards0) of + {{value, ?TUPLE(_, Msg) = ReasonMsg}, Discards} -> + Checked = maps:put(Next, ReasonMsg, Checked0), + Size = size_in_bytes(Msg), + State = State0#?DLX{discards = Discards, + msg_bytes = Bytes - Size, + msg_bytes_checkout = BytesCheckout + Size, + consumer = Con0#dlx_consumer{checked_out = Checked, + next_msg_id = Next + 1}}, + {success, Next, ReasonMsg, State}; + {empty, _} -> + State0 + end. + +dlx_delivery_effects(_CPid, []) -> + []; +dlx_delivery_effects(CPid, Msgs0) -> + Msgs1 = lists:reverse(Msgs0), + {RaftIdxs, RsnIds} = lists:unzip(Msgs1), + [{log, RaftIdxs, + fun(Log) -> + Msgs = lists:zipwith( + fun (Cmd, {Reason, H, MsgId}) -> + {MsgId, {Reason, + annotate_msg(H, rabbit_fifo:get_msg(Cmd))}} + end, Log, RsnIds), + [{send_msg, CPid, {dlx_event, self(), {dlx_delivery, Msgs}}, [cast]}] + end}]. diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index 7c814545e360..839bbf15aade 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -180,6 +180,31 @@ unused_3 = ?NIL }). +-record(messages, + { + messages = rabbit_fifo_q:new() :: rabbit_fifo_q:state(), + messages_total = 0 :: non_neg_integer(), + % queue of returned msg_in_ids - when checking out it picks from + returns = lqueue:new() :: lqueue:lqueue(term()) + }). + +-record(dlx_consumer, + {pid :: pid(), + prefetch :: non_neg_integer(), + checked_out = #{} :: #{msg_id() => + optimised_tuple(rabbit_dead_letter:reason(), msg())}, + next_msg_id = 0 :: msg_id()}). + +-record(rabbit_fifo_dlx, + {consumer :: option(#dlx_consumer{}), + %% Queue of dead-lettered messages. + discards = lqueue:new() :: lqueue:lqueue(optimised_tuple(rabbit_dead_letter:reason(), msg())), + %% Raft indexes of messages in both discards queue and dlx_consumer's checked_out map + %% so that we get the smallest ra index in O(1). + ra_indexes = rabbit_fifo_index:empty() :: rabbit_fifo_index:state(), + msg_bytes = 0 :: non_neg_integer(), + msg_bytes_checkout = 0 :: non_neg_integer()}). + -record(rabbit_fifo, {cfg :: #cfg{}, % unassigned messages @@ -207,7 +232,7 @@ % consumers that require further service are queued here service_queue = priority_queue:new() :: priority_queue:q(), %% state for at-least-once dead-lettering - dlx = rabbit_fifo_dlx:init() :: rabbit_fifo_dlx:state(), + dlx = #rabbit_fifo_dlx{} :: #rabbit_fifo_dlx{}, msg_bytes_enqueue = 0 :: non_neg_integer(), msg_bytes_checkout = 0 :: non_neg_integer(), %% one is picked if active consumer is cancelled or dies diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index 922cdea09d14..aec7988b244a 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -143,13 +143,13 @@ enqueue(QName, Correlation, Msg, {reject_publish, State0}; {error, {shutdown, delete}} -> ?LOG_DEBUG("~ts: QQ ~ts tried to register enqueuer during delete shutdown", - [?MODULE, rabbit_misc:rs(QName)]), + [?MODULE, rabbit_misc:rs(QName)]), {reject_publish, State0}; {timeout, _} -> {reject_publish, State0}; Err -> ?LOG_DEBUG("~ts: QQ ~ts error when registering enqueuer ~p", - [?MODULE, rabbit_misc:rs(QName), Err]), + [?MODULE, rabbit_misc:rs(QName), Err]), exit(Err) end; enqueue(_QName, _Correlation, _Msg, diff --git a/deps/rabbit/src/rabbit_fifo_dlx.erl b/deps/rabbit/src/rabbit_fifo_dlx.erl index 3d79f3ae84ae..e0f62075a94f 100644 --- a/deps/rabbit/src/rabbit_fifo_dlx.erl +++ b/deps/rabbit/src/rabbit_fifo_dlx.erl @@ -7,7 +7,7 @@ -module(rabbit_fifo_dlx). -include("rabbit_fifo_dlx.hrl"). --include("rabbit_fifo.hrl"). +-include("rabbit_fifo_v7.hrl"). -include_lib("kernel/include/logger.hrl"). -compile({no_auto_import, [apply/3]}). diff --git a/deps/rabbit/src/rabbit_fifo_dlx.hrl b/deps/rabbit/src/rabbit_fifo_dlx.hrl index 9fe224c1921d..d14a79c5451c 100644 --- a/deps/rabbit/src/rabbit_fifo_dlx.hrl +++ b/deps/rabbit/src/rabbit_fifo_dlx.hrl @@ -7,7 +7,8 @@ -record(dlx_consumer, {pid :: pid(), prefetch :: non_neg_integer(), - checked_out = #{} :: #{msg_id() => optimised_tuple(rabbit_dead_letter:reason(), msg())}, + checked_out = #{} :: #{msg_id() => + optimised_tuple(rabbit_dead_letter:reason(), msg())}, next_msg_id = 0 :: msg_id()}). -record(rabbit_fifo_dlx, diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 5ffe1cd35bc9..252bc0ef97dc 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -374,13 +374,11 @@ ra_machine_config(Q) when ?is_amqqueue(Q) -> PolicyConfig = gather_policy_config(Q, true), QName = amqqueue:get_name(Q), {Name, _} = amqqueue:get_pid(Q), - PolicyConfig#{ - name => Name, - queue_resource => QName, - become_leader_handler => {?MODULE, become_leader, [QName]}, - single_active_consumer_on => single_active_consumer_on(Q), - created => erlang:system_time(millisecond) - }. + PolicyConfig#{name => Name, + queue_resource => QName, + single_active_consumer_on => single_active_consumer_on(Q), + created => erlang:system_time(millisecond) + }. resolve_delivery_limit(PolVal, ArgVal) when PolVal < 0 orelse ArgVal < 0 -> @@ -679,13 +677,13 @@ handle_tick(QName, catch _:Err -> ?LOG_DEBUG("~ts: handle tick failed with ~p", - [rabbit_misc:rs(QName), Err]), + [rabbit_misc:rs(QName), Err]), ok end end); handle_tick(QName, Config, _Nodes) -> ?LOG_DEBUG("~ts: handle tick received unexpected config format ~tp", - [rabbit_misc:rs(QName), Config]). + [rabbit_misc:rs(QName), Config]). repair_leader_record(Q, Name) -> Node = node(), @@ -696,7 +694,7 @@ repair_leader_record(Q, Name) -> _ -> QName = amqqueue:get_name(Q), ?LOG_DEBUG("~ts: updating leader record to current node ~ts", - [rabbit_misc:rs(QName), Node]), + [rabbit_misc:rs(QName), Node]), ok = become_leader0(QName, Name), ok end, diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index b5b920c1ecc2..6325809d597e 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -3801,16 +3801,18 @@ message_bytes_metrics(Config) -> wait_for_messages_pending_ack(Servers, RaName, 0), rabbit_ct_helpers:await_condition( fun() -> - {3, 3, 0} == get_message_bytes(Leader, QRes) + {M, M, 0} = get_message_bytes(Leader, QRes), + M > 0 end, 30000), + {MsgSize, _, _} = get_message_bytes(Leader, QRes), subscribe(Ch, QQ, false), wait_for_messages_ready(Servers, RaName, 0), wait_for_messages_pending_ack(Servers, RaName, 1), rabbit_ct_helpers:await_condition( fun() -> - {3, 0, 3} == get_message_bytes(Leader, QRes) + {MsgSize, 0, MsgSize} == get_message_bytes(Leader, QRes) end, 30000), receive @@ -3835,7 +3837,7 @@ message_bytes_metrics(Config) -> wait_for_messages_pending_ack(Servers, RaName, 1), rabbit_ct_helpers:await_condition( fun() -> - {3, 0, 3} == get_message_bytes(Leader, QRes) + {MsgSize, 0, MsgSize} == get_message_bytes(Leader, QRes) end, 30000), rabbit_ct_client_helpers:close_channel(Ch), @@ -3844,7 +3846,7 @@ message_bytes_metrics(Config) -> wait_for_messages_pending_ack(Servers, RaName, 0), rabbit_ct_helpers:await_condition( fun() -> - {3, 3, 0} == get_message_bytes(Leader, QRes) + {MsgSize, MsgSize, 0} == get_message_bytes(Leader, QRes) end, 30000), ok. diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 7e961294a691..f0112f1d8233 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -14,7 +14,6 @@ -include_lib("rabbit_common/include/rabbit.hrl"). -include_lib("rabbit_common/include/rabbit_framing.hrl"). -include_lib("rabbit/src/rabbit_fifo.hrl"). --include_lib("rabbit/src/rabbit_fifo_dlx.hrl"). % -define(PROTOMOD, rabbit_framing_amqp_0_9_1). %%%=================================================================== @@ -101,13 +100,203 @@ test_init(Name) -> -define(FUNCTION_NAME_B, atom_to_binary(?FUNCTION_NAME)). -define(LINE_B, integer_to_binary(?LINE)). - enq_enq_checkout_compat_test(C) -> enq_enq_checkout_test(C, {auto, 2, simple_prefetch}). enq_enq_checkout_v4_test(C) -> enq_enq_checkout_test(C, {auto, {simple_prefetch, 2}}). +discarded_bytes_test(Config) -> + Conf = #{name => ?FUNCTION_NAME_B, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B) + }, + CPid = spawn(fun () -> ok end), + Cid = {?FUNCTION_NAME_B, CPid}, + Msg = crypto:strong_rand_bytes(1000), + {State1, _} = enq(Config, ?LINE, 1, Msg, init(Conf)), + %% enqueues should not increment discarded bytes + ?assertMatch(#{num_messages := 1, + discarded_bytes := 0}, rabbit_fifo:overview(State1)), + Spec = {auto, {simple_prefetch, 2}}, + {State2, #{key := CKey, + next_msg_id := NextMsgId}, _Effects} = + checkout(Config, ?LINE, Cid, Spec, State1), + #{discarded_bytes := DiscBytes2} = rabbit_fifo:overview(State2), + ?assert(DiscBytes2 > 0), + {State3, _} = settle(Config, CKey, ?LINE, [NextMsgId], State2), + #{num_messages := 0, + discarded_bytes := DiscBytes3} = rabbit_fifo:overview(State3), + %% disc bytes increment shoudl include message size _and_ settle size + ?assert(DiscBytes3 - DiscBytes2 > 1000), + + {State4, _, _} = apply(meta(Config, ?LINE), + {down, CPid, noconnection}, State3), + #{discarded_bytes := DiscBytes4} = rabbit_fifo:overview(State4), + ?assert(DiscBytes4 > DiscBytes3), + {State5, _, _} = apply(meta(Config, ?LINE), + {nodeup, node()}, State4), + #{discarded_bytes := DiscBytes5} = rabbit_fifo:overview(State5), + ?assert(DiscBytes5 > DiscBytes4), + + {State6, _} = enq(Config, ?LINE, 2, Msg, State5), + #{num_messages := 1, + discarded_bytes := DiscBytes5} = rabbit_fifo:overview(State6), + {State7, _, _} = apply(meta(Config, ?LINE), + rabbit_fifo:make_return(CKey, [NextMsgId + 1]), + State6), + #{num_messages := 1, + discarded_bytes := DiscBytes7} = rabbit_fifo:overview(State7), + ?assert(DiscBytes7 > DiscBytes5 andalso DiscBytes7 - DiscBytes5 < 1000), + + %% discard without at-least-once dead lettering configured should + %% discard the full message body + {State8, _, _} = apply(meta(Config, ?LINE), + rabbit_fifo:make_discard(CKey, [NextMsgId + 2]), + State7), + #{num_messages := 0, + discarded_bytes := DiscBytes8} = rabbit_fifo:overview(State8), + ?assert(DiscBytes8 - DiscBytes7 > 1000), + + {State9, _} = enq(Config, ?LINE, 3, Msg, State8), + #{num_messages := 1, + discarded_bytes := DiscBytes9} = rabbit_fifo:overview(State9), + + %% update config to have a delivery-limit + Conf2 = Conf#{delivery_limit => 1}, + {State10, ok, _} = apply(meta(Config, 5), + rabbit_fifo:make_update_config(Conf2), State9), + #{num_messages := 1, + discarded_bytes := DiscBytes10} = rabbit_fifo:overview(State10), + ?assert(DiscBytes10 > DiscBytes9), + + {State11, _, _} = apply(meta(Config, ?LINE), + rabbit_fifo:make_return(CKey, [NextMsgId + 3]), + State10), + #{num_messages := 1, + discarded_bytes := DiscBytes11} = rabbit_fifo:overview(State11), + ?assert(DiscBytes11 > DiscBytes10), + + {State12, _, _} = apply(meta(Config, ?LINE), + rabbit_fifo:make_return(CKey, [NextMsgId + 4]), + State11), + + %% delivery-limit was reached and message was discarded + #{num_messages := 0, + discarded_bytes := DiscBytes12} = rabbit_fifo:overview(State12), + ?assert(DiscBytes12 - DiscBytes11 > 1000), + + %% at-least-once dead lettering + Conf3 = Conf2#{dead_letter_handler => at_least_once}, + {State13, ok, _} = apply(meta(Config, ?LINE), + rabbit_fifo:make_update_config(Conf3), State12), + + {State14, _} = enq(Config, ?LINE, 4, Msg, State13), + + #{num_messages := 1, + discarded_bytes := DiscBytes14} = rabbit_fifo:overview(State14), + + {State15, _, _} = apply(meta(Config, ?LINE), + rabbit_fifo:make_discard(CKey, [NextMsgId + 5]), + State14), + #{num_messages := 1, + discarded_bytes := DiscBytes15} = rabbit_fifo:overview(State15), + ?assert(DiscBytes15 > DiscBytes14 andalso + DiscBytes15 - DiscBytes14 < 1000), + + %% attach dlx consumer + + DlxPid = spawn(fun () -> ok end), + {State16, _, _} = apply(meta(Config, ?LINE), + rabbit_fifo_dlx:make_checkout(DlxPid, 2), + State15), + #{num_messages := 1, + discarded_bytes := DiscBytes16} = rabbit_fifo:overview(State16), + ?assert(DiscBytes16 > DiscBytes15), + + {State17, _, _} = apply(meta(Config, ?LINE), + rabbit_fifo_dlx:make_settle([0]), + State16), + #{num_messages := 0, + discarded_bytes := DiscBytes17} = rabbit_fifo:overview(State17), + ?assert(DiscBytes17 - DiscBytes16 > 1000), + + {State18, _} = enq(Config, ?LINE, 5, Msg, State17), + #{num_messages := 1, + discarded_bytes := DiscBytes17} = rabbit_fifo:overview(State18), + + {State19, _, _} = apply(meta(Config, ?LINE), + rabbit_fifo:make_modify(CKey, [NextMsgId + 5], + false, false, #{}), + State18), + #{num_messages := 1, + discarded_bytes := DiscBytes19} = rabbit_fifo:overview(State19), + ?assert(DiscBytes19 > DiscBytes17), + + %% change the dlx handler + Conf4 = Conf3#{dead_letter_handler => {at_most_once, {?MODULE, ?FUNCTION_NAME, []}}, + max_length => 2}, + {State20, ok, _} = apply(meta(Config, ?LINE), + rabbit_fifo:make_update_config(Conf4), State19), + #{num_messages := 1, + discarded_bytes := DiscBytes20} = rabbit_fifo:overview(State20), + + {State21, _, _} = apply(meta(Config, ?LINE), + rabbit_fifo:make_modify(CKey, [NextMsgId + 6], + true, true, #{}), + State20), + #{num_messages := 0, + discarded_bytes := DiscBytes21} = rabbit_fifo:overview(State21), + ?assert(DiscBytes21 - DiscBytes20 > 1000), + + %% unsubsrcibe + {State22, _, _} = apply(meta(Config, ?LINE), + make_checkout(Cid, cancel, #{}), State21), + #{num_messages := 0, + discarded_bytes := DiscBytes22} = rabbit_fifo:overview(State22), + ?assert(DiscBytes22 > DiscBytes21), + + {State23, _} = enq(Config, ?LINE, 6, Msg, State22), + #{num_messages := 1, + discarded_bytes := DiscBytes23} = rabbit_fifo:overview(State23), + ?assert(DiscBytes22 =:= DiscBytes23), + + {State24, _} = enq(Config, ?LINE, 7, Msg, State23), + #{num_messages := 2, + discarded_bytes := DiscBytes24} = rabbit_fifo:overview(State24), + ?assert(DiscBytes23 =:= DiscBytes24), + + %% drop head should increment + {State25, _} = enq(Config, ?LINE, 8, Msg, State24), + #{num_messages := 2, + discarded_bytes := DiscBytes25} = rabbit_fifo:overview(State25), + ?assert(DiscBytes25 - DiscBytes24 > 1000), + + %% duplicate enqueue should also increment discarded bytes + {State26, _} = enq(Config, ?LINE, 8, Msg, State25), + #{num_messages := 2, + discarded_bytes := DiscBytes26} = rabbit_fifo:overview(State26), + ?assert(DiscBytes26 - DiscBytes25 > 1000), + %% test expiration + {State27, _, _} = apply(meta(Config, ?LINE), + rabbit_fifo:make_purge(), State26), + #{num_messages := 0, + discarded_bytes := _DiscBytes27} = rabbit_fifo:overview(State27), + + Conf5 = Conf4#{msg_ttl => 1000, + max_length => undefined}, + {State28, ok, _} = apply(meta(Config, ?LINE), + rabbit_fifo:make_update_config(Conf5), State27), + {State29, _} = enq_ts(Config, ?LINE, 9, Msg, 0, State28), + #{num_messages := 1, + discarded_bytes := DiscBytes29} = rabbit_fifo:overview(State29), + {State30, _} = enq_ts(Config, ?LINE, 10, Msg, 3000, State29), + % {State31, _} = enq_ts(Config, ?LINE, 11, Msg, 5000, State30), + + #{num_messages := 1, + discarded_bytes := DiscBytes30} = rabbit_fifo:overview(State30), + ?assert(DiscBytes30 - DiscBytes29 > 1000), + ok. + enq_enq_checkout_test(Config, Spec) -> Cid = {?FUNCTION_NAME_B, self()}, {State1, _} = enq(Config, 1, 1, first, test_init(?FUNCTION_NAME)), @@ -501,8 +690,6 @@ return_dequeue_delivery_limit_test(C) -> Init = init(#{name => test, queue_resource => rabbit_misc:r("/", queue, atom_to_binary(test, utf8)), - max_in_memory_length => 0, - release_cursor_interval => 0, delivery_limit => 1}), {State0, _} = enq(C, 1, 1, msg, Init), @@ -2221,7 +2408,8 @@ reject_publish_applied_after_limit_test(Config) -> overflow_strategy => reject_publish, dead_letter_handler => undefined }, - {State5, ok, Efx1} = apply(meta(Config, 5), rabbit_fifo:make_update_config(Conf), State4), + {State5, ok, Efx1} = apply(meta(Config, 5), + rabbit_fifo:make_update_config(Conf), State4), ?ASSERT_EFF({send_msg, P, {queue_status, reject_publish}, [ra_event]}, P == Pid1, Efx1), Pid2 = test_util:fake_pid(node()), @@ -2341,6 +2529,12 @@ enq(Config, Idx, MsgSeq, Msg, State) -> rabbit_fifo:make_enqueue(self(), MsgSeq, Msg), State)). +enq_ts(Config, Idx, MsgSeq, Msg, Ts, State) -> + strip_reply( + apply(meta(Config, Idx, Ts, {notify, MsgSeq, self()}), + rabbit_fifo:make_enqueue(self(), MsgSeq, Msg), + State)). + deq(Config, Idx, Cid, Settlement, Msg, State0) -> {State, _, Effs} = apply(meta(Config, Idx), diff --git a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl index 639403566aae..1d8297a452ca 100644 --- a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl @@ -453,30 +453,31 @@ stats(Config) -> ]), #'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = DLX}), declare_queue(Ch, TargetQ, []), - Msg = <<"12">>, %% 2 bytes per message + Msg = <<"12">>, %% 2 + 4(basic props) bytes per message [ok = amqp_channel:cast(Ch, #'basic.publish'{routing_key = SourceQ}, - #amqp_msg{props = #'P_basic'{expiration = <<"0">>}, + #amqp_msg{props = #'P_basic'{expiration = <<"0">>}, payload = Msg}) || _ <- lists:seq(1, 10)], %% 10 messages in total RaName = ra_name(SourceQ), %% Binding from target queue to DLX is missing. Therefore %% * 10 msgs should be discarded (i.e. in discards queue or checked out to dlx_worker) - %% * 20 bytes (=10msgs*2bytes) should be discarded (i.e. in discards queue or checked out to dlx_worker) - eventually(?_assertEqual([{10, 20}], + %% * 60 bytes (=10msgs*(2+4)bytes) should be discarded + %% (i.e. in discards queue or checked out to dlx_worker) + eventually(?_assertEqual([{10, 60}], dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))), ?assertMatch([#{ %% 2 msgs (=Prefetch) should be checked out to dlx_worker num_discard_checked_out := 2, %% 4 bytes (=2msgs*2bytes) should be checked out to dlx_worker - discard_checkout_message_bytes := 4, + discard_checkout_message_bytes := B, %% 8 msgs (=10-2) should be in discards queue num_discarded := 8, %% 16 bytes (=8msgs*2bytes) should be in discards queue - discard_message_bytes := 16, + discard_message_bytes := B2, %% 10 msgs in total num_messages := 10 - }], + }] when B > 0 andalso B2 > B, dirty_query([Server], RaName, fun rabbit_fifo:overview/1)), ?assertEqual(10, counted(messages_dead_lettered_expired_total, Config)), ?assertEqual(0, counted(messages_dead_lettered_confirmed_total, Config)), @@ -548,12 +549,12 @@ switch_strategy(Config) -> [#{ %% 2 msgs (=Prefetch) should be checked out to dlx_worker num_discard_checked_out := 2, - discard_checkout_message_bytes := 2, + discard_checkout_message_bytes := B, %% 3 msgs (=5-2) should be in discards queue num_discarded := 3, - discard_message_bytes := 3, + discard_message_bytes := B2, num_messages := 5 - }], + }] when B > 0 andalso B2 > 0, dirty_query([Server], RaName, fun rabbit_fifo:overview/1))), ok = rabbit_ct_broker_helpers:set_policy(Config, Server, PolicyName, SourceQ, <<"queues">>, @@ -580,7 +581,7 @@ reject_publish_source_queue_max_length(Config) -> %% Test that source quorum queue rejects messages when source quorum queue's max-length-bytes is reached. %% max-length-bytes should also take into account dead-lettered messages. reject_publish_source_queue_max_length_bytes(Config) -> - reject_publish(Config, {<<"x-max-length-bytes">>, long, 1}). + reject_publish(Config, {<<"x-max-length-bytes">>, long, 4}). reject_publish(Config, QArg) when is_tuple(QArg) -> Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), @@ -604,7 +605,7 @@ reject_publish(Config, QArg) when is_tuple(QArg) -> ok = publish_confirm(Ch, SourceQ), ok = publish_confirm(Ch, SourceQ), RaName = ra_name(SourceQ), - eventually(?_assertMatch([{2, 2}], %% 2 messages with 1 byte each + eventually(?_assertMatch([{2, _}], %% 2 messages with 1 byte each dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))), %% Now, we have 2 expired messages in the source quorum queue's discards queue. @@ -613,7 +614,7 @@ reject_publish(Config, QArg) when is_tuple(QArg) -> %% Fix the dead-letter routing topology. ok = rabbit_ct_broker_helpers:set_policy(Config, Server, PolicyName, SourceQ, <<"queues">>, [{<<"dead-letter-routing-key">>, TargetQ}]), - eventually(?_assertEqual([{0, 0}], + eventually(?_assertMatch([{0, _}], dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1)), 500, 6), %% Publish should be allowed again. ok = publish_confirm(Ch, SourceQ), @@ -659,7 +660,7 @@ reject_publish_max_length_target_quorum_queue(Config) -> amqp_channel:call(Ch, #'basic.get'{queue = TargetQ}), 30000) end || N <- lists:seq(1,4)], - eventually(?_assertEqual([{0, 0}], + eventually(?_assertMatch([{0, _}], dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1)), 500, 10), ?assertEqual(4, counted(messages_dead_lettered_expired_total, Config)), eventually(?_assertEqual(4, counted(messages_dead_lettered_confirmed_total, Config))). @@ -710,7 +711,7 @@ reject_publish_down_target_quorum_queue(Config) -> sets:add_element(Msg, S) end, sets:new([{version, 2}]), lists:seq(1, 50)), ?assertEqual(50, sets:size(Received)), - eventually(?_assertEqual([{0, 0}], + eventually(?_assertMatch([{0, _}], dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1)), 500, 10), ?assertEqual(50, counted(messages_dead_lettered_expired_total, Config)), eventually(?_assertEqual(50, counted(messages_dead_lettered_confirmed_total, Config))). @@ -736,16 +737,16 @@ reject_publish_target_classic_queue(Config) -> ok = amqp_channel:cast(Ch, #'basic.publish'{routing_key = SourceQ}, #amqp_msg{payload = Msg}), ok = amqp_channel:cast(Ch, #'basic.publish'{routing_key = SourceQ}, #amqp_msg{payload = Msg}), %% By now we expect target classic queue confirmed 1 message and rejected 1 message. - eventually(?_assertEqual([{1, 1}], + eventually(?_assertMatch([{1, _}], dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))), - consistently(?_assertEqual([{1, 1}], + consistently(?_assertMatch([{1, _}], dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))), ?assertEqual(2, counted(messages_dead_lettered_expired_total, Config)), ?assertEqual(1, counted(messages_dead_lettered_confirmed_total, Config)), %% Let's make space in the target queue for the rejected message. {#'basic.get_ok'{}, #amqp_msg{payload = Msg}} = amqp_channel:call(Ch, #'basic.get'{queue = TargetQ}), eventually(?_assertEqual(2, counted(messages_dead_lettered_confirmed_total, Config)), 500, 6), - ?assertEqual([{0, 0}], dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1)), + ?assertMatch([{0, _}], dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1)), {#'basic.get_ok'{}, #amqp_msg{payload = Msg}} = amqp_channel:call(Ch, #'basic.get'{queue = TargetQ}), ok. @@ -804,7 +805,7 @@ target_quorum_queue_delete_create(Config) -> Send100Msgs(), %% Expect no message to get stuck in dlx worker. wait_for_min_messages(Config, TargetQ, 200), - eventually(?_assertEqual([{0, 0}], + eventually(?_assertMatch([{0, _}], dirty_query([Server], ra_name(SourceQ), fun rabbit_fifo:query_stat_dlx/1)), 500, 10), ?assertEqual(300, counted(messages_dead_lettered_expired_total, Config)), ?assertEqual(300, counted(messages_dead_lettered_confirmed_total, Config)), @@ -885,7 +886,7 @@ many_target_queues(Config) -> after 30_000 -> exit(deliver_timeout) end, - ?awaitMatch([{0, 0}], + ?awaitMatch([{0, _}], dirty_query([Server1], RaName, fun rabbit_fifo:query_stat_dlx/1), ?DEFAULT_WAIT, ?DEFAULT_INTERVAL), ok = rabbit_ct_broker_helpers:stop_node(Config, Server3), @@ -898,14 +899,14 @@ many_target_queues(Config) -> %% Nodes 2 and 3 are down. %% rabbit_fifo_dlx_worker should wait until all queues confirm the message %% before acking it to the source queue. - ?awaitMatch([{1, 2}], + ?awaitMatch([{1, _}], dirty_query([Server1], RaName, fun rabbit_fifo:query_stat_dlx/1), ?DEFAULT_WAIT, ?DEFAULT_INTERVAL), ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg2}}, amqp_channel:call(Ch, #'basic.get'{queue = TargetQ1})), ok = rabbit_ct_broker_helpers:start_node(Config, Server2), ok = rabbit_ct_broker_helpers:start_node(Config, Server3), - ?awaitMatch([{0, 0}], + ?awaitMatch([{0, _}], dirty_query([Server1], RaName, fun rabbit_fifo:query_stat_dlx/1), 3000, 500), ?awaitMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg2}}, diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index e28b2ee07fae..15545923a478 100644 --- a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl @@ -7,7 +7,6 @@ -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). -include_lib("rabbit/src/rabbit_fifo.hrl"). --include_lib("rabbit/src/rabbit_fifo_dlx.hrl"). -include_lib("rabbit_common/include/rabbit_framing.hrl"). -include_lib("rabbit_common/include/rabbit.hrl"). @@ -1629,7 +1628,7 @@ upgrade_prop(Conf0, Commands) -> fun (_) -> true end, FromVersion), %% perform conversion - #rabbit_fifo{} = V4 = element(1, rabbit_fifo:apply( + #rabbit_fifo{} = V4 = element(1, rabbit_fifo:apply_( meta(length(PreEntries) + 1), {machine_version, FromVersion, ToVersion}, V3)), @@ -2100,7 +2099,7 @@ do_apply(Cmd, #t{effects = Effs, EnqCmds0 end, - {St, Effects} = case rabbit_fifo:apply(meta(Index), Cmd, S0) of + {St, Effects} = case rabbit_fifo:apply_(meta(Index), Cmd, S0) of {S, _, E} when is_list(E) -> {S, E}; {S, _, E} -> From 9c55f28c7b35a12cd6f1cf487775117a4fcb5053 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 6 Oct 2025 10:12:01 +0100 Subject: [PATCH 08/28] QQ: fix test affected by automated rename of rabbit_fifo:apply/3 --- deps/rabbit/test/rabbit_fifo_prop_SUITE.erl | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index 15545923a478..e866f7a79ffe 100644 --- a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl @@ -1628,7 +1628,7 @@ upgrade_prop(Conf0, Commands) -> fun (_) -> true end, FromVersion), %% perform conversion - #rabbit_fifo{} = V4 = element(1, rabbit_fifo:apply_( + #rabbit_fifo{} = V4 = element(1, rabbit_fifo:apply( meta(length(PreEntries) + 1), {machine_version, FromVersion, ToVersion}, V3)), @@ -2099,7 +2099,7 @@ do_apply(Cmd, #t{effects = Effs, EnqCmds0 end, - {St, Effects} = case rabbit_fifo:apply_(meta(Index), Cmd, S0) of + {St, Effects} = case rabbit_fifo:apply(meta(Index), Cmd, S0) of {S, _, E} when is_list(E) -> {S, E}; {S, _, E} -> From a29c9dd6aae2ef3ef1b262a4917da1d969a3ab2a Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 6 Oct 2025 12:31:57 +0100 Subject: [PATCH 09/28] QQ: Formatting and test flake improvement --- deps/rabbit/src/rabbit_fifo.erl | 72 ++++++++++----------- deps/rabbit/test/quorum_queue_SUITE.erl | 21 +----- deps/rabbit/test/rabbit_fifo_prop_SUITE.erl | 2 +- 3 files changed, 39 insertions(+), 56 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 1c50fc746eca..266c4a13a1ec 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -365,11 +365,12 @@ apply_(_, #checkout{spec = {dequeue, _}}, #?STATE{cfg = #cfg{consumer_strategy = single_active}} = State0) -> {State0, {error, {unsupported, single_active_consumer}}}; apply_(#{index := Index, - system_time := Ts, - from := From} = Meta, #checkout{spec = {dequeue, Settlement}, - meta = ConsumerMeta, - consumer_id = ConsumerId}, - #?STATE{consumers = Consumers} = State00) -> + system_time := Ts, + from := From} = Meta, + #checkout{spec = {dequeue, Settlement}, + meta = ConsumerMeta, + consumer_id = ConsumerId}, + #?STATE{consumers = Consumers} = State00) -> %% dequeue always updates last_active State0 = State00#?STATE{last_active = Ts}, %% all dequeue operations result in keeping the queue from expiring @@ -416,8 +417,8 @@ apply_(#{index := Index, end end; apply_(#{index := _Idx} = Meta, - #checkout{spec = Spec, - consumer_id = ConsumerId}, State0) + #checkout{spec = Spec, + consumer_id = ConsumerId}, State0) when Spec == cancel orelse Spec == remove -> case consumer_key_from_id(ConsumerId, State0) of @@ -432,9 +433,9 @@ apply_(#{index := _Idx} = Meta, {State0, {error, consumer_not_found}, []} end; apply_(#{index := Idx} = Meta, - #checkout{spec = Spec0, - meta = ConsumerMeta, - consumer_id = {_, Pid} = ConsumerId}, State0) -> + #checkout{spec = Spec0, + meta = ConsumerMeta, + consumer_id = {_, Pid} = ConsumerId}, State0) -> %% might be better to check machine_version IsV4 = tuple_size(Spec0) == 2, %% normalise spec format @@ -475,7 +476,7 @@ apply_(#{index := Idx} = Meta, num_checked_out => map_size(Checked)}}, checkout(Meta, State0, State2, [{monitor, process, Pid} | Effs], Reply); apply_(#{index := Index}, #purge{}, - #?STATE{messages_total = Total} = State0) -> + #?STATE{messages_total = Total} = State0) -> NumReady = messages_ready(State0), State1 = State0#?STATE{messages = rabbit_fifo_q:new(), messages_total = Total - NumReady, @@ -491,11 +492,11 @@ apply_(#{index := _Idx}, #garbage_collection{}, State) -> apply_(Meta, {timeout, expire_msgs}, State) -> checkout(Meta, State, State, []); apply_(#{system_time := Ts} = Meta, - {down, Pid, noconnection}, - #?STATE{consumers = Cons0, - cfg = #cfg{consumer_strategy = single_active}, - waiting_consumers = Waiting0, - enqueuers = Enqs0} = State0) -> + {down, Pid, noconnection}, + #?STATE{consumers = Cons0, + cfg = #cfg{consumer_strategy = single_active}, + waiting_consumers = Waiting0, + enqueuers = Enqs0} = State0) -> Node = node(Pid), %% if the pid refers to an active or cancelled consumer, %% mark it as suspected and return it to the waiting queue @@ -538,9 +539,9 @@ apply_(#{system_time := Ts} = Meta, Effects = [{monitor, node, Node} | Effects1], checkout(Meta, State0, State#?STATE{enqueuers = Enqs}, Effects); apply_(#{system_time := Ts} = Meta, - {down, Pid, noconnection}, - #?STATE{consumers = Cons0, - enqueuers = Enqs0} = State0) -> + {down, Pid, noconnection}, + #?STATE{consumers = Cons0, + enqueuers = Enqs0} = State0) -> %% A node has been disconnected. This doesn't necessarily mean that %% any processes on this node are down, they _may_ come back so here %% we just mark them as suspected (effectively deactivated) @@ -580,7 +581,7 @@ apply_(Meta, {down, Pid, _Info}, State0) -> {State1, Effects1} = activate_next_consumer(handle_down(Meta, Pid, State0)), checkout(Meta, State0, State1, Effects1); apply_(Meta, {nodeup, Node}, #?STATE{consumers = Cons0, - enqueuers = Enqs0} = State0) -> + enqueuers = Enqs0} = State0) -> %% A node we are monitoring has come back. %% If we have suspected any processes of being %% down we should now re-issue the monitors for them to detect if they're @@ -628,10 +629,10 @@ apply_(Meta, #purge_nodes{nodes = Nodes}, State0) -> end, {State0, []}, Nodes), {State, ok, Effects}; apply_(Meta, - #update_config{config = #{} = Conf}, - #?STATE{cfg = #cfg{dead_letter_handler = OldDLH, - resource = QRes}, - dlx = DlxState0} = State0) -> + #update_config{config = #{} = Conf}, + #?STATE{cfg = #cfg{dead_letter_handler = OldDLH, + resource = QRes}, + dlx = DlxState0} = State0) -> NewDLH = maps:get(dead_letter_handler, Conf, OldDLH), {DlxState, Effects0} = update_config(OldDLH, NewDLH, QRes, DlxState0), @@ -656,19 +657,18 @@ live_indexes(#?STATE{cfg = #cfg{}, messages = Messages, consumers = Consumers, dlx = #?DLX{discards = Discards}}) -> + MsgsIdxs = rabbit_fifo_q:indexes(Messages), DlxIndexes = lqueue:fold(fun (?TUPLE(_, ?MSG(I, _)), Acc) -> [I | Acc] - end, [], Discards), - RtnIndexes = [I || ?MSG(I, _) <- lqueue:to_list(Returns)], - CheckedIdxs = maps:fold( - fun (_Cid, #consumer{checked_out = Ch}, Acc0) -> - maps:fold( - fun (_MsgId, ?MSG(I, _), Acc) -> - [I | Acc] - end, Acc0, Ch) - end, RtnIndexes ++ DlxIndexes, Consumers), - - CheckedIdxs ++ rabbit_fifo_q:indexes(Messages). + end, MsgsIdxs, Discards), + RtnIndexes = lqueue:fold(fun(?MSG(I, _), Acc) -> [I | Acc] end, + DlxIndexes, Returns), + maps:fold(fun (_Cid, #consumer{checked_out = Ch}, Acc0) -> + maps:fold( + fun (_MsgId, ?MSG(I, _), Acc) -> + [I | Acc] + end, Acc0, Ch) + end, RtnIndexes, Consumers). -spec snapshot_installed(Meta, State, OldMeta, OldState) -> ra_machine:effects() when @@ -676,7 +676,7 @@ live_indexes(#?STATE{cfg = #cfg{}, State :: state(), OldMeta :: ra_snapshot:meta(), OldState :: state(). -snapshot_installed(_Meta, #?MODULE{cfg = #cfg{resource = _QR}, +snapshot_installed(_Meta, #?MODULE{cfg = #cfg{}, consumers = Consumers} = State, _OldMeta, _OldState) -> %% here we need to redliver all pending consumer messages diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index 6325809d597e..27cba0ac03b9 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -1612,7 +1612,7 @@ policy_repair(Config) -> consume_all(Ch, QQ), % Ensure the queue process is unavailable - lists:foreach(fun(Srv) -> ensure_qq_proc_dead(Config, Srv, RaName) end, Servers), + [ok = ra:stop_server(quorum_queues, {RaName, Srv}) || Srv <- Servers], % Add policy with higher priority, allowing even more messages. ExpectedMaxLength3 = 30, @@ -1633,24 +1633,7 @@ policy_repair(Config) -> ]), % Restart the queue process. - {ok, Queue} = - rabbit_ct_broker_helpers:rpc( - Config, - 0, - rabbit_amqqueue, - lookup, - [{resource, <<"/">>, queue, QQ}]), - lists:foreach( - fun(Srv) -> - rabbit_ct_broker_helpers:rpc( - Config, - Srv, - rabbit_quorum_queue, - recover, - [foo, [Queue]] - ) - end, - Servers), + [ok = ra:restart_server(quorum_queues, {RaName, Srv}) || Srv <- Servers], % Wait for the queue to be available again. lists:foreach(fun(Srv) -> diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index e866f7a79ffe..36dc3f2c8b1f 100644 --- a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl @@ -1137,7 +1137,7 @@ is_same_otp_version(ConfigOrNode) -> OurOTP =:= OtherOTP. two_nodes(Node) -> - Size = 500, + Size = 300, run_proper( fun () -> ?FORALL({Length, Bytes, DeliveryLimit, SingleActive}, From c27dbec84735d22ae782e929e5ec2e19dc169b4f Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 7 Oct 2025 16:30:04 +0100 Subject: [PATCH 10/28] QQ: optimise memory for message references. Currently a message references' smallest representation is an improper list with two integer terms; the Raft index and the message size in bytes. This has a heap overhead of 2 words + 2 words for the cons cell in the queue data structure. This commit introduces a new more compact form that is used if the Raft index is less than 44 bits and the message size is less than 32KiB. In this case the Raft index and message size is packed into a 59 bit integer which is the largest unsigned integer that is still an immediate term and thus has 0 heap overhead as the integer value fits into the pointer word completely. This reduces per message overhead to the 2 words for the cons cell and thus halves message memory use for a lot of scenarios. If a message cannot be packed it will fall back to the current list based representation. --- deps/rabbit/src/rabbit_fifo.erl | 216 +++++++++++------- deps/rabbit/src/rabbit_fifo.hrl | 25 +- deps/rabbit/src/rabbit_fifo_dlx.erl | 4 +- deps/rabbit/src/rabbit_fifo_q.erl | 34 ++- deps/rabbit/test/rabbit_fifo_SUITE.erl | 4 +- .../rabbit_fifo_dlx_integration_SUITE.erl | 6 +- 6 files changed, 188 insertions(+), 101 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 266c4a13a1ec..2868d1d72989 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -77,7 +77,7 @@ get_msg_header/1, get_header/2, annotate_msg/2, - get_msg/1, + get_msg_from_cmd/1, %% protocol helpers make_enqueue/3, @@ -386,8 +386,9 @@ apply_(#{index := Index, {once, {simple_prefetch, 1}}, 0, State0), case checkout_one(Meta, false, State1, []) of - {success, _, MsgId, - ?MSG(RaftIdx, Header), _ExpiredMsg, State2, Effects0} -> + {success, _, MsgId, Msg, _ExpiredMsg, State2, Effects0} -> + RaftIdx = get_msg_idx(Msg), + Header = get_msg_header(Msg), {State4, Effects1} = case Settlement of unsettled -> @@ -658,15 +659,16 @@ live_indexes(#?STATE{cfg = #cfg{}, consumers = Consumers, dlx = #?DLX{discards = Discards}}) -> MsgsIdxs = rabbit_fifo_q:indexes(Messages), - DlxIndexes = lqueue:fold(fun (?TUPLE(_, ?MSG(I, _)), Acc) -> + DlxIndexes = lqueue:fold(fun (?TUPLE(_, Msg), Acc) -> + I = get_msg_idx(Msg), [I | Acc] end, MsgsIdxs, Discards), - RtnIndexes = lqueue:fold(fun(?MSG(I, _), Acc) -> [I | Acc] end, + RtnIndexes = lqueue:fold(fun(Msg, Acc) -> [get_msg_idx(Msg) | Acc] end, DlxIndexes, Returns), maps:fold(fun (_Cid, #consumer{checked_out = Ch}, Acc0) -> maps:fold( - fun (_MsgId, ?MSG(I, _), Acc) -> - [I | Acc] + fun (_MsgId, Msg, Acc) -> + [get_msg_idx(Msg) | Acc] end, Acc0, Ch) end, RtnIndexes, Consumers). @@ -901,7 +903,9 @@ get_checked_out(CKey, From, To, #?STATE{consumers = Consumers}) -> case find_consumer(CKey, Consumers) of {_CKey, #consumer{checked_out = Checked}} -> [begin - ?MSG(I, H) = maps:get(K, Checked), + Msg = maps:get(K, Checked), + I = get_msg_idx(Msg), + H = get_msg_header(Msg), {K, {I, H}} end || K <- lists:seq(From, To), maps:is_key(K, Checked)]; _ -> @@ -1025,12 +1029,14 @@ handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, {ConsumerKey, #consumer{checked_out = Checked}} -> {RaAux, ToReturn} = maps:fold( - fun (MsgId, ?MSG(Idx, Header), {RA0, Acc}) -> + fun (MsgId, Msg, {RA0, Acc}) -> + Idx = get_msg_idx(Msg), + Header = get_msg_header(Msg), %% it is possible this is not found if the consumer %% crashed and the message got removed case ra_aux:log_fetch(Idx, RA0) of {{_Term, _Meta, Cmd}, RA} -> - Msg = get_msg(Cmd), + Msg = get_msg_from_cmd(Cmd), {RA, [{MsgId, Idx, Header, Msg} | Acc]}; {undefined, RA} -> {RA, Acc} @@ -1070,13 +1076,14 @@ handle_aux(_, _, {get_checked_out, ConsumerKey, MsgIds}, Aux0, RaAux0) -> #{ConsumerKey := #consumer{checked_out = Checked}} -> {RaState, IdMsgs} = maps:fold( - fun (MsgId, ?MSG(Idx, Header), {S0, Acc}) -> + fun (MsgId, Msg, {S0, Acc}) -> + Idx = get_msg_idx(Msg), + Header = get_msg_idx(Msg), %% it is possible this is not found if the consumer %% crashed and the message got removed case ra_aux:log_fetch(Idx, S0) of {{_Term, _Meta, Cmd}, S} -> - Msg = get_msg(Cmd), - {S, [{MsgId, {Header, Msg}} | Acc]}; + {S, [{MsgId, {Header, get_msg_from_cmd(Cmd)}} | Acc]}; {undefined, S} -> {S, Acc} end @@ -1121,11 +1128,13 @@ handle_aux(_RaState, {call, _From}, {peek, Pos}, Aux0, RaAux0) -> MacState = ra_aux:machine_state(RaAux0), case query_peek(Pos, MacState) of - {ok, ?MSG(Idx, Header)} -> + {ok, Msg} -> + Idx = get_msg_idx(Msg), + Header = get_msg_header(Msg), %% need to re-hydrate from the log {{_, _, Cmd}, RaAux} = ra_aux:log_fetch(Idx, RaAux0), - Msg = get_msg(Cmd), - {reply, {ok, {Header, Msg}}, Aux0, RaAux}; + ActualMsg = get_msg_from_cmd(Cmd), + {reply, {ok, {Header, ActualMsg}}, Aux0, RaAux}; Err -> {reply, Err, Aux0, RaAux0} end; @@ -1320,11 +1329,16 @@ query_peek(Pos, State0) when Pos > 0 -> {error, no_message_at_pos}; {Msg, _State} when Pos == 1 -> - {ok, Msg}; + {ok, unpack(Msg)}; {_Msg, State} -> query_peek(Pos-1, State) end. +unpack(Packed) when ?IS_PACKED(Packed) -> + ?MSG(?PACKED_IDX(Packed), ?PACKED_SZ(Packed)); +unpack(Msg) -> + Msg. + query_notify_decorators_info(#?STATE{consumers = Consumers} = State) -> MaxActivePriority = maps:fold( fun(_, #consumer{credit = C, @@ -1568,7 +1582,8 @@ decr_total(#?STATE{messages_total = Tot} = State) -> drop_head(#?STATE{discarded_bytes = DiscardedBytes0} = State0, Effects) -> case take_next_msg(State0) of - {?MSG(_Idx, Header) = Msg, State1} -> + {Msg, State1} -> + Header = get_msg_header(Msg), State = decr_total(add_bytes_drop(Header, State1)), #?STATE{cfg = #cfg{dead_letter_handler = DLH}, dlx = DlxState} = State, @@ -1638,6 +1653,13 @@ update_expiry_header(RaCmdTs, TTL, Header) -> update_expiry_header(ExpiryTs, Header) -> update_header(expiry, fun(Ts) -> Ts end, ExpiryTs, Header). +make_msg(Idx, Sz) + when Idx =< ?PACKED_IDX_MAX andalso + (is_integer(Sz) andalso Sz =< ?PACKED_SZ_MAX) -> + ?PACK(Idx, Sz); +make_msg(Idx, Hdr) -> + ?MSG(Idx, Hdr). + maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, {MetaSize, BodySize}, Effects, #?STATE{msg_bytes_enqueue = Enqueue, @@ -1647,7 +1669,7 @@ maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, Size = MetaSize + BodySize, Header0 = maybe_set_msg_ttl(RawMsg, Ts, Size, State0), Header = maybe_set_msg_delivery_count(RawMsg, Header0), - Msg = ?MSG(RaftIdx, Header), + Msg = make_msg(RaftIdx, Header), PTag = priority_tag(RawMsg), State = State0#?STATE{msg_bytes_enqueue = Enqueue + Size, messages_total = Total + 1, @@ -1673,7 +1695,7 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, % it is the next expected seqno Header0 = maybe_set_msg_ttl(RawMsg, Ts, Size, State0), Header = maybe_set_msg_delivery_count(RawMsg, Header0), - Msg = ?MSG(RaftIdx, Header), + Msg = make_msg(RaftIdx, Header), Enq = Enq0#enqueuer{next_seqno = MsgSeqNo + 1}, MsgCache = case can_immediately_deliver(State0) of true -> @@ -1734,7 +1756,8 @@ complete(Meta, ConsumerKey, [MsgId], messages_total = Tot} = State0, Effects) -> case maps:take(MsgId, Checked0) of - {?MSG(_Idx, Hdr), Checked} -> + {Msg, Checked} -> + Hdr = get_msg_header(Msg), SettledSize = get_header(size, Hdr), Con = Con0#consumer{checked_out = Checked, credit = increase_credit(Con0, 1)}, @@ -1755,7 +1778,8 @@ complete(Meta, ConsumerKey, MsgIds, = lists:foldl( fun (MsgId, {S0, Ch0}) -> case maps:take(MsgId, Ch0) of - {?MSG(_Idx, Hdr), Ch} -> + {Msg, Ch} -> + Hdr = get_msg_header(Msg), S = get_header(size, Hdr) + S0, {S, Ch}; error -> @@ -1822,8 +1846,9 @@ cancel_consumer_effects(ConsumerId, [{mod_call, rabbit_quorum_queue, cancel_consumer_handler, [QName, ConsumerId]} | Effects]. -update_msg_header(Key, Fun, Def, ?MSG(Idx, Header)) -> - ?MSG(Idx, update_header(Key, Fun, Def, Header)). +update_msg_header(Key, Fun, Def, Msg) -> + ?MSG(get_msg_idx(Msg), + update_header(Key, Fun, Def, get_msg_header(Msg))). update_header(expiry, _, Expiry, Size) when is_integer(Size) -> @@ -1840,8 +1865,15 @@ update_header(Key, UpdateFun, Default, Header) when is_map_key(size, Header) -> maps:update_with(Key, UpdateFun, Default, Header). +get_msg_idx(?MSG(Idx, _Header)) -> + Idx; +get_msg_idx(Packed) when ?IS_PACKED(Packed) -> + ?PACKED_IDX(Packed). + get_msg_header(?MSG(_Idx, Header)) -> - Header. + Header; +get_msg_header(Packed) when ?IS_PACKED(Packed) -> + ?PACKED_SZ(Packed). get_header(size, Size) when is_integer(Size) -> @@ -1878,7 +1910,7 @@ annotate_msg(Header, Msg0) -> Msg0 end. -return_one(Meta, MsgId, ?MSG(_, _) = Msg0, DelivFailed, Anns, +return_one(Meta, MsgId, Msg0, DelivFailed, Anns, #?STATE{returns = Returns, consumers = Consumers, dlx = DlxState0, @@ -1941,7 +1973,7 @@ checkout(#{index := Index} = Meta, {State, Reply, Effects}. checkout0(Meta, {success, ConsumerKey, MsgId, - ?MSG(_, _) = Msg, ExpiredMsg, State, Effects}, + Msg, ExpiredMsg, State, Effects}, SendAcc0) -> DelMsg = {MsgId, Msg}, SendAcc = case maps:get(ConsumerKey, SendAcc0, undefined) of @@ -2032,15 +2064,15 @@ chunk_disk_msgs([], _Bytes, [[] | Chunks]) -> Chunks; chunk_disk_msgs([], _Bytes, Chunks) -> Chunks; -chunk_disk_msgs([{_MsgId, ?MSG(_RaftIdx, Header)} = Msg | Rem], +chunk_disk_msgs([{_MsgId, Msg} = ConsumerMsg | Rem], Bytes, Chunks) when Bytes >= ?DELIVERY_CHUNK_LIMIT_B -> - Size = get_header(size, Header), - chunk_disk_msgs(Rem, Size, [[Msg] | Chunks]); -chunk_disk_msgs([{_MsgId, ?MSG(_RaftIdx, Header)} = Msg | Rem], Bytes, + Size = get_header(size, get_msg_header(Msg)), + chunk_disk_msgs(Rem, Size, [[ConsumerMsg] | Chunks]); +chunk_disk_msgs([{_MsgId, Msg} = ConsumerMsg | Rem], Bytes, [CurChunk | Chunks]) -> - Size = get_header(size, Header), - chunk_disk_msgs(Rem, Bytes + Size, [[Msg | CurChunk] | Chunks]). + Size = get_header(size, get_msg_header(Msg)), + chunk_disk_msgs(Rem, Bytes + Size, [[ConsumerMsg | CurChunk] | Chunks]). add_delivery_effects(Effects0, AccMap, _State) when map_size(AccMap) == 0 -> @@ -2064,7 +2096,7 @@ take_next_msg(#?STATE{returns = Returns0, case rabbit_fifo_q:out(Messages0) of empty -> empty; - {?MSG(_RaftIdx, _) = Msg, Messages} -> + {Msg, Messages} -> {Msg, State#?STATE{messages = Messages}} end end. @@ -2083,10 +2115,18 @@ delivery_effect(ConsumerKey, [{MsgId, ?MSG(Idx, Header)}], {CTag, CPid} = consumer_id(ConsumerKey, State), {send_msg, CPid, {delivery, CTag, [{MsgId, {Header, RawMsg}}]}, ?DELIVERY_SEND_MSG_OPTS}; +delivery_effect(ConsumerKey, [{MsgId, Msg}], + #?STATE{msg_cache = {Idx, RawMsg}} = State) + when is_integer(Msg) andalso ?PACKED_IDX(Msg) == Idx -> + Header = get_msg_header(Msg), + {CTag, CPid} = consumer_id(ConsumerKey, State), + {send_msg, CPid, {delivery, CTag, [{MsgId, {Header, RawMsg}}]}, + ?DELIVERY_SEND_MSG_OPTS}; delivery_effect(ConsumerKey, Msgs, #?STATE{} = State) -> {CTag, CPid} = consumer_id(ConsumerKey, State), - {RaftIdxs, _Num} = lists:foldr(fun ({_, ?MSG(I, _)}, {Acc, N}) -> - {[I | Acc], N+1} + {RaftIdxs, _Num} = lists:foldr(fun ({_, Msg}, {Acc, N}) -> + + {[get_msg_idx(Msg) | Acc], N+1} end, {[], 0}, Msgs), {log_ext, RaftIdxs, fun (ReadPlan) -> @@ -2113,7 +2153,7 @@ reply_log_effect(RaftIdx, MsgId, Header, Ready, From) -> []; ([Cmd]) -> [{reply, From, {wrap_reply, - {dequeue, {MsgId, {Header, get_msg(Cmd)}}, Ready}}}] + {dequeue, {MsgId, {Header, get_msg_from_cmd(Cmd)}}, Ready}}}] end}. checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> @@ -2199,11 +2239,12 @@ expire_msgs(RaCmdTs, Result, State, Effects) -> when is_integer(Expiry), RaCmdTs >= Expiry -> expire(RaCmdTs, State, Effects); _ -> + %% packed messages never have an expiry {Result, State, Effects} end. expire(RaCmdTs, State0, Effects) -> - {?MSG(_Idx, Header) = Msg, + {Msg, #?STATE{cfg = #cfg{dead_letter_handler = DLH}, dlx = DlxState0, messages_total = Tot, @@ -2212,12 +2253,13 @@ expire(RaCmdTs, State0, Effects) -> take_next_msg(State0), {DlxState, _RetainedBytes, DlxEffects} = discard_or_dead_letter([Msg], expired, DLH, DlxState0), - DiscardedSize = get_header(size, Header) + ?ENQ_OVERHEAD, + Header = get_msg_header(Msg), + Size = get_header(size, Header), + DiscardedSize = Size + ?ENQ_OVERHEAD, State = State1#?STATE{dlx = DlxState, messages_total = Tot - 1, discarded_bytes = DiscardedBytes0 + DiscardedSize, - msg_bytes_enqueue = - MsgBytesEnqueue - get_header(size, Header)}, + msg_bytes_enqueue = MsgBytesEnqueue - Size}, expire_msgs(RaCmdTs, true, State, DlxEffects ++ Effects). timer_effect(RaCmdTs, State, Effects) -> @@ -2759,8 +2801,8 @@ convert(Meta, 7, To, State) -> smallest_raft_index(#?STATE{messages = Messages, dlx = #?DLX{discards = Discards}} = State) -> - SmallestDlxRaIdx = lqueue:fold(fun (?TUPLE(_, ?MSG(I, _)), Acc) -> - min(I, Acc) + SmallestDlxRaIdx = lqueue:fold(fun (?TUPLE(_, Msg), Acc) -> + min(get_msg_idx(Msg), Acc) end, undefined, Discards), SmallestMsgsRaIdx = rabbit_fifo_q:get_lowest_index(Messages), %% scan consumers and returns queue here instead @@ -2768,13 +2810,13 @@ smallest_raft_index(#?STATE{messages = Messages, smallest_checked_out(#?STATE{returns = Returns, consumers = Consumers}, Min) -> - SmallestSoFar = lqueue:fold(fun (?MSG(I, _), Acc) -> - min(I, Acc) + SmallestSoFar = lqueue:fold(fun (Msg, Acc) -> + min(get_msg_idx(Msg), Acc) end, Min, Returns), maps:fold(fun (_Cid, #consumer{checked_out = Ch}, Acc0) -> maps:fold( - fun (_MsgId, ?MSG(I, _), Acc) -> - min(I, Acc) + fun (_MsgId, Msg, Acc) -> + min(get_msg_idx(Msg), Acc) end, Acc0, Ch) end, SmallestSoFar, Consumers). @@ -2814,11 +2856,11 @@ can_immediately_deliver(#?STATE{service_queue = SQ, incr(I) -> I + 1. -get_msg(#?ENQ_V2{msg = M}) -> +get_msg_from_cmd(#?ENQ_V2{msg = M}) -> M; -get_msg(#enqueue{msg = M}) -> +get_msg_from_cmd(#enqueue{msg = M}) -> M; -get_msg(#requeue{msg = M}) -> +get_msg_from_cmd(#requeue{msg = M}) -> M. initial_delivery_count({credited, Count}) -> @@ -3017,9 +3059,11 @@ exec_read(Flru0, ReadPlan, Msgs) -> try ra_log_read_plan:execute(ReadPlan, Flru0) of {Entries, Flru} -> %% return a list in original order - {lists:map(fun ({MsgId, ?MSG(Idx, Header)}) -> + {lists:map(fun ({MsgId, Msg}) -> + Idx = get_msg_idx(Msg), + Header = get_msg_header(Msg), Cmd = maps:get(Idx, Entries), - {MsgId, {Header, get_msg(Cmd)}} + {MsgId, {Header, get_msg_from_cmd(Cmd)}} end, Msgs), Flru} catch exit:{missing_key, _} when Flru0 =/= undefined -> @@ -3075,19 +3119,21 @@ dlx_apply(_Meta, {dlx, {settle, MsgIds}}, at_least_once, #?DLX{consumer = #dlx_consumer{checked_out = Checked0}} = State0) -> Acked = maps:with(MsgIds, Checked0), {DBytes, State} = - maps:fold(fun(MsgId, ?TUPLE(_Rsn, ?MSG(Idx, Hdr)), - {Sz, - #?DLX{consumer = #dlx_consumer{checked_out = Checked} = C, - msg_bytes_checkout = BytesCheckout, - ra_indexes = Indexes0} = S}) -> - Indexes = rabbit_fifo_index:delete(Idx, Indexes0), - Size = get_header(size, Hdr), - {Sz + Size + ?ENQ_OVERHEAD, - S#?DLX{consumer = C#dlx_consumer{checked_out = - maps:remove(MsgId, Checked)}, - msg_bytes_checkout = BytesCheckout - Size, - ra_indexes = Indexes}} - end, {0, State0}, Acked), + maps:fold( + fun(MsgId, ?TUPLE(_Rsn, Msg), + {Sz, #?DLX{consumer = #dlx_consumer{checked_out = Checked} = C, + msg_bytes_checkout = BytesCheckout, + ra_indexes = Indexes0} = S}) -> + Idx = get_msg_idx(Msg), + Hdr = get_msg_header(Msg), + Indexes = rabbit_fifo_index:delete(Idx, Indexes0), + Size = get_header(size, Hdr), + {Sz + Size + ?ENQ_OVERHEAD, + S#?DLX{consumer = C#dlx_consumer{checked_out = + maps:remove(MsgId, Checked)}, + msg_bytes_checkout = BytesCheckout - Size, + ra_indexes = Indexes}} + end, {0, State0}, Acked), {State, DBytes, [{mod_call, rabbit_global_counters, messages_dead_lettered_confirmed, [rabbit_quorum_queue, at_least_once, maps:size(Acked)]}]}; @@ -3289,18 +3335,20 @@ discard_or_dead_letter(Msgs, Reason, undefined, State) -> [{mod_call, rabbit_global_counters, messages_dead_lettered, [Reason, rabbit_quorum_queue, disabled, length(Msgs)]}]}; discard_or_dead_letter(Msgs0, Reason, {at_most_once, {Mod, Fun, Args}}, State) -> - Idxs = [I || ?MSG(I, _) <- Msgs0], + Idxs = lists:map(fun get_msg_idx/1, Msgs0), %% TODO: this could be turned into a log_ext effect instead to avoid %% reading from disk inside the qq process Effect = {log, Idxs, fun (Log) -> Lookup = maps:from_list(lists:zip(Idxs, Log)), Msgs = [begin + Idx = get_msg_idx(Msg), + Hdr = get_msg_header(Msg), Cmd = maps:get(Idx, Lookup), %% ensure header delivery count %% is copied to the message container - annotate_msg(H, rabbit_fifo:get_msg(Cmd)) - end || ?MSG(Idx, H) <- Msgs0], + annotate_msg(Hdr, rabbit_fifo:get_msg_from_cmd(Cmd)) + end || Msg <- Msgs0], [{mod_call, Mod, Fun, Args ++ [Reason, Msgs]}] end}, {State, 0, [Effect]}; @@ -3309,13 +3357,13 @@ discard_or_dead_letter(Msgs, Reason, at_least_once, State0) RetainedBytes = lists:foldl(fun (M, Acc) -> Acc + size_in_bytes(M) + ?ENQ_OVERHEAD end, 0, Msgs), - State = lists:foldl(fun(?MSG(Idx, _) = Msg, - #?DLX{discards = D0, - msg_bytes = B0, - ra_indexes = I0} = S0) -> + State = lists:foldl(fun(Msg, #?DLX{discards = D0, + msg_bytes = B0, + ra_indexes = I0} = S0) -> MsgSize = size_in_bytes(Msg), D = lqueue:in(?TUPLE(Reason, Msg), D0), B = B0 + MsgSize, + Idx = get_msg_idx(Msg), I = rabbit_fifo_index:append(Idx, I0), S0#?DLX{discards = D, msg_bytes = B, @@ -3335,8 +3383,10 @@ dlx_checkout(at_least_once, #?DLX{consumer = #dlx_consumer{}} = State) -> dlx_checkout(_, State) -> {State, []}. -dlx_checkout0({success, MsgId, ?TUPLE(Reason, ?MSG(Idx, H)), State}, SendAcc) -> - DelMsg = {Idx, {Reason, H, MsgId}}, +dlx_checkout0({success, MsgId, ?TUPLE(Reason, Msg), State}, SendAcc) -> + Idx = get_msg_idx(Msg), + Hdr = get_msg_header(Msg), + DelMsg = {Idx, {Reason, Hdr, MsgId}}, dlx_checkout0(dlx_checkout_one(State), [DelMsg | SendAcc]); dlx_checkout0(#?DLX{consumer = #dlx_consumer{pid = Pid}} = State, SendAcc) -> Effects = dlx_delivery_effects(Pid, SendAcc), @@ -3347,19 +3397,19 @@ dlx_checkout_one(#?DLX{consumer = #dlx_consumer{checked_out = Checked, when map_size(Checked) >= Prefetch -> State; dlx_checkout_one(#?DLX{discards = Discards0, - msg_bytes = Bytes, - msg_bytes_checkout = BytesCheckout, - consumer = #dlx_consumer{checked_out = Checked0, - next_msg_id = Next} = Con0} = State0) -> + msg_bytes = Bytes, + msg_bytes_checkout = BytesCheckout, + consumer = #dlx_consumer{checked_out = Checked0, + next_msg_id = Next} = Con0} = State0) -> case lqueue:out(Discards0) of {{value, ?TUPLE(_, Msg) = ReasonMsg}, Discards} -> Checked = maps:put(Next, ReasonMsg, Checked0), Size = size_in_bytes(Msg), State = State0#?DLX{discards = Discards, - msg_bytes = Bytes - Size, - msg_bytes_checkout = BytesCheckout + Size, - consumer = Con0#dlx_consumer{checked_out = Checked, - next_msg_id = Next + 1}}, + msg_bytes = Bytes - Size, + msg_bytes_checkout = BytesCheckout + Size, + consumer = Con0#dlx_consumer{checked_out = Checked, + next_msg_id = Next + 1}}, {success, Next, ReasonMsg, State}; {empty, _} -> State0 @@ -3375,7 +3425,7 @@ dlx_delivery_effects(CPid, Msgs0) -> Msgs = lists:zipwith( fun (Cmd, {Reason, H, MsgId}) -> {MsgId, {Reason, - annotate_msg(H, rabbit_fifo:get_msg(Cmd))}} + annotate_msg(H, rabbit_fifo:get_msg_from_cmd(Cmd))}} end, Log, RsnIds), [{send_msg, CPid, {dlx_event, self(), {dlx_delivery, Msgs}}, [cast]}] end}]. diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index 839bbf15aade..2da17bf2295e 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -21,6 +21,26 @@ -define(DELIVERY_SEND_MSG_OPTS, [local, ra_event]). +%% constants for packed msg references where both the raft index and the size +%% is packed into a single immidate term +%% +%% 59 bytes as immedate ints are signed +-define(PACKED_MAX, 16#7FFF_FFFF_FFFF_FFF). +%% index bits - enough for 2000 days at 100k indexes p/sec +-define(PACKED_IDX_BITS, 44). +-define(PACKED_IDX_MAX, 16#FFFF_FFFF_FFF). +-define(PACKED_SZ_BITS, 15). %% size +-define(PACKED_SZ_MAX, 16#7FFF). %% 15 bits + +-define(PACK(Idx, Sz), + (Idx bxor (Sz bsl ?PACKED_IDX_BITS))). +-define(PACKED_IDX(PackedInt), + (PackedInt band ?PACKED_IDX_MAX)). +-define(PACKED_SZ(PackedInt), + ((PackedInt bsr 44) band 16#7FFF)). + +-define(IS_PACKED(Int), (Int >= 0 andalso Int =< ?PACKED_MAX)). + -type optimised_tuple(A, B) :: nonempty_improper_list(A, B). -type option(T) :: undefined | T. @@ -57,7 +77,10 @@ -type msg_size() :: non_neg_integer(). %% the size in bytes of the msg payload --type msg() :: optimised_tuple(ra:index(), msg_header()). +%% 60 byte integer, immediate +-type packed_msg() :: 0..?PACKED_MAX. + +-type msg() :: packed_msg() | optimised_tuple(ra:index(), msg_header()). -type delivery_msg() :: {msg_id(), {msg_header(), raw_msg()}}. %% A tuple consisting of the message id, and the headered message. diff --git a/deps/rabbit/src/rabbit_fifo_dlx.erl b/deps/rabbit/src/rabbit_fifo_dlx.erl index e0f62075a94f..562fc43f7db4 100644 --- a/deps/rabbit/src/rabbit_fifo_dlx.erl +++ b/deps/rabbit/src/rabbit_fifo_dlx.erl @@ -165,7 +165,7 @@ discard(Msgs0, Reason, {at_most_once, {Mod, Fun, Args}}, State) -> Cmd = maps:get(Idx, Lookup), %% ensure header delivery count %% is copied to the message container - annotate_msg(H, rabbit_fifo:get_msg(Cmd)) + annotate_msg(H, rabbit_fifo:get_msg_from_cmd(Cmd)) end || ?MSG(Idx, H) <- Msgs0], [{mod_call, Mod, Fun, Args ++ [Reason, Msgs]}] end}, @@ -238,7 +238,7 @@ delivery_effects(CPid, Msgs0) -> Msgs = lists:zipwith( fun (Cmd, {Reason, H, MsgId}) -> {MsgId, {Reason, - annotate_msg(H, rabbit_fifo:get_msg(Cmd))}} + annotate_msg(H, rabbit_fifo:get_msg_from_cmd(Cmd))}} end, Log, RsnIds), [{send_msg, CPid, {dlx_event, self(), {dlx_delivery, Msgs}}, [cast]}] end}]. diff --git a/deps/rabbit/src/rabbit_fifo_q.erl b/deps/rabbit/src/rabbit_fifo_q.erl index 5f9d2e194d40..9bca67135154 100644 --- a/deps/rabbit/src/rabbit_fifo_q.erl +++ b/deps/rabbit/src/rabbit_fifo_q.erl @@ -85,10 +85,10 @@ from_lqueue(LQ) -> -spec indexes(state()) -> [ra:index()]. indexes(#?MODULE{hi = {Hi1, Hi2}, no = {No1, No2}}) -> - A = lists:map(fun (?MSG(I, _)) -> I end, Hi1), - B = lists:foldl(fun (?MSG(I, _), Acc) -> [I | Acc] end, A, Hi2), - C = lists:foldl(fun (?MSG(I, _), Acc) -> [I | Acc] end, B, No1), - lists:foldl(fun (?MSG(I, _), Acc) -> [I | Acc] end, C, No2). + A = lists:map(fun msg_idx/1, Hi1), + B = lists:foldl(fun msg_idx_fld/2, A, Hi2), + C = lists:foldl(fun msg_idx_fld/2, B, No1), + lists:foldl(fun msg_idx_fld/2, C, No2). -spec get_lowest_index(state()) -> undefined | ra:index(). get_lowest_index(#?MODULE{len = 0}) -> @@ -96,14 +96,14 @@ get_lowest_index(#?MODULE{len = 0}) -> get_lowest_index(#?MODULE{hi = Hi, no = No}) -> case peek(Hi) of empty -> - ?MSG(NoIdx, _) = peek(No), - NoIdx; - ?MSG(HiIdx, _) -> + msg_idx(peek(No)); + HiMsg -> + HiIdx = msg_idx(HiMsg), case peek(No) of - ?MSG(NoIdx, _) -> - min(HiIdx, NoIdx); empty -> - HiIdx + HiIdx; + NoMsg -> + min(HiIdx, msg_idx(NoMsg)) end end. @@ -128,8 +128,10 @@ overview(#?MODULE{len = Len, next(#?MODULE{hi = ?NON_EMPTY = Hi, no = ?NON_EMPTY = No, dequeue_counter = ?WEIGHT}) -> - ?MSG(HiIdx, _) = HiMsg = peek(Hi), - ?MSG(NoIdx, _) = NoMsg = peek(No), + HiMsg = peek(Hi), + NoMsg = peek(No), + HiIdx = msg_idx(HiMsg), + NoIdx = msg_idx(NoMsg), %% always favour hi priority messages when it is safe to do so, %% i.e. the index is lower than the next index for the 'no' queue case HiIdx < NoIdx of @@ -159,3 +161,11 @@ drop({In, [_]}) -> {[], lists:reverse(In)}; drop({In, [_ | Out]}) -> {In, Out}. + +msg_idx_fld(Msg, Acc) when is_list(Acc) -> + [msg_idx(Msg) | Acc]. + +msg_idx(?MSG(Idx, _Header)) -> + Idx; +msg_idx(Packed) when ?IS_PACKED(Packed) -> + ?PACKED_IDX(Packed). diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index f0112f1d8233..5c0b78502071 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -2881,10 +2881,12 @@ queue_ttl_with_single_active_consumer_test(Config) -> query_peek_test(Config) -> State0 = test_init(test), ?assertEqual({error, no_message_at_pos}, rabbit_fifo:query_peek(1, State0)), + {State1, _} = enq(Config, 1, 1, first, State0), - {State2, _} = enq(Config, 2, 2, second, State1), ?assertMatch({ok, [1 | _]}, rabbit_fifo:query_peek(1, State1)), ?assertEqual({error, no_message_at_pos}, rabbit_fifo:query_peek(2, State1)), + + {State2, _} = enq(Config, 2, 2, second, State1), ?assertMatch({ok, [1 | _]}, rabbit_fifo:query_peek(1, State2)), ?assertMatch({ok, [2 | _]}, rabbit_fifo:query_peek(2, State2)), ?assertEqual({error, no_message_at_pos}, rabbit_fifo:query_peek(3, State2)), diff --git a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl index 1d8297a452ca..ce8e242fd794 100644 --- a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl @@ -806,10 +806,12 @@ target_quorum_queue_delete_create(Config) -> %% Expect no message to get stuck in dlx worker. wait_for_min_messages(Config, TargetQ, 200), eventually(?_assertMatch([{0, _}], - dirty_query([Server], ra_name(SourceQ), fun rabbit_fifo:query_stat_dlx/1)), 500, 10), + dirty_query([Server], ra_name(SourceQ), + fun rabbit_fifo:query_stat_dlx/1)), 500, 10), ?assertEqual(300, counted(messages_dead_lettered_expired_total, Config)), ?assertEqual(300, counted(messages_dead_lettered_confirmed_total, Config)), - #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = TargetQ}). + #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = TargetQ}), + ok. %% Test that %% 1. Message is only acked to source queue once publisher confirms got received from **all** target queues. From ee9043ba2ed9dccf74bd277562dde568910e6970 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 10 Oct 2025 12:25:04 +0100 Subject: [PATCH 11/28] QQ: fix effects ordering Not really a problem but effects passed to checkout/4 would be reversed which could potentially cause future confusion. --- deps/rabbit/src/rabbit_fifo.erl | 40 ++++++++++++++++---------- deps/rabbit/test/rabbit_fifo_SUITE.erl | 22 ++++++-------- 2 files changed, 34 insertions(+), 28 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 2868d1d72989..2a4336409c4d 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -272,7 +272,7 @@ apply_(_Meta, #register_enqueuer{pid = Pid}, end, {State, Res, [{monitor, process, Pid}]}; apply_(Meta, #settle{msg_ids = MsgIds, - consumer_key = Key}, + consumer_key = Key}, #?STATE{consumers = Consumers} = State) -> case find_consumer(Key, Consumers) of {ConsumerKey, Con0} -> @@ -284,7 +284,7 @@ apply_(Meta, #settle{msg_ids = MsgIds, {State, ok} end; apply_(Meta, #discard{consumer_key = ConsumerKey, - msg_ids = MsgIds}, + msg_ids = MsgIds}, #?STATE{consumers = Consumers } = State0) -> case find_consumer(ConsumerKey, Consumers) of {ActualConsumerKey, #consumer{} = Con} -> @@ -293,8 +293,8 @@ apply_(Meta, #discard{consumer_key = ConsumerKey, {State0, ok} end; apply_(Meta, #return{consumer_key = ConsumerKey, - msg_ids = MsgIds}, - #?STATE{consumers = Cons} = State) -> + msg_ids = MsgIds}, + #?STATE{consumers = Cons} = State) -> case find_consumer(ConsumerKey, Cons) of {ActualConsumerKey, #consumer{checked_out = Checked}} -> return(Meta, ActualConsumerKey, MsgIds, false, @@ -303,11 +303,11 @@ apply_(Meta, #return{consumer_key = ConsumerKey, {State, ok} end; apply_(Meta, #modify{consumer_key = ConsumerKey, - delivery_failed = DelFailed, - undeliverable_here = Undel, - annotations = Anns, - msg_ids = MsgIds}, - #?STATE{consumers = Cons} = State) -> + delivery_failed = DelFailed, + undeliverable_here = Undel, + annotations = Anns, + msg_ids = MsgIds}, + #?STATE{consumers = Cons} = State) -> case find_consumer(ConsumerKey, Cons) of {ActualConsumerKey, #consumer{checked_out = Checked}} when Undel == false -> @@ -1984,8 +1984,8 @@ checkout0(Meta, {success, ConsumerKey, MsgId, end, checkout0(Meta, checkout_one(Meta, ExpiredMsg, State, Effects), SendAcc); checkout0(_Meta, {_Activity, ExpiredMsg, State0, Effects0}, SendAcc) -> - Effects = add_delivery_effects(Effects0, SendAcc, State0), - {State0, ExpiredMsg, lists:reverse(Effects)}. + Effects = add_delivery_effects([], SendAcc, State0), + {State0, ExpiredMsg, Effects0 ++ lists:reverse(Effects)}. evaluate_limit(Idx, State1, State2, OuterEffects) -> case evaluate_limit0(Idx, State1, State2, []) of @@ -2987,14 +2987,24 @@ do_snapshot(MacVer, Ts, #snapshot{index = _ChIdx, RaAux, DiscardedBytes, Force) when is_integer(MacVer) andalso MacVer >= 8 -> LastAppliedIdx = ra_aux:last_applied(RaAux), - #?STATE{} = MacState = ra_aux:machine_state(RaAux), + #?STATE{consumers = Consumers, + enqueuers = Enqueuers} = MacState = ra_aux:machine_state(RaAux), TimeSince = Ts - SnapTime, MsgsTot = messages_total(MacState), %% if the approximate snapshot size * 2 can be reclaimed it is worth %% taking a snapshot - %% TODO: take number of enqueues and consumers into account - ApproxSnapSize = 4096 + (32 * MsgsTot), - EnoughDataRemoved = DiscardedBytes - LastDiscardedBytes > (ApproxSnapSize * 2), + %% take number of enqueues and consumers into account + %% message: 32 bytes + %% enqueuer: 96 bytes + %% consumer: 256 bytes + NumEnqueuers = map_size(Enqueuers), + NumConsumers = map_size(Consumers), + ApproxSnapSize = 4096 + + (MsgsTot * 32) + + (NumEnqueuers * 96) + + (NumConsumers * 256), + + EnoughDataRemoved = DiscardedBytes - LastDiscardedBytes > (ApproxSnapSize * 3), {CheckMinInterval, _CheckMinIndexes, _CheckMaxIndexes} = persistent_term:get(quorum_queue_checkpoint_config, diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 5c0b78502071..2680bdf89498 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -556,7 +556,6 @@ untracked_enq_deq_test(Config) -> State0), {_State2, _, Effs} = apply(meta(Config, 3), make_checkout(Cid, {dequeue, settled}, #{}), State1), - ct:pal("Effs ~p", [State1]), ?ASSERT_EFF({log, [1], _}, Effs), ok. @@ -792,7 +791,6 @@ requeue_test(Config) -> [_Monitor, {log_ext, [1], _Fun, _}]} = checkout(Config, ?LINE, Cid, 1, State0), [{MsgId, {H1, _}}] = rabbit_fifo:get_checked_out(CKey, MsgId, MsgId, State1), - ct:pal("query consumers ~p", [rabbit_fifo:query_consumers(State1)]), [{append, Requeue, _}] = rabbit_fifo:make_requeue(CKey, {notify, 1, self()}, [{MsgId, 1, H1, Msg1}], []), @@ -962,7 +960,6 @@ discarded_message_with_dead_letter_handler_emits_log_effect_test(Config) -> ?assertEqual(undefined, mc:get_annotation(acquired_count, McOut)), ?assertEqual(1, mc:get_annotation(delivery_count, McOut)), - ok. discard_after_cancel_test(Config) -> @@ -1911,18 +1908,18 @@ single_active_consumer_priority_test(Config) -> ], {#rabbit_fifo{ cfg = #cfg{resource = Resource}}, StateMachineEvents} = run_log(Config, S0, Entries, fun single_active_invariant/1), ModCalls = [ S || S = {mod_call, rabbit_quorum_queue, update_consumer_handler, _} <- StateMachineEvents ], - - %% C1 should be added as single_active + + %% C1 should be added as single_active assert_update_consumer_handler_state_transition(C1, Resource, true, single_active, lists:nth(1, ModCalls)), - %% C1 should transition to waiting because ... - assert_update_consumer_handler_state_transition(C1, Resource, false, waiting, lists:nth(2, ModCalls)), %% C2 should become single_active - assert_update_consumer_handler_state_transition(C2, Resource, true, single_active, lists:nth(3, ModCalls)), - %% C2 should transition as waiting because ... - assert_update_consumer_handler_state_transition(C2, Resource, false, waiting, lists:nth(4, ModCalls)), + assert_update_consumer_handler_state_transition(C2, Resource, true, single_active, lists:nth(2, ModCalls)), + %% C1 should transition to waiting + assert_update_consumer_handler_state_transition(C1, Resource, false, waiting, lists:nth(3, ModCalls)), %% C3 is added as single_active - assert_update_consumer_handler_state_transition(C3, Resource, true, single_active, lists:nth(5, ModCalls)), - + assert_update_consumer_handler_state_transition(C3, Resource, true, single_active, lists:nth(4, ModCalls)), + %% C2 should transition as waiting + assert_update_consumer_handler_state_transition(C2, Resource, false, waiting, lists:nth(5, ModCalls)), + ok. assert_update_consumer_handler_state_transition(ConsumerId, Resource, IsActive, UpdatedState, ModCall) -> @@ -2376,7 +2373,6 @@ reject_publish_purge_test(Config) -> rabbit_fifo:make_enqueue(Pid1, 2, two), State2), {State4, ok, Efx} = apply(meta(Config, 4, ?LINE, {notify, 2, Pid1}), rabbit_fifo:make_enqueue(Pid1, 3, three), State3), - % ct:pal("Efx ~tp", [Efx]), ?ASSERT_EFF({send_msg, P, {queue_status, reject_publish}, [ra_event]}, P == Pid1, Efx), {_State5, {purge, 3}, Efx1} = apply(meta(Config, 5), rabbit_fifo:make_purge(), State4), ?ASSERT_EFF({send_msg, P, {queue_status, go}, [ra_event]}, P == Pid1, Efx1), From 41cef464280da3c44a7b6d467214d42eb3048a3c Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 14 Oct 2025 12:27:05 +0100 Subject: [PATCH 12/28] minor refactoring --- deps/rabbit/src/rabbit_fifo.erl | 35 ++++++++++++++++++--------------- 1 file changed, 19 insertions(+), 16 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 2a4336409c4d..144a70630ff0 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -417,9 +417,8 @@ apply_(#{index := Index, {State, {dequeue, empty}, Effects} end end; -apply_(#{index := _Idx} = Meta, - #checkout{spec = Spec, - consumer_id = ConsumerId}, State0) +apply_(Meta, #checkout{spec = Spec, + consumer_id = ConsumerId}, State0) when Spec == cancel orelse Spec == remove -> case consumer_key_from_id(ConsumerId, State0) of @@ -463,18 +462,9 @@ apply_(#{index := Idx} = Meta, {Consumer, State1} = update_consumer(Meta, ConsumerKey, ConsumerId, ConsumerMeta, Spec, Priority, State0), {State2, Effs} = activate_next_consumer(State1, []), - #consumer{checked_out = Checked, - credit = Credit, - delivery_count = DeliveryCount, - next_msg_id = NextMsgId} = Consumer, - - %% reply with a consumer summary - Reply = {ok, #{next_msg_id => NextMsgId, - credit => Credit, - key => ConsumerKey, - delivery_count => DeliveryCount, - is_active => is_active(ConsumerKey, State2), - num_checked_out => map_size(Checked)}}, + + %% reply with a consumer infos + Reply = {ok, consumer_info(ConsumerKey, Consumer, State2)}, checkout(Meta, State0, State2, [{monitor, process, Pid} | Effs], Reply); apply_(#{index := Index}, #purge{}, #?STATE{messages_total = Total} = State0) -> @@ -685,7 +675,7 @@ snapshot_installed(_Meta, #?MODULE{cfg = #cfg{}, %% to local consumers %% TODO: with some additional state (raft indexes assigned to consumer) %% we could reduce the number of resends but it is questionable if this - %% complexity is worth the effort. rabbit_fifo_index will de-duplicate + %% complexity is worth the effort. rabbit_fifo_client will de-duplicate %% deliveries anyway SendAcc = maps:fold( fun (_ConsumerKey, #consumer{cfg = #consumer_cfg{tag = Tag, @@ -3439,3 +3429,16 @@ dlx_delivery_effects(CPid, Msgs0) -> end, Log, RsnIds), [{send_msg, CPid, {dlx_event, self(), {dlx_delivery, Msgs}}, [cast]}] end}]. + +consumer_info(ConsumerKey, + #consumer{checked_out = Checked, + credit = Credit, + delivery_count = DeliveryCount, + next_msg_id = NextMsgId}, + State) -> + #{next_msg_id => NextMsgId, + credit => Credit, + key => ConsumerKey, + delivery_count => DeliveryCount, + is_active => is_active(ConsumerKey, State), + num_checked_out => map_size(Checked)}. From c57c47aff3cb57e49886a3c10b63a0c96b09aee5 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 17 Oct 2025 09:31:48 +0100 Subject: [PATCH 13/28] QQ: allow unlimited returns. Base delivery-limit on the delivery-count _not_ the acquired_count. This allows messages to be explicitly returned to the queue an unlimited number of times. The delivery-limit will only be reached for returns triggered by the consuming process crashing with any reason other than noconnection. --- deps/rabbit/src/rabbit_fifo.erl | 39 ++-- deps/rabbit/test/quorum_queue_SUITE.erl | 198 +++++++++--------- deps/rabbit/test/rabbit_fifo_SUITE.erl | 85 ++++++-- .../rabbit_fifo_dlx_integration_SUITE.erl | 12 +- 4 files changed, 192 insertions(+), 142 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 144a70630ff0..e1da4789c3a1 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -303,20 +303,20 @@ apply_(Meta, #return{consumer_key = ConsumerKey, {State, ok} end; apply_(Meta, #modify{consumer_key = ConsumerKey, - delivery_failed = DelFailed, - undeliverable_here = Undel, + delivery_failed = DeliveryFailed, + undeliverable_here = UndelHere, annotations = Anns, msg_ids = MsgIds}, #?STATE{consumers = Cons} = State) -> case find_consumer(ConsumerKey, Cons) of {ActualConsumerKey, #consumer{checked_out = Checked}} - when Undel == false -> - return(Meta, ActualConsumerKey, MsgIds, DelFailed, + when UndelHere == false -> + return(Meta, ActualConsumerKey, MsgIds, DeliveryFailed, Anns, Checked, [], State); {ActualConsumerKey, #consumer{} = Con} - when Undel == true -> + when UndelHere == true -> discard(Meta, MsgIds, ActualConsumerKey, - Con, DelFailed, Anns, State); + Con, DeliveryFailed, Anns, State); _ -> {State, ok} end; @@ -499,7 +499,10 @@ apply_(#{system_time := Ts} = Meta, %% and checked out messages should be returned Effs = consumer_update_active_effects( S0, C0, false, suspected_down, E0), - {St, Effs1} = return_all(Meta, S0, Effs, CKey, C0, true), + %% TODO: set a timer instead of reaturn all here to allow + %% a disconnected node a configurable bit of time to be + %% reconnected + {St, Effs1} = return_all(Meta, S0, Effs, CKey, C0, false), %% if the consumer was cancelled there is a chance it got %% removed when returning hence we need to be defensive here Waiting = case St#?STATE.consumers of @@ -549,7 +552,7 @@ apply_(#{system_time := Ts} = Meta, status = up} = C0, {St0, Eff}) when node(P) =:= Node -> C = C0#consumer{status = suspected_down}, - {St, Eff0} = return_all(Meta, St0, Eff, CKey, C, true), + {St, Eff0} = return_all(Meta, St0, Eff, CKey, C, false), Eff1 = consumer_update_active_effects(St, C, false, suspected_down, Eff0), {St, Eff1}; @@ -1900,7 +1903,7 @@ annotate_msg(Header, Msg0) -> Msg0 end. -return_one(Meta, MsgId, Msg0, DelivFailed, Anns, +return_one(Meta, MsgId, Msg0, DeliveryFailed, Anns, #?STATE{returns = Returns, consumers = Consumers, dlx = DlxState0, @@ -1909,12 +1912,14 @@ return_one(Meta, MsgId, Msg0, DelivFailed, Anns, dead_letter_handler = DLH}} = State0, Effects0, ConsumerKey) -> #consumer{checked_out = Checked0} = Con0 = maps:get(ConsumerKey, Consumers), - Msg = incr_msg(Msg0, DelivFailed, Anns), + Msg = incr_msg_headers(Msg0, DeliveryFailed, Anns), Header = get_msg_header(Msg), %% TODO: do not use acquired count here as that includes all deliberate %% returns, use delivery_count header instead - case get_header(acquired_count, Header) of - AcquiredCount when AcquiredCount > DeliveryLimit -> + case get_header(delivery_count, Header) of + DeliveryCount + when is_integer(DeliveryCount) andalso + DeliveryCount > DeliveryLimit -> {DlxState, RetainedBytes, DlxEffects} = discard_or_dead_letter([Msg], delivery_limit, DLH, DlxState0), %% subtract retained bytes as complete/6 will add them on irrespective @@ -1937,10 +1942,10 @@ return_one(Meta, MsgId, Msg0, DelivFailed, Anns, end. return_all(Meta, #?STATE{consumers = Cons} = State0, Effects0, ConsumerKey, - #consumer{checked_out = Checked} = Con, DelivFailed) -> + #consumer{checked_out = Checked} = Con, DeliveryFailed) -> State = State0#?STATE{consumers = Cons#{ConsumerKey => Con}}, lists:foldl(fun ({MsgId, Msg}, {S, E}) -> - return_one(Meta, MsgId, Msg, DelivFailed, #{}, + return_one(Meta, MsgId, Msg, DeliveryFailed, #{}, S, E, ConsumerKey) end, {State, Effects0}, lists:sort(maps:to_list(Checked))). @@ -3028,7 +3033,7 @@ discard(Meta, MsgIds, ConsumerKey, undefined -> false; Msg0 -> - {true, incr_msg(Msg0, DelFailed, Anns)} + {true, incr_msg_headers(Msg0, DelFailed, Anns)} end end, MsgIds), {DlxState, RetainedBytes, Effects} = @@ -3037,7 +3042,7 @@ discard(Meta, MsgIds, ConsumerKey, discarded_bytes = DiscardedBytes0 - RetainedBytes}, complete_and_checkout(Meta, MsgIds, ConsumerKey, Con, Effects, State). -incr_msg(Msg0, DelFailed, Anns) -> +incr_msg_headers(Msg0, DeliveryFailed, Anns) -> Msg1 = update_msg_header(acquired_count, fun incr/1, 1, Msg0), Msg2 = case map_size(Anns) > 0 of true -> @@ -3048,7 +3053,7 @@ incr_msg(Msg0, DelFailed, Anns) -> false -> Msg1 end, - case DelFailed of + case DeliveryFailed of true -> update_msg_header(delivery_count, fun incr/1, 1, Msg2); false -> diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index 27cba0ac03b9..451dd9d51965 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -3502,6 +3502,21 @@ subscribe_redelivery_count(Config) -> exit(basic_deliver_timeout_3) end. +simulate_consumer_crash(Self, QQ, Config, Server) -> + Conn = rabbit_ct_client_helpers:open_unmanaged_connection(Config, Server), + {ok, Chan} = amqp_connection:open_channel(Conn), + qos(Chan, 2, false), + subscribe(Chan, QQ, false), + receive + {_, _} = All -> + %% need to do this to trigger the right channel + %% conditions + amqp_channel:close(Chan, 541, <<"oh no">>), + Self ! All + after 1000 -> + exit(timeout) + end. + subscribe_redelivery_limit(Config) -> [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -3513,37 +3528,35 @@ subscribe_redelivery_limit(Config) -> publish(Ch, QQ), wait_for_messages(Config, [[QQ, <<"1">>, <<"1">>, <<"0">>]]), - subscribe(Ch, QQ, false), + + Self = self(), + FailedMsgFun = fun () -> simulate_consumer_crash(Self, QQ, Config, Server) end, + + _ = spawn_monitor(FailedMsgFun), DCHeader = <<"x-delivery-count">>, receive - {#'basic.deliver'{delivery_tag = DeliveryTag, - redelivered = false}, + {#'basic.deliver'{redelivered = false}, #amqp_msg{props = #'P_basic'{headers = H0}}} -> - ?assertMatch(undefined, rabbit_basic:header(DCHeader, H0)), - amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, - multiple = false, - requeue = true}) + ?assertMatch(undefined, rabbit_basic:header(DCHeader, H0)) + after 5000 -> + flush(1), + ct:fail(no_delivery) end, - wait_for_messages(Config, [[QQ, <<"1">>, <<"0">>, <<"1">>]]), + %% one more consumer failure should cause the message to be dropped + _ = spawn_monitor(FailedMsgFun), + receive - {#'basic.deliver'{delivery_tag = DeliveryTag1, - redelivered = true}, + {#'basic.deliver'{redelivered = true}, #amqp_msg{props = #'P_basic'{headers = H1}}} -> - ?assertMatch({DCHeader, _, 1}, rabbit_basic:header(DCHeader, H1)), - amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag1, - multiple = false, - requeue = true}) + ?assertMatch({DCHeader, _, 1}, rabbit_basic:header(DCHeader, H1)) + after 5000 -> + flush(1), + ct:fail(no_delivery) end, - wait_for_messages(Config, [[QQ, <<"0">>, <<"0">>, <<"0">>]]), - receive - {#'basic.deliver'{redelivered = true}, #amqp_msg{}} -> - throw(unexpected_redelivery) - after 5000 -> - ok - end. + ok. subscribe_redelivery_limit_disable(Config) -> [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -3555,39 +3568,32 @@ subscribe_redelivery_limit_disable(Config) -> {<<"x-delivery-limit">>, long, -1}])), publish(Ch, QQ), wait_for_messages(Config, [[QQ, <<"1">>, <<"1">>, <<"0">>]]), - subscribe(Ch, QQ, false), + Self = self(), + FailedMsgFun = fun () -> simulate_consumer_crash(Self, QQ, Config, Server) end, + + _ = spawn_monitor(FailedMsgFun), DCHeader = <<"x-delivery-count">>, receive - {#'basic.deliver'{delivery_tag = DeliveryTag, - redelivered = false}, + {#'basic.deliver'{redelivered = false}, #amqp_msg{props = #'P_basic'{headers = H0}}} -> - ?assertMatch(undefined, rabbit_basic:header(DCHeader, H0)), - amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, - multiple = false, - requeue = true}) + ?assertMatch(undefined, rabbit_basic:header(DCHeader, H0)) + after 5000 -> + flush(1), + ct:fail(no_delivery) end, - wait_for_messages(Config, [[QQ, <<"1">>, <<"0">>, <<"1">>]]), - %% set an operator policy, this should always win - ok = rabbit_ct_broker_helpers:set_operator_policy( - Config, 0, <<"delivery-limit">>, QQ, <<"queues">>, - [{<<"delivery-limit">>, 0}]), - + _ = spawn_monitor(FailedMsgFun), receive - {#'basic.deliver'{delivery_tag = DeliveryTag2, - redelivered = true}, + {#'basic.deliver'{redelivered = true}, #amqp_msg{props = #'P_basic'{}}} -> - % ?assertMatch(undefined, rabbit_basic:header(DCHeader, H0)), - amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag2, - multiple = false, - requeue = true}) + ok after ?TIMEOUT -> flush(1), ct:fail("message did not arrive as expected") end, - wait_for_messages(Config, [[QQ, <<"0">>, <<"0">>, <<"0">>]]), - ok = rabbit_ct_broker_helpers:clear_operator_policy(Config, 0, <<"delivery-limit">>), + timer:sleep(100), + wait_for_messages(Config, [[QQ, <<"1">>, <<"1">>, <<"0">>]]), ok. %% Test that consumer credit is increased correctly. @@ -3600,36 +3606,38 @@ subscribe_redelivery_limit_many(Config) -> declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, {<<"x-delivery-limit">>, long, 1}])), + SomePid = spawn(fun() -> ok end), + FailedMsgFun = fun () -> simulate_consumer_crash(SomePid, QQ, Config, Server) end, + publish_many(Ch, QQ, 5), wait_for_messages(Config, [[QQ, <<"5">>, <<"5">>, <<"0">>]]), - qos(Ch, 2, false), - subscribe(Ch, QQ, false), - wait_for_messages(Config, [[QQ, <<"5">>, <<"3">>, <<"2">>]]), - - nack(Ch, false, true), - nack(Ch, false, true), - wait_for_messages(Config, [[QQ, <<"5">>, <<"3">>, <<"2">>]]), - - nack(Ch, false, true), - nack(Ch, false, true), - wait_for_messages(Config, [[QQ, <<"3">>, <<"1">>, <<"2">>]]), - - nack(Ch, false, true), - nack(Ch, false, true), - wait_for_messages(Config, [[QQ, <<"3">>, <<"1">>, <<"2">>]]), + spawn(FailedMsgFun), + wait_for_messages(Config, [[QQ, <<"5">>, <<"5">>, <<"0">>]]), - nack(Ch, false, true), - nack(Ch, false, true), - wait_for_messages(Config, [[QQ, <<"1">>, <<"0">>, <<"1">>]]), + spawn(FailedMsgFun), + wait_for_messages(Config, [[QQ, <<"3">>, <<"3">>, <<"0">>]]), - nack(Ch, false, true), - wait_for_messages(Config, [[QQ, <<"1">>, <<"0">>, <<"1">>]]), + spawn_and_wait(FailedMsgFun), + spawn_and_wait(FailedMsgFun), + wait_for_messages(Config, [[QQ, <<"1">>, <<"1">>, <<"0">>]]), - nack(Ch, false, true), + spawn_and_wait(FailedMsgFun), + spawn_and_wait(FailedMsgFun), wait_for_messages(Config, [[QQ, <<"0">>, <<"0">>, <<"0">>]]), ok. +spawn_and_wait(Fun) -> + {_, Ref} = spawn_monitor(Fun), + receive + {'DOWN', Ref, _, _, _} -> + ok + after 5000 -> + flush(1), + ct:fail(spawn_and_wait_timout) + end. + + subscribe_redelivery_policy(Config) -> [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -3644,37 +3652,32 @@ subscribe_redelivery_policy(Config) -> publish(Ch, QQ), wait_for_messages(Config, [[QQ, <<"1">>, <<"1">>, <<"0">>]]), - subscribe(Ch, QQ, false), + Self = self(), + FailedMsgFun = fun () -> simulate_consumer_crash(Self, QQ, Config, Server) end, DCHeader = <<"x-delivery-count">>, + spawn(FailedMsgFun), receive - {#'basic.deliver'{delivery_tag = DeliveryTag, - redelivered = false}, + {#'basic.deliver'{redelivered = false}, #amqp_msg{props = #'P_basic'{headers = H0}}} -> - ?assertMatch(undefined, rabbit_basic:header(DCHeader, H0)), - amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, - multiple = false, - requeue = true}) + ?assertMatch(undefined, rabbit_basic:header(DCHeader, H0)) + after 5000 -> + flush(1), + ct:fail(timeout) end, - wait_for_messages(Config, [[QQ, <<"1">>, <<"0">>, <<"1">>]]), + wait_for_messages(Config, [[QQ, <<"1">>, <<"1">>, <<"0">>]]), + spawn(FailedMsgFun), receive - {#'basic.deliver'{delivery_tag = DeliveryTag1, - redelivered = true}, + {#'basic.deliver'{redelivered = true}, #amqp_msg{props = #'P_basic'{headers = H1}}} -> - ?assertMatch({DCHeader, _, 1}, rabbit_basic:header(DCHeader, H1)), - amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag1, - multiple = false, - requeue = true}) + ?assertMatch({DCHeader, _, 1}, rabbit_basic:header(DCHeader, H1)) + after 5000 -> + flush(1), + ct:fail(timeout) end, wait_for_messages(Config, [[QQ, <<"0">>, <<"0">>, <<"0">>]]), - receive - {#'basic.deliver'{redelivered = true}, #amqp_msg{}} -> - throw(unexpected_redelivery) - after 5000 -> - ok - end, ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"delivery-limit">>). subscribe_redelivery_limit_with_dead_letter(Config) -> @@ -3694,28 +3697,29 @@ subscribe_redelivery_limit_with_dead_letter(Config) -> publish(Ch, QQ), wait_for_messages(Config, [[QQ, <<"1">>, <<"1">>, <<"0">>]]), - subscribe(Ch, QQ, false), DCHeader = <<"x-delivery-count">>, + Self = self(), + FailedMsgFun = fun () -> simulate_consumer_crash(Self, QQ, Config, Server) end, + spawn(FailedMsgFun), receive - {#'basic.deliver'{delivery_tag = DeliveryTag, - redelivered = false}, + {#'basic.deliver'{redelivered = false}, #amqp_msg{props = #'P_basic'{headers = H0}}} -> - ?assertMatch(undefined, rabbit_basic:header(DCHeader, H0)), - amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, - multiple = false, - requeue = true}) + ?assertMatch(undefined, rabbit_basic:header(DCHeader, H0)) + after 5000 -> + flush(1), + ct:fail(timeout) end, - wait_for_messages(Config, [[QQ, <<"1">>, <<"0">>, <<"1">>]]), + wait_for_messages(Config, [[QQ, <<"1">>, <<"1">>, <<"0">>]]), + spawn(FailedMsgFun), receive - {#'basic.deliver'{delivery_tag = DeliveryTag1, - redelivered = true}, + {#'basic.deliver'{redelivered = true}, #amqp_msg{props = #'P_basic'{headers = H1}}} -> - ?assertMatch({DCHeader, _, 1}, rabbit_basic:header(DCHeader, H1)), - amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag1, - multiple = false, - requeue = true}) + ?assertMatch({DCHeader, _, 1}, rabbit_basic:header(DCHeader, H1)) + after 5000 -> + flush(1), + ct:fail(timeout) end, wait_for_messages(Config, [[QQ, <<"0">>, <<"0">>, <<"0">>]]), diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 2680bdf89498..449b919f6ced 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -112,6 +112,8 @@ discarded_bytes_test(Config) -> }, CPid = spawn(fun () -> ok end), Cid = {?FUNCTION_NAME_B, CPid}, + CPid2 = spawn(fun () -> ok end), + Cid2 = {?FUNCTION_NAME_B, CPid2}, Msg = crypto:strong_rand_bytes(1000), {State1, _} = enq(Config, ?LINE, 1, Msg, init(Conf)), %% enqueues should not increment discarded bytes @@ -162,7 +164,7 @@ discarded_bytes_test(Config) -> discarded_bytes := DiscBytes9} = rabbit_fifo:overview(State9), %% update config to have a delivery-limit - Conf2 = Conf#{delivery_limit => 1}, + Conf2 = Conf#{delivery_limit => 0}, {State10, ok, _} = apply(meta(Config, 5), rabbit_fifo:make_update_config(Conf2), State9), #{num_messages := 1, @@ -170,20 +172,17 @@ discarded_bytes_test(Config) -> ?assert(DiscBytes10 > DiscBytes9), {State11, _, _} = apply(meta(Config, ?LINE), - rabbit_fifo:make_return(CKey, [NextMsgId + 3]), + {down, CPid, blah}, State10), - #{num_messages := 1, + #{num_messages := 0, discarded_bytes := DiscBytes11} = rabbit_fifo:overview(State11), - ?assert(DiscBytes11 > DiscBytes10), - - {State12, _, _} = apply(meta(Config, ?LINE), - rabbit_fifo:make_return(CKey, [NextMsgId + 4]), - State11), + ?assert(DiscBytes11 - DiscBytes10 > 1000), - %% delivery-limit was reached and message was discarded - #{num_messages := 0, - discarded_bytes := DiscBytes12} = rabbit_fifo:overview(State12), - ?assert(DiscBytes12 - DiscBytes11 > 1000), + %% checkout again + Spec = {auto, {simple_prefetch, 2}}, + {State12, #{key := CKey2, + next_msg_id := C2NextMsgId}, _} = + checkout(Config, ?LINE, Cid2, Spec, State11), %% at-least-once dead lettering Conf3 = Conf2#{dead_letter_handler => at_least_once}, @@ -196,7 +195,7 @@ discarded_bytes_test(Config) -> discarded_bytes := DiscBytes14} = rabbit_fifo:overview(State14), {State15, _, _} = apply(meta(Config, ?LINE), - rabbit_fifo:make_discard(CKey, [NextMsgId + 5]), + rabbit_fifo:make_discard(CKey2, [C2NextMsgId]), State14), #{num_messages := 1, discarded_bytes := DiscBytes15} = rabbit_fifo:overview(State15), @@ -225,7 +224,7 @@ discarded_bytes_test(Config) -> discarded_bytes := DiscBytes17} = rabbit_fifo:overview(State18), {State19, _, _} = apply(meta(Config, ?LINE), - rabbit_fifo:make_modify(CKey, [NextMsgId + 5], + rabbit_fifo:make_modify(CKey2, [C2NextMsgId + 1], false, false, #{}), State18), #{num_messages := 1, @@ -233,7 +232,8 @@ discarded_bytes_test(Config) -> ?assert(DiscBytes19 > DiscBytes17), %% change the dlx handler - Conf4 = Conf3#{dead_letter_handler => {at_most_once, {?MODULE, ?FUNCTION_NAME, []}}, + Conf4 = Conf3#{dead_letter_handler => + {at_most_once, {?MODULE, ?FUNCTION_NAME, []}}, max_length => 2}, {State20, ok, _} = apply(meta(Config, ?LINE), rabbit_fifo:make_update_config(Conf4), State19), @@ -241,7 +241,7 @@ discarded_bytes_test(Config) -> discarded_bytes := DiscBytes20} = rabbit_fifo:overview(State20), {State21, _, _} = apply(meta(Config, ?LINE), - rabbit_fifo:make_modify(CKey, [NextMsgId + 6], + rabbit_fifo:make_modify(CKey2, [C2NextMsgId + 2], true, true, #{}), State20), #{num_messages := 0, @@ -250,8 +250,10 @@ discarded_bytes_test(Config) -> %% unsubsrcibe {State22, _, _} = apply(meta(Config, ?LINE), - make_checkout(Cid, cancel, #{}), State21), + make_checkout(Cid2, remove, #{}), State21), + ct:pal("State22 ~p", [State22]), #{num_messages := 0, + num_consumers := 0, discarded_bytes := DiscBytes22} = rabbit_fifo:overview(State22), ?assert(DiscBytes22 > DiscBytes21), @@ -686,6 +688,8 @@ return_multiple_test(Config) -> ok. return_dequeue_delivery_limit_test(C) -> + %% now tests that more returns than the delivery limit _does _not_ + %% cause the message to be removed Init = init(#{name => test, queue_resource => rabbit_misc:r("/", queue, atom_to_binary(test, utf8)), @@ -696,14 +700,18 @@ return_dequeue_delivery_limit_test(C) -> Cid2 = {<<"cid2">>, self()}, Msg = rabbit_fifo:make_enqueue(self(), 1, msg), - {State1, {MsgId1, _}} = deq(C, 2, Cid, unsettled, Msg, State0), - {State2, _, _} = apply(meta(C, 4), rabbit_fifo:make_return(Cid, [MsgId1]), + {State1, {MsgId1, _}} = deq(C, ?LINE, Cid, unsettled, Msg, State0), + % debugger:start(), + % int:i(rabbit_fifo), + % int:break(rabbit_fifo, 1914), + {State2, _, _} = apply(meta(C, ?LINE), rabbit_fifo:make_return(Cid, [MsgId1]), State1), - {State3, {MsgId2, _}} = deq(C, 2, Cid2, unsettled, Msg, State2), - {State4, _, _} = apply(meta(C, 4), rabbit_fifo:make_return(Cid2, [MsgId2]), + ct:pal("State2 ~p", [State2]), + {State3, {MsgId2, _}} = deq(C, ?LINE, Cid2, unsettled, Msg, State2), + {State4, _, _} = apply(meta(C, ?LINE), rabbit_fifo:make_return(Cid2, [MsgId2]), State3), - ?assertMatch(#{num_messages := 0}, rabbit_fifo:overview(State4)), + ?assertMatch(#{num_messages := 1}, rabbit_fifo:overview(State4)), ok. return_non_existent_test(Config) -> @@ -747,6 +755,37 @@ return_checked_out_limit_test(Config) -> {#rabbit_fifo{} = State, ok, _} = apply(meta(Config, 4), rabbit_fifo:make_return(Cid, [MsgId + 1]), State2), + ?assertEqual(1, rabbit_fifo:query_messages_total(State)), + ok. + +down_checked_out_limit_test(Config) -> + Cid = {<<"cid">>, self()}, + Init = init(#{name => test, + queue_resource => rabbit_misc:r("/", queue, + atom_to_binary(test, utf8)), + release_cursor_interval => 0, + max_in_memory_length => 0, + delivery_limit => 1}), + Msg1 = rabbit_fifo:make_enqueue(self(), 1, first), + {State0, _} = enq(Config, 1, 1, Msg1, Init), + {State1, #{key := _, + next_msg_id := _C1MsgId}, Effects1} = + checkout(Config, ?LINE, Cid, 1, State0), + ?ASSERT_EFF({log_ext, [1], _Fun, _Local}, Effects1), + % returning immediately checks out the same message again + {State2, ok, _Effects2} = + apply(meta(Config, 3), {down, self(), error}, State1), + + {State3, #{key := _, + next_msg_id := _C2MsgId}, Effects3} = + checkout(Config, ?LINE, Cid, 1, State2), + ?ASSERT_EFF({log_ext, [1], _Fun, _Local}, Effects3), + + {State4, ok, _Effects4} = + apply(meta(Config, ?LINE), {down, self(), error}, State3), + % {#rabbit_fifo{} = State, ok, _} = + % apply(meta(Config, 4), rabbit_fifo:make_return(Cid, [MsgId + 1]), State2), + State = State4, ?assertEqual(0, rabbit_fifo:query_messages_total(State)), ok. @@ -1665,7 +1704,7 @@ active_flag_updated_when_consumer_suspected_unsuspected_test(Config) -> ], {State1, _} = run_log(Config, State0, Entries), {State2, _, Effects2} = apply(meta(Config, 3), - {down, Pid1, noconnection}, State1), + {down, Pid1, noconnection}, State1), % 1 effect to update the metrics of each consumer % (they belong to the same node), % 1 more effect to monitor the node, diff --git a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl index ce8e242fd794..c93afa144b14 100644 --- a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl @@ -227,13 +227,15 @@ delivery_limit(Config) -> {Server, Ch, SourceQ, TargetQ} = declare_topology(Config, [{<<"x-delivery-limit">>, long, 0}]), publish(Ch, SourceQ), wait_for_messages_ready([Server], ra_name(SourceQ), 1), - DelTag = basic_get_tag(Ch, SourceQ, false), - amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DelTag, - multiple = false, - requeue = true}), + _DelTag = basic_get_tag(Ch, SourceQ, false), + amqp_channel:close(Ch, 541, <<"boo">>), + Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server), + % amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DelTag, + % multiple = false, + % requeue = true}), {_, #amqp_msg{props = #'P_basic'{headers = Headers}}} = ?awaitMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg">>}}, - amqp_channel:call(Ch, #'basic.get'{queue = TargetQ}), + amqp_channel:call(Ch2, #'basic.get'{queue = TargetQ}), 30000), assert_dlx_headers(Headers, <<"delivery_limit">>, SourceQ), ?assertEqual(1, counted(messages_dead_lettered_delivery_limit_total, Config)), From 2ff1b30df42670e0190d52a98be970a145737eed Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 24 Oct 2025 17:04:07 +0100 Subject: [PATCH 14/28] QQ: Better noconnection handling instead of immedately returning all pending messages for a consumer that goes down with the `noconnection` reason we set a timer to give the node some time to re-connect should the connection issue only be transient and the consumer is still running. --- deps/rabbit/src/rabbit_fifo.erl | 268 ++++++---- deps/rabbit/src/rabbit_fifo.hrl | 4 +- deps/rabbit/test/quorum_queue_SUITE.erl | 1 + deps/rabbit/test/rabbit_fifo_SUITE.erl | 562 +++++++++++++-------- deps/rabbit/test/rabbit_fifo_int_SUITE.erl | 3 +- 5 files changed, 518 insertions(+), 320 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index e1da4789c3a1..1d5cfa81e558 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -11,6 +11,7 @@ -compile(inline_list_funcs). -compile(inline). -compile({no_auto_import, [apply/3]}). +-dialyzer({nowarn_function, convert_v7_to_v8/2}). -dialyzer(no_improper_lists). -include("rabbit_fifo.hrl"). @@ -482,95 +483,146 @@ apply_(#{index := _Idx}, #garbage_collection{}, State) -> {State, ok, [{aux, garbage_collection}]}; apply_(Meta, {timeout, expire_msgs}, State) -> checkout(Meta, State, State, []); -apply_(#{system_time := Ts} = Meta, - {down, Pid, noconnection}, - #?STATE{consumers = Cons0, - cfg = #cfg{consumer_strategy = single_active}, - waiting_consumers = Waiting0, - enqueuers = Enqs0} = State0) -> - Node = node(Pid), - %% if the pid refers to an active or cancelled consumer, - %% mark it as suspected and return it to the waiting queue - {State1, Effects0} = - maps:fold( - fun(CKey, ?CONSUMER_PID(P) = C0, {S0, E0}) - when node(P) =:= Node -> - %% the consumer should be returned to waiting - %% and checked out messages should be returned - Effs = consumer_update_active_effects( - S0, C0, false, suspected_down, E0), - %% TODO: set a timer instead of reaturn all here to allow - %% a disconnected node a configurable bit of time to be - %% reconnected - {St, Effs1} = return_all(Meta, S0, Effs, CKey, C0, false), - %% if the consumer was cancelled there is a chance it got - %% removed when returning hence we need to be defensive here - Waiting = case St#?STATE.consumers of - #{CKey := C} -> - Waiting0 ++ [{CKey, C}]; - _ -> - Waiting0 - end, - {St#?STATE{consumers = maps:remove(CKey, St#?STATE.consumers), - waiting_consumers = Waiting, - last_active = Ts}, - Effs1}; - (_, _, S) -> - S - end, {State0, []}, maps:iterator(Cons0, ordered)), - WaitingConsumers = update_waiting_consumer_status(Node, State1, - suspected_down), - - %% select a new consumer from the waiting queue and run a checkout - State2 = State1#?STATE{waiting_consumers = WaitingConsumers}, - {State, Effects1} = activate_next_consumer(State2, Effects0), - - %% mark any enquers as suspected - Enqs = maps:map(fun(P, E) when node(P) =:= Node -> - E#enqueuer{status = suspected_down}; - (_, E) -> E - end, Enqs0), - Effects = [{monitor, node, Node} | Effects1], - checkout(Meta, State0, State#?STATE{enqueuers = Enqs}, Effects); +% apply_(#{system_time := Ts} = Meta, +% {down, Pid, noconnection}, +% #?STATE{consumers = Cons0, +% cfg = #cfg{consumer_strategy = single_active}, +% waiting_consumers = Waiting0, +% enqueuers = Enqs0} = State0) -> +% Node = node(Pid), +% %% if the pid refers to an active or cancelled consumer, +% %% mark it as suspected and return it to the waiting queue +% {State1, Effects0} = +% maps:fold( +% fun(CKey, ?CONSUMER_PID(P) = #consumer{status = Status} = C0, {S0, E0}) +% when is_atom(Status) andalso node(P) =:= Node -> +% %% the consumer should be returned to waiting +% %% and checked out messages should be returned +% Effs = consumer_update_active_effects( +% S0, C0, false, {suspected_down, Status} , E0), +% %% TODO: set a timer instead of reaturn all here to allow +% %% a disconnected node a configurable bit of time to be +% %% reconnected +% {St, Effs1} = return_all(Meta, S0, Effs, CKey, C0, false), +% %% if the consumer was cancelled there is a chance it got +% %% removed when returning hence we need to be defensive here +% Waiting = case St#?STATE.consumers of +% #{CKey := C} -> +% Waiting0 ++ [{CKey, C}]; +% _ -> +% Waiting0 +% end, +% {St#?STATE{consumers = maps:remove(CKey, St#?STATE.consumers), +% waiting_consumers = Waiting, +% last_active = Ts}, +% Effs1}; +% (_, _, S) -> +% S +% end, {State0, []}, maps:iterator(Cons0, ordered)), +% WaitingConsumers = update_waiting_consumer_status(Node, State1, +% {suspected_down, up}), + +% %% select a new consumer from the waiting queue and run a checkout +% State2 = State1#?STATE{waiting_consumers = WaitingConsumers}, +% {State, Effects1} = activate_next_consumer(State2, Effects0), + +% %% mark any enquers as suspected +% Enqs = maps:map(fun(P, E) when node(P) =:= Node -> +% E#enqueuer{status = suspected_down}; +% (_, E) -> E +% end, Enqs0), +% Effects = [{monitor, node, Node} | Effects1], +% checkout(Meta, State0, State#?STATE{enqueuers = Enqs}, Effects); apply_(#{system_time := Ts} = Meta, {down, Pid, noconnection}, #?STATE{consumers = Cons0, enqueuers = Enqs0} = State0) -> + %% A node has been disconnected. This doesn't necessarily mean that %% any processes on this node are down, they _may_ come back so here %% we just mark them as suspected (effectively deactivated) %% and return all checked out messages to the main queue for delivery to any %% live consumers - %% - %% all pids for the disconnected node will be marked as suspected not just - %% the one we got the `down' command for + Node = node(Pid), - {State, Effects1} = + {Cons, Effects1} = maps:fold( fun(CKey, #consumer{cfg = #consumer_cfg{pid = P}, - status = up} = C0, - {St0, Eff}) when node(P) =:= Node -> - C = C0#consumer{status = suspected_down}, - {St, Eff0} = return_all(Meta, St0, Eff, CKey, C, false), - Eff1 = consumer_update_active_effects(St, C, false, + status = Status} = C0, + {Cns0, Eff}) when P =:= Pid -> + TargetStatus = case Status of + {suspected_down, T} -> T; + _ -> + Status + end, + C = C0#consumer{status = {suspected_down, TargetStatus}}, + % down consumer still has messages assigned + % TODO: make timeout configurable + Eff0 = [{timer, {consumer_down_timeout, CKey}, 10_000} | Eff], + Eff1 = consumer_update_active_effects(State0, C, false, suspected_down, Eff0), - {St, Eff1}; - (_, _, {St, Eff}) -> - {St, Eff} - end, {State0, []}, maps:iterator(Cons0, ordered)), - Enqs = maps:map(fun(P, E) when node(P) =:= Node -> - E#enqueuer{status = suspected_down}; - (_, E) -> E - end, Enqs0), + {Cns0#{CKey => C}, Eff1}; + (_, _, St) -> + St + end, {Cons0, []}, maps:iterator(Cons0, ordered)), + Enqs = case Enqs0 of + #{Pid := E} -> + Enqs0#{Pid := E#enqueuer{status = suspected_down}}; + _ -> + Enqs0 + end, + WaitingConsumers = update_waiting_consumer_status(Pid, State0, + {suspected_down, up}), % Monitor the node so that we can "unsuspect" these processes when the node % comes back, then re-issue all monitors and discover the final fate of % these processes - Effects = [{monitor, node, Node} | Effects1], - checkout(Meta, State0, State#?STATE{enqueuers = Enqs, - last_active = Ts}, Effects); + checkout(Meta, State0, State0#?STATE{enqueuers = Enqs, + waiting_consumers = WaitingConsumers, + consumers = Cons, + last_active = Ts}, Effects); +apply_(Meta, {timeout, {consumer_down_timeout, CKey}}, + #?STATE{cfg = #cfg{consumer_strategy = competing}, + consumers = Consumers} = State0) -> + + case find_consumer(CKey, Consumers) of + {_CKey, #consumer{status = {suspected_down, _}} = Consumer} -> + %% the consumer is still suspected and has timed out + %% return all messages + {State1, Effects0} = return_all(Meta, State0, [], CKey, + Consumer, false), + checkout(Meta, State0, State1, Effects0); + _ -> + {State0, []} + end; +apply_(#{system_time := Ts} = Meta, {timeout, {consumer_down_timeout, CKey}}, + #?STATE{cfg = #cfg{consumer_strategy = single_active}, + waiting_consumers = Waiting0, + consumers = Consumers} = State0) -> + + case find_consumer(CKey, Consumers) of + {_CKey, #consumer{status = {suspected_down, Status}} = Consumer} -> + %% the consumer is still suspected and has timed out + %% return all messages + {State1, Effects0} = return_all(Meta, State0, [], CKey, + Consumer, false), + Waiting = case State1#?STATE.consumers of + #{CKey := C} when Status =/= cancelled -> + Waiting0 ++ + [{CKey, C#consumer{status = {suspected_down, up}}}]; + _ -> + Waiting0 + end, + State2 = State1#?STATE{consumers = maps:remove(CKey, State1#?STATE.consumers), + waiting_consumers = Waiting, + last_active = Ts}, + {State, Effects1} = activate_next_consumer(State2, Effects0), + checkout(Meta, State0, State, Effects1); + _ -> + {State0, []} + end; apply_(Meta, {down, Pid, _Info}, State0) -> {State1, Effects1} = activate_next_consumer(handle_down(Meta, Pid, State0)), checkout(Meta, State0, State1, Effects1); @@ -580,33 +632,37 @@ apply_(Meta, {nodeup, Node}, #?STATE{consumers = Cons0, %% If we have suspected any processes of being %% down we should now re-issue the monitors for them to detect if they're %% actually down or not - Monitors = [{monitor, process, P} - || P <- suspected_pids_for(Node, State0)], + %% send leader change events to all disconnected enqueuers to prompt them + %% to resend any messages stuck during disconnection, + %% ofc it may not be a leader change per se but it has the same effect + Effects0 = lists:flatten([[{monitor, process, P}, + {send_msg, P, leader_change, ra_event}] + || P <- suspected_pids_for(Node, State0)]), Enqs1 = maps:map(fun(P, E) when node(P) =:= Node -> E#enqueuer{status = up}; (_, E) -> E end, Enqs0), - %% send leader change events to all disconnected enqueuers to prompt them - %% to resend any messages stuck during disconnection, - %% ofc it may not be a leader change per se - Effects0 = maps:fold(fun(P, _E, Acc) when node(P) =:= Node -> - [{send_msg, P, leader_change, ra_event} | Acc]; - (_, _E, Acc) -> Acc - end, Monitors, Enqs0), ConsumerUpdateActiveFun = consumer_active_flag_update_function(State0), %% mark all consumers as up {State1, Effects1} = maps:fold( - fun(ConsumerKey, ?CONSUMER_PID(P) = C, {SAcc, EAcc}) - when (node(P) =:= Node) and - (C#consumer.status =/= cancelled) -> + fun(ConsumerKey, + ?CONSUMER_PID(P) = + #consumer{status = {suspected_down, NextStatus}} = C, + {SAcc, EAcc0}) + when node(P) =:= Node -> EAcc1 = ConsumerUpdateActiveFun(SAcc, ConsumerKey, - C, true, up, EAcc), + C, true, NextStatus, EAcc0), + %% cancel timers + EAcc = [{timer, + {consumer_down_timeout, ConsumerKey}, + infinity} | EAcc1], + {update_or_remove_con(Meta, ConsumerKey, - C#consumer{status = up}, - SAcc), EAcc1}; + C#consumer{status = NextStatus}, + SAcc), EAcc}; (_, _, Acc) -> Acc end, {State0, Effects0}, maps:iterator(Cons0, ordered)), @@ -696,8 +752,16 @@ snapshot_installed(_Meta, #?MODULE{cfg = #cfg{}, Effs. convert_v7_to_v8(#{} = _Meta, StateV7) -> + %% the structure is intact for now + Cons0 = element(#?STATE.consumers, StateV7), + Cons = maps:map(fun (_CKey, #consumer{status = suspected_down} = C) -> + C#consumer{status = {suspected_down, up}}; + (_CKey, C) -> + C + end, Cons0), StateV8 = StateV7, StateV8#?STATE{discarded_bytes = 0, + consumers = Cons, unused_0 = ?NIL}. purge_node(Meta, Node, State, Effects) -> @@ -763,15 +827,16 @@ handle_waiting_consumer_down(Pid, State = State0#?STATE{waiting_consumers = StillUp}, {Effects, State}. -update_waiting_consumer_status(Node, +update_waiting_consumer_status(DownPidOrNode, #?STATE{waiting_consumers = WaitingConsumers}, Status) -> sort_waiting( - [case node(Pid) of - Node -> - {ConsumerKey, Consumer#consumer{status = Status}}; - _ -> - {ConsumerKey, Consumer} + [if is_pid(DownPidOrNode) andalso DownPidOrNode == Pid -> + {ConsumerKey, Consumer#consumer{status = Status}}; + is_atom(DownPidOrNode) andalso DownPidOrNode == node(Pid) -> + {ConsumerKey, Consumer#consumer{status = Status}}; + true -> + {ConsumerKey, Consumer} end || {ConsumerKey, ?CONSUMER_PID(Pid) = Consumer} <- WaitingConsumers, Consumer#consumer.status =/= cancelled]). @@ -1221,7 +1286,9 @@ query_waiting_consumers(#?STATE{waiting_consumers = WaitingConsumers}) -> query_consumer_count(#?STATE{consumers = Consumers, waiting_consumers = WaitingConsumers}) -> Up = maps:filter(fun(_ConsumerKey, #consumer{status = Status}) -> - Status =/= suspected_down + %% TODO: should this really not include suspected + %% consumers? + is_atom(Status) end, Consumers), maps:size(Up) + length(WaitingConsumers). @@ -1234,8 +1301,8 @@ query_consumers(#?STATE{consumers = Consumers, competing -> fun(_ConsumerKey, #consumer{status = Status}) -> case Status of - suspected_down -> - {false, Status}; + {suspected_down, _} -> + {false, suspected_down}; _ -> {true, Status} end @@ -1522,7 +1589,10 @@ activate_next_consumer(#?STATE{consumers = Cons0, end. active_consumer({CKey, #consumer{status = Status} = Consumer, _I}) - when Status == up orelse Status == quiescing -> + when Status == up orelse + Status == quiescing orelse + Status == {suspected_down, up} orelse + Status == {suspected_down, quiescing} -> {CKey, Consumer}; active_consumer({_CKey, #consumer{status = _}, I}) -> active_consumer(maps:next(I)); @@ -1928,7 +1998,7 @@ return_one(Meta, MsgId, Msg0, DeliveryFailed, Anns, State1 = State0#?STATE{dlx = DlxState, discarded_bytes = DiscardedBytes0 - RetainedBytes}, {State, Effects} = complete(Meta, ConsumerKey, [MsgId], - Con0, State1, Effects0), + Con0, State1, Effects0), {State, DlxEffects ++ Effects}; _ -> Checked = maps:remove(MsgId, Checked0), @@ -2715,7 +2785,7 @@ suspected_pids_for(Node, #?STATE{consumers = Cons0, waiting_consumers = WaitingConsumers0}) -> Cons = maps:fold(fun(_Key, #consumer{cfg = #consumer_cfg{pid = P}, - status = suspected_down}, + status = {suspected_down, _}}, Acc) when node(P) =:= Node -> [P | Acc]; @@ -2730,7 +2800,7 @@ suspected_pids_for(Node, #?STATE{consumers = Cons0, end, Cons, maps:iterator(Enqs0, ordered)), lists:foldl(fun({_Key, #consumer{cfg = #consumer_cfg{pid = P}, - status = suspected_down}}, Acc) + status = {suspected_down, _}}}, Acc) when node(P) =:= Node -> [P | Acc]; (_, Acc) -> Acc @@ -2741,7 +2811,7 @@ is_expired(Ts, #?STATE{cfg = #cfg{expires = Expires}, consumers = Consumers}) when is_number(LastActive) andalso is_number(Expires) -> %% TODO: should it be active consumers? - Active = maps:filter(fun (_, #consumer{status = suspected_down}) -> + Active = maps:filter(fun (_, #consumer{status = {suspected_down, _}}) -> false; (_, _) -> true diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index 2da17bf2295e..656a6e997b7d 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -153,9 +153,11 @@ lifetime = once :: once | auto, priority = 0 :: integer()}). +-type consumer_status() :: up | cancelled | quiescing. + -record(consumer, {cfg = #consumer_cfg{}, - status = up :: up | suspected_down | cancelled | quiescing, + status = up :: consumer_status() | {suspected_down, consumer_status()}, next_msg_id = 0 :: msg_id(), checked_out = #{} :: #{msg_id() => msg()}, %% max number of messages that can be sent diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index 451dd9d51965..b12b51e7ba63 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -3314,6 +3314,7 @@ reconnect_consumer_and_publish(Config) -> Up = [Leader, F2], rabbit_ct_broker_helpers:block_traffic_between(F1, Leader), rabbit_ct_broker_helpers:block_traffic_between(F1, F2), + wait_for_messages_ready(Up, RaName, 1), wait_for_messages_pending_ack(Up, RaName, 0), wait_for_messages_ready([F1], RaName, 0), diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 449b919f6ced..33348f6f46cc 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -1,13 +1,11 @@ -module(rabbit_fifo_SUITE). - %% rabbit_fifo unit tests suite -compile(nowarn_export_all). -compile(export_all). -compile({no_auto_import, [apply/3]}). --export([ - ]). +-export([]). -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). @@ -15,29 +13,23 @@ -include_lib("rabbit_common/include/rabbit_framing.hrl"). -include_lib("rabbit/src/rabbit_fifo.hrl"). -% -define(PROTOMOD, rabbit_framing_amqp_0_9_1). %%%=================================================================== %%% Common Test callbacks %%%=================================================================== all() -> [ - {group, tests}, - {group, machine_version_conversion} + {group, tests} ]. - -%% replicate eunit like test resultion +%% replicate eunit like test resolution all_tests() -> [F || {F, 1} <- ?MODULE:module_info(functions), re:run(atom_to_list(F), "_test$") /= nomatch]. groups() -> [ - {tests, [shuffle], all_tests()}, - {machine_version_conversion, [shuffle], - [convert_v2_to_v3, - convert_v3_to_v4]} + {tests, [shuffle], all_tests()} ]. init_per_group(tests, Config) -> @@ -83,10 +75,10 @@ end_per_testcase(_Group, _Config) -> (_) -> false end, Effects))). --define(assertNoEffect(EfxPat, Effects), - ?assert(not lists:any(fun (EfxPat) -> true; - (_) -> false - end, Effects))). +% -define(assertNoEffect(EfxPat, Effects), +% ?assert(not lists:any(fun (EfxPat) -> true; +% (_) -> false +% end, Effects))). -define(ASSERT(Guard, Fun), {assert, fun (S) -> ?assertMatch(Guard, S), _ = Fun(S) end}). @@ -902,6 +894,47 @@ removed_consumer_returns_unsettled_test(Config) -> ?ASSERT_EFF({monitor, process, _}, Effects), ok. +cancelled_down_with_noconnection_comes_back_test(Config) -> + R = rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME, utf8)), + State0 = init(#{name => ?FUNCTION_NAME, + queue_resource => R}), + + {CK1, {_, C1Pid} = C1} = {?LINE, {?LINE_B, test_util:fake_pid(n1)}}, + {CK2, {_, C2Pid} = C2} = {?LINE, {?LINE_B, test_util:fake_pid(n2)}}, + Entries = + [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {credited, 0}}, #{})}, + {?LINE, rabbit_fifo:make_credit(CK2, 1, 0, false)}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up, + credit = 1}, + CK2 := #consumer{status = up, + credit = 1}}}), + {?LINE, rabbit_fifo:make_enqueue(self(), 1, one)}, + {?LINE, rabbit_fifo:make_enqueue(self(), 2, two)}, + {CK1, make_checkout(C1, cancel, #{})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = cancelled, + credit = 0}, + CK2 := #consumer{status = up, + credit = 0}}}), + {?LINE, {down, C1Pid, noconnection}}, + {?LINE, {down, C2Pid, noconnection}}, + ?ASSERT(#rabbit_fifo{consumers = + #{CK1 := #consumer{status = {suspected_down, cancelled}, + credit = 0}, + CK2 := #consumer{status = {suspected_down, up}, + credit = 0}}}), + {?LINE, {nodeup, node(C1Pid)}}, + {?LINE, {nodeup, node(C2Pid)}}, + ?ASSERT(#rabbit_fifo{consumers = + #{CK1 := #consumer{status = cancelled, + credit = 0}, + CK2 := #consumer{status = up, + credit = 0}}}) + ], + {_State1, _} = run_log(Config, State0, Entries), + ok. + down_with_noconnection_marks_suspect_and_node_is_monitored_test(Config) -> Pid = spawn(fun() -> ok end), Cid = {?FUNCTION_NAME_B, Pid}, @@ -910,45 +943,47 @@ down_with_noconnection_marks_suspect_and_node_is_monitored_test(Config) -> {State0, Effects0} = enq(Config, 1, 1, second, test_init(test)), ?ASSERT_EFF({monitor, process, P}, P =:= Self, Effects0), {State1, #{key := CKey}, Effects1} = checkout(Config, ?LINE, Cid, 1, State0), - #consumer{credit = 0} = maps:get(CKey, State1#rabbit_fifo.consumers), + #consumer{credit = 0, + checked_out = CH1} = maps:get(CKey, State1#rabbit_fifo.consumers), + ?assertMatch(#{0 := _}, CH1), ?ASSERT_EFF({monitor, process, P}, P =:= Pid, Effects1), % monitor both enqueuer and consumer % because we received a noconnection we now need to monitor the node - {State2a, _, _} = apply(meta(Config, 3), {down, Pid, noconnection}, State1), - #consumer{credit = 1, - checked_out = Ch, - status = suspected_down} = maps:get(CKey, State2a#rabbit_fifo.consumers), - ?assertEqual(#{}, Ch), - %% validate consumer has credit - {State2, _, Effects2} = apply(meta(Config, 3), {down, Self, noconnection}, State2a), - ?ASSERT_EFF({monitor, node, _}, Effects2), - ?assertNoEffect({demonitor, process, _}, Effects2), + {State1b, _, _} = apply(meta(Config, ?LINE), {down, Self, noconnection}, State1), + {State2, _, Effs} = apply(meta(Config, ?LINE), {down, Pid, noconnection}, State1b), + ?ASSERT_EFF({timer, {consumer_down_timeout, K}, _Timeout}, K == CKey, Effs), + Node = node(), + ?ASSERT_EFF({monitor, node, N}, N == Node , Effs), + + #consumer{credit = 0, + checked_out = CH1, + status = {suspected_down, up}} = maps:get(CKey, State2#rabbit_fifo.consumers), + + %% test enter_state(leader, to ensure that the consumer_down_timeout events + % when the node comes up we need to retry the process monitors for the % disconnected processes - {State3, _, Effects3} = apply(meta(Config, 3), {nodeup, Node}, State2), - #consumer{status = up} = maps:get(CKey, State3#rabbit_fifo.consumers), - % try to re-monitor the suspect processes + {State3, _, Effects3} = apply(meta(Config, ?LINE), {nodeup, Node}, State2), + #consumer{status = up, + credit = 0, + checked_out = CH1} = maps:get(CKey, State3#rabbit_fifo.consumers), + % try to re-monitor the suspect process ?ASSERT_EFF({monitor, process, P}, P =:= Pid, Effects3), ?ASSERT_EFF({monitor, process, P}, P =:= Self, Effects3), - ok. + %% consumer proc is prodded to resend any pending commands that might + %% have been dropped during the disconnection + ?ASSERT_EFF({send_msg, CPid, leader_change, ra_event}, CPid == Pid, Effects3), -down_with_noconnection_returns_unack_test(Config) -> - Pid = spawn(fun() -> ok end), - Cid = {?FUNCTION_NAME_B, Pid}, - Msg = rabbit_fifo:make_enqueue(self(), 1, second), - {State0, _} = enq(Config, 1, 1, second, test_init(test)), - ?assertEqual(1, rabbit_fifo_q:len(State0#rabbit_fifo.messages)), - ?assertEqual(0, lqueue:len(State0#rabbit_fifo.returns)), - {State1, {_, _}} = deq(Config, 2, Cid, unsettled, Msg, State0), - ?assertEqual(0, rabbit_fifo_q:len(State1#rabbit_fifo.messages)), - ?assertEqual(0, lqueue:len(State1#rabbit_fifo.returns)), - {State2a, _, _} = apply(meta(Config, 3), {down, Pid, noconnection}, State1), - ?assertEqual(0, rabbit_fifo_q:len(State2a#rabbit_fifo.messages)), - ?assertEqual(1, lqueue:len(State2a#rabbit_fifo.returns)), - ?assertMatch(#consumer{checked_out = Ch, - status = suspected_down} - when map_size(Ch) == 0, - maps:get(Cid, State2a#rabbit_fifo.consumers)), + %% ALTERNATIVE PATH + %% the node does not come back before the timeout + + {State4, _, []} = apply(meta(Config, ?LINE), + {timeout, {consumer_down_timeout, CKey}}, + State2), + #consumer{status = {suspected_down, up}, + credit = 1, + checked_out = CH2} = maps:get(CKey, State4#rabbit_fifo.consumers), + ?assertEqual(#{}, CH2), ok. down_with_noproc_enqueuer_is_cleaned_up_test(Config) -> @@ -1161,6 +1196,44 @@ purge_with_checkout_test(Config) -> ?assertEqual(1, maps:size(Checked)), ok. +cancelled_consumer_comes_back_after_noconnection_test(Config) -> + S0 = init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => false}), + + Pid1 = test_util:fake_pid(node()), + C1Pid = test_util:fake_pid(n1@banana), + {CK1, C1} = {?LINE, {?LINE_B, C1Pid}}, + Entries = + [ + %% add a consumer + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{priority => 1})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = []}), + + %% enqueue a message + {?LINE , rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, + + {?LINE , rabbit_fifo:make_checkout(C1, cancel, #{})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = cancelled, + checked_out = Ch}}} + when map_size(Ch) == 1), + {?LINE, {down, C1Pid, noconnection}}, + ?ASSERT(#rabbit_fifo{consumers = + #{CK1 := #consumer{status = {suspected_down, cancelled}, + checked_out = Ch}}} + when map_size(Ch) == 1), + %% node comes back + {?LINE, {nodeup, n1@banana}}, + ?ASSERT(#rabbit_fifo{consumers = + #{CK1 := #consumer{status = cancelled, + checked_out = Ch}}} + when map_size(Ch) == 1) + ], + {_S1, _} = run_log(Config, S0, Entries, fun (_) -> true end), + + ok. + down_noproc_returns_checked_out_in_order_test(Config) -> S0 = test_init(?FUNCTION_NAME), %% enqueue 100 @@ -1182,29 +1255,6 @@ down_noproc_returns_checked_out_in_order_test(Config) -> ?assertEqual(lists:sort(Returns), Returns), ok. -down_noconnection_returns_checked_out_test(Config) -> - S0 = test_init(?FUNCTION_NAME), - NumMsgs = 20, - S1 = lists:foldl(fun (Num, FS0) -> - {FS, _} = enq(Config, Num, Num, Num, FS0), - FS - end, S0, lists:seq(1, NumMsgs)), - ?assertEqual(NumMsgs, rabbit_fifo_q:len(S1#rabbit_fifo.messages)), - Cid = {<<"cid">>, self()}, - {S2, #{key := CKey}, _} = checkout(Config, ?LINE, Cid, 1000, S1), - #consumer{checked_out = Checked} = maps:get(CKey, S2#rabbit_fifo.consumers), - ?assertEqual(NumMsgs, maps:size(Checked)), - %% simulate down - {S, _, _} = apply(meta(Config, 102), {down, self(), noconnection}, S2), - Returns = lqueue:to_list(S#rabbit_fifo.returns), - ?assertEqual(NumMsgs, length(Returns)), - ?assertMatch(#consumer{checked_out = Ch} - when map_size(Ch) == 0, - maps:get(CKey, S#rabbit_fifo.consumers)), - %% validate returns are in order - ?assertEqual(lists:sort(Returns), Returns), - ok. - single_active_consumer_basic_get_test(Config) -> Cid = {?FUNCTION_NAME, self()}, State0 = init(#{name => ?FUNCTION_NAME, @@ -1454,81 +1504,66 @@ single_active_returns_messages_on_noconnection_test(Config) -> R = rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), State0 = init(#{name => ?FUNCTION_NAME, queue_resource => R, - release_cursor_interval => 0, single_active_consumer_on => true}), % adding some consumers {CK1, {_, DownPid} = C1} = {?LINE, {?LINE_B, test_util:fake_pid(n1)}}, - Entries = [ - {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})} - ], - {State1, _} = run_log(Config, State0, Entries), - {State2, _} = enq(Config, 4, 1, msg1, State1), - % simulate node goes down - {State3, _, _} = apply(meta(Config, ?LINE), {down, DownPid, noconnection}, State2), - ct:pal("state3 ~p", [State3]), - %% assert the consumer is up - ?assertMatch([_], lqueue:to_list(State3#rabbit_fifo.returns)), - ?assertMatch([{_, #consumer{checked_out = Checked, - credit = 1}}] - when map_size(Checked) == 0, - rabbit_fifo:query_waiting_consumers(State3)), - + Entries = + [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {?LINE, rabbit_fifo:make_enqueue(self(), 1, msg)}, + {?LINE, {down, DownPid, noconnection}}, + {?LINE, {timeout, {consumer_down_timeout, CK1}}}, + ?ASSERT(#rabbit_fifo{consumers = Cons, + waiting_consumers = + [{CK1, #consumer{status = {suspected_down, up}}}]} + when map_size(Cons) == 0), + ?ASSERT(_, fun (#rabbit_fifo{returns = Rtns}) -> + lqueue:len(Rtns) == 1 + end) + ], + {_State1, _} = run_log(Config, State0, Entries), ok. single_active_consumer_replaces_consumer_when_down_noconnection_test(Config) -> R = rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME, utf8)), State0 = init(#{name => ?FUNCTION_NAME, queue_resource => R, - release_cursor_interval => 0, single_active_consumer_on => true}), {CK1, {_, DownPid} = C1} = {?LINE, {?LINE_B, test_util:fake_pid(n1)}}, {CK2, C2} = {?LINE, {?LINE_B, test_util:fake_pid(n2)}}, {CK3, C3} = {?LINE, {?LINE_B, test_util:fake_pid(n3)}}, - Entries = [ - {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, - {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, - {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{})}, - {?LINE, rabbit_fifo:make_enqueue(self(), 1, msg)} - ], - {State1, _} = run_log(Config, State0, Entries), - %% assert the consumer is up - ?assertMatch(#{CK1 := #consumer{status = up}}, - State1#rabbit_fifo.consumers), - - % simulate node goes down - {State2, _, _} = apply(meta(Config, ?LINE), - {down, DownPid, noconnection}, State1), - - %% assert a new consumer is in place and it is up - ?assertMatch([{CK2, #consumer{status = up, - checked_out = Ch}}] - when map_size(Ch) == 1, - maps:to_list(State2#rabbit_fifo.consumers)), - - %% the disconnected consumer has been returned to waiting - ?assert(lists:any(fun ({Con, _}) -> Con =:= CK1 end, - rabbit_fifo:query_waiting_consumers(State2))), - ?assertEqual(2, length(rabbit_fifo:query_waiting_consumers(State2))), - - % simulate node comes back up - {State3, _, _} = apply(meta(Config, 2), {nodeup, node(DownPid)}, State2), - - %% the consumer is still active and the same as before - ?assertMatch([{CK2, #consumer{status = up}}], - maps:to_list(State3#rabbit_fifo.consumers)), - % the waiting consumers should be un-suspected - ?assertEqual(2, length(rabbit_fifo:query_waiting_consumers(State3))), - lists:foreach(fun({_, #consumer{status = Status}}) -> - ?assert(Status /= suspected_down) - end, rabbit_fifo:query_waiting_consumers(State3)), + Entries = + [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, + {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{})}, + {?LINE, rabbit_fifo:make_enqueue(self(), 1, msg)}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}}), + {?LINE, {down, DownPid, noconnection}}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = {suspected_down, up}}}}), + {?LINE, {timeout, {consumer_down_timeout, CK1}}}, + ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = up, + checked_out = Ch2}}, + waiting_consumers = + [_, {CK1, #consumer{checked_out = Ch1}}]} + when map_size(Ch2) == 1 andalso + map_size(Ch1) == 0), + {?LINE, {nodeup, node(DownPid)}}, + ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = up}}, + waiting_consumers = + [ + {CK1, #consumer{status = up}}, + {CK3, #consumer{status = up}} + ]}) + ], + {_State1, _} = run_log(Config, State0, Entries), ok. single_active_consumer_all_disconnected_test(Config) -> R = rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME, utf8)), State0 = init(#{name => ?FUNCTION_NAME, queue_resource => R, - release_cursor_interval => 0, single_active_consumer_on => true}), {CK1, {_, C1Pid} = C1} = {?LINE, {?LINE_B, test_util:fake_pid(n1)}}, @@ -1537,13 +1572,24 @@ single_active_consumer_all_disconnected_test(Config) -> [ {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, - ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}}), + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = [{CK2, #consumer{status = up}}]}), {?LINE, {down, C1Pid, noconnection}}, - ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = up}}}), {?LINE, {down, C2Pid, noconnection}}, - ?ASSERT(#rabbit_fifo{consumers = C} when map_size(C) == 0), + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = {suspected_down, up}}}, + waiting_consumers = + [{CK2, #consumer{status = {suspected_down, up}}}]}), {?LINE, {nodeup, node(C2Pid)}}, - ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = up, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = {suspected_down, up}}}, + waiting_consumers = [{CK2, #consumer{status = up}}]}), + drop_effects, + {?LINE, {nodeup, node(C1Pid)}}, + {assert_effs, + fun (Effs) -> + ?ASSERT_EFF({timer, {consumer_down_timeout, K}, infinity}, + K == CK1, Effs) + end}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up, credit = 1}}}) ], {_State1, _} = run_log(Config, State0, Entries), @@ -1625,7 +1671,7 @@ query_consumers_test(Config) -> {State1, _} = run_log(Config, State0, Entries), Consumers0 = State1#rabbit_fifo.consumers, Consumer = maps:get(CK2, Consumers0), - Consumers1 = maps:put(CK2, Consumer#consumer{status = suspected_down}, + Consumers1 = maps:put(CK2, Consumer#consumer{status = {suspected_down, up}}, Consumers0), State2 = State1#rabbit_fifo{consumers = Consumers1}, @@ -1700,27 +1746,44 @@ active_flag_updated_when_consumer_suspected_unsuspected_test(Config) -> {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{})}, - {CK4, make_checkout(C4, {auto, {simple_prefetch, 1}}, #{})} + {CK4, make_checkout(C4, {auto, {simple_prefetch, 1}}, #{})}, + {?LINE, {down, Pid1, noconnection}}, + {?LINE, {down, Pid2, noconnection}}, + {?LINE, {down, Pid3, noconnection}} ], - {State1, _} = run_log(Config, State0, Entries), - {State2, _, Effects2} = apply(meta(Config, 3), - {down, Pid1, noconnection}, State1), - % 1 effect to update the metrics of each consumer - % (they belong to the same node), - % 1 more effect to monitor the node, - % 1 more decorators effect - ?assertEqual(4 + 1, length(Effects2)), + {State2, Effects2} = run_log(Config, State0, Entries), + + ?ASSERT_EFF({mod_call, rabbit_quorum_queue, update_consumer_handler, + [_QueueName, C, _, _, _, _, suspected_down, []]}, + C == C1, Effects2), + ?ASSERT_EFF({mod_call, rabbit_quorum_queue, update_consumer_handler, + [_QueueName, C, _, _, _, _, suspected_down, []]}, + C == C2, Effects2), + ?ASSERT_EFF({mod_call, rabbit_quorum_queue, update_consumer_handler, + [_QueueName, C, _, _, _, _, suspected_down, []]}, + C == C3, Effects2), + ?ASSERT_EFF({mod_call, rabbit_quorum_queue, update_consumer_handler, + [_QueueName, C, _, _, _, _, suspected_down, []]}, + C == C4, Effects2), {_, _, Effects3} = apply(meta(Config, 4), {nodeup, node(self())}, State2), - % for each consumer: 1 effect to update the metrics, - % 1 effect to monitor the consumer PID, 1 more decorators effect - ?assertEqual(4 + 4, length(Effects3)), + ?ASSERT_EFF({mod_call, rabbit_quorum_queue, update_consumer_handler, + [_QueueName, C, _, _, _, _, up, []]}, + C == C1, Effects3), + ?ASSERT_EFF({mod_call, rabbit_quorum_queue, update_consumer_handler, + [_QueueName, C, _, _, _, _, up, []]}, + C == C2, Effects3), + ?ASSERT_EFF({mod_call, rabbit_quorum_queue, update_consumer_handler, + [_QueueName, C, _, _, _, _, up, []]}, + C == C3, Effects3), + ?ASSERT_EFF({mod_call, rabbit_quorum_queue, update_consumer_handler, + [_QueueName, C, _, _, _, _, up, []]}, + C == C4, Effects3), ok. active_flag_not_updated_when_consumer_suspected_unsuspected_and_single_active_consumer_is_on_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), - release_cursor_interval => 0, single_active_consumer_on => true}), DummyFunction = fun() -> ok end, @@ -1733,20 +1796,31 @@ active_flag_not_updated_when_consumer_suspected_unsuspected_and_single_active_co {CK2, C2} = {?LINE, {?LINE_B, Pid2}}, {CK3, C3} = {?LINE, {?LINE_B, Pid2}}, {CK4, C4} = {?LINE, {?LINE_B, Pid3}}, - Entries = [ - {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, - {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, - {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{})}, - {CK4, make_checkout(C4, {auto, {simple_prefetch, 1}}, #{})} - ], - {State1, _} = run_log(Config, State0, Entries), - {State2, _, Effects2} = apply(meta(Config, 2), {down, Pid1, noconnection}, State1), - % one monitor and one consumer status update (deactivated) - ?assertEqual(2, length(Effects2)), - - {_, _, Effects3} = apply(meta(Config, 3), {nodeup, node(self())}, State2), - % for each consumer: 1 effect to monitor the consumer PID - ?assertEqual(5, length(Effects3)), + Entries = + [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, + {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{})}, + {CK4, make_checkout(C4, {auto, {simple_prefetch, 1}}, #{})}, + drop_effects, + {?LINE, {down, Pid1, noconnection}}, + {assert_effs, + fun (Effs) -> + ?ASSERT_EFF({timer, {consumer_down_timeout, K}, T}, + K == CK1 andalso is_integer(T), Effs) + end}, + drop_effects, + {?LINE, {nodeup, node()}}, + {assert_effs, + fun (Effs) -> + ?ASSERT_EFF({monitor, process, P}, P == Pid1, Effs), + ?ASSERT_EFF({timer, {consumer_down_timeout, K}, infinity}, + K == CK1, Effs), + ?ASSERT_NO_EFF({monitor, process, P}, P == Pid2, Effs), + ?ASSERT_NO_EFF({monitor, process, P}, P == Pid3, Effs) + end} + ], + {_State1, _} = run_log(Config, State0, Entries), ok. single_active_cancelled_with_unacked_test(Config) -> @@ -2067,7 +2141,7 @@ single_active_consumer_quiescing_resumes_after_cancel_test(Config) -> {?LINE, make_checkout(C2, cancel, #{})}, ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = quiescing, checked_out = Ch}}, - waiting_consumers = []} + waiting_consumers = []} when map_size(Ch) == 1), %% settle @@ -2110,7 +2184,8 @@ single_active_consumer_higher_waiting_disconnected_test(Config) -> {?LINE, {down, C2Pid, noconnection}}, ?ASSERT( #rabbit_fifo{consumers = #{CK1 := #consumer{status = quiescing}}, - waiting_consumers = [{CK2, #consumer{status = suspected_down}}]}), + waiting_consumers = + [{CK2, #consumer{status = {suspected_down, up}}}]}), %% settle {?LINE, rabbit_fifo:make_settle(CK1, [0])}, %% C1 should be reactivated @@ -2204,6 +2279,7 @@ single_active_consumer_higher_waiting_requeue_test(Config) -> {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1), ok. + single_active_consumer_quiescing_disconnected_test(Config) -> S0 = init(#{name => ?FUNCTION_NAME, queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), @@ -2229,13 +2305,14 @@ single_active_consumer_quiescing_disconnected_test(Config) -> {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 2})}, ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = quiescing}}, waiting_consumers = [{CK2, _}]}), - %% C1 is disconnected, + %% C1 is disconnected and times out {?LINE, {down, C1Pid, noconnection}}, + {?LINE, {timeout, {consumer_down_timeout, CK1}}}, ?ASSERT( #rabbit_fifo{consumers = #{CK2 := #consumer{status = up, checked_out = Ch2}}, waiting_consumers = - [{CK1, #consumer{status = suspected_down, + [{CK1, #consumer{status = {suspected_down, up}, checked_out = Ch1}}]} when map_size(Ch2) == 1 andalso map_size(Ch1) == 0), @@ -2245,7 +2322,7 @@ single_active_consumer_quiescing_disconnected_test(Config) -> #rabbit_fifo{consumers = #{CK2 := #consumer{status = up, checked_out = Ch2}}, waiting_consumers = - [{CK1, #consumer{status = suspected_down, + [{CK1, #consumer{status = {suspected_down, up}, checked_out = Ch1}}]} when map_size(Ch2) == 1 andalso map_size(Ch1) == 0), @@ -2259,6 +2336,85 @@ single_active_consumer_quiescing_disconnected_test(Config) -> ok. +single_active_consumer_quiescing_cancelled_test(Config) -> + S0 = init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => true}), + + Pid1 = test_util:fake_pid(node()), + C1Pid = test_util:fake_pid(n1@banana), + C2Pid = test_util:fake_pid(n2@banana), + % % adding some consumers + {CK1, C1} = {?LINE, {?LINE_B, C1Pid}}, + {CK2, C2} = {?LINE, {?LINE_B, C2Pid}}, + Entries = + [ + %% add a consumer + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{priority => 1})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = []}), + + %% enqueue a message + {?LINE, rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, + + %% add a consumer with a higher priority, current is quiescing + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 2})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = quiescing}}, + waiting_consumers = [{CK2, _}]}), + %% C1 is cancelled + {?LINE, make_checkout(C1, cancel, #{})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = cancelled}, + CK2 := #consumer{status = up}}, + waiting_consumers = []}) + ], + {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1), + + ok. + +single_active_consumer_cancelled_with_pending_disconnected_test(Config) -> + S0 = init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => true}), + + Pid1 = test_util:fake_pid(node()), + C1Pid = test_util:fake_pid(n1@banana), + C2Pid = test_util:fake_pid(n2@banana), + % % adding some consumers + {CK1, C1} = {?LINE, {?LINE_B, C1Pid}}, + {CK2, C2} = {?LINE, {?LINE_B, C2Pid}}, + Entries = + [ + %% add a consumer + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = []}), + + %% enqueue a message + {?LINE , rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, + + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, + %% cancel with messages pending + {?LINE, make_checkout(C1, cancel, #{})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = cancelled}, + CK2 := #consumer{status = up}}, + waiting_consumers = []}), + %% C1 is disconnected and times out + {?LINE, {down, C1Pid, noconnection}}, + ?ASSERT(#rabbit_fifo{consumers = + #{CK1 := #consumer{status = {suspected_down, cancelled}}, + CK2 := #consumer{status = up}}, + waiting_consumers = []}), + {?LINE, {timeout, {consumer_down_timeout, CK1}}}, + %% cancelled consumer should have been removed + ?ASSERT(#rabbit_fifo{consumers = + #{CK2 := #consumer{status = up}} = Cons, + waiting_consumers = []} + when map_size(Cons) == 1) + ], + {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1), + + ok. + single_active_consumer_quiescing_receives_no_further_messages_test(Config) -> S0 = init(#{name => ?FUNCTION_NAME, queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), @@ -2645,9 +2801,18 @@ run_log(Config, InitState, Entries, Invariant) -> run_log(Module, Config, InitState, Entries, Invariant) -> lists:foldl( - fun ({assert, Fun}, {Acc0, Efx0}) -> + fun + ({assert, Fun}, {Acc0, Efx0}) -> _ = Fun(Acc0), {Acc0, Efx0}; + (drop_effects,{Acc, _}) -> + {Acc, []}; + (dump_state,{Acc, _}) -> + ct:pal("State ~p", [Acc]), + {Acc, []}; + ({assert_effs, Fun}, {Acc0, Efx0}) -> + _ = Fun(Efx0), + {Acc0, Efx0}; ({Idx, E}, {Acc0, Efx0}) -> case Module:apply(meta(Config, Idx, Idx, {notify, Idx, self()}), E, Acc0) of @@ -2736,32 +2901,9 @@ machine_version_waiting_consumer_test(C) -> ?assertEqual(1, priority_queue:len(S)), ok. -convert_v2_to_v3(Config) -> - ConfigV2 = [{machine_version, 2} | Config], - ConfigV3 = [{machine_version, 3} | Config], - - Cid1 = {ctag1, self()}, - Cid2 = {ctag2, self()}, - MaxCredits = 20, - Entries = [{1, make_checkout(Cid1, {auto, 10, credited}, #{})}, - {2, make_checkout(Cid2, {auto, MaxCredits, simple_prefetch}, - #{prefetch => MaxCredits})}], - - %% run log in v2 - {State, _} = run_log(ConfigV2, test_init(?FUNCTION_NAME), Entries), - - %% convert from v2 to v3 - {#rabbit_fifo{consumers = Consumers}, ok, _} = - apply(meta(ConfigV3, 3), {machine_version, 2, 3}, State), - - ?assertEqual(2, maps:size(Consumers)), - ?assertMatch(#consumer{cfg = #consumer_cfg{credit_mode = {simple_prefetch, MaxCredits}}}, - maps:get(Cid2, Consumers)), - ok. - -convert_v3_to_v4(Config) -> - ConfigV3 = [{machine_version, 3} | Config], - ConfigV4 = [{machine_version, 4} | Config], +convert_v7_to_v8_test(Config) -> + ConfigV7 = [{machine_version, 7} | Config], + ConfigV8 = [{machine_version, 8} | Config], EPid = test_util:fake_pid(node()), Pid1 = test_util:fake_pid(node()), @@ -2769,45 +2911,29 @@ convert_v3_to_v4(Config) -> Cid2 = {ctag2, self()}, MaxCredits = 2, Entries = [ - {1, rabbit_fifo_v3:make_enqueue(EPid, 1, banana)}, - {2, rabbit_fifo_v3:make_enqueue(EPid, 2, apple)}, - {3, rabbit_fifo_v3:make_enqueue(EPid, 3, orange)}, + {1, rabbit_fifo_v7:make_enqueue(EPid, 1, banana)}, + {2, rabbit_fifo_v7:make_enqueue(EPid, 2, apple)}, + {3, rabbit_fifo_v7:make_enqueue(EPid, 3, orange)}, {4, make_checkout(Cid1, {auto, 10, credited}, #{})}, {5, make_checkout(Cid2, {auto, MaxCredits, simple_prefetch}, #{prefetch => MaxCredits})}, - {6, {down, Pid1, error}}], + {6, {down, Pid1, noconnection}}], %% run log in v3 Name = ?FUNCTION_NAME, - Init = rabbit_fifo_v3:init( + Init = rabbit_fifo_v7:init( #{name => Name, - queue_resource => rabbit_misc:r("/", queue, atom_to_binary(Name)), - release_cursor_interval => 0}), - {State, _} = run_log(rabbit_fifo_v3, ConfigV3, Init, Entries, - fun (_) -> true end), - - %% convert from v3 to v4 - {#rabbit_fifo{consumers = Consumers, - returns = Returns}, ok, _} = - apply(meta(ConfigV4, ?LINE), {machine_version, 3, 4}, State), - - ?assertEqual(1, maps:size(Consumers)), - ?assertMatch(#consumer{cfg = #consumer_cfg{credit_mode = - {simple_prefetch, MaxCredits}}}, - maps:get(Cid2, Consumers)), - ?assertNot(is_map_key(Cid1, Consumers)), - %% assert delivery_count is copied to acquired_count - #consumer{checked_out = Ch2} = maps:get(Cid2, Consumers), - ?assertMatch(#{0 := ?MSG(_, #{delivery_count := 1, - acquired_count := 1}), - 1 := ?MSG(_, #{delivery_count := 1, - acquired_count := 1})}, Ch2), - - ?assertMatch(?MSG(_, #{delivery_count := 1, - acquired_count := 1}), lqueue:get(Returns)), + queue_resource => rabbit_misc:r("/", queue, atom_to_binary(Name))}), + {StateV7, _} = run_log(rabbit_fifo_v7, ConfigV7, Init, Entries, + fun (_) -> true end), + {#rabbit_fifo{consumers = Consumers}, ok, _} = + apply(meta(ConfigV8, ?LINE), {machine_version, 7, 8}, StateV7), + ?assertMatch(#consumer{status = {suspected_down, up}}, + maps:get(Cid1, Consumers)), ok. + queue_ttl_test(C) -> QName = rabbit_misc:r(<<"/">>, queue, <<"test">>), Conf = #{name => ?FUNCTION_NAME, diff --git a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl index a275ede639d5..ccdea08a0279 100644 --- a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl @@ -217,9 +217,8 @@ lost_return_is_resent_on_applied_after_leader_change(Config) -> RaEvt, F5), %% this should resend the never applied enqueue {_, _, F7} = process_ra_events(receive_ra_events(1, 0), ClusterName, F6), - {_, _, F8} = process_ra_events(receive_ra_events(1, 0), ClusterName, F7), - ?assertEqual(0, rabbit_fifo_client:pending_size(F8)), + ?assertEqual(0, rabbit_fifo_client:pending_size(F7)), flush(), ok. From 7a8fc1c6f7d623525ffa0745040ad8dd75bd9d15 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 7 Nov 2025 14:34:13 +0000 Subject: [PATCH 15/28] QQ strict priority queue --- deps/rabbit/Makefile | 2 +- deps/rabbit/src/rabbit_fifo.erl | 84 ++++----- deps/rabbit/src/rabbit_fifo.hrl | 4 +- deps/rabbit/src/rabbit_fifo_pq.erl | 196 ++++++++++++++++++++ deps/rabbit/src/rabbit_fifo_q.erl | 21 ++- deps/rabbit/test/rabbit_fifo_SUITE.erl | 8 +- deps/rabbit/test/rabbit_fifo_pq_SUITE.erl | 148 +++++++++++++++ deps/rabbit/test/rabbit_fifo_prop_SUITE.erl | 54 ++---- deps/rabbit/test/rabbit_fifo_q_SUITE.erl | 1 - 9 files changed, 425 insertions(+), 93 deletions(-) create mode 100644 deps/rabbit/src/rabbit_fifo_pq.erl create mode 100644 deps/rabbit/test/rabbit_fifo_pq_SUITE.erl diff --git a/deps/rabbit/Makefile b/deps/rabbit/Makefile index 9e17257fd00a..17418b572915 100644 --- a/deps/rabbit/Makefile +++ b/deps/rabbit/Makefile @@ -273,7 +273,7 @@ PARALLEL_CT_SET_2_C = disk_monitor dynamic_qq unit_disk_monitor unit_file_handle PARALLEL_CT_SET_2_D = queue_length_limits queue_parallel quorum_queue_member_reconciliation rabbit_fifo rabbit_fifo_dlx rabbit_stream_coordinator PARALLEL_CT_SET_3_A = definition_import per_user_connection_channel_limit_partitions per_vhost_connection_limit_partitions policy priority_queue_recovery rabbit_fifo_v0 rabbit_stream_sac_coordinator_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_B = cluster_upgrade list_consumers_sanity_check list_queues_online_and_offline logging lqueue maintenance_mode rabbit_fifo_q rabbit_fifo_pq PARALLEL_CT_SET_3_C = cli_forget_cluster_node feature_flags_v2 mc_unit 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/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 1d5cfa81e558..c107767ce7b3 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -2,8 +2,9 @@ %% License, v. 2.0. If a copy of the MPL was not distributed with this %% file, You can obtain one at https://mozilla.org/MPL/2.0/. %% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. - +%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. +%% The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. +%% All rights reserved. -module(rabbit_fifo). -behaviour(ra_machine). @@ -339,9 +340,9 @@ apply_(#{index := Idx} = Meta, State0 = add_bytes_return(Header, State00), Con = Con0#consumer{checked_out = maps:remove(MsgId, Checked0), credit = increase_credit(Con0, 1)}, - State1 = State0#?STATE{messages = rabbit_fifo_q:in(no, - ?MSG(Idx, Header), - Messages)}, + State1 = State0#?STATE{messages = rabbit_fifo_pq:in(4, + ?MSG(Idx, Header), + Messages)}, State2 = update_or_remove_con(Meta, ConsumerKey, Con, State1), {State3, Effects} = activate_next_consumer({State2, []}), checkout(Meta, State0, State3, Effects); @@ -470,7 +471,7 @@ apply_(#{index := Idx} = Meta, apply_(#{index := Index}, #purge{}, #?STATE{messages_total = Total} = State0) -> NumReady = messages_ready(State0), - State1 = State0#?STATE{messages = rabbit_fifo_q:new(), + State1 = State0#?STATE{messages = rabbit_fifo_pq:new(), messages_total = Total - NumReady, returns = lqueue:new(), msg_bytes_enqueue = 0 @@ -707,7 +708,7 @@ live_indexes(#?STATE{cfg = #cfg{}, messages = Messages, consumers = Consumers, dlx = #?DLX{discards = Discards}}) -> - MsgsIdxs = rabbit_fifo_q:indexes(Messages), + MsgsIdxs = rabbit_fifo_pq:indexes(Messages), DlxIndexes = lqueue:fold(fun (?TUPLE(_, Msg), Acc) -> I = get_msg_idx(Msg), [I | Acc] @@ -759,8 +760,17 @@ convert_v7_to_v8(#{} = _Meta, StateV7) -> (_CKey, C) -> C end, Cons0), + Msgs = element(#?STATE.messages, StateV7), + {Hi, No} = rabbit_fifo_q:to_queues(Msgs), + Pq0 = queue:fold(fun (I, Acc) -> + rabbit_fifo_pq:in(9, I, Acc) + end, rabbit_fifo_pq:new(), Hi), + Pq = queue:fold(fun (I, Acc) -> + rabbit_fifo_pq:in(4, I, Acc) + end, Pq0, No), StateV8 = StateV7, StateV8#?STATE{discarded_bytes = 0, + messages = Pq, consumers = Cons, unused_0 = ?NIL}. @@ -934,8 +944,11 @@ overview(#?STATE{consumers = Cons, #{} end, MsgsRet = lqueue:len(Returns), - #{num_hi := MsgsHi, - num_no := MsgsNo} = rabbit_fifo_q:overview(Messages), + %% TODO emit suitable overview metrics + #{ + % num_hi := MsgsHi, + % num_no := MsgsNo + } = rabbit_fifo_pq:overview(Messages), Overview = #{type => ?STATE, config => Conf, @@ -944,13 +957,14 @@ overview(#?STATE{consumers = Cons, num_checked_out => num_checked_out(State), num_enqueuers => maps:size(Enqs), num_ready_messages => messages_ready(State), - num_ready_messages_high => MsgsHi, - num_ready_messages_normal => MsgsNo, + % num_ready_messages_high => MsgsHi, + % num_ready_messages_normal => MsgsNo, num_ready_messages_return => MsgsRet, num_messages => messages_total(State), enqueue_message_bytes => EnqueueBytes, checkout_message_bytes => CheckoutBytes, - discarded_bytes => DiscardedBytes + discarded_bytes => DiscardedBytes, + smallest_raft_index => smallest_raft_index(State) }, DlxOverview = dlx_overview(DlxState), maps:merge(maps:merge(Overview, DlxOverview), SacOverview). @@ -1427,7 +1441,7 @@ usage(Name) when is_atom(Name) -> messages_ready(#?STATE{messages = M, returns = R}) -> - rabbit_fifo_q:len(M) + lqueue:len(R). + rabbit_fifo_pq:len(M) + lqueue:len(R). messages_total(#?STATE{messages_total = Total, dlx = DlxState}) -> @@ -1736,7 +1750,7 @@ maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, PTag = priority_tag(RawMsg), State = State0#?STATE{msg_bytes_enqueue = Enqueue + Size, messages_total = Total + 1, - messages = rabbit_fifo_q:in(PTag, Msg, Messages) + messages = rabbit_fifo_pq:in(PTag, Msg, Messages) }, {ok, State, Effects}; maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, @@ -1769,7 +1783,7 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, PTag = priority_tag(RawMsg), State = State0#?STATE{msg_bytes_enqueue = BytesEnqueued + Size, messages_total = Total + 1, - messages = rabbit_fifo_q:in(PTag, Msg, Messages), + messages = rabbit_fifo_pq:in(PTag, Msg, Messages), enqueuers = Enqueuers0#{From => Enq}, msg_cache = MsgCache }, @@ -2158,7 +2172,7 @@ take_next_msg(#?STATE{returns = Returns0, {{value, NextMsg}, Returns} -> {NextMsg, State#?STATE{returns = Returns}}; {empty, _} -> - case rabbit_fifo_q:out(Messages0) of + case rabbit_fifo_pq:out(Messages0) of empty -> empty; {Msg, Messages} -> @@ -2170,7 +2184,7 @@ get_next_msg(#?STATE{returns = Returns0, messages = Messages0}) -> case lqueue:get(Returns0, empty) of empty -> - rabbit_fifo_q:get(Messages0); + rabbit_fifo_pq:get(Messages0); Msg -> Msg end. @@ -2283,7 +2297,7 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> checkout_one(Meta, ExpiredMsg, InitState#?STATE{service_queue = SQ1}, Effects1); {empty, _} -> - case rabbit_fifo_q:len(Messages0) of + case rabbit_fifo_pq:len(Messages0) of 0 -> {nochange, ExpiredMsg, InitState, Effects1}; _ -> @@ -2869,7 +2883,7 @@ smallest_raft_index(#?STATE{messages = Messages, SmallestDlxRaIdx = lqueue:fold(fun (?TUPLE(_, Msg), Acc) -> min(get_msg_idx(Msg), Acc) end, undefined, Discards), - SmallestMsgsRaIdx = rabbit_fifo_q:get_lowest_index(Messages), + SmallestMsgsRaIdx = rabbit_fifo_pq:get_lowest_index(Messages), %% scan consumers and returns queue here instead smallest_checked_out(State, min(SmallestDlxRaIdx, SmallestMsgsRaIdx)). @@ -3027,14 +3041,13 @@ priority_tag(Msg) -> case mc:is(Msg) of true -> case mc:priority(Msg) of - P when is_integer(P) andalso - P > 4 -> - hi; + P when is_integer(P) -> + min(P, 31); _ -> - no + 4 end; false -> - no + 4 end. do_snapshot(MacVer, Ts, Ch, @@ -3260,29 +3273,6 @@ dlx_apply(_, Cmd, DLH, State) -> %% down: 90 bytes %% enqueue overhead 210 -% messages_get_next_msg(#messages{returns = Returns0, -% messages = Messages0}) -> -% case lqueue:get(Returns0, empty) of -% empty -> -% rabbit_fifo_q:get(Messages0); -% Msg -> -% Msg -% end. - -% messages_take_next_msg(#messages{returns = Returns0, -% messages = Messages0} = Msgs) -> -% case lqueue:out(Returns0) of -% {{value, NextMsg}, Returns} -> -% {NextMsg, Msgs#messages{returns = Returns}}; -% {empty, _} -> -% case rabbit_fifo_q:out(Messages0) of -% empty -> -% empty; -% {?MSG(_RaftIdx, _) = Msg, Messages} -> -% {Msg, Msgs#messages{messages = Messages}} -% end -% end. - ensure_worker_started(QRef, #?DLX{consumer = undefined}) -> start_worker(QRef); ensure_worker_started(QRef, #?DLX{consumer = #dlx_consumer{pid = Pid}}) -> diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index 656a6e997b7d..efeab29c32bb 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -207,7 +207,7 @@ -record(messages, { - messages = rabbit_fifo_q:new() :: rabbit_fifo_q:state(), + messages = rabbit_fifo_pq:new() :: rabbit_fifo_pq:state(), messages_total = 0 :: non_neg_integer(), % queue of returned msg_in_ids - when checking out it picks from returns = lqueue:new() :: lqueue:lqueue(term()) @@ -233,7 +233,7 @@ -record(rabbit_fifo, {cfg :: #cfg{}, % unassigned messages - messages = rabbit_fifo_q:new() :: rabbit_fifo_q:state(), + messages = rabbit_fifo_pq:new() :: rabbit_fifo_pq:state(), messages_total = 0 :: non_neg_integer(), % queue of returned msg_in_ids - when checking out it picks from returns = lqueue:new() :: lqueue:lqueue(term()), diff --git a/deps/rabbit/src/rabbit_fifo_pq.erl b/deps/rabbit/src/rabbit_fifo_pq.erl new file mode 100644 index 000000000000..1efb1303092b --- /dev/null +++ b/deps/rabbit/src/rabbit_fifo_pq.erl @@ -0,0 +1,196 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. +%% The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. +%% All rights reserved. +-module(rabbit_fifo_pq). + +-include("rabbit_fifo.hrl"). +-export([ + new/0, + in/3, + out/1, + get/1, + len/1, + to_list/1, + from_list/1, + from_lqueue/1, + indexes/1, + get_lowest_index/1, + overview/1 + ]). + +% -define(NON_EMPTY, {_, [_|_]}). +-define(EMPTY, {[], []}). + +%% a weighted priority queue with only two priorities +-type priority() :: 0..31. +-type queue() :: {list(msg()), list(msg())}. + +-record(?MODULE, {buckets = #{} :: #{priority() => queue()}, + len = 0 :: non_neg_integer(), + bitmap = 0 :: integer()}). + +-opaque state() :: #?MODULE{}. + +-export_type([state/0, + priority/0]). + +-spec new() -> state(). +new() -> + #?MODULE{}. + +-spec in(priority(), msg(), state()) -> state(). +in(Priority0, Item, #?MODULE{buckets = Buckets0, + bitmap = Bitmap0, + len = Len} = State) + when Priority0 >= 0 andalso + Priority0 =< 31 -> + %% invert priority + Priority = 31 - Priority0, + case Buckets0 of + #{Priority := Queue0} -> + %% there are messages for the priority already + State#?MODULE{buckets = Buckets0#{Priority => in(Item, Queue0)}, + len = Len + 1}; + _ -> + Bitmap = Bitmap0 bor (1 bsl Priority), + %% there are no messages for the priority + State#?MODULE{buckets = Buckets0#{Priority => in(Item, ?EMPTY)}, + bitmap = Bitmap, + len = Len + 1} + end. + +first_set_bit(0) -> + 32; +first_set_bit(Bitmap) -> + count_trailing(Bitmap band -Bitmap). + +-define(DEBRUIJN_SEQ, 16#077CB531). +-define(DEBRUIJN_LOOKUP, + {0, 1, 28, 2, 29, 14, 24, 3, 30, 22, 20, 15, 25, 17, 4, 8, + 31, 27, 13, 23, 21, 19, 16, 7, 26, 12, 18, 6, 11, 5, 10, 9}). + +count_trailing(N) -> + Lookup = ((N * ?DEBRUIJN_SEQ) bsr 27) band 31, + element(Lookup + 1, ?DEBRUIJN_LOOKUP). + +-spec out(state()) -> + empty | {msg(), state()}. +out(#?MODULE{len = 0}) -> + empty; +out(#?MODULE{buckets = Buckets, + len = Len, + bitmap = Bitmap0} = State) -> + Priority = first_set_bit(Bitmap0), + #{Priority := Q0} = Buckets, + Msg = peek(Q0), + case drop(Q0) of + ?EMPTY -> + %% zero bit in bitmap + %% as we know the bit is set we just need to xor rather than + %% create a mask then xor + Bitmap = Bitmap0 bxor (1 bsl Priority), + {Msg, State#?MODULE{buckets = maps:remove(Priority, Buckets), + len = Len - 1, + bitmap = Bitmap}}; + Q -> + {Msg, State#?MODULE{buckets = maps:put(Priority, Q, Buckets), + len = Len - 1}} + end. + +-spec get(state()) -> empty | msg(). +get(#?MODULE{len = 0}) -> + empty; +get(#?MODULE{buckets = Buckets, + bitmap = Bitmap}) -> + Priority = first_set_bit(Bitmap), + #{Priority := Q0} = Buckets, + peek(Q0). + +-spec len(state()) -> non_neg_integer(). +len(#?MODULE{len = Len}) -> + Len. + +-spec from_list([{priority(), term()}]) -> state(). +from_list(Items) when is_list(Items) -> + lists:foldl(fun ({P, Item}, Acc) -> + in(P, Item, Acc) + end, new(), Items). + +-spec to_list(state()) -> [msg()]. +to_list(State) -> + to_list(out(State), []). + +-spec from_lqueue(lqueue:lqueue(msg())) -> state(). +from_lqueue(LQ) -> + lqueue:fold(fun (Item, Acc) -> + in(4, Item, Acc) + end, new(), LQ). + +-spec indexes(state()) -> [ra:index()]. +indexes(#?MODULE{buckets = Buckets}) -> + maps:fold( + fun (_P, {L1, L2}, Acc0) -> + Acc = lists:foldl(fun msg_idx_fld/2, Acc0, L1), + lists:foldl(fun msg_idx_fld/2, Acc, L2) + end, [], Buckets). + +-spec get_lowest_index(state()) -> undefined | ra:index(). +get_lowest_index(#?MODULE{len = 0}) -> + undefined; +get_lowest_index(#?MODULE{buckets = Buckets}) -> + lists:min( + maps:fold(fun (_, Q, Acc) -> + case peek(Q) of + empty -> + Acc; + Msg -> + [msg_idx(Msg) | Acc] + end + end, [], Buckets)). + +-spec overview(state()) -> + #{len := non_neg_integer(), + num_active_priorities := 0..32, + lowest_index := ra:index()}. +overview(#?MODULE{len = Len, + buckets = Buckets} = State) -> + #{len => Len, + num_active_priorities => map_size(Buckets), + lowest_index => get_lowest_index(State)}. + +%% internals + +%% invariant, if the queue is non empty so is the Out (right) list. +in(X, ?EMPTY) -> + {[], [X]}; +in(X, {In, Out}) -> + {[X | In], Out}. + +peek(?EMPTY) -> + empty; +peek({_, [H | _]}) -> + H. + +drop({In, [_]}) -> + %% the last Out one + {[], lists:reverse(In)}; +drop({In, [_ | Out]}) -> + {In, Out}. + +msg_idx_fld(Msg, Acc) when is_list(Acc) -> + [msg_idx(Msg) | Acc]. + +msg_idx(?MSG(Idx, _Header)) -> + Idx; +msg_idx(Packed) when ?IS_PACKED(Packed) -> + ?PACKED_IDX(Packed). + +to_list(empty, Acc) -> + lists:reverse(Acc); +to_list({Item, State}, Acc) -> + to_list(out(State), [Item | Acc]). + diff --git a/deps/rabbit/src/rabbit_fifo_q.erl b/deps/rabbit/src/rabbit_fifo_q.erl index 9bca67135154..1ab5e83a048f 100644 --- a/deps/rabbit/src/rabbit_fifo_q.erl +++ b/deps/rabbit/src/rabbit_fifo_q.erl @@ -1,3 +1,10 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. +%% The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. +%% All rights reserved. -module(rabbit_fifo_q). -include("rabbit_fifo.hrl"). @@ -7,6 +14,7 @@ out/1, get/1, len/1, + to_queues/1, from_lqueue/1, indexes/1, get_lowest_index/1, @@ -19,8 +27,10 @@ %% a weighted priority queue with only two priorities --record(?MODULE, {hi = ?EMPTY :: {list(msg()), list(msg())}, %% high - no = ?EMPTY :: {list(msg()), list(msg())}, %% normal +-type queue() :: {list(msg()), list(msg())}. + +-record(?MODULE, {hi = ?EMPTY :: queue(), %% high + no = ?EMPTY :: queue(), %% normal len = 0 :: non_neg_integer(), dequeue_counter = 0 :: non_neg_integer()}). @@ -76,6 +86,13 @@ get(#?MODULE{} = State) -> len(#?MODULE{len = Len}) -> Len. +-spec to_queues(state()) -> {High :: queue(), + Normal :: queue()}. +to_queues(#?MODULE{hi = Hi, + no = No}) -> + {Hi, No}. + + -spec from_lqueue(lqueue:lqueue(msg())) -> state(). from_lqueue(LQ) -> lqueue:fold(fun (Item, Acc) -> diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 33348f6f46cc..65b9336e3f55 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -857,7 +857,7 @@ cancelled_checkout_out_test(Config) -> % cancelled checkout should not return pending messages to queue {State2, _, _} = apply(meta(Config, 4), rabbit_fifo:make_checkout(Cid, cancel, #{}), State1), - ?assertEqual(1, rabbit_fifo_q:len(State2#rabbit_fifo.messages)), + ?assertEqual(1, rabbit_fifo_pq:len(State2#rabbit_fifo.messages)), ?assertEqual(0, lqueue:len(State2#rabbit_fifo.returns)), ?assertEqual(0, priority_queue:len(State2#rabbit_fifo.service_queue)), @@ -1057,7 +1057,7 @@ enqueued_msg_with_delivery_count_test(Config) -> {at_most_once, {somemod, somefun, [somearg]}}}), Mc = mc:set_annotation(delivery_count, 2, mk_mc(<<"first">>)), {#rabbit_fifo{messages = Msgs}, _} = enq(Config, 1, 1, Mc, State00), - ?assertMatch(?MSG(_, #{delivery_count := 2}), rabbit_fifo_q:get(Msgs)), + ?assertMatch(?MSG(_, #{delivery_count := 2}), rabbit_fifo_pq:get(Msgs)), ok. get_log_eff(Effs) -> @@ -1141,7 +1141,7 @@ duplicate_delivery_test(Config) -> {#rabbit_fifo{messages = Messages} = State, _} = enq(Config, 2, 1, first, State0), ?assertEqual(1, rabbit_fifo:query_messages_total(State)), - ?assertEqual(1, rabbit_fifo_q:len(Messages)), + ?assertEqual(1, rabbit_fifo_pq:len(Messages)), ok. state_enter_monitors_and_notifications_test(Config) -> @@ -1241,7 +1241,7 @@ down_noproc_returns_checked_out_in_order_test(Config) -> {FS, _} = enq(Config, Num, Num, Num, FS0), FS end, S0, lists:seq(1, 100)), - ?assertEqual(100, rabbit_fifo_q:len(S1#rabbit_fifo.messages)), + ?assertEqual(100, rabbit_fifo_pq:len(S1#rabbit_fifo.messages)), Cid = {<<"cid">>, self()}, {S2, #{key := CKey}, _} = checkout(Config, ?LINE, Cid, 1000, S1), #consumer{checked_out = Checked} = maps:get(CKey, S2#rabbit_fifo.consumers), diff --git a/deps/rabbit/test/rabbit_fifo_pq_SUITE.erl b/deps/rabbit/test/rabbit_fifo_pq_SUITE.erl new file mode 100644 index 000000000000..4634d165ce21 --- /dev/null +++ b/deps/rabbit/test/rabbit_fifo_pq_SUITE.erl @@ -0,0 +1,148 @@ +-module(rabbit_fifo_pq_SUITE). + +-compile(nowarn_export_all). +-compile(export_all). + +-include_lib("proper/include/proper.hrl"). +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("rabbit/src/rabbit_fifo.hrl"). + +all() -> + [ + {group, tests} + ]. + + +all_tests() -> + [ + basics, + property + ]. + + +groups() -> + [ + {tests, [parallel], 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) -> + Config. + +end_per_testcase(_TestCase, _Config) -> + ok. + +%%%=================================================================== +%%% Test cases +%%%=================================================================== + +-define(MSG(L), ?MSG(L, L)). + +basics(_Config) -> + Q0 = rabbit_fifo_pq:new(), + Q1 = lists:foldl( + fun ({P, I}, Q) -> + rabbit_fifo_pq:in(P, I, Q) + end, Q0, [ + {1, ?MSG(1)}, + {2, ?MSG(2)}, + {3, ?MSG(3)}, + {4, ?MSG(4)}, + {5, ?MSG(5)} + ]), + + ?assertEqual([1,2,3,4,5], lists:sort(rabbit_fifo_pq:indexes(Q1))), + ?assertMatch(#{len := 5, + num_active_priorities := 5, + lowest_index := 1}, rabbit_fifo_pq:overview(Q1)), + {?MSG(5), Q2} = rabbit_fifo_pq:out(Q1), + {?MSG(4), Q3} = rabbit_fifo_pq:out(Q2), + {?MSG(3), Q4} = rabbit_fifo_pq:out(Q3), + {?MSG(2), Q5} = rabbit_fifo_pq:out(Q4), + {?MSG(1), Q6} = rabbit_fifo_pq:out(Q5), + empty = rabbit_fifo_pq:out(Q6), + ok. + +hi_is_prioritised(_Config) -> + Q0 = rabbit_fifo_q:new(), + %% when `hi' has a lower index than the next 'no' then it is still + %% prioritied (as this is safe to do). + Q1 = lists:foldl( + fun ({P, I}, Q) -> + rabbit_fifo_q:in(P, I, Q) + end, Q0, [ + {hi, ?MSG(1)}, + {hi, ?MSG(2)}, + {hi, ?MSG(3)}, + {hi, ?MSG(4)}, + {no, ?MSG(5)} + ]), + {?MSG(1), Q2} = rabbit_fifo_q:out(Q1), + {?MSG(2), Q3} = rabbit_fifo_q:out(Q2), + {?MSG(3), Q4} = rabbit_fifo_q:out(Q3), + {?MSG(4), Q5} = rabbit_fifo_q:out(Q4), + {?MSG(5), Q6} = rabbit_fifo_q:out(Q5), + empty = rabbit_fifo_q:out(Q6), + ok. + +get_lowest_index(_Config) -> + Q0 = rabbit_fifo_q:new(), + Q1 = rabbit_fifo_q:in(hi, ?MSG(1, ?LINE), Q0), + Q2 = rabbit_fifo_q:in(no, ?MSG(2, ?LINE), Q1), + Q3 = rabbit_fifo_q:in(no, ?MSG(3, ?LINE), Q2), + {_, Q4} = rabbit_fifo_q:out(Q3), + {_, Q5} = rabbit_fifo_q:out(Q4), + {_, Q6} = rabbit_fifo_q:out(Q5), + + ?assertEqual(undefined, rabbit_fifo_q:get_lowest_index(Q0)), + ?assertEqual(1, rabbit_fifo_q:get_lowest_index(Q1)), + ?assertEqual(1, rabbit_fifo_q:get_lowest_index(Q2)), + ?assertEqual(1, rabbit_fifo_q:get_lowest_index(Q3)), + ?assertEqual(2, rabbit_fifo_q:get_lowest_index(Q4)), + ?assertEqual(3, rabbit_fifo_q:get_lowest_index(Q5)), + ?assertEqual(undefined, rabbit_fifo_q:get_lowest_index(Q6)). + +property(_Config) -> + run_proper( + fun () -> + ?FORALL(Ops, op_gen(256), + queue_prop(Ops)) + end, [], 25), + ok. + +queue_prop(Ops) -> + %% create the expected output order + SortedOps = lists:append([begin + [I || {Pr, _} = I <- Ops, Pr == X] + end || X <- lists:seq(31, 0, -1)]), + + Sut0 = rabbit_fifo_pq:from_list(Ops), + Out = rabbit_fifo_pq:to_list(Sut0), + [element(2, O) || O <- SortedOps] == Out. + +%%% helpers + +-type item() :: {rabbit_fifo_pq:priority(), integer()}. +op_gen(Size) -> + ?LET(Ops, resize(Size, list(item())), Ops). + +run_proper(Fun, Args, NumTests) -> + ?assert( + proper:counterexample( + erlang:apply(Fun, Args), + [{numtests, NumTests}, + {on_output, fun(".", _) -> ok; % don't print the '.'s on new lines + (F, A) -> ct:pal(?LOW_IMPORTANCE, F, A) + end}])). diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index 36dc3f2c8b1f..dbaa0e1469d4 100644 --- a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl @@ -10,7 +10,7 @@ -include_lib("rabbit_common/include/rabbit_framing.hrl"). -include_lib("rabbit_common/include/rabbit.hrl"). --define(MACHINE_VERSION, 4). +-define(MACHINE_VERSION, 8). %%%=================================================================== %%% Common Test callbacks @@ -394,7 +394,6 @@ scenario21(_Config) -> rabbit_fifo:make_settle(C1, [1]) ], run_snapshot_test(#{name => ?FUNCTION_NAME, - release_cursor_interval => 1, dead_letter_handler => {at_most_once, {?MODULE, banana, []}}}, Commands), ok. @@ -411,7 +410,6 @@ scenario22(_Config) -> make_enqueue(E,5,msg(<<"5">>)) ], run_snapshot_test(#{name => ?FUNCTION_NAME, - release_cursor_interval => 1, max_length => 3, dead_letter_handler => {at_most_once, {?MODULE, banana, []}}}, Commands), @@ -432,7 +430,6 @@ scenario24(_Config) -> {down, E, noconnection} %% 7 ], run_snapshot_test(#{name => ?FUNCTION_NAME, - release_cursor_interval => 0, deliver_limit => undefined, max_length => 3, overflow_strategy => drop_head, @@ -459,7 +456,6 @@ scenario25(_Config) -> ], run_snapshot_test(#{name => ?FUNCTION_NAME, max_bytes => undefined, - release_cursor_interval => 0, deliver_limit => undefined, overflow_strategy => drop_head, dead_letter_handler => {at_most_once, {?MODULE, banana, []}} @@ -489,7 +485,6 @@ scenario26(_Config) -> {down, C1Pid, noconnection} ], run_snapshot_test(#{name => ?FUNCTION_NAME, - release_cursor_interval => 0, deliver_limit => undefined, max_length => 8, overflow_strategy => drop_head, @@ -504,7 +499,7 @@ scenario28(_Config) -> delivery_limit => undefined, max_in_memory_bytes => undefined, max_length => 1,name => ?FUNCTION_NAME,overflow_strategy => drop_head, - release_cursor_interval => 100,single_active_consumer_on => false}, + single_active_consumer_on => false}, Commands = [ make_enqueue(E,2,msg( <<>>)), make_enqueue(E,3,msg( <<>>)), @@ -558,7 +553,6 @@ scenario27(_Config) -> ?assert( single_active_prop(#{name => ?FUNCTION_NAME, max_bytes => undefined, - release_cursor_interval => 100, deliver_limit => 1, max_length => 1, max_in_memory_length => 8, @@ -581,7 +575,6 @@ scenario30(_Config) -> make_enqueue(E,3,msg(<<>>)) %% 5 ], run_snapshot_test(#{name => ?FUNCTION_NAME, - release_cursor_interval => 0, deliver_limit => undefined, max_length => 1, max_in_memory_length => 1, @@ -613,7 +606,6 @@ scenario31(_Config) -> {purge} %% 4 ], run_snapshot_test(#{name => ?FUNCTION_NAME, - release_cursor_interval => 0, deliver_limit => undefined, overflow_strategy => drop_head, dead_letter_handler => {at_most_once, {?MODULE, banana, []}} @@ -631,7 +623,6 @@ scenario32(_Config) -> make_enqueue(E1,4,msg(<<0,0,0,0>>)) %% 3 ], run_snapshot_test(#{name => ?FUNCTION_NAME, - release_cursor_interval => 0, max_length => 3, deliver_limit => undefined, overflow_strategy => drop_head, @@ -656,7 +647,6 @@ scenario29(_Config) -> {down, E, noconnection} %% 8 ], run_snapshot_test(#{name => ?FUNCTION_NAME, - release_cursor_interval => 0, deliver_limit => undefined, max_length => 5, max_in_memory_length => 1, @@ -679,7 +669,6 @@ scenario23(_Config) -> make_enqueue(E,4,msg(<<>>)) %% 6 ], run_snapshot_test(#{name => ?FUNCTION_NAME, - release_cursor_interval => 0, deliver_limit => undefined, max_length => 2, overflow_strategy => drop_head, @@ -1046,10 +1035,11 @@ single_active_ordering_03(_Config) -> make_checkout(C2, {auto,1,simple_prefetch}), make_settle(C1, [0]), make_checkout(C1, cancel), - {down, C1Pid, noconnection} + {down, C1Pid, noconnection}, + {timeout, {consumer_down_timeout, C1}} + ], - Conf0 = config(?FUNCTION_NAME, 0, 0, true, 0), - Conf = Conf0#{release_cursor_interval => 100}, + Conf = config(?FUNCTION_NAME, 0, 0, true, 0), Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), try run_log(test_init(Conf), Entries) of @@ -1137,7 +1127,7 @@ is_same_otp_version(ConfigOrNode) -> OurOTP =:= OtherOTP. two_nodes(Node) -> - Size = 300, + Size = 100, run_proper( fun () -> ?FORALL({Length, Bytes, DeliveryLimit, SingleActive}, @@ -1445,8 +1435,7 @@ config(Name, Length, MaxBytes, SingleActive, DeliveryLimit, map_max(0) -> undefined; map_max(N) -> N. -max_length_prop(Conf0, Commands) -> - Conf = Conf0#{release_cursor_interval => 100}, +max_length_prop(Conf, Commands) -> Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), Invariant = fun (#rabbit_fifo{cfg = #cfg{max_length = MaxLen}} = S) -> @@ -1486,8 +1475,7 @@ validate_idx_order(Idxs, ReleaseCursorIdx) -> %% * if new consumer subscribes, messages are checked out to new consumer %% * if dlx_worker fails receiving DOWN, messages are still in state. -single_active_prop(Conf0, Commands, ValidateOrder) -> - Conf = Conf0#{release_cursor_interval => 100}, +single_active_prop(Conf, Commands, ValidateOrder) -> Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), %% invariant: there can only be one active consumer at any one time @@ -1518,21 +1506,19 @@ single_active_prop(Conf0, Commands, ValidateOrder) -> false end. -different_nodes_prop(Node, Conf0, Commands) -> - Conf = Conf0#{release_cursor_interval => 100}, +different_nodes_prop(Node, Conf, Commands) -> Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), InitState = test_init(Conf), Fun = fun(_) -> true end, - MachineVersion = 6, + MachineVersion = 8, {State1, _Effs1} = run_log(InitState, Entries, Fun, MachineVersion), {State2, _Effs2} = erpc:call(Node, ?MODULE, run_log, [InitState, Entries, Fun, MachineVersion]), State1 =:= State2. -messages_total_prop(Conf0, Commands) -> - Conf = Conf0#{release_cursor_interval => 100}, +messages_total_prop(Conf, Commands) -> Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), InitState = test_init(Conf), @@ -1545,7 +1531,7 @@ messages_total_invariant() -> returns = R, dlx = #rabbit_fifo_dlx{discards = D, consumer = DlxCon}} = S) -> - Base = rabbit_fifo_q:len(M) + lqueue:len(R), + Base = rabbit_fifo_pq:len(M) + lqueue:len(R), Tot0 = maps:fold(fun (_, #consumer{checked_out = Ch}, Acc) -> Acc + map_size(Ch) end, Base, C), @@ -1566,8 +1552,7 @@ messages_total_invariant() -> end end. -simple_prefetch_prop(Conf0, Commands, WithCheckoutCancel) -> - Conf = Conf0#{release_cursor_interval => 100}, +simple_prefetch_prop(Conf, Commands, WithCheckoutCancel) -> Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), InitState = test_init(Conf), @@ -1612,12 +1597,11 @@ valid_simple_prefetch(Prefetch, _, CheckedOut, false, CId) valid_simple_prefetch(_, _, _, _, _) -> true. -upgrade_prop(Conf0, Commands) -> - FromVersion = 3, - ToVersion = 4, +upgrade_prop(Conf, Commands) -> + FromVersion = 7, + ToVersion = 8, FromMod = rabbit_fifo:which_module(FromVersion), ToMod = rabbit_fifo:which_module(ToVersion), - Conf = Conf0#{release_cursor_interval => 0}, Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), InitState = test_init_v(Conf, FromVersion), @@ -1931,8 +1915,7 @@ expand(Ops, Config, EnqFun) -> InitConfig0 = #{name => proper, queue_resource => #resource{virtual_host = <<"/">>, kind = queue, - name = <<"blah">>}, - release_cursor_interval => 1}, + name = <<"blah">>}}, InitConfig = case Config of #{dead_letter_handler := at_least_once} -> %% Configure rabbit_fifo config with at_least_once so that @@ -2217,7 +2200,6 @@ test_init(Conf) -> test_init(Mod, Conf) -> Default = #{queue_resource => blah, - release_cursor_interval => 0, metrics_handler => {?MODULE, metrics_handler, []}}, Mod:init(maps:merge(Default, Conf)). diff --git a/deps/rabbit/test/rabbit_fifo_q_SUITE.erl b/deps/rabbit/test/rabbit_fifo_q_SUITE.erl index 8edc27e4a979..18b33dcf10b6 100644 --- a/deps/rabbit/test/rabbit_fifo_q_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_q_SUITE.erl @@ -16,7 +16,6 @@ all() -> all_tests() -> [ - hi, basics, hi_is_prioritised, get_lowest_index, From 3c38b14e95282c438e2562c47f866dd9b2566e79 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 10 Nov 2025 16:48:31 +0000 Subject: [PATCH 16/28] refactor how tests get message counts --- deps/rabbitmq_ct_helpers/src/queue_utils.erl | 34 ++++++++++++++------ 1 file changed, 24 insertions(+), 10 deletions(-) diff --git a/deps/rabbitmq_ct_helpers/src/queue_utils.erl b/deps/rabbitmq_ct_helpers/src/queue_utils.erl index d2c69792fde0..802c31bd6acd 100644 --- a/deps/rabbitmq_ct_helpers/src/queue_utils.erl +++ b/deps/rabbitmq_ct_helpers/src/queue_utils.erl @@ -26,27 +26,29 @@ wait_for_messages_ready(Servers, QName, Ready) -> wait_for_messages(Servers, QName, Ready, - fun rabbit_fifo:query_messages_ready/1, + num_ready_messages, ?WFM_DEFAULT_NUMS). wait_for_messages_pending_ack(Servers, QName, Ready) -> wait_for_messages(Servers, QName, Ready, - fun rabbit_fifo:query_messages_checked_out/1, + num_checked_out, ?WFM_DEFAULT_NUMS). wait_for_messages_total(Servers, QName, Total) -> wait_for_messages(Servers, QName, Total, - fun rabbit_fifo:query_messages_total/1, + num_messages, ?WFM_DEFAULT_NUMS). -wait_for_messages(Servers, QName, Total, Fun) -> - wait_for_messages(Servers, QName, Total, Fun, ?WFM_DEFAULT_NUMS). +wait_for_messages(Servers, QName, Total, Key) -> + wait_for_messages(Servers, QName, Total, Key, ?WFM_DEFAULT_NUMS). -wait_for_messages(Servers, QName, Number, Fun, 0) -> - Msgs = dirty_query(Servers, QName, Fun), +wait_for_messages(Servers, QName, Number, MetricKey, 0) -> + ServerIds = [{QName, S} || S <- Servers], + Msgs = query_messages(ServerIds, MetricKey), ?assertEqual([Number || _ <- lists:seq(1, length(Servers))], Msgs); -wait_for_messages(Servers, QName, Number, Fun, N) -> - Msgs = dirty_query(Servers, QName, Fun), +wait_for_messages(Servers, QName, Number, MetricKey, N) -> + ServerIds = [{QName, S} || S <- Servers], + Msgs = query_messages(ServerIds, MetricKey), ct:log("Got messages ~tp ~tp", [QName, Msgs]), %% hack to allow the check to succeed in mixed versions clusters if at %% least one node matches the criteria rather than all nodes for @@ -65,9 +67,21 @@ wait_for_messages(Servers, QName, Number, Fun, N) -> ok; _ -> timer:sleep(?WFM_SLEEP), - wait_for_messages(Servers, QName, Number, Fun, N - 1) + wait_for_messages(Servers, QName, Number, MetricKey, N - 1) end. +query_messages(ServerIds, Key) -> + [begin + try ra:member_overview(ServerId) of + {ok, #{machine := #{Key := Value}}, _} -> + Value; + _ -> + undefined + catch _:_Err -> + undefined + end + end || ServerId <- ServerIds]. + wait_for_messages(Config, Stats) -> wait_for_messages(Config, lists:sort(Stats), ?WFM_DEFAULT_NUMS). From 74e06db1ccc0a0189aa77a16129eab49ca10c2b2 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 11 Nov 2025 12:00:36 +0000 Subject: [PATCH 17/28] QQ: Make rabbit_fifo_client:stat/2 backwards compatible. --- deps/rabbit/src/rabbit_fifo_client.erl | 14 ++++++++++---- deps/rabbit/src/rabbit_quorum_queue.erl | 9 ++++++--- deps/rabbit/test/quorum_queue_SUITE.erl | 10 +++++----- 3 files changed, 21 insertions(+), 12 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index aec7988b244a..5b74052315ee 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -521,10 +521,16 @@ stat(Leader) -> stat(Leader, Timeout) -> %% short timeout as we don't want to spend too long if it is going to %% fail anyway - case ra:local_query(Leader, fun rabbit_fifo:query_stat/1, Timeout) of - {ok, {_, {R, C}}, _} -> {ok, R, C}; - {error, _} = Error -> Error; - {timeout, _} = Error -> Error + %% TODO: the overview is too large to be super efficient + %% but we use it for backwards compatibilty + case ra:member_overview(Leader, Timeout) of + {ok, #{machine := #{num_ready_messages := R, + num_checked_out := C}}, _} -> + {ok, R, C}; + {error, _} = Error -> + Error; + {timeout, _} = Error -> + Error end. update_machine_state(Server, Conf) -> diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 252bc0ef97dc..b9e1330dd683 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -1167,9 +1167,12 @@ stat(Q, Timeout) when ?is_amqqueue(Q) -> Leader = amqqueue:get_pid(Q), try case rabbit_fifo_client:stat(Leader, Timeout) of - {ok, _, _} = Success -> Success; - {error, _} -> {ok, 0, 0}; - {timeout, _} -> {ok, 0, 0} + {ok, _, _} = Success -> + Success; + {error, _} -> + {ok, 0, 0}; + {timeout, _} -> + {ok, 0, 0} end catch _:_ -> diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index b12b51e7ba63..3bf89f4ae38f 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -3961,9 +3961,9 @@ queue_length_limit_policy_cleared(Config) -> {<<"overflow">>, <<"reject-publish">>}]), timer:sleep(1000), RaName = ra_name(QQ), - QueryFun = fun rabbit_fifo:overview/1, - ?awaitMatch({ok, {_, #{config := #{max_length := 2}}}, _}, - rpc:call(Server, ra, local_query, [RaName, QueryFun]), + % QueryFun = fun rabbit_fifo:overview/1, + ?awaitMatch({ok, #{machine := #{config := #{max_length := 2}}}, _}, + ra:member_overview({RaName, Server}), ?DEFAULT_AWAIT), #'confirm.select_ok'{} = amqp_channel:call(Ch, #'confirm.select'{}), ok = publish_confirm(Ch, QQ), @@ -3972,8 +3972,8 @@ queue_length_limit_policy_cleared(Config) -> wait_for_messages_ready(Servers, RaName, 3), fail = publish_confirm(Ch, QQ), ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"max-length">>), - ?awaitMatch({ok, {_, #{config := #{max_length := undefined}}}, _}, - rpc:call(Server, ra, local_query, [RaName, QueryFun]), + ?awaitMatch({ok, #{machine := #{config := #{max_length := undefined}}}, _}, + ra:member_overview({RaName, Server}), ?DEFAULT_AWAIT), ok = publish_confirm(Ch, QQ), wait_for_messages_ready(Servers, RaName, 4). From 235bd37ebaa058770701af66f3cb3bfa050f0ea0 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 12 Nov 2025 09:18:35 +0000 Subject: [PATCH 18/28] QQ: record lengths of each priority queue And emit the number of messages for each priority in the overview map. --- deps/rabbit/src/rabbit_fifo.erl | 30 ++--- deps/rabbit/src/rabbit_fifo_pq.erl | 127 ++++++++++++---------- deps/rabbit/test/rabbit_fifo_pq_SUITE.erl | 17 ++- 3 files changed, 98 insertions(+), 76 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index c107767ce7b3..b757d2b458f8 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -449,7 +449,7 @@ apply_(#{index := Idx} = Meta, {Life, _Credit, credited} -> {Life, credited} end, - Priority = get_priority(ConsumerMeta), + Priority = get_consumer_priority(ConsumerMeta), ConsumerKey = case consumer_key_from_id(ConsumerId, State0) of {ok, K} -> K; @@ -1747,10 +1747,10 @@ maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, Header0 = maybe_set_msg_ttl(RawMsg, Ts, Size, State0), Header = maybe_set_msg_delivery_count(RawMsg, Header0), Msg = make_msg(RaftIdx, Header), - PTag = priority_tag(RawMsg), + Priority = msg_priority(RawMsg), State = State0#?STATE{msg_bytes_enqueue = Enqueue + Size, messages_total = Total + 1, - messages = rabbit_fifo_pq:in(PTag, Msg, Messages) + messages = rabbit_fifo_pq:in(Priority, Msg, Messages) }, {ok, State, Effects}; maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, @@ -1780,10 +1780,10 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, false -> undefined end, - PTag = priority_tag(RawMsg), + Priority = msg_priority(RawMsg), State = State0#?STATE{msg_bytes_enqueue = BytesEnqueued + Size, messages_total = Total + 1, - messages = rabbit_fifo_pq:in(PTag, Msg, Messages), + messages = rabbit_fifo_pq:in(Priority, Msg, Messages), enqueuers = Enqueuers0#{From => Enq}, msg_cache = MsgCache }, @@ -2835,9 +2835,9 @@ is_expired(Ts, #?STATE{cfg = #cfg{expires = Expires}, is_expired(_Ts, _State) -> false. -get_priority(#{priority := Priority}) -> +get_consumer_priority(#{priority := Priority}) -> Priority; -get_priority(#{args := Args}) -> +get_consumer_priority(#{args := Args}) -> %% fallback, v3 option case rabbit_misc:table_lookup(Args, <<"x-priority">>) of {_Type, Value} -> @@ -2845,7 +2845,7 @@ get_priority(#{args := Args}) -> _ -> 0 end; -get_priority(_) -> +get_consumer_priority(_) -> 0. notify_decorators_effect(QName, MaxActivePriority, IsEmpty) -> @@ -3037,21 +3037,23 @@ maps_search(Pred, {K, V, I}) -> maps_search(Pred, Map) when is_map(Map) -> maps_search(Pred, maps:next(maps:iterator(Map))). -priority_tag(Msg) -> +-define(DEFAULT_PRIORITY, 4). +-define(MAX_PRIORITY, 31). + +msg_priority(Msg) -> case mc:is(Msg) of true -> case mc:priority(Msg) of P when is_integer(P) -> - min(P, 31); + min(P, ?MAX_PRIORITY); _ -> - 4 + ?DEFAULT_PRIORITY end; false -> - 4 + ?DEFAULT_PRIORITY end. -do_snapshot(MacVer, Ts, Ch, - RaAux, DiscardedBytes, Force) +do_snapshot(MacVer, Ts, Ch, RaAux, DiscardedBytes, Force) when element(1, Ch) == checkpoint andalso is_integer(MacVer) andalso MacVer >= 8 -> diff --git a/deps/rabbit/src/rabbit_fifo_pq.erl b/deps/rabbit/src/rabbit_fifo_pq.erl index 1efb1303092b..371e212d1ec3 100644 --- a/deps/rabbit/src/rabbit_fifo_pq.erl +++ b/deps/rabbit/src/rabbit_fifo_pq.erl @@ -8,6 +8,7 @@ -module(rabbit_fifo_pq). -include("rabbit_fifo.hrl"). + -export([ new/0, in/3, @@ -22,30 +23,32 @@ overview/1 ]). -% -define(NON_EMPTY, {_, [_|_]}). --define(EMPTY, {[], []}). +-define(STATE, pq). +-define(EMPTY, {0, [], []}). -%% a weighted priority queue with only two priorities +%% supports 32 priorities, needs to be a power of 2 to support the De Bruijn +%% lookup method. 64 would push the bitmap into an erlang big number so we +%% have to settle for 32 -type priority() :: 0..31. --type queue() :: {list(msg()), list(msg())}. +-type queue() :: {non_neg_integer(), list(msg()), list(msg())}. --record(?MODULE, {buckets = #{} :: #{priority() => queue()}, - len = 0 :: non_neg_integer(), - bitmap = 0 :: integer()}). +-record(?STATE, {buckets = #{} :: #{priority() => queue()}, + len = 0 :: non_neg_integer(), + bitmap = 0 :: integer()}). --opaque state() :: #?MODULE{}. +-opaque state() :: #?STATE{}. -export_type([state/0, priority/0]). -spec new() -> state(). new() -> - #?MODULE{}. + #?STATE{}. -spec in(priority(), msg(), state()) -> state(). -in(Priority0, Item, #?MODULE{buckets = Buckets0, - bitmap = Bitmap0, - len = Len} = State) +in(Priority0, Item, #?STATE{buckets = Buckets0, + bitmap = Bitmap0, + len = Len} = State) when Priority0 >= 0 andalso Priority0 =< 31 -> %% invert priority @@ -53,37 +56,23 @@ in(Priority0, Item, #?MODULE{buckets = Buckets0, case Buckets0 of #{Priority := Queue0} -> %% there are messages for the priority already - State#?MODULE{buckets = Buckets0#{Priority => in(Item, Queue0)}, - len = Len + 1}; + State#?STATE{buckets = Buckets0#{Priority => in(Item, Queue0)}, + len = Len + 1}; _ -> Bitmap = Bitmap0 bor (1 bsl Priority), %% there are no messages for the priority - State#?MODULE{buckets = Buckets0#{Priority => in(Item, ?EMPTY)}, - bitmap = Bitmap, - len = Len + 1} + State#?STATE{buckets = Buckets0#{Priority => in(Item, ?EMPTY)}, + bitmap = Bitmap, + len = Len + 1} end. -first_set_bit(0) -> - 32; -first_set_bit(Bitmap) -> - count_trailing(Bitmap band -Bitmap). - --define(DEBRUIJN_SEQ, 16#077CB531). --define(DEBRUIJN_LOOKUP, - {0, 1, 28, 2, 29, 14, 24, 3, 30, 22, 20, 15, 25, 17, 4, 8, - 31, 27, 13, 23, 21, 19, 16, 7, 26, 12, 18, 6, 11, 5, 10, 9}). - -count_trailing(N) -> - Lookup = ((N * ?DEBRUIJN_SEQ) bsr 27) band 31, - element(Lookup + 1, ?DEBRUIJN_LOOKUP). - -spec out(state()) -> empty | {msg(), state()}. -out(#?MODULE{len = 0}) -> +out(#?STATE{len = 0}) -> empty; -out(#?MODULE{buckets = Buckets, - len = Len, - bitmap = Bitmap0} = State) -> +out(#?STATE{buckets = Buckets, + len = Len, + bitmap = Bitmap0} = State) -> Priority = first_set_bit(Bitmap0), #{Priority := Q0} = Buckets, Msg = peek(Q0), @@ -93,25 +82,25 @@ out(#?MODULE{buckets = Buckets, %% as we know the bit is set we just need to xor rather than %% create a mask then xor Bitmap = Bitmap0 bxor (1 bsl Priority), - {Msg, State#?MODULE{buckets = maps:remove(Priority, Buckets), - len = Len - 1, - bitmap = Bitmap}}; + {Msg, State#?STATE{buckets = maps:remove(Priority, Buckets), + len = Len - 1, + bitmap = Bitmap}}; Q -> - {Msg, State#?MODULE{buckets = maps:put(Priority, Q, Buckets), - len = Len - 1}} + {Msg, State#?STATE{buckets = maps:put(Priority, Q, Buckets), + len = Len - 1}} end. -spec get(state()) -> empty | msg(). -get(#?MODULE{len = 0}) -> +get(#?STATE{len = 0}) -> empty; -get(#?MODULE{buckets = Buckets, - bitmap = Bitmap}) -> +get(#?STATE{buckets = Buckets, + bitmap = Bitmap}) -> Priority = first_set_bit(Bitmap), #{Priority := Q0} = Buckets, peek(Q0). -spec len(state()) -> non_neg_integer(). -len(#?MODULE{len = Len}) -> +len(#?STATE{len = Len}) -> Len. -spec from_list([{priority(), term()}]) -> state(). @@ -131,17 +120,17 @@ from_lqueue(LQ) -> end, new(), LQ). -spec indexes(state()) -> [ra:index()]. -indexes(#?MODULE{buckets = Buckets}) -> +indexes(#?STATE{buckets = Buckets}) -> maps:fold( - fun (_P, {L1, L2}, Acc0) -> + fun (_P, {_, L1, L2}, Acc0) -> Acc = lists:foldl(fun msg_idx_fld/2, Acc0, L1), lists:foldl(fun msg_idx_fld/2, Acc, L2) end, [], Buckets). -spec get_lowest_index(state()) -> undefined | ra:index(). -get_lowest_index(#?MODULE{len = 0}) -> +get_lowest_index(#?STATE{len = 0}) -> undefined; -get_lowest_index(#?MODULE{buckets = Buckets}) -> +get_lowest_index(#?STATE{buckets = Buckets}) -> lists:min( maps:fold(fun (_, Q, Acc) -> case peek(Q) of @@ -154,32 +143,38 @@ get_lowest_index(#?MODULE{buckets = Buckets}) -> -spec overview(state()) -> #{len := non_neg_integer(), + detail := #{priority() => pos_integer()}, num_active_priorities := 0..32, lowest_index := ra:index()}. -overview(#?MODULE{len = Len, - buckets = Buckets} = State) -> +overview(#?STATE{len = Len, + buckets = Buckets} = State) -> + Detail = maps:fold(fun (P0, {C, _, _}, Acc) -> + P = 31-P0, + Acc#{P => C} + end, #{}, Buckets), #{len => Len, + detail => Detail, num_active_priorities => map_size(Buckets), lowest_index => get_lowest_index(State)}. -%% internals +%% INTERNAL %% invariant, if the queue is non empty so is the Out (right) list. in(X, ?EMPTY) -> - {[], [X]}; -in(X, {In, Out}) -> - {[X | In], Out}. + {1, [], [X]}; +in(X, {C, In, Out}) -> + {C+1, [X | In], Out}. peek(?EMPTY) -> empty; -peek({_, [H | _]}) -> +peek({_, _, [H | _]}) -> H. -drop({In, [_]}) -> +drop({C, In, [_]}) -> %% the last Out one - {[], lists:reverse(In)}; -drop({In, [_ | Out]}) -> - {In, Out}. + {C-1, [], lists:reverse(In)}; +drop({C, In, [_ | Out]}) -> + {C-1, In, Out}. msg_idx_fld(Msg, Acc) when is_list(Acc) -> [msg_idx(Msg) | Acc]. @@ -194,3 +189,17 @@ to_list(empty, Acc) -> to_list({Item, State}, Acc) -> to_list(out(State), [Item | Acc]). +first_set_bit(0) -> + 32; +first_set_bit(Bitmap) -> + count_trailing(Bitmap band -Bitmap). + +-define(DEBRUIJN_SEQ, 16#077CB531). +-define(DEBRUIJN_LOOKUP, + {0, 1, 28, 2, 29, 14, 24, 3, 30, 22, 20, 15, 25, 17, 4, 8, + 31, 27, 13, 23, 21, 19, 16, 7, 26, 12, 18, 6, 11, 5, 10, 9}). + +count_trailing(N) -> + Lookup = ((N * ?DEBRUIJN_SEQ) bsr 27) band 31, + element(Lookup + 1, ?DEBRUIJN_LOOKUP). + diff --git a/deps/rabbit/test/rabbit_fifo_pq_SUITE.erl b/deps/rabbit/test/rabbit_fifo_pq_SUITE.erl index 4634d165ce21..a64ee780def7 100644 --- a/deps/rabbit/test/rabbit_fifo_pq_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_pq_SUITE.erl @@ -65,6 +65,11 @@ basics(_Config) -> ?assertEqual([1,2,3,4,5], lists:sort(rabbit_fifo_pq:indexes(Q1))), ?assertMatch(#{len := 5, + detail := #{1 := 1, + 2 := 1, + 3 := 1, + 4 := 1, + 5 := 1}, num_active_priorities := 5, lowest_index := 1}, rabbit_fifo_pq:overview(Q1)), {?MSG(5), Q2} = rabbit_fifo_pq:out(Q1), @@ -112,14 +117,16 @@ get_lowest_index(_Config) -> ?assertEqual(1, rabbit_fifo_q:get_lowest_index(Q3)), ?assertEqual(2, rabbit_fifo_q:get_lowest_index(Q4)), ?assertEqual(3, rabbit_fifo_q:get_lowest_index(Q5)), - ?assertEqual(undefined, rabbit_fifo_q:get_lowest_index(Q6)). + ?assertEqual(undefined, rabbit_fifo_q:get_lowest_index(Q6)), + ok. + property(_Config) -> run_proper( fun () -> ?FORALL(Ops, op_gen(256), queue_prop(Ops)) - end, [], 25), + end, [], 100), ok. queue_prop(Ops) -> @@ -130,11 +137,15 @@ queue_prop(Ops) -> Sut0 = rabbit_fifo_pq:from_list(Ops), Out = rabbit_fifo_pq:to_list(Sut0), + #{detail := Detail, + len := Len} = rabbit_fifo_pq:overview(Sut0), + DetailSum = maps:fold(fun (_, C, Acc) -> Acc + C end, 0, Detail), + DetailSum == Len andalso [element(2, O) || O <- SortedOps] == Out. %%% helpers --type item() :: {rabbit_fifo_pq:priority(), integer()}. +-type item() :: {rabbit_fifo_pq:priority(), non_neg_integer()}. op_gen(Size) -> ?LET(Ops, resize(Size, list(item())), Ops). From 417a2d9e760eb9d06e9dc4d234098574163abf49 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 13 Nov 2025 10:23:53 +0100 Subject: [PATCH 19/28] Redeliver all pending consumer messages in correct order --- deps/rabbit/src/rabbit_fifo.erl | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index b757d2b458f8..24d5cc2e87c9 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -744,7 +744,8 @@ snapshot_installed(_Meta, #?MODULE{cfg = #cfg{}, Acc) -> case node(Pid) == node() of true -> - Acc#{{Tag, Pid} => maps:to_list(Checked)}; + Iter = maps:iterator(Checked, ordered), + Acc#{{Tag, Pid} => maps:to_list(Iter)}; false -> Acc end From 67a3b13c26da26ed97ac3f718f53dfe4f6969a71 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 13 Nov 2025 10:43:03 +0100 Subject: [PATCH 20/28] Redeliver in correct order delivery_effects/2 expects the messages in reversed order (and will from now on return the effects in the correct order). --- deps/rabbit/src/rabbit_fifo.erl | 33 ++++++++++++++++----------------- 1 file changed, 16 insertions(+), 17 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 24d5cc2e87c9..b68530e649fd 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -744,14 +744,13 @@ snapshot_installed(_Meta, #?MODULE{cfg = #cfg{}, Acc) -> case node(Pid) == node() of true -> - Iter = maps:iterator(Checked, ordered), + Iter = maps:iterator(Checked, reversed), Acc#{{Tag, Pid} => maps:to_list(Iter)}; false -> Acc end end, #{}, Consumers), - Effs = add_delivery_effects([], SendAcc, State), - Effs. + delivery_effects(SendAcc, State). convert_v7_to_v8(#{} = _Meta, StateV7) -> %% the structure is intact for now @@ -2063,9 +2062,8 @@ checkout0(Meta, {success, ConsumerKey, MsgId, SendAcc0#{ConsumerKey => [DelMsg | LogMsgs]} end, checkout0(Meta, checkout_one(Meta, ExpiredMsg, State, Effects), SendAcc); -checkout0(_Meta, {_Activity, ExpiredMsg, State0, Effects0}, SendAcc) -> - Effects = add_delivery_effects([], SendAcc, State0), - {State0, ExpiredMsg, Effects0 ++ lists:reverse(Effects)}. +checkout0(_Meta, {_Activity, ExpiredMsg, State, Effects}, SendAcc) -> + {State, ExpiredMsg, Effects ++ delivery_effects(SendAcc, State)}. evaluate_limit(Idx, State1, State2, OuterEffects) -> case evaluate_limit0(Idx, State1, State2, []) of @@ -2154,18 +2152,19 @@ chunk_disk_msgs([{_MsgId, Msg} = ConsumerMsg | Rem], Bytes, Size = get_header(size, get_msg_header(Msg)), chunk_disk_msgs(Rem, Bytes + Size, [[ConsumerMsg | CurChunk] | Chunks]). -add_delivery_effects(Effects0, AccMap, _State) - when map_size(AccMap) == 0 -> +delivery_effects(AccMap, _State) + when map_size(AccMap) =:= 0 -> %% does this ever happen? - Effects0; -add_delivery_effects(Effects0, AccMap, State) -> - maps:fold(fun (C, DiskMsgs, Efs) - when is_list(DiskMsgs) -> - lists:foldl( - fun (Msgs, E) -> - [delivery_effect(C, Msgs, State) | E] - end, Efs, chunk_disk_msgs(DiskMsgs, 0, [[]])) - end, Effects0, AccMap). + []; +delivery_effects(AccMap, State) -> + Effs = maps:fold(fun(C, DiskMsgs, Efs) + when is_list(DiskMsgs) -> + lists:foldl( + fun (Msgs, E) -> + [delivery_effect(C, Msgs, State) | E] + end, Efs, chunk_disk_msgs(DiskMsgs, 0, [[]])) + end, [], AccMap), + lists:reverse(Effs). take_next_msg(#?STATE{returns = Returns0, messages = Messages0} = State) -> From eb69cd775a94f3125b50996d878122f69dfc79af Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 14 Nov 2025 09:55:59 +0000 Subject: [PATCH 21/28] QQ: display per priority message counts in mgmt UI. This could do with some improvement but has some utility for now. --- deps/rabbit/src/rabbit_fifo.erl | 6 +++++- deps/rabbit/src/rabbit_quorum_queue.erl | 4 ++++ .../priv/www/js/tmpl/quorum-queue-stats.ejs | 16 +++++++++------- 3 files changed, 18 insertions(+), 8 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index b68530e649fd..bf104cb84847 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -948,6 +948,8 @@ overview(#?STATE{consumers = Cons, #{ % num_hi := MsgsHi, % num_no := MsgsNo + num_active_priorities := NumActivePriorities, + detail := Detail } = rabbit_fifo_pq:overview(Messages), Overview = #{type => ?STATE, @@ -964,7 +966,9 @@ overview(#?STATE{consumers = Cons, enqueue_message_bytes => EnqueueBytes, checkout_message_bytes => CheckoutBytes, discarded_bytes => DiscardedBytes, - smallest_raft_index => smallest_raft_index(State) + smallest_raft_index => smallest_raft_index(State), + num_active_priorities => NumActivePriorities, + messages_by_priority => Detail }, DlxOverview = dlx_overview(DlxState), maps:merge(maps:merge(Overview, DlxOverview), SacOverview). diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index b9e1330dd683..3d48d465ce47 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -620,6 +620,10 @@ handle_tick(QName, [{messages_ready_normal, V} | Acc]; (num_ready_messages_return, V, Acc) -> [{messages_ready_returned, V} | Acc]; + (messages_by_priority, V, Acc) -> + [{messages_by_priority, V} | Acc]; + (num_active_priorities, V, Acc) -> + [{messages_active_priorities, V} | Acc]; (_, _, Acc) -> Acc end, info(Q, Keys), Overview), diff --git a/deps/rabbitmq_management/priv/www/js/tmpl/quorum-queue-stats.ejs b/deps/rabbitmq_management/priv/www/js/tmpl/quorum-queue-stats.ejs index 98d266ef5306..412915c68bee 100644 --- a/deps/rabbitmq_management/priv/www/js/tmpl/quorum-queue-stats.ejs +++ b/deps/rabbitmq_management/priv/www/js/tmpl/quorum-queue-stats.ejs @@ -25,6 +25,12 @@ Open files <%= fmt_table_short(queue.open_files) %> + <% if (queue.hasOwnProperty('messages_by_priority')) { %> + + Messages by priority + <%= fmt_table_short(queue.messages_by_priority) %> + + <% } %> <% if (queue.hasOwnProperty('delivery_limit')) { %> Delivery limit @@ -40,8 +46,7 @@ Total Ready Unacked - High priority - Normal priority + Active priorities Returned Dead-lettered @@ -62,10 +67,7 @@ <%= fmt_num_thousands(queue.messages_unacknowledged) %> - <%= fmt_num_thousands(queue.messages_ready_high) %> - - - <%= fmt_num_thousands(queue.messages_ready_normal) %> + <%= fmt_num_thousands(queue.messages_active_priorities) %> <%= fmt_num_thousands(queue.messages_ready_returned) %> @@ -105,4 +107,4 @@ <%= fmt_bytes(queue.memory) %> - \ No newline at end of file + From fe176151abd5d3da89dfab73d012c22837ac0ef7 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 14 Nov 2025 11:48:36 +0100 Subject: [PATCH 22/28] Fix extracting message header --- deps/rabbit/src/rabbit_fifo.erl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index bf104cb84847..1c0e8ec1b887 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -1154,7 +1154,7 @@ handle_aux(_, _, {get_checked_out, ConsumerKey, MsgIds}, Aux0, RaAux0) -> maps:fold( fun (MsgId, Msg, {S0, Acc}) -> Idx = get_msg_idx(Msg), - Header = get_msg_idx(Msg), + Header = get_msg_header(Msg), %% it is possible this is not found if the consumer %% crashed and the message got removed case ra_aux:log_fetch(Idx, S0) of From b4f44ad813f758e308dda4981a8000515be52eb2 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 14 Nov 2025 14:10:37 +0100 Subject: [PATCH 23/28] Tiny performance improvements --- deps/rabbit/src/rabbit_fifo.erl | 30 +++++++++++++++--------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 1c0e8ec1b887..c8526e728b9f 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -2032,10 +2032,10 @@ return_one(Meta, MsgId, Msg0, DeliveryFailed, Anns, return_all(Meta, #?STATE{consumers = Cons} = State0, Effects0, ConsumerKey, #consumer{checked_out = Checked} = Con, DeliveryFailed) -> State = State0#?STATE{consumers = Cons#{ConsumerKey => Con}}, - lists:foldl(fun ({MsgId, Msg}, {S, E}) -> - return_one(Meta, MsgId, Msg, DeliveryFailed, #{}, - S, E, ConsumerKey) - end, {State, Effects0}, lists:sort(maps:to_list(Checked))). + maps:fold(fun (MsgId, Msg, {S, E}) -> + return_one(Meta, MsgId, Msg, DeliveryFailed, #{}, + S, E, ConsumerKey) + end, {State, Effects0}, maps:iterator(Checked, ordered)). checkout(Meta, OldState, State0, Effects0) -> checkout(Meta, OldState, State0, Effects0, ok). @@ -2059,11 +2059,11 @@ checkout0(Meta, {success, ConsumerKey, MsgId, Msg, ExpiredMsg, State, Effects}, SendAcc0) -> DelMsg = {MsgId, Msg}, - SendAcc = case maps:get(ConsumerKey, SendAcc0, undefined) of - undefined -> - SendAcc0#{ConsumerKey => [DelMsg]}; - LogMsgs -> - SendAcc0#{ConsumerKey => [DelMsg | LogMsgs]} + SendAcc = case SendAcc0 of + #{ConsumerKey := LogMsgs} -> + SendAcc0#{ConsumerKey := [DelMsg | LogMsgs]}; + #{} -> + SendAcc0#{ConsumerKey => [DelMsg]} end, checkout0(Meta, checkout_one(Meta, ExpiredMsg, State, Effects), SendAcc); checkout0(_Meta, {_Activity, ExpiredMsg, State, Effects}, SendAcc) -> @@ -3240,7 +3240,7 @@ dlx_apply(_, {dlx, {checkout, Pid, Prefetch}}, dlx_apply(_, {dlx, {checkout, ConsumerPid, Prefetch}}, at_least_once, #?DLX{consumer = #dlx_consumer{pid = OldConsumerPid, - checked_out = CheckedOutOldConsumer}, + checked_out = CheckedOldConsumer}, discards = Discards0, msg_bytes = Bytes, msg_bytes_checkout = BytesCheckout} = State0) -> @@ -3258,13 +3258,13 @@ dlx_apply(_, {dlx, {checkout, ConsumerPid, Prefetch}}, %% such that these messages will be re-delivered to the new consumer. %% When inserting back into the discards queue, we respect the original order in which messages %% were discarded. - Checked0 = maps:to_list(CheckedOutOldConsumer), - Checked1 = lists:keysort(1, Checked0), - {Discards, BytesMoved} = lists:foldr( - fun({_Id, ?TUPLE(_, Msg) = RsnMsg}, {D, B}) -> + {Discards, BytesMoved} = maps:fold( + fun(_Id, ?TUPLE(_, Msg) = RsnMsg, {D, B}) -> Size = get_header(size, get_msg_header(Msg)), {lqueue:in_r(RsnMsg, D), B + Size} - end, {Discards0, 0}, Checked1), + end, + {Discards0, 0}, + maps:iterator(CheckedOldConsumer, reversed)), State = State0#?DLX{consumer = #dlx_consumer{pid = ConsumerPid, prefetch = Prefetch}, discards = Discards, From 4399315331eea41bed95ba65deb7e6690d1735cf Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 14 Nov 2025 15:28:57 +0100 Subject: [PATCH 24/28] Optimise get_lowest_index/1 Avoid building a list. --- deps/rabbit/src/rabbit_fifo_pq.erl | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo_pq.erl b/deps/rabbit/src/rabbit_fifo_pq.erl index 371e212d1ec3..f8687d0e604e 100644 --- a/deps/rabbit/src/rabbit_fifo_pq.erl +++ b/deps/rabbit/src/rabbit_fifo_pq.erl @@ -131,15 +131,14 @@ indexes(#?STATE{buckets = Buckets}) -> get_lowest_index(#?STATE{len = 0}) -> undefined; get_lowest_index(#?STATE{buckets = Buckets}) -> - lists:min( - maps:fold(fun (_, Q, Acc) -> - case peek(Q) of - empty -> - Acc; - Msg -> - [msg_idx(Msg) | Acc] - end - end, [], Buckets)). + maps:fold(fun (_, Q, Acc) -> + case peek(Q) of + empty -> + Acc; + Msg -> + min(msg_idx(Msg), Acc) + end + end, undefined, Buckets). -spec overview(state()) -> #{len := non_neg_integer(), From c104dffc2163cc4a346981a72f088c057ae23686 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 14 Nov 2025 17:31:43 +0100 Subject: [PATCH 25/28] Include DLX consumer in smallest_raft_index/1 The oldest message might be checked out by the DLX consumer. --- deps/rabbit/src/rabbit_fifo.erl | 43 ++++++++++++++++++--------------- 1 file changed, 24 insertions(+), 19 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index c8526e728b9f..e214cf2ac944 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -2883,25 +2883,30 @@ convert(Meta, 7, To, State) -> convert(Meta, 8, To, convert_v7_to_v8(Meta, State)). smallest_raft_index(#?STATE{messages = Messages, - dlx = #?DLX{discards = Discards}} = State) -> - SmallestDlxRaIdx = lqueue:fold(fun (?TUPLE(_, Msg), Acc) -> - min(get_msg_idx(Msg), Acc) - end, undefined, Discards), - SmallestMsgsRaIdx = rabbit_fifo_pq:get_lowest_index(Messages), - %% scan consumers and returns queue here instead - smallest_checked_out(State, min(SmallestDlxRaIdx, SmallestMsgsRaIdx)). - -smallest_checked_out(#?STATE{returns = Returns, - consumers = Consumers}, Min) -> - SmallestSoFar = lqueue:fold(fun (Msg, Acc) -> - min(get_msg_idx(Msg), Acc) - end, Min, Returns), - maps:fold(fun (_Cid, #consumer{checked_out = Ch}, Acc0) -> - maps:fold( - fun (_MsgId, Msg, Acc) -> - min(get_msg_idx(Msg), Acc) - end, Acc0, Ch) - end, SmallestSoFar, Consumers). + returns = Returns, + consumers = Consumers, + dlx = #?DLX{consumer = DlxConsumer, + discards = Discards}}) -> + Min0 = rabbit_fifo_pq:get_lowest_index(Messages), + Min1 = lqueue:fold(fun (Msg, Acc) -> + min(get_msg_idx(Msg), Acc) + end, Min0, Returns), + Min2 = maps:fold(fun (_Cid, #consumer{checked_out = Ch}, Acc0) -> + maps:fold(fun (_MsgId, Msg, Acc) -> + min(get_msg_idx(Msg), Acc) + end, Acc0, Ch) + end, Min1, Consumers), + Min = lqueue:fold(fun (?TUPLE(_Reason, Msg), Acc) -> + min(get_msg_idx(Msg), Acc) + end, Min2, Discards), + case DlxConsumer of + undefined -> + Min; + #dlx_consumer{checked_out = Checked} -> + maps:fold(fun(_MsgId, ?TUPLE(_Reason, Msg), Acc) -> + min(get_msg_idx(Msg), Acc) + end, Min, Checked) + end. make_requeue(ConsumerKey, Notify, [{MsgId, Idx, Header, Msg}], Acc) -> lists:reverse([{append, From 5a56c37e8d10324bae1ca8dd19c288129aad40eb Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 18 Nov 2025 10:52:48 +0000 Subject: [PATCH 26/28] fix rabbit_fifo_prop_SUITE --- deps/rabbit/test/rabbit_fifo_prop_SUITE.erl | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index dbaa0e1469d4..461a50f5cb95 100644 --- a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl @@ -2210,18 +2210,23 @@ test_init_v(Conf, Version) -> test_init(rabbit_fifo:which_module(Version), Conf). meta(Idx) -> - meta(Idx, 3). + meta(Idx, rabbit_fifo:version()). meta(Idx, Vsn) -> - #{machine_version => Vsn, index => Idx, term => 1, system_time => 0}. + #{machine_version => Vsn, + index => Idx, + term => 1, + reply_mode => {notify, Idx, self()}, + system_time => 0}. make_checkout(Cid, Spec) -> make_checkout(Cid, Spec, #{}). + make_checkout(Cid, Spec, Meta) -> rabbit_fifo:make_checkout(Cid, Spec, Meta). make_enqueue(Pid, Seq, Msg) -> - rabbit_fifo:make_enqueue(Pid, Seq, Msg). + rabbit_fifo:make_enqueue_old(Pid, Seq, Msg). make_settle(Cid, MsgIds) -> rabbit_fifo:make_settle(Cid, MsgIds). From 341306c54f6b452f2e28be628112a95792405e45 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 19 Nov 2025 12:02:20 +0000 Subject: [PATCH 27/28] QQ: Message expiration scans all priorities Expiration is now done for all priorities using a separate command that performs a "shallow" expiry run. I.e. each priority will have messages dropped until it encounters the next non expired one. This means that it can still leave expired messages on queue if they are preceeded by message that are not yet or never will be expired. This is consistent with existing expiration logic. --- deps/rabbit/src/rabbit_fifo.erl | 275 ++++++++++++++-------- deps/rabbit/src/rabbit_fifo.hrl | 3 +- deps/rabbit/src/rabbit_fifo_pq.erl | 72 +++++- deps/rabbit/src/rabbit_quorum_queue.erl | 7 +- deps/rabbit/test/quorum_queue_SUITE.erl | 14 +- deps/rabbit/test/rabbit_fifo_SUITE.erl | 46 +++- deps/rabbit/test/rabbit_fifo_pq_SUITE.erl | 69 ++++++ 7 files changed, 368 insertions(+), 118 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index e214cf2ac944..5f6a1caf1fc8 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -38,6 +38,7 @@ false -> {0, erts_debug:size(Msg)} end). + -else. -define(SIZE(Msg), mc:size(Msg)). -endif. @@ -83,6 +84,7 @@ %% protocol helpers make_enqueue/3, + make_enqueue_old/3, make_register_enqueuer/1, make_checkout/3, make_settle/2, @@ -238,14 +240,16 @@ apply(Meta, {machine_version, FromVersion, ToVersion}, VXState) -> State = convert(Meta, FromVersion, ToVersion, VXState), %% TODO: force snapshot now? {State, ok, [{aux, {dlx, setup}}]}; -apply(Meta, Cmd, #?STATE{discarded_bytes = DiscBytes} = State) -> +apply(#{system_time := Ts} = Meta, Cmd, + #?STATE{discarded_bytes = DiscBytes} = State) -> %% add estimated discared_bytes %% TODO: optimise! %% this is the simplest way to record the discarded bytes for most %% commands but it is a bit mory garby as almost always creates a new %% state copy before even processing the command Bytes = estimate_discarded_size(Cmd), - apply_(Meta, Cmd, State#?STATE{discarded_bytes = DiscBytes + Bytes}). + apply_(Meta, Cmd, State#?STATE{discarded_bytes = DiscBytes + Bytes, + last_command_time = Ts}). apply_(Meta, #enqueue{pid = From, seq = Seq, msg = RawMsg}, State00) -> @@ -430,6 +434,7 @@ apply_(Meta, #checkout{spec = Spec, Spec)), Reply = {ok, consumer_cancel_info(ConsumerKey, State1)}, {State, _, Effects} = checkout(Meta, State0, State1, Effects1), + {State, Reply, Effects}; error -> {State0, {error, consumer_not_found}, []} @@ -483,57 +488,11 @@ apply_(#{index := Index}, #purge{}, apply_(#{index := _Idx}, #garbage_collection{}, State) -> {State, ok, [{aux, garbage_collection}]}; apply_(Meta, {timeout, expire_msgs}, State) -> - checkout(Meta, State, State, []); -% apply_(#{system_time := Ts} = Meta, -% {down, Pid, noconnection}, -% #?STATE{consumers = Cons0, -% cfg = #cfg{consumer_strategy = single_active}, -% waiting_consumers = Waiting0, -% enqueuers = Enqs0} = State0) -> -% Node = node(Pid), -% %% if the pid refers to an active or cancelled consumer, -% %% mark it as suspected and return it to the waiting queue -% {State1, Effects0} = -% maps:fold( -% fun(CKey, ?CONSUMER_PID(P) = #consumer{status = Status} = C0, {S0, E0}) -% when is_atom(Status) andalso node(P) =:= Node -> -% %% the consumer should be returned to waiting -% %% and checked out messages should be returned -% Effs = consumer_update_active_effects( -% S0, C0, false, {suspected_down, Status} , E0), -% %% TODO: set a timer instead of reaturn all here to allow -% %% a disconnected node a configurable bit of time to be -% %% reconnected -% {St, Effs1} = return_all(Meta, S0, Effs, CKey, C0, false), -% %% if the consumer was cancelled there is a chance it got -% %% removed when returning hence we need to be defensive here -% Waiting = case St#?STATE.consumers of -% #{CKey := C} -> -% Waiting0 ++ [{CKey, C}]; -% _ -> -% Waiting0 -% end, -% {St#?STATE{consumers = maps:remove(CKey, St#?STATE.consumers), -% waiting_consumers = Waiting, -% last_active = Ts}, -% Effs1}; -% (_, _, S) -> -% S -% end, {State0, []}, maps:iterator(Cons0, ordered)), -% WaitingConsumers = update_waiting_consumer_status(Node, State1, -% {suspected_down, up}), - -% %% select a new consumer from the waiting queue and run a checkout -% State2 = State1#?STATE{waiting_consumers = WaitingConsumers}, -% {State, Effects1} = activate_next_consumer(State2, Effects0), - -% %% mark any enquers as suspected -% Enqs = maps:map(fun(P, E) when node(P) =:= Node -> -% E#enqueuer{status = suspected_down}; -% (_, E) -> E -% end, Enqs0), -% Effects = [{monitor, node, Node} | Effects1], -% checkout(Meta, State0, State#?STATE{enqueuers = Enqs}, Effects); + apply_(Meta, {timeout, {expire_msgs, shallow}}, State); +apply_(#{system_time := Ts} = Meta, + {timeout, {expire_msgs, shallow}}, State0) -> + {State, Effects} = expire_shallow(Ts, State0), + checkout(Meta, State0, State, Effects); apply_(#{system_time := Ts} = Meta, {down, Pid, noconnection}, #?STATE{consumers = Cons0, @@ -594,6 +553,7 @@ apply_(Meta, {timeout, {consumer_down_timeout, CKey}}, %% return all messages {State1, Effects0} = return_all(Meta, State0, [], CKey, Consumer, false), + checkout(Meta, State0, State1, Effects0); _ -> {State0, []} @@ -752,7 +712,7 @@ snapshot_installed(_Meta, #?MODULE{cfg = #cfg{}, end, #{}, Consumers), delivery_effects(SendAcc, State). -convert_v7_to_v8(#{} = _Meta, StateV7) -> +convert_v7_to_v8(#{system_time := Ts} = _Meta, StateV7) -> %% the structure is intact for now Cons0 = element(#?STATE.consumers, StateV7), Cons = maps:map(fun (_CKey, #consumer{status = suspected_down} = C) -> @@ -772,7 +732,7 @@ convert_v7_to_v8(#{} = _Meta, StateV7) -> StateV8#?STATE{discarded_bytes = 0, messages = Pq, consumers = Cons, - unused_0 = ?NIL}. + last_command_time = Ts}. purge_node(Meta, Node, State, Effects) -> lists:foldl(fun(Pid, {S0, E0}) -> @@ -859,7 +819,7 @@ state_enter(leader, cfg = #cfg{resource = QRes, dead_letter_handler = DLH}, dlx = DlxState} = State) -> - TimerEffs = timer_effect(erlang:system_time(millisecond), State, []), + TimerEffs = timer_effect(State, []), % return effects to monitor all current consumers and enqueuers Pids = lists:usort(maps:keys(Enqs) ++ [P || ?CONSUMER_PID(P) <- maps:values(Cons)] @@ -1047,10 +1007,9 @@ handle_aux(RaftState, Tag, Cmd, AuxV3, RaAux) handle_aux(RaftState, Tag, Cmd, AuxV4, RaAux); handle_aux(leader, cast, eval, #?AUX{last_decorators_state = LastDec, - % bytes_in = BytesIn, - % bytes_out = BytesOut, last_checkpoint = Check0} = Aux0, RaAux) -> + #?STATE{cfg = #cfg{resource = QName}, discarded_bytes = DiscardedBytes} = MacState = ra_aux:machine_state(RaAux), @@ -1069,7 +1028,7 @@ handle_aux(leader, cast, eval, %% this is called after each batch of commands have been applied %% set timer for message expire %% should really be the last applied index ts but this will have to do - Effects1 = timer_effect(Ts, MacState, Effects0), + Effects1 = timer_effect(MacState, Effects0), case query_notify_decorators_info(MacState) of LastDec -> {no_reply, Aux0#?AUX{last_checkpoint = Check}, RaAux, Effects1}; @@ -1081,11 +1040,13 @@ handle_aux(leader, cast, eval, end; handle_aux(_RaftState, cast, eval, #?AUX{last_checkpoint = Check0} = Aux0, RaAux) -> + Ts = erlang:system_time(millisecond), + EffMacVer = ra_aux:effective_machine_version(RaAux), #?STATE{discarded_bytes = DiscardedBytes} = ra_aux:machine_state(RaAux), {Check, Effects} = do_snapshot(EffMacVer, Ts, Check0, RaAux, - DiscardedBytes, false), + DiscardedBytes, false), {no_reply, Aux0#?AUX{last_checkpoint = Check}, RaAux, Effects}; handle_aux(_RaftState, cast, {bytes_in, {MetaSize, BodySize}}, #?AUX{bytes_in = Bytes} = Aux0, @@ -1237,8 +1198,12 @@ handle_aux(leader, _, {dlx, setup}, Aux, RaAux) -> _ -> ok end, + {no_reply, Aux, RaAux}; +handle_aux(_, _, {dlx, teardown, Pid}, Aux, RaAux) -> + terminate_dlx_worker(Pid), {no_reply, Aux, RaAux}. + eval_gc(RaAux, MacState, #?AUX{gc = #aux_gc{last_raft_idx = LastGcIdx} = Gc} = AuxState) -> {Idx, _} = ra_aux:log_last_index_term(RaAux), @@ -1513,6 +1478,7 @@ cancel_consumer0(Meta, ConsumerKey, {S, Effects2} = maybe_return_all(Meta, ConsumerKey, Consumer, S0, Effects0, Reason), + %% The effects are emitted before the consumer is actually removed %% if the consumer has unacked messages. This is a bit weird but %% in line with what classic queues do (from an external point of @@ -1774,6 +1740,9 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, {Res, State, [{monitor, process, From} | Effects]}; #enqueuer{next_seqno = MsgSeqNo} = Enq0 -> % it is the next expected seqno + % TODO: it is not good to query the `mc' container inside the + % statemachine as it may be modified to behave differently without + % concern for the state machine Header0 = maybe_set_msg_ttl(RawMsg, Ts, Size, State0), Header = maybe_set_msg_delivery_count(RawMsg, Header0), Msg = make_msg(RaftIdx, Header), @@ -1963,16 +1932,20 @@ get_header(_Key, Size) when is_integer(Size) -> undefined; get_header(size, ?TUPLE(Size, Expiry)) - when is_integer(Size), is_integer(Expiry) -> + when is_integer(Size) andalso + is_integer(Expiry) -> Size; get_header(expiry, ?TUPLE(Size, Expiry)) - when is_integer(Size), is_integer(Expiry) -> + when is_integer(Size) andalso + is_integer(Expiry) -> Expiry; get_header(_Key, ?TUPLE(Size, Expiry)) - when is_integer(Size), is_integer(Expiry) -> + when is_integer(Size) andalso + is_integer(Expiry) -> undefined; get_header(Key, Header) - when is_map(Header) andalso is_map_key(size, Header) -> + when is_map(Header) andalso + is_map_key(size, Header) -> maps:get(Key, Header, undefined). annotate_msg(Header, Msg0) -> @@ -2017,7 +1990,7 @@ return_one(Meta, MsgId, Msg0, DeliveryFailed, Anns, discarded_bytes = DiscardedBytes0 - RetainedBytes}, {State, Effects} = complete(Meta, ConsumerKey, [MsgId], Con0, State1, Effects0), - {State, DlxEffects ++ Effects}; + {State, Effects ++ DlxEffects}; _ -> Checked = maps:remove(MsgId, Checked0), Con = Con0#consumer{checked_out = Checked, @@ -2184,7 +2157,7 @@ take_next_msg(#?STATE{returns = Returns0, end end. -get_next_msg(#?STATE{returns = Returns0, +peek_next_msg(#?STATE{returns = Returns0, messages = Messages0}) -> case lqueue:get(Returns0, empty) of empty -> @@ -2235,12 +2208,13 @@ reply_log_effect(RaftIdx, MsgId, Header, Ready, From) -> fun ([]) -> []; ([Cmd]) -> - [{reply, From, {wrap_reply, - {dequeue, {MsgId, {Header, get_msg_from_cmd(Cmd)}}, Ready}}}] + [{reply, From, + {wrap_reply, + {dequeue, {MsgId, {Header, get_msg_from_cmd(Cmd)}}, Ready}}}] end}. checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> - %% Before checking out any messsage to any consumer, + %% Before checking out any message to any consumer, %% first remove all expired messages from the head of the queue. {ExpiredMsg, #?STATE{service_queue = SQ0, messages = Messages0, @@ -2309,30 +2283,83 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> end end. +msg_is_expired(Ts, ?MSG(_, _) = Msg) -> + Header = get_msg_header(Msg), + case get_header(expiry, Header) of + undefined -> + false; + Expiry -> + Ts >= Expiry + end; +msg_is_expired(_Ts, _) -> + false. + %% dequeue all expired messages expire_msgs(RaCmdTs, Result, State, Effects) -> %% In the normal case, there are no expired messages. %% Therefore, first lqueue:get/2 to check whether we need to lqueue:out/1 %% because the latter can be much slower than the former. - case get_next_msg(State) of - ?MSG(_, ?TUPLE(Size, Expiry)) - when is_integer(Size), is_integer(Expiry), RaCmdTs >= Expiry -> - expire(RaCmdTs, State, Effects); - ?MSG(_, #{expiry := Expiry}) - when is_integer(Expiry), RaCmdTs >= Expiry -> + case msg_is_expired(RaCmdTs, peek_next_msg(State)) of + true -> expire(RaCmdTs, State, Effects); - _ -> - %% packed messages never have an expiry + false -> {Result, State, Effects} end. +expire_shallow(Ts, #?STATE{cfg = #cfg{dead_letter_handler = DLH}, + returns = Returns0, + messages = Messages0, + dlx = DlxState0, + discarded_bytes = DiscardedBytes0, + messages_total = Tot, + msg_bytes_enqueue = MsgBytesEnqueue} = State0) -> + + {Expired0, Returns} = case lqueue:peek(Returns0) of + empty -> + {[], Returns0}; + {value, Returned} -> + case msg_is_expired(Ts, Returned) of + true -> + {[Returned], lqueue:drop(Returns0)}; + false -> + {[], Returns0} + end + end, + + {Expired, Messages} = rabbit_fifo_pq:take_while( + fun (Msg) -> msg_is_expired(Ts, Msg) end, + Messages0), + + ExpMsgs = Expired0 ++ Expired, + + {DlxState, _RetainedBytes, DlxEffects} = + discard_or_dead_letter(ExpMsgs, expired, DLH, DlxState0), + + NumExpired = length(ExpMsgs), + + %% calculate total sizes + Size = lists:foldl(fun (Msg, Acc) -> + Header = get_msg_header(Msg), + Acc + get_header(size, Header) + end, 0, ExpMsgs), + + DiscardedSize = Size + (NumExpired * ?ENQ_OVERHEAD), + State = State0#?STATE{dlx = DlxState, + returns = Returns, + messages = Messages, + messages_total = Tot - NumExpired, + discarded_bytes = DiscardedBytes0 + DiscardedSize, + msg_bytes_enqueue = MsgBytesEnqueue - Size}, + {State, DlxEffects}. + expire(RaCmdTs, State0, Effects) -> {Msg, #?STATE{cfg = #cfg{dead_letter_handler = DLH}, dlx = DlxState0, messages_total = Tot, discarded_bytes = DiscardedBytes0, - msg_bytes_enqueue = MsgBytesEnqueue} = State1} = + msg_bytes_enqueue = MsgBytesEnqueue + } = State1} = take_next_msg(State0), {DlxState, _RetainedBytes, DlxEffects} = discard_or_dead_letter([Msg], expired, DLH, DlxState0), @@ -2343,26 +2370,50 @@ expire(RaCmdTs, State0, Effects) -> messages_total = Tot - 1, discarded_bytes = DiscardedBytes0 + DiscardedSize, msg_bytes_enqueue = MsgBytesEnqueue - Size}, - expire_msgs(RaCmdTs, true, State, DlxEffects ++ Effects). - -timer_effect(RaCmdTs, State, Effects) -> - T = case get_next_msg(State) of - ?MSG(_, ?TUPLE(Size, Expiry)) - when is_integer(Size) andalso - is_integer(Expiry) -> - %% Next message contains 'expiry' header. - %% (Re)set timer so that message will be dropped or - %% dead-lettered on time. - max(0, Expiry - RaCmdTs); - ?MSG(_, #{expiry := Expiry}) - when is_integer(Expiry) -> - max(0, Expiry - RaCmdTs); - _ -> - %% Next message does not contain 'expiry' header. - %% Therefore, do not set timer or cancel timer if it was set. - infinity - end, - [{timer, expire_msgs, T} | Effects]. + expire_msgs(RaCmdTs, true, State, Effects ++ DlxEffects). + +timer_effect(#?STATE{messages_total = 0}, Effects) -> + Effects; +timer_effect(#?STATE{returns = Returns, + last_command_time = Ts, + messages = Messages}, Effects) -> + %% TODO: most queues don't use message ttls, to avoid doing this frequently + %% when not required we could keep a flag in the machine state to indicate + %% if a ttl has ever been seen in the queue and avoid this code path based + %% on the value of that flag. + ReturnedExpiry = case lqueue:peek(Returns) of + empty -> + undefined; + {value, Returned} -> + get_header(expiry, get_msg_header(Returned)) + end, + + %% this checks the next messages of all priorities and returnes the smallest + %% expiry time or undefined + NextExpiry = rabbit_fifo_pq:fold_priorities_next( + fun (Msg, Acc) -> + Header = get_msg_header(Msg), + case get_header(expiry, Header) of + undefined -> + Acc; + Expiry when Acc == undefined -> + max(0, Expiry - Ts); + Expiry -> + CalcExpiry = max(0, Expiry - Ts), + case CalcExpiry < Acc of + true -> + CalcExpiry; + false -> + Acc + end + end + end, ReturnedExpiry, Messages), + case NextExpiry of + undefined -> + Effects; + Timeout -> + [{timer, expire_msgs, Timeout} | Effects] + end. update_or_remove_con(Meta, ConsumerKey, #consumer{cfg = #consumer_cfg{lifetime = once}, @@ -2677,6 +2728,13 @@ make_enqueue(Pid, Seq, Msg) msg = Msg, size = ?SIZE(Msg)}. +make_enqueue_old(Pid, Seq, Msg) -> + %% we just use this version of the prop test, + %% TODO: refactor prop test + #enqueue{msg = Msg, + seq = Seq, + pid = Pid}. + -spec make_register_enqueuer(pid()) -> protocol(). make_register_enqueuer(Pid) -> #register_enqueuer{pid = Pid}. @@ -3086,6 +3144,7 @@ do_snapshot(MacVer, Ts, #snapshot{index = _ChIdx, %% message: 32 bytes %% enqueuer: 96 bytes %% consumer: 256 bytes + %% TODO: refine this NumEnqueuers = map_size(Enqueuers), NumConsumers = map_size(Consumers), ApproxSnapSize = 4096 + @@ -3307,6 +3366,9 @@ start_worker(QRef) -> ensure_worker_terminated(#?DLX{consumer = undefined}) -> ok; ensure_worker_terminated(#?DLX{consumer = #dlx_consumer{pid = Pid}}) -> + terminate_dlx_worker(Pid). + +terminate_dlx_worker(Pid) -> case is_local_and_alive(Pid) of true -> %% Note that we can't return a mod_call effect here @@ -3349,15 +3411,20 @@ update_config(OldDLH, NewDLH, QRes, State0) -> [OldDLH, NewDLH, rabbit_misc:rs(QRes)]]}, {State1, Effects0} = switch_from(OldDLH, QRes, State0), {State, Effects} = switch_to(NewDLH, State1, Effects0), - {State, [LogOnLeader|Effects]}. + {State, [LogOnLeader | Effects]}. -switch_from(at_least_once, QRes, State) -> +switch_from(at_least_once, QRes, DlxState) -> %% Switch from at-least-once to some other strategy. %% TODO: do worker teardown in aux handler - ensure_worker_terminated(State), - {Num, Bytes} = dlx_stat(State), + {Num, Bytes} = dlx_stat(DlxState), + Pid = case DlxState of + #?DLX{consumer = #dlx_consumer{pid = P}} -> + P; + _ -> undefined + end, %% Log only on leader. - {#?DLX{}, [{mod_call, logger, info, + {#?DLX{}, [{aux, {dlx, teardown, Pid}}, + {mod_call, logger, info, ["Deleted ~b dead-lettered messages (with total messages size of ~b bytes) in ~ts", [Num, Bytes, rabbit_misc:rs(QRes)]]}]}; switch_from(_, _, State) -> @@ -3367,7 +3434,7 @@ switch_to(at_least_once, _, Effects) -> %% Switch from some other strategy to at-least-once. %% Dlx worker needs to be started on the leader. %% The cleanest way to determine the Ra state of this node is delegation to handle_aux. - {#?DLX{}, [{aux, {dlx, setup}} | Effects]}; + {#?DLX{}, Effects ++ [{aux, {dlx, setup}}]}; switch_to(_, State, Effects) -> {State, Effects}. diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index efeab29c32bb..ba06f5643f5b 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -175,7 +175,6 @@ -record(enqueuer, {next_seqno = 1 :: msg_seqno(), - % out of order enqueues - sorted list unused = ?NIL, status = up :: up | suspected_down, %% it is useful to have a record of when this was blocked @@ -250,7 +249,7 @@ % rabbit_fifo_index can be slow when calculating the smallest % index when there are large gaps but should be faster than gb_trees % for normal appending operations as it's backed by a map - unused_0 = ?NIL, + last_command_time = 0, unused_1 = ?NIL, % consumers need to reflect consumer state at time of snapshot consumers = #{} :: #{consumer_key() => consumer()}, diff --git a/deps/rabbit/src/rabbit_fifo_pq.erl b/deps/rabbit/src/rabbit_fifo_pq.erl index f8687d0e604e..aa018c80b5f7 100644 --- a/deps/rabbit/src/rabbit_fifo_pq.erl +++ b/deps/rabbit/src/rabbit_fifo_pq.erl @@ -20,7 +20,10 @@ from_lqueue/1, indexes/1, get_lowest_index/1, - overview/1 + overview/1, + take_while/2, + any_priority_next/2, + fold_priorities_next/3 ]). -define(STATE, pq). @@ -156,8 +159,75 @@ overview(#?STATE{len = Len, num_active_priorities => map_size(Buckets), lowest_index => get_lowest_index(State)}. +-spec take_while(fun ((msg()) -> boolean()), state()) -> + {[msg()], state()}. +take_while(Fun, #?STATE{len = Len, + buckets = Buckets0} = State) + when is_function(Fun) -> + {Buckets, Acc} = maps:fold( + fun (P, Q0, {B0, Items0}) -> + case take_while(Q0, Fun, Items0) of + {?EMPTY, Items} -> + {maps:remove(P, B0), Items}; + {Q, Items} -> + {B0#{P => Q}, Items} + end + end, + {Buckets0, []}, + maps:iterator(Buckets0, ordered)), + + %% TODO: optimise updates + %% update bitmap + Bitmap = maps:fold(fun (P, _Q, B) -> B bor (1 bsl P) end, 0, Buckets), + + {lists:reverse(Acc), + State#?STATE{len = Len - length(Acc), + buckets = Buckets, + bitmap = Bitmap}}. + +-spec any_priority_next(fun ((msg()) -> boolean()), state()) -> + boolean(). +any_priority_next(Fun, #?STATE{buckets = Buckets0}) + when is_function(Fun) -> + maps_any(Fun, maps:next(maps:iterator(Buckets0))). + +-spec fold_priorities_next(fun ((msg(), Acc) -> Acc), Acc, state()) -> + Acc when Acc :: term(). +fold_priorities_next(Fun, Acc, #?STATE{buckets = Buckets0}) + when is_function(Fun) -> + maps:fold(fun (_P, Q, A) -> + Fun(peek(Q), A) + end, Acc, Buckets0). + %% INTERNAL +maps_any(_Fun, none) -> + false; +maps_any(Fun, {_, Q, I}) -> + case Fun(peek(Q)) of + true -> + true; + false -> + maps_any(Fun, maps:next(I)) + end. + +take_while(?EMPTY, _Fun, Acc) -> + {?EMPTY, Acc}; +take_while(Q, Fun, Acc) -> + case peek(Q) of + empty -> + {Q, Acc}; + Msg -> + case Fun(Msg) of + true -> + take_while(drop(Q), Fun, [Msg | Acc]); + false -> + {Q, Acc} + end + end. + + + %% invariant, if the queue is non empty so is the Out (right) list. in(X, ?EMPTY) -> {1, [], [X]}; diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 3d48d465ce47..100207d4d1dd 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -1987,7 +1987,7 @@ make_ra_conf(Q, ServerId, Membership, MacVersion) Membership, MacVersion). make_ra_conf(Q, ServerId, TickTimeout, - _SnapshotInterval, CheckpointInterval, + _SnapshotInterval, _CheckpointInterval, Membership, MacVersion) -> QName = amqqueue:get_name(Q), #resource{name = QNameBin} = QName, @@ -1998,8 +1998,9 @@ make_ra_conf(Q, ServerId, TickTimeout, Formatter = {?MODULE, format_ra_event, [QName]}, LogCfg = #{uid => UId, min_snapshot_interval => 0, - min_checkpoint_interval => CheckpointInterval, - max_checkpoints => 3}, + % min_checkpoint_interval => CheckpointInterval, + % max_checkpoints => 3, + major_compaction_strategy => {num_minors, 32}}, rabbit_misc:maps_put_truthy(membership, Membership, #{cluster_name => ClusterName, id => ServerId, diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index 3bf89f4ae38f..6a7f9ae8149a 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -2147,6 +2147,8 @@ dead_letter_policy(Config) -> %% Test that messages are at most once dead letter in the correct order %% for reason 'maxlen'. at_most_once_dead_letter_order_maxlen(Config) -> + check_quorum_queues_v8_compat(Config), + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), Ch = rabbit_ct_client_helpers:open_channel(Config, Server), @@ -2261,16 +2263,18 @@ at_most_once_dead_letter_order_delivery_limit(Config) -> #'basic.publish'{routing_key = QQ}, #amqp_msg{payload = <<"m2">>}), - ok = subscribe(Ch, QQ, false), + Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server), + ok = subscribe(Ch2, QQ, false), receive {_, #amqp_msg{payload = P1}} -> ?assertEqual(<<"m1">>, P1) end, receive {_, #amqp_msg{payload = P2}} -> ?assertEqual(<<"m2">>, P2) end, - ok = amqp_channel:call(Ch, #'basic.nack'{delivery_tag = 0, - multiple = true, - requeue = true}), + amqp_channel:close(Ch2), + % ok = amqp_channel:call(Ch, #'basic.nack'{delivery_tag = 0, + % multiple = true, + % requeue = true}), wait_for_consensus(DLQ, Config), wait_for_messages_ready(Servers, ra_name(DLQ), 2), @@ -5337,7 +5341,7 @@ check_quorum_queues_v8_compat(Config) -> true -> ok; false -> - throw({skip, "test will only work on QQ machine version > 8"}) + throw({skip, "test will only work on QQ machine version >= 8"}) end. lists_interleave([], _List) -> diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 65b9336e3f55..6e34535fa573 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -1620,8 +1620,11 @@ single_active_consumer_state_enter_leader_include_waiting_consumers_test(Config) ], {State1, _} = run_log(Config, State0, Entries), Effects = rabbit_fifo:state_enter(leader, State1), - %% 2 effects for each consumer process (channel process), 1 effect for the node, - ?assertEqual(2 * 3 + 1 + 1 + 1, length(Effects)). + ct:pal("Efx ~p", [Effects]), + %% 2 effects for each consumer process (channel process), + %% 1 effect for the node, + %% 1 for decorators + ?assertEqual(2 * 3 + 1 + 1, length(Effects)). single_active_consumer_state_enter_eol_include_waiting_consumers_test(Config) -> Resource = rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), @@ -3223,6 +3226,39 @@ modify_test(Config) -> ok. +priorities_expire_test(Config) -> + State0 = init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r("/", queue, + ?FUNCTION_NAME_B)}), + Pid1 = spawn(fun() -> ok end), + + Entries = + [ + {?LINE, make_enqueue(Pid1, 1, + mk_mc(<<"p1">>, #'P_basic'{priority = 9, + expiration = <<"100">>}))}, + {?LINE, make_enqueue(Pid1, 2, + mk_mc(<<"p1">>, #'P_basic'{priority = 9, + expiration = <<"100000">>}))}, + {?LINE, make_enqueue(Pid1, 3, + mk_mc(<<"p7">>, #'P_basic'{priority = 7, + expiration = <<"100">>}))}, + {?LINE, make_enqueue(Pid1, 4, + mk_mc(<<"p7">>, #'P_basic'{priority = 7, + expiration = <<"100000">>}))}, + {?LINE, make_enqueue(Pid1, 5, + mk_mc(<<"p7b">>, #'P_basic'{priority = 3}))}, + + {?LINE + 101, {timeout, {expire_msgs, shallow}}}, + + ?ASSERT(_, fun(State) -> + ?assertMatch(#{num_messages := 3}, + rabbit_fifo:overview(State)) + end) + ], + {_State2, _} = run_log(Config, State0, Entries), + ok. + %% Utility %% @@ -3232,6 +3268,7 @@ apply(Meta, Entry, State) -> rabbit_fifo:apply(Meta, Entry, State). init_aux(Conf) -> rabbit_fifo:init_aux(Conf). handle_aux(S, T, C, A, A2) -> rabbit_fifo:handle_aux(S, T, C, A, A2). make_checkout(C, S, M) -> rabbit_fifo:make_checkout(C, S, M). +make_enqueue(P, S, M) -> rabbit_fifo:make_enqueue(P, S, M). cid(A) when is_atom(A) -> atom_to_binary(A, utf8). @@ -3242,10 +3279,13 @@ single_active_invariant( #rabbit_fifo{consumers = Cons}) -> end, Cons)). mk_mc(Body) -> + mk_mc(Body, #'P_basic'{}). + +mk_mc(Body, BasicProps) -> mc_amqpl:from_basic_message( #basic_message{routing_keys = [<<"">>], exchange_name = #resource{name = <<"x">>, kind = exchange, virtual_host = <<"v">>}, - content = #content{properties = #'P_basic'{}, + content = #content{properties = BasicProps, payload_fragments_rev = [Body]}}). diff --git a/deps/rabbit/test/rabbit_fifo_pq_SUITE.erl b/deps/rabbit/test/rabbit_fifo_pq_SUITE.erl index a64ee780def7..c53ad03f1490 100644 --- a/deps/rabbit/test/rabbit_fifo_pq_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_pq_SUITE.erl @@ -17,6 +17,8 @@ all() -> all_tests() -> [ basics, + take_while, + any_priority_next, property ]. @@ -80,6 +82,73 @@ basics(_Config) -> empty = rabbit_fifo_pq:out(Q6), ok. +take_while(_Config) -> + Q1 = lists:foldl( + fun ({P, I}, Q) -> + rabbit_fifo_pq:in(P, I, Q) + end, rabbit_fifo_pq:new(), + [ + {1, ?MSG(1)}, {1, ?MSG(2)}, {1, ?MSG(3)}, + {2, ?MSG(1)}, {2, ?MSG(2)}, {2, ?MSG(3)}, + {3, ?MSG(1)}, {3, ?MSG(2)}, {3, ?MSG(3)}, + {4, ?MSG(1)}, {4, ?MSG(2)}, {4, ?MSG(3)}, + {5, ?MSG(1, 10)}, {5, ?MSG(2, 20)}, {5, ?MSG(3, 30)} + ]), + + {Taken, Q2} = rabbit_fifo_pq:take_while(fun (?MSG(I, _)) -> + I < 3 + end, Q1), + ?assertMatch([ + ?MSG(1, 10), ?MSG(2, 20), + ?MSG(1, 1), ?MSG(2, 2), + ?MSG(1, 1), ?MSG(2, 2), + ?MSG(1, 1), ?MSG(2, 2), + ?MSG(1, 1), ?MSG(2, 2) + ], Taken), + + + ?assertEqual(5, rabbit_fifo_pq:len(Q2)), + ?assertEqual(10, length(Taken)), + {?MSG(3, 30), Q3} = rabbit_fifo_pq:out(Q2), + {?MSG(3), Q4} = rabbit_fifo_pq:out(Q3), + {?MSG(3), Q5} = rabbit_fifo_pq:out(Q4), + {?MSG(3), Q6} = rabbit_fifo_pq:out(Q5), + {?MSG(3), _Q7} = rabbit_fifo_pq:out(Q6), + + + {_Taken2, Q} = rabbit_fifo_pq:take_while(fun (?MSG(_, _)) -> + true + end, Q2), + + ct:pal("Q ~p", [Q]), + + ok. + +any_priority_next(_Config) -> + Q0 = rabbit_fifo_pq:new(), + + ?assertNot(rabbit_fifo_pq:any_priority_next(fun (_) -> true end, Q0)), + + Q1 = lists:foldl(fun ({P, I}, Q) -> + rabbit_fifo_pq:in(P, I, Q) + end, Q0, + [ + {1, ?MSG(1)}, {1, ?MSG(2)}, {1, ?MSG(3)}, + {2, ?MSG(1)}, {2, ?MSG(2)}, {2, ?MSG(3)}, + {3, ?MSG(2)}, {3, ?MSG(3)}, + {4, ?MSG(1)}, {4, ?MSG(2)}, {4, ?MSG(3)}, + {5, ?MSG(1)}, {5, ?MSG(2)}, {5, ?MSG(3)} + ]), + + ?assert(rabbit_fifo_pq:any_priority_next(fun (?MSG(I, _)) -> + I > 1 + end, Q1)), + ?assertNot(rabbit_fifo_pq:any_priority_next(fun (?MSG(I, _)) -> + I > 6 + end, Q1)), + + ok. + hi_is_prioritised(_Config) -> Q0 = rabbit_fifo_q:new(), %% when `hi' has a lower index than the next 'no' then it is still From b909a815a39f552ecbae477cd7155ad326a07a5a Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 26 Nov 2025 17:29:47 +0000 Subject: [PATCH 28/28] WIP: consumer timeouts in queue --- deps/rabbit/src/rabbit_fifo.erl | 88 +++++++++++++++---------- deps/rabbit/src/rabbit_fifo.hrl | 29 +++++--- deps/rabbit/test/quorum_queue_SUITE.erl | 3 +- deps/rabbit/test/rabbit_fifo_SUITE.erl | 26 ++++---- 4 files changed, 91 insertions(+), 55 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 5f6a1caf1fc8..3773b51344cb 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -219,6 +219,7 @@ update_config(Conf, State) -> competing end, Cfg = State#?STATE.cfg, + DefConsumerTimeout = maps:get(consumer_timeout, Conf, 1800), LastActive = maps:get(created, Conf, undefined), State#?STATE{cfg = Cfg#cfg{dead_letter_handler = DLH, @@ -228,7 +229,8 @@ update_config(Conf, State) -> consumer_strategy = ConsumerStrategy, delivery_limit = DeliveryLimit, expires = Expires, - msg_ttl = MsgTTL}, + msg_ttl = MsgTTL, + default_consumer_timeout_s = DefConsumerTimeout}, last_active = LastActive}. % msg_ids are scoped per consumer @@ -677,7 +679,7 @@ live_indexes(#?STATE{cfg = #cfg{}, DlxIndexes, Returns), maps:fold(fun (_Cid, #consumer{checked_out = Ch}, Acc0) -> maps:fold( - fun (_MsgId, Msg, Acc) -> + fun (_MsgId, ?C_MSG(Msg), Acc) -> [get_msg_idx(Msg) | Acc] end, Acc0, Ch) end, RtnIndexes, Consumers). @@ -704,8 +706,11 @@ snapshot_installed(_Meta, #?MODULE{cfg = #cfg{}, Acc) -> case node(Pid) == node() of true -> - Iter = maps:iterator(Checked, reversed), - Acc#{{Tag, Pid} => maps:to_list(Iter)}; + Iter = maps:iterator(Checked, ordered), + Msgs = maps:fold(fun (K, ?C_MSG(M), Ac0) -> + [{K, M} | Ac0] + end, [], Iter), + Acc#{{Tag, Pid} => Msgs}; false -> Acc end @@ -715,8 +720,14 @@ snapshot_installed(_Meta, #?MODULE{cfg = #cfg{}, convert_v7_to_v8(#{system_time := Ts} = _Meta, StateV7) -> %% the structure is intact for now Cons0 = element(#?STATE.consumers, StateV7), - Cons = maps:map(fun (_CKey, #consumer{status = suspected_down} = C) -> - C#consumer{status = {suspected_down, up}}; + %% TODO: use default for now + Timeout = Ts + 1_800_000, + Cons = maps:map(fun (_CKey, #consumer{status = suspected_down, + checked_out = Ch0} = C) -> + Ch = maps:map(fun (_, M) -> ?C_MSG(Timeout, M) end, + Ch0), + C#consumer{status = {suspected_down, up}, + checked_out = Ch}; (_CKey, C) -> C end, Cons0), @@ -732,6 +743,7 @@ convert_v7_to_v8(#{system_time := Ts} = _Meta, StateV7) -> StateV8#?STATE{discarded_bytes = 0, messages = Pq, consumers = Cons, + next_consumer_timeout = Timeout, last_command_time = Ts}. purge_node(Meta, Node, State, Effects) -> @@ -939,7 +951,7 @@ get_checked_out(CKey, From, To, #?STATE{consumers = Consumers}) -> case find_consumer(CKey, Consumers) of {_CKey, #consumer{checked_out = Checked}} -> [begin - Msg = maps:get(K, Checked), + ?C_MSG(Msg) = maps:get(K, Checked), I = get_msg_idx(Msg), H = get_msg_header(Msg), {K, {I, H}} @@ -974,9 +986,10 @@ which_module(8) -> ?MODULE. gc = #aux_gc{} :: #aux_gc{}, tick_pid :: undefined | pid(), cache = #{} :: map(), - last_checkpoint :: tuple() | #snapshot{}, - bytes_in = 0 :: non_neg_integer(), - bytes_out = 0 :: non_neg_integer()}). + last_checkpoint :: tuple() | #snapshot{} + % bytes_in = 0 :: non_neg_integer(), + % bytes_out = 0 :: non_neg_integer() + }). init_aux(Name) when is_atom(Name) -> %% TODO: catch specific exception throw if table already exists @@ -1001,9 +1014,10 @@ handle_aux(RaftState, Tag, Cmd, AuxV3, RaAux) gc = element(5, AuxV3), tick_pid = element(6, AuxV3), cache = element(7, AuxV3), - last_checkpoint = element(8, AuxV3), - bytes_in = element(9, AuxV3), - bytes_out = 0}, + last_checkpoint = element(8, AuxV3) + % bytes_in = element(9, AuxV3), + % bytes_out = 0 + }, handle_aux(RaftState, Tag, Cmd, AuxV4, RaAux); handle_aux(leader, cast, eval, #?AUX{last_decorators_state = LastDec, @@ -1048,14 +1062,14 @@ handle_aux(_RaftState, cast, eval, {Check, Effects} = do_snapshot(EffMacVer, Ts, Check0, RaAux, DiscardedBytes, false), {no_reply, Aux0#?AUX{last_checkpoint = Check}, RaAux, Effects}; -handle_aux(_RaftState, cast, {bytes_in, {MetaSize, BodySize}}, - #?AUX{bytes_in = Bytes} = Aux0, - RaAux) -> - {no_reply, Aux0#?AUX{bytes_in = Bytes + MetaSize + BodySize}, RaAux, []}; -handle_aux(_RaftState, cast, {bytes_out, BodySize}, - #?AUX{bytes_out = Bytes} = Aux0, - RaAux) -> - {no_reply, Aux0#?AUX{bytes_out = Bytes + BodySize}, RaAux, []}; +% handle_aux(_RaftState, cast, {bytes_in, {MetaSize, BodySize}}, +% #?AUX{bytes_in = Bytes} = Aux0, +% RaAux) -> +% {no_reply, Aux0#?AUX{bytes_in = Bytes + MetaSize + BodySize}, RaAux, []}; +% handle_aux(_RaftState, cast, {bytes_out, BodySize}, +% #?AUX{bytes_out = Bytes} = Aux0, +% RaAux) -> +% {no_reply, Aux0#?AUX{bytes_out = Bytes + BodySize}, RaAux, []}; handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, consumer_key = Key} = Ret, Corr, Pid}, Aux0, RaAux0) -> @@ -1066,7 +1080,7 @@ handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, {ConsumerKey, #consumer{checked_out = Checked}} -> {RaAux, ToReturn} = maps:fold( - fun (MsgId, Msg, {RA0, Acc}) -> + fun (MsgId, ?C_MSG(Msg), {RA0, Acc}) -> Idx = get_msg_idx(Msg), Header = get_msg_header(Msg), %% it is possible this is not found if the consumer @@ -1113,7 +1127,7 @@ handle_aux(_, _, {get_checked_out, ConsumerKey, MsgIds}, Aux0, RaAux0) -> #{ConsumerKey := #consumer{checked_out = Checked}} -> {RaState, IdMsgs} = maps:fold( - fun (MsgId, Msg, {S0, Acc}) -> + fun (MsgId, ?C_MSG(Msg), {S0, Acc}) -> Idx = get_msg_idx(Msg), Header = get_msg_header(Msg), %% it is possible this is not found if the consumer @@ -1201,6 +1215,11 @@ handle_aux(leader, _, {dlx, setup}, Aux, RaAux) -> {no_reply, Aux, RaAux}; handle_aux(_, _, {dlx, teardown, Pid}, Aux, RaAux) -> terminate_dlx_worker(Pid), + {no_reply, Aux, RaAux}; +handle_aux(_, _, Unhandled, Aux, RaAux) -> + #?STATE{cfg = #cfg{resource = QR}} = ra_aux:machine_state(RaAux), + ?LOG_DEBUG("~ts: rabbit_fifo: unhandled aux command ~P", + [rabbit_misc:rs(QR), Unhandled, 10]), {no_reply, Aux, RaAux}. @@ -1782,8 +1801,8 @@ return(Meta, ConsumerKey, lists:foldl( fun(MsgId, Acc = {S0, E0}) -> case Checked of - #{MsgId := Msg} -> - return_one(Meta, MsgId, Msg, IncrDelCount, Anns, + #{MsgId := CMsg} -> + return_one(Meta, MsgId, CMsg, IncrDelCount, Anns, S0, E0, ConsumerKey); #{} -> Acc @@ -1806,7 +1825,7 @@ complete(Meta, ConsumerKey, [MsgId], messages_total = Tot} = State0, Effects) -> case maps:take(MsgId, Checked0) of - {Msg, Checked} -> + {?C_MSG(Msg), Checked} -> Hdr = get_msg_header(Msg), SettledSize = get_header(size, Hdr), Con = Con0#consumer{checked_out = Checked, @@ -1828,7 +1847,7 @@ complete(Meta, ConsumerKey, MsgIds, = lists:foldl( fun (MsgId, {S0, Ch0}) -> case maps:take(MsgId, Ch0) of - {Msg, Ch} -> + {?C_MSG(Msg), Ch} -> Hdr = get_msg_header(Msg), S = get_header(size, Hdr) + S0, {S, Ch}; @@ -1964,7 +1983,7 @@ annotate_msg(Header, Msg0) -> Msg0 end. -return_one(Meta, MsgId, Msg0, DeliveryFailed, Anns, +return_one(Meta, MsgId, ?C_MSG(Msg0), DeliveryFailed, Anns, #?STATE{returns = Returns, consumers = Consumers, dlx = DlxState0, @@ -2005,8 +2024,8 @@ return_one(Meta, MsgId, Msg0, DeliveryFailed, Anns, return_all(Meta, #?STATE{consumers = Cons} = State0, Effects0, ConsumerKey, #consumer{checked_out = Checked} = Con, DeliveryFailed) -> State = State0#?STATE{consumers = Cons#{ConsumerKey => Con}}, - maps:fold(fun (MsgId, Msg, {S, E}) -> - return_one(Meta, MsgId, Msg, DeliveryFailed, #{}, + maps:fold(fun (MsgId, CMsg, {S, E}) -> + return_one(Meta, MsgId, CMsg, DeliveryFailed, #{}, S, E, ConsumerKey) end, {State, Effects0}, maps:iterator(Checked, ordered)). @@ -2248,7 +2267,10 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> credit = Credit, delivery_count = DelCnt0, cfg = Cfg} = Con0 -> - Checked = maps:put(Next, Msg, Checked0), + Timeout = (Ts div 1000) + State0#?STATE.cfg#cfg.default_consumer_timeout_s, + Checked = maps:put(Next, + ?C_MSG(Timeout * 1000, Msg), + Checked0), DelCnt = case credit_api_v2(Cfg) of true -> add(DelCnt0, 1); false -> DelCnt0 + 1 @@ -2950,7 +2972,7 @@ smallest_raft_index(#?STATE{messages = Messages, min(get_msg_idx(Msg), Acc) end, Min0, Returns), Min2 = maps:fold(fun (_Cid, #consumer{checked_out = Ch}, Acc0) -> - maps:fold(fun (_MsgId, Msg, Acc) -> + maps:fold(fun (_MsgId, ?C_MSG(Msg), Acc) -> min(get_msg_idx(Msg), Acc) end, Acc0, Ch) end, Min1, Consumers), @@ -3185,7 +3207,7 @@ discard(Meta, MsgIds, ConsumerKey, case maps:get(Id, Checked, undefined) of undefined -> false; - Msg0 -> + ?C_MSG(Msg0) -> {true, incr_msg_headers(Msg0, DelFailed, Anns)} end end, MsgIds), diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index ba06f5643f5b..6a22b13d363b 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -12,6 +12,8 @@ %% Raw message data is always stored on disk. -define(MSG(Index, Header), ?TUPLE(Index, Header)). +-define(C_MSG(Timeout, Msg), {Timeout, Msg}). +-define(C_MSG(Msg), {_, Msg}). -define(NIL, []). -define(IS_HEADER(H), @@ -82,6 +84,9 @@ -type msg() :: packed_msg() | optimised_tuple(ra:index(), msg_header()). +%% a consumer message +-type c_msg() :: {LockExpiration :: milliseconds(), msg()}. + -type delivery_msg() :: {msg_id(), {msg_header(), raw_msg()}}. %% A tuple consisting of the message id, and the headered message. @@ -115,7 +120,8 @@ username => binary(), prefetch => non_neg_integer(), args => list(), - priority => non_neg_integer() + priority => 0..255, + timeout => milliseconds() }. %% static meta data associated with a consumer @@ -139,7 +145,9 @@ -define(LOW_LIMIT, 0.8). -define(DELIVERY_CHUNK_LIMIT_B, 128_000). +-type seconds() :: non_neg_integer(). -type milliseconds() :: non_neg_integer(). + -record(consumer_cfg, {meta = #{} :: consumer_meta(), pid :: pid(), @@ -157,9 +165,13 @@ -record(consumer, {cfg = #consumer_cfg{}, - status = up :: consumer_status() | {suspected_down, consumer_status()}, + status = up :: consumer_status() | + {suspected_down, consumer_status()} | + %% a message has been pending for longer than the + %% consumer timeout + {timeout, consumer_status()}, next_msg_id = 0 :: msg_id(), - checked_out = #{} :: #{msg_id() => msg()}, + checked_out = #{} :: #{msg_id() => c_msg()}, %% max number of messages that can be sent %% decremented for each delivery credit = 0 :: non_neg_integer(), @@ -200,7 +212,7 @@ delivery_limit :: option(non_neg_integer()), expires :: option(milliseconds()), msg_ttl :: option(milliseconds()), - unused_2 = ?NIL, + default_consumer_timeout_s = 1800 :: seconds(), unused_3 = ?NIL }). @@ -250,7 +262,7 @@ % index when there are large gaps but should be faster than gb_trees % for normal appending operations as it's backed by a map last_command_time = 0, - unused_1 = ?NIL, + next_consumer_timeout = undefined :: option(milliseconds()), % consumers need to reflect consumer state at time of snapshot consumers = #{} :: #{consumer_key() => consumer()}, % consumers that require further service are queued here @@ -273,8 +285,8 @@ queue_resource := rabbit_types:r('queue'), dead_letter_handler => dead_letter_handler(), become_leader_handler => applied_mfa(), - checkpoint_min_indexes => non_neg_integer(), - checkpoint_max_indexes => non_neg_integer(), + % checkpoint_min_indexes => non_neg_integer(), + % checkpoint_max_indexes => non_neg_integer(), max_length => non_neg_integer(), max_bytes => non_neg_integer(), overflow_strategy => drop_head | reject_publish, @@ -282,5 +294,6 @@ delivery_limit => non_neg_integer() | -1, expires => non_neg_integer(), msg_ttl => non_neg_integer(), - created => non_neg_integer() + created => non_neg_integer(), + consumer_timeout => seconds() }. diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index 6a7f9ae8149a..a0dc1f72387d 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -1787,7 +1787,8 @@ consumer_message_is_delevered_after_snapshot(Config) -> %% then purge #'queue.purge_ok'{} = amqp_channel:call(Ch0, #'queue.purge'{queue = QQ}), - MacVer = lists:min([V || {ok, V} <- erpc:multicall(Nodes, rabbit_fifo, version, [])]), + MacVer = lists:min([V || {ok, V} <- + erpc:multicall(Nodes, rabbit_fifo, version, [])]), ct:pal("machine version is ~b", [MacVer]), %% only await snapshot if all members have at least machine version 8 diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 6e34535fa573..19122fd3bdcc 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -3189,10 +3189,10 @@ modify_test(Config) -> fun (#rabbit_fifo{consumers = #{CK1 := #consumer{checked_out = Ch}}}) -> ?assertMatch( - ?MSG(_, #{acquired_count := 1, - anns := #{<<"x-opt-blah">> := <<"blah1">>}} = H) + ?C_MSG(?MSG(_, #{acquired_count := 1, + anns := #{<<"x-opt-blah">> := <<"blah1">>}} = H)) when not is_map_key(delivery_count, H), - maps:get(1, Ch)) + maps:get(1, Ch)) end), %% delivery_failed = true does increment delivery_count {?LINE, rabbit_fifo:make_modify(CK1, [1], true, false, @@ -3203,9 +3203,9 @@ modify_test(Config) -> fun (#rabbit_fifo{consumers = #{CK1 := #consumer{checked_out = Ch}}}) -> ?assertMatch( - ?MSG(_, #{delivery_count := 1, - acquired_count := 2, - anns := #{<<"x-opt-blah">> := <<"blah2">>}}), + ?C_MSG(?MSG(_, #{delivery_count := 1, + acquired_count := 2, + anns := #{<<"x-opt-blah">> := <<"blah2">>}})), maps:get(2, Ch)) end), %% delivery_failed = true and undeliverable_here = true is the same as discard @@ -3214,13 +3214,13 @@ modify_test(Config) -> ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{next_msg_id = 3, checked_out = Ch}}} when map_size(Ch) == 0, - fun (#rabbit_fifo{dlx = #rabbit_fifo_dlx{discards = Discards}}) -> - ?assertMatch([[_| - ?MSG(_, #{delivery_count := 2, - acquired_count := 3, - anns := #{<<"x-opt-blah">> := <<"blah3">>}})]], - lqueue:to_list(Discards)) - end) + fun (#rabbit_fifo{dlx = #rabbit_fifo_dlx{discards = Discards}}) -> + ?assertMatch([[_| + ?MSG(_, #{delivery_count := 2, + acquired_count := 3, + anns := #{<<"x-opt-blah">> := <<"blah3">>}})]], + lqueue:to_list(Discards)) + end) ], {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1),