Skip to content

Commit c14bf13

Browse files
Merge pull request #10472 from rabbitmq/rework-mirrored_supervisor-child-id
Rework `mirrored_supervisor` child ID format
2 parents ea2119e + 29f4858 commit c14bf13

21 files changed

+608
-399
lines changed

deps/rabbit/app.bzl

+1-4
Original file line numberDiff line numberDiff line change
@@ -169,7 +169,6 @@ def all_beam_files(name = "all_beam_files"):
169169
"src/rabbit_mirror_queue_slave.erl",
170170
"src/rabbit_mirror_queue_sync.erl",
171171
"src/rabbit_mnesia.erl",
172-
"src/rabbit_mnesia_to_khepri_record_converter.erl",
173172
"src/rabbit_msg_file.erl",
174173
"src/rabbit_msg_record.erl",
175174
"src/rabbit_msg_store.erl",
@@ -433,7 +432,6 @@ def all_test_beam_files(name = "all_test_beam_files"):
433432
"src/rabbit_mirror_queue_slave.erl",
434433
"src/rabbit_mirror_queue_sync.erl",
435434
"src/rabbit_mnesia.erl",
436-
"src/rabbit_mnesia_to_khepri_record_converter.erl",
437435
"src/rabbit_msg_file.erl",
438436
"src/rabbit_msg_record.erl",
439437
"src/rabbit_msg_store.erl",
@@ -543,7 +541,6 @@ def all_srcs(name = "all_srcs"):
543541
"include/gm_specs.hrl",
544542
"include/internal_user.hrl",
545543
"include/mc.hrl",
546-
"include/mirrored_supervisor.hrl",
547544
"include/rabbit_global_counters.hrl",
548545
"include/vhost.hrl",
549546
"include/vhost_v2.hrl",
@@ -557,6 +554,7 @@ def all_srcs(name = "all_srcs"):
557554
filegroup(
558555
name = "private_hdrs",
559556
srcs = [
557+
"src/mirrored_supervisor.hrl",
560558
"src/rabbit_feature_flags.hrl",
561559
"src/rabbit_fifo.hrl",
562560
"src/rabbit_fifo_dlx.hrl",
@@ -715,7 +713,6 @@ def all_srcs(name = "all_srcs"):
715713
"src/rabbit_mirror_queue_slave.erl",
716714
"src/rabbit_mirror_queue_sync.erl",
717715
"src/rabbit_mnesia.erl",
718-
"src/rabbit_mnesia_to_khepri_record_converter.erl",
719716
"src/rabbit_msg_file.erl",
720717
"src/rabbit_msg_record.erl",
721718
"src/rabbit_msg_store.erl",

deps/rabbit/src/mirrored_supervisor.erl

+5-1
Original file line numberDiff line numberDiff line change
@@ -137,7 +137,11 @@
137137
-type startlink_err() :: {'already_started', pid()} | 'shutdown' | term().
138138
-type startlink_ret() :: {'ok', pid()} | 'ignore' | {'error', startlink_err()}.
139139

140-
-type group_name() :: any().
140+
-type group_name() :: module().
141+
-type child_id() :: term(). %% supervisor:child_id() is not exported.
142+
143+
-export_type([group_name/0,
144+
child_id/0]).
141145

142146
-spec start_link(GroupName, Module, Args) -> startlink_ret() when
143147
GroupName :: group_name(),

deps/rabbit/src/rabbit_db_m2k_converter.erl

+1-10
Original file line numberDiff line numberDiff line change
@@ -15,8 +15,7 @@
1515
-include_lib("rabbit_common/include/rabbit.hrl").
1616

1717
%% Functions for `rabbit_db_*_m2k_converter' modules to call.
18-
-export([with_correlation_id/2,
19-
get_sub_state/2]).
18+
-export([with_correlation_id/2]).
2019

2120
%% `mnesia_to_khepri_converter' callbacks.
2221
-export([init_copy_to_khepri/4,
@@ -69,14 +68,6 @@ with_correlation_id(
6968
run_async_fun(Fun, State0)
7069
end.
7170

72-
-spec get_sub_state(Module, State) -> Ret when
73-
Module :: module(),
74-
State :: state(),
75-
Ret :: any().
76-
77-
get_sub_state(Module, #?MODULE{sub_states = SubStates}) ->
78-
maps:get(Module, SubStates).
79-
8071
%% `mnesia_to_khepri_converter' callbacks
8172

8273
-spec init_copy_to_khepri(StoreId, MigrationId, Tables, Migrations) ->

deps/rabbit/src/rabbit_db_msup.erl

+18-14
Original file line numberDiff line numberDiff line change
@@ -73,11 +73,11 @@ table_definitions() ->
7373
%% -------------------------------------------------------------------
7474

7575
-spec create_or_update(Group, Overall, Delegate, ChildSpec, Id) -> Ret when
76-
Group :: any(),
76+
Group :: mirrored_supervisor:group_name(),
7777
Overall :: pid(),
7878
Delegate :: pid() | undefined,
7979
ChildSpec :: supervisor2:child_spec(),
80-
Id :: {any(), any()},
80+
Id :: mirrored_supervisor:child_id(),
8181
Ret :: start | undefined | pid().
8282

8383
create_or_update(Group, Overall, Delegate, ChildSpec, Id) ->
@@ -129,8 +129,8 @@ write_in_mnesia(Group, Overall, ChildSpec, Id) ->
129129
ok = mnesia:write(?TABLE, S, write),
130130
ChildSpec.
131131

132-
create_or_update_in_khepri(Group, Overall, Delegate, ChildSpec, {SimpleId, _} = Id) ->
133-
Path = khepri_mirrored_supervisor_path(Group, SimpleId),
132+
create_or_update_in_khepri(Group, Overall, Delegate, ChildSpec, Id) ->
133+
Path = khepri_mirrored_supervisor_path(Group, Id),
134134
S = #mirrored_sup_childspec{key = {Group, Id},
135135
mirroring_pid = Overall,
136136
childspec = ChildSpec},
@@ -169,8 +169,8 @@ create_or_update_in_khepri(Group, Overall, Delegate, ChildSpec, {SimpleId, _} =
169169
%% -------------------------------------------------------------------
170170

171171
-spec delete(Group, Id) -> ok when
172-
Group :: any(),
173-
Id :: any().
172+
Group :: mirrored_supervisor:group_name(),
173+
Id :: mirrored_supervisor:child_id().
174174

175175
delete(Group, Id) ->
176176
rabbit_khepri:handle_fallback(
@@ -184,16 +184,16 @@ delete_in_mnesia(Group, Id) ->
184184
ok = mnesia:delete({?TABLE, {Group, Id}})
185185
end).
186186

187-
delete_in_khepri(Group, {SimpleId, _}) ->
188-
ok = rabbit_khepri:delete(khepri_mirrored_supervisor_path(Group, SimpleId)).
187+
delete_in_khepri(Group, Id) ->
188+
ok = rabbit_khepri:delete(khepri_mirrored_supervisor_path(Group, Id)).
189189

190190
%% -------------------------------------------------------------------
191191
%% find_mirror().
192192
%% -------------------------------------------------------------------
193193

194194
-spec find_mirror(Group, Id) -> Ret when
195-
Group :: any(),
196-
Id :: any(),
195+
Group :: mirrored_supervisor:group_name(),
196+
Id :: mirrored_supervisor:child_id(),
197197
Ret :: {ok, pid()} | {error, not_found}.
198198

199199
find_mirror(Group, Id) ->
@@ -214,8 +214,8 @@ find_mirror_in_mnesia(Group, Id) ->
214214
_ -> {error, not_found}
215215
end.
216216

217-
find_mirror_in_khepri(Group, {SimpleId, _}) ->
218-
case rabbit_khepri:get(khepri_mirrored_supervisor_path(Group, SimpleId)) of
217+
find_mirror_in_khepri(Group, Id) ->
218+
case rabbit_khepri:get(khepri_mirrored_supervisor_path(Group, Id)) of
219219
{ok, #mirrored_sup_childspec{mirroring_pid = Pid}} ->
220220
{ok, Pid};
221221
_ ->
@@ -269,7 +269,7 @@ update_all_in_khepri(Overall, OldOverall) ->
269269
%% -------------------------------------------------------------------
270270

271271
-spec delete_all(Group) -> ok when
272-
Group :: any().
272+
Group :: mirrored_supervisor:group_name().
273273

274274
delete_all(Group) ->
275275
rabbit_khepri:handle_fallback(
@@ -324,5 +324,9 @@ clear_in_khepri() ->
324324
khepri_mirrored_supervisor_path() ->
325325
[?MODULE, mirrored_supervisor_childspec].
326326

327+
khepri_mirrored_supervisor_path(Group, Id)
328+
when is_atom(Id) orelse is_binary(Id) ->
329+
[?MODULE, mirrored_supervisor_childspec, Group, Id];
327330
khepri_mirrored_supervisor_path(Group, Id) ->
328-
[?MODULE, mirrored_supervisor_childspec, Group] ++ Id.
331+
IdPath = Group:id_to_khepri_path(Id),
332+
[?MODULE, mirrored_supervisor_childspec, Group] ++ IdPath.

deps/rabbit/src/rabbit_db_msup_m2k_converter.erl

+7-50
Original file line numberDiff line numberDiff line change
@@ -2,7 +2,7 @@
22
%% License, v. 2.0. If a copy of the MPL was not distributed with this
33
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
44
%%
5-
%% Copyright (c) 2007-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
5+
%% Copyright (c) 2022-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
66
%%
77

88
-module(rabbit_db_msup_m2k_converter).
@@ -19,7 +19,7 @@
1919
copy_to_khepri/3,
2020
delete_from_khepri/3]).
2121

22-
-record(?MODULE, {record_converters :: [module()]}).
22+
-record(?MODULE, {}).
2323

2424
-spec init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when
2525
StoreId :: khepri:store_id(),
@@ -33,8 +33,7 @@ init_copy_to_khepri(_StoreId, _MigrationId, Tables) ->
3333
%% Clean up any previous attempt to copy the Mnesia table to Khepri.
3434
lists:foreach(fun clear_data_in_khepri/1, Tables),
3535

36-
Converters = discover_converters(?MODULE),
37-
SubState = #?MODULE{record_converters = Converters},
36+
SubState = #?MODULE{},
3837
{ok, SubState}.
3938

4039
-spec copy_to_khepri(Table, Record, State) -> Ret when
@@ -47,17 +46,13 @@ init_copy_to_khepri(_StoreId, _MigrationId, Tables) ->
4746
%% @private
4847

4948
copy_to_khepri(mirrored_sup_childspec = Table,
50-
#mirrored_sup_childspec{} = Record0,
49+
#mirrored_sup_childspec{key = {Group, Id} = Key} = Record,
5150
State) ->
52-
#?MODULE{record_converters = Converters} =
53-
rabbit_db_m2k_converter:get_sub_state(?MODULE, State),
54-
Record = upgrade_record(Converters, Table, Record0),
55-
#mirrored_sup_childspec{key = {Group, {SimpleId, _}} = Key} = Record,
5651
?LOG_DEBUG(
5752
"Mnesia->Khepri data copy: [~0p] key: ~0p",
5853
[Table, Key],
5954
#{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
60-
Path = rabbit_db_msup:khepri_mirrored_supervisor_path(Group, SimpleId),
55+
Path = rabbit_db_msup:khepri_mirrored_supervisor_path(Group, Id),
6156
rabbit_db_m2k_converter:with_correlation_id(
6257
fun(CorrId) ->
6358
Extra = #{async => CorrId},
@@ -81,10 +76,8 @@ copy_to_khepri(Table, Record, State) ->
8176
Reason :: any().
8277
%% @private
8378

84-
delete_from_khepri(mirrored_sup_childspec = Table, Key0, State) ->
85-
#?MODULE{record_converters = Converters} =
86-
rabbit_db_m2k_converter:get_sub_state(?MODULE, State),
87-
{Group, Id} = Key = upgrade_key(Converters, Table, Key0),
79+
delete_from_khepri(
80+
mirrored_sup_childspec = Table, {Group, Id} = Key, State) ->
8881
?LOG_DEBUG(
8982
"Mnesia->Khepri data delete: [~0p] key: ~0p",
9083
[Table, Key],
@@ -109,39 +102,3 @@ clear_data_in_khepri(mirrored_sup_childspec) ->
109102
ok -> ok;
110103
Error -> throw(Error)
111104
end.
112-
113-
%% Khepri paths don't support tuples or records, so the key part of the
114-
%% #mirrored_sup_childspec{} used by some plugins must be transformed in a
115-
%% valid Khepri path during the migration from Mnesia to Khepri.
116-
%% `rabbit_db_msup_m2k_converter` iterates over all declared converters, which
117-
%% must implement `rabbit_mnesia_to_khepri_record_converter` behaviour callbacks.
118-
%%
119-
%% This mechanism could be reused by any other rabbit_db_*_m2k_converter
120-
121-
discover_converters(MigrationMod) ->
122-
Apps = rabbit_misc:rabbitmq_related_apps(),
123-
AttrsPerApp = rabbit_misc:module_attributes_from_apps(
124-
rabbit_mnesia_records_to_khepri_db, Apps),
125-
discover_converters(MigrationMod, AttrsPerApp, []).
126-
127-
discover_converters(MigrationMod, [{_App, _AppMod, AppConverters} | Rest],
128-
Converters0) ->
129-
Converters =
130-
lists:foldl(fun({Module, Mod}, Acc) when Module =:= MigrationMod ->
131-
[Mod | Acc];
132-
(_, Acc) ->
133-
Acc
134-
end, Converters0, AppConverters),
135-
discover_converters(MigrationMod, Rest, Converters);
136-
discover_converters(_MigrationMod, [], Converters) ->
137-
Converters.
138-
139-
upgrade_record(Converters, Table, Record) ->
140-
lists:foldl(fun(Mod, Record0) ->
141-
Mod:upgrade_record(Table, Record0)
142-
end, Record, Converters).
143-
144-
upgrade_key(Converters, Table, Key) ->
145-
lists:foldl(fun(Mod, Key0) ->
146-
Mod:upgrade_key(Table, Key0)
147-
end, Key, Converters).

deps/rabbit/src/rabbit_mnesia_to_khepri_record_converter.erl

-24
This file was deleted.

deps/rabbit/test/mirrored_supervisor_SUITE.erl

+1-1
Original file line numberDiff line numberDiff line change
@@ -331,7 +331,7 @@ childspec(Id) ->
331331
{id(Id), {?SERVER, start_link, [Id]}, transient, 16#ffffffff, worker, [?MODULE]}.
332332

333333
id(Id) ->
334-
{[Id], Id}.
334+
Id.
335335

336336
pid_of(Id) ->
337337
{received, Pid, ping} = call(Id, ping),

deps/rabbit/test/rabbit_db_msup_SUITE.erl

+1-1
Original file line numberDiff line numberDiff line change
@@ -87,7 +87,7 @@ create_or_update1(_Config) ->
8787
passed.
8888

8989
id(Id) ->
90-
{[Id], Id}.
90+
Id.
9191

9292
find_mirror(Config) ->
9393
passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, find_mirror1, [Config]).

deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl

+31-24
Original file line numberDiff line numberDiff line change
@@ -1089,8 +1089,14 @@ stop_rabbitmq_nodes(Config) ->
10891089
case FindCrashes of
10901090
true ->
10911091
%% TODO: Make the ignore list configurable.
1092-
IgnoredCrashes = ["** force_vhost_failure"],
1093-
find_crashes_in_logs(NodeConfigs, IgnoredCrashes);
1092+
IgnoredCrashes0 = ["** force_vhost_failure"],
1093+
case rabbit_ct_helpers:get_config(Config, ignored_crashes) of
1094+
undefined ->
1095+
find_crashes_in_logs(NodeConfigs, IgnoredCrashes0);
1096+
IgnoredCrashes1 ->
1097+
find_crashes_in_logs(
1098+
NodeConfigs, IgnoredCrashes0 ++ IgnoredCrashes1)
1099+
end;
10941100
false ->
10951101
ok
10961102
end,
@@ -1172,7 +1178,11 @@ capture_gen_server_termination(
11721178
Ret = re:run(Line, Prefix ++ "( .*|\\*.*|)$", ReOpts),
11731179
case Ret of
11741180
{match, [Suffix]} ->
1175-
case lists:member(Suffix, IgnoredCrashes) of
1181+
Ignore = lists:any(
1182+
fun(IgnoredCrash) ->
1183+
string:find(Suffix, IgnoredCrash) =/= nomatch
1184+
end, IgnoredCrashes),
1185+
case Ignore of
11761186
false ->
11771187
capture_gen_server_termination(
11781188
Rest, Prefix, [Line | Acc], Count, IgnoredCrashes);
@@ -1259,31 +1269,28 @@ rabbitmqctl(Config, Node, Args, Timeout) ->
12591269
_ ->
12601270
CanUseSecondary
12611271
end,
1272+
WithPlugins0 = rabbit_ct_helpers:get_config(Config,
1273+
broker_with_plugins),
1274+
WithPlugins = case is_list(WithPlugins0) of
1275+
true -> lists:nth(I + 1, WithPlugins0);
1276+
false -> WithPlugins0
1277+
end,
12621278
Rabbitmqctl = case UseSecondaryUmbrella of
12631279
true ->
12641280
case BazelRunSecCmd of
12651281
undefined ->
1266-
SrcDir = ?config(
1267-
secondary_rabbit_srcdir,
1268-
Config),
1269-
SecDepsDir = ?config(
1270-
secondary_erlang_mk_depsdir,
1271-
Config),
1272-
SecNewScriptsDir = filename:join(
1273-
[SecDepsDir,
1274-
SrcDir,
1275-
"sbin"]),
1276-
SecOldScriptsDir = filename:join(
1277-
[SecDepsDir,
1278-
"rabbit",
1279-
"scripts"]),
1280-
SecNewScriptsDirExists = filelib:is_dir(
1281-
SecNewScriptsDir),
1282-
SecScriptsDir =
1283-
case SecNewScriptsDirExists of
1284-
true -> SecNewScriptsDir;
1285-
false -> SecOldScriptsDir
1286-
end,
1282+
SrcDir = case WithPlugins of
1283+
false ->
1284+
?config(
1285+
secondary_rabbit_srcdir,
1286+
Config);
1287+
_ ->
1288+
?config(
1289+
secondary_current_srcdir,
1290+
Config)
1291+
end,
1292+
SecScriptsDir = filename:join(
1293+
[SrcDir, "sbin"]),
12871294
rabbit_misc:format(
12881295
"~ts/rabbitmqctl", [SecScriptsDir]);
12891296
_ ->

deps/rabbitmq_federation/BUILD.bazel

+1-1
Original file line numberDiff line numberDiff line change
@@ -96,7 +96,7 @@ rabbitmq_integration_suite(
9696
"test/rabbit_federation_test_util.beam",
9797
],
9898
flaky = True,
99-
shard_count = 2,
99+
shard_count = 3,
100100
)
101101

102102
rabbitmq_integration_suite(

0 commit comments

Comments
 (0)