Skip to content

Commit 215f218

Browse files
Merge pull request #12641 from rabbitmq/rabbitmq-server-12640-for-v4.0.x
Backport of #12640 to v4.0.x
2 parents ae05ff2 + 7078a02 commit 215f218

File tree

3 files changed

+237
-22
lines changed

3 files changed

+237
-22
lines changed

deps/rabbit/src/rabbit_fifo.erl

+2
Original file line numberDiff line numberDiff line change
@@ -853,6 +853,8 @@ overview(#?STATE{consumers = Cons,
853853
Conf = #{name => Cfg#cfg.name,
854854
resource => Cfg#cfg.resource,
855855
dead_lettering_enabled => undefined =/= Cfg#cfg.dead_letter_handler,
856+
dead_letter_handler => Cfg#cfg.dead_letter_handler,
857+
overflow_strategy => Cfg#cfg.overflow_strategy,
856858
max_length => Cfg#cfg.max_length,
857859
max_bytes => Cfg#cfg.max_bytes,
858860
consumer_strategy => Cfg#cfg.consumer_strategy,

deps/rabbit/src/rabbit_quorum_queue.erl

+42-11
Original file line numberDiff line numberDiff line change
@@ -316,9 +316,8 @@ declare_queue_error(Error, Queue, Leader, ActingUser) ->
316316
ra_machine(Q) ->
317317
{module, rabbit_fifo, ra_machine_config(Q)}.
318318

319-
ra_machine_config(Q) when ?is_amqqueue(Q) ->
319+
gather_policy_config(Q, IsQueueDeclaration) ->
320320
QName = amqqueue:get_name(Q),
321-
{Name, _} = amqqueue:get_pid(Q),
322321
%% take the minimum value of the policy and the queue arg if present
323322
MaxLength = args_policy_lookup(<<"max-length">>, fun min/2, Q),
324323
OverflowBin = args_policy_lookup(<<"overflow">>, fun policy_has_precedence/2, Q),
@@ -327,28 +326,42 @@ ra_machine_config(Q) when ?is_amqqueue(Q) ->
327326
DeliveryLimit = case args_policy_lookup(<<"delivery-limit">>,
328327
fun resolve_delivery_limit/2, Q) of
329328
undefined ->
330-
rabbit_log:info("~ts: delivery_limit not set, defaulting to ~b",
331-
[rabbit_misc:rs(QName), ?DEFAULT_DELIVERY_LIMIT]),
329+
case IsQueueDeclaration of
330+
true ->
331+
rabbit_log:info(
332+
"~ts: delivery_limit not set, defaulting to ~b",
333+
[rabbit_misc:rs(QName), ?DEFAULT_DELIVERY_LIMIT]);
334+
false ->
335+
ok
336+
end,
332337
?DEFAULT_DELIVERY_LIMIT;
333338
DL ->
334339
DL
335340
end,
336341
Expires = args_policy_lookup(<<"expires">>, fun min/2, Q),
337342
MsgTTL = args_policy_lookup(<<"message-ttl">>, fun min/2, Q),
338-
#{name => Name,
339-
queue_resource => QName,
340-
dead_letter_handler => dead_letter_handler(Q, Overflow),
341-
become_leader_handler => {?MODULE, become_leader, [QName]},
343+
DeadLetterHandler = dead_letter_handler(Q, Overflow),
344+
#{dead_letter_handler => DeadLetterHandler,
342345
max_length => MaxLength,
343346
max_bytes => MaxBytes,
344-
single_active_consumer_on => single_active_consumer_on(Q),
345347
delivery_limit => DeliveryLimit,
346348
overflow_strategy => Overflow,
347-
created => erlang:system_time(millisecond),
348349
expires => Expires,
349350
msg_ttl => MsgTTL
350351
}.
351352

