Merge pull request #3121 from rabbitmq/quorum-queues-v2

QQ: introduce new machine version (2)
This commit is contained in:
Karl Nilsson 2022-03-08 16:19:04 +00:00 committed by GitHub
commit 4b193608bb
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
49 changed files with 8571 additions and 2032 deletions

View File

@ -144,7 +144,9 @@ _APP_ENV = """[
%% interval at which connection/channel tracking executes post operations
{tracking_execution_timeout, 15000},
{stream_messages_soft_limit, 256},
{track_auth_attempt_source, false}
{track_auth_attempt_source, false},
{dead_letter_worker_consumer_prefetch, 32},
{dead_letter_worker_publisher_confirm_timeout, 180000}
]
"""
@ -401,7 +403,7 @@ suites = [
":quorum_queue_utils",
],
flaky = True,
shard_count = 3,
shard_count = 7,
),
rabbitmq_integration_suite(
PACKAGE,
@ -698,7 +700,7 @@ suites = [
),
rabbitmq_suite(
name = "rabbit_fifo_prop_SUITE",
size = "medium",
size = "large",
additional_beam = [
":test_util",
],
@ -716,6 +718,37 @@ suites = [
"@proper//:erlang_app",
],
),
rabbitmq_suite(
name = "rabbit_fifo_dlx_SUITE",
size = "small",
additional_hdrs = [
"src/rabbit_fifo.hrl",
"src/rabbit_fifo_dlx.hrl",
],
deps = [
"//deps/rabbit_common:erlang_app",
],
),
rabbitmq_integration_suite(
PACKAGE,
name = "rabbit_fifo_dlx_integration_SUITE",
size = "medium",
additional_beam = [
":test_util",
":quorum_queue_utils",
":quorum_queue_SUITE_beam_files",
],
additional_hdrs = [
"src/rabbit_fifo.hrl",
"src/rabbit_fifo_dlx.hrl",
],
runtime_deps = [
"@ra//:erlang_app",
],
deps = [
"@proper//:erlang_app",
],
),
rabbitmq_suite(
name = "rabbit_fifo_SUITE",
size = "medium",

View File

@ -122,7 +122,9 @@ define PROJECT_ENV
%% interval at which connection/channel tracking executes post operations
{tracking_execution_timeout, 15000},
{stream_messages_soft_limit, 256},
{track_auth_attempt_source, false}
{track_auth_attempt_source, false},
{dead_letter_worker_consumer_prefetch, 32},
{dead_letter_worker_publisher_confirm_timeout, 180000}
]
endef

View File

@ -778,6 +778,7 @@ declare_args() ->
{<<"x-message-ttl">>, fun check_message_ttl_arg/2},
{<<"x-dead-letter-exchange">>, fun check_dlxname_arg/2},
{<<"x-dead-letter-routing-key">>, fun check_dlxrk_arg/2},
{<<"x-dead-letter-strategy">>, fun check_dlxstrategy_arg/2},
{<<"x-max-length">>, fun check_non_neg_int_arg/2},
{<<"x-max-length-bytes">>, fun check_non_neg_int_arg/2},
{<<"x-max-in-memory-length">>, fun check_non_neg_int_arg/2},
@ -946,6 +947,22 @@ check_dlxrk_arg(Val, Args) when is_binary(Val) ->
check_dlxrk_arg(_Val, _Args) ->
{error, {unacceptable_type, "expected a string"}}.
-define(KNOWN_DLX_STRATEGIES, [<<"at-most-once">>, <<"at-least-once">>]).
check_dlxstrategy_arg({longstr, Val}, _Args) ->
case lists:member(Val, ?KNOWN_DLX_STRATEGIES) of
true -> ok;
false -> {error, invalid_dlx_strategy}
end;
check_dlxstrategy_arg({Type, _}, _Args) ->
{error, {unacceptable_type, Type}};
check_dlxstrategy_arg(Val, _Args) when is_binary(Val) ->
case lists:member(Val, ?KNOWN_DLX_STRATEGIES) of
true -> ok;
false -> {error, invalid_dlx_strategy}
end;
check_dlxstrategy_arg(_Val, _Args) ->
{error, invalid_dlx_strategy}.
-define(KNOWN_OVERFLOW_MODES, [<<"drop-head">>, <<"reject-publish">>, <<"reject-publish-dlx">>]).
check_overflow({longstr, Val}, _Args) ->
case lists:member(Val, ?KNOWN_OVERFLOW_MODES) of
@ -1503,7 +1520,8 @@ get_queue_consumer_info(Q, ConsumerInfoKeys) ->
[lists:zip(ConsumerInfoKeys,
[amqqueue:get_name(Q), ChPid, CTag,
AckRequired, Prefetch, Active, ActivityStatus, Args]) ||
{ChPid, CTag, AckRequired, Prefetch, Active, ActivityStatus, Args, _} <- consumers(Q)].
{ChPid, CTag, AckRequired, Prefetch, Active, ActivityStatus, Args, _}
<- consumers(Q)].
-spec stat(amqqueue:amqqueue()) ->
{'ok', non_neg_integer(), non_neg_integer()}.
@ -1657,8 +1675,8 @@ credit(Q, CTag, Credit, Drain, QStates) ->
{'ok', non_neg_integer(), qmsg(), rabbit_queue_type:state()} |
{'empty', rabbit_queue_type:state()} |
{protocol_error, Type :: atom(), Reason :: string(), Args :: term()}.
basic_get(Q, NoAck, LimiterPid, CTag, QStates0) ->
rabbit_queue_type:dequeue(Q, NoAck, LimiterPid, CTag, QStates0).
basic_get(Q, NoAck, LimiterPid, CTag, QStates) ->
rabbit_queue_type:dequeue(Q, NoAck, LimiterPid, CTag, QStates).
-spec basic_consume(amqqueue:amqqueue(), boolean(), pid(), pid(), boolean(),
@ -1670,7 +1688,7 @@ basic_get(Q, NoAck, LimiterPid, CTag, QStates0) ->
{protocol_error, Type :: atom(), Reason :: string(), Args :: term()}.
basic_consume(Q, NoAck, ChPid, LimiterPid,
LimiterActive, ConsumerPrefetchCount, ConsumerTag,
ExclusiveConsume, Args, OkMsg, ActingUser, Contexts) ->
ExclusiveConsume, Args, OkMsg, ActingUser, QStates) ->
QName = amqqueue:get_name(Q),
%% first phase argument validation
@ -1686,7 +1704,7 @@ basic_consume(Q, NoAck, ChPid, LimiterPid,
args => Args,
ok_msg => OkMsg,
acting_user => ActingUser},
rabbit_queue_type:consume(Q, Spec, Contexts).
rabbit_queue_type:consume(Q, Spec, QStates).
-spec basic_cancel(amqqueue:amqqueue(), rabbit_types:ctag(), any(),
rabbit_types:username(),

View File

@ -728,10 +728,14 @@ maybe_deliver_or_enqueue(Delivery = #delivery{message = Message},
with_dlx(
DLX,
fun (X) ->
rabbit_global_counters:messages_dead_lettered(maxlen, rabbit_classic_queue,
at_most_once, 1),
QName = qname(State),
rabbit_dead_letter:publish(Message, maxlen, X, RK, QName)
end,
fun () -> ok end),
fun () -> rabbit_global_counters:messages_dead_lettered(maxlen, rabbit_classic_queue,
disabled, 1)
end),
%% Drop publish and nack to publisher
send_reject_publish(Delivery, Delivered, State);
_ ->
@ -763,6 +767,8 @@ deliver_or_enqueue(Delivery = #delivery{message = Message,
{undelivered, State2 = #q{ttl = 0, dlx = undefined,
backing_queue_state = BQS,
msg_id_to_channel = MTC}} ->
rabbit_global_counters:messages_dead_lettered(expired, rabbit_classic_queue,
disabled, 1),
{BQS1, MTC1} = discard(Delivery, BQ, BQS, MTC, amqqueue:get_name(Q)),
State2#q{backing_queue_state = BQS1, msg_id_to_channel = MTC1};
{undelivered, State2 = #q{backing_queue_state = BQS}} ->
@ -804,6 +810,9 @@ maybe_drop_head(AlreadyDropped, State = #q{backing_queue = BQ,
State#q.dlx,
fun (X) -> dead_letter_maxlen_msg(X, State) end,
fun () ->
rabbit_global_counters:messages_dead_lettered(maxlen,
rabbit_classic_queue,
disabled, 1),
{_, BQS1} = BQ:drop(false, BQS),
State#q{backing_queue_state = BQS1}
end));
@ -1012,11 +1021,18 @@ drop_expired_msgs(State) ->
drop_expired_msgs(Now, State = #q{backing_queue_state = BQS,
backing_queue = BQ }) ->
ExpirePred = fun (#message_properties{expiry = Exp}) -> Now >= Exp end,
ExpirePredIncrement = fun(Properties) ->
ExpirePred(Properties) andalso
rabbit_global_counters:messages_dead_lettered(expired,
rabbit_classic_queue,
disabled,
1) =:= ok
end,
{Props, State1} =
with_dlx(
State#q.dlx,
fun (X) -> dead_letter_expired_msgs(ExpirePred, X, State) end,
fun () -> {Next, BQS1} = BQ:dropwhile(ExpirePred, BQS),
fun () -> {Next, BQS1} = BQ:dropwhile(ExpirePredIncrement, BQS),
{Next, State#q{backing_queue_state = BQS1}} end),
ensure_ttl_timer(case Props of
undefined -> undefined;
@ -1058,6 +1074,8 @@ dead_letter_msgs(Fun, Reason, X, State = #q{dlx_routing_key = RK,
QName = qname(State),
{Res, Acks1, BQS1} =
Fun(fun (Msg, AckTag, Acks) ->
rabbit_global_counters:messages_dead_lettered(Reason, rabbit_classic_queue,
at_most_once, 1),
rabbit_dead_letter:publish(Msg, Reason, X, RK, QName),
[AckTag | Acks]
end, [], BQS),
@ -1575,7 +1593,9 @@ handle_cast({reject, false, AckTags, ChPid}, State) ->
dead_letter_rejected_msgs(
AckTags, X, State1)
end) end,
fun () -> ack(AckTags, ChPid, State) end));
fun () -> rabbit_global_counters:messages_dead_lettered(rejected, rabbit_classic_queue,
disabled, length(AckTags)),
ack(AckTags, ChPid, State) end));
handle_cast({delete_exclusive, ConnPid}, State) ->
log_delete_exclusive(ConnPid, State),

View File

@ -2165,22 +2165,18 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ex
confirm = Confirm,
msg_seq_no = MsgSeqNo},
_RoutedToQueueNames = [QName]}, State0 = #ch{queue_states = QueueStates0}) -> %% optimisation when there is one queue
AllNames = case rabbit_amqqueue:lookup(QName) of
{ok, Q0} ->
case amqqueue:get_options(Q0) of
#{extra_bcc := BCC} -> [QName, rabbit_misc:r(QName#resource.virtual_host, queue, BCC)];
_ -> [QName]
end;
_ -> []
end,
Qs = rabbit_amqqueue:lookup(AllNames),
{QueueNames, Qs} = case rabbit_amqqueue:lookup(QName) of
{ok, Q} ->
{[QName], [Q]};
_ -> {[], []}
end,
case rabbit_queue_type:deliver(Qs, Delivery, QueueStates0) of
{ok, QueueStates, Actions} ->
rabbit_global_counters:messages_routed(amqp091, erlang:min(1, length(Qs))),
%% NB: the order here is important since basic.returns must be
%% sent before confirms.
ok = process_routing_mandatory(Mandatory, Qs, Message, State0),
State1 = process_routing_confirm(Confirm, AllNames, MsgSeqNo, XName, State0),
State1 = process_routing_confirm(Confirm, QueueNames, MsgSeqNo, XName, State0),
%% Actions must be processed after registering confirms as actions may
%% contain rejections of publishes
State = handle_queue_actions(Actions, State1#ch{queue_states = QueueStates}),
@ -2208,21 +2204,15 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ex
confirm = Confirm,
msg_seq_no = MsgSeqNo},
RoutedToQueueNames}, State0 = #ch{queue_states = QueueStates0}) ->
Qs0 = rabbit_amqqueue:lookup(RoutedToQueueNames),
AllQueueNames = lists:map(fun amqqueue:get_name/1, Qs0),
AllExtraBCCs = infer_extra_bcc(Qs0),
%% Collect implicit BCC targets these queues may have
Qs = case AllExtraBCCs of
[] -> Qs0;
ExtraNames -> Qs0 ++ rabbit_amqqueue:lookup(ExtraNames)
end,
Qs = rabbit_amqqueue:lookup(RoutedToQueueNames),
QueueNames = lists:map(fun amqqueue:get_name/1, Qs),
case rabbit_queue_type:deliver(Qs, Delivery, QueueStates0) of
{ok, QueueStates, Actions} ->
rabbit_global_counters:messages_routed(amqp091, length(Qs)),
%% NB: the order here is important since basic.returns must be
%% sent before confirms.
ok = process_routing_mandatory(Mandatory, Qs, Message, State0),
State1 = process_routing_confirm(Confirm, AllQueueNames,
State1 = process_routing_confirm(Confirm, QueueNames,
MsgSeqNo, XName, State0),
%% Actions must be processed after registering confirms as actions may
%% contain rejections of publishes
@ -2231,7 +2221,7 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ex
fine ->
?INCR_STATS(exchange_stats, XName, 1, publish),
[?INCR_STATS(queue_exchange_stats, {QName, XName}, 1, publish)
|| QName <- AllQueueNames];
|| QName <- QueueNames];
_ ->
ok
end,
@ -2243,28 +2233,6 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ex
[rabbit_misc:rs(Resource)])
end.
-spec infer_extra_bcc([amqqueue:amqqueue()]) -> [rabbit_amqqueue:name()].
infer_extra_bcc([]) ->
[];
infer_extra_bcc([Q]) ->
case amqqueue:get_options(Q) of
#{extra_bcc := BCC} ->
#resource{virtual_host = VHost} = amqqueue:get_name(Q),
[rabbit_misc:r(VHost, queue, BCC)];
_ ->
[]
end;
infer_extra_bcc(Qs) ->
lists:foldl(fun(Q, Acc) ->
case amqqueue:get_options(Q) of
#{extra_bcc := BCC} ->
#resource{virtual_host = VHost} = amqqueue:get_name(Q),
[rabbit_misc:r(VHost, queue, BCC) | Acc];
_ ->
Acc
end
end, [], Qs).
process_routing_mandatory(_Mandatory = true,
_RoutedToQs = [],
Msg, State) ->

View File

