diff --git a/deps/rabbit/Makefile b/deps/rabbit/Makefile index 89048261923c..f5e7f40365e4 100644 --- a/deps/rabbit/Makefile +++ b/deps/rabbit/Makefile @@ -263,7 +263,7 @@ define ct_master.erl endef PARALLEL_CT_SET_1_A = unit_rabbit_ssl unit_cluster_formation_locking_mocks unit_cluster_formation_sort_nodes unit_collections unit_config_value_encryption unit_connection_tracking -PARALLEL_CT_SET_1_B = amqp_address amqp_auth amqp_credit_api_v2 amqp_filter_prop amqp_filter_sql amqp_filter_sql_unit amqp_dotnet amqp_jms signal_handling single_active_consumer unit_access_control_authn_authz_context_propagation unit_access_control_credential_validation unit_amqp091_content_framing unit_amqp091_server_properties unit_app_management +PARALLEL_CT_SET_1_B = amqp_address amqp_auth amqp_filter_prop amqp_filter_sql amqp_filter_sql_unit amqp_dotnet amqp_jms signal_handling single_active_consumer unit_access_control_authn_authz_context_propagation unit_access_control_credential_validation unit_amqp091_content_framing unit_amqp091_server_properties unit_app_management PARALLEL_CT_SET_1_C = amqp_proxy_protocol amqpl_consumer_ack backing_queue bindings rabbit_db_maintenance rabbit_db_msup rabbit_db_policy rabbit_db_queue rabbit_db_topic_exchange cluster_limit cluster_minority term_to_binary_compat_prop topic_permission transactions unicode unit_access_control PARALLEL_CT_SET_1_D = amqqueue_backward_compatibility channel_interceptor channel_operation_timeout classic_queue classic_queue_prop config_schema peer_discovery_dns peer_discovery_tmp_hidden_node per_node_limit per_user_connection_channel_limit diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 0c195b04ab29..33e741312bad 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 @@ -42,6 +44,8 @@ %% ra_machine callbacks init/1, apply/3, + live_indexes/1, + snapshot_installed/4, state_enter/2, tick/2, overview/1, @@ -58,7 +62,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, @@ -69,13 +72,12 @@ 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, + get_msg_from_cmd/1, %% protocol helpers make_enqueue/3, @@ -142,7 +144,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{} | @@ -185,8 +186,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) -> @@ -195,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), @@ -219,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,17 +228,33 @@ 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()}. -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, #register_enqueuer{pid = Pid}, +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) -> State = case maps:is_key(Pid, Enqueuers0) of @@ -257,8 +271,8 @@ apply(_Meta, #register_enqueuer{pid = Pid}, ok end, {State, Res, [{monitor, process, Pid}]}; -apply(Meta, #settle{msg_ids = MsgIds, - consumer_key = Key}, +apply_(Meta, #settle{msg_ids = MsgIds, + consumer_key = Key}, #?STATE{consumers = Consumers} = State) -> case find_consumer(Key, Consumers) of {ConsumerKey, Con0} -> @@ -269,20 +283,8 @@ 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}, +apply_(Meta, #discard{consumer_key = ConsumerKey, + msg_ids = MsgIds}, #?STATE{consumers = Consumers } = State0) -> case find_consumer(ConsumerKey, Consumers) of {ActualConsumerKey, #consumer{} = Con} -> @@ -290,9 +292,9 @@ apply(Meta, #discard{consumer_key = ConsumerKey, _ -> {State0, ok} end; -apply(Meta, #return{consumer_key = ConsumerKey, - msg_ids = MsgIds}, - #?STATE{consumers = Cons} = State) -> +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, @@ -300,12 +302,12 @@ 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) -> +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 -> @@ -318,16 +320,13 @@ 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, + 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 @@ -339,24 +338,16 @@ apply(#{index := Idx, 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} = 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, []} end; -apply(Meta, #credit{consumer_key = ConsumerKey} = Credit, +apply_(Meta, #credit{consumer_key = ConsumerKey} = Credit, #?STATE{consumers = Cons} = State) -> case Cons of #{ConsumerKey := Con} -> @@ -370,15 +361,16 @@ 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, - system_time := Ts, - from := From} = Meta, #checkout{spec = {dequeue, Settlement}, - meta = ConsumerMeta, - consumer_id = ConsumerId}, - #?STATE{consumers = Consumers} = State00) -> +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 @@ -394,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 -> @@ -424,9 +417,9 @@ apply(#{index := Index, {State, {dequeue, empty}, Effects} end end; -apply(#{index := _Idx} = Meta, - #checkout{spec = Spec, - consumer_id = ConsumerId}, State0) +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 @@ -440,10 +433,10 @@ apply(#{index := _Idx} = Meta, error -> {State0, {error, consumer_not_found}, []} end; -apply(#{index := Idx} = Meta, - #checkout{spec = Spec0, - meta = ConsumerMeta, - consumer_id = {_, Pid} = ConsumerId}, State0) -> +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 @@ -483,32 +476,10 @@ 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, - returns = Returns, - ra_indexes = Indexes0 - } = State0) -> +apply_(#{index := Index}, #purge{}, + #?STATE{messages_total = Total} = 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 @@ -518,22 +489,21 @@ apply(#{index := Index}, #purge{}, {State, _, Effects} = evaluate_limit(Index, false, 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(#{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) -> +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} = - 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 @@ -555,7 +525,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), @@ -570,10 +540,9 @@ 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, - {down, Pid, noconnection}, - #?STATE{consumers = Cons0, +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 @@ -586,7 +555,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 -> @@ -597,7 +566,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 @@ -610,20 +579,17 @@ apply(#{machine_version := Vsn, 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(#{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}; @@ -640,7 +606,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) -> @@ -651,95 +617,107 @@ 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}, {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, - #update_config{config = #{} = Conf}, - #?STATE{cfg = #cfg{dead_letter_handler = OldDLH, - resource = QRes}, - dlx = DlxState0} = State0) -> +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, []}. -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) -> +-spec live_indexes(state()) -> [ra:index()]. +live_indexes(#?STATE{cfg = #cfg{}, + returns = Returns, + messages = Messages, + consumers = Consumers, + dlx = #?DLX{discards = Discards}}) -> + MsgsIdxs = rabbit_fifo_q:indexes(Messages), + DlxIndexes = lqueue:fold(fun (?TUPLE(_, Msg), Acc) -> + I = get_msg_idx(Msg), + [I | Acc] + end, MsgsIdxs, Discards), + 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, Acc) -> + [get_msg_idx(Msg) | Acc] + end, Acc0, Ch) + end, RtnIndexes, Consumers). + +-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{}, + 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), + Effs = add_delivery_effects([], SendAcc, State), + Effs. + +convert_v7_to_v8(#{} = _Meta, StateV7) -> + StateV8 = StateV7, + StateV8#?STATE{discarded_bytes = 0, + unused_0 = ?NIL}. + +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), @@ -748,7 +726,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). @@ -807,40 +785,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), @@ -850,11 +822,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) -> @@ -868,11 +847,11 @@ 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, dlx = DlxState, + discarded_bytes = DiscardedBytes, messages = Messages, returns = Returns, waiting_consumers = WaitingConsumers} = State) -> @@ -886,8 +865,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), @@ -913,17 +891,11 @@ 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) + 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()) -> @@ -932,7 +904,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)]; _ -> @@ -940,36 +914,32 @@ 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; 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()}). +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_v4). + +-record(snapshot, {index :: ra:index(), + timestamp :: milliseconds(), + messages_total = 0 :: non_neg_integer(), + discarded_bytes = 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{}, + last_checkpoint :: tuple() | #snapshot{}, bytes_in = 0 :: non_neg_integer(), bytes_out = 0 :: non_neg_integer()}). @@ -978,37 +948,48 @@ 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, - 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); + 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 -> + 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, + % 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), - {Check, Effects0} = do_checkpoints(Ts, Check0, RaAux, BytesIn, false), + 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_snapshot(EffMacVer, Ts, Check0, RaAux, + DiscardedBytes, false), %% this is called after each batch of commands have been applied %% set timer for message expire @@ -1024,16 +1005,21 @@ handle_aux(leader, cast, eval, last_decorators_state = NewLast}, RaAux, Effects} end; handle_aux(_RaftState, cast, eval, - #?AUX{last_checkpoint = Check0, - bytes_in = BytesIn} = Aux0, - RaAux) -> + #?AUX{last_checkpoint = Check0} = Aux0, RaAux) -> Ts = erlang:system_time(millisecond), - {Check, Effects} = do_checkpoints(Ts, Check0, RaAux, BytesIn, false), + 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), {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) -> @@ -1044,12 +1030,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} @@ -1089,13 +1077,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 @@ -1104,22 +1093,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}; @@ -1152,30 +1129,39 @@ 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; 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) -> + #?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)]), - {Check, Effects} = do_checkpoints(Ts, Check0, RaAux, BytesIn, true), + [rabbit_misc:rs(QR), ra_aux:last_applied(RaAux)]), + EffMacVer = ra_aux:effective_machine_version(RaAux), + {Check, Effects} = do_snapshot(EffMacVer, Ts, Check0, RaAux, + 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, @@ -1237,9 +1223,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. @@ -1339,7 +1322,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 @@ -1347,11 +1330,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, @@ -1376,11 +1364,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, @@ -1389,39 +1372,9 @@ 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. -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 @@ -1557,7 +1510,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), @@ -1616,9 +1569,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} -> @@ -1630,16 +1581,18 @@ 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{discarded_bytes = DiscardedBytes0} = 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, State1} -> + Header = get_msg_header(Msg), + State = decr_total(add_bytes_drop(Header, State1)), #?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)}; + dlx = DlxState} = State, + {_, _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. @@ -1670,7 +1623,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) -> @@ -1702,44 +1655,49 @@ 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}, + {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), + 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, - 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, + {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), + Msg = make_msg(RaftIdx, Header), Enq = Enq0#enqueuer{next_seqno = MsgSeqNo + 1}, MsgCache = case can_immediately_deliver(State0) of true -> @@ -1748,8 +1706,7 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, undefined end, PTag = priority_tag(RawMsg), - State = State0#?STATE{msg_bytes_enqueue = Enqueue + BodySize, - enqueue_count = EnqCount + 1, + State = State0#?STATE{msg_bytes_enqueue = BytesEnqueued + Size, messages_total = Total + 1, messages = rabbit_fifo_q:in(PTag, Msg, Messages), enqueuers = Enqueuers0#{From => Enq}, @@ -1759,13 +1716,17 @@ 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(#{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. @@ -1786,56 +1747,55 @@ 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 complete(Meta, ConsumerKey, [MsgId], #consumer{checked_out = Checked0} = Con0, - #?STATE{ra_indexes = Indexes0, - msg_bytes_checkout = BytesCheckout, - messages_total = Tot} = State0) -> + #?STATE{msg_bytes_checkout = BytesCheckout, + discarded_bytes = DiscBytes, + 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), - 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, + discarded_bytes = DiscBytes + SettledSize + ?ENQ_OVERHEAD, + messages_total = Tot - 1}, + 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) -> - {SettledSize, Checked, Indexes} + #?STATE{msg_bytes_checkout = BytesCheckout, + discarded_bytes = DiscBytes, + messages_total = Tot} = State0, Effects) -> + {SettledSize, Checked} = lists:foldl( - fun (MsgId, {S0, Ch0, Idxs}) -> + 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, 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, + discarded_bytes = DiscBytes + SettledSize + (Len *?ENQ_OVERHEAD), + messages_total = Tot - Len}, + Effects}. increase_credit(#consumer{cfg = #consumer_cfg{lifetime = once}, credit = Credit}, _) -> @@ -1862,11 +1822,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}, @@ -1887,8 +1848,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) -> @@ -1905,8 +1867,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) -> @@ -1943,23 +1912,31 @@ 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, + discarded_bytes = DiscardedBytes0, 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), + %% 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} = - rabbit_fifo_dlx:discard([Msg], delivery_limit, DLH, DlxState0), - State1 = State0#?STATE{dlx = DlxState}, - State = complete(Meta, ConsumerKey, [MsgId], Con0, State1), - {State, DlxEffects ++ Effects0}; + {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}; _ -> Checked = maps:remove(MsgId, Checked0), Con = Con0#consumer{checked_out = Checked, @@ -1988,7 +1965,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, @@ -2000,7 +1977,7 @@ checkout(#{index := Index} = Meta, end. 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 @@ -2011,8 +1988,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(_Index, Result, #?STATE{cfg = #cfg{max_length = undefined, @@ -2083,15 +2060,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 -> @@ -2107,9 +2084,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}}; @@ -2117,11 +2092,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, Messages} -> + {Msg, State#?STATE{messages = Messages}} end end. @@ -2139,10 +2111,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) -> @@ -2169,7 +2149,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) -> @@ -2255,25 +2235,27 @@ 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, - ra_indexes = Indexes0, 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), - Indexes = rabbit_fifo_index:delete(Idx, Indexes0), + {DlxState, _RetainedBytes, DlxEffects} = + discard_or_dead_letter([Msg], expired, DLH, DlxState0), + Header = get_msg_header(Msg), + Size = get_header(size, Header), + DiscardedSize = Size + ?ENQ_OVERHEAD, State = State1#?STATE{dlx = DlxState, - ra_indexes = Indexes, messages_total = Tot - 1, - msg_bytes_enqueue = - MsgBytesEnqueue - get_header(size, Header)}, + discarded_bytes = DiscardedBytes0 + DiscardedSize, + msg_bytes_enqueue = MsgBytesEnqueue - Size}, expire_msgs(RaCmdTs, true, State, DlxEffects ++ Effects). timer_effect(RaCmdTs, State, Effects) -> @@ -2576,7 +2558,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). @@ -2587,7 +2569,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). @@ -2598,17 +2580,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) -> @@ -2616,14 +2596,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}. @@ -2658,18 +2631,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(). @@ -2724,45 +2690,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) - when node(P) =:= Node -> - [P | Acc]; - (_, _, Acc) -> - Acc - end, [], Cons0, Vsn), - Enqs = rabbit_fifo_maps:fold(fun(P, #enqueuer{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, Cons, Enqs0, Vsn), + end, Cons, maps:iterator(Enqs0, ordered)), lists:foldl(fun({_Key, #consumer{cfg = #consumer_cfg{pid = P}, status = suspected_down}}, Acc) @@ -2816,7 +2782,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); @@ -2825,22 +2791,30 @@ 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) -> + convert(Meta, 8, To, convert_v7_to_v8(Meta, State)). smallest_raft_index(#?STATE{messages = Messages, - ra_indexes = Indexes, - dlx = DlxState}) -> - SmallestDlxRaIdx = rabbit_fifo_dlx:smallest_raft_index(DlxState), + dlx = #?DLX{discards = Discards}} = State) -> + SmallestDlxRaIdx = lqueue:fold(fun (?TUPLE(_, Msg), Acc) -> + min(get_msg_idx(Msg), Acc) + end, undefined, Discards), 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, 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). make_requeue(ConsumerKey, Notify, [{MsgId, Idx, Header, Msg}], Acc) -> lists:reverse([{append, @@ -2878,11 +2852,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}) -> @@ -2994,85 +2968,63 @@ priority_tag(Msg) -> no end. - -do_checkpoints(Ts, #checkpoint{index = ChIdx, - timestamp = ChTime, - smallest_index = LastSmallest, - bytes_in = LastBytesIn, - indexes = MinIndexes} = Check0, - RaAux, BytesIn, Force) -> +do_snapshot(MacVer, Ts, Ch, + 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, DiscardedBytes, Force); +do_snapshot(MacVer, Ts, #snapshot{index = _ChIdx, + timestamp = SnapTime, + discarded_bytes = LastDiscardedBytes} = Snap0, + RaAux, DiscardedBytes, Force) + when is_integer(MacVer) andalso 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, + #?STATE{consumers = Consumers, + enqueuers = Enqueuers} = MacState = ra_aux:machine_state(RaAux), + TimeSince = Ts - SnapTime, 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} = + %% if the approximate snapshot size * 2 can be reclaimed it is worth + %% taking a snapshot + %% 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, {?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 + EnoughDataRemoved) orelse + 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)]}; + {#snapshot{index = LastAppliedIdx, + timestamp = Ts, + messages_total = MsgsTot, + discarded_bytes = DiscardedBytes}, + [{release_cursor, LastAppliedIdx, MacState}]}; false -> - {Check0#checkpoint{smallest_index = NewSmallest}, - release_cursor(LastSmallest, NewSmallest)} + {Snap0, []} 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}, + discarded_bytes = DiscardedBytes0, dlx = DlxState0} = State0) -> %% We publish to dead-letter exchange in the same order %% as messages got rejected by the client. @@ -3085,9 +3037,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) -> @@ -3112,9 +3065,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 -> @@ -3125,3 +3080,358 @@ exec_read(Flru0, ReadPlan, Msgs) -> %% retry without segment cache exec_read(undefined, ReadPlan, Msgs) end. + +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), + {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)]}]}; +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 = 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(Hdr, rabbit_fifo:get_msg_from_cmd(Cmd)) + end || Msg <- 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, #?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, + 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), 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), + {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_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 b8b69bff7f45..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. @@ -105,6 +128,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 @@ -179,6 +203,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 @@ -186,9 +235,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{}}, @@ -197,19 +248,21 @@ % 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()}, % 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 %% 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/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index 462d80362e93..494b21472263 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -147,13 +147,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, @@ -377,24 +377,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), %% ??? @@ -418,19 +406,15 @@ checkout(ConsumerTag, CreditMode, #{} = Meta, NextMsgId - 1 end end, - DeliveryCount = case rabbit_fifo:is_v4() of - true -> credit_api_v2; - false -> {credit_api_v1, 0} - end, + DeliveryCount = credit_api_v2, 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, - delivery_count = DeliveryCount}, - CDels0), + SDels = maps:update_with(ConsumerTag, + fun (C) -> C#consumer{ack = Ack} end, + #consumer{key = ConsumerKey, + last_msg_id = LastMsgId, + ack = Ack, + delivery_count = DeliveryCount}, + CDels0), {ok, Reply, State0#state{leader = Leader, consumers = SDels}}; Err -> @@ -1042,13 +1026,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..562fc43f7db4 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]}). @@ -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(), @@ -164,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}, @@ -237,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}]. @@ -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_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_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..9bca67135154 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,20 +82,28 @@ 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_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}) -> undefined; 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. @@ -119,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 @@ -150,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/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 601144d9076f..394b139a9467 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -375,13 +375,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 -> @@ -588,8 +586,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 @@ -680,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(), @@ -697,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, @@ -1993,7 +1990,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, @@ -2003,7 +2000,7 @@ make_ra_conf(Q, ServerId, TickTimeout, FName = rabbit_misc:rs(QName), Formatter = {?MODULE, format_ra_event, [QName]}, LogCfg = #{uid => UId, - snapshot_interval => SnapshotInterval, + min_snapshot_interval => 0, min_checkpoint_interval => CheckpointInterval, max_checkpoints => 3}, rabbit_misc:maps_put_truthy(membership, Membership, diff --git a/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl b/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl deleted file mode 100644 index 83f91cfda63f..000000000000 --- a/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl +++ /dev/null @@ -1,212 +0,0 @@ -%% 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) 2016-2023 VMware, Inc. or its affiliates. All rights reserved. - --module(amqp_credit_api_v2_SUITE). - --compile([export_all, nowarn_export_all]). - --include_lib("common_test/include/ct.hrl"). --include_lib("eunit/include/eunit.hrl"). --include_lib("amqp_client/include/amqp_client.hrl"). - -all() -> - [ - {group, cluster_size_1} - ]. - -groups() -> - [ - {cluster_size_1, [], - [credit_api_v2]} - ]. - -suite() -> - [ - {timetrap, {minutes, 10}} - ]. - -init_per_suite(Config) -> - {ok, _} = application:ensure_all_started(amqp10_client), - rabbit_ct_helpers:log_environment(), - rabbit_ct_helpers:run_setup_steps(Config, []). - -end_per_suite(Config) -> - rabbit_ct_helpers:run_teardown_steps(Config). - -init_per_group(_Group, Config0) -> - Config = rabbit_ct_helpers:merge_app_env( - Config0, {rabbit, [{forced_feature_flags_on_init, []}]}), - rabbit_ct_helpers:run_steps(Config, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()). - -end_per_group(_Group, Config) -> - rabbit_ct_helpers:run_steps(Config, - rabbit_ct_client_helpers:teardown_steps() ++ - rabbit_ct_broker_helpers:teardown_steps()). - - -credit_api_v2(Config) -> - %% Feature flag rabbitmq_4.0.0 enables credit API v2. - FeatureFlag = 'rabbitmq_4.0.0', - ?assertNot(rabbit_ct_broker_helpers:is_feature_flag_enabled(Config, FeatureFlag)), - - CQ = <<"classic queue">>, - QQ = <<"quorum queue">>, - CQAddr = rabbitmq_amqp_address:queue(CQ), - QQAddr = rabbitmq_amqp_address:queue(QQ), - - Ch = rabbit_ct_client_helpers:open_channel(Config), - #'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = CQ}), - #'queue.declare_ok'{} = amqp_channel:call( - Ch, #'queue.declare'{ - queue = QQ, - durable = true, - arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]}), - ok = rabbit_ct_client_helpers:close_channel(Ch), - - Host = ?config(rmq_hostname, Config), - Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp), - OpnConf = #{address => Host, - port => Port, - container_id => <<"my container">>, - sasl => {plain, <<"guest">>, <<"guest">>}}, - {ok, Connection} = amqp10_client:open_connection(OpnConf), - {ok, Session} = amqp10_client:begin_session_sync(Connection), - - {ok, CQSender} = amqp10_client:attach_sender_link(Session, <<"cq sender">>, CQAddr), - {ok, QQSender} = amqp10_client:attach_sender_link(Session, <<"qq sender">>, QQAddr), - receive {amqp10_event, {link, CQSender, credited}} -> ok - after 30_000 -> ct:fail(credited_timeout) - end, - receive {amqp10_event, {link, QQSender, credited}} -> ok - after 30_000 -> ct:fail(credited_timeout) - end, - - %% Send 40 messages to each queue. - NumMsgs = 40, - [begin - Bin = integer_to_binary(N), - ok = amqp10_client:send_msg(CQSender, amqp10_msg:new(Bin, Bin, true)), - ok = amqp10_client:send_msg(QQSender, amqp10_msg:new(Bin, Bin, true)) - end || N <- lists:seq(1, NumMsgs)], - ok = amqp10_client:detach_link(CQSender), - ok = amqp10_client:detach_link(QQSender), - - %% Consume with credit API v1 - CQAttachArgs = #{handle => 100, - name => <<"cq receiver 1">>, - role => {receiver, #{address => CQAddr, - durable => configuration}, self()}, - snd_settle_mode => unsettled, - rcv_settle_mode => first, - filter => #{}}, - {ok, CQReceiver1} = amqp10_client:attach_link(Session, CQAttachArgs), - QQAttachArgs = #{handle => 200, - name => <<"qq receiver 1">>, - role => {receiver, #{address => QQAddr, - durable => configuration}, self()}, - snd_settle_mode => unsettled, - rcv_settle_mode => first, - filter => #{}}, - {ok, QQReceiver1} = amqp10_client:attach_link(Session, QQAttachArgs), - - ok = consume_and_accept(10, CQReceiver1), - ok = consume_and_accept(10, QQReceiver1), - - ?assertEqual(ok, rabbit_ct_broker_helpers:enable_feature_flag(Config, FeatureFlag)), - flush(enabled_feature_flag), - - %% Consume with credit API v2 - {ok, CQReceiver2} = amqp10_client:attach_receiver_link( - Session, <<"cq receiver 2">>, CQAddr, unsettled), - {ok, QQReceiver2} = amqp10_client:attach_receiver_link( - Session, <<"qq receiver 2">>, QQAddr, unsettled), - ok = consume_and_accept(10, CQReceiver2), - ok = consume_and_accept(10, QQReceiver2), - - %% Consume via with credit API v1 - ok = consume_and_accept(10, CQReceiver1), - ok = consume_and_accept(10, QQReceiver1), - - %% Detach the credit API v1 links and attach with the same output handle. - ok = detach_sync(CQReceiver1), - ok = detach_sync(QQReceiver1), - {ok, CQReceiver3} = amqp10_client:attach_link(Session, CQAttachArgs), - {ok, QQReceiver3} = amqp10_client:attach_link(Session, QQAttachArgs), - - %% The new links should use credit API v2 - ok = consume_and_accept(10, CQReceiver3), - ok = consume_and_accept(10, QQReceiver3), - - flush(pre_drain), - %% Draining should also work. - ok = amqp10_client:flow_link_credit(CQReceiver3, 10, never, true), - receive {amqp10_event, {link, CQReceiver3, credit_exhausted}} -> ok - after 30_000 -> ct:fail({missing_credit_exhausted, ?LINE}) - end, - receive Unexpected1 -> ct:fail({unexpected, ?LINE, Unexpected1}) - after 20 -> ok - end, - - ok = amqp10_client:flow_link_credit(QQReceiver3, 10, never, true), - receive {amqp10_event, {link, QQReceiver3, credit_exhausted}} -> ok - after 30_000 -> ct:fail({missing_credit_exhausted, ?LINE}) - end, - receive Unexpected2 -> ct:fail({unexpected, ?LINE, Unexpected2}) - after 20 -> ok - end, - - ok = detach_sync(CQReceiver2), - ok = detach_sync(QQReceiver2), - ok = detach_sync(CQReceiver3), - ok = detach_sync(QQReceiver3), - ok = amqp10_client:end_session(Session), - receive {amqp10_event, {session, Session, {ended, _}}} -> ok - after 30_000 -> ct:fail(missing_ended) - end, - ok = amqp10_client:close_connection(Connection), - receive {amqp10_event, {connection, Connection, {closed, normal}}} -> ok - after 30_000 -> ct:fail(missing_closed) - end. - -consume_and_accept(NumMsgs, Receiver) -> - ok = amqp10_client:flow_link_credit(Receiver, NumMsgs, never), - Msgs = receive_messages(Receiver, NumMsgs), - ok = amqp10_client_session:disposition( - Receiver, - amqp10_msg:delivery_id(hd(Msgs)), - amqp10_msg:delivery_id(lists:last(Msgs)), - true, - accepted). - -receive_messages(Receiver, N) -> - receive_messages0(Receiver, N, []). - -receive_messages0(_Receiver, 0, Acc) -> - lists:reverse(Acc); -receive_messages0(Receiver, N, Acc) -> - receive - {amqp10_msg, Receiver, Msg} -> - receive_messages0(Receiver, N - 1, [Msg | Acc]) - after 30_000 -> - exit({timeout, {num_received, length(Acc)}, {num_missing, N}}) - end. - -detach_sync(Receiver) -> - ok = amqp10_client:detach_link(Receiver), - receive {amqp10_event, {link, Receiver, {detached, normal}}} -> ok - after 30_000 -> ct:fail({missing_detached, Receiver}) - end. - -flush(Prefix) -> - receive - Msg -> - ct:pal("~ts flushed: ~p~n", [Prefix, Msg]), - flush(Prefix) - after 1 -> - ok - end. diff --git a/deps/rabbit/test/dynamic_qq_SUITE.erl b/deps/rabbit/test/dynamic_qq_SUITE.erl index e23f0223b443..cc8a322645a6 100644 --- a/deps/rabbit/test/dynamic_qq_SUITE.erl +++ b/deps/rabbit/test/dynamic_qq_SUITE.erl @@ -222,7 +222,7 @@ quorum_unaffected_after_vhost_failure(Config) -> forget_cluster_node(Config) -> %% Tests that quorum queues shrink when forget_cluster_node %% operations are issues. - quorum_queue_SUITE:check_quorum_queues_v4_compat(Config), + quorum_queue_SUITE:check_quorum_queues_ff_v4_compat(Config), [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), Ch = rabbit_ct_client_helpers:open_channel(Config, Server), diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index 1a3fed31227a..f68eea87025c 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, @@ -1114,7 +1115,7 @@ get_in_minority(Config) -> ok = rabbit_quorum_queue:restart_server({RaName, Server2}). single_active_consumer_priority_take_over(Config) -> - check_quorum_queues_v4_compat(Config), + check_quorum_queues_ff_v4_compat(Config), [Server0, Server1, _Server2] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -1163,7 +1164,7 @@ single_active_consumer_priority_take_over_requeue(Config) -> single_active_consumer_priority_take_over_base(-1, Config). single_active_consumer_priority_take_over_base(DelLimit, Config) -> - check_quorum_queues_v4_compat(Config), + check_quorum_queues_ff_v4_compat(Config), [Server0, Server1, _Server2] = Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -1223,7 +1224,7 @@ single_active_consumer_priority_take_over_base(DelLimit, Config) -> ok. single_active_consumer_priority(Config) -> - check_quorum_queues_v4_compat(Config), + check_quorum_queues_ff_v4_compat(Config), [Server0, Server1, Server2] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -1444,8 +1445,8 @@ force_vhost_queues_shrink_member_to_current_member(Config) -> end. force_checkpoint_on_queue(Config) -> - check_quorum_queues_v4_compat(Config), - + check_quorum_queues_ff_v4_compat(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), @@ -1464,19 +1465,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), @@ -1493,26 +1494,26 @@ 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). force_checkpoint(Config) -> - check_quorum_queues_v4_compat(Config), + check_quorum_queues_ff_v4_compat(Config), [Server0, _Server1, _Server2] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -1624,7 +1625,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, @@ -1645,24 +1646,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) -> @@ -1735,7 +1719,7 @@ subscribe_from_each(Config) -> ok. dont_leak_file_handles(Config) -> - check_quorum_queues_v4_compat(Config), + check_quorum_queues_ff_v4_compat(Config), [Server0 | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -1783,15 +1767,78 @@ 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_v4_compat(Config), + check_quorum_queues_ff_v4_compat(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'{}), @@ -1813,7 +1860,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 @@ -1851,7 +1898,7 @@ gh_12635(Config) -> priority_queue_fifo(Config) -> %% testing: if hi priority messages are published before lo priority %% messages they are always consumed first (fifo) - check_quorum_queues_v4_compat(Config), + check_quorum_queues_ff_v4_compat(Config), [Server0 | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), Ch = rabbit_ct_client_helpers:open_channel(Config, Server0), Queue = ?config(queue_name, Config), @@ -1883,7 +1930,7 @@ priority_queue_fifo(Config) -> priority_queue_2_1_ratio(Config) -> %% testing: if lo priority messages are published before hi priority %% messages are consumed in a 2:1 hi to lo ratio - check_quorum_queues_v4_compat(Config), + check_quorum_queues_ff_v4_compat(Config), [Server0 | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), Ch = rabbit_ct_client_helpers:open_channel(Config, Server0), Queue = ?config(queue_name, Config), @@ -3329,7 +3376,7 @@ subscribe_redelivery_limit(Config) -> end. subscribe_redelivery_limit_disable(Config) -> - check_quorum_queues_v4_compat(Config), + check_quorum_queues_ff_v4_compat(Config), [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -3569,16 +3616,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 @@ -3603,7 +3652,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), @@ -3612,7 +3661,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. @@ -3727,7 +3776,7 @@ queue_length_limit_reject_publish(Config) -> ok. queue_length_limit_policy_cleared(Config) -> - check_quorum_queues_v4_compat(Config), + check_quorum_queues_ff_v4_compat(Config), [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -4787,7 +4836,7 @@ select_nodes_with_least_replicas_node_down(Config) -> || Q <- Qs]. requeue_multiple_true(Config) -> - check_quorum_queues_v4_compat(Config), + check_quorum_queues_ff_v4_compat(Config), Ch = rabbit_ct_client_helpers:open_channel(Config), QQ = ?config(queue_name, Config), @@ -4827,7 +4876,7 @@ requeue_multiple_true(Config) -> amqp_channel:call(Ch, #'queue.delete'{queue = QQ})). requeue_multiple_false(Config) -> - check_quorum_queues_v4_compat(Config), + check_quorum_queues_ff_v4_compat(Config), Ch = rabbit_ct_client_helpers:open_channel(Config), QQ = ?config(queue_name, Config), @@ -5114,7 +5163,17 @@ basic_get(Ch, Q, NoAck, Attempt) -> basic_get(Ch, Q, NoAck, Attempt - 1) end. -check_quorum_queues_v4_compat(Config) -> +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. + +check_quorum_queues_ff_v4_compat(Config) -> case rabbit_ct_broker_helpers:is_feature_flag_enabled(Config, 'rabbitmq_4.0.0') of true -> ok; diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 298e12e401da..cb909cb6129e 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)), @@ -529,8 +718,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), @@ -633,7 +820,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}], []), @@ -803,7 +989,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) -> @@ -1752,18 +1937,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) -> @@ -2217,7 +2402,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), @@ -2249,7 +2433,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()), @@ -2369,6 +2554,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), @@ -2467,50 +2658,21 @@ 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, 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), - {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}), - {no_reply, _, _, - [{checkpoint, 1, _}, - {release_cursor, 0}]} = handle_aux(follower, cast, force_checkpoint, Aux0, State1), - ok. - - %% machine version conversion test machine_version_test(C) -> @@ -2744,10 +2906,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 fc2ad83aa88f..db676c7c0d7f 100644 --- a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl @@ -158,8 +158,11 @@ end_per_testcase(Testcase, Config) -> delete_queue(Ch, ?config(target_queue_6, Config)), #'exchange.delete_ok'{} = amqp_channel:call(Ch, #'exchange.delete'{exchange = ?config(dead_letter_exchange, Config)}), - DlxWorkers = rabbit_ct_broker_helpers:rpc_all(Config, supervisor, which_children, [rabbit_fifo_dlx_sup]), - ?assert(lists:all(fun(L) -> L =:= [] end, DlxWorkers)), + DlxWorkersFun = fun () -> + rabbit_ct_broker_helpers:rpc_all(Config, supervisor, which_children, + [rabbit_fifo_dlx_sup]) + end, + eventually(?_assert(lists:all(fun(L) -> L =:= [] end, DlxWorkersFun()))), Config1 = rabbit_ct_helpers:run_steps( Config, @@ -449,30 +452,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)), @@ -544,12 +548,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">>, @@ -576,7 +580,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), @@ -600,7 +604,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. @@ -609,7 +613,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), @@ -655,7 +659,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))). @@ -706,7 +710,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))). @@ -732,16 +736,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. @@ -800,11 +804,13 @@ 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}], - dirty_query([Server], ra_name(SourceQ), fun rabbit_fifo:query_stat_dlx/1)), 500, 10), + 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)), - #'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. @@ -881,7 +887,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), @@ -894,14 +900,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_int_SUITE.erl b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl index c160bd473f5e..a275ede639d5 100644 --- a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl @@ -39,13 +39,11 @@ all_tests() -> cancel_checkout_with_pending_using_cancel_reason, cancel_checkout_with_pending_using_remove_reason, lost_delivery, - credit_api_v1, credit_api_v2, untracked_enqueue, flow, test_queries, - duplicate_delivery, - usage + duplicate_delivery ]. groups() -> @@ -281,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), @@ -524,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 => @@ -655,52 +637,6 @@ lost_delivery(Config) -> end), ok. -credit_api_v1(Config) -> - meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> false end), - ClusterName = ?config(cluster_name, Config), - ServerId = ?config(node_id, Config), - ok = start_cluster(ClusterName, [ServerId]), - F0 = rabbit_fifo_client:init([ServerId], 4), - {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, m1, F0), - {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, m2, F1), - {_, _, F3} = process_ra_events(receive_ra_events(2, 0), ClusterName, F2), - %% checkout with 0 prefetch - CTag = <<"my-tag">>, - {ok, _, F4} = rabbit_fifo_client:checkout(CTag, {credited, 0}, #{}, F3), - %% assert no deliveries - {_, _, F5} = process_ra_events(receive_ra_events(), ClusterName, F4, [], [], - fun - (D, _) -> error({unexpected_delivery, D}) - end), - %% provide some credit - {F6, []} = rabbit_fifo_client:credit_v1(CTag, 1, false, F5), - {[{_, _, _, _, m1}], [{send_credit_reply, 1}], F7} = - process_ra_events(receive_ra_events(1, 1), ClusterName, F6), - - %% credit and drain - Drain = true, - {F8, []} = rabbit_fifo_client:credit_v1(CTag, 4, Drain, F7), - AvailableAfterCheckout = 0, - {[{_, _, _, _, m2}], - [{send_credit_reply, AvailableAfterCheckout}, - {credit_reply_v1, CTag, _CreditAfterCheckout = 3, - AvailableAfterCheckout, Drain}], - F9} = process_ra_events(receive_ra_events(2, 1), ClusterName, F8), - flush(), - - %% enqueue another message - at this point the consumer credit should be - %% all used up due to the drain - {ok, F10, []} = rabbit_fifo_client:enqueue(ClusterName, m3, F9), - %% assert no deliveries - {_, _, F11} = process_ra_events(receive_ra_events(), ClusterName, F10, [], [], - fun - (D, _) -> error({unexpected_delivery, D}) - end), - %% credit again and receive the last message - {F12, []} = rabbit_fifo_client:credit_v1(CTag, 10, false, F11), - {[{_, _, _, _, m3}], _, _} = process_ra_events(receive_ra_events(1, 1), ClusterName, F12), - ok. - credit_api_v2(Config) -> ClusterName = ?config(cluster_name, Config), ServerId = ?config(node_id, Config), diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index e269a599ce23..36dc3f2c8b1f 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"). @@ -61,7 +60,6 @@ all_tests() -> scenario32, upgrade, messages_total, - ra_indexes, simple_prefetch, simple_prefetch_without_checkout_cancel, simple_prefetch_01, @@ -910,30 +908,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), @@ -1163,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}, @@ -1592,38 +1566,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)), 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), diff --git a/rabbitmq-components.mk b/rabbitmq-components.mk index 112c8b06450d..d98ab5fcd8c9 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.0 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