353+
ra_machine_config(Q) when ?is_amqqueue(Q) ->
354+
PolicyConfig = gather_policy_config(Q, true),
355+
QName = amqqueue:get_name(Q),
356+
{Name, _} = amqqueue:get_pid(Q),
357+
PolicyConfig#{
358+
name => Name,
359+
queue_resource => QName,
360+
become_leader_handler => {?MODULE, become_leader, [QName]},
361+
single_active_consumer_on => single_active_consumer_on(Q),
362+
created => erlang:system_time(millisecond)
363+
}.
364+
352365
resolve_delivery_limit(PolVal, ArgVal)
353366
when PolVal < 0 orelse ArgVal < 0 ->
354367
max(PolVal, ArgVal);
@@ -624,7 +637,9 @@ handle_tick(QName,
624637
ok;
625638
_ ->
626639
ok
627-
end
640+
end,
641+
maybe_apply_policies(Q, Overview),
642+
ok
628643
catch
629644
_:Err ->
630645
rabbit_log:debug("~ts: handle tick failed with ~p",
@@ -708,6 +723,21 @@ system_recover(quorum_queues) ->
708723
ok
709724
end.
710725

726+
maybe_apply_policies(Q, #{config := CurrentConfig}) ->
727+
NewPolicyConfig = gather_policy_config(Q, false),
728+
729+
RelevantKeys = maps:keys(NewPolicyConfig),
730+
CurrentPolicyConfig = maps:with(RelevantKeys, CurrentConfig),
731+
732+
ShouldUpdate = NewPolicyConfig =/= CurrentPolicyConfig,
733+
case ShouldUpdate of
734+
true ->
735+
rabbit_log:debug("Re-applying policies to ~ts", [rabbit_misc:rs(amqqueue:get_name(Q))]),
736+
policy_changed(Q),
737+
ok;
738+
false -> ok
739+
end.
740+
711741
-spec recover(binary(), [amqqueue:amqqueue()]) ->
712742
{[amqqueue:amqqueue()], [amqqueue:amqqueue()]}.
713743
recover(_Vhost, Queues) ->
@@ -2064,3 +2094,4 @@ file_handle_other_reservation() ->
20642094

20652095
file_handle_release_reservation() ->
20662096
ok.
2097+

deps/rabbit/test/quorum_queue_SUITE.erl

+193-11
Original file line numberDiff line numberDiff line change
@@ -96,6 +96,7 @@ groups() ->
9696
force_shrink_member_to_current_member,
9797
force_all_queues_shrink_member_to_current_member,
9898
force_vhost_queues_shrink_member_to_current_member,
99+
policy_repair,
99100
gh_12635
100101
]
101102
++ all_tests()},
@@ -1303,20 +1304,175 @@ force_vhost_queues_shrink_member_to_current_member(Config) ->
13031304
?assertEqual(3, length(Nodes0))
13041305
end || Q <- QQs, VHost <- VHosts].
13051306