@ -445,8 +445,10 @@ recover_durable_queues(QueuesAndRecoveryTerms) ->
capabilities() ->
#{unsupported_policies => [ %% Stream policies
<<"max-age">>, <<"stream-max-segment-size-bytes">>,
<<"queue-leader-locator">>, <<"initial-cluster-size">>],
<<"max-age">>, <<"stream-max-segment-size-bytes">>,
<<"queue-leader-locator">>, <<"initial-cluster-size">>,
%% Quorum policies
<<"dead-letter-strategy">>],
queue_arguments => [<<"x-expires">>, <<"x-message-ttl">>, <<"x-dead-letter-exchange">>,
<<"x-dead-letter-routing-key">>, <<"x-max-length">>,
<<"x-max-length-bytes">>, <<"x-max-in-memory-length">>,

View File

@ -7,7 +7,9 @@
-module(rabbit_dead_letter).
-export([publish/5]).
-export([publish/5,
make_msg/5,
detect_cycles/3]).
-include_lib("rabbit_common/include/rabbit.hrl").
-include_lib("rabbit_common/include/rabbit_framing.hrl").
@ -15,6 +17,7 @@
%%----------------------------------------------------------------------------
-type reason() :: 'expired' | 'rejected' | 'maxlen' | delivery_limit.
-export_type([reason/0]).
%%----------------------------------------------------------------------------
@ -39,7 +42,7 @@ make_msg(Msg = #basic_message{content = Content,
undefined -> {RoutingKeys, fun (H) -> H end};
_ -> {[RK], fun (H) -> lists:keydelete(<<"CC">>, 1, H) end}
end,
ReasonBin = list_to_binary(atom_to_list(Reason)),
ReasonBin = atom_to_binary(Reason),
TimeSec = os:system_time(seconds),
PerMsgTTL = per_msg_ttl_header(Content#content.properties),
HeadersFun2 =

View File

@ -246,11 +246,8 @@ get_disk_free(Dir, {win32, _}) ->
rabbit_log:warning("Expected the mnesia directory absolute "
"path to start with a drive letter like "
"'C:'. The path is: '~p'", [Dir]),
case win32_get_disk_free_dir(Dir) of
{ok, Free} ->
Free;
_ -> exit(could_not_determine_disk_free)
end;
{ok, Free} = win32_get_disk_free_dir(Dir),
Free;
DriveLetter ->
case catch win32_get_disk_free_pwsh(DriveLetter) of
{ok, Free1} -> Free1;

View File

@ -408,21 +408,46 @@ info_all(VHostPath, Items, Ref, AggregatorPid) ->
route(#exchange{name = #resource{virtual_host = VHost, name = RName} = XName,
decorators = Decorators} = X,
#delivery{message = #basic_message{routing_keys = RKs}} = Delivery) ->
case RName of
<<>> ->
RKsSorted = lists:usort(RKs),
[rabbit_channel:deliver_reply(RK, Delivery) ||
RK <- RKsSorted, virtual_reply_queue(RK)],
[rabbit_misc:r(VHost, queue, RK) || RK <- RKsSorted,
not virtual_reply_queue(RK)];
_ ->
Decs = rabbit_exchange_decorator:select(route, Decorators),
lists:usort(route1(Delivery, Decs, {[X], XName, []}))
end.
QNames = case RName of
<<>> ->
RKsSorted = lists:usort(RKs),
[rabbit_channel:deliver_reply(RK, Delivery) ||
RK <- RKsSorted, virtual_reply_queue(RK)],
[rabbit_misc:r(VHost, queue, RK) || RK <- RKsSorted,
not virtual_reply_queue(RK)];
_ ->
Decs = rabbit_exchange_decorator:select(route, Decorators),
lists:usort(route1(Delivery, Decs, {[X], XName, []}))
end,
Qs = rabbit_amqqueue:lookup(QNames),
ExtraBccQNames = infer_extra_bcc(Qs),
ExtraBccQNames ++ QNames.
virtual_reply_queue(<<"amq.rabbitmq.reply-to.", _/binary>>) -> true;
virtual_reply_queue(_) -> false.
-spec infer_extra_bcc([amqqueue:amqqueue()]) -> [rabbit_amqqueue:name()].
infer_extra_bcc([]) ->
[];
infer_extra_bcc([Q]) ->
case amqqueue:get_options(Q) of
#{extra_bcc := BCC} ->
#resource{virtual_host = VHost} = amqqueue:get_name(Q),
[rabbit_misc:r(VHost, queue, BCC)];
_ ->
[]
end;
infer_extra_bcc(Qs) ->
lists:foldl(fun(Q, Acc) ->
case amqqueue:get_options(Q) of
#{extra_bcc := BCC} ->
#resource{virtual_host = VHost} = amqqueue:get_name(Q),
[rabbit_misc:r(VHost, queue, BCC) | Acc];
_ ->
Acc
end
end, [], Qs).
route1(_, _, {[], _, QNames}) ->
QNames;
route1(Delivery, Decorators,

File diff suppressed because it is too large Load Diff

View File

@ -1,13 +1,29 @@
%% 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-2021 VMware, Inc. or its affiliates. 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 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_in_id() :: non_neg_integer().
% a queue scoped monotonically incrementing integer used to enforce order
% in the unassigned messages map
-type msg_id() :: non_neg_integer().
%% A consumer-scoped monotonically incrementing integer included with a
%% {@link delivery/0.}. Used to settle deliveries using
@ -19,25 +35,26 @@
%% same process
-type msg_header() :: msg_size() |
tuple(msg_size(), Expiry :: milliseconds()) |
#{size := msg_size(),
delivery_count => non_neg_integer()}.
delivery_count => non_neg_integer(),
expiry => milliseconds()}.
%% The message header:
%% size: The size of the message payload in bytes.
%% delivery_count: the number of unsuccessful delivery attempts.
%% A non-zero value indicates a previous attempt.
%% If it only contains the size it can be condensed to an integer only
-type msg() :: {msg_header(), raw_msg()}.
%% message with a header map.
%% 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 indexed_msg() :: {ra:index(), msg()}.
-type msg() :: tuple(option(ra:index()), msg_header()).
-type prefix_msg() :: {'$prefix_msg', msg_header()}.
-type delivery_msg() :: {msg_id(), msg()}.
%% A tuple consisting of the message id and the headered message.
-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
@ -63,30 +80,23 @@
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, 3200000).
-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, 2000000).
-define(GC_MEM_LIMIT_B, 2_000_000).
-define(MB, 1048576).
-define(MB, 1_048_576).
-define(LOW_LIMIT, 0.8).
-record(consumer,
-record(consumer_cfg,
{meta = #{} :: consumer_meta(),
checked_out = #{} :: #{msg_id() => {msg_in_id(), indexed_msg()}},
next_msg_id = 0 :: msg_id(), % part of snapshot data
%% 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(),
pid :: pid(),
tag :: consumer_tag(),
%% the mode of how credit is incremented
%% simple_prefetch: credit is re-filled as deliveries are settled
%% or returned.
@ -94,8 +104,19 @@
%% command: `{consumer_credit, ReceiverDeliveryCount, Credit}'
credit_mode = simple_prefetch :: credit_mode(), % part of snapshot data
lifetime = once :: once | auto,
status = up :: up | suspected_down | cancelled,
priority = 0 :: non_neg_integer()
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{}.
@ -104,16 +125,17 @@
-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
pending = [] :: [{msg_seqno(), ra:index(), raw_msg()}],
status = up :: up |
suspected_down,
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 :: undefined | ra:index(),
blocked :: option(ra:index()),
unused_1,
unused_2
}).
@ -122,7 +144,7 @@
{name :: atom(),
resource :: rabbit_types:r('queue'),
release_cursor_interval :: option({non_neg_integer(), non_neg_integer()}),
dead_letter_handler :: option(applied_mfa()),
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()),
@ -131,9 +153,8 @@
consumer_strategy = competing :: consumer_strategy(),
%% the maximum number of unsuccessful delivery attempts permitted
delivery_limit :: option(non_neg_integer()),
max_in_memory_length :: option(non_neg_integer()),
max_in_memory_bytes :: option(non_neg_integer()),
expires :: undefined | milliseconds(),
expires :: option(milliseconds()),
msg_ttl :: option(milliseconds()),
unused_1,
unused_2
}).
@ -145,57 +166,46 @@
-record(rabbit_fifo,
{cfg :: #cfg{},
% unassigned messages
messages = lqueue:new() :: lqueue:lqueue({msg_in_id(), indexed_msg()}),
% defines the next message id
next_msg_num = 1 :: msg_in_id(),
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(prefix_msg() |
{msg_in_id(), indexed_msg()}),
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 as well as a cached value of the smallest
% ra_index of all pending enqueues
% a message to this queue
enqueuers = #{} :: #{pid() => #enqueuer{}},
% master index of all enqueue raft indexes including pending
% enqueues
% 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
% needs to be part of snapshot
consumers = #{} :: #{consumer_id() => #consumer{}},
consumers = #{} :: #{consumer_id() => consumer()},
% consumers that require further service are queued here
% needs to be part of snapshot
service_queue = priority_queue:new() :: priority_queue:q(),
%% This is a special field that is only used for snapshots
%% It represents the queued messages at the time the
%% dehydrated snapshot state was cached.
%% As release_cursors are only emitted for raft indexes where all
%% prior messages no longer contribute to the current state we can
%% replace all message payloads with their sizes (to be used for
%% overflow calculations).
%% This is done so that consumers are still served in a deterministic
%% order on recovery.
prefix_msgs = {0, [], 0, []} :: prefix_msgs(),
%% 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()}],
msg_bytes_in_memory = 0 :: non_neg_integer(),
msgs_ready_in_memory = 0 :: non_neg_integer(),
last_active :: undefined | non_neg_integer(),
unused_1,
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 => applied_mfa(),
dead_letter_handler => dead_letter_handler(),
become_leader_handler => applied_mfa(),
release_cursor_interval => non_neg_integer(),
max_length => non_neg_integer(),
@ -206,5 +216,6 @@
single_active_consumer_on => boolean(),
delivery_limit => non_neg_integer(),
expires => non_neg_integer(),
msg_ttl => non_neg_integer(),
created => non_neg_integer()
}.

View File

@ -42,7 +42,6 @@
-define(COMMAND_TIMEOUT, 30000).
-type seq() :: non_neg_integer().
-type maybe_seq() :: integer().
-type action() :: {send_credit_reply, Available :: non_neg_integer()} |
{send_drained, CTagCredit ::
{rabbit_fifo:consumer_tag(), non_neg_integer()}}.
@ -66,10 +65,6 @@
leader :: undefined | ra:server_id(),
queue_status :: undefined | go | reject_publish,
next_seq = 0 :: seq(),
%% Last applied is initialise to -1 to note that no command has yet been
%% applied, but allowing to resend messages if the first ones on the sequence
%% are lost (messages are sent from last_applied + 1)
last_applied = -1 :: maybe_seq(),
next_enqueue_seq = 1 :: seq(),
%% indicates that we've exceeded the soft limit
slow = false :: boolean(),
@ -151,7 +146,7 @@ enqueue(Correlation, Msg,
0 ->
%% the leader is running the old version
enqueue(Correlation, Msg, State0#state{queue_status = go});
1 ->
N when is_integer(N) ->
%% were running the new version on the leader do sync initialisation
%% of enqueuer session
Reg = rabbit_fifo:make_register_enqueuer(self()),
@ -183,18 +178,30 @@ enqueue(_Correlation, _Msg,
{reject_publish, State};
enqueue(Correlation, Msg,
#state{slow = Slow,
pending = Pending,
queue_status = go,
cfg = #cfg{block_handler = BlockFun}} = State0) ->
Node = pick_server(State0),
{Next, State1} = next_enqueue_seq(State0),
next_seq = Seq,
next_enqueue_seq = EnqueueSeq,
cfg = #cfg{soft_limit = SftLmt,
block_handler = BlockFun}} = State0) ->
Server = pick_server(State0),
% by default there is no correlation id
Cmd = rabbit_fifo:make_enqueue(self(), Next, Msg),
case send_command(Node, Correlation, Cmd, low, State1) of
{slow, State} when not Slow ->
Cmd = rabbit_fifo:make_enqueue(self(), EnqueueSeq, Msg),
ok = ra:pipeline_command(Server, Cmd, Seq, low),
Tag = case map_size(Pending) >= SftLmt of
true -> slow;
false -> ok
end,
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 ->
BlockFun(),
{slow, set_timer(State)};
Any ->
Any
_ ->
{ok, State}
end.
%% @doc Enqueues a message.
@ -277,11 +284,7 @@ add_delivery_count_header(Msg, _Count) ->
settle(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) ->
Node = pick_server(State0),
Cmd = rabbit_fifo:make_settle(consumer_id(ConsumerTag), MsgIds),
case send_command(Node, undefined, Cmd, normal, State0) of
{_, S} ->
% turn slow into ok for this function
{S, []}
end;
{send_command(Node, undefined, Cmd, normal, State0), []};
settle(ConsumerTag, [_|_] = MsgIds,
#state{unsent_commands = Unsent0} = State0) ->
ConsumerId = consumer_id(ConsumerTag),
@ -309,8 +312,7 @@ return(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) ->
Node = pick_server(State0),
% TODO: make rabbit_fifo return support lists of message ids
Cmd = rabbit_fifo:make_return(consumer_id(ConsumerTag), MsgIds),
{_Tag, State1} = send_command(Node, undefined, Cmd, normal, State0),
{State1, []};
{send_command(Node, undefined, Cmd, normal, State0), []};
return(ConsumerTag, [_|_] = MsgIds,
#state{unsent_commands = Unsent0} = State0) ->
ConsumerId = consumer_id(ConsumerTag),
@ -338,11 +340,7 @@ return(ConsumerTag, [_|_] = MsgIds,
discard(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) ->
Node = pick_server(State0),
Cmd = rabbit_fifo:make_discard(consumer_id(ConsumerTag), MsgIds),
case send_command(Node, undefined, Cmd, normal, State0) of
{_, S} ->
% turn slow into ok for this function
{S, []}
end;
{send_command(Node, undefined, Cmd, normal, State0), []};
discard(ConsumerTag, [_|_] = MsgIds,
#state{unsent_commands = Unsent0} = State0) ->
ConsumerId = consumer_id(ConsumerTag),
@ -429,11 +427,7 @@ credit(ConsumerTag, Credit, Drain,
Node = pick_server(State0),
Cmd = rabbit_fifo:make_credit(ConsumerId, Credit,
C#consumer.last_msg_id + 1, Drain),
case send_command(Node, undefined, Cmd, normal, State0) of
{_, S} ->
% turn slow into ok for this function
{S, []}
end.
{send_command(Node, undefined, Cmd, normal, State0), []}.
%% @doc Cancels a checkout with the rabbit_fifo queue for the consumer tag
%%
@ -531,7 +525,7 @@ update_machine_state(Server, Conf) ->
%% `{internal, AppliedCorrelations, State}' if the event contained an internally
%% handled event such as a notification and a correlation was included with
%% the command (e.g. in a call to `enqueue/3' the correlation terms are returned
%% here.
%% here).
%%
%% `{RaFifoEvent, State}' if the event contained a client message generated by
%% the `rabbit_fifo' state machine such as a delivery.
@ -584,11 +578,8 @@ handle_ra_event(From, {applied, Seqs},
Node = pick_server(State2),
%% send all the settlements and returns
State = lists:foldl(fun (C, S0) ->
case send_command(Node, undefined,
C, normal, S0) of
{T, S} when T =/= error ->
S
end
send_command(Node, undefined, C,
normal, S0)
end, State2, Commands),
UnblockFun(),
{ok, State, Actions};
@ -602,21 +593,25 @@ handle_ra_event(_, {machine, {queue_status, Status}},
%% just set the queue status
{ok, State#state{queue_status = Status}, []};
handle_ra_event(Leader, {machine, leader_change},
#state{leader = Leader} = State) ->
%% leader already known
{ok, State, []};
handle_ra_event(Leader, {machine, leader_change}, State0) ->
#state{leader = OldLeader} = State0) ->
%% we need to update leader
%% and resend any pending commands
rabbit_log:debug("~s: Detected QQ leader change from ~w to ~w",
[?MODULE, OldLeader, Leader]),
State = resend_all_pending(State0#state{leader = Leader}),
{ok, State, []};
handle_ra_event(_From, {rejected, {not_leader, Leader, _Seq}},
#state{leader = Leader} = State) ->
{ok, State, []};
handle_ra_event(_From, {rejected, {not_leader, Leader, _Seq}},
#state{leader = OldLeader} = State0) ->
rabbit_log:debug("~s: Detected QQ leader change (rejection) from ~w to ~w",
[?MODULE, OldLeader, Leader]),
State = resend_all_pending(State0#state{leader = Leader}),
{ok, cancel_timer(State), []};
handle_ra_event(_From, {rejected, {not_leader, undefined, _Seq}}, State0) ->
handle_ra_event(_From, {rejected, {not_leader, _UndefinedMaybe, _Seq}}, State0) ->
% TODO: how should these be handled? re-sent on timer or try random
{ok, State0, []};
handle_ra_event(_From, {rejected, {not_leader, Leader, Seq}}, State0) ->
State1 = State0#state{leader = Leader},
State = resend(Seq, State1),
{ok, State, []};
handle_ra_event(_, timeout, #state{cfg = #cfg{servers = Servers}} = State0) ->
case find_leader(Servers) of
undefined ->
@ -663,29 +658,27 @@ try_process_command([Server | Rem], Cmd,
try_process_command(Rem, Cmd, State)
end.
seq_applied({Seq, MaybeAction},
{Corrs, Actions0, #state{last_applied = Last} = State0})
when Seq > Last ->
State1 = do_resends(Last+1, Seq-1, State0),
{Actions, State} = maybe_add_action(MaybeAction, Actions0, State1),
seq_applied({Seq, Response},
{Corrs, Actions0, #state{} = State0}) ->
%% sequences aren't guaranteed to be applied in order as enqueues are
%% low priority commands and may be overtaken by others with a normal priority.
{Actions, State} = maybe_add_action(Response, Actions0, State0),
case maps:take(Seq, State#state.pending) of
{{undefined, _}, Pending} ->
{Corrs, Actions, State#state{pending = Pending,
last_applied = Seq}};
{{Corr, _}, Pending} ->
{[Corr | Corrs], Actions, State#state{pending = Pending,
last_applied = Seq}};
error ->
% must have already been resent or removed for some other reason
% still need to update last_applied or we may inadvertently resend
% stuff later
{Corrs, Actions, State#state{last_applied = Seq}}
{Corrs, Actions, State#state{pending = Pending}};
{{Corr, _}, Pending}
when Response /= not_enqueued ->
{[Corr | Corrs], Actions, State#state{pending = Pending}};
_ ->
{Corrs, Actions, State#state{}}
end;
seq_applied(_Seq, Acc) ->
Acc.
maybe_add_action(ok, Acc, State) ->
{Acc, State};
maybe_add_action(not_enqueued, Acc, State) ->
{Acc, State};
maybe_add_action({multi, Actions}, Acc0, State0) ->
lists:foldl(fun (Act, {Acc, State}) ->
maybe_add_action(Act, Acc, State)
@ -702,11 +695,6 @@ maybe_add_action(Action, Acc, State) ->
%% anything else is assumed to be an action
{[Action | Acc], State}.
do_resends(From, To, State) when From =< To ->
lists:foldl(fun resend/2, State, lists:seq(From, To));
do_resends(_, _, State) ->
State.
% resends a command with a new sequence number
resend(OldSeq, #state{pending = Pending0, leader = Leader} = State) ->
case maps:take(OldSeq, Pending0) of
@ -850,32 +838,42 @@ sorted_servers(#state{leader = Leader,
cfg = #cfg{servers = Servers}}) ->
[Leader | lists:delete(Leader, Servers)].
next_seq(#state{next_seq = Seq} = State) ->
{Seq, State#state{next_seq = Seq + 1}}.
next_enqueue_seq(#state{next_enqueue_seq = Seq} = State) ->
{Seq, State#state{next_enqueue_seq = Seq + 1}}.
consumer_id(ConsumerTag) ->
{ConsumerTag, self()}.
send_command(Server, Correlation, Command, Priority,
send_command(Server, Correlation, Command, _Priority,
#state{pending = Pending,
cfg = #cfg{soft_limit = SftLmt}} = State0) ->
{Seq, State} = next_seq(State0),
ok = ra:pipeline_command(Server, Command, Seq, Priority),
Tag = case maps:size(Pending) >= SftLmt of
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,
{Tag, State#state{pending = Pending#{Seq => {Correlation, Command}},
slow = Tag == slow}}.
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,
State#state{pending = Pending#{Seq => {Correlation, Command}},
next_seq = Seq + 1,
slow = Tag == slow}.
resend_command(Node, Correlation, Command,
#state{pending = Pending} = State0) ->
{Seq, State} = next_seq(State0),
#state{pending = Pending,
next_seq = Seq} = State) ->
ok = ra:pipeline_command(Node, Command, Seq),
State#state{pending = Pending#{Seq => {Correlation, Command}}}.
State#state{pending = Pending#{Seq => {Correlation, Command}},
next_seq = Seq + 1}.
add_command(_, _, [], Acc) ->
Acc;

361
deps/rabbit/src/rabbit_fifo_dlx.erl vendored Normal file
View File

@ -0,0 +1,361 @@
%% 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-2021 VMware, Inc. or its affiliates. All rights reserved.
-module(rabbit_fifo_dlx).
-include("rabbit_fifo_dlx.hrl").
-include("rabbit_fifo.hrl").
-compile({no_auto_import, [apply/3]}).
-export([
%% rabbit_fifo_dlx_client
make_checkout/2,
make_settle/1,
%% rabbit_fifo delegating DLX handling to this module
init/0,
apply/4,
discard/4,
overview/1,
checkout/2,
state_enter/4,
handle_aux/6,
dehydrate/1,
normalize/1,
stat/1,
update_config/4,
smallest_raft_index/1
]).
-record(checkout, {consumer :: pid(),
prefetch :: non_neg_integer()}).
-record(settle, {msg_ids :: [msg_id()]}).
-type protocol() :: {dlx, #checkout{} | #settle{}}.
-opaque state() :: #?MODULE{}.
-export_type([state/0,
protocol/0]).
-spec init() -> state().
init() ->
#?MODULE{}.
-spec make_checkout(pid(), non_neg_integer()) -> protocol().
make_checkout(Pid, NumUnsettled) ->
{dlx, #checkout{consumer = Pid,
prefetch = NumUnsettled
}}.
-spec make_settle([msg_id()]) -> protocol().
make_settle(MessageIds) when is_list(MessageIds) ->
{dlx, #settle{msg_ids = MessageIds}}.
-spec overview(state()) -> map().
overview(#?MODULE{consumer = undefined,
msg_bytes = MsgBytes,
msg_bytes_checkout = 0,
discards = Discards}) ->
overview0(Discards, #{}, MsgBytes, 0);
overview(#?MODULE{consumer = #dlx_consumer{checked_out = Checked},
msg_bytes = MsgBytes,
msg_bytes_checkout = MsgBytesCheckout,
discards = Discards}) ->
overview0(Discards, Checked, MsgBytes, MsgBytesCheckout).
overview0(Discards, Checked, MsgBytes, MsgBytesCheckout) ->
#{num_discarded => lqueue:len(Discards),
num_discard_checked_out => maps:size(Checked),
discard_message_bytes => MsgBytes,
discard_checkout_message_bytes => MsgBytesCheckout}.
-spec stat(state()) ->
{Num :: non_neg_integer(), Bytes :: non_neg_integer()}.
stat(#?MODULE{consumer = Con,
discards = Discards,
msg_bytes = MsgBytes,
msg_bytes_checkout = MsgBytesCheckout}) ->
Num0 = lqueue:len(Discards),
Num = case Con of
undefined ->
Num0;
#dlx_consumer{checked_out = Checked} ->
%% O(1) because Erlang maps maintain their own size
Num0 + maps:size(Checked)
end,
Bytes = MsgBytes + MsgBytesCheckout,
{Num, Bytes}.
-spec apply(ra_machine:command_meta_data(), protocol(), dead_letter_handler(), state()) ->
{state(), ra_machine:effects()}.
apply(_Meta, {dlx, #settle{msg_ids = MsgIds}}, at_least_once,
#?MODULE{consumer = #dlx_consumer{checked_out = Checked0}} = State0) ->
Acked = maps:with(MsgIds, Checked0),
State = maps:fold(
fun(MsgId, ?TUPLE(_Rsn, ?MSG(Idx, _) = Msg),
#?MODULE{consumer = #dlx_consumer{checked_out = Checked} = C,
msg_bytes_checkout = BytesCheckout,
ra_indexes = Indexes0} = S) ->
Indexes = rabbit_fifo_index:delete(Idx, Indexes0),
S#?MODULE{consumer = C#dlx_consumer{checked_out =
maps:remove(MsgId, Checked)},
msg_bytes_checkout = BytesCheckout - size_in_bytes(Msg),
ra_indexes = Indexes}
end, State0, Acked),
{State, [{mod_call, rabbit_global_counters, messages_dead_lettered_confirmed,
[rabbit_quorum_queue, at_least_once, maps:size(Acked)]}]};
apply(_, {dlx, #checkout{consumer = Pid,
prefetch = Prefetch}},
at_least_once,
#?MODULE{consumer = undefined} = State0) ->
State = State0#?MODULE{consumer = #dlx_consumer{pid = Pid,
prefetch = Prefetch}},
{State, []};
apply(_, {dlx, #checkout{consumer = ConsumerPid,
prefetch = Prefetch}},
at_least_once,
#?MODULE{consumer = #dlx_consumer{checked_out = CheckedOutOldConsumer},
discards = Discards0,
msg_bytes = Bytes,
msg_bytes_checkout = BytesCheckout} = State0) ->
%% Since we allow only a single consumer, the new consumer replaces the old consumer.
%% All checked out messages to the old consumer need to be returned to the discards queue
%% such that these messages will be re-delivered to the new consumer.
%% When inserting back into the discards queue, we respect the original order in which messages
%% were discarded.
Checked0 = maps:to_list(CheckedOutOldConsumer),
Checked1 = lists:keysort(1, Checked0),
{Discards, BytesMoved} = lists:foldr(
fun({_Id, ?TUPLE(_, Msg) = RsnMsg}, {D, B}) ->
{lqueue:in_r(RsnMsg, D), B + size_in_bytes(Msg)}
end, {Discards0, 0}, Checked1),
State = State0#?MODULE{consumer = #dlx_consumer{pid = ConsumerPid,
prefetch = Prefetch},
discards = Discards,
msg_bytes = Bytes + BytesMoved,
msg_bytes_checkout = BytesCheckout - BytesMoved},
{State, []};
apply(_, Cmd, DLH, State) ->
rabbit_log:debug("Ignoring command ~p for dead_letter_handler ~p", [Cmd, DLH]),
{State, []}.
-spec discard([msg()], rabbit_dead_letter:reason(), dead_letter_handler(), state()) ->
{state(), ra_machine:effects()}.
discard(Msgs, Reason, undefined, State) ->
{State, [{mod_call, rabbit_global_counters, messages_dead_lettered,
[Reason, rabbit_quorum_queue, disabled, length(Msgs)]}]};
discard(Msgs0, Reason, {at_most_once, {Mod, Fun, Args}}, State) ->
Idxs = [I || ?MSG(I, _) <- Msgs0],
Effect = {log, Idxs,
fun (Log) ->
Lookup = maps:from_list(lists:zip(Idxs, Log)),
Msgs = [begin
{enqueue, _, _, Msg} = maps:get(Idx, Lookup),
Msg
end || ?MSG(Idx, _) <- 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,
#?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),
D = lqueue:in(?TUPLE(Reason, Msg), D0),
B = B0 + MsgSize,
I = rabbit_fifo_index:append(Idx, I0),
S0#?MODULE{discards = D,
msg_bytes = B,
ra_indexes = I}
end, State0, Msgs),
{State, [{mod_call, rabbit_global_counters, messages_dead_lettered,
[Reason, rabbit_quorum_queue, at_least_once, length(Msgs)]}]}.
-spec checkout(dead_letter_handler(), state()) ->
{state(), ra_machine:effects()}.
checkout(at_least_once, #?MODULE{consumer = #dlx_consumer{}} = State) ->
checkout0(checkout_one(State), []);
checkout(_, State) ->
{State, []}.
checkout0({success, MsgId, ?TUPLE(Reason, ?MSG(Idx, _)), State}, SendAcc) ->
DelMsg = {Idx, {Reason, MsgId}},
checkout0(checkout_one(State), [DelMsg | SendAcc]);
checkout0(#?MODULE{consumer = #dlx_consumer{pid = Pid}} = State, SendAcc) ->
Effects = delivery_effects(Pid, SendAcc),
{State, Effects}.
checkout_one(#?MODULE{consumer = #dlx_consumer{checked_out = Checked,
prefetch = Prefetch}} = State)
when map_size(Checked) >= Prefetch ->
State;
checkout_one(#?MODULE{discards = Discards0,
msg_bytes = Bytes,
msg_bytes_checkout = BytesCheckout,
consumer = #dlx_consumer{checked_out = Checked0,
next_msg_id = Next} = Con0} = State0) ->
case lqueue:out(Discards0) of
{{value, ?TUPLE(_, Msg) = ReasonMsg}, Discards} ->
Checked = maps:put(Next, ReasonMsg, Checked0),
Size = size_in_bytes(Msg),
State = State0#?MODULE{discards = Discards,
msg_bytes = Bytes - Size,
msg_bytes_checkout = BytesCheckout + Size,
consumer = Con0#dlx_consumer{checked_out = Checked,
next_msg_id = Next + 1}},
{success, Next, ReasonMsg, State};
{empty, _} ->
State0
end.
size_in_bytes(MSG) ->
Header = rabbit_fifo:get_msg_header(MSG),
rabbit_fifo:get_header(size, Header).
delivery_effects(_CPid, []) ->
[];
delivery_effects(CPid, Msgs0) ->
Msgs1 = lists:reverse(Msgs0),
{RaftIdxs, RsnIds} = lists:unzip(Msgs1),
[{log, RaftIdxs,
fun(Log) ->
Msgs = lists:zipwith(fun ({enqueue, _, _, Msg}, {Reason, MsgId}) ->
{MsgId, {Reason, Msg}}
end, Log, RsnIds),
[{send_msg, CPid, {dlx_delivery, Msgs}, [ra_event]}]
end}].
-spec state_enter(ra_server:ra_state() | eol, rabbit_types:r('queue'), dead_letter_handler(), state()) ->
ra_machine:effects().
state_enter(leader, QRes, at_least_once, State) ->
ensure_worker_started(QRes, State),
[];
state_enter(_, _, at_least_once, State) ->
ensure_worker_terminated(State),
[];
state_enter(_, _, _, _) ->
[].
ensure_worker_started(QRef, #?MODULE{consumer = undefined}) ->
start_worker(QRef);
ensure_worker_started(QRef, #?MODULE{consumer = #dlx_consumer{pid = Pid}}) ->
case is_local_and_alive(Pid) of
true ->
rabbit_log:debug("rabbit_fifo_dlx_worker ~p already started for ~s",
[Pid, rabbit_misc:rs(QRef)]);
false ->
start_worker(QRef)
end.
%% Ensure that starting the rabbit_fifo_dlx_worker succeeds.
%% Therefore, do not use an effect.
%% Also therefore, if starting the rabbit_fifo_dlx_worker fails, let the
%% Ra server process crash in which case another Ra node will become leader.
start_worker(QRef) ->
{ok, Pid} = supervisor:start_child(rabbit_fifo_dlx_sup, [QRef]),
rabbit_log:debug("started rabbit_fifo_dlx_worker ~p for ~s",
[Pid, rabbit_misc:rs(QRef)]).
ensure_worker_terminated(#?MODULE{consumer = undefined}) ->
ok;
ensure_worker_terminated(#?MODULE{consumer = #dlx_consumer{pid = Pid}}) ->
case is_local_and_alive(Pid) of
true ->
%% Note that we can't return a mod_call effect here
%% because mod_call is executed on the leader only.
ok = supervisor:terminate_child(rabbit_fifo_dlx_sup, Pid),
rabbit_log:debug("terminated rabbit_fifo_dlx_worker ~p", [Pid]);
false ->
ok
end.
local_alive_consumer_pid(#?MODULE{consumer = undefined}) ->
undefined;
local_alive_consumer_pid(#?MODULE{consumer = #dlx_consumer{pid = Pid}}) ->
case is_local_and_alive(Pid) of
true ->
Pid;
false ->
undefined
end.
is_local_and_alive(Pid)
when node(Pid) =:= node() ->
is_process_alive(Pid);
is_local_and_alive(_) ->
false.
-spec update_config(Old :: dead_letter_handler(), New :: dead_letter_handler(),
rabbit_types:r('queue'), state()) ->
{state(), ra_machine:effects()}.
update_config(at_least_once, at_least_once, _, State) ->
case local_alive_consumer_pid(State) of
undefined ->
{State, []};
Pid ->
%% Notify rabbit_fifo_dlx_worker about potentially updated policies.
{State, [{send_msg, Pid, lookup_topology, ra_event}]}
end;
update_config(SameDLH, SameDLH, _, State) ->
{State, []};
update_config(OldDLH, NewDLH, QRes, State0) ->
LogOnLeader = {mod_call, rabbit_log, debug,
["Switching dead_letter_handler from ~p to ~p for ~s",
[OldDLH, NewDLH, rabbit_misc:rs(QRes)]]},
{State1, Effects0} = switch_from(OldDLH, QRes, State0),
{State, Effects} = switch_to(NewDLH, State1, Effects0),
{State, [LogOnLeader|Effects]}.
-spec switch_from(Old :: dead_letter_handler(), rabbit_types:r('queue'), state()) ->
{state(), ra_machine:effects()}.
switch_from(at_least_once, QRes, State) ->
%% Switch from at-least-once to some other strategy.
ensure_worker_terminated(State),
{Num, Bytes} = stat(State),
%% Log only on leader.
{init(), [{mod_call, rabbit_log, info,
["Deleted ~b dead-lettered messages (with total messages size of ~b bytes) in ~s",
[Num, Bytes, rabbit_misc:rs(QRes)]]}]};
switch_from(_, _, State) ->
{State, []}.
-spec switch_to(New :: dead_letter_handler(), state(), ra_machine:effects()) ->
{state(), ra_machine:effects()}.
switch_to(at_least_once, _, Effects) ->
%% Switch from some other strategy to at-least-once.
%% Dlx worker needs to be started on the leader.
%% The cleanest way to determine the Ra state of this node is delegation to handle_aux.
{init(), [{aux, {dlx, setup}} | Effects]};
switch_to(_, State, Effects) ->
{State, Effects}.
-spec handle_aux(ra_server:ra_state(), Cmd :: term(), Aux :: term(),
rabbit_types:r('queue'), dead_letter_handler(), state()) ->
term().
handle_aux(leader, {dlx, setup}, Aux, QRes, at_least_once, State) ->
ensure_worker_started(QRes, State),
Aux;
handle_aux(_, _, Aux, _, _, _) ->
Aux.
-spec dehydrate(state()) ->
state().
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).

21
deps/rabbit/src/rabbit_fifo_dlx.hrl vendored Normal file
View File

@ -0,0 +1,21 @@
%% 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-2021 VMware, Inc. or its affiliates. All rights reserved.
-record(dlx_consumer,
{pid :: pid(),
prefetch :: non_neg_integer(),
checked_out = #{} :: #{msg_id() => tuple(rabbit_dead_letter:reason(), msg())},
next_msg_id = 0 :: msg_id()}).
-record(rabbit_fifo_dlx,
{consumer :: option(#dlx_consumer{}),
%% Queue of dead-lettered messages.
discards = lqueue:new() :: lqueue:lqueue(tuple(rabbit_dead_letter:reason(), msg())),
%% Raft indexes of messages in both discards queue and dlx_consumer's checked_out map
%% so that we get the smallest ra index in O(1).
ra_indexes = rabbit_fifo_index:empty() :: rabbit_fifo_index:state(),
msg_bytes = 0 :: non_neg_integer(),
msg_bytes_checkout = 0 :: non_neg_integer()}).

View File

@ -0,0 +1,91 @@
%% 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-2021 VMware, Inc. or its affiliates. All rights reserved.
-module(rabbit_fifo_dlx_client).
-export([checkout/3, settle/2, handle_ra_event/3,
overview/1]).
-record(state,{
queue_resource :: rabbit_types:r(queue),
leader :: ra:server_id(),
last_msg_id :: non_neg_integer() | -1
}).
-type state() :: #state{}.
-type action() :: {deliver, [{rabbit_amqqueue:name(),
MsgId :: non_neg_integer(),
rabbit_types:message(),
rabbit_dead_letter:reason()}]}.
-type actions() :: [action()].
-export_type([state/0,
actions/0]).
-spec settle([non_neg_integer()], state()) ->
{ok, state()}.
settle(MsgIds, #state{leader = Leader} = State)
when is_list(MsgIds) ->
Cmd = rabbit_fifo_dlx:make_settle(MsgIds),
ra:pipeline_command(Leader, Cmd),
{ok, State}.
-spec checkout(rabbit_amqqueue:name(), ra:server_id(), non_neg_integer()) ->
{ok, state()} | {error, ra_command_failed}.
checkout(QResource, Leader, NumUnsettled) ->
Cmd = rabbit_fifo_dlx:make_checkout(self(), NumUnsettled),
State = #state{queue_resource = QResource,
leader = Leader,
last_msg_id = -1},
process_command(Cmd, State, 5).
process_command(_Cmd, _State, 0) ->
{error, ra_command_failed};
process_command(Cmd, #state{leader = Leader} = State, Tries) ->
case ra:process_command(Leader, Cmd, 60_000) of
{ok, ok, Leader} ->
{ok, State#state{leader = Leader}};
{ok, ok, L} ->
rabbit_log:warning("Failed to process command ~p on quorum queue leader ~p because actual leader is ~p.",
[Cmd, Leader, L]),
{error, ra_command_failed};
Err ->
rabbit_log:warning("Failed to process command ~p on quorum queue leader ~p: ~p~n"
"Trying ~b more time(s)...",
[Cmd, Leader, Err, Tries]),
process_command(Cmd, State, Tries - 1)
end.
-spec handle_ra_event(ra:server_id(), term(), state()) ->
{ok, state(), actions()}.
handle_ra_event(Leader, {machine, {dlx_delivery, _} = Del}, #state{leader = Leader} = State) ->
handle_delivery(Del, State);
handle_ra_event(From, Evt, State) ->
rabbit_log:debug("Ignoring ra event ~p from ~p", [Evt, From]),
{ok, State, []}.
handle_delivery({dlx_delivery, [{FstId, _} | _] = IdMsgs},
#state{queue_resource = QRes,
last_msg_id = Prev} = State0) ->
%% Assert that messages get delivered in order since deliveries are node local.
%% (In contrast to rabbit_fifo_client, we expect neither duplicate nor missing messages.)
%% Let it crash if this assertion is wrong.
FstId = Prev + 1,
%% Format as a deliver action.
Del = {deliver, transform_msgs(QRes, IdMsgs)},
{LastId, _} = lists:last(IdMsgs),
State = State0#state{last_msg_id = LastId},
{ok, State, [Del]}.
transform_msgs(QRes, Msgs) ->
lists:map(
fun({MsgId, {Reason, Msg}}) ->
{QRes, MsgId, Msg, Reason}
end, Msgs).
-spec overview(state()) -> map().
overview(#state{leader = Leader,
last_msg_id = LastMsgId}) ->
#{leader => Leader,
last_msg_id => LastMsgId}.

34
deps/rabbit/src/rabbit_fifo_dlx_sup.erl vendored Normal file
View File

@ -0,0 +1,34 @@
%% 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) 2018-2021 VMware, Inc. or its affiliates. All rights reserved.
-module(rabbit_fifo_dlx_sup).
-behaviour(supervisor).
-rabbit_boot_step({?MODULE,
[{description, "supervisor of quorum queue dead-letter workers"},
{mfa, {rabbit_sup, start_supervisor_child, [?MODULE]}},
{requires, kernel_ready},
{enables, core_initialized}]}).
%% supervisor callback
-export([init/1]).
%% client API
-export([start_link/0]).
start_link() ->
supervisor:start_link({local, ?MODULE}, ?MODULE, []).
init([]) ->
SupFlags = #{strategy => simple_one_for_one,
intensity => 1,
period => 5},
Worker = rabbit_fifo_dlx_worker,
ChildSpec = #{id => Worker,
start => {Worker, start_link, []},
type => worker,
modules => [Worker]},
{ok, {SupFlags, [ChildSpec]}}.

View File

@ -0,0 +1,642 @@
%% 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-2021 VMware, Inc. or its affiliates. All rights reserved.
%% One rabbit_fifo_dlx_worker process exists per (source) quorum queue that has at-least-once dead lettering
%% enabled. The rabbit_fifo_dlx_worker process is co-located on the quorum queue leader node.
%% Its job is to consume from the quorum queue's 'discards' queue (containing dead lettered messages)
%% and to forward each dead lettered message at least once to every target queue.
%% This is in contrast to at-most-once semantics of rabbit_dead_letter:publish/5 which is
%% the only option for classic queues and was the only option for quorum queues in RMQ <= v3.9
%%
%% Some parts of this module resemble the channel process in the sense that it needs to keep track what messages
%% are consumed but not acked yet and what messages are published but not confirmed yet.
%% Compared to the channel process, this module is protocol independent since it does not deal with AMQP clients.
%%
%% This module consumes directly from the rabbit_fifo_dlx_client bypassing the rabbit_queue_type interface,
%% but publishes via the rabbit_queue_type interface.
%% While consuming via rabbit_queue_type interface would have worked in practice (by using a special consumer argument,
%% e.g. {<<"x-internal-queue">>, longstr, <<"discards">>}) using the rabbit_fifo_dlx_client directly provides
%% separation of concerns making things easier to test, to debug, and to understand.
-module(rabbit_fifo_dlx_worker).
-include_lib("rabbit_common/include/rabbit.hrl").
-include_lib("rabbit_common/include/rabbit_framing.hrl").
-behaviour(gen_server).
-export([start_link/1]).
%% gen_server callbacks
-export([init/1, terminate/2, handle_continue/2,
handle_cast/2, handle_call/3, handle_info/2,
code_change/3, format_status/2]).
-define(HIBERNATE_AFTER, 4*60*1000).
-record(pending, {
%% consumed_msg_id is not to be confused with consumer delivery tag.
%% The latter represents a means for AMQP clients to (multi-)ack to a channel process.
%% However, queues are not aware of delivery tags.
%% This rabbit_fifo_dlx_worker does not have the concept of delivery tags because it settles (acks)
%% message IDs directly back to the queue (and there is no AMQP consumer).
consumed_msg_id :: non_neg_integer(),
delivery :: rabbit_types:delivery(),
reason :: rabbit_dead_letter:reason(),
%% target queues for which publisher confirm has not been received yet
unsettled = [] :: [rabbit_amqqueue:name()],
%% target queues for which publisher rejection was received recently
rejected = [] :: [rabbit_amqqueue:name()],
%% target queues for which publisher confirm was received
settled = [] :: [rabbit_amqqueue:name()],
%% Number of times the message was published (i.e. rabbit_queue_type:deliver/3 invoked).
%% Can be 0 if the message was never published (for example no route exists).
publish_count = 0 :: non_neg_integer(),
%% Epoch time in milliseconds when the message was last published (i.e. rabbit_queue_type:deliver/3 invoked).
%% It can be 'undefined' if the message was never published (for example no route exists).
last_published_at :: undefined | integer(),
%% Epoch time in milliseconds when the message was consumed from the source quorum queue.
%% This value never changes.
%% It's mainly informational and meant for debugging to understand for how long the message
%% is sitting around without having received all publisher confirms.
consumed_at :: integer()
}).
-record(state, {
%% source queue
queue_ref :: rabbit_amqqueue:name(),
%% monitors source queue
monitor_ref :: reference(),
%% configured (x-)dead-letter-exchange of source queue
exchange_ref :: rabbit_exchange:name() | undefined,
%% configured (x-)dead-letter-routing-key of source queue
routing_key,
%% client of source queue
dlx_client_state :: rabbit_fifo_dlx_client:state(),
%% clients of target queues
queue_type_state :: rabbit_queue_type:state(),
%% Consumed messages for which we are awaiting publisher confirms.
pendings = #{} :: #{OutSeq :: non_neg_integer() => #pending{}},
%% Consumed message IDs for which we received all publisher confirms.
settled_ids = [] :: [non_neg_integer()],
%% next outgoing message sequence number
next_out_seq = 1,
%% If no publisher confirm was received for at least settle_timeout milliseconds, message will be redelivered.
%% To prevent duplicates in the target queue and to ensure message will eventually be acked to the source queue,
%% set this value higher than the maximum time it takes for a queue to settle a message.
settle_timeout :: non_neg_integer(),
%% Timer firing every settle_timeout milliseconds
%% redelivering messages for which not all publisher confirms were received.
%% If there are no pending messages, this timer will eventually be cancelled to allow
%% this worker to hibernate.
timer :: undefined | reference(),
logged = #{} :: map()
}).
-type state() :: #state{}.
start_link(QRef) ->
gen_server:start_link(?MODULE, QRef, [{hibernate_after, ?HIBERNATE_AFTER}]).
-spec init(rabbit_amqqueue:name()) ->
{ok, undefined, {continue, rabbit_amqqueue:name()}}.
init(QRef) ->
{ok, undefined, {continue, QRef}}.
-spec handle_continue(rabbit_amqqueue:name(), undefined) ->
{noreply, state()}.
handle_continue(QRef, undefined) ->
{ok, Prefetch} = application:get_env(rabbit,
dead_letter_worker_consumer_prefetch),
{ok, SettleTimeout} = application:get_env(rabbit,
dead_letter_worker_publisher_confirm_timeout),
{ok, Q} = rabbit_amqqueue:lookup(QRef),
{ClusterName, _MaybeOldLeaderNode} = amqqueue:get_pid(Q),
{ok, ConsumerState} = rabbit_fifo_dlx_client:checkout(QRef,
{ClusterName, node()},
Prefetch),
{noreply, lookup_topology(#state{queue_ref = QRef,
queue_type_state = rabbit_queue_type:init(),
settle_timeout = SettleTimeout,
dlx_client_state = ConsumerState,
monitor_ref = erlang:monitor(process, ClusterName)
})}.
terminate(_Reason, State) ->
cancel_timer(State).
handle_call(Request, From, State) ->
rabbit_log:info("~s received unhandled call from ~p: ~p", [?MODULE, From, Request]),
{noreply, State}.
handle_cast({queue_event, QRef, {_From, {machine, lookup_topology}}},
#state{queue_ref = QRef} = State0) ->
State = lookup_topology(State0),
redeliver_and_ack(State);
handle_cast({queue_event, QRef, {From, Evt}},
#state{queue_ref = QRef,
dlx_client_state = DlxState0} = State0) ->
%% received dead-letter message from source queue
{ok, DlxState, Actions} = rabbit_fifo_dlx_client:handle_ra_event(From, Evt, DlxState0),
State1 = State0#state{dlx_client_state = DlxState},
State = handle_queue_actions(Actions, State1),
{noreply, State};
handle_cast({queue_event, QRef, Evt},
#state{queue_type_state = QTypeState0} = State0) ->
case rabbit_queue_type:handle_event(QRef, Evt, QTypeState0) of
{ok, QTypeState1, Actions} ->
%% received e.g. confirm from target queue
State1 = State0#state{queue_type_state = QTypeState1},
State = handle_queue_actions(Actions, State1),
{noreply, State};
eol ->
remove_queue(QRef, State0);
{protocol_error, _Type, _Reason, _Args} ->
{noreply, State0}
end;
handle_cast(settle_timeout, State0) ->
State = State0#state{timer = undefined},
redeliver_and_ack(State);
handle_cast(Request, State) ->
rabbit_log:info("~s received unhandled cast ~p", [?MODULE, Request]),
{noreply, State}.
redeliver_and_ack(State0) ->
State1 = redeliver_messages(State0),
State2 = ack(State1),
State = maybe_set_timer(State2),
{noreply, State}.
handle_info({'DOWN', Ref, process, _, _},
#state{monitor_ref = Ref,
queue_ref = QRef}) ->
%% Source quorum queue is down. Therefore, terminate ourself.
%% The new leader will re-create another dlx_worker.
rabbit_log:debug("~s terminating itself because leader of ~s is down...",
[?MODULE, rabbit_misc:rs(QRef)]),
supervisor:terminate_child(rabbit_fifo_dlx_sup, self());
handle_info({'DOWN', _MRef, process, QPid, Reason},
#state{queue_type_state = QTypeState0} = State0) ->
%% received from target classic queue
case rabbit_queue_type:handle_down(QPid, Reason, QTypeState0) of
{ok, QTypeState, Actions} ->
State = State0#state{queue_type_state = QTypeState},
{noreply, handle_queue_actions(Actions, State)};
{eol, QTypeState, QRef} ->
remove_queue(QRef, State0#state{queue_type_state = QTypeState})
end;
handle_info(Info, State) ->
rabbit_log:info("~s received unhandled info ~p", [?MODULE, Info]),
{noreply, State}.
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
remove_queue(QRef, #state{pendings = Pendings0,
queue_type_state = QTypeState0} = State) ->
Pendings = maps:map(fun(_Seq, #pending{unsettled = Unsettled} = Pending) ->
Pending#pending{unsettled = lists:delete(QRef, Unsettled)}
end, Pendings0),
QTypeState = rabbit_queue_type:remove(QRef, QTypeState0),
%% Wait for max 1s (we don't want to block the gen_server process for a longer time)
%% until target queue is deleted from ETS table to prevent subsequently consumed message(s)
%% from being routed to a deleted target queue. (If that happens for a target quorum or
%% stream queue and that queue gets re-created with the same name, these messages will
%% be stuck in our 'pendings' state.)
wait_for_queue_deleted(QRef, 20),
{noreply, State#state{pendings = Pendings,
queue_type_state = QTypeState}}.
wait_for_queue_deleted(QRef, 0) ->
rabbit_log:debug("Received deletion event for ~s but queue still exists in ETS table.",
[rabbit_misc:rs(QRef)]);
wait_for_queue_deleted(QRef, N) ->
case rabbit_amqqueue:lookup(QRef) of
{error, not_found} ->
ok;
_ ->
timer:sleep(50),
wait_for_queue_deleted(QRef, N-1)
end.
-spec lookup_topology(state()) -> state().
lookup_topology(#state{queue_ref = {resource, Vhost, queue, _} = QRef} = State) ->
{ok, Q} = rabbit_amqqueue:lookup(QRef),
DLRKey = rabbit_queue_type_util:args_policy_lookup(<<"dead-letter-routing-key">>,
fun(_Pol, QArg) -> QArg end, Q),
DLX = rabbit_queue_type_util:args_policy_lookup(<<"dead-letter-exchange">>,
fun(_Pol, QArg) -> QArg end, Q),
DLXRef = rabbit_misc:r(Vhost, exchange, DLX),
State#state{exchange_ref = DLXRef,
routing_key = DLRKey}.
-spec handle_queue_actions(rabbit_queue_type:actions() | rabbit_fifo_dlx_client:actions(), state()) ->
state().
handle_queue_actions(Actions, State0) ->
lists:foldl(
fun ({deliver, Msgs}, S0) ->
S1 = handle_deliver(Msgs, S0),
maybe_set_timer(S1);
({settled, QRef, MsgSeqs}, S0) ->
S1 = handle_settled(QRef, MsgSeqs, S0),
S2 = ack(S1),
maybe_cancel_timer(S2);
({rejected, QRef, MsgSeqs}, S0) ->
handle_rejected(QRef, MsgSeqs, S0);
({queue_down, _QRef}, S0) ->
%% target classic queue is down, but not deleted
S0
end, State0, Actions).
handle_deliver(Msgs, #state{queue_ref = QRef} = State0)
when is_list(Msgs) ->
{DLX, State} = lookup_dlx(State0),
lists:foldl(fun({_QRef, MsgId, Msg, Reason}, S) ->
forward(Msg, MsgId, QRef, DLX, Reason, S)
end, State, Msgs).
handle_rejected(QRef, MsgSeqNos, #state{pendings = Pendings0} = State)
when is_list(MsgSeqNos) ->
Pendings = lists:foldl(fun(SeqNo, P) ->
rejected(SeqNo, [QRef], P)
end, Pendings0, MsgSeqNos),
State#state{pendings = Pendings}.
rejected(SeqNo, Qs, Pendings)
when is_list(Qs) ->
case maps:is_key(SeqNo, Pendings) of
true ->
maps:update_with(SeqNo,
fun(#pending{unsettled = Unsettled,
rejected = Rejected} = P) ->
P#pending{unsettled = Unsettled -- Qs,
rejected = Qs ++ Rejected}
end,
Pendings);
false ->
rabbit_log:debug("Ignoring rejection for unknown sequence number ~b "
"from target dead letter queues ~p",
[SeqNo, Qs]),
Pendings
end.
-spec lookup_dlx(state()) ->
{rabbit_types:exchange() | not_found, state()}.
lookup_dlx(#state{exchange_ref = DLXRef} = State0) ->
case rabbit_exchange:lookup(DLXRef) of
{error, not_found} ->
State = log_missing_dlx_once(State0),
{not_found, State};
{ok, X} ->
{X, State0}
end.
-spec forward(rabbit_types:message(), non_neg_integer(), rabbit_amqqueue:name(),
rabbit_types:exchange() | not_found, rabbit_dead_letter:reason(), state()) ->
state().
forward(ConsumedMsg, ConsumedMsgId, ConsumedQRef, DLX, Reason,
#state{next_out_seq = OutSeq,
pendings = Pendings,
exchange_ref = DLXRef,
routing_key = RKey} = State0) ->
#basic_message{routing_keys = RKeys} = Msg = rabbit_dead_letter:make_msg(ConsumedMsg, Reason,
DLXRef, RKey, ConsumedQRef),
%% Field 'mandatory' is set to false because we check ourselves whether the message is routable.
Delivery = rabbit_basic:delivery(_Mandatory = false, _Confirm = true, Msg, OutSeq),
{TargetQs, State3} = case DLX of
not_found ->
{[], State0};
_ ->
RouteToQs0 = rabbit_exchange:route(DLX, Delivery),
{RouteToQs1, Cycles} = rabbit_dead_letter:detect_cycles(Reason, Msg, RouteToQs0),
State1 = log_cycles(Cycles, RKeys, State0),
RouteToQs = rabbit_amqqueue:lookup(RouteToQs1),
State2 = case RouteToQs of
[] ->
log_no_route_once(State1);
_ ->
State1
end,
{RouteToQs, State2}
end,
Now = os:system_time(millisecond),
Pend0 = #pending{
consumed_msg_id = ConsumedMsgId,
consumed_at = Now,
delivery = Delivery,
reason = Reason
},
case TargetQs of
[] ->
%% We can't deliver this message since there is no target queue we can route to.
%% We buffer this message and retry to send every settle_timeout milliseonds.
State3#state{next_out_seq = OutSeq + 1,
pendings = maps:put(OutSeq, Pend0, Pendings)};
_ ->
Pend = Pend0#pending{publish_count = 1,
last_published_at = Now,
unsettled = queue_names(TargetQs)},
State = State3#state{next_out_seq = OutSeq + 1,
pendings = maps:put(OutSeq, Pend, Pendings)},
deliver_to_queues(Delivery, TargetQs, State)
end.
-spec deliver_to_queues(rabbit_types:delivery(), [amqqueue:amqqueue()], state()) ->
state().
deliver_to_queues(#delivery{msg_seq_no = SeqNo} = Delivery, Qs, #state{queue_type_state = QTypeState0,
pendings = Pendings} = State0) ->
{State, Actions} = case rabbit_queue_type:deliver(Qs, Delivery, QTypeState0) of
{ok, QTypeState, Actions0} ->
{State0#state{queue_type_state = QTypeState}, Actions0};
{error, Reason} ->
%% rabbit_queue_type:deliver/3 does not tell us which target queue failed.
%% Therefore, reject all target queues. We need to reject them such that
%% we won't rely on rabbit_fifo_client to re-deliver on behalf of us
%% (and therefore preventing messages to get stuck in our 'unsettled' state).
QNames = queue_names(Qs),
rabbit_log:debug("Failed to deliver message with seq_no ~b to queues ~p: ~p",
[SeqNo, QNames, Reason]),
{State0#state{pendings = rejected(SeqNo, QNames, Pendings)}, []}
end,
handle_queue_actions(Actions, State).
handle_settled(QRef, MsgSeqs, State) ->
lists:foldl(fun (MsgSeq, S) ->
handle_settled0(QRef, MsgSeq, S)
end, State, MsgSeqs).
handle_settled0(QRef, MsgSeq, #state{pendings = Pendings,
settled_ids = SettledIds} = State) ->
case maps:find(MsgSeq, Pendings) of
{ok, #pending{unsettled = [QRef],
rejected = [],
consumed_msg_id = ConsumedId}} ->
State#state{pendings = maps:remove(MsgSeq, Pendings),
settled_ids = [ConsumedId | SettledIds]};
{ok, #pending{unsettled = [],
rejected = [QRef],
consumed_msg_id = ConsumedId}} ->
State#state{pendings = maps:remove(MsgSeq, Pendings),
settled_ids = [ConsumedId | SettledIds]};
{ok, #pending{unsettled = Unsettled,
rejected = Rejected,
settled = Settled} = Pend0} ->
Pend = Pend0#pending{unsettled = lists:delete(QRef, Unsettled),
rejected = lists:delete(QRef, Rejected),
settled = [QRef | Settled]},
State#state{pendings = maps:update(MsgSeq, Pend, Pendings)};
error ->
rabbit_log:debug("Ignoring publisher confirm for unknown sequence number ~b "
"from target dead letter ~s",
[MsgSeq, rabbit_misc:rs(QRef)]),
State
end.
ack(#state{settled_ids = []} = State) ->
State;
ack(#state{settled_ids = Ids,
dlx_client_state = DlxState0} = State) ->
{ok, DlxState} = rabbit_fifo_dlx_client:settle(Ids, DlxState0),
State#state{settled_ids = [],
dlx_client_state = DlxState}.
%% Re-deliver messages that
%% 1. timed out waiting on publisher confirm, or
%% 2. got rejected by target queue, or
%% 3. never got sent due to routing topology misconfiguration.
-spec redeliver_messages(state()) ->
state().
redeliver_messages(#state{pendings = Pendings,
settle_timeout = SettleTimeout} = State0) ->
case lookup_dlx(State0) of
{not_found, State} ->
%% Configured dead-letter-exchange does (still) not exist.
%% Keep the same Pendings in our state until user creates or re-configures the dead-letter-exchange.
State;
{DLX, State} ->
Now = os:system_time(millisecond),
maps:fold(fun(OutSeq, #pending{last_published_at = LastPub} = Pend, S0)
when LastPub + SettleTimeout =< Now ->
%% Publisher confirm timed out.
redeliver(Pend, DLX, OutSeq, S0);
(OutSeq, #pending{last_published_at = undefined} = Pend, S0) ->
%% Message was never published due to dead-letter routing topology misconfiguration.
redeliver(Pend, DLX, OutSeq, S0);
(_OutSeq, _Pending, S) ->
%% Publisher confirm did not time out.
S
end, State, Pendings)
end.
redeliver(#pending{delivery = #delivery{message = #basic_message{content = Content}}} = Pend,
DLX, OutSeq, #state{routing_key = undefined} = State) ->
%% No dead-letter-routing-key defined for source quorum queue.
%% Therefore use all of messages's original routing keys (which can include CC and BCC recipients).
%% This complies with the behaviour of the rabbit_dead_letter module.
%% We stored these original routing keys in the 1st (i.e. most recent) x-death entry.
#content{properties = #'P_basic'{headers = Headers}} =
rabbit_binary_parser:ensure_content_decoded(Content),
{array, [{table, MostRecentDeath}|_]} = rabbit_misc:table_lookup(Headers, <<"x-death">>),
{<<"routing-keys">>, array, Routes0} = lists:keyfind(<<"routing-keys">>, 1, MostRecentDeath),
Routes = [Route || {longstr, Route} <- Routes0],
redeliver0(Pend, DLX, Routes, OutSeq, State);
redeliver(Pend, DLX, OutSeq, #state{routing_key = DLRKey} = State) ->
redeliver0(Pend, DLX, [DLRKey], OutSeq, State).
redeliver0(#pending{delivery = #delivery{message = BasicMsg} = Delivery0,
unsettled = Unsettled0,
settled = Settled,
publish_count = PublishCount,
reason = Reason,
consumed_msg_id = ConsumedId} = Pend0,
DLX, DLRKeys, OutSeq,
#state{pendings = Pendings,
settled_ids = SettledIds,
exchange_ref = DLXRef,
queue_type_state = QTypeState} = State0)
when is_list(DLRKeys) ->
Delivery = Delivery0#delivery{message = BasicMsg#basic_message{exchange_name = DLXRef,
routing_keys = DLRKeys}},
RouteToQs0 = rabbit_exchange:route(DLX, Delivery),
%% rabbit_exchange:route/2 can route to target queues that do not exist (e.g. in case of default exchange).
%% Therefore, filter out non-existent target queues.
RouteToQs1 = queue_names(rabbit_amqqueue:lookup(RouteToQs0)),
case {RouteToQs1, Settled} of
{[], [_|_]} ->
%% Routes changed dynamically so that we don't await any publisher confirms anymore.
%% Since we also received at least once publisher confirm (mandatory flag semantics),
%% we can ack the messasge to the source quorum queue.
State0#state{pendings = maps:remove(OutSeq, Pendings),
settled_ids = [ConsumedId | SettledIds]};
_ ->
%% Do not redeliver message to a target queue
%% 1. for which we already received a publisher confirm, or
Unsettled = RouteToQs1 -- Settled,
%% 2. whose queue client redelivers on our behalf.
%% Note that a quorum queue client does not redeliver on our behalf if it previously
%% rejected the message. This is why we always redeliver rejected messages here.
RouteToQs2 = Unsettled -- clients_redeliver(Unsettled0, QTypeState),
{RouteToQs, Cycles} = rabbit_dead_letter:detect_cycles(Reason, BasicMsg, RouteToQs2),
State1 = log_cycles(Cycles, DLRKeys, State0),
case RouteToQs of
[] ->
State1;
_ ->
Pend = Pend0#pending{publish_count = PublishCount + 1,
last_published_at = os:system_time(millisecond),
delivery = Delivery,
%% Override 'unsettled' because topology could have changed.
unsettled = Unsettled,
%% Any target queue that rejected previously and still need
%% to be routed to is moved back to 'unsettled'.
rejected = []},
State = State0#state{pendings = maps:update(OutSeq, Pend, Pendings)},
deliver_to_queues(Delivery, rabbit_amqqueue:lookup(RouteToQs), State)
end
end.
%% Returns queues whose queue clients take care of redelivering messages.
-spec clients_redeliver([rabbit_amqqueue:name()], rabbit_queue_type:state()) ->
[rabbit_amqqueue:name()].
clients_redeliver(Qs, QTypeState) ->
lists:filter(fun(Q) ->
case rabbit_queue_type:module(Q, QTypeState) of
{ok, rabbit_quorum_queue} ->
% If #enqueue{} Raft command does not get applied
% rabbit_fifo_client will resend.
true;
{ok, rabbit_stream_queue} ->
true;
_ ->
false
end
end, Qs).
maybe_set_timer(#state{timer = TRef} = State)
when is_reference(TRef) ->
State;
maybe_set_timer(#state{timer = undefined,
pendings = Pendings} = State)
when map_size(Pendings) =:= 0 ->
State;
maybe_set_timer(#state{timer = undefined,
settle_timeout = SettleTimeout} = State) ->
TRef = erlang:send_after(SettleTimeout, self(), {'$gen_cast', settle_timeout}),
State#state{timer = TRef}.
maybe_cancel_timer(#state{timer = TRef,
pendings = Pendings} = State)
when is_reference(TRef),
map_size(Pendings) =:= 0 ->
erlang:cancel_timer(TRef, [{async, true}, {info, false}]),
State#state{timer = undefined};
maybe_cancel_timer(State) ->
State.
cancel_timer(#state{timer = TRef} = State)
when is_reference(TRef) ->
erlang:cancel_timer(TRef, [{async, true}, {info, false}]),
State#state{timer = undefined};
cancel_timer(State) ->
State.
queue_names(Qs)
when is_list(Qs) ->
lists:map(fun amqqueue:get_name/1, Qs).
format_status(_Opt, [_PDict, #state{
queue_ref = QueueRef,
exchange_ref = ExchangeRef,
routing_key = RoutingKey,
dlx_client_state = DlxClientState,
queue_type_state = QueueTypeState,
pendings = Pendings,
settled_ids = SettledIds,
next_out_seq = NextOutSeq,
settle_timeout = SettleTimeout,
timer = Timer,
logged = Logged
}]) ->
S = #{queue_ref => QueueRef,
exchange_ref => ExchangeRef,
routing_key => RoutingKey,
dlx_client_state => rabbit_fifo_dlx_client:overview(DlxClientState),
queue_type_state => QueueTypeState,
pendings => maps:map(fun(_, P) -> format_pending(P) end, Pendings),
settled_ids => SettledIds,
next_out_seq => NextOutSeq,
settle_timeout => SettleTimeout,
timer_is_active => Timer =/= undefined,
logged => Logged},
[{data, [{"State", S}]}].
format_pending(#pending{consumed_msg_id = ConsumedMsgId,
delivery = _DoNotLogLargeBinary,
reason = Reason,
unsettled = Unsettled,
rejected = Rejected,
settled = Settled,
publish_count = PublishCount,
last_published_at = LastPublishedAt,
consumed_at = ConsumedAt}) ->
#{consumed_msg_id => ConsumedMsgId,
reason => Reason,
unsettled => Unsettled,
rejected => Rejected,
settled => Settled,
publish_count => PublishCount,
last_published_at => LastPublishedAt,
consumed_at => ConsumedAt}.
log_missing_dlx_once(#state{exchange_ref = SameDlx,
logged = #{missing_dlx := SameDlx}} = State) ->
State;
log_missing_dlx_once(#state{exchange_ref = DlxResource,
queue_ref = QueueResource,
logged = Logged} = State) ->
rabbit_log:warning("Cannot forward any dead-letter messages from source quorum ~s because "
"its configured dead-letter-exchange ~s does not exist. "
"Either create the configured dead-letter-exchange or re-configure "
"the dead-letter-exchange policy for the source quorum queue to prevent "
"dead-lettered messages from piling up in the source quorum queue. "
"This message will not be logged again.",
[rabbit_misc:rs(QueueResource), rabbit_misc:rs(DlxResource)]),
State#state{logged = maps:put(missing_dlx, DlxResource, Logged)}.
log_no_route_once(#state{exchange_ref = SameDlx,
routing_key = SameRoutingKey,
logged = #{no_route := {SameDlx, SameRoutingKey}}} = State) ->
State;
log_no_route_once(#state{queue_ref = QueueResource,
exchange_ref = DlxResource,
routing_key = RoutingKey,
logged = Logged} = State) ->
rabbit_log:warning("Cannot forward any dead-letter messages from source quorum ~s "
"with configured dead-letter-exchange ~s and configured "
"dead-letter-routing-key '~s'. This can happen either if the dead-letter "
"routing topology is misconfigured (for example no queue bound to "
"dead-letter-exchange or wrong dead-letter-routing-key configured) or if "
"non-mirrored classic queues are bound whose host node is down. "
"Fix this issue to prevent dead-lettered messages from piling up "
"in the source quorum queue. "
"This message will not be logged again.",
[rabbit_misc:rs(QueueResource), rabbit_misc:rs(DlxResource), RoutingKey]),
State#state{logged = maps:put(no_route, {DlxResource, RoutingKey}, Logged)}.
log_cycles(Cycles, RoutingKeys, State) ->
lists:foldl(fun(Cycle, S) -> log_cycle_once(Cycle, RoutingKeys, S) end, State, Cycles).
log_cycle_once(Queues, _, #state{logged = Logged} = State)
when is_map_key({cycle, Queues}, Logged) ->
State;
log_cycle_once(Queues, RoutingKeys, #state{exchange_ref = DlxResource,
queue_ref = QueueResource,
logged = Logged} = State) ->
rabbit_log:warning("Dead-letter queues cycle detected for source quorum ~s "
"with dead-letter exchange ~s and routing keys ~p: ~p "
"This message will not be logged again.",
[rabbit_misc:rs(QueueResource), rabbit_misc:rs(DlxResource),
RoutingKeys, Queues]),
State#state{logged = maps:put({cycle, Queues}, true, Logged)}.

View File

@ -7,14 +7,15 @@
delete/2,
size/1,
smallest/1,
map/2
map/2,
normalize/1
]).
-compile({no_auto_import, [size/1]}).
%% the empty atom is a lot smaller (4 bytes) than e.g. `undefined` (13 bytes).
%% This matters as the data map gets persisted as part of the snapshot
-define(NIL, '').
-define(NIL, []).
-record(?MODULE, {data = #{} :: #{integer() => ?NIL},
smallest :: undefined | non_neg_integer(),
@ -40,15 +41,29 @@ append(Key,
#?MODULE{data = Data,
smallest = Smallest,
largest = Largest} = State)
when Key > Largest orelse Largest =:= undefined ->
when is_integer(Key) andalso
(Key > Largest orelse
Largest =:= undefined) ->
State#?MODULE{data = maps:put(Key, ?NIL, Data),
smallest = ra_lib:default(Smallest, Key),
largest = Key}.
largest = Key};
append(Key,
#?MODULE{data = Data,
largest = Largest,
smallest = Smallest} = State)
when is_integer(Key) ->
State#?MODULE{data = maps:put(Key, ?NIL, Data),
smallest = min(Key, ra_lib:default(Smallest, Key)),
largest = max(Key, ra_lib:default(Largest, Key))
};
append(undefined, State) ->
State.
-spec delete(Index :: integer(), state()) -> state().
-spec delete(Index :: integer() | undefined, state()) -> state().
delete(Smallest, #?MODULE{data = Data0,
largest = Largest,
smallest = Smallest} = State) ->
smallest = Smallest} = State)
when is_integer(Smallest) ->
Data = maps:remove(Smallest, Data0),
case find_next(Smallest + 1, Largest, Data) of
undefined ->
@ -65,11 +80,10 @@ delete(Key, #?MODULE{data = Data} = State) ->
size(#?MODULE{data = Data}) ->
maps:size(Data).
-spec smallest(state()) -> undefined | integer().
-spec smallest(state()) -> undefined | non_neg_integer().
smallest(#?MODULE{smallest = Smallest}) ->
Smallest.
-spec map(fun(), state()) -> state().
map(F, #?MODULE{data = Data} = State) ->
State#?MODULE{data = maps:map(F, Data)}.
@ -91,6 +105,10 @@ 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").

2248
deps/rabbit/src/rabbit_fifo_v1.erl vendored Normal file

File diff suppressed because it is too large Load Diff

211
deps/rabbit/src/rabbit_fifo_v1.hrl vendored Normal file
View File

@ -0,0 +1,211 @@
-type option(T) :: undefined | T.
-type raw_msg() :: term().
%% The raw message. It is opaque to rabbit_fifo.
-type msg_in_id() :: non_neg_integer().
% a queue scoped monotonically incrementing integer used to enforce order
% in the unassigned messages map
-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() |
#{size := msg_size(),
delivery_count => non_neg_integer()}.
%% The message header:
%% delivery_count: the number of unsuccessful delivery attempts.
%% A non-zero value indicates a previous attempt.
%% If it only contains the size it can be condensed to an integer only
-type msg() :: {msg_header(), raw_msg()}.
%% message with a header map.
-type msg_size() :: non_neg_integer().
%% the size in bytes of the msg payload
-type indexed_msg() :: {ra:index(), msg()}.
-type prefix_msg() :: {'$prefix_msg', msg_header()}.
-type delivery_msg() :: {msg_id(), 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() :: simple_prefetch | credited.
%% 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, 3200000).
-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, 2000000).
-define(MB, 1048576).
-define(LOW_LIMIT, 0.8).
-define(STATE, rabbit_fifo).
-record(consumer,
{meta = #{} :: consumer_meta(),
checked_out = #{} :: #{msg_id() => {msg_in_id(), indexed_msg()}},
next_msg_id = 0 :: msg_id(), % part of snapshot data
%% 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(),
%% 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 = simple_prefetch :: credit_mode(), % part of snapshot data
lifetime = once :: once | auto,
status = up :: up | suspected_down | cancelled,
priority = 0 :: non_neg_integer()
}).
-type consumer() :: #consumer{}.
-type consumer_strategy() :: competing | single_active.
-type milliseconds() :: non_neg_integer().
-record(enqueuer,
{next_seqno = 1 :: msg_seqno(),
% out of order enqueues - sorted list
pending = [] :: [{msg_seqno(), ra:index(), raw_msg()}],
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 :: undefined | 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 :: option(applied_mfa()),
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()),
max_in_memory_length :: option(non_neg_integer()),
max_in_memory_bytes :: option(non_neg_integer()),
expires :: undefined | milliseconds(),
unused_1,
unused_2
}).
-type prefix_msgs() :: {list(), list()} |
{non_neg_integer(), list(),
non_neg_integer(), list()}.
-record(?STATE,
{cfg :: #cfg{},
% unassigned messages
messages = lqueue:new() :: lqueue:lqueue({msg_in_id(), indexed_msg()}),
% defines the next message id
next_msg_num = 1 :: msg_in_id(),
% queue of returned msg_in_ids - when checking out it picks from
returns = lqueue:new() :: lqueue:lqueue(prefix_msg() |
{msg_in_id(), indexed_msg()}),
% a counter of enqueues - used to trigger shadow copy points
enqueue_count = 0 :: non_neg_integer(),
% a map containing all the live processes that have ever enqueued
% a message to this queue as well as a cached value of the smallest
% ra_index of all pending enqueues
enqueuers = #{} :: #{pid() => #enqueuer{}},
% master index of all enqueue raft indexes including pending
% enqueues
% 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(),
release_cursors = lqueue:new() :: lqueue:lqueue({release_cursor,
ra:index(), #?STATE{}}),
% consumers need to reflect consumer state at time of snapshot
% needs to be part of snapshot
consumers = #{} :: #{consumer_id() => #consumer{}},
% consumers that require further service are queued here
% needs to be part of snapshot
service_queue = priority_queue:new() :: priority_queue:q(),
%% This is a special field that is only used for snapshots
%% It represents the queued messages at the time the
%% dehydrated snapshot state was cached.
%% As release_cursors are only emitted for raft indexes where all
%% prior messages no longer contribute to the current state we can
%% replace all message payloads with their sizes (to be used for
%% overflow calculations).
%% This is done so that consumers are still served in a deterministic
%% order on recovery.
prefix_msgs = {0, [], 0, []} :: prefix_msgs(),
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()}],
msg_bytes_in_memory = 0 :: non_neg_integer(),
msgs_ready_in_memory = 0 :: non_neg_integer(),
last_active :: undefined | non_neg_integer(),
unused_1,
unused_2
}).
-type config() :: #{name := atom(),
queue_resource := rabbit_types:r('queue'),
dead_letter_handler => applied_mfa(),
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(),
created => non_neg_integer()
}.

View File

@ -31,7 +31,9 @@
publisher_created/1,
publisher_deleted/1,
consumer_created/1,
consumer_deleted/1
consumer_deleted/1,
messages_dead_lettered/4,
messages_dead_lettered_confirmed/3
]).
%% PROTOCOL COUNTERS:
@ -126,11 +128,59 @@
}
]).
-define(MESSAGES_DEAD_LETTERED_EXPIRED, 1).
-define(MESSAGES_DEAD_LETTERED_REJECTED, 2).
%% The following two counters are mutually exclusive because
%% quorum queue dead-letter-strategy at-least-once is incompatible with overflow drop-head.
-define(MESSAGES_DEAD_LETTERED_MAXLEN, 3).
-define(MESSAGES_DEAD_LETTERED_CONFIRMED, 3).
-define(MESSAGES_DEAD_LETTERED_DELIVERY_LIMIT, 4).
-define(MESSAGES_DEAD_LETTERED_COUNTERS,
[
{
messages_dead_lettered_expired_total, ?MESSAGES_DEAD_LETTERED_EXPIRED, counter,
"Total number of messages dead-lettered due to message TTL exceeded"
},
{
messages_dead_lettered_rejected_total, ?MESSAGES_DEAD_LETTERED_REJECTED, counter,
"Total number of messages dead-lettered due to basic.reject or basic.nack"
}
]).
-define(MESSAGES_DEAD_LETTERED_MAXLEN_COUNTER,
{
messages_dead_lettered_maxlen_total, ?MESSAGES_DEAD_LETTERED_MAXLEN, counter,
"Total number of messages dead-lettered due to overflow drop-head or reject-publish-dlx"
}).
-define(MESSAGES_DEAD_LETTERED_CONFIRMED_COUNTER,
{
messages_dead_lettered_confirmed_total, ?MESSAGES_DEAD_LETTERED_CONFIRMED, counter,
"Total number of messages dead-lettered and confirmed by target queues"
}).
-define(MESSAGES_DEAD_LETTERED_DELIVERY_LIMIT_COUNTER,
{
messages_dead_lettered_delivery_limit_total, ?MESSAGES_DEAD_LETTERED_DELIVERY_LIMIT, counter,
"Total number of messages dead-lettered due to delivery-limit exceeded"
}).
boot_step() ->
init([{protocol, amqp091}]),
init([{protocol, amqp091}, {queue_type, rabbit_classic_queue}]),
init([{protocol, amqp091}, {queue_type, rabbit_quorum_queue}]),
init([{protocol, amqp091}, {queue_type, rabbit_stream_queue}]).
init([{protocol, amqp091}, {queue_type, rabbit_stream_queue}]),
init([{queue_type, rabbit_classic_queue}, {dead_letter_strategy, disabled}],
[?MESSAGES_DEAD_LETTERED_MAXLEN_COUNTER]),
init([{queue_type, rabbit_classic_queue}, {dead_letter_strategy, at_most_once}],
[?MESSAGES_DEAD_LETTERED_MAXLEN_COUNTER]),
init([{queue_type, rabbit_quorum_queue}, {dead_letter_strategy, disabled}],
[?MESSAGES_DEAD_LETTERED_MAXLEN_COUNTER,
?MESSAGES_DEAD_LETTERED_DELIVERY_LIMIT_COUNTER]),
init([{queue_type, rabbit_quorum_queue}, {dead_letter_strategy, at_most_once}],
[?MESSAGES_DEAD_LETTERED_MAXLEN_COUNTER,
?MESSAGES_DEAD_LETTERED_DELIVERY_LIMIT_COUNTER]),
init([{queue_type, rabbit_quorum_queue}, {dead_letter_strategy, at_least_once}],
[?MESSAGES_DEAD_LETTERED_CONFIRMED_COUNTER,
?MESSAGES_DEAD_LETTERED_DELIVERY_LIMIT_COUNTER
]).
init(Labels) ->
init(Labels, []).
@ -138,13 +188,15 @@ init(Labels) ->
init(Labels = [{protocol, Protocol}, {queue_type, QueueType}], Extra) ->
_ = seshat_counters:new_group(?MODULE),
Counters = seshat_counters:new(?MODULE, Labels, ?PROTOCOL_QUEUE_TYPE_COUNTERS ++ Extra),
persistent_term:put({?MODULE, Protocol, QueueType}, Counters),
ok;
persistent_term:put({?MODULE, Protocol, QueueType}, Counters);
init(Labels = [{protocol, Protocol}], Extra) ->
_ = seshat_counters:new_group(?MODULE),
Counters = seshat_counters:new(?MODULE, Labels, ?PROTOCOL_COUNTERS ++ Extra),
persistent_term:put({?MODULE, Protocol}, Counters),
ok.
persistent_term:put({?MODULE, Protocol}, Counters);
init(Labels = [{queue_type, QueueType}, {dead_letter_strategy, DLS}], Extra) ->
_ = seshat_counters:new_group(?MODULE),
Counters = seshat_counters:new(?MODULE, Labels, ?MESSAGES_DEAD_LETTERED_COUNTERS ++ Extra),
persistent_term:put({?MODULE, QueueType, DLS}, Counters).
overview() ->
seshat_counters:overview(?MODULE).
@ -209,8 +261,20 @@ consumer_created(Protocol) ->
consumer_deleted(Protocol) ->
counters:add(fetch(Protocol), ?CONSUMERS, -1).
messages_dead_lettered(Reason, QueueType, DeadLetterStrategy, Num) ->
Index = case Reason of
expired -> ?MESSAGES_DEAD_LETTERED_EXPIRED;
rejected -> ?MESSAGES_DEAD_LETTERED_REJECTED;
maxlen -> ?MESSAGES_DEAD_LETTERED_MAXLEN;
delivery_limit -> ?MESSAGES_DEAD_LETTERED_DELIVERY_LIMIT
end,
counters:add(fetch(QueueType, DeadLetterStrategy), Index, Num).
messages_dead_lettered_confirmed(rabbit_quorum_queue, at_least_once, Num) ->
counters:add(fetch(rabbit_quorum_queue, at_least_once), ?MESSAGES_DEAD_LETTERED_CONFIRMED, Num).
fetch(Protocol) ->
persistent_term:get({?MODULE, Protocol}).
fetch(Protocol, QueueType) ->
persistent_term:get({?MODULE, Protocol, QueueType}).
fetch(Elem2, Elem3) ->
persistent_term:get({?MODULE, Elem2, Elem3}).

View File

@ -9,12 +9,13 @@
-ignore_xref([
{lg, trace, 4},
{lg, stop, 0},
{lg_callgrind, profile_many, 3}
]).
-ignore_xref([{maps, from_list, 1}]).
-export([boot/0]).
-export([trace/1, profile/0, profile/1]).
-export([trace/1, trace_qq/0, profile/0, profile/1]).
-export([connections/0]).
boot() ->
@ -59,6 +60,27 @@ trace(Input) ->
{send, true}]
)).
trace_qq() ->
dbg:stop_clear(),
lg:trace([ra_server,
ra_server_proc,
rabbit_fifo,
queue,
rabbit_fifo_index
],
lg_file_tracer,
"traces.lz4",
maps:from_list([
{mode, profile}
% {process_dump, true},
% {running, true},
% {send, true}
]
)),
timer:sleep(10000),
lg:stop(),
profile().
profile() ->
profile("callgrind.out").

View File

@ -30,6 +30,7 @@ register() ->
{Class, Name} <- [{policy_validator, <<"alternate-exchange">>},
{policy_validator, <<"dead-letter-exchange">>},
{policy_validator, <<"dead-letter-routing-key">>},
{policy_validator, <<"dead-letter-strategy">>},
{policy_validator, <<"message-ttl">>},
{policy_validator, <<"expires">>},
{policy_validator, <<"max-length">>},
@ -85,6 +86,13 @@ validate_policy0(<<"dead-letter-routing-key">>, Value)
validate_policy0(<<"dead-letter-routing-key">>, Value) ->
{error, "~p is not a valid dead letter routing key", [Value]};
validate_policy0(<<"dead-letter-strategy">>, <<"at-most-once">>) ->
ok;
validate_policy0(<<"dead-letter-strategy">>, <<"at-least-once">>) ->
ok;
validate_policy0(<<"dead-letter-strategy">>, Value) ->
{error, "~p is not a valid dead letter strategy", [Value]};
validate_policy0(<<"message-ttl">>, Value)
when is_integer(Value), Value >= 0 ->
ok;

View File

@ -80,7 +80,7 @@
state :: queue_state()}).
-record(?STATE, {ctxs = #{} :: #{queue_ref() => #ctx{} | queue_ref()},
-record(?STATE, {ctxs = #{} :: #{queue_ref() => #ctx{}},
monitor_registry = #{} :: #{pid() => queue_ref()}
}).
@ -436,10 +436,10 @@ handle_event(QRef, Evt, Ctxs) ->
-spec module(queue_ref(), state()) ->
{ok, module()} | {error, not_found}.
module(QRef, Ctxs) ->
module(QRef, State) ->
%% events can arrive after a queue state has been cleared up
%% so need to be defensive here
case get_ctx(QRef, Ctxs, undefined) of
case get_ctx(QRef, State, undefined) of
#ctx{module = Mod} ->
{ok, Mod};
undefined ->
@ -577,12 +577,7 @@ get_ctx_with(QRef, Contexts, undefined) when ?QREF(QRef) ->
get_ctx(QRef, #?STATE{ctxs = Contexts}, Default) ->
Ref = qref(QRef),
%% if we use a QRef it should always be initialised
case maps:get(Ref, Contexts, undefined) of
#ctx{} = Ctx ->
Ctx;
undefined ->
Default
end.
maps:get(Ref, Contexts, Default).
set_ctx(Q, Ctx, #?STATE{ctxs = Contexts} = State)
when ?is_amqqueue(Q) ->

View File

@ -26,7 +26,7 @@
-export([credit/4]).
-export([purge/1]).
-export([stateless_deliver/2, deliver/3, deliver/2]).
-export([dead_letter_publish/4]).
-export([dead_letter_publish/5]).
-export([queue_name/1]).
-export([cluster_state/1, status/2]).
-export([update_consumer_handler/8, update_consumer/9]).
@ -71,6 +71,7 @@
-include_lib("stdlib/include/qlc.hrl").
-include_lib("rabbit_common/include/rabbit.hrl").
-include_lib("rabbit_common/include/rabbit_framing.hrl").
-include("amqqueue.hrl").
-type msg_id() :: non_neg_integer().
@ -94,7 +95,9 @@
single_active_consumer_pid,
single_active_consumer_ctag,
messages_ram,
message_bytes_ram
message_bytes_ram,
messages_dlx,
message_bytes_dlx
]).
-define(INFO_KEYS, [name, durable, auto_delete, arguments, pid, messages, messages_ready,
@ -227,18 +230,17 @@ ra_machine_config(Q) when ?is_amqqueue(Q) ->
{Name, _} = amqqueue:get_pid(Q),
%% take the minimum value of the policy and the queue arg if present
MaxLength = args_policy_lookup(<<"max-length">>, fun min/2, Q),
%% prefer the policy defined strategy if available
Overflow = args_policy_lookup(<<"overflow">>, fun (A, _B) -> A end , Q),
OverflowBin = args_policy_lookup(<<"overflow">>, fun policyHasPrecedence/2, Q),
Overflow = overflow(OverflowBin, drop_head, QName),
MaxBytes = args_policy_lookup(<<"max-length-bytes">>, fun min/2, Q),
MaxMemoryLength = args_policy_lookup(<<"max-in-memory-length">>, fun min/2, Q),
MaxMemoryBytes = args_policy_lookup(<<"max-in-memory-bytes">>, fun min/2, Q),
DeliveryLimit = args_policy_lookup(<<"delivery-limit">>, fun min/2, Q),
Expires = args_policy_lookup(<<"expires">>,
fun (A, _B) -> A end,
Q),
Expires = args_policy_lookup(<<"expires">>, fun policyHasPrecedence/2, Q),
MsgTTL = args_policy_lookup(<<"message-ttl">>, fun min/2, Q),
#{name => Name,
queue_resource => QName,
dead_letter_handler => dlx_mfa(Q),
dead_letter_handler => dead_letter_handler(Q, Overflow),
become_leader_handler => {?MODULE, become_leader, [QName]},
max_length => MaxLength,
max_bytes => MaxBytes,
@ -246,11 +248,17 @@ ra_machine_config(Q) when ?is_amqqueue(Q) ->
max_in_memory_bytes => MaxMemoryBytes,
single_active_consumer_on => single_active_consumer_on(Q),
delivery_limit => DeliveryLimit,
overflow_strategy => overflow(Overflow, drop_head, QName),
overflow_strategy => Overflow,
created => erlang:system_time(millisecond),
expires => Expires
expires => Expires,
msg_ttl => MsgTTL
}.
policyHasPrecedence(Policy, _QueueArg) ->
Policy.
queueArgHasPrecedence(_Policy, QueueArg) ->
QueueArg.
single_active_consumer_on(Q) ->
QArguments = amqqueue:get_arguments(Q),
case rabbit_misc:table_lookup(QArguments, <<"x-single-active-consumer">>) of
@ -293,7 +301,7 @@ become_leader(QName, Name) ->
end,
%% as this function is called synchronously when a ra node becomes leader
%% we need to ensure there is no chance of blocking as else the ra node
%% may not be able to establish it's leadership
%% may not be able to establish its leadership
spawn(fun() ->
rabbit_misc:execute_mnesia_transaction(
fun() ->
@ -377,19 +385,20 @@ filter_quorum_critical(Queues, ReplicaStates) ->
capabilities() ->
#{unsupported_policies =>
[ %% Classic policies
<<"message-ttl">>, <<"max-priority">>, <<"queue-mode">>,
<<"max-priority">>, <<"queue-mode">>,
<<"single-active-consumer">>, <<"ha-mode">>, <<"ha-params">>,
<<"ha-sync-mode">>, <<"ha-promote-on-shutdown">>, <<"ha-promote-on-failure">>,
<<"queue-master-locator">>,
%% Stream policies
<<"max-age">>, <<"stream-max-segment-size-bytes">>,
<<"queue-leader-locator">>, <<"initial-cluster-size">>],
queue_arguments => [<<"x-expires">>, <<"x-dead-letter-exchange">>,
<<"x-dead-letter-routing-key">>, <<"x-max-length">>,
<<"x-max-length-bytes">>, <<"x-max-in-memory-length">>,
<<"x-max-in-memory-bytes">>, <<"x-overflow">>,
<<"x-single-active-consumer">>, <<"x-queue-type">>,
<<"x-quorum-initial-group-size">>, <<"x-delivery-limit">>],
queue_arguments => [<<"x-dead-letter-exchange">>, <<"x-dead-letter-routing-key">>,
<<"x-dead-letter-strategy">>, <<"x-expires">>, <<"x-max-length">>,
<<"x-max-length-bytes">>, <<"x-max-in-memory-length">>,
<<"x-max-in-memory-bytes">>, <<"x-overflow">>,
<<"x-single-active-consumer">>, <<"x-queue-type">>,
<<"x-quorum-initial-group-size">>, <<"x-delivery-limit">>,
<<"x-message-ttl">>],
consumer_arguments => [<<"x-priority">>, <<"x-credit">>],
server_named => false}.
@ -405,12 +414,11 @@ spawn_deleter(QName) ->
end).
spawn_notify_decorators(QName, Fun, Args) ->
spawn(fun () ->
notify_decorators(QName, Fun, Args)
end).
%% run in ra process for now
catch notify_decorators(QName, Fun, Args).
handle_tick(QName,
{Name, MR, MU, M, C, MsgBytesReady, MsgBytesUnack},
{Name, MR, MU, M, C, MsgBytesReady, MsgBytesUnack, MsgBytesDiscard},
Nodes) ->
%% this makes calls to remote processes so cannot be run inside the
%% ra server
@ -429,8 +437,8 @@ handle_tick(QName,
{consumer_utilisation, Util},
{message_bytes_ready, MsgBytesReady},
{message_bytes_unacknowledged, MsgBytesUnack},
{message_bytes, MsgBytesReady + MsgBytesUnack},
{message_bytes_persistent, MsgBytesReady + MsgBytesUnack},
{message_bytes, MsgBytesReady + MsgBytesUnack + MsgBytesDiscard},
{message_bytes_persistent, MsgBytesReady + MsgBytesUnack + MsgBytesDiscard},
{messages_persistent, M}
| infos(QName, ?STATISTICS_KEYS -- [consumers])],
@ -839,8 +847,11 @@ deliver(true, Delivery, QState0) ->
rabbit_fifo_client:enqueue(Delivery#delivery.msg_seq_no,
Delivery#delivery.message, QState0).
deliver(QSs, #delivery{confirm = Confirm} = Delivery0) ->
Delivery = clean_delivery(Delivery0),
deliver(QSs, #delivery{message = #basic_message{content = Content0} = Msg,
confirm = Confirm} = Delivery0) ->
%% TODO: we could also consider clearing out the message id here
Content = prepare_content(Content0),
Delivery = Delivery0#delivery{message = Msg#basic_message{content = Content}},
lists:foldl(
fun({Q, stateless}, {Qs, Actions}) ->
QRef = amqqueue:get_pid(Q),
@ -1253,30 +1264,66 @@ reclaim_memory(Vhost, QueueName) ->
ra_log_wal:force_roll_over({?RA_WAL_NAME, Node}).
%%----------------------------------------------------------------------------
dlx_mfa(Q) ->
DLX = init_dlx(args_policy_lookup(<<"dead-letter-exchange">>,
fun res_arg/2, Q), Q),
DLXRKey = args_policy_lookup(<<"dead-letter-routing-key">>,
fun res_arg/2, Q),
{?MODULE, dead_letter_publish, [DLX, DLXRKey, amqqueue:get_name(Q)]}.
init_dlx(undefined, _Q) ->
undefined;
init_dlx(DLX, Q) when ?is_amqqueue(Q) ->
dead_letter_handler(Q, Overflow) ->
Exchange = args_policy_lookup(<<"dead-letter-exchange">>, fun queueArgHasPrecedence/2, Q),
RoutingKey = args_policy_lookup(<<"dead-letter-routing-key">>, fun queueArgHasPrecedence/2, Q),
Strategy = args_policy_lookup(<<"dead-letter-strategy">>, fun queueArgHasPrecedence/2, Q),
QName = amqqueue:get_name(Q),
rabbit_misc:r(QName, exchange, DLX).
dlh(Exchange, RoutingKey, Strategy, Overflow, QName).
res_arg(_PolVal, ArgVal) -> ArgVal.
dlh(undefined, undefined, undefined, _, _) ->
undefined;
dlh(undefined, RoutingKey, undefined, _, QName) ->
rabbit_log:warning("Disabling dead-lettering for ~s despite configured dead-letter-routing-key '~s' "
"because dead-letter-exchange is not configured.",
[rabbit_misc:rs(QName), RoutingKey]),
undefined;
dlh(undefined, _, Strategy, _, QName) ->
rabbit_log:warning("Disabling dead-lettering for ~s despite configured dead-letter-strategy '~s' "
"because dead-letter-exchange is not configured.",
[rabbit_misc:rs(QName), Strategy]),
undefined;
dlh(Exchange, RoutingKey, <<"at-least-once">>, reject_publish, QName) ->
%% Feature flag stream_queue includes the rabbit_queue_type refactor
%% which is required by rabbit_fifo_dlx_worker.
case rabbit_feature_flags:is_enabled(stream_queue) of
true ->
at_least_once;
false ->
rabbit_log:warning("Falling back to dead-letter-strategy at-most-once for ~s "
"because feature flag stream_queue is disabled.",
[rabbit_misc:rs(QName)]),
dlh_at_most_once(Exchange, RoutingKey, QName)
end;
dlh(Exchange, RoutingKey, <<"at-least-once">>, drop_head, QName) ->
rabbit_log:warning("Falling back to dead-letter-strategy at-most-once for ~s "
"because configured dead-letter-strategy at-least-once is incompatible with "
"effective overflow strategy drop-head. To enable dead-letter-strategy "
"at-least-once, set overflow strategy to reject-publish.",
[rabbit_misc:rs(QName)]),
dlh_at_most_once(Exchange, RoutingKey, QName);
dlh(Exchange, RoutingKey, _, _, QName) ->
dlh_at_most_once(Exchange, RoutingKey, QName).
dead_letter_publish(undefined, _, _, _) ->
ok;
dead_letter_publish(X, RK, QName, ReasonMsgs) ->
dlh_at_most_once(Exchange, RoutingKey, QName) ->
DLX = rabbit_misc:r(QName, exchange, Exchange),
MFA = {?MODULE, dead_letter_publish, [DLX, RoutingKey, QName]},
{at_most_once, MFA}.
dead_letter_publish(X, RK, QName, Reason, Msgs) ->
case rabbit_exchange:lookup(X) of
{ok, Exchange} ->
[rabbit_dead_letter:publish(Msg, Reason, Exchange, RK, QName)
|| {Reason, Msg} <- ReasonMsgs];
lists:foreach(fun(Msg) ->
rabbit_dead_letter:publish(Msg, Reason, Exchange, RK, QName)
end, Msgs),
rabbit_global_counters:messages_dead_lettered(Reason, ?MODULE, at_most_once, length(Msgs));
{error, not_found} ->
ok
%% Even though dead-letter-strategy is at_most_once,
%% when configured dead-letter-exchange does not exist,
%% we increment counter for dead-letter-strategy 'disabled' because
%% 1. we know for certain that the message won't be delivered, and
%% 2. that's in line with classic queue behaviour
rabbit_global_counters:messages_dead_lettered(Reason, ?MODULE, disabled, length(Msgs))
end.
find_quorum_queues(VHost) ->
@ -1438,6 +1485,28 @@ i(message_bytes_ram, Q) when ?is_amqqueue(Q) ->
{timeout, _} ->
0
end;
i(messages_dlx, Q) when ?is_amqqueue(Q) ->
QPid = amqqueue:get_pid(Q),
case ra:local_query(QPid,
fun rabbit_fifo:query_stat_dlx/1) of
{ok, {_, {Num, _}}, _} ->
Num;
{error, _} ->
0;
{timeout, _} ->
0
end;
i(message_bytes_dlx, Q) when ?is_amqqueue(Q) ->
QPid = amqqueue:get_pid(Q),
case ra:local_query(QPid,
fun rabbit_fifo:query_stat_dlx/1) of
{ok, {_, {_, Bytes}}, _} ->
Bytes;
{error, _} ->
0;
{timeout, _} ->
0
end;
i(_K, _Q) -> ''.
open_files(Name) ->
@ -1582,7 +1651,7 @@ overflow(undefined, Def, _QName) -> Def;
overflow(<<"reject-publish">>, _Def, _QName) -> reject_publish;
overflow(<<"drop-head">>, _Def, _QName) -> drop_head;
overflow(<<"reject-publish-dlx">> = V, Def, QName) ->
rabbit_log:warning("Invalid overflow strategy ~p for quorum queue: ~p",
rabbit_log:warning("Invalid overflow strategy ~p for quorum queue: ~s",
[V, rabbit_misc:rs(QName)]),
Def.
@ -1626,19 +1695,15 @@ notify_decorators(QName, F, A) ->
end.
%% remove any data that a quorum queue doesn't need
clean_delivery(#delivery{message =
#basic_message{content = Content0} = Msg} = Delivery) ->
Content = case Content0 of
#content{properties = none} ->
Content0;
#content{protocol = none} ->
Content0;
#content{properties = Props,
protocol = Proto} ->
Content0#content{properties = none,
properties_bin = Proto:encode_properties(Props)}
end,
%% TODO: we could also consider clearing out the message id here
Delivery#delivery{message = Msg#basic_message{content = Content}}.
prepare_content(#content{properties = none} = Content) ->
Content;
prepare_content(#content{protocol = none} = Content) ->
Content;
prepare_content(#content{properties = #'P_basic'{expiration = undefined} = Props,
protocol = Proto} = Content) ->
Content#content{properties = none,
properties_bin = Proto:encode_properties(Props)};
prepare_content(Content) ->
%% expiration is set. Therefore, leave properties decoded so that
%% rabbit_fifo can directly parse it without having to decode again.
Content.

View File

@ -957,7 +957,9 @@ capabilities() ->
<<"single-active-consumer">>, <<"delivery-limit">>,
<<"ha-mode">>, <<"ha-params">>, <<"ha-sync-mode">>,
<<"ha-promote-on-shutdown">>, <<"ha-promote-on-failure">>,
<<"queue-master-locator">>],
<<"queue-master-locator">>,
%% Quorum policies
<<"dead-letter-strategy">>],
queue_arguments => [<<"x-dead-letter-exchange">>, <<"x-dead-letter-routing-key">>,
<<"x-max-length">>, <<"x-max-length-bytes">>,
<<"x-single-active-consumer">>, <<"x-queue-type">>,

View File

@ -20,7 +20,7 @@ memory() ->
{Sums, _Other} = sum_processes(
lists:append(All), distinguishers(), [memory]),
[Qs, QsSlave, Qqs, Ssqs, Srqs, SCoor, ConnsReader, ConnsWriter, ConnsChannel,
[Qs, QsSlave, Qqs, DlxWorkers, Ssqs, Srqs, SCoor, ConnsReader, ConnsWriter, ConnsChannel,
ConnsOther, MsgIndexProc, MgmtDbProc, Plugins] =
[aggregate(Names, Sums, memory, fun (X) -> X end)
|| Names <- distinguished_interesting_sups()],
@ -55,7 +55,7 @@ memory() ->
OtherProc = Processes
- ConnsReader - ConnsWriter - ConnsChannel - ConnsOther
- Qs - QsSlave - Qqs - Ssqs - Srqs - SCoor - MsgIndexProc - Plugins
- Qs - QsSlave - Qqs - DlxWorkers - Ssqs - Srqs - SCoor - MsgIndexProc - Plugins
- MgmtDbProc - MetricsProc,
[
@ -69,6 +69,7 @@ memory() ->
{queue_procs, Qs},
{queue_slave_procs, QsSlave},
{quorum_queue_procs, Qqs},
{quorum_queue_dlx_procs, DlxWorkers},
{stream_queue_procs, Ssqs},
{stream_queue_replica_reader_procs, Srqs},
{stream_queue_coordinator_procs, SCoor},
@ -118,7 +119,7 @@ binary() ->
sets:add_element({Ptr, Sz}, Acc0)
end, Acc, Info)
end, distinguishers(), [{binary, sets:new()}]),
[Other, Qs, QsSlave, Qqs, Ssqs, Srqs, Scoor, ConnsReader, ConnsWriter,
[Other, Qs, QsSlave, Qqs, DlxWorkers, Ssqs, Srqs, Scoor, ConnsReader, ConnsWriter,
ConnsChannel, ConnsOther, MsgIndexProc, MgmtDbProc, Plugins] =
[aggregate(Names, [{other, Rest} | Sums], binary, fun sum_binary/1)
|| Names <- [[other] | distinguished_interesting_sups()]],
@ -129,6 +130,7 @@ binary() ->
{queue_procs, Qs},
{queue_slave_procs, QsSlave},
{quorum_queue_procs, Qqs},
{quorum_queue_dlx_procs, DlxWorkers},
{stream_queue_procs, Ssqs},
{stream_queue_replica_reader_procs, Srqs},
{stream_queue_coordinator_procs, Scoor},
@ -175,7 +177,7 @@ bytes(Words) -> try
end.
interesting_sups() ->
[queue_sups(), quorum_sups(), stream_server_sups(), stream_reader_sups(),
[queue_sups(), quorum_sups(), dlx_sups(), stream_server_sups(), stream_reader_sups(),
conn_sups() | interesting_sups0()].
queue_sups() ->
@ -192,6 +194,7 @@ quorum_sups() ->
supervisor:which_children(ra_server_sup_sup)]
end.
dlx_sups() -> [rabbit_fifo_dlx_sup].
stream_server_sups() -> [osiris_server_sup].
stream_reader_sups() -> [osiris_replica_reader_sup].
@ -248,6 +251,7 @@ distinguished_interesting_sups() ->
with(queue_sups(), master),
with(queue_sups(), slave),
with(quorum_sups(), quorum),
dlx_sups(),
stream_server_sups(),
stream_reader_sups(),
with(quorum_sups(), stream),

View File

@ -12,8 +12,9 @@
-include_lib("kernel/include/file.hrl").
-include_lib("amqp_client/include/amqp_client.hrl").
-include_lib("eunit/include/eunit.hrl").
-include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl").
-compile([nowarn_export_all, export_all]).
-compile([export_all, nowarn_export_all]).
-import(quorum_queue_utils, [wait_for_messages/2]).
@ -28,6 +29,7 @@ groups() ->
dead_letter_nack_requeue,
dead_letter_nack_requeue_multiple,
dead_letter_reject,
dead_letter_reject_many,
dead_letter_reject_requeue,
dead_letter_max_length_drop_head,
dead_letter_missing_exchange,
@ -46,25 +48,37 @@ groups() ->
dead_letter_headers_CC,
dead_letter_headers_CC_with_routing_key,
dead_letter_headers_first_death,
dead_letter_headers_first_death_route
],
Opts = [],
dead_letter_headers_first_death_route,
dead_letter_ttl,
dead_letter_routing_key_cycle_ttl,
dead_letter_headers_reason_expired,
dead_letter_headers_reason_expired_per_message,
dead_letter_extra_bcc],
DisabledMetricTests = [metric_maxlen,
metric_rejected,
metric_expired_queue_msg_ttl,
metric_expired_per_msg_msg_ttl],
Opts = [shuffle],
[
{dead_letter_tests, [],
{dead_letter_tests, Opts,
[
{classic_queue, Opts, DeadLetterTests ++ [dead_letter_ttl,
dead_letter_max_length_reject_publish_dlx,
dead_letter_routing_key_cycle_ttl,
dead_letter_headers_reason_expired,
dead_letter_headers_reason_expired_per_message]},
{mirrored_queue, Opts, DeadLetterTests ++ [dead_letter_ttl,
dead_letter_max_length_reject_publish_dlx,
dead_letter_routing_key_cycle_ttl,
dead_letter_headers_reason_expired,
dead_letter_headers_reason_expired_per_message]},
{quorum_queue, Opts, DeadLetterTests}
]}
].
{classic_queue, Opts, [{at_most_once, Opts, [dead_letter_max_length_reject_publish_dlx | DeadLetterTests]},
{disabled, Opts, DisabledMetricTests}]},
{mirrored_queue, Opts, [{at_most_once, Opts, [dead_letter_max_length_reject_publish_dlx | DeadLetterTests]},
{disabled, Opts, DisabledMetricTests}]},
{quorum_queue, Opts, [{at_most_once, Opts, DeadLetterTests},
{disabled, Opts, DisabledMetricTests},
{at_least_once, Opts, DeadLetterTests --
[
%% dead-letter-strategy at-least-once is incompatible with overflow drop-head
dead_letter_max_length_drop_head,
dead_letter_routing_key_cycle_max_length,
dead_letter_headers_reason_maxlen,
%% tested separately in rabbit_fifo_dlx_integration_SUITE
dead_letter_missing_exchange
]}
]
}]}].
suite() ->
[
@ -75,8 +89,10 @@ suite() ->
%% Testsuite setup/teardown.
%% -------------------------------------------------------------------
init_per_suite(Config) ->
init_per_suite(Config0) ->
rabbit_ct_helpers:log_environment(),
Config = rabbit_ct_helpers:merge_app_env(
Config0, {rabbit, [{dead_letter_worker_publisher_confirm_timeout, 2000}]}),
rabbit_ct_helpers:run_setup_steps(Config).
end_per_suite(Config) ->
@ -87,16 +103,6 @@ init_per_group(classic_queue, Config) ->
Config,
[{queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]},
{queue_durable, false}]);
init_per_group(quorum_queue, Config) ->
case rabbit_ct_broker_helpers:enable_feature_flag(Config, quorum_queue) of
ok ->
rabbit_ct_helpers:set_config(
Config,
[{queue_args, [{<<"x-queue-type">>, longstr, <<"quorum">>}]},
{queue_durable, true}]);
Skip ->
Skip
end;
init_per_group(mirrored_queue, Config) ->
rabbit_ct_broker_helpers:set_ha_policy(Config, 0, <<"^max_length.*queue">>,
<<"all">>, [{<<"ha-sync-mode">>, <<"automatic">>}]),
@ -105,6 +111,51 @@ init_per_group(mirrored_queue, Config) ->
{queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]},
{queue_durable, false}]),
rabbit_ct_helpers:run_steps(Config1, []);
init_per_group(quorum_queue, Config) ->
case rabbit_ct_broker_helpers:enable_feature_flag(Config, quorum_queue) of
ok ->
rabbit_ct_helpers:set_config(
Config,
[{queue_args, [{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-delivery-limit">>, long, 100}]},
{queue_durable, true}]);
Skip ->
Skip
end;
init_per_group(at_most_once, Config) ->
case outer_group_name(Config) of
quorum_queue ->
QueueArgs0 = rabbit_ct_helpers:get_config(Config, queue_args),
QueueArgs = lists:keystore(<<"x-dead-letter-strategy">>,
1,
QueueArgs0,
{<<"x-dead-letter-strategy">>, longstr, <<"at-most-once">>}),
rabbit_ct_helpers:set_config(Config, {queue_args, QueueArgs});
_ ->
Config
end;
init_per_group(at_least_once, Config) ->
case outer_group_name(Config) of
quorum_queue ->
QueueArgs0 = rabbit_ct_helpers:get_config(Config, queue_args),
QueueArgs1 = lists:keystore(<<"x-dead-letter-strategy">>,
1,
QueueArgs0,
{<<"x-dead-letter-strategy">>, longstr, <<"at-least-once">>}),
QueueArgs = lists:keystore(<<"x-overflow">>,
1,
QueueArgs1,
{<<"x-overflow">>, longstr, <<"reject-publish">>}),
Config1 = rabbit_ct_helpers:set_config(Config, {queue_args, QueueArgs}),
case rabbit_ct_broker_helpers:enable_feature_flag(Config1, stream_queue) of
ok ->
Config1;
Skip ->
Skip
end;
_ ->
Config
end;
init_per_group(Group, Config) ->
case lists:member({group, Group}, all()) of
true ->
@ -137,12 +188,14 @@ init_per_testcase(Testcase, Config) ->
Q3 = rabbit_data_coercion:to_binary(io_lib:format("~p_~p_3", [Group, Testcase])),
Policy = rabbit_data_coercion:to_binary(io_lib:format("~p_~p_policy", [Group, Testcase])),
DLXExchange = rabbit_data_coercion:to_binary(io_lib:format("~p_~p_dlx_exchange",
[Group, Testcase])),
[Group, Testcase])),
Counters = get_global_counters(Config),
Config1 = rabbit_ct_helpers:set_config(Config, [{dlx_exchange, DLXExchange},
{queue_name, Q},
{queue_name_dlx, Q2},
{queue_name_dlx_2, Q3},
{policy, Policy}]),
{policy, Policy},
{counters, Counters}]),
rabbit_ct_helpers:testcase_started(Config1, Testcase).
end_per_testcase(Testcase, Config) ->
@ -159,7 +212,7 @@ end_per_testcase(Testcase, Config) ->
%%
%% Messages are dead-lettered when:
%% 1) message is rejected with basic.reject or basic.nack with requeue=false
%% 2) message ttl expires (not implemented in quorum queues)
%% 2) message ttl expires
%% 3) queue length limit is exceeded (only drop-head implemented in quorum queues)
%%
%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%
@ -201,7 +254,8 @@ dead_letter_nack(Config) ->
consume_empty(Ch, QName),
%% Consume the first two messages from the dead letter queue
consume(Ch, DLXQName, [P1, P2]),
consume_empty(Ch, DLXQName).
consume_empty(Ch, DLXQName),
?assertEqual(3, counted(messages_dead_lettered_rejected_total, Config)).
%% 1) message is rejected with basic.nack, requeue=false and multiple=true
dead_letter_multiple_nack(Config) ->
@ -228,7 +282,8 @@ dead_letter_multiple_nack(Config) ->
consume(Ch, DLXQName, [P1, P2, P3]),
consume_empty(Ch, DLXQName),
%% Queue is empty
consume_empty(Ch, QName).
consume_empty(Ch, QName),
?assertEqual(3, counted(messages_dead_lettered_rejected_total, Config)).
%% 1) message is rejected with basic.nack, requeue=true and multiple=false. Dead-lettering does not take place
dead_letter_nack_requeue(Config) ->
@ -257,7 +312,8 @@ dead_letter_nack_requeue(Config) ->
consume(Ch, QName, [P3]),
consume_empty(Ch, QName),
%% Dead letter queue is empty
consume_empty(Ch, DLXQName).
consume_empty(Ch, DLXQName),
?assertEqual(0, counted(messages_dead_lettered_rejected_total, Config)).
%% 1) message is rejected with basic.nack, requeue=true and multiple=true. Dead-lettering does not take place
dead_letter_nack_requeue_multiple(Config) ->
@ -286,7 +342,8 @@ dead_letter_nack_requeue_multiple(Config) ->
consume(Ch, QName, [P1, P2, P3]),
consume_empty(Ch, QName),
%% Dead letter queue is empty
consume_empty(Ch, DLXQName).
consume_empty(Ch, DLXQName),
?assertEqual(0, counted(messages_dead_lettered_rejected_total, Config)).
%% 1) message is rejected with basic.reject, requeue=false
dead_letter_reject(Config) ->
@ -313,7 +370,39 @@ dead_letter_reject(Config) ->
consume_empty(Ch, DLXQName),
%% Consume the last two from the queue
_ = consume(Ch, QName, [P2, P3]),
consume_empty(Ch, QName).
consume_empty(Ch, QName),
?assertEqual(1, counted(messages_dead_lettered_rejected_total, Config)).
%% 1) Many messages are rejected. They get dead-lettered in correct order.
dead_letter_reject_many(Config) ->
{_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
QName = ?config(queue_name, Config),
DLXQName = ?config(queue_name_dlx, Config),
declare_dead_letter_queues(Ch, Config, QName, DLXQName),
%% Publish 100 messages
Payloads = lists:map(fun erlang:integer_to_binary/1, lists:seq(1, 100)),
publish(Ch, QName, Payloads),
wait_for_messages(Config, [[QName, <<"100">>, <<"100">>, <<"0">>]]),
%% Reject all messages using same consumer
amqp_channel:subscribe(Ch, #'basic.consume'{queue = QName}, self()),
CTag = receive #'basic.consume_ok'{consumer_tag = C} -> C end,
[begin
receive {#'basic.deliver'{consumer_tag = CTag, delivery_tag = DTag}, #amqp_msg{payload = P}} ->
amqp_channel:cast(Ch, #'basic.reject'{delivery_tag = DTag, requeue = false})
after 5000 ->
amqp_channel:call(Ch, #'basic.cancel'{consumer_tag = CTag}),
exit(timeout)
end
end || P <- Payloads],
amqp_channel:call(Ch, #'basic.cancel'{consumer_tag = CTag}),
%% Consume all messages from dead letter queue in correct order (i.e. from payload <<1>> to <<100>>)
wait_for_messages(Config, [[DLXQName, <<"100">>, <<"100">>, <<"0">>]]),
_ = consume(Ch, DLXQName, Payloads),
consume_empty(Ch, DLXQName),
?assertEqual(100, counted(messages_dead_lettered_rejected_total, Config)).
%% 1) Message is rejected with basic.reject, requeue=true. Dead-lettering does not take place.
dead_letter_reject_requeue(Config) ->
@ -339,7 +428,8 @@ dead_letter_reject_requeue(Config) ->
_ = consume(Ch, QName, [P1, P2, P3]),
consume_empty(Ch, QName),
%% Dead letter is empty
consume_empty(Ch, DLXQName).
consume_empty(Ch, DLXQName),
?assertEqual(0, counted(messages_dead_lettered_rejected_total, Config)).
%% 2) Message ttl expires
dead_letter_ttl(Config) ->
@ -353,7 +443,8 @@ dead_letter_ttl(Config) ->
publish(Ch, QName, [P1]),
wait_for_messages(Config, [[DLXQName, <<"1">>, <<"1">>, <<"0">>]]),
consume_empty(Ch, QName),
[_] = consume(Ch, DLXQName, [P1]).
[_] = consume(Ch, DLXQName, [P1]),
?assertEqual(1, counted(messages_dead_lettered_expired_total, Config)).
%% 3) The queue length limit is exceeded, message dropped is dead lettered.
%% Default strategy: drop-head
@ -377,7 +468,8 @@ dead_letter_max_length_drop_head(Config) ->
%% Consume the dropped ones from the dead letter queue
wait_for_messages(Config, [[DLXQName, <<"2">>, <<"2">>, <<"0">>]]),
_ = consume(Ch, DLXQName, [P1, P2]),
consume_empty(Ch, DLXQName).
consume_empty(Ch, DLXQName),
?assertEqual(2, counted(messages_dead_lettered_maxlen_total, Config)).
%% Another strategy: reject-publish-dlx
dead_letter_max_length_reject_publish_dlx(Config) ->
@ -402,7 +494,8 @@ dead_letter_max_length_reject_publish_dlx(Config) ->
%% Consume the dropped ones from the dead letter queue
wait_for_messages(Config, [[DLXQName, <<"2">>, <<"2">>, <<"0">>]]),
_ = consume(Ch, DLXQName, [P2, P3]),
consume_empty(Ch, DLXQName).
consume_empty(Ch, DLXQName),
?assertEqual(2, counted(messages_dead_lettered_maxlen_total, Config)).
%% Dead letter exchange does not have to be declared when the queue is declared, but it should
%% exist by the time messages need to be dead-lettered; if it is missing then, the messages will
@ -454,8 +547,9 @@ dead_letter_missing_exchange(Config) ->
%% Consume the rejected message from the dead letter queue
wait_for_messages(Config, [[DLXQName, <<"1">>, <<"1">>, <<"0">>]]),
{#'basic.get_ok'{}, #amqp_msg{payload = P2}} =
amqp_channel:call(Ch, #'basic.get'{queue = DLXQName}),
consume_empty(Ch, DLXQName).
amqp_channel:call(Ch, #'basic.get'{queue = DLXQName}),
consume_empty(Ch, DLXQName),
?assertEqual(1, counted(messages_dead_lettered_rejected_total, Config)).
%%
%% ROUTING
@ -490,8 +584,13 @@ dead_letter_routing_key(Config) ->
amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DTag1,
multiple = false,
requeue = false}),
%% Both queues are empty as the message could not been routed in the dlx exchange
wait_for_messages(Config, [[QName, <<"0">>, <<"0">>, <<"0">>]]),
case group_name(Config) of
at_most_once ->
%% Both queues are empty as the message could not been routed in the dlx exchange
wait_for_messages(Config, [[QName, <<"0">>, <<"0">>, <<"0">>]]);
at_least_once ->
wait_for_messages(Config, [[QName, <<"1">>, <<"0">>, <<"0">>]])
end,
consume_empty(Ch, QName),
consume_empty(Ch, DLXQName),
%% Bind the dlx queue with the original queue routing key
@ -506,10 +605,17 @@ dead_letter_routing_key(Config) ->
multiple = false,
requeue = false}),
%% Message can now be routed using the recently binded key
wait_for_messages(Config, [[DLXQName, <<"1">>, <<"1">>, <<"0">>]]),
consume(Ch, DLXQName, [P2]),
consume_empty(Ch, QName).
case group_name(Config) of
at_most_once ->
wait_for_messages(Config, [[DLXQName, <<"1">>, <<"1">>, <<"0">>]]),
consume(Ch, DLXQName, [P2]);
at_least_once ->
wait_for_messages(Config, [[DLXQName, <<"2">>, <<"2">>, <<"0">>]]),
consume(Ch, DLXQName, [P1, P2]),
?assertEqual(2, counted(messages_dead_lettered_confirmed_total, Config))
end,
consume_empty(Ch, QName),
?assertEqual(2, counted(messages_dead_lettered_rejected_total, Config)).
%% 4a) If a specific routing key was not set for the queue, use routing keys added by the
%% CC and BCC headers
@ -618,9 +724,10 @@ dead_letter_routing_key_cycle_max_length(Config) ->
amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DTag}),
%% Queue is empty, P1 has not been republished in a loop
wait_for_messages(Config, [[QName, <<"0">>, <<"0">>, <<"0">>]]),
consume_empty(Ch, QName).
consume_empty(Ch, QName),
?assertEqual(1, counted(messages_dead_lettered_maxlen_total, Config)).
%% 7) Message is dead lettered due to message ttl. Not yet implemented in quorum queues
%% 7) Message is dead lettered due to message ttl.
dead_letter_routing_key_cycle_ttl(Config) ->
{_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
Args = ?config(queue_args, Config),
@ -637,7 +744,8 @@ dead_letter_routing_key_cycle_ttl(Config) ->
%% Publish messages
publish(Ch, QName, [P1, P2]),
wait_for_messages(Config, [[QName, <<"0">>, <<"0">>, <<"0">>]]),
consume_empty(Ch, QName).
consume_empty(Ch, QName),
?assertEqual(2, counted(messages_dead_lettered_expired_total, Config)).
%% 5) Messages continue to be republished as there are manual rejections
dead_letter_routing_key_cycle_with_reject(Config) ->
@ -665,7 +773,8 @@ dead_letter_routing_key_cycle_with_reject(Config) ->
requeue = false}),
%% Message its being republished
wait_for_messages(Config, [[QName, <<"1">>, <<"1">>, <<"0">>]]),
[_] = consume(Ch, QName, [P]).
[_] = consume(Ch, QName, [P]),
?assertEqual(2, counted(messages_dead_lettered_rejected_total, Config)).
%%
%% For any given queue, a DLX can be defined by clients using the queue's arguments,
@ -677,7 +786,9 @@ dead_letter_policy(Config) ->
{_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
QName = ?config(queue_name, Config),
DLXQName = ?config(queue_name_dlx, Config),
Args = ?config(queue_args, Config),
Args0 = ?config(queue_args, Config),
%% declaring a quorum queue with x-dead-letter-strategy without defining a DLX will fail
Args = proplists:delete(<<"x-dead-letter-strategy">>, Args0),
Durable = ?config(queue_durable, Config),
DLXExchange = ?config(dlx_exchange, Config),
@ -1124,11 +1235,15 @@ dead_letter_headers_first_death(Config) ->
?assertEqual({longstr, <<>>},
rabbit_misc:table_lookup(Headers2, <<"x-first-death-exchange">>)).
%% Route dead-letter messages to different target queues according to first death reason.
%% Test that headers exchange's x-match binding argument set to all-with-x and any-with-x
%% works as expected. The use case being tested here:
%% Route dead-letter messages to different target queues
%% according to first death reason and first death queue.
dead_letter_headers_first_death_route(Config) ->
{_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
QName = ?config(queue_name, Config),
DLXMaxLengthQName = ?config(queue_name_dlx, Config),
QName1 = ?config(queue_name, Config),
QName2 = <<"dead_letter_headers_first_death_route_source_queue_2">>,
DLXExpiredQName = ?config(queue_name_dlx, Config),
DLXRejectedQName = ?config(queue_name_dlx_2, Config),
Args = ?config(queue_args, Config),
Durable = ?config(queue_durable, Config),
@ -1136,42 +1251,157 @@ dead_letter_headers_first_death_route(Config) ->
#'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = DLXExchange,
type = <<"headers">>}),
#'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = QName,
arguments = [{<<"x-dead-letter-exchange">>, longstr, DLXExchange},
{<<"x-max-length">>, long, 1} | Args],
#'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = QName1,
arguments = [{<<"x-dead-letter-exchange">>, longstr, DLXExchange} | Args],
durable = Durable}),
#'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = DLXMaxLengthQName,
#'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = QName2,
arguments = [{<<"x-dead-letter-exchange">>, longstr, DLXExchange} | Args],
durable = Durable}),
#'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = DLXExpiredQName,
durable = Durable}),
#'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = DLXRejectedQName,
durable = Durable}),
MatchAnyWithX = {<<"x-match">>, longstr, <<"any-with-x">>},
#'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{queue = DLXMaxLengthQName,
#'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{queue = DLXExpiredQName,
exchange = DLXExchange,
arguments = [MatchAnyWithX,
{<<"x-first-death-reason">>, longstr, <<"maxlen">>}]
arguments = [{<<"x-match">>, longstr, <<"all-with-x">>},
{<<"x-first-death-reason">>, longstr, <<"expired">>},
{<<"x-first-death-queue">>, longstr, QName1}]
}),
#'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{queue = DLXRejectedQName,
exchange = DLXExchange,
arguments = [MatchAnyWithX,
arguments = [{<<"x-match">>, longstr, <<"any-with-x">>},
{<<"x-first-death-reason">>, longstr, <<"rejected">>}]
}),
%% Send 1st message to 1st source queue and let it expire.
P1 = <<"msg1">>,
amqp_channel:call(Ch, #'basic.publish'{routing_key = QName1},
#amqp_msg{payload = P1,
props = #'P_basic'{expiration = <<"0">>}}),
%% The 1st message gets dead-lettered to DLXExpiredQName.
wait_for_messages(Config, [[DLXExpiredQName, <<"1">>, <<"1">>, <<"0">>]]),
_ = consume(Ch, DLXExpiredQName, [P1]),
consume_empty(Ch, DLXExpiredQName),
wait_for_messages(Config, [[QName1, <<"0">>, <<"0">>, <<"0">>]]),
%% Send 2nd message to 2nd source queue and let it expire.
P2 = <<"msg2">>,
%% Publish 2 messages
publish(Ch, QName, [P1, P2]),
%% The 1st message gets dropped from head of queue, dead-lettered and routed to DLXMaxLengthQName.
wait_for_messages(Config, [[DLXMaxLengthQName, <<"1">>, <<"1">>, <<"0">>]]),
_ = consume(Ch, DLXMaxLengthQName, [P1]),
consume_empty(Ch, DLXMaxLengthQName),
%% Reject the 2nd message.
wait_for_messages(Config, [[QName, <<"1">>, <<"1">>, <<"0">>]]),
[DTag] = consume(Ch, QName, [P2]),
amqp_channel:call(Ch, #'basic.publish'{routing_key = QName2},
#amqp_msg{payload = P2,
props = #'P_basic'{expiration = <<"0">>}}),
%% Send 2nd message should not be routed by the dead letter headers exchange.
rabbit_ct_helpers:consistently(?_assertEqual(#'basic.get_empty'{},
amqp_channel:call(Ch, #'basic.get'{queue = DLXExpiredQName}))),
%% Send and reject the 3rd message.
P3 = <<"msg3">>,
publish(Ch, QName2, [P3]),
timer:sleep(1000),
[DTag] = consume(Ch, QName2, [P3]),
amqp_channel:cast(Ch, #'basic.reject'{delivery_tag = DTag,
requeue = false}),
%% The 2nd message gets dead-lettered and routed to DLXRejectedQName.
%% The 3rd message gets dead-lettered to DLXRejectedQName.
wait_for_messages(Config, [[DLXRejectedQName, <<"1">>, <<"1">>, <<"0">>]]),
_ = consume(Ch, DLXRejectedQName, [P2]),
consume_empty(Ch, DLXRejectedQName).
_ = consume(Ch, DLXRejectedQName, [P3]),
consume_empty(Ch, DLXRejectedQName),
_ = amqp_channel:call(Ch, #'queue.delete'{queue = QName2}),
ok.
%% Route dead-letter messages also to extra BCC queues of target queues.
dead_letter_extra_bcc(Config) ->
{_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
SourceQ = ?config(queue_name, Config),
TargetQ = ?config(queue_name_dlx, Config),
ExtraBCCQ = ?config(queue_name_dlx_2, Config),
Durable = ?config(queue_durable, Config),
declare_dead_letter_queues(Ch, Config, SourceQ, TargetQ, [{<<"x-message-ttl">>, long, 0}]),
#'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = ExtraBCCQ,
durable = Durable}),
rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, set_queue_options,
[TargetQ, #{extra_bcc => ExtraBCCQ}]),
%% Publish message
P = <<"msg">>,
publish(Ch, SourceQ, [P]),
wait_for_messages(Config, [[TargetQ, <<"1">>, <<"1">>, <<"0">>],
[ExtraBCCQ, <<"1">>, <<"1">>, <<"0">>]]),
consume_empty(Ch, SourceQ),
[_] = consume(Ch, TargetQ, [P]),
[_] = consume(Ch, ExtraBCCQ, [P]),
ok.
set_queue_options(QName, Options) ->
rabbit_misc:execute_mnesia_transaction(
fun() ->
rabbit_amqqueue:update(rabbit_misc:r(<<"/">>, queue, QName),
fun(Q) ->
amqqueue:set_options(Q, Options)
end)
end).
metric_maxlen(Config) ->
{_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
QName = ?config(queue_name, Config),
#'queue.declare_ok'{} = amqp_channel:call(
Ch, #'queue.declare'{queue = QName,
arguments = [{<<"x-max-length">>, long, 1},
{<<"x-overflow">>, longstr, <<"drop-head">>} |
?config(queue_args, Config)],
durable = ?config(queue_durable, Config)}),
%% Publish 1000 messages
Payloads = lists:map(fun erlang:integer_to_binary/1, lists:seq(1, 1000)),
publish(Ch, QName, Payloads),
?awaitMatch(999, counted(messages_dead_lettered_maxlen_total, Config), 3000, 300).
metric_rejected(Config) ->
{_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
QName = ?config(queue_name, Config),
#'queue.declare_ok'{} = amqp_channel:call(
Ch, #'queue.declare'{queue = QName,
arguments = ?config(queue_args, Config),
durable = ?config(queue_durable, Config)}),
%% Publish 1000 messages
Payloads = lists:map(fun erlang:integer_to_binary/1, lists:seq(1, 1000)),
publish(Ch, QName, Payloads),
wait_for_messages(Config, [[QName, <<"1000">>, <<"1000">>, <<"0">>]]),
%% Reject all messages using same consumer
amqp_channel:subscribe(Ch, #'basic.consume'{queue = QName}, self()),
CTag = receive #'basic.consume_ok'{consumer_tag = C} -> C end,
[begin
receive {#'basic.deliver'{consumer_tag = CTag, delivery_tag = DTag}, #amqp_msg{payload = P}} ->
amqp_channel:cast(Ch, #'basic.reject'{delivery_tag = DTag, requeue = false})
after 5000 ->
amqp_channel:call(Ch, #'basic.cancel'{consumer_tag = CTag}),
exit(timeout)
end
end || P <- Payloads],
amqp_channel:call(Ch, #'basic.cancel'{consumer_tag = CTag}),
?awaitMatch(1000, counted(messages_dead_lettered_rejected_total, Config), 3000, 300).
metric_expired_queue_msg_ttl(Config) ->
{_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
QName = ?config(queue_name, Config),
#'queue.declare_ok'{} = amqp_channel:call(
Ch, #'queue.declare'{queue = QName,
arguments = [{<<"x-message-ttl">>, long, 0} |
?config(queue_args, Config)],
durable = ?config(queue_durable, Config)}),
%% Publish 1000 messages
Payloads = lists:map(fun erlang:integer_to_binary/1, lists:seq(1, 1000)),
publish(Ch, QName, Payloads),
?awaitMatch(1000, counted(messages_dead_lettered_expired_total, Config), 3000, 300).
metric_expired_per_msg_msg_ttl(Config) ->
{_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
QName = ?config(queue_name, Config),
#'queue.declare_ok'{} = amqp_channel:call(
Ch, #'queue.declare'{queue = QName,
arguments = ?config(queue_args, Config),
durable = ?config(queue_durable, Config)}),
%% Publish 1000 messages
Payloads = lists:map(fun erlang:integer_to_binary/1, lists:seq(1, 1000)),
[amqp_channel:call(Ch, #'basic.publish'{routing_key = QName},
#amqp_msg{payload = Payload,
props = #'P_basic'{expiration = <<"0">>}})
|| Payload <- Payloads],
?awaitMatch(1000, counted(messages_dead_lettered_expired_total, Config), 3000, 300).
%%%%%%%%%%%%%%%%%%%%%%%%
%% Test helpers
@ -1190,8 +1420,9 @@ declare_dead_letter_queues(Ch, Config, QName, DLXQName, ExtraArgs) ->
%% Declare queue
DeadLetterArgs = [{<<"x-dead-letter-exchange">>, longstr, DLXExchange},
{<<"x-dead-letter-routing-key">>, longstr, DLXQName}],
#'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = QName, arguments = DeadLetterArgs ++ Args ++ ExtraArgs, durable = Durable}),
#'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = QName,
arguments = DeadLetterArgs ++ Args ++ ExtraArgs,
durable = Durable}),
%% Declare and bind DLX queue
#'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = DLXQName, durable = Durable}),
#'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{queue = DLXQName,
@ -1211,7 +1442,7 @@ publish(Ch, QName, Payloads, Headers) ->
consume(Ch, QName, Payloads) ->
[begin
{#'basic.get_ok'{delivery_tag = DTag}, #amqp_msg{payload = Payload}} =
amqp_channel:call(Ch, #'basic.get'{queue = QName}),
amqp_channel:call(Ch, #'basic.get'{queue = QName}),
DTag
end || Payload <- Payloads].
@ -1224,3 +1455,31 @@ sync_mirrors(QName, Config) ->
rabbit_ct_broker_helpers:rabbitmqctl(Config, 0, [<<"sync_queue">>, QName]);
_ -> ok
end.
get_global_counters(Config) ->
rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_global_counters, overview, []).
%% Returns the delta of Metric between testcase start and now.
counted(Metric, Config) ->
QueueType = queue_type(outer_group_name(Config)),
Strategy = group_name(Config),
OldCounters = ?config(counters, Config),
Counters = get_global_counters(Config),
metric(QueueType, Strategy, Metric, Counters) -
metric(QueueType, Strategy, Metric, OldCounters).
metric(QueueType, Strategy, Metric, Counters) ->
Metrics = maps:get([{queue_type, QueueType}, {dead_letter_strategy, Strategy}], Counters),
maps:get(Metric, Metrics).
group_name(Config) ->
proplists:get_value(name, ?config(tc_group_properties, Config)).
outer_group_name(Config) ->
[{name, Name} | _] = hd(?config(tc_group_path, Config)),
Name.
queue_type(quorum_queue) ->
rabbit_quorum_queue;
queue_type(_) ->
rabbit_classic_queue.

View File

@ -14,7 +14,7 @@
-compile(export_all).
-define(TIMEOUT, 30000).
-define(TIMEOUT, 30_000).
-import(quorum_queue_utils, [wait_for_messages/2]).
@ -46,6 +46,7 @@ groups() ->
consume_and_multiple_nack,
basic_cancel,
purge,
purge_no_consumer,
basic_recover,
delete_immediately_by_resource
],
@ -596,6 +597,18 @@ purge(Config) ->
rabbit_ct_client_helpers:close_channel(Ch),
ok.
purge_no_consumer(Config) ->
{_, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
QName = ?config(queue_name, Config),
declare_queue(Ch, Config, QName),
publish(Ch, QName, [<<"msg1">>, <<"msg2">>]),
wait_for_messages(Config, [[QName, <<"2">>, <<"2">>, <<"0">>]]),
{'queue.purge_ok', 2} = amqp_channel:call(Ch, #'queue.purge'{queue = QName}),
wait_for_messages(Config, [[QName, <<"0">>, <<"0">>, <<"0">>]]),
rabbit_ct_client_helpers:close_channel(Ch),
ok.
basic_recover(Config) ->
{_, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
QName = ?config(queue_name, Config),

View File

@ -3,7 +3,6 @@
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2018-2021 VMware, Inc. or its affiliates. All rights reserved.
%%
-module(quorum_queue_SUITE).
@ -20,7 +19,6 @@
ra_name/1]).
-compile([nowarn_export_all, export_all]).
-compile(export_all).
-define(DEFAULT_AWAIT, 10000).
@ -81,11 +79,12 @@ groups() ->
quorum_cluster_size_7,
node_removal_is_not_quorum_critical
]},
{clustered_with_partitions, [], [
reconnect_consumer_and_publish,
reconnect_consumer_and_wait,
reconnect_consumer_and_wait_channel_down
]}
{clustered_with_partitions, [],
[
reconnect_consumer_and_publish,
reconnect_consumer_and_wait,
reconnect_consumer_and_wait_channel_down
]}
]}
].
@ -122,21 +121,16 @@ all_tests() ->
subscribe_redelivery_limit,
subscribe_redelivery_policy,
subscribe_redelivery_limit_with_dead_letter,
queue_length_in_memory_limit_basic_get,
queue_length_in_memory_limit_subscribe,
queue_length_in_memory_limit,
queue_length_in_memory_limit_returns,
queue_length_in_memory_bytes_limit_basic_get,
queue_length_in_memory_bytes_limit_subscribe,
queue_length_in_memory_bytes_limit,
queue_length_in_memory_purge,
in_memory,
purge,
consumer_metrics,
invalid_policy,
delete_if_empty,
delete_if_unused,
queue_ttl,
peek,
message_ttl,
per_message_ttl,
per_message_ttl_mixed_expiry,
consumer_priorities,
cancel_consumer_gh_3729
].
@ -963,14 +957,11 @@ invalid_policy(Config) ->
ok = rabbit_ct_broker_helpers:set_policy(
Config, 0, <<"ha">>, <<"invalid_policy.*">>, <<"queues">>,
[{<<"ha-mode">>, <<"all">>}]),
ok = rabbit_ct_broker_helpers:set_policy(
Config, 0, <<"ttl">>, <<"invalid_policy.*">>, <<"queues">>,
[{<<"message-ttl">>, 5}]),
Info = rpc:call(Server, rabbit_quorum_queue, infos,
[rabbit_misc:r(<<"/">>, queue, QQ)]),
?assertEqual('', proplists:get_value(policy, Info)),
ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"ha">>),
ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"ttl">>).
ok.
dead_letter_to_quorum_queue(Config) ->
[Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
@ -1894,7 +1885,11 @@ subscribe_redelivery_count(Config) ->
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
QQ = ?config(queue_name, Config),
?assertEqual({'queue.declare_ok', QQ, 0, 0},
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])),
declare(Ch, QQ,
[
{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-max-in-memory-length">>, long, 0}
])),
RaName = ra_name(QQ),
publish(Ch, QQ),
@ -1924,6 +1919,7 @@ subscribe_redelivery_count(Config) ->
multiple = false,
requeue = true})
after 5000 ->
flush(1),
exit(basic_deliver_timeout_2)
end,
@ -2265,287 +2261,25 @@ queue_length_limit_reject_publish(Config) ->
ok = publish_confirm(Ch, QQ),
ok.
queue_length_in_memory_limit_basic_get(Config) ->
purge(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
QQ = ?config(queue_name, Config),
?assertEqual({'queue.declare_ok', QQ, 0, 0},
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-max-in-memory-length">>, long, 1}])),
RaName = ra_name(QQ),
Msg1 = <<"msg1">>,
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = QQ},
#amqp_msg{props = #'P_basic'{delivery_mode = 2},
payload = Msg1}),
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = QQ},
#amqp_msg{props = #'P_basic'{delivery_mode = 2},
payload = <<"msg2">>}),
wait_for_messages(Config, [[QQ, <<"2">>, <<"2">>, <<"0">>]]),
?assertEqual([{1, byte_size(Msg1)}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)),
?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg1}},
amqp_channel:call(Ch, #'basic.get'{queue = QQ,
no_ack = true})),
?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg2">>}},
amqp_channel:call(Ch, #'basic.get'{queue = QQ,
no_ack = true})).
queue_length_in_memory_limit_subscribe(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
QQ = ?config(queue_name, Config),
?assertEqual({'queue.declare_ok', QQ, 0, 0},
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-max-in-memory-length">>, long, 1}])),
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])),
RaName = ra_name(QQ),
Msg1 = <<"msg1">>,
Msg2 = <<"msg11">>,
publish(Ch, QQ, Msg1),
publish(Ch, QQ, Msg2),
wait_for_messages(Config, [[QQ, <<"2">>, <<"2">>, <<"0">>]]),
?assertEqual([{1, byte_size(Msg1)}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)),
{'queue.purge_ok', 2} = amqp_channel:call(Ch, #'queue.purge'{queue = QQ}),
subscribe(Ch, QQ, false),
receive
{#'basic.deliver'{delivery_tag = DeliveryTag1,
redelivered = false},
#amqp_msg{payload = Msg1}} ->
amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DeliveryTag1,
multiple = false})
end,
?assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)),
receive
{#'basic.deliver'{delivery_tag = DeliveryTag2,
redelivered = false},
#amqp_msg{payload = Msg2}} ->
amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DeliveryTag2,
multiple = false})
end.
queue_length_in_memory_limit(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
QQ = ?config(queue_name, Config),
?assertEqual({'queue.declare_ok', QQ, 0, 0},
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-max-in-memory-length">>, long, 2}])),
RaName = ra_name(QQ),
Msg1 = <<"msg1">>,
Msg2 = <<"msg11">>,
Msg3 = <<"msg111">>,
Msg4 = <<"msg1111">>,
Msg5 = <<"msg1111">>,
publish(Ch, QQ, Msg1),
publish(Ch, QQ, Msg2),
publish(Ch, QQ, Msg3),
wait_for_messages(Config, [[QQ, <<"3">>, <<"3">>, <<"0">>]]),
?assertEqual([{2, byte_size(Msg1) + byte_size(Msg2)}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)),
?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg1}},
amqp_channel:call(Ch, #'basic.get'{queue = QQ,
no_ack = true})),
wait_for_messages(Config, [[QQ, <<"2">>, <<"2">>, <<"0">>]]),
publish(Ch, QQ, Msg4),
wait_for_messages(Config, [[QQ, <<"3">>, <<"3">>, <<"0">>]]),
?assertEqual([{2, byte_size(Msg2) + byte_size(Msg4)}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)),
publish(Ch, QQ, Msg5),
wait_for_messages(Config, [[QQ, <<"4">>, <<"4">>, <<"0">>]]),
ExpectedMsgs = [Msg2, Msg3, Msg4, Msg5],
validate_queue(Ch, QQ, ExpectedMsgs),
ok.
queue_length_in_memory_limit_returns(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
QQ = ?config(queue_name, Config),
?assertEqual({'queue.declare_ok', QQ, 0, 0},
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-max-in-memory-length">>, long, 2}])),
RaName = ra_name(QQ),
Msg1 = <<"msg1">>,
Msg2 = <<"msg11">>,
Msg3 = <<"msg111">>,
Msg4 = <<"msg111">>,
publish(Ch, QQ, Msg1),
publish(Ch, QQ, Msg2),
wait_for_messages(Config, [[QQ, <<"2">>, <<"2">>, <<"0">>]]),
?assertEqual([{2, byte_size(Msg1) + byte_size(Msg2)}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)),
?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg1}},
amqp_channel:call(Ch, #'basic.get'{queue = QQ,
no_ack = false})),
{#'basic.get_ok'{delivery_tag = DTag2}, #amqp_msg{payload = Msg2}} =
amqp_channel:call(Ch, #'basic.get'{queue = QQ,
no_ack = false}),
publish(Ch, QQ, Msg3),
publish(Ch, QQ, Msg4),
%% Ensure that returns are subject to in memory limits too
wait_for_messages(Config, [[QQ, <<"4">>, <<"2">>, <<"2">>]]),
amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DTag2,
multiple = true,
requeue = true}),
wait_for_messages(Config, [[QQ, <<"4">>, <<"4">>, <<"0">>]]),
?assertEqual([{2, byte_size(Msg3) + byte_size(Msg4)}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)).
queue_length_in_memory_bytes_limit_basic_get(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
QQ = ?config(queue_name, Config),
?assertEqual({'queue.declare_ok', QQ, 0, 0},
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-max-in-memory-bytes">>, long, 6}])),
RaName = ra_name(QQ),
Msg1 = <<"msg1">>,
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = QQ},
#amqp_msg{props = #'P_basic'{delivery_mode = 2},
payload = Msg1}),
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = QQ},
#amqp_msg{props = #'P_basic'{delivery_mode = 2},
payload = <<"msg2">>}),
wait_for_messages(Config, [[QQ, <<"2">>, <<"2">>, <<"0">>]]),
?assertEqual([{1, byte_size(Msg1)}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)),
?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg1}},
amqp_channel:call(Ch, #'basic.get'{queue = QQ,
no_ack = true})),
?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg2">>}},
amqp_channel:call(Ch, #'basic.get'{queue = QQ,
no_ack = true})).
queue_length_in_memory_bytes_limit_subscribe(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
QQ = ?config(queue_name, Config),
?assertEqual({'queue.declare_ok', QQ, 0, 0},
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-max-in-memory-bytes">>, long, 6}])),
RaName = ra_name(QQ),
Msg1 = <<"msg1">>,
Msg2 = <<"msg11">>,
publish(Ch, QQ, Msg1),
publish(Ch, QQ, Msg2),
wait_for_messages(Config, [[QQ, <<"2">>, <<"2">>, <<"0">>]]),
?assertEqual([{1, byte_size(Msg1)}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)),
subscribe(Ch, QQ, false),
receive
{#'basic.deliver'{delivery_tag = DeliveryTag1,
redelivered = false},
#amqp_msg{payload = Msg1}} ->
amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DeliveryTag1,
multiple = false})
end,
?assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)),
receive
{#'basic.deliver'{delivery_tag = DeliveryTag2,
redelivered = false},
#amqp_msg{payload = Msg2}} ->
amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DeliveryTag2,
multiple = false})
end.
queue_length_in_memory_bytes_limit(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
QQ = ?config(queue_name, Config),
?assertEqual({'queue.declare_ok', QQ, 0, 0},
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-max-in-memory-bytes">>, long, 12}])),
RaName = ra_name(QQ),
Msg1 = <<"msg1">>,
Msg2 = <<"msg11">>,
Msg3 = <<"msg111">>,
Msg4 = <<"msg1111">>,
publish(Ch, QQ, Msg1),
publish(Ch, QQ, Msg2),
publish(Ch, QQ, Msg3),
wait_for_messages(Config, [[QQ, <<"3">>, <<"3">>, <<"0">>]]),
?assertEqual([{2, byte_size(Msg1) + byte_size(Msg2)}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)),
?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg1}},
amqp_channel:call(Ch, #'basic.get'{queue = QQ,
no_ack = true})),
wait_for_messages(Config, [[QQ, <<"2">>, <<"2">>, <<"0">>]]),
publish(Ch, QQ, Msg4),
wait_for_messages(Config, [[QQ, <<"3">>, <<"3">>, <<"0">>]]),
?assertEqual([{2, byte_size(Msg2) + byte_size(Msg4)}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)).
queue_length_in_memory_purge(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
QQ = ?config(queue_name, Config),
?assertEqual({'queue.declare_ok', QQ, 0, 0},
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-max-in-memory-length">>, long, 2}])),
RaName = ra_name(QQ),
Msg1 = <<"msg1">>,
Msg2 = <<"msg11">>,
Msg3 = <<"msg111">>,
publish(Ch, QQ, Msg1),
publish(Ch, QQ, Msg2),
publish(Ch, QQ, Msg3),
wait_for_messages(Config, [[QQ, <<"3">>, <<"3">>, <<"0">>]]),
?assertEqual([{2, byte_size(Msg1) + byte_size(Msg2)}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)),
{'queue.purge_ok', 3} = amqp_channel:call(Ch, #'queue.purge'{queue = QQ}),
?assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)).
?assertEqual([0], dirty_query([Server], RaName, fun rabbit_fifo:query_messages_total/1)).
peek(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
@ -2580,7 +2314,26 @@ peek(Config) ->
wait_for_messages(Config, [[QQ, <<"2">>, <<"2">>, <<"0">>]]),
ok.
in_memory(Config) ->
message_ttl(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
QQ = ?config(queue_name, Config),
?assertEqual({'queue.declare_ok', QQ, 0, 0},
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-message-ttl">>, long, 2000}])),
Msg1 = <<"msg1">>,
Msg2 = <<"msg11">>,
publish(Ch, QQ, Msg1),
publish(Ch, QQ, Msg2),
wait_for_messages(Config, [[QQ, <<"2">>, <<"2">>, <<"0">>]]),
timer:sleep(2000),
wait_for_messages(Config, [[QQ, <<"0">>, <<"0">>, <<"0">>]]),
ok.
per_message_ttl(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
@ -2588,37 +2341,66 @@ in_memory(Config) ->
?assertEqual({'queue.declare_ok', QQ, 0, 0},
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])),
RaName = ra_name(QQ),
Msg1 = <<"msg1">>,
Msg2 = <<"msg11">>,
publish(Ch, QQ, Msg1),
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = QQ},
#amqp_msg{props = #'P_basic'{delivery_mode = 2,
expiration = <<"2000">>},
payload = Msg1}),
wait_for_messages(Config, [[QQ, <<"1">>, <<"1">>, <<"0">>]]),
?assertEqual([{1, byte_size(Msg1)}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)),
timer:sleep(2000),
wait_for_messages(Config, [[QQ, <<"0">>, <<"0">>, <<"0">>]]),
ok.
per_message_ttl_mixed_expiry(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
QQ = ?config(queue_name, Config),
?assertEqual({'queue.declare_ok', QQ, 0, 0},
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])),
Msg1 = <<"msg1">>,
Msg2 = <<"msg2">>,
%% message with no expiration
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = QQ},
#amqp_msg{props = #'P_basic'{delivery_mode = 2},
payload = Msg1}),
%% followed by message with expiration
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = QQ},
#amqp_msg{props = #'P_basic'{delivery_mode = 2,
expiration = <<"500">>},
payload = Msg2}),
wait_for_messages(Config, [[QQ, <<"2">>, <<"2">>, <<"0">>]]),
timer:sleep(1000),
wait_for_messages(Config, [[QQ, <<"2">>, <<"2">>, <<"0">>]]),
subscribe(Ch, QQ, false),
wait_for_messages(Config, [[QQ, <<"1">>, <<"0">>, <<"1">>]]),
?assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)),
publish(Ch, QQ, Msg2),
wait_for_messages(Config, [[QQ, <<"2">>, <<"0">>, <<"2">>]]),
?assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)),
receive
{#'basic.deliver'{delivery_tag = DeliveryTag}, #amqp_msg{}} ->
{#'basic.deliver'{delivery_tag = DeliveryTag},
#amqp_msg{payload = Msg1}} ->
amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DeliveryTag,
multiple = false})
after 2000 ->
flush(10),
ct:fail("basic deliver timeout")
end,
wait_for_messages(Config, [[QQ, <<"1">>, <<"0">>, <<"1">>]]),
?assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)).
%% the second message should NOT be received as it has expired
receive
{#'basic.deliver'{}, #amqp_msg{payload = Msg2}} ->
flush(10),
ct:fail("unexpected delivery")
after 500 ->
ok
end,
ok.
consumer_metrics(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,201 @@
%% 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) 2018-2021 VMware, Inc. or its affiliates. All rights reserved.
-module(rabbit_fifo_dlx_SUITE).
-compile(nowarn_export_all).
-compile(export_all).
-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.hrl").
%%%===================================================================
%%% Common Test callbacks
%%%===================================================================
all() ->
[
{group, tests}
].
groups() ->
[
{tests, [], [handler_undefined,
handler_at_most_once,
discard_dlx_consumer,
switch_strategies,
last_consumer_wins]}
].
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
%%%===================================================================
handler_undefined(_Config) ->
S = rabbit_fifo_dlx:init(),
Handler = undefined,
?assertEqual({S, [{mod_call, rabbit_global_counters, messages_dead_lettered,
[because, rabbit_quorum_queue, disabled, 1]}]},
rabbit_fifo_dlx:discard([make_msg(1)], because, Handler, S)),
ok.
handler_at_most_once(_Config) ->
S = rabbit_fifo_dlx:init(),
Handler = {at_most_once, {m, f, [a]}},
{S, Effects} = rabbit_fifo_dlx:discard([make_msg(1),
make_msg(2)], because, Handler, S),
?assertMatch([{log, [1, 2], _}], Effects),
ok.
discard_dlx_consumer(_Config) ->
Handler = at_least_once,
S0 = rabbit_fifo_dlx:init(),
?assertEqual(#{num_discarded => 0,
num_discard_checked_out => 0,
discard_message_bytes => 0,
discard_checkout_message_bytes => 0}, rabbit_fifo_dlx:overview(S0)),
%% message without dlx consumer
{S1, [{mod_call, rabbit_global_counters, messages_dead_lettered,
[because, rabbit_quorum_queue, at_least_once, 1]}]} =
rabbit_fifo_dlx:discard([make_msg(1)], because, Handler, S0),
{S2, []} = rabbit_fifo_dlx:checkout(Handler, S1),
?assertEqual(#{num_discarded => 1,
num_discard_checked_out => 0,
discard_message_bytes => 1,
discard_checkout_message_bytes => 0}, rabbit_fifo_dlx:overview(S2)),
%% with dlx consumer
Checkout = rabbit_fifo_dlx:make_checkout(self(), 2),
{S3, []} = rabbit_fifo_dlx:apply(meta(2), Checkout, Handler, S2),
{S4, DeliveryEffects0} = rabbit_fifo_dlx:checkout(Handler, S3),
?assertEqual(#{num_discarded => 0,
num_discard_checked_out => 1,
discard_message_bytes => 0,
discard_checkout_message_bytes => 1}, rabbit_fifo_dlx:overview(S4)),
?assertMatch([{log, [1], _}], DeliveryEffects0),
%% more messages than dlx consumer's prefetch
{S5, [_ModCallGlobalCounter]} = rabbit_fifo_dlx:discard([make_msg(3), make_msg(4)], because, Handler, S4),
{S6, DeliveryEffects1} = rabbit_fifo_dlx:checkout(Handler, S5),
?assertEqual(#{num_discarded => 1,
num_discard_checked_out => 2,
discard_message_bytes => 1,
discard_checkout_message_bytes => 2}, rabbit_fifo_dlx:overview(S6)),
?assertMatch([{log, [3], _}], DeliveryEffects1),
?assertEqual({3, 3}, rabbit_fifo_dlx:stat(S6)),
%% dlx consumer acks messages
Settle = rabbit_fifo_dlx:make_settle([0,1]),
{S7, [{mod_call, rabbit_global_counters, messages_dead_lettered_confirmed,
[rabbit_quorum_queue, at_least_once, 2]}]} =
rabbit_fifo_dlx:apply(meta(5), Settle, Handler, S6),
{S8, DeliveryEffects2} = rabbit_fifo_dlx:checkout(Handler, S7),
?assertEqual(#{num_discarded => 0,
num_discard_checked_out => 1,
discard_message_bytes => 0,
discard_checkout_message_bytes => 1}, rabbit_fifo_dlx:overview(S8)),
?assertMatch([{log, [4], _}], DeliveryEffects2),
?assertEqual({1, 1}, rabbit_fifo_dlx:stat(S8)),
ok.
switch_strategies(_Config) ->
QRes = #resource{virtual_host = <<"/">>,
kind = queue,
name = <<"blah">>},
application:set_env(rabbit, dead_letter_worker_consumer_prefetch, 1),
application:set_env(rabbit, dead_letter_worker_publisher_confirm_timeout, 1000),
{ok, _} = rabbit_fifo_dlx_sup:start_link(),
S0 = rabbit_fifo_dlx:init(),
Handler0 = undefined,
Handler1 = at_least_once,
%% Switching from undefined to at_least_once should start dlx consumer.
{S1, Effects0} = rabbit_fifo_dlx:update_config(Handler0, Handler1, QRes, S0),
?assertEqual([{mod_call, rabbit_log, debug,
["Switching dead_letter_handler from ~p to ~p for ~s",
[undefined, at_least_once, "queue 'blah' in vhost '/'"]]},
{aux, {dlx, setup}}],
Effects0),
rabbit_fifo_dlx:handle_aux(leader, {dlx, setup}, fake_aux, QRes, Handler1, S1),
[{_, WorkerPid, worker, _}] = supervisor:which_children(rabbit_fifo_dlx_sup),
{S2, _} = rabbit_fifo_dlx:discard([make_msg(1)], because, Handler1, S1),
Checkout = rabbit_fifo_dlx:make_checkout(WorkerPid, 1),
{S3, _} = rabbit_fifo_dlx:apply(meta(2), Checkout, Handler1, S2),
{S4, _} = rabbit_fifo_dlx:checkout(Handler1, S3),
?assertMatch(#{num_discard_checked_out := 1}, rabbit_fifo_dlx:overview(S4)),
%% Switching from at_least_once to undefined should terminate dlx consumer.
{S5, Effects} = rabbit_fifo_dlx:update_config(Handler1, Handler0, QRes, S4),
?assertEqual([{mod_call, rabbit_log, debug,
["Switching dead_letter_handler from ~p to ~p for ~s",
[at_least_once, undefined, "queue 'blah' in vhost '/'"]]},
{mod_call, rabbit_log, info,
["Deleted ~b dead-lettered messages (with total messages size of ~b bytes) in ~s",
[1, 1, "queue 'blah' in vhost '/'"]]}],
Effects),
?assertMatch([_, {active, 0}, _, _],
supervisor:count_children(rabbit_fifo_dlx_sup)),
?assertMatch(#{num_discarded := 0}, rabbit_fifo_dlx:overview(S5)),
ok.
last_consumer_wins(_Config) ->
S0 = rabbit_fifo_dlx:init(),
Handler = at_least_once,
Msgs = [make_msg(1), make_msg(2), make_msg(3), make_msg(4)],
{S1, [{mod_call, rabbit_global_counters, messages_dead_lettered,
[because, rabbit_quorum_queue, at_least_once, 4]}]} =
rabbit_fifo_dlx:discard(Msgs, because, Handler, S0),
Checkout = rabbit_fifo_dlx:make_checkout(self(), 10),
{S2, []} = rabbit_fifo_dlx:apply(meta(5), Checkout, Handler, S1),
{S3, DeliveryEffects0} = rabbit_fifo_dlx:checkout(Handler, S2),
?assertMatch([{log, [1, 2, 3, 4], _}], DeliveryEffects0),
?assertEqual(#{num_discarded => 0,
num_discard_checked_out => 4,
discard_message_bytes => 0,
discard_checkout_message_bytes => 4}, rabbit_fifo_dlx:overview(S3)),
%% When another (or the same) consumer (re)subscribes,
%% we expect this new consumer to be checked out and delivered all messages
%% from the previous consumer.
{S4, []} = rabbit_fifo_dlx:apply(meta(6), Checkout, Handler, S3),
{S5, DeliveryEffects1} = rabbit_fifo_dlx:checkout(Handler, S4),
?assertMatch([{log, [1, 2, 3, 4], _}], DeliveryEffects1),
?assertEqual(#{num_discarded => 0,
num_discard_checked_out => 4,
discard_message_bytes => 0,
discard_checkout_message_bytes => 4}, rabbit_fifo_dlx:overview(S5)),
ok.
make_msg(RaftIdx) ->
?MSG(RaftIdx, _Bytes = 1).
meta(Idx) ->
#{index => Idx,
term => 1,
system_time => 0,
from => {make_ref(), self()}}.

View File

@ -0,0 +1,957 @@
%% 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-2021 VMware, Inc. or its affiliates. All rights reserved.
-module(rabbit_fifo_dlx_integration_SUITE).
%% Integration tests for at-least-once dead-lettering comprising mainly
%% rabbit_fifo_dlx, rabbit_fifo_dlx_worker, rabbit_fifo_dlx_client
%% rabbit_quorum_queue, rabbit_fifo.
%%
%% Some at-least-once dead-lettering tests can also be found in
%% module dead_lettering_SUITE.
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
-include_lib("amqp_client/include/amqp_client.hrl").
-include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl").
-import(quorum_queue_utils, [wait_for_messages_ready/3,
wait_for_min_messages/3,
dirty_query/3,
ra_name/1]).
-import(rabbit_ct_helpers, [eventually/1,
eventually/3,
consistently/1]).
-import(quorum_queue_SUITE, [publish/2,
consume/3]).
-compile([nowarn_export_all, export_all]).
all() ->
[
{group, single_node},
{group, cluster_size_3}
].
groups() ->
[
{single_node, [shuffle], [
expired,
rejected,
delivery_limit,
target_queue_not_bound,
target_queue_deleted,
dlx_missing,
cycle,
stats,
drop_head_falls_back_to_at_most_once,
switch_strategy,
reject_publish_source_queue_max_length,
reject_publish_source_queue_max_length_bytes,
reject_publish_target_classic_queue,
reject_publish_target_quorum_queue,
target_quorum_queue_delete_create
]},
{cluster_size_3, [], [
many_target_queues,
single_dlx_worker
]}
].
init_per_suite(Config0) ->
rabbit_ct_helpers:log_environment(),
Config1 = rabbit_ct_helpers:merge_app_env(
Config0, {rabbit, [{quorum_tick_interval, 1000},
{dead_letter_worker_consumer_prefetch, 2},
{dead_letter_worker_publisher_confirm_timeout, 1000}
]}),
Config2 = rabbit_ct_helpers:merge_app_env(
Config1, {aten, [{poll_interval, 1000}]}),
rabbit_ct_helpers:run_setup_steps(Config2).
end_per_suite(Config) ->
rabbit_ct_helpers:run_teardown_steps(Config).
init_per_group(single_node = Group, Config) ->
init_per_group(Group, Config, 1);
init_per_group(cluster_size_3 = Group, Config) ->
init_per_group(Group, Config, 3).
init_per_group(Group, Config, NodesCount) ->
Config1 = rabbit_ct_helpers:set_config(Config,
[{rmq_nodes_count, NodesCount},
{rmq_nodename_suffix, Group},
{tcp_ports_base},
{net_ticktime, 10}]),
Config2 = rabbit_ct_helpers:run_steps(Config1,
[fun merge_app_env/1 ] ++
rabbit_ct_broker_helpers:setup_steps()),
ok = rabbit_ct_broker_helpers:rpc(
Config2, 0, application, set_env,
[rabbit, channel_tick_interval, 100]),
case rabbit_ct_broker_helpers:enable_feature_flag(Config2, quorum_queue) of
ok -> case rabbit_ct_broker_helpers:enable_feature_flag(Config2, stream_queue) of
ok -> Config2;
Skip -> Skip
end;
Skip -> Skip
end.
end_per_group(_, Config) ->
rabbit_ct_helpers:run_steps(Config,
rabbit_ct_broker_helpers:teardown_steps()).
merge_app_env(Config) ->
rabbit_ct_helpers:merge_app_env(
rabbit_ct_helpers:merge_app_env(Config,
{rabbit, [{core_metrics_gc_interval, 100}]}),
{ra, [{min_wal_roll_over_interval, 30000}]}).
init_per_testcase(Testcase, Config) ->
case {Testcase, rabbit_ct_helpers:is_mixed_versions()} of
{single_dlx_worker, true} ->
{skip, "single_dlx_worker is not mixed version compatible because process "
"rabbit_fifo_dlx_sup does not exist in 3.9"};
_ ->
Config1 = rabbit_ct_helpers:testcase_started(Config, Testcase),
T = rabbit_data_coercion:to_binary(Testcase),
Counters = get_global_counters(Config1),
Config2 = rabbit_ct_helpers:set_config(Config1,
[{source_queue, <<T/binary, "_source">>},
{dead_letter_exchange, <<T/binary, "_dlx">>},
{target_queue_1, <<T/binary, "_target_1">>},
{target_queue_2, <<T/binary, "_target_2">>},
{target_queue_3, <<T/binary, "_target_3">>},
{target_queue_4, <<T/binary, "_target_4">>},
{target_queue_5, <<T/binary, "_target_5">>},
{target_queue_6, <<T/binary, "_target_6">>},
{policy, <<T/binary, "_policy">>},
{counters, Counters}
]),
rabbit_ct_helpers:run_steps(Config2, rabbit_ct_client_helpers:setup_steps())
end.
end_per_testcase(Testcase, Config) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
delete_queue(Ch, ?config(source_queue, Config)),
delete_queue(Ch, ?config(target_queue_1, Config)),
delete_queue(Ch, ?config(target_queue_2, Config)),
delete_queue(Ch, ?config(target_queue_3, Config)),
delete_queue(Ch, ?config(target_queue_4, Config)),
delete_queue(Ch, ?config(target_queue_5, Config)),
delete_queue(Ch, ?config(target_queue_6, Config)),
#'exchange.delete_ok'{} = amqp_channel:call(Ch, #'exchange.delete'{exchange = ?config(dead_letter_exchange, Config)}),
Config1 = rabbit_ct_helpers:run_steps(
Config,
rabbit_ct_client_helpers:teardown_steps()),
rabbit_ct_helpers:testcase_finished(Config1, Testcase).
declare_topology(Config, AdditionalQArgs) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
SourceQ = ?config(source_queue, Config),
TargetQ = ?config(target_queue_1, Config),
DLX = ?config(dead_letter_exchange, Config),
declare_queue(Ch, SourceQ, lists:keymerge(1, AdditionalQArgs,
[{<<"x-dead-letter-exchange">>, longstr, DLX},
{<<"x-dead-letter-routing-key">>, longstr, <<"k1">>},
{<<"x-dead-letter-strategy">>, longstr, <<"at-least-once">>},
{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-queue-type">>, longstr, <<"quorum">>}
])),
#'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = DLX}),
declare_queue(Ch, TargetQ, []),
bind_queue(Ch, TargetQ, DLX, <<"k1">>),
{Server, Ch, SourceQ, TargetQ}.
%% Test that at-least-once dead-lettering works for message dead-lettered due to message TTL.
expired(Config) ->
{_Server, Ch, SourceQ, TargetQ} = declare_topology(Config, []),
Msg = <<"msg">>,
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = SourceQ},
#amqp_msg{props = #'P_basic'{expiration = <<"0">>},
payload = Msg}),
{_, #amqp_msg{props = #'P_basic'{headers = Headers,
expiration = undefined}}} =
?awaitMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ}),
1000),
assert_dlx_headers(Headers, <<"expired">>, SourceQ),
?assertEqual(1, counted(messages_dead_lettered_expired_total, Config)),
eventually(?_assertEqual(1, counted(messages_dead_lettered_confirmed_total, Config))).
%% Test that at-least-once dead-lettering works for message dead-lettered due to rejected by consumer.
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),
amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DelTag,
multiple = false,
requeue = false}),
{_, #amqp_msg{props = #'P_basic'{headers = Headers}}} =
?awaitMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg">>}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ}),
1000),
assert_dlx_headers(Headers, <<"rejected">>, SourceQ),
?assertEqual(1, counted(messages_dead_lettered_rejected_total, Config)),
eventually(?_assertEqual(1, counted(messages_dead_lettered_confirmed_total, Config))).
%% Test that at-least-once dead-lettering works for message dead-lettered due to delivery-limit exceeded.
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),
amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DelTag,
multiple = false,
requeue = true}),
{_, #amqp_msg{props = #'P_basic'{headers = Headers}}} =
?awaitMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg">>}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ}),
1000),
assert_dlx_headers(Headers, <<"delivery_limit">>, SourceQ),
?assertEqual(1, counted(messages_dead_lettered_delivery_limit_total, Config)),
eventually(?_assertEqual(1, counted(messages_dead_lettered_confirmed_total, Config))).
assert_dlx_headers(Headers, Reason, SourceQ) ->
?assertEqual({longstr, Reason}, rabbit_misc:table_lookup(Headers, <<"x-first-death-reason">>)),
?assertEqual({longstr, SourceQ}, rabbit_misc:table_lookup(Headers, <<"x-first-death-queue">>)),
?assertEqual({longstr, <<>>}, rabbit_misc:table_lookup(Headers, <<"x-first-death-exchange">>)),
{array, [{table, Death}]} = rabbit_misc:table_lookup(Headers, <<"x-death">>),
?assertEqual({longstr, SourceQ}, rabbit_misc:table_lookup(Death, <<"queue">>)),
?assertEqual({longstr, Reason}, rabbit_misc:table_lookup(Death, <<"reason">>)),
?assertEqual({longstr, <<>>}, rabbit_misc:table_lookup(Death, <<"exchange">>)),
?assertEqual({long, 1}, rabbit_misc:table_lookup(Death, <<"count">>)),
?assertEqual({array, [{longstr, SourceQ}]}, rabbit_misc:table_lookup(Death, <<"routing-keys">>)),
case Reason of
<<"expired">> ->
?assertEqual({longstr, <<"0">>}, rabbit_misc:table_lookup(Death, <<"original-expiration">>));
_ ->
ok
end.
%% Test that message is not lost despite no route from dead-letter exchange to target queue.
%% Once the route becomes available, the message is delivered to the target queue
%% and acked to the source quorum queue.
target_queue_not_bound(Config) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
SourceQ = ?config(source_queue, Config),
TargetQ = ?config(target_queue_1, Config),
DLX = ?config(dead_letter_exchange, Config),
declare_queue(Ch, SourceQ, [
{<<"x-dead-letter-exchange">>, longstr, DLX},
{<<"x-dead-letter-routing-key">>, longstr, <<"k1">>},
{<<"x-dead-letter-strategy">>, longstr, <<"at-least-once">>},
{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-queue-type">>, longstr, <<"quorum">>}
]),
#'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = DLX}),
declare_queue(Ch, TargetQ, []),
Msg = <<"msg">>,
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = SourceQ},
#amqp_msg{props = #'P_basic'{expiration = <<"0">>},
payload = Msg}),
RaName = ra_name(SourceQ),
%% Binding from target queue to DLX is missing.
%% Therefore, 1 message should be kept in discards queue.
eventually(?_assertMatch([{1, _}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))),
consistently(?_assertMatch([{1, _}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))),
?assertEqual(1, counted(messages_dead_lettered_expired_total, Config)),
?assertEqual(0, counted(messages_dead_lettered_confirmed_total, Config)),
%% Fix dead-letter toplology misconfiguration.
bind_queue(Ch, TargetQ, DLX, <<"k1">>),
%% Binding from target queue to DLX is now present.
%% Therefore, message should be delivered to target queue and acked to source queue.
eventually(?_assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1)),
500, 10),
?assertMatch({#'basic.get_ok'{}, #amqp_msg{props = #'P_basic'{expiration = undefined},
payload = Msg}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ})),
?assertEqual(1, counted(messages_dead_lettered_expired_total, Config)),
eventually(?_assertEqual(1, counted(messages_dead_lettered_confirmed_total, Config))).
%% Test that message is not lost when target queue gets deleted
%% because dead-letter routing topology should always be respected.
target_queue_deleted(Config) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
SourceQ = ?config(source_queue, Config),
TargetQ = ?config(target_queue_1, Config),
DLX = ?config(dead_letter_exchange, Config),
declare_queue(Ch, SourceQ, [
{<<"x-dead-letter-exchange">>, longstr, DLX},
{<<"x-dead-letter-routing-key">>, longstr, <<"k1">>},
{<<"x-dead-letter-strategy">>, longstr, <<"at-least-once">>},
{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-queue-type">>, longstr, <<"quorum">>}
]),
#'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = DLX}),
%% Make target queue a quorum queue to provoke sending an 'eol' message to dlx worker.
declare_queue(Ch, TargetQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
bind_queue(Ch, TargetQ, DLX, <<"k1">>),
Msg1 = <<"m1">>,
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = SourceQ},
#amqp_msg{props = #'P_basic'{expiration = <<"0">>},
payload = Msg1}),
RaName = ra_name(SourceQ),
eventually(?_assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg1}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ}))),
eventually(?_assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))),
#'queue.delete_ok'{message_count = 0} = amqp_channel:call(Ch, #'queue.delete'{queue = TargetQ}),
Msg2 = <<"m2">>,
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = SourceQ},
#amqp_msg{props = #'P_basic'{expiration = <<"0">>},
payload = Msg2}),
%% Message should not be lost despite deleted target queue.
eventually(?_assertMatch([{1, _}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))),
consistently(?_assertMatch([{1, _}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))),
%% Message should be delivered once target queue is recreated.
%% (This time we simply create a classic target queue.)
declare_queue(Ch, TargetQ, []),
bind_queue(Ch, TargetQ, DLX, <<"k1">>),
eventually(?_assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg2}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ})), 500, 5),
eventually(?_assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))),
?assertEqual(2, counted(messages_dead_lettered_expired_total, Config)),
?assertEqual(2, counted(messages_dead_lettered_confirmed_total, Config)).
%% Test that message is not lost when configured dead-letter exchange does not exist.
%% Once the exchange gets declared, the message is delivered to the target queue
%% and acked to the source quorum queue.
dlx_missing(Config) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
SourceQ = ?config(source_queue, Config),
TargetQ = ?config(target_queue_1, Config),
DLX = ?config(dead_letter_exchange, Config),
declare_queue(Ch, SourceQ, [
{<<"x-dead-letter-exchange">>, longstr, DLX},
{<<"x-dead-letter-routing-key">>, longstr, <<"k1">>},
{<<"x-dead-letter-strategy">>, longstr, <<"at-least-once">>},
{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-queue-type">>, longstr, <<"quorum">>}
]),
declare_queue(Ch, TargetQ, []),
Msg = <<"msg">>,
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = SourceQ},
#amqp_msg{props = #'P_basic'{expiration = <<"0">>},
payload = Msg}),
RaName = ra_name(SourceQ),
%% DLX is missing. Therefore, 1 message should be kept in discards queue.
eventually(?_assertMatch([{1, _}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))),
consistently(?_assertMatch([{1, _}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))),
%% Fix dead-letter toplology misconfiguration.
#'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = DLX}),
bind_queue(Ch, TargetQ, DLX, <<"k1">>),
%% DLX is now present.
%% Therefore, message should be delivered to target queue and acked to source queue.
eventually(?_assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1)),
500, 8),
?assertMatch({#'basic.get_ok'{}, #amqp_msg{props = #'P_basic'{expiration = undefined},
payload = Msg}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ})),
?assertEqual(1, counted(messages_dead_lettered_expired_total, Config)),
eventually(?_assertEqual(1, counted(messages_dead_lettered_confirmed_total, Config))).
%% Test that message is not lost when it cycles.
%% Once the cycle is resolved, the message is delivered to the target queue and acked to
%% the source quorum queue.
cycle(Config) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
SourceQ = ?config(source_queue, Config),
TargetQ = ?config(target_queue_1, Config),
PolicyName = ?config(policy, Config),
declare_queue(Ch, SourceQ, [
{<<"x-dead-letter-exchange">>, longstr, <<"">>},
{<<"x-dead-letter-strategy">>, longstr, <<"at-least-once">>},
{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-queue-type">>, longstr, <<"quorum">>}
]),
Msg = <<"msg">>,
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = SourceQ},
#amqp_msg{props = #'P_basic'{expiration = <<"0">>},
payload = Msg}),
RaName = ra_name(SourceQ),
%% Message cycled when it was dead-lettered:
%% source queue -> default exchange -> source queue
%% Therefore, 1 message should be kept in discards queue.
eventually(?_assertMatch([{1, _}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))),
consistently(?_assertMatch([{1, _}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))),
%% Fix the cycle such that dead-lettering flows like this:
%% source queue -> default exchange -> target queue
declare_queue(Ch, TargetQ, []),
ok = rabbit_ct_broker_helpers:set_policy(Config, Server, PolicyName,
SourceQ, <<"queues">>,
[{<<"dead-letter-routing-key">>, TargetQ}]),
eventually(?_assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1)),
500, 8),
?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ})),
ok = rabbit_ct_broker_helpers:clear_policy(Config, Server, PolicyName),
?assertEqual(1, counted(messages_dead_lettered_expired_total, Config)),
eventually(?_assertEqual(1, counted(messages_dead_lettered_confirmed_total, Config))).
%% Test that rabbit_fifo_dlx tracks statistics correctly.
stats(Config) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
SourceQ = ?config(source_queue, Config),
TargetQ = ?config(target_queue_1, Config),
DLX = ?config(dead_letter_exchange, Config),
declare_queue(Ch, SourceQ, [
{<<"x-dead-letter-exchange">>, longstr, DLX},
{<<"x-dead-letter-routing-key">>, longstr, <<"k1">>},
{<<"x-dead-letter-strategy">>, longstr, <<"at-least-once">>},
{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-queue-type">>, longstr, <<"quorum">>}
]),
#'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = DLX}),
declare_queue(Ch, TargetQ, []),
Msg = <<"12">>, %% 2 bytes per message
[ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = SourceQ},
#amqp_msg{props = #'P_basic'{expiration = <<"0">>},
payload = Msg})
|| _ <- lists:seq(1, 10)], %% 10 messages in total
RaName = ra_name(SourceQ),
%% Binding from target queue to DLX is missing. Therefore
%% * 10 msgs should be discarded (i.e. in discards queue or checked out to dlx_worker)
%% * 20 bytes (=10msgs*2bytes) should be discarded (i.e. in discards queue or checked out to dlx_worker)
eventually(?_assertEqual([{10, 20}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))),
?assertMatch([#{
%% 2 msgs (=Prefetch) should be checked out to dlx_worker
num_discard_checked_out := 2,
%% 4 bytes (=2msgs*2bytes) should be checked out to dlx_worker
discard_checkout_message_bytes := 4,
%% 8 msgs (=10-2) should be in discards queue
num_discarded := 8,
%% 16 bytes (=8msgs*2bytes) should be in discards queue
discard_message_bytes := 16,
%% 10 msgs in total
num_messages := 10
}],
dirty_query([Server], RaName, fun rabbit_fifo:overview/1)),
?assertEqual(10, counted(messages_dead_lettered_expired_total, Config)),
?assertEqual(0, counted(messages_dead_lettered_confirmed_total, Config)),
%% Fix dead-letter toplology misconfiguration.
bind_queue(Ch, TargetQ, DLX, <<"k1">>),
%% Binding from target queue to DLX is now present.
%% Therefore, all messages should be delivered to target queue and acked to source queue.
%% Therefore, all stats should be decremented back to 0.
eventually(?_assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1)),
500, 10),
?assertMatch([#{
num_discard_checked_out := 0,
discard_checkout_message_bytes := 0,
num_discarded := 0,
discard_message_bytes := 0,
num_messages := 0
}],
dirty_query([Server], RaName, fun rabbit_fifo:overview/1)),
[?assertMatch({#'basic.get_ok'{}, #amqp_msg{props = #'P_basic'{expiration = undefined},
payload = Msg}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ})) || _ <- lists:seq(1, 10)],
?assertEqual(10, counted(messages_dead_lettered_confirmed_total, Config)).
%% Test that configuring overflow (default) drop-head will fall back to
%% dead-letter-strategy at-most-once despite configuring at-least-once.
drop_head_falls_back_to_at_most_once(Config) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
SourceQ = ?config(source_queue, Config),
DLX = ?config(dead_letter_exchange, Config),
declare_queue(Ch, SourceQ, [
{<<"x-dead-letter-exchange">>, longstr, DLX},
{<<"x-dead-letter-strategy">>, longstr, <<"at-least-once">>},
{<<"x-overflow">>, longstr, <<"drop-head">>},
{<<"x-queue-type">>, longstr, <<"quorum">>}
]),
consistently(
?_assertMatch(
[_, {active, 0}, _, _],
rabbit_ct_broker_helpers:rpc(Config, Server, supervisor, count_children, [rabbit_fifo_dlx_sup]))).
%% Test that dynamically switching dead-letter-strategy works.
switch_strategy(Config) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
SourceQ = ?config(source_queue, Config),
RaName = ra_name(SourceQ),
DLX = ?config(dead_letter_exchange, Config),
PolicyName = ?config(policy, Config),
declare_queue(Ch, SourceQ, [
{<<"x-dead-letter-exchange">>, longstr, DLX},
{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-queue-type">>, longstr, <<"quorum">>}
]),
%% default strategy is at-most-once
assert_active_dlx_workers(0, Config, Server),
ok = rabbit_ct_broker_helpers:set_policy(Config, Server, PolicyName,
SourceQ, <<"queues">>,
[{<<"dead-letter-strategy">>, <<"at-least-once">>}]),
assert_active_dlx_workers(1, Config, Server),
[ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = SourceQ},
#amqp_msg{props = #'P_basic'{expiration = <<"0">>},
payload = <<"m">>}) %% 1 byte per message
|| _ <- lists:seq(1, 5)],
eventually(
?_assertMatch(
[#{
%% 2 msgs (=Prefetch) should be checked out to dlx_worker
num_discard_checked_out := 2,
discard_checkout_message_bytes := 2,
%% 3 msgs (=5-2) should be in discards queue
num_discarded := 3,
discard_message_bytes := 3,
num_messages := 5
}],
dirty_query([Server], RaName, fun rabbit_fifo:overview/1))),
ok = rabbit_ct_broker_helpers:set_policy(Config, Server, PolicyName,
SourceQ, <<"queues">>,
[{<<"dead-letter-strategy">>, <<"at-most-once">>}]),
assert_active_dlx_workers(0, Config, Server),
?assertMatch(
[#{
num_discard_checked_out := 0,
discard_checkout_message_bytes := 0,
num_discarded := 0,
discard_message_bytes := 0,
num_messages := 0
}],
dirty_query([Server], RaName, fun rabbit_fifo:overview/1)),
ok = rabbit_ct_broker_helpers:clear_policy(Config, Server, PolicyName),
?assertEqual(5, counted(messages_dead_lettered_expired_total, Config)),
?assertEqual(0, counted(messages_dead_lettered_confirmed_total, Config)).
%% Test that source quorum queue rejects messages when source quorum queue's max-length is reached.
%% max-length should also take into account dead-lettered messages.
reject_publish_source_queue_max_length(Config) ->
reject_publish(Config, {<<"x-max-length">>, long, 1}).
%% Test that source quorum queue rejects messages when source quorum queue's max-length-bytes is reached.
%% max-length-bytes should also take into account dead-lettered messages.
reject_publish_source_queue_max_length_bytes(Config) ->
reject_publish(Config, {<<"x-max-length-bytes">>, long, 1}).
reject_publish(Config, QArg) when is_tuple(QArg) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
SourceQ = ?config(source_queue, Config),
TargetQ = ?config(target_queue_1, Config),
PolicyName = ?config(policy, Config),
%% This routing key prevents messages from being routed to target dead-letter queue.
ok = rabbit_ct_broker_helpers:set_policy(Config, Server, PolicyName, SourceQ, <<"queues">>,
[{<<"dead-letter-routing-key">>, <<"fake">>}]),
declare_queue(Ch, SourceQ, [
{<<"x-dead-letter-exchange">>, longstr, <<"">>},
{<<"x-dead-letter-strategy">>, longstr, <<"at-least-once">>},
{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-message-ttl">>, long, 0},
QArg
]),
declare_queue(Ch, TargetQ, []),
#'confirm.select_ok'{} = amqp_channel:call(Ch, #'confirm.select'{}),
ok = publish_confirm(Ch, SourceQ),
ok = publish_confirm(Ch, SourceQ),
RaName = ra_name(SourceQ),
eventually(?_assertMatch([{2, 2}], %% 2 messages with 1 byte each
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))),
%% Now, we have 2 expired messages in the source quorum queue's discards queue.
%% Now that we are over the limit we expect publishes to be rejected.
?assertEqual(fail, publish_confirm(Ch, SourceQ)),
%% Fix the dead-letter routing topology.
ok = rabbit_ct_broker_helpers:set_policy(Config, Server, PolicyName, SourceQ, <<"queues">>,
[{<<"dead-letter-routing-key">>, TargetQ}]),
eventually(?_assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1)), 500, 6),
%% Publish should be allowed again.
ok = publish_confirm(Ch, SourceQ),
%% Consume the 3 expired messages from the target dead-letter queue.
?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"m">>}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ})),
?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"m">>}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ})),
eventually(?_assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"m">>}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ}))),
ok = rabbit_ct_broker_helpers:clear_policy(Config, Server, PolicyName).
%% Test that message gets delivered to target quorum queue eventually when it gets rejected initially.
reject_publish_target_quorum_queue(Config) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
SourceQ = ?config(source_queue, Config),
RaName = ra_name(SourceQ),
TargetQ = ?config(target_queue_1, Config),
declare_queue(Ch, SourceQ, [{<<"x-dead-letter-exchange">>, longstr, <<"">>},
{<<"x-dead-letter-routing-key">>, longstr, TargetQ},
{<<"x-dead-letter-strategy">>, longstr, <<"at-least-once">>},
{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-queue-type">>, longstr, <<"quorum">>}
]),
declare_queue(Ch, TargetQ, [{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-max-length">>, long, 1}
]),
Msg = <<"m">>,
%% Send 4 messages although target queue has max-length of 1.
[ok,ok,ok,ok] = [begin
amqp_channel:cast(Ch, #'basic.publish'{routing_key = SourceQ},
#amqp_msg{props = #'P_basic'{expiration = integer_to_binary(N)},
payload = Msg})
end || N <- lists:seq(1,4)],
%% Make space in target queue by consuming messages one by one
%% allowing for more dead-lettered messages to reach the target queue.
[begin
timer:sleep(2000),
{#'basic.get_ok'{}, #amqp_msg{payload = Msg}} = amqp_channel:call(Ch, #'basic.get'{queue = TargetQ})
end || _ <- lists:seq(1,4)],
eventually(?_assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1)), 500, 10),
?assertEqual(4, counted(messages_dead_lettered_expired_total, Config)),
eventually(?_assertEqual(4, counted(messages_dead_lettered_confirmed_total, Config))).
%% Test that message gets eventually delivered to target classic queue when it gets rejected initially.
reject_publish_target_classic_queue(Config) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
SourceQ = ?config(source_queue, Config),
RaName = ra_name(SourceQ),
TargetQ = ?config(target_queue_1, Config),
declare_queue(Ch, SourceQ, [{<<"x-dead-letter-exchange">>, longstr, <<"">>},
{<<"x-dead-letter-routing-key">>, longstr, TargetQ},
{<<"x-dead-letter-strategy">>, longstr, <<"at-least-once">>},
{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-message-ttl">>, long, 1}
]),
declare_queue(Ch, TargetQ, [{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-max-length">>, long, 1}
]),
Msg = <<"m">>,
ok = amqp_channel:cast(Ch, #'basic.publish'{routing_key = SourceQ}, #amqp_msg{payload = Msg}),
ok = amqp_channel:cast(Ch, #'basic.publish'{routing_key = SourceQ}, #amqp_msg{payload = Msg}),
%% By now we expect target classic queue confirmed 1 message and rejected 1 message.
eventually(?_assertEqual([{1, 1}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))),
consistently(?_assertEqual([{1, 1}],
dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1))),
?assertEqual(2, counted(messages_dead_lettered_expired_total, Config)),
?assertEqual(1, counted(messages_dead_lettered_confirmed_total, Config)),
%% Let's make space in the target queue for the rejected message.
{#'basic.get_ok'{}, #amqp_msg{payload = Msg}} = amqp_channel:call(Ch, #'basic.get'{queue = TargetQ}),
eventually(?_assertEqual(2, counted(messages_dead_lettered_confirmed_total, Config)), 500, 6),
?assertEqual([{0, 0}], dirty_query([Server], RaName, fun rabbit_fifo:query_stat_dlx/1)),
{#'basic.get_ok'{}, #amqp_msg{payload = Msg}} = amqp_channel:call(Ch, #'basic.get'{queue = TargetQ}),
ok.
publish_confirm(Ch, QName) ->
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = QName},
#amqp_msg{payload = <<"m">>}),
amqp_channel:register_confirm_handler(Ch, self()),
receive
#'basic.ack'{} ->
ok;
#'basic.nack'{} ->
fail
after 2500 ->
ct:fail(confirm_timeout)
end.
%% Test that all dead-lettered messages reach target quorum queue eventually
%% when target queue is deleted and recreated with same name
%% and when dead-letter-exchange is default exchange.
target_quorum_queue_delete_create(Config) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
SourceQ = ?config(source_queue, Config),
TargetQ = ?config(target_queue_1, Config),
declare_queue(Ch, SourceQ, [{<<"x-dead-letter-exchange">>, longstr, <<"">>},
{<<"x-dead-letter-routing-key">>, longstr, TargetQ},
{<<"x-dead-letter-strategy">>, longstr, <<"at-least-once">>},
{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-message-ttl">>, long, 1}
]),
DeclareTargetQueue = fun() ->
declare_queue(Ch, TargetQ,
[{<<"x-queue-type">>, longstr, <<"quorum">>}])
end,
Send100Msgs = fun() ->
[ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = SourceQ},
#amqp_msg{payload = <<"msg">>})
|| _ <- lists:seq(1, 100)]
end,
DeclareTargetQueue(),
Send100Msgs(),
%% Delete and recreate target queue (immediately or after some while).
timer:sleep(rand:uniform(50)),
%% Log the current number of messages.
rabbit_ct_broker_helpers:rabbitmqctl_list(
Config, 0, ["list_queues", "name", "messages", "messages_ready",
"messages_unacknowledged"]),
#'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = TargetQ}),
Send100Msgs(),
timer:sleep(rand:uniform(200)),
DeclareTargetQueue(),
Send100Msgs(),
%% Expect no message to get stuck in dlx worker.
wait_for_min_messages(Config, TargetQ, 200),
eventually(?_assertEqual([{0, 0}],
dirty_query([Server], ra_name(SourceQ), fun rabbit_fifo:query_stat_dlx/1)), 500, 10),
?assertEqual(300, counted(messages_dead_lettered_expired_total, Config)),
?assertEqual(300, counted(messages_dead_lettered_confirmed_total, Config)),
#'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = TargetQ}).
%% Test that
%% 1. Message is only acked to source queue once publisher confirms got received from **all** target queues.
%% 2. Target queue can be classic queue, quorum queue, or stream queue.
%%
%% Lesson learnt by writing this test:
%% If there are multiple target queues, messages will not be sent to target non-mirrored classic queues
%% (even if durable) when their host node is temporarily down because these queues get (temporarily) deleted. See:
%% https://github.com/rabbitmq/rabbitmq-server/blob/cf76b479300b767b8ea450293d096cbf729ed734/deps/rabbit/src/rabbit_amqqueue.erl#L1955-L1964
many_target_queues(Config) ->
[Server1, Server2, Server3] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server1),
Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server2),
SourceQ = ?config(source_queue, Config),
RaName = ra_name(SourceQ),
TargetQ1 = ?config(target_queue_1, Config),
TargetQ2 = ?config(target_queue_2, Config),
TargetQ3 = ?config(target_queue_3, Config),
TargetQ4 = ?config(target_queue_4, Config),
TargetQ5 = ?config(target_queue_5, Config),
TargetQ6 = ?config(target_queue_6, Config),
DLX = ?config(dead_letter_exchange, Config),
DLRKey = <<"k1">>,
%% Create topology:
%% * source quorum queue with 1 replica on node 1
%% * target non-mirrored classic queue on node 1
%% * target quorum queue with 3 replicas
%% * target stream queue with 3 replicas
%% * target mirrored classic queue with 3 replicas (leader on node 1)
%% * target mirrored classic queue with 1 replica (leader on node 2)
%% * target mirrored classic queue with 3 replica (leader on node 2)
declare_queue(Ch, SourceQ, [{<<"x-dead-letter-exchange">>, longstr, DLX},
{<<"x-dead-letter-routing-key">>, longstr, DLRKey},
{<<"x-dead-letter-strategy">>, longstr, <<"at-least-once">>},
{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-quorum-initial-group-size">>, long, 1}
]),
#'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = DLX}),
declare_queue(Ch, TargetQ1, []),
bind_queue(Ch, TargetQ1, DLX, DLRKey),
declare_queue(Ch, TargetQ2, [{<<"x-queue-type">>, longstr, <<"quorum">>},
{<<"x-quorum-initial-group-size">>, long, 3}
]),
bind_queue(Ch, TargetQ2, DLX, DLRKey),
declare_queue(Ch, TargetQ3, [{<<"x-queue-type">>, longstr, <<"stream">>},
{<<"x-initial-cluster-size">>, long, 3}
]),
bind_queue(Ch, TargetQ3, DLX, DLRKey),
ok = rabbit_ct_broker_helpers:set_policy(Config, Server1, <<"mirror-q4">>, TargetQ4, <<"queues">>,
[{<<"ha-mode">>, <<"all">>},
{<<"queue-master-locator">>, <<"client-local">>}]),
declare_queue(Ch, TargetQ4, []),
bind_queue(Ch, TargetQ4, DLX, DLRKey),
ok = rabbit_ct_broker_helpers:set_policy(Config, Server1, <<"mirror-q5">>, TargetQ5, <<"queues">>,
[{<<"ha-mode">>, <<"exactly">>},
{<<"ha-params">>, 1},
{<<"queue-master-locator">>, <<"client-local">>}]),
declare_queue(Ch2, TargetQ5, []),
bind_queue(Ch2, TargetQ5, DLX, DLRKey),
ok = rabbit_ct_broker_helpers:set_policy(Config, Server1, <<"mirror-q6">>, TargetQ6, <<"queues">>,
[{<<"ha-mode">>, <<"all">>},
{<<"queue-master-locator">>, <<"client-local">>}]),
declare_queue(Ch2, TargetQ6, []),
bind_queue(Ch2, TargetQ6, DLX, DLRKey),
Msg1 = <<"m1">>,
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = SourceQ},
#amqp_msg{props = #'P_basic'{expiration = <<"5">>},
payload = Msg1}),
eventually(?_assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg1}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ1}))),
eventually(?_assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg1}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ2}))),
%% basic.get not supported by stream queues
#'basic.qos_ok'{} = amqp_channel:call(Ch, #'basic.qos'{prefetch_count = 2}),
CTag = <<"ctag">>,
amqp_channel:subscribe(
Ch,
#'basic.consume'{queue = TargetQ3,
consumer_tag = CTag,
arguments = [{<<"x-stream-offset">>, long, 0}]},
self()),
receive
#'basic.consume_ok'{consumer_tag = CTag} ->
ok
after 2000 ->
exit(consume_ok_timeout)
end,
receive
{#'basic.deliver'{consumer_tag = CTag},
#amqp_msg{payload = Msg1}} ->
ok
after 2000 ->
exit(deliver_timeout)
end,
eventually(?_assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg1}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ4}))),
eventually(?_assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg1}},
amqp_channel:call(Ch2, #'basic.get'{queue = TargetQ5}))),
eventually(?_assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg1}},
amqp_channel:call(Ch2, #'basic.get'{queue = TargetQ6}))),
eventually(?_assertEqual([{0, 0}],
dirty_query([Server1], RaName, fun rabbit_fifo:query_stat_dlx/1))),
ok = rabbit_ct_broker_helpers:stop_node(Config, Server3),
ok = rabbit_ct_broker_helpers:stop_node(Config, Server2),
Msg2 = <<"m2">>,
ok = amqp_channel:cast(Ch,
#'basic.publish'{routing_key = SourceQ},
#amqp_msg{props = #'P_basic'{expiration = <<"1">>},
payload = Msg2}),
%% Nodes 2 and 3 are down.
%% rabbit_fifo_dlx_worker should wait until all queues confirm the message
%% before acking it to the source queue.
eventually(?_assertEqual([{1, 2}],
dirty_query([Server1], RaName, fun rabbit_fifo:query_stat_dlx/1))),
consistently(?_assertEqual([{1, 2}],
dirty_query([Server1], RaName, fun rabbit_fifo:query_stat_dlx/1))),
?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg2}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ1})),
ok = rabbit_ct_broker_helpers:start_node(Config, Server2),
ok = rabbit_ct_broker_helpers:start_node(Config, Server3),
eventually(?_assertEqual([{0, 0}],
dirty_query([Server1], RaName, fun rabbit_fifo:query_stat_dlx/1)), 500, 6),
?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg2}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ2})),
receive
{#'basic.deliver'{consumer_tag = CTag},
#amqp_msg{payload = Msg2}} ->
ok
after 0 ->
exit(deliver_timeout)
end,
?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg2}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ4})),
eventually(?_assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg2}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ5}))),
%%TODO why is the 1st message (m1) a duplicate?
?awaitMatch({#'basic.get_ok'{}, #amqp_msg{payload = Msg2}},
amqp_channel:call(Ch, #'basic.get'{queue = TargetQ6}), 2, 200),
?assertEqual(2, counted(messages_dead_lettered_expired_total, Config)),
?assertEqual(2, counted(messages_dead_lettered_confirmed_total, Config)).
%% Test that there is a single active rabbit_fifo_dlx_worker that is co-located with the quorum queue leader.
single_dlx_worker(Config) ->
[Server1, Server2, _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server1),
SourceQ = ?config(source_queue, Config),
DLX = ?config(dead_letter_exchange, Config),
declare_queue(Ch, SourceQ, [
{<<"x-dead-letter-exchange">>, longstr, DLX},
{<<"x-dead-letter-strategy">>, longstr, <<"at-least-once">>},
{<<"x-overflow">>, longstr, <<"reject-publish">>},
{<<"x-queue-type">>, longstr, <<"quorum">>}
]),
?assertMatch(
[[_, {active, 1}, _, _],
[_, {active, 0}, _, _],
[_, {active, 0}, _, _]],
rabbit_ct_broker_helpers:rpc_all(Config, supervisor, count_children, [rabbit_fifo_dlx_sup])),
ok = rabbit_ct_broker_helpers:stop_node(Config, Server1),
RaName = ra_name(SourceQ),
{ok, _, {_, Leader0}} = ra:members({RaName, Server2}),
?assertNotEqual(Server1, Leader0),
[Follower0] = Servers -- [Server1, Leader0],
assert_active_dlx_workers(1, Config, Leader0),
assert_active_dlx_workers(0, Config, Follower0),
ok = rabbit_ct_broker_helpers:start_node(Config, Server1),
consistently(
?_assertMatch(
[_, {active, 0}, _, _],
rabbit_ct_broker_helpers:rpc(Config, Server1, supervisor, count_children, [rabbit_fifo_dlx_sup], 1000))),
Pid = rabbit_ct_broker_helpers:rpc(Config, Leader0, erlang, whereis, [RaName]),
true = rabbit_ct_broker_helpers:rpc(Config, Leader0, erlang, exit, [Pid, kill]),
{ok, _, {_, Leader1}} = ?awaitMatch({ok, _, _},
ra:members({RaName, Follower0}),
1000),
?assertNotEqual(Leader0, Leader1),
[Follower1, Follower2] = Servers -- [Leader1],
assert_active_dlx_workers(0, Config, Follower1),
assert_active_dlx_workers(0, Config, Follower2),
assert_active_dlx_workers(1, Config, Leader1).
assert_active_dlx_workers(N, Config, Server) ->
?assertMatch(
[_, {active, N}, _, _],
rabbit_ct_broker_helpers:rpc(Config, Server, supervisor, count_children, [rabbit_fifo_dlx_sup], 1000)).
declare_queue(Channel, Queue, Args) ->
#'queue.declare_ok'{} = amqp_channel:call(Channel, #'queue.declare'{
queue = Queue,
durable = true,
arguments = Args
}).
bind_queue(Channel, Queue, Exchange, RoutingKey) ->
#'queue.bind_ok'{} = amqp_channel:call(Channel, #'queue.bind'{
queue = Queue,
exchange = Exchange,
routing_key = RoutingKey
}).
delete_queue(Channel, Queue) ->
%% We implicitly test here that we don't end up with duplicate messages.
#'queue.delete_ok'{message_count = 0} = amqp_channel:call(Channel, #'queue.delete'{queue = Queue}).
get_global_counters(Config) ->
rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_global_counters, overview, []).
%% Returns the delta of Metric between testcase start and now.
counted(Metric, Config) ->
OldCounters = ?config(counters, Config),
Counters = get_global_counters(Config),
metric(Metric, Counters) -
metric(Metric, OldCounters).
metric(Metric, Counters) ->
Metrics = maps:get([{queue_type, rabbit_quorum_queue}, {dead_letter_strategy, at_least_once}], Counters),
maps:get(Metric, Metrics).

View File

@ -295,7 +295,7 @@ returns_after_down(Config) ->
F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
{ok, F1} = rabbit_fifo_client:enqueue(msg1, F0),
{_, _, F2} = process_ra_events(receive_ra_events(1, 0), F1),
% start a customer in a separate processes
% start a consumer in a separate processes
% that exits after checkout
Self = self(),
_Pid = spawn(fun () ->
@ -375,7 +375,7 @@ discard(Config) ->
machine => {module, rabbit_fifo,
#{queue_resource => discard,
dead_letter_handler =>
{?MODULE, dead_letter_handler, [self()]}}}},
{at_most_once, {?MODULE, dead_letter_handler, [self()]}}}}},
_ = rabbit_quorum_queue:start_server(Conf),
ok = ra:trigger_election(ServerId),
_ = ra:members(ServerId),
@ -387,8 +387,9 @@ discard(Config) ->
F3 = discard_next_delivery(F2, 5000),
{empty, _F4} = rabbit_fifo_client:dequeue(<<"tag1">>, settled, F3),
receive
{dead_letter, Letters} ->
[{_, msg1}] = Letters,
{dead_letter, Reason, Letters} ->
[msg1] = Letters,
rejected = Reason,
ok
after 500 ->
flush(),
@ -510,8 +511,8 @@ test_queries(Config) ->
rabbit_quorum_queue:stop_server(ServerId),
ok.
dead_letter_handler(Pid, Msgs) ->
Pid ! {dead_letter, Msgs}.
dead_letter_handler(Pid, Reason, Msgs) ->
Pid ! {dead_letter, Reason, Msgs}.
dequeue(Config) ->
ClusterName = ?config(cluster_name, Config),

File diff suppressed because it is too large Load Diff

View File

@ -1,5 +1,6 @@
-module(rabbit_stream_coordinator_SUITE).
-compile(nowarn_export_all).
-compile(export_all).
-export([

View File

@ -24,6 +24,7 @@ groups() ->
alternate_exchange,
dead_letter_exchange,
dead_letter_routing_key,
dead_letter_strategy,
message_ttl,
expires,
max_length,
@ -86,6 +87,13 @@ dead_letter_exchange(_Config) ->
dead_letter_routing_key(_Config) ->
requires_binary_value(<<"dead-letter-routing-key">>).
dead_letter_strategy(_Config) ->
test_valid_and_invalid_values(<<"dead-letter-strategy">>,
%% valid values
[<<"at-most-once">>, <<"at-least-once">>],
%% invalid values
[<<"unknown">>, <<"dead-letter-strategy">>, <<"undefined">>]).
message_ttl(_Config) ->
requires_non_negative_integer_value(<<"message-ttl">>).

View File

@ -112,7 +112,7 @@
-record(basic_message,
{exchange_name, %% The exchange where the message was received
routing_keys = [], %% Routing keys used during publish
content, %% The message content
content, %% The message #content record
id, %% A `rabbit_guid:gen()` generated id
is_persistent}). %% Whether the message was published as persistent

View File

@ -80,7 +80,7 @@ defmodule RabbitMQ.CLI.Ctl.Commands.DeleteQueueCommand do
end
def output({:ok, qlen}, _options) do
{:ok, "Queue was successfully deleted with #{qlen} messages"}
{:ok, "Queue was successfully deleted with #{qlen} ready messages"}
end
## Use default output for all non-special case outputs

View File

@ -266,7 +266,7 @@ consume(Ch, QName, Count) ->
self()),
CTag = receive #'basic.consume_ok'{consumer_tag = C} -> C end,
[begin
Exp = list_to_binary(integer_to_list(I)),
Exp = integer_to_binary(I),
receive {#'basic.deliver'{consumer_tag = CTag},
#amqp_msg{payload = Exp}} ->
ok
@ -287,7 +287,7 @@ accumulate_without_acknowledging(Ch, CTag, Remaining, Acc) when Remaining =:= 0
amqp_channel:call(Ch, #'basic.cancel'{consumer_tag = CTag}),
lists:reverse(Acc);
accumulate_without_acknowledging(Ch, CTag, Remaining, Acc) ->
receive {#'basic.deliver'{consumer_tag = CTag, delivery_tag = DTag}, _MSg} ->
receive {#'basic.deliver'{consumer_tag = CTag, delivery_tag = DTag}, _Msg} ->
accumulate_without_acknowledging(Ch, CTag, Remaining - 1, [DTag | Acc])
after 5000 ->
amqp_channel:call(Ch, #'basic.cancel'{consumer_tag = CTag}),

View File

@ -50,7 +50,10 @@
await_condition/1,
await_condition/2,
await_condition_with_retries/2
await_condition_with_retries/2,
eventually/1, eventually/3,
consistently/1, consistently/3
]).
-define(SSL_CERT_PASSWORD, "test").
@ -1047,6 +1050,45 @@ await_condition_with_retries(ConditionFun, RetriesLeft) ->
ok
end.
%% Pass in any EUnit test object. Example:
%% eventually(?_assertEqual(1, Actual))
eventually({Line, Assertion} = TestObj)
when is_integer(Line), Line >= 0, is_function(Assertion, 0) ->
eventually(TestObj, 200, 5).
eventually({Line, _}, _, 0) ->
erlang:error({assert_timeout_line, Line});
eventually({Line, Assertion} = TestObj, PollInterval, PollCount)
when is_integer(Line), Line >= 0, is_function(Assertion, 0),
is_integer(PollInterval), PollInterval >= 0,
is_integer(PollCount), PollCount >= 0 ->
case catch Assertion() of
ok ->
ok;
Err ->
ct:pal(?LOW_IMPORTANCE,
"Retrying in ~bms for ~b more times due to failed assertion in line ~b: ~p",
[PollInterval, PollCount - 1, Line, Err]),
timer:sleep(PollInterval),
eventually(TestObj, PollInterval, PollCount - 1)
end.
%% Pass in any EUnit test object. Example:
%% consistently(?_assertEqual(1, Actual))
consistently({Line, Assertion} = TestObj)
when is_integer(Line), Line >= 0, is_function(Assertion, 0) ->
consistently(TestObj, 200, 5).
consistently(_, _, 0) ->
ok;
consistently({Line, Assertion} = TestObj, PollInterval, PollCount)
when is_integer(Line), Line >= 0, is_function(Assertion, 0),
is_integer(PollInterval), PollInterval >= 0,
is_integer(PollCount), PollCount >= 0 ->
Assertion(),
timer:sleep(PollInterval),
consistently(TestObj, PollInterval, PollCount - 1).
%% -------------------------------------------------------------------
%% Cover-related functions.
%% -------------------------------------------------------------------

View File

@ -174,7 +174,7 @@ const QUEUE_EXTRA_CONTENT_REQUESTS = [];
// All help ? popups
var HELP = {
'delivery-limit':
'The number of allowed unsuccessful delivery attempts. Once a message has been delivered unsuccessfully this many times it will be dropped or dead-lettered, depending on the queue configuration.',
'The number of allowed unsuccessful delivery attempts. Once a message has been delivered unsuccessfully more than this many times it will be dropped or dead-lettered, depending on the queue configuration.',
'exchange-auto-delete':
'If yes, the exchange will delete itself after at least one queue or exchange has been bound to this one, and then all queues or exchanges have been unbound.',
@ -218,6 +218,9 @@ var HELP = {
'queue-dead-letter-routing-key':
'Optional replacement routing key to use when a message is dead-lettered. If this is not set, the message\'s original routing key will be used.<br/>(Sets the "<a target="_blank" href="https://rabbitmq.com/dlx.html">x-dead-letter-routing-key</a>" argument.)',
'queue-dead-letter-strategy':
'Valid values are <code>at-most-once</code> or <code>at-least-once</code>. It defaults to <code>at-most-once</code>. This setting is understood only by quorum queues. If <code>at-least-once</code> is set, <code>Overflow behaviour</code> must be set to <code>reject-publish</code>. Otherwise, dead letter strategy will fall back to <code>at-most-once</code>.',
'queue-single-active-consumer':
'If set, makes sure only one consumer at a time consumes from the queue and fails over to another registered consumer in case the active one is cancelled or dies.<br/>(Sets the "<a target="_blank" href="https://rabbitmq.com/consumers.html#single-active-consumer">x-single-active-consumer</a>" argument.)',
@ -246,11 +249,14 @@ var HELP = {
'Set the queue initial cluster size.',
'queue-type':
'Set the queue type, determining the type of queue to use: raft-based high availability or classic queue. Valid values are <code>quorum</code> or <code>classic</code>. It defaults to <code>classic<code>. <br/>',
'Set the queue type, determining the type of queue to use: raft-based high availability or classic queue. Valid values are <code>quorum</code> or <code>classic</code>. It defaults to <code>classic</code>. <br/>',
'queue-messages':
'<p>Message counts.</p><p>Note that "in memory" and "persistent" are not mutually exclusive; persistent messages can be in memory as well as on disc, and transient messages can be paged out if memory is tight. Non-durable queues will consider all messages to be transient.</p>',
'queue-dead-lettered':
'Applies to messages dead-lettered with dead-letter-strategy <code>at-least-once</code>.',
'queue-message-body-bytes':
'<p>The sum total of the sizes of the message bodies in this queue. This only counts message bodies; it does not include message properties (including headers) or metadata used by the queue.</p><p>Note that "in memory" and "persistent" are not mutually exclusive; persistent messages can be in memory as well as on disc, and transient messages can be paged out if memory is tight. Non-durable queues will consider all messages to be transient.</p><p>If a message is routed to multiple queues on publication, its body will be stored only once (in memory and on disk) and shared between queues. The value shown here does not take account of this effect.</p>',

View File

@ -9,6 +9,7 @@
var sections = {'queue_procs' : ['classic', 'Classic queues (masters)'],
'queue_slave_procs' : ['classic', 'Classic queues (mirrors)'],
'quorum_queue_procs' : ['quorum', 'Quorum queues'],
'quorum_queue_dlx_procs' : ['quorum', 'Dead letter workers'],
'stream_queue_procs' : ['stream', 'Stream queues'],
'stream_queue_replica_reader_procs' : ['stream', 'Stream queues (replica reader)'],
'stream_queue_coordinator_procs' : ['stream', 'Stream queues (coordinator)'],
@ -26,12 +27,12 @@
<span class="clear">&nbsp;</span>
<div class="box">
<%
var key = [[{name: 'Classic Queues', colour: 'classic',
keys: [['queue_procs', 'queues'],
['queue_slave_procs', 'mirrors']]},
{name: 'Quorum Queues', colour: 'quorum',
keys: [['quorum_queue_procs', 'quorum']]},
keys: [['quorum_queue_procs', 'quorum'],
['quorum_queue_dlx_procs', 'dead letter workers']]},
{name: 'Streams', colour: 'stream',
keys: [['stream_queue_procs', 'stream'],
['stream_queue_replica_reader_procs', 'stream replica reader'],

View File

@ -9,6 +9,7 @@
var sections = {'queue_procs' : ['classic', 'Classic queues (masters)'],
'queue_slave_procs' : ['classic', 'Classic queues (mirrors)'],
'quorum_queue_procs' : ['quorum', 'Quorum queues'],
'quorum_queue_dlx_procs' : ['quorum', 'Dead letter workers'],
'stream_queue_procs' : ['stream', 'Stream queues'],
'stream_queue_replica_reader_procs' : ['stream', 'Stream queues (replica reader)'],
'stream_queue_coordinator_procs' : ['stream', 'Stream queues (coordinator)'],
@ -38,7 +39,8 @@ var key = [[{name: 'Classic Queues', colour: 'classic',
keys: [['queue_procs', 'queues'],
['queue_slave_procs', 'mirrors']]},
{name: 'Quorum Queues', colour: 'quorum',
keys: [['quorum_queue_procs','quorum']]},
keys: [['quorum_queue_procs','quorum'],
['quorum_queue_dlx_procs', 'dead letter workers']]},
{name: 'Streams', colour: 'stream',
keys: [['stream_queue_procs', 'stream'],
['stream_queue_replica_reader_procs', 'stream replica reader'],

View File

@ -103,7 +103,8 @@
<span class="argument-link" field="definition" key="overflow" type="string">Overflow behaviour</span> <span class="help" id="queue-overflow"></span> |
<span class="argument-link" field="definition" key="expires" type="number">Auto expire</span> </br>
<span class="argument-link" field="definition" key="dead-letter-exchange" type="string">Dead letter exchange</span> |
<span class="argument-link" field="definition" key="dead-letter-routing-key" type="string">Dead letter routing key</span><br />
<span class="argument-link" field="definition" key="dead-letter-routing-key" type="string">Dead letter routing key</span><br/>
<span class="argument-link" field="definition" key="message-ttl" type="number">Message TTL</span><span class="help" id="queue-message-ttl"></span></br>
</td>
<tr>
<td>Queues [Classic]</td>
@ -114,7 +115,6 @@
<span class="argument-link" field="definition" key="ha-promote-on-shutdown" type="string" value="">HA mirror promotion on shutdown</span> <span class="help" id="policy-ha-promote-on-shutdown"></span> |
<span class="argument-link" field="definition" key="ha-promote-on-failure" type="string" value="">HA mirror promotion on failure</span> <span class="help" id="policy-ha-promote-on-failure"></span>
</br>
<span class="argument-link" field="definition" key="message-ttl" type="number">Message TTL</span> |
<span class="argument-link" field="definition" key="queue-mode" type="string" value="lazy">Lazy mode</span> |
<span class="argument-link" field="definition" key="queue-version" type="number">Version</span> <span class="help" id="queue-version"></span> |
<span class="argument-link" field="definition" key="queue-master-locator" type="string">Master Locator</span></br>
@ -128,7 +128,9 @@
<span class="argument-link" field="definition" key="max-in-memory-bytes" type="number">Max in memory bytes</span>
<span class="help" id="queue-max-in-memory-bytes"></span> |
<span class="argument-link" field="definition" key="delivery-limit" type="number">Delivery limit</span>
<span class="help" id="delivery-limit"></span>
<span class="help" id="delivery-limit"></span></br>
<span class="argument-link" field="definition" key="dead-letter-strategy" type="string">Dead letter strategy</span>
<span class="help" id="queue-dead-letter-strategy"></span>
</td>
</tr>
<tr>
@ -271,13 +273,14 @@
<span class="argument-link" field="definitionop" key="max-length" type="number">Max length</span> |
<span class="argument-link" field="definitionop" key="max-length-bytes" type="number">Max length bytes</span> |
<span class="argument-link" field="definitionop" key="overflow" type="string">Overflow behaviour</span>
<span class="help" id="queue-overflow"></span>
<span class="help" id="queue-overflow"></span></br>
<span class="argument-link" field="definitionop" key="message-ttl" type="number">Message TTL</span>
<span class="help" id="queue-message-ttl"></span>
</td>
</tr>
<tr>
<td>Queues [Classic]</td>
<td>
<span class="argument-link" field="definitionop" key="message-ttl" type="number">Message TTL</span> |
<span class="argument-link" field="definitionop" key="expires" type="number">Auto expire</span>
</td>
</tr>

View File

@ -167,6 +167,9 @@
<th class="horizontal">Unacked</th>
<% if (is_quorum(queue)) { %>
<th class="horizontal">In memory ready</th>
<th class="horizontal">Dead-lettered
<span class="help" id="queue-dead-lettered"></span>
</th>
<% } %>
<% if (is_classic(queue)) { %>
<th class="horizontal">In memory</th>
@ -192,6 +195,9 @@
<td class="r">
<%= fmt_num_thousands(queue.messages_ram) %>
</td>
<td class="r">
<%= fmt_num_thousands(queue.messages_dlx) %>
</td>
<% } %>
<% if (is_classic(queue)) { %>
<td class="r">
@ -224,6 +230,11 @@
<%= fmt_bytes(queue.message_bytes_ram) %>
</td>
<% } %>
<% if (is_quorum(queue)) { %>
<td class="r">
<%= fmt_bytes(queue.message_bytes_dlx) %>
</td>
<% } %>
<% if (is_classic(queue)) { %>
<td class="r">
<%= fmt_bytes(queue.message_bytes_persistent) %>

View File

@ -312,13 +312,11 @@
<tr>
<td>Add</td>
<td>
<% if (queue_type == "classic") { %>
<span class="argument-link" field="arguments" key="x-message-ttl" type="number">Message TTL</span> <span class="help" id="queue-message-ttl"></span> |
<% } %>
<% if (queue_type != "stream") { %>
<span class="argument-link" field="arguments" key="x-expires" type="number">Auto expire</span> <span class="help" id="queue-expires"></span> |
<span class="argument-link" field="arguments" key="x-overflow" type="string">Overflow behaviour</span> <span class="help" id="queue-overflow"></span> |
<span class="argument-link" field="arguments" key="x-single-active-consumer" type="boolean">Single active consumer</span> <span class="help" id="queue-single-active-consumer"></span><br/>
<span class="argument-link" field="arguments" key="x-message-ttl" type="number">Message TTL</span> <span class="help" id="queue-message-ttl"></span> |
<span class="argument-link" field="arguments" key="x-overflow" type="string">Overflow behaviour</span> <span class="help" id="queue-overflow"></span><br/>
<span class="argument-link" field="arguments" key="x-single-active-consumer" type="boolean">Single active consumer</span> <span class="help" id="queue-single-active-consumer"></span> |
<span class="argument-link" field="arguments" key="x-dead-letter-exchange" type="string">Dead letter exchange</span> <span class="help" id="queue-dead-letter-exchange"></span> |
<span class="argument-link" field="arguments" key="x-dead-letter-routing-key" type="string">Dead letter routing key</span> <span class="help" id="queue-dead-letter-routing-key"></span><br/>
<span class="argument-link" field="arguments" key="x-max-length" type="number">Max length</span> <span class="help" id="queue-max-length"></span> |
@ -334,7 +332,8 @@
<span class="argument-link" field="arguments" key="x-delivery-limit" type="number">Delivery limit</span><span class="help" id="delivery-limit"></span>
| <span class="argument-link" field="arguments" key="x-max-in-memory-length" type="number">Max in memory length</span><span class="help" id="queue-max-in-memory-length"></span>
| <span class="argument-link" field="arguments" key="x-max-in-memory-bytes" type="number">Max in memory bytes</span><span class="help" id="queue-max-in-memory-bytes"></span>
| <span class="argument-link" field="arguments" key="x-quorum-initial-group-size" type="number">Initial cluster size</span><span class="help" id="queue-quorum-initial-group-size"></span><br/>
| <span class="argument-link" field="arguments" key="x-quorum-initial-group-size" type="number">Initial cluster size</span><span class="help" id="queue-initial-cluster-size"></span><br/>
<span class="argument-link" field="arguments" key="x-dead-letter-strategy" type="string">Dead letter strategy</span><span class="help" id="queue-dead-letter-strategy"></span><br/>
<% } %>
<% if (queue_type == "stream") { %>
<span class="argument-link" field="arguments" key="x-max-age" type="string">Max time retention</span><span class="help" id="queue-max-age"></span>

View File

@ -67,6 +67,34 @@ To generate these:
| rabbitmq_global_publishers | Publishers currently connected |
| rabbitmq_global_consumers | Consumers currently connected |
#### Dead letter global counters
| Metric | Description |
| --- | --- |
| rabbitmq_global_messages_dead_lettered_confirmed_total | Total number of messages dead-lettered and confirmed by target queues |
| rabbitmq_global_messages_dead_lettered_delivery_limit_total | Total number of messages dead-lettered due to delivery-limit exceeded |
| rabbitmq_global_messages_dead_lettered_expired_total | Total number of messages dead-lettered due to message TTL exceeded |
| rabbitmq_global_messages_dead_lettered_maxlen_total | Total number of messages dead-lettered due to overflow drop-head or reject-publish-dlx |
| rabbitmq_global_messages_dead_lettered_rejected_total | Total number of messages dead-lettered due to basic.reject or basic.nack |
Metrics `rabbitmq_global_messages_dead_lettered_*` have labels `queue_type` and `dead_letter_strategy`.
Label `queue_type` denotes the type of queue messages were discarded from. It can have value
* `rabbit_classic_queue`, or
* `rabbit_quorum_queue`
(Queue type `rabbit_stream_queue` does not dead letter messages.)
Note that metrics `rabbitmq_global_messages_dead_lettered_*` with label `queue_type` set to `rabbit_quorum_queue`
might miss some counter updates in certain failure scenarios, i.e. the reported Prometheus value could be
slightly lower than the actual number of messages dead lettered (and confirmed).
(This is because in the current implementation quorum queue leaders update the counters asynchronously.)
Label `dead_letter_strategy` can have value
* `disabled` if queue has no dead-letter-exchange configured or if configured dead-letter-exchange does not exist implying messages get dropped, or
* `at_most_once` if queue's configured dead-lettered-exchange exists, or
* `at_least_once` if queue type is `rabbit_quorum_queue` with configured `dead-letter-exchange` and `dead-letter-strategy` set to `at-least-once` and `overflow` set to `reject-publish`.
#### Stream global counters
These metrics are specific to the stream protocol.

View File

@ -3,23 +3,25 @@
# otp_path: "/path/to/otp/lib/erlang"
deps_dirs:
- "deps/*"
- "deps/rabbit/apps/*"
diagnostics:
# disabled:
# - bound_var_in_pattern
disabled:
- bound_var_in_pattern
enabled:
- crossref
- dialyzer
# - elvis
- compiler
# - elvis
include_dirs:
- "deps"
- "deps/*/include"
# lenses:
# enabled:
# - ct-run-test
# - show-behaviour-usages
# disabled: []
lenses:
enabled:
- ct-run-test
- show-behaviour-usages
- suggest-spec
- function-references
disabled: []
# macros:
# - name: DEFINED_WITH_VALUE
# value: 42