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/BUILD.bazel b/deps/rabbit/BUILD.bazel index c829b5597e3a..a3ebb5349775 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", @@ -722,6 +725,7 @@ rabbitmq_suite( ], deps = [ "//deps/rabbit_common:erlang_app", + "@meck//:erlang_app", "@proper//:erlang_app", "@ra//:erlang_app", ], @@ -735,6 +739,15 @@ rabbitmq_suite( ], ) +rabbitmq_suite( + name = "rabbit_fifo_q_SUITE", + size = "small", + deps = [ + "//deps/rabbit_common:erlang_app", + "@proper//: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 44095b8a7d13..17bfb089dcc4 100644 --- a/deps/rabbit/app.bzl +++ b/deps/rabbit/app.bzl @@ -146,8 +146,10 @@ 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", "src/rabbit_file.erl", "src/rabbit_global_counters.erl", "src/rabbit_guid.erl", @@ -399,8 +401,10 @@ 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", "src/rabbit_file.erl", "src/rabbit_global_counters.erl", "src/rabbit_guid.erl", @@ -541,6 +545,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", ], @@ -672,8 +677,10 @@ 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", "src/rabbit_file.erl", "src/rabbit_global_counters.erl", "src/rabbit_guid.erl", @@ -1288,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"], @@ -2142,3 +2150,13 @@ def test_suite_beam_files(name = "test_suite_beam_files"): erlc_opts = "//:test_erlc_opts", 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"], + hdrs = ["src/rabbit_fifo.hrl"], + app_name = "rabbit", + erlc_opts = "//:test_erlc_opts", + deps = ["@proper//:erlang_app"], + ) 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 3a90e2879842..be63597c3f96 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,22 @@ 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) -> +update_header_from_anns(undefined, Anns) -> + update_header_from_anns(#'v1_0.header'{durable = true}, Anns); +update_header_from_anns(Header, Anns) -> + DeliveryCount = case Anns of + #{delivery_count := C} -> C; + _ -> 0 + end, Redelivered = case Anns of #{redelivered := R} -> R; _ -> false end, - not Redelivered. + FirstAcq = not Redelivered andalso + DeliveryCount =:= 0 andalso + not is_map_key(deaths, Anns), + Header#'v1_0.header'{first_acquirer = FirstAcq, + delivery_count = {uint, DeliveryCount}}. encode_deaths(Deaths) -> lists:map( 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/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 4e0029b02ba1..3b527d3d838c 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, @@ -1861,20 +1861,30 @@ 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. - discard; + +%% 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 +%% 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'{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, @@ -1981,7 +1991,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 @@ -2788,7 +2798,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( @@ -2853,7 +2863,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). @@ -3441,12 +3451,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 @@ -3497,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_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..2da8d55f7a6f 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 -> @@ -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, _Anns}, 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 -> @@ -413,7 +422,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 +577,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 +654,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 67270f4c1c30..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,9 +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 + {'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 bc1a85af08d8..7d357beadc13 100644 --- a/deps/rabbit/src/rabbit_fifo.erl +++ b/deps/rabbit/src/rabbit_fifo.erl @@ -14,7 +14,28 @@ -dialyzer(no_improper_lists). -include("rabbit_fifo.hrl"). --include_lib("rabbit_common/include/rabbit.hrl"). + +-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}}). + +-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 @@ -30,7 +51,7 @@ which_module/1, %% aux init_aux/1, - handle_aux/6, + handle_aux/5, % queries query_messages_ready/1, query_messages_checked_out/1, @@ -47,12 +68,12 @@ query_peek/2, query_notify_decorators_info/1, usage/1, + is_v4/0, %% misc - dehydrate_state/1, - normalize/1, get_msg_header/1, get_header/2, + annotate_msg/2, get_msg/1, %% protocol helpers @@ -61,8 +82,10 @@ make_checkout/3, make_settle/2, make_return/2, + is_return/1, make_discard/2, make_credit/4, + make_modify/5, make_purge/0, make_purge_nodes/1, make_update_config/1, @@ -71,16 +94,23 @@ -ifdef(TEST). -export([update_header/4, - chunk_disk_msgs/3]). + chunk_disk_msgs/3, + smallest_raft_index/1, + make_requeue/4]). -endif. -import(serial_number, [add/2, diff/2]). +-define(ENQ_V2, e). %% command records representing all the protocol actions that are supported -record(enqueue, {pid :: option(pid()), seq :: option(msg_seqno()), msg :: raw_msg()}). --record(requeue, {consumer_id :: consumer_id(), +-record(?ENQ_V2, {seq :: option(msg_seqno()), + msg :: raw_msg(), + size :: {MetadataSize :: non_neg_integer(), + PayloadSize :: non_neg_integer()}}). +-record(requeue, {consumer_key :: consumer_key(), msg_id :: msg_id(), index :: ra:index(), header :: msg_header(), @@ -89,23 +119,30 @@ -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()}). +-record(modify, {consumer_key :: consumer_key(), + msg_ids :: [msg_id()], + delivery_failed :: boolean(), + undeliverable_here :: boolean(), + annotations :: mc:annotations()}). -record(purge, {}). -record(purge_nodes, {nodes :: [node()]}). -record(update_config, {config :: config()}). -record(garbage_collection, {}). +% -record(eval_consumer_timeouts, {consumer_keys :: [consumer_key()]}). -opaque protocol() :: #enqueue{} | + #?ENQ_V2{} | #requeue{} | #register_enqueuer{} | #checkout{} | @@ -113,6 +150,7 @@ #return{} | #discard{} | #credit{} | + #modify{} | #purge{} | #purge_nodes{} | #update_config{} | @@ -126,7 +164,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, @@ -134,6 +172,7 @@ credit_mode/0, consumer_meta/0, consumer_id/0, + consumer_key/0, client_msg/0, msg/0, msg_id/0, @@ -147,8 +186,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 +205,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 @@ -189,16 +228,19 @@ 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, size = Size}, State00) -> + apply_enqueue(Meta, EnqPid, Seq, RawMsg, Size, 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,234 +249,198 @@ 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) -> - 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}, - #?MODULE{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}, - complete_and_checkout(Meta, MsgIds, ConsumerId, Con, Effects, State); +apply(Meta, #discard{consumer_key = ConsumerKey, + msg_ids = MsgIds}, + #?STATE{consumers = Consumers } = State0) -> + case find_consumer(ConsumerKey, Consumers) of + {ConsumerKey, #consumer{} = Con} -> + discard(Meta, MsgIds, ConsumerKey, Con, true, #{}, State0); _ -> {State0, ok} end; -apply(Meta, #return{msg_ids = MsgIds, consumer_id = ConsumerId}, - #?MODULE{consumers = Cons0} = State) -> - case Cons0 of - #{ConsumerId := #consumer{checked_out = Checked0}} -> - Returned = maps:with(MsgIds, Checked0), - return(Meta, ConsumerId, Returned, [], State); +apply(Meta, #return{consumer_key = ConsumerKey, + msg_ids = MsgIds}, + #?STATE{consumers = Cons} = State) -> + case find_consumer(ConsumerKey, Cons) of + {ActualConsumerKey, #consumer{checked_out = Checked}} -> + return(Meta, ActualConsumerKey, MsgIds, false, + #{}, Checked, [], State); + _ -> + {State, ok} + end; +apply(Meta, #modify{consumer_key = ConsumerKey, + delivery_failed = DelFailed, + undeliverable_here = Undel, + annotations = Anns, + msg_ids = MsgIds}, + #?STATE{consumers = Cons} = State) -> + case find_consumer(ConsumerKey, Cons) of + {ConsumerKey, #consumer{checked_out = Checked}} + when Undel == false -> + return(Meta, ConsumerKey, MsgIds, DelFailed, + Anns, Checked, [], State); + {ConsumerKey, #consumer{} = Con} + when Undel == true -> + discard(Meta, MsgIds, ConsumerKey, Con, DelFailed, Anns, State); _ -> {State, ok} end; apply(#{index := Idx} = Meta, - #requeue{consumer_id = ConsumerId, + #requeue{consumer_key = ConsumerKey, msg_id = MsgId, index = OldIdx, - header = Header0, - msg = _Msg}, - #?MODULE{consumers = Cons0, - messages = Messages, - ra_indexes = Indexes0, - enqueue_count = EnqCount} = State00) -> - case Cons0 of - #{ConsumerId := #consumer{checked_out = Checked0} = Con0} + header = Header0}, + #?STATE{consumers = Cons, + messages = Messages, + ra_indexes = Indexes0, + enqueue_count = EnqCount} = State00) -> + %% the actual consumer key was looked up in the aux handler so we + %% dont need to use find_consumer/2 here + case Cons of + #{ConsumerKey := #consumer{checked_out = Checked0} = Con0} when is_map_key(MsgId, Checked0) -> %% construct a message with the current raft index - %% and update delivery count before adding it to the message queue - Header = update_header(delivery_count, fun incr/1, 1, Header0), + %% and update acquired count before adding it to the message queue + Header = update_header(acquired_count, fun incr/1, 1, Header0), State0 = add_bytes_return(Header, State00), Con = Con0#consumer{checked_out = maps:remove(MsgId, Checked0), - credit = increase_credit(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}, - 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); + credit = increase_credit(Con0, 1)}, + State1 = State0#?STATE{ra_indexes = rabbit_fifo_index:delete(OldIdx, + Indexes0), + messages = rabbit_fifo_q:in(lo, + ?MSG(Idx, Header), + Messages), + enqueue_count = EnqCount + 1}, + State2 = update_or_remove_con(Meta, ConsumerKey, Con, State1), + checkout(Meta, State0, State2, []); _ -> {State00, ok, []} 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) -> - case Cons0 of - #{ConsumerId := #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), - State1 = State0#?MODULE{service_queue = ServiceQueue, - consumers = maps:update(ConsumerId, Con1, Cons0)}, - {State2, ok, Effects} = checkout(Meta, State0, State1, []), - - #?MODULE{consumers = Cons1 = #{ConsumerId := Con2}} = State2, - #consumer{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#?MODULE{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. - {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(ConsumerId, Con, Cons1), - State = State2#?MODULE{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 - %% 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. - case lists:keytake(ConsumerId, 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#?MODULE{waiting_consumers = - [{ConsumerId, 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, Available, false}, - ?DELIVERY_SEND_MSG_OPTS}}; - false -> - {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, _}}, - #?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), + 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}}; _ -> - {_, 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, ?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; + {success, _, MsgId, + ?MSG(RaftIdx, Header), _ExpiredMsg, State2, Effects0} -> + {State4, Effects1} = + case Settlement of + unsettled -> + {_, Pid} = ConsumerId, + {State2, [{monitor, process, Pid} | Effects0]}; + settled -> + %% immediately settle the checkout + {State3, _, SettleEffects} = + apply(Meta, make_settle(ConsumerId, [MsgId]), + State2), + {State3, SettleEffects ++ Effects0} + end, + Effects2 = [reply_log_effect(RaftIdx, MsgId, Header, + messages_ready(State4), From) + | Effects1], + {State, _DroppedMsg, Effects} = + evaluate_limit(Index, false, State0, State4, Effects2), + {State, '$ra_no_reply', Effects}; {nochange, _ExpiredMsg = true, State2, Effects0} -> %% All ready messages expired. - State3 = State2#?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), + {State, {dequeue, empty}, Effects} end end; +apply(#{index := _Idx} = Meta, + #checkout{spec = Spec, + consumer_id = ConsumerId}, State0) + when Spec == cancel orelse + Spec == remove -> + case consumer_key_from_id(ConsumerId, State0) of + {ok, ConsumerKey} -> + {State1, Effects1} = activate_next_consumer( + cancel_consumer(Meta, ConsumerKey, State0, [], + Spec)), + Reply = {ok, consumer_cancel_info(ConsumerKey, State1)}, + {State, _, Effects} = checkout(Meta, State0, State1, Effects1), + {State, Reply, Effects}; + error -> + {State0, {error, consumer_not_found}, []} + end; apply(#{index := Idx} = Meta, - #checkout{spec = 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), + #checkout{spec = Spec0, + meta = ConsumerMeta, + consumer_id = {_, Pid} = ConsumerId}, State0) -> + %% might be better to check machine_version + IsV4 = tuple_size(Spec0) == 2, + %% normalise spec format + Spec = case Spec0 of + {_, _} -> + Spec0; + {Life, Prefetch, simple_prefetch} -> + {Life, {simple_prefetch, Prefetch}}; + {Life, _Credit, credited} -> + {Life, credited} + end, + Priority = get_priority(ConsumerMeta), + ConsumerKey = case consumer_key_from_id(ConsumerId, State0) of + {ok, K} -> + K; + error when IsV4 -> + %% if the consumer does not already exist use the + %% raft index as it's unique identifier in future + %% settle, credit, return and discard operations + Idx; + error -> + ConsumerId + end, + {Consumer, State1} = update_consumer(Meta, ConsumerKey, ConsumerId, + ConsumerMeta, Spec, Priority, State0), {State2, Effs} = activate_next_consumer(State1, []), #consumer{checked_out = Checked, credit = Credit, @@ -444,90 +450,88 @@ 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{}, - #?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 = rabbit_fifo_q: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}]); + {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, 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(CKey, ?CONSUMER_PID(P) = C0, {S0, E0}) + when node(P) =:= Node -> + %% the consumer should be returned to waiting + %% and checked out messages should be returned + Effs = consumer_update_active_effects( + S0, C0, false, suspected_down, E0), + {St, Effs1} = return_all(Meta, S0, Effs, CKey, C0, true), + %% if the consumer was cancelled there is a chance it got + %% removed when returning hence we need to be defensive here + Waiting = case St#?STATE.consumers of + #{CKey := C} -> + Waiting0 ++ [{CKey, C}]; + _ -> + Waiting0 + end, + {St#?STATE{consumers = maps:remove(CKey, St#?STATE.consumers), + waiting_consumers = Waiting, + last_active = Ts}, + Effs1}; + (_, _, S) -> + S + end, {State0, []}, Cons0), 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 +540,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,19 +557,12 @@ apply(#{system_time := Ts, machine_version := MachineVersion} = Meta, {State, Effects1} = maps:fold( - fun({_, P} = Cid, #consumer{checked_out = Checked0, - status = up} = C0, + fun(CKey, #consumer{cfg = #consumer_cfg{pid = P}, + 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, + C = C0#consumer{status = suspected_down}, + {St, Eff0} = return_all(Meta, St0, Eff, CKey, C, true), + Eff1 = consumer_update_active_effects(St, C, false, suspected_down, Eff0), {St, Eff1}; (_, _, {St, Eff}) -> @@ -581,15 +578,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); -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) -> + checkout(Meta, State0, State#?STATE{enqueuers = Enqs, + last_active = Ts}, Effects); +apply(#{index := _Idx} = Meta, {down, Pid, _Info}, State0) -> + {State1, Effects1} = activate_next_consumer( + handle_down(Meta, Pid, State0)), + checkout(Meta, State0, State1, Effects1); +apply(Meta, {nodeup, Node}, #?STATE{consumers = Cons0, + enqueuers = Enqs0, + 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 @@ -604,293 +601,187 @@ apply(Meta, {nodeup, Node}, #?MODULE{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_con(Meta, ConsumerKey, C#consumer{status = up}, SAcc), EAcc1}; (_, _, Acc) -> 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) -> {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}, - #?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, 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{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}), + 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, - #?MODULE{cfg = #cfg{dead_letter_handler = DLH}, +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#?MODULE{dlx = DlxState}, - {State, ok, Effects} = checkout(Meta, State0, State1, Effects0), - update_smallest_raft_index(IncomingRaftIdx, State, Effects); + State1 = State0#?STATE{dlx = DlxState}, + 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_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(#{} = _Meta, StateV3) -> + %% TODO: consider emitting release cursors as checkpoints + Messages0 = rabbit_fifo_v3:get_field(messages, StateV3), + Returns0 = lqueue:to_list(rabbit_fifo_v3:get_field(returns, StateV3)), + Consumers0 = rabbit_fifo_v3:get_field(consumers, StateV3), + Consumers = maps:map( + fun (_, #consumer{checked_out = Ch0} = C) -> + Ch = maps:map( + fun (_, ?MSG(I, #{delivery_count := DC} = H)) -> + ?MSG(I, H#{acquired_count => DC}); + (_, Msg) -> + Msg + end, Ch0), + C#consumer{checked_out = Ch} + end, Consumers0), + Returns = lqueue:from_list( + lists:map(fun (?MSG(I, #{delivery_count := DC} = H)) -> + ?MSG(I, H#{acquired_count => DC}); + (Msg) -> + Msg + end, Returns0)), + + Messages = rabbit_fifo_q:from_lqueue(Messages0), + #?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}) -> {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) -> +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 - 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( - #?MODULE{cfg = #cfg{consumer_strategy = competing}}) -> - fun(State, ConsumerId, Consumer, Active, ActivityStatus, Effects) -> - consumer_update_active_effects(State, ConsumerId, Consumer, Active, + #?STATE{cfg = #cfg{consumer_strategy = competing}}) -> + fun(State, _ConsumerKey, Consumer, Active, ActivityStatus, Effects) -> + consumer_update_active_effects(State, Consumer, Active, ActivityStatus, Effects) end; consumer_active_flag_update_function( - #?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, + Down = lists:filter(fun({_, ?CONSUMER_PID(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#?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 - Node -> - {ConsumerId, Consumer#consumer{status = Status}}; - _ -> - {ConsumerId, Consumer} - end - end || {{_, Pid} = ConsumerId, Consumer} <- WaitingConsumers, - Consumer#consumer.status =/= cancelled]. + sort_waiting( + [case node(Pid) of + Node -> + {ConsumerKey, Consumer#consumer{status = Status}}; + _ -> + {ConsumerKey, Consumer} + end || {ConsumerKey, ?CONSUMER_PID(Pid) = Consumer} + <- WaitingConsumers, Consumer#consumer.status =/= cancelled]). -spec state_enter(ra_server:ra_state() | eol, state()) -> ra_machine:effects(). -state_enter(RaState, #?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 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]), @@ -902,24 +793,28 @@ 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 = 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))] ++ - [{aux, eol} | Effects]; + [{aux, eol} + | Effects]; state_enter0(_, _, Effects) -> %% catch all as not handling all states Effects. -spec tick(non_neg_integer(), state()) -> ra_machine:effects(). -tick(Ts, #?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 +823,16 @@ 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, + enqueue_count = EnqCount, + msg_bytes_enqueue = EnqueueBytes, + msg_bytes_checkout = CheckoutBytes, + cfg = Cfg, + dlx = DlxState, + messages = Messages, + returns = Returns, + waiting_consumers = WaitingConsumers} = State) -> Conf = #{name => Cfg#cfg.name, resource => Cfg#cfg.resource, release_cursor_interval => Cfg#cfg.release_cursor_interval, @@ -949,38 +845,48 @@ overview(#?MODULE{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}; _ -> #{} end, - Overview = #{type => ?MODULE, + 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), 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_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, + num_release_cursors => 0, %% backwards compat enqueue_message_bytes => EnqueueBytes, checkout_message_bytes => CheckoutBytes, + release_cursors => [], %% backwards compat in_memory_message_bytes => 0, %% backwards compat + num_in_memory_ready_messages => 0, %% backwards compat + release_cursor_enqueue_counter => EnqCount, smallest_raft_index => smallest_raft_index(State) }, 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, #?MODULE{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}} @@ -990,15 +896,21 @@ 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) -> ?MODULE. --define(AUX, aux_v2). +-define(AUX, aux_v3). +-record(checkpoint, {index :: ra:index(), + timestamp :: milliseconds(), + enqueue_count :: non_neg_integer(), + smallest_index :: undefined | ra:index(), + messages_total :: non_neg_integer()}). -record(aux_gc, {last_raft_idx = 0 :: ra:index()}). -record(aux, {name :: atom(), capacity :: term(), @@ -1007,58 +919,72 @@ which_module(3) -> ?MODULE. last_decorators_state :: term(), capacity :: term(), gc = #aux_gc{} :: #aux_gc{}, - tick_pid, - cache = #{} :: map()}). + tick_pid :: undefined | pid(), + 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 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}}. + capacity = {inactive, Now, 1, 1.0}, + last_checkpoint = #checkpoint{index = 0, + timestamp = erlang:system_time(millisecond), + enqueue_count = 0, + messages_total = 0}}. 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}; + AuxV2 = init_aux(Name), + Aux = AuxV2#?AUX{capacity = Cap, + gc = Gc}, + handle_aux(RaftState, Tag, Cmd, Aux, RaAux); +handle_aux(RaftState, Tag, Cmd, AuxV2, RaAux) + when element(1, AuxV2) == aux_v2 -> + Name = element(2, AuxV2), + AuxV3 = init_aux(Name), + handle_aux(RaftState, Tag, Cmd, AuxV3, RaAux); handle_aux(_RaftState, cast, {#return{msg_ids = MsgIds, - consumer_id = ConsumerId}, Corr, Pid}, - Aux0, Log0, #?MODULE{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}; + consumer_key = Key} = Ret, Corr, Pid}, + Aux0, RaAux0) -> + case ra_aux:machine_state(RaAux0) of + #?STATE{cfg = #cfg{delivery_limit = undefined}, + consumers = Consumers} -> + case find_consumer(Key, Consumers) of + {ConsumerKey, #consumer{checked_out = Checked}} -> + {RaAux, ToReturn} = + maps:fold( + fun (MsgId, ?MSG(Idx, Header), {RA0, Acc}) -> + %% it is possible this is not found if the consumer + %% crashed and the message got removed + case ra_aux:log_fetch(Idx, RA0) of + {{_Term, _Meta, Cmd}, RA} -> + Msg = get_msg(Cmd), + {RA, [{MsgId, Idx, Header, Msg} | Acc]}; + {undefined, RA} -> + {RA, Acc} + end + end, {RaAux0, []}, maps:with(MsgIds, Checked)), + + Appends = make_requeue(ConsumerKey, {notify, Corr, Pid}, + lists:sort(ToReturn), []), + {no_reply, Aux0, RaAux, Appends}; + _ -> + {no_reply, Aux0, RaAux0} + end; _ -> - {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 -> @@ -1069,110 +995,129 @@ 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, ConsumerId, MsgIds}, - Aux0, Log0, #?MODULE{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 - #{ConsumerId := #consumer{checked_out = Checked}} -> - {Log, IdMsgs} = + #{ConsumerKey := #consumer{checked_out = Checked}} -> + {RaState, IdMsgs} = maps:fold( - fun (MsgId, ?MSG(Idx, Header), {L0, 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_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, #?MODULE{}) -> - %% 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) -> +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, Log, 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}, Log, Effects} + | Effects1], + {no_reply, Aux0#?AUX{last_checkpoint = Check, + 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, + #?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)}, 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), + 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, Log}; + {no_reply, Aux, RaAux}; handle_aux(_RaState, {call, _From}, oldest_entry_timestamp, - #?AUX{cache = Cache} = Aux0, - Log0, #?MODULE{} = 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, - #?MODULE{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, #?MODULE{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 @@ -1187,9 +1132,10 @@ eval_gc(Log, #?MODULE{cfg = #cfg{resource = QR}} = MacState, AuxState end. -force_eval_gc(Log, #?MODULE{cfg = #cfg{resource = QR}}, +force_eval_gc(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 -> @@ -1212,7 +1158,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 +1166,34 @@ query_messages_checked_out(#?MODULE{consumers = Consumers}) -> query_messages_total(State) -> messages_total(State). -query_processes(#?MODULE{enqueuers = Enqs, consumers = Cons0}) -> - Cons = maps:fold(fun({_, P}, V, S) -> S#{P => V} end, #{}, Cons0), +query_processes(#?STATE{enqueuers = Enqs, consumers = Cons0}) -> + Cons = maps:fold(fun(_, ?CONSUMER_PID(P) = V, S) -> + S#{P => V} + end, #{}, Cons0), maps:keys(maps:merge(Enqs, Cons)). -query_ra_indexes(#?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}) -> - Up = maps:filter(fun(_ConsumerId, #consumer{status = Status}) -> +query_consumer_count(#?STATE{consumers = Consumers, + waiting_consumers = WaitingConsumers}) -> + Up = maps:filter(fun(_ConsumerKey, #consumer{status = Status}) -> Status =/= suspected_down end, Consumers), maps:size(Up) + length(WaitingConsumers). -query_consumers(#?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 + case ConsumerStrategy of competing -> - fun(_ConsumerId, - #consumer{status = Status}) -> + fun(_ConsumerKey, #consumer{status = Status}) -> case Status of suspected_down -> {false, Status}; @@ -1255,7 +1203,7 @@ query_consumers(#?MODULE{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}; @@ -1267,11 +1215,13 @@ query_consumers(#?MODULE{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), @@ -1282,46 +1232,49 @@ query_consumers(#?MODULE{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( - #?MODULE{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. -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 +1288,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, @@ -1359,14 +1312,19 @@ usage(Name) when is_atom(Name) -> [{_, Use}] -> Use end. +-spec is_v4() -> boolean(). +is_v4() -> + %% Quorum queue v4 is introduced in RabbitMQ 4.0.0 + rabbit_feature_flags:is_enabled('rabbitmq_4.0.0'). + %%% Internal -messages_ready(#?MODULE{messages = M, - returns = R}) -> - lqueue:len(M) + lqueue:len(R). +messages_ready(#?STATE{messages = M, + returns = R}) -> + rabbit_fifo_q: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. @@ -1375,18 +1333,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; @@ -1400,119 +1358,161 @@ 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, +cancel_consumer(Meta, ConsumerKey, + #?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, + 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, - #?MODULE{consumers = Cons0, - cfg = #cfg{consumer_strategy = single_active}, - waiting_consumers = Waiting0} = State0, + 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, - 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 - 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#?MODULE{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(#?MODULE{cfg = #cfg{resource = QName}}, - ConsumerId, - #consumer{cfg = #consumer_cfg{meta = Meta}}, +consumer_update_active_effects(#?STATE{cfg = #cfg{resource = QName}}, + #consumer{cfg = #consumer_cfg{pid = CPid, + tag = CTag, + meta = Meta}}, Active, ActivityStatus, Effects) -> Ack = maps:get(ack, Meta, undefined), Prefetch = maps:get(prefetch, Meta, undefined), Args = maps:get(args, Meta, []), [{mod_call, rabbit_quorum_queue, update_consumer_handler, - [QName, ConsumerId, false, Ack, Prefetch, Active, ActivityStatus, Args]} + [QName, {CTag, CPid}, false, Ack, Prefetch, Active, ActivityStatus, Args]} | Effects]. -cancel_consumer0(Meta, ConsumerId, - #?MODULE{consumers = C0} = S0, Effects0, Reason) -> +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(consumer_id(Consumer), S, Effects2), {S, Effects}; _ -> %% already removed: do nothing {S0, Effects0} end. -activate_next_consumer(#?MODULE{cfg = #cfg{consumer_strategy = competing}} = State0, - Effects0) -> - {State0, Effects0}; -activate_next_consumer(#?MODULE{consumers = Cons, - waiting_consumers = Waiting0} = State0, +activate_next_consumer({State, Effects}) -> + activate_next_consumer(State, Effects). + +activate_next_consumer(#?STATE{cfg = #cfg{consumer_strategy = competing}} = State, + Effects) -> + {State, Effects}; +activate_next_consumer(#?STATE{consumers = Cons0, + waiting_consumers = Waiting0} = State0, Effects0) -> - 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, - #?MODULE{service_queue = ServiceQueue} = State0, - ServiceQueue1 = maybe_queue_consumer(NextConsumerId, - Consumer, - ServiceQueue), - State = State0#?MODULE{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 -> + %% invariant, the waiting list always need to be sorted by consumers that are + %% up - then by priority + NextConsumer = + case Waiting0 of + [{_, #consumer{status = up}} = Next | _] -> + Next; + _ -> + undefined + end, + + case {active_consumer(Cons0), NextConsumer} of + {undefined, {NextCKey, #consumer{cfg = NextCCfg} = NextC}} -> + Remaining = tl(Waiting0), + %% TODO: can this happen? + Consumer = case maps:get(NextCKey, Cons0, undefined) of + undefined -> + NextC; + Existing -> + %% there was an exisiting non-active consumer + %% just update the existing cancelled consumer + %% with the new config + Existing#consumer{cfg = NextCCfg} + end, + #?STATE{service_queue = ServiceQueue} = State0, + ServiceQueue1 = maybe_queue_consumer(NextCKey, + Consumer, + ServiceQueue), + State = State0#?STATE{consumers = Cons0#{NextCKey => Consumer}, + service_queue = ServiceQueue1, + waiting_consumers = Remaining}, + Effects = consumer_update_active_effects(State, Consumer, + true, single_active, + Effects0), + {State, Effects}; + {{ActiveCKey, ?CONSUMER_PRIORITY(ActivePriority) = + #consumer{checked_out = ActiveChecked} = Active}, + {NextCKey, ?CONSUMER_PRIORITY(WaitingPriority) = Consumer}} + when WaitingPriority > ActivePriority andalso + map_size(ActiveChecked) == 0 -> + Remaining = tl(Waiting0), + %% the next consumer is a higher priority and should take over + %% and this consumer does not have any pending messages + #?STATE{service_queue = ServiceQueue} = State0, + ServiceQueue1 = maybe_queue_consumer(NextCKey, + Consumer, + ServiceQueue), + Cons1 = Cons0#{NextCKey => Consumer}, + Cons = maps:remove(ActiveCKey, Cons1), + Waiting = add_waiting({ActiveCKey, Active}, Remaining), + State = State0#?STATE{consumers = Cons, + service_queue = ServiceQueue1, + waiting_consumers = Waiting}, + 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 + Cons = maps:update(ActiveCKey, + Active#consumer{status = quiescing}, + Cons0), + {State0#?STATE{consumers = Cons}, Effects0}; + _ -> + %% no activation {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({CKey, #consumer{status = Status} = Consumer, _I}) + when Status == up orelse Status == quiescing -> + {CKey, Consumer}; +active_consumer({_CKey, #consumer{status = _}, I}) -> active_consumer(maps:next(I)); active_consumer(none) -> undefined; @@ -1520,68 +1520,63 @@ 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, + 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, ConsumerId, Consumer), - {S1#?MODULE{consumers = maps:remove(ConsumerId, S1#?MODULE.consumers), - last_active = Ts}, - Effects1} + _ -> + {S1, Effects} = return_all(Meta, S0, Effects0, ConsumerKey, + Consumer, Reason == down), + {S1#?STATE{consumers = maps:remove(ConsumerKey, S1#?STATE.consumers), + last_active = Ts}, + Effects} end. apply_enqueue(#{index := RaftIdx, - system_time := Ts} = Meta, From, Seq, RawMsg, 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}; + checkout(Meta, State0, State1, Effects1); {out_of_sequence, State, Effects} -> {State, not_enqueued, Effects}; {duplicate, State, Effects} -> {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 -> {State0, Effects} end. -maybe_set_msg_ttl(#basic_message{content = #content{properties = none}}, - RaCmdTs, Header, - #?MODULE{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}}) -> - %% 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, - #?MODULE{cfg = #cfg{msg_ttl = MsgTTL}}) -> + #?STATE{cfg = #cfg{msg_ttl = MsgTTL}}) -> case mc:is(Msg) of true -> TTL = min(MsgTTL, mc:ttl(Msg)), @@ -1590,6 +1585,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) -> @@ -1605,64 +1614,43 @@ 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, - #?MODULE{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}; - 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 = - {Base, Interval}}}, - Dehydrated = dehydrate_state(State), - Cursor = {release_cursor, RaftIdx, Dehydrated}, - Cursors = lqueue:in(Cursor, Cursors0), - State#?MODULE{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) -> +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), - Header = maybe_set_msg_ttl(RawMsg, Ts, Size, State0), + Size = BodySize, + Header0 = maybe_set_msg_ttl(RawMsg, Ts, BodySize, State0), + Header = maybe_set_msg_delivery_count(RawMsg, Header0), 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) - }, + PTag = priority_tag(RawMsg), + State = State0#?STATE{msg_bytes_enqueue = Enqueue + Size, + enqueue_count = EnqCount + 1, + messages_total = Total + 1, + messages = rabbit_fifo_q:in(PTag, Msg, Messages) + }, {ok, State, Effects}; -maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, Effects0, - #?MODULE{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#?MODULE{enqueuers = Enqueuers0#{From => #enqueuer{}}}, + 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), + 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 @@ -1671,13 +1659,14 @@ 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 - }, + PTag = priority_tag(RawMsg), + State = State0#?STATE{msg_bytes_enqueue = Enqueue + BodySize, + enqueue_count = EnqCount + 1, + messages_total = Total + 1, + messages = rabbit_fifo_q:in(PTag, Msg, Messages), + enqueuers = Enqueuers0#{From => Enq}, + msg_cache = MsgCache + }, {ok, State, Effects0}; #enqueuer{next_seqno = Next} when MsgSeqNo > Next -> @@ -1688,52 +1677,53 @@ maybe_enqueue(RaftIdx, Ts, From, MsgSeqNo, RawMsg, Effects0, {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#?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, - {State, ok, Effects} = checkout(Meta, State0, State2, Effects1), - update_smallest_raft_index(IncomingRaftIdx, State, Effects). +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 := Msg} -> + return_one(Meta, MsgId, Msg, IncrDelCount, Anns, + S0, E0, ConsumerKey); + #{} -> + Acc + end + end, {State0, Effects0}, MsgIds), + State2 = case State1#?STATE.consumers of + #{ConsumerKey := Con} -> + update_or_remove_con(Meta, ConsumerKey, Con, State1); + _ -> + State1 + end, + checkout(Meta, State0, State2, Effects1). % used to process messages that are finished -complete(Meta, ConsumerId, [DiscardedMsgId], +complete(Meta, ConsumerKey, [MsgId], #consumer{checked_out = Checked0} = Con0, - #?MODULE{ra_indexes = Indexes0, - msg_bytes_checkout = BytesCheckout, - messages_total = Tot} = State0) -> - case maps:take(DiscardedMsgId, Checked0) of + #?STATE{ra_indexes = Indexes0, + msg_bytes_checkout = BytesCheckout, + messages_total = Tot} = State0) -> + case maps:take(MsgId, Checked0) of {?MSG(Idx, Hdr), Checked} -> SettledSize = get_header(size, Hdr), Indexes = rabbit_fifo_index:delete(Idx, Indexes0), Con = Con0#consumer{checked_out = Checked, - credit = increase_credit(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}; + credit = increase_credit(Con0, 1)}, + State1 = update_or_remove_con(Meta, ConsumerKey, Con, State0), + State1#?STATE{ra_indexes = Indexes, + msg_bytes_checkout = BytesCheckout - SettledSize, + messages_total = Tot - 1}; error -> State0 end; -complete(Meta, ConsumerId, DiscardedMsgIds, +complete(Meta, ConsumerKey, MsgIds, #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}) -> @@ -1744,94 +1734,51 @@ complete(Meta, ConsumerId, 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(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}. - -increase_credit(_Meta, #consumer{cfg = #consumer_cfg{lifetime = once}, - credit = Credit}, _) -> + credit = increase_credit(Con0, Len)}, + State1 = update_or_remove_con(Meta, ConsumerKey, Con, State0), + State1#?STATE{ra_indexes = Indexes, + msg_bytes_checkout = BytesCheckout - SettledSize, + messages_total = Tot - Len}. + +increase_credit(#consumer{cfg = #consumer_cfg{lifetime = once}, + credit = Credit}, _) -> %% once consumers cannot increment credit Credit; -increase_credit(_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(#{machine_version := MachineVersion}, - #consumer{cfg = #consumer_cfg{credit_mode = {simple_prefetch, MaxCredit}}, +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 MachineVersion >= 3 andalso MaxCredit > 0 -> + 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(#{} = Meta, MsgIds, ConsumerKey, #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). + State1 = complete(Meta, ConsumerKey, MsgIds, Con0, State0), + %% a completion could have removed the active/quiescing consumer + {State2, Effects1} = activate_next_consumer(State1, Effects0), + checkout(Meta, State0, State2, Effects1). cancel_consumer_effects(ConsumerId, - #?MODULE{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]. -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, - 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#?MODULE{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#?MODULE{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)). @@ -1842,11 +1789,12 @@ 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_key(size, Header) -> maps:update_with(Key, UpdateFun, Default, Header). get_msg_header(?MSG(_Idx, Header)) -> @@ -1871,76 +1819,87 @@ 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, - Effects0, ConsumerId) -> - #consumer{checked_out = Checked0} = Con0 = maps:get(ConsumerId, Consumers), - Msg = update_msg_header(delivery_count, fun incr/1, 1, 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 Header of + #{delivery_count := DelCount} -> + mc:set_annotation(delivery_count, DelCount, Msg); + _ -> + Msg + end; + _ -> + Msg0 + end. + +return_one(Meta, MsgId, ?MSG(_, _) = Msg0, DelivFailed, Anns, + #?STATE{returns = Returns, + consumers = Consumers, + dlx = DlxState0, + cfg = #cfg{delivery_limit = DeliveryLimit, + dead_letter_handler = DLH}} = State0, + Effects0, ConsumerKey) -> + #consumer{checked_out = Checked0} = Con0 = maps:get(ConsumerKey, Consumers), + Msg = incr_msg(Msg0, DelivFailed, Anns), Header = get_msg_header(Msg), - case get_header(delivery_count, Header) of - DeliveryCount when DeliveryCount > DeliveryLimit -> - {DlxState, DlxEffects} = rabbit_fifo_dlx:discard([Msg], delivery_limit, DLH, DlxState0), - State1 = State0#?MODULE{dlx = DlxState}, - State = complete(Meta, ConsumerId, [MsgId], Con0, State1), + case get_header(acquired_count, Header) of + AcquiredCount when AcquiredCount > DeliveryLimit -> + {DlxState, DlxEffects} = + rabbit_fifo_dlx:discard([Msg], delivery_limit, DLH, DlxState0), + State1 = State0#?STATE{dlx = DlxState}, + State = complete(Meta, ConsumerKey, [MsgId], Con0, State1), {State, DlxEffects ++ Effects0}; _ -> Checked = maps:remove(MsgId, Checked0), - Con = 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(Con0, 1)}, {add_bytes_return( Header, - State0#?MODULE{consumers = Consumers#{ConsumerId => Con}, - returns = lqueue:in(Msg, Returns)}), + State0#?STATE{consumers = Consumers#{ConsumerKey => Con}, + returns = lqueue:in(Msg, Returns)}), Effects0} end. -return_all(Meta, #?MODULE{consumers = Cons} = State0, Effects0, ConsumerId, - #consumer{checked_out = Checked} = Con) -> - State = State0#?MODULE{consumers = Cons#{ConsumerId => Con}}, +return_all(Meta, #?STATE{consumers = Cons} = State0, Effects0, ConsumerKey, + #consumer{checked_out = Checked} = Con, DelivFailed) -> + State = State0#?STATE{consumers = Cons#{ConsumerKey => Con}}, lists:foldl(fun ({MsgId, Msg}, {S, E}) -> - return_one(Meta, MsgId, Msg, S, E, ConsumerId) + return_one(Meta, MsgId, Msg, DelivFailed, #{}, + S, E, ConsumerKey) end, {State, Effects0}, lists:sort(maps:to_list(Checked))). checkout(Meta, OldState, State0, Effects0) -> checkout(Meta, OldState, State0, Effects0, ok). checkout(#{index := Index} = Meta, - #?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 -> - {State, Reply, Effects}; {State, _, Effects} -> - update_smallest_raft_index(Index, Reply, State, Effects) + {State, Reply, Effects} end. -checkout0(Meta, {success, ConsumerId, MsgId, - ?MSG(_RaftIdx, _Header) = Msg, ExpiredMsg, State, Effects}, +checkout0(Meta, {success, ConsumerKey, MsgId, + ?MSG(_, _) = 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) -> @@ -1948,13 +1907,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 +1924,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 +1932,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 +1950,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,39 +1987,41 @@ 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, _} -> + case rabbit_fifo_q:out(Messages0) of + empty -> empty; - {{value, ?MSG(RaftIdx, _) = Msg}, Messages} -> + {?MSG(RaftIdx, _) = Msg, Messages} -> %% add index here Indexes = rabbit_fifo_index:append(RaftIdx, Indexes0), - {Msg, State#?MODULE{messages = Messages, - ra_indexes = Indexes}} + {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); + rabbit_fifo_q:get(Messages0); Msg -> Msg end. -delivery_effect({CTag, CPid}, [{MsgId, ?MSG(Idx, Header)}], - #?MODULE{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), @@ -2070,7 +2031,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,38 +2046,39 @@ 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 - {{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} -> + {Msg, 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 + case maps:get(ConsumerKey, Cons0) of + #consumer{credit = Credit, + status = Status} + when Credit =:= 0 orelse + Status =/= up -> + %% not an active consumer but still in the consumers + %% map - this can happen when draining + %% or when higher priority single active consumers + %% take over, recurse without consumer in service + %% queue checkout_one(Meta, ExpiredMsg, - InitState#?MODULE{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, delivery_count = DelCnt0, cfg = Cfg} = Con0 -> - Checked = maps:put(Next, ConsumerMsg, Checked0), + Checked = maps:put(Next, Msg, Checked0), DelCnt = case credit_api_v2(Cfg) of true -> add(DelCnt0, 1); false -> DelCnt0 + 1 @@ -2124,24 +2087,25 @@ 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)), - State = update_or_remove_sub( - Meta, ConsumerId, Con, - State0#?MODULE{service_queue = SQ1, - msg_bytes_checkout = BytesCheckout + Size, - msg_bytes_enqueue = BytesEnqueue - Size}), - {success, ConsumerId, Next, ConsumerMsg, ExpiredMsg, + Size = get_header(size, get_msg_header(Msg)), + State1 = + State0#?STATE{service_queue = SQ1, + msg_bytes_checkout = BytesCheckout + Size, + msg_bytes_enqueue = BytesEnqueue - Size}, + State = update_or_remove_con( + Meta, ConsumerKey, Con, State1), + {success, ConsumerKey, Next, Msg, ExpiredMsg, State, Effects1} end; empty -> {nochange, ExpiredMsg, InitState, Effects1} end; {{value, _ConsumerId}, SQ1} -> - %% consumer did not exist but was queued, recurse + %% consumer was not active 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 + case rabbit_fifo_q:len(Messages0) of 0 -> {nochange, ExpiredMsg, InitState, Effects1}; _ -> @@ -2167,25 +2131,30 @@ 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) -> 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 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) -> @@ -2197,31 +2166,42 @@ timer_effect(RaCmdTs, State, Effects) -> end, [{timer, expire_msgs, T} | Effects]. -update_or_remove_sub(Meta, ConsumerId, +update_or_remove_con(Meta, ConsumerKey, #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(ConsumerKey, 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(ConsumerKey, 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)}. +update_or_remove_con(_Meta, ConsumerKey, + #consumer{status = quiescing, + checked_out = Checked} = Con0, + #?STATE{consumers = Cons, + waiting_consumers = Waiting} = State) + when map_size(Checked) == 0 -> + Con = Con0#consumer{status = up}, + State#?STATE{consumers = maps:remove(ConsumerKey, Cons), + waiting_consumers = add_waiting({ConsumerKey, Con}, Waiting)}; +update_or_remove_con(_Meta, ConsumerKey, + #consumer{} = Con, + #?STATE{consumers = Cons, + service_queue = ServiceQueue} = State) -> + State#?STATE{consumers = maps:put(ConsumerKey, Con, Cons), + service_queue = maybe_queue_consumer(ConsumerKey, Con, + ServiceQueue)}. maybe_queue_consumer(Key, #consumer{credit = Credit, status = up, - cfg = #consumer_cfg{priority = P}}, ServiceQueue) + 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 @@ -2234,15 +2214,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, - #?MODULE{cfg = #cfg{consumer_strategy = competing}, - consumers = Cons0} = State0) -> +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, @@ -2250,34 +2232,35 @@ 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, - #?MODULE{cfg = #cfg{consumer_strategy = single_active}, - consumers = Cons0, - waiting_consumers = Waiting, - service_queue = _ServiceQueue0} = 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}, + consumers = Cons0, + waiting_consumers = Waiting0, + service_queue = _ServiceQueue0} = State) -> %% if it is the current active consumer, just update %% if it is a cancelled active consumer, add to waiting unless it is the only %% one, then merge case active_consumer(Cons0) of - {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_con(Meta, ConsumerKey, Consumer, State)}; + undefined when is_map_key(ConsumerKey, Cons0) -> %% there is no active consumer and the current consumer is in the %% consumers map and thus must be cancelled, in this case we can just %% merge and effectively make this the current active one - Consumer0 = maps:get(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_con(Meta, ConsumerKey, Consumer, State)}; _ -> %% 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, @@ -2285,17 +2268,43 @@ update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta, priority = Priority, credit_mode = Mode}, credit = Credit, - delivery_count = initial_delivery_count(ConsumerMeta)}, - {Consumer, - State0#?MODULE{waiting_consumers = - Waiting ++ [{ConsumerId, Consumer}]}} + delivery_count = DeliveryCount}, + Waiting = add_waiting({ConsumerKey, Consumer}, Waiting0), + {Consumer, State#?STATE{waiting_consumers = Waiting}} end. -merge_consumer(Meta, #consumer{cfg = CCfg, checked_out = Checked} = Consumer, - ConsumerMeta, {Life, Credit, Mode0}, Priority) -> +add_waiting({Key, _} = New, Waiting) -> + sort_waiting(lists:keystore(Key, 1, Waiting, New)). + +sort_waiting(Waiting) -> + lists:sort(fun + ({_, ?CONSUMER_PRIORITY(P1) = #consumer{status = up}}, + {_, ?CONSUMER_PRIORITY(P2) = #consumer{status = up}}) + when P1 =/= P2 -> + P2 =< P1; + ({C1, #consumer{status = up, + credit = Cr1}}, + {C2, #consumer{status = up, + credit = Cr2}}) -> + %% both are up, priority the same + if Cr1 == Cr2 -> + %% same credit + %% sort by key, first attached priority + C1 =< C2; + true -> + %% else sort by credit + Cr2 =< Cr1 + end; + (_, {_, #consumer{status = Status}}) -> + %% not up + Status /= up + end, Waiting). + +merge_consumer(_Meta, #consumer{cfg = CCfg, checked_out = Checked} = Consumer, + ConsumerMeta, {Life, Mode}, Priority) -> + Credit = included_credit(Mode), NumChecked = map_size(Checked), NewCredit = max(0, Credit - NumChecked), - Mode = credit_mode(Meta, Credit, Mode0), Consumer#consumer{cfg = CCfg#consumer_cfg{priority = Priority, meta = ConsumerMeta, credit_mode = Mode, @@ -2303,39 +2312,138 @@ 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 -> - {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{}, - 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(), - 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, - returns = Returns, - messages = Messages, - release_cursors = Cursors, - dlx = DlxState} = State) -> - State#?MODULE{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, +included_credit({simple_prefetch, Credit}) -> + Credit; +included_credit({credited, _}) -> + 0; +included_credit(credited) -> + 0. + +credit_active_consumer( + #credit{credit = LinkCreditRcv, + delivery_count = DeliveryCountRcv, + drain = Drain, + consumer_key = ConsumerKey}, + #consumer{delivery_count = DeliveryCountSnd, + cfg = Cfg} = Con0, + Meta, + #?STATE{consumers = Cons0, + service_queue = ServiceQueue0} = State0) -> + LinkCreditSnd = link_credit_snd(DeliveryCountRcv, LinkCreditRcv, + DeliveryCountSnd, Cfg), + %% grant the credit + Con1 = Con0#consumer{credit = LinkCreditSnd}, + ServiceQueue = maybe_queue_consumer(ConsumerKey, Con1, ServiceQueue0), + State1 = State0#?STATE{service_queue = ServiceQueue, + consumers = maps:update(ConsumerKey, Con1, Cons0)}, + {State2, ok, Effects} = checkout(Meta, State0, State1, []), + + #?STATE{consumers = Cons1 = #{ConsumerKey := Con2}} = State2, + #consumer{cfg = #consumer_cfg{pid = CPid, + tag = CTag}, + credit = PostCred, + delivery_count = PostDeliveryCount} = Con2, + Available = messages_ready(State2), + case credit_api_v2(Cfg) of + true -> + {Credit, DeliveryCount, State} = + case Drain andalso PostCred > 0 of + true -> + AdvancedDeliveryCount = add(PostDeliveryCount, PostCred), + ZeroCredit = 0, + Con = Con2#consumer{delivery_count = AdvancedDeliveryCount, + credit = ZeroCredit}, + Cons = maps:update(ConsumerKey, Con, Cons1), + State3 = State2#?STATE{consumers = Cons}, + {ZeroCredit, AdvancedDeliveryCount, State3}; + false -> + {PostCred, PostDeliveryCount, State2} + end, + %% We must send the delivery effects to the queue client + %% before credit_reply such that session process can send to + %% AMQP 1.0 client TRANSFERs before FLOW. + {State, ok, Effects ++ [{send_msg, CPid, + {credit_reply, CTag, DeliveryCount, + Credit, Available, Drain}, + ?DELIVERY_SEND_MSG_OPTS}]}; + false -> + %% We must always send a send_credit_reply because basic.credit + %% is synchronous. + %% Additionally, we keep the bug of credit API v1 that we + %% send to queue client the + %% send_drained reply before the delivery effects (resulting + %% in the wrong behaviour that the session process sends to + %% AMQP 1.0 client the FLOW before the TRANSFERs). + %% We have to keep this bug because old rabbit_fifo_client + %% implementations expect a send_drained Ra reply + %% (they can't handle such a Ra effect). + CreditReply = {send_credit_reply, Available}, + case Drain of + true -> + AdvancedDeliveryCount = PostDeliveryCount + PostCred, + Con = Con2#consumer{delivery_count = AdvancedDeliveryCount, + credit = 0}, + Cons = maps:update(ConsumerKey, Con, Cons1), + State = State2#?STATE{consumers = Cons}, + Reply = {multi, [CreditReply, + {send_drained, {CTag, PostCred}}]}, + {State, Reply, Effects}; + false -> + {State2, CreditReply, Effects} + end + end. + +credit_inactive_consumer( + #credit{credit = LinkCreditRcv, + delivery_count = DeliveryCountRcv, + drain = Drain, + consumer_key = ConsumerKey}, + #consumer{cfg = #consumer_cfg{pid = CPid, + tag = CTag} = Cfg, + delivery_count = DeliveryCountSnd} = Con0, + Waiting0, State0) -> + %% No messages are available for inactive consumers. + Available = 0, + LinkCreditSnd = link_credit_snd(DeliveryCountRcv, + LinkCreditRcv, + DeliveryCountSnd, + Cfg), + case credit_api_v2(Cfg) of + true -> + {Credit, DeliveryCount} = + case Drain of + true -> + %% By issuing drain=true, the client says "either send a transfer or a flow frame". + %% Since there are no messages to send to an inactive consumer, we advance the + %% delivery-count consuming all link-credit and send a credit_reply with drain=true + %% to the session which causes the session to send a flow frame to the client. + AdvancedDeliveryCount = add(DeliveryCountSnd, LinkCreditSnd), + {0, AdvancedDeliveryCount}; + false -> + {LinkCreditSnd, DeliveryCountSnd} + end, + %% Grant the credit. + Con = Con0#consumer{credit = Credit, + delivery_count = DeliveryCount}, + Waiting = add_waiting({ConsumerKey, Con}, Waiting0), + State = State0#?STATE{waiting_consumers = Waiting}, + {State, ok, + {send_msg, CPid, + {credit_reply, CTag, DeliveryCount, Credit, Available, Drain}, + ?DELIVERY_SEND_MSG_OPTS}}; + false -> + %% Credit API v1 doesn't support draining an inactive consumer. + %% Grant the credit. + Con = Con0#consumer{credit = LinkCreditSnd}, + Waiting = add_waiting({ConsumerKey, Con}, Waiting0), + State = State0#?STATE{waiting_consumers = Waiting}, + {State, {send_credit_reply, Available}} + end. + +is_over_limit(#?STATE{cfg = #cfg{max_length = undefined, max_bytes = undefined}}) -> false; -is_over_limit(#?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 +2451,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) -> @@ -2359,40 +2467,82 @@ 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) -> - #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, + size = ?SIZE(Msg)}; + _ -> + #enqueue{pid = Pid, seq = Seq, msg = Msg} + end. -spec make_register_enqueuer(pid()) -> protocol(). make_register_enqueuer(Pid) -> #register_enqueuer{pid = Pid}. --spec make_checkout(consumer_id(), - checkout_spec(), consumer_meta()) -> protocol(). -make_checkout({_, _} = ConsumerId, 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 + cancel; + _ -> + Spec0 + end, #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 is_return(protocol()) -> boolean(). +is_return(Command) -> + is_record(Command, return). -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}. +-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{}. @@ -2408,52 +2558,47 @@ 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, - msg_bytes_enqueue = Enqueue + Size}. + 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; + 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. - -all_nodes(#?MODULE{consumers = Cons0, - enqueuers = Enqs0, - waiting_consumers = WaitingConsumers0}) -> - Nodes0 = maps:fold(fun({_, P}, _, Acc) -> +all_nodes(#?STATE{consumers = Cons0, + enqueuers = Enqs0, + waiting_consumers = WaitingConsumers0}) -> + Nodes0 = maps:fold(fun(_, ?CONSUMER_PID(P), Acc) -> Acc#{node(P) => ok} end, #{}, Cons0), Nodes1 = maps:fold(fun(P, _, Acc) -> Acc#{node(P) => ok} end, Nodes0, Enqs0), maps:keys( - lists:foldl(fun({{_, P}, _}, Acc) -> + lists:foldl(fun({_, ?CONSUMER_PID(P)}, Acc) -> Acc#{node(P) => ok} end, Nodes1, WaitingConsumers0)). -all_pids_for(Node, #?MODULE{consumers = Cons0, - enqueuers = Enqs0, - waiting_consumers = WaitingConsumers0}) -> - Cons = maps:fold(fun({_, P}, _, Acc) +all_pids_for(Node, #?STATE{consumers = Cons0, + enqueuers = Enqs0, + waiting_consumers = WaitingConsumers0}) -> + Cons = maps:fold(fun(_, ?CONSUMER_PID(P), Acc) when node(P) =:= Node -> [P | Acc]; (_, _, Acc) -> Acc @@ -2463,17 +2608,18 @@ all_pids_for(Node, #?MODULE{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 end, Enqs, WaitingConsumers0). -suspected_pids_for(Node, #?MODULE{consumers = Cons0, - enqueuers = Enqs0, - waiting_consumers = WaitingConsumers0}) -> - Cons = maps:fold(fun({_, P}, - #consumer{status = suspected_down}, +suspected_pids_for(Node, #?STATE{consumers = Cons0, + enqueuers = Enqs0, + waiting_consumers = WaitingConsumers0}) -> + Cons = maps:fold(fun(_Key, + #consumer{cfg = #consumer_cfg{pid = P}, + status = suspected_down}, Acc) when node(P) =:= Node -> [P | Acc]; @@ -2484,14 +2630,15 @@ suspected_pids_for(Node, #?MODULE{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 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) -> @@ -2506,13 +2653,17 @@ is_expired(Ts, #?MODULE{cfg = #cfg{expires = Expires}, is_expired(_Ts, _State) -> false. -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} -> + {_Type, Value} -> Value; - _ -> 0 + _ -> + 0 end; -get_priority_from_args(_) -> +get_priority(_) -> 0. notify_decorators_effect(QName, MaxActivePriority, IsEmpty) -> @@ -2523,41 +2674,38 @@ 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, convert_v1_to_v2(State)); -convert(2, To, State) -> - convert(3, To, convert_v2_to_v3(State)). - -smallest_raft_index(#?MODULE{messages = Messages, - ra_indexes = Indexes, - dlx = DlxState}) -> +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, + 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]). -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, @@ -2567,8 +2715,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 @@ -2581,24 +2729,24 @@ can_immediately_deliver(#?MODULE{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 @@ -2609,3 +2757,185 @@ 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))). + +priority_tag(Msg) -> + case mc:is(Msg) of + true -> + case mc:priority(Msg) of + P when is_integer(P) andalso + P > 4 -> + hi; + _ -> + lo + end; + 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, + 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 > ?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, + smallest_index = NewSmallest, + messages_total = MsgsTot}, + [{checkpoint, LastAppliedIdx, MacState} | + release_cursor(LastSmallest, NewSmallest)]}; + false -> + {Check0#checkpoint{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(_, _) -> + []. + +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 92e15ef91268..a436b5df8adf 100644 --- a/deps/rabbit/src/rabbit_fifo.hrl +++ b/deps/rabbit/src/rabbit_fifo.hrl @@ -39,12 +39,14 @@ -type msg_header() :: msg_size() | optimised_tuple(msg_size(), Expiry :: milliseconds()) | #{size := msg_size(), + acquired_count => non_neg_integer(), delivery_count => non_neg_integer(), expiry => milliseconds()}. %% The message header: %% size: The size of the message payload in bytes. -%% delivery_count: the number of unsuccessful delivery attempts. +%% 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. @@ -53,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. @@ -64,32 +66,39 @@ -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()} | +-type checkout_spec() :: {once | auto, + Num :: non_neg_integer(), + credited | simple_prefetch} | + {dequeue, settled | unsettled} | - cancel. + cancel | remove | + %% 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() + priority => non_neg_integer() }. %% 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(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. @@ -99,6 +108,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(), @@ -107,15 +117,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 + status = up :: up | suspected_down | cancelled | quiescing, + next_msg_id = 0 :: msg_id(), checked_out = #{} :: #{msg_id() => msg()}, %% max number of messages that can be sent %% decremented for each delivery @@ -128,8 +138,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, @@ -164,14 +172,10 @@ 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 = 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()), @@ -187,13 +191,9 @@ % 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() => 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 @@ -202,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(), 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/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl index 0653f6f09e57..20d57d89577f 100644 --- a/deps/rabbit/src/rabbit_fifo_client.erl +++ b/deps/rabbit/src/rabbit_fifo_client.erl @@ -14,14 +14,15 @@ -export([ init/1, init/2, - checkout/5, - cancel_checkout/2, + checkout/4, + cancel_checkout/3, enqueue/3, enqueue/4, dequeue/4, settle/3, return/3, discard/3, + modify/6, credit_v1/4, credit/5, handle_ra_event/4, @@ -38,13 +39,17 @@ -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(). --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 + %% 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 }). -record(cfg, {servers = [] :: [ra:server_id()], @@ -59,12 +64,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() }). @@ -112,6 +116,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} -> @@ -135,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, @@ -145,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. @@ -194,6 +197,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, @@ -203,14 +208,9 @@ 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, 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; @@ -218,15 +218,25 @@ 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); - _ -> - Msg - end. - +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">>, AcqCount, Msg0), + %% the "delivery-count" header in the AMQP spec does not include + %% returns (released outcomes) + rabbit_fifo:annotate_msg(Header, Msg1); + false -> + Msg0 + end, + 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}. %% @doc Settle a message. Permanently removes message from the queue. %% @param ConsumerTag the tag uniquely identifying the consumer. @@ -236,15 +246,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 +275,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,20 +300,35 @@ 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), {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. %% @@ -320,29 +346,45 @@ 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) andalso + is_tuple(CreditMode) -> Servers = sorted_servers(State0), - ConsumerId = {ConsumerTag, self()}, - Cmd = rabbit_fifo:make_checkout(ConsumerId, - {auto, NumUnsettled, CreditMode}, - Meta), + ConsumerId = consumer_id(ConsumerTag), + Spec = case rabbit_fifo:is_v4() of + true -> + case CreditMode of + {simple_prefetch, 0} -> + {auto, {simple_prefetch, + ?UNLIMITED_PREFETCH_COUNT}}; + _ -> + {auto, CreditMode} + end; + false -> + case CreditMode of + {credited, _} -> + {auto, 0, credited}; + {simple_prefetch, 0} -> + {auto, ?UNLIMITED_PREFETCH_COUNT, simple_prefetch}; + {simple_prefetch, Num} -> + {auto, Num, simple_prefetch} + end + end, + Cmd = rabbit_fifo:make_checkout(ConsumerId, Spec, 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 @@ -356,19 +398,21 @@ checkout(ConsumerTag, NumUnsettled, 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, + 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 +436,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,12 +456,12 @@ 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 +%% @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. @@ -426,18 +470,29 @@ 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(), rabbit_queue_type:cancel_reason(), 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, Reason, + #state{consumers = Consumers} = State0) + when is_atom(Reason) -> + case Consumers of + #{ConsumerTag := #consumer{key = Cid}} -> + Servers = sorted_servers(State0), + ConsumerId = {ConsumerTag, self()}, + 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 + {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 @@ -549,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 @@ -681,7 +736,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}; @@ -713,7 +768,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 +784,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 +804,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 +820,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 @@ -785,13 +840,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). @@ -805,17 +854,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,35 +892,32 @@ 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, - #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 rabbit_fifo:is_return(Command) of + true -> + %% returns are sent to the aux machine for pre-evaluation + ra:cast_aux_command(Server, {Command, Seq, self()}); + _ -> + ra:pipeline_command(Server, Command, Seq, Priority) + end, 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, @@ -940,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 12326a13c490..4e787172d1a4 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 @@ -160,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), @@ -192,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), @@ -233,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}]. @@ -357,14 +357,10 @@ 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}) -> rabbit_fifo_index:smallest(Indexes). + +annotate_msg(H, Msg) -> + rabbit_fifo:annotate_msg(H, Msg). diff --git a/deps/rabbit/src/rabbit_fifo_index.erl b/deps/rabbit/src/rabbit_fifo_index.erl index 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 new file mode 100644 index 000000000000..779ba586ec57 --- /dev/null +++ b/deps/rabbit/src/rabbit_fifo_q.erl @@ -0,0 +1,152 @@ +-module(rabbit_fifo_q). + +-include("rabbit_fifo.hrl"). +-export([ + new/0, + in/3, + out/1, + get/1, + len/1, + from_lqueue/1, + get_lowest_index/1, + overview/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 | {msg(), state()}. +out(#?MODULE{len = 0}) -> + empty; +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} -> + {Msg, State#?MODULE{hi = drop(Hi0), + dequeue_counter = C, + len = Len - 1}}; + {lo, Msg} -> + {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 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, _) -> + min(HiIdx, LoIdx); + empty -> + HiIdx + 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, + 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, Out}) -> + {[X | In], Out}. + +peek(?EMPTY) -> + empty; +peek({_, [H | _]}) -> + H. + +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 new file mode 100644 index 000000000000..60ee6be9dc4b --- /dev/null +++ b/deps/rabbit/src/rabbit_fifo_v3.erl @@ -0,0 +1,2574 @@ +%% 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, + 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, + + get_field/2 + ]). + +-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}. + +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}}}; +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)}. + +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_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 4f3db9a3231c..23e588c99e34 100644 --- a/deps/rabbit/src/rabbit_queue_type.erl +++ b/deps/rabbit/src/rabbit_queue_type.erl @@ -114,8 +114,9 @@ -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}. +%% 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(), channel_pid := pid(), limiter_pid => pid() | none, @@ -135,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, @@ -189,7 +196,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 @@ -225,7 +233,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()}. @@ -707,7 +715,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/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index eb92b3670e9a..e9a492a66881 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. @@ -129,11 +134,12 @@ -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 --define(UNLIMITED_PREFETCH_COUNT, 2000). %% something large for ra +% -define(UNLIMITED_PREFETCH_COUNT, 2000). %% something large for ra +-define(MIN_CHECKPOINT_INTERVAL, 8192). %% the ra default is 16384 %%----------- QQ policies --------------------------------------------------- @@ -529,6 +535,7 @@ handle_tick(QName, 0 -> 0; _ -> rabbit_fifo:usage(Name) end, + Keys = ?STATISTICS_KEYS -- [leader, consumers, messages_dlx, @@ -538,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}, @@ -553,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 @@ -569,12 +589,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 @@ -761,6 +781,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]), @@ -824,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). @@ -871,31 +897,26 @@ 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} = - case Mode of - {credited, C} -> - Meta = if C =:= credit_api_v1 -> - #{}; - is_integer(C) -> - #{initial_delivery_count => C} - end, - {credited, 0, 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}), - {ok, QState} = rabbit_fifo_client:checkout(ConsumerTag, EffectivePrefetch, - CreditMode, ConsumerMeta, - QState0), + Prefetch = case Mode of + {simple_prefetch, Declared} -> + Declared; + _ -> + 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, + priority => Priority}, + {ok, _Infos, QState} = rabbit_fifo_client:checkout(ConsumerTag, + Mode, ConsumerMeta, + QState0), case single_active_consumer_on(Q) of true -> %% get the leader from state @@ -910,10 +931,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} -> @@ -925,17 +946,18 @@ 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. 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, @@ -1800,18 +1822,27 @@ 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). - -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), 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, @@ -1819,8 +1850,7 @@ make_ra_conf(Q, ServerId, TickTimeout, SnapshotInterval, Membership) -> friendly_name => FName, metrics_key => QName, initial_members => Members, - log_init_args => #{uid => UId, - snapshot_interval => SnapshotInterval}, + log_init_args => LogCfg, tick_timeout => TickTimeout, machine => RaMachine, ra_event_formatter => Formatter}). @@ -1828,7 +1858,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}. @@ -1974,3 +2004,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. diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl index 40d7c560c9f6..d70a278222c0 100644 --- a/deps/rabbit/test/amqp_client_SUITE.erl +++ b/deps/rabbit/test/amqp_client_SUITE.erl @@ -81,10 +81,15 @@ 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, detach_requeues_one_session_quorum_queue, detach_requeues_drop_head_classic_queue, @@ -109,7 +114,9 @@ groups() -> handshake_timeout, credential_expires, attach_to_exclusive_queue, - classic_priority_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, @@ -209,13 +216,21 @@ 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 -> {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, ['rabbitmq_4.0.0']) 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 @@ -225,40 +240,57 @@ 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 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 credit_api_v2 is disabled"} + {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_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 -> - %% Cancel API v2 reuses feature flag credit_api_v2. + 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 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, 'rabbitmq_4.0.0') of + ok -> + rabbit_ct_helpers:testcase_started(Config, T); + {skip, _} -> + {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 @@ -281,7 +313,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 -> @@ -383,6 +415,234 @@ reliable_send_receive(QType, Outcome, Config) -> ok = end_session_sync(Session2), ok = amqp10_client:close_connection(Connection2). +%% 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( + 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)), + ?assertMatch(#{delivery_count := 0, + first_acquirer := true}, + amqp10_msg:headers(M1)), + ok = amqp10_client:settle_msg(Receiver, M1, {modified, false, true, #{}}), + + {ok, M2a} = amqp10_client:get_msg(Receiver), + ?assertEqual([<<"m2">>], amqp10_msg:body(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)), + ?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([<<"m2">>], amqp10_msg:body(M2c)), + ?assertMatch(#{delivery_count := 1, + first_acquirer := false}, + amqp10_msg:headers(M2c)), + ok = amqp10_client:settle_msg(Receiver, M2c, + {modified, true, false, + #{<<"x-opt-key">> => <<"val 1">>}}), + + {ok, M2d} = amqp10_client:get_msg(Receiver), + ?assertEqual([<<"m2">>], amqp10_msg:body(M2d)), + ?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}}, + 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). + +%% 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) -> @@ -1947,12 +2207,147 @@ 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). -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), @@ -2059,6 +2454,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 @@ -2076,16 +2588,17 @@ 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). -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), @@ -2234,9 +2747,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), @@ -2259,16 +2771,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), @@ -2782,7 +3296,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, _} -> @@ -2963,7 +3477,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. @@ -3014,7 +3535,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">>, @@ -3924,31 +4445,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)), @@ -3959,13 +4492,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). @@ -4076,6 +4609,8 @@ 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} }}), @@ -4106,15 +4641,24 @@ dead_letter_reject(Config) -> ok = wait_for_accepted(Tag), {ok, Msg1} = amqp10_client:get_msg(Receiver), + ?assertMatch(#{delivery_count := 0}, 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), - ok = amqp10_client:settle_msg(Receiver, Msg3, accepted), + ?assertMatch(#{delivery_count := 2, + first_acquirer := false}, amqp10_msg:headers(Msg3)), ?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, @@ -4152,6 +4696,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), diff --git a/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl b/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl index 669eb54348e9..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,8 +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, 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 9b3ed61e84a0..e1bf5abea72b 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, @@ -68,11 +69,13 @@ 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, 'rabbitmq_4.0.0'), + Config2. end_per_group(_, Config) -> rabbit_ct_helpers:run_teardown_steps(Config, @@ -115,22 +118,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 +141,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 +188,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 +253,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 +266,13 @@ run_java_test(Config, Class) -> ], [{cd, TestProjectDir}]), {ok, _} = Ret. + +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..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" "roundtrip-q" + 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 = "/amq/queue/stream_q2" + let address = "/queues/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 = "/queues/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 = "/queues/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 = "/queues/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" "/queues/fragmentation" 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" "/queues/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" "/queues/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" "/queues/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" "/queues/reject" new Message "testing reject" |> sender.Send let m = receiver.Receive() receiver.Reject(m) @@ -345,76 +346,70 @@ module Test = let redelivery uri = use c = connect uri - let sender, receiver = senderReceiver c "test" "redelivery-q" + let sender, receiver = senderReceiver c "test" "/queues/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", "/queues/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" "/queues/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 in + [ + "/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 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 *) - - 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) = @@ -454,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 )) @@ -471,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" @@ -485,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" @@ -521,10 +516,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 +528,7 @@ let main argv = fragmentation uri 0 | [AsLower "message_annotations"; uri] -> - messageAnnotations uri + message_annotations uri 0 | [AsLower "footer"; uri] -> footer uri @@ -544,6 +539,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/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/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. diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index 15b75fac4a69..d34253beb793 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, @@ -171,7 +173,11 @@ 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, + requeue_multiple_true, + requeue_multiple_false ]. memory_tests() -> @@ -236,6 +242,9 @@ init_per_group(Group, Config) -> {skip, _} -> Ret; Config2 -> + 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]), @@ -943,6 +952,7 @@ publish_confirm(Ch, QName, Timeout) -> ct:pal("NOT CONFIRMED! ~ts", [QName]), fail after Timeout -> + flush(1), exit(confirm_timeout) end. @@ -990,6 +1000,185 @@ 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. + +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), @@ -1157,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">>]]), @@ -1169,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. @@ -1243,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), @@ -1255,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), @@ -1274,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">>]]), @@ -1284,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) -> @@ -1683,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( @@ -3280,12 +3474,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), @@ -3559,6 +3755,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) @@ -3609,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), @@ -3621,13 +3899,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) -> @@ -3740,3 +4025,19 @@ 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, 'rabbitmq_4.0.0') of + true -> + ok; + false -> + throw({skip, "test needs feature flag rabbitmq_4.0.0"}) + 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 80f6093129eb..753704affd09 100644 --- a/deps/rabbit/test/rabbit_fifo_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl @@ -14,15 +14,16 @@ -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). %%%=================================================================== %%% Common Test callbacks %%%=================================================================== all() -> [ - {group, machine_version_2}, - {group, machine_version_3}, + {group, tests}, {group, machine_version_conversion} ]. @@ -34,19 +35,28 @@ all_tests() -> groups() -> [ - {machine_version_2, [shuffle], all_tests()}, - {machine_version_3, [shuffle], all_tests()}, - {machine_version_conversion, [shuffle], [convert_v2_to_v3]} + {tests, [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(tests, 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. %%%=================================================================== @@ -59,7 +69,10 @@ end_per_group(_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; @@ -76,37 +89,58 @@ end_per_group(_Group, _Config) -> (_) -> false end, Effects))). +-define(ASSERT(Guard, Fun), + {assert, fun (S) -> ?assertMatch(Guard, S), _ = Fun(S) end}). +-define(ASSERT(Guard), + ?ASSERT(Guard, fun (_) -> true end)). + 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)). +-define(LINE_B, integer_to_binary(?LINE)). + +enq_enq_checkout_compat_test(C) -> + enq_enq_checkout_test(C, {auto, 2, simple_prefetch}). + +enq_enq_checkout_v4_test(C) -> + enq_enq_checkout_test(C, {auto, {simple_prefetch, 2}}). + +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), - ct:pal("~tp", [Effects]), + {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), rabbit_fifo: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; (_) -> @@ -114,151 +148,198 @@ 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), 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. + +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, 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 @@ -267,52 +348,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. @@ -321,104 +407,125 @@ 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), 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), - 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(C, 2, 100), Enq2, S1), + Idx2 = ?LINE, + {S2, ok, _} = apply(meta(Config, Idx2, 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), - {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). -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), - % no release cursor effect at this point - {State4, _} = settle(C, Cid, 4, 1, State3), - {_Final, Effects1} = settle(C, Cid, 5, 0, State4), - % empty queue forwards release cursor all the way - ?ASSERT_EFF({release_cursor, 5, _}, Effects1), +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), 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), - % the release cursor is the smallest raft index that does not - % contribute to the state of the application - % ?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_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) -> @@ -444,33 +551,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 doesn't 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, @@ -479,124 +580,173 @@ 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), + {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(C, 3), rabbit_fifo:make_return(Cid, [MsgId]), State1), + {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}}]}, _}] + [{send_msg, _, {delivery, _, [{_MsgId2, {#{acquired_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 := 1, + acquired_count := 2}, first}}]}, _}] + = Fun3([Msg1]), ok. -cancelled_checkout_empty_queue_test(C) -> +requeue_test(Config) -> Cid = {<<"cid">>, self()}, - {State1, _} = check_auto(C, Cid, 2, test_init(test)), + 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, {#{acquired_count := 1}, first}}]}, _}] + = Fun2([Msg1]), + ok. + +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), - ?assertEqual(1, lqueue:len(State2#rabbit_fifo.messages)), + {State2, _, _} = apply(meta(Config, 4), + rabbit_fifo:make_checkout(Cid, cancel, #{}), State1), + ?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)), {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) -> +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 = {<<"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)), - ?assertEqual(1, lqueue:len(State0#rabbit_fifo.messages)), + {State0, _} = enq(Config, 1, 1, second, test_init(test)), + ?assertEqual(1, rabbit_fifo_q:len(State0#rabbit_fifo.messages)), ?assertEqual(0, lqueue:len(State0#rabbit_fifo.returns)), - {State1, {_, _}} = deq(C, 2, Cid, unsettled, Msg, State0), - ?assertEqual(0, lqueue:len(State1#rabbit_fifo.messages)), + {State1, {_, _}} = deq(Config, 2, Cid, unsettled, Msg, State0), + ?assertEqual(0, rabbit_fifo_q:len(State1#rabbit_fifo.messages)), ?assertEqual(0, lqueue:len(State1#rabbit_fifo.returns)), - {State2a, _, _} = apply(meta(C, 3), {down, Pid, noconnection}, State1), - ?assertEqual(0, lqueue:len(State2a#rabbit_fifo.messages)), + {State2a, _, _} = apply(meta(Config, 3), {down, Pid, noconnection}, State1), + ?assertEqual(0, rabbit_fifo_q:len(State2a#rabbit_fifo.messages)), ?assertEqual(1, lqueue:len(State2a#rabbit_fifo.returns)), ?assertMatch(#consumer{checked_out = Ch, status = suspected_down} @@ -604,49 +754,72 @@ 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), rabbit_fifo:make_enqueue(Pid, 1, first), State00), + {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), + + 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), ?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), - ?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) -> {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">>), @@ -656,12 +829,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]), @@ -673,17 +845,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{}, @@ -700,38 +872,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)), + ?assertEqual(1, rabbit_fifo_q: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), @@ -749,47 +921,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)), + ?assertEqual(100, rabbit_fifo_q: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)), @@ -797,30 +970,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)), + ?assertEqual(NumMsgs, rabbit_fifo_q: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, @@ -829,27 +1002,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)), + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), 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, @@ -860,12 +1034,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)), %% @@ -878,22 +1052,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)), @@ -903,62 +1076,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 @@ -968,12 +1141,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 @@ -984,7 +1157,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 @@ -997,33 +1170,34 @@ 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}), + queue_resource => rabbit_misc:r("/", queue, ?FUNCTION_NAME_B), + single_active_consumer_on => true}), - DummyFunction = fun() -> ok end, - Pid1 = spawn(DummyFunction), - Pid2 = spawn(DummyFunction), - Pid3 = spawn(DummyFunction), + 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}}, #{})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = up}}}), + % the channel of the active consumer goes down + {?LINE, {down, Pid1, noproc}} + ], + {State2, Effects} = run_log(Config, State0, Entries), - [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), - - % 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 @@ -1035,8 +1209,11 @@ single_active_consumer_cancel_consumer_when_channel_is_down_test(C) -> ?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(C, 3), {down, Pid2, noproc}, State2), + {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 @@ -1050,7 +1227,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))), @@ -1060,33 +1238,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, @@ -1096,56 +1263,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))), @@ -1154,190 +1312,167 @@ 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), - %% assert the consumer is up - ?assertMatch(#{C1 := #consumer{status = up}}, State1#rabbit_fifo.consumers), - - % simulate node goes down - {State2, _, _} = apply(meta(C, 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), - %% 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), - %% ensure n2 is the active consumer as this node as been registered - %% as up again - ?assertMatch([{{<<"ctag_n2">>, _}, #consumer{status = up, - credit = 1}}], - maps:to_list(State4#rabbit_fifo.consumers)), - ok. - -single_active_consumer_state_enter_leader_include_waiting_consumers_test(C) -> + {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}}, #{})}, + ?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), + ok. + +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), @@ -1345,32 +1480,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}), @@ -1380,200 +1517,574 @@ 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. -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. + +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 = quiescing}}, + 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_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}), + + 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 quiescing + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 2})}, + ?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 = quiescing, + 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 quiescing + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 2})}, + ?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 = quiescing}}, + 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_quiescing_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 quiescing + {CK2, make_checkout(C2, {auto, {simple_prefetch, 1}}, #{priority => 2})}, + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = quiescing}}, + 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_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}), + + 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 quiescing + {CK2, make_checkout(C2, {auto, {simple_prefetch, 10}}, #{priority => 2})}, + ?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 quiescing consumer + ?ASSERT(#rabbit_fifo{consumers = #{CK1 := #consumer{status = quiescing, + 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(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), make_register_enqueuer(Pid1), State0), + {State1, ok, [_]} = apply(meta(Config, 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(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), make_register_enqueuer(Pid2), State2), + {State3, ok, [_]} = apply(meta(Config, 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), - % ct:pal("Efx ~tp", [Efx]), + {State4, ok, _} = apply(meta(Config, 4, ?LINE, {notify, 4, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 2, two), State3), + {State5, ok, Efx} = apply(meta(Config, 5, ?LINE, {notify, 4, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 3, three), State4), %% 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(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), @@ -1581,27 +2092,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), 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(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(Config, 3, ?LINE, {notify, 2, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 2, two), State2), + {State4, ok, Efx} = apply(meta(Config, 4, ?LINE, {notify, 2, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 3, three), State3), % ct:pal("Efx ~tp", [Efx]), ?ASSERT_EFF({send_msg, P, {queue_status, reject_publish}, [ra_event]}, P == Pid1, Efx), - {_State5, {purge, 3}, Efx1} = apply(meta(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 @@ -1609,12 +2122,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), 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), - % ct:pal("Efx ~tp", [Efx]), - ?ASSERT_NO_EFF({send_msg, P, {queue_status, reject_publish}, [ra_event]}, P == Pid1, Efx), + {State1, ok, [_]} = apply(meta(Config, 1, ?LINE, {notify, 1, Pid1}), + make_register_enqueuer(Pid1), State0), + {State2, ok, _} = apply(meta(Config, 2, ?LINE, {notify, 1, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 1, one), State1), + {State3, ok, _} = apply(meta(Config, 3, ?LINE, {notify, 1, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 2, two), State2), + {State4, ok, Efx} = apply(meta(Config, 4, ?LINE, {notify, 1, Pid1}), + rabbit_fifo:make_enqueue(Pid1, 3, three), State3), + ?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, @@ -1623,78 +2140,81 @@ 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), + {State1, _, _} = apply(meta(Config, 1, ?LINE, {notify, 1, EnqPid}), rabbit_fifo:make_enqueue(EnqPid, 1, msg1), State0), - {State2, _, _} = apply(meta(C, 2), + {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, - [#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), + {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] - ]}}] , rabbit_fifo:tick(1, State)), + ?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. 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: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} = 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]), @@ -1724,8 +2244,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)). @@ -1737,17 +2269,36 @@ credit(Config, Cid, Idx, Credit, DelCnt, Drain, State) -> 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 - {Acc, _, Efx} when is_list(Efx) -> - {Acc, Efx0 ++ Efx}; - {Acc, _, Efx} -> - {Acc, Efx0 ++ [Efx]}; - {Acc, _} -> - {Acc, Efx0} - end - end, {InitState, []}, Entries). + 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 Module: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 @@ -1755,16 +2306,18 @@ run_log(Config, InitState, Entries) -> aux_test(_) -> _ = ra_machine_ets:start_link(), Aux0 = init_aux(aux_test), - MacState = init(#{name => aux_test, - queue_resource => - rabbit_misc:r(<<"/">>, queue, <<"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, + last_applied => LastApplied}, 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, _, + [{release_cursor, LastApplied}]} = handle_aux(leader, cast, tick, Aux, State), [X] = ets:lookup(rabbit_fifo_usage, aux_test), meck:unload(), ?assert(X > 0.0), @@ -1832,9 +2385,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 +2401,55 @@ 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], + + EPid = test_util:fake_pid(node()), + Pid1 = test_util:fake_pid(node()), + Cid1 = {ctag1, Pid1}, + Cid2 = {ctag2, self()}, + MaxCredits = 2, + Entries = [ + {1, rabbit_fifo_v3:make_enqueue(EPid, 1, banana)}, + {2, rabbit_fifo_v3:make_enqueue(EPid, 2, apple)}, + {3, rabbit_fifo_v3:make_enqueue(EPid, 3, orange)}, + {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, + queue_resource => rabbit_misc:r("/", queue, atom_to_binary(Name)), + release_cursor_interval => 0}), + {State, _} = run_log(rabbit_fifo_v3, ConfigV3, Init, Entries, + fun (_) -> true end), + + %% convert from v3 to v4 + {#rabbit_fifo{consumers = Consumers, + returns = Returns}, ok, _} = + apply(meta(ConfigV4, ?LINE), {machine_version, 3, 4}, State), + + ?assertEqual(1, maps:size(Consumers)), + ?assertMatch(#consumer{cfg = #consumer_cfg{credit_mode = + {simple_prefetch, MaxCredits}}}, + maps:get(Cid2, Consumers)), + ?assertNot(is_map_key(Cid1, Consumers)), + %% assert delivery_count is copied to acquired_count + #consumer{checked_out = Ch2} = maps:get(Cid2, Consumers), + ?assertMatch(#{0 := ?MSG(_, #{delivery_count := 1, + acquired_count := 1}), + 1 := ?MSG(_, #{delivery_count := 1, + acquired_count := 1})}, Ch2), + + ?assertMatch(?MSG(_, #{delivery_count := 1, + acquired_count := 1}), lqueue:get(Returns)), + + ok. + queue_ttl_test(C) -> QName = rabbit_misc:r(<<"/">>, queue, <<"test">>), Conf = #{name => ?FUNCTION_NAME, @@ -1867,7 +2469,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 +2490,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), @@ -1897,11 +2499,11 @@ 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), - rabbit_fifo:make_checkout(Deq, {dequeue, unsettled}, #{}), + make_checkout(Deq, {dequeue, unsettled}, #{}), E1), {log, [2], Fun2} = get_log_eff(Effs2), @@ -1915,7 +2517,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, @@ -1930,12 +2532,12 @@ 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), - rabbit_fifo:make_checkout(Cid, cancel, #{}), S1), + {S2, _, _} = apply(meta(Config, ?LINE, Now), + 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), @@ -1943,7 +2545,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 @@ -1953,11 +2555,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)), @@ -1965,56 +2567,29 @@ 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), - rabbit_fifo:make_checkout(Cid, {once, 2, simple_prefetch}, - #{args => Args}), + apply(meta(Config, ?LINE), + make_checkout(Cid, {auto, {simple_prefetch, 2}}, + #{args => Args}), test_init(test)), {S2, _, _} = - apply(meta(C, 3), - rabbit_fifo: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) -> - State0 = test_init(?FUNCTION_NAME), - Cid = {<<"basic.get1">>, self()}, - {_State, {dequeue, empty}, Effects} = - apply(meta(C, 2, 1234), - rabbit_fifo:make_checkout(Cid, {dequeue, unsettled}, #{}), - State0), - - ?ASSERT_EFF({release_cursor, _, _}, Effects), - ok. - -expire_message_should_emit_release_cursor_test(C) -> - Conf = #{name => ?FUNCTION_NAME, - queue_resource => rabbit_misc:r(<<"/">>, queue, <<"test">>), - 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), rabbit_fifo:make_enqueue(self(), 1, Msg), S0), - {_S, ok, Effs} = apply(meta(C, 2, 101), - rabbit_fifo:make_enqueue(self(), 2, Msg), - S1), - ?ASSERT_EFF({release_cursor, 1, _}, Effs), - ok. - header_test(_) -> H0 = Size = 5, ?assertEqual(Size, rabbit_fifo:get_header(size, H0)), @@ -2086,18 +2661,80 @@ 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), - rabbit_fifo: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), - rabbit_fifo: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}, #{}), - State2), + make_checkout(Cid, cancel, #{}), State1), + {_State3, #{next_msg_id := 1, + num_checked_out := 1}, _} = + 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 @@ -2106,8 +2743,22 @@ 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) -> 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)). + +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 787b60a30d00..2ae8e4bc55f8 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, @@ -31,6 +33,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, @@ -64,6 +69,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,19 +96,18 @@ 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, - #{}, 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 - 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, @@ -180,7 +186,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 +221,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 +274,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 @@ -284,28 +290,101 @@ 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, + {FC5, _} = + receive + {ra_event, Qname2, + {machine, {delivery, _, [{MsgId1, {_, _Msg1Out}}]}} = Evt2} -> + {ok, FC4, Actions2} = + rabbit_fifo_client:handle_ra_event(Qname2, Qname2, Evt2, FC3), + [{deliver, _tag, true, + [{_, _, _, _, Msg1Out}]}] = Actions2, + ?assert(mc:is(Msg1Out)), + ?assertEqual(1, mc:get_annotation(<<"x-delivery-count">>, Msg1Out)), + %% 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_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. + 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">>, 10, - simple_prefetch, - #{}, F), - Self ! checkout_done - end), - receive checkout_done -> ok after 1000 -> exit(checkout_done_timeout) end, - timer:sleep(1000), + {_, 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 + 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. @@ -378,8 +457,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 +480,70 @@ 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, F4} = rabbit_fifo_client:cancel_checkout(<<"tag">>, F3), + {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">>, cancel, 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. + +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) -> @@ -415,8 +553,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} -> @@ -441,6 +580,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]), @@ -450,7 +590,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, 0}, #{}, F3), %% assert no deliveries {_, _, F5} = process_ra_events(receive_ra_events(), ClusterName, F4, [], [], fun @@ -497,9 +637,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, 0, credited, #{initial_delivery_count => DC0}, F3), + CTag, {credited, DC0}, #{}, F3), %% assert no deliveries {_, _, F5} = process_ra_events(receive_ra_events(), ClusterName, F4, [], [], fun @@ -598,7 +738,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), @@ -626,8 +766,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), @@ -687,7 +827,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 @@ -745,3 +885,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]}}). diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index c151c1cd0214..273597982f31 100644 --- a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl @@ -11,9 +11,10 @@ -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). %%%=================================================================== %%% Common Test callbacks @@ -62,10 +63,6 @@ all_tests() -> scenario31, scenario32, upgrade, - upgrade_snapshots, - upgrade_snapshots_scenario1, - upgrade_snapshots_scenario2, - upgrade_snapshots_v2_to_v3, messages_total, simple_prefetch, simple_prefetch_without_checkout_cancel, @@ -88,8 +85,8 @@ all_tests() -> dlx_06, dlx_07, dlx_08, - dlx_09 - % single_active_ordering_02 + dlx_09, + single_active_ordering_02 ]. groups() -> @@ -110,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)}, @@ -693,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(#{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(#{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}, @@ -765,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 = [ @@ -788,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, @@ -856,8 +799,6 @@ snapshots(_Config) -> Bytes, SingleActiveConsumer, DeliveryLimit, - InMemoryLength, - InMemoryBytes, Overflow, DeadLetterHandler), ?FORALL(O, ?LET(Ops, log_gen(256), expand(Ops, Config)), @@ -867,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 @@ -886,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)), @@ -897,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))) @@ -924,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 @@ -940,10 +877,8 @@ upgrade(_Config) -> Bytes, SingleActive, DeliveryLimit, - InMemoryLength, - undefined, drop_head, - {?MODULE, banana, []} + undefined ), ?FORALL(O, ?LET(Ops, log_gen(Size), expand(Ops, Config)), collect({log_size, length(O)}, @@ -951,36 +886,8 @@ upgrade(_Config) -> end) end, [], Size). -upgrade_snapshots(_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(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 () -> @@ -996,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))) @@ -1034,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}, @@ -1048,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))) @@ -1059,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}, @@ -1073,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) @@ -1105,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) -> @@ -1132,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. @@ -1153,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. @@ -1173,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), @@ -1198,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)}, @@ -1235,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. @@ -1257,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. @@ -1278,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. @@ -1298,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. @@ -1324,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. @@ -1352,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. @@ -1385,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. @@ -1431,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. @@ -1453,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; @@ -1485,7 +1364,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; _ -> @@ -1531,7 +1410,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}, @@ -1555,7 +1434,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() -> @@ -1564,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), @@ -1590,7 +1469,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) -> @@ -1598,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. @@ -1628,24 +1511,26 @@ 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), + {V3, _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{} = V4 = element(1, rabbit_fifo:apply( + meta(length(PreEntries) + 1), + {machine_version, FromVersion, ToVersion}, + 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, @@ -1653,42 +1538,18 @@ 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)), - 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) + run_log(V4, PostEntries, fun (_) -> true end, ToVersion) end || SplitPos <- lists:seq(1, length(Entries))], - - {_, V1Effs} = run_log(InitState, Entries, fun (_) -> true end, - rabbit_fifo_v1), - [begin - Res = rabbit_fifo:apply(meta(Idx + 1), {machine_version, 1, 2}, RCS) , - #rabbit_fifo{} = V2 = element(1, Res), - %% assert invariants - Fields = [num_ready_messages, - smallest_raft_index, - num_enqueuers, - num_consumers, - enqueue_message_bytes, - checkout_message_bytes - ], - V1Overview = maps:with(Fields, rabbit_fifo_v1:overview(RCS)), - V2Overview = maps:with(Fields, rabbit_fifo:overview(V2)), - case V1Overview == V2Overview of - true -> ok; - false -> - ct:pal("upgrade_prop failed expected~n~tp~nGot:~n~tp", - [V1Overview, V2Overview]), - ?assertEqual(V1Overview, V2Overview) - end - end || {release_cursor, Idx, RCS} <- V1Effs], true. %% single active consumer ordering invariant: @@ -1720,27 +1581,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(Conf, Commands) -> - try run_upgrade_snapshot_test(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 -> @@ -1772,28 +1613,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, @@ -1812,14 +1631,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} ]))))). @@ -1946,16 +1759,21 @@ 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], - properties = none}}. + properties = #'P_basic'{}}}. checkout_cancel_gen(Pid) -> {checkout, Pid, cancel}. @@ -1974,7 +1792,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) -> @@ -2000,9 +1819,11 @@ expand(Ops, Config, EnqFun) -> _ -> InitConfig0 end, + IsV4 = rabbit_feature_flags:is_enabled('rabbitmq_4.0.0'), 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, @@ -2024,7 +1845,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, @@ -2054,9 +1875,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">>, @@ -2084,13 +1911,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} -> @@ -2113,7 +1951,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). @@ -2181,145 +2020,17 @@ 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}, Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), - {State0, Effects} = run_log(test_init(Conf), Entries, Invariant, rabbit_fifo), - 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, rabbit_fifo), - 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(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, rabbit_fifo_v1), - 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, rabbit_fifo_v1), - %% Perform conversion and recover in V2. - Res = rabbit_fifo:apply(meta(SnapIdx + 1), {machine_version, 1, 2}, SnapState), - #rabbit_fifo{} = V2 = element(1, Res), - {StateV2, _} = run_log(V2, FilteredV2, Invariant, rabbit_fifo, 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: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(), + MsgTotFun(S) + 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(Conf), Entries, Invariant, rabbit_fifo, 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, rabbit_fifo, 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), - %% 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}. @@ -2332,45 +2043,64 @@ 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, InvariantFun, FifoMod) -> - run_log(InitState, Entries, InvariantFun, FifoMod, 3). - -run_log(InitState, Entries, InvariantFun, FifoMod, MachineVersion) -> - Invariant = fun(E, S) -> - case InvariantFun(S) of + run_log(InitState, Entries, fun(_) -> true end). + +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, Effs) -> + case InvariantFun(E, S, Effs) of true -> ok; false -> 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), + 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). 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). + +test_init_v(Conf, Version) -> + test_init(rabbit_fifo:which_module(Version), Conf). meta(Idx) -> meta(Idx, 3). 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..942ba591c3c9 --- /dev/null +++ b/deps/rabbit/test/rabbit_fifo_q_SUITE.erl @@ -0,0 +1,208 @@ +-module(rabbit_fifo_q_SUITE). + +-compile(nowarn_export_all). +-compile(export_all). + +-include_lib("proper/include/proper.hrl"). +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("rabbit/src/rabbit_fifo.hrl"). + +all() -> + [ + {group, tests} + ]. + + +all_tests() -> + [ + hi, + basics, + hi_is_prioritised, + get_lowest_index, + single_priority_behaves_like_queue + ]. + + +groups() -> + [ + {tests, [parallel], all_tests()} + ]. + +init_per_suite(Config) -> + Config. + +end_per_suite(_Config) -> + ok. + +init_per_group(_Group, Config) -> + Config. + +end_per_group(_Group, _Config) -> + ok. + +init_per_testcase(_TestCase, Config) -> + Config. + +end_per_testcase(_TestCase, _Config) -> + ok. + +%%%=================================================================== +%%% Test cases +%%%=================================================================== + +-define(MSG(L), ?MSG(L, L)). + +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(1)} + ]), + {?MSG(1), Q2} = rabbit_fifo_q:out(Q1), + empty = 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(1)}, + {lo, ?MSG(2)}, + {hi, ?MSG(3)}, + {lo, ?MSG(4)}, + {hi, ?MSG(5)} + ]), + {?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) -> + 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)}, + {hi, ?MSG(2)}, + {hi, ?MSG(3)}, + {hi, ?MSG(4)}, + {lo, ?MSG(5)} + ]), + {?MSG(1), Q2} = rabbit_fifo_q:out(Q1), + {?MSG(2), Q3} = rabbit_fifo_q:out(Q2), + {?MSG(3), Q4} = rabbit_fifo_q:out(Q3), + {?MSG(4), Q5} = rabbit_fifo_q:out(Q4), + {?MSG(5), Q6} = rabbit_fifo_q:out(Q5), + empty = rabbit_fifo_q:out(Q6), + ok. + +get_lowest_index(_Config) -> + Q0 = rabbit_fifo_q:new(), + Q1 = rabbit_fifo_q:in(hi, ?MSG(1, ?LINE), Q0), + Q2 = rabbit_fifo_q:in(lo, ?MSG(2, ?LINE), Q1), + Q3 = rabbit_fifo_q:in(lo, ?MSG(3, ?LINE), Q2), + {_, Q4} = rabbit_fifo_q:out(Q3), + {_, Q5} = rabbit_fifo_q:out(Q4), + {_, Q6} = rabbit_fifo_q:out(Q5), + + ?assertEqual(undefined, rabbit_fifo_q:get_lowest_index(Q0)), + ?assertEqual(1, rabbit_fifo_q:get_lowest_index(Q1)), + ?assertEqual(1, rabbit_fifo_q:get_lowest_index(Q2)), + ?assertEqual(1, rabbit_fifo_q:get_lowest_index(Q3)), + ?assertEqual(2, rabbit_fifo_q:get_lowest_index(Q4)), + ?assertEqual(3, rabbit_fifo_q:get_lowest_index(Q5)), + ?assertEqual(undefined, rabbit_fifo_q:get_lowest_index(Q6)). + +-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, V0}, Q1} -> + {V0, Q1}; + Res0 -> + Res0 + end, + {V2, S} = case rabbit_fifo_q:out(S0) of + empty -> + {empty, S0}; + 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/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() -> 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) %> 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)); 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 fdb82dada0c4..d3110c5f5cd9 100755 --- a/moduleindex.yaml +++ b/moduleindex.yaml @@ -647,8 +647,10 @@ rabbit: - rabbit_fifo_dlx_sup - rabbit_fifo_dlx_worker - rabbit_fifo_index +- rabbit_fifo_q - rabbit_fifo_v0 - rabbit_fifo_v1 +- rabbit_fifo_v3 - rabbit_file - rabbit_global_counters - rabbit_guid