Skip to content

Commit 12bf3e0

Browse files
committed
Fix force_checkpoint tests and CLI command
1 parent b54ab1d commit 12bf3e0

File tree

5 files changed

+39
-76
lines changed

5 files changed

+39
-76
lines changed

deps/rabbit/src/rabbit_fifo.erl

Lines changed: 0 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -925,26 +925,6 @@ which_module(3) -> rabbit_fifo_v3;
925925
which_module(4) -> ?MODULE;
926926
which_module(5) -> ?MODULE.
927927

928-
-define(AUX, aux_v3).
929-
930-
-record(checkpoint, {index :: ra:index(),
931-
timestamp :: milliseconds(),
932-
smallest_index :: undefined | ra:index(),
933-
messages_total :: non_neg_integer(),
934-
indexes = ?CHECK_MIN_INDEXES :: non_neg_integer(),
935-
unused_1 = ?NIL}).
936-
-record(aux_gc, {last_raft_idx = 0 :: ra:index()}).
937-
-record(aux, {name :: atom(),
938-
capacity :: term(),
939-
gc = #aux_gc{} :: #aux_gc{}}).
940-
-record(?AUX, {name :: atom(),
941-
last_decorators_state :: term(),
942-
capacity :: term(),
943-
gc = #aux_gc{} :: #aux_gc{},
944-
tick_pid :: undefined | pid(),
945-
cache = #{} :: map(),
946-
last_checkpoint :: #checkpoint{}}).
947-
948928
init_aux(Name) when is_atom(Name) ->
949929
%% TODO: catch specific exception throw if table already exists
950930
ok = ra_machine_ets:create_table(rabbit_fifo_usage,

deps/rabbit/src/rabbit_fifo.hrl

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -227,3 +227,23 @@
227227
msg_ttl => non_neg_integer(),
228228
created => non_neg_integer()
229229
}.
230+
231+
-define(AUX, aux_v3).
232+
233+
-record(checkpoint, {index :: ra:index(),
234+
timestamp :: milliseconds(),
235+
smallest_index :: undefined | ra:index(),
236+
messages_total :: non_neg_integer(),
237+
indexes = ?CHECK_MIN_INDEXES :: non_neg_integer(),
238+
unused_1 = ?NIL}).
239+
-record(aux_gc, {last_raft_idx = 0 :: ra:index()}).
240+
-record(aux, {name :: atom(),
241+
capacity :: term(),
242+
gc = #aux_gc{} :: #aux_gc{}}).
243+
-record(?AUX, {name :: atom(),
244+
last_decorators_state :: term(),
245+
capacity :: term(),
246+
gc = #aux_gc{} :: #aux_gc{},
247+
tick_pid :: undefined | pid(),
248+
cache = #{} :: map(),
249+
last_checkpoint :: #checkpoint{}}).

deps/rabbit/src/rabbit_quorum_queue.erl

Lines changed: 4 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -2089,13 +2089,13 @@ force_all_queues_shrink_member_to_current_member(ListQQFun) when is_function(Lis
20892089
force_checkpoint_on_queue(QName) ->
20902090
Node = node(),
20912091
QNameFmt = rabbit_misc:rs(QName),
2092-
case rabbit_amqqueue:lookup(QName) of
2092+
case rabbit_db_queue:get_durable(QName) of
20932093
{ok, Q} when ?amqqueue_is_classic(Q) ->
20942094
{error, classic_queue_not_supported};
20952095
{ok, Q} when ?amqqueue_is_quorum(Q) ->
20962096
{RaName, _} = amqqueue:get_pid(Q),
2097-
rpc:call(Node, ra, cast_aux_command, [{RaName, Node}, force_checkpoint]),
2098-
rabbit_log:debug("Sent command to force checkpoint ~ts", [QNameFmt]);
2097+
rabbit_log:debug("Sending command to force ~ts to take a checkpoint", [QNameFmt]),
2098+
rpc:call(Node, ra, cast_aux_command, [{RaName, Node}, force_checkpoint]);
20992099
{ok, _Q} ->
21002100
{error, not_quorum_queue};
21012101
{error, _} = E ->
@@ -2114,8 +2114,7 @@ force_checkpoint(VhostSpec, QueueSpec) ->
21142114
{QName, {error, Err}}
21152115
end
21162116
end
2117-
|| Q <- rabbit_amqqueue:list(),
2118-
amqqueue:get_type(Q) == ?MODULE,
2117+
|| Q <- rabbit_db_queue:get_all_durable_by_type(?MODULE),
21192118
is_match(amqqueue:get_vhost(Q), VhostSpec)
21202119
andalso is_match(get_resource_name(amqqueue:get_name(Q)), QueueSpec)].
21212120

@@ -2179,4 +2178,3 @@ file_handle_other_reservation() ->
21792178

21802179
file_handle_release_reservation() ->
21812180
ok.
2182-

deps/rabbit/test/quorum_queue_SUITE.erl

Lines changed: 13 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@
1010
-include_lib("eunit/include/eunit.hrl").
1111
-include_lib("amqp_client/include/amqp_client.hrl").
1212
-include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl").
13+
-include_lib("rabbit/src/rabbit_fifo.hrl").
1314

1415
-import(queue_utils, [wait_for_messages_ready/3,
1516
wait_for_messages_pending_ack/3,
@@ -1327,12 +1328,9 @@ force_checkpoint_on_queue(Config) ->
13271328
% Wait for initial checkpoint and make sure it's 0; checkpoint hasn't been triggered yet.
13281329
rabbit_ct_helpers:await_condition(
13291330
fun() ->
1330-
{ok, #{aux := Aux1}, _} = rpc:call(Server0, ra, member_overview, [{RaName, Server0}]),
1331-
{aux_v3, _, _, _, _, _, _, {checkpoint, Index, _, _, _, _, _}} = Aux1,
1332-
case Index of
1333-
0 -> true;
1334-
_ -> false
1335-
end
1331+
{ok, #{aux := Aux}, _} = rpc:call(Server0, ra, member_overview, [{RaName, Server0}]),
1332+
#aux_v3{last_checkpoint = #checkpoint{index = Index}} = Aux,
1333+
Index =:= 0
13361334
end),
13371335

13381336
rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue,
@@ -1341,19 +1339,17 @@ force_checkpoint_on_queue(Config) ->
13411339
% Wait for initial checkpoint and make sure it's not 0
13421340
rabbit_ct_helpers:await_condition(
13431341
fun() ->
1344-
{ok, #{aux := Aux1}, _} = rpc:call(Server0, ra, member_overview, [{RaName, Server0}]),
1345-
{aux_v3, _, _, _, _, _, _, {checkpoint, Index, _, _, _, _, _}} = Aux1,
1346-
case Index of
1347-
0 -> false;
1348-
_ -> true
1349-
end
1342+
{ok, #{aux := Aux}, _} = rpc:call(Server0, ra, member_overview, [{RaName, Server0}]),
1343+
#aux_v3{last_checkpoint = #checkpoint{index = Index}} = Aux,
1344+
Index =/= 0
13501345
end).
13511346

13521347
force_checkpoint(Config) ->
13531348
[Server0, _Server1, _Server2] =
13541349
rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
13551350
Ch = rabbit_ct_client_helpers:open_channel(Config, Server0),
13561351
QQ = ?config(queue_name, Config),
1352+
QQName = rabbit_misc:r(<<"/">>, queue, QQ),
13571353
CQ = <<"force_checkpoint_cq">>,
13581354
RaName = ra_name(QQ),
13591355

@@ -1366,24 +1362,12 @@ force_checkpoint(Config) ->
13661362
rabbit_ct_client_helpers:publish(Ch, QQ, 3),
13671363
wait_for_messages_ready([Server0], RaName, 3),
13681364

1369-
meck:expect(rabbit_quorum_queue, force_checkpoint_on_queue, fun(Q) -> ok end),
1370-
1371-
rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue,
1365+
ForceCheckpointRes = rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_quorum_queue,
13721366
force_checkpoint, [<<".*">>, <<".*">>]),
1373-
1374-
% Waiting here to make sure checkpoint has been forced
1375-
rabbit_ct_helpers:await_condition(
1376-
fun() ->
1377-
{ok, #{aux := Aux1}, _} = rpc:call(Server0, ra, member_overview, [{RaName, Server0}]),
1378-
{aux_v3, _, _, _, _, _, _, {checkpoint, Index, _, _, _, _, _}} = Aux1,
1379-
case Index of
1380-
0 -> false;
1381-
_ -> true
1382-
end
1383-
end),
1384-
1385-
% Make sure force_checkpoint_on_queue was only called for the quorun queue
1386-
?assertEqual(1, meck:num_calls(rabbit_quorum_queue, force_checkpoint_on_queue, '_')).
1367+
ExpectedRes = [{QQName, {ok}}],
1368+
1369+
% Result should only have quorum queue
1370+
?assertEqual(ExpectedRes, ForceCheckpointRes).
13871371

13881372
% Tests that, if the process of a QQ is dead in the moment of declaring a policy
13891373
% that affects such queue, when the process is made available again, the policy

deps/rabbitmq_cli/lib/rabbitmq/cli/queues/commands/force_checkpoint_command.ex

Lines changed: 2 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -35,9 +35,6 @@ defmodule RabbitMQ.CLI.Queues.Commands.ForceCheckpointCommand do
3535
args = [vhost_pat, queue_pat]
3636

3737
case :rabbit_misc.rpc_call(node_name, :rabbit_quorum_queue, :force_checkpoint, args) do
38-
{:error, _} = error ->
39-
error
40-
4138
{:badrpc, _} = error ->
4239
error
4340

@@ -46,15 +43,15 @@ defmodule RabbitMQ.CLI.Queues.Commands.ForceCheckpointCommand do
4643
do: [
4744
{:vhost, vhost},
4845
{:name, name},
49-
{:result, format_result(res)}
46+
{:result, res}
5047
]
5148

5249
results ->
5350
for {{:resource, vhost, _kind, name}, res} <- results,
5451
do: [
5552
{:vhost, vhost},
5653
{:name, name},
57-
{:result, format_result(res)}
54+
{:result, res}
5855
]
5956
end
6057
end
@@ -88,20 +85,4 @@ defmodule RabbitMQ.CLI.Queues.Commands.ForceCheckpointCommand do
8885
def banner([], _) do
8986
"Forcing checkpoint for all matching quorum queues..."
9087
end
91-
92-
#
93-
# Implementation
94-
#
95-
96-
defp format_result({:ok}) do
97-
"ok"
98-
end
99-
100-
defp format_result({:error, :timeout}) do
101-
"error: the operation timed out and may not have been completed"
102-
end
103-
104-
defp format_result({:error, err}) do
105-
to_string(:io_lib.format("error: ~W", [err, 10]))
106-
end
10788
end

0 commit comments

Comments
 (0)