1307+
1308+
% Tests that, if the process of a QQ is dead in the moment of declaring a policy
1309+
% that affects such queue, when the process is made available again, the policy
1310+
% will eventually get applied. (https://github.com/rabbitmq/rabbitmq-server/issues/7863)
1311+
policy_repair(Config) ->
1312+
[Server0, _Server1, _Server2] = Servers =
1313+
rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
1314+
Ch = rabbit_ct_client_helpers:open_channel(Config, Server0),
1315+
#'confirm.select_ok'{} = amqp_channel:call(Ch, #'confirm.select'{}),
1316+
1317+
QQ = ?config(queue_name, Config),
1318+
?assertEqual({'queue.declare_ok', QQ, 0, 0},
1319+
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])),
1320+
RaName = ra_name(QQ),
1321+
ExpectedMaxLength1 = 10,
1322+
Priority1 = 1,
1323+
ok = rabbit_ct_broker_helpers:rpc(
1324+
Config,
1325+
0,
1326+
rabbit_policy,
1327+
set,
1328+
[
1329+
<<"/">>,
1330+
<<QQ/binary, "_1">>,
1331+
QQ,
1332+
[{<<"max-length">>, ExpectedMaxLength1}, {<<"overflow">>, <<"reject-publish">>}],
1333+
Priority1,
1334+
<<"quorum_queues">>,
1335+
<<"acting-user">>
1336+
]),
1337+
1338+
% Wait for the policy to apply
1339+
QueryFun = fun rabbit_fifo:overview/1,
1340+
?awaitMatch({ok, {_, #{config := #{max_length := ExpectedMaxLength1}}}, _},
1341+
rpc:call(Server0, ra, local_query, [RaName, QueryFun]),
1342+
?DEFAULT_AWAIT),
1343+
1344+
% Check the policy has been applied
1345+
% Insert MaxLength1 + some messages but after consuming all messages only
1346+
% MaxLength1 are retrieved.
1347+
% Checking twice to ensure consistency
1348+
publish_confirm_many(Ch, QQ, ExpectedMaxLength1 + 1),
1349+
% +1 because QQs let one pass
1350+
wait_for_messages_ready(Servers, RaName, ExpectedMaxLength1 + 1),
1351+
fail = publish_confirm(Ch, QQ),
1352+
fail = publish_confirm(Ch, QQ),
1353+
consume_all(Ch, QQ),
1354+
1355+
% Set higher priority policy, allowing more messages
1356+
ExpectedMaxLength2 = 20,
1357+
Priority2 = 2,
1358+
ok = rabbit_ct_broker_helpers:rpc(
1359+
Config,
1360+
0,
1361+
rabbit_policy,
1362+
set,
1363+
[
1364+
<<"/">>,
1365+
<<QQ/binary, "_2">>,
1366+
QQ,
1367+
[{<<"max-length">>, ExpectedMaxLength2}, {<<"overflow">>, <<"reject-publish">>}],
1368+
Priority2,
1369+
<<"quorum_queues">>,
1370+
<<"acting-user">>
1371+
]),
1372+
1373+
% Wait for the policy to apply
1374+
?awaitMatch({ok, {_, #{config := #{max_length := ExpectedMaxLength2}}}, _},
1375+
rpc:call(Server0, ra, local_query, [RaName, QueryFun]),
1376+
?DEFAULT_AWAIT),
1377+
1378+
% Check the policy has been applied
1379+
% Insert MaxLength2 + some messages but after consuming all messages only
1380+
% MaxLength2 are retrieved.
1381+
% Checking twice to ensure consistency.
1382+
% + 1 because QQs let one pass
1383+
publish_confirm_many(Ch, QQ, ExpectedMaxLength2 + 1),
1384+
wait_for_messages_ready(Servers, RaName, ExpectedMaxLength2 + 1),
1385+
fail = publish_confirm(Ch, QQ),
1386+
fail = publish_confirm(Ch, QQ),
1387+
consume_all(Ch, QQ),
1388+
1389+
% Ensure the queue process is unavailable
1390+
lists:foreach(fun(Srv) -> ensure_qq_proc_dead(Config, Srv, RaName) end, Servers),
1391+
1392+
% Add policy with higher priority, allowing even more messages.
1393+
ExpectedMaxLength3 = 30,
1394+
Priority3 = 3,
1395+
ok = rabbit_ct_broker_helpers:rpc(
1396+
Config,
1397+
0,
1398+
rabbit_policy,
1399+
set,
1400+
[
1401+
<<"/">>,
1402+
<<QQ/binary, "_3">>,
1403+
QQ,
1404+
[{<<"max-length">>, ExpectedMaxLength3}, {<<"overflow">>, <<"reject-publish">>}],
1405+
Priority3,
1406+
<<"quorum_queues">>,
1407+
<<"acting-user">>
1408+
]),
1409+
1410+
% Restart the queue process.
1411+
{ok, Queue} =
1412+
rabbit_ct_broker_helpers:rpc(
1413+
Config,
1414+
0,
1415+
rabbit_amqqueue,
1416+
lookup,
1417+
[{resource, <<"/">>, queue, QQ}]),
1418+
lists:foreach(
1419+
fun(Srv) ->
1420+
rabbit_ct_broker_helpers:rpc(
1421+
Config,
1422+
Srv,
1423+
rabbit_quorum_queue,
1424+
recover,
1425+
[foo, [Queue]]
1426+
)
1427+
end,
1428+
Servers),
1429+
1430+
% Wait for the queue to be available again.
1431+
lists:foreach(fun(Srv) ->
1432+
rabbit_ct_helpers:await_condition(
1433+
fun () ->
1434+
is_pid(
1435+
rabbit_ct_broker_helpers:rpc(
1436+
Config,
1437+
Srv,
1438+
erlang,
1439+
whereis,
1440+
[RaName]))
1441+
end)
1442+
end,
1443+
Servers),
1444+
1445+
% Wait for the policy to apply
1446+
?awaitMatch({ok, {_, #{config := #{max_length := ExpectedMaxLength3}}}, _},
1447+
rpc:call(Server0, ra, local_query, [RaName, QueryFun]),
1448+
?DEFAULT_AWAIT),
1449+
1450+
% Check the policy has been applied
1451+
% Insert MaxLength3 + some messages but after consuming all messages only
1452+
% MaxLength3 are retrieved.
1453+
% Checking twice to ensure consistency.
1454+
% + 1 because QQs let one pass
1455+
publish_confirm_many(Ch, QQ, ExpectedMaxLength3 + 1),
1456+
wait_for_messages_ready(Servers, RaName, ExpectedMaxLength3 + 1),
1457+
fail = publish_confirm(Ch, QQ),
1458+
fail = publish_confirm(Ch, QQ),
1459+
consume_all(Ch, QQ).
1460+
1461+
13061462
gh_12635(Config) ->
13071463
% https://github.com/rabbitmq/rabbitmq-server/issues/12635
13081464
[Server0, _Server1, Server2] =
13091465
rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
13101466

13111467
ok = rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,
1312-
[rabbit, quorum_min_checkpoint_interval, 1]),
1468+
[rabbit, quorum_min_checkpoint_interval, 1]),
13131469

13141470
Ch0 = rabbit_ct_client_helpers:open_channel(Config, Server0),
13151471
#'confirm.select_ok'{} = amqp_channel:call(Ch0, #'confirm.select'{}),
13161472
QQ = ?config(queue_name, Config),
13171473
RaName = ra_name(QQ),
13181474
?assertEqual({'queue.declare_ok', QQ, 0, 0},
1319-
declare(Ch0, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])),
1475+
declare(Ch0, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])),
13201476

13211477
%% stop member to simulate slow or down member
13221478
ok = rpc:call(Server2, ra, stop_server, [quorum_queues, {RaName, Server2}]),
@@ -1327,10 +1483,10 @@ gh_12635(Config) ->
13271483
%% force a checkpoint on leader
13281484
ok = rpc:call(Server0, ra, cast_aux_command, [{RaName, Server0}, force_checkpoint]),
13291485
rabbit_ct_helpers:await_condition(
1330-
fun () ->
1331-
{ok, #{log := Log}, _} = rpc:call(Server0, ra, member_overview, [{RaName, Server0}]),
1332-
undefined =/= maps:get(latest_checkpoint_index, Log)
1333-
end),
1486+
fun () ->
1487+
{ok, #{log := Log}, _} = rpc:call(Server0, ra, member_overview, [{RaName, Server0}]),
1488+
undefined =/= maps:get(latest_checkpoint_index, Log)
1489+
end),
13341490

13351491
%% publish 1 more message
13361492
publish_confirm(Ch0, QQ),
@@ -1346,10 +1502,10 @@ gh_12635(Config) ->
13461502
#'queue.purge_ok'{} = amqp_channel:call(Ch0, #'queue.purge'{queue = QQ}),
13471503

13481504
rabbit_ct_helpers:await_condition(
1349-
fun () ->
1350-
{ok, #{log := Log}, _} = rpc:call(Server0, ra, member_overview, [{RaName, Server0}]),
1351-
undefined =/= maps:get(snapshot_index, Log)
1352-
end),
1505+
fun () ->
1506+
{ok, #{log := Log}, _} = rpc:call(Server0, ra, member_overview, [{RaName, Server0}]),
1507+
undefined =/= maps:get(snapshot_index, Log)
1508+
end),
13531509
%% restart the down member
13541510
ok = rpc:call(Server2, ra, restart_server, [quorum_queues, {RaName, Server2}]),
13551511
Pid2 = rpc:call(Server2, erlang, whereis, [RaName]),
@@ -1359,11 +1515,12 @@ gh_12635(Config) ->
13591515
{'DOWN',Ref, process,_, _} ->
13601516
ct:fail("unexpected DOWN")
13611517
after 500 ->
1362-
ok
1518+
ok
13631519
end,
13641520
flush(1),
13651521
ok.
13661522

1523+
13671524
priority_queue_fifo(Config) ->
13681525
%% testing: if hi priority messages are published before lo priority
13691526
%% messages they are always consumed first (fifo)
@@ -4397,3 +4554,28 @@ lists_interleave([Item | Items], List)
43974554
{Left, Right} = lists:split(2, List),
43984555
Left ++ [Item | lists_interleave(Items, Right)].
43994556

4557+
publish_confirm_many(Ch, Queue, Count) ->
4558+
lists:foreach(fun(_) -> publish_confirm(Ch, Queue) end, lists:seq(1, Count)).
4559+
4560+
consume_all(Ch, QQ) ->
4561+
Consume = fun C(Acc) ->
4562+
case amqp_channel:call(Ch, #'basic.get'{queue = QQ}) of
4563+
{#'basic.get_ok'{}, Msg} ->
4564+
C([Msg | Acc]);
4565+
_ ->
4566+
Acc
4567+
end
4568+
end,
4569+
Consume([]).
4570+
4571+
ensure_qq_proc_dead(Config, Server, RaName) ->
4572+
case rabbit_ct_broker_helpers:rpc(Config, Server, erlang, whereis, [RaName]) of
4573+
undefined ->
4574+
ok;
4575+
Pid ->
4576+
rabbit_ct_broker_helpers:rpc(Config, Server, erlang, exit, [Pid, kill]),
4577+
%% Give some time for the supervisor to restart the process
4578+
timer:sleep(500),
4579+
ensure_qq_proc_dead(Config, Server, RaName)
4580+
end.
4581+

0 commit comments

Comments
 (0)