Skip to content

Commit db51669

Browse files
committed
rabbit_db: Add init/0 function
This function sits on top of `rabbit_mnesia:init/0`. In the future, it will take care of initializing the right database layer, whatever it is. The start of `mnesia_sync` is now part of this initialization instead of a separate boot step in `rabbit` because its is specific to our use of Mnesia. This patch also introduces the following functions: * `is_virgin_node/0` * `dir/0` They also wrap the same functions in `rabbit_mnesia`. In addition, `rabbit_db` provides `is_virgin_node/1` to query the state of a remote node. This is used by `rabbit_ff_controller` in the feature flags subsystem. At this point, the underlying equivalent functions in `rabbit_mnesia` become private to this module (and other modules implementing the interaction with Mnesia). Other parts of RabbitMQ, including plugins, should now use `rabbit_db`, not `rabbit_mnesia`.
1 parent 3a0618e commit db51669

File tree

5 files changed

+100
-23
lines changed

5 files changed

+100
-23
lines changed

deps/rabbit/src/rabbit.erl

Lines changed: 1 addition & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -64,16 +64,10 @@
6464
{enables, external_infrastructure}]}).
6565

6666
-rabbit_boot_step({database,
67-
[{mfa, {rabbit_mnesia, init, []}},
67+
[{mfa, {rabbit_db, init, []}},
6868
{requires, file_handle_cache},
6969
{enables, external_infrastructure}]}).
7070

71-
-rabbit_boot_step({database_sync,
72-
[{description, "database sync"},
73-
{mfa, {rabbit_sup, start_child, [mnesia_sync]}},
74-
{requires, database},
75-
{enables, external_infrastructure}]}).
76-
7771
-rabbit_boot_step({networking_metadata_store,
7872
[{description, "networking infrastructure"},
7973
{mfa, {rabbit_sup, start_child, [rabbit_networking_store]}},
@@ -1113,7 +1107,6 @@ get_default_data_param(Param) ->
11131107

11141108
data_dir() ->
11151109
{ok, DataDir} = application:get_env(rabbit, data_dir),
1116-
?assertEqual(DataDir, rabbit_mnesia:dir()),
11171110
DataDir.
11181111

11191112
%%---------------------------------------------------------------------------

deps/rabbit/src/rabbit_db.erl

Lines changed: 92 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -10,8 +10,100 @@
1010
-include_lib("kernel/include/logger.hrl").
1111
-include_lib("stdlib/include/assert.hrl").
1212

13+
-include_lib("rabbit_common/include/logging.hrl").
14+
15+
-export([init/0,
16+
is_virgin_node/0, is_virgin_node/1,
17+
dir/0]).
1318
-export([run/1]).
1419

20+
%% Default timeout for operations on remote nodes.
21+
-define(TIMEOUT, 60000).
22+
23+
%% -------------------------------------------------------------------
24+
%% DB initialization.
25+
%% -------------------------------------------------------------------
26+
27+
-spec init() -> Ret when
28+
Ret :: ok | {error, any()}.
29+
%% @doc Initializes the DB layer.
30+
31+
init() ->
32+
IsVirgin = is_virgin_node(),
33+
?LOG_DEBUG(
34+
"DB: this node is virgin: ~ts",
35+
[IsVirgin],
36+
#{domain => ?RMQLOG_DOMAIN_DB}),
37+
case init_mnesia() of
38+
ok ->
39+
?LOG_DEBUG(
40+
"DB: initialization successeful",
41+
#{domain => ?RMQLOG_DOMAIN_DB}),
42+
ok;
43+
Error ->
44+
?LOG_DEBUG(
45+
"DB: initialization failed: ~0p",
46+
[Error],
47+
#{domain => ?RMQLOG_DOMAIN_DB}),
48+
Error
49+
end.
50+
51+
init_mnesia() ->
52+
?LOG_DEBUG(
53+
"DB: initialize Mnesia",
54+
#{domain => ?RMQLOG_DOMAIN_DB}),
55+
ok = rabbit_mnesia:init(),
56+
?assertEqual(rabbit:data_dir(), mnesia_dir()),
57+
rabbit_sup:start_child(mnesia_sync).
58+
59+
-spec is_virgin_node() -> IsVirgin when
60+
IsVirgin :: boolean().
61+
%% @doc Indicates if this RabbitMQ node is virgin.
62+
%%
63+
%% @returns `true' if the node is virgin, `false' if it is not.
64+
%%
65+
%% @see is_virgin_node/1.
66+
67+
is_virgin_node() ->
68+
ThisNode = node(),
69+
is_virgin_node(ThisNode).
70+
71+
-spec is_virgin_node(Node) -> IsVirgin | undefined when
72+
Node :: node(),
73+
IsVirgin :: boolean().
74+
%% @doc Indicates if the given RabbitMQ node is virgin.
75+
%%
76+
%% A virgin node is a node starting for the first time. It could be a brand
77+
%% new node or a node having been reset.
78+
%%
79+
%% @returns `true' if the node is virgin, `false' if it is not, or `undefined'
80+
%% if the given node is remote and we couldn't determine it.
81+
82+
is_virgin_node(Node) when is_atom(Node) ->
83+
is_virgin_node_with_mnesia(Node).
84+
85+
is_virgin_node_with_mnesia(Node) when Node =:= node() ->
86+
rabbit_mnesia:is_virgin_node();
87+
is_virgin_node_with_mnesia(Node) ->
88+
try
89+
erpc:call(Node, rabbit_mnesia, is_virgin_node, [], ?TIMEOUT)
90+
catch
91+
_:_ ->
92+
undefined
93+
end.
94+
95+
-spec dir() -> DBDir when
96+
DBDir :: file:filename().
97+
%% @doc Returns the directory where the database stores its data.
98+
%%
99+
%% @returns the directory path.
100+
101+
dir() ->
102+
mnesia_dir().
103+
104+
mnesia_dir() ->
105+
rabbit_mnesia:dir().
106+
15107
%% -------------------------------------------------------------------
16108
%% run().
17109
%% -------------------------------------------------------------------

deps/rabbit/src/rabbit_ff_controller.erl

Lines changed: 1 addition & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -762,7 +762,7 @@ check_required_and_enable(
762762
VirginNodesWhereDisabled =
763763
lists:filter(
764764
fun(Node) ->
765-
case is_virgin_node(Node) of
765+
case rabbit_db:is_virgin_node(Node) of
766766
IsVirgin when is_boolean(IsVirgin) ->
767767
IsVirgin;
768768
undefined ->
@@ -853,14 +853,6 @@ update_feature_state_and_enable(
853853
Error
854854
end.
855855

856-
is_virgin_node(Node) ->
857-
case rpc_call(Node, rabbit_mnesia, is_virgin_node, [], ?TIMEOUT) of
858-
IsVirgin when is_boolean(IsVirgin) ->
859-
IsVirgin;
860-
{error, _} ->
861-
undefined
862-
end.
863-
864856
restore_feature_flag_state(
865857
Nodes, NodesWhereDisabled, Inventory, FeatureName) ->
866858
NodesWhereEnabled = Nodes -- NodesWhereDisabled,

deps/rabbit/src/rabbit_node_monitor.erl

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -62,12 +62,12 @@ start_link() -> gen_server:start_link({local, ?SERVER}, ?MODULE, [], []).
6262
-spec running_nodes_filename() -> string().
6363

6464
running_nodes_filename() ->
65-
filename:join(rabbit_mnesia:dir(), "nodes_running_at_shutdown").
65+
filename:join(rabbit:data_dir(), "nodes_running_at_shutdown").
6666

6767
-spec cluster_status_filename() -> string().
6868

6969
cluster_status_filename() ->
70-
filename:join(rabbit_mnesia:dir(), "cluster_nodes.config").
70+
filename:join(rabbit:data_dir(), "cluster_nodes.config").
7171

7272
coordination_filename() ->
7373
filename:join(rabbit:data_dir(), "coordination").

deps/rabbit/test/feature_flags_v2_SUITE.erl

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1272,11 +1272,11 @@ have_required_feature_flag_in_cluster_and_add_member_without_it(
12721272
?assert(rabbit_feature_flags:is_supported(FeatureName)),
12731273
?assertNot(rabbit_feature_flags:is_enabled(FeatureName)),
12741274

1275-
MnesiaDir = rabbit_mnesia:dir(),
1276-
ok = filelib:ensure_path(MnesiaDir),
1277-
SomeFile = filename:join(MnesiaDir, "some-mnesia-file.db"),
1275+
DBDir = rabbit_db:dir(),
1276+
ok = filelib:ensure_path(DBDir),
1277+
SomeFile = filename:join(DBDir, "some-file.db"),
12781278
ok = file:write_file(SomeFile, <<>>),
1279-
?assertNot(rabbit_mnesia:is_virgin_node()),
1279+
?assertNot(rabbit_db:is_virgin_node()),
12801280
ok
12811281
end,
12821282
[]),

0 commit comments

Comments
 (0)