From 6c8c62bafe7ce7890045e7bff8e0a42e592be092 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 28 Feb 2024 10:18:57 +0000 Subject: [PATCH 01/45] Quorum queues v4 scaffolding. Create the new version but not including any changes yet. fix QQ: force delete followers after leader has terminated. Also try a longer sleep for mqtt_shared_SUITE so that the delete operation stands a chance to time out and move on to the forced deletion stage. In some mixed machine version scenarios some followers will never apply the poison pill command so we may as well force delete them just in case. QQ: skip test in amqp_client that cannot pass with mixed machine versions QQ: remove dead code Code relating to prior machine versions and state conversions. formatting / readability rabbit_fifo_prop_SUITE fixes --- deps/rabbit/app.bzl | 4 + deps/rabbit/src/rabbit_fifo.erl | 1074 ++++---- deps/rabbit/src/rabbit_fifo_v3.erl | 2569 +++++++++++++++++++ deps/rabbit/src/rabbit_fifo_v3.hrl | 226 ++ deps/rabbit/src/rabbit_quorum_queue.erl | 3 + deps/rabbit/test/amqp_client_SUITE.erl | 9 +- deps/rabbit/test/rabbit_fifo_SUITE.erl | 76 +- deps/rabbit/test/rabbit_fifo_prop_SUITE.erl | 89 +- deps/rabbitmq_mqtt/test/shared_SUITE.erl | 4 +- moduleindex.yaml | 1 + 10 files changed, 3394 insertions(+), 661 deletions(-) create mode 100644 deps/rabbit/src/rabbit_fifo_v3.erl create mode 100644 deps/rabbit/src/rabbit_fifo_v3.hrl diff --git a/deps/rabbit/app.bzl b/deps/rabbit/app.bzl index 44095b8a7d13..71aad1a6d8d4 100644 --- a/deps/rabbit/app.bzl +++ b/deps/rabbit/app.bzl @@ -148,6 +148,7 @@ def all_beam_files(name = "all_beam_files"): "src/rabbit_fifo_index.erl", "src/rabbit_fifo_v0.erl", "src/rabbit_fifo_v1.erl", + "src/rabbit_fifo_v3.erl", "src/rabbit_file.erl", "src/rabbit_global_counters.erl", "src/rabbit_guid.erl", @@ -401,6 +402,7 @@ def all_test_beam_files(name = "all_test_beam_files"): "src/rabbit_fifo_index.erl", "src/rabbit_fifo_v0.erl", "src/rabbit_fifo_v1.erl", + "src/rabbit_fifo_v3.erl", "src/rabbit_file.erl", "src/rabbit_global_counters.erl", "src/rabbit_guid.erl", @@ -541,6 +543,7 @@ def all_srcs(name = "all_srcs"): "src/rabbit_fifo_dlx.hrl", "src/rabbit_fifo_v0.hrl", "src/rabbit_fifo_v1.hrl", + "src/rabbit_fifo_v3.hrl", "src/rabbit_stream_coordinator.hrl", "src/rabbit_stream_sac_coordinator.hrl", ], @@ -674,6 +677,7 @@ def all_srcs(name = "all_srcs"): "src/rabbit_fifo_index.erl", "src/rabbit_fifo_v0.erl", "src/rabbit_fifo_v1.erl", + "src/rabbit_fifo_v3.erl", "src/rabbit_file.erl", "src/rabbit_global_counters.erl", "src/rabbit_guid.erl", diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index bc1a85af08d8..31cfe0c5c659 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -16,6 +16,8 @@ -include("rabbit_fifo.hrl"). -include_lib("rabbit_common/include/rabbit.hrl"). +-define(STATE, ?MODULE). + -export([ %% ra_machine callbacks init/1, @@ -126,7 +128,7 @@ -type client_msg() :: delivery(). %% the messages `rabbit_fifo' can send to consumers. --opaque state() :: #?MODULE{}. +-opaque state() :: #?STATE{}. -export_type([protocol/0, delivery/0, @@ -147,8 +149,8 @@ -spec init(config()) -> state(). init(#{name := Name, queue_resource := Resource} = Conf) -> - update_config(Conf, #?MODULE{cfg = #cfg{name = Name, - resource = Resource}}). + update_config(Conf, #?STATE{cfg = #cfg{name = Name, + resource = Resource}}). update_config(Conf, State) -> DLH = maps:get(dead_letter_handler, Conf, undefined), @@ -166,21 +168,21 @@ update_config(Conf, State) -> false -> competing end, - Cfg = State#?MODULE.cfg, + Cfg = State#?STATE.cfg, RCISpec = {RCI, RCI}, LastActive = maps:get(created, Conf, undefined), - State#?MODULE{cfg = Cfg#cfg{release_cursor_interval = RCISpec, - 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}. + State#?STATE{cfg = Cfg#cfg{release_cursor_interval = RCISpec, + 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 @@ -191,14 +193,14 @@ apply(Meta, #enqueue{pid = From, seq = Seq, msg = RawMsg}, State00) -> apply_enqueue(Meta, From, Seq, RawMsg, State00); apply(_Meta, #register_enqueuer{pid = Pid}, - #?MODULE{enqueuers = Enqueuers0, - cfg = #cfg{overflow_strategy = Overflow}} = State0) -> + #?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#?MODULE{enqueuers = Enqueuers0#{Pid => #enqueuer{}}} + State0#?STATE{enqueuers = Enqueuers0#{Pid => #enqueuer{}}} end, Res = case is_over_limit(State) of true when Overflow == reject_publish -> @@ -207,9 +209,8 @@ apply(_Meta, #register_enqueuer{pid = Pid}, ok end, {State, Res, [{monitor, process, Pid}]}; -apply(Meta, - #settle{msg_ids = MsgIds, consumer_id = ConsumerId}, - #?MODULE{consumers = Cons0} = State) -> +apply(Meta, #settle{msg_ids = MsgIds, consumer_id = ConsumerId}, + #?STATE{consumers = Cons0} = State) -> case Cons0 of #{ConsumerId := Con0} -> complete_and_checkout(Meta, MsgIds, ConsumerId, @@ -218,28 +219,31 @@ apply(Meta, {State, ok} end; apply(Meta, #discard{msg_ids = MsgIds, consumer_id = ConsumerId}, - #?MODULE{consumers = Cons, - dlx = DlxState0, - cfg = #cfg{dead_letter_handler = DLH}} = State0) -> + #?STATE{consumers = Cons, + dlx = DlxState0, + cfg = #cfg{dead_letter_handler = DLH}} = State0) -> case Cons of #{ConsumerId := #consumer{checked_out = Checked} = Con} -> - % Publishing to dead-letter exchange must maintain same order as messages got rejected. - DiscardMsgs = lists:filtermap(fun(Id) -> - case maps:get(Id, Checked, undefined) of - undefined -> - false; - Msg -> - {true, Msg} - end - end, MsgIds), - {DlxState, Effects} = rabbit_fifo_dlx:discard(DiscardMsgs, rejected, DLH, DlxState0), - State = State0#?MODULE{dlx = DlxState}, + % Publishing to dead-letter exchange must maintain same order as + % messages got rejected. + DiscardMsgs = lists:filtermap( + fun(Id) -> + case maps:get(Id, Checked, undefined) of + undefined -> + false; + Msg -> + {true, Msg} + end + end, MsgIds), + {DlxState, Effects} = rabbit_fifo_dlx:discard(DiscardMsgs, rejected, + DLH, DlxState0), + State = State0#?STATE{dlx = DlxState}, complete_and_checkout(Meta, MsgIds, ConsumerId, Con, Effects, State); _ -> {State0, ok} end; apply(Meta, #return{msg_ids = MsgIds, consumer_id = ConsumerId}, - #?MODULE{consumers = Cons0} = State) -> + #?STATE{consumers = Cons0} = State) -> case Cons0 of #{ConsumerId := #consumer{checked_out = Checked0}} -> Returned = maps:with(MsgIds, Checked0), @@ -253,10 +257,10 @@ apply(#{index := Idx} = Meta, index = OldIdx, header = Header0, msg = _Msg}, - #?MODULE{consumers = Cons0, - messages = Messages, - ra_indexes = Indexes0, - enqueue_count = EnqCount} = State00) -> + #?STATE{consumers = Cons0, + messages = Messages, + ra_indexes = Indexes0, + enqueue_count = EnqCount} = State00) -> case Cons0 of #{ConsumerId := #consumer{checked_out = Checked0} = Con0} when is_map_key(MsgId, Checked0) -> @@ -266,9 +270,10 @@ apply(#{index := Idx} = Meta, State0 = add_bytes_return(Header, State00), Con = Con0#consumer{checked_out = maps:remove(MsgId, Checked0), credit = increase_credit(Meta, Con0, 1)}, - State1 = State0#?MODULE{ra_indexes = rabbit_fifo_index:delete(OldIdx, Indexes0), - messages = lqueue:in(?MSG(Idx, Header), Messages), - enqueue_count = EnqCount + 1}, + State1 = State0#?STATE{ra_indexes = rabbit_fifo_index:delete(OldIdx, + Indexes0), + messages = lqueue:in(?MSG(Idx, Header), Messages), + enqueue_count = EnqCount + 1}, State2 = update_or_remove_sub(Meta, ConsumerId, Con, State1), {State, Ret, Effs} = checkout(Meta, State0, State2, []), update_smallest_raft_index(Idx, Ret, @@ -279,21 +284,22 @@ apply(#{index := Idx} = Meta, end; apply(Meta, #credit{credit = LinkCreditRcv, delivery_count = DeliveryCountRcv, drain = Drain, consumer_id = ConsumerId = {CTag, CPid}}, - #?MODULE{consumers = Cons0, - service_queue = ServiceQueue0, - waiting_consumers = Waiting0} = State0) -> + #?STATE{consumers = Cons0, + service_queue = ServiceQueue0, + waiting_consumers = Waiting0} = State0) -> case Cons0 of #{ConsumerId := #consumer{delivery_count = DeliveryCountSnd, cfg = Cfg} = Con0} -> - LinkCreditSnd = link_credit_snd(DeliveryCountRcv, LinkCreditRcv, DeliveryCountSnd, Cfg), + LinkCreditSnd = link_credit_snd(DeliveryCountRcv, LinkCreditRcv, + DeliveryCountSnd, Cfg), %% grant the credit Con1 = Con0#consumer{credit = LinkCreditSnd}, ServiceQueue = maybe_queue_consumer(ConsumerId, Con1, ServiceQueue0), - State1 = State0#?MODULE{service_queue = ServiceQueue, - consumers = maps:update(ConsumerId, Con1, Cons0)}, + State1 = State0#?STATE{service_queue = ServiceQueue, + consumers = maps:update(ConsumerId, Con1, Cons0)}, {State2, ok, Effects} = checkout(Meta, State0, State1, []), - #?MODULE{consumers = Cons1 = #{ConsumerId := Con2}} = State2, + #?STATE{consumers = Cons1 = #{ConsumerId := Con2}} = State2, #consumer{credit = PostCred, delivery_count = PostDeliveryCount} = Con2, Available = messages_ready(State2), @@ -307,23 +313,29 @@ apply(Meta, #credit{credit = LinkCreditRcv, delivery_count = DeliveryCountRcv, Con = Con2#consumer{delivery_count = AdvancedDeliveryCount, credit = ZeroCredit}, Cons = maps:update(ConsumerId, Con, Cons1), - State3 = State2#?MODULE{consumers = Cons}, + State3 = State2#?STATE{consumers = Cons}, {ZeroCredit, AdvancedDeliveryCount, State3}; false -> {PostCred, PostDeliveryCount, State2} end, - %% We must send to queue client delivery effects before credit_reply such - %% that session process can send to AMQP 1.0 client TRANSFERs before FLOW. + %% We must send to queue client delivery effects 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}, + {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). + %% 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 -> @@ -331,26 +343,31 @@ apply(Meta, #credit{credit = LinkCreditRcv, delivery_count = DeliveryCountRcv, Con = Con2#consumer{delivery_count = AdvancedDeliveryCount, credit = 0}, Cons = maps:update(ConsumerId, Con, Cons1), - State = State2#?MODULE{consumers = Cons}, - Reply = {multi, [CreditReply, {send_drained, {CTag, PostCred}}]}, + State = State2#?STATE{consumers = Cons}, + Reply = {multi, [CreditReply, + {send_drained, {CTag, PostCred}}]}, {State, Reply, Effects}; false -> {State2, CreditReply, Effects} end end; _ when Waiting0 /= [] -> - %%TODO next time when we bump the machine version: - %% 1. Do not put consumer at head of waiting_consumers if NewCredit == 0 + %% TODO next time when we bump the machine version: + %% 1. Do not put consumer at head of waiting_consumers if + %% NewCredit == 0 %% to reduce likelihood of activating a 0 credit consumer. - %% 2. Support Drain == true, i.e. advance delivery-count, consuming all link-credit since there - %% are no messages available for an inactive consumer and send credit_reply with Drain=true. + %% 2. Support Drain == true, i.e. advance delivery-count, + %% consuming all link-credit since there + %% are no messages available for an inactive consumer and + %% send credit_reply with Drain=true. case lists:keytake(ConsumerId, 1, Waiting0) of {value, {_, Con0 = #consumer{delivery_count = DeliveryCountSnd, cfg = Cfg}}, Waiting} -> - LinkCreditSnd = link_credit_snd(DeliveryCountRcv, LinkCreditRcv, DeliveryCountSnd, Cfg), + LinkCreditSnd = link_credit_snd(DeliveryCountRcv, LinkCreditRcv, + DeliveryCountSnd, Cfg), %% grant the credit Con = Con0#consumer{credit = LinkCreditSnd}, - State = State0#?MODULE{waiting_consumers = + State = State0#?STATE{waiting_consumers = [{ConsumerId, Con} | Waiting]}, %% No messages are available for inactive consumers. Available = 0, @@ -358,7 +375,8 @@ apply(Meta, #credit{credit = LinkCreditRcv, delivery_count = DeliveryCountRcv, true -> {State, ok, {send_msg, CPid, - {credit_reply, CTag, DeliveryCountSnd, LinkCreditSnd, Available, false}, + {credit_reply, CTag, DeliveryCountSnd, LinkCreditSnd, + Available, false}, ?DELIVERY_SEND_MSG_OPTS}}; false -> {State, {send_credit_reply, Available}} @@ -371,16 +389,16 @@ apply(Meta, #credit{credit = LinkCreditRcv, delivery_count = DeliveryCountRcv, {State0, ok} end; apply(_, #checkout{spec = {dequeue, _}}, - #?MODULE{cfg = #cfg{consumer_strategy = single_active}} = State0) -> + #?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}, - #?MODULE{consumers = Consumers} = State00) -> + #?STATE{consumers = Consumers} = State00) -> %% dequeue always updates last_active - State0 = State00#?MODULE{last_active = Ts}, + State0 = State00#?STATE{last_active = Ts}, %% all dequeue operations result in keeping the queue from expiring Exists = maps:is_key(ConsumerId, Consumers), case messages_ready(State0) of @@ -394,21 +412,25 @@ apply(#{index := Index, {once, 1, simple_prefetch}, 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), + {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), Reply = '$ra_no_reply', case {DroppedMsg, ExpiredMsg} of {false, false} -> @@ -418,9 +440,13 @@ apply(#{index := Index, end; {nochange, _ExpiredMsg = true, State2, Effects0} -> %% All ready messages expired. - State3 = State2#?MODULE{consumers = maps:remove(ConsumerId, State2#?MODULE.consumers)}, - {State, _, Effects} = evaluate_limit(Index, false, State0, State3, Effects0), - update_smallest_raft_index(Index, {dequeue, empty}, State, Effects) + State3 = State2#?STATE{consumers = + maps:remove(ConsumerId, + State2#?STATE.consumers)}, + {State, _, Effects} = evaluate_limit(Index, false, State0, + State3, Effects0), + update_smallest_raft_index(Index, {dequeue, empty}, + State, Effects) end end; apply(#{index := Idx} = Meta, @@ -448,32 +474,35 @@ apply(Meta, #checkout{spec = Spec, meta = ConsumerMeta, num_checked_out => map_size(Checked)}}, checkout(Meta, State0, State2, [{monitor, process, Pid} | Effs], Reply); apply(#{index := Index}, #purge{}, - #?MODULE{messages_total = Total, - returns = Returns, - ra_indexes = Indexes0 - } = State0) -> + #?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. + %% 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. + %% 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'. + %% 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#?MODULE{ra_indexes = Indexes, - messages = lqueue:new(), - messages_total = Total - NumReady, - returns = lqueue:new(), - msg_bytes_enqueue = 0 - }, + State1 = State0#?STATE{ra_indexes = Indexes, + messages = lqueue:new(), + messages_total = Total - NumReady, + returns = lqueue:new(), + msg_bytes_enqueue = 0 + }, Effects0 = [garbage_collection], Reply = {purge, NumReady}, {State, _, Effects} = evaluate_limit(Index, false, State0, @@ -483,51 +512,44 @@ apply(#{index := Idx}, #garbage_collection{}, State) -> update_smallest_raft_index(Idx, ok, State, [{aux, garbage_collection}]); apply(Meta, {timeout, expire_msgs}, State) -> checkout(Meta, State, State, []); -apply(#{system_time := Ts, machine_version := MachineVersion} = Meta, +apply(#{system_time := Ts} = Meta, {down, Pid, noconnection}, - #?MODULE{consumers = Cons0, - cfg = #cfg{consumer_strategy = single_active}, - waiting_consumers = Waiting0, - enqueuers = Enqs0} = State0) -> + #?STATE{consumers = Cons0, + cfg = #cfg{consumer_strategy = single_active}, + waiting_consumers = Waiting0, + enqueuers = Enqs0} = State0) -> Node = node(Pid), %% if the pid refers to an active or cancelled consumer, %% mark it as suspected and return it to the waiting queue {State1, Effects0} = - maps:fold(fun({_, P} = Cid, 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, Cid, C0, false, suspected_down, E0), - C1 = case MachineVersion of - V when V >= 3 -> - C0; - 2 -> - Checked = C0#consumer.checked_out, - Credit = increase_credit(Meta, C0, maps:size(Checked)), - C0#consumer{credit = Credit} - end, - {St, Effs1} = return_all(Meta, S0, Effs, Cid, C1), - %% if the consumer was cancelled there is a chance it got - %% removed when returning hence we need to be defensive here - Waiting = case St#?MODULE.consumers of - #{Cid := C} -> - Waiting0 ++ [{Cid, C}]; - _ -> - Waiting0 - end, - {St#?MODULE{consumers = maps:remove(Cid, St#?MODULE.consumers), - waiting_consumers = Waiting, - last_active = Ts}, - Effs1}; - (_, _, S) -> - S - end, {State0, []}, Cons0), + maps:fold( + fun({_, P} = Cid, 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, Cid, C0, false, suspected_down, E0), + {St, Effs1} = return_all(Meta, S0, Effs, Cid, C0), + %% 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 + #{Cid := C} -> + Waiting0 ++ [{Cid, C}]; + _ -> + Waiting0 + end, + {St#?STATE{consumers = maps:remove(Cid, St#?STATE.consumers), + waiting_consumers = Waiting, + last_active = Ts}, + Effs1}; + (_, _, S) -> + S + end, {State0, []}, Cons0), WaitingConsumers = update_waiting_consumer_status(Node, State1, suspected_down), %% select a new consumer from the waiting queue and run a checkout - State2 = State1#?MODULE{waiting_consumers = WaitingConsumers}, + State2 = State1#?STATE{waiting_consumers = WaitingConsumers}, {State, Effects1} = activate_next_consumer(State2, Effects0), %% mark any enquers as suspected @@ -536,10 +558,10 @@ apply(#{system_time := Ts, machine_version := MachineVersion} = Meta, (_, E) -> E end, Enqs0), Effects = [{monitor, node, Node} | Effects1], - checkout(Meta, State0, State#?MODULE{enqueuers = Enqs}, Effects); -apply(#{system_time := Ts, machine_version := MachineVersion} = Meta, + checkout(Meta, State0, State#?STATE{enqueuers = Enqs}, Effects); +apply(#{system_time := Ts} = Meta, {down, Pid, noconnection}, - #?MODULE{consumers = Cons0, + #?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 @@ -553,17 +575,9 @@ apply(#{system_time := Ts, machine_version := MachineVersion} = Meta, {State, Effects1} = maps:fold( - fun({_, P} = Cid, #consumer{checked_out = Checked0, - status = up} = C0, + fun({_, P} = Cid, #consumer{status = up} = C0, {St0, Eff}) when node(P) =:= Node -> - C = case MachineVersion of - V when V >= 3 -> - C0#consumer{status = suspected_down}; - 2 -> - Credit = increase_credit(Meta, C0, map_size(Checked0)), - C0#consumer{status = suspected_down, - credit = Credit} - end, + C = C0#consumer{status = suspected_down}, {St, Eff0} = return_all(Meta, St0, Eff, Cid, C), Eff1 = consumer_update_active_effects(St, Cid, C, false, suspected_down, Eff0), @@ -581,15 +595,15 @@ apply(#{system_time := Ts, machine_version := MachineVersion} = Meta, % these processes Effects = [{monitor, node, Node} | Effects1], - checkout(Meta, State0, State#?MODULE{enqueuers = Enqs, - last_active = Ts}, Effects); + checkout(Meta, State0, State#?STATE{enqueuers = Enqs, + last_active = Ts}, Effects); apply(#{index := Idx} = Meta, {down, Pid, _Info}, State0) -> {State1, Effects1} = handle_down(Meta, Pid, State0), {State, Reply, Effects} = checkout(Meta, State0, State1, Effects1), update_smallest_raft_index(Idx, Reply, State, Effects); -apply(Meta, {nodeup, Node}, #?MODULE{consumers = Cons0, - enqueuers = Enqs0, - service_queue = _SQ0} = State0) -> +apply(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 @@ -616,8 +630,8 @@ apply(Meta, {nodeup, Node}, #?MODULE{consumers = Cons0, Acc end, {State0, Monitors}, Cons0), Waiting = update_waiting_consumer_status(Node, State1, up), - State2 = State1#?MODULE{enqueuers = Enqs1, - waiting_consumers = Waiting}, + State2 = State1#?STATE{enqueuers = Enqs1, + waiting_consumers = Waiting}, {State, Effects} = activate_next_consumer(State2, Effects1), checkout(Meta, State0, State, Effects); apply(_, {nodedown, _Node}, State) -> @@ -629,21 +643,22 @@ apply(#{index := Idx} = Meta, #purge_nodes{nodes = Nodes}, State0) -> update_smallest_raft_index(Idx, ok, State, Effects); apply(#{index := Idx} = Meta, #update_config{config = #{dead_letter_handler := NewDLH} = Conf}, - #?MODULE{cfg = #cfg{dead_letter_handler = OldDLH, - resource = QRes}, - dlx = DlxState0} = State0) -> - {DlxState, Effects0} = rabbit_fifo_dlx:update_config(OldDLH, NewDLH, QRes, DlxState0), - State1 = update_config(Conf, State0#?MODULE{dlx = DlxState}), + #?STATE{cfg = #cfg{dead_letter_handler = OldDLH, + resource = QRes}, + dlx = DlxState0} = State0) -> + {DlxState, Effects0} = rabbit_fifo_dlx:update_config(OldDLH, NewDLH, QRes, + DlxState0), + State1 = update_config(Conf, State0#?STATE{dlx = DlxState}), {State, Reply, Effects} = checkout(Meta, State0, State1, Effects0), update_smallest_raft_index(Idx, Reply, State, Effects); apply(_Meta, {machine_version, FromVersion, ToVersion}, V0State) -> State = convert(FromVersion, ToVersion, V0State), {State, ok, [{aux, {dlx, setup}}]}; apply(#{index := IncomingRaftIdx} = Meta, {dlx, _} = Cmd, - #?MODULE{cfg = #cfg{dead_letter_handler = DLH}, + #?STATE{cfg = #cfg{dead_letter_handler = DLH}, dlx = DlxState0} = State0) -> {DlxState, Effects0} = rabbit_fifo_dlx:apply(Meta, Cmd, DLH, DlxState0), - State1 = State0#?MODULE{dlx = DlxState}, + State1 = State0#?STATE{dlx = DlxState}, {State, ok, Effects} = checkout(Meta, State0, State1, Effects0), update_smallest_raft_index(IncomingRaftIdx, State, Effects); apply(_Meta, Cmd, State) -> @@ -651,168 +666,22 @@ apply(_Meta, Cmd, State) -> rabbit_log:debug("rabbit_fifo: unhandled command ~W", [Cmd, 10]), {State, ok, []}. -convert_msg({RaftIdx, {Header, empty}}) when is_integer(RaftIdx) -> - ?MSG(RaftIdx, Header); -convert_msg({RaftIdx, {Header, _Msg}}) when is_integer(RaftIdx) -> - ?MSG(RaftIdx, Header); -convert_msg({'$empty_msg', Header}) -> - %% dummy index - ?MSG(undefined, Header); -convert_msg({'$prefix_msg', Header}) -> - %% dummy index - ?MSG(undefined, Header); -convert_msg({Header, empty}) -> - convert_msg(Header); -convert_msg(Header) when ?IS_HEADER(Header) -> - ?MSG(undefined, Header). - -convert_consumer_v1_to_v2({ConsumerTag, Pid}, CV1) -> - Meta = element(2, CV1), - CheckedOut = element(3, CV1), - NextMsgId = element(4, CV1), - Credit = element(5, CV1), - DeliveryCount = element(6, CV1), - CreditMode = element(7, CV1), - LifeTime = element(8, CV1), - Status = element(9, CV1), - Priority = element(10, CV1), - #consumer{cfg = #consumer_cfg{tag = ConsumerTag, - pid = Pid, - meta = Meta, - credit_mode = CreditMode, - lifetime = LifeTime, - priority = Priority}, - credit = Credit, - status = Status, - delivery_count = DeliveryCount, - next_msg_id = NextMsgId, - checked_out = maps:map( - fun (_, {Tag, _} = Msg) when is_atom(Tag) -> - convert_msg(Msg); - (_, {_Seq, Msg}) -> - convert_msg(Msg) - end, CheckedOut) - }. - -convert_v1_to_v2(V1State0) -> - V1State = rabbit_fifo_v1:enqueue_all_pending(V1State0), - IndexesV1 = rabbit_fifo_v1:get_field(ra_indexes, V1State), - ReturnsV1 = rabbit_fifo_v1:get_field(returns, V1State), - MessagesV1 = rabbit_fifo_v1:get_field(messages, V1State), - ConsumersV1 = rabbit_fifo_v1:get_field(consumers, V1State), - WaitingConsumersV1 = rabbit_fifo_v1:get_field(waiting_consumers, V1State), - %% remove all raft idx in messages from index - {_, PrefReturns, _, PrefMsgs} = rabbit_fifo_v1:get_field(prefix_msgs, V1State), - V2PrefMsgs = lists:foldl(fun(Hdr, Acc) -> - lqueue:in(convert_msg(Hdr), Acc) - end, lqueue:new(), PrefMsgs), - V2PrefReturns = lists:foldl(fun(Hdr, Acc) -> - lqueue:in(convert_msg(Hdr), Acc) - end, lqueue:new(), PrefReturns), - MessagesV2 = lqueue:fold(fun ({_, Msg}, Acc) -> - lqueue:in(convert_msg(Msg), Acc) - end, V2PrefMsgs, MessagesV1), - ReturnsV2 = lqueue:fold(fun ({_SeqId, Msg}, Acc) -> - lqueue:in(convert_msg(Msg), Acc) - end, V2PrefReturns, ReturnsV1), - ConsumersV2 = maps:map( - fun (ConsumerId, CV1) -> - convert_consumer_v1_to_v2(ConsumerId, CV1) - end, ConsumersV1), - WaitingConsumersV2 = lists:map( - fun ({ConsumerId, CV1}) -> - {ConsumerId, convert_consumer_v1_to_v2(ConsumerId, CV1)} - end, WaitingConsumersV1), - EnqueuersV1 = rabbit_fifo_v1:get_field(enqueuers, V1State), - EnqueuersV2 = maps:map(fun (_EnqPid, Enq) -> - Enq#enqueuer{unused = undefined} - end, EnqueuersV1), - - %% do after state conversion - %% The (old) format of dead_letter_handler in RMQ < v3.10 is: - %% {Module, Function, Args} - %% The (new) format of dead_letter_handler in RMQ >= v3.10 is: - %% undefined | {at_most_once, {Module, Function, Args}} | at_least_once - %% - %% Note that the conversion must convert both from old format to new format - %% as well as from new format to new format. The latter is because quorum queues - %% created in RMQ >= v3.10 are still initialised with rabbit_fifo_v0 as described in - %% https://github.com/rabbitmq/ra/blob/e0d1e6315a45f5d3c19875d66f9d7bfaf83a46e3/src/ra_machine.erl#L258-L265 - DLH = case rabbit_fifo_v1:get_cfg_field(dead_letter_handler, V1State) of - {_M, _F, _A = [_DLX = undefined|_]} -> - %% queue was declared in RMQ < v3.10 and no DLX configured - undefined; - {_M, _F, _A} = MFA -> - %% queue was declared in RMQ < v3.10 and DLX configured - {at_most_once, MFA}; - Other -> - Other - end, - - Cfg = #cfg{name = rabbit_fifo_v1:get_cfg_field(name, V1State), - resource = rabbit_fifo_v1:get_cfg_field(resource, V1State), - release_cursor_interval = rabbit_fifo_v1:get_cfg_field(release_cursor_interval, V1State), - dead_letter_handler = DLH, - become_leader_handler = rabbit_fifo_v1:get_cfg_field(become_leader_handler, V1State), - %% TODO: what if policy enabling reject_publish was applied before conversion? - overflow_strategy = rabbit_fifo_v1:get_cfg_field(overflow_strategy, V1State), - max_length = rabbit_fifo_v1:get_cfg_field(max_length, V1State), - max_bytes = rabbit_fifo_v1:get_cfg_field(max_bytes, V1State), - consumer_strategy = rabbit_fifo_v1:get_cfg_field(consumer_strategy, V1State), - delivery_limit = rabbit_fifo_v1:get_cfg_field(delivery_limit, V1State), - expires = rabbit_fifo_v1:get_cfg_field(expires, V1State) - }, - - MessagesConsumersV2 = maps:fold(fun(_ConsumerId, #consumer{checked_out = Checked}, Acc) -> - Acc + maps:size(Checked) - end, 0, ConsumersV2), - MessagesWaitingConsumersV2 = lists:foldl(fun({_ConsumerId, #consumer{checked_out = Checked}}, Acc) -> - Acc + maps:size(Checked) - end, 0, WaitingConsumersV2), - MessagesTotal = lqueue:len(MessagesV2) + - lqueue:len(ReturnsV2) + - MessagesConsumersV2 + - MessagesWaitingConsumersV2, - - #?MODULE{cfg = Cfg, - messages = MessagesV2, - messages_total = MessagesTotal, - returns = ReturnsV2, - enqueue_count = rabbit_fifo_v1:get_field(enqueue_count, V1State), - enqueuers = EnqueuersV2, - ra_indexes = IndexesV1, - release_cursors = rabbit_fifo_v1:get_field(release_cursors, V1State), - consumers = ConsumersV2, - service_queue = rabbit_fifo_v1:get_field(service_queue, V1State), - msg_bytes_enqueue = rabbit_fifo_v1:get_field(msg_bytes_enqueue, V1State), - msg_bytes_checkout = rabbit_fifo_v1:get_field(msg_bytes_checkout, V1State), - waiting_consumers = WaitingConsumersV2, - last_active = rabbit_fifo_v1:get_field(last_active, V1State) - }. - -convert_v2_to_v3(#rabbit_fifo{consumers = ConsumersV2} = StateV2) -> - ConsumersV3 = maps:map(fun(_, C) -> - convert_consumer_v2_to_v3(C) - end, ConsumersV2), - StateV2#rabbit_fifo{consumers = ConsumersV3}. - -convert_consumer_v2_to_v3(C = #consumer{cfg = Cfg = #consumer_cfg{credit_mode = simple_prefetch, - meta = #{prefetch := Prefetch}}}) -> - C#consumer{cfg = Cfg#consumer_cfg{credit_mode = {simple_prefetch, Prefetch}}}; -convert_consumer_v2_to_v3(C) -> - C. +convert_v3_to_v4(#rabbit_fifo{} = StateV3) -> + %% nothing to convert - yet + StateV3. 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, State)). + end, {State, Effects}, + all_pids_for(Node, State)). -%% any downs that are not noconnection -handle_down(Meta, Pid, #?MODULE{consumers = Cons0, - enqueuers = Enqs0} = State0) -> +%% any downs that re not noconnection +handle_down(Meta, Pid, #?STATE{consumers = Cons0, + enqueuers = Enqs0} = State0) -> % Remove any enqueuer for the down pid - State1 = State0#?MODULE{enqueuers = maps:remove(Pid, Enqs0)}, + 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 @@ -823,27 +692,29 @@ handle_down(Meta, Pid, #?MODULE{consumers = Cons0, end, {State2, Effects1}, DownConsumers). consumer_active_flag_update_function( - #?MODULE{cfg = #cfg{consumer_strategy = competing}}) -> + #?STATE{cfg = #cfg{consumer_strategy = competing}}) -> fun(State, ConsumerId, Consumer, Active, ActivityStatus, Effects) -> consumer_update_active_effects(State, ConsumerId, Consumer, Active, ActivityStatus, Effects) end; consumer_active_flag_update_function( - #?MODULE{cfg = #cfg{consumer_strategy = single_active}}) -> + #?STATE{cfg = #cfg{consumer_strategy = single_active}}) -> fun(_, _, _, _, _, Effects) -> Effects end. handle_waiting_consumer_down(_Pid, - #?MODULE{cfg = #cfg{consumer_strategy = competing}} = State) -> + #?STATE{cfg = #cfg{consumer_strategy = competing}} + = State) -> {[], State}; handle_waiting_consumer_down(_Pid, - #?MODULE{cfg = #cfg{consumer_strategy = single_active}, - waiting_consumers = []} = State) -> + #?STATE{cfg = #cfg{consumer_strategy = single_active}, + waiting_consumers = []} = State) -> {[], State}; handle_waiting_consumer_down(Pid, - #?MODULE{cfg = #cfg{consumer_strategy = single_active}, - waiting_consumers = WaitingConsumers0} = State0) -> + #?STATE{cfg = #cfg{consumer_strategy = single_active}, + waiting_consumers = WaitingConsumers0} + = State0) -> % get cancel effects for down waiting consumers Down = lists:filter(fun({{_, P}, _}) -> P =:= Pid end, WaitingConsumers0), @@ -854,11 +725,11 @@ handle_waiting_consumer_down(Pid, % update state to have only up waiting consumers StillUp = lists:filter(fun({{_, P}, _}) -> P =/= Pid end, WaitingConsumers0), - State = State0#?MODULE{waiting_consumers = StillUp}, + State = State0#?STATE{waiting_consumers = StillUp}, {Effects, State}. update_waiting_consumer_status(Node, - #?MODULE{waiting_consumers = WaitingConsumers}, + #?STATE{waiting_consumers = WaitingConsumers}, Status) -> [begin case node(Pid) of @@ -872,19 +743,19 @@ update_waiting_consumer_status(Node, -spec state_enter(ra_server:ra_state() | eol, state()) -> ra_machine:effects(). -state_enter(RaState, #?MODULE{cfg = #cfg{dead_letter_handler = DLH, - resource = QRes}, - dlx = DlxState} = State) -> +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, #?MODULE{consumers = Cons, - enqueuers = Enqs, - waiting_consumers = WaitingConsumers, - cfg = #cfg{name = Name, - resource = Resource, - become_leader_handler = BLH} - } = State, +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 @@ -902,9 +773,9 @@ state_enter0(leader, #?MODULE{consumers = Cons, {Mod, Fun, Args} -> [{mod_call, Mod, Fun, Args ++ [Name]} | Effects] end; -state_enter0(eol, #?MODULE{enqueuers = Enqs, - consumers = Custs0, - waiting_consumers = WaitingConsumers0}, +state_enter0(eol, #?STATE{enqueuers = Enqs, + consumers = Custs0, + waiting_consumers = WaitingConsumers0}, Effects) -> Custs = maps:fold(fun({_, P}, V, S) -> S#{P => V} end, #{}, Custs0), WaitingConsumers1 = lists:foldl(fun({{_, P}, V}, Acc) -> Acc#{P => V} end, @@ -912,14 +783,20 @@ state_enter0(eol, #?MODULE{enqueuers = Enqs, AllConsumers = maps:merge(Custs, WaitingConsumers1), [{send_msg, P, eol, ra_event} || P <- maps:keys(maps:merge(Enqs, AllConsumers))] ++ - [{aux, eol} | Effects]; + [{aux, eol} + | Effects]; +state_enter0(State, #?STATE{cfg = #cfg{resource = _Resource}}, Effects) + when State =/= leader -> + FHReservation = {mod_call, rabbit_quorum_queue, + file_handle_other_reservation, []}, + [FHReservation | Effects]; state_enter0(_, _, Effects) -> %% catch all as not handling all states Effects. -spec tick(non_neg_integer(), state()) -> ra_machine:effects(). -tick(Ts, #?MODULE{cfg = #cfg{name = _Name, - resource = QName}} = State) -> +tick(Ts, #?STATE{cfg = #cfg{name = _Name, + resource = QName}} = State) -> case is_expired(Ts, State) of true -> [{mod_call, rabbit_quorum_queue, spawn_deleter, [QName]}]; @@ -928,15 +805,15 @@ tick(Ts, #?MODULE{cfg = #cfg{name = _Name, end. -spec overview(state()) -> map(). -overview(#?MODULE{consumers = Cons, - enqueuers = Enqs, - release_cursors = Cursors, - enqueue_count = EnqCount, - msg_bytes_enqueue = EnqueueBytes, - msg_bytes_checkout = CheckoutBytes, - cfg = Cfg, - dlx = DlxState, - waiting_consumers = WaitingConsumers} = State) -> +overview(#?STATE{consumers = Cons, + enqueuers = Enqs, + release_cursors = Cursors, + enqueue_count = EnqCount, + msg_bytes_enqueue = EnqueueBytes, + msg_bytes_checkout = CheckoutBytes, + cfg = Cfg, + dlx = DlxState, + waiting_consumers = WaitingConsumers} = State) -> Conf = #{name => Cfg#cfg.name, resource => Cfg#cfg.resource, release_cursor_interval => Cfg#cfg.release_cursor_interval, @@ -956,7 +833,7 @@ overview(#?MODULE{consumers = Cons, _ -> #{} end, - Overview = #{type => ?MODULE, + Overview = #{type => ?STATE, config => Conf, num_consumers => map_size(Cons), num_active_consumers => query_consumer_count(State), @@ -978,7 +855,7 @@ overview(#?MODULE{consumers = Cons, -spec get_checked_out(consumer_id(), msg_id(), msg_id(), state()) -> [delivery_msg()]. -get_checked_out(Cid, From, To, #?MODULE{consumers = Consumers}) -> +get_checked_out(Cid, From, To, #?STATE{consumers = Consumers}) -> case Consumers of #{Cid := #consumer{checked_out = Checked}} -> [begin @@ -990,12 +867,13 @@ get_checked_out(Cid, From, To, #?MODULE{consumers = Consumers}) -> end. -spec version() -> pos_integer(). -version() -> 3. +version() -> 4. which_module(0) -> rabbit_fifo_v0; which_module(1) -> rabbit_fifo_v1; -which_module(2) -> ?MODULE; -which_module(3) -> ?MODULE. +which_module(2) -> rabbit_fifo_v3; +which_module(3) -> rabbit_fifo_v3; +which_module(4) -> ?STATE. -define(AUX, aux_v2). @@ -1033,8 +911,8 @@ handle_aux(follower, _, garbage_collection, Aux, Log, MacState) -> {no_reply, force_eval_gc(Log, MacState, Aux), Log}; handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, consumer_id = ConsumerId}, Corr, Pid}, - Aux0, Log0, #?MODULE{cfg = #cfg{delivery_limit = undefined}, - consumers = Consumers}) -> + Aux0, Log0, #?STATE{cfg = #cfg{delivery_limit = undefined}, + consumers = Consumers}) -> case Consumers of #{ConsumerId := #consumer{checked_out = Checked}} -> {Log, ToReturn} = @@ -1071,8 +949,8 @@ handle_aux(leader, _, {handle_tick, [QName, Overview, Nodes]}, end, {no_reply, Aux#?AUX{tick_pid = NewPid}, Log}; handle_aux(_, _, {get_checked_out, ConsumerId, MsgIds}, - Aux0, Log0, #?MODULE{cfg = #cfg{}, - consumers = Consumers}) -> + Aux0, Log0, #?STATE{cfg = #cfg{}, + consumers = Consumers}) -> case Consumers of #{ConsumerId := #consumer{checked_out = Checked}} -> {Log, IdMsgs} = @@ -1093,11 +971,11 @@ handle_aux(_, _, {get_checked_out, ConsumerId, MsgIds}, {reply, {error, consumer_not_found}, Aux0, Log0} end; handle_aux(leader, cast, {#return{} = Ret, Corr, Pid}, - Aux0, Log, #?MODULE{}) -> + Aux0, Log, #?STATE{}) -> %% for returns with a delivery limit set we can just return as before {no_reply, Aux0, Log, [{append, Ret, {notify, Corr, Pid}}]}; handle_aux(leader, cast, eval, #?AUX{last_decorators_state = LastDec} = Aux0, - Log, #?MODULE{cfg = #cfg{resource = QName}} = MacState) -> + Log, #?STATE{cfg = #cfg{resource = QName}} = MacState) -> %% 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 @@ -1129,7 +1007,7 @@ handle_aux(_RaState, cast, eol, #?AUX{name = Name} = Aux, Log, _) -> {no_reply, Aux, Log}; handle_aux(_RaState, {call, _From}, oldest_entry_timestamp, #?AUX{cache = Cache} = Aux0, - Log0, #?MODULE{} = State) -> + Log0, #?STATE{} = State) -> {CachedIdx, CachedTs} = maps:get(oldest_entry, Cache, {undefined, undefined}), case smallest_raft_index(State) of %% if there are no entries, we return current timestamp @@ -1164,13 +1042,13 @@ handle_aux(_RaState, {call, _From}, {peek, Pos}, Aux0, {reply, Err, Aux0, Log0} end; handle_aux(RaState, _, {dlx, _} = Cmd, Aux0, Log, - #?MODULE{dlx = DlxState, - cfg = #cfg{dead_letter_handler = DLH, - resource = QRes}}) -> + #?STATE{dlx = DlxState, + cfg = #cfg{dead_letter_handler = DLH, + resource = QRes}}) -> Aux = rabbit_fifo_dlx:handle_aux(RaState, Cmd, Aux0, QRes, DLH, DlxState), {no_reply, Aux, Log}. -eval_gc(Log, #?MODULE{cfg = #cfg{resource = QR}} = MacState, +eval_gc(Log, #?STATE{cfg = #cfg{resource = QR}} = MacState, #?AUX{gc = #aux_gc{last_raft_idx = LastGcIdx} = Gc} = AuxState) -> {Idx, _} = ra_log:last_index_term(Log), {memory, Mem} = erlang:process_info(self(), memory), @@ -1187,7 +1065,7 @@ eval_gc(Log, #?MODULE{cfg = #cfg{resource = QR}} = MacState, AuxState end. -force_eval_gc(Log, #?MODULE{cfg = #cfg{resource = QR}}, +force_eval_gc(Log, #?STATE{cfg = #cfg{resource = QR}}, #?AUX{gc = #aux_gc{last_raft_idx = LastGcIdx} = Gc} = AuxState) -> {Idx, _} = ra_log:last_index_term(Log), {memory, Mem} = erlang:process_info(self(), memory), @@ -1212,7 +1090,7 @@ process_is_alive(_) -> query_messages_ready(State) -> messages_ready(State). -query_messages_checked_out(#?MODULE{consumers = Consumers}) -> +query_messages_checked_out(#?STATE{consumers = Consumers}) -> maps:fold(fun (_, #consumer{checked_out = C}, S) -> maps:size(C) + S end, 0, Consumers). @@ -1220,32 +1098,32 @@ query_messages_checked_out(#?MODULE{consumers = Consumers}) -> query_messages_total(State) -> messages_total(State). -query_processes(#?MODULE{enqueuers = Enqs, consumers = Cons0}) -> +query_processes(#?STATE{enqueuers = Enqs, consumers = Cons0}) -> Cons = maps:fold(fun({_, P}, V, S) -> S#{P => V} end, #{}, Cons0), maps:keys(maps:merge(Enqs, Cons)). -query_ra_indexes(#?MODULE{ra_indexes = RaIndexes}) -> +query_ra_indexes(#?STATE{ra_indexes = RaIndexes}) -> RaIndexes. -query_waiting_consumers(#?MODULE{waiting_consumers = WaitingConsumers}) -> +query_waiting_consumers(#?STATE{waiting_consumers = WaitingConsumers}) -> WaitingConsumers. -query_consumer_count(#?MODULE{consumers = Consumers, - waiting_consumers = WaitingConsumers}) -> +query_consumer_count(#?STATE{consumers = Consumers, + waiting_consumers = WaitingConsumers}) -> Up = maps:filter(fun(_ConsumerId, #consumer{status = Status}) -> Status =/= suspected_down end, Consumers), maps:size(Up) + length(WaitingConsumers). -query_consumers(#?MODULE{consumers = Consumers, - waiting_consumers = WaitingConsumers, - cfg = #cfg{consumer_strategy = ConsumerStrategy}} = State) -> +query_consumers(#?STATE{consumers = Consumers, + waiting_consumers = WaitingConsumers, + cfg = #cfg{consumer_strategy = ConsumerStrategy}} + = State) -> ActiveActivityStatusFun = case ConsumerStrategy of competing -> - fun(_ConsumerId, - #consumer{status = Status}) -> + fun(_ConsumerId, #consumer{status = Status}) -> case Status of suspected_down -> {false, Status}; @@ -1304,8 +1182,8 @@ query_consumers(#?MODULE{consumers = Consumers, query_single_active_consumer( - #?MODULE{cfg = #cfg{consumer_strategy = single_active}, - consumers = Consumers}) -> + #?STATE{cfg = #cfg{consumer_strategy = single_active}, + consumers = Consumers}) -> case active_consumer(Consumers) of undefined -> {error, no_value}; @@ -1315,13 +1193,13 @@ query_single_active_consumer( query_single_active_consumer(_) -> disabled. -query_stat(#?MODULE{consumers = Consumers} = State) -> +query_stat(#?STATE{consumers = Consumers} = State) -> {messages_ready(State), maps:size(Consumers)}. -query_in_memory_usage(#?MODULE{ }) -> +query_in_memory_usage(#?STATE{ }) -> {0, 0}. -query_stat_dlx(#?MODULE{dlx = DlxState}) -> +query_stat_dlx(#?STATE{dlx = DlxState}) -> rabbit_fifo_dlx:stat(DlxState). query_peek(Pos, State0) when Pos > 0 -> @@ -1335,7 +1213,7 @@ query_peek(Pos, State0) when Pos > 0 -> query_peek(Pos-1, State) end. -query_notify_decorators_info(#?MODULE{consumers = Consumers} = State) -> +query_notify_decorators_info(#?STATE{consumers = Consumers} = State) -> MaxActivePriority = maps:fold( fun(_, #consumer{credit = C, status = up, @@ -1361,12 +1239,12 @@ usage(Name) when is_atom(Name) -> %%% Internal -messages_ready(#?MODULE{messages = M, - returns = R}) -> +messages_ready(#?STATE{messages = M, + returns = R}) -> lqueue:len(M) + lqueue:len(R). -messages_total(#?MODULE{messages_total = Total, - dlx = DlxState}) -> +messages_total(#?STATE{messages_total = Total, + dlx = DlxState}) -> {DlxTotal, _} = rabbit_fifo_dlx:stat(DlxState), Total + DlxTotal. @@ -1400,25 +1278,25 @@ moving_average(Time, HalfLife, Next, Current) -> Weight = math:exp(Time * math:log(0.5) / HalfLife), Next * (1 - Weight) + Current * Weight. -num_checked_out(#?MODULE{consumers = Cons}) -> +num_checked_out(#?STATE{consumers = Cons}) -> maps:fold(fun (_, #consumer{checked_out = C}, Acc) -> maps:size(C) + Acc end, 0, Cons). cancel_consumer(Meta, ConsumerId, - #?MODULE{cfg = #cfg{consumer_strategy = competing}} = State, + #?STATE{cfg = #cfg{consumer_strategy = competing}} = State, Effects, Reason) -> cancel_consumer0(Meta, ConsumerId, State, Effects, Reason); cancel_consumer(Meta, ConsumerId, - #?MODULE{cfg = #cfg{consumer_strategy = single_active}, - waiting_consumers = []} = State, + #?STATE{cfg = #cfg{consumer_strategy = single_active}, + waiting_consumers = []} = State, Effects, Reason) -> %% single active consumer on, no consumers are waiting cancel_consumer0(Meta, ConsumerId, State, Effects, Reason); cancel_consumer(Meta, ConsumerId, - #?MODULE{consumers = Cons0, - cfg = #cfg{consumer_strategy = single_active}, - waiting_consumers = Waiting0} = State0, + #?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 @@ -1434,10 +1312,10 @@ cancel_consumer(Meta, ConsumerId, Effects = cancel_consumer_effects(ConsumerId, State0, Effects0), % A waiting consumer isn't supposed to have any checked out messages, % so nothing special to do here - {State0#?MODULE{waiting_consumers = Waiting}, Effects} + {State0#?STATE{waiting_consumers = Waiting}, Effects} end. -consumer_update_active_effects(#?MODULE{cfg = #cfg{resource = QName}}, +consumer_update_active_effects(#?STATE{cfg = #cfg{resource = QName}}, ConsumerId, #consumer{cfg = #consumer_cfg{meta = Meta}}, Active, ActivityStatus, @@ -1450,7 +1328,7 @@ consumer_update_active_effects(#?MODULE{cfg = #cfg{resource = QName}}, | Effects]. cancel_consumer0(Meta, ConsumerId, - #?MODULE{consumers = C0} = S0, Effects0, Reason) -> + #?STATE{consumers = C0} = S0, Effects0, Reason) -> case C0 of #{ConsumerId := Consumer} -> {S, Effects2} = maybe_return_all(Meta, ConsumerId, Consumer, @@ -1467,11 +1345,11 @@ cancel_consumer0(Meta, ConsumerId, {S0, Effects0} end. -activate_next_consumer(#?MODULE{cfg = #cfg{consumer_strategy = competing}} = State0, +activate_next_consumer(#?STATE{cfg = #cfg{consumer_strategy = competing}} = State0, Effects0) -> {State0, Effects0}; -activate_next_consumer(#?MODULE{consumers = Cons, - waiting_consumers = Waiting0} = State0, +activate_next_consumer(#?STATE{consumers = Cons, + waiting_consumers = Waiting0} = State0, Effects0) -> case has_active_consumer(Cons) of false -> @@ -1489,11 +1367,11 @@ activate_next_consumer(#?MODULE{consumers = Cons, %% with the new config Existing#consumer{cfg = NextCCfg} end, - #?MODULE{service_queue = ServiceQueue} = State0, + #?STATE{service_queue = ServiceQueue} = State0, ServiceQueue1 = maybe_queue_consumer(NextConsumerId, Consumer, ServiceQueue), - State = State0#?MODULE{consumers = Cons#{NextConsumerId => Consumer}, + State = State0#?STATE{consumers = Cons#{NextConsumerId => Consumer}, service_queue = ServiceQueue1, waiting_consumers = Remaining}, Effects = consumer_update_active_effects(State, NextConsumerId, @@ -1533,8 +1411,8 @@ maybe_return_all(#{system_time := Ts} = Meta, ConsumerId, S0), Effects0}; down -> {S1, Effects1} = return_all(Meta, S0, Effects0, ConsumerId, Consumer), - {S1#?MODULE{consumers = maps:remove(ConsumerId, S1#?MODULE.consumers), - last_active = Ts}, + {S1#?STATE{consumers = maps:remove(ConsumerId, S1#?STATE.consumers), + last_active = Ts}, Effects1} end. @@ -1550,17 +1428,17 @@ apply_enqueue(#{index := RaftIdx, {State, ok, Effects} end. -decr_total(#?MODULE{messages_total = Tot} = State) -> - State#?MODULE{messages_total = Tot - 1}. +decr_total(#?STATE{messages_total = Tot} = State) -> + State#?STATE{messages_total = Tot - 1}. -drop_head(#?MODULE{ra_indexes = Indexes0} = State0, Effects) -> +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#?MODULE{ra_indexes = Indexes}, + State2 = State1#?STATE{ra_indexes = Indexes}, State3 = decr_total(add_bytes_drop(Header, State2)), - #?MODULE{cfg = #cfg{dead_letter_handler = DLH}, - dlx = DlxState} = State = State3, + #?STATE{cfg = #cfg{dead_letter_handler = DLH}, + dlx = DlxState} = State = State3, {_, DlxEffects} = rabbit_fifo_dlx:discard([Msg], maxlen, DLH, DlxState), {State, DlxEffects ++ Effects}; empty -> @@ -1569,11 +1447,11 @@ drop_head(#?MODULE{ra_indexes = Indexes0} = State0, Effects) -> maybe_set_msg_ttl(#basic_message{content = #content{properties = none}}, RaCmdTs, Header, - #?MODULE{cfg = #cfg{msg_ttl = PerQueueMsgTTL}}) -> + #?STATE{cfg = #cfg{msg_ttl = PerQueueMsgTTL}}) -> update_expiry_header(RaCmdTs, PerQueueMsgTTL, Header); maybe_set_msg_ttl(#basic_message{content = #content{properties = Props}}, RaCmdTs, Header, - #?MODULE{cfg = #cfg{msg_ttl = PerQueueMsgTTL}}) -> + #?STATE{cfg = #cfg{msg_ttl = PerQueueMsgTTL}}) -> %% rabbit_quorum_queue will leave the properties decoded if and only if %% per message message TTL is set. %% We already check in the channel that expiration must be valid. @@ -1581,7 +1459,7 @@ maybe_set_msg_ttl(#basic_message{content = #content{properties = Props}}, TTL = min(PerMsgMsgTTL, PerQueueMsgTTL), update_expiry_header(RaCmdTs, TTL, Header); maybe_set_msg_ttl(Msg, RaCmdTs, Header, - #?MODULE{cfg = #cfg{msg_ttl = MsgTTL}}) -> + #?STATE{cfg = #cfg{msg_ttl = MsgTTL}}) -> case mc:is(Msg) of true -> TTL = min(MsgTTL, mc:ttl(Msg)), @@ -1606,56 +1484,57 @@ update_expiry_header(ExpiryTs, Header) -> update_header(expiry, fun(Ts) -> Ts end, ExpiryTs, Header). maybe_store_release_cursor(RaftIdx, - #?MODULE{cfg = #cfg{release_cursor_interval = {Base, C}} = Cfg, - enqueue_count = EC, - release_cursors = Cursors0} = State0) + #?STATE{cfg = #cfg{release_cursor_interval = + {Base, C}} = Cfg, + enqueue_count = EC, + release_cursors = Cursors0} = State0) when EC >= C -> case messages_total(State0) of 0 -> %% message must have been immediately dropped - State0#?MODULE{enqueue_count = 0}; + State0#?STATE{enqueue_count = 0}; Total -> Interval = case Base of 0 -> 0; _ -> min(max(Total, Base), ?RELEASE_CURSOR_EVERY_MAX) end, - State = State0#?MODULE{cfg = Cfg#cfg{release_cursor_interval = + State = State0#?STATE{cfg = Cfg#cfg{release_cursor_interval = {Base, Interval}}}, Dehydrated = dehydrate_state(State), Cursor = {release_cursor, RaftIdx, Dehydrated}, Cursors = lqueue:in(Cursor, Cursors0), - State#?MODULE{enqueue_count = 0, - release_cursors = Cursors} + State#?STATE{enqueue_count = 0, + release_cursors = Cursors} end; maybe_store_release_cursor(_RaftIdx, State) -> State. maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, Effects, - #?MODULE{msg_bytes_enqueue = Enqueue, - enqueue_count = EnqCount, - messages = Messages, - messages_total = Total} = State0) -> + #?STATE{msg_bytes_enqueue = Enqueue, + enqueue_count = EnqCount, + messages = Messages, + messages_total = Total} = State0) -> % direct enqueue without tracking Size = message_size(RawMsg), Header = maybe_set_msg_ttl(RawMsg, Ts, Size, State0), Msg = ?MSG(RaftIdx, Header), - State = State0#?MODULE{msg_bytes_enqueue = Enqueue + Size, - enqueue_count = EnqCount + 1, - messages_total = Total + 1, - messages = lqueue:in(Msg, Messages) - }, + State = State0#?STATE{msg_bytes_enqueue = Enqueue + Size, + enqueue_count = EnqCount + 1, + messages_total = Total + 1, + messages = lqueue:in(Msg, Messages) + }, {ok, State, Effects}; maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, Effects0, - #?MODULE{msg_bytes_enqueue = Enqueue, - enqueue_count = EnqCount, - enqueuers = Enqueuers0, - messages = Messages, - messages_total = Total} = State0) -> + #?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#?MODULE{enqueuers = Enqueuers0#{From => #enqueuer{}}}, + State1 = State0#?STATE{enqueuers = Enqueuers0#{From => #enqueuer{}}}, {Res, State, Effects} = maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, Effects0, State1), {Res, State, [{monitor, process, From} | Effects]}; @@ -1671,13 +1550,13 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, Effects0, false -> undefined end, - State = State0#?MODULE{msg_bytes_enqueue = Enqueue + Size, - enqueue_count = EnqCount + 1, - messages_total = Total + 1, - messages = lqueue:in(Msg, Messages), - enqueuers = Enqueuers0#{From => Enq}, - msg_cache = MsgCache - }, + State = State0#?STATE{msg_bytes_enqueue = Enqueue + Size, + enqueue_count = EnqCount + 1, + messages_total = Total + 1, + messages = lqueue:in(Msg, Messages), + enqueuers = Enqueuers0#{From => Enq}, + msg_cache = MsgCache + }, {ok, State, Effects0}; #enqueuer{next_seqno = Next} when MsgSeqNo > Next -> @@ -1688,34 +1567,28 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, Effects0, {duplicate, State0, Effects0} end. -return(#{index := IncomingRaftIdx, machine_version := MachineVersion} = Meta, +return(#{index := IncomingRaftIdx} = Meta, ConsumerId, Returned, Effects0, State0) -> {State1, Effects1} = maps:fold( fun(MsgId, Msg, {S0, E0}) -> - return_one(Meta, MsgId, Msg, S0, E0, ConsumerId) + return_one(Meta, MsgId, Msg, + S0, E0, ConsumerId) end, {State0, Effects0}, Returned), - State2 = - case State1#?MODULE.consumers of - #{ConsumerId := Con} - when MachineVersion >= 3 -> - update_or_remove_sub(Meta, ConsumerId, Con, State1); - #{ConsumerId := Con0} - when MachineVersion =:= 2 -> - Credit = increase_credit(Meta, Con0, map_size(Returned)), - Con = Con0#consumer{credit = Credit}, - update_or_remove_sub(Meta, ConsumerId, Con, State1); - _ -> - State1 - end, + State2 = case State1#?STATE.consumers of + #{ConsumerId := Con} -> + update_or_remove_sub(Meta, ConsumerId, Con, State1); + _ -> + State1 + end, {State, ok, Effects} = checkout(Meta, State0, State2, Effects1), update_smallest_raft_index(IncomingRaftIdx, State, Effects). % used to process messages that are finished complete(Meta, ConsumerId, [DiscardedMsgId], #consumer{checked_out = Checked0} = Con0, - #?MODULE{ra_indexes = Indexes0, - msg_bytes_checkout = BytesCheckout, - messages_total = Tot} = State0) -> + #?STATE{ra_indexes = Indexes0, + msg_bytes_checkout = BytesCheckout, + messages_total = Tot} = State0) -> case maps:take(DiscardedMsgId, Checked0) of {?MSG(Idx, Hdr), Checked} -> SettledSize = get_header(size, Hdr), @@ -1723,17 +1596,17 @@ complete(Meta, ConsumerId, [DiscardedMsgId], Con = Con0#consumer{checked_out = Checked, credit = increase_credit(Meta, Con0, 1)}, State1 = update_or_remove_sub(Meta, ConsumerId, Con, State0), - State1#?MODULE{ra_indexes = Indexes, - msg_bytes_checkout = BytesCheckout - SettledSize, - messages_total = Tot - 1}; + State1#?STATE{ra_indexes = Indexes, + msg_bytes_checkout = BytesCheckout - SettledSize, + messages_total = Tot - 1}; error -> State0 end; complete(Meta, ConsumerId, DiscardedMsgIds, #consumer{checked_out = Checked0} = Con0, - #?MODULE{ra_indexes = Indexes0, - msg_bytes_checkout = BytesCheckout, - messages_total = Tot} = State0) -> + #?STATE{ra_indexes = Indexes0, + msg_bytes_checkout = BytesCheckout, + messages_total = Tot} = State0) -> {SettledSize, Checked, Indexes} = lists:foldl( fun (MsgId, {S0, Ch0, Idxs}) -> @@ -1749,9 +1622,9 @@ complete(Meta, ConsumerId, DiscardedMsgIds, Con = Con0#consumer{checked_out = Checked, credit = increase_credit(Meta, Con0, Len)}, State1 = update_or_remove_sub(Meta, ConsumerId, Con, State0), - State1#?MODULE{ra_indexes = Indexes, - msg_bytes_checkout = BytesCheckout - SettledSize, - messages_total = Tot - Len}. + State1#?STATE{ra_indexes = Indexes, + msg_bytes_checkout = BytesCheckout - SettledSize, + messages_total = Tot - Len}. increase_credit(_Meta, #consumer{cfg = #consumer_cfg{lifetime = once}, credit = Credit}, _) -> @@ -1762,10 +1635,11 @@ increase_credit(_Meta, #consumer{cfg = #consumer_cfg{lifetime = auto, credit = Credit}, _) -> %% credit_mode: `credited' also doesn't automatically increment credit Credit; -increase_credit(#{machine_version := MachineVersion}, - #consumer{cfg = #consumer_cfg{credit_mode = {simple_prefetch, MaxCredit}}, +increase_credit(_Meta, + #consumer{cfg = #consumer_cfg{credit_mode = + {simple_prefetch, MaxCredit}}, credit = Current}, Credit) - when MachineVersion >= 3 andalso MaxCredit > 0 -> + when MaxCredit > 0 -> min(MaxCredit, Current + Credit); increase_credit(_Meta, #consumer{credit = Current}, Credit) -> Current + Credit. @@ -1778,7 +1652,7 @@ complete_and_checkout(#{index := IncomingRaftIdx} = Meta, MsgIds, ConsumerId, update_smallest_raft_index(IncomingRaftIdx, State, Effects). cancel_consumer_effects(ConsumerId, - #?MODULE{cfg = #cfg{resource = QName}} = _State, + #?STATE{cfg = #cfg{resource = QName}} = _State, Effects) -> [{mod_call, rabbit_quorum_queue, cancel_consumer_handler, [QName, ConsumerId]} | Effects]. @@ -1787,8 +1661,8 @@ update_smallest_raft_index(Idx, State, Effects) -> update_smallest_raft_index(Idx, ok, State, Effects). update_smallest_raft_index(IncomingRaftIdx, Reply, - #?MODULE{cfg = Cfg, - release_cursors = Cursors0} = State0, + #?STATE{cfg = Cfg, + release_cursors = Cursors0} = State0, Effects) -> Total = messages_total(State0), %% TODO: optimise @@ -1800,9 +1674,9 @@ update_smallest_raft_index(IncomingRaftIdx, Reply, %% reset the release cursor interval #cfg{release_cursor_interval = {Base, _}} = Cfg, RCI = {Base, Base}, - State = State0#?MODULE{cfg = Cfg#cfg{release_cursor_interval = RCI}, - release_cursors = lqueue:new(), - enqueue_count = 0}, + State = State0#?STATE{cfg = Cfg#cfg{release_cursor_interval = RCI}, + release_cursors = lqueue:new(), + enqueue_count = 0}, {State, Reply, Effects ++ [{release_cursor, IncomingRaftIdx, State}]}; undefined -> {State0, Reply, Effects}; @@ -1813,7 +1687,7 @@ update_smallest_raft_index(IncomingRaftIdx, Reply, {Cursor, Cursors} -> %% we can emit a release cursor when we've passed the smallest %% release cursor available. - {State0#?MODULE{release_cursors = Cursors}, Reply, + {State0#?STATE{release_cursors = Cursors}, Reply, Effects ++ [Cursor]} end end. @@ -1871,42 +1745,37 @@ get_header(Key, Header) when is_map(Header) andalso is_map_key(size, Header) -> maps:get(Key, Header, undefined). -return_one(#{machine_version := MachineVersion} = Meta, - MsgId, Msg0, - #?MODULE{returns = Returns, - consumers = Consumers, - dlx = DlxState0, - cfg = #cfg{delivery_limit = DeliveryLimit, - dead_letter_handler = DLH}} = State0, +return_one(Meta, MsgId, Msg0, + #?STATE{returns = Returns, + consumers = Consumers, + dlx = DlxState0, + cfg = #cfg{delivery_limit = DeliveryLimit, + dead_letter_handler = DLH}} = State0, Effects0, ConsumerId) -> #consumer{checked_out = Checked0} = Con0 = maps:get(ConsumerId, Consumers), Msg = update_msg_header(delivery_count, fun incr/1, 1, Msg0), Header = get_msg_header(Msg), case get_header(delivery_count, Header) of DeliveryCount when DeliveryCount > DeliveryLimit -> - {DlxState, DlxEffects} = rabbit_fifo_dlx:discard([Msg], delivery_limit, DLH, DlxState0), - State1 = State0#?MODULE{dlx = DlxState}, + {DlxState, DlxEffects} = + rabbit_fifo_dlx:discard([Msg], delivery_limit, DLH, DlxState0), + State1 = State0#?STATE{dlx = DlxState}, State = complete(Meta, ConsumerId, [MsgId], Con0, State1), {State, DlxEffects ++ Effects0}; _ -> Checked = maps:remove(MsgId, Checked0), - Con = case MachineVersion of - V when V >= 3 -> - Con0#consumer{checked_out = Checked, - credit = increase_credit(Meta, Con0, 1)}; - 2 -> - Con0#consumer{checked_out = Checked} - end, + Con = Con0#consumer{checked_out = Checked, + credit = increase_credit(Meta, Con0, 1)}, {add_bytes_return( Header, - State0#?MODULE{consumers = Consumers#{ConsumerId => Con}, - returns = lqueue:in(Msg, Returns)}), + State0#?STATE{consumers = Consumers#{ConsumerId => Con}, + returns = lqueue:in(Msg, Returns)}), Effects0} end. -return_all(Meta, #?MODULE{consumers = Cons} = State0, Effects0, ConsumerId, +return_all(Meta, #?STATE{consumers = Cons} = State0, Effects0, ConsumerId, #consumer{checked_out = Checked} = Con) -> - State = State0#?MODULE{consumers = Cons#{ConsumerId => Con}}, + State = State0#?STATE{consumers = Cons#{ConsumerId => Con}}, lists:foldl(fun ({MsgId, Msg}, {S, E}) -> return_one(Meta, MsgId, Msg, S, E, ConsumerId) end, {State, Effects0}, lists:sort(maps:to_list(Checked))). @@ -1915,15 +1784,16 @@ checkout(Meta, OldState, State0, Effects0) -> checkout(Meta, OldState, State0, Effects0, ok). checkout(#{index := Index} = Meta, - #?MODULE{cfg = #cfg{resource = _QName}} = OldState, + #?STATE{} = OldState, State0, Effects0, Reply) -> - {#?MODULE{cfg = #cfg{dead_letter_handler = DLH}, - dlx = DlxState0} = State1, ExpiredMsg, Effects1} = + {#?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? - State2 = State1#?MODULE{msg_cache = undefined, %% by this time the cache should be used - dlx = DlxState}, + %% 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, false, Effects} when ExpiredMsg == false -> @@ -1948,13 +1818,13 @@ checkout0(_Meta, {_Activity, ExpiredMsg, State0, Effects0}, SendAcc) -> {State0, ExpiredMsg, lists:reverse(Effects)}. evaluate_limit(_Index, Result, _BeforeState, - #?MODULE{cfg = #cfg{max_length = undefined, - max_bytes = undefined}} = State, + #?STATE{cfg = #cfg{max_length = undefined, + max_bytes = undefined}} = State, Effects) -> {State, Result, Effects}; evaluate_limit(Index, Result, BeforeState, - #?MODULE{cfg = #cfg{overflow_strategy = Strategy}, - enqueuers = Enqs0} = State0, + #?STATE{cfg = #cfg{overflow_strategy = Strategy}, + enqueuers = Enqs0} = State0, Effects0) -> case is_over_limit(State0) of true when Strategy == drop_head -> @@ -1965,7 +1835,7 @@ evaluate_limit(Index, Result, BeforeState, %% they need to block {Enqs, Effects} = maps:fold( - fun (P, #enqueuer{blocked = undefined} = E0, {Enqs, Acc}) -> + fun (P, #enqueuer{blocked = undefined} = E0, {Enqs, Acc}) -> E = E0#enqueuer{blocked = Index}, {Enqs#{P => E}, [{send_msg, P, {queue_status, reject_publish}, @@ -1973,7 +1843,7 @@ evaluate_limit(Index, Result, BeforeState, (_P, _E, Acc) -> Acc end, {Enqs0, Effects0}, Enqs0), - {State0#?MODULE{enqueuers = Enqs}, Result, Effects}; + {State0#?STATE{enqueuers = Enqs}, Result, Effects}; false when Strategy == reject_publish -> %% TODO: optimise as this case gets called for every command %% pretty much @@ -1991,7 +1861,7 @@ evaluate_limit(Index, Result, BeforeState, (_P, _E, Acc) -> Acc end, {Enqs0, Effects0}, Enqs0), - {State0#?MODULE{enqueuers = Enqs}, Result, Effects}; + {State0#?STATE{enqueuers = Enqs}, Result, Effects}; _ -> {State0, Result, Effects0} end; @@ -2028,13 +1898,13 @@ add_delivery_effects(Effects0, AccMap, State) -> end, Efs, chunk_disk_msgs(DiskMsgs, 0, [[]])) end, Effects0, AccMap). -take_next_msg(#?MODULE{returns = Returns0, - messages = Messages0, - ra_indexes = Indexes0 - } = State) -> +take_next_msg(#?STATE{returns = Returns0, + messages = Messages0, + ra_indexes = Indexes0 + } = State) -> case lqueue:out(Returns0) of {{value, NextMsg}, Returns} -> - {NextMsg, State#?MODULE{returns = Returns}}; + {NextMsg, State#?STATE{returns = Returns}}; {empty, _} -> case lqueue:out(Messages0) of {empty, _} -> @@ -2042,13 +1912,13 @@ take_next_msg(#?MODULE{returns = Returns0, {{value, ?MSG(RaftIdx, _) = Msg}, Messages} -> %% add index here Indexes = rabbit_fifo_index:append(RaftIdx, Indexes0), - {Msg, State#?MODULE{messages = Messages, + {Msg, State#?STATE{messages = Messages, ra_indexes = Indexes}} end end. -get_next_msg(#?MODULE{returns = Returns0, - messages = Messages0}) -> +get_next_msg(#?STATE{returns = Returns0, + messages = Messages0}) -> case lqueue:get(Returns0, empty) of empty -> lqueue:get(Messages0, empty); @@ -2057,7 +1927,7 @@ get_next_msg(#?MODULE{returns = Returns0, end. delivery_effect({CTag, CPid}, [{MsgId, ?MSG(Idx, Header)}], - #?MODULE{msg_cache = {Idx, RawMsg}}) -> + #?STATE{msg_cache = {Idx, RawMsg}}) -> {send_msg, CPid, {delivery, CTag, [{MsgId, {Header, RawMsg}}]}, ?DELIVERY_SEND_MSG_OPTS}; delivery_effect({CTag, CPid}, Msgs, _State) -> @@ -2070,7 +1940,8 @@ delivery_effect({CTag, CPid}, Msgs, _State) -> fun (Cmd, {MsgId, ?MSG(_Idx, Header)}) -> {MsgId, {Header, get_msg(Cmd)}} end, Log, Msgs), - [{send_msg, CPid, {delivery, CTag, DelMsgs}, ?DELIVERY_SEND_MSG_OPTS}] + [{send_msg, CPid, {delivery, CTag, DelMsgs}, + ?DELIVERY_SEND_MSG_OPTS}] end, {local, node(CPid)}}. @@ -2084,11 +1955,11 @@ reply_log_effect(RaftIdx, MsgId, Header, Ready, From) -> 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, #?MODULE{service_queue = SQ0, - messages = Messages0, - msg_bytes_checkout = BytesCheckout, - msg_bytes_enqueue = BytesEnqueue, - consumers = Cons0} = InitState, Effects1} = + {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 @@ -2104,12 +1975,14 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> %% can happen when draining %% recurse without consumer on queue checkout_one(Meta, ExpiredMsg, - InitState#?MODULE{service_queue = SQ1}, Effects1); + InitState#?STATE{service_queue = SQ1}, + Effects1); #consumer{status = S} when S =:= cancelled orelse S =:= suspected_down -> checkout_one(Meta, ExpiredMsg, - InitState#?MODULE{service_queue = SQ1}, Effects1); + InitState#?STATE{service_queue = SQ1}, + Effects1); #consumer{checked_out = Checked0, next_msg_id = Next, credit = Credit, @@ -2125,11 +1998,12 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> credit = Credit - 1, delivery_count = DelCnt}, Size = get_header(size, get_msg_header(ConsumerMsg)), + State1 = + State0#?STATE{service_queue = SQ1, + msg_bytes_checkout = BytesCheckout + Size, + msg_bytes_enqueue = BytesEnqueue - Size}, State = update_or_remove_sub( - Meta, ConsumerId, Con, - State0#?MODULE{service_queue = SQ1, - msg_bytes_checkout = BytesCheckout + Size, - msg_bytes_enqueue = BytesEnqueue - Size}), + Meta, ConsumerId, Con, State1), {success, ConsumerId, Next, ConsumerMsg, ExpiredMsg, State, Effects1} end; @@ -2139,7 +2013,7 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> {{value, _ConsumerId}, SQ1} -> %% consumer did not exist but was queued, recurse checkout_one(Meta, ExpiredMsg, - InitState#?MODULE{service_queue = SQ1}, Effects1); + InitState#?STATE{service_queue = SQ1}, Effects1); {empty, _} -> case lqueue:len(Messages0) of 0 -> @@ -2167,17 +2041,20 @@ expire_msgs(RaCmdTs, Result, State, Effects) -> expire(RaCmdTs, State0, Effects) -> {?MSG(Idx, Header) = Msg, - #?MODULE{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), + #?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#?MODULE{dlx = DlxState, - ra_indexes = Indexes, - messages_total = Tot - 1, - msg_bytes_enqueue = MsgBytesEnqueue - get_header(size, Header)}, + 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) -> @@ -2185,7 +2062,8 @@ timer_effect(RaCmdTs, State, Effects) -> ?MSG(_, ?TUPLE(Size, Expiry)) when is_integer(Size), is_integer(Expiry) -> %% Next message contains 'expiry' header. - %% (Re)set timer so that mesage will be dropped or dead-lettered on time. + %% (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) -> @@ -2201,27 +2079,29 @@ update_or_remove_sub(Meta, ConsumerId, #consumer{cfg = #consumer_cfg{lifetime = once}, checked_out = Checked, credit = 0} = Con, - #?MODULE{consumers = Cons} = State) -> + #?STATE{consumers = Cons} = State) -> case map_size(Checked) of 0 -> #{system_time := Ts} = Meta, % we're done with this consumer - State#?MODULE{consumers = maps:remove(ConsumerId, Cons), - last_active = Ts}; + State#?STATE{consumers = maps:remove(ConsumerId, Cons), + last_active = Ts}; _ -> % there are unsettled items so need to keep around - State#?MODULE{consumers = maps:put(ConsumerId, Con, Cons)} + State#?STATE{consumers = maps:put(ConsumerId, Con, Cons)} end; update_or_remove_sub(_Meta, ConsumerId, #consumer{cfg = #consumer_cfg{}} = Con, - #?MODULE{consumers = Cons, - service_queue = ServiceQueue} = State) -> - State#?MODULE{consumers = maps:put(ConsumerId, Con, Cons), - service_queue = maybe_queue_consumer(ConsumerId, Con, ServiceQueue)}. + #?STATE{consumers = Cons, + service_queue = ServiceQueue} = State) -> + State#?STATE{consumers = maps:put(ConsumerId, Con, Cons), + service_queue = maybe_queue_consumer(ConsumerId, Con, + ServiceQueue)}. maybe_queue_consumer(Key, #consumer{credit = Credit, status = up, - cfg = #consumer_cfg{priority = P}}, ServiceQueue) + 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 @@ -2236,8 +2116,8 @@ maybe_queue_consumer(_Key, _Consumer, ServiceQueue) -> update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta, {Life, Credit, Mode0} = Spec, Priority, - #?MODULE{cfg = #cfg{consumer_strategy = competing}, - consumers = Cons0} = State0) -> + #?STATE{cfg = #cfg{consumer_strategy = competing}, + consumers = Cons0} = State0) -> Consumer = case Cons0 of #{ConsumerId := #consumer{} = Consumer0} -> merge_consumer(Meta, Consumer0, ConsumerMeta, Spec, Priority); @@ -2255,10 +2135,10 @@ update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta, {Consumer, update_or_remove_sub(Meta, ConsumerId, Consumer, State0)}; update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta, {Life, Credit, Mode0} = Spec, Priority, - #?MODULE{cfg = #cfg{consumer_strategy = single_active}, - consumers = Cons0, - waiting_consumers = Waiting, - service_queue = _ServiceQueue0} = State0) -> + #?STATE{cfg = #cfg{consumer_strategy = single_active}, + consumers = Cons0, + waiting_consumers = Waiting, + service_queue = _ServiceQueue0} = State0) -> %% 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 @@ -2287,7 +2167,7 @@ update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta, credit = Credit, delivery_count = initial_delivery_count(ConsumerMeta)}, {Consumer, - State0#?MODULE{waiting_consumers = + State0#?STATE{waiting_consumers = Waiting ++ [{ConsumerId, Consumer}]}} end. @@ -2303,39 +2183,38 @@ merge_consumer(Meta, #consumer{cfg = CCfg, checked_out = Checked} = Consumer, status = up, credit = NewCredit}. -credit_mode(#{machine_version := Vsn}, Credit, simple_prefetch) - when Vsn >= 3 -> +credit_mode(_Meta, Credit, simple_prefetch) -> {simple_prefetch, Credit}; credit_mode(_, _, Mode) -> Mode. %% creates a dehydrated version of the current state to be cached and %% potentially used to for a snaphot at a later point -dehydrate_state(#?MODULE{cfg = #cfg{}, +dehydrate_state(#?STATE{cfg = #cfg{}, dlx = DlxState} = State) -> % no messages are kept in memory, no need to % overly mutate the current state apart from removing indexes and cursors - State#?MODULE{ra_indexes = rabbit_fifo_index:empty(), + State#?STATE{ra_indexes = rabbit_fifo_index:empty(), release_cursors = lqueue:new(), enqueue_count = 0, msg_cache = undefined, dlx = rabbit_fifo_dlx:dehydrate(DlxState)}. %% make the state suitable for equality comparison -normalize(#?MODULE{ra_indexes = _Indexes, +normalize(#?STATE{ra_indexes = _Indexes, returns = Returns, messages = Messages, release_cursors = Cursors, dlx = DlxState} = State) -> - State#?MODULE{returns = lqueue:from_list(lqueue:to_list(Returns)), + State#?STATE{returns = lqueue:from_list(lqueue:to_list(Returns)), messages = lqueue:from_list(lqueue:to_list(Messages)), release_cursors = lqueue:from_list(lqueue:to_list(Cursors)), dlx = rabbit_fifo_dlx:normalize(DlxState)}. -is_over_limit(#?MODULE{cfg = #cfg{max_length = undefined, +is_over_limit(#?STATE{cfg = #cfg{max_length = undefined, max_bytes = undefined}}) -> false; -is_over_limit(#?MODULE{cfg = #cfg{max_length = MaxLength, +is_over_limit(#?STATE{cfg = #cfg{max_length = MaxLength, max_bytes = MaxBytes}, msg_bytes_enqueue = BytesEnq, dlx = DlxState} = State) -> @@ -2343,10 +2222,10 @@ is_over_limit(#?MODULE{cfg = #cfg{max_length = MaxLength, (messages_ready(State) + NumDlx > MaxLength) orelse (BytesEnq + BytesDlx > MaxBytes). -is_below_soft_limit(#?MODULE{cfg = #cfg{max_length = undefined, +is_below_soft_limit(#?STATE{cfg = #cfg{max_length = undefined, max_bytes = undefined}}) -> false; -is_below_soft_limit(#?MODULE{cfg = #cfg{max_length = MaxLength, +is_below_soft_limit(#?STATE{cfg = #cfg{max_length = MaxLength, max_bytes = MaxBytes}, msg_bytes_enqueue = BytesEnq, dlx = DlxState} = State) -> @@ -2408,16 +2287,16 @@ make_update_config(Config) -> #update_config{config = Config}. add_bytes_drop(Header, - #?MODULE{msg_bytes_enqueue = Enqueue} = State) -> + #?STATE{msg_bytes_enqueue = Enqueue} = State) -> Size = get_header(size, Header), - State#?MODULE{msg_bytes_enqueue = Enqueue - Size}. + State#?STATE{msg_bytes_enqueue = Enqueue - Size}. add_bytes_return(Header, - #?MODULE{msg_bytes_checkout = Checkout, + #?STATE{msg_bytes_checkout = Checkout, msg_bytes_enqueue = Enqueue} = State) -> Size = get_header(size, Header), - State#?MODULE{msg_bytes_checkout = Checkout - Size, + State#?STATE{msg_bytes_checkout = Checkout - Size, msg_bytes_enqueue = Enqueue + Size}. message_size(#basic_message{content = Content}) -> @@ -2436,9 +2315,9 @@ message_size(Msg) -> end. -all_nodes(#?MODULE{consumers = Cons0, - enqueuers = Enqs0, - waiting_consumers = WaitingConsumers0}) -> +all_nodes(#?STATE{consumers = Cons0, + enqueuers = Enqs0, + waiting_consumers = WaitingConsumers0}) -> Nodes0 = maps:fold(fun({_, P}, _, Acc) -> Acc#{node(P) => ok} end, #{}, Cons0), @@ -2450,8 +2329,8 @@ all_nodes(#?MODULE{consumers = Cons0, Acc#{node(P) => ok} end, Nodes1, WaitingConsumers0)). -all_pids_for(Node, #?MODULE{consumers = Cons0, - enqueuers = Enqs0, +all_pids_for(Node, #?STATE{consumers = Cons0, + enqueuers = Enqs0, waiting_consumers = WaitingConsumers0}) -> Cons = maps:fold(fun({_, P}, _, Acc) when node(P) =:= Node -> @@ -2469,9 +2348,9 @@ all_pids_for(Node, #?MODULE{consumers = Cons0, (_, Acc) -> Acc end, Enqs, WaitingConsumers0). -suspected_pids_for(Node, #?MODULE{consumers = Cons0, - enqueuers = Enqs0, - waiting_consumers = WaitingConsumers0}) -> +suspected_pids_for(Node, #?STATE{consumers = Cons0, + enqueuers = Enqs0, + waiting_consumers = WaitingConsumers0}) -> Cons = maps:fold(fun({_, P}, #consumer{status = suspected_down}, Acc) @@ -2491,7 +2370,7 @@ suspected_pids_for(Node, #?MODULE{consumers = Cons0, (_, Acc) -> Acc end, Enqs, WaitingConsumers0). -is_expired(Ts, #?MODULE{cfg = #cfg{expires = Expires}, +is_expired(Ts, #?STATE{cfg = #cfg{expires = Expires}, last_active = LastActive, consumers = Consumers}) when is_number(LastActive) andalso is_number(Expires) -> @@ -2528,11 +2407,13 @@ convert(To, To, State) -> convert(0, To, State) -> convert(1, To, rabbit_fifo_v1:convert_v0_to_v1(State)); convert(1, To, State) -> - convert(2, To, convert_v1_to_v2(State)); + convert(2, To, rabbit_fifo_v3:convert_v1_to_v2(State)); convert(2, To, State) -> - convert(3, To, convert_v2_to_v3(State)). + convert(3, To, rabbit_fifo_v3:convert_v2_to_v3(State)); +convert(3, To, State) -> + convert(4, To, convert_v3_to_v4(State)). -smallest_raft_index(#?MODULE{messages = Messages, +smallest_raft_index(#?STATE{messages = Messages, ra_indexes = Indexes, dlx = DlxState}) -> SmallestDlxRaIdx = rabbit_fifo_dlx:smallest_raft_index(DlxState), @@ -2567,8 +2448,8 @@ make_requeue(ConsumerId, Notify, [{MsgId, Idx, Header, Msg} | Rem], Acc) -> make_requeue(_ConsumerId, _Notify, [], []) -> []. -can_immediately_deliver(#?MODULE{service_queue = SQ, - consumers = Consumers} = State) -> +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 @@ -2595,8 +2476,7 @@ initial_delivery_count(_) -> %% credit API v1 0. --spec credit_api_v2(#consumer_cfg{}) -> - boolean(). +-spec credit_api_v2(#consumer_cfg{}) -> boolean(). credit_api_v2(#consumer_cfg{meta = ConsumerMeta}) -> maps:is_key(initial_delivery_count, ConsumerMeta). diff --git a/deps/rabbit/src/rabbit_fifo_v3.erl b/deps/rabbit/src/rabbit_fifo_v3.erl new file mode 100644 index 000000000000..70691aa9feb9 --- /dev/null +++ b/deps/rabbit/src/rabbit_fifo_v3.erl @@ -0,0 +1,2569 @@ +%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. + +-module(rabbit_fifo_v3). + +-behaviour(ra_machine). + +-compile(inline_list_funcs). +-compile(inline). +-compile({no_auto_import, [apply/3]}). +-dialyzer(no_improper_lists). + +-include("rabbit_fifo_v3.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +-define(STATE, rabbit_fifo). + +-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/6, + % 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, + + %% misc + dehydrate_state/1, + normalize/1, + get_msg_header/1, + get_header/2, + get_msg/1, + + %% protocol helpers + make_enqueue/3, + make_register_enqueuer/1, + make_checkout/3, + make_settle/2, + make_return/2, + make_discard/2, + make_credit/4, + make_purge/0, + make_purge_nodes/1, + make_update_config/1, + make_garbage_collection/0, + convert_v1_to_v2/1, + convert_v2_to_v3/1 + ]). + +-ifdef(TEST). +-export([update_header/4, + chunk_disk_msgs/3]). +-endif. + +%% command records representing all the protocol actions that are supported +-record(enqueue, {pid :: option(pid()), + seq :: option(msg_seqno()), + msg :: raw_msg()}). +-record(requeue, {consumer_id :: consumer_id(), + 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_id :: consumer_id(), + msg_ids :: [msg_id()]}). +-record(return, {consumer_id :: consumer_id(), + msg_ids :: [msg_id()]}). +-record(discard, {consumer_id :: consumer_id(), + msg_ids :: [msg_id()]}). +-record(credit, {consumer_id :: consumer_id(), + credit :: non_neg_integer(), + delivery_count :: non_neg_integer(), + drain :: boolean()}). +-record(purge, {}). +-record(purge_nodes, {nodes :: [node()]}). +-record(update_config, {config :: config()}). +-record(garbage_collection, {}). + +-opaque protocol() :: + #enqueue{} | + #requeue{} | + #register_enqueuer{} | + #checkout{} | + #settle{} | + #return{} | + #discard{} | + #credit{} | + #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_tag/0, + consumer_meta/0, + consumer_id/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), + RCI = maps:get(release_cursor_interval, Conf, ?RELEASE_CURSOR_EVERY), + Overflow = maps:get(overflow_strategy, Conf, drop_head), + MaxLength = maps:get(max_length, Conf, undefined), + MaxBytes = maps:get(max_bytes, Conf, undefined), + DeliveryLimit = maps:get(delivery_limit, Conf, undefined), + 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, + RCISpec = {RCI, RCI}, + + LastActive = maps:get(created, Conf, undefined), + State#?STATE{cfg = Cfg#cfg{release_cursor_interval = RCISpec, + 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(), Reply :: term(), ra_machine:effects()} | + {state(), Reply :: term()}. +apply(Meta, #enqueue{pid = From, seq = Seq, + msg = RawMsg}, State00) -> + apply_enqueue(Meta, From, Seq, RawMsg, 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_id = ConsumerId}, + #?STATE{consumers = Cons0} = State) -> + case Cons0 of + #{ConsumerId := Con0} -> + complete_and_checkout(Meta, MsgIds, ConsumerId, + Con0, [], State); + _ -> + {State, ok} + end; +apply(Meta, #discard{msg_ids = MsgIds, consumer_id = ConsumerId}, + #?STATE{consumers = Cons, + dlx = DlxState0, + cfg = #cfg{dead_letter_handler = DLH}} = State0) -> + case Cons of + #{ConsumerId := #consumer{checked_out = Checked} = Con} -> + % Publishing to dead-letter exchange must maintain same order as messages got rejected. + DiscardMsgs = lists:filtermap(fun(Id) -> + case maps:get(Id, Checked, undefined) of + undefined -> + false; + Msg -> + {true, Msg} + end + end, MsgIds), + {DlxState, Effects} = rabbit_fifo_dlx:discard(DiscardMsgs, rejected, DLH, DlxState0), + State = State0#?STATE{dlx = DlxState}, + complete_and_checkout(Meta, MsgIds, ConsumerId, Con, Effects, State); + _ -> + {State0, ok} + end; +apply(Meta, #return{msg_ids = MsgIds, consumer_id = ConsumerId}, + #?STATE{consumers = Cons0} = State) -> + case Cons0 of + #{ConsumerId := #consumer{checked_out = Checked0}} -> + Returned = maps:with(MsgIds, Checked0), + return(Meta, ConsumerId, Returned, [], State); + _ -> + {State, ok} + end; +apply(#{index := Idx} = Meta, + #requeue{consumer_id = ConsumerId, + msg_id = MsgId, + index = OldIdx, + header = Header0, + msg = _Msg}, + #?STATE{consumers = Cons0, + messages = Messages, + ra_indexes = Indexes0, + enqueue_count = EnqCount} = State00) -> + case Cons0 of + #{ConsumerId := #consumer{checked_out = Checked0} = Con0} + when is_map_key(MsgId, Checked0) -> + %% construct a message with the current raft index + %% and update delivery count before adding it to the message queue + Header = update_header(delivery_count, fun incr/1, 1, Header0), + State0 = add_bytes_return(Header, State00), + Con = Con0#consumer{checked_out = maps:remove(MsgId, Checked0), + credit = increase_credit(Meta, Con0, 1)}, + State1 = State0#?STATE{ra_indexes = rabbit_fifo_index:delete(OldIdx, Indexes0), + messages = lqueue:in(?MSG(Idx, Header), Messages), + enqueue_count = EnqCount + 1}, + State2 = update_or_remove_sub(Meta, ConsumerId, Con, State1), + {State, Ret, Effs} = checkout(Meta, State0, State2, []), + update_smallest_raft_index(Idx, Ret, + maybe_store_release_cursor(Idx, State), + Effs); + _ -> + {State00, ok, []} + end; +apply(Meta, #credit{credit = NewCredit, delivery_count = RemoteDelCnt, + drain = Drain, consumer_id = ConsumerId}, + #?STATE{consumers = Cons0, + service_queue = ServiceQueue0, + waiting_consumers = Waiting0} = State0) -> + case Cons0 of + #{ConsumerId := #consumer{delivery_count = DelCnt} = Con0} -> + %% this can go below 0 when credit is reduced + C = max(0, RemoteDelCnt + NewCredit - DelCnt), + %% grant the credit + Con1 = Con0#consumer{credit = C}, + ServiceQueue = maybe_queue_consumer(ConsumerId, Con1, + ServiceQueue0), + Cons = maps:put(ConsumerId, Con1, Cons0), + {State1, ok, Effects} = + checkout(Meta, State0, + State0#?STATE{service_queue = ServiceQueue, + consumers = Cons}, []), + Response = {send_credit_reply, messages_ready(State1)}, + %% by this point all checkouts for the updated credit value + %% should be processed so we can evaluate the drain + case Drain of + false -> + %% just return the result of the checkout + {State1, Response, Effects}; + true -> + Con = #consumer{credit = PostCred} = + maps:get(ConsumerId, State1#?STATE.consumers), + %% add the outstanding credit to the delivery count + DeliveryCount = Con#consumer.delivery_count + PostCred, + Consumers = maps:put(ConsumerId, + Con#consumer{delivery_count = DeliveryCount, + credit = 0}, + State1#?STATE.consumers), + Drained = Con#consumer.credit, + {CTag, _} = ConsumerId, + {State1#?STATE{consumers = Consumers}, + %% returning a multi response with two client actions + %% for the channel to execute + {multi, [Response, {send_drained, {CTag, Drained}}]}, + Effects} + end; + _ when Waiting0 /= [] -> + %% there are waiting consuemrs + case lists:keytake(ConsumerId, 1, Waiting0) of + {value, {_, Con0 = #consumer{delivery_count = DelCnt}}, Waiting} -> + %% the consumer is a waiting one + %% grant the credit + C = max(0, RemoteDelCnt + NewCredit - DelCnt), + Con = Con0#consumer{credit = C}, + State = State0#?STATE{waiting_consumers = + [{ConsumerId, Con} | Waiting]}, + {State, {send_credit_reply, messages_ready(State)}}; + false -> + {State0, ok} + end; + _ -> + %% credit for unknown consumer - just ignore + {State0, ok} + 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 = maps:is_key(ConsumerId, Consumers), + case messages_ready(State0) of + 0 -> + update_smallest_raft_index(Index, {dequeue, empty}, State0, []); + _ when Exists -> + %% a dequeue using the same consumer_id isn't possible at this point + {State0, {dequeue, empty}}; + _ -> + {_, State1} = update_consumer(Meta, ConsumerId, ConsumerMeta, + {once, 1, simple_prefetch}, 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), + Reply = '$ra_no_reply', + case {DroppedMsg, ExpiredMsg} of + {false, false} -> + {State, Reply, Effects}; + _ -> + update_smallest_raft_index(Index, Reply, State, Effects) + end; + {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), + update_smallest_raft_index(Index, {dequeue, empty}, State, Effects) + end + end; +apply(#{index := Idx} = Meta, + #checkout{spec = cancel, + consumer_id = ConsumerId}, State0) -> + {State1, Effects1} = cancel_consumer(Meta, ConsumerId, State0, [], + consumer_cancel), + {State, Reply, Effects} = checkout(Meta, State0, State1, Effects1), + update_smallest_raft_index(Idx, Reply, State, Effects); +apply(Meta, #checkout{spec = Spec, meta = ConsumerMeta, + consumer_id = {_, Pid} = ConsumerId}, State0) -> + Priority = get_priority_from_args(ConsumerMeta), + {Consumer, State1} = update_consumer(Meta, 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, + delivery_count => DeliveryCount, + 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 = lqueue:new(), + messages_total = Total - NumReady, + returns = lqueue:new(), + msg_bytes_enqueue = 0 + }, + Effects0 = [garbage_collection], + Reply = {purge, NumReady}, + {State, _, Effects} = evaluate_limit(Index, false, State0, + State1, Effects0), + update_smallest_raft_index(Index, Reply, State, Effects); +apply(#{index := Idx}, #garbage_collection{}, State) -> + update_smallest_raft_index(Idx, ok, State, [{aux, garbage_collection}]); +apply(Meta, {timeout, expire_msgs}, State) -> + checkout(Meta, State, State, []); +apply(#{system_time := Ts, machine_version := MachineVersion} = Meta, + {down, Pid, noconnection}, + #?STATE{consumers = Cons0, + cfg = #cfg{consumer_strategy = single_active}, + waiting_consumers = Waiting0, + enqueuers = Enqs0} = State0) -> + Node = node(Pid), + %% if the pid refers to an active or cancelled consumer, + %% mark it as suspected and return it to the waiting queue + {State1, Effects0} = + maps:fold(fun({_, P} = Cid, 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, Cid, C0, false, suspected_down, E0), + C1 = case MachineVersion of + V when V >= 3 -> + C0; + 2 -> + Checked = C0#consumer.checked_out, + Credit = increase_credit(Meta, C0, maps:size(Checked)), + C0#consumer{credit = Credit} + end, + {St, Effs1} = return_all(Meta, S0, Effs, Cid, C1), + %% 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 + #{Cid := C} -> + Waiting0 ++ [{Cid, C}]; + _ -> + Waiting0 + end, + {St#?STATE{consumers = maps:remove(Cid, St#?STATE.consumers), + waiting_consumers = Waiting, + last_active = Ts}, + Effs1}; + (_, _, S) -> + S + end, {State0, []}, Cons0), + WaitingConsumers = update_waiting_consumer_status(Node, State1, + suspected_down), + + %% select a new consumer from the waiting queue and run a checkout + State2 = State1#?STATE{waiting_consumers = WaitingConsumers}, + {State, Effects1} = activate_next_consumer(State2, Effects0), + + %% mark any enquers as suspected + Enqs = maps:map(fun(P, E) when node(P) =:= Node -> + E#enqueuer{status = suspected_down}; + (_, E) -> E + end, Enqs0), + Effects = [{monitor, node, Node} | Effects1], + checkout(Meta, State0, State#?STATE{enqueuers = Enqs}, Effects); +apply(#{system_time := Ts, machine_version := MachineVersion} = 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} = + maps:fold( + fun({_, P} = Cid, #consumer{checked_out = Checked0, + status = up} = C0, + {St0, Eff}) when node(P) =:= Node -> + C = case MachineVersion of + V when V >= 3 -> + C0#consumer{status = suspected_down}; + 2 -> + Credit = increase_credit(Meta, C0, map_size(Checked0)), + C0#consumer{status = suspected_down, + credit = Credit} + end, + {St, Eff0} = return_all(Meta, St0, Eff, Cid, C), + Eff1 = consumer_update_active_effects(St, Cid, C, false, + suspected_down, Eff0), + {St, Eff1}; + (_, _, {St, Eff}) -> + {St, Eff} + end, {State0, []}, Cons0), + 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(#{index := Idx} = Meta, {down, Pid, _Info}, State0) -> + {State1, Effects1} = handle_down(Meta, Pid, State0), + {State, Reply, Effects} = checkout(Meta, State0, State1, Effects1), + update_smallest_raft_index(Idx, Reply, State, Effects); +apply(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, 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} = + maps:fold(fun({_, P} = ConsumerId, C, {SAcc, EAcc}) + when (node(P) =:= Node) and + (C#consumer.status =/= cancelled) -> + EAcc1 = ConsumerUpdateActiveFun(SAcc, ConsumerId, + C, true, up, EAcc), + {update_or_remove_sub(Meta, ConsumerId, + C#consumer{status = up}, + SAcc), EAcc1}; + (_, _, Acc) -> + Acc + end, {State0, Monitors}, Cons0), + 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(#{index := Idx} = Meta, #purge_nodes{nodes = Nodes}, State0) -> + {State, Effects} = lists:foldl(fun(Node, {S, E}) -> + purge_node(Meta, Node, S, E) + end, {State0, []}, Nodes), + update_smallest_raft_index(Idx, ok, State, Effects); +apply(#{index := Idx} = Meta, + #update_config{config = #{dead_letter_handler := NewDLH} = Conf}, + #?STATE{cfg = #cfg{dead_letter_handler = OldDLH, + resource = QRes}, + dlx = DlxState0} = State0) -> + {DlxState, Effects0} = rabbit_fifo_dlx:update_config(OldDLH, NewDLH, QRes, DlxState0), + State1 = update_config(Conf, State0#?STATE{dlx = DlxState}), + {State, Reply, Effects} = checkout(Meta, State0, State1, Effects0), + update_smallest_raft_index(Idx, Reply, State, Effects); +apply(_Meta, {machine_version, FromVersion, ToVersion}, V0State) -> + State = convert(FromVersion, ToVersion, V0State), + {State, ok, [{aux, {dlx, setup}}]}; +apply(#{index := IncomingRaftIdx} = 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}, + {State, ok, Effects} = checkout(Meta, State0, State1, Effects0), + update_smallest_raft_index(IncomingRaftIdx, State, Effects); +apply(_Meta, Cmd, State) -> + %% handle unhandled commands gracefully + rabbit_log:debug("rabbit_fifo: unhandled command ~W", [Cmd, 10]), + {State, ok, []}. + +convert_msg({RaftIdx, {Header, empty}}) when is_integer(RaftIdx) -> + ?MSG(RaftIdx, Header); +convert_msg({RaftIdx, {Header, _Msg}}) when is_integer(RaftIdx) -> + ?MSG(RaftIdx, Header); +convert_msg({'$empty_msg', Header}) -> + %% dummy index + ?MSG(undefined, Header); +convert_msg({'$prefix_msg', Header}) -> + %% dummy index + ?MSG(undefined, Header); +convert_msg({Header, empty}) -> + convert_msg(Header); +convert_msg(Header) when ?IS_HEADER(Header) -> + ?MSG(undefined, Header). + +convert_consumer_v1_to_v2({ConsumerTag, Pid}, CV1) -> + Meta = element(2, CV1), + CheckedOut = element(3, CV1), + NextMsgId = element(4, CV1), + Credit = element(5, CV1), + DeliveryCount = element(6, CV1), + CreditMode = element(7, CV1), + LifeTime = element(8, CV1), + Status = element(9, CV1), + Priority = element(10, CV1), + #consumer{cfg = #consumer_cfg{tag = ConsumerTag, + pid = Pid, + meta = Meta, + credit_mode = CreditMode, + lifetime = LifeTime, + priority = Priority}, + credit = Credit, + status = Status, + delivery_count = DeliveryCount, + next_msg_id = NextMsgId, + checked_out = maps:map( + fun (_, {Tag, _} = Msg) when is_atom(Tag) -> + convert_msg(Msg); + (_, {_Seq, Msg}) -> + convert_msg(Msg) + end, CheckedOut) + }. + +convert_v1_to_v2(V1State0) -> + V1State = rabbit_fifo_v1:enqueue_all_pending(V1State0), + IndexesV1 = rabbit_fifo_v1:get_field(ra_indexes, V1State), + ReturnsV1 = rabbit_fifo_v1:get_field(returns, V1State), + MessagesV1 = rabbit_fifo_v1:get_field(messages, V1State), + ConsumersV1 = rabbit_fifo_v1:get_field(consumers, V1State), + WaitingConsumersV1 = rabbit_fifo_v1:get_field(waiting_consumers, V1State), + %% remove all raft idx in messages from index + {_, PrefReturns, _, PrefMsgs} = rabbit_fifo_v1:get_field(prefix_msgs, V1State), + V2PrefMsgs = lists:foldl(fun(Hdr, Acc) -> + lqueue:in(convert_msg(Hdr), Acc) + end, lqueue:new(), PrefMsgs), + V2PrefReturns = lists:foldl(fun(Hdr, Acc) -> + lqueue:in(convert_msg(Hdr), Acc) + end, lqueue:new(), PrefReturns), + MessagesV2 = lqueue:fold(fun ({_, Msg}, Acc) -> + lqueue:in(convert_msg(Msg), Acc) + end, V2PrefMsgs, MessagesV1), + ReturnsV2 = lqueue:fold(fun ({_SeqId, Msg}, Acc) -> + lqueue:in(convert_msg(Msg), Acc) + end, V2PrefReturns, ReturnsV1), + ConsumersV2 = maps:map( + fun (ConsumerId, CV1) -> + convert_consumer_v1_to_v2(ConsumerId, CV1) + end, ConsumersV1), + WaitingConsumersV2 = lists:map( + fun ({ConsumerId, CV1}) -> + {ConsumerId, convert_consumer_v1_to_v2(ConsumerId, CV1)} + end, WaitingConsumersV1), + EnqueuersV1 = rabbit_fifo_v1:get_field(enqueuers, V1State), + EnqueuersV2 = maps:map(fun (_EnqPid, Enq) -> + Enq#enqueuer{unused = undefined} + end, EnqueuersV1), + + %% do after state conversion + %% The (old) format of dead_letter_handler in RMQ < v3.10 is: + %% {Module, Function, Args} + %% The (new) format of dead_letter_handler in RMQ >= v3.10 is: + %% undefined | {at_most_once, {Module, Function, Args}} | at_least_once + %% + %% Note that the conversion must convert both from old format to new format + %% as well as from new format to new format. The latter is because quorum queues + %% created in RMQ >= v3.10 are still initialised with rabbit_fifo_v0 as described in + %% https://github.com/rabbitmq/ra/blob/e0d1e6315a45f5d3c19875d66f9d7bfaf83a46e3/src/ra_machine.erl#L258-L265 + DLH = case rabbit_fifo_v1:get_cfg_field(dead_letter_handler, V1State) of + {_M, _F, _A = [_DLX = undefined|_]} -> + %% queue was declared in RMQ < v3.10 and no DLX configured + undefined; + {_M, _F, _A} = MFA -> + %% queue was declared in RMQ < v3.10 and DLX configured + {at_most_once, MFA}; + Other -> + Other + end, + + Cfg = #cfg{name = rabbit_fifo_v1:get_cfg_field(name, V1State), + resource = rabbit_fifo_v1:get_cfg_field(resource, V1State), + release_cursor_interval = rabbit_fifo_v1:get_cfg_field(release_cursor_interval, V1State), + dead_letter_handler = DLH, + become_leader_handler = rabbit_fifo_v1:get_cfg_field(become_leader_handler, V1State), + %% TODO: what if policy enabling reject_publish was applied before conversion? + overflow_strategy = rabbit_fifo_v1:get_cfg_field(overflow_strategy, V1State), + max_length = rabbit_fifo_v1:get_cfg_field(max_length, V1State), + max_bytes = rabbit_fifo_v1:get_cfg_field(max_bytes, V1State), + consumer_strategy = rabbit_fifo_v1:get_cfg_field(consumer_strategy, V1State), + delivery_limit = rabbit_fifo_v1:get_cfg_field(delivery_limit, V1State), + expires = rabbit_fifo_v1:get_cfg_field(expires, V1State) + }, + + MessagesConsumersV2 = maps:fold(fun(_ConsumerId, #consumer{checked_out = Checked}, Acc) -> + Acc + maps:size(Checked) + end, 0, ConsumersV2), + MessagesWaitingConsumersV2 = lists:foldl(fun({_ConsumerId, #consumer{checked_out = Checked}}, Acc) -> + Acc + maps:size(Checked) + end, 0, WaitingConsumersV2), + MessagesTotal = lqueue:len(MessagesV2) + + lqueue:len(ReturnsV2) + + MessagesConsumersV2 + + MessagesWaitingConsumersV2, + + #?STATE{cfg = Cfg, + messages = MessagesV2, + messages_total = MessagesTotal, + returns = ReturnsV2, + enqueue_count = rabbit_fifo_v1:get_field(enqueue_count, V1State), + enqueuers = EnqueuersV2, + ra_indexes = IndexesV1, + release_cursors = rabbit_fifo_v1:get_field(release_cursors, V1State), + consumers = ConsumersV2, + service_queue = rabbit_fifo_v1:get_field(service_queue, V1State), + msg_bytes_enqueue = rabbit_fifo_v1:get_field(msg_bytes_enqueue, V1State), + msg_bytes_checkout = rabbit_fifo_v1:get_field(msg_bytes_checkout, V1State), + waiting_consumers = WaitingConsumersV2, + last_active = rabbit_fifo_v1:get_field(last_active, V1State) + }. + +convert_v2_to_v3(#rabbit_fifo{consumers = ConsumersV2} = StateV2) -> + ConsumersV3 = maps:map(fun(_, C) -> + convert_consumer_v2_to_v3(C) + end, ConsumersV2), + StateV2#rabbit_fifo{consumers = ConsumersV3}. + +convert_consumer_v2_to_v3(C = #consumer{cfg = Cfg = #consumer_cfg{credit_mode = simple_prefetch, + meta = #{prefetch := Prefetch}}}) -> + C#consumer{cfg = Cfg#consumer_cfg{credit_mode = {simple_prefetch, Prefetch}}}; +convert_consumer_v2_to_v3(C) -> + C. + +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, State)). + +%% any downs that re not noconnection +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), + % return checked out messages to main queue + % Find the consumers for the down pid + DownConsumers = maps:keys( + maps:filter(fun({_, P}, _) -> P =:= Pid end, Cons0)), + lists:foldl(fun(ConsumerId, {S, E}) -> + cancel_consumer(Meta, ConsumerId, S, E, down) + end, {State2, Effects1}, DownConsumers). + +consumer_active_flag_update_function( + #?STATE{cfg = #cfg{consumer_strategy = competing}}) -> + fun(State, ConsumerId, Consumer, Active, ActivityStatus, Effects) -> + consumer_update_active_effects(State, ConsumerId, 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({{_, P}, _}) -> P =:= Pid end, + WaitingConsumers0), + Effects = lists:foldl(fun ({ConsumerId, _}, Effects) -> + cancel_consumer_effects(ConsumerId, State0, + Effects) + end, [], Down), + % update state to have only up waiting consumers + StillUp = lists:filter(fun({{_, P}, _}) -> P =/= Pid end, + WaitingConsumers0), + State = State0#?STATE{waiting_consumers = StillUp}, + {Effects, State}. + +update_waiting_consumer_status(Node, + #?STATE{waiting_consumers = WaitingConsumers}, + Status) -> + [begin + case node(Pid) of + Node -> + {ConsumerId, Consumer#consumer{status = Status}}; + _ -> + {ConsumerId, Consumer} + end + end || {{_, Pid} = ConsumerId, 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 || {_, P} <- maps:keys(Cons)] + ++ [P || {{_, 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]), + FHReservation = [{mod_call, rabbit_quorum_queue, + file_handle_leader_reservation, [Resource]}], + NotifyDecs = notify_decorators_startup(Resource), + Effects = TimerEffs ++ Mons ++ Nots ++ NodeMons ++ FHReservation ++ [NotifyDecs], + case BLH of + undefined -> + Effects; + {Mod, Fun, Args} -> + [{mod_call, Mod, Fun, Args ++ [Name]} | Effects] + end; +state_enter0(eol, #?STATE{enqueuers = Enqs, + consumers = Custs0, + waiting_consumers = WaitingConsumers0}, + Effects) -> + Custs = maps:fold(fun({_, P}, V, S) -> S#{P => V} end, #{}, Custs0), + WaitingConsumers1 = lists:foldl(fun({{_, 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}, + {mod_call, rabbit_quorum_queue, file_handle_release_reservation, []} | Effects]; +state_enter0(State, #?STATE{cfg = #cfg{resource = _Resource}}, Effects) + when State =/= leader -> + FHReservation = {mod_call, rabbit_quorum_queue, file_handle_other_reservation, []}, + [FHReservation | 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{name = _Name, + 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, + release_cursors = Cursors, + enqueue_count = EnqCount, + msg_bytes_enqueue = EnqueueBytes, + msg_bytes_checkout = CheckoutBytes, + cfg = Cfg, + dlx = DlxState, + waiting_consumers = WaitingConsumers} = State) -> + Conf = #{name => Cfg#cfg.name, + resource => Cfg#cfg.resource, + release_cursor_interval => Cfg#cfg.release_cursor_interval, + dead_lettering_enabled => undefined =/= Cfg#cfg.dead_letter_handler, + 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 + {SacConsumerId, _} -> + NumWaiting = length(WaitingConsumers), + #{single_active_consumer_id => SacConsumerId, + single_active_num_waiting_consumers => NumWaiting}; + _ -> + #{} + end, + 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_in_memory_ready_messages => 0, %% backwards compat + num_messages => messages_total(State), + num_release_cursors => lqueue:len(Cursors), + release_cursors => [I || {_, I, _} <- lqueue:to_list(Cursors)], + release_cursor_enqueue_counter => EnqCount, + enqueue_message_bytes => EnqueueBytes, + checkout_message_bytes => CheckoutBytes, + in_memory_message_bytes => 0, %% backwards compat + smallest_raft_index => smallest_raft_index(State) + }, + DlxOverview = rabbit_fifo_dlx:overview(DlxState), + maps:merge(maps:merge(Overview, DlxOverview), SacOverview). + +-spec get_checked_out(consumer_id(), msg_id(), msg_id(), state()) -> + [delivery_msg()]. +get_checked_out(Cid, From, To, #?STATE{consumers = Consumers}) -> + case Consumers of + #{Cid := #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() -> 3. + +which_module(0) -> rabbit_fifo_v0; +which_module(1) -> rabbit_fifo_v1; +which_module(2) -> ?STATE; +which_module(3) -> ?STATE. + +-define(AUX, aux_v2). + +-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, + cache = #{} :: map()}). + +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(micro_seconds), + #?AUX{name = Name, + capacity = {inactive, Now, 1, 1.0}}. + +handle_aux(RaftState, Tag, Cmd, #aux{name = Name, + capacity = Cap, + gc = Gc}, Log, MacState) -> + %% convert aux state to new version + Aux = #?AUX{name = Name, + capacity = Cap, + gc = Gc}, + handle_aux(RaftState, Tag, Cmd, Aux, Log, MacState); +handle_aux(leader, _, garbage_collection, Aux, Log, MacState) -> + {no_reply, force_eval_gc(Log, MacState, Aux), Log}; +handle_aux(follower, _, garbage_collection, Aux, Log, MacState) -> + {no_reply, force_eval_gc(Log, MacState, Aux), Log}; +handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, + consumer_id = ConsumerId}, Corr, Pid}, + Aux0, Log0, #?STATE{cfg = #cfg{delivery_limit = undefined}, + consumers = Consumers}) -> + case Consumers of + #{ConsumerId := #consumer{checked_out = Checked}} -> + {Log, ToReturn} = + maps:fold( + fun (MsgId, ?MSG(Idx, Header), {L0, Acc}) -> + %% it is possible this is not found if the consumer + %% crashed and the message got removed + case ra_log:fetch(Idx, L0) of + {{_, _, {_, _, Cmd, _}}, L} -> + Msg = get_msg(Cmd), + {L, [{MsgId, Idx, Header, Msg} | Acc]}; + {undefined, L} -> + {L, Acc} + end + end, {Log0, []}, maps:with(MsgIds, Checked)), + + Appends = make_requeue(ConsumerId, {notify, Corr, Pid}, + lists:sort(ToReturn), []), + {no_reply, Aux0, Log, Appends}; + _ -> + {no_reply, Aux0, Log0} + end; +handle_aux(leader, _, {handle_tick, [QName, Overview, Nodes]}, + #?AUX{tick_pid = Pid} = Aux, Log, _) -> + 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, + {no_reply, Aux#?AUX{tick_pid = NewPid}, Log}; +handle_aux(_, _, {get_checked_out, ConsumerId, MsgIds}, + Aux0, Log0, #?STATE{cfg = #cfg{}, + consumers = Consumers}) -> + case Consumers of + #{ConsumerId := #consumer{checked_out = Checked}} -> + {Log, IdMsgs} = + maps:fold( + fun (MsgId, ?MSG(Idx, Header), {L0, Acc}) -> + %% it is possible this is not found if the consumer + %% crashed and the message got removed + case ra_log:fetch(Idx, L0) of + {{_, _, {_, _, Cmd, _}}, L} -> + Msg = get_msg(Cmd), + {L, [{MsgId, {Header, Msg}} | Acc]}; + {undefined, L} -> + {L, Acc} + end + end, {Log0, []}, maps:with(MsgIds, Checked)), + {reply, {ok, IdMsgs}, Aux0, Log}; + _ -> + {reply, {error, consumer_not_found}, Aux0, Log0} + end; +handle_aux(leader, cast, {#return{} = Ret, Corr, Pid}, + Aux0, Log, #?STATE{}) -> + %% for returns with a delivery limit set we can just return as before + {no_reply, Aux0, Log, [{append, Ret, {notify, Corr, Pid}}]}; +handle_aux(leader, cast, eval, #?AUX{last_decorators_state = LastDec} = Aux0, + Log, #?STATE{cfg = #cfg{resource = QName}} = MacState) -> + %% 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 + Ts = erlang:system_time(millisecond), + Effects0 = timer_effect(Ts, MacState, []), + case query_notify_decorators_info(MacState) of + LastDec -> + {no_reply, Aux0, Log, Effects0}; + {MaxActivePriority, IsEmpty} = NewLast -> + Effects = [notify_decorators_effect(QName, MaxActivePriority, IsEmpty) + | Effects0], + {no_reply, Aux0#?AUX{last_decorators_state = NewLast}, Log, Effects} + end; +handle_aux(_RaftState, cast, eval, Aux0, Log, _MacState) -> + {no_reply, Aux0, Log}; +handle_aux(_RaState, cast, Cmd, #?AUX{capacity = Use0} = Aux0, + Log, _MacState) + when Cmd == active orelse Cmd == inactive -> + {no_reply, Aux0#?AUX{capacity = update_use(Use0, Cmd)}, Log}; +handle_aux(_RaState, cast, tick, #?AUX{name = Name, + capacity = Use0} = State0, + Log, MacState) -> + true = ets:insert(rabbit_fifo_usage, + {Name, capacity(Use0)}), + Aux = eval_gc(Log, MacState, State0), + {no_reply, Aux, Log}; +handle_aux(_RaState, cast, eol, #?AUX{name = Name} = Aux, Log, _) -> + ets:delete(rabbit_fifo_usage, Name), + {no_reply, Aux, Log}; +handle_aux(_RaState, {call, _From}, oldest_entry_timestamp, + #?AUX{cache = Cache} = Aux0, + Log0, #?STATE{} = State) -> + {CachedIdx, CachedTs} = maps:get(oldest_entry, Cache, {undefined, undefined}), + case smallest_raft_index(State) 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, Log0}; + CachedIdx -> + %% cache hit + {reply, {ok, CachedTs}, Aux0, Log0}; + Idx when is_integer(Idx) -> + case ra_log:fetch(Idx, Log0) of + {{_, _, {_, #{ts := Timestamp}, _, _}}, Log1} -> + Aux1 = Aux0#?AUX{cache = Cache#{oldest_entry => + {Idx, Timestamp}}}, + {reply, {ok, Timestamp}, Aux1, Log1}; + {undefined, Log1} -> + %% fetch failed + {reply, {error, failed_to_get_timestamp}, Aux0, Log1} + end + end; +handle_aux(_RaState, {call, _From}, {peek, Pos}, Aux0, + Log0, MacState) -> + case query_peek(Pos, MacState) of + {ok, ?MSG(Idx, Header)} -> + %% need to re-hydrate from the log + {{_, _, {_, _, Cmd, _}}, Log} = ra_log:fetch(Idx, Log0), + Msg = get_msg(Cmd), + {reply, {ok, {Header, Msg}}, Aux0, Log}; + Err -> + {reply, Err, Aux0, Log0} + end; +handle_aux(RaState, _, {dlx, _} = Cmd, Aux0, Log, + #?STATE{dlx = DlxState, + cfg = #cfg{dead_letter_handler = DLH, + resource = QRes}}) -> + Aux = rabbit_fifo_dlx:handle_aux(RaState, Cmd, Aux0, QRes, DLH, DlxState), + {no_reply, Aux, Log}. + +eval_gc(Log, #?STATE{cfg = #cfg{resource = QR}} = MacState, + #?AUX{gc = #aux_gc{last_raft_idx = LastGcIdx} = Gc} = AuxState) -> + {Idx, _} = ra_log:last_index_term(Log), + {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), + rabbit_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(Log, #?STATE{cfg = #cfg{resource = QR}}, + #?AUX{gc = #aux_gc{last_raft_idx = LastGcIdx} = Gc} = AuxState) -> + {Idx, _} = ra_log:last_index_term(Log), + {memory, Mem} = erlang:process_info(self(), memory), + case Idx > LastGcIdx of + true -> + garbage_collect(), + {memory, MemAfter} = erlang:process_info(self(), memory), + rabbit_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({_, 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(_ConsumerId, #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(_ConsumerId, + #consumer{status = Status}) -> + case Status of + suspected_down -> + {false, Status}; + _ -> + {true, Status} + end + end; + single_active -> + SingleActiveConsumer = query_single_active_consumer(State), + fun({Tag, Pid} = _Consumer, _) -> + case SingleActiveConsumer of + {value, {Tag, Pid}} -> + {true, single_active}; + _ -> + {false, waiting} + end + end + end, + FromConsumers = + maps:fold(fun (_, #consumer{status = cancelled}, Acc) -> + Acc; + ({Tag, Pid}, + #consumer{cfg = #consumer_cfg{meta = Meta}} = Consumer, + Acc) -> + {Active, ActivityStatus} = + ActiveActivityStatusFun({Tag, Pid}, Consumer), + maps:put({Tag, Pid}, + {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; + ({{Tag, Pid}, + #consumer{cfg = #consumer_cfg{meta = Meta}} = Consumer}, + Acc) -> + {Active, ActivityStatus} = + ActiveActivityStatusFun({Tag, Pid}, Consumer), + maps:put({Tag, Pid}, + {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}; + {ActiveCid, _} -> + {value, ActiveCid} + 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. + +%%% Internal + +messages_ready(#?STATE{messages = M, + returns = R}) -> + lqueue: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(micro_seconds), + {inactive, Now, Now - Since, Avg}; +update_use({inactive, Since, Active, Avg}, active) -> + Now = erlang:monotonic_time(micro_seconds), + {active, Now, use_avg(Active, Now - Since, Avg)}. + +capacity({active, Since, Avg}) -> + use_avg(erlang:monotonic_time(micro_seconds) - Since, 0, Avg); +capacity({inactive, _, 1, 1.0}) -> + 1.0; +capacity({inactive, Since, Active, Avg}) -> + use_avg(Active, erlang:monotonic_time(micro_seconds) - 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, ConsumerId, + #?STATE{cfg = #cfg{consumer_strategy = competing}} = State, + Effects, Reason) -> + cancel_consumer0(Meta, ConsumerId, State, Effects, Reason); +cancel_consumer(Meta, ConsumerId, + #?STATE{cfg = #cfg{consumer_strategy = single_active}, + waiting_consumers = []} = State, + Effects, Reason) -> + %% single active consumer on, no consumers are waiting + cancel_consumer0(Meta, ConsumerId, State, Effects, Reason); +cancel_consumer(Meta, ConsumerId, + #?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 + #{ConsumerId := #consumer{status = _}} -> + % The active consumer is to be removed + {State1, Effects1} = cancel_consumer0(Meta, ConsumerId, State0, + Effects0, Reason), + activate_next_consumer(State1, Effects1); + _ -> + % The cancelled consumer is not active or cancelled + % Just remove it from idle_consumers + Waiting = lists:keydelete(ConsumerId, 1, Waiting0), + Effects = cancel_consumer_effects(ConsumerId, 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} + end. + +consumer_update_active_effects(#?STATE{cfg = #cfg{resource = QName}}, + ConsumerId, + #consumer{cfg = #consumer_cfg{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, ConsumerId, false, Ack, Prefetch, Active, ActivityStatus, Args]} + | Effects]. + +cancel_consumer0(Meta, ConsumerId, + #?STATE{consumers = C0} = S0, Effects0, Reason) -> + case C0 of + #{ConsumerId := Consumer} -> + {S, Effects2} = maybe_return_all(Meta, ConsumerId, 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(ConsumerId, S, Effects2), + {S, Effects}; + _ -> + %% already removed: do nothing + {S0, Effects0} + end. + +activate_next_consumer(#?STATE{cfg = #cfg{consumer_strategy = competing}} = State0, + Effects0) -> + {State0, Effects0}; +activate_next_consumer(#?STATE{consumers = Cons, + waiting_consumers = Waiting0} = State0, + Effects0) -> + case has_active_consumer(Cons) of + false -> + case lists:filter(fun ({_, #consumer{status = Status}}) -> + Status == up + end, Waiting0) of + [{NextConsumerId, #consumer{cfg = NextCCfg} = NextConsumer} | _] -> + Remaining = lists:keydelete(NextConsumerId, 1, Waiting0), + Consumer = case maps:get(NextConsumerId, Cons, undefined) of + undefined -> + NextConsumer; + 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(NextConsumerId, + Consumer, + ServiceQueue), + State = State0#?STATE{consumers = Cons#{NextConsumerId => Consumer}, + service_queue = ServiceQueue1, + waiting_consumers = Remaining}, + Effects = consumer_update_active_effects(State, NextConsumerId, + Consumer, true, + single_active, Effects0), + {State, Effects}; + [] -> + {State0, Effects0} + end; + true -> + {State0, Effects0} + end. + +has_active_consumer(Consumers) -> + active_consumer(Consumers) /= undefined. + +active_consumer({Cid, #consumer{status = up} = Consumer, _I}) -> + {Cid, Consumer}; +active_consumer({_Cid, #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)). + +maybe_return_all(#{system_time := Ts} = Meta, ConsumerId, + #consumer{cfg = CCfg} = Consumer, S0, + Effects0, Reason) -> + case Reason of + consumer_cancel -> + {update_or_remove_sub( + Meta, ConsumerId, + Consumer#consumer{cfg = CCfg#consumer_cfg{lifetime = once}, + credit = 0, + status = cancelled}, + S0), Effects0}; + down -> + {S1, Effects1} = return_all(Meta, S0, Effects0, ConsumerId, Consumer), + {S1#?STATE{consumers = maps:remove(ConsumerId, S1#?STATE.consumers), + last_active = Ts}, + Effects1} + end. + +apply_enqueue(#{index := RaftIdx, + system_time := Ts} = Meta, From, Seq, RawMsg, State0) -> + case maybe_enqueue(RaftIdx, Ts, From, Seq, RawMsg, [], State0) of + {ok, State1, Effects1} -> + {State, ok, Effects} = checkout(Meta, State0, State1, Effects1), + {maybe_store_release_cursor(RaftIdx, State), ok, Effects}; + {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, DlxEffects ++ Effects}; + empty -> + {State0, Effects} + end. + +maybe_set_msg_ttl(#basic_message{content = #content{properties = none}}, + RaCmdTs, Header, + #?STATE{cfg = #cfg{msg_ttl = PerQueueMsgTTL}}) -> + update_expiry_header(RaCmdTs, PerQueueMsgTTL, Header); +maybe_set_msg_ttl(#basic_message{content = #content{properties = Props}}, + RaCmdTs, Header, + #?STATE{cfg = #cfg{msg_ttl = PerQueueMsgTTL}}) -> + %% rabbit_quorum_queue will leave the properties decoded if and only if + %% per message message TTL is set. + %% We already check in the channel that expiration must be valid. + {ok, PerMsgMsgTTL} = rabbit_basic:parse_expiration(Props), + TTL = min(PerMsgMsgTTL, PerQueueMsgTTL), + update_expiry_header(RaCmdTs, TTL, Header); +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. + +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_store_release_cursor(RaftIdx, + #?STATE{cfg = #cfg{release_cursor_interval = {Base, C}} = Cfg, + enqueue_count = EC, + release_cursors = Cursors0} = State0) + when EC >= C -> + case messages_total(State0) of + 0 -> + %% message must have been immediately dropped + State0#?STATE{enqueue_count = 0}; + Total -> + Interval = case Base of + 0 -> 0; + _ -> + min(max(Total, Base), ?RELEASE_CURSOR_EVERY_MAX) + end, + State = State0#?STATE{cfg = Cfg#cfg{release_cursor_interval = + {Base, Interval}}}, + Dehydrated = dehydrate_state(State), + Cursor = {release_cursor, RaftIdx, Dehydrated}, + Cursors = lqueue:in(Cursor, Cursors0), + State#?STATE{enqueue_count = 0, + release_cursors = Cursors} + end; +maybe_store_release_cursor(_RaftIdx, State) -> + State. + +maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, Effects, + #?STATE{msg_bytes_enqueue = Enqueue, + enqueue_count = EnqCount, + messages = Messages, + messages_total = Total} = State0) -> + % direct enqueue without tracking + Size = message_size(RawMsg), + Header = maybe_set_msg_ttl(RawMsg, Ts, Size, State0), + Msg = ?MSG(RaftIdx, Header), + State = State0#?STATE{msg_bytes_enqueue = Enqueue + Size, + enqueue_count = EnqCount + 1, + messages_total = Total + 1, + messages = lqueue:in(Msg, Messages) + }, + {ok, State, Effects}; +maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, 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, Effects0, State1), + {Res, State, [{monitor, process, From} | Effects]}; + #enqueuer{next_seqno = MsgSeqNo} = Enq0 -> + % it is the next expected seqno + Size = message_size(RawMsg), + Header = maybe_set_msg_ttl(RawMsg, Ts, Size, State0), + Msg = ?MSG(RaftIdx, Header), + Enq = Enq0#enqueuer{next_seqno = MsgSeqNo + 1}, + MsgCache = case can_immediately_deliver(State0) of + true -> + {RaftIdx, RawMsg}; + false -> + undefined + end, + State = State0#?STATE{msg_bytes_enqueue = Enqueue + Size, + enqueue_count = EnqCount + 1, + messages_total = Total + 1, + messages = lqueue:in(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(#{index := IncomingRaftIdx, machine_version := MachineVersion} = Meta, + ConsumerId, Returned, Effects0, State0) -> + {State1, Effects1} = maps:fold( + fun(MsgId, Msg, {S0, E0}) -> + return_one(Meta, MsgId, Msg, S0, E0, ConsumerId) + end, {State0, Effects0}, Returned), + State2 = + case State1#?STATE.consumers of + #{ConsumerId := Con} + when MachineVersion >= 3 -> + update_or_remove_sub(Meta, ConsumerId, Con, State1); + #{ConsumerId := Con0} + when MachineVersion =:= 2 -> + Credit = increase_credit(Meta, Con0, map_size(Returned)), + Con = Con0#consumer{credit = Credit}, + update_or_remove_sub(Meta, ConsumerId, Con, State1); + _ -> + State1 + end, + {State, ok, Effects} = checkout(Meta, State0, State2, Effects1), + update_smallest_raft_index(IncomingRaftIdx, State, Effects). + +% used to process messages that are finished +complete(Meta, ConsumerId, [DiscardedMsgId], + #consumer{checked_out = Checked0} = Con0, + #?STATE{ra_indexes = Indexes0, + msg_bytes_checkout = BytesCheckout, + messages_total = Tot} = State0) -> + case maps:take(DiscardedMsgId, 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(Meta, Con0, 1)}, + State1 = update_or_remove_sub(Meta, ConsumerId, Con, State0), + State1#?STATE{ra_indexes = Indexes, + msg_bytes_checkout = BytesCheckout - SettledSize, + messages_total = Tot - 1}; + error -> + State0 + end; +complete(Meta, ConsumerId, DiscardedMsgIds, + #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}, DiscardedMsgIds), + Len = map_size(Checked0) - map_size(Checked), + Con = Con0#consumer{checked_out = Checked, + credit = increase_credit(Meta, Con0, Len)}, + State1 = update_or_remove_sub(Meta, ConsumerId, Con, State0), + State1#?STATE{ra_indexes = Indexes, + msg_bytes_checkout = BytesCheckout - SettledSize, + messages_total = Tot - Len}. + +increase_credit(_Meta, #consumer{cfg = #consumer_cfg{lifetime = once}, + credit = Credit}, _) -> + %% once consumers cannot increment credit + Credit; +increase_credit(_Meta, #consumer{cfg = #consumer_cfg{lifetime = auto, + credit_mode = credited}, + credit = Credit}, _) -> + %% credit_mode: `credited' also doesn't automatically increment credit + Credit; +increase_credit(#{machine_version := MachineVersion}, + #consumer{cfg = #consumer_cfg{credit_mode = {simple_prefetch, MaxCredit}}, + credit = Current}, Credit) + when MachineVersion >= 3 andalso MaxCredit > 0 -> + min(MaxCredit, Current + Credit); +increase_credit(_Meta, #consumer{credit = Current}, Credit) -> + Current + Credit. + +complete_and_checkout(#{index := IncomingRaftIdx} = Meta, MsgIds, ConsumerId, + #consumer{} = Con0, + Effects0, State0) -> + State1 = complete(Meta, ConsumerId, MsgIds, Con0, State0), + {State, ok, Effects} = checkout(Meta, State0, State1, Effects0), + update_smallest_raft_index(IncomingRaftIdx, State, Effects). + +cancel_consumer_effects(ConsumerId, + #?STATE{cfg = #cfg{resource = QName}} = _State, + Effects) -> + [{mod_call, rabbit_quorum_queue, + cancel_consumer_handler, [QName, ConsumerId]} | Effects]. + +update_smallest_raft_index(Idx, State, Effects) -> + update_smallest_raft_index(Idx, ok, State, Effects). + +update_smallest_raft_index(IncomingRaftIdx, Reply, + #?STATE{cfg = Cfg, + release_cursors = Cursors0} = State0, + Effects) -> + Total = messages_total(State0), + %% TODO: optimise + case smallest_raft_index(State0) of + undefined when Total == 0 -> + % there are no messages on queue anymore and no pending enqueues + % we can forward release_cursor all the way until + % the last received command, hooray + %% reset the release cursor interval + #cfg{release_cursor_interval = {Base, _}} = Cfg, + RCI = {Base, Base}, + State = State0#?STATE{cfg = Cfg#cfg{release_cursor_interval = RCI}, + release_cursors = lqueue:new(), + enqueue_count = 0}, + {State, Reply, Effects ++ [{release_cursor, IncomingRaftIdx, State}]}; + undefined -> + {State0, Reply, Effects}; + Smallest when is_integer(Smallest) -> + case find_next_cursor(Smallest, Cursors0) of + empty -> + {State0, Reply, Effects}; + {Cursor, Cursors} -> + %% we can emit a release cursor when we've passed the smallest + %% release cursor available. + {State0#?STATE{release_cursors = Cursors}, Reply, + Effects ++ [Cursor]} + end + end. + +find_next_cursor(Idx, Cursors) -> + find_next_cursor(Idx, Cursors, empty). + +find_next_cursor(Smallest, Cursors0, Potential) -> + case lqueue:out(Cursors0) of + {{value, {_, Idx, _} = Cursor}, Cursors} when Idx < Smallest -> + %% we found one but it may not be the largest one + find_next_cursor(Smallest, Cursors, Cursor); + _ when Potential == empty -> + empty; + _ -> + {Potential, Cursors0} + end. + +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), is_integer(Expiry) -> + update_header(Key, UpdateFun, Default, #{size => Size, + expiry => Expiry}); +update_header(Key, UpdateFun, Default, Header) + when is_map(Header), 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). + +return_one(#{machine_version := MachineVersion} = Meta, + MsgId, Msg0, + #?STATE{returns = Returns, + consumers = Consumers, + dlx = DlxState0, + cfg = #cfg{delivery_limit = DeliveryLimit, + dead_letter_handler = DLH}} = State0, + Effects0, ConsumerId) -> + #consumer{checked_out = Checked0} = Con0 = maps:get(ConsumerId, Consumers), + Msg = update_msg_header(delivery_count, fun incr/1, 1, Msg0), + Header = get_msg_header(Msg), + case get_header(delivery_count, Header) of + DeliveryCount when DeliveryCount > DeliveryLimit -> + {DlxState, DlxEffects} = rabbit_fifo_dlx:discard([Msg], delivery_limit, DLH, DlxState0), + State1 = State0#?STATE{dlx = DlxState}, + State = complete(Meta, ConsumerId, [MsgId], Con0, State1), + {State, DlxEffects ++ Effects0}; + _ -> + Checked = maps:remove(MsgId, Checked0), + Con = case MachineVersion of + V when V >= 3 -> + Con0#consumer{checked_out = Checked, + credit = increase_credit(Meta, Con0, 1)}; + 2 -> + Con0#consumer{checked_out = Checked} + end, + {add_bytes_return( + Header, + State0#?STATE{consumers = Consumers#{ConsumerId => Con}, + returns = lqueue:in(Msg, Returns)}), + Effects0} + end. + +return_all(Meta, #?STATE{consumers = Cons} = State0, Effects0, ConsumerId, + #consumer{checked_out = Checked} = Con) -> + State = State0#?STATE{consumers = Cons#{ConsumerId => Con}}, + lists:foldl(fun ({MsgId, Msg}, {S, E}) -> + return_one(Meta, MsgId, Msg, S, E, ConsumerId) + 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{cfg = #cfg{resource = _QName}} = 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? + State2 = State1#?STATE{msg_cache = undefined, %% by this time the cache should be used + dlx = DlxState}, + Effects2 = DlxDeliveryEffects ++ Effects1, + case evaluate_limit(Index, false, OldState, State2, Effects2) of + {State, false, Effects} when ExpiredMsg == false -> + {State, Reply, Effects}; + {State, _, Effects} -> + update_smallest_raft_index(Index, Reply, State, Effects) + end. + +checkout0(Meta, {success, ConsumerId, MsgId, + ?MSG(_RaftIdx, _Header) = Msg, ExpiredMsg, State, Effects}, + SendAcc0) -> + DelMsg = {MsgId, Msg}, + SendAcc = case maps:get(ConsumerId, SendAcc0, undefined) of + undefined -> + SendAcc0#{ConsumerId => [DelMsg]}; + LogMsgs -> + SendAcc0#{ConsumerId => [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, _BeforeState, + #?STATE{cfg = #cfg{max_length = undefined, + max_bytes = undefined}} = State, + Effects) -> + {State, 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} = + 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), + {State0#?STATE{enqueuers = Enqs}, Result, Effects}; + _ -> + {State0, Result, Effects0} + end; + false -> + {State0, Result, Effects0} + end. + + +%% [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 lqueue:out(Messages0) of + {empty, _} -> + empty; + {{value, ?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 -> + lqueue:get(Messages0, empty); + Msg -> + Msg + end. + +delivery_effect({CTag, CPid}, [{MsgId, ?MSG(Idx, Header)}], + #?STATE{msg_cache = {Idx, RawMsg}}) -> + {send_msg, CPid, {delivery, CTag, [{MsgId, {Header, RawMsg}}]}, + [local, ra_event]}; +delivery_effect({CTag, CPid}, Msgs, _State) -> + RaftIdxs = lists:foldr(fun ({_, ?MSG(I, _)}, Acc) -> + [I | Acc] + end, [], Msgs), + {log, RaftIdxs, + fun(Log) -> + DelMsgs = lists:zipwith( + fun (Cmd, {MsgId, ?MSG(_Idx, Header)}) -> + {MsgId, {Header, get_msg(Cmd)}} + end, Log, Msgs), + [{send_msg, CPid, {delivery, CTag, DelMsgs}, [local, ra_event]}] + 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, ConsumerId}, SQ1} + when is_map_key(ConsumerId, Cons0) -> + case take_next_msg(InitState) of + {ConsumerMsg, State0} -> + %% there are consumers waiting to be serviced + %% process consumer checkout + case maps:get(ConsumerId, Cons0) of + #consumer{credit = 0} -> + %% no credit but was still on queue + %% can happen when draining + %% recurse without consumer on queue + checkout_one(Meta, ExpiredMsg, + InitState#?STATE{service_queue = SQ1}, Effects1); + #consumer{status = cancelled} -> + checkout_one(Meta, ExpiredMsg, + InitState#?STATE{service_queue = SQ1}, Effects1); + #consumer{status = suspected_down} -> + checkout_one(Meta, ExpiredMsg, + InitState#?STATE{service_queue = SQ1}, Effects1); + #consumer{checked_out = Checked0, + next_msg_id = Next, + credit = Credit, + delivery_count = DelCnt} = Con0 -> + Checked = maps:put(Next, ConsumerMsg, Checked0), + Con = Con0#consumer{checked_out = Checked, + next_msg_id = Next + 1, + credit = Credit - 1, + delivery_count = DelCnt + 1}, + Size = get_header(size, get_msg_header(ConsumerMsg)), + State = update_or_remove_sub( + Meta, ConsumerId, Con, + State0#?STATE{service_queue = SQ1, + msg_bytes_checkout = BytesCheckout + Size, + msg_bytes_enqueue = BytesEnqueue - Size}), + {success, ConsumerId, Next, ConsumerMsg, ExpiredMsg, + State, Effects1} + end; + empty -> + {nochange, ExpiredMsg, InitState, Effects1} + end; + {{value, _ConsumerId}, SQ1} -> + %% consumer did not exist but was queued, recurse + checkout_one(Meta, ExpiredMsg, + InitState#?STATE{service_queue = SQ1}, Effects1); + {empty, _} -> + case lqueue: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), is_integer(Expiry) -> + %% Next message contains 'expiry' header. + %% (Re)set timer so that mesage 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_sub(Meta, ConsumerId, + #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(ConsumerId, Cons), + last_active = Ts}; + _ -> + % there are unsettled items so need to keep around + State#?STATE{consumers = maps:put(ConsumerId, Con, Cons)} + end; +update_or_remove_sub(_Meta, ConsumerId, + #consumer{cfg = #consumer_cfg{}} = Con, + #?STATE{consumers = Cons, + service_queue = ServiceQueue} = State) -> + State#?STATE{consumers = maps:put(ConsumerId, Con, Cons), + service_queue = uniq_queue_in(ConsumerId, Con, ServiceQueue)}. + +uniq_queue_in(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; +uniq_queue_in(_Key, _Consumer, ServiceQueue) -> + ServiceQueue. + +update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta, + {Life, Credit, Mode0} = Spec, Priority, + #?STATE{cfg = #cfg{consumer_strategy = competing}, + consumers = Cons0} = State0) -> + Consumer = case Cons0 of + #{ConsumerId := #consumer{} = Consumer0} -> + merge_consumer(Meta, Consumer0, ConsumerMeta, Spec, Priority); + _ -> + Mode = credit_mode(Meta, Credit, Mode0), + #consumer{cfg = #consumer_cfg{tag = Tag, + pid = Pid, + lifetime = Life, + meta = ConsumerMeta, + priority = Priority, + credit_mode = Mode}, + credit = Credit} + end, + {Consumer, update_or_remove_sub(Meta, ConsumerId, Consumer, State0)}; +update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta, + {Life, Credit, Mode0} = Spec, Priority, + #?STATE{cfg = #cfg{consumer_strategy = single_active}, + consumers = Cons0, + waiting_consumers = Waiting, + service_queue = _ServiceQueue0} = State0) -> + %% 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 + {ConsumerId, #consumer{status = up} = Consumer0} -> + Consumer = merge_consumer(Meta, Consumer0, ConsumerMeta, + Spec, Priority), + {Consumer, update_or_remove_sub(Meta, ConsumerId, Consumer, State0)}; + undefined when is_map_key(ConsumerId, 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(ConsumerId, Cons0), + Consumer = merge_consumer(Meta, Consumer0, ConsumerMeta, + Spec, Priority), + {Consumer, update_or_remove_sub(Meta, ConsumerId, Consumer, State0)}; + _ -> + %% add as a new waiting consumer + Mode = credit_mode(Meta, Credit, Mode0), + Consumer = #consumer{cfg = #consumer_cfg{tag = Tag, + pid = Pid, + lifetime = Life, + meta = ConsumerMeta, + priority = Priority, + credit_mode = Mode}, + credit = Credit}, + + {Consumer, + State0#?STATE{waiting_consumers = + Waiting ++ [{ConsumerId, Consumer}]}} + end. + +merge_consumer(Meta, #consumer{cfg = CCfg, checked_out = Checked} = Consumer, + ConsumerMeta, {Life, Credit, Mode0}, Priority) -> + NumChecked = map_size(Checked), + NewCredit = max(0, Credit - NumChecked), + Mode = credit_mode(Meta, Credit, Mode0), + Consumer#consumer{cfg = CCfg#consumer_cfg{priority = Priority, + meta = ConsumerMeta, + credit_mode = Mode, + lifetime = Life}, + status = up, + credit = NewCredit}. + +credit_mode(#{machine_version := Vsn}, Credit, simple_prefetch) + when Vsn >= 3 -> + {simple_prefetch, Credit}; +credit_mode(_, _, Mode) -> + Mode. + +maybe_queue_consumer(ConsumerId, #consumer{credit = Credit} = Con, + ServiceQueue0) -> + case Credit > 0 of + true -> + % consumer needs service - check if already on service queue + uniq_queue_in(ConsumerId, Con, ServiceQueue0); + false -> + ServiceQueue0 + end. + +%% creates a dehydrated version of the current state to be cached and +%% potentially used to for a snaphot at a later point +dehydrate_state(#?STATE{cfg = #cfg{}, + dlx = DlxState} = State) -> + % no messages are kept in memory, no need to + % overly mutate the current state apart from removing indexes and cursors + State#?STATE{ra_indexes = rabbit_fifo_index:empty(), + release_cursors = lqueue:new(), + enqueue_count = 0, + msg_cache = undefined, + dlx = rabbit_fifo_dlx:dehydrate(DlxState)}. + +%% make the state suitable for equality comparison +normalize(#?STATE{ra_indexes = _Indexes, + returns = Returns, + messages = Messages, + release_cursors = Cursors, + dlx = DlxState} = State) -> + State#?STATE{returns = lqueue:from_list(lqueue:to_list(Returns)), + messages = lqueue:from_list(lqueue:to_list(Messages)), + release_cursors = lqueue:from_list(lqueue:to_list(Cursors)), + dlx = rabbit_fifo_dlx:normalize(DlxState)}. + +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) -> + #enqueue{pid = Pid, seq = Seq, msg = Msg}. + +-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, Spec, Meta) -> + #checkout{consumer_id = ConsumerId, + spec = Spec, meta = Meta}. + +-spec make_settle(consumer_id(), [msg_id()]) -> protocol(). +make_settle(ConsumerId, MsgIds) when is_list(MsgIds) -> + #settle{consumer_id = ConsumerId, msg_ids = MsgIds}. + +-spec make_return(consumer_id(), [msg_id()]) -> protocol(). +make_return(ConsumerId, MsgIds) -> + #return{consumer_id = ConsumerId, msg_ids = MsgIds}. + +-spec make_discard(consumer_id(), [msg_id()]) -> protocol(). +make_discard(ConsumerId, MsgIds) -> + #discard{consumer_id = ConsumerId, msg_ids = MsgIds}. + +-spec make_credit(consumer_id(), non_neg_integer(), non_neg_integer(), + boolean()) -> protocol(). +make_credit(ConsumerId, Credit, DeliveryCount, Drain) -> + #credit{consumer_id = ConsumerId, + credit = Credit, + delivery_count = DeliveryCount, + drain = Drain}. + +-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(#basic_message{content = Content}) -> + #content{payload_fragments_rev = PFR} = Content, + iolist_size(PFR); +message_size(B) when is_binary(B) -> + byte_size(B); +message_size(Msg) -> + case mc:is(Msg) of + true -> + {_, PayloadSize} = mc:size(Msg), + PayloadSize; + false -> + %% probably only hit this for testing so ok to use erts_debug + erts_debug:size(Msg) + end. + + +all_nodes(#?STATE{consumers = Cons0, + enqueuers = Enqs0, + waiting_consumers = WaitingConsumers0}) -> + Nodes0 = maps:fold(fun({_, 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({{_, P}, _}, Acc) -> + Acc#{node(P) => ok} + end, Nodes1, WaitingConsumers0)). + +all_pids_for(Node, #?STATE{consumers = Cons0, + enqueuers = Enqs0, + waiting_consumers = WaitingConsumers0}) -> + Cons = maps:fold(fun({_, P}, _, Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, _, Acc) -> Acc + end, [], Cons0), + Enqs = maps:fold(fun(P, _, Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, _, Acc) -> Acc + end, Cons, Enqs0), + lists:foldl(fun({{_, P}, _}, Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, Acc) -> Acc + end, Enqs, WaitingConsumers0). + +suspected_pids_for(Node, #?STATE{consumers = Cons0, + enqueuers = Enqs0, + waiting_consumers = WaitingConsumers0}) -> + Cons = maps:fold(fun({_, P}, + #consumer{status = suspected_down}, + Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, _, Acc) -> Acc + end, [], Cons0), + Enqs = maps:fold(fun(P, #enqueuer{status = suspected_down}, Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, _, Acc) -> Acc + end, Cons, Enqs0), + lists:foldl(fun({{_, P}, + #consumer{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_from_args(#{args := Args}) -> + case rabbit_misc:table_lookup(Args, <<"x-priority">>) of + {_Key, Value} -> + Value; + _ -> 0 + end; +get_priority_from_args(_) -> + 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(To, To, State) -> + State; +convert(0, To, State) -> + convert(1, To, rabbit_fifo_v1:convert_v0_to_v1(State)); +convert(1, To, State) -> + convert(2, To, convert_v1_to_v2(State)); +convert(2, To, State) -> + convert(3, To, convert_v2_to_v3(State)). + +smallest_raft_index(#?STATE{messages = Messages, + ra_indexes = Indexes, + dlx = DlxState}) -> + SmallestDlxRaIdx = rabbit_fifo_dlx:smallest_raft_index(DlxState), + SmallestMsgsRaIdx = case lqueue:get(Messages, undefined) of + ?MSG(I, _) when is_integer(I) -> + I; + _ -> + undefined + end, + SmallestRaIdx = rabbit_fifo_index:smallest(Indexes), + lists:min([SmallestDlxRaIdx, SmallestMsgsRaIdx, SmallestRaIdx]). + +make_requeue(ConsumerId, Notify, [{MsgId, Idx, Header, Msg}], Acc) -> + lists:reverse([{append, + #requeue{consumer_id = ConsumerId, + index = Idx, + header = Header, + msg_id = MsgId, + msg = Msg}, + Notify} + | Acc]); +make_requeue(ConsumerId, Notify, [{MsgId, Idx, Header, Msg} | Rem], Acc) -> + make_requeue(ConsumerId, Notify, Rem, + [{append, + #requeue{consumer_id = ConsumerId, + 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(#enqueue{msg = M}) -> + M; +get_msg(#requeue{msg = M}) -> + M. diff --git a/deps/rabbit/src/rabbit_fifo_v3.hrl b/deps/rabbit/src/rabbit_fifo_v3.hrl new file mode 100644 index 000000000000..9b1078265dc6 --- /dev/null +++ b/deps/rabbit/src/rabbit_fifo_v3.hrl @@ -0,0 +1,226 @@ +%% 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-2024 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(IS_HEADER(H), + (is_integer(H) andalso H >= 0) orelse + is_list(H) orelse + (is_map(H) andalso is_map_key(size, H))). + +-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(), + 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. +%% 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(option(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 consumer_tag() :: binary(). +%% An arbitrary binary tag used to distinguish between different consumers +%% set up by the same process. See: {@link rabbit_fifo_client:checkout/3.} + +-type delivery() :: {delivery, consumer_tag(), [delivery_msg()]}. +%% Represents the delivery of one or more rabbit_fifo messages. + +-type consumer_id() :: {consumer_tag(), pid()}. +%% The entity that receives messages. Uniquely identifies a consumer. + +-type credit_mode() :: credited | + %% machine_version 2 + simple_prefetch | + %% machine_version 3 + {simple_prefetch, MaxCredit :: non_neg_integer()}. +%% determines how credit is replenished + +-type checkout_spec() :: {once | auto, Num :: non_neg_integer(), + credit_mode()} | + {dequeue, settled | unsettled} | + cancel. + +-type consumer_meta() :: #{ack => boolean(), + username => binary(), + prefetch => non_neg_integer(), + args => list()}. +%% static meta data associated with a consumer + +-type applied_mfa() :: {module(), atom(), list()}. +% represents a partially applied module call + +-define(RELEASE_CURSOR_EVERY, 2048). +-define(RELEASE_CURSOR_EVERY_MAX, 3_200_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). + +-record(consumer_cfg, + {meta = #{} :: consumer_meta(), + pid :: pid(), + tag :: consumer_tag(), + %% 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: `{consumer_credit, ReceiverDeliveryCount, Credit}' + credit_mode :: credit_mode(), % part of snapshot data + lifetime = once :: once | auto, + priority = 0 :: non_neg_integer()}). + +-record(consumer, + {cfg = #consumer_cfg{}, + status = up :: up | suspected_down | cancelled | waiting, + next_msg_id = 0 :: msg_id(), % part of snapshot data + checked_out = #{} :: #{msg_id() => msg()}, + %% max number of messages that can be sent + %% decremented for each delivery + credit = 0 : non_neg_integer(), + %% total number of checked out messages - ever + %% incremented for each delivery + delivery_count = 0 :: non_neg_integer() + }). + +-type consumer() :: #consumer{}. + +-type consumer_strategy() :: competing | single_active. + +-type milliseconds() :: non_neg_integer(). + +-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, + 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, + unused_2 + }). + +-record(cfg, + {name :: atom(), + resource :: rabbit_types:r('queue'), + release_cursor_interval :: option({non_neg_integer(), non_neg_integer()}), + 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_1, + unused_2 + }). + +-type prefix_msgs() :: {list(), list()} | + {non_neg_integer(), list(), + non_neg_integer(), list()}. + +-record(rabbit_fifo, + {cfg :: #cfg{}, + % unassigned messages + messages = lqueue:new() :: lqueue:lqueue(msg()), + 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(), + %% A release cursor is essentially a snapshot for a past raft index. + %% Working assumption: Messages are consumed in a FIFO-ish order because + %% the log is truncated only until the oldest message. + release_cursors = lqueue:new() :: lqueue:lqueue({release_cursor, + ra:index(), #rabbit_fifo{}}), + % consumers need to reflect consumer state at time of snapshot + consumers = #{} :: #{consumer_id() => 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(), + %% waiting consumers, one is picked active consumer is cancelled or dies + %% used only when single active consumer is on + waiting_consumers = [] :: [{consumer_id(), consumer()}], + last_active :: option(non_neg_integer()), + msg_cache :: option({ra:index(), raw_msg()}), + unused_2 + }). + +-type config() :: #{name := atom(), + queue_resource := rabbit_types:r('queue'), + dead_letter_handler => dead_letter_handler(), + become_leader_handler => applied_mfa(), + release_cursor_interval => non_neg_integer(), + max_length => non_neg_integer(), + max_bytes => non_neg_integer(), + max_in_memory_length => non_neg_integer(), + max_in_memory_bytes => non_neg_integer(), + overflow_strategy => drop_head | reject_publish, + single_active_consumer_on => boolean(), + delivery_limit => non_neg_integer(), + 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 eb92b3670e9a..8d455287dea7 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -761,6 +761,9 @@ delete(Q, _IfUnused, _IfEmpty, ActingUser) when ?amqqueue_is_quorum(Q) -> MRef = erlang:monitor(process, Leader), receive {'DOWN', MRef, process, _, _} -> + %% leader is down, + %% force delete remaining members + ok = force_delete_queue(lists:delete(Leader, Servers)), ok after Timeout -> erlang:demonitor(MRef, [flush]), diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl index 40d7c560c9f6..39c462939b0c 100644 --- a/deps/rabbit/test/amqp_client_SUITE.erl +++ b/deps/rabbit/test/amqp_client_SUITE.erl @@ -2963,7 +2963,14 @@ quorum_queue_on_old_node(Config) -> queue_and_client_different_nodes(1, 0, <<"quorum">>, Config). quorum_queue_on_new_node(Config) -> - queue_and_client_different_nodes(0, 1, <<"quorum">>, Config). + Versions = rabbit_ct_broker_helpers:rpc_all(Config, rabbit_fifo, version, []), + case lists:usort(Versions) of + [_] -> + %% all are one version, go ahead with the test + queue_and_client_different_nodes(0, 1, <<"quorum">>, Config); + _ -> + {skip, "this test cannot pass with mixed QQ machine versions"} + end. %% In mixed version tests, run the queue leader with old code %% and queue client with new code, or vice versa. diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 80f6093129eb..95dc00a226a0 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -36,13 +36,18 @@ groups() -> [ {machine_version_2, [shuffle], all_tests()}, {machine_version_3, [shuffle], all_tests()}, - {machine_version_conversion, [shuffle], [convert_v2_to_v3]} + {machine_version_4, [shuffle], all_tests()}, + {machine_version_conversion, [shuffle], + [convert_v2_to_v3, + convert_v3_to_v4]} ]. init_per_group(machine_version_2, Config) -> [{machine_version, 2} | Config]; init_per_group(machine_version_3, Config) -> [{machine_version, 3} | Config]; +init_per_group(machine_version_4, Config) -> + [{machine_version, 4} | Config]; init_per_group(machine_version_conversion, Config) -> Config. @@ -91,7 +96,6 @@ enq_enq_checkout_test(C) -> apply(meta(C, 3), rabbit_fifo:make_checkout(Cid, {once, 2, simple_prefetch}, #{}), State2), - ct:pal("~tp", [Effects]), ?ASSERT_EFF({monitor, _, _}, Effects), ?ASSERT_EFF({log, [1,2], _Fun, _Local}, Effects), ok. @@ -101,7 +105,7 @@ credit_enq_enq_checkout_settled_credit_v1_test(C) -> {State1, _} = enq(C, 1, 1, first, test_init(test)), {State2, _} = enq(C, 2, 2, second, State1), {State3, _, Effects} = - apply(meta(C, 3), rabbit_fifo:make_checkout(Cid, {auto, 1, credited}, #{}), State2), + apply(meta(C, 3), make_checkout(Cid, {auto, 1, credited}, #{}), State2), ?ASSERT_EFF({monitor, _, _}, Effects), ?ASSERT_EFF({log, [1], _Fun, _Local}, Effects), %% settle the delivery this should _not_ result in further messages being @@ -1687,14 +1691,17 @@ meta(Config, Idx, Timestamp) -> enq(Config, Idx, MsgSeq, Msg, State) -> strip_reply( - rabbit_fifo:apply(meta(Config, Idx), rabbit_fifo:make_enqueue(self(), MsgSeq, Msg), State)). + rabbit_fifo:apply(meta(Config, Idx), + rabbit_fifo:make_enqueue(self(), MsgSeq, Msg), State)). deq(Config, Idx, Cid, Settlement, Msg, State0) -> {State, _, Effs} = apply(meta(Config, Idx), rabbit_fifo:make_checkout(Cid, {dequeue, Settlement}, #{}), State0), - {value, {log, [_Idx], Fun}} = lists:search(fun(E) -> element(1, E) == log end, Effs), + {value, {log, [_Idx], Fun}} = lists:search(fun(E) -> + element(1, E) == log + end, Effs), [{reply, _From, {wrap_reply, {dequeue, {MsgId, _}, _}}}] = Fun([Msg]), @@ -1832,9 +1839,9 @@ convert_v2_to_v3(Config) -> Cid1 = {ctag1, self()}, Cid2 = {ctag2, self()}, MaxCredits = 20, - Entries = [{1, rabbit_fifo:make_checkout(Cid1, {auto, 10, credited}, #{})}, - {2, rabbit_fifo:make_checkout(Cid2, {auto, MaxCredits, simple_prefetch}, - #{prefetch => MaxCredits})}], + Entries = [{1, make_checkout(Cid1, {auto, 10, credited}, #{})}, + {2, make_checkout(Cid2, {auto, MaxCredits, simple_prefetch}, + #{prefetch => MaxCredits})}], %% run log in v2 {State, _} = run_log(ConfigV2, test_init(?FUNCTION_NAME), Entries), @@ -1848,6 +1855,36 @@ convert_v2_to_v3(Config) -> maps:get(Cid2, Consumers)), ok. +convert_v3_to_v4(Config) -> + ConfigV3 = [{machine_version, 3} | Config], + ConfigV4 = [{machine_version, 4} | Config], + + Cid1 = {ctag1, self()}, + Cid2 = {ctag2, self()}, + MaxCredits = 20, + Entries = [{1, make_checkout(Cid1, {auto, 10, credited}, #{})}, + {2, make_checkout(Cid2, {auto, MaxCredits, simple_prefetch}, + #{prefetch => MaxCredits})}], + + %% run log in v3 + Name = ?FUNCTION_NAME, + Init = rabbit_fifo_v3:init( + #{name => Name, + max_in_memory_length => 0, + queue_resource => rabbit_misc:r("/", queue, atom_to_binary(Name)), + release_cursor_interval => 0}), + {State, _} = run_log(ConfigV3, Init, Entries), + + %% convert from v3 to v4 + {#rabbit_fifo{consumers = Consumers}, ok, _} = + apply(meta(ConfigV4, 4), {machine_version, 3, 4}, State), + + ?assertEqual(2, maps:size(Consumers)), + ?assertMatch(#consumer{cfg = #consumer_cfg{credit_mode = + {simple_prefetch, MaxCredits}}}, + maps:get(Cid2, Consumers)), + ok. + queue_ttl_test(C) -> QName = rabbit_misc:r(<<"/">>, queue, <<"test">>), Conf = #{name => ?FUNCTION_NAME, @@ -1867,7 +1904,7 @@ queue_ttl_test(C) -> [{aux, {handle_tick, [_, _, _]}}] = rabbit_fifo:tick(Now + 1000, S1), %% cancelling the consumer should then {S2, _, _} = apply(meta(C, 2, Now), - rabbit_fifo:make_checkout(Cid, cancel, #{}), S1), + make_checkout(Cid, cancel, #{}), S1), %% last_active should have been reset when consumer was cancelled %% last_active = 2500 [{aux, {handle_tick, [_, _, _]}}] = rabbit_fifo:tick(Now + 1000, S2), @@ -1888,7 +1925,7 @@ queue_ttl_test(C) -> %% dequeue should set last applied {S1Deq, {dequeue, empty}, _} = apply(meta(C, 2, Now), - rabbit_fifo:make_checkout(Cid, {dequeue, unsettled}, #{}), + make_checkout(Cid, {dequeue, unsettled}, #{}), S0), [{aux, {handle_tick, [_, _, _]}}] = rabbit_fifo:tick(Now + 1000, S1Deq), @@ -1901,7 +1938,7 @@ queue_ttl_test(C) -> Deq = {<<"deq1">>, self()}, {E2, _, Effs2} = apply(meta(C, 3, Now), - rabbit_fifo:make_checkout(Deq, {dequeue, unsettled}, #{}), + make_checkout(Deq, {dequeue, unsettled}, #{}), E1), {log, [2], Fun2} = get_log_eff(Effs2), @@ -1935,7 +1972,7 @@ queue_ttl_with_single_active_consumer_test(C) -> [{aux, {handle_tick, [_, _, _]}}] = rabbit_fifo:tick(Now + 1000, S1), %% cancelling the consumer should then {S2, _, _} = apply(meta(C, 2, Now), - rabbit_fifo:make_checkout(Cid, cancel, #{}), S1), + make_checkout(Cid, cancel, #{}), S1), %% last_active should have been reset when consumer was cancelled %% last_active = 2500 [{aux, {handle_tick, [_, _, _]}}] = rabbit_fifo:tick(Now + 1000, S2), @@ -1972,13 +2009,12 @@ checkout_priority_test(C) -> Args = [{<<"x-priority">>, long, 1}], {S1, _, _} = apply(meta(C, 3), - rabbit_fifo:make_checkout(Cid, {once, 2, simple_prefetch}, - #{args => Args}), + make_checkout(Cid, {once, 2, simple_prefetch}, + #{args => Args}), test_init(test)), {S2, _, _} = apply(meta(C, 3), - rabbit_fifo:make_checkout(Cid2, {once, 2, simple_prefetch}, - #{args => []}), + make_checkout(Cid2, {once, 2, simple_prefetch}, #{args => []}), S1), {S3, E3} = enq(C, 1, 1, first, S2), ct:pal("E3 ~tp ~tp", [E3, self()]), @@ -1994,7 +2030,7 @@ empty_dequeue_should_emit_release_cursor_test(C) -> Cid = {<<"basic.get1">>, self()}, {_State, {dequeue, empty}, Effects} = apply(meta(C, 2, 1234), - rabbit_fifo:make_checkout(Cid, {dequeue, unsettled}, #{}), + make_checkout(Cid, {dequeue, unsettled}, #{}), State0), ?ASSERT_EFF({release_cursor, _, _}, Effects), @@ -2089,14 +2125,14 @@ checkout_metadata_test(Config) -> {State1, {ok, #{next_msg_id := 0, num_checked_out := 0}}, _} = apply(meta(Config, ?LINE), - rabbit_fifo:make_checkout(Cid, {auto, 1, simple_prefetch}, #{}), + make_checkout(Cid, {auto, 1, simple_prefetch}, #{}), State0), {State2, _, _} = apply(meta(Config, ?LINE), - rabbit_fifo:make_checkout(Cid, cancel, #{}), State1), + make_checkout(Cid, cancel, #{}), State1), {_State3, {ok, #{next_msg_id := 1, num_checked_out := 1}}, _} = apply(meta(Config, ?LINE), - rabbit_fifo:make_checkout(Cid, {auto, 1, simple_prefetch}, #{}), + make_checkout(Cid, {auto, 1, simple_prefetch}, #{}), State2), ok. diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index c151c1cd0214..5372e0ce9280 100644 --- a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl @@ -62,7 +62,7 @@ all_tests() -> scenario31, scenario32, upgrade, - upgrade_snapshots, + upgrade_snapshots_v1_v2, upgrade_snapshots_scenario1, upgrade_snapshots_scenario2, upgrade_snapshots_v2_to_v3, @@ -698,17 +698,17 @@ upgrade_snapshots_scenario1(_Config) -> Commands = [make_enqueue(E,1,msg(<<"msg1">>)), make_enqueue(E,2,msg(<<"msg2">>)), make_enqueue(E,3,msg(<<"msg3">>))], - run_upgrade_snapshot_test(#{name => ?FUNCTION_NAME, - delivery_limit => 100, - max_length => 1, - max_bytes => 100, - max_in_memory_length => undefined, - max_in_memory_bytes => undefined, - overflow_strategy => drop_head, - single_active_consumer_on => false, - dead_letter_handler => {?MODULE, banana, []} - }, - Commands), + run_upgrade_snapshot_test_v1_v2(#{name => ?FUNCTION_NAME, + delivery_limit => 100, + max_length => 1, + max_bytes => 100, + max_in_memory_length => undefined, + max_in_memory_bytes => undefined, + overflow_strategy => drop_head, + single_active_consumer_on => false, + dead_letter_handler => {?MODULE, banana, []} + }, + Commands), ok. upgrade_snapshots_scenario2(_Config) -> @@ -719,17 +719,17 @@ upgrade_snapshots_scenario2(_Config) -> make_enqueue(E,1,msg(<<"msg1">>)), make_enqueue(E,2,msg(<<"msg2">>)), rabbit_fifo:make_settle(C, [0])], - run_upgrade_snapshot_test(#{name => ?FUNCTION_NAME, - delivery_limit => undefined, - max_length => undefined, - max_bytes => undefined, - max_in_memory_length => undefined, - max_in_memory_bytes => undefined, - overflow_strategy => drop_head, - single_active_consumer_on => false, - dead_letter_handler => {?MODULE, banana, []} - }, - Commands), + run_upgrade_snapshot_test_v1_v2(#{name => ?FUNCTION_NAME, + delivery_limit => undefined, + max_length => undefined, + max_bytes => undefined, + max_in_memory_length => undefined, + max_in_memory_bytes => undefined, + overflow_strategy => drop_head, + single_active_consumer_on => false, + dead_letter_handler => {?MODULE, banana, []} + }, + Commands), ok. single_active_01(_Config) -> @@ -951,7 +951,7 @@ upgrade(_Config) -> end) end, [], Size). -upgrade_snapshots(_Config) -> +upgrade_snapshots_v1_v2(_Config) -> Size = 256, run_proper( fun () -> @@ -974,9 +974,10 @@ upgrade_snapshots(_Config) -> drop_head, {?MODULE, banana, []} ), - ?FORALL(O, ?LET(Ops, log_gen_upgrade_snapshots(Size), expand(Ops, Config)), + ?FORALL(O, ?LET(Ops, log_gen_upgrade_snapshots(Size), + expand(Ops, Config)), collect({log_size, length(O)}, - upgrade_snapshots_prop(Config, O))) + upgrade_snapshots_prop_v1_v2(Config, O))) end) end, [], Size). @@ -1729,8 +1730,8 @@ snapshots_prop(Conf, Commands) -> false end. -upgrade_snapshots_prop(Conf, Commands) -> - try run_upgrade_snapshot_test(Conf, Commands) of +upgrade_snapshots_prop_v1_v2(Conf, Commands) -> + try run_upgrade_snapshot_test_v1_v2(Conf, Commands) of _ -> true catch Err -> @@ -1812,12 +1813,15 @@ log_gen_upgrade_snapshots(Size) -> {2, requeue} ])}}, {2, checkout_gen(oneof(CPids))}, - %% v2 fixes a bug that exists in v1 where a cancelled consumer is revived. - %% Therefore, there is an expected behavioural difference between v1 and v2 + %% v2 fixes a bug that exists in v1 where a cancelled + %% consumer is revived. + %% Therefore, there is an expected behavioural + %% difference between v1 and v2 %% and below line must be commented out. % {1, checkout_cancel_gen(oneof(CPids))}, - %% Likewise there is a behavioural difference between v1 and v2 - %% when 'up' is followed by 'down' where v2 behaves correctly. + %% Likewise there is a behavioural difference between + %% v1 and v2 when 'up' is followed by 'down' where + %% v2 behaves correctly. %% Therefore, below line must be commented out. % {1, down_gen(oneof(EPids ++ CPids))}, {1, nodeup_gen(Nodes)}, @@ -2228,7 +2232,7 @@ run_snapshot_test0(Conf0, Commands, Invariant) -> end || {release_cursor, SnapIdx, SnapState} <- Cursors], ok. -run_upgrade_snapshot_test(Conf, Commands) -> +run_upgrade_snapshot_test_v1_v2(Conf, Commands) -> ct:pal("running test with ~b commands using config ~tp", [length(Commands), Conf]), Indexes = lists:seq(1, length(Commands)), @@ -2249,9 +2253,9 @@ run_upgrade_snapshot_test(Conf, Commands) -> %% Recover in V1. {StateV1, _} = run_log(SnapState, FilteredV1, Invariant, rabbit_fifo_v1), %% Perform conversion and recover in V2. - Res = rabbit_fifo:apply(meta(SnapIdx + 1), {machine_version, 1, 2}, SnapState), + Res = rabbit_fifo_v3:apply(meta(SnapIdx + 1), {machine_version, 1, 2}, SnapState), #rabbit_fifo{} = V2 = element(1, Res), - {StateV2, _} = run_log(V2, FilteredV2, Invariant, rabbit_fifo, 2), + {StateV2, _} = run_log(V2, FilteredV2, Invariant, rabbit_fifo_v3, 2), %% Invariant: Recovering a V1 snapshot in V1 or V2 should end up in the same %% number of messages. Fields = [num_messages, @@ -2262,7 +2266,7 @@ run_upgrade_snapshot_test(Conf, Commands) -> checkout_message_bytes ], V1Overview = maps:with(Fields, rabbit_fifo_v1:overview(StateV1)), - V2Overview = maps:with(Fields, rabbit_fifo:overview(StateV2)), + V2Overview = maps:with(Fields, rabbit_fifo_v3:overview(StateV2)), case V1Overview == V2Overview of true -> ok; false -> @@ -2281,7 +2285,8 @@ run_upgrade_snapshot_test_v2_to_v3(Conf, Commands) -> Entries = lists:zip(Indexes, Commands), Invariant = fun(_) -> true end, %% Run the whole command log in v2 to emit release cursors. - {_, Effects} = run_log(test_init(Conf), Entries, Invariant, rabbit_fifo, 2), + {_, Effects} = run_log(test_init(rabbit_fifo_v3, Conf), Entries, Invariant, + rabbit_fifo, 2), Cursors = [ C || {release_cursor, _, _} = C <- Effects], [begin %% Drop all entries below and including the snapshot. @@ -2361,16 +2366,16 @@ run_log(InitState, Entries, InvariantFun, FifoMod, MachineVersion) -> end, {InitState, []}, Entries). test_init(Conf) -> + test_init(rabbit_fifo, Conf). + +test_init(Mod, Conf) -> Default = #{queue_resource => blah, release_cursor_interval => 0, metrics_handler => {?MODULE, metrics_handler, []}}, - rabbit_fifo:init(maps:merge(Default, Conf)). + Mod:init(maps:merge(Default, Conf)). test_init_v1(Conf) -> - Default = #{queue_resource => blah, - release_cursor_interval => 0, - metrics_handler => {?MODULE, metrics_handler, []}}, - rabbit_fifo_v1:init(maps:merge(Default, Conf)). + test_init(rabbit_fifo_v1, Conf). meta(Idx) -> meta(Idx, 3). diff --git a/deps/rabbitmq_mqtt/test/shared_SUITE.erl b/deps/rabbitmq_mqtt/test/shared_SUITE.erl index 656948e0763d..a401b664df6a 100644 --- a/deps/rabbitmq_mqtt/test/shared_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/shared_SUITE.erl @@ -829,7 +829,9 @@ delete_create_queue(Config) -> timer:sleep(2), delete_queue(Ch, [CQ1, QQ]), %% Give queues some time to be fully deleted - timer:sleep(2000), + %% TODO: wait longer for quorum queues in mixed mode as it can take longer + %% for deletion to complete, delete timeout is 5s so we need to exceed that + timer:sleep(6000), %% We expect confirms for all messages. %% Confirm here does not mean that messages made it ever to the deleted queues. diff --git a/moduleindex.yaml b/moduleindex.yaml index 687500c4096e..c2a161b9af1c 100755 --- a/moduleindex.yaml +++ b/moduleindex.yaml @@ -646,6 +646,7 @@ rabbit: - rabbit_fifo_index - rabbit_fifo_v0 - rabbit_fifo_v1 +- rabbit_fifo_v3 - rabbit_file - rabbit_global_counters - rabbit_guid From 3cd3a0ccc82e88d410590f5fe989fdcb332042da Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 11 Mar 2024 12:53:16 +0000 Subject: [PATCH 02/45] QQ: add v4 ff and new more compact enqueue command. Also update rabbit_fifo_* suites to test more relevant code versions where applicable. add ff mock QQ: always use the updated credit mode format QQv4: use more compact consumer reference in settle, credit, return This introudces a new type: consumer_key() which is either the consumer_id or the raft index the checkout was processed at. If the consumer is using one of the updated credit spec formats rabbit_fifo will use the raft index as the primary key for the consumer such that the rabbit fifo client can then use the more space efficient integer index instead of the full consumer id in subsequent commands. There is compatibility code to still accept the consumer id in settle, return, discard and credit commands but this is slighlyt slower and of course less space efficient. The old form will be used in cases where the fifo client may have already remove the local consumer state (as happens after a cancel). Lots of test refactorings of the rabbit_fifo_SUITE to begin to use the new forms. --- deps/rabbit/src/rabbit_core_ff.erl | 7 + deps/rabbit/src/rabbit_fifo.erl | 612 +++++++++++++------- deps/rabbit/src/rabbit_fifo.hrl | 38 +- deps/rabbit/src/rabbit_fifo_client.erl | 142 +++-- deps/rabbit/src/rabbit_fifo_v3.erl | 20 +- deps/rabbit/src/rabbit_queue_type.erl | 3 +- deps/rabbit/src/rabbit_quorum_queue.erl | 23 +- deps/rabbit/test/quorum_queue_SUITE.erl | 4 + deps/rabbit/test/rabbit_fifo_SUITE.erl | 592 ++++++++++--------- deps/rabbit/test/rabbit_fifo_int_SUITE.erl | 38 +- deps/rabbit/test/rabbit_fifo_prop_SUITE.erl | 92 +-- 11 files changed, 939 insertions(+), 632 deletions(-) diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl index 67270f4c1c30..ea3e4f0112d2 100644 --- a/deps/rabbit/src/rabbit_core_ff.erl +++ b/deps/rabbit/src/rabbit_core_ff.erl @@ -192,3 +192,10 @@ doc_url => "https://www.rabbitmq.com/docs/clustering#replica-placement", stability => stable }}). + +-rabbit_feature_flag( + {quorum_queues_v4, + #{desc => "Unlocks QQ v4 goodies", + stability => stable, + depends_on => [quorum_queue] + }}). diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 31cfe0c5c659..f96a02f304e7 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -18,6 +18,11 @@ -define(STATE, ?MODULE). +-define(CONSUMER_PID(Pid), #consumer{cfg = #consumer_cfg{pid = Pid}}). +-define(CONSUMER_TAG_PID(Tag, Pid), + #consumer{cfg = #consumer_cfg{tag = Tag, + pid = Pid}}). + -export([ %% ra_machine callbacks init/1, @@ -49,6 +54,7 @@ query_peek/2, query_notify_decorators_info/1, usage/1, + is_v4/0, %% misc dehydrate_state/1, @@ -77,12 +83,15 @@ -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(requeue, {consumer_id :: consumer_id(), +-record(?ENQ_V2, {seq :: option(msg_seqno()), + msg :: raw_msg()}). +-record(requeue, {consumer_key :: consumer_key(), msg_id :: msg_id(), index :: ra:index(), header :: msg_header(), @@ -91,13 +100,13 @@ -record(checkout, {consumer_id :: consumer_id(), spec :: checkout_spec(), meta :: consumer_meta()}). --record(settle, {consumer_id :: consumer_id(), +-record(settle, {consumer_key :: consumer_key(), msg_ids :: [msg_id()]}). --record(return, {consumer_id :: consumer_id(), +-record(return, {consumer_key :: consumer_key(), msg_ids :: [msg_id()]}). --record(discard, {consumer_id :: consumer_id(), +-record(discard, {consumer_key :: consumer_key(), msg_ids :: [msg_id()]}). --record(credit, {consumer_id :: consumer_id(), +-record(credit, {consumer_key :: consumer_key(), credit :: non_neg_integer(), delivery_count :: rabbit_queue_type:delivery_count(), drain :: boolean()}). @@ -108,6 +117,7 @@ -opaque protocol() :: #enqueue{} | + #?ENQ_V2{} | #requeue{} | #register_enqueuer{} | #checkout{} | @@ -192,6 +202,9 @@ update_config(Conf, State) -> apply(Meta, #enqueue{pid = From, seq = Seq, msg = RawMsg}, State00) -> apply_enqueue(Meta, From, Seq, RawMsg, State00); +apply(#{reply_mode := {notify, _Corr, EnqPid}} = Meta, + #?ENQ_V2{seq = Seq, msg = RawMsg}, State00) -> + apply_enqueue(Meta, EnqPid, Seq, RawMsg, State00); apply(_Meta, #register_enqueuer{pid = Pid}, #?STATE{enqueuers = Enqueuers0, cfg = #cfg{overflow_strategy = Overflow}} = State0) -> @@ -209,21 +222,25 @@ apply(_Meta, #register_enqueuer{pid = Pid}, ok end, {State, Res, [{monitor, process, Pid}]}; -apply(Meta, #settle{msg_ids = MsgIds, consumer_id = ConsumerId}, - #?STATE{consumers = Cons0} = State) -> - case Cons0 of - #{ConsumerId := Con0} -> - complete_and_checkout(Meta, MsgIds, ConsumerId, +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(Meta, #discard{msg_ids = MsgIds, consumer_id = ConsumerId}, - #?STATE{consumers = Cons, +apply(Meta, #discard{consumer_key = ConsumerKey, + msg_ids = MsgIds}, + #?STATE{consumers = Consumers, dlx = DlxState0, cfg = #cfg{dead_letter_handler = DLH}} = State0) -> - case Cons of - #{ConsumerId := #consumer{checked_out = Checked} = Con} -> + case find_consumer(ConsumerKey, Consumers) of + {ConsumerKey, #consumer{checked_out = Checked} = Con} -> % Publishing to dead-letter exchange must maintain same order as % messages got rejected. DiscardMsgs = lists:filtermap( @@ -238,21 +255,22 @@ apply(Meta, #discard{msg_ids = MsgIds, consumer_id = ConsumerId}, {DlxState, Effects} = rabbit_fifo_dlx:discard(DiscardMsgs, rejected, DLH, DlxState0), State = State0#?STATE{dlx = DlxState}, - complete_and_checkout(Meta, MsgIds, ConsumerId, Con, Effects, State); + complete_and_checkout(Meta, MsgIds, ConsumerKey, Con, Effects, State); _ -> {State0, ok} end; -apply(Meta, #return{msg_ids = MsgIds, consumer_id = ConsumerId}, +apply(Meta, #return{consumer_key = ConsumerKey, + msg_ids = MsgIds}, #?STATE{consumers = Cons0} = State) -> - case Cons0 of - #{ConsumerId := #consumer{checked_out = Checked0}} -> + case find_consumer(ConsumerKey, Cons0) of + {ActualConsumerKey, #consumer{checked_out = Checked0}} -> Returned = maps:with(MsgIds, Checked0), - return(Meta, ConsumerId, Returned, [], State); + return(Meta, ActualConsumerKey, Returned, [], State); _ -> {State, ok} end; apply(#{index := Idx} = Meta, - #requeue{consumer_id = ConsumerId, + #requeue{consumer_key = ConsumerKey, msg_id = MsgId, index = OldIdx, header = Header0, @@ -262,19 +280,19 @@ apply(#{index := Idx} = Meta, ra_indexes = Indexes0, enqueue_count = EnqCount} = State00) -> case Cons0 of - #{ConsumerId := #consumer{checked_out = Checked0} = Con0} + #{ConsumerKey := #consumer{checked_out = Checked0} = Con0} when is_map_key(MsgId, Checked0) -> %% construct a message with the current raft index %% and update delivery count before adding it to the message queue Header = update_header(delivery_count, fun incr/1, 1, Header0), State0 = add_bytes_return(Header, State00), Con = Con0#consumer{checked_out = maps:remove(MsgId, Checked0), - credit = increase_credit(Meta, Con0, 1)}, + credit = increase_credit(Con0, 1)}, State1 = State0#?STATE{ra_indexes = rabbit_fifo_index:delete(OldIdx, Indexes0), messages = lqueue:in(?MSG(Idx, Header), Messages), enqueue_count = EnqCount + 1}, - State2 = update_or_remove_sub(Meta, ConsumerId, Con, State1), + State2 = update_or_remove_sub(Meta, ConsumerKey, Con, State1), {State, Ret, Effs} = checkout(Meta, State0, State2, []), update_smallest_raft_index(Idx, Ret, maybe_store_release_cursor(Idx, State), @@ -282,44 +300,48 @@ apply(#{index := Idx} = Meta, _ -> {State00, ok, []} end; -apply(Meta, #credit{credit = LinkCreditRcv, delivery_count = DeliveryCountRcv, - drain = Drain, consumer_id = ConsumerId = {CTag, CPid}}, +apply(Meta, #credit{credit = LinkCreditRcv, + delivery_count = DeliveryCountRcv, + drain = Drain, + consumer_key = ConsumerKey}, #?STATE{consumers = Cons0, service_queue = ServiceQueue0, waiting_consumers = Waiting0} = State0) -> case Cons0 of - #{ConsumerId := #consumer{delivery_count = DeliveryCountSnd, - cfg = Cfg} = Con0} -> + #{ConsumerKey := #consumer{delivery_count = DeliveryCountSnd, + cfg = Cfg} = Con0} -> LinkCreditSnd = link_credit_snd(DeliveryCountRcv, LinkCreditRcv, DeliveryCountSnd, Cfg), %% grant the credit Con1 = Con0#consumer{credit = LinkCreditSnd}, - ServiceQueue = maybe_queue_consumer(ConsumerId, Con1, ServiceQueue0), + ServiceQueue = maybe_queue_consumer(ConsumerKey, Con1, ServiceQueue0), State1 = State0#?STATE{service_queue = ServiceQueue, - consumers = maps:update(ConsumerId, Con1, Cons0)}, + consumers = maps:update(ConsumerKey, Con1, Cons0)}, {State2, ok, Effects} = checkout(Meta, State0, State1, []), - #?STATE{consumers = Cons1 = #{ConsumerId := Con2}} = State2, - #consumer{credit = PostCred, + #?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(ConsumerId, Con, Cons1), - State3 = State2#?STATE{consumers = Cons}, - {ZeroCredit, AdvancedDeliveryCount, State3}; - false -> - {PostCred, PostDeliveryCount, State2} - end, - %% We must send to queue client delivery effects before - %% credit_reply such that session process can send to + 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, @@ -342,7 +364,7 @@ apply(Meta, #credit{credit = LinkCreditRcv, delivery_count = DeliveryCountRcv, AdvancedDeliveryCount = PostDeliveryCount + PostCred, Con = Con2#consumer{delivery_count = AdvancedDeliveryCount, credit = 0}, - Cons = maps:update(ConsumerId, Con, Cons1), + Cons = maps:update(ConsumerKey, Con, Cons1), State = State2#?STATE{consumers = Cons}, Reply = {multi, [CreditReply, {send_drained, {CTag, PostCred}}]}, @@ -360,7 +382,7 @@ apply(Meta, #credit{credit = LinkCreditRcv, delivery_count = DeliveryCountRcv, %% consuming all link-credit since there %% are no messages available for an inactive consumer and %% send credit_reply with Drain=true. - case lists:keytake(ConsumerId, 1, Waiting0) of + case lists:keytake(ConsumerKey, 1, Waiting0) of {value, {_, Con0 = #consumer{delivery_count = DeliveryCountSnd, cfg = Cfg}}, Waiting} -> LinkCreditSnd = link_credit_snd(DeliveryCountRcv, LinkCreditRcv, @@ -368,14 +390,15 @@ apply(Meta, #credit{credit = LinkCreditRcv, delivery_count = DeliveryCountRcv, %% grant the credit Con = Con0#consumer{credit = LinkCreditSnd}, State = State0#?STATE{waiting_consumers = - [{ConsumerId, Con} | Waiting]}, + [{ConsumerKey, Con} | Waiting]}, %% No messages are available for inactive consumers. Available = 0, case credit_api_v2(Cfg) of true -> {State, ok, - {send_msg, CPid, - {credit_reply, CTag, DeliveryCountSnd, LinkCreditSnd, + {send_msg, Cfg#consumer_cfg.pid, + {credit_reply, Cfg#consumer_cfg.tag, + DeliveryCountSnd, LinkCreditSnd, Available, false}, ?DELIVERY_SEND_MSG_OPTS}}; false -> @@ -408,8 +431,8 @@ apply(#{index := Index, %% a dequeue using the same consumer_id isn't possible at this point {State0, {dequeue, empty}}; _ -> - {_, State1} = update_consumer(Meta, ConsumerId, ConsumerMeta, - {once, 1, simple_prefetch}, 0, + {_, State1} = update_consumer(Meta, ConsumerId, ConsumerId, ConsumerMeta, + {once, {simple_prefetch, 1}}, 0, State0), case checkout_one(Meta, false, State1, []) of {success, _, MsgId, @@ -452,15 +475,46 @@ apply(#{index := Index, apply(#{index := Idx} = Meta, #checkout{spec = cancel, consumer_id = ConsumerId}, State0) -> - {State1, Effects1} = cancel_consumer(Meta, ConsumerId, State0, [], - consumer_cancel), - {State, Reply, Effects} = checkout(Meta, State0, State1, Effects1), - update_smallest_raft_index(Idx, Reply, State, Effects); -apply(Meta, #checkout{spec = Spec, meta = ConsumerMeta, - consumer_id = {_, Pid} = ConsumerId}, State0) -> + case consumer_key_from_id(ConsumerId, State0) of + {ok, ConsumerKey} -> + {State1, Effects1} = cancel_consumer(Meta, ConsumerKey, State0, [], + consumer_cancel), + Reply = {ok, consumer_cancel_info(ConsumerKey, State1)}, + {State, _, Effects} = checkout(Meta, State0, State1, Effects1), + update_smallest_raft_index(Idx, Reply, State, Effects); + error -> + update_smallest_raft_index(Idx, {error, consumer_not_found}, + State0, []) + 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_from_args(ConsumerMeta), - {Consumer, State1} = update_consumer(Meta, ConsumerId, ConsumerMeta, - Spec, Priority, State0), + 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, @@ -470,7 +524,9 @@ apply(Meta, #checkout{spec = Spec, meta = ConsumerMeta, %% 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{}, @@ -523,22 +579,22 @@ apply(#{system_time := Ts} = Meta, %% mark it as suspected and return it to the waiting queue {State1, Effects0} = maps:fold( - fun({_, P} = Cid, C0, {S0, E0}) + 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, Cid, C0, false, suspected_down, E0), - {St, Effs1} = return_all(Meta, S0, Effs, Cid, C0), + S0, C0, false, suspected_down, E0), + {St, Effs1} = return_all(Meta, S0, Effs, CKey, C0), %% 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 - #{Cid := C} -> - Waiting0 ++ [{Cid, C}]; + #{CKey := C} -> + Waiting0 ++ [{CKey, C}]; _ -> Waiting0 end, - {St#?STATE{consumers = maps:remove(Cid, St#?STATE.consumers), + {St#?STATE{consumers = maps:remove(CKey, St#?STATE.consumers), waiting_consumers = Waiting, last_active = Ts}, Effs1}; @@ -575,11 +631,12 @@ apply(#{system_time := Ts} = Meta, {State, Effects1} = maps:fold( - fun({_, P} = Cid, #consumer{status = up} = C0, + 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, Cid, C), - Eff1 = consumer_update_active_effects(St, Cid, C, false, + {St, Eff0} = return_all(Meta, St0, Eff, CKey, C), + Eff1 = consumer_update_active_effects(St, C, false, suspected_down, Eff0), {St, Eff1}; (_, _, {St, Eff}) -> @@ -618,12 +675,12 @@ apply(Meta, {nodeup, Node}, #?STATE{consumers = Cons0, ConsumerUpdateActiveFun = consumer_active_flag_update_function(State0), %% mark all consumers as up {State1, Effects1} = - maps:fold(fun({_, P} = ConsumerId, C, {SAcc, EAcc}) + maps:fold(fun(ConsumerKey, ?CONSUMER_PID(P) = C, {SAcc, EAcc}) when (node(P) =:= Node) and (C#consumer.status =/= cancelled) -> - EAcc1 = ConsumerUpdateActiveFun(SAcc, ConsumerId, + EAcc1 = ConsumerUpdateActiveFun(SAcc, ConsumerKey, C, true, up, EAcc), - {update_or_remove_sub(Meta, ConsumerId, + {update_or_remove_sub(Meta, ConsumerKey, C#consumer{status = up}, SAcc), EAcc1}; (_, _, Acc) -> @@ -685,16 +742,17 @@ handle_down(Meta, Pid, #?STATE{consumers = Cons0, {Effects1, State2} = handle_waiting_consumer_down(Pid, State1), % return checked out messages to main queue % Find the consumers for the down pid - DownConsumers = maps:keys( - maps:filter(fun({_, P}, _) -> P =:= Pid end, Cons0)), - lists:foldl(fun(ConsumerId, {S, E}) -> - cancel_consumer(Meta, ConsumerId, S, E, down) + DownConsumers = maps:keys(maps:filter(fun(_CKey, ?CONSUMER_PID(P)) -> + P =:= Pid + end, Cons0)), + lists:foldl(fun(ConsumerKey, {S, E}) -> + cancel_consumer(Meta, ConsumerKey, S, E, down) end, {State2, Effects1}, DownConsumers). consumer_active_flag_update_function( #?STATE{cfg = #cfg{consumer_strategy = competing}}) -> - fun(State, ConsumerId, Consumer, Active, ActivityStatus, Effects) -> - consumer_update_active_effects(State, ConsumerId, Consumer, Active, + fun(State, _ConsumerKey, Consumer, Active, ActivityStatus, Effects) -> + consumer_update_active_effects(State, Consumer, Active, ActivityStatus, Effects) end; consumer_active_flag_update_function( @@ -718,12 +776,15 @@ handle_waiting_consumer_down(Pid, % get cancel effects for down waiting consumers Down = lists:filter(fun({{_, P}, _}) -> P =:= Pid end, WaitingConsumers0), - Effects = lists:foldl(fun ({ConsumerId, _}, Effects) -> + 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({{_, P}, _}) -> P =/= Pid end, + StillUp = lists:filter(fun({_CKey, ?CONSUMER_PID(P)}) -> + P =/= Pid + end, WaitingConsumers0), State = State0#?STATE{waiting_consumers = StillUp}, {Effects, State}. @@ -734,11 +795,11 @@ update_waiting_consumer_status(Node, [begin case node(Pid) of Node -> - {ConsumerId, Consumer#consumer{status = Status}}; + {ConsumerKey, Consumer#consumer{status = Status}}; _ -> - {ConsumerId, Consumer} + {ConsumerKey, Consumer} end - end || {{_, Pid} = ConsumerId, Consumer} <- WaitingConsumers, + end || {{_, Pid} = ConsumerKey, Consumer} <- WaitingConsumers, Consumer#consumer.status =/= cancelled]. -spec state_enter(ra_server:ra_state() | eol, state()) -> @@ -826,9 +887,11 @@ overview(#?STATE{consumers = Cons, delivery_limit => Cfg#cfg.delivery_limit }, SacOverview = case active_consumer(Cons) of - {SacConsumerId, _} -> + {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}; _ -> #{} @@ -910,11 +973,11 @@ handle_aux(leader, _, garbage_collection, Aux, Log, MacState) -> handle_aux(follower, _, garbage_collection, Aux, Log, MacState) -> {no_reply, force_eval_gc(Log, MacState, Aux), Log}; handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, - consumer_id = ConsumerId}, Corr, Pid}, + consumer_key = ConsumerKey}, Corr, Pid}, Aux0, Log0, #?STATE{cfg = #cfg{delivery_limit = undefined}, consumers = Consumers}) -> case Consumers of - #{ConsumerId := #consumer{checked_out = Checked}} -> + #{ConsumerKey := #consumer{checked_out = Checked}} -> {Log, ToReturn} = maps:fold( fun (MsgId, ?MSG(Idx, Header), {L0, Acc}) -> @@ -929,7 +992,7 @@ handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, end end, {Log0, []}, maps:with(MsgIds, Checked)), - Appends = make_requeue(ConsumerId, {notify, Corr, Pid}, + Appends = make_requeue(ConsumerKey, {notify, Corr, Pid}, lists:sort(ToReturn), []), {no_reply, Aux0, Log, Appends}; _ -> @@ -948,11 +1011,11 @@ handle_aux(leader, _, {handle_tick, [QName, Overview, Nodes]}, Pid end, {no_reply, Aux#?AUX{tick_pid = NewPid}, Log}; -handle_aux(_, _, {get_checked_out, ConsumerId, MsgIds}, +handle_aux(_, _, {get_checked_out, ConsumerKey, MsgIds}, Aux0, Log0, #?STATE{cfg = #cfg{}, consumers = Consumers}) -> case Consumers of - #{ConsumerId := #consumer{checked_out = Checked}} -> + #{ConsumerKey := #consumer{checked_out = Checked}} -> {Log, IdMsgs} = maps:fold( fun (MsgId, ?MSG(Idx, Header), {L0, Acc}) -> @@ -1111,7 +1174,7 @@ query_waiting_consumers(#?STATE{waiting_consumers = WaitingConsumers}) -> query_consumer_count(#?STATE{consumers = Consumers, waiting_consumers = WaitingConsumers}) -> - Up = maps:filter(fun(_ConsumerId, #consumer{status = Status}) -> + Up = maps:filter(fun(_ConsumerKey, #consumer{status = Status}) -> Status =/= suspected_down end, Consumers), maps:size(Up) + length(WaitingConsumers). @@ -1123,7 +1186,7 @@ query_consumers(#?STATE{consumers = Consumers, ActiveActivityStatusFun = case ConsumerStrategy of competing -> - fun(_ConsumerId, #consumer{status = Status}) -> + fun(_ConsumerKey, #consumer{status = Status}) -> case Status of suspected_down -> {false, Status}; @@ -1237,6 +1300,9 @@ usage(Name) when is_atom(Name) -> [{_, Use}] -> Use end. +is_v4() -> + rabbit_feature_flags:is_enabled(quorum_queues_v4). + %%% Internal messages_ready(#?STATE{messages = M, @@ -1283,62 +1349,63 @@ num_checked_out(#?STATE{consumers = Cons}) -> maps:size(C) + Acc end, 0, Cons). -cancel_consumer(Meta, ConsumerId, +cancel_consumer(Meta, ConsumerKey, #?STATE{cfg = #cfg{consumer_strategy = competing}} = State, Effects, Reason) -> - cancel_consumer0(Meta, ConsumerId, State, Effects, Reason); -cancel_consumer(Meta, ConsumerId, + 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, ConsumerId, State, Effects, Reason); -cancel_consumer(Meta, ConsumerId, + 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 - #{ConsumerId := #consumer{status = _}} -> + #{ConsumerKey := #consumer{status = _}} -> % The active consumer is to be removed - {State1, Effects1} = cancel_consumer0(Meta, ConsumerId, State0, + {State1, Effects1} = cancel_consumer0(Meta, ConsumerKey, State0, Effects0, Reason), activate_next_consumer(State1, Effects1); _ -> % The cancelled consumer is not active or cancelled % Just remove it from idle_consumers - Waiting = lists:keydelete(ConsumerId, 1, Waiting0), - Effects = cancel_consumer_effects(ConsumerId, State0, Effects0), + Waiting = lists:keydelete(ConsumerKey, 1, Waiting0), + Effects = cancel_consumer_effects(ConsumerKey, 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} end. consumer_update_active_effects(#?STATE{cfg = #cfg{resource = QName}}, - ConsumerId, - #consumer{cfg = #consumer_cfg{meta = Meta}}, + #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, ConsumerId, false, Ack, Prefetch, Active, ActivityStatus, Args]} + [QName, {CTag, CPid}, false, Ack, Prefetch, Active, ActivityStatus, Args]} | Effects]. -cancel_consumer0(Meta, ConsumerId, +cancel_consumer0(Meta, ConsumerKey, #?STATE{consumers = C0} = S0, Effects0, Reason) -> case C0 of - #{ConsumerId := Consumer} -> - {S, Effects2} = maybe_return_all(Meta, ConsumerId, Consumer, + #{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(ConsumerId, S, Effects2), + Effects = cancel_consumer_effects(ConsumerKey, S, Effects2), {S, Effects}; _ -> %% already removed: do nothing @@ -1356,9 +1423,9 @@ activate_next_consumer(#?STATE{consumers = Cons, case lists:filter(fun ({_, #consumer{status = Status}}) -> Status == up end, Waiting0) of - [{NextConsumerId, #consumer{cfg = NextCCfg} = NextConsumer} | _] -> - Remaining = lists:keydelete(NextConsumerId, 1, Waiting0), - Consumer = case maps:get(NextConsumerId, Cons, undefined) of + [{NextCKey, #consumer{cfg = NextCCfg} = NextConsumer} | _] -> + Remaining = lists:keydelete(NextCKey, 1, Waiting0), + Consumer = case maps:get(NextCKey, Cons, undefined) of undefined -> NextConsumer; Existing -> @@ -1368,15 +1435,15 @@ activate_next_consumer(#?STATE{consumers = Cons, Existing#consumer{cfg = NextCCfg} end, #?STATE{service_queue = ServiceQueue} = State0, - ServiceQueue1 = maybe_queue_consumer(NextConsumerId, + ServiceQueue1 = maybe_queue_consumer(NextCKey, Consumer, ServiceQueue), - State = State0#?STATE{consumers = Cons#{NextConsumerId => Consumer}, - service_queue = ServiceQueue1, - waiting_consumers = Remaining}, - Effects = consumer_update_active_effects(State, NextConsumerId, - Consumer, true, - single_active, Effects0), + State = State0#?STATE{consumers = Cons#{NextCKey => Consumer}, + service_queue = ServiceQueue1, + waiting_consumers = Remaining}, + Effects = consumer_update_active_effects(State, Consumer, + true, single_active, + Effects0), {State, Effects}; [] -> {State0, Effects0} @@ -1388,9 +1455,9 @@ activate_next_consumer(#?STATE{consumers = Cons, has_active_consumer(Consumers) -> active_consumer(Consumers) /= undefined. -active_consumer({Cid, #consumer{status = up} = Consumer, _I}) -> - {Cid, Consumer}; -active_consumer({_Cid, #consumer{status = _}, I}) -> +active_consumer({CKey, #consumer{status = up} = Consumer, _I}) -> + {CKey, Consumer}; +active_consumer({_CKey, #consumer{status = _}, I}) -> active_consumer(maps:next(I)); active_consumer(none) -> undefined; @@ -1398,20 +1465,27 @@ active_consumer(M) when is_map(M) -> I = maps:iterator(M), active_consumer(maps:next(I)). -maybe_return_all(#{system_time := Ts} = Meta, ConsumerId, +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 consumer_cancel -> {update_or_remove_sub( - Meta, ConsumerId, + Meta, ConsumerKey, Consumer#consumer{cfg = CCfg#consumer_cfg{lifetime = once}, credit = 0, status = cancelled}, S0), Effects0}; down -> - {S1, Effects1} = return_all(Meta, S0, Effects0, ConsumerId, Consumer), - {S1#?STATE{consumers = maps:remove(ConsumerId, S1#?STATE.consumers), + {S1, Effects1} = return_all(Meta, S0, Effects0, ConsumerKey, Consumer), + {S1#?STATE{consumers = maps:remove(ConsumerKey, S1#?STATE.consumers), last_active = Ts}, Effects1} end. @@ -1568,15 +1642,15 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, Effects0, end. return(#{index := IncomingRaftIdx} = Meta, - ConsumerId, Returned, Effects0, State0) -> + ConsumerKey, Returned, Effects0, State0) -> {State1, Effects1} = maps:fold( fun(MsgId, Msg, {S0, E0}) -> return_one(Meta, MsgId, Msg, - S0, E0, ConsumerId) + S0, E0, ConsumerKey) end, {State0, Effects0}, Returned), State2 = case State1#?STATE.consumers of - #{ConsumerId := Con} -> - update_or_remove_sub(Meta, ConsumerId, Con, State1); + #{ConsumerKey := Con} -> + update_or_remove_sub(Meta, ConsumerKey, Con, State1); _ -> State1 end, @@ -1584,7 +1658,7 @@ return(#{index := IncomingRaftIdx} = Meta, update_smallest_raft_index(IncomingRaftIdx, State, Effects). % used to process messages that are finished -complete(Meta, ConsumerId, [DiscardedMsgId], +complete(Meta, ConsumerKey, [DiscardedMsgId], #consumer{checked_out = Checked0} = Con0, #?STATE{ra_indexes = Indexes0, msg_bytes_checkout = BytesCheckout, @@ -1594,15 +1668,15 @@ complete(Meta, ConsumerId, [DiscardedMsgId], SettledSize = get_header(size, Hdr), Indexes = rabbit_fifo_index:delete(Idx, Indexes0), Con = Con0#consumer{checked_out = Checked, - credit = increase_credit(Meta, Con0, 1)}, - State1 = update_or_remove_sub(Meta, ConsumerId, Con, State0), + credit = increase_credit(Con0, 1)}, + State1 = update_or_remove_sub(Meta, ConsumerKey, Con, State0), State1#?STATE{ra_indexes = Indexes, msg_bytes_checkout = BytesCheckout - SettledSize, messages_total = Tot - 1}; error -> State0 end; -complete(Meta, ConsumerId, DiscardedMsgIds, +complete(Meta, ConsumerKey, DiscardedMsgIds, #consumer{checked_out = Checked0} = Con0, #?STATE{ra_indexes = Indexes0, msg_bytes_checkout = BytesCheckout, @@ -1620,40 +1694,44 @@ complete(Meta, ConsumerId, DiscardedMsgIds, end, {0, Checked0, Indexes0}, DiscardedMsgIds), Len = map_size(Checked0) - map_size(Checked), Con = Con0#consumer{checked_out = Checked, - credit = increase_credit(Meta, Con0, Len)}, - State1 = update_or_remove_sub(Meta, ConsumerId, Con, State0), + credit = increase_credit(Con0, Len)}, + State1 = update_or_remove_sub(Meta, ConsumerKey, Con, State0), State1#?STATE{ra_indexes = Indexes, msg_bytes_checkout = BytesCheckout - SettledSize, messages_total = Tot - Len}. -increase_credit(_Meta, #consumer{cfg = #consumer_cfg{lifetime = once}, - credit = Credit}, _) -> +increase_credit(#consumer{cfg = #consumer_cfg{lifetime = once}, + credit = Credit}, _) -> %% once consumers cannot increment credit Credit; -increase_credit(_Meta, #consumer{cfg = #consumer_cfg{lifetime = auto, - credit_mode = credited}, - 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(_Meta, - #consumer{cfg = #consumer_cfg{credit_mode = +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(_Meta, #consumer{credit = Current}, Credit) -> +increase_credit(#consumer{credit = Current}, Credit) -> Current + Credit. -complete_and_checkout(#{index := IncomingRaftIdx} = Meta, MsgIds, ConsumerId, +complete_and_checkout(#{index := IncomingRaftIdx} = Meta, MsgIds, ConsumerKey, #consumer{} = Con0, Effects0, State0) -> - State1 = complete(Meta, ConsumerId, MsgIds, Con0, State0), + State1 = complete(Meta, ConsumerKey, MsgIds, Con0, State0), {State, ok, Effects} = checkout(Meta, State0, State1, Effects0), update_smallest_raft_index(IncomingRaftIdx, State, Effects). cancel_consumer_effects(ConsumerId, - #?STATE{cfg = #cfg{resource = QName}} = _State, - Effects) -> + #?STATE{cfg = #cfg{resource = QName}}, + Effects) when is_tuple(ConsumerId) -> [{mod_call, rabbit_quorum_queue, cancel_consumer_handler, [QName, ConsumerId]} | Effects]. @@ -1751,8 +1829,8 @@ return_one(Meta, MsgId, Msg0, dlx = DlxState0, cfg = #cfg{delivery_limit = DeliveryLimit, dead_letter_handler = DLH}} = State0, - Effects0, ConsumerId) -> - #consumer{checked_out = Checked0} = Con0 = maps:get(ConsumerId, Consumers), + Effects0, ConsumerKey) -> + #consumer{checked_out = Checked0} = Con0 = maps:get(ConsumerKey, Consumers), Msg = update_msg_header(delivery_count, fun incr/1, 1, Msg0), Header = get_msg_header(Msg), case get_header(delivery_count, Header) of @@ -1760,24 +1838,24 @@ return_one(Meta, MsgId, Msg0, {DlxState, DlxEffects} = rabbit_fifo_dlx:discard([Msg], delivery_limit, DLH, DlxState0), State1 = State0#?STATE{dlx = DlxState}, - State = complete(Meta, ConsumerId, [MsgId], Con0, State1), + State = complete(Meta, ConsumerKey, [MsgId], Con0, State1), {State, DlxEffects ++ Effects0}; _ -> Checked = maps:remove(MsgId, Checked0), Con = Con0#consumer{checked_out = Checked, - credit = increase_credit(Meta, Con0, 1)}, + credit = increase_credit(Con0, 1)}, {add_bytes_return( Header, - State0#?STATE{consumers = Consumers#{ConsumerId => Con}, + State0#?STATE{consumers = Consumers#{ConsumerKey => Con}, returns = lqueue:in(Msg, Returns)}), Effects0} end. -return_all(Meta, #?STATE{consumers = Cons} = State0, Effects0, ConsumerId, +return_all(Meta, #?STATE{consumers = Cons} = State0, Effects0, ConsumerKey, #consumer{checked_out = Checked} = Con) -> - State = State0#?STATE{consumers = Cons#{ConsumerId => Con}}, + State = State0#?STATE{consumers = Cons#{ConsumerKey => Con}}, lists:foldl(fun ({MsgId, Msg}, {S, E}) -> - return_one(Meta, MsgId, Msg, S, E, ConsumerId) + return_one(Meta, MsgId, Msg, S, E, ConsumerKey) end, {State, Effects0}, lists:sort(maps:to_list(Checked))). checkout(Meta, OldState, State0, Effects0) -> @@ -1802,15 +1880,15 @@ checkout(#{index := Index} = Meta, update_smallest_raft_index(Index, Reply, State, Effects) end. -checkout0(Meta, {success, ConsumerId, MsgId, +checkout0(Meta, {success, ConsumerKey, MsgId, ?MSG(_RaftIdx, _Header) = Msg, ExpiredMsg, State, Effects}, SendAcc0) -> DelMsg = {MsgId, Msg}, - SendAcc = case maps:get(ConsumerId, SendAcc0, undefined) of + SendAcc = case maps:get(ConsumerKey, SendAcc0, undefined) of undefined -> - SendAcc0#{ConsumerId => [DelMsg]}; + SendAcc0#{ConsumerKey => [DelMsg]}; LogMsgs -> - SendAcc0#{ConsumerId => [DelMsg | LogMsgs]} + SendAcc0#{ConsumerKey => [DelMsg | LogMsgs]} end, checkout0(Meta, checkout_one(Meta, ExpiredMsg, State, Effects), SendAcc); checkout0(_Meta, {_Activity, ExpiredMsg, State0, Effects0}, SendAcc) -> @@ -1926,11 +2004,13 @@ get_next_msg(#?STATE{returns = Returns0, Msg end. -delivery_effect({CTag, CPid}, [{MsgId, ?MSG(Idx, Header)}], - #?STATE{msg_cache = {Idx, RawMsg}}) -> +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({CTag, CPid}, Msgs, _State) -> +delivery_effect(ConsumerKey, Msgs, State) -> + {CTag, CPid} = consumer_id(ConsumerKey, State), RaftIdxs = lists:foldr(fun ({_, ?MSG(I, _)}, Acc) -> [I | Acc] end, [], Msgs), @@ -1963,13 +2043,13 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> expire_msgs(Ts, ExpiredMsg0, InitState0, Effects0), case priority_queue:out(SQ0) of - {{value, ConsumerId}, SQ1} - when is_map_key(ConsumerId, Cons0) -> + {{value, ConsumerKey}, SQ1} + when is_map_key(ConsumerKey, Cons0) -> case take_next_msg(InitState) of {ConsumerMsg, State0} -> %% there are consumers waiting to be serviced %% process consumer checkout - case maps:get(ConsumerId, Cons0) of + case maps:get(ConsumerKey, Cons0) of #consumer{credit = 0} -> %% no credit but was still on queue %% can happen when draining @@ -2003,8 +2083,8 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> msg_bytes_checkout = BytesCheckout + Size, msg_bytes_enqueue = BytesEnqueue - Size}, State = update_or_remove_sub( - Meta, ConsumerId, Con, State1), - {success, ConsumerId, Next, ConsumerMsg, ExpiredMsg, + Meta, ConsumerKey, Con, State1), + {success, ConsumerKey, Next, ConsumerMsg, ExpiredMsg, State, Effects1} end; empty -> @@ -2060,7 +2140,8 @@ expire(RaCmdTs, State0, Effects) -> timer_effect(RaCmdTs, State, Effects) -> T = case get_next_msg(State) of ?MSG(_, ?TUPLE(Size, Expiry)) - when is_integer(Size), is_integer(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. @@ -2075,7 +2156,7 @@ timer_effect(RaCmdTs, State, Effects) -> end, [{timer, expire_msgs, T} | Effects]. -update_or_remove_sub(Meta, ConsumerId, +update_or_remove_sub(Meta, ConsumerKey, #consumer{cfg = #consumer_cfg{lifetime = once}, checked_out = Checked, credit = 0} = Con, @@ -2084,18 +2165,18 @@ update_or_remove_sub(Meta, ConsumerId, 0 -> #{system_time := Ts} = Meta, % we're done with this consumer - State#?STATE{consumers = maps:remove(ConsumerId, Cons), + State#?STATE{consumers = maps:remove(ConsumerKey, Cons), last_active = Ts}; _ -> % there are unsettled items so need to keep around - State#?STATE{consumers = maps:put(ConsumerId, Con, Cons)} + State#?STATE{consumers = maps:put(ConsumerKey, Con, Cons)} end; -update_or_remove_sub(_Meta, ConsumerId, - #consumer{cfg = #consumer_cfg{}} = Con, +update_or_remove_sub(_Meta, ConsumerKey, + #consumer{} = Con, #?STATE{consumers = Cons, service_queue = ServiceQueue} = State) -> - State#?STATE{consumers = maps:put(ConsumerId, Con, Cons), - service_queue = maybe_queue_consumer(ConsumerId, Con, + State#?STATE{consumers = maps:put(ConsumerKey, Con, Cons), + service_queue = maybe_queue_consumer(ConsumerKey, Con, ServiceQueue)}. maybe_queue_consumer(Key, #consumer{credit = Credit, @@ -2114,15 +2195,17 @@ maybe_queue_consumer(Key, #consumer{credit = Credit, maybe_queue_consumer(_Key, _Consumer, ServiceQueue) -> ServiceQueue. -update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta, - {Life, Credit, Mode0} = Spec, Priority, +update_consumer(Meta, ConsumerKey, {Tag, Pid}, ConsumerMeta, + {Life, Mode} = Spec, Priority, #?STATE{cfg = #cfg{consumer_strategy = competing}, consumers = Cons0} = State0) -> Consumer = case Cons0 of - #{ConsumerId := #consumer{} = Consumer0} -> - merge_consumer(Meta, Consumer0, ConsumerMeta, Spec, Priority); + #{ConsumerKey := #consumer{} = Consumer0} -> + merge_consumer(Meta, Consumer0, ConsumerMeta, + Spec, Priority); _ -> - Mode = credit_mode(Meta, Credit, Mode0), + Credit = included_credit(Mode), + DeliveryCount = initial_delivery_count(Mode), #consumer{cfg = #consumer_cfg{tag = Tag, pid = Pid, lifetime = Life, @@ -2130,11 +2213,11 @@ update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta, priority = Priority, credit_mode = Mode}, credit = Credit, - delivery_count = initial_delivery_count(ConsumerMeta)} + delivery_count = DeliveryCount} end, - {Consumer, update_or_remove_sub(Meta, ConsumerId, Consumer, State0)}; -update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta, - {Life, Credit, Mode0} = Spec, Priority, + {Consumer, update_or_remove_sub(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 = Waiting, @@ -2143,21 +2226,22 @@ update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta, %% if it is a cancelled active consumer, add to waiting unless it is the only %% one, then merge case active_consumer(Cons0) of - {ConsumerId, #consumer{status = up} = Consumer0} -> + {ConsumerKey, #consumer{status = up} = Consumer0} -> Consumer = merge_consumer(Meta, Consumer0, ConsumerMeta, Spec, Priority), - {Consumer, update_or_remove_sub(Meta, ConsumerId, Consumer, State0)}; - undefined when is_map_key(ConsumerId, Cons0) -> + {Consumer, update_or_remove_sub(Meta, ConsumerKey, Consumer, State0)}; + 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(ConsumerId, Cons0), + Consumer0 = maps:get(ConsumerKey, Cons0), Consumer = merge_consumer(Meta, Consumer0, ConsumerMeta, Spec, Priority), - {Consumer, update_or_remove_sub(Meta, ConsumerId, Consumer, State0)}; + {Consumer, update_or_remove_sub(Meta, ConsumerKey, Consumer, State0)}; _ -> %% add as a new waiting consumer - Mode = credit_mode(Meta, Credit, Mode0), + Credit = included_credit(Mode), + DeliveryCount = initial_delivery_count(Mode), Consumer = #consumer{cfg = #consumer_cfg{tag = Tag, pid = Pid, lifetime = Life, @@ -2165,17 +2249,17 @@ update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta, priority = Priority, credit_mode = Mode}, credit = Credit, - delivery_count = initial_delivery_count(ConsumerMeta)}, + delivery_count = DeliveryCount}, {Consumer, State0#?STATE{waiting_consumers = - Waiting ++ [{ConsumerId, Consumer}]}} + Waiting ++ [{ConsumerKey, Consumer}]}} end. -merge_consumer(Meta, #consumer{cfg = CCfg, checked_out = Checked} = Consumer, - ConsumerMeta, {Life, Credit, Mode0}, Priority) -> +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), - Mode = credit_mode(Meta, Credit, Mode0), Consumer#consumer{cfg = CCfg#consumer_cfg{priority = Priority, meta = ConsumerMeta, credit_mode = Mode, @@ -2183,11 +2267,12 @@ merge_consumer(Meta, #consumer{cfg = CCfg, checked_out = Checked} = Consumer, status = up, credit = NewCredit}. -credit_mode(_Meta, Credit, simple_prefetch) -> - {simple_prefetch, Credit}; -credit_mode(_, _, Mode) -> - Mode. - +included_credit({simple_prefetch, Credit}) -> + Credit; +included_credit({credited, _}) -> + 0; +included_credit(credited) -> + 0. %% creates a dehydrated version of the current state to be cached and %% potentially used to for a snaphot at a later point dehydrate_state(#?STATE{cfg = #cfg{}, @@ -2240,7 +2325,14 @@ 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) -> - #enqueue{pid = Pid, seq = Seq, msg = Msg}. + case is_v4() of + true when is_pid(Pid) andalso + is_integer(Seq) -> + %% more compact format + #?ENQ_V2{seq = Seq, msg = Msg}; + _ -> + #enqueue{pid = Pid, seq = Seq, msg = Msg} + end. -spec make_register_enqueuer(pid()) -> protocol(). make_register_enqueuer(Pid) -> @@ -2252,22 +2344,22 @@ make_checkout({_, _} = ConsumerId, Spec, Meta) -> #checkout{consumer_id = ConsumerId, spec = Spec, meta = Meta}. --spec make_settle(consumer_id(), [msg_id()]) -> protocol(). -make_settle(ConsumerId, MsgIds) when is_list(MsgIds) -> - #settle{consumer_id = ConsumerId, msg_ids = MsgIds}. +-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_id(), [msg_id()]) -> protocol(). -make_return(ConsumerId, MsgIds) -> - #return{consumer_id = ConsumerId, msg_ids = MsgIds}. +make_return(ConsumerKey, MsgIds) -> + #return{consumer_key = ConsumerKey, msg_ids = MsgIds}. -spec make_discard(consumer_id(), [msg_id()]) -> protocol(). -make_discard(ConsumerId, MsgIds) -> - #discard{consumer_id = ConsumerId, msg_ids = MsgIds}. +make_discard(ConsumerKey, MsgIds) -> + #discard{consumer_key = ConsumerKey, msg_ids = MsgIds}. --spec make_credit(consumer_id(), rabbit_queue_type:credit(), +-spec make_credit(consumer_key(), rabbit_queue_type:credit(), non_neg_integer(), boolean()) -> protocol(). -make_credit(ConsumerId, Credit, DeliveryCount, Drain) -> - #credit{consumer_id = ConsumerId, +make_credit(Key, Credit, DeliveryCount, Drain) -> + #credit{consumer_key = Key, credit = Credit, delivery_count = DeliveryCount, drain = Drain}. @@ -2385,6 +2477,7 @@ is_expired(Ts, #?STATE{cfg = #cfg{expires = Expires}, is_expired(_Ts, _State) -> false. +%%TODO: provide first class means of configuring get_priority_from_args(#{args := Args}) -> case rabbit_misc:table_lookup(Args, <<"x-priority">>) of {_Key, Value} -> @@ -2426,19 +2519,19 @@ smallest_raft_index(#?STATE{messages = Messages, SmallestRaIdx = rabbit_fifo_index:smallest(Indexes), lists:min([SmallestDlxRaIdx, SmallestMsgsRaIdx, SmallestRaIdx]). -make_requeue(ConsumerId, Notify, [{MsgId, Idx, Header, Msg}], Acc) -> +make_requeue(ConsumerKey, Notify, [{MsgId, Idx, Header, Msg}], Acc) -> lists:reverse([{append, - #requeue{consumer_id = ConsumerId, + #requeue{consumer_key = ConsumerKey, index = Idx, header = Header, msg_id = MsgId, msg = Msg}, Notify} | Acc]); -make_requeue(ConsumerId, Notify, [{MsgId, Idx, Header, Msg} | Rem], Acc) -> - make_requeue(ConsumerId, Notify, Rem, +make_requeue(ConsumerKey, Notify, [{MsgId, Idx, Header, Msg} | Rem], Acc) -> + make_requeue(ConsumerKey, Notify, Rem, [{append, - #requeue{consumer_id = ConsumerId, + #requeue{consumer_key = ConsumerKey, index = Idx, header = Header, msg_id = MsgId, @@ -2462,23 +2555,24 @@ can_immediately_deliver(#?STATE{service_queue = SQ, incr(I) -> I + 1. +get_msg(#?ENQ_V2{msg = M}) -> + M; get_msg(#enqueue{msg = M}) -> M; get_msg(#requeue{msg = M}) -> M. --spec initial_delivery_count(consumer_meta()) -> - rabbit_queue_type:delivery_count(). -initial_delivery_count(#{initial_delivery_count := Count}) -> +initial_delivery_count({credited, Count}) -> %% credit API v2 Count; initial_delivery_count(_) -> %% credit API v1 0. --spec credit_api_v2(#consumer_cfg{}) -> boolean(). -credit_api_v2(#consumer_cfg{meta = ConsumerMeta}) -> - maps:is_key(initial_delivery_count, ConsumerMeta). +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 @@ -2489,3 +2583,79 @@ link_credit_snd(DeliveryCountRcv, LinkCreditRcv, DeliveryCountSnd, ConsumerCfg) %% 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))). + +>>>>>>> e9e71cc724 (QQ: add v4 ff and new more compact enqueue command.) diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index 92e15ef91268..dc6ee204cf1b 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -64,24 +64,32 @@ -type consumer_id() :: {rabbit_types:ctag(), pid()}. %% The entity that receives messages. Uniquely identifies a consumer. --type credit_mode() :: credited | - %% machine_version 2 - simple_prefetch | - %% machine_version 3 - {simple_prefetch, MaxCredit :: non_neg_integer()}. +-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(), - credit_mode()} | + credited, + simple_prefetch} | + {dequeue, settled | unsettled} | - cancel. + cancel | + %% new v4 format + {once | auto, credit_mode()}. -type consumer_meta() :: #{ack => boolean(), username => binary(), prefetch => non_neg_integer(), - args => list(), - %% set if and only if credit API v2 is in use - initial_delivery_count => rabbit_queue_type:delivery_count() + args => list() + % %% set if and only if credit API v2 is in use + % initial_delivery_count => rabbit_queue_type:delivery_count() }. %% static meta data associated with a consumer @@ -107,15 +115,15 @@ %% simple_prefetch: credit is re-filled as deliveries are settled %% or returned. %% credited: credit can only be changed by receiving a consumer_credit - %% command: `{consumer_credit, ReceiverDeliveryCount, Credit}' - credit_mode :: credit_mode(), % part of snapshot data + %% 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 | waiting, - next_msg_id = 0 :: msg_id(), % part of snapshot data + next_msg_id = 0 :: msg_id(), checked_out = #{} :: #{msg_id() => msg()}, %% max number of messages that can be sent %% decremented for each delivery @@ -193,7 +201,7 @@ release_cursors = lqueue:new() :: lqueue:lqueue({release_cursor, ra:index(), #rabbit_fifo{}}), % consumers need to reflect consumer state at time of snapshot - consumers = #{} :: #{consumer_id() => consumer()}, + consumers = #{} :: #{consumer_id() | ra:index() => consumer()}, % consumers that require further service are queued here service_queue = priority_queue:new() :: priority_queue:q(), %% state for at-least-once dead-lettering @@ -202,7 +210,7 @@ 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_id(), consumer()}], + waiting_consumers = [] :: [{consumer_id() | ra:index(), consumer()}], last_active :: option(non_neg_integer()), msg_cache :: option({ra:index(), raw_msg()}), unused_2 diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index 0653f6f09e57..f60f8c33978d 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -14,7 +14,7 @@ -export([ init/1, init/2, - checkout/5, + checkout/4, cancel_checkout/2, enqueue/3, enqueue/4, @@ -41,10 +41,13 @@ -type seq() :: non_neg_integer(). --record(consumer, {last_msg_id :: seq() | -1 | undefined, +-record(consumer, {key :: rabbit_fifo:consumer_key(), + % status = up :: up | cancelled, + last_msg_id :: seq() | -1 | undefined, ack = false :: boolean(), %% Remove this field when feature flag credit_api_v2 becomes required. - delivery_count :: {credit_api_v1, rabbit_queue_type:delivery_count()} | credit_api_v2 + delivery_count :: {credit_api_v1, rabbit_queue_type:delivery_count()} | + credit_api_v2 }). -record(cfg, {servers = [] :: [ra:server_id()], @@ -59,12 +62,11 @@ next_enqueue_seq = 1 :: seq(), %% indicates that we've exceeded the soft limit slow = false :: boolean(), - unsent_commands = #{} :: #{rabbit_fifo:consumer_id() => + unsent_commands = #{} :: #{rabbit_fifo:consumer_key() => {[seq()], [seq()], [seq()]}}, pending = #{} :: #{seq() => {term(), rabbit_fifo:command()}}, - consumer_deliveries = #{} :: #{rabbit_types:ctag() => - #consumer{}}, + consumers = #{} :: #{rabbit_types:ctag() => #consumer{}}, timer_state :: term() }). @@ -194,6 +196,8 @@ enqueue(QName, Msg, State) -> dequeue(QueueName, ConsumerTag, Settlement, #state{cfg = #cfg{timeout = Timeout}} = State0) -> ServerId = pick_server(State0), + %% dequeue never really needs to assign a consumer key so we just use + %% the old ConsumerId format here ConsumerId = consumer_id(ConsumerTag), case ra:process_command(ServerId, rabbit_fifo:make_checkout(ConsumerId, @@ -236,15 +240,16 @@ add_delivery_count_header(Msg, Count) -> -spec settle(rabbit_types:ctag(), [rabbit_fifo:msg_id()], state()) -> {state(), list()}. settle(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) -> + ConsumerKey = consumer_key(ConsumerTag, State0), ServerId = pick_server(State0), - Cmd = rabbit_fifo:make_settle(consumer_id(ConsumerTag), MsgIds), + Cmd = rabbit_fifo:make_settle(ConsumerKey, MsgIds), {send_command(ServerId, undefined, Cmd, normal, State0), []}; settle(ConsumerTag, [_|_] = MsgIds, #state{unsent_commands = Unsent0} = State0) -> - ConsumerId = consumer_id(ConsumerTag), + ConsumerKey = consumer_key(ConsumerTag, State0), %% we've reached the soft limit so will stash the command to be %% sent once we have seen enough notifications - Unsent = maps:update_with(ConsumerId, + Unsent = maps:update_with(ConsumerKey, fun ({Settles, Returns, Discards}) -> %% MsgIds has fewer elements than Settles. %% Therefore put it on the left side of the ++ operator. @@ -264,16 +269,16 @@ settle(ConsumerTag, [_|_] = MsgIds, -spec return(rabbit_types:ctag(), [rabbit_fifo:msg_id()], state()) -> {state(), list()}. return(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) -> + ConsumerKey = consumer_key(ConsumerTag, State0), ServerId = pick_server(State0), - % TODO: make rabbit_fifo return support lists of message ids - Cmd = rabbit_fifo:make_return(consumer_id(ConsumerTag), MsgIds), + Cmd = rabbit_fifo:make_return(ConsumerKey, MsgIds), {send_command(ServerId, undefined, Cmd, normal, State0), []}; return(ConsumerTag, [_|_] = MsgIds, #state{unsent_commands = Unsent0} = State0) -> - ConsumerId = consumer_id(ConsumerTag), + ConsumerKey = consumer_key(ConsumerTag, State0), %% we've reached the soft limit so will stash the command to be %% sent once we have seen enough notifications - Unsent = maps:update_with(ConsumerId, + Unsent = maps:update_with(ConsumerKey, fun ({Settles, Returns, Discards}) -> {Settles, Returns ++ MsgIds, Discards} end, {[], MsgIds, []}, Unsent0), @@ -289,15 +294,16 @@ return(ConsumerTag, [_|_] = MsgIds, -spec discard(rabbit_types:ctag(), [rabbit_fifo:msg_id()], state()) -> {state(), list()}. discard(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) -> + ConsumerKey = consumer_key(ConsumerTag, State0), ServerId = pick_server(State0), - Cmd = rabbit_fifo:make_discard(consumer_id(ConsumerTag), MsgIds), + Cmd = rabbit_fifo:make_discard(ConsumerKey, MsgIds), {send_command(ServerId, undefined, Cmd, normal, State0), []}; discard(ConsumerTag, [_|_] = MsgIds, #state{unsent_commands = Unsent0} = State0) -> - ConsumerId = consumer_id(ConsumerTag), + ConsumerKey = consumer_key(ConsumerTag, State0), %% we've reached the soft limit so will stash the command to be %% sent once we have seen enough notifications - Unsent = maps:update_with(ConsumerId, + Unsent = maps:update_with(ConsumerKey, fun ({Settles, Returns, Discards}) -> {Settles, Returns, Discards ++ MsgIds} end, {[], [], MsgIds}, Unsent0), @@ -320,14 +326,21 @@ discard(ConsumerTag, [_|_] = MsgIds, %% %% @returns `{ok, State}' or `{error | timeout, term()}' -spec checkout(rabbit_types:ctag(), - NumUnsettled :: non_neg_integer(), CreditMode :: rabbit_fifo:credit_mode(), Meta :: rabbit_fifo:consumer_meta(), - state()) -> {ok, state()} | {error | timeout, term()}. -checkout(ConsumerTag, NumUnsettled, CreditMode, Meta, - #state{consumer_deliveries = CDels0} = State0) -> + state()) -> + {ok, ConsumerInfos :: map(), state()} | + {error | timeout, term()}. +checkout(ConsumerTag, CreditMode, Meta, + #state{consumers = CDels0} = State0) + when is_binary(ConsumerTag) -> Servers = sorted_servers(State0), - ConsumerId = {ConsumerTag, self()}, + ConsumerId = consumer_id(ConsumerTag), + NumUnsettled = case CreditMode of + credited -> 0; + {simple_prefetch, Num} -> + Num + end, Cmd = rabbit_fifo:make_checkout(ConsumerId, {auto, NumUnsettled, CreditMode}, Meta), @@ -335,14 +348,10 @@ checkout(ConsumerTag, NumUnsettled, CreditMode, Meta, Ack = maps:get(ack, Meta, true), case try_process_command(Servers, Cmd, State0) of - {ok, Reply, Leader} -> + {ok, {ok, Reply}, Leader} -> LastMsgId = case Reply of - ok -> - %% this is the pre 3.11.1 / 3.10.9 - %% reply format - -1; - {ok, #{num_checked_out := NumChecked, - next_msg_id := NextMsgId}} -> + #{num_checked_out := NumChecked, + next_msg_id := NextMsgId} -> case NumChecked > 0 of true -> %% we cannot know if the pending messages @@ -360,15 +369,17 @@ checkout(ConsumerTag, NumUnsettled, CreditMode, Meta, true -> credit_api_v2; false -> {credit_api_v1, 0} end, + ConsumerKey = maps:get(key, Reply, ConsumerId), SDels = maps:update_with( ConsumerTag, fun (C) -> C#consumer{ack = Ack} end, - #consumer{last_msg_id = LastMsgId, + #consumer{key = ConsumerKey, + last_msg_id = LastMsgId, ack = Ack, delivery_count = DeliveryCount}, CDels0), - {ok, State0#state{leader = Leader, - consumer_deliveries = SDels}}; + {ok, Reply, State0#state{leader = Leader, + consumers = SDels}}; Err -> Err end. @@ -392,7 +403,7 @@ query_single_active_consumer(#state{leader = Leader}) -> state()) -> {state(), rabbit_queue_type:actions()}. credit_v1(ConsumerTag, Credit, Drain, - State = #state{consumer_deliveries = CDels}) -> + #state{consumers = CDels} = State) -> #consumer{delivery_count = {credit_api_v1, Count}} = maps:get(ConsumerTag, CDels), credit(ConsumerTag, Count, Credit, Drain, State). @@ -412,9 +423,9 @@ credit_v1(ConsumerTag, Credit, Drain, state()) -> {state(), rabbit_queue_type:actions()}. credit(ConsumerTag, DeliveryCount, Credit, Drain, State) -> - ConsumerId = consumer_id(ConsumerTag), + ConsumerKey = consumer_key(ConsumerTag, State), ServerId = pick_server(State), - Cmd = rabbit_fifo:make_credit(ConsumerId, Credit, DeliveryCount, Drain), + Cmd = rabbit_fifo:make_credit(ConsumerKey, Credit, DeliveryCount, Drain), {send_command(ServerId, undefined, Cmd, normal, State), []}. %% @doc Cancels a checkout with the rabbit_fifo queue for the consumer tag @@ -428,16 +439,26 @@ credit(ConsumerTag, DeliveryCount, Credit, Drain, State) -> %% @returns `{ok, State}' or `{error | timeout, term()}' -spec cancel_checkout(rabbit_types:ctag(), state()) -> {ok, state()} | {error | timeout, term()}. -cancel_checkout(ConsumerTag, #state{consumer_deliveries = CDels} = State0) -> - Servers = sorted_servers(State0), - ConsumerId = {ConsumerTag, self()}, - Cmd = rabbit_fifo:make_checkout(ConsumerId, cancel, #{}), - State = State0#state{consumer_deliveries = maps:remove(ConsumerTag, CDels)}, - case try_process_command(Servers, Cmd, State) of - {ok, _, Leader} -> - {ok, State#state{leader = Leader}}; - Err -> - Err +cancel_checkout(ConsumerTag, #state{consumers = Consumers} = State0) -> + case Consumers of + #{ConsumerTag := #consumer{}} -> + Servers = sorted_servers(State0), + ConsumerId = {ConsumerTag, self()}, + %% TODO: send any pending commands for consumer + %% checkout always uses the ConsumerId, rather than the key + Cmd = rabbit_fifo:make_checkout(ConsumerId, cancel, #{}), + State = State0#state{consumers = maps:remove(ConsumerTag, Consumers)}, + case try_process_command(Servers, Cmd, State0) of + {ok, _, Leader} -> + {ok, State#state{leader = Leader}}; + Err -> + Err + end; + _ -> + %% TODO: when we implement the `delete' checkout spec we could + %% fallback to that to make sure there is little chance a consumer + %% sticks around in the machine + {ok, State0} end. %% @doc Purges all the messages from a rabbit_fifo queue and returns the number @@ -713,7 +734,7 @@ maybe_auto_ack(false, {deliver, Tag, _Ack, Msgs} = Deliver, State0) -> {ok, State, [Deliver] ++ Actions}. handle_delivery(QName, Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs}, - #state{consumer_deliveries = CDels0} = State0) + #state{consumers = CDels0} = State0) when is_map_key(Tag, CDels0) -> QRef = qref(Leader), {LastId, _} = lists:last(IdMsgs), @@ -729,7 +750,7 @@ handle_delivery(QName, Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs}, %% In this case we can't reliably know what the next expected message %% id should be so have to accept whatever message comes next maybe_auto_ack(Ack, Del, - State0#state{consumer_deliveries = + State0#state{consumers = update_consumer(Tag, LastId, length(IdMsgs), C, CDels0)}); @@ -749,7 +770,7 @@ handle_delivery(QName, Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs}, XDel = {deliver, Tag, Ack, transform_msgs(QName, QRef, Missing ++ IdMsgs)}, maybe_auto_ack(Ack, XDel, - State0#state{consumer_deliveries = + State0#state{consumers = update_consumer(Tag, LastId, length(IdMsgs) + NumMissing, C, CDels0)}) @@ -765,14 +786,14 @@ handle_delivery(QName, Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs}, C when FstId =:= 0 -> % the very first delivery maybe_auto_ack(Ack, Del, - State0#state{consumer_deliveries = + State0#state{consumers = update_consumer(Tag, LastId, length(IdMsgs), C#consumer{last_msg_id = LastId}, CDels0)}) end; handle_delivery(_QName, _Leader, {delivery, Tag, [_ | _] = IdMsgs}, - #state{consumer_deliveries = CDels0} = State0) + #state{consumers = CDels0} = State0) when not is_map_key(Tag, CDels0) -> %% Note: %% https://github.com/rabbitmq/rabbitmq-server/issues/3729 @@ -805,17 +826,17 @@ update_consumer(Tag, LastId, DelCntIncr, Consumer, Consumers) -> delivery_count = D}, Consumers). -add_delivery_count(DelCntIncr, Tag, #state{consumer_deliveries = CDels0} = State) -> +add_delivery_count(DelCntIncr, Tag, #state{consumers = CDels0} = State) -> Con = #consumer{last_msg_id = LastMsgId} = maps:get(Tag, CDels0), CDels = update_consumer(Tag, LastMsgId, DelCntIncr, Con, CDels0), - State#state{consumer_deliveries = CDels}. + State#state{consumers = CDels}. get_missing_deliveries(State, From, To, ConsumerTag) -> %% find local server - ConsumerId = consumer_id(ConsumerTag), - rabbit_log:debug("get_missing_deliveries for ~w from ~b to ~b", - [ConsumerId, From, To]), - Cmd = {get_checked_out, ConsumerId, lists:seq(From, To)}, + ConsumerKey = consumer_key(ConsumerTag, State), + rabbit_log:debug("get_missing_deliveries for consumer '~s' from ~b to ~b", + [ConsumerTag, From, To]), + Cmd = {get_checked_out, ConsumerKey, lists:seq(From, To)}, ServerId = find_local_or_leader(State), case ra:aux_command(ServerId, Cmd) of {ok, Missing} -> @@ -843,7 +864,16 @@ sorted_servers(#state{leader = Leader, cfg = #cfg{servers = Servers}}) -> [Leader | lists:delete(Leader, Servers)]. -consumer_id(ConsumerTag) -> +consumer_key(ConsumerTag, #state{consumers = Consumers}) -> + case Consumers of + #{ConsumerTag := #consumer{key = Key}} -> + Key; + _ -> + %% if no consumer found fall back to using the ConsumerId + consumer_id(ConsumerTag) + end. + +consumer_id(ConsumerTag) when is_binary(ConsumerTag) -> {ConsumerTag, self()}. send_command(Server, Correlation, Command, _Priority, diff --git a/deps/rabbit/src/rabbit_fifo_v3.erl b/deps/rabbit/src/rabbit_fifo_v3.erl index 70691aa9feb9..caab81c951ac 100644 --- a/deps/rabbit/src/rabbit_fifo_v3.erl +++ b/deps/rabbit/src/rabbit_fifo_v3.erl @@ -174,16 +174,16 @@ update_config(Conf, State) -> LastActive = maps:get(created, Conf, undefined), State#?STATE{cfg = Cfg#cfg{release_cursor_interval = RCISpec, - 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}. + 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 diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index 4f3db9a3231c..325392f456f4 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -115,7 +115,8 @@ -opaque state() :: #?STATE{}. %% Delete atom 'credit_api_v1' when feature flag credit_api_v2 becomes required. --type consume_mode() :: {simple_prefetch, non_neg_integer()} | {credited, Initial :: delivery_count() | credit_api_v1}. +-type consume_mode() :: {simple_prefetch, non_neg_integer()} | + {credited, Initial :: delivery_count() | credit_api_v1}. -type consume_spec() :: #{no_ack := boolean(), channel_pid := pid(), limiter_pid => pid() | none, diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 8d455287dea7..95a83a48bf1c 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -874,7 +874,7 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) -> ConsumerTag = quorum_ctag(ConsumerTag0), %% consumer info is used to describe the consumer properties AckRequired = not NoAck, - {CreditMode, EffectivePrefetch, DeclaredPrefetch, ConsumerMeta0} = + {CreditMode, DeclaredPrefetch, ConsumerMeta0} = case Mode of {credited, C} -> Meta = if C =:= credit_api_v1 -> @@ -882,23 +882,22 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) -> is_integer(C) -> #{initial_delivery_count => C} end, - {credited, 0, 0, Meta}; + {credited, 0, Meta}; {simple_prefetch = M, Declared} -> Effective = case Declared of 0 -> ?UNLIMITED_PREFETCH_COUNT; _ -> Declared end, - {M, Effective, Declared, #{}} + {{M, Effective}, Declared, #{}} end, - ConsumerMeta = maps:merge( - ConsumerMeta0, - #{ack => AckRequired, - prefetch => DeclaredPrefetch, - args => Args, - username => ActingUser}), - {ok, QState} = rabbit_fifo_client:checkout(ConsumerTag, EffectivePrefetch, - CreditMode, ConsumerMeta, - QState0), + ConsumerMeta = maps:merge(ConsumerMeta0, + #{ack => AckRequired, + prefetch => DeclaredPrefetch, + args => Args, + username => ActingUser}), + {ok, _Infos, QState} = rabbit_fifo_client:checkout(ConsumerTag, + CreditMode, ConsumerMeta, + QState0), case single_active_consumer_on(Q) of true -> %% get the leader from state diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index 15b75fac4a69..ccba3705014a 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -236,6 +236,10 @@ init_per_group(Group, Config) -> {skip, _} -> Ret; Config2 -> + _ = rabbit_ct_broker_helpers:enable_feature_flag(Config2, + message_containers), + _ = rabbit_ct_broker_helpers:enable_feature_flag(Config2, + quorum_queues_v4), ok = rabbit_ct_broker_helpers:rpc( Config2, 0, application, set_env, [rabbit, channel_tick_interval, 100]), diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 95dc00a226a0..f8c306760a85 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -21,8 +21,7 @@ all() -> [ - {group, machine_version_2}, - {group, machine_version_3}, + {group, machine_version_4}, {group, machine_version_conversion} ]. @@ -34,24 +33,28 @@ all_tests() -> groups() -> [ - {machine_version_2, [shuffle], all_tests()}, - {machine_version_3, [shuffle], all_tests()}, {machine_version_4, [shuffle], all_tests()}, {machine_version_conversion, [shuffle], [convert_v2_to_v3, convert_v3_to_v4]} ]. -init_per_group(machine_version_2, Config) -> - [{machine_version, 2} | Config]; -init_per_group(machine_version_3, Config) -> - [{machine_version, 3} | Config]; init_per_group(machine_version_4, Config) -> [{machine_version, 4} | Config]; init_per_group(machine_version_conversion, Config) -> Config. -end_per_group(_Group, _Config) -> +init_per_testcase(_Testcase, Config) -> + FF = ?config(machine_version, Config) == 4, + ok = meck:new(rabbit_feature_flags, [passthrough]), + meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> FF end), + Config. + +end_per_group(_, Config) -> + Config. + +end_per_testcase(_Group, _Config) -> + meck:unload(), ok. %%%=================================================================== @@ -83,34 +86,50 @@ end_per_group(_Group, _Config) -> test_init(Name) -> init(#{name => Name, - max_in_memory_length => 0, queue_resource => rabbit_misc:r("/", queue, atom_to_binary(Name)), release_cursor_interval => 0}). -enq_enq_checkout_test(C) -> - Cid = {<<"enq_enq_checkout_test">>, self()}, - {State1, _} = enq(C, 1, 1, first, test_init(test)), - {State2, _} = enq(C, 2, 2, second, State1), +-define(FUNCTION_NAME_B, atom_to_binary(?FUNCTION_NAME)). + +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}}). + +enq_enq_checkout_test(Config, Spec) -> + Cid = {?FUNCTION_NAME_B, self()}, + {State1, _} = enq(Config, 1, 1, first, test_init(?FUNCTION_NAME)), + {State2, _} = enq(Config, 2, 2, second, State1), ?assertEqual(2, rabbit_fifo:query_messages_total(State2)), - {_State3, _, Effects} = - apply(meta(C, 3), - rabbit_fifo:make_checkout(Cid, {once, 2, simple_prefetch}, #{}), - State2), + {State3, #{key := CKey, + next_msg_id := NextMsgId}, Effects} = + checkout(Config, ?LINE, Cid, Spec, State2), ?ASSERT_EFF({monitor, _, _}, Effects), - ?ASSERT_EFF({log, [1,2], _Fun, _Local}, Effects), + ?ASSERT_EFF({log, [1, 2], _Fun, _Local}, Effects), + + {State4, _} = settle(Config, CKey, ?LINE, + [NextMsgId, NextMsgId+1], State3), + ?assertMatch(#{num_messages := 0, + num_ready_messages := 0, + num_checked_out := 0, + num_consumers := 1}, + rabbit_fifo:overview(State4)), ok. -credit_enq_enq_checkout_settled_credit_v1_test(C) -> +credit_enq_enq_checkout_settled_credit_v1_test(Config) -> Cid = {?FUNCTION_NAME, self()}, - {State1, _} = enq(C, 1, 1, first, test_init(test)), - {State2, _} = enq(C, 2, 2, second, State1), - {State3, _, Effects} = - apply(meta(C, 3), make_checkout(Cid, {auto, 1, credited}, #{}), State2), - ?ASSERT_EFF({monitor, _, _}, Effects), - ?ASSERT_EFF({log, [1], _Fun, _Local}, Effects), + {State1, _} = enq(Config, 1, 1, first, test_init(test)), + {State2, _} = enq(Config, 2, 2, second, State1), + {State3, #{key := CKey, + next_msg_id := NextMsgId}, Effects3} = + checkout(Config, ?LINE, Cid, {auto, 0, credited}, State2), + ?ASSERT_EFF({monitor, _, _}, Effects3), + {State4, Effects4} = credit(Config, CKey, ?LINE, 1, 0, false, State3), + ?ASSERT_EFF({log, [1], _Fun, _Local}, Effects4), %% settle the delivery this should _not_ result in further messages being %% delivered - {State4, SettledEffects} = settle(C, Cid, 4, 1, State3), + {State5, SettledEffects} = settle(Config, CKey, ?LINE, NextMsgId, State4), ?assertEqual(false, lists:any(fun ({log, _, _, _}) -> true; (_) -> @@ -118,151 +137,146 @@ credit_enq_enq_checkout_settled_credit_v1_test(C) -> end, SettledEffects)), %% granting credit (3) should deliver the second msg if the receivers %% delivery count is (1) - {State5, CreditEffects} = credit(C, Cid, 5, 1, 1, false, State4), - % ?debugFmt("CreditEffects ~tp ~n~tp", [CreditEffects, State4]), + {State6, CreditEffects} = credit(Config, CKey, ?LINE, 1, 1, false, State5), ?ASSERT_EFF({log, [2], _, _}, CreditEffects), - {_State6, FinalEffects} = enq(C, 6, 3, third, State5), + {_State, FinalEffects} = enq(Config, 6, 3, third, State6), ?assertEqual(false, lists:any(fun ({log, _, _, _}) -> true; (_) -> false end, FinalEffects)), ok. -credit_enq_enq_checkout_settled_credit_v2_test(C) -> +credit_enq_enq_checkout_settled_credit_v2_test(Config) -> + InitDelCnt = 16#ff_ff_ff_ff, Ctag = ?FUNCTION_NAME, Cid = {Ctag, self()}, - {State1, _} = enq(C, 1, 1, first, test_init(test)), - {State2, _} = enq(C, 2, 2, second, State1), - {State3, _, Effects} = apply(meta(C, 3), - rabbit_fifo:make_checkout( - Cid, - {auto, 1, credited}, - %% denotes that credit API v2 is used - #{initial_delivery_count => 16#ff_ff_ff_ff}), - State2), - ?ASSERT_EFF({monitor, _, _}, Effects), - ?ASSERT_EFF({log, [1], _Fun, _Local}, Effects), + {State1, _} = enq(Config, 1, 1, first, test_init(test)), + {State2, _} = enq(Config, 2, 2, second, State1), + {State3, #{key := CKey, + next_msg_id := NextMsgId}, Effects3} = + checkout(Config, ?LINE, Cid, {auto, {credited, InitDelCnt}}, State2), + ?ASSERT_EFF({monitor, _, _}, Effects3), + {State4, Effects4} = credit(Config, CKey, ?LINE, 1, InitDelCnt, false, State3), + ?ASSERT_EFF({log, [1], _Fun, _Local}, Effects4), %% Settling the delivery should not grant new credit. - {State4, SettledEffects} = settle(C, Cid, 4, 1, State3), + {State5, SettledEffects} = settle(Config, CKey, 4, NextMsgId, State4), ?assertEqual(false, lists:any(fun ({log, _, _, _}) -> true; (_) -> false end, SettledEffects)), - {State5, CreditEffects} = credit(C, Cid, 5, 1, 0, false, State4), + {State6, CreditEffects} = credit(Config, CKey, ?LINE, 1, 0, false, State5), ?ASSERT_EFF({log, [2], _, _}, CreditEffects), %% The credit_reply should be sent **after** the delivery. ?assertEqual({send_msg, self(), {credit_reply, Ctag, _DeliveryCount = 1, _Credit = 0, _Available = 0, _Drain = false}, ?DELIVERY_SEND_MSG_OPTS}, lists:last(CreditEffects)), - {_State6, FinalEffects} = enq(C, 6, 3, third, State5), + {_State, FinalEffects} = enq(Config, 6, 3, third, State6), ?assertEqual(false, lists:any(fun ({log, _, _, _}) -> true; (_) -> false end, FinalEffects)). -credit_with_drained_v1_test(C) -> - Ctag = ?FUNCTION_NAME, +credit_with_drained_v1_test(Config) -> + Ctag = ?FUNCTION_NAME_B, Cid = {Ctag, self()}, State0 = test_init(test), %% checkout with a single credit - {State1, _, _} = - apply(meta(C, 1), rabbit_fifo:make_checkout(Cid, {auto, 1, credited},#{}), - State0), - ?assertMatch(#rabbit_fifo{consumers = #{Cid := #consumer{credit = 1, - delivery_count = 0}}}, + {State1, #{key := CKey}, _} = checkout(Config, ?LINE, Cid, {auto, 0, credited}, State0), + ?assertMatch(#rabbit_fifo{consumers = #{CKey := #consumer{credit = 0, + delivery_count = 0}}}, State1), + {State2, _Effects2} = credit(Config, CKey, ?LINE, 1, 0, false, State1), + ?assertMatch(#rabbit_fifo{consumers = #{CKey := #consumer{credit = 1, + delivery_count = 0}}}, + State2), {State, Result, _} = - apply(meta(C, 3), rabbit_fifo:make_credit(Cid, 5, 0, true), State1), - ?assertMatch(#rabbit_fifo{consumers = #{Cid := #consumer{credit = 0, - delivery_count = 5}}}, + apply(meta(Config, ?LINE), rabbit_fifo:make_credit(Cid, 5, 0, true), State2), + ?assertMatch(#rabbit_fifo{consumers = #{CKey := #consumer{credit = 0, + delivery_count = 5}}}, State), ?assertEqual({multi, [{send_credit_reply, 0}, {send_drained, {Ctag, 5}}]}, - Result), + Result), ok. -credit_with_drained_v2_test(C) -> +credit_with_drained_v2_test(Config) -> Ctag = ?FUNCTION_NAME, Cid = {Ctag, self()}, State0 = test_init(test), %% checkout with a single credit - {State1, _, _} = apply(meta(C, 1), - rabbit_fifo:make_checkout( - Cid, - {auto, 1, credited}, - %% denotes that credit API v2 is used - #{initial_delivery_count => 0}), - State0), - ?assertMatch(#rabbit_fifo{consumers = #{Cid := #consumer{credit = 1, + {State1, #{key := CKey}, _} = checkout(Config, ?LINE, Cid, {auto, {credited, 0}}, State0), + ?assertMatch(#rabbit_fifo{consumers = #{CKey := #consumer{credit = 0, delivery_count = 0}}}, State1), - {State, ok, Effects} = apply(meta(C, 3), rabbit_fifo:make_credit(Cid, 5, 0, true), State1), - ?assertMatch(#rabbit_fifo{consumers = #{Cid := #consumer{credit = 0, + {State2, _Effects2} = credit(Config, CKey, ?LINE, 1, 0, false, State1), + ?assertMatch(#rabbit_fifo{consumers = #{CKey := #consumer{credit = 1, + delivery_count = 0}}}, + State2), + {State, _, Effects} = + apply(meta(Config, ?LINE), rabbit_fifo:make_credit(CKey, 5, 0, true), State2), + ?assertMatch(#rabbit_fifo{consumers = #{CKey := #consumer{credit = 0, delivery_count = 5}}}, State), ?assertEqual([{send_msg, self(), - {credit_reply, Ctag, _DeliveryCount = 5, _Credit = 0, _Available = 0, _Drain = true}, + {credit_reply, Ctag, _DeliveryCount = 5, + _Credit = 0, _Available = 0, _Drain = true}, ?DELIVERY_SEND_MSG_OPTS}], Effects). -credit_and_drain_v1_test(C) -> +credit_and_drain_v1_test(Config) -> Ctag = ?FUNCTION_NAME, Cid = {Ctag, self()}, - {State1, _} = enq(C, 1, 1, first, test_init(test)), - {State2, _} = enq(C, 2, 2, second, State1), + {State1, _} = enq(Config, 1, 1, first, test_init(test)), + {State2, _} = enq(Config, 2, 2, second, State1), %% checkout without any initial credit (like AMQP 1.0 would) {State3, _, CheckEffs} = - apply(meta(C, 3), rabbit_fifo:make_checkout(Cid, {auto, 0, credited}, #{}), + apply(meta(Config, 3), rabbit_fifo:make_checkout(Cid, {auto, 0, credited}, #{}), State2), ?ASSERT_NO_EFF({log, _, _, _}, CheckEffs), {State4, {multi, [{send_credit_reply, 0}, {send_drained, {Ctag, 2}}]}, - Effects} = apply(meta(C, 4), rabbit_fifo:make_credit(Cid, 4, 0, true), State3), + Effects} = apply(meta(Config, 4), rabbit_fifo:make_credit(Cid, 4, 0, true), State3), ?assertMatch(#rabbit_fifo{consumers = #{Cid := #consumer{credit = 0, delivery_count = 4}}}, State4), ?ASSERT_EFF({log, [1, 2], _, _}, Effects), - {_State5, EnqEffs} = enq(C, 5, 2, third, State4), + {_State5, EnqEffs} = enq(Config, 5, 2, third, State4), ?ASSERT_NO_EFF({log, _, _, _}, EnqEffs), ok. -credit_and_drain_v2_test(C) -> - Ctag = ?FUNCTION_NAME, +credit_and_drain_v2_test(Config) -> + Ctag = ?FUNCTION_NAME_B, Cid = {Ctag, self()}, - {State1, _} = enq(C, 1, 1, first, test_init(test)), - {State2, _} = enq(C, 2, 2, second, State1), - {State3, _, CheckEffs} = apply(meta(C, 3), - rabbit_fifo:make_checkout( - Cid, - %% checkout without any initial credit (like AMQP 1.0 would) - {auto, 0, credited}, - %% denotes that credit API v2 is used - #{initial_delivery_count => 16#ff_ff_ff_ff - 1}), - State2), + {State1, _} = enq(Config, 1, 1, first, test_init(test)), + {State2, _} = enq(Config, 2, 2, second, State1), + {State3, #{key := CKey}, CheckEffs} = checkout(Config, ?LINE, Cid, + {auto, {credited, 16#ff_ff_ff_ff - 1}}, + State2), ?ASSERT_NO_EFF({log, _, _, _}, CheckEffs), - {State4, ok, Effects} = apply(meta(C, 4), - rabbit_fifo:make_credit(Cid, 4, 16#ff_ff_ff_ff - 1, true), - State3), - ?assertMatch(#rabbit_fifo{consumers = #{Cid := #consumer{credit = 0, - delivery_count = 2}}}, + {State4, Effects} = credit(Config, CKey, ?LINE, 4, 16#ff_ff_ff_ff - 1, + true, State3), + ?assertMatch(#rabbit_fifo{consumers = #{CKey := #consumer{credit = 0, + delivery_count = 2}}}, State4), ?ASSERT_EFF({log, [1, 2], _, _}, Effects), %% The credit_reply should be sent **after** the deliveries. ?assertEqual({send_msg, self(), - {credit_reply, Ctag, _DeliveryCount = 2, _Credit = 0, _Available = 0, _Drain = true}, + {credit_reply, Ctag, _DeliveryCount = 2, _Credit = 0, + _Available = 0, _Drain = true}, ?DELIVERY_SEND_MSG_OPTS}, lists:last(Effects)), - {_State5, EnqEffs} = enq(C, 5, 2, third, State4), - ?ASSERT_NO_EFF({log, _, _, _}, EnqEffs). + {_State5, EnqEffs} = enq(Config, 5, 2, third, State4), + ?ASSERT_NO_EFF({log, _, _, _}, EnqEffs), + ok. enq_enq_deq_test(C) -> - Cid = {?FUNCTION_NAME, self()}, + Cid = {?FUNCTION_NAME_B, self()}, {State1, _} = enq(C, 1, 1, first, test_init(test)), {State2, _} = enq(C, 2, 2, second, State1), % get returns a reply value @@ -271,52 +285,57 @@ enq_enq_deq_test(C) -> {_State3, _, [{log, [1], Fun}, {monitor, _, _}]} = - apply(meta(C, 3), rabbit_fifo:make_checkout(Cid, {dequeue, unsettled}, #{}), + apply(meta(C, 3), make_checkout(Cid, {dequeue, unsettled}, #{}), State2), ct:pal("Out ~tp", [Fun([Msg1])]), ok. -enq_enq_deq_deq_settle_test(C) -> - Cid = {?FUNCTION_NAME, self()}, - {State1, _} = enq(C, 1, 1, first, test_init(test)), - {State2, _} = enq(C, 2, 2, second, State1), +enq_enq_deq_deq_settle_test(Config) -> + Cid = {?FUNCTION_NAME_B, self()}, + {State1, _} = enq(Config, 1, 1, first, test_init(test)), + {State2, _} = enq(Config, 2, 2, second, State1), % get returns a reply value {State3, '$ra_no_reply', [{log, [1], _}, {monitor, _, _}]} = - apply(meta(C, 3), rabbit_fifo:make_checkout(Cid, {dequeue, unsettled}, #{}), + apply(meta(Config, 3), make_checkout(Cid, {dequeue, unsettled}, #{}), State2), - {_State4, {dequeue, empty}} = - apply(meta(C, 4), rabbit_fifo:make_checkout(Cid, {dequeue, unsettled}, #{}), + {State4, {dequeue, empty}} = + apply(meta(Config, 4), make_checkout(Cid, {dequeue, unsettled}, #{}), State3), + + {State, _} = settle(Config, Cid, ?LINE, 0, State4), + + ?assertMatch(#{num_consumers := 0}, rabbit_fifo:overview(State)), ok. -enq_enq_checkout_get_settled_test(C) -> +enq_enq_checkout_get_settled_test(Config) -> Cid = {?FUNCTION_NAME, self()}, - {State1, _} = enq(C, 1, 1, first, test_init(test)), + {State1, _} = enq(Config, 1, 1, first, test_init(test)), % get returns a reply value {State2, _, Effs} = - apply(meta(C, 3), rabbit_fifo:make_checkout(Cid, {dequeue, settled}, #{}), + apply(meta(Config, 3), make_checkout(Cid, {dequeue, settled}, #{}), State1), ?ASSERT_EFF({log, [1], _}, Effs), ?assertEqual(0, rabbit_fifo:query_messages_total(State2)), ok. -checkout_get_empty_test(C) -> +checkout_get_empty_test(Config) -> Cid = {?FUNCTION_NAME, self()}, - State = test_init(test), - {_State2, {dequeue, empty}, _} = - apply(meta(C, 1), rabbit_fifo:make_checkout(Cid, {dequeue, unsettled}, #{}), State), + State0 = test_init(test), + {State, {dequeue, empty}, _} = checkout(Config, ?LINE, Cid, + {dequeue, unsettled}, State0), + ?assertMatch(#{num_consumers := 0}, rabbit_fifo:overview(State)), ok. -untracked_enq_deq_test(C) -> +untracked_enq_deq_test(Config) -> Cid = {?FUNCTION_NAME, self()}, State0 = test_init(test), - {State1, _, _} = apply(meta(C, 1), + {State1, _, _} = apply(meta(Config, 1), rabbit_fifo:make_enqueue(undefined, undefined, first), State0), {_State2, _, Effs} = - apply(meta(C, 3), rabbit_fifo:make_checkout(Cid, {dequeue, settled}, #{}), State1), + apply(meta(Config, 3), make_checkout(Cid, {dequeue, settled}, #{}), State1), ?ASSERT_EFF({log, [1], _}, Effs), ok. @@ -327,102 +346,115 @@ enq_expire_deq_test(C) -> S0 = rabbit_fifo:init(Conf), Msg = #basic_message{content = #content{properties = none, payload_fragments_rev = []}}, - {S1, ok, _} = apply(meta(C, 1, 100), rabbit_fifo:make_enqueue(self(), 1, Msg), S0), + {S1, ok, _} = apply(meta(C, 1, 100, {notify, 1, self()}), + rabbit_fifo:make_enqueue(self(), 1, Msg), S0), Cid = {?FUNCTION_NAME, self()}, {_S2, {dequeue, empty}, Effs} = - apply(meta(C, 2, 101), rabbit_fifo:make_checkout(Cid, {dequeue, unsettled}, #{}), S1), + apply(meta(C, 2, 101), make_checkout(Cid, {dequeue, unsettled}, #{}), S1), ?ASSERT_EFF({mod_call, rabbit_global_counters, messages_dead_lettered, [expired, rabbit_quorum_queue, disabled, 1]}, Effs), ok. -enq_expire_enq_deq_test(C) -> +enq_expire_enq_deq_test(Config) -> S0 = test_init(test), %% Msg1 and Msg2 get enqueued in the same millisecond, %% but only Msg1 expires immediately. - Msg1 = #basic_message{content = #content{properties = #'P_basic'{expiration = <<"0">>}, - payload_fragments_rev = [<<"msg1">>]}}, + Msg1 = mc_amqpl:from_basic_message( + #basic_message{routing_keys = [<<"">>], + exchange_name = #resource{name = <<"x">>, + kind = exchange, + virtual_host = <<"v">>}, + content = #content{properties = #'P_basic'{expiration = <<"0">>}, + payload_fragments_rev = [<<"msg1">>]}}), Enq1 = rabbit_fifo:make_enqueue(self(), 1, Msg1), - {S1, ok, _} = apply(meta(C, 1, 100), Enq1, S0), + {S1, ok, _} = apply(meta(Config, 1, 100, {notify, 1, self()}), Enq1, S0), Msg2 = #basic_message{content = #content{properties = none, payload_fragments_rev = [<<"msg2">>]}}, Enq2 = rabbit_fifo:make_enqueue(self(), 2, Msg2), - {S2, ok, _} = apply(meta(C, 2, 100), Enq2, S1), + {S2, ok, _} = apply(meta(Config, 2, 100, {notify, 2, self()}), Enq2, S1), Cid = {?FUNCTION_NAME, self()}, {_S3, _, Effs} = - apply(meta(C, 3, 101), rabbit_fifo:make_checkout(Cid, {dequeue, unsettled}, #{}), S2), + apply(meta(Config, 3, 101), make_checkout(Cid, {dequeue, unsettled}, #{}), S2), {log, [2], Fun} = get_log_eff(Effs), [{reply, _From, {wrap_reply, {dequeue, {_MsgId, _HeaderMsg}, ReadyMsgCount}}}] = Fun([Enq2]), ?assertEqual(0, ReadyMsgCount). -enq_expire_deq_enq_enq_deq_deq_test(C) -> +enq_expire_deq_enq_enq_deq_deq_test(Config) -> S0 = test_init(test), - Msg1 = #basic_message{content = #content{properties = #'P_basic'{expiration = <<"0">>}, - payload_fragments_rev = [<<"msg1">>]}}, - {S1, ok, _} = apply(meta(C, 1, 100), rabbit_fifo:make_enqueue(self(), 1, Msg1), S0), - {S2, {dequeue, empty}, _} = apply(meta(C, 2, 101), - rabbit_fifo:make_checkout({c1, self()}, {dequeue, unsettled}, #{}), S1), - {S3, _} = enq(C, 3, 2, msg2, S2), - {S4, _} = enq(C, 4, 3, msg3, S3), + Msg1 = #basic_message{content = + #content{properties = #'P_basic'{expiration = <<"0">>}, + payload_fragments_rev = [<<"msg1">>]}}, + {S1, ok, _} = apply(meta(Config, 1, 100, {notify, 1, self()}), + rabbit_fifo:make_enqueue(self(), 1, Msg1), S0), + {S2, {dequeue, empty}, _} = apply(meta(Config, 2, 101), + make_checkout({c1, self()}, + {dequeue, unsettled}, #{}), S1), + {S3, _} = enq(Config, 3, 2, msg2, S2), + {S4, _} = enq(Config, 4, 3, msg3, S3), {S5, '$ra_no_reply', [{log, [3], _}, {monitor, _, _}]} = - apply(meta(C, 5), rabbit_fifo:make_checkout({c2, self()}, {dequeue, unsettled}, #{}), S4), + apply(meta(Config, 5), make_checkout({c2, self()}, {dequeue, unsettled}, #{}), S4), {_S6, '$ra_no_reply', [{log, [4], _}, {monitor, _, _}]} = - apply(meta(C, 6), rabbit_fifo:make_checkout({c3, self()}, {dequeue, unsettled}, #{}), S5). + apply(meta(Config, 6), make_checkout({c3, self()}, {dequeue, unsettled}, #{}), S5), + ok. -release_cursor_test(C) -> - Cid = {?FUNCTION_NAME, self()}, - {State1, _} = enq(C, 1, 1, first, test_init(test)), - {State2, _} = enq(C, 2, 2, second, State1), - {State3, _} = check(C, Cid, 3, 10, State2), +release_cursor_test(Config) -> + Cid = {?FUNCTION_NAME_B, self()}, + {State1, _} = enq(Config, 1, 1, first, test_init(test)), + {State2, _} = enq(Config, 2, 2, second, State1), + {State3, #{key := CKey}, _} = checkout(Config, ?LINE, Cid, + {auto, {simple_prefetch, 10}}, + State2), % no release cursor effect at this point - {State4, _} = settle(C, Cid, 4, 1, State3), - {_Final, Effects1} = settle(C, Cid, 5, 0, State4), + {State4, _} = settle(Config, CKey, ?LINE, 1, State3), + Settle0Idx = ?LINE, + {_Final, Effects1} = settle(Config, CKey, Settle0Idx, 0, State4), + ct:pal("Effects1 ~p", [Effects1]), % empty queue forwards release cursor all the way - ?ASSERT_EFF({release_cursor, 5, _}, Effects1), - ok. - -checkout_enq_settle_test(C) -> - Cid = {?FUNCTION_NAME, self()}, - {State1, [{monitor, _, _} | _]} = check(C, Cid, 1, test_init(test)), - {State2, Effects0} = enq(C, 2, 1, first, State1), - %% TODO: this should go back to a send_msg effect after optimisation - % ?ASSERT_EFF({log, [2], _, _}, Effects0), - ?ASSERT_EFF({send_msg, _, - {delivery, ?FUNCTION_NAME, - [{0, {_, first}}]}, _}, - Effects0), - {State3, _} = enq(C, 3, 2, second, State2), - {_, _Effects} = settle(C, Cid, 4, 0, State3), + ?ASSERT_EFF({release_cursor, _, __}, Effects1), + ok. + +checkout_enq_settle_test(Config) -> + Cid = {?FUNCTION_NAME_B, self()}, + {State1, #{key := CKey, + next_msg_id := NextMsgId}, + [{monitor, _, _} | _]} = checkout(Config, ?LINE, Cid, 1, test_init(test)), + {State2, Effects0} = enq(Config, 2, 1, first, State1), + ?ASSERT_EFF({send_msg, _, {delivery, _, [{0, {_, first}}]}, _}, Effects0), + {State3, _} = enq(Config, 3, 2, second, State2), + {_, Effects} = settle(Config, CKey, 4, NextMsgId, State3), % the release cursor is the smallest raft index that does not % contribute to the state of the application - % ?ASSERT_EFF({release_cursor, 2, _}, Effects), + ?ASSERT_EFF({release_cursor, 2, _}, Effects), ok. -duplicate_enqueue_test(C) -> - Cid = {<<"duplicate_enqueue_test">>, self()}, - {State1, [ {monitor, _, _} | _]} = check_n(C, Cid, 5, 5, test_init(test)), - {State2, Effects2} = enq(C, 2, 1, first, State1), - % ?ASSERT_EFF({log, [2], _, _}, Effects2), +duplicate_enqueue_test(Config) -> + Cid = {?FUNCTION_NAME_B, self()}, + MsgSeq = 1, + {State1, [ {monitor, _, _} | _]} = check_n(Config, Cid, 5, 5, test_init(test)), + {State2, Effects2} = enq(Config, 2, MsgSeq, first, State1), ?ASSERT_EFF({send_msg, _, {delivery, _, [{_, {_, first}}]}, _}, Effects2), - {_State3, Effects3} = enq(C, 3, 1, first, State2), + {_State3, Effects3} = enq(Config, 3, MsgSeq, first, State2), ?ASSERT_NO_EFF({log, [_], _, _}, Effects3), ok. -return_test(C) -> +return_test(Config) -> Cid = {<<"cid">>, self()}, Cid2 = {<<"cid2">>, self()}, - {State0, _} = enq(C, 1, 1, msg, test_init(test)), - {State1, _} = check_auto(C, Cid, 2, State0), - {State2, _} = check_auto(C, Cid2, 3, State1), - {State3, _, _} = apply(meta(C, 4), rabbit_fifo:make_return(Cid, [0]), State2), - ?assertMatch(#{Cid := #consumer{checked_out = C1}} when map_size(C1) == 0, - State3#rabbit_fifo.consumers), - ?assertMatch(#{Cid2 := #consumer{checked_out = C2}} when map_size(C2) == 1, - State3#rabbit_fifo.consumers), + {State0, _} = enq(Config, 1, 1, msg, test_init(test)), + {State1, #{key := C1Key, + next_msg_id := MsgId}, _} = checkout(Config, ?LINE, Cid, 1, State0), + {State2, #{key := C2Key}, _} = checkout(Config, ?LINE, Cid2, 1, State1), + {State3, _, _} = apply(meta(Config, 4), + rabbit_fifo:make_return(C1Key, [MsgId]), State2), + ?assertMatch(#{C1Key := #consumer{checked_out = C1}} + when map_size(C1) == 0, State3#rabbit_fifo.consumers), + ?assertMatch(#{C2Key := #consumer{checked_out = C2}} + when map_size(C2) == 1, State3#rabbit_fifo.consumers), ok. return_dequeue_delivery_limit_test(C) -> @@ -448,33 +480,27 @@ return_dequeue_delivery_limit_test(C) -> ?assertMatch(#{num_messages := 0}, rabbit_fifo:overview(State4)), ok. -return_non_existent_test(C) -> +return_non_existent_test(Config) -> Cid = {<<"cid">>, self()}, - {State0, _} = enq(C, 1, 1, second, test_init(test)), - % return non-existent - {_State2, _} = apply(meta(C, 3), rabbit_fifo:make_return(Cid, [99]), State0), + {State0, _} = enq(Config, 1, 1, second, test_init(test)), + % return non-existent, check it doesnt crash + {_State2, _} = apply(meta(Config, 3), rabbit_fifo:make_return(Cid, [99]), State0), ok. -return_checked_out_test(C) -> +return_checked_out_test(Config) -> Cid = {<<"cid">>, self()}, - {State0, _} = enq(C, 1, 1, first, test_init(test)), - {State1, [_Monitor, - {log, [1], Fun, _} - | _ ] - } = check_auto(C, Cid, 2, State0), - - Msg1 = rabbit_fifo:make_enqueue(self(), 1, first), - - [{send_msg, _, {delivery, _, [{MsgId, _}]}, _}] = Fun([Msg1]), + {State0, _} = enq(Config, 1, 1, first, test_init(test)), + {State1, #{key := CKey, + next_msg_id := MsgId}, Effects1} = + checkout(Config, ?LINE, Cid, 1, State0), + ?ASSERT_EFF({log, [1], _Fun, _Local}, Effects1), % returning immediately checks out the same message again - {_, ok, [ - {log, [1], _, _} - % {send_msg, _, {delivery, _, [{_, _}]}, _}, - ]} = - apply(meta(C, 3), rabbit_fifo:make_return(Cid, [MsgId]), State1), + {_State, ok, Effects2} = + apply(meta(Config, 3), rabbit_fifo:make_return(CKey, [MsgId]), State1), + ?ASSERT_EFF({log, [1], _Fun, _Local}, Effects2), ok. -return_checked_out_limit_test(C) -> +return_checked_out_limit_test(Config) -> Cid = {<<"cid">>, self()}, Init = init(#{name => test, queue_resource => rabbit_misc:r("/", queue, @@ -483,37 +509,36 @@ return_checked_out_limit_test(C) -> max_in_memory_length => 0, delivery_limit => 1}), Msg1 = rabbit_fifo:make_enqueue(self(), 1, first), - {State0, _} = enq(C, 1, 1, first, Init), - {State1, [_Monitor, - {log, [1], Fun1, _} - | _ ]} = check_auto(C, Cid, 2, State0), - [{send_msg, _, {delivery, _, [{MsgId, _}]}, _}] = Fun1([Msg1]), + {State0, _} = enq(Config, 1, 1, Msg1, Init), + {State1, #{key := CKey, + next_msg_id := MsgId}, Effects1} = + checkout(Config, ?LINE, Cid, 1, State0), + ?ASSERT_EFF({log, [1], _Fun, _Local}, Effects1), % returning immediately checks out the same message again - {State2, ok, [ - {log, [1], Fun2, _} - ]} = - apply(meta(C, 3), rabbit_fifo:make_return(Cid, [MsgId]), State1), - [{send_msg, _, {delivery, _, [{MsgId2, _}]}, _}] = Fun2([Msg1]), + {State2, ok, Effects2} = + apply(meta(Config, 3), rabbit_fifo:make_return(CKey, [MsgId]), State1), + ?ASSERT_EFF({log, [1], _Fun, _Local}, Effects2), + {#rabbit_fifo{} = State, ok, _} = - apply(meta(C, 4), rabbit_fifo:make_return(Cid, [MsgId2]), State2), + apply(meta(Config, 4), rabbit_fifo:make_return(Cid, [MsgId + 1]), State2), ?assertEqual(0, rabbit_fifo:query_messages_total(State)), ok. -return_auto_checked_out_test(C) -> +return_auto_checked_out_test(Config) -> Cid = {<<"cid">>, self()}, Msg1 = rabbit_fifo:make_enqueue(self(), 1, first), - {State00, _} = enq(C, 1, 1, first, test_init(test)), - {State0, _} = enq(C, 2, 2, second, State00), + {State00, _} = enq(Config, 1, 1, first, test_init(test)), + {State0, _} = enq(Config, 2, 2, second, State00), % it first active then inactive as the consumer took on but cannot take % any more {State1, [_Monitor, {log, [1], Fun1, _} - ]} = check_auto(C, Cid, 2, State0), + ]} = check_auto(Config, Cid, 2, State0), [{send_msg, _, {delivery, _, [{MsgId, _}]}, _}] = Fun1([Msg1]), % return should include another delivery - {_State2, _, Effects} = apply(meta(C, 3), rabbit_fifo:make_return(Cid, [MsgId]), State1), + {_State2, _, Effects} = apply(meta(Config, 3), + rabbit_fifo:make_return(Cid, [MsgId]), State1), [{log, [1], Fun2, _} | _] = Effects, - [{send_msg, _, {delivery, _, [{_MsgId2, {#{delivery_count := 1}, first}}]}, _}] = Fun2([Msg1]), ok. @@ -611,7 +636,8 @@ down_with_noconnection_returns_unack_test(C) -> down_with_noproc_enqueuer_is_cleaned_up_test(C) -> State00 = test_init(test), Pid = spawn(fun() -> ok end), - {State0, _, Effects0} = apply(meta(C, 1), rabbit_fifo:make_enqueue(Pid, 1, first), State00), + {State0, _, Effects0} = apply(meta(C, 1, ?LINE, {notify, 1, Pid}), + rabbit_fifo:make_enqueue(Pid, 1, first), State00), ?ASSERT_EFF({monitor, process, _}, Effects0), {State1, _, _} = apply(meta(C, 3), {down, Pid, noproc}, State0), % ensure there are no enqueuers @@ -1490,45 +1516,43 @@ single_active_with_credited_v1_test(C) -> rabbit_fifo:query_waiting_consumers(State3)), ok. -single_active_with_credited_v2_test(C) -> +single_active_with_credited_v2_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME)), + queue_resource => rabbit_misc:r("/", queue, + ?FUNCTION_NAME_B), release_cursor_interval => 0, single_active_consumer_on => true}), C1 = {<<"ctag1">>, self()}, - {State1, _, _} = apply(meta(C, 1), - make_checkout(C1, - {auto, 0, credited}, - %% denotes that credit API v2 is used - #{initial_delivery_count => 0}), - State0), + {State1, {ok, #{key := CKey1}}, _} = + apply(meta(Config, 1), + make_checkout(C1, {auto, {credited, 0}}, #{}), State0), C2 = {<<"ctag2">>, self()}, - {State2, _, _} = apply(meta(C, 2), - make_checkout(C2, - {auto, 0, credited}, - %% denotes that credit API v2 is used - #{initial_delivery_count => 0}), - State1), + {State2, {ok, #{key := CKey2}}, _} = + apply(meta(Config, 2), + make_checkout(C2, {auto, {credited, 0}}, #{}), State1), %% add some credit - C1Cred = rabbit_fifo:make_credit(C1, 5, 0, false), - {State3, ok, Effects1} = apply(meta(C, 3), C1Cred, State2), + C1Cred = rabbit_fifo:make_credit(CKey1, 5, 0, false), + {State3, ok, Effects1} = apply(meta(Config, 3), C1Cred, State2), ?assertEqual([{send_msg, self(), - {credit_reply, <<"ctag1">>, _DeliveryCount = 0, _Credit = 5, _Available = 0, _Drain = false}, + {credit_reply, <<"ctag1">>, _DeliveryCount = 0, _Credit = 5, + _Available = 0, _Drain = false}, ?DELIVERY_SEND_MSG_OPTS}], Effects1), - C2Cred = rabbit_fifo:make_credit(C2, 4, 0, false), - {State, ok, Effects2} = apply(meta(C, 4), C2Cred, State3), + C2Cred = rabbit_fifo:make_credit(CKey2, 4, 0, false), + {State, ok, Effects2} = apply(meta(Config, 4), C2Cred, State3), ?assertEqual({send_msg, self(), - {credit_reply, <<"ctag2">>, _DeliveryCount = 0, _Credit = 4, _Available = 0, _Drain = false}, + {credit_reply, <<"ctag2">>, _DeliveryCount = 0, _Credit = 4, + _Available = 0, _Drain = false}, ?DELIVERY_SEND_MSG_OPTS}, Effects2), %% both consumers should have credit - ?assertMatch(#{C1 := #consumer{credit = 5}}, + ?assertMatch(#{CKey1 := #consumer{credit = 5}}, State#rabbit_fifo.consumers), - ?assertMatch([{C2, #consumer{credit = 4}}], - rabbit_fifo:query_waiting_consumers(State)). + ?assertMatch([{CKey2, #consumer{credit = 4}}], + rabbit_fifo:query_waiting_consumers(State)), + ok. register_enqueuer_test(C) -> State0 = init(#{name => ?FUNCTION_NAME, @@ -1539,23 +1563,31 @@ register_enqueuer_test(C) -> overflow_strategy => reject_publish}), %% simply registering should be ok when we're below limit Pid1 = test_util:fake_pid(node()), - {State1, ok, [_]} = apply(meta(C, 1), make_register_enqueuer(Pid1), State0), + {State1, ok, [_]} = apply(meta(C, 1, ?LINE, {notify, 1, Pid1}), + make_register_enqueuer(Pid1), State0), - {State2, ok, _} = apply(meta(C, 2), rabbit_fifo:make_enqueue(Pid1, 1, one), State1), + {State2, ok, _} = apply(meta(C, 2, ?LINE, {notify, 2, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 1, one), State1), %% register another enqueuer shoudl be ok Pid2 = test_util:fake_pid(node()), - {State3, ok, [_]} = apply(meta(C, 3), make_register_enqueuer(Pid2), State2), + {State3, ok, [_]} = apply(meta(C, 3, ?LINE, {notify, 3, Pid2}), + make_register_enqueuer(Pid2), State2), - {State4, ok, _} = apply(meta(C, 4), rabbit_fifo:make_enqueue(Pid1, 2, two), State3), - {State5, ok, Efx} = apply(meta(C, 5), rabbit_fifo:make_enqueue(Pid1, 3, three), State4), + {State4, ok, _} = apply(meta(C, 4, ?LINE, {notify, 4, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 2, two), State3), + {State5, ok, Efx} = apply(meta(C, 5, ?LINE, {notify, 4, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 3, three), State4), % ct:pal("Efx ~tp", [Efx]), %% validate all registered enqueuers are notified of overflow state - ?ASSERT_EFF({send_msg, P, {queue_status, reject_publish}, [ra_event]}, P == Pid1, Efx), - ?ASSERT_EFF({send_msg, P, {queue_status, reject_publish}, [ra_event]}, P == Pid2, Efx), + ?ASSERT_EFF({send_msg, P, {queue_status, reject_publish}, [ra_event]}, + P == Pid1, Efx), + ?ASSERT_EFF({send_msg, P, {queue_status, reject_publish}, [ra_event]}, + P == Pid2, Efx), %% this time, registry should return reject_publish - {State6, reject_publish, [_]} = apply(meta(C, 6), make_register_enqueuer( - test_util:fake_pid(node())), State5), + {State6, reject_publish, [_]} = + apply(meta(C, 6), make_register_enqueuer( + test_util:fake_pid(node())), State5), ?assertMatch(#{num_enqueuers := 3}, rabbit_fifo:overview(State6)), @@ -1595,9 +1627,12 @@ reject_publish_purge_test(C) -> %% simply registering should be ok when we're below limit Pid1 = test_util:fake_pid(node()), {State1, ok, [_]} = apply(meta(C, 1), make_register_enqueuer(Pid1), State0), - {State2, ok, _} = apply(meta(C, 2), rabbit_fifo:make_enqueue(Pid1, 1, one), State1), - {State3, ok, _} = apply(meta(C, 3), rabbit_fifo:make_enqueue(Pid1, 2, two), State2), - {State4, ok, Efx} = apply(meta(C, 4), rabbit_fifo:make_enqueue(Pid1, 3, three), State3), + {State2, ok, _} = apply(meta(C, 2, ?LINE, {notify, 2, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 1, one), State1), + {State3, ok, _} = apply(meta(C, 3, ?LINE, {notify, 2, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 2, two), State2), + {State4, ok, Efx} = apply(meta(C, 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(C, 5), rabbit_fifo:make_purge(), State4), @@ -1613,10 +1648,14 @@ reject_publish_applied_after_limit_test(C) -> State0 = init(InitConf), %% simply registering should be ok when we're below limit Pid1 = test_util:fake_pid(node()), - {State1, ok, [_]} = apply(meta(C, 1), make_register_enqueuer(Pid1), State0), - {State2, ok, _} = apply(meta(C, 2), rabbit_fifo:make_enqueue(Pid1, 1, one), State1), - {State3, ok, _} = apply(meta(C, 3), rabbit_fifo:make_enqueue(Pid1, 2, two), State2), - {State4, ok, Efx} = apply(meta(C, 4), rabbit_fifo:make_enqueue(Pid1, 3, three), State3), + {State1, ok, [_]} = apply(meta(C, 1, ?LINE, {notify, 1, Pid1}), + make_register_enqueuer(Pid1), State0), + {State2, ok, _} = apply(meta(C, 2, ?LINE, {notify, 1, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 1, one), State1), + {State3, ok, _} = apply(meta(C, 3, ?LINE, {notify, 1, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 2, two), State2), + {State4, ok, Efx} = apply(meta(C, 4, ?LINE, {notify, 1, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 3, three), State3), % ct:pal("Efx ~tp", [Efx]), ?ASSERT_NO_EFF({send_msg, P, {queue_status, reject_publish}, [ra_event]}, P == Pid1, Efx), %% apply new config @@ -1646,21 +1685,20 @@ purge_nodes_test(C) -> queue_resource => rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME, utf8)), single_active_consumer_on => false}), - {State1, _, _} = apply(meta(C, 1), + {State1, _, _} = apply(meta(C, 1, ?LINE, {notify, 1, EnqPid}), rabbit_fifo:make_enqueue(EnqPid, 1, msg1), State0), - {State2, _, _} = apply(meta(C, 2), + {State2, _, _} = apply(meta(C, 2, ?LINE, {notify, 2, EnqPid2}), rabbit_fifo:make_enqueue(EnqPid2, 1, msg2), State1), {State3, _} = check(C, Cid, 3, 1000, State2), {State4, _, _} = apply(meta(C, 4), {down, EnqPid, noconnection}, State3), - ?assertMatch( - [{aux, {handle_tick, - [#resource{}, _Metrics, - [ThisNode, Node] - ]}}] , rabbit_fifo:tick(1, State4)), + ?assertMatch([{aux, {handle_tick, + [#resource{}, _Metrics, + [ThisNode, Node]]}}], + rabbit_fifo:tick(1, State4)), %% assert there are both enqueuers and consumers {State, _, _} = apply(meta(C, 5), rabbit_fifo:make_purge_nodes([Node]), @@ -1672,26 +1710,29 @@ purge_nodes_test(C) -> ?assertMatch(#rabbit_fifo{consumers = Cons} when map_size(Cons) == 0, State), - ?assertMatch( - [{aux, {handle_tick, - [#resource{}, _Metrics, - [ThisNode] - ]}}] , rabbit_fifo:tick(1, State)), + ?assertMatch([{aux, {handle_tick, + [#resource{}, _Metrics, + [ThisNode]]}}], + rabbit_fifo:tick(1, State)), ok. meta(Config, Idx) -> meta(Config, Idx, 0). meta(Config, Idx, Timestamp) -> + meta(Config, Idx, Timestamp, no_reply). + +meta(Config, Idx, Timestamp, ReplyMode) -> #{machine_version => ?config(machine_version, Config), index => Idx, term => 1, system_time => Timestamp, + reply_mode => ReplyMode, from => {make_ref(), self()}}. enq(Config, Idx, MsgSeq, Msg, State) -> strip_reply( - rabbit_fifo:apply(meta(Config, Idx), + rabbit_fifo:apply(meta(Config, Idx, 0, {notify, MsgSeq, self()}), rabbit_fifo:make_enqueue(self(), MsgSeq, Msg), State)). deq(Config, Idx, Cid, Settlement, Msg, State0) -> @@ -1731,8 +1772,20 @@ check(Config, Cid, Idx, Num, State) -> rabbit_fifo:make_checkout(Cid, {auto, Num, simple_prefetch}, #{}), State)). -settle(Config, Cid, Idx, MsgId, State) -> - strip_reply(apply(meta(Config, Idx), rabbit_fifo:make_settle(Cid, [MsgId]), State)). +checkout(Config, Idx, Cid, Credit, State) + when is_integer(Credit) -> + checkout(Config, Idx, Cid, {auto, {simple_prefetch, Credit}}, State); +checkout(Config, Idx, Cid, Spec, State) -> + checkout_reply( + apply(meta(Config, Idx), + rabbit_fifo:make_checkout(Cid, Spec, #{}), + State)). + +settle(Config, Cid, Idx, MsgId, State) when is_integer(MsgId) -> + settle(Config, Cid, Idx, [MsgId], State); +settle(Config, Cid, Idx, MsgIds, State) when is_list(MsgIds) -> + strip_reply(apply(meta(Config, Idx), + rabbit_fifo:make_settle(Cid, MsgIds), State)). return(Config, Cid, Idx, MsgId, State) -> strip_reply(apply(meta(Config, Idx), rabbit_fifo:make_return(Cid, [MsgId]), State)). @@ -1743,10 +1796,18 @@ credit(Config, Cid, Idx, Credit, DelCnt, Drain, State) -> strip_reply({State, _, Effects}) -> {State, Effects}. +% strip_reply({State, Effects}) -> +% {State, Effects}. + +checkout_reply({State, {ok, CInfo}, Effects}) when is_map(CInfo) -> + {State, CInfo, Effects}; +checkout_reply(Oth) -> + Oth. run_log(Config, InitState, Entries) -> lists:foldl(fun ({Idx, E}, {Acc0, Efx0}) -> - case apply(meta(Config, Idx), E, Acc0) of + case apply(meta(Config, Idx, Idx, {notify, Idx, self()}), + E, Acc0) of {Acc, _, Efx} when is_list(Efx) -> {Acc, Efx0 ++ Efx}; {Acc, _, Efx} -> @@ -1934,7 +1995,7 @@ queue_ttl_test(C) -> = rabbit_fifo:tick(Now + 2500, S1Deq), %% Enqueue message, Msg = rabbit_fifo:make_enqueue(self(), 1, msg1), - {E1, _, _} = apply(meta(C, 2, Now), Msg, S0), + {E1, _, _} = apply(meta(C, 2, Now, {notify, 2, self()}), Msg, S0), Deq = {<<"deq1">>, self()}, {E2, _, Effs2} = apply(meta(C, 3, Now), @@ -2044,8 +2105,9 @@ expire_message_should_emit_release_cursor_test(C) -> S0 = rabbit_fifo:init(Conf), Msg = #basic_message{content = #content{properties = none, payload_fragments_rev = []}}, - {S1, ok, _} = apply(meta(C, 1, 100), rabbit_fifo:make_enqueue(self(), 1, Msg), S0), - {_S, ok, Effs} = apply(meta(C, 2, 101), + {S1, ok, _} = apply(meta(C, 1, 100, {notify, 1, self()}), + rabbit_fifo:make_enqueue(self(), 1, Msg), S0), + {_S, ok, Effs} = apply(meta(C, 2, 101, {notify, 2, self()}), rabbit_fifo:make_enqueue(self(), 2, Msg), S1), ?ASSERT_EFF({release_cursor, 1, _}, Effs), diff --git a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl index 787b60a30d00..9575b91b5488 100644 --- a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl @@ -64,6 +64,8 @@ init_per_testcase(TestCase, Config) -> meck:new(rabbit_quorum_queue, [passthrough]), meck:expect(rabbit_quorum_queue, handle_tick, fun (_, _, _) -> ok end), meck:expect(rabbit_quorum_queue, cancel_consumer_handler, fun (_, _) -> ok end), + meck:new(rabbit_feature_flags, []), + meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> true end), ra_server_sup_sup:remove_all(?RA_SYSTEM), ServerName2 = list_to_atom(atom_to_list(TestCase) ++ "2"), ServerName3 = list_to_atom(atom_to_list(TestCase) ++ "3"), @@ -89,7 +91,7 @@ basics(Config) -> ConsumerTag = UId, ok = start_cluster(ClusterName, [ServerId]), FState0 = rabbit_fifo_client:init([ServerId]), - {ok, FState1} = rabbit_fifo_client:checkout(ConsumerTag, 1, simple_prefetch, + {ok, FState1} = rabbit_fifo_client:checkout(ConsumerTag, {simple_prefetch, 1}, #{}, FState0), rabbit_quorum_queue:wal_force_roll_over(node()), @@ -180,7 +182,7 @@ duplicate_delivery(Config) -> ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), F0 = rabbit_fifo_client:init([ServerId]), - {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F0), + {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, #{}, F0), {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, corr1, msg1, F1), Fun = fun Loop(S0) -> receive @@ -215,7 +217,7 @@ usage(Config) -> ServerId = ?config(node_id, Config), ok = start_cluster(ClusterName, [ServerId]), F0 = rabbit_fifo_client:init([ServerId]), - {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F0), + {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), @@ -268,7 +270,7 @@ detects_lost_delivery(Config) -> F000 = rabbit_fifo_client:init([ServerId]), {ok, F00, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F000), {_, _, F0} = process_ra_events(receive_ra_events(1, 0), ClusterName, F00), - {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F0), + {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, #{}, F0), {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, msg2, F1), {ok, F3, []} = rabbit_fifo_client:enqueue(ClusterName, msg3, F2), % lose first delivery @@ -297,8 +299,8 @@ returns_after_down(Config) -> Self = self(), _Pid = spawn(fun () -> F = rabbit_fifo_client:init([ServerId]), - {ok, _} = rabbit_fifo_client:checkout(<<"tag">>, 10, - simple_prefetch, + {ok, _} = rabbit_fifo_client:checkout(<<"tag">>, + {simple_prefetch, 10}, #{}, F), Self ! checkout_done end), @@ -378,8 +380,8 @@ discard(Config) -> _ = ra:members(ServerId), F0 = rabbit_fifo_client:init([ServerId]), - {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, - simple_prefetch, #{}, F0), + {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, + #{}, F0), {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F1), F3 = discard_next_delivery(ClusterName, F2, 5000), {empty, _F4} = rabbit_fifo_client:dequeue(ClusterName, <<"tag1">>, settled, F3), @@ -401,11 +403,14 @@ cancel_checkout(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:checkout(<<"tag">>, 10, simple_prefetch, #{}, F1), - {_, _, F3} = process_ra_events(receive_ra_events(1, 1), ClusterName, F2, [], [], fun (_, S) -> S end), + {ok, F2} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, + #{}, F1), + {_, _, F3} = process_ra_events(receive_ra_events(1, 1), ClusterName, F2, + [], [], fun (_, S) -> S end), {ok, F4} = rabbit_fifo_client:cancel_checkout(<<"tag">>, F3), {F5, _} = rabbit_fifo_client:return(<<"tag">>, [0], F4), - {ok, _, {_, _, _, _, m1}, F5} = rabbit_fifo_client:dequeue(ClusterName, <<"d1">>, settled, F5), + {ok, _, {_, _, _, _, m1}, F5} = + rabbit_fifo_client:dequeue(ClusterName, <<"d1">>, settled, F5), ok. lost_delivery(Config) -> @@ -415,8 +420,9 @@ lost_delivery(Config) -> F0 = rabbit_fifo_client:init([ServerId], 4), {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, m1, F0), {_, _, F2} = process_ra_events( - receive_ra_events(1, 0), ClusterName, F1, [], [], fun (_, S) -> S end), - {ok, F3} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F2), + receive_ra_events(1, 0), ClusterName, F1, [], [], + fun (_, S) -> S end), + {ok, F3} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, #{}, F2), %% drop a delivery, simulating e.g. a full distribution buffer receive {ra_event, _, Evt} -> @@ -450,7 +456,7 @@ credit_api_v1(Config) -> {_, _, 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, 0, credited, #{}, F3), + {ok, F4} = rabbit_fifo_client:checkout(CTag, credited, #{}, F3), %% assert no deliveries {_, _, F5} = process_ra_events(receive_ra_events(), ClusterName, F4, [], [], fun @@ -499,7 +505,7 @@ credit_api_v2(Config) -> DC1 = 0, %% = DC0 + 1 using 32 bit serial number arithmetic {ok, F4} = rabbit_fifo_client:checkout( %% initial_delivery_count in consumer meta means credit API v2. - CTag, 0, credited, #{initial_delivery_count => DC0}, F3), + CTag, credited, #{initial_delivery_count => DC0}, F3), %% assert no deliveries {_, _, F5} = process_ra_events(receive_ra_events(), ClusterName, F4, [], [], fun @@ -598,7 +604,7 @@ test_queries(Config) -> exit(ready_timeout) end, F0 = rabbit_fifo_client:init([ServerId], 4), - {ok, _} = rabbit_fifo_client:checkout(<<"tag">>, 1, simple_prefetch, #{}, F0), + {ok, _} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 1}, #{}, F0), {ok, {_, Ready}, _} = ra:local_query(ServerId, fun rabbit_fifo:query_messages_ready/1), ?assertEqual(1, Ready), diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index 5372e0ce9280..fe99ac0cbbbf 100644 --- a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl @@ -15,6 +15,8 @@ -define(record_info(T,R),lists:zip(record_info(fields,T),tl(tuple_to_list(R)))). +-define(MACHINE_VERSION, 4). + %%%=================================================================== %%% Common Test callbacks %%%=================================================================== @@ -943,7 +945,7 @@ upgrade(_Config) -> InMemoryLength, undefined, drop_head, - {?MODULE, banana, []} + undefined ), ?FORALL(O, ?LET(Ops, log_gen(Size), expand(Ops, Config)), collect({log_size, length(O)}, @@ -1486,7 +1488,7 @@ max_length_prop(Conf0, Commands) -> #{num_ready_messages := MsgReady} = rabbit_fifo:overview(S), MsgReady =< MaxLen end, - try run_log(test_init(Conf), Entries, Invariant, rabbit_fifo) of + try run_log(test_init(Conf), Entries, Invariant) of {_State, _Effects} -> true; _ -> @@ -1532,7 +1534,7 @@ single_active_prop(Conf0, Commands, ValidateOrder) -> map_size(Up) =< 1 end, - try run_log(test_init(Conf), Entries, Invariant, rabbit_fifo) of + try run_log(test_init(Conf), Entries, Invariant) of {_State, Effects} when ValidateOrder -> %% validate message ordering lists:foldl(fun ({send_msg, Pid, {delivery, Tag, Msgs}, ra_event}, @@ -1556,7 +1558,7 @@ messages_total_prop(Conf0, Commands) -> Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), InitState = test_init(Conf), - run_log(InitState, Entries, messages_total_invariant(), rabbit_fifo), + run_log(InitState, Entries, messages_total_invariant()), true. messages_total_invariant() -> @@ -1591,7 +1593,8 @@ simple_prefetch_prop(Conf0, Commands, WithCheckoutCancel) -> Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), InitState = test_init(Conf), - run_log(InitState, Entries, simple_prefetch_invariant(WithCheckoutCancel), rabbit_fifo), + run_log(InitState, Entries, + simple_prefetch_invariant(WithCheckoutCancel)), true. simple_prefetch_invariant(WithCheckoutCancel) -> @@ -1629,19 +1632,25 @@ valid_simple_prefetch(_, _, _, _, _) -> true. upgrade_prop(Conf0, Commands) -> + FromVersion = 3, + ToVersion = 4, + FromMod = rabbit_fifo:which_module(FromVersion), + ToMod = rabbit_fifo:which_module(ToVersion), Conf = Conf0#{release_cursor_interval => 0}, Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), - InitState = test_init_v1(Conf), + InitState = test_init_v(Conf, FromVersion), [begin {PreEntries, PostEntries} = lists:split(SplitPos, Entries), %% run log v1 - {V1, _V1Effs} = run_log(InitState, PreEntries, fun (_) -> true end, - rabbit_fifo_v1), + {V1, _V1Effs} = run_log(InitState, PreEntries, + fun (_) -> true end, FromVersion), %% perform conversion - #rabbit_fifo{} = V2 = element(1, rabbit_fifo:apply(meta(length(PreEntries) + 1), - {machine_version, 1, 2}, V1)), + #rabbit_fifo{} = V2 = element(1, rabbit_fifo:apply( + meta(length(PreEntries) + 1), + {machine_version, FromVersion, ToVersion}, + V1)), %% assert invariants %% %% Note that we cannot test for num_messages because rabbit_fifo_v1:messages_total/1 @@ -1654,8 +1663,8 @@ upgrade_prop(Conf0, Commands) -> enqueue_message_bytes, checkout_message_bytes ], - V1Overview = maps:with(Fields, rabbit_fifo_v1:overview(V1)), - V2Overview = maps:with(Fields, rabbit_fifo:overview(V2)), + V1Overview = maps:with(Fields, FromMod:overview(V1)), + V2Overview = maps:with(Fields, ToMod:overview(V2)), case V1Overview == V2Overview of true -> ok; false -> @@ -1664,13 +1673,13 @@ upgrade_prop(Conf0, Commands) -> ?assertEqual(V1Overview, V2Overview) end, %% check we can run the post entries from the converted state - run_log(V2, PostEntries) + run_log(V2, PostEntries, fun (_) -> true end, ToVersion) end || SplitPos <- lists:seq(1, length(Entries))], - {_, V1Effs} = run_log(InitState, Entries, fun (_) -> true end, - rabbit_fifo_v1), + {_, V1Effs} = run_log(InitState, Entries, fun (_) -> true end, FromVersion), [begin - Res = rabbit_fifo:apply(meta(Idx + 1), {machine_version, 1, 2}, RCS) , + Res = rabbit_fifo:apply(meta(Idx + 1), + {machine_version, FromVersion, ToVersion}, RCS) , #rabbit_fifo{} = V2 = element(1, Res), %% assert invariants Fields = [num_ready_messages, @@ -1680,8 +1689,8 @@ upgrade_prop(Conf0, Commands) -> enqueue_message_bytes, checkout_message_bytes ], - V1Overview = maps:with(Fields, rabbit_fifo_v1:overview(RCS)), - V2Overview = maps:with(Fields, rabbit_fifo:overview(V2)), + V1Overview = maps:with(Fields, FromMod:overview(RCS)), + V2Overview = maps:with(Fields, ToMod:overview(V2)), case V1Overview == V2Overview of true -> ok; false -> @@ -2028,7 +2037,7 @@ handle_op({enqueue, Pid, When, Data}, Enqs = maps:update_with(Pid, fun (Seq) -> Seq + 1 end, 1, Enqs0), MsgSeq = maps:get(Pid, Enqs), {EnqSt, Msg} = Fun({EnqSt0, Data}), - Cmd = rabbit_fifo:make_enqueue(Pid, MsgSeq, Msg), + Cmd = make_enqueue(Pid, MsgSeq, Msg), case When of enqueue -> do_apply(Cmd, T#t{enqueuers = Enqs, @@ -2204,7 +2213,7 @@ run_snapshot_test0(Conf0, Commands, Invariant) -> Conf = Conf0#{max_in_memory_length => 0}, Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), - {State0, Effects} = run_log(test_init(Conf), Entries, Invariant, rabbit_fifo), + {State0, Effects} = run_log(test_init(Conf), Entries, Invariant), State = rabbit_fifo:normalize(State0), Cursors = [ C || {release_cursor, _, _} = C <- Effects], @@ -2214,7 +2223,7 @@ run_snapshot_test0(Conf0, Commands, Invariant) -> (_) -> false end, Entries), % ct:pal("release_cursor: ~b from ~w~n", [SnapIdx, element(1, hd_or(Filtered))]), - {S0, _} = run_log(SnapState, Filtered, Invariant, rabbit_fifo), + {S0, _} = run_log(SnapState, Filtered, Invariant), S = rabbit_fifo:normalize(S0), % assert log can be restored from any release cursor index case S of @@ -2239,7 +2248,7 @@ run_upgrade_snapshot_test_v1_v2(Conf, Commands) -> Entries = lists:zip(Indexes, Commands), Invariant = fun(_) -> true end, %% Run the whole command log in v1 to emit release cursors. - {_, Effects} = run_log(test_init_v1(Conf), Entries, Invariant, rabbit_fifo_v1), + {_, Effects} = run_log(test_init_v1(Conf), Entries, Invariant, 1), Cursors = [ C || {release_cursor, _, _} = C <- Effects], [begin %% Drop all entries below and including the snapshot. @@ -2251,11 +2260,11 @@ run_upgrade_snapshot_test_v1_v2(Conf, Commands) -> %% requires one additional Raft index for the conversion command from V1 to V2. FilteredV2 = lists:keymap(fun(Idx) -> Idx + 1 end, 1, FilteredV1), %% Recover in V1. - {StateV1, _} = run_log(SnapState, FilteredV1, Invariant, rabbit_fifo_v1), + {StateV1, _} = run_log(SnapState, FilteredV1, Invariant, 1), %% Perform conversion and recover in V2. Res = rabbit_fifo_v3:apply(meta(SnapIdx + 1), {machine_version, 1, 2}, SnapState), #rabbit_fifo{} = V2 = element(1, Res), - {StateV2, _} = run_log(V2, FilteredV2, Invariant, rabbit_fifo_v3, 2), + {StateV2, _} = run_log(V2, FilteredV2, Invariant, 2), %% Invariant: Recovering a V1 snapshot in V1 or V2 should end up in the same %% number of messages. Fields = [num_messages, @@ -2285,8 +2294,7 @@ run_upgrade_snapshot_test_v2_to_v3(Conf, Commands) -> Entries = lists:zip(Indexes, Commands), Invariant = fun(_) -> true end, %% Run the whole command log in v2 to emit release cursors. - {_, Effects} = run_log(test_init(rabbit_fifo_v3, Conf), Entries, Invariant, - rabbit_fifo, 2), + {_, Effects} = run_log(test_init(rabbit_fifo_v3, Conf), Entries, Invariant, 2), Cursors = [ C || {release_cursor, _, _} = C <- Effects], [begin %% Drop all entries below and including the snapshot. @@ -2298,11 +2306,11 @@ run_upgrade_snapshot_test_v2_to_v3(Conf, Commands) -> %% requires one additional Raft index for the conversion command from V2 to V3. FilteredV3 = lists:keymap(fun(Idx) -> Idx + 1 end, 1, FilteredV2), %% Recover in V2. - {StateV2, _} = run_log(SnapState, FilteredV2, Invariant, rabbit_fifo, 2), + {StateV2, _} = run_log(SnapState, FilteredV2, Invariant, 2), %% Perform conversion and recover in V3. Res = rabbit_fifo:apply(meta(SnapIdx + 1), {machine_version, 2, 3}, SnapState), #rabbit_fifo{} = V3 = element(1, Res), - {StateV3, _} = run_log(V3, FilteredV3, Invariant, rabbit_fifo, 3), + {StateV3, _} = run_log(V3, FilteredV3, Invariant, 3), %% Invariant: Recovering a V2 snapshot in V2 or V3 should end up in the same %% number of messages given that no "return", "down", or "cancel consumer" %% Ra commands are used. @@ -2337,12 +2345,13 @@ prefixes(Source, N, Acc) -> prefixes(Source, N+1, [X | Acc]). run_log(InitState, Entries) -> - run_log(InitState, Entries, fun(_) -> true end, rabbit_fifo). + run_log(InitState, Entries, fun(_) -> true end). -run_log(InitState, Entries, InvariantFun, FifoMod) -> - run_log(InitState, Entries, InvariantFun, FifoMod, 3). +run_log(InitState, Entries, InvariantFun) -> + run_log(InitState, Entries, InvariantFun, ?MACHINE_VERSION). -run_log(InitState, Entries, InvariantFun, FifoMod, MachineVersion) -> +run_log(InitState, Entries, InvariantFun, MachineVersion) + when is_integer(MachineVersion) -> Invariant = fun(E, S) -> case InvariantFun(S) of true -> ok; @@ -2350,9 +2359,17 @@ run_log(InitState, Entries, InvariantFun, FifoMod, MachineVersion) -> throw({invariant, E, S}) end end, - - lists:foldl(fun ({Idx, E}, {Acc0, Efx0}) -> - case FifoMod:apply(meta(Idx, MachineVersion), E, Acc0) of + FifoMod = rabbit_fifo:which_module(MachineVersion), + + lists:foldl(fun ({Idx, E0}, {Acc0, Efx0}) -> + {Meta, E} = case E0 of + {M1, E1} when is_map(M1) -> + M0 = meta(Idx, MachineVersion), + {maps:merge(M0, M1), E1}; + _ -> + {meta(Idx, MachineVersion), E0} + end, + case FifoMod:apply(Meta, E, Acc0) of {Acc, _, Efx} when is_list(Efx) -> Invariant(E, Acc), {Acc, Efx0 ++ Efx}; @@ -2377,6 +2394,9 @@ test_init(Mod, Conf) -> test_init_v1(Conf) -> test_init(rabbit_fifo_v1, Conf). +test_init_v(Conf, Version) -> + test_init(rabbit_fifo:which_module(Version), Conf). + meta(Idx) -> meta(Idx, 3). @@ -2389,7 +2409,7 @@ make_checkout(Cid, Spec, Meta) -> rabbit_fifo:make_checkout(Cid, Spec, Meta). make_enqueue(Pid, Seq, Msg) -> - rabbit_fifo:make_enqueue(Pid, Seq, Msg). + rabbit_fifo_v3:make_enqueue(Pid, Seq, Msg). make_settle(Cid, MsgIds) -> rabbit_fifo:make_settle(Cid, MsgIds). From 09e271ab4c78881235d244829929a736b97749fe Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 19 Mar 2024 15:33:04 +0000 Subject: [PATCH 03/45] More test refactoring and new API fixes rabbit_fifo_prop_SUITE refactoring and other fixes. fixss bzl bzl fixes --- deps/rabbit/BUILD.bazel | 1 + deps/rabbit/app.bzl | 2 +- deps/rabbit/src/rabbit_core_ff.erl | 3 +- deps/rabbit/src/rabbit_fifo.erl | 159 +-- deps/rabbit/src/rabbit_fifo.hrl | 6 +- deps/rabbit/src/rabbit_fifo_client.erl | 37 +- deps/rabbit/src/rabbit_queue_type.erl | 2 +- deps/rabbit/src/rabbit_quorum_queue.erl | 42 +- deps/rabbit/test/amqp_credit_api_v2_SUITE.erl | 2 + deps/rabbit/test/quorum_queue_SUITE.erl | 4 +- deps/rabbit/test/rabbit_fifo_SUITE.erl | 987 +++++++++--------- deps/rabbit/test/rabbit_fifo_int_SUITE.erl | 39 +- deps/rabbit/test/rabbit_fifo_prop_SUITE.erl | 614 ++++------- 13 files changed, 815 insertions(+), 1083 deletions(-) diff --git a/deps/rabbit/BUILD.bazel b/deps/rabbit/BUILD.bazel index c829b5597e3a..b151bc8c2f09 100644 --- a/deps/rabbit/BUILD.bazel +++ b/deps/rabbit/BUILD.bazel @@ -723,6 +723,7 @@ rabbitmq_suite( deps = [ "//deps/rabbit_common:erlang_app", "@proper//:erlang_app", + "@meck//:erlang_app", "@ra//:erlang_app", ], ) diff --git a/deps/rabbit/app.bzl b/deps/rabbit/app.bzl index 71aad1a6d8d4..388edccc0290 100644 --- a/deps/rabbit/app.bzl +++ b/deps/rabbit/app.bzl @@ -1295,7 +1295,7 @@ def test_suite_beam_files(name = "test_suite_beam_files"): hdrs = ["src/rabbit_fifo.hrl"], app_name = "rabbit", erlc_opts = "//:test_erlc_opts", - deps = ["//deps/rabbit_common:erlang_app"], + deps = ["//deps/rabbit_common:erlang_app", "@proper//:erlang_app"], ) erlang_bytecode( name = "rabbit_fifo_dlx_SUITE_beam_files", diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl index ea3e4f0112d2..fc1f069a225d 100644 --- a/deps/rabbit/src/rabbit_core_ff.erl +++ b/deps/rabbit/src/rabbit_core_ff.erl @@ -197,5 +197,6 @@ {quorum_queues_v4, #{desc => "Unlocks QQ v4 goodies", stability => stable, - depends_on => [quorum_queue] + depends_on => [quorum_queue, + credit_api_v2] }}). diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index f96a02f304e7..1720d3fb28e2 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -79,7 +79,8 @@ -ifdef(TEST). -export([update_header/4, - chunk_disk_msgs/3]). + chunk_disk_msgs/3, + smallest_raft_index/1]). -endif. -import(serial_number, [add/2, diff/2]). @@ -146,6 +147,7 @@ credit_mode/0, consumer_meta/0, consumer_id/0, + consumer_key/0, client_msg/0, msg/0, msg_id/0, @@ -273,12 +275,13 @@ apply(#{index := Idx} = Meta, #requeue{consumer_key = ConsumerKey, msg_id = MsgId, index = OldIdx, - header = Header0, - msg = _Msg}, + header = Header0}, #?STATE{consumers = Cons0, 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 Cons0 of #{ConsumerKey := #consumer{checked_out = Checked0} = Con0} when is_map_key(MsgId, Checked0) -> @@ -423,7 +426,7 @@ apply(#{index := Index, %% dequeue always updates last_active State0 = State00#?STATE{last_active = Ts}, %% all dequeue operations result in keeping the queue from expiring - Exists = maps:is_key(ConsumerId, Consumers), + Exists = find_consumer(ConsumerId, Consumers) /= undefined, case messages_ready(State0) of 0 -> update_smallest_raft_index(Index, {dequeue, empty}, State0, []); @@ -774,7 +777,7 @@ handle_waiting_consumer_down(Pid, waiting_consumers = WaitingConsumers0} = State0) -> % get cancel effects for down waiting consumers - Down = lists:filter(fun({{_, P}, _}) -> P =:= Pid end, + Down = lists:filter(fun({_, ?CONSUMER_PID(P)}) -> P =:= Pid end, WaitingConsumers0), Effects = lists:foldl(fun ({_ConsumerKey, Consumer}, Effects) -> ConsumerId = consumer_id(Consumer), @@ -799,8 +802,8 @@ update_waiting_consumer_status(Node, _ -> {ConsumerKey, Consumer} end - end || {{_, Pid} = ConsumerKey, Consumer} <- WaitingConsumers, - Consumer#consumer.status =/= cancelled]. + end || {ConsumerKey, ?CONSUMER_PID(Pid) = Consumer} + <- WaitingConsumers, Consumer#consumer.status =/= cancelled]. -spec state_enter(ra_server:ra_state() | eol, state()) -> ra_machine:effects(). @@ -821,8 +824,8 @@ state_enter0(leader, #?STATE{consumers = Cons, 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 || {_, P} <- maps:keys(Cons)] - ++ [P || {{_, 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]), @@ -835,12 +838,15 @@ state_enter0(leader, #?STATE{consumers = Cons, [{mod_call, Mod, Fun, Args ++ [Name]} | Effects] end; state_enter0(eol, #?STATE{enqueuers = Enqs, - consumers = Custs0, + consumers = Cons0, waiting_consumers = WaitingConsumers0}, Effects) -> - Custs = maps:fold(fun({_, P}, V, S) -> S#{P => V} end, #{}, Custs0), - WaitingConsumers1 = lists:foldl(fun({{_, P}, V}, Acc) -> Acc#{P => V} end, - #{}, WaitingConsumers0), + 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))] ++ @@ -916,11 +922,11 @@ overview(#?STATE{consumers = Cons, DlxOverview = rabbit_fifo_dlx:overview(DlxState), maps:merge(maps:merge(Overview, DlxOverview), SacOverview). --spec get_checked_out(consumer_id(), msg_id(), msg_id(), state()) -> +-spec get_checked_out(consumer_key(), msg_id(), msg_id(), state()) -> [delivery_msg()]. -get_checked_out(Cid, From, To, #?STATE{consumers = Consumers}) -> - case Consumers of - #{Cid := #consumer{checked_out = Checked}} -> +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}} @@ -973,11 +979,12 @@ handle_aux(leader, _, garbage_collection, Aux, Log, MacState) -> handle_aux(follower, _, garbage_collection, Aux, Log, MacState) -> {no_reply, force_eval_gc(Log, MacState, Aux), Log}; handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, - consumer_key = ConsumerKey}, Corr, Pid}, + consumer_key = Key}, Corr, Pid}, Aux0, Log0, #?STATE{cfg = #cfg{delivery_limit = undefined}, consumers = Consumers}) -> - case Consumers of - #{ConsumerKey := #consumer{checked_out = Checked}} -> + + case find_consumer(Key, Consumers) of + {ConsumerKey, #consumer{checked_out = Checked}} -> {Log, ToReturn} = maps:fold( fun (MsgId, ?MSG(Idx, Header), {L0, Acc}) -> @@ -1162,7 +1169,9 @@ query_messages_total(State) -> messages_total(State). query_processes(#?STATE{enqueuers = Enqs, consumers = Cons0}) -> - Cons = maps:fold(fun({_, P}, V, S) -> S#{P => V} end, #{}, Cons0), + Cons = maps:fold(fun(_, ?CONSUMER_PID(P) = V, S) -> + S#{P => V} + end, #{}, Cons0), maps:keys(maps:merge(Enqs, Cons)). @@ -1184,7 +1193,7 @@ query_consumers(#?STATE{consumers = Consumers, cfg = #cfg{consumer_strategy = ConsumerStrategy}} = State) -> ActiveActivityStatusFun = - case ConsumerStrategy of + case ConsumerStrategy of competing -> fun(_ConsumerKey, #consumer{status = Status}) -> case Status of @@ -1196,7 +1205,7 @@ query_consumers(#?STATE{consumers = Consumers, end; single_active -> SingleActiveConsumer = query_single_active_consumer(State), - fun({Tag, Pid} = _Consumer, _) -> + fun(_, ?CONSUMER_TAG_PID(Tag, Pid)) -> case SingleActiveConsumer of {value, {Tag, Pid}} -> {true, single_active}; @@ -1208,11 +1217,13 @@ query_consumers(#?STATE{consumers = Consumers, FromConsumers = maps:fold(fun (_, #consumer{status = cancelled}, Acc) -> Acc; - (Key = {Tag, Pid}, - #consumer{cfg = #consumer_cfg{meta = Meta}} = Consumer, + (Key, + #consumer{cfg = #consumer_cfg{tag = Tag, + pid = Pid, + meta = Meta}} = Consumer, Acc) -> {Active, ActivityStatus} = - ActiveActivityStatusFun(Key, Consumer), + ActiveActivityStatusFun(Key, Consumer), maps:put(Key, {Pid, Tag, maps:get(ack, Meta, undefined), @@ -1223,35 +1234,38 @@ query_consumers(#?STATE{consumers = Consumers, maps:get(username, Meta, undefined)}, Acc) end, #{}, Consumers), - FromWaitingConsumers = - lists:foldl(fun ({_, #consumer{status = cancelled}}, Acc) -> - Acc; - (Key = {{Tag, Pid}, - #consumer{cfg = #consumer_cfg{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}) -> + 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}; - {ActiveCid, _} -> - {value, ActiveCid} + {_CKey, ?CONSUMER_TAG_PID(Tag, Pid)} -> + {value, {Tag, Pid}} end; query_single_active_consumer(_) -> disabled. @@ -1374,11 +1388,16 @@ cancel_consumer(Meta, ConsumerKey, _ -> % The cancelled consumer is not active or cancelled % Just remove it from idle_consumers - Waiting = lists:keydelete(ConsumerKey, 1, Waiting0), - Effects = cancel_consumer_effects(ConsumerKey, 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} + 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}}, @@ -1405,7 +1424,7 @@ cancel_consumer0(Meta, ConsumerKey, %% 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(ConsumerKey, S, Effects2), + Effects = cancel_consumer_effects(consumer_id(Consumer), S, Effects2), {S, Effects}; _ -> %% already removed: do nothing @@ -2410,21 +2429,21 @@ message_size(Msg) -> all_nodes(#?STATE{consumers = Cons0, enqueuers = Enqs0, waiting_consumers = WaitingConsumers0}) -> - Nodes0 = maps:fold(fun({_, P}, _, Acc) -> + 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({{_, P}, _}, Acc) -> + lists:foldl(fun({_, ?CONSUMER_PID(P)}, Acc) -> Acc#{node(P) => ok} end, Nodes1, WaitingConsumers0)). all_pids_for(Node, #?STATE{consumers = Cons0, enqueuers = Enqs0, - waiting_consumers = WaitingConsumers0}) -> - Cons = maps:fold(fun({_, P}, _, Acc) + waiting_consumers = WaitingConsumers0}) -> + Cons = maps:fold(fun(_, ?CONSUMER_PID(P), Acc) when node(P) =:= Node -> [P | Acc]; (_, _, Acc) -> Acc @@ -2434,7 +2453,7 @@ all_pids_for(Node, #?STATE{consumers = Cons0, [P | Acc]; (_, _, Acc) -> Acc end, Cons, Enqs0), - lists:foldl(fun({{_, P}, _}, Acc) + lists:foldl(fun({_, ?CONSUMER_PID(P)}, Acc) when node(P) =:= Node -> [P | Acc]; (_, Acc) -> Acc @@ -2443,8 +2462,9 @@ all_pids_for(Node, #?STATE{consumers = Cons0, suspected_pids_for(Node, #?STATE{consumers = Cons0, enqueuers = Enqs0, waiting_consumers = WaitingConsumers0}) -> - Cons = maps:fold(fun({_, P}, - #consumer{status = suspected_down}, + Cons = maps:fold(fun(_Key, + #consumer{cfg = #consumer_cfg{pid = P}, + status = suspected_down}, Acc) when node(P) =:= Node -> [P | Acc]; @@ -2455,8 +2475,9 @@ suspected_pids_for(Node, #?STATE{consumers = Cons0, [P | Acc]; (_, _, Acc) -> Acc end, Cons, Enqs0), - lists:foldl(fun({{_, P}, - #consumer{status = suspected_down}}, Acc) + lists:foldl(fun({_Key, + #consumer{cfg = #consumer_cfg{pid = P}, + status = suspected_down}}, Acc) when node(P) =:= Node -> [P | Acc]; (_, Acc) -> Acc @@ -2507,8 +2528,8 @@ convert(3, To, State) -> convert(4, To, convert_v3_to_v4(State)). smallest_raft_index(#?STATE{messages = Messages, - ra_indexes = Indexes, - dlx = DlxState}) -> + ra_indexes = Indexes, + dlx = DlxState}) -> SmallestDlxRaIdx = rabbit_fifo_dlx:smallest_raft_index(DlxState), SmallestMsgsRaIdx = case lqueue:get(Messages, undefined) of ?MSG(I, _) when is_integer(I) -> diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index dc6ee204cf1b..d68170d60c8e 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -75,9 +75,9 @@ {simple_prefetch, MaxCredit :: non_neg_integer()}. %% determines how credit is replenished --type checkout_spec() :: {once | auto, Num :: non_neg_integer(), - credited, - simple_prefetch} | +-type checkout_spec() :: {once | auto, + Num :: non_neg_integer(), + credited | simple_prefetch} | {dequeue, settled | unsettled} | cancel | diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index f60f8c33978d..786db1fa88d3 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -38,6 +38,7 @@ -define(SOFT_LIMIT, 32). -define(TIMER_TIME, 10000). -define(COMMAND_TIMEOUT, 30000). +-define(UNLIMITED_PREFETCH_COUNT, 2000). %% something large for ra -type seq() :: non_neg_integer(). @@ -114,6 +115,9 @@ enqueue(QName, Correlation, Msg, cfg = #cfg{servers = Servers, timeout = Timeout}} = State0) -> %% the first publish, register and enqueuer for this process. + %% TODO: we _only_ need to pre-register an enqueuer to discover if the + %% queue overflow is `reject_publish` and the queue can accept new messages + %% if the queue does not have `reject_publish` set we can skip this step Reg = rabbit_fifo:make_register_enqueuer(self()), case ra:process_command(Servers, Reg, Timeout) of {ok, reject_publish, Leader} -> @@ -331,19 +335,32 @@ discard(ConsumerTag, [_|_] = MsgIds, state()) -> {ok, ConsumerInfos :: map(), state()} | {error | timeout, term()}. -checkout(ConsumerTag, CreditMode, Meta, +checkout(ConsumerTag, CreditMode, #{} = Meta, #state{consumers = CDels0} = State0) - when is_binary(ConsumerTag) -> + when is_binary(ConsumerTag) andalso + is_tuple(CreditMode) -> Servers = sorted_servers(State0), ConsumerId = consumer_id(ConsumerTag), - NumUnsettled = case CreditMode of - credited -> 0; + 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} -> - Num - end, - Cmd = rabbit_fifo:make_checkout(ConsumerId, - {auto, NumUnsettled, CreditMode}, - Meta), + {auto, Num, simple_prefetch} + end + end, + Cmd = rabbit_fifo:make_checkout(ConsumerId, Spec, Meta), %% ??? Ack = maps:get(ack, Meta, true), @@ -365,7 +382,7 @@ checkout(ConsumerTag, CreditMode, Meta, NextMsgId - 1 end end, - DeliveryCount = case maps:is_key(initial_delivery_count, Meta) of + DeliveryCount = case rabbit_fifo:is_v4() of true -> credit_api_v2; false -> {credit_api_v1, 0} end, diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index 325392f456f4..401c537ba30b 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -115,7 +115,7 @@ -opaque state() :: #?STATE{}. %% Delete atom 'credit_api_v1' when feature flag credit_api_v2 becomes required. --type consume_mode() :: {simple_prefetch, non_neg_integer()} | +-type consume_mode() :: {simple_prefetch, Prefetch :: non_neg_integer()} | {credited, Initial :: delivery_count() | credit_api_v1}. -type consume_spec() :: #{no_ack := boolean(), channel_pid := pid(), diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 95a83a48bf1c..22b0fbe4f9c2 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -874,29 +874,19 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) -> ConsumerTag = quorum_ctag(ConsumerTag0), %% consumer info is used to describe the consumer properties AckRequired = not NoAck, - {CreditMode, DeclaredPrefetch, ConsumerMeta0} = - case Mode of - {credited, C} -> - Meta = if C =:= credit_api_v1 -> - #{}; - is_integer(C) -> - #{initial_delivery_count => C} - end, - {credited, 0, Meta}; - {simple_prefetch = M, Declared} -> - Effective = case Declared of - 0 -> ?UNLIMITED_PREFETCH_COUNT; - _ -> Declared - end, - {{M, Effective}, Declared, #{}} - end, - ConsumerMeta = maps:merge(ConsumerMeta0, - #{ack => AckRequired, - prefetch => DeclaredPrefetch, - args => Args, - username => ActingUser}), + Prefetch = case Mode of + {simple_prefetch, Declared} -> + Declared; + _ -> + 0 + end, + + ConsumerMeta = #{ack => AckRequired, + prefetch => Prefetch, + args => Args, + username => ActingUser}, {ok, _Infos, QState} = rabbit_fifo_client:checkout(ConsumerTag, - CreditMode, ConsumerMeta, + Mode, ConsumerMeta, QState0), case single_active_consumer_on(Q) of true -> @@ -912,10 +902,10 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) -> rabbit_core_metrics:consumer_created( ChPid, ConsumerTag, ExclusiveConsume, AckRequired, QName, - DeclaredPrefetch, ActivityStatus == single_active, %% Active + Prefetch, ActivityStatus == single_active, %% Active ActivityStatus, Args), emit_consumer_created(ChPid, ConsumerTag, ExclusiveConsume, - AckRequired, QName, DeclaredPrefetch, + AckRequired, QName, Prefetch, Args, none, ActingUser), {ok, QState}; {error, Error} -> @@ -927,10 +917,10 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) -> rabbit_core_metrics:consumer_created( ChPid, ConsumerTag, ExclusiveConsume, AckRequired, QName, - DeclaredPrefetch, true, %% Active + Prefetch, true, %% Active up, Args), emit_consumer_created(ChPid, ConsumerTag, ExclusiveConsume, - AckRequired, QName, DeclaredPrefetch, + AckRequired, QName, Prefetch, Args, none, ActingUser), {ok, QState} end. diff --git a/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl b/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl index 669eb54348e9..ef25602234ce 100644 --- a/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl +++ b/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl @@ -126,6 +126,8 @@ credit_api_v2(Config) -> ?assertEqual(ok, rabbit_ct_broker_helpers:enable_feature_flag(Config, ?FUNCTION_NAME)), + ?assertEqual(ok, + rabbit_ct_broker_helpers:enable_feature_flag(Config, quorum_queues_v4)), flush(enabled_feature_flag), %% Consume with credit API v2 diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index ccba3705014a..728ebead0648 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -3284,12 +3284,14 @@ cancel_consumer_gh_3729(Config) -> ct:fail("basic.cancel_ok timeout") end, - D = #'queue.declare'{queue = QQ, passive = true, arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]}, + D = #'queue.declare'{queue = QQ, passive = true, + arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]}, F = fun() -> #'queue.declare_ok'{queue = QQ, message_count = MC, consumer_count = CC} = amqp_channel:call(Ch, D), + ct:pal("Mc ~b CC ~b", [MC, CC]), MC =:= 1 andalso CC =:= 0 end, rabbit_ct_helpers:await_condition(F, 30000), diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index f8c306760a85..1db936bfbdb8 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -90,6 +90,7 @@ test_init(Name) -> release_cursor_interval => 0}). -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}). @@ -232,7 +233,7 @@ credit_and_drain_v1_test(Config) -> {State2, _} = enq(Config, 2, 2, second, State1), %% checkout without any initial credit (like AMQP 1.0 would) {State3, _, CheckEffs} = - apply(meta(Config, 3), rabbit_fifo:make_checkout(Cid, {auto, 0, credited}, #{}), + apply(meta(Config, 3), make_checkout(Cid, {auto, 0, credited}, #{}), State2), ?ASSERT_NO_EFF({log, _, _, _}, CheckEffs), @@ -531,100 +532,108 @@ return_auto_checked_out_test(Config) -> {State0, _} = enq(Config, 2, 2, second, State00), % it first active then inactive as the consumer took on but cannot take % any more - {State1, [_Monitor, - {log, [1], Fun1, _} - ]} = check_auto(Config, Cid, 2, State0), + {State1, #{key := CKey, + next_msg_id := MsgId}, + [_Monitor, {log, [1], Fun1, _} ]} = checkout(Config, ?LINE, Cid, 1, State0), [{send_msg, _, {delivery, _, [{MsgId, _}]}, _}] = Fun1([Msg1]), % return should include another delivery {_State2, _, Effects} = apply(meta(Config, 3), - rabbit_fifo:make_return(Cid, [MsgId]), State1), + rabbit_fifo:make_return(CKey, [MsgId]), State1), [{log, [1], Fun2, _} | _] = Effects, [{send_msg, _, {delivery, _, [{_MsgId2, {#{delivery_count := 1}, first}}]}, _}] - = Fun2([Msg1]), + = Fun2([Msg1]), ok. -cancelled_checkout_empty_queue_test(C) -> - Cid = {<<"cid">>, self()}, - {State1, _} = check_auto(C, Cid, 2, test_init(test)), +cancelled_checkout_empty_queue_test(Config) -> + Cid = {?FUNCTION_NAME_B, self()}, + {State1, #{key := _CKey, + next_msg_id := _NextMsgId}, _} = + checkout(Config, ?LINE, Cid, 1, test_init(test)),%% prefetch of 1 % cancelled checkout should clear out service_queue also, else we'd get a % build up of these - {State2, _, Effects} = apply(meta(C, 3), rabbit_fifo:make_checkout(Cid, cancel, #{}), State1), + {State2, _, Effects} = apply(meta(Config, 3), + make_checkout(Cid, cancel, #{}), State1), ?assertEqual(0, map_size(State2#rabbit_fifo.consumers)), ?assertEqual(0, priority_queue:len(State2#rabbit_fifo.service_queue)), ?ASSERT_EFF({release_cursor, _, _}, Effects), ok. -cancelled_checkout_out_test(C) -> +cancelled_checkout_out_test(Config) -> Cid = {<<"cid">>, self()}, - {State00, _} = enq(C, 1, 1, first, test_init(test)), - {State0, _} = enq(C, 2, 2, second, State00), - {State1, _} = check_auto(C, Cid, 3, State0),%% prefetch of 1 + {State00, _} = enq(Config, 1, 1, first, test_init(test)), + {State0, _} = enq(Config, 2, 2, second, State00), + {State1, #{key := CKey, + next_msg_id := NextMsgId}, _} = + checkout(Config, ?LINE, Cid, 1, State0),%% prefetch of 1 % cancelled checkout should not return pending messages to queue - {State2, _, _} = apply(meta(C, 4), rabbit_fifo:make_checkout(Cid, cancel, #{}), State1), + {State2, _, _} = apply(meta(Config, 4), + rabbit_fifo:make_checkout(Cid, cancel, #{}), State1), ?assertEqual(1, lqueue:len(State2#rabbit_fifo.messages)), ?assertEqual(0, lqueue:len(State2#rabbit_fifo.returns)), ?assertEqual(0, priority_queue:len(State2#rabbit_fifo.service_queue)), {State3, {dequeue, empty}} = - apply(meta(C, 5), rabbit_fifo:make_checkout(Cid, {dequeue, settled}, #{}), State2), + apply(meta(Config, 5), make_checkout(Cid, {dequeue, settled}, #{}), State2), %% settle {State4, ok, _} = - apply(meta(C, 6), rabbit_fifo:make_settle(Cid, [0]), State3), + apply(meta(Config, 6), rabbit_fifo:make_settle(CKey, [NextMsgId]), State3), {_State, _, [{log, [2], _Fun} | _]} = - apply(meta(C, 7), rabbit_fifo:make_checkout(Cid, {dequeue, settled}, #{}), State4), + apply(meta(Config, 7), make_checkout(Cid, {dequeue, settled}, #{}), State4), ok. -down_with_noproc_consumer_returns_unsettled_test(C) -> - Cid = {<<"down_consumer_returns_unsettled_test">>, self()}, - {State0, _} = enq(C, 1, 1, second, test_init(test)), - {State1, [{monitor, process, Pid} | _]} = check(C, Cid, 2, State0), - {State2, _, _} = apply(meta(C, 3), {down, Pid, noproc}, State1), - {_State, Effects} = check(C, Cid, 4, State2), +down_with_noproc_consumer_returns_unsettled_test(Config) -> + Cid = {?FUNCTION_NAME_B, self()}, + {State0, _} = enq(Config, 1, 1, second, test_init(test)), + {State1, #{key := CKey}, + [{monitor, process, Pid} | _]} = checkout(Config, ?LINE, Cid, 1, State0), + {State2, _, _} = apply(meta(Config, 3), {down, Pid, noproc}, State1), + {_State, #{key := CKey2}, Effects} = checkout(Config, ?LINE, Cid, 1, State2), + ?assertNotEqual(CKey, CKey2), ?ASSERT_EFF({monitor, process, _}, Effects), ok. -down_with_noconnection_marks_suspect_and_node_is_monitored_test(C) -> +down_with_noconnection_marks_suspect_and_node_is_monitored_test(Config) -> Pid = spawn(fun() -> ok end), - Cid = {<<"down_with_noconnect">>, Pid}, + Cid = {?FUNCTION_NAME_B, Pid}, Self = self(), Node = node(Pid), - {State0, Effects0} = enq(C, 1, 1, second, test_init(test)), + {State0, Effects0} = enq(Config, 1, 1, second, test_init(test)), ?ASSERT_EFF({monitor, process, P}, P =:= Self, Effects0), - {State1, Effects1} = check_auto(C, Cid, 2, State0), - #consumer{credit = 0} = maps:get(Cid, State1#rabbit_fifo.consumers), + {State1, #{key := CKey}, Effects1} = checkout(Config, ?LINE, Cid, 1, State0), + #consumer{credit = 0} = maps:get(CKey, State1#rabbit_fifo.consumers), ?ASSERT_EFF({monitor, process, P}, P =:= Pid, Effects1), % monitor both enqueuer and consumer % because we received a noconnection we now need to monitor the node - {State2a, _, _} = apply(meta(C, 3), {down, Pid, noconnection}, State1), + {State2a, _, _} = apply(meta(Config, 3), {down, Pid, noconnection}, State1), #consumer{credit = 1, checked_out = Ch, - status = suspected_down} = maps:get(Cid, State2a#rabbit_fifo.consumers), + status = suspected_down} = maps:get(CKey, State2a#rabbit_fifo.consumers), ?assertEqual(#{}, Ch), %% validate consumer has credit - {State2, _, Effects2} = apply(meta(C, 3), {down, Self, noconnection}, State2a), + {State2, _, Effects2} = apply(meta(Config, 3), {down, Self, noconnection}, State2a), ?ASSERT_EFF({monitor, node, _}, Effects2), ?assertNoEffect({demonitor, process, _}, Effects2), % when the node comes up we need to retry the process monitors for the % disconnected processes - {State3, _, Effects3} = apply(meta(C, 3), {nodeup, Node}, State2), - #consumer{status = up} = maps:get(Cid, State3#rabbit_fifo.consumers), + {State3, _, Effects3} = apply(meta(Config, 3), {nodeup, Node}, State2), + #consumer{status = up} = maps:get(CKey, State3#rabbit_fifo.consumers), % try to re-monitor the suspect processes ?ASSERT_EFF({monitor, process, P}, P =:= Pid, Effects3), ?ASSERT_EFF({monitor, process, P}, P =:= Self, Effects3), ok. -down_with_noconnection_returns_unack_test(C) -> +down_with_noconnection_returns_unack_test(Config) -> Pid = spawn(fun() -> ok end), - Cid = {<<"down_with_noconnect">>, Pid}, + Cid = {?FUNCTION_NAME_B, Pid}, Msg = rabbit_fifo:make_enqueue(self(), 1, second), - {State0, _} = enq(C, 1, 1, second, test_init(test)), + {State0, _} = enq(Config, 1, 1, second, test_init(test)), ?assertEqual(1, lqueue:len(State0#rabbit_fifo.messages)), ?assertEqual(0, lqueue:len(State0#rabbit_fifo.returns)), - {State1, {_, _}} = deq(C, 2, Cid, unsettled, Msg, State0), + {State1, {_, _}} = deq(Config, 2, Cid, unsettled, Msg, State0), ?assertEqual(0, lqueue:len(State1#rabbit_fifo.messages)), ?assertEqual(0, lqueue:len(State1#rabbit_fifo.returns)), - {State2a, _, _} = apply(meta(C, 3), {down, Pid, noconnection}, State1), + {State2a, _, _} = apply(meta(Config, 3), {down, Pid, noconnection}, State1), ?assertEqual(0, lqueue:len(State2a#rabbit_fifo.messages)), ?assertEqual(1, lqueue:len(State2a#rabbit_fifo.returns)), ?assertMatch(#consumer{checked_out = Ch, @@ -633,39 +642,44 @@ down_with_noconnection_returns_unack_test(C) -> maps:get(Cid, State2a#rabbit_fifo.consumers)), ok. -down_with_noproc_enqueuer_is_cleaned_up_test(C) -> +down_with_noproc_enqueuer_is_cleaned_up_test(Config) -> State00 = test_init(test), Pid = spawn(fun() -> ok end), - {State0, _, Effects0} = apply(meta(C, 1, ?LINE, {notify, 1, Pid}), + {State0, _, Effects0} = apply(meta(Config, 1, ?LINE, {notify, 1, Pid}), rabbit_fifo:make_enqueue(Pid, 1, first), State00), ?ASSERT_EFF({monitor, process, _}, Effects0), - {State1, _, _} = apply(meta(C, 3), {down, Pid, noproc}, State0), + {State1, _, _} = apply(meta(Config, 3), {down, Pid, noproc}, State0), % ensure there are no enqueuers ?assert(0 =:= maps:size(State1#rabbit_fifo.enqueuers)), ok. -discarded_message_without_dead_letter_handler_is_removed_test(C) -> - Cid = {<<"completed_consumer_yields_demonitor_effect_test">>, self()}, - {State0, _} = enq(C, 1, 1, first, test_init(test)), - {State1, Effects1} = check_n(C, Cid, 2, 10, State0), +discarded_message_without_dead_letter_handler_is_removed_test(Config) -> + Cid = {?FUNCTION_NAME_B, self()}, + {State0, _} = enq(Config, 1, 1, first, test_init(test)), + {State1, #{key := CKey, + next_msg_id := MsgId}, Effects1} = + checkout(Config, ?LINE, Cid, 10, State0), ?ASSERT_EFF({log, [1], _Fun, _}, Effects1), - {_State2, _, Effects2} = apply(meta(C, 1), - rabbit_fifo:make_discard(Cid, [0]), State1), + {_State2, _, Effects2} = apply(meta(Config, 1), + rabbit_fifo:make_discard(CKey, [MsgId]), State1), ?ASSERT_NO_EFF({log, [1], _Fun, _}, Effects2), ok. -discarded_message_with_dead_letter_handler_emits_log_effect_test(C) -> - Cid = {<<"cid1">>, self()}, +discarded_message_with_dead_letter_handler_emits_log_effect_test(Config) -> + Cid = {?FUNCTION_NAME_B, self()}, State00 = init(#{name => test, queue_resource => rabbit_misc:r(<<"/">>, queue, <<"test">>), max_in_memory_length => 0, dead_letter_handler => {at_most_once, {somemod, somefun, [somearg]}}}), Msg1 = rabbit_fifo:make_enqueue(self(), 1, first), - {State0, _} = enq(C, 1, 1, first, State00), - {State1, Effects1} = check_n(C, Cid, 2, 10, State0), + {State0, _} = enq(Config, 1, 1, first, State00), + {State1, #{key := CKey, + next_msg_id := MsgId}, Effects1} = + checkout(Config, ?LINE, Cid, 10, State0), ?ASSERT_EFF({log, [1], _, _}, Effects1), - {_State2, _, Effects2} = apply(meta(C, 1), rabbit_fifo:make_discard(Cid, [0]), State1), + {_State2, _, Effects2} = apply(meta(Config, 1), + rabbit_fifo:make_discard(CKey, [MsgId]), State1), % assert mod call effect with appended reason and message {value, {log, [1], Fun}} = lists:search(fun (E) -> element(1, E) == log end, Effects2), @@ -676,7 +690,7 @@ get_log_eff(Effs) -> {value, Log} = lists:search(fun (E) -> element(1, E) == log end, Effs), Log. -mixed_send_msg_and_log_effects_are_correctly_ordered_test(C) -> +mixed_send_msg_and_log_effects_are_correctly_ordered_test(Config) -> Cid = {cid(?FUNCTION_NAME), self()}, State00 = init(#{name => test, queue_resource => rabbit_misc:r(<<"/">>, queue, <<"test">>), @@ -686,12 +700,11 @@ mixed_send_msg_and_log_effects_are_correctly_ordered_test(C) -> {somemod, somefun, [somearg]}}}), %% enqueue two messages Msg1 = rabbit_fifo:make_enqueue(self(), 1, first), - {State0, _} = enq(C, 1, 1, first, State00), + {State0, _} = enq(Config, 1, 1, first, State00), Msg2 = rabbit_fifo:make_enqueue(self(), 2, snd), - {State1, _} = enq(C, 2, 2, snd, State0), + {State1, _} = enq(Config, 2, 2, snd, State0), - {_State2, Effects1} = check_n(C, Cid, 3, 10, State1), - ct:pal("Effects ~w", [Effects1]), + {_State2, _, Effects1} = checkout(Config, ?LINE, Cid, 10, State1), {log, [1, 2], Fun, _} = get_log_eff(Effects1), [{send_msg, _, {delivery, _Cid, [{0,{0,first}},{1,{0,snd}}]}, [local,ra_event]}] = Fun([Msg1, Msg2]), @@ -703,17 +716,17 @@ mixed_send_msg_and_log_effects_are_correctly_ordered_test(C) -> ?ASSERT_NO_EFF({send_msg, _, _, _}, Effects1), ok. -tick_test(C) -> +tick_test(Config) -> Cid = {<<"c">>, self()}, Cid2 = {<<"c2">>, self()}, Msg1 = rabbit_fifo:make_enqueue(self(), 1, <<"fst">>), Msg2 = rabbit_fifo:make_enqueue(self(), 2, <<"snd">>), - {S0, _} = enq(C, 1, 1, <<"fst">>, test_init(?FUNCTION_NAME)), - {S1, _} = enq(C, 2, 2, <<"snd">>, S0), - {S2, {MsgId, _}} = deq(C, 3, Cid, unsettled, Msg1, S1), - {S3, {_, _}} = deq(C, 4, Cid2, unsettled, Msg2, S2), - {S4, _, _} = apply(meta(C, 5), rabbit_fifo:make_return(Cid, [MsgId]), S3), + {S0, _} = enq(Config, 1, 1, <<"fst">>, test_init(?FUNCTION_NAME)), + {S1, _} = enq(Config, 2, 2, <<"snd">>, S0), + {S2, {MsgId, _}} = deq(Config, 3, Cid, unsettled, Msg1, S1), + {S3, {_, _}} = deq(Config, 4, Cid2, unsettled, Msg2, S2), + {S4, _, _} = apply(meta(Config, 5), rabbit_fifo:make_return(Cid, [MsgId]), S3), [{aux, {handle_tick, [#resource{}, @@ -730,38 +743,38 @@ tick_test(C) -> ok. -delivery_query_returns_deliveries_test(C) -> +delivery_query_returns_deliveries_test(Config) -> Tag = atom_to_binary(?FUNCTION_NAME, utf8), Cid = {Tag, self()}, - Commands = [ - rabbit_fifo:make_checkout(Cid, {auto, 5, simple_prefetch}, #{}), - rabbit_fifo:make_enqueue(self(), 1, one), - rabbit_fifo:make_enqueue(self(), 2, two), - rabbit_fifo:make_enqueue(self(), 3, tre), - rabbit_fifo:make_enqueue(self(), 4, for) + CKey = ?LINE, + Entries = [ + {CKey, make_checkout(Cid, {auto, {simple_prefetch, 5}}, #{})}, + {?LINE, rabbit_fifo:make_enqueue(self(), 1, one)}, + {?LINE, rabbit_fifo:make_enqueue(self(), 2, two)}, + {?LINE, rabbit_fifo:make_enqueue(self(), 3, tre)}, + {?LINE, rabbit_fifo:make_enqueue(self(), 4, for)} ], - Indexes = lists:seq(1, length(Commands)), - Entries = lists:zip(Indexes, Commands), - {State, _Effects} = run_log(C, test_init(help), Entries), + {State, _Effects} = run_log(Config, test_init(help), Entries), % 3 deliveries are returned - [{0, {_, _}}] = rabbit_fifo:get_checked_out(Cid, 0, 0, State), + [{0, {_, _}}] = rabbit_fifo:get_checked_out(CKey, 0, 0, State), [_, _, _] = rabbit_fifo:get_checked_out(Cid, 1, 3, State), ok. -duplicate_delivery_test(C) -> - {State0, _} = enq(C, 1, 1, first, test_init(test)), - {#rabbit_fifo{messages = Messages} = State, _} = enq(C, 2, 1, first, State0), +duplicate_delivery_test(Config) -> + {State0, _} = enq(Config, 1, 1, first, test_init(test)), + {#rabbit_fifo{messages = Messages} = State, _} = + enq(Config, 2, 1, first, State0), ?assertEqual(1, rabbit_fifo:query_messages_total(State)), ?assertEqual(1, lqueue:len(Messages)), ok. -state_enter_monitors_and_notifications_test(C) -> +state_enter_monitors_and_notifications_test(Config) -> Oth = spawn(fun () -> ok end), - {State0, _} = enq(C, 1, 1, first, test_init(test)), + {State0, _} = enq(Config, 1, 1, first, test_init(test)), Cid = {<<"adf">>, self()}, OthCid = {<<"oth">>, Oth}, - {State1, _} = check(C, Cid, 2, State0), - {State, _} = check(C, OthCid, 3, State1), + {State1, _, _} = checkout(Config, ?LINE, Cid, 1, State0), + {State, _, _} = checkout(Config, ?LINE, OthCid, 1, State1), Self = self(), Effects = rabbit_fifo:state_enter(leader, State), @@ -779,47 +792,48 @@ state_enter_monitors_and_notifications_test(C) -> ?ASSERT_EFF({monitor, process, _}, Effects), ok. -purge_test(C) -> +purge_test(Config) -> Cid = {<<"purge_test">>, self()}, - {State1, _} = enq(C, 1, 1, first, test_init(test)), - {State2, {purge, 1}, _} = apply(meta(C, 2), rabbit_fifo:make_purge(), State1), - {State3, _} = enq(C, 3, 2, second, State2), + {State1, _} = enq(Config, 1, 1, first, test_init(test)), + {State2, {purge, 1}, _} = apply(meta(Config, 2), rabbit_fifo:make_purge(), State1), + {State3, _} = enq(Config, 3, 2, second, State2), % get returns a reply value {_State4, _, Effs} = - apply(meta(C, 4), rabbit_fifo:make_checkout(Cid, {dequeue, unsettled}, #{}), State3), + apply(meta(Config, 4), make_checkout(Cid, {dequeue, unsettled}, #{}), State3), ?ASSERT_EFF({log, [3], _}, Effs), ok. -purge_with_checkout_test(C) -> +purge_with_checkout_test(Config) -> Cid = {<<"purge_test">>, self()}, - {State0, _} = check_auto(C, Cid, 1, test_init(?FUNCTION_NAME)), - {State1, _} = enq(C, 2, 1, <<"first">>, State0), - {State2, _} = enq(C, 3, 2, <<"second">>, State1), + {State0, #{key := CKey}, _} = checkout(Config, ?LINE, Cid, 1, + test_init(?FUNCTION_NAME)), + {State1, _} = enq(Config, 2, 1, <<"first">>, State0), + {State2, _} = enq(Config, 3, 2, <<"second">>, State1), %% assert message bytes are non zero ?assert(State2#rabbit_fifo.msg_bytes_checkout > 0), ?assert(State2#rabbit_fifo.msg_bytes_enqueue > 0), - {State3, {purge, 1}, _} = apply(meta(C, 2), rabbit_fifo:make_purge(), State2), + {State3, {purge, 1}, _} = apply(meta(Config, 2), rabbit_fifo:make_purge(), State2), ?assert(State2#rabbit_fifo.msg_bytes_checkout > 0), ?assertEqual(0, State3#rabbit_fifo.msg_bytes_enqueue), ?assertEqual(1, rabbit_fifo:query_messages_total(State3)), - #consumer{checked_out = Checked} = maps:get(Cid, State3#rabbit_fifo.consumers), + #consumer{checked_out = Checked} = maps:get(CKey, State3#rabbit_fifo.consumers), ?assertEqual(1, maps:size(Checked)), ok. -down_noproc_returns_checked_out_in_order_test(C) -> +down_noproc_returns_checked_out_in_order_test(Config) -> S0 = test_init(?FUNCTION_NAME), %% enqueue 100 S1 = lists:foldl(fun (Num, FS0) -> - {FS, _} = enq(C, Num, Num, Num, FS0), + {FS, _} = enq(Config, Num, Num, Num, FS0), FS end, S0, lists:seq(1, 100)), ?assertEqual(100, lqueue:len(S1#rabbit_fifo.messages)), Cid = {<<"cid">>, self()}, - {S2, _} = check(C, Cid, 101, 1000, S1), - #consumer{checked_out = Checked} = maps:get(Cid, S2#rabbit_fifo.consumers), + {S2, #{key := CKey}, _} = checkout(Config, ?LINE, Cid, 1000, S1), + #consumer{checked_out = Checked} = maps:get(CKey, S2#rabbit_fifo.consumers), ?assertEqual(100, maps:size(Checked)), %% simulate down - {S, _, _} = apply(meta(C, 102), {down, self(), noproc}, S2), + {S, _, _} = apply(meta(Config, 102), {down, self(), noproc}, S2), Returns = lqueue:to_list(S#rabbit_fifo.returns), ?assertEqual(100, length(Returns)), ?assertEqual(0, maps:size(S#rabbit_fifo.consumers)), @@ -827,30 +841,30 @@ down_noproc_returns_checked_out_in_order_test(C) -> ?assertEqual(lists:sort(Returns), Returns), ok. -down_noconnection_returns_checked_out_test(C) -> +down_noconnection_returns_checked_out_test(Config) -> S0 = test_init(?FUNCTION_NAME), NumMsgs = 20, S1 = lists:foldl(fun (Num, FS0) -> - {FS, _} = enq(C, Num, Num, Num, FS0), + {FS, _} = enq(Config, Num, Num, Num, FS0), FS end, S0, lists:seq(1, NumMsgs)), ?assertEqual(NumMsgs, lqueue:len(S1#rabbit_fifo.messages)), Cid = {<<"cid">>, self()}, - {S2, _} = check(C, Cid, 101, 1000, S1), - #consumer{checked_out = Checked} = maps:get(Cid, S2#rabbit_fifo.consumers), + {S2, #{key := CKey}, _} = checkout(Config, ?LINE, Cid, 1000, S1), + #consumer{checked_out = Checked} = maps:get(CKey, S2#rabbit_fifo.consumers), ?assertEqual(NumMsgs, maps:size(Checked)), %% simulate down - {S, _, _} = apply(meta(C, 102), {down, self(), noconnection}, S2), + {S, _, _} = apply(meta(Config, 102), {down, self(), noconnection}, S2), Returns = lqueue:to_list(S#rabbit_fifo.returns), ?assertEqual(NumMsgs, length(Returns)), ?assertMatch(#consumer{checked_out = Ch} when map_size(Ch) == 0, - maps:get(Cid, S#rabbit_fifo.consumers)), + maps:get(CKey, S#rabbit_fifo.consumers)), %% validate returns are in order ?assertEqual(lists:sort(Returns), Returns), ok. -single_active_consumer_basic_get_test(C) -> +single_active_consumer_basic_get_test(Config) -> Cid = {?FUNCTION_NAME, self()}, State0 = init(#{name => ?FUNCTION_NAME, queue_resource => rabbit_misc:r("/", queue, @@ -859,27 +873,28 @@ single_active_consumer_basic_get_test(C) -> single_active_consumer_on => true}), ?assertEqual(single_active, State0#rabbit_fifo.cfg#cfg.consumer_strategy), ?assertEqual(0, map_size(State0#rabbit_fifo.consumers)), - {State1, _} = enq(C, 1, 1, first, State0), + {State1, _} = enq(Config, 1, 1, first, State0), {_State, {error, {unsupported, single_active_consumer}}} = - apply(meta(C, 2), rabbit_fifo:make_checkout(Cid, {dequeue, unsettled}, #{}), + apply(meta(Config, 2), make_checkout(Cid, {dequeue, unsettled}, #{}), State1), ok. -single_active_consumer_revive_test(C) -> +single_active_consumer_revive_test(Config) -> S0 = init(#{name => ?FUNCTION_NAME, queue_resource => rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME, utf8)), single_active_consumer_on => true}), Cid1 = {<<"one">>, self()}, Cid2 = {<<"two">>, self()}, - {S1, _} = check_auto(C, Cid1, 1, S0), - {S2, _} = check_auto(C, Cid2, 2, S1), - {S3, _} = enq(C, 3, 1, first, S2), + {S1, #{key := CKey1}, _} = checkout(Config, ?LINE, Cid1, 1, S0), + {S2, #{key := _CKey2}, _} = checkout(Config, ?LINE, Cid2, 1, S1), + {S3, _} = enq(Config, 3, 1, first, S2), %% cancel the active consumer whilst it has a message pending - {S4, _, _} = rabbit_fifo:apply(meta(C, 4), make_checkout(Cid1, cancel, #{}), S3), - {S5, _} = check_auto(C, Cid1, 5, S4), + {S4, _, _} = rabbit_fifo:apply(meta(Config, ?LINE), + make_checkout(Cid1, cancel, #{}), S3), + %% the revived consumer should have the original key + {S5, #{key := CKey1}, _} = checkout(Config, ?LINE, Cid1, 1, S4), - ct:pal("S5 ~tp", [S5]), ?assertEqual(1, rabbit_fifo:query_messages_checked_out(S5)), ?assertEqual(1, rabbit_fifo:query_messages_total(S5)), Consumers = S5#rabbit_fifo.consumers, @@ -890,12 +905,12 @@ single_active_consumer_revive_test(C) -> ?assertEqual(1, map_size(Up)), %% settle message and ensure it is handled correctly - {S6, _} = settle(C, Cid1, 6, 0, S5), + {S6, _} = settle(Config, CKey1, 6, 0, S5), ?assertEqual(0, rabbit_fifo:query_messages_checked_out(S6)), ?assertEqual(0, rabbit_fifo:query_messages_total(S6)), %% requeue message and check that is handled - {S6b, _} = return(C, Cid1, 6, 0, S5), + {S6b, _} = return(Config, CKey1, 6, 0, S5), ?assertEqual(1, rabbit_fifo:query_messages_checked_out(S6b)), ?assertEqual(1, rabbit_fifo:query_messages_total(S6b)), %% @@ -908,22 +923,21 @@ single_active_consumer_revive_test(C) -> single_active_consumer_revive_2_test(C) -> S0 = init(#{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME, utf8)), + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), single_active_consumer_on => true}), Cid1 = {<<"one">>, self()}, - {S1, _} = check_auto(C, Cid1, 1, S0), + {S1, #{key := CKey}, _} = checkout(C, ?LINE, Cid1, 1, S0), {S2, _} = enq(C, 3, 1, first, S1), %% cancel the active consumer whilst it has a message pending {S3, _, _} = rabbit_fifo:apply(meta(C, 4), make_checkout(Cid1, cancel, #{}), S2), - {S4, _} = check_auto(C, Cid1, 5, S3), + {S4, #{key := CKey}, _} = checkout(C, ?LINE, Cid1, 5, S3), ?assertEqual(1, rabbit_fifo:query_consumer_count(S4)), ?assertEqual(0, length(rabbit_fifo:query_waiting_consumers(S4))), ?assertEqual(1, rabbit_fifo:query_messages_total(S4)), ?assertEqual(1, rabbit_fifo:query_messages_checked_out(S4)), - ok. -single_active_consumer_test(C) -> +single_active_consumer_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, queue_resource => rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME, utf8)), @@ -933,62 +947,62 @@ single_active_consumer_test(C) -> ?assertEqual(0, map_size(State0#rabbit_fifo.consumers)), % adding some consumers - AddConsumer = fun(CTag, State) -> - {NewState, _, _} = apply( - meta(C, 1), - make_checkout({CTag, self()}, - {once, 1, simple_prefetch}, - #{}), - State), - NewState - end, - State1 = lists:foldl(AddConsumer, State0, - [<<"ctag1">>, <<"ctag2">>, <<"ctag3">>, <<"ctag4">>]), C1 = {<<"ctag1">>, self()}, C2 = {<<"ctag2">>, self()}, C3 = {<<"ctag3">>, self()}, C4 = {<<"ctag4">>, self()}, + CK1 = ?LINE, + CK2 = ?LINE, + CK3 = ?LINE, + CK4 = ?LINE, + Entries = [ + {CK1, make_checkout(C1, {once, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {once, {simple_prefetch, 1}}, #{})}, + {CK3, make_checkout(C3, {once, {simple_prefetch, 1}}, #{})}, + {CK4, make_checkout(C4, {once, {simple_prefetch, 1}}, #{})} + ], + {State1, _} = run_log(Config, State0, Entries), % the first registered consumer is the active one, the others are waiting ?assertEqual(1, map_size(State1#rabbit_fifo.consumers)), - ?assertMatch(#{C1 := _}, State1#rabbit_fifo.consumers), + ?assertMatch(#{CK1 := _}, State1#rabbit_fifo.consumers), ?assertMatch(#{single_active_consumer_id := C1, single_active_num_waiting_consumers := 3}, rabbit_fifo:overview(State1)), ?assertEqual(3, length(rabbit_fifo:query_waiting_consumers(State1))), - ?assertNotEqual(false, lists:keyfind(C2, 1, rabbit_fifo:query_waiting_consumers(State1))), - ?assertNotEqual(false, lists:keyfind(C3, 1, rabbit_fifo:query_waiting_consumers(State1))), - ?assertNotEqual(false, lists:keyfind(C4, 1, rabbit_fifo:query_waiting_consumers(State1))), + ?assertNotEqual(false, lists:keyfind(CK2, 1, rabbit_fifo:query_waiting_consumers(State1))), + ?assertNotEqual(false, lists:keyfind(CK3, 1, rabbit_fifo:query_waiting_consumers(State1))), + ?assertNotEqual(false, lists:keyfind(CK4, 1, rabbit_fifo:query_waiting_consumers(State1))), % cancelling a waiting consumer - {State2, _, Effects1} = apply(meta(C, 2), + {State2, _, Effects1} = apply(meta(Config, ?LINE), make_checkout(C3, cancel, #{}), State1), % the active consumer should still be in place ?assertEqual(1, map_size(State2#rabbit_fifo.consumers)), - ?assertMatch(#{C1 := _}, State2#rabbit_fifo.consumers), + ?assertMatch(#{CK1 := _}, State2#rabbit_fifo.consumers), % the cancelled consumer has been removed from waiting consumers ?assertMatch(#{single_active_consumer_id := C1, single_active_num_waiting_consumers := 2}, rabbit_fifo:overview(State2)), ?assertEqual(2, length(rabbit_fifo:query_waiting_consumers(State2))), - ?assertNotEqual(false, lists:keyfind(C2, 1, rabbit_fifo:query_waiting_consumers(State2))), - ?assertNotEqual(false, lists:keyfind(C4, 1, rabbit_fifo:query_waiting_consumers(State2))), + ?assertNotEqual(false, lists:keyfind(CK2, 1, rabbit_fifo:query_waiting_consumers(State2))), + ?assertNotEqual(false, lists:keyfind(CK4, 1, rabbit_fifo:query_waiting_consumers(State2))), % there are some effects to unregister the consumer ?ASSERT_EFF({mod_call, rabbit_quorum_queue, cancel_consumer_handler, [_, Con]}, Con == C3, Effects1), % cancelling the active consumer - {State3, _, Effects2} = apply(meta(C, 3), + {State3, _, Effects2} = apply(meta(Config, ?LINE), make_checkout(C1, cancel, #{}), State2), % the second registered consumer is now the active one ?assertEqual(1, map_size(State3#rabbit_fifo.consumers)), - ?assertMatch(#{C2 := _}, State3#rabbit_fifo.consumers), + ?assertMatch(#{CK2 := _}, State3#rabbit_fifo.consumers), % the new active consumer is no longer in the waiting list ?assertEqual(1, length(rabbit_fifo:query_waiting_consumers(State3))), - ?assertNotEqual(false, lists:keyfind(C4, 1, + ?assertNotEqual(false, lists:keyfind(CK4, 1, rabbit_fifo:query_waiting_consumers(State3))), %% should have a cancel consumer handler mod_call effect and %% an active new consumer effect @@ -998,12 +1012,12 @@ single_active_consumer_test(C) -> update_consumer_handler, _}, Effects2), % cancelling the active consumer - {State4, _, Effects3} = apply(meta(C, 4), + {State4, _, Effects3} = apply(meta(Config, ?LINE), make_checkout(C2, cancel, #{}), State3), % the last waiting consumer became the active one ?assertEqual(1, map_size(State4#rabbit_fifo.consumers)), - ?assertMatch(#{C4 := _}, State4#rabbit_fifo.consumers), + ?assertMatch(#{CK4 := _}, State4#rabbit_fifo.consumers), % the waiting consumer list is now empty ?assertEqual(0, length(rabbit_fifo:query_waiting_consumers(State4))), % there are some effects to unregister the consumer and @@ -1014,7 +1028,7 @@ single_active_consumer_test(C) -> update_consumer_handler, _}, Effects3), % cancelling the last consumer - {State5, _, Effects4} = apply(meta(C, 5), + {State5, _, Effects4} = apply(meta(Config, ?LINE), make_checkout(C4, cancel, #{}), State4), % no active consumer anymore @@ -1027,33 +1041,33 @@ single_active_consumer_test(C) -> ok. -single_active_consumer_cancel_consumer_when_channel_is_down_test(C) -> +single_active_consumer_cancel_consumer_when_channel_is_down_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r("/", queue, - atom_to_binary(?FUNCTION_NAME, utf8)), - release_cursor_interval => 0, - single_active_consumer_on => true}), - - DummyFunction = fun() -> ok end, - Pid1 = spawn(DummyFunction), - Pid2 = spawn(DummyFunction), - Pid3 = spawn(DummyFunction), + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + release_cursor_interval => 0, + single_active_consumer_on => true}), - [C1, C2, C3, C4] = Consumers = - [{<<"ctag1">>, Pid1}, {<<"ctag2">>, Pid2}, - {<<"ctag3">>, Pid2}, {<<"ctag4">>, Pid3}], - % adding some consumers - AddConsumer = fun({CTag, ChannelId}, State) -> - {NewState, _, _} = apply( - meta(C, 1), - make_checkout({CTag, ChannelId}, {once, 1, simple_prefetch}, #{}), - State), - NewState - end, - State1 = lists:foldl(AddConsumer, State0, Consumers), + Pid1 = spawn(fun() -> ok end), + Pid2 = spawn(fun() -> ok end), + Pid3 = spawn(fun() -> ok end), + C1 = {<<"ctag1">>, Pid1}, + C2 = {<<"ctag2">>, Pid2}, + C3 = {<<"ctag3">>, Pid2}, + C4 = {<<"ctag4">>, Pid3}, + CK1 = ?LINE, + CK2 = ?LINE, + CK3 = ?LINE, + CK4 = ?LINE, + Entries = [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, + {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{})}, + {CK4, make_checkout(C4, {auto, {simple_prefetch, 1}}, #{})} + ], + {State1, _} = run_log(Config, State0, Entries), % the channel of the active consumer goes down - {State2, _, Effects} = apply(meta(C, 2), {down, Pid1, noproc}, State1), + {State2, _, Effects} = apply(meta(Config, 2), {down, Pid1, noproc}, State1), % fell back to another consumer ?assertEqual(1, map_size(State2#rabbit_fifo.consumers)), % there are still waiting consumers @@ -1066,7 +1080,7 @@ single_active_consumer_cancel_consumer_when_channel_is_down_test(C) -> update_consumer_handler, _}, Effects), % the channel of the active consumer and a waiting consumer goes down - {State3, _, Effects2} = apply(meta(C, 3), {down, Pid2, noproc}, State2), + {State3, _, Effects2} = apply(meta(Config, ?LINE), {down, Pid2, noproc}, State2), % fell back to another consumer ?assertEqual(1, map_size(State3#rabbit_fifo.consumers)), % no more waiting consumer @@ -1080,7 +1094,8 @@ single_active_consumer_cancel_consumer_when_channel_is_down_test(C) -> update_consumer_handler, _}, Effects2), % the last channel goes down - {State4, _, Effects3} = apply(meta(C, 4), {down, Pid3, doesnotmatter}, State3), + {State4, _, Effects3} = apply(meta(Config, ?LINE), + {down, Pid3, doesnotmatter}, State3), % no more consumers ?assertEqual(0, map_size(State4#rabbit_fifo.consumers)), ?assertEqual(0, length(rabbit_fifo:query_waiting_consumers(State4))), @@ -1090,33 +1105,22 @@ single_active_consumer_cancel_consumer_when_channel_is_down_test(C) -> ok. -single_active_returns_messages_on_noconnection_test(C) -> - R = rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME, utf8)), +single_active_returns_messages_on_noconnection_test(Config) -> + R = rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), State0 = init(#{name => ?FUNCTION_NAME, queue_resource => R, release_cursor_interval => 0, single_active_consumer_on => true}), - Meta = meta(C, 1), - Nodes = [n1], - ConsumerIds = [{_, DownPid}] = - [begin - B = atom_to_binary(N, utf8), - {<<"ctag_", B/binary>>, - test_util:fake_pid(N)} - end || N <- Nodes], % adding some consumers - State1 = lists:foldl( - fun(CId, Acc0) -> - {Acc, _, _} = - apply(Meta, - make_checkout(CId, - {auto, 1, simple_prefetch}, #{}), - Acc0), - Acc - end, State0, ConsumerIds), - {State2, _} = enq(C, 4, 1, msg1, State1), + {CK1, {_, DownPid} = C1} = {?LINE, {?LINE_B, test_util:fake_pid(n1)}}, + Entries = [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})} + ], + {State1, _} = run_log(Config, State0, Entries), + {State2, _} = enq(Config, 4, 1, msg1, State1), % simulate node goes down - {State3, _, _} = apply(meta(C, 5), {down, DownPid, noconnection}, State2), + {State3, _, _} = apply(meta(Config, ?LINE), {down, DownPid, noconnection}, State2), + ct:pal("state3 ~p", [State3]), %% assert the consumer is up ?assertMatch([_], lqueue:to_list(State3#rabbit_fifo.returns)), ?assertMatch([{_, #consumer{checked_out = Checked, @@ -1126,56 +1130,47 @@ single_active_returns_messages_on_noconnection_test(C) -> ok. -single_active_consumer_replaces_consumer_when_down_noconnection_test(C) -> +single_active_consumer_replaces_consumer_when_down_noconnection_test(Config) -> R = rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME, utf8)), State0 = init(#{name => ?FUNCTION_NAME, queue_resource => R, release_cursor_interval => 0, single_active_consumer_on => true}), - Meta = meta(C, 1), - Nodes = [n1, n2, node()], - ConsumerIds = [C1 = {_, DownPid}, C2, _C3] = - [begin - B = atom_to_binary(N, utf8), - {<<"ctag_", B/binary>>, - test_util:fake_pid(N)} - end || N <- Nodes], - % adding some consumers - State1a = lists:foldl( - fun(CId, Acc0) -> - {Acc, _, _} = - apply(Meta, - make_checkout(CId, - {once, 1, simple_prefetch}, #{}), - Acc0), - Acc - end, State0, ConsumerIds), + {CK1, {_, DownPid} = C1} = {?LINE, {?LINE_B, test_util:fake_pid(n1)}}, + {CK2, C2} = {?LINE, {?LINE_B, test_util:fake_pid(n2)}}, + {CK3, C3} = {?LINE, {?LINE_B, test_util:fake_pid(n3)}}, + Entries = [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, + {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{})}, + {?LINE, rabbit_fifo:make_enqueue(self(), 1, msg)} + ], + {State1, _} = run_log(Config, State0, Entries), %% assert the consumer is up - ?assertMatch(#{C1 := #consumer{status = up}}, - State1a#rabbit_fifo.consumers), - - {State1, _} = enq(C, 10, 1, msg, State1a), + ?assertMatch(#{CK1 := #consumer{status = up}}, + State1#rabbit_fifo.consumers), % simulate node goes down - {State2, _, _} = apply(meta(C, 5), {down, DownPid, noconnection}, State1), + {State2, _, _} = apply(meta(Config, ?LINE), + {down, DownPid, noconnection}, State1), %% assert a new consumer is in place and it is up - ?assertMatch([{C2, #consumer{status = up, - checked_out = Ch}}] + ?assertMatch([{CK2, #consumer{status = up, + checked_out = Ch}}] when map_size(Ch) == 1, maps:to_list(State2#rabbit_fifo.consumers)), %% the disconnected consumer has been returned to waiting - ?assert(lists:any(fun ({Con,_}) -> Con =:= C1 end, + ?assert(lists:any(fun ({Con, _}) -> Con =:= CK1 end, rabbit_fifo:query_waiting_consumers(State2))), ?assertEqual(2, length(rabbit_fifo:query_waiting_consumers(State2))), % simulate node comes back up - {State3, _, _} = apply(meta(C, 2), {nodeup, node(DownPid)}, State2), + {State3, _, _} = apply(meta(Config, 2), {nodeup, node(DownPid)}, State2), %% the consumer is still active and the same as before - ?assertMatch([{C2, #consumer{status = up}}], + ?assertMatch([{CK2, #consumer{status = up}}], maps:to_list(State3#rabbit_fifo.consumers)), % the waiting consumers should be un-suspected ?assertEqual(2, length(rabbit_fifo:query_waiting_consumers(State3))), @@ -1184,190 +1179,176 @@ single_active_consumer_replaces_consumer_when_down_noconnection_test(C) -> end, rabbit_fifo:query_waiting_consumers(State3)), ok. -single_active_consumer_all_disconnected_test(C) -> +single_active_consumer_all_disconnected_test(Config) -> R = rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME, utf8)), State0 = init(#{name => ?FUNCTION_NAME, queue_resource => R, release_cursor_interval => 0, single_active_consumer_on => true}), - Meta = meta(C, 1), - Nodes = [n1, n2], - ConsumerIds = [C1 = {_, C1Pid}, C2 = {_, C2Pid}] = - [begin - B = atom_to_binary(N, utf8), - {<<"ctag_", B/binary>>, - test_util:fake_pid(N)} - end || N <- Nodes], - % adding some consumers - State1 = lists:foldl( - fun(CId, Acc0) -> - {Acc, _, _} = - apply(Meta, - make_checkout(CId, - {once, 1, simple_prefetch}, #{}), - Acc0), - Acc - end, State0, ConsumerIds), + + {CK1, {_, C1Pid} = C1} = {?LINE, {?LINE_B, test_util:fake_pid(n1)}}, + {CK2, {_, C2Pid} = C2} = {?LINE, {?LINE_B, test_util:fake_pid(n2)}}, + Entries = [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})} + ], + {State1, _} = run_log(Config, State0, Entries), %% assert the consumer is up - ?assertMatch(#{C1 := #consumer{status = up}}, State1#rabbit_fifo.consumers), + ?assertMatch(#{CK1 := #consumer{status = up}}, State1#rabbit_fifo.consumers), % simulate node goes down - {State2, _, _} = apply(meta(C, 5), {down, C1Pid, noconnection}, State1), + {State2, _, _} = apply(meta(Config, 5), {down, C1Pid, noconnection}, State1), %% assert the consumer fails over to the consumer on n2 - ?assertMatch(#{C2 := #consumer{status = up}}, State2#rabbit_fifo.consumers), - {State3, _, _} = apply(meta(C, 6), {down, C2Pid, noconnection}, State2), + ?assertMatch(#{CK2 := #consumer{status = up}}, State2#rabbit_fifo.consumers), + {State3, _, _} = apply(meta(Config, 6), {down, C2Pid, noconnection}, State2), %% assert these no active consumer after both nodes are maked as down ?assertMatch([], maps:to_list(State3#rabbit_fifo.consumers)), %% n2 comes back - {State4, _, _} = apply(meta(C, 7), {nodeup, node(C2Pid)}, State3), + {State4, _, _} = apply(meta(Config, 7), {nodeup, node(C2Pid)}, State3), %% ensure n2 is the active consumer as this node as been registered %% as up again - ?assertMatch([{{<<"ctag_n2">>, _}, #consumer{status = up, - credit = 1}}], + ?assertMatch([{CK2, #consumer{status = up, + credit = 1}}], maps:to_list(State4#rabbit_fifo.consumers)), ok. -single_active_consumer_state_enter_leader_include_waiting_consumers_test(C) -> +single_active_consumer_state_enter_leader_include_waiting_consumers_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, - queue_resource => - rabbit_misc:r("/", queue, - atom_to_binary(?FUNCTION_NAME, utf8)), + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), release_cursor_interval => 0, single_active_consumer_on => true}), - DummyFunction = fun() -> ok end, - Pid1 = spawn(DummyFunction), - Pid2 = spawn(DummyFunction), - Pid3 = spawn(DummyFunction), - - Meta = meta(C, 1), - % adding some consumers - AddConsumer = fun({CTag, ChannelId}, State) -> - {NewState, _, _} = apply( - Meta, - make_checkout({CTag, ChannelId}, - {once, 1, simple_prefetch}, #{}), - State), - NewState - end, - State1 = lists:foldl(AddConsumer, State0, [{<<"ctag1">>, Pid1}, - {<<"ctag2">>, Pid2}, - {<<"ctag3">>, Pid2}, - {<<"ctag4">>, Pid3}]), - + Pid1 = spawn(fun() -> ok end), + Pid2 = spawn(fun() -> ok end), + Pid3 = spawn(fun() -> ok end), + C1 = {<<"ctag1">>, Pid1}, + C2 = {<<"ctag2">>, Pid2}, + C3 = {<<"ctag3">>, Pid2}, + C4 = {<<"ctag4">>, Pid3}, + CK1 = ?LINE, + CK2 = ?LINE, + CK3 = ?LINE, + CK4 = ?LINE, + Entries = [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, + {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{})}, + {CK4, make_checkout(C4, {auto, {simple_prefetch, 1}}, #{})} + ], + {State1, _} = run_log(Config, State0, Entries), Effects = rabbit_fifo:state_enter(leader, State1), %% 2 effects for each consumer process (channel process), 1 effect for the node, ?assertEqual(2 * 3 + 1 + 1 + 1, length(Effects)). -single_active_consumer_state_enter_eol_include_waiting_consumers_test(C) -> - Resource = rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME, utf8)), +single_active_consumer_state_enter_eol_include_waiting_consumers_test(Config) -> + Resource = rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), State0 = init(#{name => ?FUNCTION_NAME, queue_resource => Resource, release_cursor_interval => 0, single_active_consumer_on => true}), - DummyFunction = fun() -> ok end, - Pid1 = spawn(DummyFunction), - Pid2 = spawn(DummyFunction), - Pid3 = spawn(DummyFunction), - - Meta = meta(C, 1), - % adding some consumers - AddConsumer = fun({CTag, ChannelId}, State) -> - {NewState, _, _} = apply( - Meta, - make_checkout({CTag, ChannelId}, - {once, 1, simple_prefetch}, #{}), - State), - NewState - end, - State1 = lists:foldl(AddConsumer, State0, - [{<<"ctag1">>, Pid1}, {<<"ctag2">>, Pid2}, - {<<"ctag3">>, Pid2}, {<<"ctag4">>, Pid3}]), - + Pid1 = spawn(fun() -> ok end), + Pid2 = spawn(fun() -> ok end), + Pid3 = spawn(fun() -> ok end), + {CK1, C1} = {?LINE, {?LINE_B, Pid1}}, + {CK2, C2} = {?LINE, {?LINE_B, Pid2}}, + {CK3, C3} = {?LINE, {?LINE_B, Pid2}}, + {CK4, C4} = {?LINE, {?LINE_B, Pid3}}, + Entries = [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, + {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{})}, + {CK4, make_checkout(C4, {auto, {simple_prefetch, 1}}, #{})} + ], + {State1, _} = run_log(Config, State0, Entries), Effects = rabbit_fifo:state_enter(eol, State1), %% 1 effect for each consumer process (channel process), %% 1 effect for eol to handle rabbit_fifo_usage entries - ?assertEqual(4, length(Effects)). + ?assertEqual(4, length(Effects)), + ok. -query_consumers_test(C) -> +query_consumers_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, queue_resource => rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME, utf8)), release_cursor_interval => 0, single_active_consumer_on => false}), - % adding some consumers - AddConsumer = fun(CTag, State) -> - {NewState, _, _} = apply( - meta(C, 1), - make_checkout({CTag, self()}, - {once, 1, simple_prefetch}, #{}), - State), - NewState - end, - State1 = lists:foldl(AddConsumer, State0, [<<"ctag1">>, <<"ctag2">>, <<"ctag3">>, <<"ctag4">>]), + % % adding some consumers + {CK1, C1} = {?LINE, {?LINE_B, self()}}, + {CK2, C2} = {?LINE, {?LINE_B, self()}}, + {CK3, C3} = {?LINE, {?LINE_B, self()}}, + {CK4, C4} = {?LINE, {?LINE_B, self()}}, + Entries = [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, + {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{})}, + {CK4, make_checkout(C4, {auto, {simple_prefetch, 1}}, #{})} + ], + {State1, _} = run_log(Config, State0, Entries), Consumers0 = State1#rabbit_fifo.consumers, - Consumer = maps:get({<<"ctag2">>, self()}, Consumers0), - Consumers1 = maps:put({<<"ctag2">>, self()}, - Consumer#consumer{status = suspected_down}, Consumers0), + Consumer = maps:get(CK2, Consumers0), + Consumers1 = maps:put(CK2, Consumer#consumer{status = suspected_down}, + Consumers0), State2 = State1#rabbit_fifo{consumers = Consumers1}, ?assertEqual(3, rabbit_fifo:query_consumer_count(State2)), Consumers2 = rabbit_fifo:query_consumers(State2), ?assertEqual(4, maps:size(Consumers2)), - maps:fold(fun(_Key, {Pid, Tag, _, _, Active, ActivityStatus, _, _}, _Acc) -> - ?assertEqual(self(), Pid), - case Tag of - <<"ctag2">> -> - ?assertNot(Active), - ?assertEqual(suspected_down, ActivityStatus); - _ -> - ?assert(Active), - ?assertEqual(up, ActivityStatus) - end - end, [], Consumers2). - -query_consumers_when_single_active_consumer_is_on_test(C) -> + maps:fold(fun(Key, {Pid, _Tag, _, _, Active, ActivityStatus, _, _}, _Acc) -> + ?assertEqual(self(), Pid), + case Key of + CK2 -> + ?assertNot(Active), + ?assertEqual(suspected_down, ActivityStatus); + _ -> + ?assert(Active), + ?assertEqual(up, ActivityStatus) + end + end, [], Consumers2), + ok. + +query_consumers_when_single_active_consumer_is_on_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r("/", queue, - atom_to_binary(?FUNCTION_NAME, utf8)), + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), release_cursor_interval => 0, single_active_consumer_on => true}), - Meta = meta(C, 1), - % adding some consumers - AddConsumer = fun(CTag, State) -> - {NewState, _, _} = apply( - Meta, - make_checkout({CTag, self()}, - {once, 1, simple_prefetch}, #{}), - State), - NewState - end, - State1 = lists:foldl(AddConsumer, State0, [<<"ctag1">>, <<"ctag2">>, <<"ctag3">>, <<"ctag4">>]), + % % adding some consumers + {CK1, C1} = {?LINE, {?LINE_B, self()}}, + {CK2, C2} = {?LINE, {?LINE_B, self()}}, + {CK3, C3} = {?LINE, {?LINE_B, self()}}, + {CK4, C4} = {?LINE, {?LINE_B, self()}}, + Entries = [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, + {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{})}, + {CK4, make_checkout(C4, {auto, {simple_prefetch, 1}}, #{})} + ], + {State1, _} = run_log(Config, State0, Entries), ?assertEqual(4, rabbit_fifo:query_consumer_count(State1)), Consumers = rabbit_fifo:query_consumers(State1), ?assertEqual(4, maps:size(Consumers)), - maps:fold(fun(_Key, {Pid, Tag, _, _, Active, ActivityStatus, _, _}, _Acc) -> + maps:fold(fun(Key, {Pid, _Tag, _, _, Active, ActivityStatus, _, _}, _Acc) -> ?assertEqual(self(), Pid), - case Tag of - <<"ctag1">> -> + case Key of + CK1 -> ?assert(Active), ?assertEqual(single_active, ActivityStatus); _ -> ?assertNot(Active), ?assertEqual(waiting, ActivityStatus) end - end, [], Consumers). + end, [], Consumers), + ok. -active_flag_updated_when_consumer_suspected_unsuspected_test(C) -> +active_flag_updated_when_consumer_suspected_unsuspected_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r("/", queue, - atom_to_binary(?FUNCTION_NAME, utf8)), - release_cursor_interval => 0, - single_active_consumer_on => false}), + queue_resource => rabbit_misc:r("/", queue, + ?FUNCTION_NAME_B), + release_cursor_interval => 0, + single_active_consumer_on => false}), DummyFunction = fun() -> ok end, Pid1 = spawn(DummyFunction), @@ -1375,32 +1356,34 @@ active_flag_updated_when_consumer_suspected_unsuspected_test(C) -> Pid3 = spawn(DummyFunction), % adding some consumers - AddConsumer = fun({CTag, ChannelId}, State) -> - {NewState, _, _} = - apply( - meta(C, 1), - rabbit_fifo:make_checkout({CTag, ChannelId}, - {once, 1, simple_prefetch}, - #{}), - State), - NewState - end, - State1 = lists:foldl(AddConsumer, State0, - [{<<"ctag1">>, Pid1}, {<<"ctag2">>, Pid2}, {<<"ctag3">>, Pid2}, {<<"ctag4">>, Pid3}]), - - {State2, _, Effects2} = apply(meta(C, 3), + {CK1, C1} = {?LINE, {?LINE_B, Pid1}}, + {CK2, C2} = {?LINE, {?LINE_B, Pid2}}, + {CK3, C3} = {?LINE, {?LINE_B, Pid2}}, + {CK4, C4} = {?LINE, {?LINE_B, Pid3}}, + Entries = [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, + {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{})}, + {CK4, make_checkout(C4, {auto, {simple_prefetch, 1}}, #{})} + ], + {State1, _} = run_log(Config, State0, Entries), + {State2, _, Effects2} = apply(meta(Config, 3), {down, Pid1, noconnection}, State1), - % 1 effect to update the metrics of each consumer (they belong to the same node), 1 more effect to monitor the node, 1 more decorators effect + % 1 effect to update the metrics of each consumer + % (they belong to the same node), + % 1 more effect to monitor the node, + % 1 more decorators effect ?assertEqual(4 + 1, length(Effects2)), - {_, _, Effects3} = apply(meta(C, 4), {nodeup, node(self())}, State2), - % for each consumer: 1 effect to update the metrics, 1 effect to monitor the consumer PID, 1 more decorators effect - ?assertEqual(4 + 4, length(Effects3)). + {_, _, Effects3} = apply(meta(Config, 4), {nodeup, node(self())}, State2), + % for each consumer: 1 effect to update the metrics, + % 1 effect to monitor the consumer PID, 1 more decorators effect + ?assertEqual(4 + 4, length(Effects3)), + ok. -active_flag_not_updated_when_consumer_suspected_unsuspected_and_single_active_consumer_is_on_test(C) -> +active_flag_not_updated_when_consumer_suspected_unsuspected_and_single_active_consumer_is_on_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r("/", queue, - atom_to_binary(?FUNCTION_NAME, utf8)), + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), release_cursor_interval => 0, single_active_consumer_on => true}), @@ -1410,109 +1393,98 @@ active_flag_not_updated_when_consumer_suspected_unsuspected_and_single_active_co Pid3 = spawn(DummyFunction), % adding some consumers - AddConsumer = fun({CTag, ChannelId}, State) -> - {NewState, _, _} = apply( - meta(C, 1), - make_checkout({CTag, ChannelId}, - {once, 1, simple_prefetch}, #{}), - State), - NewState - end, - State1 = lists:foldl(AddConsumer, State0, - [{<<"ctag1">>, Pid1}, {<<"ctag2">>, Pid2}, - {<<"ctag3">>, Pid2}, {<<"ctag4">>, Pid3}]), - - {State2, _, Effects2} = apply(meta(C, 2), {down, Pid1, noconnection}, State1), + {CK1, C1} = {?LINE, {?LINE_B, Pid1}}, + {CK2, C2} = {?LINE, {?LINE_B, Pid2}}, + {CK3, C3} = {?LINE, {?LINE_B, Pid2}}, + {CK4, C4} = {?LINE, {?LINE_B, Pid3}}, + Entries = [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, + {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{})}, + {CK4, make_checkout(C4, {auto, {simple_prefetch, 1}}, #{})} + ], + {State1, _} = run_log(Config, State0, Entries), + {State2, _, Effects2} = apply(meta(Config, 2), {down, Pid1, noconnection}, State1), % one monitor and one consumer status update (deactivated) ?assertEqual(2, length(Effects2)), - {_, _, Effects3} = apply(meta(C, 3), {nodeup, node(self())}, State2), + {_, _, Effects3} = apply(meta(Config, 3), {nodeup, node(self())}, State2), % for each consumer: 1 effect to monitor the consumer PID - ?assertEqual(5, length(Effects3)). + ?assertEqual(5, length(Effects3)), + ok. -single_active_cancelled_with_unacked_test(C) -> +single_active_cancelled_with_unacked_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r("/", queue, - atom_to_binary(?FUNCTION_NAME, utf8)), + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), release_cursor_interval => 0, single_active_consumer_on => true}), - C1 = {<<"ctag1">>, self()}, - C2 = {<<"ctag2">>, self()}, - % adding some consumers - AddConsumer = fun(Con, S0) -> - {S, _, _} = apply( - meta(C, 1), - make_checkout(Con, - {auto, 1, simple_prefetch}, - #{}), - S0), - S - end, - State1 = lists:foldl(AddConsumer, State0, [C1, C2]), + % % adding some consumers + {CK1, C1} = {?LINE, {?LINE_B, self()}}, + {CK2, C2} = {?LINE, {?LINE_B, self()}}, + Entries = [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})} + ], + {State1, _} = run_log(Config, State0, Entries), %% enqueue 2 messages - {State2, _Effects2} = enq(C, 3, 1, msg1, State1), - {State3, _Effects3} = enq(C, 4, 2, msg2, State2), + {State2, _Effects2} = enq(Config, 3, 1, msg1, State1), + {State3, _Effects3} = enq(Config, 4, 2, msg2, State2), %% one should be checked ou to C1 %% cancel C1 - {State4, _, _} = apply(meta(C, 5), + {State4, _, _} = apply(meta(Config, ?LINE), make_checkout(C1, cancel, #{}), State3), %% C2 should be the active consumer - ?assertMatch(#{C2 := #consumer{status = up, - checked_out = #{0 := _}}}, + ?assertMatch(#{CK2 := #consumer{status = up, + checked_out = #{0 := _}}}, State4#rabbit_fifo.consumers), %% C1 should be a cancelled consumer - ?assertMatch(#{C1 := #consumer{status = cancelled, - cfg = #consumer_cfg{lifetime = once}, - checked_out = #{0 := _}}}, + ?assertMatch(#{CK1 := #consumer{status = cancelled, + cfg = #consumer_cfg{lifetime = once}, + checked_out = #{0 := _}}}, State4#rabbit_fifo.consumers), ?assertMatch([], rabbit_fifo:query_waiting_consumers(State4)), %% Ack both messages - {State5, _Effects5} = settle(C, C1, 1, 0, State4), + {State5, _Effects5} = settle(Config, CK1, ?LINE, 0, State4), %% C1 should now be cancelled - {State6, _Effects6} = settle(C, C2, 2, 0, State5), + {State6, _Effects6} = settle(Config, CK2, ?LINE, 0, State5), %% C2 should remain - ?assertMatch(#{C2 := #consumer{status = up}}, + ?assertMatch(#{CK2 := #consumer{status = up}}, State6#rabbit_fifo.consumers), %% C1 should be gone - ?assertNotMatch(#{C1 := _}, + ?assertNotMatch(#{CK1 := _}, State6#rabbit_fifo.consumers), ?assertMatch([], rabbit_fifo:query_waiting_consumers(State6)), ok. -single_active_with_credited_v1_test(C) -> +single_active_with_credited_v1_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME)), + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), release_cursor_interval => 0, single_active_consumer_on => true}), - C1 = {<<"ctag1">>, self()}, - C2 = {<<"ctag2">>, self()}, - % adding some consumers - AddConsumer = fun(Con, S0) -> - {S, _, _} = apply( - meta(C, 1), - make_checkout(Con, - {auto, 0, credited}, - #{}), - S0), - S - end, - State1 = lists:foldl(AddConsumer, State0, [C1, C2]), + % % adding some consumers + {CK1, C1} = {?LINE, {?LINE_B, self()}}, + {CK2, C2} = {?LINE, {?LINE_B, self()}}, + Entries = [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})} + ], + {State1, _} = run_log(Config, State0, Entries), %% add some credit - C1Cred = rabbit_fifo:make_credit(C1, 5, 0, false), - {State2, _, _Effects2} = apply(meta(C, 3), C1Cred, State1), - C2Cred = rabbit_fifo:make_credit(C2, 4, 0, false), - {State3, _} = apply(meta(C, 4), C2Cred, State2), + C1Cred = rabbit_fifo:make_credit(CK1, 5, 0, false), + {State2, _, _Effects2} = apply(meta(Config, ?LINE), C1Cred, State1), + C2Cred = rabbit_fifo:make_credit(CK2, 4, 0, false), + {State3, _} = apply(meta(Config, ?LINE), C2Cred, State2), %% both consumers should have credit - ?assertMatch(#{C1 := #consumer{credit = 5}}, + ?assertMatch(#{CK1 := #consumer{credit = 5}}, State3#rabbit_fifo.consumers), - ?assertMatch([{C2, #consumer{credit = 4}}], + ?assertMatch([{CK2, #consumer{credit = 4}}], rabbit_fifo:query_waiting_consumers(State3)), ok. @@ -1554,30 +1526,28 @@ single_active_with_credited_v2_test(Config) -> rabbit_fifo:query_waiting_consumers(State)), ok. -register_enqueuer_test(C) -> +register_enqueuer_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r("/", queue, - atom_to_binary(?FUNCTION_NAME, utf8)), + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), max_length => 2, max_in_memory_length => 0, overflow_strategy => reject_publish}), %% simply registering should be ok when we're below limit Pid1 = test_util:fake_pid(node()), - {State1, ok, [_]} = apply(meta(C, 1, ?LINE, {notify, 1, Pid1}), + {State1, ok, [_]} = apply(meta(Config, 1, ?LINE, {notify, 1, Pid1}), make_register_enqueuer(Pid1), State0), - {State2, ok, _} = apply(meta(C, 2, ?LINE, {notify, 2, Pid1}), + {State2, ok, _} = apply(meta(Config, 2, ?LINE, {notify, 2, Pid1}), rabbit_fifo:make_enqueue(Pid1, 1, one), State1), %% register another enqueuer shoudl be ok Pid2 = test_util:fake_pid(node()), - {State3, ok, [_]} = apply(meta(C, 3, ?LINE, {notify, 3, Pid2}), + {State3, ok, [_]} = apply(meta(Config, 3, ?LINE, {notify, 3, Pid2}), make_register_enqueuer(Pid2), State2), - {State4, ok, _} = apply(meta(C, 4, ?LINE, {notify, 4, Pid1}), + {State4, ok, _} = apply(meta(Config, 4, ?LINE, {notify, 4, Pid1}), rabbit_fifo:make_enqueue(Pid1, 2, two), State3), - {State5, ok, Efx} = apply(meta(C, 5, ?LINE, {notify, 4, Pid1}), + {State5, ok, Efx} = apply(meta(Config, 5, ?LINE, {notify, 4, Pid1}), rabbit_fifo:make_enqueue(Pid1, 3, three), State4), - % ct:pal("Efx ~tp", [Efx]), %% validate all registered enqueuers are notified of overflow state ?ASSERT_EFF({send_msg, P, {queue_status, reject_publish}, [ra_event]}, P == Pid1, Efx), @@ -1586,30 +1556,27 @@ register_enqueuer_test(C) -> %% this time, registry should return reject_publish {State6, reject_publish, [_]} = - apply(meta(C, 6), make_register_enqueuer( - test_util:fake_pid(node())), State5), + apply(meta(Config, 6), make_register_enqueuer( + test_util:fake_pid(node())), State5), ?assertMatch(#{num_enqueuers := 3}, rabbit_fifo:overview(State6)), - Pid3 = test_util:fake_pid(node()), %% remove two messages this should make the queue fall below the 0.8 limit {State7, _, Efx7} = - apply(meta(C, 7), + apply(meta(Config, 7), rabbit_fifo:make_checkout({<<"a">>, Pid3}, {dequeue, settled}, #{}), State6), ?ASSERT_EFF({log, [_], _}, Efx7), - % ct:pal("Efx7 ~tp", [_Efx7]), {State8, _, Efx8} = - apply(meta(C, 8), + apply(meta(Config, 8), rabbit_fifo:make_checkout({<<"a">>, Pid3}, {dequeue, settled}, #{}), State7), ?ASSERT_EFF({log, [_], _}, Efx8), - % ct:pal("Efx8 ~tp", [Efx8]), %% validate all registered enqueuers are notified of overflow state ?ASSERT_EFF({send_msg, P, {queue_status, go}, [ra_event]}, P == Pid1, Efx8), ?ASSERT_EFF({send_msg, P, {queue_status, go}, [ra_event]}, P == Pid2, Efx8), {_State9, _, Efx9} = - apply(meta(C, 9), + apply(meta(Config, 9), rabbit_fifo:make_checkout({<<"a">>, Pid3}, {dequeue, settled}, #{}), State8), ?ASSERT_EFF({log, [_], _}, Efx9), @@ -1617,30 +1584,29 @@ register_enqueuer_test(C) -> ?ASSERT_NO_EFF({send_msg, P, go, [ra_event]}, P == Pid2, Efx9), ok. -reject_publish_purge_test(C) -> +reject_publish_purge_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r("/", queue, - atom_to_binary(?FUNCTION_NAME, utf8)), + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), max_length => 2, max_in_memory_length => 0, overflow_strategy => reject_publish}), %% simply registering should be ok when we're below limit Pid1 = test_util:fake_pid(node()), - {State1, ok, [_]} = apply(meta(C, 1), make_register_enqueuer(Pid1), State0), - {State2, ok, _} = apply(meta(C, 2, ?LINE, {notify, 2, Pid1}), + {State1, ok, [_]} = apply(meta(Config, 1), make_register_enqueuer(Pid1), State0), + {State2, ok, _} = apply(meta(Config, 2, ?LINE, {notify, 2, Pid1}), rabbit_fifo:make_enqueue(Pid1, 1, one), State1), - {State3, ok, _} = apply(meta(C, 3, ?LINE, {notify, 2, Pid1}), + {State3, ok, _} = apply(meta(Config, 3, ?LINE, {notify, 2, Pid1}), rabbit_fifo:make_enqueue(Pid1, 2, two), State2), - {State4, ok, Efx} = apply(meta(C, 4, ?LINE, {notify, 2, Pid1}), + {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(C, 5), rabbit_fifo:make_purge(), State4), + {_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), ok. -reject_publish_applied_after_limit_test(C) -> - QName = rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME, utf8)), +reject_publish_applied_after_limit_test(Config) -> + QName = rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), InitConf = #{name => ?FUNCTION_NAME, max_in_memory_length => 0, queue_resource => QName @@ -1648,16 +1614,16 @@ reject_publish_applied_after_limit_test(C) -> State0 = init(InitConf), %% simply registering should be ok when we're below limit Pid1 = test_util:fake_pid(node()), - {State1, ok, [_]} = apply(meta(C, 1, ?LINE, {notify, 1, Pid1}), + {State1, ok, [_]} = apply(meta(Config, 1, ?LINE, {notify, 1, Pid1}), make_register_enqueuer(Pid1), State0), - {State2, ok, _} = apply(meta(C, 2, ?LINE, {notify, 1, Pid1}), + {State2, ok, _} = apply(meta(Config, 2, ?LINE, {notify, 1, Pid1}), rabbit_fifo:make_enqueue(Pid1, 1, one), State1), - {State3, ok, _} = apply(meta(C, 3, ?LINE, {notify, 1, Pid1}), + {State3, ok, _} = apply(meta(Config, 3, ?LINE, {notify, 1, Pid1}), rabbit_fifo:make_enqueue(Pid1, 2, two), State2), - {State4, ok, Efx} = apply(meta(C, 4, ?LINE, {notify, 1, Pid1}), + {State4, ok, Efx} = apply(meta(Config, 4, ?LINE, {notify, 1, Pid1}), rabbit_fifo:make_enqueue(Pid1, 3, three), State3), - % ct:pal("Efx ~tp", [Efx]), - ?ASSERT_NO_EFF({send_msg, P, {queue_status, reject_publish}, [ra_event]}, P == Pid1, Efx), + ?ASSERT_NO_EFF({send_msg, P, {queue_status, reject_publish}, [ra_event]}, + P == Pid1, Efx), %% apply new config Conf = #{name => ?FUNCTION_NAME, queue_resource => QName, @@ -1666,33 +1632,33 @@ reject_publish_applied_after_limit_test(C) -> max_in_memory_length => 0, dead_letter_handler => undefined }, - {State5, ok, Efx1} = apply(meta(C, 5), rabbit_fifo:make_update_config(Conf), State4), - ?ASSERT_EFF({send_msg, P, {queue_status, reject_publish}, [ra_event]}, P == Pid1, Efx1), + {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()), - {_State6, reject_publish, _} = apply(meta(C, 1), make_register_enqueuer(Pid2), State5), + {_State6, reject_publish, _} = + apply(meta(Config, 1), make_register_enqueuer(Pid2), State5), ok. -purge_nodes_test(C) -> +purge_nodes_test(Config) -> Node = purged@node, ThisNode = node(), EnqPid = test_util:fake_pid(Node), EnqPid2 = test_util:fake_pid(node()), ConPid = test_util:fake_pid(Node), Cid = {<<"tag">>, ConPid}, - % WaitingPid = test_util:fake_pid(Node), State0 = init(#{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r("/", queue, - atom_to_binary(?FUNCTION_NAME, utf8)), + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), single_active_consumer_on => false}), - {State1, _, _} = apply(meta(C, 1, ?LINE, {notify, 1, EnqPid}), + {State1, _, _} = apply(meta(Config, 1, ?LINE, {notify, 1, EnqPid}), rabbit_fifo:make_enqueue(EnqPid, 1, msg1), State0), - {State2, _, _} = apply(meta(C, 2, ?LINE, {notify, 2, EnqPid2}), + {State2, _, _} = apply(meta(Config, 2, ?LINE, {notify, 2, EnqPid2}), rabbit_fifo:make_enqueue(EnqPid2, 1, msg2), State1), - {State3, _} = check(C, Cid, 3, 1000, State2), - {State4, _, _} = apply(meta(C, 4), + {State3, _} = check(Config, Cid, 3, 1000, State2), + {State4, _, _} = apply(meta(Config, ?LINE), {down, EnqPid, noconnection}, State3), ?assertMatch([{aux, {handle_tick, @@ -1700,19 +1666,16 @@ purge_nodes_test(C) -> [ThisNode, Node]]}}], rabbit_fifo:tick(1, State4)), %% assert there are both enqueuers and consumers - {State, _, _} = apply(meta(C, 5), + {State, _, _} = apply(meta(Config, ?LINE), rabbit_fifo:make_purge_nodes([Node]), State4), %% assert there are no enqueuers nor consumers - ?assertMatch(#rabbit_fifo{enqueuers = Enqs} when map_size(Enqs) == 1, - State), - - ?assertMatch(#rabbit_fifo{consumers = Cons} when map_size(Cons) == 0, - State), - ?assertMatch([{aux, {handle_tick, - [#resource{}, _Metrics, - [ThisNode]]}}], + ?assertMatch(#rabbit_fifo{enqueuers = Enqs} + when map_size(Enqs) == 1, State), + ?assertMatch(#rabbit_fifo{consumers = Cons} + when map_size(Cons) == 0, State), + ?assertMatch([{aux, {handle_tick, [#resource{}, _Metrics, [ThisNode]]}}], rabbit_fifo:tick(1, State)), ok. @@ -1796,8 +1759,6 @@ credit(Config, Cid, Idx, Credit, DelCnt, Drain, State) -> strip_reply({State, _, Effects}) -> {State, Effects}. -% strip_reply({State, Effects}) -> -% {State, Effects}. checkout_reply({State, {ok, CInfo}, Effects}) when is_map(CInfo) -> {State, CInfo, Effects}; @@ -2013,7 +1974,7 @@ queue_ttl_test(C) -> = rabbit_fifo:tick(Now + 3000, E3), ok. -queue_ttl_with_single_active_consumer_test(C) -> +queue_ttl_with_single_active_consumer_test(Config) -> QName = rabbit_misc:r(<<"/">>, queue, <<"test">>), Conf = #{name => ?FUNCTION_NAME, queue_resource => QName, @@ -2028,11 +1989,11 @@ queue_ttl_with_single_active_consumer_test(C) -> = rabbit_fifo:tick(Now + 1000, S0), %% adding a consumer should not ever trigger deletion Cid = {<<"cid1">>, self()}, - {S1, _} = check_auto(C, Cid, 1, S0), + {S1, _, _} = checkout(Config, ?LINE, Cid, 1, S0), [{aux, {handle_tick, [_, _, _]}}] = rabbit_fifo:tick(Now, S1), [{aux, {handle_tick, [_, _, _]}}] = rabbit_fifo:tick(Now + 1000, S1), %% cancelling the consumer should then - {S2, _, _} = apply(meta(C, 2, Now), + {S2, _, _} = apply(meta(Config, ?LINE, Now), make_checkout(Cid, cancel, #{}), S1), %% last_active should have been reset when consumer was cancelled %% last_active = 2500 @@ -2041,7 +2002,7 @@ queue_ttl_with_single_active_consumer_test(C) -> [{mod_call, rabbit_quorum_queue, spawn_deleter, [QName]}] = rabbit_fifo:tick(Now + 2500, S2), %% Same for downs - {S2D, _, _} = apply(meta(C, 2, Now), + {S2D, _, _} = apply(meta(Config, ?LINE, Now), {down, self(), noconnection}, S1), %% last_active should have been reset when consumer was cancelled %% last_active = 2500 @@ -2051,11 +2012,11 @@ queue_ttl_with_single_active_consumer_test(C) -> = rabbit_fifo:tick(Now + 2500, S2D), ok. -query_peek_test(C) -> +query_peek_test(Config) -> State0 = test_init(test), ?assertEqual({error, no_message_at_pos}, rabbit_fifo:query_peek(1, State0)), - {State1, _} = enq(C, 1, 1, first, State0), - {State2, _} = enq(C, 2, 2, second, State1), + {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)), ?assertMatch({ok, [1 | _]}, rabbit_fifo:query_peek(1, State2)), @@ -2063,54 +2024,54 @@ query_peek_test(C) -> ?assertEqual({error, no_message_at_pos}, rabbit_fifo:query_peek(3, State2)), ok. -checkout_priority_test(C) -> +checkout_priority_test(Config) -> Cid = {<<"checkout_priority_test">>, self()}, Pid = spawn(fun () -> ok end), Cid2 = {<<"checkout_priority_test2">>, Pid}, Args = [{<<"x-priority">>, long, 1}], {S1, _, _} = - apply(meta(C, 3), - make_checkout(Cid, {once, 2, simple_prefetch}, + apply(meta(Config, ?LINE), + make_checkout(Cid, {auto, {simple_prefetch, 2}}, #{args => Args}), test_init(test)), {S2, _, _} = - apply(meta(C, 3), - make_checkout(Cid2, {once, 2, simple_prefetch}, #{args => []}), + apply(meta(Config, ?LINE), + make_checkout(Cid2, {auto, {simple_prefetch, 2}}, + #{args => []}), S1), - {S3, E3} = enq(C, 1, 1, first, S2), - ct:pal("E3 ~tp ~tp", [E3, self()]), + {S3, E3} = enq(Config, ?LINE, 1, first, S2), ?ASSERT_EFF({send_msg, P, {delivery, _, _}, _}, P == self(), E3), - {S4, E4} = enq(C, 2, 2, second, S3), + {S4, E4} = enq(Config, ?LINE, 2, second, S3), ?ASSERT_EFF({send_msg, P, {delivery, _, _}, _}, P == self(), E4), - {_S5, E5} = enq(C, 3, 3, third, S4), + {_S5, E5} = enq(Config, ?LINE, 3, third, S4), ?ASSERT_EFF({send_msg, P, {delivery, _, _}, _}, P == Pid, E5), ok. -empty_dequeue_should_emit_release_cursor_test(C) -> +empty_dequeue_should_emit_release_cursor_test(Config) -> State0 = test_init(?FUNCTION_NAME), Cid = {<<"basic.get1">>, self()}, {_State, {dequeue, empty}, Effects} = - apply(meta(C, 2, 1234), + apply(meta(Config, ?LINE, 1234), make_checkout(Cid, {dequeue, unsettled}, #{}), State0), ?ASSERT_EFF({release_cursor, _, _}, Effects), ok. -expire_message_should_emit_release_cursor_test(C) -> +expire_message_should_emit_release_cursor_test(Config) -> Conf = #{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r(<<"/">>, queue, <<"test">>), + queue_resource => rabbit_misc:r(<<"/">>, queue, ?FUNCTION_NAME_B), release_cursor_interval => 0, msg_ttl => 1}, S0 = rabbit_fifo:init(Conf), Msg = #basic_message{content = #content{properties = none, payload_fragments_rev = []}}, - {S1, ok, _} = apply(meta(C, 1, 100, {notify, 1, self()}), + {S1, ok, _} = apply(meta(Config, ?LINE, 100, {notify, 1, self()}), rabbit_fifo:make_enqueue(self(), 1, Msg), S0), - {_S, ok, Effs} = apply(meta(C, 2, 101, {notify, 2, self()}), + {_S, ok, Effs} = apply(meta(Config, ?LINE, 101, {notify, 2, self()}), rabbit_fifo:make_enqueue(self(), 2, Msg), S1), - ?ASSERT_EFF({release_cursor, 1, _}, Effs), + ?ASSERT_EFF({release_cursor, _, _}, Effs), ok. header_test(_) -> @@ -2184,18 +2145,14 @@ checkout_metadata_test(Config) -> {State0, _} = enq(Config, 2, 2, second, State00), %% NB: the consumer meta data is taken _before_ it runs a checkout %% so in this case num_checked_out will be 0 - {State1, {ok, #{next_msg_id := 0, - num_checked_out := 0}}, _} = - apply(meta(Config, ?LINE), - make_checkout(Cid, {auto, 1, simple_prefetch}, #{}), - State0), + {State1, #{next_msg_id := 0, + num_checked_out := 0}, _} = + checkout(Config, ?LINE, Cid, 1, State0), {State2, _, _} = apply(meta(Config, ?LINE), make_checkout(Cid, cancel, #{}), State1), - {_State3, {ok, #{next_msg_id := 1, - num_checked_out := 1}}, _} = - apply(meta(Config, ?LINE), - make_checkout(Cid, {auto, 1, simple_prefetch}, #{}), - State2), + {_State3, #{next_msg_id := 1, + num_checked_out := 1}, _} = + checkout(Config, ?LINE, Cid, 1, State2), ok. %% Utility diff --git a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl index 9575b91b5488..25eb9ca64aba 100644 --- a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl @@ -91,8 +91,8 @@ basics(Config) -> ConsumerTag = UId, ok = start_cluster(ClusterName, [ServerId]), FState0 = rabbit_fifo_client:init([ServerId]), - {ok, FState1} = rabbit_fifo_client:checkout(ConsumerTag, {simple_prefetch, 1}, - #{}, FState0), + {ok, _, FState1} = rabbit_fifo_client:checkout(ConsumerTag, {simple_prefetch, 1}, + #{}, FState0), rabbit_quorum_queue:wal_force_roll_over(node()), % create segment the segment will trigger a snapshot @@ -182,7 +182,7 @@ duplicate_delivery(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, _, F1} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, #{}, F0), {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, corr1, msg1, F1), Fun = fun Loop(S0) -> receive @@ -217,7 +217,7 @@ usage(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, _, 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), @@ -270,7 +270,7 @@ detects_lost_delivery(Config) -> F000 = rabbit_fifo_client:init([ServerId]), {ok, F00, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F000), {_, _, F0} = process_ra_events(receive_ra_events(1, 0), ClusterName, F00), - {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, #{}, F0), + {ok, _, F1} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, #{}, F0), {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, msg2, F1), {ok, F3, []} = rabbit_fifo_client:enqueue(ClusterName, msg3, F2), % lose first delivery @@ -299,9 +299,9 @@ returns_after_down(Config) -> Self = self(), _Pid = spawn(fun () -> F = rabbit_fifo_client:init([ServerId]), - {ok, _} = rabbit_fifo_client:checkout(<<"tag">>, - {simple_prefetch, 10}, - #{}, F), + {ok, _, _} = rabbit_fifo_client:checkout(<<"tag">>, + {simple_prefetch, 10}, + #{}, F), Self ! checkout_done end), receive checkout_done -> ok after 1000 -> exit(checkout_done_timeout) end, @@ -380,7 +380,7 @@ discard(Config) -> _ = ra:members(ServerId), F0 = rabbit_fifo_client:init([ServerId]), - {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, + {ok, _, F1} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, #{}, F0), {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F1), F3 = discard_next_delivery(ClusterName, F2, 5000), @@ -403,8 +403,8 @@ cancel_checkout(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:checkout(<<"tag">>, {simple_prefetch, 10}, - #{}, F1), + {ok, _, F2} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, + #{}, F1), {_, _, F3} = process_ra_events(receive_ra_events(1, 1), ClusterName, F2, [], [], fun (_, S) -> S end), {ok, F4} = rabbit_fifo_client:cancel_checkout(<<"tag">>, F3), @@ -422,7 +422,7 @@ lost_delivery(Config) -> {_, _, F2} = process_ra_events( receive_ra_events(1, 0), ClusterName, F1, [], [], fun (_, S) -> S end), - {ok, F3} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, #{}, F2), + {ok, _, F3} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, #{}, F2), %% drop a delivery, simulating e.g. a full distribution buffer receive {ra_event, _, Evt} -> @@ -447,6 +447,7 @@ lost_delivery(Config) -> 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]), @@ -456,7 +457,7 @@ credit_api_v1(Config) -> {_, _, 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, #{}, F3), + {ok, _, F4} = rabbit_fifo_client:checkout(CTag, {credited, 0}, #{}, F3), %% assert no deliveries {_, _, F5} = process_ra_events(receive_ra_events(), ClusterName, F4, [], [], fun @@ -503,9 +504,9 @@ credit_api_v2(Config) -> CTag = <<"my-tag">>, DC0 = 16#ff_ff_ff_ff, DC1 = 0, %% = DC0 + 1 using 32 bit serial number arithmetic - {ok, F4} = rabbit_fifo_client:checkout( + {ok, _, F4} = rabbit_fifo_client:checkout( %% initial_delivery_count in consumer meta means credit API v2. - CTag, credited, #{initial_delivery_count => DC0}, F3), + CTag, {credited, DC0}, #{}, F3), %% assert no deliveries {_, _, F5} = process_ra_events(receive_ra_events(), ClusterName, F4, [], [], fun @@ -604,7 +605,7 @@ test_queries(Config) -> exit(ready_timeout) end, F0 = rabbit_fifo_client:init([ServerId], 4), - {ok, _} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 1}, #{}, F0), + {ok, _, _} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 1}, #{}, F0), {ok, {_, Ready}, _} = ra:local_query(ServerId, fun rabbit_fifo:query_messages_ready/1), ?assertEqual(1, Ready), @@ -632,8 +633,8 @@ dequeue(Config) -> {ok, F2_, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F1b), {_, _, F2} = process_ra_events(receive_ra_events(1, 0), ClusterName, F2_), - % {ok, {{0, {_, msg1}}, _}, F3} = rabbit_fifo_client:dequeue(ClusterName, Tag, settled, F2), - {ok, _, {_, _, 0, _, msg1}, F3} = rabbit_fifo_client:dequeue(ClusterName, Tag, settled, F2), + {ok, _, {_, _, 0, _, msg1}, F3} = + rabbit_fifo_client:dequeue(ClusterName, Tag, settled, F2), {ok, F4_, []} = rabbit_fifo_client:enqueue(ClusterName, msg2, F3), {_, _, F4} = process_ra_events(receive_ra_events(1, 0), ClusterName, F4_), {ok, _, {_, _, MsgId, _, msg2}, F5} = rabbit_fifo_client:dequeue(ClusterName, Tag, unsettled, F4), @@ -693,7 +694,7 @@ receive_ra_events(Acc) -> end. process_ra_events(Events, ClusterName, State) -> - DeliveryFun = fun ({deliver, _, Tag, Msgs}, S) -> + DeliveryFun = fun ({deliver, Tag, _, Msgs}, S) -> MsgIds = [element(1, M) || M <- Msgs], {S0, _} = rabbit_fifo_client:settle(Tag, MsgIds, S), S0 diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index fe99ac0cbbbf..c3837ce7f6b9 100644 --- a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl @@ -11,10 +11,9 @@ -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"). --define(record_info(T,R),lists:zip(record_info(fields,T),tl(tuple_to_list(R)))). - -define(MACHINE_VERSION, 4). %%%=================================================================== @@ -64,10 +63,6 @@ all_tests() -> scenario31, scenario32, upgrade, - upgrade_snapshots_v1_v2, - upgrade_snapshots_scenario1, - upgrade_snapshots_scenario2, - upgrade_snapshots_v2_to_v3, messages_total, simple_prefetch, simple_prefetch_without_checkout_cancel, @@ -90,8 +85,8 @@ all_tests() -> dlx_06, dlx_07, dlx_08, - dlx_09 - % single_active_ordering_02 + dlx_09, + single_active_ordering_02 ]. groups() -> @@ -112,18 +107,18 @@ end_per_group(_Group, _Config) -> ok. init_per_testcase(_TestCase, Config) -> + ok = meck:new(rabbit_feature_flags, [passthrough]), + meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> false end), Config. end_per_testcase(_TestCase, _Config) -> + meck:unload(), ok. %%%=================================================================== %%% Test cases %%%=================================================================== -% -type log_op() :: -% {enqueue, pid(), maybe(msg_seqno()), Msg :: raw_msg()}. - scenario2(_Config) -> C1 = {<<>>, c:pid(0,346,1)}, C2 = {<<>>,c:pid(0,379,1)}, @@ -695,45 +690,6 @@ scenario23(_Config) -> Commands), ok. -upgrade_snapshots_scenario1(_Config) -> - E = c:pid(0,327,1), - Commands = [make_enqueue(E,1,msg(<<"msg1">>)), - make_enqueue(E,2,msg(<<"msg2">>)), - make_enqueue(E,3,msg(<<"msg3">>))], - run_upgrade_snapshot_test_v1_v2(#{name => ?FUNCTION_NAME, - delivery_limit => 100, - max_length => 1, - max_bytes => 100, - max_in_memory_length => undefined, - max_in_memory_bytes => undefined, - overflow_strategy => drop_head, - single_active_consumer_on => false, - dead_letter_handler => {?MODULE, banana, []} - }, - Commands), - ok. - -upgrade_snapshots_scenario2(_Config) -> - E = c:pid(0,240,0), - CPid = c:pid(0,242,0), - C = {<<>>, CPid}, - Commands = [make_checkout(C, {auto,1,simple_prefetch}), - make_enqueue(E,1,msg(<<"msg1">>)), - make_enqueue(E,2,msg(<<"msg2">>)), - rabbit_fifo:make_settle(C, [0])], - run_upgrade_snapshot_test_v1_v2(#{name => ?FUNCTION_NAME, - delivery_limit => undefined, - max_length => undefined, - max_bytes => undefined, - max_in_memory_length => undefined, - max_in_memory_bytes => undefined, - overflow_strategy => drop_head, - single_active_consumer_on => false, - dead_letter_handler => {?MODULE, banana, []} - }, - Commands), - ok. - single_active_01(_Config) -> C1Pid = test_util:fake_pid(rabbit@fake_node1), C1 = {<<0>>, C1Pid}, @@ -767,15 +723,14 @@ single_active_02(_Config) -> make_checkout(C2, cancel), {down,E,noconnection} ], - Conf = config(?FUNCTION_NAME, undefined, undefined, true, 1, undefined, undefined), + Conf = config(?FUNCTION_NAME, undefined, undefined, true, 1, + undefined, undefined), ?assert(single_active_prop(Conf, Commands, false)), ok. single_active_03(_Config) -> C1Pid = test_util:fake_pid(node()), C1 = {<<0>>, C1Pid}, - % C2Pid = test_util:fake_pid(rabbit@fake_node2), - % C2 = {<<>>, C2Pid}, Pid = test_util:fake_pid(node()), E = test_util:fake_pid(rabbit@fake_node2), Commands = [ @@ -790,67 +745,53 @@ single_active_03(_Config) -> ok. single_active_04(_Config) -> - % C1Pid = test_util:fake_pid(node()), - % C1 = {<<0>>, C1Pid}, - % C2Pid = test_util:fake_pid(rabbit@fake_node2), - % C2 = {<<>>, C2Pid}, - % Pid = test_util:fake_pid(node()), E = test_util:fake_pid(rabbit@fake_node2), Commands = [ - - % make_checkout(C1, {auto,2,simple_prefetch}), make_enqueue(E, 1, msg(<<>>)), make_enqueue(E, 2, msg(<<>>)), make_enqueue(E, 3, msg(<<>>)), make_enqueue(E, 4, msg(<<>>)) - % {down, Pid, noconnection}, - % {nodeup, node()} ], - Conf = config(?FUNCTION_NAME, 3, 587, true, 3, 7, undefined), + Conf = config(?FUNCTION_NAME, 3, 587, true, 3), ?assert(single_active_prop(Conf, Commands, true)), ok. test_run_log(_Config) -> - Fun = {-1, fun ({Prev, _}) -> {Prev + 1, Prev + 1} end}, + meck:expect(rabbit_feature_flags, is_enabled, + fun (_) -> true end), run_proper( fun () -> - ?FORALL({Length, Bytes, SingleActiveConsumer, DeliveryLimit, InMemoryLength, - InMemoryBytes}, - frequency([{10, {0, 0, false, 0, 0, 0}}, + ?FORALL({Length, Bytes, SingleActiveConsumer, DeliveryLimit}, + frequency([{10, {0, 0, false, 0}}, {5, {oneof([range(1, 10), undefined]), oneof([range(1, 1000), undefined]), boolean(), - oneof([range(1, 3), undefined]), - oneof([range(1, 10), undefined]), - oneof([range(1, 1000), undefined]) + oneof([range(1, 3), undefined]) }}]), - ?FORALL(O, ?LET(Ops, log_gen(100), expand(Ops, Fun)), + ?FORALL(O, ?LET(Ops, log_gen(100), expand(Ops, #{})), collect({log_size, length(O)}, dump_generated( config(?FUNCTION_NAME, Length, Bytes, SingleActiveConsumer, - DeliveryLimit, - InMemoryLength, - InMemoryBytes), O)))) + DeliveryLimit), O)))) end, [], 10). snapshots(_Config) -> + meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> true end), run_proper( fun () -> ?FORALL({Length, Bytes, SingleActiveConsumer, - DeliveryLimit, InMemoryLength, InMemoryBytes, - Overflow, DeadLetterHandler}, - frequency([{10, {0, 0, false, 0, 0, 0, drop_head, undefined}}, + DeliveryLimit, Overflow, DeadLetterHandler}, + frequency([{10, {0, 0, false, 0, drop_head, undefined}}, {5, {oneof([range(1, 10), undefined]), oneof([range(1, 1000), undefined]), boolean(), oneof([range(1, 3), undefined]), - oneof([range(1, 10), undefined]), - oneof([range(1, 1000), undefined]), oneof([drop_head, reject_publish]), - oneof([undefined, {at_most_once, {?MODULE, banana, []}}]) + oneof([undefined, + {at_most_once, {?MODULE, banana, []}}]) }}]), begin Config = config(?FUNCTION_NAME, @@ -858,8 +799,6 @@ snapshots(_Config) -> Bytes, SingleActiveConsumer, DeliveryLimit, - InMemoryLength, - InMemoryBytes, Overflow, DeadLetterHandler), ?FORALL(O, ?LET(Ops, log_gen(256), expand(Ops, Config)), @@ -869,17 +808,15 @@ snapshots(_Config) -> end, [], 256). snapshots_dlx(_Config) -> + meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> true end), Size = 256, run_proper( fun () -> - ?FORALL({Length, Bytes, SingleActiveConsumer, - DeliveryLimit, InMemoryLength, InMemoryBytes}, - frequency([{10, {0, 0, false, 0, 0, 0}}, + ?FORALL({Length, Bytes, SingleActiveConsumer, DeliveryLimit}, + frequency([{10, {0, 0, false, 0}}, {5, {oneof([range(1, 10), undefined]), oneof([range(1, 1000), undefined]), boolean(), - oneof([range(1, 3), undefined]), - oneof([range(1, 10), undefined]), oneof([range(1, 1000), undefined]) }}]), begin @@ -888,8 +825,6 @@ snapshots_dlx(_Config) -> Bytes, SingleActiveConsumer, DeliveryLimit, - InMemoryLength, - InMemoryBytes, reject_publish, at_least_once), ?FORALL(O, ?LET(Ops, log_gen_dlx(Size), expand(Ops, Config)), @@ -899,25 +834,24 @@ snapshots_dlx(_Config) -> end, [], Size). single_active(_Config) -> - Size = 300, + %% validates that there can only ever be a single active consumer at a time + %% as well as that message deliveries are done in order + Size = 1000, + meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> true end), run_proper( fun () -> - ?FORALL({Length, Bytes, DeliveryLimit, InMemoryLength, InMemoryBytes}, - frequency([{10, {0, 0, 0, 0, 0}}, + ?FORALL({Length, Bytes, DeliveryLimit}, + frequency([{10, {undefined, undefined, undefined}}, {5, {oneof([range(1, 10), undefined]), oneof([range(1, 1000), undefined]), - oneof([range(1, 3), undefined]), - oneof([range(1, 10), undefined]), - oneof([range(1, 1000), undefined]) + oneof([range(1, 3), undefined]) }}]), begin Config = config(?FUNCTION_NAME, Length, Bytes, true, - DeliveryLimit, - InMemoryLength, - InMemoryBytes), + DeliveryLimit), ?FORALL(O, ?LET(Ops, log_gen(Size), expand(Ops, Config)), collect({log_size, length(O)}, single_active_prop(Config, O, false))) @@ -926,14 +860,15 @@ single_active(_Config) -> upgrade(_Config) -> Size = 256, + %% upgrade is always done using _old_ command formats + meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> false end), run_proper( fun () -> - ?FORALL({Length, Bytes, DeliveryLimit, InMemoryLength, SingleActive}, - frequency([{5, {undefined, undefined, undefined, undefined, false}}, + ?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([range(1, 10), 0, undefined]), oneof([true, false]) }}]), begin @@ -942,8 +877,6 @@ upgrade(_Config) -> Bytes, SingleActive, DeliveryLimit, - InMemoryLength, - undefined, drop_head, undefined ), @@ -953,37 +886,8 @@ upgrade(_Config) -> end) end, [], Size). -upgrade_snapshots_v1_v2(_Config) -> - Size = 256, - run_proper( - fun () -> - ?FORALL({Length, Bytes, DeliveryLimit, InMemoryLength, SingleActive}, - frequency([{5, {undefined, undefined, undefined, undefined, false}}, - {5, {oneof([range(1, 10), undefined]), - oneof([range(1, 1000), undefined]), - oneof([range(1, 3), undefined]), - oneof([range(1, 10), 0, undefined]), - oneof([true, false]) - }}]), - begin - Config = config(?FUNCTION_NAME, - Length, - Bytes, - SingleActive, - DeliveryLimit, - InMemoryLength, - undefined, - drop_head, - {?MODULE, banana, []} - ), - ?FORALL(O, ?LET(Ops, log_gen_upgrade_snapshots(Size), - expand(Ops, Config)), - collect({log_size, length(O)}, - upgrade_snapshots_prop_v1_v2(Config, O))) - end) - end, [], Size). - -upgrade_snapshots_v2_to_v3(_Config) -> +messages_total(_Config) -> + meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> false end), Size = 256, run_proper( fun () -> @@ -999,36 +903,7 @@ upgrade_snapshots_v2_to_v3(_Config) -> Length, Bytes, SingleActive, - DeliveryLimit, - undefined, - undefined - ), - ?FORALL(O, ?LET(Ops, log_gen_upgrade_snapshots_v2_to_v3(Size), expand(Ops, Config)), - collect({log_size, length(O)}, - upgrade_snapshots_prop_v2_to_v3(Config, O))) - end) - end, [], Size). - -messages_total(_Config) -> - Size = 1000, - run_proper( - fun () -> - ?FORALL({Length, Bytes, DeliveryLimit, InMemoryLength, SingleActive}, - frequency([{5, {undefined, undefined, undefined, undefined, false}}, - {5, {oneof([range(1, 10), undefined]), - oneof([range(1, 1000), undefined]), - oneof([range(1, 3), undefined]), - oneof([range(1, 10), 0, undefined]), - oneof([true, false]) - }}]), - begin - Config = config(?FUNCTION_NAME, - Length, - Bytes, - SingleActive, - DeliveryLimit, - InMemoryLength, - undefined), + DeliveryLimit), ?FORALL(O, ?LET(Ops, log_gen(Size), expand(Ops, Config)), collect({log_size, length(O)}, messages_total_prop(Config, O))) @@ -1037,6 +912,7 @@ messages_total(_Config) -> simple_prefetch(_Config) -> Size = 500, + meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> true end), run_proper( fun () -> ?FORALL({Length, Bytes, DeliveryLimit, SingleActive}, @@ -1051,9 +927,7 @@ simple_prefetch(_Config) -> Length, Bytes, SingleActive, - DeliveryLimit, - undefined, - undefined), + DeliveryLimit), ?FORALL(O, ?LET(Ops, log_gen(Size), expand(Ops, Config)), collect({log_size, length(O)}, simple_prefetch_prop(Config, O, true))) @@ -1062,6 +936,7 @@ simple_prefetch(_Config) -> simple_prefetch_without_checkout_cancel(_Config) -> Size = 256, + meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> true end), run_proper( fun () -> ?FORALL({Length, Bytes, DeliveryLimit, SingleActive}, @@ -1076,10 +951,9 @@ simple_prefetch_without_checkout_cancel(_Config) -> Length, Bytes, SingleActive, - DeliveryLimit, - undefined, - undefined), - ?FORALL(O, ?LET(Ops, log_gen_without_checkout_cancel(Size), expand(Ops, Config)), + DeliveryLimit), + ?FORALL(O, ?LET(Ops, log_gen_without_checkout_cancel(Size), + expand(Ops, Config)), collect({log_size, length(O)}, simple_prefetch_prop(Config, O, false))) end) @@ -1108,19 +982,19 @@ simple_prefetch_01(_Config) -> single_active_ordering(_Config) -> Size = 500, + meck:expect(rabbit_feature_flags, is_enabled, fun (_) -> true end), Fun = {-1, fun ({Prev, _}) -> {Prev + 1, Prev + 1} end}, run_proper( fun () -> ?FORALL(O, ?LET(Ops, log_gen_ordered(Size), expand(Ops, Fun)), collect({log_size, length(O)}, - single_active_prop(config(?FUNCTION_NAME, - undefined, - undefined, - true, - undefined, - undefined, - undefined), O, - true))) + single_active_prop( + config(?FUNCTION_NAME, + undefined, + undefined, + true, + undefined), O, + true))) end, [], Size). single_active_ordering_01(_Config) -> @@ -1135,7 +1009,7 @@ single_active_ordering_01(_Config) -> make_enqueue(E2, 1, msg(<<"2">>)), make_settle(C1, [0]) ], - Conf = config(?FUNCTION_NAME, 0, 0, true, 0, 0, 0), + Conf = config(?FUNCTION_NAME, 0, 0, true, 0), ?assert(single_active_prop(Conf, Commands, true)), ok. @@ -1156,7 +1030,7 @@ single_active_ordering_02(_Config) -> {down,E,noproc}, make_settle(C1, [0]) ], - Conf = config(?FUNCTION_NAME, 0, 0, true, 0, 0, 0), + Conf = config(?FUNCTION_NAME, 0, 0, true, 0), ?assert(single_active_prop(Conf, Commands, true)), ok. @@ -1176,7 +1050,7 @@ single_active_ordering_03(_Config) -> make_checkout(C1, cancel), {down, C1Pid, noconnection} ], - Conf0 = config(?FUNCTION_NAME, 0, 0, true, 0, 0, 0), + Conf0 = config(?FUNCTION_NAME, 0, 0, true, 0), Conf = Conf0#{release_cursor_interval => 100}, Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), @@ -1201,21 +1075,17 @@ max_length(_Config) -> Size = 1000, run_proper( fun () -> - ?FORALL({Length, SingleActiveConsumer, DeliveryLimit, - InMemoryLength}, + ?FORALL({Length, SingleActiveConsumer, DeliveryLimit}, {oneof([range(1, 100), undefined]), boolean(), - range(1, 3), - range(1, 10) + range(1, 3) }, begin Config = config(?FUNCTION_NAME, Length, undefined, SingleActiveConsumer, - DeliveryLimit, - InMemoryLength, - undefined), + DeliveryLimit), ?FORALL(O, ?LET(Ops, log_gen_config(Size), expand(Ops, Config)), collect({log_size, length(O)}, @@ -1238,7 +1108,8 @@ dlx_01(_Config) -> rabbit_fifo:make_discard(C1, [1]), rabbit_fifo_dlx:make_settle([1]) ], - Config = config(?FUNCTION_NAME, 8, undefined, false, 2, 5, 100, reject_publish, at_least_once), + Config = config(?FUNCTION_NAME, 8, undefined, false, 2, + reject_publish, at_least_once), ?assert(snapshots_prop(Config, Commands)), ok. @@ -1260,7 +1131,8 @@ dlx_02(_Config) -> rabbit_fifo_dlx:make_settle([0]) %% Release cursor A got emitted. ], - Config = config(?FUNCTION_NAME, 10, undefined, false, 5, 5, 100, reject_publish, at_least_once), + Config = config(?FUNCTION_NAME, 10, undefined, false, 5, + reject_publish, at_least_once), ?assert(snapshots_prop(Config, Commands)), ok. @@ -1281,7 +1153,8 @@ dlx_03(_Config) -> rabbit_fifo_dlx:make_settle([0]) %% Release cursor A got emitted. ], - Config = config(?FUNCTION_NAME, 10, undefined, false, 5, 5, 100, reject_publish, at_least_once), + Config = config(?FUNCTION_NAME, 10, undefined, false, 5, + reject_publish, at_least_once), ?assert(snapshots_prop(Config, Commands)), ok. @@ -1301,7 +1174,8 @@ dlx_04(_Config) -> rabbit_fifo:make_discard(C1, [0,1,2,3,4,5]), rabbit_fifo_dlx:make_settle([0,1,2]) ], - Config = config(?FUNCTION_NAME, undefined, undefined, true, 1, 5, 136, reject_publish, at_least_once), + Config = config(?FUNCTION_NAME, undefined, undefined, true, 1, + reject_publish, at_least_once), ?assert(snapshots_prop(Config, Commands)), ok. @@ -1327,7 +1201,8 @@ dlx_05(_Config) -> rabbit_fifo_dlx:make_settle([0]) %% 2 in checkout ], - Config = config(?FUNCTION_NAME, 0, 0, false, 0, 0, 0, reject_publish, at_least_once), + Config = config(?FUNCTION_NAME, 0, 0, false, 0, + reject_publish, at_least_once), ?assert(snapshots_prop(Config, Commands)), ok. @@ -1355,7 +1230,8 @@ dlx_06(_Config) -> rabbit_fifo_dlx:make_settle([0,1]) %% 3 in dlx_checkout ], - Config = config(?FUNCTION_NAME, undefined, 749, false, 1, 1, 131, reject_publish, at_least_once), + Config = config(?FUNCTION_NAME, undefined, 749, false, 1, + reject_publish, at_least_once), ?assert(snapshots_prop(Config, Commands)), ok. @@ -1388,7 +1264,7 @@ dlx_07(_Config) -> rabbit_fifo_dlx:make_settle([0,1]) %% 3 in checkout ], - Config = config(?FUNCTION_NAME, undefined, undefined, false, undefined, undefined, undefined, + Config = config(?FUNCTION_NAME, undefined, undefined, false, undefined, reject_publish, at_least_once), ?assert(snapshots_prop(Config, Commands)), ok. @@ -1434,7 +1310,7 @@ dlx_08(_Config) -> rabbit_fifo_dlx:make_settle([1]), rabbit_fifo_dlx:make_settle([2]) ], - Config = config(?FUNCTION_NAME, undefined, undefined, false, undefined, undefined, undefined, + Config = config(?FUNCTION_NAME, undefined, undefined, false, undefined, reject_publish, at_least_once), ?assert(snapshots_prop(Config, Commands)), ok. @@ -1456,25 +1332,25 @@ dlx_09(_Config) -> rabbit_fifo:make_discard(C1, [2]) %% 1,2 in discards ], - Config = config(?FUNCTION_NAME, undefined, undefined, false, undefined, undefined, undefined, + Config = config(?FUNCTION_NAME, undefined, undefined, false, undefined, reject_publish, at_least_once), ?assert(snapshots_prop(Config, Commands)), ok. -config(Name, Length, Bytes, SingleActive, DeliveryLimit, InMemoryLength, InMemoryBytes) -> -config(Name, Length, Bytes, SingleActive, DeliveryLimit, InMemoryLength, InMemoryBytes, +config(Name, Length, MaxBytes, SingleActive, DeliveryLimit) -> + config(Name, Length, MaxBytes, SingleActive, DeliveryLimit, drop_head, {at_most_once, {?MODULE, banana, []}}). -config(Name, Length, Bytes, SingleActive, DeliveryLimit, - InMemoryLength, InMemoryBytes, Overflow, DeadLetterHandler) -> +config(Name, Length, MaxBytes, SingleActive, DeliveryLimit, + Overflow, DeadLetterHandler) -> #{name => Name, max_length => map_max(Length), - max_bytes => map_max(Bytes), + max_bytes => map_max(MaxBytes), dead_letter_handler => DeadLetterHandler, single_active_consumer_on => SingleActive, delivery_limit => map_max(DeliveryLimit), - max_in_memory_length => map_max(InMemoryLength), - max_in_memory_bytes => map_max(InMemoryBytes), + % max_in_memory_length => map_max(InMemoryLength), + % max_in_memory_bytes => map_max(InMemoryBytes), overflow_strategy => Overflow}. map_max(0) -> undefined; @@ -1602,10 +1478,13 @@ simple_prefetch_invariant(WithCheckoutCancel) -> maps:fold( fun(_, _, false) -> false; - (Id, #consumer{cfg = #consumer_cfg{credit_mode = {simple_prefetch, MaxCredit}}, + (Id, #consumer{cfg = #consumer_cfg{credit_mode = + {simple_prefetch, MaxCredit}}, checked_out = CheckedOut, credit = Credit}, true) -> - valid_simple_prefetch(MaxCredit, Credit, maps:size(CheckedOut), WithCheckoutCancel, Id) + valid_simple_prefetch(MaxCredit, Credit, + maps:size(CheckedOut), + WithCheckoutCancel, Id) end, true, Consumers) end. @@ -1643,19 +1522,15 @@ upgrade_prop(Conf0, Commands) -> [begin {PreEntries, PostEntries} = lists:split(SplitPos, Entries), %% run log v1 - {V1, _V1Effs} = run_log(InitState, PreEntries, + {V3, _V1Effs} = run_log(InitState, PreEntries, fun (_) -> true end, FromVersion), %% perform conversion - #rabbit_fifo{} = V2 = element(1, rabbit_fifo:apply( + #rabbit_fifo{} = V4 = element(1, rabbit_fifo:apply( meta(length(PreEntries) + 1), {machine_version, FromVersion, ToVersion}, - V1)), + V3)), %% assert invariants - %% - %% Note that we cannot test for num_messages because rabbit_fifo_v1:messages_total/1 - %% relies on ra_indexes not to be empty. However ra_indexes are empty in snapshots - %% in which case the number of messages checked out to consumers will not be included. Fields = [num_ready_messages, smallest_raft_index, num_enqueuers, @@ -1663,24 +1538,24 @@ upgrade_prop(Conf0, Commands) -> enqueue_message_bytes, checkout_message_bytes ], - V1Overview = maps:with(Fields, FromMod:overview(V1)), - V2Overview = maps:with(Fields, ToMod:overview(V2)), - case V1Overview == V2Overview of + V3Overview = maps:with(Fields, FromMod:overview(V3)), + V4Overview = maps:with(Fields, ToMod:overview(V4)), + case V3Overview == V4Overview of true -> ok; false -> ct:pal("upgrade_prop failed expected~n~tp~nGot:~n~tp", - [V1Overview, V2Overview]), - ?assertEqual(V1Overview, V2Overview) + [V3Overview, V4Overview]), + ?assertEqual(V3Overview, V4Overview) end, %% check we can run the post entries from the converted state - run_log(V2, PostEntries, fun (_) -> true end, ToVersion) + run_log(V4, PostEntries, fun (_) -> true end, ToVersion) end || SplitPos <- lists:seq(1, length(Entries))], - {_, V1Effs} = run_log(InitState, Entries, fun (_) -> true end, FromVersion), + {_, V3Effs} = run_log(InitState, Entries, fun (_) -> true end, FromVersion), [begin Res = rabbit_fifo:apply(meta(Idx + 1), {machine_version, FromVersion, ToVersion}, RCS) , - #rabbit_fifo{} = V2 = element(1, Res), + #rabbit_fifo{} = V4 = element(1, Res), %% assert invariants Fields = [num_ready_messages, smallest_raft_index, @@ -1689,16 +1564,16 @@ upgrade_prop(Conf0, Commands) -> enqueue_message_bytes, checkout_message_bytes ], - V1Overview = maps:with(Fields, FromMod:overview(RCS)), - V2Overview = maps:with(Fields, ToMod:overview(V2)), - case V1Overview == V2Overview of + V3Overview = maps:with(Fields, FromMod:overview(RCS)), + V4Overview = maps:with(Fields, ToMod:overview(V4)), + case V3Overview == V4Overview of true -> ok; false -> ct:pal("upgrade_prop failed expected~n~tp~nGot:~n~tp", - [V1Overview, V2Overview]), - ?assertEqual(V1Overview, V2Overview) + [V3Overview, V4Overview]), + ?assertEqual(V3Overview, V4Overview) end - end || {release_cursor, Idx, RCS} <- V1Effs], + end || {release_cursor, Idx, RCS} <- V3Effs], true. %% single active consumer ordering invariant: @@ -1730,27 +1605,7 @@ dump_generated(Conf, Commands) -> true. snapshots_prop(Conf, Commands) -> - try run_snapshot_test(Conf, Commands, messages_total_invariant()) of - _ -> true - catch - Err -> - ct:pal("Commands: ~tp~nConf~tp~n", [Commands, Conf]), - ct:pal("Err: ~tp~n", [Err]), - false - end. - -upgrade_snapshots_prop_v1_v2(Conf, Commands) -> - try run_upgrade_snapshot_test_v1_v2(Conf, Commands) of - _ -> true - catch - Err -> - ct:pal("Commands: ~tp~nConf~tp~n", [Commands, Conf]), - ct:pal("Err: ~tp~n", [Err]), - false - end. - -upgrade_snapshots_prop_v2_to_v3(Conf, Commands) -> - try run_upgrade_snapshot_test_v2_to_v3(Conf, Commands) of + try run_snapshot_test(Conf, Commands) of _ -> true catch Err -> @@ -1782,28 +1637,6 @@ log_gen(Size) -> {1, purge} ]))))). -%% Does not use "return", "down", or "checkout cancel" Ra commands -%% since these 3 commands change behaviour across v2 and v3 fixing -%% a bug where to many credits are granted to the consumer. -log_gen_upgrade_snapshots_v2_to_v3(Size) -> - Nodes = [node(), - fakenode@fake, - fakenode@fake2 - ], - ?LET(EPids, vector(2, pid_gen(Nodes)), - ?LET(CPids, vector(2, pid_gen(Nodes)), - resize(Size, - list( - frequency( - [{20, enqueue_gen(oneof(EPids))}, - {40, {input_event, - frequency([{10, settle}, - {2, discard}, - {2, requeue}])}}, - {1, checkout_gen(oneof(CPids))}, - {1, purge} - ]))))). - log_gen_upgrade_snapshots(Size) -> Nodes = [node(), fakenode@fake, @@ -1822,17 +1655,8 @@ log_gen_upgrade_snapshots(Size) -> {2, requeue} ])}}, {2, checkout_gen(oneof(CPids))}, - %% v2 fixes a bug that exists in v1 where a cancelled - %% consumer is revived. - %% Therefore, there is an expected behavioural - %% difference between v1 and v2 - %% and below line must be commented out. - % {1, checkout_cancel_gen(oneof(CPids))}, - %% Likewise there is a behavioural difference between - %% v1 and v2 when 'up' is followed by 'down' where - %% v2 behaves correctly. - %% Therefore, below line must be commented out. - % {1, down_gen(oneof(EPids ++ CPids))}, + {1, checkout_cancel_gen(oneof(CPids))}, + {1, down_gen(oneof(EPids ++ CPids))}, {1, nodeup_gen(Nodes)}, {1, purge} ]))))). @@ -1959,12 +1783,17 @@ enqueue_gen(Pid) -> enqueue_gen(Pid, _Enq, _Del) -> ?LET(E, {enqueue, Pid, enqueue, msg_gen()}, E). -%% It's fair to assume that every message enqueued is a #basic_message. -%% That's what the channel expects and what rabbit_quorum_queue invokes rabbit_fifo_client with. msg_gen() -> ?LET(Bin, binary(), - #basic_message{content = #content{payload_fragments_rev = [Bin], - properties = none}}). + mc:prepare( + store, mc_amqpl:from_basic_message( + #basic_message{exchange_name = #resource{name = <<"e">>, + kind = exchange, + virtual_host = <<"/">>}, + routing_keys = [<<>>], + content = + #content{payload_fragments_rev = [Bin], + properties = #'P_basic'{}}}))). msg(Bin) when is_binary(Bin) -> #basic_message{content = #content{payload_fragments_rev = [Bin], @@ -1987,7 +1816,8 @@ checkout_gen(Pid) -> config :: map(), log = [] :: list(), down = #{} :: #{pid() => noproc | noconnection}, - enq_cmds = #{} :: #{ra:index() => rabbit_fifo:enqueue()} + enq_cmds = #{} :: #{ra:index() => rabbit_fifo:enqueue()}, + is_v4 = false :: boolean() }). expand(Ops, Config) -> @@ -2013,9 +1843,11 @@ expand(Ops, Config, EnqFun) -> _ -> InitConfig0 end, + IsV4 = rabbit_feature_flags:is_enabled(quorum_queues_v4), T = #t{state = rabbit_fifo:init(InitConfig), enq_body_fun = EnqFun, - config = Config}, + config = Config, + is_v4 = IsV4}, #t{effects = Effs} = T1 = lists:foldl(fun handle_op/2, T, Ops), %% process the remaining effect #t{log = Log} = lists:foldl(fun do_apply/2, @@ -2067,9 +1899,15 @@ handle_op({checkout, CId, Prefetch}, #t{consumers = Cons0} = T) -> %% ignore if it already exists T; _ -> - Cons = maps:put(CId, ok, Cons0), - Cmd = rabbit_fifo:make_checkout(CId, - {auto, Prefetch, simple_prefetch}, + Spec = case T#t.is_v4 of + true -> + {auto, {simple_prefetch, Prefetch}}; + false -> + {auto, Prefetch, simple_prefetch} + end, + + Cons = maps:put(CId, T#t.index, Cons0), + Cmd = rabbit_fifo:make_checkout(CId, Spec, #{ack => true, prefetch => Prefetch, username => <<"user">>, @@ -2097,13 +1935,24 @@ handle_op({input_event, requeue}, #t{effects = Effs} = T) -> T end; handle_op({input_event, Settlement}, #t{effects = Effs, - down = Down} = T) -> + consumers = Cons, + down = Down, + is_v4 = IsV4} = T) -> case queue:out(Effs) of {{value, {settle, CId, MsgIds}}, Q} -> + CKey = case maps:get(CId, Cons, undefined) of + K when is_integer(K) andalso IsV4 -> + K; + _ -> + CId + end, Cmd = case Settlement of - settle -> rabbit_fifo:make_settle(CId, MsgIds); - return -> rabbit_fifo:make_return(CId, MsgIds); - discard -> rabbit_fifo:make_discard(CId, MsgIds) + settle -> + rabbit_fifo:make_settle(CKey, MsgIds); + return -> + rabbit_fifo:make_return(CKey, MsgIds); + discard -> + rabbit_fifo:make_discard(CKey, MsgIds) end, do_apply(Cmd, T#t{effects = Q}); {{value, {enqueue, Pid, _, _} = Cmd}, Q} -> @@ -2126,7 +1975,8 @@ handle_op(purge, T) -> handle_op({update_config, Changes}, #t{config = Conf} = T) -> Config = maps:merge(Conf, Changes), do_apply(rabbit_fifo:make_update_config(Config), T); -handle_op({checkout_dlx, Prefetch}, #t{config = #{dead_letter_handler := at_least_once}} = T) -> +handle_op({checkout_dlx, Prefetch}, + #t{config = #{dead_letter_handler := at_least_once}} = T) -> Cmd = rabbit_fifo_dlx:make_checkout(ignore_pid, Prefetch), do_apply(Cmd, T). @@ -2194,145 +2044,28 @@ run_proper(Fun, Args, NumTests) -> end}])). run_snapshot_test(Conf, Commands) -> - run_snapshot_test(Conf, Commands, fun (_) -> true end). - -run_snapshot_test(Conf, Commands, Invariant) -> - %% create every incremental permutation of the commands lists - %% and run the snapshot tests against that - ct:pal("running snapshot test with ~b commands using config ~tp", - [length(Commands), Conf]), - [begin - % ct:pal("~w running commands to ~w~n", [?FUNCTION_NAME, lists:last(C)]), - run_snapshot_test0(Conf, C, Invariant) - end || C <- prefixes(Commands, 1, [])]. - -run_snapshot_test0(Conf, Commands) -> - run_snapshot_test0(Conf, Commands, fun (_) -> true end). - -run_snapshot_test0(Conf0, Commands, Invariant) -> - Conf = Conf0#{max_in_memory_length => 0}, + %% tests that release cursor indexes are definitely lower than + %% the smallest raft index referring to a message Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), - {State0, Effects} = run_log(test_init(Conf), Entries, Invariant), - State = rabbit_fifo:normalize(State0), - Cursors = [ C || {release_cursor, _, _} = C <- Effects], - - [begin - %% drop all entries below and including the snapshot - Filtered = lists:dropwhile(fun({X, _}) when X =< SnapIdx -> true; - (_) -> false - end, Entries), - % ct:pal("release_cursor: ~b from ~w~n", [SnapIdx, element(1, hd_or(Filtered))]), - {S0, _} = run_log(SnapState, Filtered, Invariant), - S = rabbit_fifo:normalize(S0), - % assert log can be restored from any release cursor index - case S of - State -> ok; - _ -> - ct:pal("Snapshot tests failed run log:~n" - "~tp~n from snapshot index ~b " - "with snapshot state~n~tp~n Entries~n~tp~n" - "Config: ~tp~n", - [Filtered, SnapIdx, SnapState, Entries, Conf]), - ct:pal("Expected~n~tp~nGot:~n~tp~n", [?record_info(rabbit_fifo, State), - ?record_info(rabbit_fifo, S)]), - ?assertEqual(State, S) - end - end || {release_cursor, SnapIdx, SnapState} <- Cursors], - ok. - -run_upgrade_snapshot_test_v1_v2(Conf, Commands) -> - ct:pal("running test with ~b commands using config ~tp", + ct:pal("running snapshot test 2 with ~b commands using config ~tp", [length(Commands), Conf]), - Indexes = lists:seq(1, length(Commands)), - Entries = lists:zip(Indexes, Commands), - Invariant = fun(_) -> true end, - %% Run the whole command log in v1 to emit release cursors. - {_, Effects} = run_log(test_init_v1(Conf), Entries, Invariant, 1), - Cursors = [ C || {release_cursor, _, _} = C <- Effects], - [begin - %% Drop all entries below and including the snapshot. - FilteredV1 = lists:dropwhile(fun({X, _}) when X =< SnapIdx -> true; - (_) -> false - end, Entries), - %% For V2 we will apply the same commands to the snapshot state as for V1. - %% However, we need to increment all Raft indexes by 1 because V2 - %% requires one additional Raft index for the conversion command from V1 to V2. - FilteredV2 = lists:keymap(fun(Idx) -> Idx + 1 end, 1, FilteredV1), - %% Recover in V1. - {StateV1, _} = run_log(SnapState, FilteredV1, Invariant, 1), - %% Perform conversion and recover in V2. - Res = rabbit_fifo_v3:apply(meta(SnapIdx + 1), {machine_version, 1, 2}, SnapState), - #rabbit_fifo{} = V2 = element(1, Res), - {StateV2, _} = run_log(V2, FilteredV2, Invariant, 2), - %% Invariant: Recovering a V1 snapshot in V1 or V2 should end up in the same - %% number of messages. - Fields = [num_messages, - num_ready_messages, - num_enqueuers, - num_consumers, - enqueue_message_bytes, - checkout_message_bytes - ], - V1Overview = maps:with(Fields, rabbit_fifo_v1:overview(StateV1)), - V2Overview = maps:with(Fields, rabbit_fifo_v3:overview(StateV2)), - case V1Overview == V2Overview of - true -> ok; - false -> - ct:pal("property failed, expected:~n~tp~ngot:~n~tp~nstate v1:~n~tp~nstate v2:~n~tp~n" - "snapshot index: ~tp", - [V1Overview, V2Overview, StateV1, ?record_info(rabbit_fifo, StateV2), SnapIdx]), - ?assertEqual(V1Overview, V2Overview) - end - end || {release_cursor, SnapIdx, SnapState} <- Cursors], - ok. + Fun = fun (_E, S, Effs) -> + MsgTotFun = messages_total_invariant(), + case lists:reverse( + [C || {release_cursor, _, _} = C <- Effs]) of + [] -> + MsgTotFun(S); + [{release_cursor, Idx, _} | _] -> + %% ensure the current state has no active messages + %% below or equal to the release cursor index + rabbit_fifo:smallest_raft_index(S) > Idx andalso + MsgTotFun(S) + end + end, + _ = run_log(test_init(Conf), Entries, Fun), + true. -run_upgrade_snapshot_test_v2_to_v3(Conf, Commands) -> - ct:pal("running test with ~b commands using config ~tp", - [length(Commands), Conf]), - Indexes = lists:seq(1, length(Commands)), - Entries = lists:zip(Indexes, Commands), - Invariant = fun(_) -> true end, - %% Run the whole command log in v2 to emit release cursors. - {_, Effects} = run_log(test_init(rabbit_fifo_v3, Conf), Entries, Invariant, 2), - Cursors = [ C || {release_cursor, _, _} = C <- Effects], - [begin - %% Drop all entries below and including the snapshot. - FilteredV2 = lists:dropwhile(fun({X, _}) when X =< SnapIdx -> true; - (_) -> false - end, Entries), - %% For V3 we will apply the same commands to the snapshot state as for V2. - %% However, we need to increment all Raft indexes by 1 because V3 - %% requires one additional Raft index for the conversion command from V2 to V3. - FilteredV3 = lists:keymap(fun(Idx) -> Idx + 1 end, 1, FilteredV2), - %% Recover in V2. - {StateV2, _} = run_log(SnapState, FilteredV2, Invariant, 2), - %% Perform conversion and recover in V3. - Res = rabbit_fifo:apply(meta(SnapIdx + 1), {machine_version, 2, 3}, SnapState), - #rabbit_fifo{} = V3 = element(1, Res), - {StateV3, _} = run_log(V3, FilteredV3, Invariant, 3), - %% Invariant: Recovering a V2 snapshot in V2 or V3 should end up in the same - %% number of messages given that no "return", "down", or "cancel consumer" - %% Ra commands are used. - Fields = [num_messages, - num_ready_messages, - num_enqueuers, - num_consumers, - enqueue_message_bytes, - checkout_message_bytes - ], - V2Overview = maps:with(Fields, rabbit_fifo:overview(StateV2)), - V3Overview = maps:with(Fields, rabbit_fifo:overview(StateV3)), - case V2Overview == V3Overview of - true -> ok; - false -> - ct:pal("property failed, expected:~n~tp~ngot:~n~tp~nstate v2:~n~tp~nstate v3:~n~tp~n" - "snapshot index: ~tp", - [V2Overview, V3Overview, StateV2, ?record_info(rabbit_fifo, StateV3), SnapIdx]), - ?assertEqual(V2Overview, V3Overview) - end - end || {release_cursor, SnapIdx, SnapState} <- Cursors], - ok. hd_or([H | _]) -> H; hd_or(_) -> {undefined}. @@ -2350,10 +2083,16 @@ run_log(InitState, Entries) -> run_log(InitState, Entries, InvariantFun) -> run_log(InitState, Entries, InvariantFun, ?MACHINE_VERSION). +run_log(InitState, Entries, InvariantFun0, MachineVersion) + when is_function(InvariantFun0, 1) -> + InvariantFun = fun (_E, S, _Effs) -> + InvariantFun0(S) + end, + run_log(InitState, Entries, InvariantFun, MachineVersion); run_log(InitState, Entries, InvariantFun, MachineVersion) when is_integer(MachineVersion) -> - Invariant = fun(E, S) -> - case InvariantFun(S) of + Invariant = fun(E, S, Effs) -> + case InvariantFun(E, S, Effs) of true -> ok; false -> throw({invariant, E, S}) @@ -2369,15 +2108,16 @@ run_log(InitState, Entries, InvariantFun, MachineVersion) _ -> {meta(Idx, MachineVersion), E0} end, + case FifoMod:apply(Meta, E, Acc0) of {Acc, _, Efx} when is_list(Efx) -> - Invariant(E, Acc), + Invariant(E, Acc, lists:flatten(Efx)), {Acc, Efx0 ++ Efx}; {Acc, _, Efx} -> - Invariant(E, Acc), + Invariant(E, Acc, lists:flatten(Efx)), {Acc, Efx0 ++ [Efx]}; {Acc, _} -> - Invariant(E, Acc), + Invariant(E, Acc, []), {Acc, Efx0} end end, {InitState, []}, Entries). @@ -2409,7 +2149,7 @@ make_checkout(Cid, Spec, Meta) -> rabbit_fifo:make_checkout(Cid, Spec, Meta). make_enqueue(Pid, Seq, Msg) -> - rabbit_fifo_v3:make_enqueue(Pid, Seq, Msg). + rabbit_fifo:make_enqueue(Pid, Seq, Msg). make_settle(Cid, MsgIds) -> rabbit_fifo:make_settle(Cid, MsgIds). From 922459d747599baf41c8c1a1de3fe992de2b50ca Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 2 Apr 2024 11:08:22 +0100 Subject: [PATCH 04/45] First pass SAC consumer priority implementation. Single active consumers will be activated if they have a higher priority than the currently active consumer. if the currently active consumer has pending messages, no further messages will be assigned to the consumer and the activation of the new consumer will happen once all pending messages are settled. This is to ensure processing order. Consumers with the same priority will internally be ordered to favour those with credit then those that attached first. QQ: add SAC consumer priority integration tests Dialyzer fix QQ: add check for ff in tests --- deps/rabbit/src/rabbit_fifo.erl | 247 ++++++--- deps/rabbit/src/rabbit_fifo.hrl | 5 +- deps/rabbit/src/rabbit_quorum_queue.erl | 8 +- deps/rabbit/test/quorum_queue_SUITE.erl | 139 ++++- deps/rabbit/test/rabbit_fifo_SUITE.erl | 483 ++++++++++++++++-- .../test/single_active_consumer_SUITE.erl | 4 +- 6 files changed, 752 insertions(+), 134 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 1720d3fb28e2..e23c4354e052 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -19,6 +19,7 @@ -define(STATE, ?MODULE). -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}}). @@ -295,7 +296,7 @@ apply(#{index := Idx} = Meta, Indexes0), messages = lqueue:in(?MSG(Idx, Header), Messages), enqueue_count = EnqCount + 1}, - State2 = update_or_remove_sub(Meta, ConsumerKey, Con, State1), + State2 = update_or_remove_con(Meta, ConsumerKey, Con, State1), {State, Ret, Effs} = checkout(Meta, State0, State2, []), update_smallest_raft_index(Idx, Ret, maybe_store_release_cursor(Idx, State), @@ -393,7 +394,7 @@ apply(Meta, #credit{credit = LinkCreditRcv, %% grant the credit Con = Con0#consumer{credit = LinkCreditSnd}, State = State0#?STATE{waiting_consumers = - [{ConsumerKey, Con} | Waiting]}, + add_waiting({ConsumerKey, Con}, Waiting)}, %% No messages are available for inactive consumers. Available = 0, case credit_api_v2(Cfg) of @@ -480,8 +481,9 @@ apply(#{index := Idx} = Meta, consumer_id = ConsumerId}, State0) -> case consumer_key_from_id(ConsumerId, State0) of {ok, ConsumerKey} -> - {State1, Effects1} = cancel_consumer(Meta, ConsumerKey, State0, [], - consumer_cancel), + {State1, Effects1} = activate_next_consumer( + cancel_consumer(Meta, ConsumerKey, State0, [], + consumer_cancel)), Reply = {ok, consumer_cancel_info(ConsumerKey, State1)}, {State, _, Effects} = checkout(Meta, State0, State1, Effects1), update_smallest_raft_index(Idx, Reply, State, Effects); @@ -504,7 +506,7 @@ apply(#{index := Idx} = Meta, {Life, _Credit, credited} -> {Life, credited} end, - Priority = get_priority_from_args(ConsumerMeta), + Priority = get_priority(ConsumerMeta), ConsumerKey = case consumer_key_from_id(ConsumerId, State0) of {ok, K} -> K; @@ -658,7 +660,8 @@ apply(#{system_time := Ts} = Meta, checkout(Meta, State0, State#?STATE{enqueuers = Enqs, last_active = Ts}, Effects); apply(#{index := Idx} = Meta, {down, Pid, _Info}, State0) -> - {State1, Effects1} = handle_down(Meta, Pid, State0), + {State1, Effects1} = activate_next_consumer( + handle_down(Meta, Pid, State0)), {State, Reply, Effects} = checkout(Meta, State0, State1, Effects1), update_smallest_raft_index(Idx, Reply, State, Effects); apply(Meta, {nodeup, Node}, #?STATE{consumers = Cons0, @@ -683,7 +686,7 @@ apply(Meta, {nodeup, Node}, #?STATE{consumers = Cons0, (C#consumer.status =/= cancelled) -> EAcc1 = ConsumerUpdateActiveFun(SAcc, ConsumerKey, C, true, up, EAcc), - {update_or_remove_sub(Meta, ConsumerKey, + {update_or_remove_con(Meta, ConsumerKey, C#consumer{status = up}, SAcc), EAcc1}; (_, _, Acc) -> @@ -795,15 +798,16 @@ handle_waiting_consumer_down(Pid, update_waiting_consumer_status(Node, #?STATE{waiting_consumers = WaitingConsumers}, Status) -> - [begin - case node(Pid) of - Node -> - {ConsumerKey, Consumer#consumer{status = Status}}; - _ -> - {ConsumerKey, Consumer} - end - end || {ConsumerKey, ?CONSUMER_PID(Pid) = Consumer} - <- WaitingConsumers, Consumer#consumer.status =/= cancelled]. + sort_waiting( + [begin + case node(Pid) of + Node -> + {ConsumerKey, Consumer#consumer{status = Status}}; + _ -> + {ConsumerKey, Consumer} + end + end || {ConsumerKey, ?CONSUMER_PID(Pid) = Consumer} + <- WaitingConsumers, Consumer#consumer.status =/= cancelled]). -spec state_enter(ra_server:ra_state() | eol, state()) -> ra_machine:effects(). @@ -1382,9 +1386,8 @@ cancel_consumer(Meta, ConsumerKey, case Cons0 of #{ConsumerKey := #consumer{status = _}} -> % The active consumer is to be removed - {State1, Effects1} = cancel_consumer0(Meta, ConsumerKey, State0, - Effects0, Reason), - activate_next_consumer(State1, Effects1); + cancel_consumer0(Meta, ConsumerKey, State0, + Effects0, Reason); _ -> % The cancelled consumer is not active or cancelled % Just remove it from idle_consumers @@ -1431,50 +1434,85 @@ cancel_consumer0(Meta, ConsumerKey, {S0, Effects0} end. +activate_next_consumer({State, Effects}) -> + activate_next_consumer(State, Effects). + activate_next_consumer(#?STATE{cfg = #cfg{consumer_strategy = competing}} = State0, Effects0) -> {State0, Effects0}; activate_next_consumer(#?STATE{consumers = Cons, waiting_consumers = Waiting0} = State0, Effects0) -> - case has_active_consumer(Cons) of - false -> - case lists:filter(fun ({_, #consumer{status = Status}}) -> - Status == up - end, Waiting0) of - [{NextCKey, #consumer{cfg = NextCCfg} = NextConsumer} | _] -> - Remaining = lists:keydelete(NextCKey, 1, Waiting0), - Consumer = case maps:get(NextCKey, Cons, undefined) of - undefined -> - NextConsumer; - 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 = Cons#{NextCKey => Consumer}, - service_queue = ServiceQueue1, - waiting_consumers = Remaining}, - Effects = consumer_update_active_effects(State, Consumer, - true, single_active, - Effects0), - {State, Effects}; - [] -> - {State0, Effects0} - end; - true -> + %% 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(Cons), NextConsumer} of + {undefined, {NextCKey, #consumer{cfg = NextCCfg} = NextC}} -> + Remaining = tl(Waiting0), + %% TODO: can this happen? + Consumer = case maps:get(NextCKey, Cons, 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 = Cons#{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), + State = State0#?STATE{consumers = maps:remove(ActiveCKey, + Cons#{NextCKey => Consumer}), + service_queue = ServiceQueue1, + waiting_consumers = + add_waiting({ActiveCKey, Active}, Remaining)}, + Effects = consumer_update_active_effects(State, Consumer, + true, single_active, + Effects0), + {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 + {State0#?STATE{consumers = + Cons#{ActiveCKey => Active#consumer{status = fading}}}, + Effects0}; + _ -> + %% no activation {State0, Effects0} end. -has_active_consumer(Consumers) -> - active_consumer(Consumers) /= undefined. - -active_consumer({CKey, #consumer{status = up} = Consumer, _I}) -> +active_consumer({CKey, #consumer{status = Status} = Consumer, _I}) + when Status == up orelse Status == fading -> {CKey, Consumer}; active_consumer({_CKey, #consumer{status = _}, I}) -> active_consumer(maps:next(I)); @@ -1496,7 +1534,7 @@ maybe_return_all(#{system_time := Ts} = Meta, ConsumerKey, Effects0, Reason) -> case Reason of consumer_cancel -> - {update_or_remove_sub( + {update_or_remove_con( Meta, ConsumerKey, Consumer#consumer{cfg = CCfg#consumer_cfg{lifetime = once}, credit = 0, @@ -1669,7 +1707,7 @@ return(#{index := IncomingRaftIdx} = Meta, end, {State0, Effects0}, Returned), State2 = case State1#?STATE.consumers of #{ConsumerKey := Con} -> - update_or_remove_sub(Meta, ConsumerKey, Con, State1); + update_or_remove_con(Meta, ConsumerKey, Con, State1); _ -> State1 end, @@ -1677,25 +1715,25 @@ return(#{index := IncomingRaftIdx} = Meta, update_smallest_raft_index(IncomingRaftIdx, State, Effects). % used to process messages that are finished -complete(Meta, ConsumerKey, [DiscardedMsgId], +complete(Meta, ConsumerKey, [MsgId], #consumer{checked_out = Checked0} = Con0, #?STATE{ra_indexes = Indexes0, msg_bytes_checkout = BytesCheckout, messages_total = Tot} = State0) -> - case maps:take(DiscardedMsgId, Checked0) of + 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_sub(Meta, ConsumerKey, Con, State0), + 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, DiscardedMsgIds, +complete(Meta, ConsumerKey, MsgIds, #consumer{checked_out = Checked0} = Con0, #?STATE{ra_indexes = Indexes0, msg_bytes_checkout = BytesCheckout, @@ -1710,11 +1748,11 @@ complete(Meta, ConsumerKey, DiscardedMsgIds, error -> {S0, Ch0, Idxs} end - end, {0, Checked0, Indexes0}, DiscardedMsgIds), + 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_sub(Meta, ConsumerKey, Con, State0), + State1 = update_or_remove_con(Meta, ConsumerKey, Con, State0), State1#?STATE{ra_indexes = Indexes, msg_bytes_checkout = BytesCheckout - SettledSize, messages_total = Tot - Len}. @@ -1745,7 +1783,9 @@ complete_and_checkout(#{index := IncomingRaftIdx} = Meta, MsgIds, ConsumerKey, #consumer{} = Con0, Effects0, State0) -> State1 = complete(Meta, ConsumerKey, MsgIds, Con0, State0), - {State, ok, Effects} = checkout(Meta, State0, State1, Effects0), + %% a completion could have removed the active/fading consumer + {State2, Effects1} = activate_next_consumer(State1, Effects0), + {State, ok, Effects} = checkout(Meta, State0, State2, Effects1), update_smallest_raft_index(IncomingRaftIdx, State, Effects). cancel_consumer_effects(ConsumerId, @@ -2069,16 +2109,15 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> %% there are consumers waiting to be serviced %% process consumer checkout case maps:get(ConsumerKey, Cons0) of - #consumer{credit = 0} -> - %% no credit but was still on queue - %% can happen when draining - %% recurse without consumer on queue - checkout_one(Meta, ExpiredMsg, - InitState#?STATE{service_queue = SQ1}, - Effects1); - #consumer{status = S} - when S =:= cancelled orelse - S =:= suspected_down -> + #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); @@ -2101,7 +2140,7 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> State0#?STATE{service_queue = SQ1, msg_bytes_checkout = BytesCheckout + Size, msg_bytes_enqueue = BytesEnqueue - Size}, - State = update_or_remove_sub( + State = update_or_remove_con( Meta, ConsumerKey, Con, State1), {success, ConsumerKey, Next, ConsumerMsg, ExpiredMsg, State, Effects1} @@ -2110,7 +2149,7 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> {nochange, ExpiredMsg, InitState, Effects1} end; {{value, _ConsumerId}, SQ1} -> - %% consumer did not exist but was queued, recurse + %% consumer was not active but was queued, recurse checkout_one(Meta, ExpiredMsg, InitState#?STATE{service_queue = SQ1}, Effects1); {empty, _} -> @@ -2175,7 +2214,7 @@ timer_effect(RaCmdTs, State, Effects) -> end, [{timer, expire_msgs, T} | Effects]. -update_or_remove_sub(Meta, ConsumerKey, +update_or_remove_con(Meta, ConsumerKey, #consumer{cfg = #consumer_cfg{lifetime = once}, checked_out = Checked, credit = 0} = Con, @@ -2190,7 +2229,16 @@ update_or_remove_sub(Meta, ConsumerKey, % there are unsettled items so need to keep around State#?STATE{consumers = maps:put(ConsumerKey, Con, Cons)} end; -update_or_remove_sub(_Meta, ConsumerKey, +update_or_remove_con(_Meta, ConsumerKey, + #consumer{status = fading, + 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) -> @@ -2234,7 +2282,7 @@ update_consumer(Meta, ConsumerKey, {Tag, Pid}, ConsumerMeta, credit = Credit, delivery_count = DeliveryCount} end, - {Consumer, update_or_remove_sub(Meta, ConsumerKey, Consumer, State0)}; + {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}, @@ -2248,7 +2296,7 @@ update_consumer(Meta, ConsumerKey, {Tag, Pid}, ConsumerMeta, {ConsumerKey, #consumer{status = up} = Consumer0} -> Consumer = merge_consumer(Meta, Consumer0, ConsumerMeta, Spec, Priority), - {Consumer, update_or_remove_sub(Meta, ConsumerKey, Consumer, State0)}; + {Consumer, update_or_remove_con(Meta, ConsumerKey, Consumer, State0)}; 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 @@ -2256,7 +2304,7 @@ update_consumer(Meta, ConsumerKey, {Tag, Pid}, ConsumerMeta, Consumer0 = maps:get(ConsumerKey, Cons0), Consumer = merge_consumer(Meta, Consumer0, ConsumerMeta, Spec, Priority), - {Consumer, update_or_remove_sub(Meta, ConsumerKey, Consumer, State0)}; + {Consumer, update_or_remove_con(Meta, ConsumerKey, Consumer, State0)}; _ -> %% add as a new waiting consumer Credit = included_credit(Mode), @@ -2271,9 +2319,37 @@ update_consumer(Meta, ConsumerKey, {Tag, Pid}, ConsumerMeta, delivery_count = DeliveryCount}, {Consumer, State0#?STATE{waiting_consumers = - Waiting ++ [{ConsumerKey, Consumer}]}} + add_waiting({ConsumerKey, Consumer}, 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), @@ -2499,13 +2575,16 @@ is_expired(_Ts, _State) -> false. %%TODO: provide first class means of configuring -get_priority_from_args(#{args := Args}) -> +get_priority(#{priority := Priority}) -> + Priority; +get_priority(#{args := Args}) -> + %% fallback, v3 option case rabbit_misc:table_lookup(Args, <<"x-priority">>) of {_Key, Value} -> Value; _ -> 0 end; -get_priority_from_args(_) -> +get_priority(_) -> 0. notify_decorators_effect(QName, MaxActivePriority, IsEmpty) -> @@ -2678,5 +2757,3 @@ maps_search(Pred, {K, V, I}) -> end; maps_search(Pred, Map) when is_map(Map) -> maps_search(Pred, maps:next(maps:iterator(Map))). - ->>>>>>> e9e71cc724 (QQ: add v4 ff and new more compact enqueue command.) diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index d68170d60c8e..7263e43cbe99 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -87,7 +87,8 @@ -type consumer_meta() :: #{ack => boolean(), username => binary(), prefetch => non_neg_integer(), - args => list() + args => list(), + priority => non_neg_integer() % %% set if and only if credit API v2 is in use % initial_delivery_count => rabbit_queue_type:delivery_count() }. @@ -122,7 +123,7 @@ -record(consumer, {cfg = #consumer_cfg{}, - status = up :: up | suspected_down | cancelled | waiting, + status = up :: up | suspected_down | cancelled | fading, next_msg_id = 0 :: msg_id(), checked_out = #{} :: #{msg_id() => msg()}, %% max number of messages that can be sent diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 22b0fbe4f9c2..b0b9bda4767b 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -881,10 +881,16 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) -> 0 end, + Priority = case rabbit_misc:table_lookup(Args, <<"x-priority">>) of + {_Key, Value} -> + Value; + _ -> 0 + end, ConsumerMeta = #{ack => AckRequired, prefetch => Prefetch, args => Args, - username => ActingUser}, + username => ActingUser, + priority => Priority}, {ok, _Infos, QState} = rabbit_fifo_client:checkout(ConsumerTag, Mode, ConsumerMeta, QState0), diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index 728ebead0648..cb7f6b1ee6cf 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -90,7 +90,9 @@ groups() -> leader_locator_policy, status, format, - add_member_2 + add_member_2, + single_active_consumer_priority_take_over, + single_active_consumer_priority ] ++ all_tests()}, {cluster_size_5, [], [start_queue, @@ -947,6 +949,7 @@ publish_confirm(Ch, QName, Timeout) -> ct:pal("NOT CONFIRMED! ~ts", [QName]), fail after Timeout -> + flush(1), exit(confirm_timeout) end. @@ -994,6 +997,120 @@ consume_in_minority(Config) -> rabbit_quorum_queue:restart_server({RaName, Server2}), ok. +single_active_consumer_priority_take_over(Config) -> + check_quorum_queues_v4_compat(Config), + + [Server0, Server1, _Server2] = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Ch1 = rabbit_ct_client_helpers:open_channel(Config, Server0), + Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server1), + QName = ?config(queue_name, Config), + Q1 = <>, + RaNameQ1 = binary_to_atom(<<"%2F", "_", Q1/binary>>, utf8), + QueryFun = fun rabbit_fifo:query_single_active_consumer/1, + Args = [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-single-active-consumer">>, bool, true}], + ?assertEqual({'queue.declare_ok', Q1, 0, 0}, declare(Ch1, Q1, Args)), + ok = subscribe(Ch1, Q1, false, <<"ch1-ctag1">>, [{"x-priority", byte, 1}]), + ?assertMatch({ok, {_, {value, {<<"ch1-ctag1">>, _}}}, _}, + rpc:call(Server0, ra, local_query, [RaNameQ1, QueryFun])), + #'confirm.select_ok'{} = amqp_channel:call(Ch2, #'confirm.select'{}), + publish_confirm(Ch2, Q1), + %% higher priority consumer attaches + ok = subscribe(Ch2, Q1, false, <<"ch2-ctag1">>, [{"x-priority", byte, 3}]), + + %% Q1 should still have Ch1 as consumer as it has pending messages + ?assertMatch({ok, {_, {value, {<<"ch1-ctag1">>, _}}}, _}, + rpc:call(Server0, ra, local_query, + [RaNameQ1, QueryFun])), + + %% ack the message + receive + {#'basic.deliver'{consumer_tag = <<"ch1-ctag1">>, + delivery_tag = DeliveryTag}, _} -> + amqp_channel:cast(Ch1, #'basic.ack'{delivery_tag = DeliveryTag, + multiple = false}) + after 5000 -> + flush(1), + exit(basic_deliver_timeout) + end, + + ?awaitMatch({ok, {_, {value, {<<"ch2-ctag1">>, _}}}, _}, + rpc:call(Server0, ra, local_query, [RaNameQ1, QueryFun]), + ?DEFAULT_AWAIT), + ok. + +single_active_consumer_priority(Config) -> + check_quorum_queues_v4_compat(Config), + [Server0, Server1, Server2] = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch1 = rabbit_ct_client_helpers:open_channel(Config, Server0), + Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server1), + Ch3 = rabbit_ct_client_helpers:open_channel(Config, Server2), + QName = ?config(queue_name, Config), + Q1 = <>, + Q2 = <>, + Q3 = <>, + Args = [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-single-active-consumer">>, bool, true}], + ?assertEqual({'queue.declare_ok', Q1, 0, 0}, declare(Ch1, Q1, Args)), + ?assertEqual({'queue.declare_ok', Q2, 0, 0}, declare(Ch2, Q2, Args)), + ?assertEqual({'queue.declare_ok', Q3, 0, 0}, declare(Ch3, Q3, Args)), + + ok = subscribe(Ch1, Q1, false, <<"ch1-ctag1">>, [{"x-priority", byte, 3}]), + ok = subscribe(Ch1, Q2, false, <<"ch1-ctag2">>, [{"x-priority", byte, 2}]), + ok = subscribe(Ch1, Q3, false, <<"ch1-ctag3">>, [{"x-priority", byte, 1}]), + + + ok = subscribe(Ch2, Q1, false, <<"ch2-ctag1">>, [{"x-priority", byte, 1}]), + ok = subscribe(Ch2, Q2, false, <<"ch2-ctag2">>, [{"x-priority", byte, 3}]), + ok = subscribe(Ch2, Q3, false, <<"ch2-ctag3">>, [{"x-priority", byte, 2}]), + + ok = subscribe(Ch3, Q1, false, <<"ch3-ctag1">>, [{"x-priority", byte, 2}]), + ok = subscribe(Ch3, Q2, false, <<"ch3-ctag2">>, [{"x-priority", byte, 1}]), + ok = subscribe(Ch3, Q3, false, <<"ch3-ctag3">>, [{"x-priority", byte, 3}]), + + + RaNameQ1 = binary_to_atom(<<"%2F", "_", Q1/binary>>, utf8), + RaNameQ2 = binary_to_atom(<<"%2F", "_", Q2/binary>>, utf8), + RaNameQ3 = binary_to_atom(<<"%2F", "_", Q3/binary>>, utf8), + %% assert each queue has a different consumer + QueryFun = fun rabbit_fifo:query_single_active_consumer/1, + + %% Q1 should have the consumer on Ch1 + ?assertMatch({ok, {_, {value, {<<"ch1-ctag1">>, _}}}, _}, + rpc:call(Server0, ra, local_query, [RaNameQ1, QueryFun])), + + %% Q2 Ch2 + ?assertMatch({ok, {_, {value, {<<"ch2-ctag2">>, _}}}, _}, + rpc:call(Server1, ra, local_query, [RaNameQ2, QueryFun])), + + %% Q3 Ch3 + ?assertMatch({ok, {_, {value, {<<"ch3-ctag3">>, _}}}, _}, + rpc:call(Server2, ra, local_query, [RaNameQ3, QueryFun])), + + %% close Ch3 + _ = rabbit_ct_client_helpers:close_channel(Ch3), + flush(100), + + %% assert Q3 has Ch2 (priority 2) as consumer + ?assertMatch({ok, {_, {value, {<<"ch2-ctag3">>, _}}}, _}, + rpc:call(Server2, ra, local_query, [RaNameQ3, QueryFun])), + + %% close Ch2 + _ = rabbit_ct_client_helpers:close_channel(Ch2), + flush(100), + + %% assert all queues as has Ch1 as consumer + ?assertMatch({ok, {_, {value, {<<"ch1-ctag1">>, _}}}, _}, + rpc:call(Server0, ra, local_query, [RaNameQ1, QueryFun])), + ?assertMatch({ok, {_, {value, {<<"ch1-ctag2">>, _}}}, _}, + rpc:call(Server0, ra, local_query, [RaNameQ2, QueryFun])), + ?assertMatch({ok, {_, {value, {<<"ch1-ctag3">>, _}}}, _}, + rpc:call(Server0, ra, local_query, [RaNameQ3, QueryFun])), + ok. + reject_after_leader_transfer(Config) -> [Server0, Server1, Server2] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -3627,13 +3744,20 @@ consume_empty(Ch, Queue, NoAck) -> no_ack = NoAck})). subscribe(Ch, Queue, NoAck) -> + subscribe(Ch, Queue, NoAck, <<"ctag">>, []). + +subscribe(Ch, Queue, NoAck, Tag, Args) -> amqp_channel:subscribe(Ch, #'basic.consume'{queue = Queue, no_ack = NoAck, - consumer_tag = <<"ctag">>}, + arguments = Args, + consumer_tag = Tag}, self()), receive - #'basic.consume_ok'{consumer_tag = <<"ctag">>} -> + #'basic.consume_ok'{consumer_tag = Tag} -> ok + after 30000 -> + flush(100), + exit(subscribe_timeout) end. qos(Ch, Prefetch, Global) -> @@ -3746,3 +3870,12 @@ basic_get(Ch, Q, NoAck, Attempt) -> timer:sleep(100), basic_get(Ch, Q, NoAck, Attempt - 1) end. + +check_quorum_queues_v4_compat(Config) -> + case rabbit_ct_broker_helpers:is_feature_flag_enabled(Config, + quorum_queues_4) of + false -> + throw({skip, "test needs feature flag quorum_queues_v4"}); + true -> + ok + end. diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 1db936bfbdb8..0696b61dfb7a 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -21,7 +21,7 @@ all() -> [ - {group, machine_version_4}, + {group, tests}, {group, machine_version_conversion} ]. @@ -33,13 +33,13 @@ all_tests() -> groups() -> [ - {machine_version_4, [shuffle], all_tests()}, + {tests, [shuffle], all_tests()}, {machine_version_conversion, [shuffle], [convert_v2_to_v3, convert_v3_to_v4]} ]. -init_per_group(machine_version_4, Config) -> +init_per_group(tests, Config) -> [{machine_version, 4} | Config]; init_per_group(machine_version_conversion, Config) -> Config. @@ -84,6 +84,11 @@ end_per_testcase(_Group, _Config) -> (_) -> false end, Effects))). +-define(ASSERT(Guard, Fun), + {assert, fun (S) -> ?assertMatch(Guard, S), Fun end}). +-define(ASSERT(Guard), + ?ASSERT(Guard, fun () -> ok end)). + test_init(Name) -> init(#{name => Name, queue_resource => rabbit_misc:r("/", queue, atom_to_binary(Name)), @@ -882,7 +887,7 @@ single_active_consumer_basic_get_test(Config) -> single_active_consumer_revive_test(Config) -> S0 = init(#{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME, utf8)), + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), single_active_consumer_on => true}), Cid1 = {<<"one">>, self()}, Cid2 = {<<"two">>, self()}, @@ -1044,7 +1049,6 @@ single_active_consumer_test(Config) -> single_active_consumer_cancel_consumer_when_channel_is_down_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), - release_cursor_interval => 0, single_active_consumer_on => true}), Pid1 = spawn(fun() -> ok end), @@ -1062,12 +1066,14 @@ single_active_consumer_cancel_consumer_when_channel_is_down_test(Config) -> {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{})}, - {CK4, make_checkout(C4, {auto, {simple_prefetch, 1}}, #{})} + {CK4, make_checkout(C4, {auto, {simple_prefetch, 1}}, #{})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}}), + % the channel of the active consumer goes down + {?LINE, {down, Pid1, noproc}} ], - {State1, _} = run_log(Config, State0, Entries), + {State2, Effects} = run_log(Config, State0, Entries), - % the channel of the active consumer goes down - {State2, _, Effects} = apply(meta(Config, 2), {down, Pid1, noproc}, State1), + % {State2, _, Effects} = apply(meta(Config, 2), {down, Pid1, noproc}, State1), % fell back to another consumer ?assertEqual(1, map_size(State2#rabbit_fifo.consumers)), % there are still waiting consumers @@ -1079,8 +1085,11 @@ single_active_consumer_cancel_consumer_when_channel_is_down_test(Config) -> ?ASSERT_EFF({mod_call, rabbit_quorum_queue, update_consumer_handler, _}, Effects), + ct:pal("STate2 ~p", [State2]), % the channel of the active consumer and a waiting consumer goes down {State3, _, Effects2} = apply(meta(Config, ?LINE), {down, Pid2, noproc}, State2), + ct:pal("STate3 ~p", [State3]), + ct:pal("Effects2 ~p", [Effects2]), % fell back to another consumer ?assertEqual(1, map_size(State3#rabbit_fifo.consumers)), % no more waiting consumer @@ -1188,29 +1197,20 @@ single_active_consumer_all_disconnected_test(Config) -> {CK1, {_, C1Pid} = C1} = {?LINE, {?LINE_B, test_util:fake_pid(n1)}}, {CK2, {_, C2Pid} = C2} = {?LINE, {?LINE_B, test_util:fake_pid(n2)}}, - Entries = [ - {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, - {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})} - ], + Entries = + [ + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}}), + {?LINE, {down, C1Pid, noconnection}}, + ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = up}}}), + {?LINE, {down, C2Pid, noconnection}}, + ?ASSERT(#rabbit_fifo{consumers = C} when map_size(C) == 0), + {?LINE, {nodeup, node(C2Pid)}}, + ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = up, + credit = 1}}}) + ], {State1, _} = run_log(Config, State0, Entries), - - %% assert the consumer is up - ?assertMatch(#{CK1 := #consumer{status = up}}, State1#rabbit_fifo.consumers), - - % simulate node goes down - {State2, _, _} = apply(meta(Config, 5), {down, C1Pid, noconnection}, State1), - %% assert the consumer fails over to the consumer on n2 - ?assertMatch(#{CK2 := #consumer{status = up}}, State2#rabbit_fifo.consumers), - {State3, _, _} = apply(meta(Config, 6), {down, C2Pid, noconnection}, State2), - %% assert these no active consumer after both nodes are maked as down - ?assertMatch([], maps:to_list(State3#rabbit_fifo.consumers)), - %% n2 comes back - {State4, _, _} = apply(meta(Config, 7), {nodeup, node(C2Pid)}, State3), - %% ensure n2 is the active consumer as this node as been registered - %% as up again - ?assertMatch([{CK2, #consumer{status = up, - credit = 1}}], - maps:to_list(State4#rabbit_fifo.consumers)), ok. single_active_consumer_state_enter_leader_include_waiting_consumers_test(Config) -> @@ -1526,6 +1526,390 @@ single_active_with_credited_v2_test(Config) -> rabbit_fifo:query_waiting_consumers(State)), ok. +single_active_settle_after_cancel_test(Config) -> + S0 = init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => true}), + + Pid1 = test_util:fake_pid(node()), + % % adding some consumers + E1Idx = ?LINE, + {CK1, C1} = {?LINE, {?LINE_B, self()}}, + {CK2, C2} = {?LINE, {?LINE_B, self()}}, + Entries = + [ + {E1Idx , rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, + %% add a consumer + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{next_msg_id = 1, + status = up, + checked_out = Ch}}} + when map_size(Ch) == 1), + %% add another consumer + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = [{CK2, _}]}), + + %% cancel C1 + {?LINE, make_checkout(C1, cancel, #{})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = cancelled}, + CK2 := #consumer{status = up}}, + waiting_consumers = []}), + %% settle the message, C1 one should be completely removed + {?LINE, rabbit_fifo:make_settle(CK1, [0])}, + ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = up}} = C, + waiting_consumers = []} + when map_size(C) == 1) + + ], + {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1), + + ok. + +single_active_consumer_priority_test(Config) -> + S0 = init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => true}), + + Pid1 = test_util:fake_pid(node()), + % % adding some consumers + {CK1, C1} = {?LINE, {?LINE_B, self()}}, + {CK2, C2} = {?LINE, {?LINE_B, self()}}, + E1Idx = ?LINE, + {CK3, C3} = {?LINE, {?LINE_B, self()}}, + Entries = + [ + %% add a consumer + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{priority => 1})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = []}), + + %% add a consumer with a higher priority, assert it becomes active + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 2})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = up}}, + waiting_consumers = [_]}), + + %% enqueue a message + {E1Idx , rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, + ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{next_msg_id = 1, + status = up, + checked_out = Ch}}} + when map_size(Ch) == 1), + + %% add en even higher consumer, but the current active has a message pending + %% so can't be immedately replaced + {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{priority => 3})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = fading}}, + waiting_consumers = [_, _]}), + %% settle the message, the higher priority should become the active, + %% completing the replacement + {?LINE, rabbit_fifo:make_settle(CK2, [0])}, + ?ASSERT(#rabbit_fifo{consumers = #{CK3 := #consumer{status = up, + checked_out = Ch}}, + waiting_consumers = [_, _]} + when map_size(Ch) == 0) + + ], + {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1), + + ok. + + +single_active_consumer_priority_cancel_active_test(Config) -> + S0 = init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => true}), + + % % adding some consumers + {CK1, C1} = {?LINE, {?LINE_B, self()}}, + {CK2, C2} = {?LINE, {?LINE_B, self()}}, + {CK3, C3} = {?LINE, {?LINE_B, self()}}, + Entries = + [ + %% add a consumer + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{priority => 2})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = []}), + + %% add two consumers each with a lower priority + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 1})}, + {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{priority => 0})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = [_, _]}), + + {?LINE, make_checkout(C1, cancel, #{})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = up}}, + waiting_consumers = [{CK3, _}]}) + ], + {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1), + + ok. + +single_active_consumer_update_priority_test(Config) -> + S0 = init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => true}), + + Pid1 = test_util:fake_pid(node()), + % % adding some consumers + {CK1, C1} = {?LINE, {?LINE_B, self()}}, + {CK2, C2} = {?LINE, {?LINE_B, self()}}, + Entries = + [ + %% add a consumer + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{priority => 2})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = []}), + %% add abother consumer with lower priority + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 1})}, + %% update the current active consumer to lower priority + {?LINE, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{priority => 0})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = up}}, + waiting_consumers = [_]}), + %% back to original priority + {?LINE, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{priority => 2})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = [_]}), + {?LINE , rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{checked_out = Ch}}, + waiting_consumers = [{CK2, _}]} + when map_size(Ch) == 1), + %% update priority for C2 + {?LINE, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 3})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{checked_out = Ch}}, + waiting_consumers = [{CK2, _}]} + when map_size(Ch) == 1), + %% settle should cause the existing active to be replaced + {?LINE, rabbit_fifo:make_settle(CK1, [0])}, + ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = up}}, + waiting_consumers = [{CK1, _}]}) + ], + {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1), + ok. + +single_active_consumer_fading_resumes_after_cancel_test(Config) -> + S0 = init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => true}), + + Pid1 = test_util:fake_pid(node()), + % % adding some consumers + {CK1, C1} = {?LINE, {?LINE_B, self()}}, + {CK2, C2} = {?LINE, {?LINE_B, self()}}, + Entries = + [ + %% add a consumer + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{priority => 1})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = []}), + + %% enqueue a message + {?LINE , rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, + + %% add a consumer with a higher priority, current is fading + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 2})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = fading}}, + waiting_consumers = [{CK2, _}]}), + + %% C2 cancels + {?LINE, make_checkout(C2, cancel, #{})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = fading, + checked_out = Ch}}, + waiting_consumers = []} + when map_size(Ch) == 1), + + %% settle + {?LINE, rabbit_fifo:make_settle(CK1, [0])}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up, + credit = 1}}, + waiting_consumers = []}) + ], + + {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1), + + ok. + +single_active_consumer_higher_waiting_disconnected_test(Config) -> + S0 = init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => true}), + + Pid1 = test_util:fake_pid(node()), + C1Pid = test_util:fake_pid(n1@banana), + C2Pid = test_util:fake_pid(n2@banana), + % % adding some consumers + {CK1, C1} = {?LINE, {?LINE_B, C1Pid}}, + {CK2, C2} = {?LINE, {?LINE_B, C2Pid}}, + Entries = + [ + %% add a consumer + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{priority => 1})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = []}), + + %% enqueue a message + {?LINE , rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, + + %% add a consumer with a higher priority, current is fading + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 2})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = fading}}, + waiting_consumers = [{CK2, _}]}), + %% C2 is disconnected, + {?LINE, {down, C2Pid, noconnection}}, + ?ASSERT( + #rabbit_fifo{consumers = #{CK1 := #consumer{status = fading}}, + waiting_consumers = [{CK2, #consumer{status = suspected_down}}]}), + %% settle + {?LINE, rabbit_fifo:make_settle(CK1, [0])}, + %% C1 should be reactivated + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up, + credit = 1}}, + waiting_consumers = [_]}), + %% C2 comes back up and takes over + {?LINE, {nodeup, n2@banana}}, + ?ASSERT( + #rabbit_fifo{consumers = #{CK2 := #consumer{status = up}}, + waiting_consumers = [{CK1, #consumer{status = up}}]}) + ], + {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1), + + ok. + +single_active_consumer_fading_disconnected_test(Config) -> + S0 = init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => true}), + + Pid1 = test_util:fake_pid(node()), + C1Pid = test_util:fake_pid(n1@banana), + C2Pid = test_util:fake_pid(n2@banana), + % % adding some consumers + {CK1, C1} = {?LINE, {?LINE_B, C1Pid}}, + {CK2, C2} = {?LINE, {?LINE_B, C2Pid}}, + Entries = + [ + %% add a consumer + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{priority => 1})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = []}), + + %% enqueue a message + {?LINE , rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, + + %% add a consumer with a higher priority, current is fading + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 2})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = fading}}, + waiting_consumers = [{CK2, _}]}), + %% C1 is disconnected, + {?LINE, {down, C1Pid, noconnection}}, + ?ASSERT( + #rabbit_fifo{consumers = #{CK2 := #consumer{status = up, + checked_out = Ch2}}, + waiting_consumers = + [{CK1, #consumer{status = suspected_down, + checked_out = Ch1}}]} + when map_size(Ch2) == 1 andalso + map_size(Ch1) == 0), + %% C1 settles which will be ignored + {?LINE, rabbit_fifo:make_settle(CK1, [0])}, + ?ASSERT( + #rabbit_fifo{consumers = #{CK2 := #consumer{status = up, + checked_out = Ch2}}, + waiting_consumers = + [{CK1, #consumer{status = suspected_down, + checked_out = Ch1}}]} + when map_size(Ch2) == 1 andalso + map_size(Ch1) == 0), + % %% C1 comes back up + {?LINE, {nodeup, n1@banana}}, + ?ASSERT( + #rabbit_fifo{consumers = #{CK2 := #consumer{status = up}}, + waiting_consumers = [{CK1, #consumer{status = up}}]}) + ], + {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1), + + ok. + +single_active_consumer_fading_receives_no_further_messages_test(Config) -> + S0 = init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => true}), + + Pid1 = test_util:fake_pid(node()), + C1Pid = test_util:fake_pid(n1@banana), + C2Pid = test_util:fake_pid(n2@banana), + % % adding some consumers + {CK1, C1} = {?LINE, {?LINE_B, C1Pid}}, + {CK2, C2} = {?LINE, {?LINE_B, C2Pid}}, + Entries = + [ + %% add a consumer, with plenty of prefetch + {CK1, make_checkout(C1, {auto, {simple_prefetch, 10}}, #{priority => 1})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = []}), + + %% enqueue a message + {?LINE, rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, + + %% add a consumer with a higher priority, current is fading + {CK2, make_checkout(C2, {auto, {simple_prefetch, 10}}, #{priority => 2})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = fading, + checked_out = Ch}}, + waiting_consumers = [{CK2, _}]} + when map_size(Ch) == 1), + + %% enqueue another message + {?LINE, rabbit_fifo:make_enqueue(Pid1, 2, msg2)}, + %% message should not be assinged to fading consumer + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = fading, + checked_out = Ch}}, + waiting_consumers = [{CK2, _}]} + when map_size(Ch) == 1) + + ], + + {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1), + ok. + +single_active_consumer_credited_favour_with_credit_test(Config) -> + S0 = init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => true}), + + C1Pid = test_util:fake_pid(n1@banana), + C2Pid = test_util:fake_pid(n2@banana), + C3Pid = test_util:fake_pid(n3@banana), + % % adding some consumers + {CK1, C1} = {?LINE, {?LINE_B, C1Pid}}, + {CK2, C2} = {?LINE, {?LINE_B, C2Pid}}, + {CK3, C3} = {?LINE, {?LINE_B, C3Pid}}, + Entries = + [ + %% add a consumer + {CK1, make_checkout(C1, {auto, {credited, 0}}, #{priority => 3})}, + {CK2, make_checkout(C2, {auto, {credited, 0}}, #{priority => 1})}, + {CK3, make_checkout(C3, {auto, {credited, 0}}, #{priority => 1})}, + %% waiting are sorted by arrival order + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = [{CK2, _}, {CK3, _}]}), + + %% give credit to C3 + {?LINE , rabbit_fifo:make_credit(CK3, 1, 0, false)}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}, + waiting_consumers = [{CK3, _}, {CK2, _}]}), + %% cancel the current active consumer + {CK1, make_checkout(C1, cancel, #{})}, + %% C3 should become active due having credits + ?ASSERT(#rabbit_fifo{consumers = #{CK3 := #consumer{status = up, + credit = 1}}, + waiting_consumers = [{CK2, _}]}) + ], + + {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1), + ok. + + + register_enqueuer_test(Config) -> State0 = init(#{name => ?FUNCTION_NAME, queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), @@ -1766,17 +2150,26 @@ checkout_reply(Oth) -> Oth. run_log(Config, InitState, Entries) -> - lists:foldl(fun ({Idx, E}, {Acc0, Efx0}) -> - case apply(meta(Config, Idx, Idx, {notify, Idx, self()}), - E, Acc0) of - {Acc, _, Efx} when is_list(Efx) -> - {Acc, Efx0 ++ Efx}; - {Acc, _, Efx} -> - {Acc, Efx0 ++ [Efx]}; - {Acc, _} -> - {Acc, Efx0} - end - end, {InitState, []}, Entries). + run_log(Config, InitState, Entries, fun (_) -> true end). +run_log(Config, InitState, Entries, Invariant) -> + lists:foldl( + fun ({assert, Fun}, {Acc0, Efx0}) -> + _ = Fun(Acc0), + {Acc0, Efx0}; + ({Idx, E}, {Acc0, Efx0}) -> + case apply(meta(Config, Idx, Idx, {notify, Idx, self()}), + E, Acc0) of + {Acc, _, Efx} when is_list(Efx) -> + ?assert(Invariant(Acc)), + {Acc, Efx0 ++ Efx}; + {Acc, _, Efx} -> + ?assert(Invariant(Acc)), + {Acc, Efx0 ++ [Efx]}; + {Acc, _} -> + ?assert(Invariant(Acc)), + {Acc, Efx0} + end + end, {InitState, []}, Entries). %% AUX Tests @@ -2166,3 +2559,9 @@ make_checkout(C, S, M) -> rabbit_fifo:make_checkout(C, S, M). cid(A) when is_atom(A) -> atom_to_binary(A, utf8). + +single_active_invariant( #rabbit_fifo{consumers = Cons}) -> + 1 >= map_size(maps:filter(fun (_, #consumer{status = S}) -> + S == up + end, Cons)). + diff --git a/deps/rabbit/test/single_active_consumer_SUITE.erl b/deps/rabbit/test/single_active_consumer_SUITE.erl index 6945d213b85a..ac682ad95712 100644 --- a/deps/rabbit/test/single_active_consumer_SUITE.erl +++ b/deps/rabbit/test/single_active_consumer_SUITE.erl @@ -11,13 +11,15 @@ -include_lib("eunit/include/eunit.hrl"). -include_lib("amqp_client/include/amqp_client.hrl"). +-compile(nowarn_export_all). -compile(export_all). -define(TIMEOUT, 30000). all() -> [ - {group, classic_queue}, {group, quorum_queue} + {group, classic_queue}, + {group, quorum_queue} ]. groups() -> From daade0e517b89f99bc298193f80e600ea2bafe3b Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 1 May 2024 14:16:51 +0100 Subject: [PATCH 05/45] QQ: add new consumer cancel option: 'remove' This option immediately removes and returns all messages for a consumer instead of the softer 'cancel' option which keeps the consumer around until all pending messages have been either settled or returned. This involves a change to the rabbit_queue_type:cancel/5 API to rabbit_queue_type:cancel/3. --- deps/rabbit/src/rabbit_fifo.erl | 25 ++++++--- deps/rabbit/src/rabbit_fifo.hrl | 2 +- deps/rabbit/src/rabbit_fifo_client.erl | 37 +++++++++++--- deps/rabbit/src/rabbit_quorum_queue.erl | 2 +- deps/rabbit/test/amqp_client_SUITE.erl | 8 +-- deps/rabbit/test/rabbit_fifo_SUITE.erl | 12 +++++ deps/rabbit/test/rabbit_fifo_int_SUITE.erl | 59 ++++++++++++++++++++++ 7 files changed, 124 insertions(+), 21 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index e23c4354e052..fbbd3171cfe0 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -477,13 +477,15 @@ apply(#{index := Index, end end; apply(#{index := Idx} = Meta, - #checkout{spec = cancel, - consumer_id = ConsumerId}, State0) -> + #checkout{spec = Spec, + consumer_id = ConsumerId}, State0) + when Spec == cancel orelse + Spec == remove -> case consumer_key_from_id(ConsumerId, State0) of {ok, ConsumerKey} -> {State1, Effects1} = activate_next_consumer( cancel_consumer(Meta, ConsumerKey, State0, [], - consumer_cancel)), + Spec)), Reply = {ok, consumer_cancel_info(ConsumerKey, State1)}, {State, _, Effects} = checkout(Meta, State0, State1, Effects1), update_smallest_raft_index(Idx, Reply, State, Effects); @@ -1533,14 +1535,14 @@ maybe_return_all(#{system_time := Ts} = Meta, ConsumerKey, #consumer{cfg = CCfg} = Consumer, S0, Effects0, Reason) -> case Reason of - consumer_cancel -> + cancel -> {update_or_remove_con( Meta, ConsumerKey, Consumer#consumer{cfg = CCfg#consumer_cfg{lifetime = once}, credit = 0, status = cancelled}, S0), Effects0}; - down -> + _ -> {S1, Effects1} = return_all(Meta, S0, Effects0, ConsumerKey, Consumer), {S1#?STATE{consumers = maps:remove(ConsumerKey, S1#?STATE.consumers), last_active = Ts}, @@ -2433,9 +2435,16 @@ make_enqueue(Pid, Seq, Msg) -> make_register_enqueuer(Pid) -> #register_enqueuer{pid = Pid}. --spec make_checkout(consumer_id(), - checkout_spec(), consumer_meta()) -> protocol(). -make_checkout({_, _} = ConsumerId, Spec, Meta) -> +-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 + make_checkout(ConsumerId, cancel, Meta); + _ -> + Spec0 + end, #checkout{consumer_id = ConsumerId, spec = Spec, meta = Meta}. diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index 7263e43cbe99..58d4be87116b 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -80,7 +80,7 @@ credited | simple_prefetch} | {dequeue, settled | unsettled} | - cancel | + cancel | remove | %% new v4 format {once | auto, credit_mode()}. diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index 786db1fa88d3..a48173d44165 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -16,6 +16,7 @@ init/2, checkout/4, cancel_checkout/2, + cancel_checkout/3, enqueue/3, enqueue/4, dequeue/4, @@ -445,6 +446,9 @@ credit(ConsumerTag, DeliveryCount, Credit, Drain, State) -> Cmd = rabbit_fifo:make_credit(ConsumerKey, Credit, DeliveryCount, Drain), {send_command(ServerId, undefined, Cmd, normal, State), []}. +cancel_checkout(ConsumerTag, State) -> + cancel_checkout(ConsumerTag, cancel, State). + %% @doc Cancels a checkout with the rabbit_fifo queue for the consumer tag %% %% This is a synchronous call. I.e. the call will block until the command @@ -454,18 +458,35 @@ credit(ConsumerTag, DeliveryCount, Credit, Drain, State) -> %% @param State The {@module} state. %% %% @returns `{ok, State}' or `{error | timeout, term()}' --spec cancel_checkout(rabbit_types:ctag(), state()) -> +-spec cancel_checkout(rabbit_types:ctag(), Reason :: cancel | remove, state()) -> {ok, state()} | {error | timeout, term()}. -cancel_checkout(ConsumerTag, #state{consumers = Consumers} = State0) -> +cancel_checkout(ConsumerTag, Reason, + #state{consumers = Consumers, + unsent_commands = Unsent} = State0) + when is_atom(Reason) -> case Consumers of - #{ConsumerTag := #consumer{}} -> + #{ConsumerTag := #consumer{key = Cid}} -> Servers = sorted_servers(State0), ConsumerId = {ConsumerTag, self()}, - %% TODO: send any pending commands for consumer - %% checkout always uses the ConsumerId, rather than the key - Cmd = rabbit_fifo:make_checkout(ConsumerId, cancel, #{}), - State = State0#state{consumers = maps:remove(ConsumerTag, Consumers)}, - case try_process_command(Servers, Cmd, State0) of + %% send any pending commands for consumer before cancelling + Commands = case Unsent of + #{Cid := {Settled, Returns, Discards}} -> + add_command(Cid, settle, Settled, + add_command(Cid, return, Returns, + add_command(Cid, discard, + Discards, []))); + _ -> + [] + end, + ServerId = pick_server(State0), + %% send all the settlements, discards and returns + State1 = lists:foldl(fun (C, S0) -> + send_command(ServerId, undefined, C, + normal, S0) + end, State0, Commands), + Cmd = rabbit_fifo:make_checkout(ConsumerId, Reason, #{}), + State = State1#state{consumers = maps:remove(ConsumerTag, Consumers)}, + case try_process_command(Servers, Cmd, State) of {ok, _, Leader} -> {ok, State#state{leader = Leader}}; Err -> diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index b0b9bda4767b..1460d69c7866 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -133,7 +133,7 @@ -define(DELETE_TIMEOUT, 5000). -define(MEMBER_CHANGE_TIMEOUT, 20_000). -define(SNAPSHOT_INTERVAL, 8192). %% the ra default is 4096 --define(UNLIMITED_PREFETCH_COUNT, 2000). %% something large for ra +% -define(UNLIMITED_PREFETCH_COUNT, 2000). %% something large for ra %%----------- QQ policies --------------------------------------------------- diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl index 39c462939b0c..8346bc3186dc 100644 --- a/deps/rabbit/test/amqp_client_SUITE.erl +++ b/deps/rabbit/test/amqp_client_SUITE.erl @@ -2076,9 +2076,11 @@ single_active_consumer(QType, Config) -> %% In addition to consumer cancellation, detaching a link therefore causes in flight deliveries to be requeued. %% That's okay given that AMQP receivers can stop a link (figure 2.46) before detaching. %% -%% Note that this behaviour is different from merely consumer cancellation in AMQP legacy: -%% "After a consumer is cancelled there will be no future deliveries dispatched to it. Note that there can -%% still be "in flight" deliveries dispatched previously. Cancelling a consumer will neither discard nor requeue them." +%% Note that this behaviour is different from merely consumer cancellation in +%% AMQP legacy: +%% "After a consumer is cancelled there will be no future deliveries dispatched to it. +%% Note that there can still be "in flight" deliveries dispatched previously. +%% Cancelling a consumer will neither discard nor requeue them." %% [https://www.rabbitmq.com/consumers.html#unsubscribing] detach_requeues_one_session_classic_queue(Config) -> detach_requeue_one_session(<<"classic">>, Config). diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 0696b61dfb7a..948b6d450183 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -598,6 +598,18 @@ down_with_noproc_consumer_returns_unsettled_test(Config) -> ?ASSERT_EFF({monitor, process, _}, Effects), ok. +removed_consumer_returns_unsettled_test(Config) -> + Cid = {?FUNCTION_NAME_B, self()}, + {State0, _} = enq(Config, 1, 1, second, test_init(test)), + {State1, #{key := CKey}, + [{monitor, process, _Pid} | _]} = checkout(Config, ?LINE, Cid, 1, State0), + Remove = rabbit_fifo:make_checkout(Cid, remove, #{}), + {State2, _, _} = apply(meta(Config, 3), Remove, State1), + {_State, #{key := CKey2}, Effects} = checkout(Config, ?LINE, Cid, 1, State2), + ?assertNotEqual(CKey, CKey2), + ?ASSERT_EFF({monitor, process, _}, Effects), + ok. + down_with_noconnection_marks_suspect_and_node_is_monitored_test(Config) -> Pid = spawn(fun() -> ok end), Cid = {?FUNCTION_NAME_B, Pid}, diff --git a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl index 25eb9ca64aba..1809de4f54c4 100644 --- a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl @@ -31,6 +31,9 @@ all_tests() -> dequeue, discard, cancel_checkout, + cancel_checkout_with_remove, + cancel_checkout_with_pending_using_cancel_reason, + cancel_checkout_with_pending_using_remove_reason, lost_delivery, credit_api_v1, credit_api_v2, @@ -413,6 +416,62 @@ cancel_checkout(Config) -> rabbit_fifo_client:dequeue(ClusterName, <<"d1">>, settled, F5), ok. +cancel_checkout_with_remove(Config) -> + 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:checkout(<<"tag">>, {simple_prefetch, 10}, + #{}, F1), + {_, _, F3} = process_ra_events(receive_ra_events(1, 1), ClusterName, F2, + [], [], fun (_, S) -> S end), + {ok, F4} = rabbit_fifo_client:cancel_checkout(<<"tag">>, remove, F3), + %% settle here to prove that message is returned by "remove" cancellation + %% and not settled by late settlement + {F5, _} = rabbit_fifo_client:settle(<<"tag">>, [0], F4), + {ok, _, {_, _, _, _, m1}, F5} = + rabbit_fifo_client:dequeue(ClusterName, <<"d1">>, settled, F5), + ok. + +cancel_checkout_with_pending_using_cancel_reason(Config) -> + cancel_checkout_with_pending(Config, cancel). + +cancel_checkout_with_pending_using_remove_reason(Config) -> + cancel_checkout_with_pending(Config, remove). + +cancel_checkout_with_pending(Config, Reason) -> + ClusterName = ?config(cluster_name, Config), + ServerId = ?config(node_id, Config), + ok = start_cluster(ClusterName, [ServerId]), + F0 = rabbit_fifo_client:init([ServerId], 4), + F1 = lists:foldl( + fun (Num, Acc0) -> + {ok, Acc, _} = rabbit_fifo_client:enqueue(ClusterName, Num, Acc0), + Acc + end, F0, lists:seq(1, 10)), + receive_ra_events(10, 0), + {ok, _, F2} = rabbit_fifo_client:checkout(<<"tag">>, {simple_prefetch, 10}, + #{}, F1), + {Msgs, _, F3} = process_ra_events(receive_ra_events(0, 1), ClusterName, F2, + [], [], fun (_, S) -> S end), + %% settling each individually should cause the client to enter the "slow" + %% state where settled msg ids are buffered internally waiting for + %% applied events + F4 = lists:foldl( + fun({_Q, _, MsgId, _, _}, Acc0) -> + {Acc, _} = rabbit_fifo_client:settle(<<"tag">>, [MsgId], Acc0), + Acc + end, F3, Msgs), + + {ok, _F4} = rabbit_fifo_client:cancel_checkout(<<"tag">>, Reason, F4), + timer:sleep(100), + {ok, Overview, _} = ra:member_overview(ServerId), + ?assertMatch(#{machine := #{num_messages := 0, + num_consumers := 0}}, Overview), + flush(), + ok. + lost_delivery(Config) -> ClusterName = ?config(cluster_name, Config), ServerId = ?config(node_id, Config), From 44e7d09c443fd2a1d85008c12f5a4067129781ad Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 7 May 2024 13:29:54 +0100 Subject: [PATCH 06/45] QQ: capture checked out time for each consumer message. This will form the basis for queue initiated consumer timeouts. --- deps/rabbit/src/rabbit_fifo.erl | 74 +++++++++++++++---------- deps/rabbit/src/rabbit_fifo.hrl | 7 ++- deps/rabbit/src/rabbit_queue_type.erl | 3 +- deps/rabbit/test/quorum_queue_SUITE.erl | 5 +- deps/rabbit/test/rabbit_fifo_SUITE.erl | 6 +- 5 files changed, 59 insertions(+), 36 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index fbbd3171cfe0..754d55f6560e 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -72,6 +72,7 @@ make_return/2, make_discard/2, make_credit/4, + make_defer/2, make_purge/0, make_purge_nodes/1, make_update_config/1, @@ -112,10 +113,13 @@ credit :: non_neg_integer(), delivery_count :: rabbit_queue_type:delivery_count(), drain :: boolean()}). +-record(defer, {consumer_key :: consumer_key(), + msg_ids :: [msg_id()]}). -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{} | @@ -127,6 +131,7 @@ #return{} | #discard{} | #credit{} | + #defer{} | #purge{} | #purge_nodes{} | #update_config{} | @@ -251,7 +256,7 @@ apply(Meta, #discard{consumer_key = ConsumerKey, case maps:get(Id, Checked, undefined) of undefined -> false; - Msg -> + ?C_MSG(_At, Msg) -> {true, Msg} end end, MsgIds), @@ -716,8 +721,8 @@ apply(#{index := Idx} = Meta, State1 = update_config(Conf, State0#?STATE{dlx = DlxState}), {State, Reply, Effects} = checkout(Meta, State0, State1, Effects0), update_smallest_raft_index(Idx, Reply, State, Effects); -apply(_Meta, {machine_version, FromVersion, ToVersion}, V0State) -> - State = convert(FromVersion, ToVersion, V0State), +apply(Meta, {machine_version, FromVersion, ToVersion}, V0State) -> + State = convert(Meta, FromVersion, ToVersion, V0State), {State, ok, [{aux, {dlx, setup}}]}; apply(#{index := IncomingRaftIdx} = Meta, {dlx, _} = Cmd, #?STATE{cfg = #cfg{dead_letter_handler = DLH}, @@ -731,9 +736,16 @@ apply(_Meta, Cmd, State) -> rabbit_log:debug("rabbit_fifo: unhandled command ~W", [Cmd, 10]), {State, ok, []}. -convert_v3_to_v4(#rabbit_fifo{} = StateV3) -> - %% nothing to convert - yet - StateV3. +convert_v3_to_v4(#{system_time := Ts}, #rabbit_fifo{consumers = Consumers0} = StateV3) -> + Consumers = maps:map( + fun (_CKey, #consumer{checked_out = Ch0} = C) -> + Ch = maps:map( + fun (_MsgId, ?MSG(_, _) = Msg) -> + ?C_MSG(Ts, Msg) + end, Ch0), + C#consumer{checked_out = Ch} + end, Consumers0), + StateV3#?MODULE{consumers = Consumers}. purge_node(Meta, Node, State, Effects) -> lists:foldl(fun(Pid, {S0, E0}) -> @@ -934,7 +946,7 @@ 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), + ?C_MSG(_At, I, H) = maps:get(K, Checked), {K, {I, H}} end || K <- lists:seq(From, To), maps:is_key(K, Checked)]; _ -> @@ -993,7 +1005,7 @@ handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, {ConsumerKey, #consumer{checked_out = Checked}} -> {Log, ToReturn} = maps:fold( - fun (MsgId, ?MSG(Idx, Header), {L0, Acc}) -> + fun (MsgId, ?C_MSG(_, Idx, Header), {L0, Acc}) -> %% it is possible this is not found if the consumer %% crashed and the message got removed case ra_log:fetch(Idx, L0) of @@ -1031,7 +1043,7 @@ handle_aux(_, _, {get_checked_out, ConsumerKey, MsgIds}, #{ConsumerKey := #consumer{checked_out = Checked}} -> {Log, IdMsgs} = maps:fold( - fun (MsgId, ?MSG(Idx, Header), {L0, Acc}) -> + fun (MsgId, ?C_MSG(_, Idx, Header), {L0, Acc}) -> %% it is possible this is not found if the consumer %% crashed and the message got removed case ra_log:fetch(Idx, L0) of @@ -1703,7 +1715,7 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, Effects0, return(#{index := IncomingRaftIdx} = Meta, ConsumerKey, Returned, Effects0, State0) -> {State1, Effects1} = maps:fold( - fun(MsgId, Msg, {S0, E0}) -> + fun(MsgId, {_At, Msg}, {S0, E0}) -> return_one(Meta, MsgId, Msg, S0, E0, ConsumerKey) end, {State0, Effects0}, Returned), @@ -1723,7 +1735,7 @@ complete(Meta, ConsumerKey, [MsgId], msg_bytes_checkout = BytesCheckout, messages_total = Tot} = State0) -> case maps:take(MsgId, Checked0) of - {?MSG(Idx, Hdr), Checked} -> + {?C_MSG(_, Idx, Hdr), Checked} -> SettledSize = get_header(size, Hdr), Indexes = rabbit_fifo_index:delete(Idx, Indexes0), Con = Con0#consumer{checked_out = Checked, @@ -1744,7 +1756,7 @@ complete(Meta, ConsumerKey, MsgIds, = lists:foldl( fun (MsgId, {S0, Ch0, Idxs}) -> case maps:take(MsgId, Ch0) of - {?MSG(Idx, Hdr), Ch} -> + {?C_MSG(_, Idx, Hdr), Ch} -> S = get_header(size, Hdr) + S0, {S, Ch, rabbit_fifo_index:delete(Idx, Idxs)}; error -> @@ -1884,7 +1896,7 @@ get_header(Key, Header) when is_map(Header) andalso is_map_key(size, Header) -> maps:get(Key, Header, undefined). -return_one(Meta, MsgId, Msg0, +return_one(Meta, MsgId, ?MSG(_, _) = Msg0, #?STATE{returns = Returns, consumers = Consumers, dlx = DlxState0, @@ -1915,7 +1927,7 @@ return_one(Meta, MsgId, Msg0, return_all(Meta, #?STATE{consumers = Cons} = State0, Effects0, ConsumerKey, #consumer{checked_out = Checked} = Con) -> State = State0#?STATE{consumers = Cons#{ConsumerKey => Con}}, - lists:foldl(fun ({MsgId, Msg}, {S, E}) -> + lists:foldl(fun ({MsgId, ?C_MSG(_At, Msg)}, {S, E}) -> return_one(Meta, MsgId, Msg, S, E, ConsumerKey) end, {State, Effects0}, lists:sort(maps:to_list(Checked))). @@ -1942,7 +1954,7 @@ checkout(#{index := Index} = Meta, end. checkout0(Meta, {success, ConsumerKey, MsgId, - ?MSG(_RaftIdx, _Header) = Msg, ExpiredMsg, State, Effects}, + ?MSG(_, _) = Msg, ExpiredMsg, State, Effects}, SendAcc0) -> DelMsg = {MsgId, Msg}, SendAcc = case maps:get(ConsumerKey, SendAcc0, undefined) of @@ -2052,7 +2064,7 @@ take_next_msg(#?STATE{returns = Returns0, %% add index here Indexes = rabbit_fifo_index:append(RaftIdx, Indexes0), {Msg, State#?STATE{messages = Messages, - ra_indexes = Indexes}} + ra_indexes = Indexes}} end end. @@ -2107,7 +2119,7 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> {{value, ConsumerKey}, SQ1} when is_map_key(ConsumerKey, Cons0) -> case take_next_msg(InitState) of - {ConsumerMsg, State0} -> + {Msg, State0} -> %% there are consumers waiting to be serviced %% process consumer checkout case maps:get(ConsumerKey, Cons0) of @@ -2128,7 +2140,7 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> credit = Credit, delivery_count = DelCnt0, cfg = Cfg} = Con0 -> - Checked = maps:put(Next, ConsumerMsg, Checked0), + Checked = maps:put(Next, ?C_MSG(Ts, Msg), Checked0), DelCnt = case credit_api_v2(Cfg) of true -> add(DelCnt0, 1); false -> DelCnt0 + 1 @@ -2137,14 +2149,14 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> next_msg_id = Next + 1, credit = Credit - 1, delivery_count = DelCnt}, - Size = get_header(size, get_msg_header(ConsumerMsg)), + 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, ConsumerMsg, ExpiredMsg, + {success, ConsumerKey, Next, Msg, ExpiredMsg, State, Effects1} end; empty -> @@ -2468,6 +2480,10 @@ make_credit(Key, Credit, DeliveryCount, Drain) -> delivery_count = DeliveryCount, drain = Drain}. +-spec make_defer(consumer_key(), [msg_id()]) -> protocol(). +make_defer(ConsumerKey, MsgIds) when is_list(MsgIds) -> + #defer{consumer_key = ConsumerKey, msg_ids = MsgIds}. + -spec make_purge() -> protocol(). make_purge() -> #purge{}. @@ -2604,16 +2620,16 @@ notify_decorators_startup(QName) -> {mod_call, rabbit_quorum_queue, spawn_notify_decorators, [QName, startup, []]}. -convert(To, To, State) -> +convert(_Meta, To, To, State) -> State; -convert(0, To, State) -> - convert(1, To, rabbit_fifo_v1:convert_v0_to_v1(State)); -convert(1, To, State) -> - convert(2, To, rabbit_fifo_v3:convert_v1_to_v2(State)); -convert(2, To, State) -> - convert(3, To, rabbit_fifo_v3:convert_v2_to_v3(State)); -convert(3, To, State) -> - convert(4, To, convert_v3_to_v4(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)). smallest_raft_index(#?STATE{messages = Messages, ra_indexes = Indexes, diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index 58d4be87116b..2cca759c16dc 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -11,6 +11,8 @@ %% 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(C_MSG(At, Msg), {At, Msg}). +-define(C_MSG(At, Index, Header), {At, ?MSG(Index, Header)}). -define(IS_HEADER(H), (is_integer(H) andalso H >= 0) orelse @@ -108,6 +110,7 @@ -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(), @@ -125,7 +128,7 @@ {cfg = #consumer_cfg{}, status = up :: up | suspected_down | cancelled | fading, next_msg_id = 0 :: msg_id(), - checked_out = #{} :: #{msg_id() => msg()}, + checked_out = #{} :: #{msg_id() => {At :: milliseconds(), msg()}}, %% max number of messages that can be sent %% decremented for each delivery credit = 0 :: non_neg_integer(), @@ -137,8 +140,6 @@ -type consumer_strategy() :: competing | single_active. --type milliseconds() :: non_neg_integer(). - -type dead_letter_handler() :: option({at_most_once, applied_mfa()} | at_least_once). -record(enqueuer, diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index 401c537ba30b..e5ab058944bb 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -190,7 +190,8 @@ -callback is_stateful() -> boolean(). %% intitialise and return a queue type specific session context --callback init(amqqueue:amqqueue()) -> {ok, queue_state()} | {error, Reason :: term()}. +-callback init(amqqueue:amqqueue()) -> + {ok, queue_state()} | {error, Reason :: term()}. -callback close(queue_state()) -> ok. %% update the queue type state from amqqrecord diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index cb7f6b1ee6cf..8795359c411e 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -240,8 +240,9 @@ init_per_group(Group, Config) -> Config2 -> _ = rabbit_ct_broker_helpers:enable_feature_flag(Config2, message_containers), - _ = rabbit_ct_broker_helpers:enable_feature_flag(Config2, + QQV4 = rabbit_ct_broker_helpers:enable_feature_flag(Config2, quorum_queues_v4), + ct:pal("quorum_queue_v4 enable result ~p", [QQV4]), ok = rabbit_ct_broker_helpers:rpc( Config2, 0, application, set_env, [rabbit, channel_tick_interval, 100]), @@ -3873,7 +3874,7 @@ basic_get(Ch, Q, NoAck, Attempt) -> check_quorum_queues_v4_compat(Config) -> case rabbit_ct_broker_helpers:is_feature_flag_enabled(Config, - quorum_queues_4) of + quorum_queues_v4) of false -> throw({skip, "test needs feature flag quorum_queues_v4"}); true -> diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 948b6d450183..43d9b15a9b52 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -2304,12 +2304,16 @@ convert_v3_to_v4(Config) -> %% convert from v3 to v4 {#rabbit_fifo{consumers = Consumers}, ok, _} = - apply(meta(ConfigV4, 4), {machine_version, 3, 4}, State), + apply(meta(ConfigV4, 4), {machine_version, 3, 4}, State), ?assertEqual(2, maps:size(Consumers)), ?assertMatch(#consumer{cfg = #consumer_cfg{credit_mode = {simple_prefetch, MaxCredits}}}, maps:get(Cid2, Consumers)), + #consumer{checked_out = Ch1} = maps:get(Cid1, Consumers), + maps:foreach(fun (_MsgId, Msg) -> ?assert(is_tuple(Msg)) end, Ch1), + #consumer{checked_out = Ch2} = maps:get(Cid2, Consumers), + maps:foreach(fun (_MsgId, Msg) -> ?assert(is_tuple(Msg)) end, Ch2), ok. queue_ttl_test(C) -> From 1662d1402d0a71c4ee79427ae36645ef29b4f8e3 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 8 May 2024 14:10:05 +0100 Subject: [PATCH 07/45] QQ: Refactor to use the new ra_machine:handle_aux/5 API Instead of the old ra_machine:handle_aux/6 callback. --- deps/rabbit/src/rabbit_fifo.erl | 184 ++++++++++++------------ deps/rabbit/src/rabbit_quorum_queue.erl | 10 +- 2 files changed, 100 insertions(+), 94 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 754d55f6560e..b2145bcd6d62 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -38,7 +38,7 @@ which_module/1, %% aux init_aux/1, - handle_aux/6, + handle_aux/5, % queries query_messages_ready/1, query_messages_checked_out/1, @@ -972,7 +972,7 @@ which_module(4) -> ?STATE. last_decorators_state :: term(), capacity :: term(), gc = #aux_gc{} :: #aux_gc{}, - tick_pid, + tick_pid :: undefined | pid(), cache = #{} :: map()}). init_aux(Name) when is_atom(Name) -> @@ -986,45 +986,47 @@ init_aux(Name) when is_atom(Name) -> handle_aux(RaftState, Tag, Cmd, #aux{name = Name, capacity = Cap, - gc = Gc}, Log, MacState) -> + gc = Gc}, RaAux) -> %% convert aux state to new version Aux = #?AUX{name = Name, capacity = Cap, gc = Gc}, - handle_aux(RaftState, Tag, Cmd, Aux, Log, MacState); -handle_aux(leader, _, garbage_collection, Aux, Log, MacState) -> - {no_reply, force_eval_gc(Log, MacState, Aux), Log}; -handle_aux(follower, _, garbage_collection, Aux, Log, MacState) -> - {no_reply, force_eval_gc(Log, MacState, Aux), Log}; + handle_aux(RaftState, Tag, Cmd, Aux, RaAux); handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, - consumer_key = Key}, Corr, Pid}, - Aux0, Log0, #?STATE{cfg = #cfg{delivery_limit = undefined}, - consumers = Consumers}) -> - - case find_consumer(Key, Consumers) of - {ConsumerKey, #consumer{checked_out = Checked}} -> - {Log, ToReturn} = - maps:fold( - fun (MsgId, ?C_MSG(_, Idx, Header), {L0, Acc}) -> - %% it is possible this is not found if the consumer - %% crashed and the message got removed - case ra_log:fetch(Idx, L0) of - {{_, _, {_, _, Cmd, _}}, L} -> - Msg = get_msg(Cmd), - {L, [{MsgId, Idx, Header, Msg} | Acc]}; - {undefined, L} -> - {L, Acc} - end - end, {Log0, []}, maps:with(MsgIds, Checked)), - - Appends = make_requeue(ConsumerKey, {notify, Corr, Pid}, - lists:sort(ToReturn), []), - {no_reply, Aux0, Log, Appends}; + 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, ?C_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; _ -> - {no_reply, Aux0, Log0} + %% 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, Overview, Nodes]}, - #?AUX{tick_pid = Pid} = Aux, Log, _) -> +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 -> @@ -1035,35 +1037,34 @@ handle_aux(leader, _, {handle_tick, [QName, Overview, Nodes]}, %% Active TICK pid, do nothing Pid end, - {no_reply, Aux#?AUX{tick_pid = NewPid}, Log}; -handle_aux(_, _, {get_checked_out, ConsumerKey, MsgIds}, - Aux0, Log0, #?STATE{cfg = #cfg{}, - consumers = Consumers}) -> + %% 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}} -> - {Log, IdMsgs} = + {RaState, IdMsgs} = maps:fold( - fun (MsgId, ?C_MSG(_, Idx, Header), {L0, Acc}) -> + fun (MsgId, ?C_MSG(_, Idx, Header), {S0, Acc}) -> %% it is possible this is not found if the consumer %% crashed and the message got removed - case ra_log:fetch(Idx, L0) of - {{_, _, {_, _, Cmd, _}}, L} -> + case ra_aux:log_fetch(Idx, S0) of + {{_Term, _Meta, Cmd}, S} -> Msg = get_msg(Cmd), - {L, [{MsgId, {Header, Msg}} | Acc]}; - {undefined, L} -> - {L, Acc} + {S, [{MsgId, {Header, Msg}} | Acc]}; + {undefined, S} -> + {S, Acc} end - end, {Log0, []}, maps:with(MsgIds, Checked)), - {reply, {ok, IdMsgs}, Aux0, Log}; + end, {RaAux0, []}, maps:with(MsgIds, Checked)), + {reply, {ok, IdMsgs}, Aux0, RaState}; _ -> - {reply, {error, consumer_not_found}, Aux0, Log0} + {reply, {error, consumer_not_found}, Aux0, RaAux0} end; -handle_aux(leader, cast, {#return{} = Ret, Corr, Pid}, - Aux0, Log, #?STATE{}) -> - %% for returns with a delivery limit set we can just return as before - {no_reply, Aux0, Log, [{append, Ret, {notify, Corr, Pid}}]}; handle_aux(leader, cast, eval, #?AUX{last_decorators_state = LastDec} = Aux0, - Log, #?STATE{cfg = #cfg{resource = QName}} = MacState) -> + RaAux) -> + #?STATE{cfg = #cfg{resource = QName}} = MacState = + ra_aux:machine_state(RaAux), %% 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 @@ -1071,74 +1072,77 @@ handle_aux(leader, cast, eval, #?AUX{last_decorators_state = LastDec} = Aux0, Effects0 = timer_effect(Ts, MacState, []), case query_notify_decorators_info(MacState) of LastDec -> - {no_reply, Aux0, Log, Effects0}; + {no_reply, Aux0, RaAux, Effects0}; {MaxActivePriority, IsEmpty} = NewLast -> Effects = [notify_decorators_effect(QName, MaxActivePriority, IsEmpty) | Effects0], - {no_reply, Aux0#?AUX{last_decorators_state = NewLast}, Log, Effects} + {no_reply, Aux0#?AUX{last_decorators_state = NewLast}, RaAux, Effects} end; -handle_aux(_RaftState, cast, eval, Aux0, Log, _MacState) -> - {no_reply, Aux0, Log}; -handle_aux(_RaState, cast, Cmd, #?AUX{capacity = Use0} = Aux0, - Log, _MacState) +handle_aux(_RaftState, cast, eval, Aux0, RaAux) -> + {no_reply, Aux0, RaAux}; +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)}, Log}; + {no_reply, Aux0#?AUX{capacity = update_use(Use0, Cmd)}, RaAux}; handle_aux(_RaState, cast, tick, #?AUX{name = Name, capacity = Use0} = State0, - Log, MacState) -> + RaAux) -> true = ets:insert(rabbit_fifo_usage, {Name, capacity(Use0)}), - Aux = eval_gc(Log, MacState, State0), - {no_reply, Aux, Log}; -handle_aux(_RaState, cast, eol, #?AUX{name = Name} = Aux, Log, _) -> + Aux = eval_gc(RaAux, ra_aux:machine_state(RaAux), State0), + {no_reply, Aux, RaAux}; +handle_aux(_RaState, cast, eol, #?AUX{name = Name} = Aux, RaAux) -> ets:delete(rabbit_fifo_usage, Name), - {no_reply, Aux, Log}; + {no_reply, Aux, RaAux}; handle_aux(_RaState, {call, _From}, oldest_entry_timestamp, - #?AUX{cache = Cache} = Aux0, - Log0, #?STATE{} = State) -> - {CachedIdx, CachedTs} = maps:get(oldest_entry, Cache, {undefined, undefined}), - case smallest_raft_index(State) of + #?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, Log0}; + {reply, {ok, erlang:system_time(millisecond)}, Aux1, RaAux0}; CachedIdx -> %% cache hit - {reply, {ok, CachedTs}, Aux0, Log0}; + {reply, {ok, CachedTs}, Aux0, RaAux0}; Idx when is_integer(Idx) -> - case ra_log:fetch(Idx, Log0) of - {{_, _, {_, #{ts := Timestamp}, _, _}}, Log1} -> + 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, Log1}; - {undefined, Log1} -> + {reply, {ok, Timestamp}, Aux1, RaAux}; + {undefined, RaAux} -> %% fetch failed - {reply, {error, failed_to_get_timestamp}, Aux0, Log1} + {reply, {error, failed_to_get_timestamp}, Aux0, RaAux} end end; handle_aux(_RaState, {call, _From}, {peek, Pos}, Aux0, - Log0, MacState) -> + 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, _}}, Log} = ra_log:fetch(Idx, Log0), + {{_, _, Cmd}, RaAux} = ra_aux:log_fetch(Idx, RaAux0), Msg = get_msg(Cmd), - {reply, {ok, {Header, Msg}}, Aux0, Log}; + {reply, {ok, {Header, Msg}}, Aux0, RaAux}; Err -> - {reply, Err, Aux0, Log0} + {reply, Err, Aux0, RaAux0} end; -handle_aux(RaState, _, {dlx, _} = Cmd, Aux0, Log, - #?STATE{dlx = DlxState, - cfg = #cfg{dead_letter_handler = DLH, - resource = QRes}}) -> +handle_aux(_, _, garbage_collection, Aux, RaAux) -> + {no_reply, force_eval_gc(RaAux, Aux), RaAux}; +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, Log}. + {no_reply, Aux, RaAux}. -eval_gc(Log, #?STATE{cfg = #cfg{resource = QR}} = MacState, +eval_gc(RaAux, MacState, #?AUX{gc = #aux_gc{last_raft_idx = LastGcIdx} = Gc} = AuxState) -> - {Idx, _} = ra_log:last_index_term(Log), + {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 @@ -1153,9 +1157,11 @@ eval_gc(Log, #?STATE{cfg = #cfg{resource = QR}} = MacState, AuxState end. -force_eval_gc(Log, #?STATE{cfg = #cfg{resource = QR}}, +force_eval_gc(%Log, + RaAux, #?AUX{gc = #aux_gc{last_raft_idx = LastGcIdx} = Gc} = AuxState) -> - {Idx, _} = ra_log:last_index_term(Log), + {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 -> diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 1460d69c7866..53fe0f7f38df 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -129,7 +129,7 @@ -define(RPC_TIMEOUT, 1000). -define(START_CLUSTER_TIMEOUT, 5000). -define(START_CLUSTER_RPC_TIMEOUT, 60_000). %% needs to be longer than START_CLUSTER_TIMEOUT --define(TICK_TIMEOUT, 5000). %% the ra server tick time +-define(TICK_INTERVAL, 5000). %% the ra server tick time -define(DELETE_TIMEOUT, 5000). -define(MEMBER_CHANGE_TIMEOUT, 20_000). -define(SNAPSHOT_INTERVAL, 8192). %% the ra default is 4096 @@ -569,12 +569,12 @@ handle_tick(QName, Stale when length(ExpectedNodes) > 0 -> %% rabbit_nodes:list_members/0 returns [] when there %% is an error so we need to handle that case - rabbit_log:debug("~ts: stale nodes detected. Purging ~w", + rabbit_log:debug("~ts: stale nodes detected in quorum " + "queue state. Purging ~w", [rabbit_misc:rs(QName), Stale]), %% pipeline purge command ok = ra:pipeline_command(amqqueue:get_pid(Q), rabbit_fifo:make_purge_nodes(Stale)), - ok; _ -> ok @@ -1798,7 +1798,7 @@ make_ra_conf(Q, ServerId) -> make_ra_conf(Q, ServerId, Membership) -> TickTimeout = application:get_env(rabbit, quorum_tick_interval, - ?TICK_TIMEOUT), + ?TICK_INTERVAL), SnapshotInterval = application:get_env(rabbit, quorum_snapshot_interval, ?SNAPSHOT_INTERVAL), make_ra_conf(Q, ServerId, TickTimeout, SnapshotInterval, Membership). @@ -1826,7 +1826,7 @@ make_ra_conf(Q, ServerId, TickTimeout, SnapshotInterval, Membership) -> make_mutable_config(Q) -> QName = amqqueue:get_name(Q), TickTimeout = application:get_env(rabbit, quorum_tick_interval, - ?TICK_TIMEOUT), + ?TICK_INTERVAL), Formatter = {?MODULE, format_ra_event, [QName]}, #{tick_timeout => TickTimeout, ra_event_formatter => Formatter}. From 021d93bff54172982bd60e3db08f7ef58a4cda8e Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 15 May 2024 14:50:27 +0100 Subject: [PATCH 08/45] QQ hi/lo priority queue --- deps/rabbit/BUILD.bazel | 9 + deps/rabbit/app.bzl | 12 ++ deps/rabbit/src/rabbit_fifo.erl | 98 +++++----- deps/rabbit/src/rabbit_fifo.hrl | 2 +- deps/rabbit/src/rabbit_fifo_q.erl | 150 +++++++++++++++ deps/rabbit/src/rabbit_fifo_v3.erl | 19 +- deps/rabbit/test/quorum_queue_SUITE.erl | 77 +++++++- deps/rabbit/test/rabbit_fifo_SUITE.erl | 66 ++++--- deps/rabbit/test/rabbit_fifo_prop_SUITE.erl | 4 +- deps/rabbit/test/rabbit_fifo_q_SUITE.erl | 194 ++++++++++++++++++++ moduleindex.yaml | 1 + 11 files changed, 558 insertions(+), 74 deletions(-) create mode 100644 deps/rabbit/src/rabbit_fifo_q.erl create mode 100644 deps/rabbit/test/rabbit_fifo_q_SUITE.erl diff --git a/deps/rabbit/BUILD.bazel b/deps/rabbit/BUILD.bazel index b151bc8c2f09..d666dabe9304 100644 --- a/deps/rabbit/BUILD.bazel +++ b/deps/rabbit/BUILD.bazel @@ -736,6 +736,15 @@ rabbitmq_suite( ], ) +rabbitmq_suite( + name = "rabbit_fifo_q_SUITE", + size = "small", + deps = [ + "@proper//:erlang_app", + "//deps/rabbit_common:erlang_app", + ], +) + rabbitmq_integration_suite( name = "rabbit_fifo_dlx_integration_SUITE", size = "medium", diff --git a/deps/rabbit/app.bzl b/deps/rabbit/app.bzl index 388edccc0290..b09512bc2b2d 100644 --- a/deps/rabbit/app.bzl +++ b/deps/rabbit/app.bzl @@ -146,6 +146,7 @@ def all_beam_files(name = "all_beam_files"): "src/rabbit_fifo_dlx_sup.erl", "src/rabbit_fifo_dlx_worker.erl", "src/rabbit_fifo_index.erl", + "src/rabbit_fifo_q.erl", "src/rabbit_fifo_v0.erl", "src/rabbit_fifo_v1.erl", "src/rabbit_fifo_v3.erl", @@ -400,6 +401,7 @@ def all_test_beam_files(name = "all_test_beam_files"): "src/rabbit_fifo_dlx_sup.erl", "src/rabbit_fifo_dlx_worker.erl", "src/rabbit_fifo_index.erl", + "src/rabbit_fifo_q.erl", "src/rabbit_fifo_v0.erl", "src/rabbit_fifo_v1.erl", "src/rabbit_fifo_v3.erl", @@ -675,6 +677,7 @@ def all_srcs(name = "all_srcs"): "src/rabbit_fifo_dlx_sup.erl", "src/rabbit_fifo_dlx_worker.erl", "src/rabbit_fifo_index.erl", + "src/rabbit_fifo_q.erl", "src/rabbit_fifo_v0.erl", "src/rabbit_fifo_v1.erl", "src/rabbit_fifo_v3.erl", @@ -2146,3 +2149,12 @@ def test_suite_beam_files(name = "test_suite_beam_files"): erlc_opts = "//:test_erlc_opts", deps = ["//deps/amqp_client:erlang_app"], ) + erlang_bytecode( + testonly = True, + srcs = ["test/rabbit_fifo_q_SUITE.erl"], + outs = ["test/rabbit_fifo_q_SUITE.beam"], + hdrs = ["src/rabbit_fifo.hrl"], + app_name = "rabbit", + erlc_opts = "//:test_erlc_opts", + deps = ["@proper//:erlang_app"], + ) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index b2145bcd6d62..c392d016d46c 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -14,7 +14,6 @@ -dialyzer(no_improper_lists). -include("rabbit_fifo.hrl"). --include_lib("rabbit_common/include/rabbit.hrl"). -define(STATE, ?MODULE). @@ -299,7 +298,9 @@ apply(#{index := Idx} = Meta, credit = increase_credit(Con0, 1)}, State1 = State0#?STATE{ra_indexes = rabbit_fifo_index:delete(OldIdx, Indexes0), - messages = lqueue:in(?MSG(Idx, Header), Messages), + messages = rabbit_fifo_q:in(lo, + ?MSG(Idx, Header), + Messages), enqueue_count = EnqCount + 1}, State2 = update_or_remove_con(Meta, ConsumerKey, Con, State1), {State, Ret, Effs} = checkout(Meta, State0, State2, []), @@ -566,7 +567,7 @@ apply(#{index := Index}, #purge{}, end, Indexes0, Returns) end, State1 = State0#?STATE{ra_indexes = Indexes, - messages = lqueue:new(), + messages = rabbit_fifo_q:new(), messages_total = Total - NumReady, returns = lqueue:new(), msg_bytes_enqueue = 0 @@ -736,7 +737,11 @@ apply(_Meta, Cmd, State) -> rabbit_log:debug("rabbit_fifo: unhandled command ~W", [Cmd, 10]), {State, ok, []}. -convert_v3_to_v4(#{system_time := Ts}, #rabbit_fifo{consumers = Consumers0} = StateV3) -> +convert_v3_to_v4(#{system_time := Ts}, + StateV3) -> + Messages0 = rabbit_fifo_v3:get_field(messages, StateV3), + Consumers0 = rabbit_fifo_v3:get_field(consumers, StateV3), + Messages = rabbit_fifo_q:from_lqueue(Messages0), Consumers = maps:map( fun (_CKey, #consumer{checked_out = Ch0} = C) -> Ch = maps:map( @@ -745,7 +750,23 @@ convert_v3_to_v4(#{system_time := Ts}, #rabbit_fifo{consumers = Consumers0} = St end, Ch0), C#consumer{checked_out = Ch} end, Consumers0), - StateV3#?MODULE{consumers = Consumers}. + #?MODULE{cfg = rabbit_fifo_v3:get_field(cfg, StateV3), + messages = Messages, + messages_total = rabbit_fifo_v3:get_field(messages_total, StateV3), + returns = rabbit_fifo_v3:get_field(returns, StateV3), + 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), + release_cursors = rabbit_fifo_v3:get_field(release_cursors, StateV3), + consumers = Consumers, + % consumers that require further service are queued here + 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)}. purge_node(Meta, Node, State, Effects) -> lists:foldl(fun(Pid, {S0, E0}) -> @@ -1345,7 +1366,7 @@ is_v4() -> messages_ready(#?STATE{messages = M, returns = R}) -> - lqueue:len(M) + lqueue:len(R). + rabbit_fifo_q:len(M) + lqueue:len(R). messages_total(#?STATE{messages_total = Total, dlx = DlxState}) -> @@ -1596,19 +1617,6 @@ drop_head(#?STATE{ra_indexes = Indexes0} = State0, Effects) -> {State0, Effects} end. -maybe_set_msg_ttl(#basic_message{content = #content{properties = none}}, - RaCmdTs, Header, - #?STATE{cfg = #cfg{msg_ttl = PerQueueMsgTTL}}) -> - update_expiry_header(RaCmdTs, PerQueueMsgTTL, Header); -maybe_set_msg_ttl(#basic_message{content = #content{properties = Props}}, - RaCmdTs, Header, - #?STATE{cfg = #cfg{msg_ttl = PerQueueMsgTTL}}) -> - %% rabbit_quorum_queue will leave the properties decoded if and only if - %% per message message TTL is set. - %% We already check in the channel that expiration must be valid. - {ok, PerMsgMsgTTL} = rabbit_basic:parse_expiration(Props), - TTL = min(PerMsgMsgTTL, PerQueueMsgTTL), - update_expiry_header(RaCmdTs, TTL, Header); maybe_set_msg_ttl(Msg, RaCmdTs, Header, #?STATE{cfg = #cfg{msg_ttl = MsgTTL}}) -> case mc:is(Msg) of @@ -1670,10 +1678,11 @@ maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, Effects, Size = message_size(RawMsg), Header = maybe_set_msg_ttl(RawMsg, Ts, Size, State0), 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 = lqueue:in(Msg, Messages) + messages = rabbit_fifo_q:in(PTag, Msg, Messages) }, {ok, State, Effects}; maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, Effects0, @@ -1701,10 +1710,11 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, Effects0, false -> undefined end, + PTag = priority_tag(RawMsg), State = State0#?STATE{msg_bytes_enqueue = Enqueue + Size, enqueue_count = EnqCount + 1, messages_total = Total + 1, - messages = lqueue:in(Msg, Messages), + messages = rabbit_fifo_q:in(PTag, Msg, Messages), enqueuers = Enqueuers0#{From => Enq}, msg_cache = MsgCache }, @@ -1821,10 +1831,10 @@ update_smallest_raft_index(IncomingRaftIdx, Reply, #?STATE{cfg = Cfg, release_cursors = Cursors0} = State0, Effects) -> - Total = messages_total(State0), + % Total = messages_total(State0), %% TODO: optimise case smallest_raft_index(State0) of - undefined when Total == 0 -> + undefined -> % there are no messages on queue anymore and no pending enqueues % we can forward release_cursor all the way until % the last received command, hooray @@ -1835,8 +1845,8 @@ update_smallest_raft_index(IncomingRaftIdx, Reply, release_cursors = lqueue:new(), enqueue_count = 0}, {State, Reply, Effects ++ [{release_cursor, IncomingRaftIdx, State}]}; - undefined -> - {State0, Reply, Effects}; + % undefined -> + % {State0, Reply, Effects}; Smallest when is_integer(Smallest) -> case find_next_cursor(Smallest, Cursors0) of empty -> @@ -2063,10 +2073,10 @@ take_next_msg(#?STATE{returns = Returns0, {{value, NextMsg}, Returns} -> {NextMsg, State#?STATE{returns = Returns}}; {empty, _} -> - case lqueue:out(Messages0) of + case rabbit_fifo_q:out(Messages0) of {empty, _} -> empty; - {{value, ?MSG(RaftIdx, _) = Msg}, Messages} -> + {_P, ?MSG(RaftIdx, _) = Msg, Messages} -> %% add index here Indexes = rabbit_fifo_index:append(RaftIdx, Indexes0), {Msg, State#?STATE{messages = Messages, @@ -2078,7 +2088,7 @@ get_next_msg(#?STATE{returns = Returns0, messages = Messages0}) -> case lqueue:get(Returns0, empty) of empty -> - lqueue:get(Messages0, empty); + rabbit_fifo_q:get(Messages0); Msg -> Msg end. @@ -2173,7 +2183,7 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> checkout_one(Meta, ExpiredMsg, InitState#?STATE{service_queue = SQ1}, Effects1); {empty, _} -> - case lqueue:len(Messages0) of + case rabbit_fifo_q:len(Messages0) of 0 -> {nochange, ExpiredMsg, InitState, Effects1}; _ -> @@ -2407,9 +2417,10 @@ normalize(#?STATE{ra_indexes = _Indexes, release_cursors = Cursors, dlx = DlxState} = State) -> State#?STATE{returns = lqueue:from_list(lqueue:to_list(Returns)), - messages = lqueue:from_list(lqueue:to_list(Messages)), - release_cursors = lqueue:from_list(lqueue:to_list(Cursors)), - dlx = rabbit_fifo_dlx:normalize(DlxState)}. + messages = rabbit_fifo_q:normalize(Messages, + rabbit_fifo_q:new()), + release_cursors = lqueue:from_list(lqueue:to_list(Cursors)), + dlx = rabbit_fifo_dlx:normalize(DlxState)}. is_over_limit(#?STATE{cfg = #cfg{max_length = undefined, max_bytes = undefined}}) -> @@ -2517,9 +2528,6 @@ add_bytes_return(Header, State#?STATE{msg_bytes_checkout = Checkout - Size, msg_bytes_enqueue = Enqueue + Size}. -message_size(#basic_message{content = Content}) -> - #content{payload_fragments_rev = PFR} = Content, - iolist_size(PFR); message_size(B) when is_binary(B) -> byte_size(B); message_size(Msg) -> @@ -2641,12 +2649,7 @@ smallest_raft_index(#?STATE{messages = Messages, ra_indexes = Indexes, dlx = DlxState}) -> SmallestDlxRaIdx = rabbit_fifo_dlx:smallest_raft_index(DlxState), - SmallestMsgsRaIdx = case lqueue:get(Messages, undefined) of - ?MSG(I, _) when is_integer(I) -> - I; - _ -> - undefined - end, + SmallestMsgsRaIdx = rabbit_fifo_q:get_lowest_index(Messages), SmallestRaIdx = rabbit_fifo_index:smallest(Indexes), lists:min([SmallestDlxRaIdx, SmallestMsgsRaIdx, SmallestRaIdx]). @@ -2788,3 +2791,16 @@ maps_search(Pred, {K, V, 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 P > 4 -> + hi; + _ -> + lo + end; + false -> + lo + end. diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index 2cca759c16dc..29c522999fdd 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -181,7 +181,7 @@ -record(rabbit_fifo, {cfg :: #cfg{}, % unassigned messages - messages = lqueue:new() :: lqueue:lqueue(msg()), + 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()), diff --git a/deps/rabbit/src/rabbit_fifo_q.erl b/deps/rabbit/src/rabbit_fifo_q.erl new file mode 100644 index 000000000000..125040a77a59 --- /dev/null +++ b/deps/rabbit/src/rabbit_fifo_q.erl @@ -0,0 +1,150 @@ +-module(rabbit_fifo_q). + +-include("rabbit_fifo.hrl"). +-export([ + new/0, + in/3, + out/1, + get/1, + len/1, + from_lqueue/1, + normalize/2, + get_lowest_index/1 + ]). + +-define(WEIGHT, 2). +-define(NON_EMPTY, {_, [_|_]}). +-define(EMPTY, {[], []}). + +%% a weighted priority queue with only two priorities + +-record(?MODULE, {hi = ?EMPTY :: {list(msg()), list(msg())}, + lo = ?EMPTY :: {list(msg()), list(msg())}, + len = 0 :: non_neg_integer(), + dequeue_counter = 0 :: non_neg_integer()}). + +-opaque state() :: #?MODULE{}. + +-export_type([state/0]). + +-spec new() -> state(). +new() -> + #?MODULE{}. + +-spec in(hi | lo, msg(), state()) -> state(). +in(hi, Item, #?MODULE{hi = Hi, len = Len} = State) -> + State#?MODULE{hi = in(Item, Hi), + len = Len + 1}; +in(lo, Item, #?MODULE{lo = Lo, len = Len} = State) -> + State#?MODULE{lo = in(Item, Lo), + len = Len + 1}. + +-spec out(state()) -> + {empty, state()} | + {hi | lo, msg(), state()}. +out(#?MODULE{len = 0} = S) -> + {empty, S}; +out(#?MODULE{hi = Hi0, + lo = Lo0, + len = Len, + dequeue_counter = C0} = State) -> + C = case C0 of + ?WEIGHT -> + 0; + _ -> + C0 + 1 + end, + case next(State) of + {hi, Msg} -> + {hi, Msg, State#?MODULE{hi = drop(Hi0), + dequeue_counter = C, + len = Len - 1}}; + {lo, Msg} -> + {lo, Msg, State#?MODULE{lo = drop(Lo0), + dequeue_counter = C, + len = Len - 1}} + end. + +-spec get(state()) -> empty | msg(). +get(#?MODULE{len = 0}) -> + empty; +get(#?MODULE{} = State) -> + {_, Msg} = next(State), + Msg. + +-spec len(state()) -> non_neg_integer(). +len(#?MODULE{len = Len}) -> + Len. + +-spec from_lqueue(lqueue:lqueue(msg())) -> state(). +from_lqueue(LQ) -> + lqueue:fold(fun (Item, Acc) -> + in(lo, Item, Acc) + end, new(), LQ). + +-spec normalize(state(), state()) -> state(). +normalize(Q0, Acc) -> + case out(Q0) of + {empty, _} -> + Acc; + {P, Msg, Q} -> + normalize(Q, in(P, Msg, Acc)) + end. + +-spec get_lowest_index(state()) -> undefined | ra:index(). +get_lowest_index(#?MODULE{len = 0}) -> + undefined; +get_lowest_index(#?MODULE{hi = Hi, lo = Lo}) -> + case peek(Hi) of + empty -> + ?MSG(LoIdx, _) = peek(Lo), + LoIdx; + ?MSG(HiIdx, _) -> + case peek(Lo) of + ?MSG(LoIdx, _) -> + max(HiIdx, LoIdx); + empty -> + HiIdx + end + end. + +%% internals + +next(#?MODULE{hi = ?NON_EMPTY = Hi, + lo = ?NON_EMPTY = Lo, + dequeue_counter = ?WEIGHT}) -> + ?MSG(HiIdx, _) = HiMsg = peek(Hi), + ?MSG(LoIdx, _) = LoMsg = peek(Lo), + %% always favour hi priority messages when it is safe to do so, + %% i.e. the index is lower than the next index for the lo queue + case HiIdx < LoIdx of + true -> + {hi, HiMsg}; + false -> + {lo, LoMsg} + end; +next(#?MODULE{hi = ?NON_EMPTY = Hi}) -> + {hi, peek(Hi)}; +next(#?MODULE{lo = Lo}) -> + {lo, peek(Lo)}. + +%% invariant, if the queue is non empty so is the Out (right) list. +in(X, ?EMPTY) -> + {[], [X]}; +in(X, {[_] = In, []}) -> + {[X], In}; +in(X, {In, Out}) -> + {[X | In], Out}. + +peek(?EMPTY) -> + empty; +peek({_, [H | _]}) -> + H. + +drop(?EMPTY = Q) -> + Q; +drop({In, [_]}) -> + %% the last Out one + {[], lists:reverse(In)}; +drop({In, [_ | Out]}) -> + {In, Out}. diff --git a/deps/rabbit/src/rabbit_fifo_v3.erl b/deps/rabbit/src/rabbit_fifo_v3.erl index caab81c951ac..871e258609b3 100644 --- a/deps/rabbit/src/rabbit_fifo_v3.erl +++ b/deps/rabbit/src/rabbit_fifo_v3.erl @@ -70,7 +70,9 @@ make_update_config/1, make_garbage_collection/0, convert_v1_to_v2/1, - convert_v2_to_v3/1 + convert_v2_to_v3/1, + + get_field/2 ]). -ifdef(TEST). @@ -766,6 +768,21 @@ convert_v2_to_v3(#rabbit_fifo{consumers = ConsumersV2} = StateV2) -> end, ConsumersV2), StateV2#rabbit_fifo{consumers = ConsumersV3}. +get_field(Field, State) -> + Fields = record_info(fields, ?STATE), + Index = record_index_of(Field, Fields), + element(Index, State). + +record_index_of(F, Fields) -> + index_of(2, F, Fields). + +index_of(_, F, []) -> + exit({field_not_found, F}); +index_of(N, F, [F | _]) -> + N; +index_of(N, F, [_ | T]) -> + index_of(N+1, F, T). + convert_consumer_v2_to_v3(C = #consumer{cfg = Cfg = #consumer_cfg{credit_mode = simple_prefetch, meta = #{prefetch := Prefetch}}}) -> C#consumer{cfg = Cfg#consumer_cfg{credit_mode = {simple_prefetch, Prefetch}}}; diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index 8795359c411e..bfc81d771403 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -173,7 +173,9 @@ all_tests() -> cancel_consumer_gh_3729, cancel_and_consume_with_same_tag, validate_messages_on_queue, - amqpl_headers + amqpl_headers, + priority_queue_fifo, + priority_queue_2_1_ratio ]. memory_tests() -> @@ -1112,6 +1114,71 @@ single_active_consumer_priority(Config) -> rpc:call(Server0, ra, local_query, [RaNameQ3, QueryFun])), ok. +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), + [Server0 | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Ch = rabbit_ct_client_helpers:open_channel(Config, Server0), + Queue = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Queue, 0, 0}, + declare(Ch, Queue, + [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + ExpectedHi = + [begin + MsgP5 = integer_to_binary(P), + ok = amqp_channel:cast(Ch, #'basic.publish'{routing_key = Queue}, + #amqp_msg{props = #'P_basic'{priority = P}, + payload = MsgP5}), + MsgP5 + %% high priority is > 4 + end || P <- lists:seq(5, 10)], + + ExpectedLo = + [begin + MsgP1 = integer_to_binary(P), + ok = amqp_channel:cast(Ch, #'basic.publish'{routing_key = Queue}, + #amqp_msg{props = #'P_basic'{priority = P}, + payload = MsgP1}), + MsgP1 + end || P <- lists:seq(0, 4)], + + validate_queue(Ch, Queue, ExpectedHi ++ ExpectedLo), + ok. + +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), + [Server0 | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Ch = rabbit_ct_client_helpers:open_channel(Config, Server0), + Queue = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Queue, 0, 0}, + declare(Ch, Queue, + [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + ExpectedLo = + [begin + MsgP1 = integer_to_binary(P), + ok = amqp_channel:cast(Ch, #'basic.publish'{routing_key = Queue}, + #amqp_msg{props = #'P_basic'{priority = P}, + payload = MsgP1}), + MsgP1 + end || P <- lists:seq(0, 4)], + ExpectedHi = + [begin + MsgP5 = integer_to_binary(P), + ok = amqp_channel:cast(Ch, #'basic.publish'{routing_key = Queue}, + #amqp_msg{props = #'P_basic'{priority = P}, + payload = MsgP5}), + MsgP5 + %% high priority is > 4 + end || P <- lists:seq(5, 14)], + + Expected = lists_interleave(ExpectedLo, ExpectedHi), + + validate_queue(Ch, Queue, Expected), + ok. + reject_after_leader_transfer(Config) -> [Server0, Server1, Server2] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -3880,3 +3947,11 @@ check_quorum_queues_v4_compat(Config) -> true -> ok end. + +lists_interleave([], _List) -> + []; +lists_interleave([Item | Items], List) + when is_list(List) -> + {Left, Right} = lists:split(2, List), + Left ++ [Item | lists_interleave(Items, Right)]. + diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 43d9b15a9b52..2a3935f74f1e 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -15,6 +15,7 @@ -include_lib("rabbit_common/include/rabbit_framing.hrl"). -include_lib("rabbit/src/rabbit_fifo.hrl"). +-define(PROTOMOD, rabbit_framing_amqp_0_9_1). %%%=================================================================== %%% Common Test callbacks %%%=================================================================== @@ -350,7 +351,7 @@ enq_expire_deq_test(C) -> queue_resource => rabbit_misc:r(<<"/">>, queue, <<"test">>), msg_ttl => 0}, S0 = rabbit_fifo:init(Conf), - Msg = #basic_message{content = #content{properties = none, + Msg = #basic_message{content = #content{properties = #'P_basic'{}, payload_fragments_rev = []}}, {S1, ok, _} = apply(meta(C, 1, 100, {notify, 1, self()}), rabbit_fifo:make_enqueue(self(), 1, Msg), S0), @@ -370,18 +371,23 @@ enq_expire_enq_deq_test(Config) -> exchange_name = #resource{name = <<"x">>, kind = exchange, virtual_host = <<"v">>}, - content = #content{properties = #'P_basic'{expiration = <<"0">>}, + content = #content{properties = #'P_basic'{ + expiration = <<"0">>}, payload_fragments_rev = [<<"msg1">>]}}), Enq1 = rabbit_fifo:make_enqueue(self(), 1, Msg1), - {S1, ok, _} = apply(meta(Config, 1, 100, {notify, 1, self()}), Enq1, S0), - Msg2 = #basic_message{content = #content{properties = none, + Idx1 = ?LINE, + {S1, ok, _} = apply(meta(Config, Idx1, 100, {notify, 1, self()}), Enq1, S0), + Msg2 = #basic_message{content = #content{properties = #'P_basic'{}, + % class_id = 60, + % protocol = ?PROTOMOD, payload_fragments_rev = [<<"msg2">>]}}, Enq2 = rabbit_fifo:make_enqueue(self(), 2, Msg2), - {S2, ok, _} = apply(meta(Config, 2, 100, {notify, 2, self()}), Enq2, S1), + Idx2 = ?LINE, + {S2, ok, _} = apply(meta(Config, Idx2, 100, {notify, 2, self()}), Enq2, S1), Cid = {?FUNCTION_NAME, self()}, {_S3, _, Effs} = - apply(meta(Config, 3, 101), make_checkout(Cid, {dequeue, unsettled}, #{}), S2), - {log, [2], Fun} = get_log_eff(Effs), + apply(meta(Config, ?LINE, 101), make_checkout(Cid, {dequeue, unsettled}, #{}), S2), + {log, [Idx2], Fun} = get_log_eff(Effs), [{reply, _From, {wrap_reply, {dequeue, {_MsgId, _HeaderMsg}, ReadyMsgCount}}}] = Fun([Enq2]), ?assertEqual(0, ReadyMsgCount). @@ -573,7 +579,7 @@ cancelled_checkout_out_test(Config) -> % cancelled checkout should not return pending messages to queue {State2, _, _} = apply(meta(Config, 4), rabbit_fifo:make_checkout(Cid, cancel, #{}), State1), - ?assertEqual(1, lqueue:len(State2#rabbit_fifo.messages)), + ?assertEqual(1, rabbit_fifo_q:len(State2#rabbit_fifo.messages)), ?assertEqual(0, lqueue:len(State2#rabbit_fifo.returns)), ?assertEqual(0, priority_queue:len(State2#rabbit_fifo.service_queue)), @@ -645,13 +651,13 @@ down_with_noconnection_returns_unack_test(Config) -> Cid = {?FUNCTION_NAME_B, Pid}, Msg = rabbit_fifo:make_enqueue(self(), 1, second), {State0, _} = enq(Config, 1, 1, second, test_init(test)), - ?assertEqual(1, lqueue:len(State0#rabbit_fifo.messages)), + ?assertEqual(1, rabbit_fifo_q:len(State0#rabbit_fifo.messages)), ?assertEqual(0, lqueue:len(State0#rabbit_fifo.returns)), {State1, {_, _}} = deq(Config, 2, Cid, unsettled, Msg, State0), - ?assertEqual(0, lqueue:len(State1#rabbit_fifo.messages)), + ?assertEqual(0, rabbit_fifo_q:len(State1#rabbit_fifo.messages)), ?assertEqual(0, lqueue:len(State1#rabbit_fifo.returns)), {State2a, _, _} = apply(meta(Config, 3), {down, Pid, noconnection}, State1), - ?assertEqual(0, lqueue:len(State2a#rabbit_fifo.messages)), + ?assertEqual(0, rabbit_fifo_q:len(State2a#rabbit_fifo.messages)), ?assertEqual(1, lqueue:len(State2a#rabbit_fifo.returns)), ?assertMatch(#consumer{checked_out = Ch, status = suspected_down} @@ -782,7 +788,7 @@ duplicate_delivery_test(Config) -> {#rabbit_fifo{messages = Messages} = State, _} = enq(Config, 2, 1, first, State0), ?assertEqual(1, rabbit_fifo:query_messages_total(State)), - ?assertEqual(1, lqueue:len(Messages)), + ?assertEqual(1, rabbit_fifo_q:len(Messages)), ok. state_enter_monitors_and_notifications_test(Config) -> @@ -844,7 +850,7 @@ down_noproc_returns_checked_out_in_order_test(Config) -> {FS, _} = enq(Config, Num, Num, Num, FS0), FS end, S0, lists:seq(1, 100)), - ?assertEqual(100, lqueue:len(S1#rabbit_fifo.messages)), + ?assertEqual(100, rabbit_fifo_q:len(S1#rabbit_fifo.messages)), Cid = {<<"cid">>, self()}, {S2, #{key := CKey}, _} = checkout(Config, ?LINE, Cid, 1000, S1), #consumer{checked_out = Checked} = maps:get(CKey, S2#rabbit_fifo.consumers), @@ -865,7 +871,7 @@ down_noconnection_returns_checked_out_test(Config) -> {FS, _} = enq(Config, Num, Num, Num, FS0), FS end, S0, lists:seq(1, NumMsgs)), - ?assertEqual(NumMsgs, lqueue:len(S1#rabbit_fifo.messages)), + ?assertEqual(NumMsgs, rabbit_fifo_q:len(S1#rabbit_fifo.messages)), Cid = {<<"cid">>, self()}, {S2, #{key := CKey}, _} = checkout(Config, ?LINE, Cid, 1000, S1), #consumer{checked_out = Checked} = maps:get(CKey, S2#rabbit_fifo.consumers), @@ -2162,15 +2168,19 @@ checkout_reply(Oth) -> Oth. run_log(Config, InitState, Entries) -> - run_log(Config, InitState, Entries, fun (_) -> true end). + run_log(rabbit_fifo, Config, InitState, Entries, fun (_) -> true end). + run_log(Config, InitState, Entries, Invariant) -> + run_log(rabbit_fifo, Config, InitState, Entries, Invariant). + +run_log(Module, Config, InitState, Entries, Invariant) -> lists:foldl( fun ({assert, Fun}, {Acc0, Efx0}) -> _ = Fun(Acc0), {Acc0, Efx0}; ({Idx, E}, {Acc0, Efx0}) -> - case apply(meta(Config, Idx, Idx, {notify, Idx, self()}), - E, Acc0) of + case Module:apply(meta(Config, Idx, Idx, {notify, Idx, self()}), + E, Acc0) of {Acc, _, Efx} when is_list(Efx) -> ?assert(Invariant(Acc)), {Acc, Efx0 ++ Efx}; @@ -2189,16 +2199,15 @@ run_log(Config, InitState, Entries, Invariant) -> aux_test(_) -> _ = ra_machine_ets:start_link(), Aux0 = init_aux(aux_test), - MacState = init(#{name => aux_test, - queue_resource => - rabbit_misc:r(<<"/">>, queue, <<"test">>)}), + State0 = #{machine_state => + init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => false}), + log => mock_log}, ok = meck:new(ra_log, []), - Log = mock_log, meck:expect(ra_log, last_index_term, fun (_) -> {0, 0} end), - {no_reply, Aux, mock_log} = handle_aux(leader, cast, active, Aux0, - Log, MacState), - {no_reply, _Aux, mock_log} = handle_aux(leader, cast, tick, Aux, - Log, MacState), + {no_reply, Aux, State} = handle_aux(leader, cast, active, Aux0, State0), + {no_reply, _Aux, _} = handle_aux(leader, cast, tick, Aux, State), [X] = ets:lookup(rabbit_fifo_usage, aux_test), meck:unload(), ?assert(X > 0.0), @@ -2300,7 +2309,8 @@ convert_v3_to_v4(Config) -> max_in_memory_length => 0, queue_resource => rabbit_misc:r("/", queue, atom_to_binary(Name)), release_cursor_interval => 0}), - {State, _} = run_log(ConfigV3, Init, Entries), + {State, _} = run_log(rabbit_fifo_v3, ConfigV3, Init, Entries, + fun (_) -> true end), %% convert from v3 to v4 {#rabbit_fifo{consumers = Consumers}, ok, _} = @@ -2473,7 +2483,7 @@ expire_message_should_emit_release_cursor_test(Config) -> release_cursor_interval => 0, msg_ttl => 1}, S0 = rabbit_fifo:init(Conf), - Msg = #basic_message{content = #content{properties = none, + Msg = #basic_message{content = #content{properties = #'P_basic'{}, payload_fragments_rev = []}}, {S1, ok, _} = apply(meta(Config, ?LINE, 100, {notify, 1, self()}), rabbit_fifo:make_enqueue(self(), 1, Msg), S0), @@ -2570,7 +2580,7 @@ init(Conf) -> rabbit_fifo:init(Conf). make_register_enqueuer(Pid) -> rabbit_fifo:make_register_enqueuer(Pid). apply(Meta, Entry, State) -> rabbit_fifo:apply(Meta, Entry, State). init_aux(Conf) -> rabbit_fifo:init_aux(Conf). -handle_aux(S, T, C, A, L, M) -> rabbit_fifo:handle_aux(S, T, C, A, L, M). +handle_aux(S, T, C, A, A2) -> rabbit_fifo:handle_aux(S, T, C, A, A2). make_checkout(C, S, M) -> rabbit_fifo:make_checkout(C, S, M). cid(A) when is_atom(A) -> diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index c3837ce7f6b9..a2dfc4cf4a00 100644 --- a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl @@ -1443,7 +1443,7 @@ messages_total_invariant() -> returns = R, dlx = #rabbit_fifo_dlx{discards = D, consumer = DlxCon}} = S) -> - Base = lqueue:len(M) + lqueue:len(R), + Base = rabbit_fifo_q:len(M) + lqueue:len(R), Tot0 = maps:fold(fun (_, #consumer{checked_out = Ch}, Acc) -> Acc + map_size(Ch) end, Base, C), @@ -1797,7 +1797,7 @@ msg_gen() -> msg(Bin) when is_binary(Bin) -> #basic_message{content = #content{payload_fragments_rev = [Bin], - properties = none}}. + properties = #'P_basic'{}}}. checkout_cancel_gen(Pid) -> {checkout, Pid, cancel}. diff --git a/deps/rabbit/test/rabbit_fifo_q_SUITE.erl b/deps/rabbit/test/rabbit_fifo_q_SUITE.erl new file mode 100644 index 000000000000..86f0e98e21ff --- /dev/null +++ b/deps/rabbit/test/rabbit_fifo_q_SUITE.erl @@ -0,0 +1,194 @@ +-module(rabbit_fifo_q_SUITE). + +-compile(nowarn_export_all). +-compile(export_all). + +-export([ + ]). + +-include_lib("proper/include/proper.hrl"). +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("rabbit/src/rabbit_fifo.hrl"). + +all() -> + [ + {group, tests} + ]. + + +all_tests() -> + [ + hi, + basics, + hi_is_prioritised, + single_priority_behaves_like_queue + ]. + + +groups() -> + [ + {tests, [], all_tests()} + ]. + +init_per_suite(Config) -> + Config. + +end_per_suite(_Config) -> + ok. + +init_per_group(_Group, Config) -> + Config. + +end_per_group(_Group, _Config) -> + ok. + +init_per_testcase(_TestCase, Config) -> + Config. + +end_per_testcase(_TestCase, _Config) -> + ok. + +%%%=================================================================== +%%% Test cases +%%%=================================================================== + +-define(MSG(L), ?MSG(L, L)). + +hi(_Config) -> + Q0 = rabbit_fifo_q:new(), + Q1 = lists:foldl( + fun ({P, I}, Q) -> + rabbit_fifo_q:in(P, I, Q) + end, Q0, [ + {hi, ?MSG(?LINE)} + ]), + {hi, _, Q2} = rabbit_fifo_q:out(Q1), + {empty, _Q3} = rabbit_fifo_q:out(Q2), + ok. + +basics(_Config) -> + Q0 = rabbit_fifo_q:new(), + Q1 = lists:foldl( + fun ({P, I}, Q) -> + rabbit_fifo_q:in(P, I, Q) + end, Q0, [ + {hi, ?MSG(?LINE)}, + {lo, ?MSG(?LINE)}, + {hi, ?MSG(?LINE)}, + {lo, ?MSG(?LINE)}, + {hi, ?MSG(?LINE)} + ]), + {hi, _, Q2} = rabbit_fifo_q:out(Q1), + {hi, _, Q3} = rabbit_fifo_q:out(Q2), + {lo, _, Q4} = rabbit_fifo_q:out(Q3), + {hi, _, Q5} = rabbit_fifo_q:out(Q4), + {lo, _, Q6} = rabbit_fifo_q:out(Q5), + {empty, _} = rabbit_fifo_q:out(Q6), + ok. + +hi_is_prioritised(_Config) -> + Q0 = rabbit_fifo_q:new(), + %% when `hi' has a lower index than the next lo then it is still + %% prioritied (as this is safe to do). + Q1 = lists:foldl( + fun ({P, I}, Q) -> + rabbit_fifo_q:in(P, I, Q) + end, Q0, [ + {hi, ?MSG(1, ?LINE)}, + {hi, ?MSG(2, ?LINE)}, + {hi, ?MSG(3, ?LINE)}, + {hi, ?MSG(4, ?LINE)}, + {lo, ?MSG(5, ?LINE)} + ]), + {hi, _, Q2} = rabbit_fifo_q:out(Q1), + {hi, _, Q3} = rabbit_fifo_q:out(Q2), + {hi, _, Q4} = rabbit_fifo_q:out(Q3), + {hi, _, Q5} = rabbit_fifo_q:out(Q4), + {lo, _, Q6} = rabbit_fifo_q:out(Q5), + {empty, _} = rabbit_fifo_q:out(Q6), + + ok. + +-type op() :: {in, integer()} | out. + +single_priority_behaves_like_queue(_Config) -> + run_proper( + fun () -> + ?FORALL({P, Ops}, {oneof([hi, lo]), op_gen(256)}, + queue_prop(P, Ops)) + end, [], 25), + ok. + +queue_prop(P, Ops) -> + % ct:pal("Running queue_prop for ~s", [Ops]), + Que = queue:new(), + Sut = rabbit_fifo_q:new(), + {Queue, FifoQ} = lists:foldl( + fun ({in, V}, {Q0, S0}) -> + Q = queue:in(V, Q0), + S = rabbit_fifo_q:in(P, V, S0), + case queue:len(Q) == rabbit_fifo_q:len(S) of + true -> + {Q, S}; + false -> + throw(false) + end; + (out, {Q0, S0}) -> + {V1, Q} = case queue:out(Q0) of + {{value, V_}, Q1} -> + {V_, Q1}; + Res0 -> + Res0 + end, + {V2, S} = case rabbit_fifo_q:out(S0) of + {_, V, S1} -> + {V, S1}; + Res -> + Res + end, + case V1 == V2 of + true -> + {Q, S}; + false -> + ct:pal("V1 ~p, V2 ~p", [V1, V2]), + throw(false) + end + end, {Que, Sut}, Ops), + + queue:len(Queue) == rabbit_fifo_q:len(FifoQ). + + + + +%%% helpers + +op_gen(Size) -> + ?LET(Ops, + resize(Size, + list( + frequency( + [ + {20, {in, non_neg_integer()}}, + {20, out} + ] + ))), + begin + {_, Ops1} = lists:foldl( + fun ({in, I}, {Idx, Os}) -> + {Idx + 1, [{in, ?MSG(Idx, I)} | Os]}; + (out, {Idx, Os}) -> + {Idx + 1, [out | Os] } + end, {1, []}, Ops), + lists:reverse(Ops1) + end + ). + +run_proper(Fun, Args, NumTests) -> + ?assert( + proper:counterexample( + erlang:apply(Fun, Args), + [{numtests, NumTests}, + {on_output, fun(".", _) -> ok; % don't print the '.'s on new lines + (F, A) -> ct:pal(?LOW_IMPORTANCE, F, A) + end}])). diff --git a/moduleindex.yaml b/moduleindex.yaml index c2a161b9af1c..47f04ee4455a 100755 --- a/moduleindex.yaml +++ b/moduleindex.yaml @@ -644,6 +644,7 @@ rabbit: - rabbit_fifo_dlx_sup - rabbit_fifo_dlx_worker - rabbit_fifo_index +- rabbit_fifo_q - rabbit_fifo_v0 - rabbit_fifo_v1 - rabbit_fifo_v3 From 6a1cf01d388018f51174e294b745d6285afa3f77 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 6 Jun 2024 15:28:45 +0100 Subject: [PATCH 09/45] QQ: Avoid using mc:size/1 inside rabbit_fifo As we dont want to depend on external functions for things that may change the state of the queue. --- deps/rabbit/src/rabbit_fifo.erl | 59 ++++++++++++++++++--------------- 1 file changed, 33 insertions(+), 26 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index c392d016d46c..6e2d204ca854 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -92,7 +92,9 @@ seq :: option(msg_seqno()), msg :: raw_msg()}). -record(?ENQ_V2, {seq :: option(msg_seqno()), - msg :: raw_msg()}). + 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(), @@ -208,10 +210,10 @@ update_config(Conf, State) -> {state(), ra_machine:reply()}. apply(Meta, #enqueue{pid = From, seq = Seq, msg = RawMsg}, State00) -> - apply_enqueue(Meta, From, Seq, 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}, State00) -> - apply_enqueue(Meta, EnqPid, Seq, RawMsg, State00); + #?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) -> @@ -1589,8 +1591,9 @@ maybe_return_all(#{system_time := Ts} = Meta, ConsumerKey, end. apply_enqueue(#{index := RaftIdx, - system_time := Ts} = Meta, From, Seq, RawMsg, State0) -> - case maybe_enqueue(RaftIdx, Ts, From, Seq, RawMsg, [], State0) of + system_time := Ts} = Meta, From, + Seq, RawMsg, Size, State0) -> + case maybe_enqueue(RaftIdx, Ts, From, Seq, RawMsg, Size, [], State0) of {ok, State1, Effects1} -> {State, ok, Effects} = checkout(Meta, State0, State1, Effects1), {maybe_store_release_cursor(RaftIdx, State), ok, Effects}; @@ -1669,13 +1672,14 @@ maybe_store_release_cursor(RaftIdx, maybe_store_release_cursor(_RaftIdx, State) -> State. -maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, Effects, - #?STATE{msg_bytes_enqueue = Enqueue, - enqueue_count = EnqCount, - messages = Messages, - messages_total = Total} = State0) -> +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 = message_size(RawMsg), + Size = BodySize, Header = maybe_set_msg_ttl(RawMsg, Ts, Size, State0), Msg = ?MSG(RaftIdx, Header), PTag = priority_tag(RawMsg), @@ -1685,23 +1689,24 @@ maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, Effects, messages = rabbit_fifo_q:in(PTag, Msg, Messages) }, {ok, State, Effects}; -maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, Effects0, - #?STATE{msg_bytes_enqueue = Enqueue, - enqueue_count = EnqCount, - enqueuers = Enqueuers0, - messages = Messages, - messages_total = Total} = State0) -> +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, Effects0, State1), + RawMsg, Size, Effects0, + State1), {Res, State, [{monitor, process, From} | Effects]}; #enqueuer{next_seqno = MsgSeqNo} = Enq0 -> % it is the next expected seqno - Size = message_size(RawMsg), - Header = maybe_set_msg_ttl(RawMsg, Ts, Size, State0), + Header = maybe_set_msg_ttl(RawMsg, Ts, BodySize, State0), Msg = ?MSG(RaftIdx, Header), Enq = Enq0#enqueuer{next_seqno = MsgSeqNo + 1}, MsgCache = case can_immediately_deliver(State0) of @@ -1711,7 +1716,7 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, Effects0, undefined end, PTag = priority_tag(RawMsg), - State = State0#?STATE{msg_bytes_enqueue = Enqueue + Size, + State = State0#?STATE{msg_bytes_enqueue = Enqueue + BodySize, enqueue_count = EnqCount + 1, messages_total = Total + 1, messages = rabbit_fifo_q:in(PTag, Msg, Messages), @@ -2455,7 +2460,9 @@ make_enqueue(Pid, Seq, Msg) -> true when is_pid(Pid) andalso is_integer(Seq) -> %% more compact format - #?ENQ_V2{seq = Seq, msg = Msg}; + #?ENQ_V2{seq = Seq, + msg = Msg, + size = mc:size(Msg)}; _ -> #enqueue{pid = Pid, seq = Seq, msg = Msg} end. @@ -2533,11 +2540,11 @@ message_size(B) when is_binary(B) -> message_size(Msg) -> case mc:is(Msg) of true -> - {_, PayloadSize} = mc:size(Msg), - PayloadSize; + mc:size(Msg); false -> %% probably only hit this for testing so ok to use erts_debug - erts_debug:size(Msg) + {0, erts_debug:size(Msg)} + end. From e0b6e4d54742f6c6dd11346ac2dfebf3073971a9 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 25 Jun 2024 08:31:04 +0200 Subject: [PATCH 10/45] QQ bug fix: Maintain order when returning multiple Prior to this commit, quorum queues requeued messages in an undefined order, which is wrong. This commit fixes this bug and requeues messages always in the order as nacked / rejected / released by the client. We ensure that order of requeues is deterministic from the client's point of view and doesn't depend on whether the quorum queue soft limit was exceeded temporarily. So, even when rabbit_fifo_client batches requeues, the order as nacked by the client is still maintained. --- deps/rabbit/src/rabbit_fifo.erl | 47 ++++++++++---- deps/rabbit/test/quorum_queue_SUITE.erl | 86 ++++++++++++++++++++++++- deps/rabbit/test/rabbit_fifo_SUITE.erl | 29 ++++++++- 3 files changed, 145 insertions(+), 17 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 6e2d204ca854..1aa3c6306918 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -23,6 +23,20 @@ #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, @@ -250,8 +264,8 @@ apply(Meta, #discard{consumer_key = ConsumerKey, cfg = #cfg{dead_letter_handler = DLH}} = State0) -> case find_consumer(ConsumerKey, Consumers) of {ConsumerKey, #consumer{checked_out = Checked} = Con} -> - % Publishing to dead-letter exchange must maintain same order as - % messages got rejected. + %% 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 @@ -272,9 +286,8 @@ apply(Meta, #return{consumer_key = ConsumerKey, msg_ids = MsgIds}, #?STATE{consumers = Cons0} = State) -> case find_consumer(ConsumerKey, Cons0) of - {ActualConsumerKey, #consumer{checked_out = Checked0}} -> - Returned = maps:with(MsgIds, Checked0), - return(Meta, ActualConsumerKey, Returned, [], State); + {ActualConsumerKey, #consumer{checked_out = Checked}} -> + return(Meta, ActualConsumerKey, MsgIds, Checked, [], State); _ -> {State, ok} end; @@ -1734,12 +1747,19 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, end. return(#{index := IncomingRaftIdx} = Meta, - ConsumerKey, Returned, Effects0, State0) -> - {State1, Effects1} = maps:fold( - fun(MsgId, {_At, Msg}, {S0, E0}) -> - return_one(Meta, MsgId, Msg, - S0, E0, ConsumerKey) - end, {State0, Effects0}, Returned), + ConsumerKey, MsgIds, Checked, Effects0, State0) -> + %% 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 := Val} -> + {_At, Msg} = Val, + return_one(Meta, MsgId, Msg, + 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); @@ -2454,7 +2474,8 @@ is_below(undefined, _Num) -> 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(). +-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 @@ -2462,7 +2483,7 @@ make_enqueue(Pid, Seq, Msg) -> %% more compact format #?ENQ_V2{seq = Seq, msg = Msg, - size = mc:size(Msg)}; + size = ?SIZE(Msg)}; _ -> #enqueue{pid = Pid, seq = Seq, msg = Msg} end. diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index bfc81d771403..8ca389b49cf2 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -175,7 +175,9 @@ all_tests() -> validate_messages_on_queue, amqpl_headers, priority_queue_fifo, - priority_queue_2_1_ratio + priority_queue_2_1_ratio, + requeue_multiple_true, + requeue_multiple_false ]. memory_tests() -> @@ -3750,6 +3752,88 @@ select_nodes_with_least_replicas_node_down(Config) -> amqp_channel:call(Ch, #'queue.delete'{queue = Q})) || Q <- Qs]. +requeue_multiple_true(Config) -> + Ch = rabbit_ct_client_helpers:open_channel(Config), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-delivery-limit">>, long, 3}])), + Num = 100, + Payloads = [integer_to_binary(N) || N <- lists:seq(1, Num)], + [publish(Ch, QQ, P) || P <- Payloads], + + amqp_channel:subscribe(Ch, #'basic.consume'{queue = QQ}, self()), + receive #'basic.consume_ok'{} -> ok + end, + + DTags = [receive {#'basic.deliver'{redelivered = false, + delivery_tag = D}, + #amqp_msg{payload = P0}} -> + ?assertEqual(P, P0), + D + after 5000 -> ct:fail({basic_deliver_timeout, P, ?LINE}) + end || P <- Payloads], + + %% Requeue all messages. + ok = amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = lists:last(DTags), + multiple = true, + requeue = true}), + + %% We expect to get all messages re-delivered in the order in which we requeued + %% (which is the same order as messages were sent to us previously). + [receive {#'basic.deliver'{redelivered = true}, + #amqp_msg{payload = P1}} -> + ?assertEqual(P, P1) + after 5000 -> ct:fail({basic_deliver_timeout, P, ?LINE}) + end || P <- Payloads], + + ?assertEqual(#'queue.delete_ok'{message_count = 0}, + amqp_channel:call(Ch, #'queue.delete'{queue = QQ})). + +requeue_multiple_false(Config) -> + Ch = rabbit_ct_client_helpers:open_channel(Config), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-delivery-limit">>, long, 3}])), + Num = 100, + Payloads = [integer_to_binary(N) || N <- lists:seq(1, Num)], + [publish(Ch, QQ, P) || P <- Payloads], + + amqp_channel:subscribe(Ch, #'basic.consume'{queue = QQ}, self()), + receive #'basic.consume_ok'{} -> ok + end, + + DTags = [receive {#'basic.deliver'{redelivered = false, + delivery_tag = D}, + #amqp_msg{payload = P0}} -> + ?assertEqual(P, P0), + D + after 5000 -> ct:fail({basic_deliver_timeout, P, ?LINE}) + end || P <- Payloads], + + %% The delivery tags we received via AMQP 0.9.1 are ordered from 1-100. + %% Sanity check: + ?assertEqual(lists:seq(1, Num), DTags), + + %% Requeue each message individually in random order. + Tuples = [{rand:uniform(), D} || D <- DTags], + DTagsShuffled = [D || {_, D} <- lists:sort(Tuples)], + [ok = amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = D, + multiple = false, + requeue = true}) + || D <- DTagsShuffled], + + %% We expect to get all messages re-delivered in the order in which we requeued. + [receive {#'basic.deliver'{redelivered = true}, + #amqp_msg{payload = P1}} -> + ?assertEqual(integer_to_binary(D), P1) + after 5000 -> ct:fail({basic_deliver_timeout, ?LINE}) + end || D <- DTagsShuffled], + + ?assertEqual(#'queue.delete_ok'{message_count = 0}, + amqp_channel:call(Ch, #'queue.delete'{queue = QQ})). + %%---------------------------------------------------------------------------- same_elements(L1, L2) diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 2a3935f74f1e..f4179b41d3b7 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -469,6 +469,28 @@ return_test(Config) -> when map_size(C2) == 1, State3#rabbit_fifo.consumers), ok. +return_multiple_test(Config) -> + Cid = {<<"cid">>, self()}, + {State0, _} = enq(Config, 1, 1, first, test_init(?FUNCTION_NAME)), + {State1, _} = enq(Config, 2, 2, second, State0), + {State2, _} = enq(Config, 3, 3, third, State1), + + {State3, + #{key := CKey, + next_msg_id := NextMsgId}, + Effects0} = checkout(Config, ?LINE, Cid, 3, State2), + ?ASSERT_EFF({log, [1, 2, 3], _Fun, _Local}, Effects0), + + {_, _, Effects1} = apply(meta(Config, ?LINE), + rabbit_fifo:make_return( + CKey, + %% Return messages in following order: 3, 1, 2 + [NextMsgId + 2, NextMsgId, NextMsgId + 1]), + State3), + %% We expect messages to be re-delivered in the same order in which we previously returned. + ?ASSERT_EFF({log, [3, 1, 2], _Fun, _Local}, Effects1), + ok. + return_dequeue_delivery_limit_test(C) -> Init = init(#{name => test, queue_resource => rabbit_misc:r("/", queue, @@ -495,7 +517,7 @@ return_dequeue_delivery_limit_test(C) -> return_non_existent_test(Config) -> Cid = {<<"cid">>, self()}, {State0, _} = enq(Config, 1, 1, second, test_init(test)), - % return non-existent, check it doesnt crash + % return non-existent, check it doesn't crash {_State2, _} = apply(meta(Config, 3), rabbit_fifo:make_return(Cid, [99]), State0), ok. @@ -2097,8 +2119,9 @@ meta(Config, Idx, Timestamp, ReplyMode) -> enq(Config, Idx, MsgSeq, Msg, State) -> strip_reply( - rabbit_fifo:apply(meta(Config, Idx, 0, {notify, MsgSeq, self()}), - rabbit_fifo:make_enqueue(self(), MsgSeq, Msg), State)). + apply(meta(Config, Idx, 0, {notify, MsgSeq, self()}), + rabbit_fifo:make_enqueue(self(), MsgSeq, Msg), + State)). deq(Config, Idx, Cid, Settlement, Msg, State0) -> {State, _, Effs} = From ae0560efe29a9252ecec02f28315e731bc0bdd61 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 26 Jun 2024 09:51:40 +0200 Subject: [PATCH 11/45] Simplify --- deps/rabbit/src/rabbit_fifo_client.erl | 48 +++++++++----------------- 1 file changed, 16 insertions(+), 32 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index a48173d44165..506729a8e25c 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -142,7 +142,7 @@ enqueue(_QName, _Correlation, _Msg, cfg = #cfg{}} = State) -> {reject_publish, State}; enqueue(QName, Correlation, Msg, - #state{slow = Slow, + #state{slow = WasSlow, pending = Pending, queue_status = go, next_seq = Seq, @@ -152,19 +152,15 @@ enqueue(QName, Correlation, Msg, % by default there is no correlation id Cmd = rabbit_fifo:make_enqueue(self(), EnqueueSeq, Msg), ok = ra:pipeline_command(ServerId, Cmd, Seq, low), - Tag = case map_size(Pending) >= SftLmt of - true -> slow; - false -> ok - end, + IsSlow = map_size(Pending) >= SftLmt, State = State0#state{pending = Pending#{Seq => {Correlation, Cmd}}, next_seq = Seq + 1, next_enqueue_seq = EnqueueSeq + 1, - slow = Tag == slow}, - case Tag of - slow when not Slow -> - {ok, set_timer(QName, State), [{block, cluster_name(State)}]}; - _ -> - {ok, State, []} + slow = IsSlow}, + if IsSlow andalso not WasSlow -> + {ok, set_timer(QName, State), [{block, cluster_name(State)}]}; + true -> + {ok, State, []} end. %% @doc Enqueues a message. @@ -608,7 +604,7 @@ handle_ra_event(QName, From, {applied, Seqs}, %% is sequence numer agnostic: it handles any correlation terms. [{settled, QName, Corrs} | Actions0] end, - case maps:size(State1#state.pending) < SftLmt of + case map_size(State1#state.pending) < SftLmt of true when State1#state.slow == true -> % we have exited soft limit state % send any unsent commands and cancel the time as @@ -914,32 +910,20 @@ consumer_key(ConsumerTag, #state{consumers = Consumers}) -> consumer_id(ConsumerTag) when is_binary(ConsumerTag) -> {ConsumerTag, self()}. -send_command(Server, Correlation, Command, _Priority, - #state{pending = Pending, - next_seq = Seq, - cfg = #cfg{soft_limit = SftLmt}} = State) - when element(1, Command) == return -> - %% returns are sent to the aux machine for pre-evaluation - ok = ra:cast_aux_command(Server, {Command, Seq, self()}), - Tag = case map_size(Pending) >= SftLmt of - true -> slow; - false -> ok - end, - State#state{pending = Pending#{Seq => {Correlation, Command}}, - next_seq = Seq + 1, - slow = Tag == slow}; send_command(Server, Correlation, Command, Priority, #state{pending = Pending, next_seq = Seq, cfg = #cfg{soft_limit = SftLmt}} = State) -> - ok = ra:pipeline_command(Server, Command, Seq, Priority), - Tag = case map_size(Pending) >= SftLmt of - true -> slow; - false -> ok - end, + ok = case element(1, Command) of + return -> + %% 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, State#state{pending = Pending#{Seq => {Correlation, Command}}, next_seq = Seq + 1, - slow = Tag == slow}. + slow = map_size(Pending) >= SftLmt}. resend_command(ServerId, Correlation, Command, #state{pending = Pending, From a5c7533862830dc86a9ee907b9b16bad69e91b8c Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 27 Jun 2024 12:12:00 +0100 Subject: [PATCH 12/45] Add rabbit_quorum_queue:file_handle* functions back. For backwards compat. --- deps/rabbit/src/rabbit_fifo.erl | 5 ----- deps/rabbit/src/rabbit_quorum_queue.erl | 15 +++++++++++++++ 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 1aa3c6306918..bae820f08cec 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -906,11 +906,6 @@ state_enter0(eol, #?STATE{enqueuers = Enqs, || P <- maps:keys(maps:merge(Enqs, AllConsumers))] ++ [{aux, eol} | Effects]; -state_enter0(State, #?STATE{cfg = #cfg{resource = _Resource}}, Effects) - when State =/= leader -> - FHReservation = {mod_call, rabbit_quorum_queue, - file_handle_other_reservation, []}, - [FHReservation | Effects]; state_enter0(_, _, Effects) -> %% catch all as not handling all states Effects. diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 53fe0f7f38df..4b972338aa1b 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -76,6 +76,11 @@ -export([force_shrink_member_to_current_member/2, force_all_queues_shrink_member_to_current_member/0]). +%% for backwards compatibility +-export([file_handle_leader_reservation/1, + file_handle_other_reservation/0, + file_handle_release_reservation/0]). + -ifdef(TEST). -export([filter_promotable/2]). -endif. @@ -1972,3 +1977,13 @@ is_process_alive(Name, Node) -> %% as this function is used for metrics and stats and the additional %% latency isn't warranted erlang:is_pid(erpc_call(Node, erlang, whereis, [Name], ?RPC_TIMEOUT)). + +%% backwards compat +file_handle_leader_reservation(_QName) -> + ok. + +file_handle_other_reservation() -> + ok. + +file_handle_release_reservation() -> + ok. From b876548e5e99df0def9d9735dd6e81aff573b95f Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 27 Jun 2024 14:11:57 +0100 Subject: [PATCH 13/45] dialyzer fix --- deps/rabbit/src/rabbit_fifo.erl | 5 +++++ deps/rabbit/src/rabbit_fifo_client.erl | 4 ++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index bae820f08cec..1d521a6053d5 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -83,6 +83,7 @@ make_checkout/3, make_settle/2, make_return/2, + is_return/1, make_discard/2, make_credit/4, make_defer/2, @@ -2508,6 +2509,10 @@ make_settle(ConsumerKey, MsgIds) when is_list(MsgIds) -> 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_id(), [msg_id()]) -> protocol(). make_discard(ConsumerKey, MsgIds) -> #discard{consumer_key = ConsumerKey, msg_ids = MsgIds}. diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index 506729a8e25c..31228f1c0da6 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -914,8 +914,8 @@ send_command(Server, Correlation, Command, Priority, #state{pending = Pending, next_seq = Seq, cfg = #cfg{soft_limit = SftLmt}} = State) -> - ok = case element(1, Command) of - return -> + 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()}); _ -> From 0e3a20f6bcd06041016d2a5e16a9ced226b22882 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 28 Jun 2024 10:08:22 +0100 Subject: [PATCH 14/45] dynamic_qq_SUITE: avoid mixed versions failure. --- deps/rabbit/test/dynamic_qq_SUITE.erl | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/deps/rabbit/test/dynamic_qq_SUITE.erl b/deps/rabbit/test/dynamic_qq_SUITE.erl index e13237703fa8..e87f51c79c46 100644 --- a/deps/rabbit/test/dynamic_qq_SUITE.erl +++ b/deps/rabbit/test/dynamic_qq_SUITE.erl @@ -28,7 +28,7 @@ groups() -> {cluster_size_3, [], [ vhost_deletion, quorum_unaffected_after_vhost_failure, - recover_follower_after_standalone_restart, + forget_cluster_node, force_delete_if_no_consensus, takeover_on_failure, takeover_on_shutdown @@ -219,7 +219,7 @@ quorum_unaffected_after_vhost_failure(Config) -> end, 60000). -recover_follower_after_standalone_restart(Config) -> +forget_cluster_node(Config) -> %% Tests that quorum queues shrink when forget_cluster_node %% operations are issues. [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -243,10 +243,10 @@ recover_follower_after_standalone_restart(Config) -> rabbit_ct_client_helpers:close_channel(Ch), %% Restart one follower - forget_cluster_node(Config, B, C), - wait_for_messages_ready([B], Name, 15), - forget_cluster_node(Config, B, A), - wait_for_messages_ready([B], Name, 15), + forget_cluster_node(Config, C, B), + wait_for_messages_ready([C], Name, 15), + forget_cluster_node(Config, C, A), + wait_for_messages_ready([C], Name, 15), ok. From da09e8d48578b75d8d6b325f21ae953ef31a9158 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 2 Jul 2024 12:45:57 +0100 Subject: [PATCH 15/45] QQ: track number of requeues for message. To be able to calculate the correct value for the AMQP delivery_count header we need to be able to distinguish between messages that were "released" or returned in QQ speak and those that were returned due to errors such as channel termination. This commit implement such tracking as well as the calculation of a new mc annotations `delivery_count` that AMQP makes use of to set the header value accordingly. --- deps/rabbit/BUILD.bazel | 3 + deps/rabbit/app.bzl | 2 +- deps/rabbit/src/mc_amqp.erl | 39 ++----- deps/rabbit/src/rabbit_fifo.erl | 36 +++--- deps/rabbit/src/rabbit_fifo.hrl | 4 +- deps/rabbit/src/rabbit_fifo_client.erl | 34 +++--- deps/rabbit/test/amqp_system_SUITE.erl | 64 ++++++++++- .../fsharp-tests/Program.fs | 104 +++++++++--------- deps/rabbit/test/rabbit_fifo_SUITE.erl | 43 +++++++- deps/rabbit/test/rabbit_fifo_int_SUITE.erl | 98 ++++++++++++++--- 10 files changed, 290 insertions(+), 137 deletions(-) diff --git a/deps/rabbit/BUILD.bazel b/deps/rabbit/BUILD.bazel index d666dabe9304..a3e008e5b3b2 100644 --- a/deps/rabbit/BUILD.bazel +++ b/deps/rabbit/BUILD.bazel @@ -705,6 +705,9 @@ rabbitmq_suite( rabbitmq_suite( name = "rabbit_fifo_int_SUITE", size = "medium", + additional_beam = [ + ":test_test_util_beam", + ], deps = [ "//deps/rabbit_common:erlang_app", "@aten//:erlang_app", diff --git a/deps/rabbit/app.bzl b/deps/rabbit/app.bzl index b09512bc2b2d..b968962dae09 100644 --- a/deps/rabbit/app.bzl +++ b/deps/rabbit/app.bzl @@ -1326,7 +1326,7 @@ def test_suite_beam_files(name = "test_suite_beam_files"): outs = ["test/rabbit_fifo_int_SUITE.beam"], app_name = "rabbit", erlc_opts = "//:test_erlc_opts", - deps = ["//deps/rabbit_common:erlang_app"], + deps = ["//deps/rabbitmq_ct_helpers:erlang_app"], ) erlang_bytecode( name = "rabbit_fifo_prop_SUITE_beam_files", diff --git a/deps/rabbit/src/mc_amqp.erl b/deps/rabbit/src/mc_amqp.erl index 3a90e2879842..16cbdfa6e828 100644 --- a/deps/rabbit/src/mc_amqp.erl +++ b/deps/rabbit/src/mc_amqp.erl @@ -222,14 +222,7 @@ get_property(priority, Msg) -> -spec protocol_state(state(), mc:annotations()) -> iolist(). protocol_state(Msg0 = #msg_body_decoded{header = Header0, message_annotations = MA0}, Anns) -> - FirstAcquirer = first_acquirer(Anns), - Header = case Header0 of - undefined -> - #'v1_0.header'{durable = true, - first_acquirer = FirstAcquirer}; - #'v1_0.header'{} -> - Header0#'v1_0.header'{first_acquirer = FirstAcquirer} - end, + Header = update_header_from_anns(Header0, Anns), MA = protocol_state_message_annotations(MA0, Anns), Msg = Msg0#msg_body_decoded{header = Header, message_annotations = MA}, @@ -238,14 +231,7 @@ protocol_state(Msg0 = #msg_body_decoded{header = Header0, protocol_state(#msg_body_encoded{header = Header0, message_annotations = MA0, bare_and_footer = BareAndFooter}, Anns) -> - FirstAcquirer = first_acquirer(Anns), - Header = case Header0 of - undefined -> - #'v1_0.header'{durable = true, - first_acquirer = FirstAcquirer}; - #'v1_0.header'{} -> - Header0#'v1_0.header'{first_acquirer = FirstAcquirer} - end, + Header = update_header_from_anns(Header0, Anns), MA = protocol_state_message_annotations(MA0, Anns), Sections = to_sections(Header, MA, []), [encode(Sections), BareAndFooter]; @@ -269,10 +255,9 @@ protocol_state(#v1{message_annotations = MA0, _ -> undefined end, - Header = #'v1_0.header'{durable = Durable, - priority = Priority, - ttl = Ttl, - first_acquirer = first_acquirer(Anns)}, + Header = update_header_from_anns(#'v1_0.header'{durable = Durable, + priority = Priority, + ttl = Ttl}, Anns), MA = protocol_state_message_annotations(MA0, Anns), Sections = to_sections(Header, MA, []), [encode(Sections), BareAndFooter]. @@ -573,13 +558,13 @@ msg_body_encoded([{{pos, Pos}, {body, Code}}], BarePos, Msg) binary_part_bare_and_footer(Payload, Start) -> binary_part(Payload, Start, byte_size(Payload) - Start). --spec first_acquirer(mc:annotations()) -> boolean(). -first_acquirer(Anns) -> - Redelivered = case Anns of - #{redelivered := R} -> R; - _ -> false - end, - not Redelivered. +update_header_from_anns(undefined, Anns) -> + update_header_from_anns(#'v1_0.header'{durable = true}, Anns); +update_header_from_anns(Header, Anns) -> + FirstAcq = not maps:get(redelivered, Anns, false), + DeliveryCount = {uint, maps:get(delivery_count, Anns, 0)}, + Header#'v1_0.header'{first_acquirer = FirstAcq, + delivery_count = DeliveryCount}. encode_deaths(Deaths) -> lists:map( diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 1d521a6053d5..f5b3d262d7cc 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -96,7 +96,8 @@ -ifdef(TEST). -export([update_header/4, chunk_disk_msgs/3, - smallest_raft_index/1]). + smallest_raft_index/1, + make_requeue/4]). -endif. -import(serial_number, [add/2, diff/2]). @@ -308,7 +309,8 @@ apply(#{index := Idx} = Meta, when is_map_key(MsgId, Checked0) -> %% construct a message with the current raft index %% and update delivery count before adding it to the message queue - Header = update_header(delivery_count, fun incr/1, 1, Header0), + Header1 = update_header(delivery_count, fun incr/1, 1, Header0), + Header = update_header(return_count, fun incr/1, 1, Header1), State0 = add_bytes_return(Header, State00), Con = Con0#consumer{checked_out = maps:remove(MsgId, Checked0), credit = increase_credit(Con0, 1)}, @@ -1745,17 +1747,19 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, return(#{index := IncomingRaftIdx} = Meta, ConsumerKey, MsgIds, Checked, Effects0, State0) -> %% 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 := Val} -> - {_At, Msg} = Val, - return_one(Meta, MsgId, Msg, - S0, E0, ConsumerKey); - #{} -> - Acc - end - end, {State0, Effects0}, MsgIds), + {State1, Effects1} = + lists:foldl( + fun(MsgId, Acc = {S0, E0}) -> + case Checked of + #{MsgId := {_At, Msg0}} -> + Msg = update_msg_header(return_count, fun incr/1, 1, + Msg0), + return_one(Meta, MsgId, Msg, + 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); @@ -1904,11 +1908,13 @@ 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), is_integer(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(Header), is_map_key(size, Header) -> + when is_map(Header) andalso + is_map_key(size, Header) -> maps:update_with(Key, UpdateFun, Default, Header). get_msg_header(?MSG(_Idx, Header)) -> diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index 29c522999fdd..4da4db55e152 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -42,11 +42,13 @@ optimised_tuple(msg_size(), Expiry :: milliseconds()) | #{size := msg_size(), delivery_count => non_neg_integer(), + return_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. +%% 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. diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index 31228f1c0da6..cd7b2f076a04 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -208,12 +208,7 @@ dequeue(QueueName, ConsumerTag, Settlement, {ok, {dequeue, empty}, Leader} -> {empty, State0#state{leader = Leader}}; {ok, {dequeue, {MsgId, {MsgHeader, Msg0}}, MsgsReady}, Leader} -> - Count = case MsgHeader of - #{delivery_count := C} -> C; - _ -> 0 - end, - IsDelivered = Count > 0, - Msg = add_delivery_count_header(Msg0, Count), + {Msg, IsDelivered} = add_delivery_count_header(Msg0, MsgHeader), {ok, MsgsReady, {QueueName, qref(Leader), MsgId, IsDelivered, Msg}, State0#state{leader = Leader}}; @@ -223,14 +218,21 @@ dequeue(QueueName, ConsumerTag, Settlement, Err end. -add_delivery_count_header(Msg, Count) -> - case mc:is(Msg) of - true when is_integer(Count) andalso - Count > 0 -> - mc:set_annotation(<<"x-delivery-count">>, Count, Msg); +add_delivery_count_header(Msg, #{delivery_count := DelCount} = Header) + when is_integer(DelCount) -> + {case mc:is(Msg) of + true -> + %% the "delivery-count" header in the AMQP spec does not include + %% returns (released outcomes) + AmqpDelCount = DelCount - maps:get(return_count, Header, 0), + mc:set_annotation(delivery_count, AmqpDelCount, + mc:set_annotation(<<"x-delivery-count">>, + DelCount, Msg)); _ -> Msg - end. + end, DelCount > 0}; +add_delivery_count_header(Msg, _Header) -> + {Msg, false}. %% @doc Settle a message. Permanently removes message from the queue. @@ -840,13 +842,7 @@ handle_delivery(_QName, _Leader, {delivery, Tag, [_ | _] = IdMsgs}, transform_msgs(QName, QRef, Msgs) -> lists:map( fun({MsgId, {MsgHeader, Msg0}}) -> - {Msg, Redelivered} = case MsgHeader of - #{delivery_count := C} -> - {add_delivery_count_header(Msg0, C), true}; - _ -> - {Msg0, false} - end, - + {Msg, Redelivered} = add_delivery_count_header(Msg0, MsgHeader), {QName, QRef, MsgId, Redelivered, Msg} end, Msgs). diff --git a/deps/rabbit/test/amqp_system_SUITE.erl b/deps/rabbit/test/amqp_system_SUITE.erl index 9b3ed61e84a0..f275ecac98dc 100644 --- a/deps/rabbit/test/amqp_system_SUITE.erl +++ b/deps/rabbit/test/amqp_system_SUITE.erl @@ -34,6 +34,7 @@ groups() -> %% TODO at_most_once, reject, redelivery, + released, routing, invalid_routes, auth_failure, @@ -80,6 +81,13 @@ end_per_group(_, Config) -> rabbit_ct_broker_helpers:teardown_steps()). init_per_testcase(Testcase, Config) -> + enable_feature_flags(Config, + [ + message_containers_store_amqp_v1, + credit_api_v2, + quorum_queues_v4 + % amqp_address_v1 + ]), rabbit_ct_helpers:testcase_started(Config, Testcase). end_per_testcase(Testcase, Config) -> @@ -115,22 +123,20 @@ build_maven_test_project(Config) -> %% ------------------------------------------------------------------- roundtrip(Config) -> + declare_queue(Config, ?FUNCTION_NAME, "quorum"), run(Config, [{dotnet, "roundtrip"}, {java, "RoundTripTest"}]). streams(Config) -> - _ = rabbit_ct_broker_helpers:enable_feature_flag(Config, - message_containers_store_amqp_v1), - Ch = rabbit_ct_client_helpers:open_channel(Config), - amqp_channel:call(Ch, #'queue.declare'{queue = <<"stream_q2">>, - durable = true, - arguments = [{<<"x-queue-type">>, longstr, "stream"}]}), + declare_queue(Config, ?FUNCTION_NAME, "stream"), run(Config, [{dotnet, "streams"}]). roundtrip_to_amqp_091(Config) -> + declare_queue(Config, ?FUNCTION_NAME, "classic"), run(Config, [{dotnet, "roundtrip_to_amqp_091"}]). default_outcome(Config) -> + declare_queue(Config, ?FUNCTION_NAME, "classic"), run(Config, [{dotnet, "default_outcome"}]). no_routes_is_released(Config) -> @@ -140,28 +146,41 @@ no_routes_is_released(Config) -> run(Config, [{dotnet, "no_routes_is_released"}]). outcomes(Config) -> + declare_queue(Config, ?FUNCTION_NAME, "classic"), run(Config, [{dotnet, "outcomes"}]). fragmentation(Config) -> + declare_queue(Config, ?FUNCTION_NAME, "classic"), run(Config, [{dotnet, "fragmentation"}]). message_annotations(Config) -> + declare_queue(Config, ?FUNCTION_NAME, "classic"), run(Config, [{dotnet, "message_annotations"}]). footer(Config) -> + declare_queue(Config, ?FUNCTION_NAME, "classic"), run(Config, [{dotnet, "footer"}]). data_types(Config) -> + declare_queue(Config, ?FUNCTION_NAME, "classic"), run(Config, [{dotnet, "data_types"}]). reject(Config) -> + declare_queue(Config, ?FUNCTION_NAME, "classic"), run(Config, [{dotnet, "reject"}]). redelivery(Config) -> + declare_queue(Config, ?FUNCTION_NAME, "quorum"), run(Config, [{dotnet, "redelivery"}]). +released(Config) -> + declare_queue(Config, ?FUNCTION_NAME, "quorum"), + run(Config, [{dotnet, "released"}]). + routing(Config) -> Ch = rabbit_ct_client_helpers:open_channel(Config), + amqp_channel:call(Ch, #'queue.declare'{queue = <<"test">>, + durable = true}), amqp_channel:call(Ch, #'queue.declare'{queue = <<"transient_q">>, durable = false}), amqp_channel:call(Ch, #'queue.declare'{queue = <<"durable_q">>, @@ -174,6 +193,18 @@ routing(Config) -> arguments = [{<<"x-queue-type">>, longstr, <<"stream">>}]}), amqp_channel:call(Ch, #'queue.declare'{queue = <<"autodel_q">>, auto_delete = true}), + amqp_channel:call(Ch, #'queue.declare'{queue = <<"fanout_q">>, + durable = false}), + amqp_channel:call(Ch, #'queue.bind'{queue = <<"fanout_q">>, + exchange = <<"amq.fanout">> + }), + amqp_channel:call(Ch, #'queue.declare'{queue = <<"direct_q">>, + durable = false}), + amqp_channel:call(Ch, #'queue.bind'{queue = <<"direct_q">>, + exchange = <<"amq.direct">>, + routing_key = <<"direct_q">> + }), + run(Config, [ {dotnet, "routing"} ]). @@ -227,6 +258,7 @@ run_dotnet_test(Config, Method) -> [ {cd, TestProjectDir} ]), + ct:pal("~s: result ~p", [?FUNCTION_NAME, Ret]), {ok, _} = Ret. run_java_test(Config, Class) -> @@ -239,3 +271,23 @@ run_java_test(Config, Class) -> ], [{cd, TestProjectDir}]), {ok, _} = Ret. + + +enable_feature_flags(Config, Flags) -> + [begin + case rabbit_ct_broker_helpers:enable_feature_flag(Config, Flag) of + ok -> ok; + _ -> + throw({skip, "feature flag ~s could not be enabled"}) + end + end || Flag <- Flags]. + +declare_queue(Config, Name, Type) -> + Ch = rabbit_ct_client_helpers:open_channel(Config), + #'queue.declare_ok'{} = + amqp_channel:call(Ch, #'queue.declare'{queue = atom_to_binary(Name, utf8), + durable = true, + arguments = [{<<"x-queue-type">>, + longstr, Type}]}), + rabbit_ct_client_helpers:close_channel(Ch), + ok. diff --git a/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs b/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs index 7ed91f388f70..505e1390767e 100755 --- a/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs +++ b/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs @@ -161,7 +161,7 @@ module Test = let roundtrip uri = use c = connect uri - let sender, receiver = senderReceiver c "test" "roundtrip-q" + let sender, receiver = senderReceiver c "test" "/queue/roundtrip" for body in sampleTypes do let corr = "correlation" new Message(body, @@ -177,7 +177,7 @@ module Test = let streams uri = use c = connect uri let name = "streams-test" - let address = "/amq/queue/stream_q2" + let address = "/queue/streams" let sender = SenderLink(c.Session, name + "-sender" , address) //for body in sampleTypes do let body = "hi"B :> obj @@ -217,9 +217,10 @@ module Test = let roundtrip_to_amqp_091 uri = use c = connect uri - let q = "roundtrip-091-q" + let q = "roundtrip_to_amqp_091 " + let target = "/queue/roundtrip_to_amqp_091 " let corr = "correlation" - let sender = SenderLink(c.Session, q + "-sender" , q) + let sender = SenderLink(c.Session, q + "-sender" , target) new Message("hi"B, Header = Header(), Properties = new Properties(CorrelationId = corr)) @@ -242,13 +243,13 @@ module Test = assertEqual id corr () - let defaultOutcome uri = + let default_outcome uri = for (defOut, cond, defObj) in ["amqp:accepted:list", null, Accepted() :> Outcome "amqp:rejected:list", null, Rejected() :> Outcome "amqp:released:list", null, Released() :> Outcome] do - let source = new Source(Address = "default_outcome_q", + let source = new Source(Address = "/queue/default_outcome", DefaultOutcome = defObj) let attach = new Attach (Source = source, Target = Target()) @@ -263,7 +264,7 @@ module Test = "amqp:modified:list", null "amqp:madeup:list", "amqp:not-implemented"] do - let source = new Source(Address = "outcomes_q", + let source = new Source(Address = "/queue/outcomes", Outcomes = [| Symbol outcome |]) let attach = new Attach (Source = source, Target = Target()) @@ -282,15 +283,15 @@ module Test = HostName = addr.Host, ChannelMax = 256us, MaxFrameSize = frameSize) use c = connectWithOpen uri opn - let sender, receiver = senderReceiver c "test" "framentation-q" + let sender, receiver = senderReceiver c "test" "/queue/framentation" let m = new Message(String.replicate size "a") sender.Send m let m' = receive receiver assertEqual (m.Body) (m'.Body) - let messageAnnotations uri = + let message_annotations uri = use c = connect uri - let sender, receiver = senderReceiver c "test" "annotations-q" + let sender, receiver = senderReceiver c "test" "/queue/message_annotations" let ann = MessageAnnotations() let k1 = Symbol "key1" let k2 = Symbol "key2" @@ -309,7 +310,7 @@ module Test = let footer uri = use c = connect uri - let sender, receiver = senderReceiver c "test" "footer-q" + let sender, receiver = senderReceiver c "test" "/queue/footer" let footer = Footer() let k1 = Symbol "key1" let k2 = Symbol "key2" @@ -325,9 +326,9 @@ module Test = assertTrue (m.Footer.[k1] = m'.Footer.[k1]) assertTrue (m.Footer.[k2] = m'.Footer.[k2]) - let datatypes uri = + let data_types uri = use c = connect uri - let sender, receiver = senderReceiver c "test" "datatypes-q" + let sender, receiver = senderReceiver c "test" "/queue/data_types" let aSeq = amqpSequence sampleTypes (new Message(aSeq)) |> sender.Send let rtd = receive receiver @@ -337,7 +338,7 @@ module Test = let reject uri = use c = connect uri - let sender, receiver = senderReceiver c "test" "reject-q" + let sender, receiver = senderReceiver c "test" "/queue/reject" new Message "testing reject" |> sender.Send let m = receiver.Receive() receiver.Reject(m) @@ -345,58 +346,58 @@ module Test = let redelivery uri = use c = connect uri - let sender, receiver = senderReceiver c "test" "redelivery-q" + let sender, receiver = senderReceiver c "test" "/queue/redelivery" new Message "testing redelivery" |> sender.Send let m = receiver.Receive() assertTrue (m.Header.FirstAcquirer) - receiver.Close() c.Session.Close() + let session = Session(c.Conn) - let receiver = ReceiverLink(session, "test-receiver", "redelivery-q") + let receiver = ReceiverLink(session, "test-receiver", "/queue/redelivery") let m' = receive receiver assertEqual (m.Body :?> string) (m'.Body :?> string) assertTrue (not m'.Header.FirstAcquirer) + assertEqual 1u (m'.Header.DeliveryCount) assertEqual null (receiver.Receive(TimeSpan.FromMilliseconds 100.)) session.Close() + let released uri = + use c = connect uri + let sender, receiver = senderReceiver c "test" "/queue/released" + new Message "testing released" |> sender.Send + let m = receiver.Receive() + assertTrue (m.Header.FirstAcquirer) + receiver.SetCredit(0, false) + receiver.Release m + + let m' = receive receiver + assertEqual (m.Body :?> string) (m'.Body :?> string) + assertTrue (not m'.Header.FirstAcquirer) + assertEqual 0u (m'.Header.DeliveryCount) + assertEqual null (receiver.Receive(TimeSpan.FromMilliseconds 100.)) + c.Session.Close() + let routing uri = - for target, source, routingKey, succeed in - ["/queue/test", "test", "", true - "test", "/queue/test", "", true - "test", "test", "", true - - "/topic/a.b.c.d", "/topic/#.c.*", "", true - "/exchange/amq.topic", "/topic/#.c.*", "a.b.c.d", true - "/topic/w.x.y.z", "/exchange/amq.topic/#.y.*", "", true - "/exchange/amq.topic", "/exchange/amq.topic/#.y.*", "w.x.y.z", true - - "/exchange/amq.fanout", "/exchange/amq.fanout/", "", true - "/exchange/amq.direct", "/exchange/amq.direct/", "", true - "/exchange/amq.direct", "/exchange/amq.direct/a", "a", true - "/queue", "/queue/b", "b", true - - (* FIXME: The following three tests rely on the queue "test" - * created by previous tests in this function. *) - "/queue/test", "/amq/queue/test", "", true - "/amq/queue/test", "/queue/test", "", true - "/amq/queue/test", "/amq/queue/test", "", true - - (* The following tests verify that a queue created out-of-band - * in AMQP is reachable from the AMQP 1.0 world. Queues are created - * from the common_test suite. *) - "/amq/queue/transient_q", "/amq/queue/transient_q", "", true - "/amq/queue/durable_q", "/amq/queue/durable_q", "", true - "/amq/queue/quorum_q", "/amq/queue/quorum_q", "", true - "/amq/queue/stream_q", "/amq/queue/stream_q", "", true - "/amq/queue/autodel_q", "/amq/queue/autodel_q", "", true] do + for target, source, toProp, succeed in + [ + "/queue/test", "/queue/test", "", true + "/exchange/amq.fanout", "/queue/fanout_q", "", true + "/exchange/amq.direct/key/direct_q", "/queue/direct_q", "", true + null, "/queue/direct_q", "/exchange/amq.direct/key/direct_q", true + "/queue/transient_q", "/queue/transient_q", "", true + "/queue/durable_q", "/queue/durable_q", "", true + "/queue/quorum_q", "/queue/quorum_q", "", true + "/queue/stream_q", "/queue/stream_q", "", true + "/queue/autodel_q", "/queue/autodel_q", "", true] do let rnd = Random() use c = connect uri let sender = SenderLink(c.Session, "test-sender", target) let receiver = ReceiverLink(c.Session, "test-receiver", source) receiver.SetCredit(100, true) - use m = new Message(rnd.Next(10000), Properties = Properties(Subject = routingKey)) + use m = new Message(rnd.Next(10000), + Properties = Properties(To = toProp)) sender.Send m (* printfn "%s %s %s %A" target source routingKey succeed *) @@ -521,10 +522,10 @@ let main argv = roundtrip_to_amqp_091 uri 0 | [AsLower "data_types"; uri] -> - datatypes uri + data_types uri 0 | [AsLower "default_outcome"; uri] -> - defaultOutcome uri + default_outcome uri 0 | [AsLower "outcomes"; uri] -> outcomes uri @@ -533,7 +534,7 @@ let main argv = fragmentation uri 0 | [AsLower "message_annotations"; uri] -> - messageAnnotations uri + message_annotations uri 0 | [AsLower "footer"; uri] -> footer uri @@ -544,6 +545,9 @@ let main argv = | [AsLower "redelivery"; uri] -> redelivery uri 0 + | [AsLower "released"; uri] -> + released uri + 0 | [AsLower "routing"; uri] -> routing uri 0 diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index f4179b41d3b7..3e7b12f0fb3e 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -15,7 +15,7 @@ -include_lib("rabbit_common/include/rabbit_framing.hrl"). -include_lib("rabbit/src/rabbit_fifo.hrl"). --define(PROTOMOD, rabbit_framing_amqp_0_9_1). +% -define(PROTOMOD, rabbit_framing_amqp_0_9_1). %%%=================================================================== %%% Common Test callbacks %%%=================================================================== @@ -570,11 +570,44 @@ return_auto_checked_out_test(Config) -> [_Monitor, {log, [1], Fun1, _} ]} = checkout(Config, ?LINE, Cid, 1, State0), [{send_msg, _, {delivery, _, [{MsgId, _}]}, _}] = Fun1([Msg1]), % return should include another delivery - {_State2, _, Effects} = apply(meta(Config, 3), + {State2, _, Effects} = apply(meta(Config, 3), rabbit_fifo:make_return(CKey, [MsgId]), State1), [{log, [1], Fun2, _} | _] = Effects, - [{send_msg, _, {delivery, _, [{_MsgId2, {#{delivery_count := 1}, first}}]}, _}] - = Fun2([Msg1]), + [{send_msg, _, {delivery, _, [{_MsgId2, {#{delivery_count := 1, + return_count := 1}, first}}]}, _}] + = Fun2([Msg1]), + + %% a down does not increment the return_count + {State3, _, _} = apply(meta(Config, ?LINE), {down, self(), noproc}, State2), + + {_State4, #{key := _CKey2, + next_msg_id := _}, + [_, {log, [1], Fun3, _} ]} = checkout(Config, ?LINE, Cid, 1, State3), + + [{send_msg, _, {delivery, _, [{_, {#{delivery_count := 2, + return_count := 1}, first}}]}, _}] + = Fun3([Msg1]), + ok. + +requeue_test(Config) -> + Cid = {<<"cid">>, self()}, + Msg1 = rabbit_fifo:make_enqueue(self(), 1, first), + {State0, _} = enq(Config, 1, 1, first, test_init(test)), + % it first active then inactive as the consumer took on but cannot take + % any more + {State1, #{key := CKey, + next_msg_id := MsgId}, + [_Monitor, {log, [1], Fun1, _} ]} = checkout(Config, ?LINE, Cid, 1, State0), + [{send_msg, _, {delivery, _, [{MsgId, {H1, _}}]}, _}] = Fun1([Msg1]), + % return should include another delivery + [{append, Requeue, _}] = rabbit_fifo:make_requeue(CKey, {notify, 1, self()}, + [{MsgId, 1, H1, Msg1}], []), + {_State2, _, Effects} = apply(meta(Config, 3), Requeue, State1), + [{log, [_], Fun2, _} | _] = Effects, + [{send_msg, _, + {delivery, _, [{_MsgId2, {#{delivery_count := 1, + return_count := 1}, first}}]}, _}] + = Fun2([Msg1]), ok. cancelled_checkout_empty_queue_test(Config) -> @@ -1250,7 +1283,7 @@ single_active_consumer_all_disconnected_test(Config) -> ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = up, credit = 1}}}) ], - {State1, _} = run_log(Config, State0, Entries), + {_State1, _} = run_log(Config, State0, Entries), ok. single_active_consumer_state_enter_leader_include_waiting_consumers_test(Config) -> diff --git a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl index 1809de4f54c4..eba5a77a70d4 100644 --- a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl @@ -8,6 +8,7 @@ -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). -include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("rabbit_common/include/rabbit_framing.hrl"). -define(RA_EVENT_TIMEOUT, 5000). -define(RA_SYSTEM, quorum_queues). @@ -23,6 +24,7 @@ all_tests() -> return, rabbit_fifo_returns_correlation, resends_lost_command, + returns, returns_after_down, resends_after_lost_applied, handles_reject_notification, @@ -99,14 +101,13 @@ basics(Config) -> rabbit_quorum_queue:wal_force_roll_over(node()), % create segment the segment will trigger a snapshot - timer:sleep(1000), + ra_log_segment_writer:await(ra_log_segment_writer), {ok, FState2, []} = rabbit_fifo_client:enqueue(ClusterName, one, FState1), DeliverFun = fun DeliverFun(S0, F) -> receive {ra_event, From, Evt} -> - ct:pal("ra_event ~p", [Evt]), case rabbit_fifo_client:handle_ra_event(ClusterName, From, Evt, S0) of {ok, S1, [{deliver, C, true, @@ -289,28 +290,90 @@ detects_lost_delivery(Config) -> rabbit_quorum_queue:stop_server(ServerId), ok. +returns(Config) -> + ClusterName = ?config(cluster_name, Config), + ServerId = ?config(node_id, Config), + ok = start_cluster(ClusterName, [ServerId]), + + F0 = rabbit_fifo_client:init([ServerId]), + Msg1 = mk_msg(<<"msg1">>), + {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, Msg1, F0), + {_, _, _F2} = process_ra_events(receive_ra_events(1, 0), ClusterName, F1), + + FC = rabbit_fifo_client:init([ServerId]), + {ok, _, FC1} = rabbit_fifo_client:checkout(<<"tag">>, + {simple_prefetch, 10}, + #{}, FC), + + {FC3, _} = + receive + {ra_event, Qname, {machine, {delivery, _, [{MsgId, {_, _}}]}} = Evt1} -> + {ok, FC2, Actions1} = + rabbit_fifo_client:handle_ra_event(Qname, Qname, Evt1, FC1), + [{deliver, _, true, + [{_, _, _, _, Msg1Out0}]}] = Actions1, + ?assert(mc:is(Msg1Out0)), + ?assertEqual(undefined, mc:get_annotation(<<"x-delivery-count">>, Msg1Out0)), + ?assertEqual(undefined, mc:get_annotation(delivery_count, Msg1Out0)), + rabbit_fifo_client:return(<<"tag">>, [MsgId], FC2) + after 5000 -> + flush(), + exit(await_delivery_timeout) + end, + receive + {ra_event, Qname2, + {machine, {delivery, _, [{_MsgId, {_, _Msg1Out}}]}} = Evt2} -> + {ok, _FC4, Actions2} = + rabbit_fifo_client:handle_ra_event(Qname2, Qname2, Evt2, FC3), + % ct:pal("Actions2 ~p", [Actions2]), + [{deliver, _tag, true, + [{_, _, _, _, Msg1Out}]}] = Actions2, + ?assert(mc:is(Msg1Out)), + ?assertEqual(1, mc:get_annotation(<<"x-delivery-count">>, Msg1Out)), + ?assertEqual(0, mc:get_annotation(delivery_count, Msg1Out)), + ok + after 5000 -> + flush(), + exit(await_delivery_timeout) + end, + rabbit_quorum_queue:stop_server(ServerId), + ok. + returns_after_down(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:enqueue(ClusterName, msg1, F0), + Msg1 = mk_msg(<<"msg1">>), + {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, Msg1, F0), {_, _, F2} = process_ra_events(receive_ra_events(1, 0), ClusterName, F1), % start a consumer in a separate processes % that exits after checkout Self = self(), - _Pid = spawn(fun () -> - F = rabbit_fifo_client:init([ServerId]), - {ok, _, _} = rabbit_fifo_client:checkout(<<"tag">>, - {simple_prefetch, 10}, - #{}, F), - Self ! checkout_done - end), - receive checkout_done -> ok after 1000 -> exit(checkout_done_timeout) end, - timer:sleep(1000), + Pid = spawn(fun () -> + F = rabbit_fifo_client:init([ServerId]), + {ok, _, _} = rabbit_fifo_client:checkout(<<"tag">>, + {simple_prefetch, 10}, + #{}, F), + Self ! checkout_done + end), + receive checkout_done -> ok + after 1000 -> + exit(checkout_done_timeout) + end, + MonRef = erlang:monitor(process, Pid), + receive + {'DOWN', MonRef, _, _, _} -> + ok + after 5000 -> + ct:fail("waiting for process exit timed out") + end, % message should be available for dequeue - {ok, _, {_, _, _, _, msg1}, _} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F2), + {ok, _, {_, _, _, _, Msg1Out}, _} = + rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F2), + ?assertEqual(1, mc:get_annotation(<<"x-delivery-count">>, Msg1Out)), + ?assertEqual(1, mc:get_annotation(delivery_count, Msg1Out)), rabbit_quorum_queue:stop_server(ServerId), ok. @@ -811,3 +874,12 @@ flush() -> after 10 -> ok end. + +mk_msg(Body) when is_binary(Body) -> + mc_amqpl:from_basic_message( + #basic_message{routing_keys = [<<"">>], + exchange_name = #resource{name = <<"x">>, + kind = exchange, + virtual_host = <<"v">>}, + content = #content{properties = #'P_basic'{}, + payload_fragments_rev = [Body]}}). From 39a5f3b153331a6868580154320678445475eff5 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 3 Jul 2024 17:04:38 +0000 Subject: [PATCH 16/45] Use QQ consumer removal when AMQP client detaches This enables us to unskip some AMQP tests. --- deps/rabbit/src/rabbit_fifo.erl | 2 +- deps/rabbit/src/rabbit_fifo_client.erl | 8 ++--- deps/rabbit/src/rabbit_quorum_queue.erl | 3 +- deps/rabbit/test/amqp_client_SUITE.erl | 42 ++++++++++++---------- deps/rabbit/test/rabbit_fifo_int_SUITE.erl | 2 +- 5 files changed, 30 insertions(+), 27 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index f5b3d262d7cc..c5db56a6e504 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -2500,7 +2500,7 @@ make_checkout({_, _} = ConsumerId, Spec0, Meta) -> Spec = case is_v4() of false when Spec0 == remove -> %% if v4 is not active, fall back to cancel spec - make_checkout(ConsumerId, cancel, Meta); + cancel; _ -> Spec0 end, diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index cd7b2f076a04..513c248c95f9 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -15,7 +15,6 @@ init/1, init/2, checkout/4, - cancel_checkout/2, cancel_checkout/3, enqueue/3, enqueue/4, @@ -444,10 +443,7 @@ credit(ConsumerTag, DeliveryCount, Credit, Drain, State) -> Cmd = rabbit_fifo:make_credit(ConsumerKey, Credit, DeliveryCount, Drain), {send_command(ServerId, undefined, Cmd, normal, State), []}. -cancel_checkout(ConsumerTag, State) -> - cancel_checkout(ConsumerTag, cancel, State). - -%% @doc Cancels a checkout with the rabbit_fifo queue for the consumer tag +%% @doc Cancels a checkout with the rabbit_fifo queue for the consumer tag %% %% This is a synchronous call. I.e. the call will block until the command %% has been accepted by the ra process or it times out. @@ -456,7 +452,7 @@ cancel_checkout(ConsumerTag, State) -> %% @param State The {@module} state. %% %% @returns `{ok, State}' or `{error | timeout, term()}' --spec cancel_checkout(rabbit_types:ctag(), Reason :: cancel | remove, state()) -> +-spec cancel_checkout(rabbit_types:ctag(), rabbit_queue_type:cancel_reason(), state()) -> {ok, state()} | {error | timeout, term()}. cancel_checkout(ConsumerTag, Reason, #state{consumers = Consumers, diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 4b972338aa1b..e0a8d169aef7 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -938,7 +938,8 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) -> cancel(_Q, #{consumer_tag := ConsumerTag} = Spec, State) -> maybe_send_reply(self(), maps:get(ok_msg, Spec, undefined)), - rabbit_fifo_client:cancel_checkout(quorum_ctag(ConsumerTag), State). + Reason = maps:get(reason, Spec, cancel), + rabbit_fifo_client:cancel_checkout(quorum_ctag(ConsumerTag), Reason, State). emit_consumer_created(ChPid, CTag, Exclusive, AckRequired, QName, PrefetchCount, Args, Ref, ActingUser) -> rabbit_event:notify(consumer_created, diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl index 8346bc3186dc..c6e73abbe9ea 100644 --- a/deps/rabbit/test/amqp_client_SUITE.erl +++ b/deps/rabbit/test/amqp_client_SUITE.erl @@ -232,13 +232,10 @@ init_per_testcase(T, Config) {skip, "Feature flag credit_api_v2 is disabled"} end; init_per_testcase(T, Config) - when T =:= detach_requeues_one_session_classic_queue orelse - T =:= detach_requeues_one_session_quorum_queue orelse - T =:= detach_requeues_drop_head_classic_queue orelse - T =:= detach_requeues_two_connections_classic_queue orelse - T =:= detach_requeues_two_connections_quorum_queue orelse - T =:= single_active_consumer_classic_queue orelse - T =:= single_active_consumer_quorum_queue -> + when T =:= detach_requeues_one_session_classic_queue orelse + T =:= detach_requeues_drop_head_classic_queue orelse + T =:= detach_requeues_two_connections_classic_queue orelse + T =:= single_active_consumer_classic_queue -> %% Cancel API v2 reuses feature flag credit_api_v2. %% In 3.13, with cancel API v1, when a receiver detaches with unacked messages, these messages %% will remain unacked and unacked message state will be left behind in the server session @@ -252,6 +249,16 @@ init_per_testcase(T, Config) false -> {skip, "Cancel API v2 is disabled due to feature flag credit_api_v2 being disabled."} end; +init_per_testcase(T, Config) + when T =:= detach_requeues_one_session_quorum_queue orelse + T =:= single_active_consumer_quorum_queue orelse + T =:= detach_requeues_two_connections_quorum_queue -> + case rabbit_ct_broker_helpers:enable_feature_flag(Config, quorum_queues_v4) of + ok -> + rabbit_ct_helpers:testcase_started(Config, T); + {skip, _} -> + {skip, "Feature flag quorum_queues_v4 enables the consumer removal API"} + end; init_per_testcase(T = immutable_bare_message, Config) -> case rpc(Config, rabbit_feature_flags, is_enabled, [message_containers_store_amqp_v1]) of true -> @@ -1950,9 +1957,8 @@ consumer_priority(QType, Config) -> single_active_consumer_classic_queue(Config) -> single_active_consumer(<<"classic">>, Config). -single_active_consumer_quorum_queue(_Config) -> - % single_active_consumer(<<"quorum">>, Config). - {skip, "TODO: unskip when qq-v4 branch is merged"}. +single_active_consumer_quorum_queue(Config) -> + single_active_consumer(<<"quorum">>, Config). single_active_consumer(QType, Config) -> QName = atom_to_binary(?FUNCTION_NAME), @@ -2085,9 +2091,8 @@ single_active_consumer(QType, Config) -> detach_requeues_one_session_classic_queue(Config) -> detach_requeue_one_session(<<"classic">>, Config). -detach_requeues_one_session_quorum_queue(_Config) -> - % detach_requeue_one_session(<<"quorum">>, Config). - {skip, "TODO: unskip when qq-v4 branch is merged"}. +detach_requeues_one_session_quorum_queue(Config) -> + detach_requeue_one_session(<<"quorum">>, Config). detach_requeue_one_session(QType, Config) -> QName = atom_to_binary(?FUNCTION_NAME), @@ -2236,9 +2241,8 @@ detach_requeues_drop_head_classic_queue(Config) -> detach_requeues_two_connections_classic_queue(Config) -> detach_requeues_two_connections(<<"classic">>, Config). -detach_requeues_two_connections_quorum_queue(_Config) -> - % detach_requeues_two_connections(<<"quorum">>, Config). - {skip, "TODO: unskip when qq-v4 branch is merged"}. +detach_requeues_two_connections_quorum_queue(Config) -> + detach_requeues_two_connections(<<"quorum">>, Config). detach_requeues_two_connections(QType, Config) -> QName = atom_to_binary(?FUNCTION_NAME), @@ -2261,16 +2265,18 @@ detach_requeues_two_connections(QType, Config) -> %% Attach 1 sender and 2 receivers. {ok, Sender} = amqp10_client:attach_sender_link(Session0, <<"sender">>, Address, settled), ok = wait_for_credit(Sender), + {ok, Receiver0} = amqp10_client:attach_receiver_link(Session0, <<"receiver 0">>, Address, unsettled), receive {amqp10_event, {link, Receiver0, attached}} -> ok after 5000 -> ct:fail({missing_event, ?LINE}) end, + ok = gen_statem:cast(Session0, {flow_session, #'v1_0.flow'{incoming_window = {uint, 1}}}), + ok = amqp10_client:flow_link_credit(Receiver0, 50, never), + {ok, Receiver1} = amqp10_client:attach_receiver_link(Session1, <<"receiver 1">>, Address, unsettled), receive {amqp10_event, {link, Receiver1, attached}} -> ok after 5000 -> ct:fail({missing_event, ?LINE}) end, - ok = gen_statem:cast(Session0, {flow_session, #'v1_0.flow'{incoming_window = {uint, 1}}}), - ok = amqp10_client:flow_link_credit(Receiver0, 50, never), ok = amqp10_client:flow_link_credit(Receiver1, 50, never), flush(attached), diff --git a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl index eba5a77a70d4..afce3d40dd42 100644 --- a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl @@ -473,7 +473,7 @@ cancel_checkout(Config) -> #{}, F1), {_, _, F3} = process_ra_events(receive_ra_events(1, 1), ClusterName, F2, [], [], fun (_, S) -> S end), - {ok, F4} = rabbit_fifo_client:cancel_checkout(<<"tag">>, F3), + {ok, F4} = rabbit_fifo_client:cancel_checkout(<<"tag">>, cancel, F3), {F5, _} = rabbit_fifo_client:return(<<"tag">>, [0], F4), {ok, _, {_, _, _, _, m1}, F5} = rabbit_fifo_client:dequeue(ClusterName, <<"d1">>, settled, F5), From b0dde404b2b486fb13f92b8063687923084be74e Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 4 Jul 2024 15:21:03 +0200 Subject: [PATCH 17/45] Use AMQP address v2 in fsharp-tests --- deps/rabbit/test/amqp_system_SUITE.erl | 1 - .../fsharp-tests/Program.fs | 74 +++++++++---------- 2 files changed, 34 insertions(+), 41 deletions(-) diff --git a/deps/rabbit/test/amqp_system_SUITE.erl b/deps/rabbit/test/amqp_system_SUITE.erl index f275ecac98dc..d94a590be397 100644 --- a/deps/rabbit/test/amqp_system_SUITE.erl +++ b/deps/rabbit/test/amqp_system_SUITE.erl @@ -86,7 +86,6 @@ init_per_testcase(Testcase, Config) -> message_containers_store_amqp_v1, credit_api_v2, quorum_queues_v4 - % amqp_address_v1 ]), rabbit_ct_helpers:testcase_started(Config, Testcase). diff --git a/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs b/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs index 505e1390767e..287b933239ae 100755 --- a/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs +++ b/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs @@ -136,7 +136,7 @@ module Test = // tests that a message sent to an exchange that resolves no routes for the // binding key returns the Released outcome, rather than Accepted use ac = connect uri - let address = "/exchange/no_routes_is_released" + let address = "/exchanges/no_routes_is_released" let sender = SenderLink(ac.Session, "released-sender", address) let trySet (mre: AutoResetEvent) = try mre.Set() |> ignore with _ -> () @@ -161,7 +161,7 @@ module Test = let roundtrip uri = use c = connect uri - let sender, receiver = senderReceiver c "test" "/queue/roundtrip" + let sender, receiver = senderReceiver c "test" "/queues/roundtrip" for body in sampleTypes do let corr = "correlation" new Message(body, @@ -177,7 +177,7 @@ module Test = let streams uri = use c = connect uri let name = "streams-test" - let address = "/queue/streams" + let address = "/queues/streams" let sender = SenderLink(c.Session, name + "-sender" , address) //for body in sampleTypes do let body = "hi"B :> obj @@ -217,8 +217,8 @@ module Test = let roundtrip_to_amqp_091 uri = use c = connect uri - let q = "roundtrip_to_amqp_091 " - let target = "/queue/roundtrip_to_amqp_091 " + let q = "roundtrip_to_amqp_091" + let target = "/queues/roundtrip_to_amqp_091" let corr = "correlation" let sender = SenderLink(c.Session, q + "-sender" , target) @@ -249,7 +249,7 @@ module Test = "amqp:rejected:list", null, Rejected() :> Outcome "amqp:released:list", null, Released() :> Outcome] do - let source = new Source(Address = "/queue/default_outcome", + let source = new Source(Address = "/queues/default_outcome", DefaultOutcome = defObj) let attach = new Attach (Source = source, Target = Target()) @@ -264,7 +264,7 @@ module Test = "amqp:modified:list", null "amqp:madeup:list", "amqp:not-implemented"] do - let source = new Source(Address = "/queue/outcomes", + let source = new Source(Address = "/queues/outcomes", Outcomes = [| Symbol outcome |]) let attach = new Attach (Source = source, Target = Target()) @@ -283,7 +283,7 @@ module Test = HostName = addr.Host, ChannelMax = 256us, MaxFrameSize = frameSize) use c = connectWithOpen uri opn - let sender, receiver = senderReceiver c "test" "/queue/framentation" + let sender, receiver = senderReceiver c "test" "/queues/fragmentation" let m = new Message(String.replicate size "a") sender.Send m let m' = receive receiver @@ -291,7 +291,7 @@ module Test = let message_annotations uri = use c = connect uri - let sender, receiver = senderReceiver c "test" "/queue/message_annotations" + let sender, receiver = senderReceiver c "test" "/queues/message_annotations" let ann = MessageAnnotations() let k1 = Symbol "key1" let k2 = Symbol "key2" @@ -310,7 +310,7 @@ module Test = let footer uri = use c = connect uri - let sender, receiver = senderReceiver c "test" "/queue/footer" + let sender, receiver = senderReceiver c "test" "/queues/footer" let footer = Footer() let k1 = Symbol "key1" let k2 = Symbol "key2" @@ -328,7 +328,7 @@ module Test = let data_types uri = use c = connect uri - let sender, receiver = senderReceiver c "test" "/queue/data_types" + let sender, receiver = senderReceiver c "test" "/queues/data_types" let aSeq = amqpSequence sampleTypes (new Message(aSeq)) |> sender.Send let rtd = receive receiver @@ -338,7 +338,7 @@ module Test = let reject uri = use c = connect uri - let sender, receiver = senderReceiver c "test" "/queue/reject" + let sender, receiver = senderReceiver c "test" "/queues/reject" new Message "testing reject" |> sender.Send let m = receiver.Receive() receiver.Reject(m) @@ -346,14 +346,14 @@ module Test = let redelivery uri = use c = connect uri - let sender, receiver = senderReceiver c "test" "/queue/redelivery" + let sender, receiver = senderReceiver c "test" "/queues/redelivery" new Message "testing redelivery" |> sender.Send let m = receiver.Receive() assertTrue (m.Header.FirstAcquirer) c.Session.Close() let session = Session(c.Conn) - let receiver = ReceiverLink(session, "test-receiver", "/queue/redelivery") + let receiver = ReceiverLink(session, "test-receiver", "/queues/redelivery") let m' = receive receiver assertEqual (m.Body :?> string) (m'.Body :?> string) @@ -364,7 +364,7 @@ module Test = let released uri = use c = connect uri - let sender, receiver = senderReceiver c "test" "/queue/released" + let sender, receiver = senderReceiver c "test" "/queues/released" new Message "testing released" |> sender.Send let m = receiver.Receive() assertTrue (m.Header.FirstAcquirer) @@ -379,17 +379,17 @@ module Test = c.Session.Close() let routing uri = - for target, source, toProp, succeed in + for target, source, toProp in [ - "/queue/test", "/queue/test", "", true - "/exchange/amq.fanout", "/queue/fanout_q", "", true - "/exchange/amq.direct/key/direct_q", "/queue/direct_q", "", true - null, "/queue/direct_q", "/exchange/amq.direct/key/direct_q", true - "/queue/transient_q", "/queue/transient_q", "", true - "/queue/durable_q", "/queue/durable_q", "", true - "/queue/quorum_q", "/queue/quorum_q", "", true - "/queue/stream_q", "/queue/stream_q", "", true - "/queue/autodel_q", "/queue/autodel_q", "", true] do + "/queues/test", "/queues/test", "" + "/exchanges/amq.fanout", "/queues/fanout_q", "" + "/exchanges/amq.direct/direct_q", "/queues/direct_q", "" + null, "/queues/direct_q", "/exchanges/amq.direct/direct_q" + "/queues/transient_q", "/queues/transient_q", "" + "/queues/durable_q", "/queues/durable_q", "" + "/queues/quorum_q", "/queues/quorum_q", "" + "/queues/stream_q", "/queues/stream_q", "" + "/queues/autodel_q", "/queues/autodel_q", ""] do let rnd = Random() use c = connect uri @@ -399,23 +399,17 @@ module Test = use m = new Message(rnd.Next(10000), Properties = Properties(To = toProp)) sender.Send m - (* printfn "%s %s %s %A" target source routingKey succeed *) - - if succeed then - let m' = receiver.Receive(TimeSpan.FromMilliseconds 3000.) - receiver.Accept m' - assertTrue (m' <> null) - assertEqual (m.Body :?> int) (m'.Body :?> int) - else - use m' = receiver.Receive(TimeSpan.FromMilliseconds 100.) - assertEqual null m' - + (* printfn "%s %s %s %A" target source routingKey *) + let m' = receiver.Receive(TimeSpan.FromMilliseconds 3000.) + receiver.Accept m' + assertTrue (m' <> null) + assertEqual (m.Body :?> int) (m'.Body :?> int) let invalidRoutes uri = for dest, cond in - ["/exchange/missing", "amqp:not-found" + ["/exchanges/missing", "amqp:not-found" "/fruit/orange", "amqp:invalid-field"] do use ac = connect uri let trySet (mre: AutoResetEvent) = @@ -455,7 +449,7 @@ module Test = let u = Uri uri let uri = sprintf "amqp://access_failure:boo@%s:%i" u.Host u.Port use ac = connect uri - let dest = "/amq/queue/test" + let dest = "/queues/test" ac.Session.add_Closed ( new ClosedCallback (fun _ err -> printfn "session err %A" err.Condition )) @@ -472,7 +466,7 @@ module Test = let u = Uri uri let uri = sprintf "amqp://access_failure:boo@%s:%i" u.Host u.Port use ac = connect uri - let dest = "/amq/queue/test" + let dest = "/queues/test" let receiver = ReceiverLink(ac.Session, "test-receiver", dest) receiver.Close() failwith "expected exception not received" @@ -486,7 +480,7 @@ module Test = let u = Uri uri let uri = sprintf "amqp://access_failure_not_allowed:boo@%s:%i" u.Host u.Port use ac = connect uri - let dest = "/amq/queue/test" + let dest = "/queues/test" let receiver = ReceiverLink(ac.Session, "test-receiver", dest) receiver.Close() failwith "expected exception not received" From e608c88aaa475725053112714627e4ae45bd0be1 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 2 Jul 2024 12:45:57 +0100 Subject: [PATCH 18/45] QQ: track number of requeues for message. To be able to calculate the correct value for the AMQP delivery_count header we need to be able to distinguish between messages that were "released" or returned in QQ speak and those that were returned due to errors such as channel termination. This commit implement such tracking as well as the calculation of a new mc annotations `delivery_count` that AMQP makes use of to set the header value accordingly. --- deps/rabbit/test/rabbit_fifo_int_SUITE.erl | 20 +++++++------------- 1 file changed, 7 insertions(+), 13 deletions(-) diff --git a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl index afce3d40dd42..1f1552da5477 100644 --- a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl @@ -350,19 +350,13 @@ returns_after_down(Config) -> {_, _, F2} = process_ra_events(receive_ra_events(1, 0), ClusterName, F1), % start a consumer in a separate processes % that exits after checkout - Self = self(), - Pid = spawn(fun () -> - F = rabbit_fifo_client:init([ServerId]), - {ok, _, _} = rabbit_fifo_client:checkout(<<"tag">>, - {simple_prefetch, 10}, - #{}, F), - Self ! checkout_done - end), - receive checkout_done -> ok - after 1000 -> - exit(checkout_done_timeout) - end, - MonRef = erlang:monitor(process, Pid), + {_, MonRef} = spawn_monitor( + fun () -> + F = rabbit_fifo_client:init([ServerId]), + {ok, _, _} = rabbit_fifo_client:checkout(<<"tag">>, + {simple_prefetch, 10}, + #{}, F) + end), receive {'DOWN', MonRef, _, _, _} -> ok From 5cf19169a93c02f500b6987edbecf7cf2a8fb757 Mon Sep 17 00:00:00 2001 From: Michael Davis Date: Mon, 29 Jan 2024 10:42:46 -0500 Subject: [PATCH 19/45] rabbit_fifo: Use Ra checkpoints --- deps/rabbit/src/rabbit_fifo.erl | 86 +++++++------------------- deps/rabbit/test/rabbit_fifo_SUITE.erl | 4 +- 2 files changed, 26 insertions(+), 64 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index c5db56a6e504..467ec29b0e86 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -71,7 +71,6 @@ is_v4/0, %% misc - dehydrate_state/1, normalize/1, get_msg_header/1, get_header/2, @@ -321,10 +320,9 @@ apply(#{index := Idx} = Meta, Messages), enqueue_count = EnqCount + 1}, State2 = update_or_remove_con(Meta, ConsumerKey, Con, State1), - {State, Ret, Effs} = checkout(Meta, State0, State2, []), - update_smallest_raft_index(Idx, Ret, - maybe_store_release_cursor(Idx, State), - Effs); + {State3, Ret, Effs0} = checkout(Meta, State0, State2, []), + {State, Effs} = maybe_checkpoint(Idx, State3, Effs0), + update_smallest_raft_index(Idx, Ret, State, Effs); _ -> {State00, ok, []} end; @@ -1606,8 +1604,9 @@ apply_enqueue(#{index := RaftIdx, Seq, RawMsg, Size, State0) -> case maybe_enqueue(RaftIdx, Ts, From, Seq, RawMsg, Size, [], State0) of {ok, State1, Effects1} -> - {State, ok, Effects} = checkout(Meta, State0, State1, Effects1), - {maybe_store_release_cursor(RaftIdx, State), ok, Effects}; + {State2, ok, Effects2} = checkout(Meta, State0, State1, Effects1), + {State, Effects} = maybe_checkpoint(RaftIdx, State2, Effects2), + {State, ok, Effects}; {out_of_sequence, State, Effects} -> {State, not_enqueued, Effects}; {duplicate, State, Effects} -> @@ -1656,16 +1655,15 @@ update_expiry_header(RaCmdTs, TTL, Header) -> update_expiry_header(ExpiryTs, Header) -> update_header(expiry, fun(Ts) -> Ts end, ExpiryTs, Header). -maybe_store_release_cursor(RaftIdx, - #?STATE{cfg = #cfg{release_cursor_interval = - {Base, C}} = Cfg, - enqueue_count = EC, - release_cursors = Cursors0} = State0) +maybe_checkpoint(RaftIdx, + #?STATE{cfg = #cfg{release_cursor_interval = {Base, C}} = Cfg, + enqueue_count = EC} = State0, + Effects0) when EC >= C -> case messages_total(State0) of 0 -> %% message must have been immediately dropped - State0#?STATE{enqueue_count = 0}; + {State0#?STATE{enqueue_count = 0}, Effects0}; Total -> Interval = case Base of 0 -> 0; @@ -1674,14 +1672,11 @@ maybe_store_release_cursor(RaftIdx, end, State = State0#?STATE{cfg = Cfg#cfg{release_cursor_interval = {Base, Interval}}}, - Dehydrated = dehydrate_state(State), - Cursor = {release_cursor, RaftIdx, Dehydrated}, - Cursors = lqueue:in(Cursor, Cursors0), - State#?STATE{enqueue_count = 0, - release_cursors = Cursors} + Effects = Effects0 ++ [{checkpoint, RaftIdx, State}], + {State#?STATE{enqueue_count = 0}, Effects} end; -maybe_store_release_cursor(_RaftIdx, State) -> - State. +maybe_checkpoint(_RaftIdx, State, Effects) -> + {State, Effects}. maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, {_MetaSize, BodySize}, @@ -1853,10 +1848,8 @@ update_smallest_raft_index(Idx, State, Effects) -> update_smallest_raft_index(Idx, ok, State, Effects). update_smallest_raft_index(IncomingRaftIdx, Reply, - #?STATE{cfg = Cfg, - release_cursors = Cursors0} = State0, - Effects) -> - % Total = messages_total(State0), + #?STATE{cfg = Cfg} = State0, + Effects0) -> %% TODO: optimise case smallest_raft_index(State0) of undefined -> @@ -1867,35 +1860,15 @@ update_smallest_raft_index(IncomingRaftIdx, Reply, #cfg{release_cursor_interval = {Base, _}} = Cfg, RCI = {Base, Base}, State = State0#?STATE{cfg = Cfg#cfg{release_cursor_interval = RCI}, - release_cursors = lqueue:new(), enqueue_count = 0}, - {State, Reply, Effects ++ [{release_cursor, IncomingRaftIdx, State}]}; - % undefined -> - % {State0, Reply, Effects}; + Effects = Effects0 ++ [{release_cursor, IncomingRaftIdx, State}], + {State, Reply, Effects}; Smallest when is_integer(Smallest) -> - case find_next_cursor(Smallest, Cursors0) of - empty -> - {State0, Reply, Effects}; - {Cursor, Cursors} -> - %% we can emit a release cursor when we've passed the smallest - %% release cursor available. - {State0#?STATE{release_cursors = Cursors}, Reply, - Effects ++ [Cursor]} - end - end. - -find_next_cursor(Idx, Cursors) -> - find_next_cursor(Idx, Cursors, empty). - -find_next_cursor(Smallest, Cursors0, Potential) -> - case lqueue:out(Cursors0) of - {{value, {_, Idx, _} = Cursor}, Cursors} when Idx < Smallest -> - %% we found one but it may not be the largest one - find_next_cursor(Smallest, Cursors, Cursor); - _ when Potential == empty -> - empty; - _ -> - {Potential, Cursors0} + %% Promote a checkpoint smaller than `Smallest'. `Smallest' is the + %% oldest message that must remain in the log, so we can only + %% promote an existing checkpoint smaller than `Smallest'. + Effects = Effects0 ++ [{release_cursor, Smallest - 1}], + {State0, Reply, Effects} end. update_msg_header(Key, Fun, Def, ?MSG(Idx, Header)) -> @@ -2425,17 +2398,6 @@ included_credit({credited, _}) -> 0; included_credit(credited) -> 0. -%% creates a dehydrated version of the current state to be cached and -%% potentially used to for a snaphot at a later point -dehydrate_state(#?STATE{cfg = #cfg{}, - dlx = DlxState} = State) -> - % no messages are kept in memory, no need to - % overly mutate the current state apart from removing indexes and cursors - State#?STATE{ra_indexes = rabbit_fifo_index:empty(), - release_cursors = lqueue:new(), - enqueue_count = 0, - msg_cache = undefined, - dlx = rabbit_fifo_dlx:dehydrate(DlxState)}. %% make the state suitable for equality comparison normalize(#?STATE{ra_indexes = _Indexes, diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 3e7b12f0fb3e..c5c3d8c520cc 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -441,7 +441,7 @@ checkout_enq_settle_test(Config) -> {_, Effects} = settle(Config, CKey, 4, NextMsgId, State3), % the release cursor is the smallest raft index that does not % contribute to the state of the application - ?ASSERT_EFF({release_cursor, 2, _}, Effects), + ?ASSERT_EFF({release_cursor, 2}, Effects), ok. duplicate_enqueue_test(Config) -> @@ -2546,7 +2546,7 @@ expire_message_should_emit_release_cursor_test(Config) -> {_S, ok, Effs} = apply(meta(Config, ?LINE, 101, {notify, 2, self()}), rabbit_fifo:make_enqueue(self(), 2, Msg), S1), - ?ASSERT_EFF({release_cursor, _, _}, Effs), + ?ASSERT_EFF({release_cursor, _}, Effs), ok. header_test(_) -> From cb1deeac1d7f83edb2a998a55bd2787206b7a7e0 Mon Sep 17 00:00:00 2001 From: Michael Davis Date: Mon, 29 Jan 2024 10:42:50 -0500 Subject: [PATCH 20/45] quorum queues: Use a custom interval for checkpoints --- deps/rabbit/src/rabbit_quorum_queue.erl | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index e0a8d169aef7..b86cc36ac3f0 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -139,6 +139,7 @@ -define(MEMBER_CHANGE_TIMEOUT, 20_000). -define(SNAPSHOT_INTERVAL, 8192). %% the ra default is 4096 % -define(UNLIMITED_PREFETCH_COUNT, 2000). %% something large for ra +-define(MIN_CHECKPOINT_INTERVAL, 8192). %% the ra default is 16384 %%----------- QQ policies --------------------------------------------------- @@ -1807,9 +1808,14 @@ make_ra_conf(Q, ServerId, Membership) -> ?TICK_INTERVAL), SnapshotInterval = application:get_env(rabbit, quorum_snapshot_interval, ?SNAPSHOT_INTERVAL), - make_ra_conf(Q, ServerId, TickTimeout, SnapshotInterval, Membership). - -make_ra_conf(Q, ServerId, TickTimeout, SnapshotInterval, Membership) -> + CheckpointInterval = application:get_env(rabbit, + quorum_min_checkpoint_interval, + ?MIN_CHECKPOINT_INTERVAL), + make_ra_conf(Q, ServerId, TickTimeout, + SnapshotInterval, CheckpointInterval, Membership). + +make_ra_conf(Q, ServerId, TickTimeout, + SnapshotInterval, CheckpointInterval, Membership) -> QName = amqqueue:get_name(Q), RaMachine = ra_machine(Q), [{ClusterName, _} | _] = Members = members(Q), @@ -1824,7 +1830,9 @@ make_ra_conf(Q, ServerId, TickTimeout, SnapshotInterval, Membership) -> metrics_key => QName, initial_members => Members, log_init_args => #{uid => UId, - snapshot_interval => SnapshotInterval}, + snapshot_interval => SnapshotInterval, + min_checkpoint_interval => + CheckpointInterval}, tick_timeout => TickTimeout, machine => RaMachine, ra_event_formatter => Formatter}). From 24482f59b2b7450c09562907e9a16fd71f008254 Mon Sep 17 00:00:00 2001 From: Michael Davis Date: Wed, 1 May 2024 15:07:53 -0400 Subject: [PATCH 21/45] rabbit_fifo_SUITE: List actual effects in ?ASSERT_EFF failure --- deps/rabbit/test/rabbit_fifo_SUITE.erl | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index c5c3d8c520cc..761b3a038fbf 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -68,7 +68,10 @@ end_per_testcase(_Group, _Config) -> -define(ASSERT_EFF(EfxPat, Guard, Effects), ?assert(lists:any(fun (EfxPat) when Guard -> true; (_) -> false - end, Effects))). + end, Effects), + lists:flatten(io_lib:format("Expected to find effect matching " + "pattern '~s' in effect list '~0p'", + [??EfxPat, Effects])))). -define(ASSERT_NO_EFF(EfxPat, Effects), ?assert(not lists:any(fun (EfxPat) -> true; From bf133221686765be5f261ded681f9272a061c517 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 9 Jul 2024 12:36:21 +0100 Subject: [PATCH 22/45] QQ: Checkpoints modifications --- deps/rabbit/src/rabbit_fifo.erl | 265 ++++++++++---------- deps/rabbit/src/rabbit_fifo.hrl | 10 +- deps/rabbit/src/rabbit_quorum_queue.erl | 9 +- deps/rabbit/test/rabbit_fifo_SUITE.erl | 53 +--- deps/rabbit/test/rabbit_fifo_prop_SUITE.erl | 39 +-- 5 files changed, 148 insertions(+), 228 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 467ec29b0e86..0b4ad644e3bc 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -272,7 +272,7 @@ apply(Meta, #discard{consumer_key = ConsumerKey, case maps:get(Id, Checked, undefined) of undefined -> false; - ?C_MSG(_At, Msg) -> + Msg -> {true, Msg} end end, MsgIds), @@ -320,9 +320,7 @@ apply(#{index := Idx} = Meta, Messages), enqueue_count = EnqCount + 1}, State2 = update_or_remove_con(Meta, ConsumerKey, Con, State1), - {State3, Ret, Effs0} = checkout(Meta, State0, State2, []), - {State, Effs} = maybe_checkpoint(Idx, State3, Effs0), - update_smallest_raft_index(Idx, Ret, State, Effs); + checkout(Meta, State0, State2, []); _ -> {State00, ok, []} end; @@ -452,7 +450,7 @@ apply(#{index := Index, Exists = find_consumer(ConsumerId, Consumers) /= undefined, case messages_ready(State0) of 0 -> - update_smallest_raft_index(Index, {dequeue, empty}, State0, []); + {State0, {dequeue, empty}, []}; _ when Exists -> %% a dequeue using the same consumer_id isn't possible at this point {State0, {dequeue, empty}}; @@ -462,7 +460,7 @@ apply(#{index := Index, State0), case checkout_one(Meta, false, State1, []) of {success, _, MsgId, - ?MSG(RaftIdx, Header), ExpiredMsg, State2, Effects0} -> + ?MSG(RaftIdx, Header), _ExpiredMsg, State2, Effects0} -> {State4, Effects1} = case Settlement of unsettled -> @@ -478,15 +476,9 @@ apply(#{index := Index, Effects2 = [reply_log_effect(RaftIdx, MsgId, Header, messages_ready(State4), From) | Effects1], - {State, DroppedMsg, Effects} = + {State, _DroppedMsg, Effects} = evaluate_limit(Index, false, State0, State4, Effects2), - Reply = '$ra_no_reply', - case {DroppedMsg, ExpiredMsg} of - {false, false} -> - {State, Reply, Effects}; - _ -> - update_smallest_raft_index(Index, Reply, State, Effects) - end; + {State, '$ra_no_reply', Effects}; {nochange, _ExpiredMsg = true, State2, Effects0} -> %% All ready messages expired. State3 = State2#?STATE{consumers = @@ -494,11 +486,10 @@ apply(#{index := Index, State2#?STATE.consumers)}, {State, _, Effects} = evaluate_limit(Index, false, State0, State3, Effects0), - update_smallest_raft_index(Index, {dequeue, empty}, - State, Effects) + {State, {dequeue, empty}, Effects} end end; -apply(#{index := Idx} = Meta, +apply(#{index := _Idx} = Meta, #checkout{spec = Spec, consumer_id = ConsumerId}, State0) when Spec == cancel orelse @@ -510,10 +501,9 @@ apply(#{index := Idx} = Meta, Spec)), Reply = {ok, consumer_cancel_info(ConsumerKey, State1)}, {State, _, Effects} = checkout(Meta, State0, State1, Effects1), - update_smallest_raft_index(Idx, Reply, State, Effects); + {State, Reply, Effects}; error -> - update_smallest_raft_index(Idx, {error, consumer_not_found}, - State0, []) + {State0, {error, consumer_not_found}, []} end; apply(#{index := Idx} = Meta, #checkout{spec = Spec0, @@ -592,9 +582,9 @@ apply(#{index := Index}, #purge{}, Reply = {purge, NumReady}, {State, _, Effects} = evaluate_limit(Index, false, State0, State1, Effects0), - update_smallest_raft_index(Index, Reply, State, Effects); -apply(#{index := Idx}, #garbage_collection{}, State) -> - update_smallest_raft_index(Idx, ok, State, [{aux, garbage_collection}]); + {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(#{system_time := Ts} = Meta, @@ -683,11 +673,10 @@ apply(#{system_time := Ts} = Meta, Effects = [{monitor, node, Node} | Effects1], checkout(Meta, State0, State#?STATE{enqueuers = Enqs, last_active = Ts}, Effects); -apply(#{index := Idx} = Meta, {down, Pid, _Info}, State0) -> +apply(#{index := _Idx} = Meta, {down, Pid, _Info}, State0) -> {State1, Effects1} = activate_next_consumer( handle_down(Meta, Pid, State0)), - {State, Reply, Effects} = checkout(Meta, State0, State1, Effects1), - update_smallest_raft_index(Idx, Reply, State, Effects); + checkout(Meta, State0, State1, Effects1); apply(Meta, {nodeup, Node}, #?STATE{consumers = Cons0, enqueuers = Enqs0, service_queue = _SQ0} = State0) -> @@ -723,12 +712,12 @@ apply(Meta, {nodeup, Node}, #?STATE{consumers = Cons0, checkout(Meta, State0, State, Effects); apply(_, {nodedown, _Node}, State) -> {State, ok}; -apply(#{index := Idx} = Meta, #purge_nodes{nodes = Nodes}, State0) -> +apply(#{index := _Idx} = Meta, #purge_nodes{nodes = Nodes}, State0) -> {State, Effects} = lists:foldl(fun(Node, {S, E}) -> purge_node(Meta, Node, S, E) end, {State0, []}, Nodes), - update_smallest_raft_index(Idx, ok, State, Effects); -apply(#{index := Idx} = Meta, + {State, ok, Effects}; +apply(#{index := _Idx} = Meta, #update_config{config = #{dead_letter_handler := NewDLH} = Conf}, #?STATE{cfg = #cfg{dead_letter_handler = OldDLH, resource = QRes}, @@ -736,36 +725,26 @@ apply(#{index := Idx} = Meta, {DlxState, Effects0} = rabbit_fifo_dlx:update_config(OldDLH, NewDLH, QRes, DlxState0), State1 = update_config(Conf, State0#?STATE{dlx = DlxState}), - {State, Reply, Effects} = checkout(Meta, State0, State1, Effects0), - update_smallest_raft_index(Idx, Reply, State, Effects); + checkout(Meta, State0, State1, Effects0); apply(Meta, {machine_version, FromVersion, ToVersion}, V0State) -> State = convert(Meta, FromVersion, ToVersion, V0State), {State, ok, [{aux, {dlx, setup}}]}; -apply(#{index := IncomingRaftIdx} = Meta, {dlx, _} = Cmd, +apply(#{index := _IncomingRaftIdx} = 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}, - {State, ok, Effects} = checkout(Meta, State0, State1, Effects0), - update_smallest_raft_index(IncomingRaftIdx, State, Effects); + checkout(Meta, State0, State1, Effects0); apply(_Meta, Cmd, State) -> %% handle unhandled commands gracefully rabbit_log:debug("rabbit_fifo: unhandled command ~W", [Cmd, 10]), {State, ok, []}. -convert_v3_to_v4(#{system_time := Ts}, +convert_v3_to_v4(#{system_time := _Ts}, StateV3) -> Messages0 = rabbit_fifo_v3:get_field(messages, StateV3), - Consumers0 = rabbit_fifo_v3:get_field(consumers, StateV3), + Consumers = rabbit_fifo_v3:get_field(consumers, StateV3), Messages = rabbit_fifo_q:from_lqueue(Messages0), - Consumers = maps:map( - fun (_CKey, #consumer{checked_out = Ch0} = C) -> - Ch = maps:map( - fun (_MsgId, ?MSG(_, _) = Msg) -> - ?C_MSG(Ts, Msg) - end, Ch0), - C#consumer{checked_out = Ch} - end, Consumers0), #?MODULE{cfg = rabbit_fifo_v3:get_field(cfg, StateV3), messages = Messages, messages_total = rabbit_fifo_v3:get_field(messages_total, StateV3), @@ -978,7 +957,7 @@ get_checked_out(CKey, From, To, #?STATE{consumers = Consumers}) -> case find_consumer(CKey, Consumers) of {_CKey, #consumer{checked_out = Checked}} -> [begin - ?C_MSG(_At, I, H) = maps:get(K, Checked), + ?MSG(I, H) = maps:get(K, Checked), {K, {I, H}} end || K <- lists:seq(From, To), maps:is_key(K, Checked)]; _ -> @@ -992,10 +971,15 @@ 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) -> ?STATE. +which_module(4) -> ?MODULE. -define(AUX, aux_v2). +-record(checkpoint, {index :: ra:index(), + timestamp :: milliseconds(), + enqueue_count :: non_neg_integer(), + last_smallest_index :: ra:index(), + messages_total :: non_neg_integer()}). -record(aux_gc, {last_raft_idx = 0 :: ra:index()}). -record(aux, {name :: atom(), capacity :: term(), @@ -1005,7 +989,9 @@ which_module(4) -> ?STATE. capacity :: term(), gc = #aux_gc{} :: #aux_gc{}, tick_pid :: undefined | pid(), - cache = #{} :: map()}). + cache = #{} :: map(), + %% TODO: we need a state conversion for this + last_checkpoint :: #checkpoint{}}). init_aux(Name) when is_atom(Name) -> %% TODO: catch specific exception throw if table already exists @@ -1014,15 +1000,18 @@ init_aux(Name) when is_atom(Name) -> {write_concurrency, true}]), Now = erlang:monotonic_time(micro_seconds), #?AUX{name = Name, - capacity = {inactive, Now, 1, 1.0}}. + capacity = {inactive, Now, 1, 1.0}, + last_checkpoint = #checkpoint{index = 0, + timestamp = erlang:system_time(millisecond), + enqueue_count = 0}}. handle_aux(RaftState, Tag, Cmd, #aux{name = Name, capacity = Cap, gc = Gc}, RaAux) -> %% convert aux state to new version - Aux = #?AUX{name = Name, - capacity = Cap, - gc = Gc}, + AuxV2 = init_aux(Name), + Aux = AuxV2#?AUX{capacity = Cap, + gc = Gc}, handle_aux(RaftState, Tag, Cmd, Aux, RaAux); handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, consumer_key = Key} = Ret, Corr, Pid}, @@ -1034,7 +1023,7 @@ handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, {ConsumerKey, #consumer{checked_out = Checked}} -> {RaAux, ToReturn} = maps:fold( - fun (MsgId, ?C_MSG(_, Idx, Header), {RA0, Acc}) -> + 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 @@ -1069,8 +1058,15 @@ handle_aux(leader, _, {handle_tick, [QName, Overview0, Nodes]}, %% Active TICK pid, do nothing Pid end, + + Effs = case smallest_raft_index(ra_aux:machine_state(RaAux)) of + undefined -> + [{release_cursor, ra_aux:last_applied(RaAux)}]; + Smallest -> + [{release_cursor, Smallest}] + end, %% TODO: check consumer timeouts - {no_reply, Aux#?AUX{tick_pid = NewPid}, RaAux}; + {no_reply, Aux#?AUX{tick_pid = NewPid}, RaAux, Effs}; handle_aux(_, _, {get_checked_out, ConsumerKey, MsgIds}, Aux0, RaAux0) -> #?STATE{cfg = #cfg{}, consumers = Consumers} = ra_aux:machine_state(RaAux0), @@ -1078,7 +1074,7 @@ handle_aux(_, _, {get_checked_out, ConsumerKey, MsgIds}, Aux0, RaAux0) -> #{ConsumerKey := #consumer{checked_out = Checked}} -> {RaState, IdMsgs} = maps:fold( - fun (MsgId, ?C_MSG(_, Idx, Header), {S0, Acc}) -> + 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 @@ -1093,25 +1089,35 @@ handle_aux(_, _, {get_checked_out, ConsumerKey, MsgIds}, Aux0, RaAux0) -> _ -> {reply, {error, consumer_not_found}, Aux0, RaAux0} end; -handle_aux(leader, cast, eval, #?AUX{last_decorators_state = LastDec} = Aux0, +handle_aux(leader, cast, eval, + #?AUX{last_decorators_state = LastDec, + 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), + %% 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 - Ts = erlang:system_time(millisecond), - Effects0 = timer_effect(Ts, MacState, []), + Effects1 = timer_effect(Ts, MacState, Effects0), case query_notify_decorators_info(MacState) of LastDec -> - {no_reply, Aux0, RaAux, Effects0}; + {no_reply, Aux0#?AUX{last_checkpoint = Check}, RaAux, Effects1}; {MaxActivePriority, IsEmpty} = NewLast -> Effects = [notify_decorators_effect(QName, MaxActivePriority, IsEmpty) - | Effects0], - {no_reply, Aux0#?AUX{last_decorators_state = NewLast}, RaAux, Effects} + | Effects1], + {no_reply, Aux0#?AUX{last_checkpoint = Check, + last_decorators_state = NewLast}, RaAux, Effects} end; -handle_aux(_RaftState, cast, eval, Aux0, RaAux) -> - {no_reply, Aux0, RaAux}; +handle_aux(_RaftState, cast, eval, + #?AUX{last_checkpoint = Check0} = Aux0, + RaAux) -> + Ts = erlang:system_time(millisecond), + {Check, Effects} = do_checkpoints(Ts, Check0, RaAux), + {no_reply, Aux0#?AUX{last_checkpoint = Check}, RaAux, Effects}; 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}; @@ -1189,8 +1195,7 @@ eval_gc(RaAux, MacState, AuxState end. -force_eval_gc(%Log, - RaAux, +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), @@ -1604,9 +1609,7 @@ apply_enqueue(#{index := RaftIdx, Seq, RawMsg, Size, State0) -> case maybe_enqueue(RaftIdx, Ts, From, Seq, RawMsg, Size, [], State0) of {ok, State1, Effects1} -> - {State2, ok, Effects2} = checkout(Meta, State0, State1, Effects1), - {State, Effects} = maybe_checkpoint(RaftIdx, State2, Effects2), - {State, ok, Effects}; + checkout(Meta, State0, State1, Effects1); {out_of_sequence, State, Effects} -> {State, not_enqueued, Effects}; {duplicate, State, Effects} -> @@ -1655,29 +1658,6 @@ update_expiry_header(RaCmdTs, TTL, Header) -> update_expiry_header(ExpiryTs, Header) -> update_header(expiry, fun(Ts) -> Ts end, ExpiryTs, Header). -maybe_checkpoint(RaftIdx, - #?STATE{cfg = #cfg{release_cursor_interval = {Base, C}} = Cfg, - enqueue_count = EC} = State0, - Effects0) - when EC >= C -> - case messages_total(State0) of - 0 -> - %% message must have been immediately dropped - {State0#?STATE{enqueue_count = 0}, Effects0}; - Total -> - Interval = case Base of - 0 -> 0; - _ -> - min(max(Total, Base), ?RELEASE_CURSOR_EVERY_MAX) - end, - State = State0#?STATE{cfg = Cfg#cfg{release_cursor_interval = - {Base, Interval}}}, - Effects = Effects0 ++ [{checkpoint, RaftIdx, State}], - {State#?STATE{enqueue_count = 0}, Effects} - end; -maybe_checkpoint(_RaftIdx, State, Effects) -> - {State, Effects}. - maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, {_MetaSize, BodySize}, Effects, #?STATE{msg_bytes_enqueue = Enqueue, @@ -1739,14 +1719,14 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, {duplicate, State0, Effects0} end. -return(#{index := IncomingRaftIdx} = Meta, +return(#{} = Meta, ConsumerKey, MsgIds, Checked, Effects0, State0) -> %% 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 := {_At, Msg0}} -> + #{MsgId := Msg0} -> Msg = update_msg_header(return_count, fun incr/1, 1, Msg0), return_one(Meta, MsgId, Msg, @@ -1761,8 +1741,7 @@ return(#{index := IncomingRaftIdx} = Meta, _ -> State1 end, - {State, ok, Effects} = checkout(Meta, State0, State2, Effects1), - update_smallest_raft_index(IncomingRaftIdx, State, Effects). + checkout(Meta, State0, State2, Effects1). % used to process messages that are finished complete(Meta, ConsumerKey, [MsgId], @@ -1771,7 +1750,7 @@ complete(Meta, ConsumerKey, [MsgId], msg_bytes_checkout = BytesCheckout, messages_total = Tot} = State0) -> case maps:take(MsgId, Checked0) of - {?C_MSG(_, Idx, Hdr), Checked} -> + {?MSG(Idx, Hdr), Checked} -> SettledSize = get_header(size, Hdr), Indexes = rabbit_fifo_index:delete(Idx, Indexes0), Con = Con0#consumer{checked_out = Checked, @@ -1792,7 +1771,7 @@ complete(Meta, ConsumerKey, MsgIds, = lists:foldl( fun (MsgId, {S0, Ch0, Idxs}) -> case maps:take(MsgId, Ch0) of - {?C_MSG(_, Idx, Hdr), Ch} -> + {?MSG(Idx, Hdr), Ch} -> S = get_header(size, Hdr) + S0, {S, Ch, rabbit_fifo_index:delete(Idx, Idxs)}; error -> @@ -1829,14 +1808,13 @@ increase_credit(#consumer{cfg = #consumer_cfg{credit_mode = increase_credit(#consumer{credit = Current}, Credit) -> Current + Credit. -complete_and_checkout(#{index := IncomingRaftIdx} = Meta, MsgIds, ConsumerKey, +complete_and_checkout(#{} = Meta, MsgIds, ConsumerKey, #consumer{} = Con0, Effects0, State0) -> State1 = complete(Meta, ConsumerKey, MsgIds, Con0, State0), %% a completion could have removed the active/fading consumer {State2, Effects1} = activate_next_consumer(State1, Effects0), - {State, ok, Effects} = checkout(Meta, State0, State2, Effects1), - update_smallest_raft_index(IncomingRaftIdx, State, Effects). + checkout(Meta, State0, State2, Effects1). cancel_consumer_effects(ConsumerId, #?STATE{cfg = #cfg{resource = QName}}, @@ -1844,33 +1822,6 @@ cancel_consumer_effects(ConsumerId, [{mod_call, rabbit_quorum_queue, cancel_consumer_handler, [QName, ConsumerId]} | Effects]. -update_smallest_raft_index(Idx, State, Effects) -> - update_smallest_raft_index(Idx, ok, State, Effects). - -update_smallest_raft_index(IncomingRaftIdx, Reply, - #?STATE{cfg = Cfg} = State0, - Effects0) -> - %% TODO: optimise - case smallest_raft_index(State0) of - undefined -> - % there are no messages on queue anymore and no pending enqueues - % we can forward release_cursor all the way until - % the last received command, hooray - %% reset the release cursor interval - #cfg{release_cursor_interval = {Base, _}} = Cfg, - RCI = {Base, Base}, - State = State0#?STATE{cfg = Cfg#cfg{release_cursor_interval = RCI}, - enqueue_count = 0}, - Effects = Effects0 ++ [{release_cursor, IncomingRaftIdx, State}], - {State, Reply, Effects}; - Smallest when is_integer(Smallest) -> - %% Promote a checkpoint smaller than `Smallest'. `Smallest' is the - %% oldest message that must remain in the log, so we can only - %% promote an existing checkpoint smaller than `Smallest'. - Effects = Effects0 ++ [{release_cursor, Smallest - 1}], - {State0, Reply, Effects} - end. - update_msg_header(Key, Fun, Def, ?MSG(Idx, Header)) -> ?MSG(Idx, update_header(Key, Fun, Def, Header)). @@ -1943,7 +1894,7 @@ return_one(Meta, MsgId, ?MSG(_, _) = Msg0, return_all(Meta, #?STATE{consumers = Cons} = State0, Effects0, ConsumerKey, #consumer{checked_out = Checked} = Con) -> State = State0#?STATE{consumers = Cons#{ConsumerKey => Con}}, - lists:foldl(fun ({MsgId, ?C_MSG(_At, Msg)}, {S, E}) -> + lists:foldl(fun ({MsgId, Msg}, {S, E}) -> return_one(Meta, MsgId, Msg, S, E, ConsumerKey) end, {State, Effects0}, lists:sort(maps:to_list(Checked))). @@ -1954,7 +1905,7 @@ checkout(#{index := Index} = Meta, #?STATE{} = OldState, State0, Effects0, Reply) -> {#?STATE{cfg = #cfg{dead_letter_handler = DLH}, - dlx = DlxState0} = State1, ExpiredMsg, Effects1} = + 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? @@ -1963,10 +1914,8 @@ checkout(#{index := Index} = Meta, dlx = DlxState}, Effects2 = DlxDeliveryEffects ++ Effects1, case evaluate_limit(Index, false, OldState, State2, Effects2) of - {State, false, Effects} when ExpiredMsg == false -> - {State, Reply, Effects}; {State, _, Effects} -> - update_smallest_raft_index(Index, Reply, State, Effects) + {State, Reply, Effects} end. checkout0(Meta, {success, ConsumerKey, MsgId, @@ -2156,7 +2105,7 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) -> credit = Credit, delivery_count = DelCnt0, cfg = Cfg} = Con0 -> - Checked = maps:put(Next, ?C_MSG(Ts, Msg), Checked0), + Checked = maps:put(Next, Msg, Checked0), DelCnt = case credit_api_v2(Cfg) of true -> add(DelCnt0, 1); false -> DelCnt0 + 1 @@ -2800,3 +2749,55 @@ priority_tag(Msg) -> false -> lo end. + +do_checkpoints(Ts, + #checkpoint{index = ChIdx, + timestamp = ChTime, + enqueue_count = ChEnqCnt, + last_smallest_index = LastSmallest, + messages_total = LastMsgsTot} = Check0, RaAux) -> + LastAppliedIdx = ra_aux:last_applied(RaAux), + #?STATE{enqueue_count = EnqCnt} = MacState = ra_aux:machine_state(RaAux), + MsgsTot = messages_total(MacState), + Mult = case MsgsTot > 200_000 of + true -> + min(4, MsgsTot div 100_000); + false -> + 1 + end, + Since = Ts - ChTime, + NewSmallest = case smallest_raft_index(MacState) of + undefined -> + LastAppliedIdx; + Smallest -> + Smallest + end, + {Check, Effects} = case (EnqCnt - ChEnqCnt > 4096 andalso + Since > (500 * Mult)) orelse + (LastAppliedIdx - ChIdx > 65456 andalso + Since > (5000 * Mult)) orelse + (LastMsgsTot > 0 andalso MsgsTot == 0) of + true -> + %% take a checkpoint; + {#checkpoint{index = LastAppliedIdx, + timestamp = Ts, + enqueue_count = EnqCnt, + last_smallest_index = NewSmallest, + messages_total = MsgsTot}, + [{checkpoint, LastAppliedIdx, MacState} | + release_cursor(LastSmallest, NewSmallest)]}; + false -> + {Check0#checkpoint{last_smallest_index = NewSmallest}, + release_cursor(LastSmallest, NewSmallest)} + end, + + {Check, Effects}. + +release_cursor(LastSmallest, Smallest) + when is_integer(LastSmallest) andalso + is_integer(Smallest) andalso + Smallest > LastSmallest -> + [{release_cursor, Smallest}]; +release_cursor(_, _) -> + []. + diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index 4da4db55e152..0518c7fdb0d1 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -11,8 +11,8 @@ %% 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(C_MSG(At, Msg), {At, Msg}). --define(C_MSG(At, Index, Header), {At, ?MSG(Index, Header)}). +% -define(C_MSG(_At, Msg), Msg). +% -define(C_MSG(_At, Index, Header), ?MSG(Index, Header)). -define(IS_HEADER(H), (is_integer(H) andalso H >= 0) orelse @@ -101,8 +101,8 @@ -type applied_mfa() :: {module(), atom(), list()}. % represents a partially applied module call --define(RELEASE_CURSOR_EVERY, 2048). --define(RELEASE_CURSOR_EVERY_MAX, 3_200_000). +-define(RELEASE_CURSOR_EVERY, 2048 * 4). +-define(RELEASE_CURSOR_EVERY_MAX, 1_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. @@ -130,7 +130,7 @@ {cfg = #consumer_cfg{}, status = up :: up | suspected_down | cancelled | fading, next_msg_id = 0 :: msg_id(), - checked_out = #{} :: #{msg_id() => {At :: milliseconds(), msg()}}, + checked_out = #{} :: #{msg_id() => msg()}, %% max number of messages that can be sent %% decremented for each delivery credit = 0 :: non_neg_integer(), diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index b86cc36ac3f0..a2b1da64d9b6 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -1822,6 +1822,10 @@ make_ra_conf(Q, ServerId, TickTimeout, UId = ra:new_uid(ra_lib:to_binary(ClusterName)), FName = rabbit_misc:rs(QName), Formatter = {?MODULE, format_ra_event, [QName]}, + LogCfg = #{uid => UId, + snapshot_interval => SnapshotInterval, + min_checkpoint_interval => CheckpointInterval, + max_checkpoints => 3}, rabbit_misc:maps_put_truthy(membership, Membership, #{cluster_name => ClusterName, id => ServerId, @@ -1829,10 +1833,7 @@ make_ra_conf(Q, ServerId, TickTimeout, friendly_name => FName, metrics_key => QName, initial_members => Members, - log_init_args => #{uid => UId, - snapshot_interval => SnapshotInterval, - min_checkpoint_interval => - CheckpointInterval}, + log_init_args => LogCfg, tick_timeout => TickTimeout, machine => RaMachine, ra_event_formatter => Formatter}). diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 761b3a038fbf..82546bc0db34 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -417,22 +417,6 @@ enq_expire_deq_enq_enq_deq_deq_test(Config) -> apply(meta(Config, 6), make_checkout({c3, self()}, {dequeue, unsettled}, #{}), S5), ok. -release_cursor_test(Config) -> - Cid = {?FUNCTION_NAME_B, self()}, - {State1, _} = enq(Config, 1, 1, first, test_init(test)), - {State2, _} = enq(Config, 2, 2, second, State1), - {State3, #{key := CKey}, _} = checkout(Config, ?LINE, Cid, - {auto, {simple_prefetch, 10}}, - State2), - % no release cursor effect at this point - {State4, _} = settle(Config, CKey, ?LINE, 1, State3), - Settle0Idx = ?LINE, - {_Final, Effects1} = settle(Config, CKey, Settle0Idx, 0, State4), - ct:pal("Effects1 ~p", [Effects1]), - % empty queue forwards release cursor all the way - ?ASSERT_EFF({release_cursor, _, __}, Effects1), - ok. - checkout_enq_settle_test(Config) -> Cid = {?FUNCTION_NAME_B, self()}, {State1, #{key := CKey, @@ -441,10 +425,7 @@ checkout_enq_settle_test(Config) -> {State2, Effects0} = enq(Config, 2, 1, first, State1), ?ASSERT_EFF({send_msg, _, {delivery, _, [{0, {_, first}}]}, _}, Effects0), {State3, _} = enq(Config, 3, 2, second, State2), - {_, Effects} = settle(Config, CKey, 4, NextMsgId, State3), - % the release cursor is the smallest raft index that does not - % contribute to the state of the application - ?ASSERT_EFF({release_cursor, 2}, Effects), + {_, _Effects} = settle(Config, CKey, 4, NextMsgId, State3), ok. duplicate_enqueue_test(Config) -> @@ -620,11 +601,10 @@ cancelled_checkout_empty_queue_test(Config) -> checkout(Config, ?LINE, Cid, 1, test_init(test)),%% prefetch of 1 % cancelled checkout should clear out service_queue also, else we'd get a % build up of these - {State2, _, Effects} = apply(meta(Config, 3), - make_checkout(Cid, cancel, #{}), State1), + {State2, _, _Effects} = apply(meta(Config, 3), + make_checkout(Cid, cancel, #{}), State1), ?assertEqual(0, map_size(State2#rabbit_fifo.consumers)), ?assertEqual(0, priority_queue:len(State2#rabbit_fifo.service_queue)), - ?ASSERT_EFF({release_cursor, _, _}, Effects), ok. cancelled_checkout_out_test(Config) -> @@ -2525,33 +2505,6 @@ checkout_priority_test(Config) -> ?ASSERT_EFF({send_msg, P, {delivery, _, _}, _}, P == Pid, E5), ok. -empty_dequeue_should_emit_release_cursor_test(Config) -> - State0 = test_init(?FUNCTION_NAME), - Cid = {<<"basic.get1">>, self()}, - {_State, {dequeue, empty}, Effects} = - apply(meta(Config, ?LINE, 1234), - make_checkout(Cid, {dequeue, unsettled}, #{}), - State0), - - ?ASSERT_EFF({release_cursor, _, _}, Effects), - ok. - -expire_message_should_emit_release_cursor_test(Config) -> - Conf = #{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r(<<"/">>, queue, ?FUNCTION_NAME_B), - release_cursor_interval => 0, - msg_ttl => 1}, - S0 = rabbit_fifo:init(Conf), - Msg = #basic_message{content = #content{properties = #'P_basic'{}, - payload_fragments_rev = []}}, - {S1, ok, _} = apply(meta(Config, ?LINE, 100, {notify, 1, self()}), - rabbit_fifo:make_enqueue(self(), 1, Msg), S0), - {_S, ok, Effs} = apply(meta(Config, ?LINE, 101, {notify, 2, self()}), - rabbit_fifo:make_enqueue(self(), 2, Msg), - S1), - ?ASSERT_EFF({release_cursor, _}, Effs), - ok. - header_test(_) -> H0 = Size = 5, ?assertEqual(Size, rabbit_fifo:get_header(size, H0)), diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index a2dfc4cf4a00..1271e7dd6b76 100644 --- a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl @@ -1550,30 +1550,6 @@ upgrade_prop(Conf0, Commands) -> %% check we can run the post entries from the converted state run_log(V4, PostEntries, fun (_) -> true end, ToVersion) end || SplitPos <- lists:seq(1, length(Entries))], - - {_, V3Effs} = run_log(InitState, Entries, fun (_) -> true end, FromVersion), - [begin - Res = rabbit_fifo:apply(meta(Idx + 1), - {machine_version, FromVersion, ToVersion}, RCS) , - #rabbit_fifo{} = V4 = element(1, Res), - %% assert invariants - Fields = [num_ready_messages, - smallest_raft_index, - num_enqueuers, - num_consumers, - enqueue_message_bytes, - checkout_message_bytes - ], - V3Overview = maps:with(Fields, FromMod:overview(RCS)), - V4Overview = maps:with(Fields, ToMod:overview(V4)), - case V3Overview == V4Overview of - true -> ok; - false -> - ct:pal("upgrade_prop failed expected~n~tp~nGot:~n~tp", - [V3Overview, V4Overview]), - ?assertEqual(V3Overview, V4Overview) - end - end || {release_cursor, Idx, RCS} <- V3Effs], true. %% single active consumer ordering invariant: @@ -2044,24 +2020,13 @@ run_proper(Fun, Args, NumTests) -> end}])). run_snapshot_test(Conf, Commands) -> - %% tests that release cursor indexes are definitely lower than - %% the smallest raft index referring to a message Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), ct:pal("running snapshot test 2 with ~b commands using config ~tp", [length(Commands), Conf]), - Fun = fun (_E, S, Effs) -> + Fun = fun (_E, S, _Effs) -> MsgTotFun = messages_total_invariant(), - case lists:reverse( - [C || {release_cursor, _, _} = C <- Effs]) of - [] -> - MsgTotFun(S); - [{release_cursor, Idx, _} | _] -> - %% ensure the current state has no active messages - %% below or equal to the release cursor index - rabbit_fifo:smallest_raft_index(S) > Idx andalso - MsgTotFun(S) - end + MsgTotFun(S) end, _ = run_log(test_init(Conf), Entries, Fun), true. From 42c501aefe5ad78e54401c796d3c6bb1ac81aa75 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 11 Jul 2024 13:25:49 +0100 Subject: [PATCH 23/45] fixes --- deps/rabbit/src/rabbit_fifo.erl | 34 +++++++++++++++++++-------------- 1 file changed, 20 insertions(+), 14 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 0b4ad644e3bc..6a5ff473419d 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -399,9 +399,6 @@ apply(Meta, #credit{credit = LinkCreditRcv, end; _ when Waiting0 /= [] -> %% TODO next time when we bump the machine version: - %% 1. Do not put consumer at head of waiting_consumers if - %% NewCredit == 0 - %% to reduce likelihood of activating a 0 credit consumer. %% 2. Support Drain == true, i.e. advance delivery-count, %% consuming all link-credit since there %% are no messages available for an inactive consumer and @@ -973,12 +970,12 @@ which_module(2) -> rabbit_fifo_v3; which_module(3) -> rabbit_fifo_v3; which_module(4) -> ?MODULE. --define(AUX, aux_v2). +-define(AUX, aux_v3). -record(checkpoint, {index :: ra:index(), timestamp :: milliseconds(), enqueue_count :: non_neg_integer(), - last_smallest_index :: ra:index(), + smallest_index :: undefined | ra:index(), messages_total :: non_neg_integer()}). -record(aux_gc, {last_raft_idx = 0 :: ra:index()}). -record(aux, {name :: atom(), @@ -1003,7 +1000,8 @@ init_aux(Name) when is_atom(Name) -> capacity = {inactive, Now, 1, 1.0}, last_checkpoint = #checkpoint{index = 0, timestamp = erlang:system_time(millisecond), - enqueue_count = 0}}. + enqueue_count = 0, + messages_total = 0}}. handle_aux(RaftState, Tag, Cmd, #aux{name = Name, capacity = Cap, @@ -1013,6 +1011,11 @@ handle_aux(RaftState, Tag, Cmd, #aux{name = 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(_RaftState, cast, {#return{msg_ids = MsgIds, consumer_key = Key} = Ret, Corr, Pid}, Aux0, RaAux0) -> @@ -2558,7 +2561,6 @@ is_expired(Ts, #?STATE{cfg = #cfg{expires = Expires}, is_expired(_Ts, _State) -> false. -%%TODO: provide first class means of configuring get_priority(#{priority := Priority}) -> Priority; get_priority(#{args := Args}) -> @@ -2749,12 +2751,16 @@ priority_tag(Msg) -> false -> lo end. +-define(CHECK_ENQ_MIN_INTERVAL_MS, 500). +-define(CHECK_ENQ_MIN_INDEXES, 4096). +-define(CHECK_MIN_INTERVAL_MS, 5000). +-define(CHECK_MIN_INDEXES, 65456). do_checkpoints(Ts, #checkpoint{index = ChIdx, timestamp = ChTime, enqueue_count = ChEnqCnt, - last_smallest_index = LastSmallest, + smallest_index = LastSmallest, messages_total = LastMsgsTot} = Check0, RaAux) -> LastAppliedIdx = ra_aux:last_applied(RaAux), #?STATE{enqueue_count = EnqCnt} = MacState = ra_aux:machine_state(RaAux), @@ -2772,22 +2778,22 @@ do_checkpoints(Ts, Smallest -> Smallest end, - {Check, Effects} = case (EnqCnt - ChEnqCnt > 4096 andalso - Since > (500 * Mult)) orelse - (LastAppliedIdx - ChIdx > 65456 andalso - Since > (5000 * Mult)) orelse + {Check, Effects} = case (EnqCnt - ChEnqCnt > ?CHECK_ENQ_MIN_INDEXES andalso + Since > (?CHECK_ENQ_MIN_INTERVAL_MS * Mult)) orelse + (LastAppliedIdx - ChIdx > ?CHECK_MIN_INDEXES andalso + Since > (?CHECK_MIN_INTERVAL_MS * Mult)) orelse (LastMsgsTot > 0 andalso MsgsTot == 0) of true -> %% take a checkpoint; {#checkpoint{index = LastAppliedIdx, timestamp = Ts, enqueue_count = EnqCnt, - last_smallest_index = NewSmallest, + smallest_index = NewSmallest, messages_total = MsgsTot}, [{checkpoint, LastAppliedIdx, MacState} | release_cursor(LastSmallest, NewSmallest)]}; false -> - {Check0#checkpoint{last_smallest_index = NewSmallest}, + {Check0#checkpoint{smallest_index = NewSmallest}, release_cursor(LastSmallest, NewSmallest)} end, From e0fadac66610c220893c41a5de6f7ac2051a9630 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 12 Jul 2024 09:49:56 +0100 Subject: [PATCH 24/45] QQ: emit release cursors on tick for followers and leaders else followers could end up holding on to segments a bit longer after traffic stops. --- deps/rabbit/src/rabbit_fifo.erl | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 6a5ff473419d..daaa42e04eb3 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -1062,14 +1062,8 @@ handle_aux(leader, _, {handle_tick, [QName, Overview0, Nodes]}, Pid end, - Effs = case smallest_raft_index(ra_aux:machine_state(RaAux)) of - undefined -> - [{release_cursor, ra_aux:last_applied(RaAux)}]; - Smallest -> - [{release_cursor, Smallest}] - end, %% TODO: check consumer timeouts - {no_reply, Aux#?AUX{tick_pid = NewPid}, RaAux, Effs}; + {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), @@ -1130,7 +1124,13 @@ handle_aux(_RaState, cast, tick, #?AUX{name = Name, true = ets:insert(rabbit_fifo_usage, {Name, capacity(Use0)}), Aux = eval_gc(RaAux, ra_aux:machine_state(RaAux), State0), - {no_reply, Aux, RaAux}; + Effs = case smallest_raft_index(ra_aux:machine_state(RaAux)) of + undefined -> + [{release_cursor, ra_aux:last_applied(RaAux)}]; + Smallest -> + [{release_cursor, Smallest}] + 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}; From 71eec01efbc310f8193c58e4eaf70110853db39b Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 11 Jul 2024 18:09:40 +0200 Subject: [PATCH 25/45] Support draining a QQ SAC waiting consumer 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, the sending queue should advance the delivery-count consuming all link-credit and send a credit_reply with drain=true to the session proc which causes the session proc to send a flow frame to the client. --- deps/rabbit/src/rabbit_fifo.erl | 42 +++++--- deps/rabbit/test/amqp_client_SUITE.erl | 127 +++++++++++++++++++++++++ deps/rabbit/test/rabbit_fifo_SUITE.erl | 52 ++++++++++ 3 files changed, 207 insertions(+), 14 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index daaa42e04eb3..33433a9dc75b 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -398,31 +398,45 @@ apply(Meta, #credit{credit = LinkCreditRcv, end end; _ when Waiting0 /= [] -> - %% TODO next time when we bump the machine version: - %% 2. Support Drain == true, i.e. advance delivery-count, - %% consuming all link-credit since there - %% are no messages available for an inactive consumer and - %% send credit_reply with Drain=true. case lists:keytake(ConsumerKey, 1, Waiting0) of {value, {_, Con0 = #consumer{delivery_count = DeliveryCountSnd, - cfg = Cfg}}, Waiting} -> - LinkCreditSnd = link_credit_snd(DeliveryCountRcv, LinkCreditRcv, - DeliveryCountSnd, Cfg), - %% grant the credit - Con = Con0#consumer{credit = LinkCreditSnd}, - State = State0#?STATE{waiting_consumers = - add_waiting({ConsumerKey, Con}, Waiting)}, + cfg = Cfg}}, Waiting1} -> %% 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}, Waiting1), + State = State0#?STATE{waiting_consumers = Waiting}, {State, ok, {send_msg, Cfg#consumer_cfg.pid, {credit_reply, Cfg#consumer_cfg.tag, - DeliveryCountSnd, LinkCreditSnd, - Available, false}, + 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}, Waiting1), + State = State0#?STATE{waiting_consumers = Waiting}, {State, {send_credit_reply, Available}} end; false -> diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl index c6e73abbe9ea..d02e207334a4 100644 --- a/deps/rabbit/test/amqp_client_SUITE.erl +++ b/deps/rabbit/test/amqp_client_SUITE.erl @@ -85,6 +85,8 @@ groups() -> consumer_priority_quorum_queue, single_active_consumer_classic_queue, single_active_consumer_quorum_queue, + single_active_consumer_drain_classic_queue, + single_active_consumer_drain_quorum_queue, detach_requeues_one_session_classic_queue, detach_requeues_one_session_quorum_queue, detach_requeues_drop_head_classic_queue, @@ -216,6 +218,14 @@ init_per_testcase(T, Config) {skip, "Receiving with drain from quorum queues in credit API v1 have a known " "bug that they reply with send_drained before delivering the message."} end; +init_per_testcase(single_active_consumer_drain_quorum_queue = T, Config) -> + case rpc(Config, rabbit_feature_flags, is_enabled, [credit_api_v2]) of + true -> + rabbit_ct_helpers:testcase_started(Config, T); + false -> + {skip, "Draining a SAC inactive quorum queue consumer with credit API v1 " + "is known to be unsupported."} + end; init_per_testcase(T, Config) when T =:= incoming_window_closed_close_link orelse T =:= incoming_window_closed_rabbitmq_internal_flow_classic_queue orelse @@ -2065,6 +2075,123 @@ single_active_consumer(QType, Config) -> ok = end_session_sync(Session), ok = amqp10_client:close_connection(Connection). +single_active_consumer_drain_classic_queue(Config) -> + single_active_consumer_drain(<<"classic">>, Config). + +single_active_consumer_drain_quorum_queue(Config) -> + single_active_consumer_drain(<<"quorum">>, Config). + +single_active_consumer_drain(QType, Config) -> + QName = atom_to_binary(?FUNCTION_NAME), + {Connection, Session, LinkPair} = init(Config), + QProps = #{arguments => #{<<"x-queue-type">> => {utf8, QType}, + <<"x-single-active-consumer">> => true}}, + {ok, #{type := QType}} = rabbitmq_amqp_client:declare_queue(LinkPair, QName, QProps), + + %% Attach 1 sender and 2 receivers to the queue. + Address = rabbitmq_amqp_address:queue(QName), + {ok, Sender} = amqp10_client:attach_sender_link(Session, <<"test-sender">>, Address), + ok = wait_for_credit(Sender), + + %% The 1st consumer will become active. + {ok, Receiver1} = amqp10_client:attach_receiver_link( + Session, + <<"test-receiver-1">>, + Address, + unsettled), + receive {amqp10_event, {link, Receiver1, attached}} -> ok + after 5000 -> ct:fail("missing attached") + end, + %% The 2nd consumer will become inactive. + {ok, Receiver2} = amqp10_client:attach_receiver_link( + Session, + <<"test-receiver-2">>, + Address, + unsettled), + receive {amqp10_event, {link, Receiver2, attached}} -> ok + after 5000 -> ct:fail("missing attached") + end, + flush(attached), + + %% Drain both active and inactive consumer for the 1st time. + ok = amqp10_client:flow_link_credit(Receiver1, 100, never, true), + ok = amqp10_client:flow_link_credit(Receiver2, 100, never, true), + receive {amqp10_event, {link, Receiver1, credit_exhausted}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + receive {amqp10_event, {link, Receiver2, credit_exhausted}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + + %% Send 2 messages. + ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"dtag1">>, <<"m1">>)), + ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"dtag2">>, <<"m2">>)), + ok = wait_for_accepts(2), + + %% No consumer should receive a message since both should have 0 credits. + receive Unexpected0 -> ct:fail("received unexpected ~p", [Unexpected0]) + after 10 -> ok + end, + + %% Drain both active and inactive consumer for the 2nd time. + ok = amqp10_client:flow_link_credit(Receiver1, 200, never, true), + ok = amqp10_client:flow_link_credit(Receiver2, 200, never, true), + + %% Only the active consumer should receive messages. + receive {amqp10_msg, Receiver1, Msg1} -> + ?assertEqual([<<"m1">>], amqp10_msg:body(Msg1)), + ok = amqp10_client:accept_msg(Receiver1, Msg1) + after 5000 -> ct:fail({missing_msg, ?LINE}) + end, + receive {amqp10_msg, Receiver1, Msg2} -> + ?assertEqual([<<"m2">>], amqp10_msg:body(Msg2)), + ok = amqp10_client:accept_msg(Receiver1, Msg2) + after 5000 -> ct:fail({missing_msg, ?LINE}) + end, + receive {amqp10_event, {link, Receiver1, credit_exhausted}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + receive {amqp10_event, {link, Receiver2, credit_exhausted}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + + %% Cancelling the active consumer should cause the inactive to become active. + ok = amqp10_client:detach_link(Receiver1), + receive {amqp10_event, {link, Receiver1, {detached, normal}}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + + %% Send 1 more message. + ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"dtag3">>, <<"m3">>)), + ok = wait_for_accepted(<<"dtag3">>), + + %% Our 2nd (now active) consumer should have 0 credits. + receive Unexpected1 -> ct:fail("received unexpected ~p", [Unexpected1]) + after 10 -> ok + end, + + %% Drain for the 3rd time. + ok = amqp10_client:flow_link_credit(Receiver2, 300, never, true), + + receive {amqp10_msg, Receiver2, Msg3} -> + ?assertEqual([<<"m3">>], amqp10_msg:body(Msg3)), + ok = amqp10_client:accept_msg(Receiver2, Msg3) + after 5000 -> ct:fail({missing_msg, ?LINE}) + end, + receive {amqp10_event, {link, Receiver2, credit_exhausted}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + + ok = amqp10_client:detach_link(Receiver2), + receive {amqp10_event, {link, Receiver2, {detached, normal}}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + ?assertMatch({ok, #{message_count := 0}}, + rabbitmq_amqp_client:delete_queue(LinkPair, QName)), + ok = rabbitmq_amqp_client:detach_management_link_pair_sync(LinkPair), + ok = end_session_sync(Session), + ok = amqp10_client:close_connection(Connection). + %% "A session endpoint can choose to unmap its output handle for a link. In this case, the endpoint MUST %% send a detach frame to inform the remote peer that the handle is no longer attached to the link endpoint. %% If both endpoints do this, the link MAY return to a fully detached state. Note that in this case the diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 82546bc0db34..eb8fbd8e901f 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -285,6 +285,58 @@ credit_and_drain_v2_test(Config) -> ?ASSERT_NO_EFF({log, _, _, _}, EnqEffs), ok. +credit_and_drain_single_active_consumer_v2_test(Config) -> + State0 = init(#{name => ?FUNCTION_NAME, + queue_resource => rabbit_misc:r( + "/", queue, atom_to_binary(?FUNCTION_NAME)), + release_cursor_interval => 0, + single_active_consumer_on => true}), + Self = self(), + + % Send 1 message. + {State1, _} = enq(Config, 1, 1, first, State0), + + % Add 2 consumers. + Ctag1 = <<"ctag1">>, + Ctag2 = <<"ctag2">>, + C1 = {Ctag1, Self}, + C2 = {Ctag2, Self}, + CK1 = ?LINE, + CK2 = ?LINE, + Entries = [ + {CK1, make_checkout(C1, {auto, {credited, 16#ff_ff_ff_ff}}, #{})}, + {CK2, make_checkout(C2, {auto, {credited, 16#ff_ff_ff_ff}}, #{})} + ], + {State2, _} = run_log(Config, State1, Entries), + + % The 1st registered consumer is the active one, the 2nd consumer is waiting. + ?assertMatch(#{single_active_consumer_id := C1, + single_active_num_waiting_consumers := 1}, + rabbit_fifo:overview(State2)), + + % Drain the inactive consumer. + {State3, Effects0} = credit(Config, CK2, ?LINE, 5000, 16#ff_ff_ff_ff, true, State2), + % The inactive consumer should not receive any message. + % Hence, no log effect should be returned. + % Since we sent drain=true, we expect the sending queue to consume all link credit + % advancing the delivery-count. + ?assertEqual({send_msg, Self, + {credit_reply, Ctag2, _DeliveryCount = 4999, _Credit = 0, + _Available = 0, _Drain = true}, + ?DELIVERY_SEND_MSG_OPTS}, + Effects0), + + % Drain the active consumer. + {_State4, Effects1} = credit(Config, CK1, ?LINE, 1000, 16#ff_ff_ff_ff, true, State3), + ?assertMatch([ + {log, [1], _Fun, _Local}, + {send_msg, Self, + {credit_reply, Ctag1, _DeliveryCount = 999, _Credit = 0, + _Available = 0, _Drain = true}, + ?DELIVERY_SEND_MSG_OPTS} + ], + Effects1). + enq_enq_deq_test(C) -> Cid = {?FUNCTION_NAME_B, self()}, {State1, _} = enq(C, 1, 1, first, test_init(test)), From 5e30ad26901882f4a545aaa83ef11f219e0f0442 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 12 Jul 2024 12:00:47 +0200 Subject: [PATCH 26/45] Extract applying #credit{} cmd into 2 functions This commit is only refactoring and doesn't change any behaviour. --- deps/rabbit/src/rabbit_fifo.erl | 261 +++++++++++++++++--------------- 1 file changed, 137 insertions(+), 124 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 33433a9dc75b..2459fe26da0f 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -324,127 +324,19 @@ apply(#{index := Idx} = Meta, _ -> {State00, ok, []} end; -apply(Meta, #credit{credit = LinkCreditRcv, - delivery_count = DeliveryCountRcv, - drain = Drain, - consumer_key = ConsumerKey}, - #?STATE{consumers = Cons0, - service_queue = ServiceQueue0, - waiting_consumers = Waiting0} = State0) -> - case Cons0 of - #{ConsumerKey := #consumer{delivery_count = DeliveryCountSnd, - cfg = Cfg} = Con0} -> - 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; - _ when Waiting0 /= [] -> - case lists:keytake(ConsumerKey, 1, Waiting0) of - {value, {_, Con0 = #consumer{delivery_count = DeliveryCountSnd, - cfg = Cfg}}, Waiting1} -> - %% 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}, Waiting1), - State = State0#?STATE{waiting_consumers = Waiting}, - {State, ok, - {send_msg, Cfg#consumer_cfg.pid, - {credit_reply, Cfg#consumer_cfg.tag, - 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}, Waiting1), - State = State0#?STATE{waiting_consumers = Waiting}, - {State, {send_credit_reply, Available}} - end; - false -> - {State0, 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); _ -> - %% credit for unknown consumer - just ignore - {State0, ok} + 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) -> @@ -2365,12 +2257,133 @@ included_credit({credited, _}) -> 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. + %% make the state suitable for equality comparison normalize(#?STATE{ra_indexes = _Indexes, - returns = Returns, - messages = Messages, - release_cursors = Cursors, - dlx = DlxState} = State) -> + returns = Returns, + messages = Messages, + release_cursors = Cursors, + dlx = DlxState} = State) -> State#?STATE{returns = lqueue:from_list(lqueue:to_list(Returns)), messages = rabbit_fifo_q:normalize(Messages, rabbit_fifo_q:new()), From 954e37ba4c5c6fb3847084b3a5843dea7af641ba Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 12 Jul 2024 13:23:35 +0200 Subject: [PATCH 27/45] Fix default priority level Prior to this commit, when a message didn't have a priority level set, it got enqueued as high prio. This is wrong because the default priority is 4 and "for example, if 2 distinct priorities are implemented, then levels 0 to 4 are equivalent, and levels 5 to 9 are equivalent and levels 4 and 5 are distinct." Hence, by default a message without priority set, must be enqueued as low prio. --- deps/rabbit/src/rabbit_fifo.erl | 4 +- deps/rabbit/test/amqp_client_SUITE.erl | 53 ++++++++++++++++---------- 2 files changed, 36 insertions(+), 21 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 2459fe26da0f..e4aa845d7d62 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -2770,7 +2770,8 @@ priority_tag(Msg) -> case mc:is(Msg) of true -> case mc:priority(Msg) of - P when P > 4 -> + P when is_integer(P) andalso + P > 4 -> hi; _ -> lo @@ -2778,6 +2779,7 @@ priority_tag(Msg) -> false -> lo end. + -define(CHECK_ENQ_MIN_INTERVAL_MS, 500). -define(CHECK_ENQ_MIN_INDEXES, 4096). -define(CHECK_MIN_INTERVAL_MS, 5000). diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl index d02e207334a4..8fda571ac3da 100644 --- a/deps/rabbit/test/amqp_client_SUITE.erl +++ b/deps/rabbit/test/amqp_client_SUITE.erl @@ -111,7 +111,8 @@ groups() -> handshake_timeout, credential_expires, attach_to_exclusive_queue, - classic_priority_queue, + priority_classic_queue, + priority_quorum_queue, dead_letter_headers_exchange, dead_letter_reject, dead_letter_reject_message_order_classic_queue, @@ -4066,31 +4067,43 @@ attach_to_exclusive_queue(Config) -> #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}), ok = rabbit_ct_client_helpers:close_channel(Ch). -classic_priority_queue(Config) -> +priority_classic_queue(Config) -> + QArgs = #{<<"x-queue-type">> => {utf8, <<"classic">>}, + <<"x-max-priority">> => {ulong, 10}}, + priority(QArgs, Config). + +priority_quorum_queue(Config) -> + QArgs = #{<<"x-queue-type">> => {utf8, <<"quorum">>}}, + priority(QArgs, Config). + +priority(QArgs, Config) -> + {Connection, Session, LinkPair} = init(Config), QName = atom_to_binary(?FUNCTION_NAME), Address = rabbitmq_amqp_address:queue(QName), - Ch = rabbit_ct_client_helpers:open_channel(Config), - #'queue.declare_ok'{} = amqp_channel:call( - Ch, #'queue.declare'{ - queue = QName, - durable = true, - arguments = [{<<"x-max-priority">>, long, 10}]}), - OpnConf = connection_config(Config), - {ok, Connection} = amqp10_client:open_connection(OpnConf), - {ok, Session} = amqp10_client:begin_session_sync(Connection), + {ok, _} = rabbitmq_amqp_client:declare_queue(LinkPair, QName, #{arguments => QArgs}), {ok, Sender} = amqp10_client:attach_sender_link(Session, <<"test-sender">>, Address), wait_for_credit(Sender), - Out1 = amqp10_msg:set_headers(#{priority => 3, - durable => true}, amqp10_msg:new(<<"t1">>, <<"low prio">>, false)), - Out2 = amqp10_msg:set_headers(#{priority => 5, - durable => true}, amqp10_msg:new(<<"t2">>, <<"high prio">>, false)), - ok = amqp10_client:send_msg(Sender, Out1), - ok = amqp10_client:send_msg(Sender, Out2), + %% We don't set a priority on Msg1. + %% According to the AMQP spec, the default priority is 4. + Msg1 = amqp10_msg:set_headers( + #{durable => true}, + amqp10_msg:new(<<"t1">>, <<"low prio">>)), + %% Quorum queues implement 2 distinct priority levels. + %% "if 2 distinct priorities are implemented, then levels 0 to 4 are equivalent, + %% and levels 5 to 9 are equivalent and levels 4 and 5 are distinct." [§3.2.1] + %% Therefore, when we set a priority of 5 on Msg2, Msg2 will have a higher priority + %% than the default priority 4 of Msg1. + Msg2 = amqp10_msg:set_headers( + #{priority => 5, + durable => true}, + amqp10_msg:new(<<"t2">>, <<"high prio">>)), + ok = amqp10_client:send_msg(Sender, Msg1), + ok = amqp10_client:send_msg(Sender, Msg2), ok = wait_for_accepts(2), flush(accepted), - %% The high prio message should be delivered first. + %% The high prio Msg2 should overtake the low prio Msg1 and therefore be delivered first. {ok, Receiver1} = amqp10_client:attach_receiver_link(Session, <<"receiver 1">>, Address, unsettled), {ok, In1} = amqp10_client:get_msg(Receiver1), ?assertEqual([<<"high prio">>], amqp10_msg:body(In1)), @@ -4101,13 +4114,13 @@ classic_priority_queue(Config) -> {ok, Receiver2} = amqp10_client:attach_receiver_link(Session, <<"receiver 2">>, Address, settled), {ok, In2} = amqp10_client:get_msg(Receiver2), ?assertEqual([<<"low prio">>], amqp10_msg:body(In2)), - ?assertEqual(3, amqp10_msg:header(priority, In2)), ?assert(amqp10_msg:header(durable, In2)), ok = amqp10_client:detach_link(Receiver1), ok = amqp10_client:detach_link(Receiver2), ok = amqp10_client:detach_link(Sender), - ok = delete_queue(Session, QName), + {ok, #{message_count := 0}} = rabbitmq_amqp_client:delete_queue(LinkPair, QName), + ok = rabbitmq_amqp_client:detach_management_link_pair_sync(LinkPair), ok = end_session_sync(Session), ok = amqp10_client:close_connection(Connection). From 8242b82412cb1ce634f3637594b11f7a16b7edd7 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 12 Jul 2024 13:50:18 +0200 Subject: [PATCH 28/45] bazel run gazelle --- deps/rabbit/BUILD.bazel | 4 ++-- deps/rabbit/app.bzl | 5 +++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/deps/rabbit/BUILD.bazel b/deps/rabbit/BUILD.bazel index a3e008e5b3b2..a3ebb5349775 100644 --- a/deps/rabbit/BUILD.bazel +++ b/deps/rabbit/BUILD.bazel @@ -725,8 +725,8 @@ rabbitmq_suite( ], deps = [ "//deps/rabbit_common:erlang_app", - "@proper//:erlang_app", "@meck//:erlang_app", + "@proper//:erlang_app", "@ra//:erlang_app", ], ) @@ -743,8 +743,8 @@ rabbitmq_suite( name = "rabbit_fifo_q_SUITE", size = "small", deps = [ - "@proper//:erlang_app", "//deps/rabbit_common:erlang_app", + "@proper//:erlang_app", ], ) diff --git a/deps/rabbit/app.bzl b/deps/rabbit/app.bzl index b968962dae09..ae3204f7d02d 100644 --- a/deps/rabbit/app.bzl +++ b/deps/rabbit/app.bzl @@ -1298,7 +1298,7 @@ def test_suite_beam_files(name = "test_suite_beam_files"): hdrs = ["src/rabbit_fifo.hrl"], app_name = "rabbit", erlc_opts = "//:test_erlc_opts", - deps = ["//deps/rabbit_common:erlang_app", "@proper//:erlang_app"], + deps = ["//deps/rabbit_common:erlang_app"], ) erlang_bytecode( name = "rabbit_fifo_dlx_SUITE_beam_files", @@ -1326,7 +1326,7 @@ def test_suite_beam_files(name = "test_suite_beam_files"): outs = ["test/rabbit_fifo_int_SUITE.beam"], app_name = "rabbit", erlc_opts = "//:test_erlc_opts", - deps = ["//deps/rabbitmq_ct_helpers:erlang_app"], + deps = ["//deps/rabbit_common:erlang_app"], ) erlang_bytecode( name = "rabbit_fifo_prop_SUITE_beam_files", @@ -2150,6 +2150,7 @@ def test_suite_beam_files(name = "test_suite_beam_files"): deps = ["//deps/amqp_client:erlang_app"], ) erlang_bytecode( + name = "rabbit_fifo_q_SUITE_beam_files", testonly = True, srcs = ["test/rabbit_fifo_q_SUITE.erl"], outs = ["test/rabbit_fifo_q_SUITE.beam"], From bc0d914e0d8a650d615ba145194f043c00049f34 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 12 Jul 2024 15:23:25 +0200 Subject: [PATCH 29/45] Avoid deprecated time unit --- deps/rabbit/src/rabbit_fifo.erl | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index e4aa845d7d62..896012f14a06 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -901,7 +901,7 @@ 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(micro_seconds), + Now = erlang:monotonic_time(microsecond), #?AUX{name = Name, capacity = {inactive, Now, 1, 1.0}, last_checkpoint = #checkpoint{index = 0, @@ -1303,18 +1303,18 @@ update_use({inactive, _, _, _} = CUInfo, inactive) -> update_use({active, _, _} = CUInfo, active) -> CUInfo; update_use({active, Since, Avg}, inactive) -> - Now = erlang:monotonic_time(micro_seconds), + Now = erlang:monotonic_time(microsecond), {inactive, Now, Now - Since, Avg}; update_use({inactive, Since, Active, Avg}, active) -> - Now = erlang:monotonic_time(micro_seconds), + Now = erlang:monotonic_time(microsecond), {active, Now, use_avg(Active, Now - Since, Avg)}. capacity({active, Since, Avg}) -> - use_avg(erlang:monotonic_time(micro_seconds) - Since, 0, 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(micro_seconds) - Since, Avg). + use_avg(Active, erlang:monotonic_time(microsecond) - Since, Avg). use_avg(0, 0, Avg) -> Avg; From 4213c336f4d8a3c6e83a4c676a5bb48f37035307 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 12 Jul 2024 15:43:10 +0200 Subject: [PATCH 30/45] Fix aux_test --- deps/rabbit/test/rabbit_fifo_SUITE.erl | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index eb8fbd8e901f..c8b9d0246a91 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -2290,15 +2290,18 @@ run_log(Module, Config, InitState, Entries, Invariant) -> aux_test(_) -> _ = ra_machine_ets:start_link(), Aux0 = 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}, + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => false}), + log => mock_log, + 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, _} = handle_aux(leader, cast, tick, Aux, State), + {no_reply, _Aux, _, + [{release_cursor, LastApplied}]} = handle_aux(leader, cast, tick, Aux, State), [X] = ets:lookup(rabbit_fifo_usage, aux_test), meck:unload(), ?assert(X > 0.0), From 903fe74c52ef01124a9b214db9bc2def624472b1 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 15 Jul 2024 17:36:27 +0200 Subject: [PATCH 31/45] Delete dead code --- deps/rabbit/src/rabbit_fifo_q.erl | 8 ++------ deps/rabbit/test/rabbit_fifo_q_SUITE.erl | 3 --- 2 files changed, 2 insertions(+), 9 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo_q.erl b/deps/rabbit/src/rabbit_fifo_q.erl index 125040a77a59..cac4697e9058 100644 --- a/deps/rabbit/src/rabbit_fifo_q.erl +++ b/deps/rabbit/src/rabbit_fifo_q.erl @@ -10,7 +10,7 @@ from_lqueue/1, normalize/2, get_lowest_index/1 - ]). + ]). -define(WEIGHT, 2). -define(NON_EMPTY, {_, [_|_]}). @@ -111,7 +111,7 @@ get_lowest_index(#?MODULE{hi = Hi, lo = Lo}) -> %% internals next(#?MODULE{hi = ?NON_EMPTY = Hi, - lo = ?NON_EMPTY = Lo, + lo = ?NON_EMPTY = Lo, dequeue_counter = ?WEIGHT}) -> ?MSG(HiIdx, _) = HiMsg = peek(Hi), ?MSG(LoIdx, _) = LoMsg = peek(Lo), @@ -131,8 +131,6 @@ next(#?MODULE{lo = Lo}) -> %% invariant, if the queue is non empty so is the Out (right) list. in(X, ?EMPTY) -> {[], [X]}; -in(X, {[_] = In, []}) -> - {[X], In}; in(X, {In, Out}) -> {[X | In], Out}. @@ -141,8 +139,6 @@ peek(?EMPTY) -> peek({_, [H | _]}) -> H. -drop(?EMPTY = Q) -> - Q; drop({In, [_]}) -> %% the last Out one {[], lists:reverse(In)}; diff --git a/deps/rabbit/test/rabbit_fifo_q_SUITE.erl b/deps/rabbit/test/rabbit_fifo_q_SUITE.erl index 86f0e98e21ff..2c2dff85b76d 100644 --- a/deps/rabbit/test/rabbit_fifo_q_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_q_SUITE.erl @@ -3,9 +3,6 @@ -compile(nowarn_export_all). -compile(export_all). --export([ - ]). - -include_lib("proper/include/proper.hrl"). -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). From abf902e280c2df216a7c040bd9427c8dfc3a0bfd Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 15 Jul 2024 18:00:33 +0200 Subject: [PATCH 32/45] Fix rabbit_fifo_q:get_lowest_index/1 --- deps/rabbit/src/rabbit_fifo_q.erl | 2 +- deps/rabbit/test/rabbit_fifo_q_SUITE.erl | 20 +++++++++++++++++++- 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo_q.erl b/deps/rabbit/src/rabbit_fifo_q.erl index cac4697e9058..c77b440efc97 100644 --- a/deps/rabbit/src/rabbit_fifo_q.erl +++ b/deps/rabbit/src/rabbit_fifo_q.erl @@ -102,7 +102,7 @@ get_lowest_index(#?MODULE{hi = Hi, lo = Lo}) -> ?MSG(HiIdx, _) -> case peek(Lo) of ?MSG(LoIdx, _) -> - max(HiIdx, LoIdx); + min(HiIdx, LoIdx); empty -> HiIdx end diff --git a/deps/rabbit/test/rabbit_fifo_q_SUITE.erl b/deps/rabbit/test/rabbit_fifo_q_SUITE.erl index 2c2dff85b76d..01ae5f2e3167 100644 --- a/deps/rabbit/test/rabbit_fifo_q_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_q_SUITE.erl @@ -19,13 +19,14 @@ all_tests() -> hi, basics, hi_is_prioritised, + get_lowest_index, single_priority_behaves_like_queue ]. groups() -> [ - {tests, [], all_tests()} + {tests, [parallel], all_tests()} ]. init_per_suite(Config) -> @@ -107,6 +108,23 @@ hi_is_prioritised(_Config) -> ok. +get_lowest_index(_Config) -> + Q0 = rabbit_fifo_q:new(), + Q1 = rabbit_fifo_q:in(hi, ?MSG(1, ?LINE), Q0), + Q2 = rabbit_fifo_q:in(lo, ?MSG(2, ?LINE), Q1), + Q3 = rabbit_fifo_q:in(lo, ?MSG(3, ?LINE), Q2), + {hi, _, Q4} = rabbit_fifo_q:out(Q3), + {lo, _, Q5} = rabbit_fifo_q:out(Q4), + {lo, _, Q6} = rabbit_fifo_q:out(Q5), + + ?assertEqual(undefined, rabbit_fifo_q:get_lowest_index(Q0)), + ?assertEqual(1, rabbit_fifo_q:get_lowest_index(Q1)), + ?assertEqual(1, rabbit_fifo_q:get_lowest_index(Q2)), + ?assertEqual(1, rabbit_fifo_q:get_lowest_index(Q3)), + ?assertEqual(2, rabbit_fifo_q:get_lowest_index(Q4)), + ?assertEqual(3, rabbit_fifo_q:get_lowest_index(Q5)), + ?assertEqual(undefined, rabbit_fifo_q:get_lowest_index(Q6)). + -type op() :: {in, integer()} | out. single_priority_behaves_like_queue(_Config) -> From 24d0d9dbf5fa57f19ced93cad8fc1116af18f8c1 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 15 Jul 2024 18:39:23 +0200 Subject: [PATCH 33/45] Delete unused normalize functions --- deps/rabbit/src/rabbit_fifo.erl | 13 ------------- deps/rabbit/src/rabbit_fifo_dlx.erl | 8 -------- deps/rabbit/src/rabbit_fifo_index.erl | 7 +------ deps/rabbit/src/rabbit_fifo_q.erl | 10 ---------- deps/rabbit/src/rabbit_fifo_v3.erl | 12 ------------ 5 files changed, 1 insertion(+), 49 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 896012f14a06..3153dbab4508 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -71,7 +71,6 @@ is_v4/0, %% misc - normalize/1, get_msg_header/1, get_header/2, get_msg/1, @@ -2378,18 +2377,6 @@ credit_inactive_consumer( {State, {send_credit_reply, Available}} end. -%% make the state suitable for equality comparison -normalize(#?STATE{ra_indexes = _Indexes, - returns = Returns, - messages = Messages, - release_cursors = Cursors, - dlx = DlxState} = State) -> - State#?STATE{returns = lqueue:from_list(lqueue:to_list(Returns)), - messages = rabbit_fifo_q:normalize(Messages, - rabbit_fifo_q:new()), - release_cursors = lqueue:from_list(lqueue:to_list(Cursors)), - dlx = rabbit_fifo_dlx:normalize(DlxState)}. - is_over_limit(#?STATE{cfg = #cfg{max_length = undefined, max_bytes = undefined}}) -> false; diff --git a/deps/rabbit/src/rabbit_fifo_dlx.erl b/deps/rabbit/src/rabbit_fifo_dlx.erl index 12326a13c490..480b6f497fec 100644 --- a/deps/rabbit/src/rabbit_fifo_dlx.erl +++ b/deps/rabbit/src/rabbit_fifo_dlx.erl @@ -23,7 +23,6 @@ state_enter/4, handle_aux/6, dehydrate/1, - normalize/1, stat/1, update_config/4, smallest_raft_index/1 @@ -357,13 +356,6 @@ handle_aux(_, _, Aux, _, _, _) -> dehydrate(State) -> State#?MODULE{ra_indexes = rabbit_fifo_index:empty()}. --spec normalize(state()) -> - state(). -normalize(#?MODULE{discards = Discards, - ra_indexes = Indexes} = State) -> - State#?MODULE{discards = lqueue:from_list(lqueue:to_list(Discards)), - ra_indexes = rabbit_fifo_index:normalize(Indexes)}. - -spec smallest_raft_index(state()) -> option(non_neg_integer()). smallest_raft_index(#?MODULE{ra_indexes = Indexes}) -> diff --git a/deps/rabbit/src/rabbit_fifo_index.erl b/deps/rabbit/src/rabbit_fifo_index.erl index b20604386b8d..8a8fbbdb9e07 100644 --- a/deps/rabbit/src/rabbit_fifo_index.erl +++ b/deps/rabbit/src/rabbit_fifo_index.erl @@ -7,8 +7,7 @@ delete/2, size/1, smallest/1, - map/2, - normalize/1 + map/2 ]). -compile({no_auto_import, [size/1]}). @@ -105,10 +104,6 @@ find_next(Next, Last, Map) -> find_next(Next+1, Last, Map) end. --spec normalize(state()) -> state(). -normalize(State) -> - State#?MODULE{largest = undefined}. - -ifdef(TEST). -include_lib("eunit/include/eunit.hrl"). diff --git a/deps/rabbit/src/rabbit_fifo_q.erl b/deps/rabbit/src/rabbit_fifo_q.erl index c77b440efc97..cf43eaaa4e1e 100644 --- a/deps/rabbit/src/rabbit_fifo_q.erl +++ b/deps/rabbit/src/rabbit_fifo_q.erl @@ -8,7 +8,6 @@ get/1, len/1, from_lqueue/1, - normalize/2, get_lowest_index/1 ]). @@ -82,15 +81,6 @@ from_lqueue(LQ) -> in(lo, Item, Acc) end, new(), LQ). --spec normalize(state(), state()) -> state(). -normalize(Q0, Acc) -> - case out(Q0) of - {empty, _} -> - Acc; - {P, Msg, Q} -> - normalize(Q, in(P, Msg, Acc)) - end. - -spec get_lowest_index(state()) -> undefined | ra:index(). get_lowest_index(#?MODULE{len = 0}) -> undefined; diff --git a/deps/rabbit/src/rabbit_fifo_v3.erl b/deps/rabbit/src/rabbit_fifo_v3.erl index 871e258609b3..60ee6be9dc4b 100644 --- a/deps/rabbit/src/rabbit_fifo_v3.erl +++ b/deps/rabbit/src/rabbit_fifo_v3.erl @@ -52,7 +52,6 @@ %% misc dehydrate_state/1, - normalize/1, get_msg_header/1, get_header/2, get_msg/1, @@ -2320,17 +2319,6 @@ dehydrate_state(#?STATE{cfg = #cfg{}, msg_cache = undefined, dlx = rabbit_fifo_dlx:dehydrate(DlxState)}. -%% make the state suitable for equality comparison -normalize(#?STATE{ra_indexes = _Indexes, - returns = Returns, - messages = Messages, - release_cursors = Cursors, - dlx = DlxState} = State) -> - State#?STATE{returns = lqueue:from_list(lqueue:to_list(Returns)), - messages = lqueue:from_list(lqueue:to_list(Messages)), - release_cursors = lqueue:from_list(lqueue:to_list(Cursors)), - dlx = rabbit_fifo_dlx:normalize(DlxState)}. - is_over_limit(#?STATE{cfg = #cfg{max_length = undefined, max_bytes = undefined}}) -> false; From d3828fe5b79a4a7b9bfb7829286fab7b9e7bb235 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Mon, 15 Jul 2024 18:54:59 +0200 Subject: [PATCH 34/45] Generate less garbage --- deps/rabbit/src/rabbit_fifo.erl | 17 +++---- deps/rabbit/src/rabbit_fifo_q.erl | 19 ++++--- deps/rabbit/test/rabbit_fifo_q_SUITE.erl | 65 ++++++++++++------------ 3 files changed, 48 insertions(+), 53 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 3153dbab4508..a9ff09389f56 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -731,13 +731,11 @@ update_waiting_consumer_status(Node, #?STATE{waiting_consumers = WaitingConsumers}, Status) -> sort_waiting( - [begin - case node(Pid) of - Node -> - {ConsumerKey, Consumer#consumer{status = Status}}; - _ -> - {ConsumerKey, Consumer} - end + [case node(Pid) of + Node -> + {ConsumerKey, Consumer#consumer{status = Status}}; + _ -> + {ConsumerKey, Consumer} end || {ConsumerKey, ?CONSUMER_PID(Pid) = Consumer} <- WaitingConsumers, Consumer#consumer.status =/= cancelled]). @@ -1931,9 +1929,9 @@ take_next_msg(#?STATE{returns = Returns0, {NextMsg, State#?STATE{returns = Returns}}; {empty, _} -> case rabbit_fifo_q:out(Messages0) of - {empty, _} -> + empty -> empty; - {_P, ?MSG(RaftIdx, _) = Msg, Messages} -> + {?MSG(RaftIdx, _) = Msg, Messages} -> %% add index here Indexes = rabbit_fifo_index:append(RaftIdx, Indexes0), {Msg, State#?STATE{messages = Messages, @@ -2236,7 +2234,6 @@ sort_waiting(Waiting) -> Status /= up end, Waiting). - merge_consumer(_Meta, #consumer{cfg = CCfg, checked_out = Checked} = Consumer, ConsumerMeta, {Life, Mode}, Priority) -> Credit = included_credit(Mode), diff --git a/deps/rabbit/src/rabbit_fifo_q.erl b/deps/rabbit/src/rabbit_fifo_q.erl index cf43eaaa4e1e..9fd63fe4b6fd 100644 --- a/deps/rabbit/src/rabbit_fifo_q.erl +++ b/deps/rabbit/src/rabbit_fifo_q.erl @@ -39,10 +39,9 @@ in(lo, Item, #?MODULE{lo = Lo, len = Len} = State) -> len = Len + 1}. -spec out(state()) -> - {empty, state()} | - {hi | lo, msg(), state()}. -out(#?MODULE{len = 0} = S) -> - {empty, S}; + empty | {msg(), state()}. +out(#?MODULE{len = 0}) -> + empty; out(#?MODULE{hi = Hi0, lo = Lo0, len = Len, @@ -55,13 +54,13 @@ out(#?MODULE{hi = Hi0, end, case next(State) of {hi, Msg} -> - {hi, Msg, State#?MODULE{hi = drop(Hi0), - dequeue_counter = C, - len = Len - 1}}; + {Msg, State#?MODULE{hi = drop(Hi0), + dequeue_counter = C, + len = Len - 1}}; {lo, Msg} -> - {lo, Msg, State#?MODULE{lo = drop(Lo0), - dequeue_counter = C, - len = Len - 1}} + {Msg, State#?MODULE{lo = drop(Lo0), + dequeue_counter = C, + len = Len - 1}} end. -spec get(state()) -> empty | msg(). diff --git a/deps/rabbit/test/rabbit_fifo_q_SUITE.erl b/deps/rabbit/test/rabbit_fifo_q_SUITE.erl index 01ae5f2e3167..942ba591c3c9 100644 --- a/deps/rabbit/test/rabbit_fifo_q_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_q_SUITE.erl @@ -59,10 +59,10 @@ hi(_Config) -> fun ({P, I}, Q) -> rabbit_fifo_q:in(P, I, Q) end, Q0, [ - {hi, ?MSG(?LINE)} + {hi, ?MSG(1)} ]), - {hi, _, Q2} = rabbit_fifo_q:out(Q1), - {empty, _Q3} = rabbit_fifo_q:out(Q2), + {?MSG(1), Q2} = rabbit_fifo_q:out(Q1), + empty = rabbit_fifo_q:out(Q2), ok. basics(_Config) -> @@ -71,18 +71,18 @@ basics(_Config) -> fun ({P, I}, Q) -> rabbit_fifo_q:in(P, I, Q) end, Q0, [ - {hi, ?MSG(?LINE)}, - {lo, ?MSG(?LINE)}, - {hi, ?MSG(?LINE)}, - {lo, ?MSG(?LINE)}, - {hi, ?MSG(?LINE)} + {hi, ?MSG(1)}, + {lo, ?MSG(2)}, + {hi, ?MSG(3)}, + {lo, ?MSG(4)}, + {hi, ?MSG(5)} ]), - {hi, _, Q2} = rabbit_fifo_q:out(Q1), - {hi, _, Q3} = rabbit_fifo_q:out(Q2), - {lo, _, Q4} = rabbit_fifo_q:out(Q3), - {hi, _, Q5} = rabbit_fifo_q:out(Q4), - {lo, _, Q6} = rabbit_fifo_q:out(Q5), - {empty, _} = rabbit_fifo_q:out(Q6), + {?MSG(1), Q2} = rabbit_fifo_q:out(Q1), + {?MSG(3), Q3} = rabbit_fifo_q:out(Q2), + {?MSG(2), Q4} = rabbit_fifo_q:out(Q3), + {?MSG(5), Q5} = rabbit_fifo_q:out(Q4), + {?MSG(4), Q6} = rabbit_fifo_q:out(Q5), + empty = rabbit_fifo_q:out(Q6), ok. hi_is_prioritised(_Config) -> @@ -93,19 +93,18 @@ hi_is_prioritised(_Config) -> fun ({P, I}, Q) -> rabbit_fifo_q:in(P, I, Q) end, Q0, [ - {hi, ?MSG(1, ?LINE)}, - {hi, ?MSG(2, ?LINE)}, - {hi, ?MSG(3, ?LINE)}, - {hi, ?MSG(4, ?LINE)}, - {lo, ?MSG(5, ?LINE)} + {hi, ?MSG(1)}, + {hi, ?MSG(2)}, + {hi, ?MSG(3)}, + {hi, ?MSG(4)}, + {lo, ?MSG(5)} ]), - {hi, _, Q2} = rabbit_fifo_q:out(Q1), - {hi, _, Q3} = rabbit_fifo_q:out(Q2), - {hi, _, Q4} = rabbit_fifo_q:out(Q3), - {hi, _, Q5} = rabbit_fifo_q:out(Q4), - {lo, _, Q6} = rabbit_fifo_q:out(Q5), - {empty, _} = rabbit_fifo_q:out(Q6), - + {?MSG(1), Q2} = rabbit_fifo_q:out(Q1), + {?MSG(2), Q3} = rabbit_fifo_q:out(Q2), + {?MSG(3), Q4} = rabbit_fifo_q:out(Q3), + {?MSG(4), Q5} = rabbit_fifo_q:out(Q4), + {?MSG(5), Q6} = rabbit_fifo_q:out(Q5), + empty = rabbit_fifo_q:out(Q6), ok. get_lowest_index(_Config) -> @@ -113,9 +112,9 @@ get_lowest_index(_Config) -> Q1 = rabbit_fifo_q:in(hi, ?MSG(1, ?LINE), Q0), Q2 = rabbit_fifo_q:in(lo, ?MSG(2, ?LINE), Q1), Q3 = rabbit_fifo_q:in(lo, ?MSG(3, ?LINE), Q2), - {hi, _, Q4} = rabbit_fifo_q:out(Q3), - {lo, _, Q5} = rabbit_fifo_q:out(Q4), - {lo, _, Q6} = rabbit_fifo_q:out(Q5), + {_, Q4} = rabbit_fifo_q:out(Q3), + {_, Q5} = rabbit_fifo_q:out(Q4), + {_, Q6} = rabbit_fifo_q:out(Q5), ?assertEqual(undefined, rabbit_fifo_q:get_lowest_index(Q0)), ?assertEqual(1, rabbit_fifo_q:get_lowest_index(Q1)), @@ -151,14 +150,14 @@ queue_prop(P, Ops) -> end; (out, {Q0, S0}) -> {V1, Q} = case queue:out(Q0) of - {{value, V_}, Q1} -> - {V_, Q1}; + {{value, V0}, Q1} -> + {V0, Q1}; Res0 -> Res0 end, {V2, S} = case rabbit_fifo_q:out(S0) of - {_, V, S1} -> - {V, S1}; + empty -> + {empty, S0}; Res -> Res end, From 8ee4d749968b98538b2781b55ec9d6ad12c35cf3 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 16 Jul 2024 10:19:10 +0200 Subject: [PATCH 35/45] Add integration test for QQ SAC with consumer priority --- deps/rabbit/test/amqp_client_SUITE.erl | 141 ++++++++++++++++++++++++- 1 file changed, 139 insertions(+), 2 deletions(-) diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl index 8fda571ac3da..74d897790946 100644 --- a/deps/rabbit/test/amqp_client_SUITE.erl +++ b/deps/rabbit/test/amqp_client_SUITE.erl @@ -81,10 +81,13 @@ groups() -> stop_classic_queue, stop_quorum_queue, stop_stream, + priority_classic_queue, + priority_quorum_queue, consumer_priority_classic_queue, consumer_priority_quorum_queue, single_active_consumer_classic_queue, single_active_consumer_quorum_queue, + single_active_consumer_priority_quorum_queue, single_active_consumer_drain_classic_queue, single_active_consumer_drain_quorum_queue, detach_requeues_one_session_classic_queue, @@ -111,8 +114,6 @@ groups() -> handshake_timeout, credential_expires, attach_to_exclusive_queue, - priority_classic_queue, - priority_quorum_queue, dead_letter_headers_exchange, dead_letter_reject, dead_letter_reject_message_order_classic_queue, @@ -1965,6 +1966,142 @@ consumer_priority(QType, Config) -> ok = end_session_sync(Session), ok = amqp10_client:close_connection(Connection). +single_active_consumer_priority_quorum_queue(Config) -> + QType = <<"quorum">>, + QName = atom_to_binary(?FUNCTION_NAME), + {Connection, Session1, LinkPair} = init(Config), + QProps = #{arguments => #{<<"x-queue-type">> => {utf8, QType}, + <<"x-single-active-consumer">> => true}}, + {ok, #{type := QType}} = rabbitmq_amqp_client:declare_queue(LinkPair, QName, QProps), + + %% Send 6 messages. + Address = rabbitmq_amqp_address:queue(QName), + {ok, Sender} = amqp10_client:attach_sender_link(Session1, <<"test-sender">>, Address), + ok = wait_for_credit(Sender), + NumMsgs = 6, + [begin + Bin = integer_to_binary(N), + ok = amqp10_client:send_msg(Sender, amqp10_msg:new(Bin, Bin, true)) + end || N <- lists:seq(1, NumMsgs)], + ok = amqp10_client:detach_link(Sender), + + %% The 1st consumer (with default prio 0) will become active. + {ok, Recv1} = amqp10_client:attach_receiver_link( + Session1, <<"receiver 1">>, Address, unsettled), + receive {amqp10_event, {link, Recv1, attached}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + + {ok, Msg1} = amqp10_client:get_msg(Recv1), + ?assertEqual([<<"1">>], amqp10_msg:body(Msg1)), + + %% The 2nd consumer should take over thanks to higher prio. + {ok, Recv2} = amqp10_client:attach_receiver_link( + Session1, <<"receiver 2">>, Address, unsettled, none, #{}, + #{<<"rabbitmq:priority">> => {int, 1}}), + receive {amqp10_event, {link, Recv2, attached}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + flush("attched receiver 2"), + + %% To ensure in-order processing and to avoid interrupting the 1st consumer during + %% its long running task processing, neither of the 2 consumers should receive more + %% messages until the 1st consumer settles all outstanding messages. + ?assertEqual({error, timeout}, amqp10_client:get_msg(Recv1, 5)), + ?assertEqual({error, timeout}, amqp10_client:get_msg(Recv2, 5)), + ok = amqp10_client:accept_msg(Recv1, Msg1), + receive {amqp10_msg, R1, Msg2} -> + ?assertEqual([<<"2">>], amqp10_msg:body(Msg2)), + ?assertEqual(Recv2, R1), + ok = amqp10_client:accept_msg(Recv2, Msg2) + after 5000 -> ct:fail({missing_msg, ?LINE}) + end, + + %% Attaching with same prio should not take over. + {ok, Session2} = amqp10_client:begin_session_sync(Connection), + {ok, Recv3} = amqp10_client:attach_receiver_link( + Session2, <<"receiver 3">>, Address, unsettled, none, #{}, + #{<<"rabbitmq:priority">> => {int, 1}}), + receive {amqp10_event, {link, Recv3, attached}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + ?assertEqual({error, timeout}, amqp10_client:get_msg(Recv3, 5)), + ok = end_session_sync(Session2), + + {ok, Recv4} = amqp10_client:attach_receiver_link( + Session1, <<"receiver 4">>, Address, unsettled, none, #{}, + #{<<"rabbitmq:priority">> => {int, 1}}), + receive {amqp10_event, {link, Recv4, attached}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + + {ok, Recv5} = amqp10_client:attach_receiver_link( + Session1, <<"receiver 5">>, Address, unsettled, none, #{}, + #{<<"rabbitmq:priority">> => {int, 1}}), + receive {amqp10_event, {link, Recv5, attached}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + flush("attched receivers 4 and 5"), + + ok = amqp10_client:flow_link_credit(Recv4, 1, never), + ok = amqp10_client:flow_link_credit(Recv5, 2, never), + + %% Stop the active consumer. + ok = amqp10_client:detach_link(Recv2), + receive {amqp10_event, {link, Recv2, {detached, normal}}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + + %% The 5th consumer should become the active one because it is up, + %% has highest prio (1), and most credits (2). + receive {amqp10_msg, R2, Msg3} -> + ?assertEqual([<<"3">>], amqp10_msg:body(Msg3)), + ?assertEqual(Recv5, R2), + ok = amqp10_client:accept_msg(Recv5, Msg3) + after 5000 -> ct:fail({missing_msg, ?LINE}) + end, + receive {amqp10_msg, R3, Msg4} -> + ?assertEqual([<<"4">>], amqp10_msg:body(Msg4)), + ?assertEqual(Recv5, R3), + ok = amqp10_client:accept_msg(Recv5, Msg4) + after 5000 -> ct:fail({missing_msg, ?LINE}) + end, + + %% Stop the active consumer. + ok = amqp10_client:detach_link(Recv5), + receive {amqp10_event, {link, Recv5, {detached, normal}}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + + %% The 4th consumer should become the active one because it is up, + %% has highest prio (1), and most credits (1). + receive {amqp10_msg, R4, Msg5} -> + ?assertEqual([<<"5">>], amqp10_msg:body(Msg5)), + ?assertEqual(Recv4, R4), + ok = amqp10_client:accept_msg(Recv4, Msg5) + after 5000 -> ct:fail({missing_msg, ?LINE}) + end, + + %% Stop the active consumer. + ok = amqp10_client:detach_link(Recv4), + receive {amqp10_event, {link, Recv4, {detached, normal}}} -> ok + after 5000 -> ct:fail({missing_event, ?LINE}) + end, + + %% The only up consumer left is the 1st one (prio 0) which still has 1 credit. + receive {amqp10_msg, R5, Msg6} -> + ?assertEqual([<<"6">>], amqp10_msg:body(Msg6)), + ?assertEqual(Recv1, R5), + ok = amqp10_client:accept_msg(Recv1, Msg6) + after 5000 -> ct:fail({missing_msg, ?LINE}) + end, + + ok = amqp10_client:detach_link(Recv1), + {ok, #{message_count := 0}} = rabbitmq_amqp_client:delete_queue(LinkPair, QName), + ok = rabbitmq_amqp_client:detach_management_link_pair_sync(LinkPair), + ok = end_session_sync(Session1), + ok = amqp10_client:close_connection(Connection). + single_active_consumer_classic_queue(Config) -> single_active_consumer(<<"classic">>, Config). From 26c63327c656f1a33de1bcf5f24d32420298dee0 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Tue, 16 Jul 2024 15:23:07 +0200 Subject: [PATCH 36/45] Improve readability --- deps/rabbit/src/rabbit_core_ff.erl | 3 +- deps/rabbit/src/rabbit_fifo.erl | 58 ++++++++++++------------- deps/rabbit/src/rabbit_fifo_client.erl | 30 ++++++------- deps/rabbit/src/rabbit_quorum_queue.erl | 4 +- deps/rabbit/test/amqp_system_SUITE.erl | 31 +++++-------- 5 files changed, 57 insertions(+), 69 deletions(-) diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl index fc1f069a225d..0c077bdace02 100644 --- a/deps/rabbit/src/rabbit_core_ff.erl +++ b/deps/rabbit/src/rabbit_core_ff.erl @@ -195,7 +195,8 @@ -rabbit_feature_flag( {quorum_queues_v4, - #{desc => "Unlocks QQ v4 goodies", + #{desc => "Unlocks quorum queue v4 features", + doc_url => "https://github.com/rabbitmq/rabbitmq-server/pull/10637", stability => stable, depends_on => [quorum_queue, credit_api_v2] diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index a9ff09389f56..a51b26ee4fde 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -1400,10 +1400,10 @@ cancel_consumer0(Meta, ConsumerKey, activate_next_consumer({State, Effects}) -> activate_next_consumer(State, Effects). -activate_next_consumer(#?STATE{cfg = #cfg{consumer_strategy = competing}} = State0, - Effects0) -> - {State0, Effects0}; -activate_next_consumer(#?STATE{consumers = Cons, +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 @@ -1416,11 +1416,11 @@ activate_next_consumer(#?STATE{consumers = Cons, undefined end, - case {active_consumer(Cons), NextConsumer} of + case {active_consumer(Cons0), NextConsumer} of {undefined, {NextCKey, #consumer{cfg = NextCCfg} = NextC}} -> Remaining = tl(Waiting0), %% TODO: can this happen? - Consumer = case maps:get(NextCKey, Cons, undefined) of + Consumer = case maps:get(NextCKey, Cons0, undefined) of undefined -> NextC; Existing -> @@ -1433,7 +1433,7 @@ activate_next_consumer(#?STATE{consumers = Cons, ServiceQueue1 = maybe_queue_consumer(NextCKey, Consumer, ServiceQueue), - State = State0#?STATE{consumers = Cons#{NextCKey => Consumer}, + State = State0#?STATE{consumers = Cons0#{NextCKey => Consumer}, service_queue = ServiceQueue1, waiting_consumers = Remaining}, Effects = consumer_update_active_effects(State, Consumer, @@ -1452,11 +1452,12 @@ activate_next_consumer(#?STATE{consumers = Cons, ServiceQueue1 = maybe_queue_consumer(NextCKey, Consumer, ServiceQueue), - State = State0#?STATE{consumers = maps:remove(ActiveCKey, - Cons#{NextCKey => Consumer}), + 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 = - add_waiting({ActiveCKey, Active}, Remaining)}, + waiting_consumers = Waiting}, Effects = consumer_update_active_effects(State, Consumer, true, single_active, Effects0), @@ -1466,9 +1467,10 @@ activate_next_consumer(#?STATE{consumers = Cons, when WaitingPriority > ActivePriority -> %% A higher priority consumer has attached but the current one has %% pending messages - {State0#?STATE{consumers = - Cons#{ActiveCKey => Active#consumer{status = fading}}}, - Effects0}; + Cons = maps:update(ActiveCKey, + Active#consumer{status = fading}, + Cons0), + {State0#?STATE{consumers = Cons}, Effects0}; _ -> %% no activation {State0, Effects0} @@ -1504,10 +1506,10 @@ maybe_return_all(#{system_time := Ts} = Meta, ConsumerKey, status = cancelled}, S0), Effects0}; _ -> - {S1, Effects1} = return_all(Meta, S0, Effects0, ConsumerKey, Consumer), + {S1, Effects} = return_all(Meta, S0, Effects0, ConsumerKey, Consumer), {S1#?STATE{consumers = maps:remove(ConsumerKey, S1#?STATE.consumers), last_active = Ts}, - Effects1} + Effects} end. apply_enqueue(#{index := RaftIdx, @@ -1743,8 +1745,7 @@ update_header(Key, UpdateFun, Default, ?TUPLE(Size, Expiry)) update_header(Key, UpdateFun, Default, #{size => Size, expiry => Expiry}); update_header(Key, UpdateFun, Default, Header) - when is_map(Header) andalso - is_map_key(size, Header) -> + when is_map_key(size, Header) -> maps:update_with(Key, UpdateFun, Default, Header). get_msg_header(?MSG(_Idx, Header)) -> @@ -2172,8 +2173,8 @@ update_consumer(Meta, ConsumerKey, {Tag, Pid}, ConsumerMeta, {Life, Mode} = Spec, Priority, #?STATE{cfg = #cfg{consumer_strategy = single_active}, consumers = Cons0, - waiting_consumers = Waiting, - service_queue = _ServiceQueue0} = State0) -> + 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 @@ -2181,7 +2182,7 @@ update_consumer(Meta, ConsumerKey, {Tag, Pid}, ConsumerMeta, {ConsumerKey, #consumer{status = up} = Consumer0} -> Consumer = merge_consumer(Meta, Consumer0, ConsumerMeta, Spec, Priority), - {Consumer, update_or_remove_con(Meta, ConsumerKey, Consumer, State0)}; + {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 @@ -2189,7 +2190,7 @@ update_consumer(Meta, ConsumerKey, {Tag, Pid}, ConsumerMeta, Consumer0 = maps:get(ConsumerKey, Cons0), Consumer = merge_consumer(Meta, Consumer0, ConsumerMeta, Spec, Priority), - {Consumer, update_or_remove_con(Meta, ConsumerKey, Consumer, State0)}; + {Consumer, update_or_remove_con(Meta, ConsumerKey, Consumer, State)}; _ -> %% add as a new waiting consumer Credit = included_credit(Mode), @@ -2202,9 +2203,8 @@ update_consumer(Meta, ConsumerKey, {Tag, Pid}, ConsumerMeta, credit_mode = Mode}, credit = Credit, delivery_count = DeliveryCount}, - {Consumer, - State0#?STATE{waiting_consumers = - add_waiting({ConsumerKey, Consumer}, Waiting)}} + Waiting = add_waiting({ConsumerKey, Consumer}, Waiting0), + {Consumer, State#?STATE{waiting_consumers = Waiting}} end. add_waiting({Key, _} = New, Waiting) -> @@ -2496,10 +2496,8 @@ message_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}) -> @@ -2577,9 +2575,10 @@ get_priority(#{priority := Priority}) -> get_priority(#{args := Args}) -> %% fallback, v3 option case rabbit_misc:table_lookup(Args, <<"x-priority">>) of - {_Key, Value} -> + {_Type, Value} -> Value; - _ -> 0 + _ -> + 0 end; get_priority(_) -> 0. @@ -2713,7 +2712,6 @@ consumer_key_from_id(ConsumerId, {_, _, I}) -> consumer_key_from_id(_ConsumerId, none) -> error. - consumer_cancel_info(ConsumerKey, #?STATE{consumers = Consumers}) -> case Consumers of #{ConsumerKey := #consumer{checked_out = Checked}} -> diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index 513c248c95f9..1d88adca16f2 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -207,9 +207,9 @@ dequeue(QueueName, ConsumerTag, Settlement, {ok, {dequeue, empty}, Leader} -> {empty, State0#state{leader = Leader}}; {ok, {dequeue, {MsgId, {MsgHeader, Msg0}}, MsgsReady}, Leader} -> - {Msg, IsDelivered} = add_delivery_count_header(Msg0, MsgHeader), + {Msg, Redelivered} = add_delivery_count_header(Msg0, MsgHeader), {ok, MsgsReady, - {QueueName, qref(Leader), MsgId, IsDelivered, Msg}, + {QueueName, qref(Leader), MsgId, Redelivered, Msg}, State0#state{leader = Leader}}; {ok, {error, _} = Err, _Leader} -> Err; @@ -217,23 +217,23 @@ dequeue(QueueName, ConsumerTag, Settlement, Err end. -add_delivery_count_header(Msg, #{delivery_count := DelCount} = Header) +add_delivery_count_header(Msg0, #{delivery_count := DelCount} = Header) when is_integer(DelCount) -> - {case mc:is(Msg) of - true -> - %% the "delivery-count" header in the AMQP spec does not include - %% returns (released outcomes) - AmqpDelCount = DelCount - maps:get(return_count, Header, 0), - mc:set_annotation(delivery_count, AmqpDelCount, - mc:set_annotation(<<"x-delivery-count">>, - DelCount, Msg)); - _ -> - Msg - end, DelCount > 0}; + Msg = case mc:is(Msg0) of + true -> + Msg1 = mc:set_annotation(<<"x-delivery-count">>, DelCount, Msg0), + %% the "delivery-count" header in the AMQP spec does not include + %% returns (released outcomes) + AmqpDelCount = DelCount - maps:get(return_count, Header, 0), + mc:set_annotation(delivery_count, AmqpDelCount, Msg1); + false -> + Msg0 + end, + Redelivered = DelCount > 0, + {Msg, Redelivered}; add_delivery_count_header(Msg, _Header) -> {Msg, false}. - %% @doc Settle a message. Permanently removes message from the queue. %% @param ConsumerTag the tag uniquely identifying the consumer. %% @param MsgIds the message ids received with the {@link rabbit_fifo:delivery/0.} diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index a2b1da64d9b6..b782845fca21 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -886,11 +886,11 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) -> _ -> 0 end, - Priority = case rabbit_misc:table_lookup(Args, <<"x-priority">>) of {_Key, Value} -> Value; - _ -> 0 + _ -> + 0 end, ConsumerMeta = #{ack => AckRequired, prefetch => Prefetch, diff --git a/deps/rabbit/test/amqp_system_SUITE.erl b/deps/rabbit/test/amqp_system_SUITE.erl index d94a590be397..2d4e728fb6ca 100644 --- a/deps/rabbit/test/amqp_system_SUITE.erl +++ b/deps/rabbit/test/amqp_system_SUITE.erl @@ -69,11 +69,16 @@ init_per_group(Group, Config) -> dotnet -> fun build_dotnet_test_project/1; java -> fun build_maven_test_project/1 end, - rabbit_ct_helpers:run_setup_steps(Config1, [ - GroupSetupStep - ] ++ - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()). + Config2 = rabbit_ct_helpers:run_setup_steps( + Config1, + [GroupSetupStep] ++ + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()), + ok = rabbit_ct_broker_helpers:enable_feature_flag( + Config2, message_containers_store_amqp_v1), + ok = rabbit_ct_broker_helpers:enable_feature_flag( + Config2, quorum_queues_v4), + Config2. end_per_group(_, Config) -> rabbit_ct_helpers:run_teardown_steps(Config, @@ -81,12 +86,6 @@ end_per_group(_, Config) -> rabbit_ct_broker_helpers:teardown_steps()). init_per_testcase(Testcase, Config) -> - enable_feature_flags(Config, - [ - message_containers_store_amqp_v1, - credit_api_v2, - quorum_queues_v4 - ]), rabbit_ct_helpers:testcase_started(Config, Testcase). end_per_testcase(Testcase, Config) -> @@ -271,16 +270,6 @@ run_java_test(Config, Class) -> [{cd, TestProjectDir}]), {ok, _} = Ret. - -enable_feature_flags(Config, Flags) -> - [begin - case rabbit_ct_broker_helpers:enable_feature_flag(Config, Flag) of - ok -> ok; - _ -> - throw({skip, "feature flag ~s could not be enabled"}) - end - end || Flag <- Flags]. - declare_queue(Config, Name, Type) -> Ch = rabbit_ct_client_helpers:open_channel(Config), #'queue.declare_ok'{} = From f3de471dfeed0e349c85a8cafe502eb550a8d90a Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 17 Jul 2024 10:17:33 +0200 Subject: [PATCH 37/45] Change modified outcome behaviour With the new quorum queue v4 improvements where a requeue counter was added in addition to the quorum queue delivery counter, the following sentence from https://github.com/rabbitmq/rabbitmq-server/pull/6292#issue-1431275848 doesn't apply anymore: > Also the case where delivery_failed=false|undefined requires the release of the > message without incrementing the delivery_count. Again this is not something > that our queues are able to do so again we have to reject without requeue. Therefore, we simplify the modified outcome behaviour: RabbitMQ will from now on only discard the message if the modified's undeliverable-here field is true. --- deps/amqp10_client/src/amqp10_msg.erl | 3 +- deps/rabbit/src/rabbit_amqp_session.erl | 27 ++++++------- deps/rabbit/test/amqp_client_SUITE.erl | 51 +++++++++++++++++++++++++ 3 files changed, 67 insertions(+), 14 deletions(-) diff --git a/deps/amqp10_client/src/amqp10_msg.erl b/deps/amqp10_client/src/amqp10_msg.erl index 91a7efebe329..fa046cc60657 100644 --- a/deps/amqp10_client/src/amqp10_msg.erl +++ b/deps/amqp10_client/src/amqp10_msg.erl @@ -193,7 +193,8 @@ header(first_acquirer = K, header(delivery_count = K, #amqp10_msg{header = #'v1_0.header'{delivery_count = D}}) -> header_value(K, D); -header(K, #amqp10_msg{header = undefined}) -> header_value(K, undefined). +header(K, #amqp10_msg{header = undefined}) -> + header_value(K, undefined). -spec delivery_annotations(amqp10_msg()) -> #{annotations_key() => any()}. delivery_annotations(#amqp10_msg{delivery_annotations = undefined}) -> diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl index 4e0029b02ba1..c60a124c61c5 100644 --- a/deps/rabbit/src/rabbit_amqp_session.erl +++ b/deps/rabbit/src/rabbit_amqp_session.erl @@ -1861,20 +1861,21 @@ settle_op_from_outcome(#'v1_0.rejected'{}) -> discard; settle_op_from_outcome(#'v1_0.released'{}) -> requeue; -%% Keep the same Modified behaviour as in RabbitMQ 3.x -settle_op_from_outcome(#'v1_0.modified'{delivery_failed = true, - undeliverable_here = UndelHere}) - when UndelHere =/= true -> - requeue; -settle_op_from_outcome(#'v1_0.modified'{}) -> - %% If delivery_failed is not true, we can't increment its delivery_count. - %% So, we will have to reject without requeue. - %% - %% If undeliverable_here is true, this is not quite correct because - %% undeliverable_here refers to the link, and not the message in general. - %% However, we cannot filter messages from being assigned to individual consumers. - %% That's why we will have to reject it without requeue. + +%% RabbitMQ does not support any of the modified outcome fields correctly. +%% However, we still allow the client to settle with the modified outcome +%% because some client libraries such as Apache QPid make use of it: +%% https://github.com/apache/qpid-jms/blob/90eb60f59cb59b7b9ad8363ee8a843d6903b8e77/qpid-jms-client/src/main/java/org/apache/qpid/jms/JmsMessageConsumer.java#L464 +%% In such cases, it's better when RabbitMQ does not end the session. +%% See https://github.com/rabbitmq/rabbitmq-server/issues/6121 +settle_op_from_outcome(#'v1_0.modified'{undeliverable_here = true}) -> + %% This is not quite correct because undeliverable_here refers to the link, + %% and not the message in general. However, RabbitMQ cannot filter messages from + %% being assigned to individual consumers. That's why we discard. discard; +settle_op_from_outcome(#'v1_0.modified'{}) -> + requeue; + settle_op_from_outcome(Outcome) -> protocol_error( ?V_1_0_AMQP_ERROR_INVALID_FIELD, diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl index 74d897790946..c56c669b2753 100644 --- a/deps/rabbit/test/amqp_client_SUITE.erl +++ b/deps/rabbit/test/amqp_client_SUITE.erl @@ -41,6 +41,7 @@ groups() -> [ reliable_send_receive_with_outcomes_classic_queue, reliable_send_receive_with_outcomes_quorum_queue, + modified, sender_settle_mode_unsettled, sender_settle_mode_unsettled_fanout, sender_settle_mode_mixed, @@ -402,6 +403,56 @@ reliable_send_receive(QType, Outcome, Config) -> ok = end_session_sync(Session2), ok = amqp10_client:close_connection(Connection2). +%% This test case doesn't expect the correct AMQP spec behavivour. +%% We know that RabbitMQ doesn't implement the modified outcome correctly. +%% Here, we test RabbitMQ's workaround behaviour: +%% RabbitMQ discards if undeliverable-here is true. Otherwise, RabbitMQ requeues. +modified(Config) -> + QName = atom_to_binary(?FUNCTION_NAME), + {Connection, Session, LinkPair} = init(Config), + {ok, #{type := <<"quorum">>}} = rabbitmq_amqp_client:declare_queue( + LinkPair, QName, + #{arguments => #{<<"x-queue-type">> => {utf8, <<"quorum">>}}}), + Address = rabbitmq_amqp_address:queue(QName), + {ok, Sender} = amqp10_client:attach_sender_link(Session, <<"sender">>, Address), + ok = wait_for_credit(Sender), + + Msg1 = amqp10_msg:new(<<"tag1">>, <<"m1">>, true), + Msg2 = amqp10_msg:new(<<"tag2">>, <<"m2">>, true), + ok = amqp10_client:send_msg(Sender, Msg1), + ok = amqp10_client:send_msg(Sender, Msg2), + ok = amqp10_client:detach_link(Sender), + + {ok, Receiver} = amqp10_client:attach_receiver_link(Session, <<"receiver">>, Address, unsettled), + + {ok, M1} = amqp10_client:get_msg(Receiver), + ?assertEqual([<<"m1">>], amqp10_msg:body(M1)), + ok = amqp10_client:settle_msg(Receiver, M1, {modified, false, _UndeliverableHere = true, #{}}), + + {ok, M2a} = amqp10_client:get_msg(Receiver), + ?assertEqual([<<"m2">>], amqp10_msg:body(M2a)), + ok = amqp10_client:settle_msg(Receiver, M2a, {modified, false, false, #{}}), + + {ok, M2b} = amqp10_client:get_msg(Receiver), + ?assertEqual([<<"m2">>], amqp10_msg:body(M2b)), + ok = amqp10_client:settle_msg(Receiver, M2b, {modified, true, false, #{}}), + + {ok, M2c} = amqp10_client:get_msg(Receiver), + ?assertEqual([<<"m2">>], amqp10_msg:body(M2c)), + ok = amqp10_client:settle_msg(Receiver, M2c, {modified, true, false, #{<<"key">> => <<"val">>}}), + + {ok, M2d} = amqp10_client:get_msg(Receiver), + ?assertEqual([<<"m2">>], amqp10_msg:body(M2d)), + ?assertEqual(0, amqp10_msg:header(delivery_count, M2d)), + ok = amqp10_client:settle_msg(Receiver, M2d, modified), + + ok = amqp10_client:detach_link(Receiver), + ?assertMatch({ok, #{message_count := 1}}, + rabbitmq_amqp_client:delete_queue(LinkPair, QName)), + ok = rabbitmq_amqp_client:detach_management_link_pair_sync(LinkPair), + ok = end_session_sync(Session), + ok = amqp10_client:close_connection(Connection). + %% Tests that confirmations are returned correctly %% when sending many messages async to a quorum queue. sender_settle_mode_unsettled(Config) -> From 33610364260c99c8d6565dc76b43454a015fdfa0 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 17 Jul 2024 17:29:19 +0200 Subject: [PATCH 38/45] Introduce single feature flag rabbitmq_4.0.0 ## What? Merge all feature flags introduced in RabbitMQ 4.0.0 into a single feature flag called rabbitmq_4.0.0. ## Why? 1. This fixes the crash in https://github.com/rabbitmq/rabbitmq-server/pull/10637#discussion_r1681002352 2. It's better user experience. --- deps/rabbit/src/mc_amqpl.erl | 2 +- deps/rabbit/src/rabbit_amqp_session.erl | 21 ++++++----- deps/rabbit/src/rabbit_amqp_writer.erl | 6 +-- deps/rabbit/src/rabbit_amqqueue_process.erl | 2 +- deps/rabbit/src/rabbit_classic_queue.erl | 16 ++++---- deps/rabbit/src/rabbit_core_ff.erl | 37 ++++++------------- deps/rabbit/src/rabbit_fifo.erl | 4 +- deps/rabbit/src/rabbit_fifo_client.erl | 4 +- deps/rabbit/src/rabbit_queue_consumers.erl | 4 +- deps/rabbit/src/rabbit_queue_type.erl | 6 +-- deps/rabbit/test/amqp_client_SUITE.erl | 26 ++++++------- deps/rabbit/test/amqp_credit_api_v2_SUITE.erl | 20 +++------- deps/rabbit/test/amqp_system_SUITE.erl | 5 +-- deps/rabbit/test/classic_queue_SUITE.erl | 2 +- deps/rabbit/test/quorum_queue_SUITE.erl | 17 ++++----- deps/rabbit/test/rabbit_fifo_prop_SUITE.erl | 2 +- .../test/protocol_interop_SUITE.erl | 4 +- 17 files changed, 75 insertions(+), 103 deletions(-) diff --git a/deps/rabbit/src/mc_amqpl.erl b/deps/rabbit/src/mc_amqpl.erl index f1b023d3fe79..8de27294723a 100644 --- a/deps/rabbit/src/mc_amqpl.erl +++ b/deps/rabbit/src/mc_amqpl.erl @@ -176,7 +176,7 @@ convert_from(mc_amqp, Sections, Env) -> {Headers2, CorrId091} = message_id(CorrId, <<"x-correlation-id">>, Headers1), Headers = case Env of - #{message_containers_store_amqp_v1 := false} -> + #{'rabbitmq_4.0.0' := false} -> Headers3 = case AProp of undefined -> Headers2; diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl index c60a124c61c5..abfbc90f4ac6 100644 --- a/deps/rabbit/src/rabbit_amqp_session.erl +++ b/deps/rabbit/src/rabbit_amqp_session.erl @@ -187,7 +187,7 @@ send_settled :: boolean(), max_message_size :: unlimited | pos_integer(), - %% When feature flag credit_api_v2 becomes required, + %% When feature flag rabbitmq_4.0.0 becomes required, %% the following 2 fields should be deleted. credit_api_version :: 1 | 2, %% When credit API v1 is used, our session process holds the delivery-count @@ -225,7 +225,7 @@ frames :: [transfer_frame_body(), ...], queue_ack_required :: boolean(), %% Queue that sent us this message. - %% When feature flag credit_api_v2 becomes required, this field should be deleted. + %% When feature flag rabbitmq_4.0.0 becomes required, this field should be deleted. queue_pid :: pid() | credit_api_v2, delivery_id :: delivery_number(), outgoing_unsettled :: #outgoing_unsettled{} @@ -1068,17 +1068,17 @@ handle_control(#'v1_0.attach'{role = ?AMQP_ROLE_RECEIVER, QType = amqqueue:get_type(Q), %% Whether credit API v1 or v2 is used is decided only here at link attachment time. %% This decision applies to the whole life time of the link. - %% This means even when feature flag credit_api_v2 will be enabled later, this consumer will + %% This means even when feature flag rabbitmq_4.0.0 will be enabled later, this consumer will %% continue to use credit API v1. This is the safest and easiest solution avoiding %% transferring link flow control state (the delivery-count) at runtime from this session %% process to the queue process. - %% Eventually, after feature flag credit_api_v2 gets enabled and a subsequent rolling upgrade, + %% Eventually, after feature flag rabbitmq_4.0.0 gets enabled and a subsequent rolling upgrade, %% all consumers will use credit API v2. %% Streams always use credit API v2 since the stream client (rabbit_stream_queue) holds the link %% flow control state. Hence, credit API mixed version isn't an issue for streams. {CreditApiVsn, Mode, DeliveryCount, ClientFlowCtl, QueueFlowCtl, CreditReqInFlight, StashedCreditReq} = - case rabbit_feature_flags:is_enabled(credit_api_v2) orelse + case rabbit_feature_flags:is_enabled('rabbitmq_4.0.0') orelse QType =:= rabbit_stream_queue of true -> {2, @@ -1982,7 +1982,7 @@ handle_queue_actions(Actions, State) -> S0 = #state{outgoing_links = OutgoingLinks0, outgoing_pending = Pending}) -> %% credit API v1 - %% Delete this branch when feature flag credit_api_v2 becomes required. + %% Delete this branch when feature flag rabbitmq_4.0.0 becomes required. Handle = ctag_to_handle(Ctag), Link = #outgoing_link{delivery_count = Count0} = maps:get(Handle, OutgoingLinks0), {Count, Credit, S} = case Drain of @@ -2789,7 +2789,7 @@ delivery_count_rcv(undefined) -> %% credits to a queue has to synchronously wait for a credit reply from the queue: %% https://github.com/rabbitmq/rabbitmq-server/blob/b9566f4d02f7ceddd2f267a92d46affd30fb16c8/deps/rabbitmq_codegen/credit_extension.json#L43 %% This blocks our entire AMQP 1.0 session process. Since the credit reply from the -%% queue did not contain the consumr tag prior to feature flag credit_api_v2, we +%% queue did not contain the consumr tag prior to feature flag rabbitmq_4.0.0, we %% must behave here the same way as non-native AMQP 1.0: We wait until the queue %% sends us a credit reply sucht that we can correlate that reply with our consumer tag. process_credit_reply_sync( @@ -2854,7 +2854,7 @@ process_credit_reply_sync_quorum_queue(Ctag, QName, Credit, State0) -> no_return(). credit_reply_timeout(QType, QName) -> Fmt = "Timed out waiting for credit reply from ~s ~s. " - "Hint: Enable feature flag credit_api_v2", + "Hint: Enable feature flag rabbitmq_4.0.0", Args = [QType, rabbit_misc:rs(QName)], rabbit_log:error(Fmt, Args), protocol_error(?V_1_0_AMQP_ERROR_INTERNAL_ERROR, Fmt, Args). @@ -3442,12 +3442,13 @@ cap_credit(DesiredCredit) -> min(DesiredCredit, MaxCredit). ensure_mc_cluster_compat(Mc) -> - IsEnabled = rabbit_feature_flags:is_enabled(message_containers_store_amqp_v1), + Feature = 'rabbitmq_4.0.0', + IsEnabled = rabbit_feature_flags:is_enabled(Feature), case IsEnabled of true -> Mc; false -> - McEnv = #{message_containers_store_amqp_v1 => IsEnabled}, + McEnv = #{Feature => IsEnabled}, %% other nodes in the cluster may not understand the new internal %% amqp mc format - in this case we convert to AMQP legacy format %% for compatibility diff --git a/deps/rabbit/src/rabbit_amqp_writer.erl b/deps/rabbit/src/rabbit_amqp_writer.erl index c3840d5468d2..7b239a10a107 100644 --- a/deps/rabbit/src/rabbit_amqp_writer.erl +++ b/deps/rabbit/src/rabbit_amqp_writer.erl @@ -74,7 +74,7 @@ send_command_sync(Writer, ChannelNum, Performative) -> Request = {send_command, ChannelNum, Performative}, gen_server:call(Writer, Request, ?CALL_TIMEOUT). -%% Delete this function when feature flag credit_api_v2 becomes required. +%% Delete this function when feature flag rabbitmq_4.0.0 becomes required. -spec send_command_and_notify(pid(), pid(), rabbit_types:channel_number(), @@ -111,7 +111,7 @@ handle_cast({send_command, SessionPid, ChannelNum, Performative, Payload}, State State1 = internal_send_command_async(ChannelNum, Performative, Payload, State0), State = credit_flow_ack(SessionPid, State1), no_reply(State); -%% Delete below function clause when feature flag credit_api_v2 becomes required. +%% Delete below function clause when feature flag rabbitmq_4.0.0 becomes required. handle_cast({send_command_and_notify, QueuePid, SessionPid, ChannelNum, Performative, Payload}, State0) -> State1 = internal_send_command_async(ChannelNum, Performative, Payload, State0), State = credit_flow_ack(SessionPid, State1), @@ -131,7 +131,7 @@ handle_info({{'DOWN', session}, _MRef, process, SessionPid, _Reason}, credit_flow:peer_down(SessionPid), State = State0#state{monitored_sessions = maps:remove(SessionPid, Sessions)}, no_reply(State); -%% Delete below function clause when feature flag credit_api_v2 becomes required. +%% Delete below function clause when feature flag rabbitmq_4.0.0 becomes required. handle_info({'DOWN', _MRef, process, QueuePid, _Reason}, State) -> rabbit_amqqueue:notify_sent_queue_down(QueuePid), no_reply(State). diff --git a/deps/rabbit/src/rabbit_amqqueue_process.erl b/deps/rabbit/src/rabbit_amqqueue_process.erl index da9c1751f8b0..e2334235c335 100644 --- a/deps/rabbit/src/rabbit_amqqueue_process.erl +++ b/deps/rabbit/src/rabbit_amqqueue_process.erl @@ -1516,7 +1516,7 @@ handle_cast({credit, SessionPid, CTag, Credit, Drain}, backing_queue = BQ, backing_queue_state = BQS0} = State) -> %% Credit API v1. - %% Delete this function clause when feature flag credit_api_v2 becomes required. + %% Delete this function clause when feature flag rabbitmq_4.0.0 becomes required. %% Behave like non-native AMQP 1.0: Send send_credit_reply before deliveries. rabbit_classic_queue:send_credit_reply_credit_api_v1( SessionPid, amqqueue:get_name(Q), BQ:len(BQS0)), diff --git a/deps/rabbit/src/rabbit_classic_queue.erl b/deps/rabbit/src/rabbit_classic_queue.erl index 5878347349d2..7eb09637f77b 100644 --- a/deps/rabbit/src/rabbit_classic_queue.erl +++ b/deps/rabbit/src/rabbit_classic_queue.erl @@ -297,9 +297,9 @@ consume(Q, Spec, State0) when ?amqqueue_is_classic(Q) -> Err end. -%% Delete this function when feature flag credit_api_v2 becomes required. +%% Delete this function when feature flag rabbitmq_4.0.0 becomes required. consume_backwards_compat({simple_prefetch, PrefetchCount} = Mode, Args) -> - case rabbit_feature_flags:is_enabled(credit_api_v2) of + case rabbit_feature_flags:is_enabled('rabbitmq_4.0.0') of true -> {Mode, Args}; false -> {PrefetchCount, Args} end; @@ -314,8 +314,8 @@ consume_backwards_compat({credited, credit_api_v1}, Args) -> {<<"drain">>, bool, false}]} | Args]}. cancel(Q, Spec, State) -> - %% Cancel API v2 reuses feature flag credit_api_v2. - Request = case rabbit_feature_flags:is_enabled(credit_api_v2) of + %% Cancel API v2 reuses feature flag rabbitmq_4.0.0. + Request = case rabbit_feature_flags:is_enabled('rabbitmq_4.0.0') of true -> {stop_consumer, Spec#{pid => self()}}; false -> @@ -413,7 +413,7 @@ handle_event(_QName, Action, State) {ok, State, [Action]}; handle_event(_QName, {send_drained, {Ctag, Credit}}, State) -> %% This function clause should be deleted when feature flag - %% credit_api_v2 becomes required. + %% rabbitmq_4.0.0 becomes required. Action = {credit_reply_v1, Ctag, Credit, _Available = 0, _Drain = true}, {ok, State, [Action]}. @@ -568,7 +568,7 @@ capabilities() -> <<"x-max-length-bytes">>, <<"x-max-priority">>, <<"x-overflow">>, <<"x-queue-mode">>, <<"x-queue-version">>, <<"x-single-active-consumer">>, <<"x-queue-type">>, <<"x-queue-master-locator">>] - ++ case rabbit_feature_flags:is_enabled(classic_queue_leader_locator) of + ++ case rabbit_feature_flags:is_enabled('rabbitmq_4.0.0') of true -> [<<"x-queue-leader-locator">>]; false -> [] end, @@ -645,12 +645,12 @@ deliver_to_consumer(Pid, QName, CTag, AckRequired, Message) -> Evt = {deliver, CTag, AckRequired, [Message]}, send_queue_event(Pid, QName, Evt). -%% Delete this function when feature flag credit_api_v2 becomes required. +%% Delete this function when feature flag rabbitmq_4.0.0 becomes required. send_credit_reply_credit_api_v1(Pid, QName, Available) -> Evt = {send_credit_reply, Available}, send_queue_event(Pid, QName, Evt). -%% Delete this function when feature flag credit_api_v2 becomes required. +%% Delete this function when feature flag rabbitmq_4.0.0 becomes required. send_drained_credit_api_v1(Pid, QName, Ctag, Credit) -> Evt = {send_drained, {Ctag, Credit}}, send_queue_event(Pid, QName, Evt). diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl index 0c077bdace02..6501ddb8da65 100644 --- a/deps/rabbit/src/rabbit_core_ff.erl +++ b/deps/rabbit/src/rabbit_core_ff.erl @@ -165,19 +165,6 @@ depends_on => [quorum_queue] }}). --rabbit_feature_flag( - {credit_api_v2, - #{desc => "Credit and cancel API v2 between queue clients and queue processes", - stability => stable - }}). - --rabbit_feature_flag( - {message_containers_store_amqp_v1, - #{desc => "Support storing messages in message containers AMQP 1.0 disk format v1", - stability => stable, - depends_on => [message_containers] - }}). - -rabbit_feature_flag( {message_containers_deaths_v2, #{desc => "Bug fix for dead letter cycle detection", @@ -186,18 +173,16 @@ depends_on => [message_containers] }}). +%% We bundle the following separate concerns (which could have been separate feature flags) +%% into a single feature flag for better user experience: +%% 1. credit API v2 between classic / quorum queue client and classic / quorum queue server +%% 2. cancel API v2 betweeen classic queue client and classic queue server +%% 3. more compact quorum queue commands in quorum queue v4 +%% 4. store messages in message containers AMQP 1.0 disk format v1 +%% 5. support queue leader locator in classic queues -rabbit_feature_flag( - {classic_queue_leader_locator, - #{desc => "queue-leader-locator support in classic queues", - doc_url => "https://www.rabbitmq.com/docs/clustering#replica-placement", - stability => stable - }}). - --rabbit_feature_flag( - {quorum_queues_v4, - #{desc => "Unlocks quorum queue v4 features", - doc_url => "https://github.com/rabbitmq/rabbitmq-server/pull/10637", - stability => stable, - depends_on => [quorum_queue, - credit_api_v2] + {'rabbitmq_4.0.0', + #{desc => "Allows rolling upgrades from 3.13.x to 4.0.x", + stability => stable, + depends_on => [message_containers] }}). diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index a51b26ee4fde..6084f3f9b9d0 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -1281,8 +1281,10 @@ usage(Name) when is_atom(Name) -> [{_, Use}] -> Use end. +-spec is_v4() -> boolean(). is_v4() -> - rabbit_feature_flags:is_enabled(quorum_queues_v4). + %% Quorum queue v4 is introduced in RabbitMQ 4.0.0 + rabbit_feature_flags:is_enabled('rabbitmq_4.0.0'). %%% Internal diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index 1d88adca16f2..9cdd92203480 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -46,7 +46,7 @@ % status = up :: up | cancelled, last_msg_id :: seq() | -1 | undefined, ack = false :: boolean(), - %% Remove this field when feature flag credit_api_v2 becomes required. + %% Remove this field when feature flag rabbitmq_4.0.0 becomes required. delivery_count :: {credit_api_v1, rabbit_queue_type:delivery_count()} | credit_api_v2 }). @@ -734,7 +734,7 @@ maybe_add_action({multi, Actions}, Acc0, State0) -> end, {Acc0, State0}, Actions); maybe_add_action({send_drained, {Tag, Credit}}, Acc, State0) -> %% This function clause should be deleted when - %% feature flag credit_api_v2 becomes required. + %% feature flag rabbitmq_4.0.0 becomes required. State = add_delivery_count(Credit, Tag, State0), Action = {credit_reply_v1, Tag, Credit, _Avail = 0, _Drain = true}, {[Action | Acc], State}; diff --git a/deps/rabbit/src/rabbit_queue_consumers.erl b/deps/rabbit/src/rabbit_queue_consumers.erl index 7a95582a6551..a36efe3cb94c 100644 --- a/deps/rabbit/src/rabbit_queue_consumers.erl +++ b/deps/rabbit/src/rabbit_queue_consumers.erl @@ -33,7 +33,7 @@ -record(consumer, {tag, ack_required, prefetch, args, user}). %% AMQP 1.0 link flow control state, see §2.6.7 -%% Delete atom credit_api_v1 when feature flag credit_api_v2 becomes required. +%% Delete atom credit_api_v1 when feature flag rabbitmq_4.0.0 becomes required. -record(link_state, {delivery_count :: rabbit_queue_type:delivery_count() | credit_api_v1, credit :: rabbit_queue_type:credit()}). @@ -596,7 +596,7 @@ parse_credit_mode({simple_prefetch, Prefetch}, _Args) -> parse_credit_mode({credited, InitialDeliveryCount}, _Args) -> {InitialDeliveryCount, manual}; %% credit API v1 -%% i.e. below function clause should be deleted when feature flag credit_api_v2 becomes required: +%% i.e. below function clause should be deleted when feature flag rabbitmq_4.0.0 becomes required: parse_credit_mode(Prefetch, Args) when is_integer(Prefetch) -> case rabbit_misc:table_lookup(Args, <<"x-credit">>) of diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index e5ab058944bb..45184963ac8f 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -114,7 +114,7 @@ -opaque state() :: #?STATE{}. -%% Delete atom 'credit_api_v1' when feature flag credit_api_v2 becomes required. +%% Delete atom 'credit_api_v1' when feature flag rabbitmq_4.0.0 becomes required. -type consume_mode() :: {simple_prefetch, Prefetch :: non_neg_integer()} | {credited, Initial :: delivery_count() | credit_api_v1}. -type consume_spec() :: #{no_ack := boolean(), @@ -227,7 +227,7 @@ {queue_state(), actions()} | {'protocol_error', Type :: atom(), Reason :: string(), Args :: term()}. -%% Delete this callback when feature flag credit_api_v2 becomes required. +%% Delete this callback when feature flag rabbitmq_4.0.0 becomes required. -callback credit_v1(queue_name(), rabbit_types:ctag(), credit(), Drain :: boolean(), queue_state()) -> {queue_state(), actions()}. @@ -709,7 +709,7 @@ settle(#resource{kind = queue} = QRef, Op, CTag, MsgIds, Ctxs) -> end end. -%% Delete this function when feature flag credit_api_v2 becomes required. +%% Delete this function when feature flag rabbitmq_4.0.0 becomes required. -spec credit_v1(queue_name(), rabbit_types:ctag(), credit(), boolean(), state()) -> {ok, state(), actions()}. credit_v1(QName, CTag, LinkCreditSnd, Drain, Ctxs) -> diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl index c56c669b2753..4a813c01a03b 100644 --- a/deps/rabbit/test/amqp_client_SUITE.erl +++ b/deps/rabbit/test/amqp_client_SUITE.erl @@ -214,7 +214,7 @@ init_per_testcase(T, Config) T =:= drain_many_quorum_queue orelse T =:= timed_get_quorum_queue orelse T =:= available_messages_quorum_queue -> - case rpc(Config, rabbit_feature_flags, is_enabled, [credit_api_v2]) of + case rpc(Config, rabbit_feature_flags, is_enabled, ['rabbitmq_4.0.0']) of true -> rabbit_ct_helpers:testcase_started(Config, T); false -> @@ -222,7 +222,7 @@ init_per_testcase(T, Config) "bug that they reply with send_drained before delivering the message."} end; init_per_testcase(single_active_consumer_drain_quorum_queue = T, Config) -> - case rpc(Config, rabbit_feature_flags, is_enabled, [credit_api_v2]) of + case rpc(Config, rabbit_feature_flags, is_enabled, ['rabbitmq_4.0.0']) of true -> rabbit_ct_helpers:testcase_started(Config, T); false -> @@ -238,47 +238,47 @@ init_per_testcase(T, Config) %% The new RabbitMQ internal flow control %% writer proc <- session proc <- queue proc %% is only available with credit API v2. - case rpc(Config, rabbit_feature_flags, is_enabled, [credit_api_v2]) of + case rpc(Config, rabbit_feature_flags, is_enabled, ['rabbitmq_4.0.0']) of true -> rabbit_ct_helpers:testcase_started(Config, T); false -> - {skip, "Feature flag credit_api_v2 is disabled"} + {skip, "Feature flag rabbitmq_4.0.0 is disabled"} end; init_per_testcase(T, Config) when T =:= detach_requeues_one_session_classic_queue orelse T =:= detach_requeues_drop_head_classic_queue orelse T =:= detach_requeues_two_connections_classic_queue orelse T =:= single_active_consumer_classic_queue -> - %% Cancel API v2 reuses feature flag credit_api_v2. + %% Cancel API v2 reuses feature flag rabbitmq_4.0.0. %% In 3.13, with cancel API v1, when a receiver detaches with unacked messages, these messages %% will remain unacked and unacked message state will be left behind in the server session %% process state. %% In contrast, cancel API v2 in 4.x will requeue any unacked messages if the receiver detaches. %% We skip the single active consumer tests because these test cases assume that detaching a %% receiver link will requeue unacked messages. - case rpc(Config, rabbit_feature_flags, is_enabled, [credit_api_v2]) of + case rpc(Config, rabbit_feature_flags, is_enabled, ['rabbitmq_4.0.0']) of true -> rabbit_ct_helpers:testcase_started(Config, T); false -> - {skip, "Cancel API v2 is disabled due to feature flag credit_api_v2 being disabled."} + {skip, "Cancel API v2 is disabled due to feature flag rabbitmq_4.0.0 being disabled."} end; init_per_testcase(T, Config) when T =:= detach_requeues_one_session_quorum_queue orelse T =:= single_active_consumer_quorum_queue orelse T =:= detach_requeues_two_connections_quorum_queue -> - case rabbit_ct_broker_helpers:enable_feature_flag(Config, quorum_queues_v4) of + case rabbit_ct_broker_helpers:enable_feature_flag(Config, 'rabbitmq_4.0.0') of ok -> rabbit_ct_helpers:testcase_started(Config, T); {skip, _} -> - {skip, "Feature flag quorum_queues_v4 enables the consumer removal API"} + {skip, "Feature flag rabbitmq_4.0.0 enables the consumer removal API"} end; init_per_testcase(T = immutable_bare_message, Config) -> - case rpc(Config, rabbit_feature_flags, is_enabled, [message_containers_store_amqp_v1]) of + case rpc(Config, rabbit_feature_flags, is_enabled, ['rabbitmq_4.0.0']) of true -> rabbit_ct_helpers:testcase_started(Config, T); false -> {skip, "RabbitMQ is known to wrongfully modify the bare message with feature " - "flag message_containers_store_amqp_v1 disabled"} + "flag rabbitmq_4.0.0 disabled"} end; init_per_testcase(T = dead_letter_into_stream, Config) -> case rpc(Config, rabbit_feature_flags, is_enabled, [message_containers_deaths_v2]) of @@ -3106,7 +3106,7 @@ async_notify_settled_stream(Config) -> async_notify(settled, <<"stream">>, Config). async_notify_unsettled_classic_queue(Config) -> - case rabbit_ct_broker_helpers:enable_feature_flag(Config, credit_api_v2) of + case rabbit_ct_broker_helpers:enable_feature_flag(Config, 'rabbitmq_4.0.0') of ok -> async_notify(unsettled, <<"classic">>, Config); {skip, _} -> @@ -3345,7 +3345,7 @@ queue_and_client_different_nodes(QueueLeaderNode, ClientNode, QueueType, Config) true, accepted), - case rpc(Config, rabbit_feature_flags, is_enabled, [credit_api_v2]) of + case rpc(Config, rabbit_feature_flags, is_enabled, ['rabbitmq_4.0.0']) of true -> %% Send another message and drain. Tag = <<"tag">>, diff --git a/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl b/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl index ef25602234ce..76a12873e715 100644 --- a/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl +++ b/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl @@ -48,19 +48,12 @@ end_per_group(_Group, Config) -> rabbit_ct_client_helpers:teardown_steps() ++ rabbit_ct_broker_helpers:teardown_steps()). -init_per_testcase(TestCase, Config) -> - case rabbit_ct_broker_helpers:is_feature_flag_supported(Config, TestCase) of - true -> - ?assertNot(rabbit_ct_broker_helpers:is_feature_flag_enabled(Config, TestCase)), - Config; - false -> - {skip, io_lib:format("feature flag ~s is unsupported", [TestCase])} - end. - -end_per_testcase(_TestCase, Config) -> - Config. 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), @@ -124,10 +117,7 @@ credit_api_v2(Config) -> ok = consume_and_accept(10, CQReceiver1), ok = consume_and_accept(10, QQReceiver1), - ?assertEqual(ok, - rabbit_ct_broker_helpers:enable_feature_flag(Config, ?FUNCTION_NAME)), - ?assertEqual(ok, - rabbit_ct_broker_helpers:enable_feature_flag(Config, quorum_queues_v4)), + ?assertEqual(ok, rabbit_ct_broker_helpers:enable_feature_flag(Config, FeatureFlag)), flush(enabled_feature_flag), %% Consume with credit API v2 diff --git a/deps/rabbit/test/amqp_system_SUITE.erl b/deps/rabbit/test/amqp_system_SUITE.erl index 2d4e728fb6ca..e1bf5abea72b 100644 --- a/deps/rabbit/test/amqp_system_SUITE.erl +++ b/deps/rabbit/test/amqp_system_SUITE.erl @@ -74,10 +74,7 @@ init_per_group(Group, Config) -> [GroupSetupStep] ++ rabbit_ct_broker_helpers:setup_steps() ++ rabbit_ct_client_helpers:setup_steps()), - ok = rabbit_ct_broker_helpers:enable_feature_flag( - Config2, message_containers_store_amqp_v1), - ok = rabbit_ct_broker_helpers:enable_feature_flag( - Config2, quorum_queues_v4), + ok = rabbit_ct_broker_helpers:enable_feature_flag(Config2, 'rabbitmq_4.0.0'), Config2. end_per_group(_, Config) -> diff --git a/deps/rabbit/test/classic_queue_SUITE.erl b/deps/rabbit/test/classic_queue_SUITE.erl index 09c427f67664..5b54d7150fb0 100644 --- a/deps/rabbit/test/classic_queue_SUITE.erl +++ b/deps/rabbit/test/classic_queue_SUITE.erl @@ -61,7 +61,7 @@ end_per_group(_, Config) -> rabbit_ct_broker_helpers:teardown_steps()). init_per_testcase(T, Config) -> - case rabbit_ct_broker_helpers:enable_feature_flag(Config, classic_queue_leader_locator) of + case rabbit_ct_broker_helpers:enable_feature_flag(Config, 'rabbitmq_4.0.0') of ok -> rabbit_ct_helpers:testcase_started(Config, T); Skip -> diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index 8ca389b49cf2..e64f11717c09 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -242,11 +242,9 @@ init_per_group(Group, Config) -> {skip, _} -> Ret; Config2 -> - _ = rabbit_ct_broker_helpers:enable_feature_flag(Config2, - message_containers), - QQV4 = rabbit_ct_broker_helpers:enable_feature_flag(Config2, - quorum_queues_v4), - ct:pal("quorum_queue_v4 enable result ~p", [QQV4]), + Res = rabbit_ct_broker_helpers:enable_feature_flag( + Config2, 'rabbitmq_4.0.0'), + ct:pal("rabbitmq_4.0.0 enable result ~p", [Res]), ok = rabbit_ct_broker_helpers:rpc( Config2, 0, application, set_env, [rabbit, channel_tick_interval, 100]), @@ -4024,12 +4022,11 @@ basic_get(Ch, Q, NoAck, Attempt) -> end. check_quorum_queues_v4_compat(Config) -> - case rabbit_ct_broker_helpers:is_feature_flag_enabled(Config, - quorum_queues_v4) of - false -> - throw({skip, "test needs feature flag quorum_queues_v4"}); + case rabbit_ct_broker_helpers:is_feature_flag_enabled(Config, 'rabbitmq_4.0.0') of true -> - ok + ok; + false -> + throw({skip, "test needs feature flag rabbitmq_4.0.0"}) end. lists_interleave([], _List) -> diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index 1271e7dd6b76..273597982f31 100644 --- a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl @@ -1819,7 +1819,7 @@ expand(Ops, Config, EnqFun) -> _ -> InitConfig0 end, - IsV4 = rabbit_feature_flags:is_enabled(quorum_queues_v4), + IsV4 = rabbit_feature_flags:is_enabled('rabbitmq_4.0.0'), T = #t{state = rabbit_fifo:init(InitConfig), enq_body_fun = EnqFun, config = Config, diff --git a/deps/rabbitmq_mqtt/test/protocol_interop_SUITE.erl b/deps/rabbitmq_mqtt/test/protocol_interop_SUITE.erl index d85fc4fb1b14..249e335e2afd 100644 --- a/deps/rabbitmq_mqtt/test/protocol_interop_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/protocol_interop_SUITE.erl @@ -338,7 +338,7 @@ amqp_mqtt_amqp(Config) -> properties := Props = #{'Correlation-Data' := Correlation} } = MqttMsg, case rabbit_ct_broker_helpers:is_feature_flag_enabled( - Config, message_containers_store_amqp_v1) of + Config, 'rabbitmq_4.0.0') of true -> ?assertEqual({ok, ResponseTopic}, maps:find('Response-Topic', Props)); @@ -430,7 +430,7 @@ amqp_mqtt(Qos, Config) -> } = MqttMsg1, ?assertEqual([Body1], amqp10_framing:decode_bin(Payload1)), case rabbit_ct_broker_helpers:is_feature_flag_enabled( - Config, message_containers_store_amqp_v1) of + Config, 'rabbitmq_4.0.0') of true -> ?assertEqual({ok, <<"message/vnd.rabbitmq.amqp">>}, maps:find('Content-Type', Props)); From 990c10de0e1538d7f930e71d282fbc1b9c252b6f Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 18 Jul 2024 14:25:56 +0100 Subject: [PATCH 39/45] QQ: expose priority metrics in UI --- deps/rabbit/src/rabbit_fifo.erl | 16 +++++++++++++--- deps/rabbit/src/rabbit_fifo_q.erl | 19 ++++++++++++++++++- deps/rabbit/src/rabbit_quorum_queue.erl | 16 +++++++++++++++- .../priv/www/js/tmpl/queue.ejs | 16 ++++++++++++++-- 4 files changed, 60 insertions(+), 7 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 6084f3f9b9d0..9dca79ae2217 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -809,6 +809,8 @@ overview(#?STATE{consumers = Cons, 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, @@ -831,6 +833,11 @@ overview(#?STATE{consumers = Cons, _ -> #{} end, + MsgsRet = lqueue:len(Returns), + + #{len := _MsgsLen, + num_hi := MsgsHi, + num_lo := MsgsLo} = rabbit_fifo_q:overview(Messages), Overview = #{type => ?STATE, config => Conf, num_consumers => map_size(Cons), @@ -838,14 +845,17 @@ overview(#?STATE{consumers = Cons, num_checked_out => num_checked_out(State), num_enqueuers => maps:size(Enqs), num_ready_messages => messages_ready(State), - num_in_memory_ready_messages => 0, %% backwards compat + num_ready_messages_high => MsgsHi, + num_ready_messages_low => MsgsLo, + num_ready_messages_return => MsgsRet, num_messages => messages_total(State), num_release_cursors => lqueue:len(Cursors), - release_cursors => [I || {_, I, _} <- lqueue:to_list(Cursors)], - release_cursor_enqueue_counter => EnqCount, 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) }, DlxOverview = rabbit_fifo_dlx:overview(DlxState), diff --git a/deps/rabbit/src/rabbit_fifo_q.erl b/deps/rabbit/src/rabbit_fifo_q.erl index 9fd63fe4b6fd..779ba586ec57 100644 --- a/deps/rabbit/src/rabbit_fifo_q.erl +++ b/deps/rabbit/src/rabbit_fifo_q.erl @@ -8,7 +8,8 @@ get/1, len/1, from_lqueue/1, - get_lowest_index/1 + get_lowest_index/1, + overview/1 ]). -define(WEIGHT, 2). @@ -97,6 +98,22 @@ get_lowest_index(#?MODULE{hi = Hi, lo = Lo}) -> end end. +-spec overview(state()) -> + #{len := non_neg_integer(), + num_hi := non_neg_integer(), + num_lo := non_neg_integer(), + lowest_index := ra:index()}. +overview(#?MODULE{len = Len, + hi = {Hi1, Hi2}, + lo = _} = State) -> + %% TODO: this could be very slow with large backlogs, + %% consider keeping a separate counter for hi, lo messages + NumHi = length(Hi1) + length(Hi2), + #{len => Len, + num_hi => NumHi, + num_lo => Len - NumHi, + lowest_index => get_lowest_index(State)}. + %% internals next(#?MODULE{hi = ?NON_EMPTY = Hi, diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index b782845fca21..7c9451144008 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -535,6 +535,7 @@ handle_tick(QName, 0 -> 0; _ -> rabbit_fifo:usage(Name) end, + Keys = ?STATISTICS_KEYS -- [leader, consumers, messages_dlx, @@ -544,11 +545,24 @@ handle_tick(QName, ], {SacTag, SacPid} = maps:get(single_active_consumer_id, Overview, {'', ''}), + Infos0 = maps:fold( + fun(num_ready_messages_high, V, Acc) -> + [{messages_ready_high, V} | Acc]; + (num_ready_messages_low, V, Acc) -> + [{messages_ready_low, V} | Acc]; + (num_ready_messages_return, V, Acc) -> + [{messages_ready_returned, V} | Acc]; + (_, _, Acc) -> + Acc + end, info(Q, Keys), Overview), MsgBytesDiscarded = DiscardBytes + DiscardCheckoutBytes, MsgBytes = EnqueueBytes + CheckoutBytes + MsgBytesDiscarded, Infos = [{consumers, NumConsumers}, {consumer_capacity, Util}, {consumer_utilisation, Util}, + {messages, NumMessages}, + {messages_ready, NumReadyMsgs}, + {messages_unacknowledged, NumCheckedOut}, {message_bytes_ready, EnqueueBytes}, {message_bytes_unacknowledged, CheckoutBytes}, {message_bytes, MsgBytes}, @@ -559,7 +573,7 @@ handle_tick(QName, {single_active_consumer_tag, SacTag}, {single_active_consumer_pid, SacPid}, {leader, node()} - | info(Q, Keys)], + | Infos0], rabbit_core_metrics:queue_stats(QName, Infos), ok = repair_leader_record(Q, Self), case repair_amqqueue_nodes(Q) of diff --git a/deps/rabbitmq_management/priv/www/js/tmpl/queue.ejs b/deps/rabbitmq_management/priv/www/js/tmpl/queue.ejs index c402ce4875d8..ea141f0256bf 100644 --- a/deps/rabbitmq_management/priv/www/js/tmpl/queue.ejs +++ b/deps/rabbitmq_management/priv/www/js/tmpl/queue.ejs @@ -119,7 +119,9 @@ Ready Unacked <% if (is_quorum(queue)) { %> - In memory ready + High priority + Low priority + Returned Dead-lettered @@ -146,7 +148,13 @@ <% if (is_quorum(queue)) { %> - <%= fmt_num_thousands(queue.messages_ram) %> + <%= fmt_num_thousands(queue.messages_ready_high) %> + + + <%= fmt_num_thousands(queue.messages_ready_low) %> + + + <%= fmt_num_thousands(queue.messages_ready_returned) %> <%= fmt_num_thousands(queue.messages_dlx) %> @@ -184,6 +192,10 @@ <% } %> <% if (is_quorum(queue)) { %> + + + + <%= fmt_bytes(queue.message_bytes_dlx) %> From 5a9d2e68e9a814d9f4ae1c9b64125b42b995b8e5 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 31 Jul 2024 14:08:52 +0200 Subject: [PATCH 40/45] Enable skipped test after rebasing onto main --- deps/rabbit/test/amqp_client_SUITE.erl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl index 4a813c01a03b..4da2a11d136a 100644 --- a/deps/rabbit/test/amqp_client_SUITE.erl +++ b/deps/rabbit/test/amqp_client_SUITE.erl @@ -301,7 +301,7 @@ init_per_testcase(T, Config) T =:= leader_transfer_quorum_queue_credit_batches orelse T =:= leader_transfer_stream_credit_single orelse T =:= leader_transfer_stream_credit_batches -> - case rpc(Config, rabbit_feature_flags, is_supported, [credit_api_v2]) of + case rpc(Config, rabbit_feature_flags, is_supported, ['rabbitmq_4.0.0']) of true -> rabbit_ct_helpers:testcase_started(Config, T); false -> From 83a78f18266910d16b137c8ffa6457839cda275d Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 23 Jul 2024 12:29:11 +0100 Subject: [PATCH 41/45] QQ: add new command "modify" to better handle AMQP modified outcomes. This new command can be used to annotate returned or rejected messages. This commit also retains the delivery-count across dead letter boundaries such that the AMQP header delivery-count field can now include _all_ failed deliver attempts since the message was originally received. Internally the quorum queue has moved it's delivery_count header to only track the AMQP protocol delivery attempts and now introduces a new acquired_count to track all message acquisitions by consumers. --- deps/rabbit/app.bzl | 3 +- deps/rabbit/src/mc_compat.erl | 4 +- deps/rabbit/src/rabbit_amqp_session.erl | 30 ++- deps/rabbit/src/rabbit_classic_queue.erl | 9 + deps/rabbit/src/rabbit_fifo.erl | 248 +++++++++++++----- deps/rabbit/src/rabbit_fifo.hrl | 8 +- deps/rabbit/src/rabbit_fifo_client.erl | 68 +++-- deps/rabbit/src/rabbit_fifo_dlx.erl | 28 +- deps/rabbit/src/rabbit_queue_type.erl | 8 +- deps/rabbit/src/rabbit_quorum_queue.erl | 5 +- deps/rabbit/test/amqp_client_SUITE.erl | 33 ++- deps/rabbit/test/quorum_queue_SUITE.erl | 23 +- deps/rabbit/test/rabbit_fifo_SUITE.erl | 150 +++++++++-- .../rabbit_fifo_dlx_integration_SUITE.erl | 6 +- deps/rabbit/test/rabbit_fifo_int_SUITE.erl | 29 +- 15 files changed, 482 insertions(+), 170 deletions(-) diff --git a/deps/rabbit/app.bzl b/deps/rabbit/app.bzl index ae3204f7d02d..17bfb089dcc4 100644 --- a/deps/rabbit/app.bzl +++ b/deps/rabbit/app.bzl @@ -1295,7 +1295,8 @@ def test_suite_beam_files(name = "test_suite_beam_files"): testonly = True, srcs = ["test/rabbit_fifo_SUITE.erl"], outs = ["test/rabbit_fifo_SUITE.beam"], - hdrs = ["src/rabbit_fifo.hrl"], + hdrs = ["src/rabbit_fifo.hrl", + "src/rabbit_fifo_dlx.hrl"], app_name = "rabbit", erlc_opts = "//:test_erlc_opts", deps = ["//deps/rabbit_common:erlang_app"], diff --git a/deps/rabbit/src/mc_compat.erl b/deps/rabbit/src/mc_compat.erl index 702f8c0f64ca..289a5332cd58 100644 --- a/deps/rabbit/src/mc_compat.erl +++ b/deps/rabbit/src/mc_compat.erl @@ -54,7 +54,9 @@ get_annotation(?ANN_ROUTING_KEYS, #basic_message{routing_keys = RKeys}) -> get_annotation(?ANN_EXCHANGE, #basic_message{exchange_name = Ex}) -> Ex#resource.name; get_annotation(id, #basic_message{id = Id}) -> - Id. + Id; +get_annotation(_Key, #basic_message{}) -> + undefined. set_annotation(id, Value, #basic_message{} = Msg) -> Msg#basic_message{id = Value}; diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl index abfbc90f4ac6..33c48b3a990d 100644 --- a/deps/rabbit/src/rabbit_amqp_session.erl +++ b/deps/rabbit/src/rabbit_amqp_session.erl @@ -1868,14 +1868,23 @@ settle_op_from_outcome(#'v1_0.released'{}) -> %% https://github.com/apache/qpid-jms/blob/90eb60f59cb59b7b9ad8363ee8a843d6903b8e77/qpid-jms-client/src/main/java/org/apache/qpid/jms/JmsMessageConsumer.java#L464 %% In such cases, it's better when RabbitMQ does not end the session. %% See https://github.com/rabbitmq/rabbitmq-server/issues/6121 -settle_op_from_outcome(#'v1_0.modified'{undeliverable_here = true}) -> - %% This is not quite correct because undeliverable_here refers to the link, - %% and not the message in general. However, RabbitMQ cannot filter messages from - %% being assigned to individual consumers. That's why we discard. - discard; -settle_op_from_outcome(#'v1_0.modified'{}) -> - requeue; - +settle_op_from_outcome(#'v1_0.modified'{delivery_failed = DelFailed, + undeliverable_here = UndelHere, + message_annotations = Anns0 + }) -> + Anns = case Anns0 of + #'v1_0.message_annotations'{content = C} -> + C; + _ -> + [] + end, + {modify, + default(DelFailed, false), + default(UndelHere, false), + %% TODO: this must exist elsewhere + lists:foldl(fun ({{symbol, K}, V}, Acc) -> + Acc#{K => unwrap(V)} + end, #{}, Anns)}; settle_op_from_outcome(Outcome) -> protocol_error( ?V_1_0_AMQP_ERROR_INVALID_FIELD, @@ -3499,3 +3508,8 @@ format_status( permission_cache => PermissionCache, topic_permission_cache => TopicPermissionCache}, maps:update(state, State, Status). + +unwrap({_Tag, V}) -> + V; +unwrap(V) -> + V. diff --git a/deps/rabbit/src/rabbit_classic_queue.erl b/deps/rabbit/src/rabbit_classic_queue.erl index 7eb09637f77b..2907d3dbc428 100644 --- a/deps/rabbit/src/rabbit_classic_queue.erl +++ b/deps/rabbit/src/rabbit_classic_queue.erl @@ -333,6 +333,15 @@ cancel(Q, Spec, State) -> -spec settle(rabbit_amqqueue:name(), rabbit_queue_type:settle_op(), rabbit_types:ctag(), [non_neg_integer()], state()) -> {state(), rabbit_queue_type:actions()}. +settle(QName, {modify, _DelFailed, Undel, _}, CTag, MsgIds, State) -> + %% translate modify into other op + Op = case Undel of + true -> + discard; + false -> + requeue + end, + settle(QName, Op, CTag, MsgIds, State); settle(_QName, Op, _CTag, MsgIds, State = #?STATE{pid = Pid}) -> Arg = case Op of complete -> diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 9dca79ae2217..82ca4c42c340 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -73,6 +73,7 @@ %% misc get_msg_header/1, get_header/2, + annotate_msg/2, get_msg/1, %% protocol helpers @@ -84,7 +85,7 @@ is_return/1, make_discard/2, make_credit/4, - make_defer/2, + make_modify/5, make_purge/0, make_purge_nodes/1, make_update_config/1, @@ -128,8 +129,11 @@ credit :: non_neg_integer(), delivery_count :: rabbit_queue_type:delivery_count(), drain :: boolean()}). --record(defer, {consumer_key :: consumer_key(), - msg_ids :: [msg_id()]}). +-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()}). @@ -146,7 +150,7 @@ #return{} | #discard{} | #credit{} | - #defer{} | + #modify{} | #purge{} | #purge_nodes{} | #update_config{} | @@ -259,26 +263,10 @@ apply(Meta, #settle{msg_ids = MsgIds, end; apply(Meta, #discard{consumer_key = ConsumerKey, msg_ids = MsgIds}, - #?STATE{consumers = Consumers, - dlx = DlxState0, - cfg = #cfg{dead_letter_handler = DLH}} = State0) -> + #?STATE{consumers = Consumers } = State0) -> case find_consumer(ConsumerKey, Consumers) of - {ConsumerKey, #consumer{checked_out = Checked} = Con} -> - %% 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; - Msg -> - {true, Msg} - 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); + {ConsumerKey, #consumer{} = Con} -> + discard(Meta, MsgIds, ConsumerKey, Con, true, #{}, State0); _ -> {State0, ok} end; @@ -287,10 +275,28 @@ apply(Meta, #return{consumer_key = ConsumerKey, #?STATE{consumers = Cons0} = State) -> case find_consumer(ConsumerKey, Cons0) of {ActualConsumerKey, #consumer{checked_out = Checked}} -> - return(Meta, ActualConsumerKey, MsgIds, Checked, [], State); + 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 = Cons0} = State0) -> + case find_consumer(ConsumerKey, Cons0) of + {ConsumerKey, #consumer{checked_out = Checked}} + when Undel == false -> + return(Meta, ConsumerKey, MsgIds, DelFailed, + Anns, Checked, [], State0); + {ConsumerKey, #consumer{} = Con} + when Undel == true -> + discard(Meta, MsgIds, ConsumerKey, Con, DelFailed, Anns, State0); + _ -> + {State0, ok} + end; apply(#{index := Idx} = Meta, #requeue{consumer_key = ConsumerKey, msg_id = MsgId, @@ -307,8 +313,7 @@ apply(#{index := Idx} = Meta, when is_map_key(MsgId, Checked0) -> %% construct a message with the current raft index %% and update delivery count before adding it to the message queue - Header1 = update_header(delivery_count, fun incr/1, 1, Header0), - Header = update_header(return_count, fun incr/1, 1, Header1), + 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)}, @@ -506,7 +511,7 @@ apply(#{system_time := Ts} = Meta, %% and checked out messages should be returned Effs = consumer_update_active_effects( S0, C0, false, suspected_down, E0), - {St, Effs1} = return_all(Meta, S0, Effs, CKey, C0), + {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 @@ -556,7 +561,7 @@ apply(#{system_time := Ts} = Meta, status = up} = C0, {St0, Eff}) when node(P) =:= Node -> C = C0#consumer{status = suspected_down}, - {St, Eff0} = return_all(Meta, St0, Eff, CKey, C), + {St, Eff0} = return_all(Meta, St0, Eff, CKey, C, true), Eff1 = consumer_update_active_effects(St, C, false, suspected_down, Eff0), {St, Eff1}; @@ -642,28 +647,47 @@ apply(_Meta, Cmd, State) -> rabbit_log:debug("rabbit_fifo: unhandled command ~W", [Cmd, 10]), {State, ok, []}. -convert_v3_to_v4(#{system_time := _Ts}, - StateV3) -> +convert_v3_to_v4(#{} = _Meta, StateV3) -> + %% TODO: consider emitting release cursors as checkpoints Messages0 = rabbit_fifo_v3:get_field(messages, StateV3), - Consumers = rabbit_fifo_v3:get_field(consumers, 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}; + (_, Msg) -> + Msg + 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), - #?MODULE{cfg = rabbit_fifo_v3:get_field(cfg, StateV3), - messages = Messages, - messages_total = rabbit_fifo_v3:get_field(messages_total, StateV3), - returns = rabbit_fifo_v3:get_field(returns, StateV3), - 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), - release_cursors = rabbit_fifo_v3:get_field(release_cursors, StateV3), - consumers = Consumers, - % consumers that require further service are queued here - 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)}. + #?STATE{cfg = rabbit_fifo_v3:get_field(cfg, StateV3), + 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(Meta, Node, State, Effects) -> lists:foldl(fun(Pid, {S0, E0}) -> @@ -803,7 +827,6 @@ tick(Ts, #?STATE{cfg = #cfg{name = _Name, -spec overview(state()) -> map(). overview(#?STATE{consumers = Cons, enqueuers = Enqs, - release_cursors = Cursors, enqueue_count = EnqCount, msg_bytes_enqueue = EnqueueBytes, msg_bytes_checkout = CheckoutBytes, @@ -849,7 +872,7 @@ overview(#?STATE{consumers = Cons, num_ready_messages_low => MsgsLo, num_ready_messages_return => MsgsRet, num_messages => messages_total(State), - num_release_cursors => lqueue:len(Cursors), + num_release_cursors => 0, %% backwards compat enqueue_message_bytes => EnqueueBytes, checkout_message_bytes => CheckoutBytes, release_cursors => [], %% backwards compat @@ -1518,7 +1541,8 @@ maybe_return_all(#{system_time := Ts} = Meta, ConsumerKey, status = cancelled}, S0), Effects0}; _ -> - {S1, Effects} = return_all(Meta, S0, Effects0, ConsumerKey, Consumer), + {S1, Effects} = return_all(Meta, S0, Effects0, ConsumerKey, + Consumer, Reason == down), {S1#?STATE{consumers = maps:remove(ConsumerKey, S1#?STATE.consumers), last_active = Ts}, Effects} @@ -1563,6 +1587,20 @@ maybe_set_msg_ttl(Msg, RaCmdTs, Header, 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) -> @@ -1586,7 +1624,8 @@ maybe_enqueue(RaftIdx, Ts, undefined, undefined, RawMsg, messages_total = Total} = State0) -> % direct enqueue without tracking Size = BodySize, - Header = maybe_set_msg_ttl(RawMsg, Ts, Size, State0), + 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, @@ -1612,7 +1651,8 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, {Res, State, [{monitor, process, From} | Effects]}; #enqueuer{next_seqno = MsgSeqNo} = Enq0 -> % it is the next expected seqno - Header = maybe_set_msg_ttl(RawMsg, Ts, BodySize, State0), + 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 @@ -1639,17 +1679,16 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, {duplicate, State0, Effects0} end. -return(#{} = Meta, - ConsumerKey, MsgIds, Checked, Effects0, State0) -> +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. {State1, Effects1} = lists:foldl( fun(MsgId, Acc = {S0, E0}) -> case Checked of - #{MsgId := Msg0} -> - Msg = update_msg_header(return_count, fun incr/1, 1, - Msg0), - return_one(Meta, MsgId, Msg, + #{MsgId := Msg} -> + return_one(Meta, MsgId, Msg, IncrDelCount, Anns, S0, E0, ConsumerKey); #{} -> Acc @@ -1782,7 +1821,24 @@ get_header(Key, Header) when is_map(Header) andalso is_map_key(size, Header) -> maps:get(Key, Header, undefined). -return_one(Meta, MsgId, ?MSG(_, _) = Msg0, +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 is_map_key(delivery_count, Header) of + true -> + mc:set_annotation(delivery_count, + maps:get(delivery_count, Header), Msg); + false -> + Msg + end; + _ -> + Msg0 + end. + +return_one(Meta, MsgId, ?MSG(_, _) = Msg0, DelivFailed, Anns, #?STATE{returns = Returns, consumers = Consumers, dlx = DlxState0, @@ -1790,9 +1846,9 @@ return_one(Meta, MsgId, ?MSG(_, _) = Msg0, dead_letter_handler = DLH}} = State0, Effects0, ConsumerKey) -> #consumer{checked_out = Checked0} = Con0 = maps:get(ConsumerKey, Consumers), - Msg = update_msg_header(delivery_count, fun incr/1, 1, Msg0), + Msg = incr_msg(Msg0, DelivFailed, Anns), Header = get_msg_header(Msg), - case get_header(delivery_count, Header) of + case get_header(acquired_count, Header) of DeliveryCount when DeliveryCount > DeliveryLimit -> {DlxState, DlxEffects} = rabbit_fifo_dlx:discard([Msg], delivery_limit, DLH, DlxState0), @@ -1811,10 +1867,11 @@ return_one(Meta, MsgId, ?MSG(_, _) = Msg0, end. return_all(Meta, #?STATE{consumers = Cons} = State0, Effects0, ConsumerKey, - #consumer{checked_out = Checked} = Con) -> + #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, S, E, ConsumerKey) + return_one(Meta, MsgId, Msg, DelivFailed, #{}, + S, E, ConsumerKey) end, {State, Effects0}, lists:sort(maps:to_list(Checked))). checkout(Meta, OldState, State0, Effects0) -> @@ -2468,9 +2525,26 @@ make_credit(Key, Credit, DeliveryCount, Drain) -> delivery_count = DeliveryCount, drain = Drain}. --spec make_defer(consumer_key(), [msg_id()]) -> protocol(). -make_defer(ConsumerKey, MsgIds) when is_list(MsgIds) -> - #defer{consumer_key = ConsumerKey, msg_ids = MsgIds}. +-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{}. @@ -2497,7 +2571,7 @@ add_bytes_return(Header, msg_bytes_enqueue = Enqueue} = State) -> Size = get_header(size, Header), State#?STATE{msg_bytes_checkout = Checkout - Size, - msg_bytes_enqueue = Enqueue + Size}. + msg_bytes_enqueue = Enqueue + Size}. message_size(B) when is_binary(B) -> byte_size(B); @@ -2830,3 +2904,41 @@ release_cursor(LastSmallest, Smallest) 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. diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index 0518c7fdb0d1..d7a5de7fdb0d 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -11,8 +11,6 @@ %% 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(C_MSG(_At, Msg), Msg). -% -define(C_MSG(_At, Index, Header), ?MSG(Index, Header)). -define(IS_HEADER(H), (is_integer(H) andalso H >= 0) orelse @@ -199,11 +197,7 @@ % index when there are large gaps but should be faster than gb_trees % for normal appending operations as it's backed by a map ra_indexes = rabbit_fifo_index:empty() :: rabbit_fifo_index:state(), - %% A release cursor is essentially a snapshot for a past raft index. - %% Working assumption: Messages are consumed in a FIFO-ish order because - %% the log is truncated only until the oldest message. - release_cursors = lqueue:new() :: lqueue:lqueue({release_cursor, - ra:index(), #rabbit_fifo{}}), + unused_1, % consumers need to reflect consumer state at time of snapshot consumers = #{} :: #{consumer_id() | ra:index() => consumer()}, % consumers that require further service are queued here diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index 9cdd92203480..20d57d89577f 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -22,6 +22,7 @@ settle/3, return/3, discard/3, + modify/6, credit_v1/4, credit/5, handle_ra_event/4, @@ -217,20 +218,23 @@ dequeue(QueueName, ConsumerTag, Settlement, Err end. -add_delivery_count_header(Msg0, #{delivery_count := DelCount} = Header) - when is_integer(DelCount) -> +add_delivery_count_header(Msg0, #{acquired_count := AcqCount} = Header) + when is_integer(AcqCount) -> Msg = case mc:is(Msg0) of true -> - Msg1 = mc:set_annotation(<<"x-delivery-count">>, DelCount, Msg0), + Msg1 = mc:set_annotation(<<"x-delivery-count">>, AcqCount, Msg0), %% the "delivery-count" header in the AMQP spec does not include %% returns (released outcomes) - AmqpDelCount = DelCount - maps:get(return_count, Header, 0), - mc:set_annotation(delivery_count, AmqpDelCount, Msg1); + rabbit_fifo:annotate_msg(Header, Msg1); false -> Msg0 end, - Redelivered = DelCount > 0, + Redelivered = AcqCount > 0, {Msg, Redelivered}; +add_delivery_count_header(Msg, #{delivery_count := DC} = Header) -> + %% there was a delivery count but no acquired count, this means the message + %% was delivered from a quorum queue running v3 so we patch this up here + add_delivery_count_header(Msg, Header#{acquired_count => DC}); add_delivery_count_header(Msg, _Header) -> {Msg, false}. @@ -311,6 +315,20 @@ discard(ConsumerTag, [_|_] = MsgIds, end, {[], [], MsgIds}, Unsent0), {State0#state{unsent_commands = Unsent}, []}. +-spec modify(rabbit_types:ctag(), [rabbit_fifo:msg_id()], + boolean(), boolean(), mc:annotations(), state()) -> + {state(), list()}. +modify(ConsumerTag, [_|_] = MsgIds, DelFailed, Undel, Anns, + #state{} = State0) -> + ConsumerKey = consumer_key(ConsumerTag, State0), + %% we need to send any pending settles, discards or returns before we + %% send the modify as this cannot be batched + %% as it contains message specific annotations + State1 = send_pending(ConsumerKey, State0), + ServerId = pick_server(State1), + Cmd = rabbit_fifo:make_modify(ConsumerKey, MsgIds, DelFailed, Undel, Anns), + {send_command(ServerId, undefined, Cmd, normal, State1), []}. + %% @doc Register with the rabbit_fifo queue to "checkout" messages as they %% become available. %% @@ -455,29 +473,13 @@ credit(ConsumerTag, DeliveryCount, Credit, Drain, State) -> -spec cancel_checkout(rabbit_types:ctag(), rabbit_queue_type:cancel_reason(), state()) -> {ok, state()} | {error | timeout, term()}. cancel_checkout(ConsumerTag, Reason, - #state{consumers = Consumers, - unsent_commands = Unsent} = State0) + #state{consumers = Consumers} = State0) when is_atom(Reason) -> case Consumers of #{ConsumerTag := #consumer{key = Cid}} -> Servers = sorted_servers(State0), ConsumerId = {ConsumerTag, self()}, - %% send any pending commands for consumer before cancelling - Commands = case Unsent of - #{Cid := {Settled, Returns, Discards}} -> - add_command(Cid, settle, Settled, - add_command(Cid, return, Returns, - add_command(Cid, discard, - Discards, []))); - _ -> - [] - end, - ServerId = pick_server(State0), - %% send all the settlements, discards and returns - State1 = lists:foldl(fun (C, S0) -> - send_command(ServerId, undefined, C, - normal, S0) - end, State0, Commands), + State1 = send_pending(Cid, State0), Cmd = rabbit_fifo:make_checkout(ConsumerId, Reason, #{}), State = State1#state{consumers = maps:remove(ConsumerTag, Consumers)}, case try_process_command(Servers, Cmd, State) of @@ -984,3 +986,21 @@ qref(Ref) -> Ref. atom(). cluster_name(#state{cfg = #cfg{servers = [{Name, _Node} | _]}}) -> Name. + +send_pending(Cid, #state{unsent_commands = Unsent} = State0) -> + Commands = case Unsent of + #{Cid := {Settled, Returns, Discards}} -> + add_command(Cid, settle, Settled, + add_command(Cid, return, Returns, + add_command(Cid, discard, + Discards, []))); + _ -> + [] + end, + ServerId = pick_server(State0), + %% send all the settlements, discards and returns + State1 = lists:foldl(fun (C, S0) -> + send_command(ServerId, undefined, C, + normal, S0) + end, State0, Commands), + State1#state{unsent_commands = maps:remove(Cid, Unsent)}. diff --git a/deps/rabbit/src/rabbit_fifo_dlx.erl b/deps/rabbit/src/rabbit_fifo_dlx.erl index 480b6f497fec..c4c9dd488c2f 100644 --- a/deps/rabbit/src/rabbit_fifo_dlx.erl +++ b/deps/rabbit/src/rabbit_fifo_dlx.erl @@ -159,21 +159,20 @@ discard(Msgs0, Reason, {at_most_once, {Mod, Fun, Args}}, State) -> Lookup = maps:from_list(lists:zip(Idxs, Log)), Msgs = [begin Cmd = maps:get(Idx, Lookup), - rabbit_fifo:get_msg(Cmd) - end || ?MSG(Idx, _) <- Msgs0], + %% ensure header delivery count + %% is copied to the message container + annotate_msg(H, rabbit_fifo:get_msg(Cmd)) + end || ?MSG(Idx, H) <- Msgs0], [{mod_call, Mod, Fun, Args ++ [Reason, Msgs]}] end}, {State, [Effect]}; discard(Msgs, Reason, at_least_once, State0) when Reason =/= maxlen -> - State = lists:foldl(fun(?MSG(Idx, _) = Msg0, + State = lists:foldl(fun(?MSG(Idx, _) = Msg, #?MODULE{discards = D0, msg_bytes = B0, ra_indexes = I0} = S0) -> - MsgSize = size_in_bytes(Msg0), - %% Condense header to an integer representing the message size. - %% We need neither delivery_count nor expiry anymore. - Msg = ?MSG(Idx, MsgSize), + MsgSize = size_in_bytes(Msg), D = lqueue:in(?TUPLE(Reason, Msg), D0), B = B0 + MsgSize, I = rabbit_fifo_index:append(Idx, I0), @@ -191,8 +190,8 @@ checkout(at_least_once, #?MODULE{consumer = #dlx_consumer{}} = State) -> checkout(_, State) -> {State, []}. -checkout0({success, MsgId, ?TUPLE(Reason, ?MSG(Idx, _)), State}, SendAcc) -> - DelMsg = {Idx, {Reason, MsgId}}, +checkout0({success, MsgId, ?TUPLE(Reason, ?MSG(Idx, H)), State}, SendAcc) -> + DelMsg = {Idx, {Reason, H, MsgId}}, checkout0(checkout_one(State), [DelMsg | SendAcc]); checkout0(#?MODULE{consumer = #dlx_consumer{pid = Pid}} = State, SendAcc) -> Effects = delivery_effects(Pid, SendAcc), @@ -232,9 +231,11 @@ delivery_effects(CPid, Msgs0) -> {RaftIdxs, RsnIds} = lists:unzip(Msgs1), [{log, RaftIdxs, fun(Log) -> - Msgs = lists:zipwith(fun (Cmd, {Reason, MsgId}) -> - {MsgId, {Reason, rabbit_fifo:get_msg(Cmd)}} - end, Log, RsnIds), + Msgs = lists:zipwith( + fun (Cmd, {Reason, H, MsgId}) -> + {MsgId, {Reason, + annotate_msg(H, rabbit_fifo:get_msg(Cmd))}} + end, Log, RsnIds), [{send_msg, CPid, {dlx_event, self(), {dlx_delivery, Msgs}}, [cast]}] end}]. @@ -360,3 +361,6 @@ dehydrate(State) -> option(non_neg_integer()). smallest_raft_index(#?MODULE{ra_indexes = Indexes}) -> rabbit_fifo_index:smallest(Indexes). + +annotate_msg(H, M0) -> + rabbit_fifo:annotate_msg(H, M0). diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl index 45184963ac8f..23e588c99e34 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -136,7 +136,13 @@ -type delivery_options() :: #{correlation => correlation(), atom() => term()}. --type settle_op() :: 'complete' | 'requeue' | 'discard'. +-type settle_op() :: complete | + requeue | + discard | + {modify, + DeliveryFailed :: boolean(), + UndeliverableHere :: boolean(), + Annotations :: mc:annotations()}. -export_type([state/0, consume_mode/0, diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 7c9451144008..e9a492a66881 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -847,7 +847,10 @@ settle(_QName, complete, CTag, MsgIds, QState) -> settle(_QName, requeue, CTag, MsgIds, QState) -> rabbit_fifo_client:return(quorum_ctag(CTag), MsgIds, QState); settle(_QName, discard, CTag, MsgIds, QState) -> - rabbit_fifo_client:discard(quorum_ctag(CTag), MsgIds, QState). + rabbit_fifo_client:discard(quorum_ctag(CTag), MsgIds, QState); +settle(_QName, {modify, DelFailed, Undel, Anns}, CTag, MsgIds, QState) -> + rabbit_fifo_client:modify(quorum_ctag(CTag), MsgIds, DelFailed, Undel, + Anns, QState). credit_v1(_QName, CTag, Credit, Drain, QState) -> rabbit_fifo_client:credit_v1(quorum_ctag(CTag), Credit, Drain, QState). diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl index 4da2a11d136a..dcb6625bfa5b 100644 --- a/deps/rabbit/test/amqp_client_SUITE.erl +++ b/deps/rabbit/test/amqp_client_SUITE.erl @@ -427,23 +427,33 @@ modified(Config) -> {ok, M1} = amqp10_client:get_msg(Receiver), ?assertEqual([<<"m1">>], amqp10_msg:body(M1)), - ok = amqp10_client:settle_msg(Receiver, M1, {modified, false, _UndeliverableHere = true, #{}}), + ?assertEqual(0, amqp10_msg:header(delivery_count, M1)), + ok = amqp10_client:settle_msg(Receiver, M1, {modified, false, + _UndeliverableHere = true, #{}}), {ok, M2a} = amqp10_client:get_msg(Receiver), ?assertEqual([<<"m2">>], amqp10_msg:body(M2a)), - ok = amqp10_client:settle_msg(Receiver, M2a, {modified, false, false, #{}}), + ?assertEqual(0, amqp10_msg:header(delivery_count, M2a)), + ok = amqp10_client:settle_msg(Receiver, M2a, {modified, false, false, #{}}), {ok, M2b} = amqp10_client:get_msg(Receiver), ?assertEqual([<<"m2">>], amqp10_msg:body(M2b)), - ok = amqp10_client:settle_msg(Receiver, M2b, {modified, true, false, #{}}), + ?assertEqual(0, amqp10_msg:header(delivery_count, M2b)), + ok = amqp10_client:settle_msg(Receiver, M2b, {modified, true, false, #{}}), {ok, M2c} = amqp10_client:get_msg(Receiver), + ?assertEqual(1, amqp10_msg:header(delivery_count, M2c)), ?assertEqual([<<"m2">>], amqp10_msg:body(M2c)), - ok = amqp10_client:settle_msg(Receiver, M2c, {modified, true, false, #{<<"key">> => <<"val">>}}), + + + ok = amqp10_client:settle_msg(Receiver, M2c, + {modified, true, false, + #{<<"x-opt-key">> => <<"val">>}}), {ok, M2d} = amqp10_client:get_msg(Receiver), ?assertEqual([<<"m2">>], amqp10_msg:body(M2d)), - ?assertEqual(0, amqp10_msg:header(delivery_count, M2d)), + ?assertEqual(2, amqp10_msg:header(delivery_count, M2d)), + ?assertMatch(#{<<"x-opt-key">> := <<"val">>}, amqp10_msg:message_annotations(M2d)), ok = amqp10_client:settle_msg(Receiver, M2d, modified), ok = amqp10_client:detach_link(Receiver), @@ -4419,6 +4429,7 @@ dead_letter_reject(Config) -> QName1, #{arguments => #{<<"x-queue-type">> => {utf8, <<"quorum">>}, <<"x-message-ttl">> => {ulong, 20}, + <<"x-dead-letter-strategy">> => {utf8, <<"at-least-once">>}, <<"x-dead-letter-exchange">> => {utf8, <<>>}, <<"x-dead-letter-routing-key">> => {utf8, QName2} }}), @@ -4449,10 +4460,20 @@ dead_letter_reject(Config) -> ok = wait_for_accepted(Tag), {ok, Msg1} = amqp10_client:get_msg(Receiver), + ?assertMatch(#{delivery_count := 0, + first_acquirer := true}, amqp10_msg:headers(Msg1)), ok = amqp10_client:settle_msg(Receiver, Msg1, rejected), {ok, Msg2} = amqp10_client:get_msg(Receiver), - ok = amqp10_client:settle_msg(Receiver, Msg2, rejected), + ?assertMatch(#{delivery_count := 1, + first_acquirer := false}, amqp10_msg:headers(Msg2)), + ok = amqp10_client:settle_msg(Receiver, Msg2, + {modified, true, true, + #{<<"x-opt-thekey">> => <<"val">>}}), {ok, Msg3} = amqp10_client:get_msg(Receiver), + ?assertMatch(#{delivery_count := 2, + first_acquirer := false}, amqp10_msg:headers(Msg3)), + ?assertMatch(#{<<"x-opt-thekey">> := <<"val">>}, + amqp10_msg:message_annotations(Msg3)), ok = amqp10_client:settle_msg(Receiver, Msg3, accepted), ?assertEqual(Body, amqp10_msg:body_bin(Msg3)), Annotations = amqp10_msg:message_annotations(Msg3), diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index e64f11717c09..d34253beb793 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -1346,7 +1346,7 @@ test_dead_lettering(PolicySet, Config, Ch, Servers, RaName, Source, Destination) wait_for_messages_ready(Servers, RaName, 1), wait_for_messages_pending_ack(Servers, RaName, 0), wait_for_messages(Config, [[Destination, <<"0">>, <<"0">>, <<"0">>]]), - DeliveryTag = consume(Ch, Source, false), + DeliveryTag = basic_get_tag(Ch, Source, false), wait_for_messages_ready(Servers, RaName, 0), wait_for_messages_pending_ack(Servers, RaName, 1), wait_for_messages(Config, [[Destination, <<"0">>, <<"0">>, <<"0">>]]), @@ -1358,7 +1358,7 @@ test_dead_lettering(PolicySet, Config, Ch, Servers, RaName, Source, Destination) case PolicySet of true -> wait_for_messages(Config, [[Destination, <<"1">>, <<"1">>, <<"0">>]]), - _ = consume(Ch, Destination, true); + _ = basic_get_tag(Ch, Destination, true); false -> wait_for_messages(Config, [[Destination, <<"0">>, <<"0">>, <<"0">>]]) end. @@ -1432,7 +1432,7 @@ dead_letter_to_quorum_queue(Config) -> wait_for_messages_pending_ack(Servers, RaName, 0), wait_for_messages_ready(Servers, RaName2, 0), wait_for_messages_pending_ack(Servers, RaName2, 0), - DeliveryTag = consume(Ch, QQ, false), + DeliveryTag = basic_get_tag(Ch, QQ, false), wait_for_messages_ready(Servers, RaName, 0), wait_for_messages_pending_ack(Servers, RaName, 1), wait_for_messages_ready(Servers, RaName2, 0), @@ -1444,7 +1444,12 @@ dead_letter_to_quorum_queue(Config) -> wait_for_messages_pending_ack(Servers, RaName, 0), wait_for_messages_ready(Servers, RaName2, 1), wait_for_messages_pending_ack(Servers, RaName2, 0), - _ = consume(Ch, QQ2, false). + + {#'basic.get_ok'{delivery_tag = _Tag}, + #amqp_msg{} = Msg} = basic_get(Ch, QQ2, false, 1), + ct:pal("Msg ~p", [Msg]), + flush(1000), + ok. dead_letter_from_classic_to_quorum_queue(Config) -> [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -1463,7 +1468,7 @@ dead_letter_from_classic_to_quorum_queue(Config) -> wait_for_messages_ready(Servers, RaName, 0), wait_for_messages_pending_ack(Servers, RaName, 0), wait_for_messages(Config, [[CQ, <<"1">>, <<"1">>, <<"0">>]]), - DeliveryTag = consume(Ch, CQ, false), + DeliveryTag = basic_get_tag(Ch, CQ, false), wait_for_messages_ready(Servers, RaName, 0), wait_for_messages_pending_ack(Servers, RaName, 0), wait_for_messages(Config, [[CQ, <<"1">>, <<"0">>, <<"1">>]]), @@ -1473,7 +1478,7 @@ dead_letter_from_classic_to_quorum_queue(Config) -> wait_for_messages_ready(Servers, RaName, 1), wait_for_messages_pending_ack(Servers, RaName, 0), wait_for_messages(Config, [[CQ, <<"0">>, <<"0">>, <<"0">>]]), - _ = consume(Ch, QQ, false), + _ = basic_get_tag(Ch, QQ, false), rabbit_ct_client_helpers:close_channel(Ch). cleanup_queue_state_on_channel_after_publish(Config) -> @@ -1872,8 +1877,8 @@ channel_handles_ra_event(Config) -> publish(Ch1, Q2), wait_for_messages(Config, [[Q1, <<"1">>, <<"1">>, <<"0">>]]), wait_for_messages(Config, [[Q2, <<"1">>, <<"1">>, <<"0">>]]), - ?assertEqual(1, consume(Ch1, Q1, false)), - ?assertEqual(2, consume(Ch1, Q2, false)). + ?assertEqual(1, basic_get_tag(Ch1, Q1, false)), + ?assertEqual(2, basic_get_tag(Ch1, Q2, false)). declare_during_node_down(Config) -> [Server, DownServer, _] = Servers = rabbit_ct_broker_helpers:get_node_configs( @@ -3882,7 +3887,7 @@ publish(Ch, Queue, Msg) -> #amqp_msg{props = #'P_basic'{delivery_mode = 2}, payload = Msg}). -consume(Ch, Queue, NoAck) -> +basic_get_tag(Ch, Queue, NoAck) -> {GetOk, _} = Reply = amqp_channel:call(Ch, #'basic.get'{queue = Queue, no_ack = NoAck}), ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg">>}}, Reply), diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index c8b9d0246a91..6803cceace9f 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -14,6 +14,7 @@ -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). %%%=================================================================== @@ -89,9 +90,9 @@ end_per_testcase(_Group, _Config) -> end, Effects))). -define(ASSERT(Guard, Fun), - {assert, fun (S) -> ?assertMatch(Guard, S), Fun end}). + {assert, fun (S) -> ?assertMatch(Guard, S), _ = Fun(S) end}). -define(ASSERT(Guard), - ?ASSERT(Guard, fun () -> ok end)). + ?ASSERT(Guard, fun (_) -> true end)). test_init(Name) -> init(#{name => Name, @@ -609,8 +610,7 @@ return_auto_checked_out_test(Config) -> {State2, _, Effects} = apply(meta(Config, 3), rabbit_fifo:make_return(CKey, [MsgId]), State1), [{log, [1], Fun2, _} | _] = Effects, - [{send_msg, _, {delivery, _, [{_MsgId2, {#{delivery_count := 1, - return_count := 1}, first}}]}, _}] + [{send_msg, _, {delivery, _, [{_MsgId2, {#{acquired_count := 1}, first}}]}, _}] = Fun2([Msg1]), %% a down does not increment the return_count @@ -620,8 +620,8 @@ return_auto_checked_out_test(Config) -> next_msg_id := _}, [_, {log, [1], Fun3, _} ]} = checkout(Config, ?LINE, Cid, 1, State3), - [{send_msg, _, {delivery, _, [{_, {#{delivery_count := 2, - return_count := 1}, first}}]}, _}] + [{send_msg, _, {delivery, _, [{_, {#{delivery_count := 1, + acquired_count := 2}, first}}]}, _}] = Fun3([Msg1]), ok. @@ -641,8 +641,7 @@ requeue_test(Config) -> {_State2, _, Effects} = apply(meta(Config, 3), Requeue, State1), [{log, [_], Fun2, _} | _] = Effects, [{send_msg, _, - {delivery, _, [{_MsgId2, {#{delivery_count := 1, - return_count := 1}, first}}]}, _}] + {delivery, _, [{_MsgId2, {#{acquired_count := 1}, first}}]}, _}] = Fun2([Msg1]), ok. @@ -785,8 +784,10 @@ discarded_message_with_dead_letter_handler_emits_log_effect_test(Config) -> max_in_memory_length => 0, dead_letter_handler => {at_most_once, {somemod, somefun, [somearg]}}}), - Msg1 = rabbit_fifo:make_enqueue(self(), 1, first), - {State0, _} = enq(Config, 1, 1, first, State00), + + Mc = mk_mc(<<"first">>), + Msg1 = rabbit_fifo:make_enqueue(self(), 1, Mc), + {State0, _} = enq(Config, 1, 1, Mc, State00), {State1, #{key := CKey, next_msg_id := MsgId}, Effects1} = checkout(Config, ?LINE, Cid, 10, State0), @@ -796,7 +797,22 @@ discarded_message_with_dead_letter_handler_emits_log_effect_test(Config) -> % assert mod call effect with appended reason and message {value, {log, [1], Fun}} = lists:search(fun (E) -> element(1, E) == log end, Effects2), - ?assertMatch([{mod_call,somemod,somefun,[somearg,rejected,[first]]}], Fun([Msg1])), + [{mod_call, somemod, somefun, [somearg, rejected, [McOut]]}] = Fun([Msg1]), + + ?assertEqual(undefined, mc:get_annotation(acquired_count, McOut)), + ?assertEqual(1, mc:get_annotation(delivery_count, McOut)), + + ok. + +enqueued_msg_with_delivery_count_test(Config) -> + State00 = init(#{name => test, + queue_resource => rabbit_misc:r(<<"/">>, queue, <<"test">>), + max_in_memory_length => 0, + dead_letter_handler => + {at_most_once, {somemod, somefun, [somearg]}}}), + Mc = mc:set_annotation(delivery_count, 2, mk_mc(<<"first">>)), + {#rabbit_fifo{messages = Msgs}, _} = enq(Config, 1, 1, Mc, State00), + ?assertMatch(?MSG(_, #{delivery_count := 2}), rabbit_fifo_q:get(Msgs)), ok. get_log_eff(Effs) -> @@ -2389,35 +2405,49 @@ convert_v3_to_v4(Config) -> ConfigV3 = [{machine_version, 3} | Config], ConfigV4 = [{machine_version, 4} | Config], - Cid1 = {ctag1, self()}, + EPid = test_util:fake_pid(node()), + Pid1 = test_util:fake_pid(node()), + Cid1 = {ctag1, Pid1}, Cid2 = {ctag2, self()}, - MaxCredits = 20, - Entries = [{1, make_checkout(Cid1, {auto, 10, credited}, #{})}, - {2, make_checkout(Cid2, {auto, MaxCredits, simple_prefetch}, - #{prefetch => MaxCredits})}], + MaxCredits = 2, + Entries = [ + {1, rabbit_fifo_v3:make_enqueue(EPid, 1, banana)}, + {2, rabbit_fifo_v3:make_enqueue(EPid, 2, apple)}, + {3, rabbit_fifo_v3:make_enqueue(EPid, 3, orange)}, + {4, make_checkout(Cid1, {auto, 10, credited}, #{})}, + {5, make_checkout(Cid2, {auto, MaxCredits, simple_prefetch}, + #{prefetch => MaxCredits})}, + {6, {down, Pid1, error}}], %% run log in v3 Name = ?FUNCTION_NAME, Init = rabbit_fifo_v3:init( #{name => Name, - max_in_memory_length => 0, queue_resource => rabbit_misc:r("/", queue, atom_to_binary(Name)), release_cursor_interval => 0}), {State, _} = run_log(rabbit_fifo_v3, ConfigV3, Init, Entries, fun (_) -> true end), %% convert from v3 to v4 - {#rabbit_fifo{consumers = Consumers}, ok, _} = - apply(meta(ConfigV4, 4), {machine_version, 3, 4}, State), + {#rabbit_fifo{consumers = Consumers, + returns = Returns}, ok, _} = + apply(meta(ConfigV4, ?LINE), {machine_version, 3, 4}, State), - ?assertEqual(2, maps:size(Consumers)), + ?assertEqual(1, maps:size(Consumers)), ?assertMatch(#consumer{cfg = #consumer_cfg{credit_mode = {simple_prefetch, MaxCredits}}}, maps:get(Cid2, Consumers)), - #consumer{checked_out = Ch1} = maps:get(Cid1, Consumers), - maps:foreach(fun (_MsgId, Msg) -> ?assert(is_tuple(Msg)) end, Ch1), + ?assertNot(is_map_key(Cid1, Consumers)), + %% assert delivery_count is copied to acquired_count #consumer{checked_out = Ch2} = maps:get(Cid2, Consumers), - maps:foreach(fun (_MsgId, Msg) -> ?assert(is_tuple(Msg)) end, Ch2), + ?assertMatch(#{0 := ?MSG(_, #{delivery_count := 1, + acquired_count := 1}), + 1 := ?MSG(_, #{delivery_count := 1, + acquired_count := 1})}, Ch2), + + ?assertMatch(?MSG(_, #{delivery_count := 1, + acquired_count := 1}), lqueue:get(Returns)), + ok. queue_ttl_test(C) -> @@ -2641,6 +2671,72 @@ checkout_metadata_test(Config) -> checkout(Config, ?LINE, Cid, 1, State2), ok. +modify_test(Config) -> + S0 = init(#{name => ?FUNCTION_NAME, + dead_letter_handler => at_least_once, + queue_resource => + rabbit_misc:r("/", queue, ?FUNCTION_NAME_B)}), + + Pid1 = test_util:fake_pid(node()), + % % adding some consumers + E1Idx = ?LINE, + {CK1, C1} = {?LINE, {?LINE_B, self()}}, + Entries = + [ + {E1Idx , rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, + %% add a consumer + {CK1, make_checkout(C1, {auto, {simple_prefetch, 1}}, #{})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{next_msg_id = 1, + checked_out = Ch}}} + when map_size(Ch) == 1), + %% delivery_failed = false, undeliverable_here = false|true + %% this is the same as a requeue, + %% this should not increment the delivery count + {?LINE, rabbit_fifo:make_modify(CK1, [0], false, false, + #{<<"x-opt-blah">> => <<"blah1">>})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{next_msg_id = 2, + checked_out = Ch}}} + when map_size(Ch) == 1, + fun (#rabbit_fifo{consumers = + #{CK1 := #consumer{checked_out = Ch}}}) -> + ?assertMatch( + ?MSG(_, #{acquired_count := 1, + anns := #{<<"x-opt-blah">> := <<"blah1">>}} = H) + when not is_map_key(delivery_count, H), + maps:get(1, Ch)) + end), + %% delivery_failed = true does increment delivery_count + {?LINE, rabbit_fifo:make_modify(CK1, [1], true, false, + #{<<"x-opt-blah">> => <<"blah2">>})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{next_msg_id = 3, + checked_out = Ch}}} + when map_size(Ch) == 1, + fun (#rabbit_fifo{consumers = + #{CK1 := #consumer{checked_out = Ch}}}) -> + ?assertMatch( + ?MSG(_, #{delivery_count := 1, + acquired_count := 2, + anns := #{<<"x-opt-blah">> := <<"blah2">>}}), + maps:get(2, Ch)) + end), + %% delivery_failed = true and undeliverable_here = true is the same as discard + {?LINE, rabbit_fifo:make_modify(CK1, [2], true, true, + #{<<"x-opt-blah">> => <<"blah3">>})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{next_msg_id = 3, + checked_out = Ch}}} + when map_size(Ch) == 0, + fun (#rabbit_fifo{dlx = #rabbit_fifo_dlx{discards = Discards}}) -> + ?assertMatch([[_| + ?MSG(_, #{delivery_count := 2, + acquired_count := 3, + anns := #{<<"x-opt-blah">> := <<"blah3">>}})]], + lqueue:to_list(Discards)) + end) + ], + {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1), + + ok. + %% Utility init(Conf) -> rabbit_fifo:init(Conf). @@ -2658,3 +2754,11 @@ single_active_invariant( #rabbit_fifo{consumers = Cons}) -> S == up end, Cons)). +mk_mc(Body) -> + mc_amqpl:from_basic_message( + #basic_message{routing_keys = [<<"">>], + exchange_name = #resource{name = <<"x">>, + kind = exchange, + virtual_host = <<"v">>}, + content = #content{properties = #'P_basic'{}, + payload_fragments_rev = [Body]}}). diff --git a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl index baf6f72387ac..619fb4e06bdb 100644 --- a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl @@ -29,7 +29,7 @@ -import(rabbit_ct_broker_helpers, [rpc/5, rpc/6]). -import(quorum_queue_SUITE, [publish/2, - consume/3]). + basic_get_tag/3]). -define(DEFAULT_WAIT, 1000). -define(DEFAULT_INTERVAL, 200). @@ -207,7 +207,7 @@ rejected(Config) -> {Server, Ch, SourceQ, TargetQ} = declare_topology(Config, []), publish(Ch, SourceQ), wait_for_messages_ready([Server], ra_name(SourceQ), 1), - DelTag = consume(Ch, SourceQ, false), + DelTag = basic_get_tag(Ch, SourceQ, false), amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DelTag, multiple = false, requeue = false}), @@ -224,7 +224,7 @@ delivery_limit(Config) -> {Server, Ch, SourceQ, TargetQ} = declare_topology(Config, [{<<"x-delivery-limit">>, long, 0}]), publish(Ch, SourceQ), wait_for_messages_ready([Server], ra_name(SourceQ), 1), - DelTag = consume(Ch, SourceQ, false), + DelTag = basic_get_tag(Ch, SourceQ, false), amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DelTag, multiple = false, requeue = true}), diff --git a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl index 1f1552da5477..2ae8e4bc55f8 100644 --- a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl @@ -320,21 +320,38 @@ returns(Config) -> flush(), exit(await_delivery_timeout) end, + {FC5, _} = receive {ra_event, Qname2, - {machine, {delivery, _, [{_MsgId, {_, _Msg1Out}}]}} = Evt2} -> - {ok, _FC4, Actions2} = + {machine, {delivery, _, [{MsgId1, {_, _Msg1Out}}]}} = Evt2} -> + {ok, FC4, Actions2} = rabbit_fifo_client:handle_ra_event(Qname2, Qname2, Evt2, FC3), - % ct:pal("Actions2 ~p", [Actions2]), [{deliver, _tag, true, [{_, _, _, _, Msg1Out}]}] = Actions2, ?assert(mc:is(Msg1Out)), ?assertEqual(1, mc:get_annotation(<<"x-delivery-count">>, Msg1Out)), - ?assertEqual(0, mc:get_annotation(delivery_count, Msg1Out)), - ok + %% delivery_count should _not_ be incremented for a return + ?assertEqual(undefined, mc:get_annotation(delivery_count, Msg1Out)), + rabbit_fifo_client:modify(<<"tag">>, [MsgId1], true, false, #{}, FC4) after 5000 -> flush(), - exit(await_delivery_timeout) + exit(await_delivery_timeout_2) + end, + receive + {ra_event, Qname3, + {machine, {delivery, _, [{MsgId2, {_, _Msg2Out}}]}} = Evt3} -> + {ok, FC6, Actions3} = + rabbit_fifo_client:handle_ra_event(Qname3, Qname3, Evt3, FC5), + [{deliver, _, true, + [{_, _, _, _, Msg2Out}]}] = Actions3, + ?assert(mc:is(Msg2Out)), + ?assertEqual(2, mc:get_annotation(<<"x-delivery-count">>, Msg2Out)), + %% delivery_count should be incremented for a modify with delivery_failed = true + ?assertEqual(1, mc:get_annotation(delivery_count, Msg2Out)), + rabbit_fifo_client:settle(<<"tag">>, [MsgId2], FC6) + after 5000 -> + flush(), + exit(await_delivery_timeout_3) end, rabbit_quorum_queue:stop_server(ServerId), ok. From 374980cc2874e5319433c5885ed7aedf755f2bf6 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 7 Aug 2024 13:09:29 +0100 Subject: [PATCH 42/45] Type tweaks and naming --- deps/rabbit/src/rabbit_fifo.erl | 8 +++---- deps/rabbit/src/rabbit_fifo.hrl | 16 ++++--------- deps/rabbit/test/rabbit_fifo_SUITE.erl | 32 +++++++++++++------------- 3 files changed, 25 insertions(+), 31 deletions(-) diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 82ca4c42c340..95c7b0fcf596 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -1503,7 +1503,7 @@ activate_next_consumer(#?STATE{consumers = Cons0, %% A higher priority consumer has attached but the current one has %% pending messages Cons = maps:update(ActiveCKey, - Active#consumer{status = fading}, + Active#consumer{status = quiescing}, Cons0), {State0#?STATE{consumers = Cons}, Effects0}; _ -> @@ -1512,7 +1512,7 @@ activate_next_consumer(#?STATE{consumers = Cons0, end. active_consumer({CKey, #consumer{status = Status} = Consumer, _I}) - when Status == up orelse Status == fading -> + when Status == up orelse Status == quiescing -> {CKey, Consumer}; active_consumer({_CKey, #consumer{status = _}, I}) -> active_consumer(maps:next(I)); @@ -1771,7 +1771,7 @@ complete_and_checkout(#{} = Meta, MsgIds, ConsumerKey, #consumer{} = Con0, Effects0, State0) -> State1 = complete(Meta, ConsumerKey, MsgIds, Con0, State0), - %% a completion could have removed the active/fading consumer + %% a completion could have removed the active/quiescing consumer {State2, Effects1} = activate_next_consumer(State1, Effects0), checkout(Meta, State0, State2, Effects1). @@ -2185,7 +2185,7 @@ update_or_remove_con(Meta, ConsumerKey, State#?STATE{consumers = maps:put(ConsumerKey, Con, Cons)} end; update_or_remove_con(_Meta, ConsumerKey, - #consumer{status = fading, + #consumer{status = quiescing, checked_out = Checked} = Con0, #?STATE{consumers = Cons, waiting_consumers = Waiting} = State) diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl index d7a5de7fdb0d..a436b5df8adf 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -39,8 +39,8 @@ -type msg_header() :: msg_size() | optimised_tuple(msg_size(), Expiry :: milliseconds()) | #{size := msg_size(), + acquired_count => non_neg_integer(), delivery_count => non_neg_integer(), - return_count => non_neg_integer(), expiry => milliseconds()}. %% The message header: %% size: The size of the message payload in bytes. @@ -55,7 +55,7 @@ -type msg_size() :: non_neg_integer(). %% the size in bytes of the msg payload --type msg() :: optimised_tuple(option(ra:index()), msg_header()). +-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. @@ -91,8 +91,6 @@ prefetch => non_neg_integer(), args => list(), priority => non_neg_integer() - % %% set if and only if credit API v2 is in use - % initial_delivery_count => rabbit_queue_type:delivery_count() }. %% static meta data associated with a consumer @@ -126,7 +124,7 @@ -record(consumer, {cfg = #consumer_cfg{}, - status = up :: up | suspected_down | cancelled | fading, + 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 @@ -174,10 +172,6 @@ unused_2 }). --type prefix_msgs() :: {list(), list()} | - {non_neg_integer(), list(), - non_neg_integer(), list()}. - -record(rabbit_fifo, {cfg :: #cfg{}, % unassigned messages @@ -199,7 +193,7 @@ ra_indexes = rabbit_fifo_index:empty() :: rabbit_fifo_index:state(), unused_1, % consumers need to reflect consumer state at time of snapshot - consumers = #{} :: #{consumer_id() | ra:index() => consumer()}, + 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 @@ -208,7 +202,7 @@ 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_id() | ra:index(), consumer()}], + waiting_consumers = [] :: [{consumer_key(), consumer()}], last_active :: option(non_neg_integer()), msg_cache :: option({ra:index(), raw_msg()}), unused_2 diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl index 6803cceace9f..753704affd09 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -1723,7 +1723,7 @@ single_active_consumer_priority_test(Config) -> %% add en even higher consumer, but the current active has a message pending %% so can't be immedately replaced {CK3, make_checkout(C3, {auto, {simple_prefetch, 1}}, #{priority => 3})}, - ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = fading}}, + ?ASSERT(#rabbit_fifo{consumers = #{CK2 := #consumer{status = quiescing}}, waiting_consumers = [_, _]}), %% settle the message, the higher priority should become the active, %% completing the replacement @@ -1811,7 +1811,7 @@ single_active_consumer_update_priority_test(Config) -> {_S1, _} = run_log(Config, S0, Entries, fun single_active_invariant/1), ok. -single_active_consumer_fading_resumes_after_cancel_test(Config) -> +single_active_consumer_quiescing_resumes_after_cancel_test(Config) -> S0 = init(#{name => ?FUNCTION_NAME, queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), single_active_consumer_on => true}), @@ -1830,14 +1830,14 @@ single_active_consumer_fading_resumes_after_cancel_test(Config) -> %% enqueue a message {?LINE , rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, - %% add a consumer with a higher priority, current is fading + %% add a consumer with a higher priority, current is quiescing {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 2})}, - ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = fading}}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = quiescing}}, waiting_consumers = [{CK2, _}]}), %% C2 cancels {?LINE, make_checkout(C2, cancel, #{})}, - ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = fading, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = quiescing, checked_out = Ch}}, waiting_consumers = []} when map_size(Ch) == 1), @@ -1874,14 +1874,14 @@ single_active_consumer_higher_waiting_disconnected_test(Config) -> %% enqueue a message {?LINE , rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, - %% add a consumer with a higher priority, current is fading + %% add a consumer with a higher priority, current is quiescing {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 2})}, - ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = fading}}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = quiescing}}, waiting_consumers = [{CK2, _}]}), %% C2 is disconnected, {?LINE, {down, C2Pid, noconnection}}, ?ASSERT( - #rabbit_fifo{consumers = #{CK1 := #consumer{status = fading}}, + #rabbit_fifo{consumers = #{CK1 := #consumer{status = quiescing}}, waiting_consumers = [{CK2, #consumer{status = suspected_down}}]}), %% settle {?LINE, rabbit_fifo:make_settle(CK1, [0])}, @@ -1899,7 +1899,7 @@ single_active_consumer_higher_waiting_disconnected_test(Config) -> ok. -single_active_consumer_fading_disconnected_test(Config) -> +single_active_consumer_quiescing_disconnected_test(Config) -> S0 = init(#{name => ?FUNCTION_NAME, queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), single_active_consumer_on => true}), @@ -1920,9 +1920,9 @@ single_active_consumer_fading_disconnected_test(Config) -> %% enqueue a message {?LINE , rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, - %% add a consumer with a higher priority, current is fading + %% add a consumer with a higher priority, current is quiescing {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 2})}, - ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = fading}}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = quiescing}}, waiting_consumers = [{CK2, _}]}), %% C1 is disconnected, {?LINE, {down, C1Pid, noconnection}}, @@ -1954,7 +1954,7 @@ single_active_consumer_fading_disconnected_test(Config) -> ok. -single_active_consumer_fading_receives_no_further_messages_test(Config) -> +single_active_consumer_quiescing_receives_no_further_messages_test(Config) -> S0 = init(#{name => ?FUNCTION_NAME, queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), single_active_consumer_on => true}), @@ -1975,17 +1975,17 @@ single_active_consumer_fading_receives_no_further_messages_test(Config) -> %% enqueue a message {?LINE, rabbit_fifo:make_enqueue(Pid1, 1, msg1)}, - %% add a consumer with a higher priority, current is fading + %% add a consumer with a higher priority, current is quiescing {CK2, make_checkout(C2, {auto, {simple_prefetch, 10}}, #{priority => 2})}, - ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = fading, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = quiescing, checked_out = Ch}}, waiting_consumers = [{CK2, _}]} when map_size(Ch) == 1), %% enqueue another message {?LINE, rabbit_fifo:make_enqueue(Pid1, 2, msg2)}, - %% message should not be assinged to fading consumer - ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = fading, + %% message should not be assinged to quiescing consumer + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = quiescing, checked_out = Ch}}, waiting_consumers = [{CK2, _}]} when map_size(Ch) == 1) From be05f9a23045cb1204e157c2f696108c791394f3 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 7 Aug 2024 15:19:33 +0200 Subject: [PATCH 43/45] Add test for modified outcome with classic queue --- deps/rabbit/src/rabbit_amqp_session.erl | 4 +- deps/rabbit/src/rabbit_classic_queue.erl | 2 +- deps/rabbit/src/rabbit_fifo.erl | 37 ++++--- deps/rabbit/src/rabbit_fifo_dlx.erl | 4 +- deps/rabbit/test/amqp_client_SUITE.erl | 118 ++++++++++++++++++----- 5 files changed, 116 insertions(+), 49 deletions(-) diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl index 33c48b3a990d..3b527d3d838c 100644 --- a/deps/rabbit/src/rabbit_amqp_session.erl +++ b/deps/rabbit/src/rabbit_amqp_session.erl @@ -1862,7 +1862,7 @@ settle_op_from_outcome(#'v1_0.rejected'{}) -> settle_op_from_outcome(#'v1_0.released'{}) -> requeue; -%% RabbitMQ does not support any of the modified outcome fields correctly. +%% Not all queue types support the modified outcome fields correctly. %% However, we still allow the client to settle with the modified outcome %% because some client libraries such as Apache QPid make use of it: %% https://github.com/apache/qpid-jms/blob/90eb60f59cb59b7b9ad8363ee8a843d6903b8e77/qpid-jms-client/src/main/java/org/apache/qpid/jms/JmsMessageConsumer.java#L464 @@ -1871,7 +1871,7 @@ settle_op_from_outcome(#'v1_0.released'{}) -> settle_op_from_outcome(#'v1_0.modified'{delivery_failed = DelFailed, undeliverable_here = UndelHere, message_annotations = Anns0 - }) -> + }) -> Anns = case Anns0 of #'v1_0.message_annotations'{content = C} -> C; diff --git a/deps/rabbit/src/rabbit_classic_queue.erl b/deps/rabbit/src/rabbit_classic_queue.erl index 2907d3dbc428..2da8d55f7a6f 100644 --- a/deps/rabbit/src/rabbit_classic_queue.erl +++ b/deps/rabbit/src/rabbit_classic_queue.erl @@ -333,7 +333,7 @@ cancel(Q, Spec, State) -> -spec settle(rabbit_amqqueue:name(), rabbit_queue_type:settle_op(), rabbit_types:ctag(), [non_neg_integer()], state()) -> {state(), rabbit_queue_type:actions()}. -settle(QName, {modify, _DelFailed, Undel, _}, CTag, MsgIds, State) -> +settle(QName, {modify, _DelFailed, Undel, _Anns}, CTag, MsgIds, State) -> %% translate modify into other op Op = case Undel of true -> diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl index 95c7b0fcf596..7d357beadc13 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -272,8 +272,8 @@ apply(Meta, #discard{consumer_key = ConsumerKey, end; apply(Meta, #return{consumer_key = ConsumerKey, msg_ids = MsgIds}, - #?STATE{consumers = Cons0} = State) -> - case find_consumer(ConsumerKey, Cons0) of + #?STATE{consumers = Cons} = State) -> + case find_consumer(ConsumerKey, Cons) of {ActualConsumerKey, #consumer{checked_out = Checked}} -> return(Meta, ActualConsumerKey, MsgIds, false, #{}, Checked, [], State); @@ -285,34 +285,34 @@ apply(Meta, #modify{consumer_key = ConsumerKey, undeliverable_here = Undel, annotations = Anns, msg_ids = MsgIds}, - #?STATE{consumers = Cons0} = State0) -> - case find_consumer(ConsumerKey, Cons0) of + #?STATE{consumers = Cons} = State) -> + case find_consumer(ConsumerKey, Cons) of {ConsumerKey, #consumer{checked_out = Checked}} when Undel == false -> return(Meta, ConsumerKey, MsgIds, DelFailed, - Anns, Checked, [], State0); + Anns, Checked, [], State); {ConsumerKey, #consumer{} = Con} when Undel == true -> - discard(Meta, MsgIds, ConsumerKey, Con, DelFailed, Anns, State0); + discard(Meta, MsgIds, ConsumerKey, Con, DelFailed, Anns, State); _ -> - {State0, ok} + {State, ok} end; apply(#{index := Idx} = Meta, #requeue{consumer_key = ConsumerKey, msg_id = MsgId, index = OldIdx, header = Header0}, - #?STATE{consumers = Cons0, + #?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 Cons0 of + 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 delivery count before adding it to the message queue + %% 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), @@ -660,9 +660,7 @@ convert_v3_to_v4(#{} = _Meta, StateV3) -> (_, Msg) -> Msg end, Ch0), - C#consumer{checked_out = Ch}; - (_, Msg) -> - Msg + C#consumer{checked_out = Ch} end, Consumers0), Returns = lqueue:from_list( lists:map(fun (?MSG(I, #{delivery_count := DC} = H)) -> @@ -696,7 +694,7 @@ purge_node(Meta, Node, State, Effects) -> end, {State, Effects}, all_pids_for(Node, State)). -%% any downs that re not noconnection +%% any downs that are not noconnection handle_down(Meta, Pid, #?STATE{consumers = Cons0, enqueuers = Enqs0} = State0) -> % Remove any enqueuer for the down pid @@ -1827,11 +1825,10 @@ annotate_msg(Header, Msg0) -> Msg = maps:fold(fun (K, V, Acc) -> mc:set_annotation(K, V, Acc) end, Msg0, maps:get(anns, Header, #{})), - case is_map_key(delivery_count, Header) of - true -> - mc:set_annotation(delivery_count, - maps:get(delivery_count, Header), Msg); - false -> + case Header of + #{delivery_count := DelCount} -> + mc:set_annotation(delivery_count, DelCount, Msg); + _ -> Msg end; _ -> @@ -1849,7 +1846,7 @@ return_one(Meta, MsgId, ?MSG(_, _) = Msg0, DelivFailed, Anns, Msg = incr_msg(Msg0, DelivFailed, Anns), Header = get_msg_header(Msg), case get_header(acquired_count, Header) of - DeliveryCount when DeliveryCount > DeliveryLimit -> + AcquiredCount when AcquiredCount > DeliveryLimit -> {DlxState, DlxEffects} = rabbit_fifo_dlx:discard([Msg], delivery_limit, DLH, DlxState0), State1 = State0#?STATE{dlx = DlxState}, diff --git a/deps/rabbit/src/rabbit_fifo_dlx.erl b/deps/rabbit/src/rabbit_fifo_dlx.erl index c4c9dd488c2f..4e787172d1a4 100644 --- a/deps/rabbit/src/rabbit_fifo_dlx.erl +++ b/deps/rabbit/src/rabbit_fifo_dlx.erl @@ -362,5 +362,5 @@ dehydrate(State) -> smallest_raft_index(#?MODULE{ra_indexes = Indexes}) -> rabbit_fifo_index:smallest(Indexes). -annotate_msg(H, M0) -> - rabbit_fifo:annotate_msg(H, M0). +annotate_msg(H, Msg) -> + rabbit_fifo:annotate_msg(H, Msg). diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl index dcb6625bfa5b..5db2f17d56a0 100644 --- a/deps/rabbit/test/amqp_client_SUITE.erl +++ b/deps/rabbit/test/amqp_client_SUITE.erl @@ -41,7 +41,8 @@ groups() -> [ reliable_send_receive_with_outcomes_classic_queue, reliable_send_receive_with_outcomes_quorum_queue, - modified, + modified_classic_queue, + modified_quorum_queue, sender_settle_mode_unsettled, sender_settle_mode_unsettled_fanout, sender_settle_mode_mixed, @@ -403,11 +404,59 @@ reliable_send_receive(QType, Outcome, Config) -> ok = end_session_sync(Session2), ok = amqp10_client:close_connection(Connection2). -%% This test case doesn't expect the correct AMQP spec behavivour. -%% We know that RabbitMQ doesn't implement the modified outcome correctly. -%% Here, we test RabbitMQ's workaround behaviour: -%% RabbitMQ discards if undeliverable-here is true. Otherwise, RabbitMQ requeues. -modified(Config) -> +%% We test the modified outcome with classic queues. +%% We expect that classic queues implement field undeliverable-here incorrectly +%% by discarding (if true) or requeueing (if false). +%% Fields delivery-failed and message-annotations are not implemented. +modified_classic_queue(Config) -> + QName = atom_to_binary(?FUNCTION_NAME), + {Connection, Session, LinkPair} = init(Config), + {ok, #{type := <<"classic">>}} = rabbitmq_amqp_client:declare_queue( + LinkPair, QName, + #{arguments => #{<<"x-queue-type">> => {utf8, <<"classic">>}}}), + Address = rabbitmq_amqp_address:queue(QName), + {ok, Sender} = amqp10_client:attach_sender_link(Session, <<"sender">>, Address), + ok = wait_for_credit(Sender), + + Msg1 = amqp10_msg:new(<<"tag1">>, <<"m1">>, true), + Msg2 = amqp10_msg:new(<<"tag2">>, <<"m2">>, true), + ok = amqp10_client:send_msg(Sender, Msg1), + ok = amqp10_client:send_msg(Sender, Msg2), + ok = amqp10_client:detach_link(Sender), + + {ok, Receiver} = amqp10_client:attach_receiver_link(Session, <<"receiver">>, Address, unsettled), + + {ok, M1} = amqp10_client:get_msg(Receiver), + ?assertEqual([<<"m1">>], amqp10_msg:body(M1)), + ok = amqp10_client:settle_msg(Receiver, M1, {modified, false, true, #{}}), + + {ok, M2a} = amqp10_client:get_msg(Receiver), + ?assertEqual([<<"m2">>], amqp10_msg:body(M2a)), + ok = amqp10_client:settle_msg(Receiver, M2a, + {modified, false, false, #{}}), + + {ok, M2b} = amqp10_client:get_msg(Receiver), + ?assertEqual([<<"m2">>], amqp10_msg:body(M2b)), + ok = amqp10_client:settle_msg(Receiver, M2b, + {modified, true, false, #{<<"x-opt-key">> => <<"val">>}}), + + {ok, M2c} = amqp10_client:get_msg(Receiver), + ?assertEqual([<<"m2">>], amqp10_msg:body(M2c)), + ok = amqp10_client:settle_msg(Receiver, M2c, modified), + + ok = amqp10_client:detach_link(Receiver), + ?assertMatch({ok, #{message_count := 1}}, + rabbitmq_amqp_client:delete_queue(LinkPair, QName)), + ok = rabbitmq_amqp_client:detach_management_link_pair_sync(LinkPair), + ok = end_session_sync(Session), + ok = amqp10_client:close_connection(Connection). + +%% We test the modified outcome with quorum queues. +%% We expect that quorum queues implement field +%% * delivery-failed correctly +%% * undeliverable-here incorrectly by discarding (if true) or requeueing (if false) +%% * message-annotations correctly +modified_quorum_queue(Config) -> QName = atom_to_binary(?FUNCTION_NAME), {Connection, Session, LinkPair} = init(Config), {ok, #{type := <<"quorum">>}} = rabbitmq_amqp_client:declare_queue( @@ -427,34 +476,53 @@ modified(Config) -> {ok, M1} = amqp10_client:get_msg(Receiver), ?assertEqual([<<"m1">>], amqp10_msg:body(M1)), - ?assertEqual(0, amqp10_msg:header(delivery_count, M1)), + ?assertMatch(#{delivery_count := 0, + first_acquirer := true}, + amqp10_msg:headers(M1)), + ok = amqp10_client:settle_msg(Receiver, M1, {modified, false, true, #{}}), - ok = amqp10_client:settle_msg(Receiver, M1, {modified, false, - _UndeliverableHere = true, #{}}), {ok, M2a} = amqp10_client:get_msg(Receiver), ?assertEqual([<<"m2">>], amqp10_msg:body(M2a)), - ?assertEqual(0, amqp10_msg:header(delivery_count, M2a)), - + ?assertMatch(#{delivery_count := 0, + first_acquirer := true}, + amqp10_msg:headers(M2a)), ok = amqp10_client:settle_msg(Receiver, M2a, {modified, false, false, #{}}), + {ok, M2b} = amqp10_client:get_msg(Receiver), ?assertEqual([<<"m2">>], amqp10_msg:body(M2b)), - ?assertEqual(0, amqp10_msg:header(delivery_count, M2b)), - + ?assertMatch(#{delivery_count := 0, + first_acquirer := false}, + amqp10_msg:headers(M2b)), ok = amqp10_client:settle_msg(Receiver, M2b, {modified, true, false, #{}}), + {ok, M2c} = amqp10_client:get_msg(Receiver), - ?assertEqual(1, amqp10_msg:header(delivery_count, M2c)), + ?assertMatch(#{delivery_count := 1, + first_acquirer := false}, + amqp10_msg:headers(M2c)), ?assertEqual([<<"m2">>], amqp10_msg:body(M2c)), - - ok = amqp10_client:settle_msg(Receiver, M2c, {modified, true, false, - #{<<"x-opt-key">> => <<"val">>}}), + #{<<"x-opt-key">> => <<"val 1">>}}), {ok, M2d} = amqp10_client:get_msg(Receiver), ?assertEqual([<<"m2">>], amqp10_msg:body(M2d)), - ?assertEqual(2, amqp10_msg:header(delivery_count, M2d)), - ?assertMatch(#{<<"x-opt-key">> := <<"val">>}, amqp10_msg:message_annotations(M2d)), - ok = amqp10_client:settle_msg(Receiver, M2d, modified), + ?assertMatch(#{delivery_count := 2, + first_acquirer := false}, + amqp10_msg:headers(M2d)), + ?assertMatch(#{<<"x-opt-key">> := <<"val 1">>}, amqp10_msg:message_annotations(M2d)), + ok = amqp10_client:settle_msg(Receiver, M2d, + {modified, false, false, + #{<<"x-opt-key">> => <<"val 2">>, + <<"x-other">> => 99}}), + + {ok, M2e} = amqp10_client:get_msg(Receiver), + ?assertEqual([<<"m2">>], amqp10_msg:body(M2e)), + ?assertMatch(#{delivery_count := 2, + first_acquirer := false}, + amqp10_msg:headers(M2e)), + ?assertMatch(#{<<"x-opt-key">> := <<"val 2">>, + <<"x-other">> := 99}, amqp10_msg:message_annotations(M2e)), + ok = amqp10_client:settle_msg(Receiver, M2e, modified), ok = amqp10_client:detach_link(Receiver), ?assertMatch({ok, #{message_count := 1}}, @@ -4429,6 +4497,7 @@ dead_letter_reject(Config) -> QName1, #{arguments => #{<<"x-queue-type">> => {utf8, <<"quorum">>}, <<"x-message-ttl">> => {ulong, 20}, + <<"x-overflow">> => {utf8, <<"reject-publish">>}, <<"x-dead-letter-strategy">> => {utf8, <<"at-least-once">>}, <<"x-dead-letter-exchange">> => {utf8, <<>>}, <<"x-dead-letter-routing-key">> => {utf8, QName2} @@ -4463,22 +4532,22 @@ dead_letter_reject(Config) -> ?assertMatch(#{delivery_count := 0, first_acquirer := true}, amqp10_msg:headers(Msg1)), ok = amqp10_client:settle_msg(Receiver, Msg1, rejected), + {ok, Msg2} = amqp10_client:get_msg(Receiver), ?assertMatch(#{delivery_count := 1, first_acquirer := false}, amqp10_msg:headers(Msg2)), ok = amqp10_client:settle_msg(Receiver, Msg2, {modified, true, true, #{<<"x-opt-thekey">> => <<"val">>}}), + {ok, Msg3} = amqp10_client:get_msg(Receiver), ?assertMatch(#{delivery_count := 2, first_acquirer := false}, amqp10_msg:headers(Msg3)), - ?assertMatch(#{<<"x-opt-thekey">> := <<"val">>}, - amqp10_msg:message_annotations(Msg3)), - ok = amqp10_client:settle_msg(Receiver, Msg3, accepted), ?assertEqual(Body, amqp10_msg:body_bin(Msg3)), Annotations = amqp10_msg:message_annotations(Msg3), ?assertMatch( - #{<<"x-first-death-queue">> := QName1, + #{<<"x-opt-thekey">> := <<"val">>, + <<"x-first-death-queue">> := QName1, <<"x-first-death-exchange">> := <<>>, <<"x-first-death-reason">> := <<"expired">>, <<"x-last-death-queue">> := QName1, @@ -4516,6 +4585,7 @@ dead_letter_reject(Config) -> ]} = D3, ?assertEqual([Ts1, Ts3, Ts5, Ts4, Ts6, Ts2], lists:sort([Ts1, Ts2, Ts3, Ts4, Ts5, Ts6])), + ok = amqp10_client:settle_msg(Receiver, Msg3, accepted), ok = amqp10_client:detach_link(Receiver), ok = amqp10_client:detach_link(Sender), From 72d855d7338835eeed256d68e6f765b5eb24d02a Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 7 Aug 2024 17:59:20 +0200 Subject: [PATCH 44/45] Add test routing on message-annotations in modified outcome --- deps/rabbit/src/mc.erl | 4 +- deps/rabbit/src/mc_amqp.erl | 15 +++- deps/rabbit/test/amqp_client_SUITE.erl | 111 +++++++++++++++++++++++-- 3 files changed, 120 insertions(+), 10 deletions(-) diff --git a/deps/rabbit/src/mc.erl b/deps/rabbit/src/mc.erl index 74704c25c2b6..465c7054f089 100644 --- a/deps/rabbit/src/mc.erl +++ b/deps/rabbit/src/mc.erl @@ -383,6 +383,7 @@ record_death(Reason, SourceQueue, routing_keys = RKeys, count = 1, anns = DeathAnns}, + ReasonBin = atom_to_binary(Reason), Anns = case Anns0 of #{deaths := Deaths0} -> Deaths = case Deaths0 of @@ -406,7 +407,7 @@ record_death(Reason, SourceQueue, [{Key, NewDeath} | Deaths0] end end, - Anns0#{<<"x-last-death-reason">> := atom_to_binary(Reason), + Anns0#{<<"x-last-death-reason">> := ReasonBin, <<"x-last-death-queue">> := SourceQueue, <<"x-last-death-exchange">> := Exchange, deaths := Deaths}; @@ -419,7 +420,6 @@ record_death(Reason, SourceQueue, _ -> [{Key, NewDeath}] end, - ReasonBin = atom_to_binary(Reason), Anns0#{<<"x-first-death-reason">> => ReasonBin, <<"x-first-death-queue">> => SourceQueue, <<"x-first-death-exchange">> => Exchange, diff --git a/deps/rabbit/src/mc_amqp.erl b/deps/rabbit/src/mc_amqp.erl index 16cbdfa6e828..be63597c3f96 100644 --- a/deps/rabbit/src/mc_amqp.erl +++ b/deps/rabbit/src/mc_amqp.erl @@ -561,10 +561,19 @@ binary_part_bare_and_footer(Payload, Start) -> update_header_from_anns(undefined, Anns) -> update_header_from_anns(#'v1_0.header'{durable = true}, Anns); update_header_from_anns(Header, Anns) -> - FirstAcq = not maps:get(redelivered, Anns, false), - DeliveryCount = {uint, maps:get(delivery_count, Anns, 0)}, + DeliveryCount = case Anns of + #{delivery_count := C} -> C; + _ -> 0 + end, + Redelivered = case Anns of + #{redelivered := R} -> R; + _ -> false + end, + FirstAcq = not Redelivered andalso + DeliveryCount =:= 0 andalso + not is_map_key(deaths, Anns), Header#'v1_0.header'{first_acquirer = FirstAcq, - delivery_count = DeliveryCount}. + delivery_count = {uint, DeliveryCount}}. encode_deaths(Deaths) -> lists:map( diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl index 5db2f17d56a0..c9b4b21b103b 100644 --- a/deps/rabbit/test/amqp_client_SUITE.erl +++ b/deps/rabbit/test/amqp_client_SUITE.erl @@ -41,8 +41,6 @@ groups() -> [ reliable_send_receive_with_outcomes_classic_queue, reliable_send_receive_with_outcomes_quorum_queue, - modified_classic_queue, - modified_quorum_queue, sender_settle_mode_unsettled, sender_settle_mode_unsettled_fanout, sender_settle_mode_mixed, @@ -116,6 +114,9 @@ groups() -> handshake_timeout, credential_expires, attach_to_exclusive_queue, + modified_classic_queue, + modified_quorum_queue, + modified_dead_letter_headers_exchange, dead_letter_headers_exchange, dead_letter_reject, dead_letter_reject_message_order_classic_queue, @@ -496,10 +497,10 @@ modified_quorum_queue(Config) -> ok = amqp10_client:settle_msg(Receiver, M2b, {modified, true, false, #{}}), {ok, M2c} = amqp10_client:get_msg(Receiver), + ?assertEqual([<<"m2">>], amqp10_msg:body(M2c)), ?assertMatch(#{delivery_count := 1, first_acquirer := false}, amqp10_msg:headers(M2c)), - ?assertEqual([<<"m2">>], amqp10_msg:body(M2c)), ok = amqp10_client:settle_msg(Receiver, M2c, {modified, true, false, #{<<"x-opt-key">> => <<"val 1">>}}), @@ -531,6 +532,107 @@ modified_quorum_queue(Config) -> ok = end_session_sync(Session), ok = amqp10_client:close_connection(Connection). +%% Test that a message can be routed based on the message-annotations +%% provided in the modified outcome. +modified_dead_letter_headers_exchange(Config) -> + {Connection, Session, LinkPair} = init(Config), + SourceQName = <<"source quorum queue">>, + AppleQName = <<"dead letter classic queue receiving apples">>, + BananaQName = <<"dead letter quorum queue receiving bananas">>, + {ok, #{type := <<"quorum">>}} = rabbitmq_amqp_client:declare_queue( + LinkPair, + SourceQName, + #{arguments => #{<<"x-queue-type">> => {utf8, <<"quorum">>}, + <<"x-overflow">> => {utf8, <<"reject-publish">>}, + <<"x-dead-letter-strategy">> => {utf8, <<"at-least-once">>}, + <<"x-dead-letter-exchange">> => {utf8, <<"amq.headers">>}}}), + {ok, #{type := <<"classic">>}} = rabbitmq_amqp_client:declare_queue( + LinkPair, + AppleQName, + #{arguments => #{<<"x-queue-type">> => {utf8, <<"classic">>}}}), + {ok, #{type := <<"quorum">>}} = rabbitmq_amqp_client:declare_queue( + LinkPair, + BananaQName, + #{arguments => #{<<"x-queue-type">> => {utf8, <<"quorum">>}}}), + ok = rabbitmq_amqp_client:bind_queue( + LinkPair, AppleQName, <<"amq.headers">>, <<>>, + #{<<"x-fruit">> => {utf8, <<"apple">>}, + <<"x-match">> => {utf8, <<"any-with-x">>}}), + ok = rabbitmq_amqp_client:bind_queue( + LinkPair, BananaQName, <<"amq.headers">>, <<>>, + #{<<"x-fruit">> => {utf8, <<"banana">>}, + <<"x-match">> => {utf8, <<"any-with-x">>}}), + + {ok, Sender} = amqp10_client:attach_sender_link( + Session, <<"test-sender">>, rabbitmq_amqp_address:queue(SourceQName)), + wait_for_credit(Sender), + {ok, Receiver} = amqp10_client:attach_receiver_link( + Session, <<"receiver">>, rabbitmq_amqp_address:queue(SourceQName), unsettled), + {ok, ReceiverApple} = amqp10_client:attach_receiver_link( + Session, <<"receiver apple">>, rabbitmq_amqp_address:queue(AppleQName), unsettled), + {ok, ReceiverBanana} = amqp10_client:attach_receiver_link( + Session, <<"receiver banana">>, rabbitmq_amqp_address:queue(BananaQName), unsettled), + + ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"t1">>, <<"m1">>)), + ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"t2">>, <<"m2">>)), + ok = amqp10_client:send_msg(Sender, amqp10_msg:set_message_annotations( + #{"x-fruit" => <<"apple">>}, + amqp10_msg:new(<<"t3">>, <<"m3">>))), + ok = amqp10_client:send_msg(Sender, amqp10_msg:set_message_annotations( + #{"x-fruit" => <<"apple">>}, + amqp10_msg:new(<<"t4">>, <<"m4">>))), + ok = wait_for_accepts(3), + + {ok, Msg1} = amqp10_client:get_msg(Receiver), + ?assertMatch(#{delivery_count := 0, + first_acquirer := true}, + amqp10_msg:headers(Msg1)), + ok = amqp10_client:settle_msg(Receiver, Msg1, {modified, true, true, #{<<"x-fruit">> => <<"banana">>}}), + {ok, MsgBanana1} = amqp10_client:get_msg(ReceiverBanana), + ?assertEqual([<<"m1">>], amqp10_msg:body(MsgBanana1)), + ?assertMatch(#{delivery_count := 1, + first_acquirer := false}, + amqp10_msg:headers(MsgBanana1)), + ok = amqp10_client:accept_msg(ReceiverBanana, MsgBanana1), + + {ok, Msg2} = amqp10_client:get_msg(Receiver), + ok = amqp10_client:settle_msg(Receiver, Msg2, {modified, true, true, #{<<"x-fruit">> => <<"apple">>}}), + {ok, MsgApple1} = amqp10_client:get_msg(ReceiverApple), + ?assertEqual([<<"m2">>], amqp10_msg:body(MsgApple1)), + ?assertMatch(#{delivery_count := 1, + first_acquirer := false}, + amqp10_msg:headers(MsgApple1)), + ok = amqp10_client:accept_msg(ReceiverApple, MsgApple1), + + {ok, Msg3} = amqp10_client:get_msg(Receiver), + ok = amqp10_client:settle_msg(Receiver, Msg3, {modified, false, true, #{}}), + {ok, MsgApple2} = amqp10_client:get_msg(ReceiverApple), + ?assertEqual([<<"m3">>], amqp10_msg:body(MsgApple2)), + ?assertMatch(#{delivery_count := 0, + first_acquirer := false}, + amqp10_msg:headers(MsgApple2)), + ok = amqp10_client:accept_msg(ReceiverApple, MsgApple2), + + {ok, Msg4} = amqp10_client:get_msg(Receiver), + ok = amqp10_client:settle_msg(Receiver, Msg4, {modified, false, true, #{<<"x-fruit">> => <<"banana">>}}), + {ok, MsgBanana2} = amqp10_client:get_msg(ReceiverBanana), + ?assertEqual([<<"m4">>], amqp10_msg:body(MsgBanana2)), + ?assertMatch(#{delivery_count := 0, + first_acquirer := false}, + amqp10_msg:headers(MsgBanana2)), + ok = amqp10_client:accept_msg(ReceiverBanana, MsgBanana2), + + ok = detach_link_sync(Sender), + ok = detach_link_sync(Receiver), + ok = detach_link_sync(ReceiverApple), + ok = detach_link_sync(ReceiverBanana), + {ok, #{message_count := 0}} = rabbitmq_amqp_client:delete_queue(LinkPair, SourceQName), + {ok, #{message_count := 0}} = rabbitmq_amqp_client:delete_queue(LinkPair, AppleQName), + {ok, #{message_count := 0}} = rabbitmq_amqp_client:delete_queue(LinkPair, BananaQName), + ok = rabbitmq_amqp_client:detach_management_link_pair_sync(LinkPair), + ok = end_session_sync(Session), + ok = amqp10_client:close_connection(Connection). + %% Tests that confirmations are returned correctly %% when sending many messages async to a quorum queue. sender_settle_mode_unsettled(Config) -> @@ -4529,8 +4631,7 @@ dead_letter_reject(Config) -> ok = wait_for_accepted(Tag), {ok, Msg1} = amqp10_client:get_msg(Receiver), - ?assertMatch(#{delivery_count := 0, - first_acquirer := true}, amqp10_msg:headers(Msg1)), + ?assertMatch(#{delivery_count := 0}, amqp10_msg:headers(Msg1)), ok = amqp10_client:settle_msg(Receiver, Msg1, rejected), {ok, Msg2} = amqp10_client:get_msg(Receiver), From 84f262a5b0b0d778991ef27b9e8ad3b38d9bce6f Mon Sep 17 00:00:00 2001 From: David Ansari Date: Wed, 7 Aug 2024 19:07:46 +0200 Subject: [PATCH 45/45] Skip tests in mixed version tests Skip tests in mixed version tests because feature flag rabbitmq_4.0.0 is needed for the new #modify{} Ra command being sent to quorum queues. --- deps/rabbit/test/amqp_client_SUITE.erl | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl index c9b4b21b103b..d70a278222c0 100644 --- a/deps/rabbit/test/amqp_client_SUITE.erl +++ b/deps/rabbit/test/amqp_client_SUITE.erl @@ -246,6 +246,16 @@ init_per_testcase(T, Config) false -> {skip, "Feature flag rabbitmq_4.0.0 is disabled"} end; +init_per_testcase(T, Config) + when T =:= modified_quorum_queue orelse + T =:= modified_dead_letter_headers_exchange -> + case rpc(Config, rabbit_feature_flags, is_enabled, ['rabbitmq_4.0.0']) of + true -> + rabbit_ct_helpers:testcase_started(Config, T); + false -> + {skip, "Feature flag rabbitmq_4.0.0 is disabled, but needed for " + "the new #modify{} command being sent to quorum queues."} + end; init_per_testcase(T, Config) when T =:= detach_requeues_one_session_classic_queue orelse T =:= detach_requeues_drop_head_classic_queue orelse