Delete commented quorum queue in-memory tests

This commit is contained in:
David Ansari 2022-02-28 17:39:36 +01:00
parent fd2023a118
commit 2db128f683
11 changed files with 74 additions and 338 deletions

View File

@ -1,10 +1,9 @@
%% This Source Code Form is subject tconsumer_ido the terms of the Mozilla Public
%% 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.
%%
%%
%% before post gc 1M msg: 203MB, after recovery + gc: 203MB
-module(rabbit_fifo).

View File

@ -1,3 +1,9 @@
%% 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]).

View File

@ -695,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

View File

@ -1,3 +1,9 @@
%% 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").
@ -29,7 +35,7 @@
}).
-record(settle, {msg_ids :: [msg_id()]}).
-type protocol() :: {dlx, #checkout{} | #settle{}}.
-type state() :: #?MODULE{}.
-opaque state() :: #?MODULE{}.
-export_type([state/0,
protocol/0]).
@ -37,11 +43,13 @@
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}}.
@ -132,8 +140,7 @@ 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()) ->
-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,

View File

@ -1,3 +1,9 @@
%% 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(),

View File

@ -1,3 +1,9 @@
%% 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,

View File

@ -1,3 +1,9 @@
%% 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).

View File

@ -1,15 +1,25 @@
%% This module consumes from a single quroum queue's discards queue (containing dead-letttered messages)
%% and forwards the DLX messages at least once to every target queue.
%% 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 doesn't deal with AMQP clients.
%% 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 much easier to test, to debug, and to understand.
%% 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).
@ -55,22 +65,23 @@
}).
-record(state, {
%% There is one rabbit_fifo_dlx_worker per source quorum queue
%% (if dead-letter-strategy at-least-once is used).
%% source queue
queue_ref :: rabbit_amqqueue:name(),
%% monitors source queue
monitor_ref :: reference(),
%% configured (x-)dead-letter-exchange of source queue
exchange_ref,
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 publisher confirm delivery tag sequence number
%% 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,

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).
@ -122,15 +121,7 @@ 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,
@ -2270,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),
@ -2673,46 +2402,6 @@ per_message_ttl_mixed_expiry(Config) ->
end,
ok.
in_memory(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">>}])),
RaName = ra_name(QQ),
Msg1 = <<"msg1">>,
Msg2 = <<"msg11">>,
publish(Ch, QQ, 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)),
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{}} ->
amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DeliveryTag,
multiple = false})
end,
wait_for_messages(Config, [[QQ, <<"1">>, <<"0">>, <<"1">>]]),
?assertEqual([{0, 0}],
dirty_query([Server], RaName, fun rabbit_fifo:query_in_memory_usage/1)).
consumer_metrics(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),

View File

@ -1,9 +1,14 @@
%% 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("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
-include_lib("rabbit/src/rabbit_fifo.hrl").
-include_lib("rabbit/src/rabbit_fifo_dlx.hrl").

View File

@ -1,3 +1,9 @@
%% 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