Skip to content

Commit 2a0a0df

Browse files
committed
rabbit_peer_discovery: Rewrite the core logic
[Why] This work started as an effort to add peer discovery support to our Khepri integration. Indeed, as part of the task to integrate Khepri, we missed the fact that `rabbit_peer_discovery:maybe_create_cluster/1` was called from the Mnesia-specific code only. Even though we knew about it because we hit many issues caused by the fact the `join_cluster` and peer discovery use different code path to create a cluster. To add support for Khepri, the first version of this patch was to move the call to `rabbit_peer_discovery:maybe_create_cluster/1` from `rabbit_db_cluster` instead of `rabbit_mnesia`. To achieve that, it made sense to unify the code and simply call `rabbit_db_cluster:join/2` instead of duplicating the work. Unfortunately, doing so highlighted another issue: the way the node to cluster with was selected. Indeed, it could cause situations where multiple clusters are created instead of one, without resorting to out-of-band counter-measures, like a 30-second delay added in the Kubernetes operator (rabbitmq/cluster-operator#1156). This problem was even more frequent when we tried to unify the code path and call `join_cluster`. After several iterations on the patch and even more discussions with the team, we decided to rewrite the algorithm to make node selection more robust and still use `rabbit_db_cluster:join/2` to create the cluster. [How] This commit is only about the rewrite of the algorithm. Calling peer discovery from `rabbit_db_cluster` instead of `rabbit_mnesia` (and thus making peer discovery work with Khepri) will be done in a follow-up commit. We wanted the new algorithm to fulfill the following properties: 1. `rabbit_peer_discovery` should provide the ability to re-trigger it easily to re-evaluate the cluster. The new public API is `rabbit_peer_discovery:sync_desired_cluster/0`. 2. The selection of the node to join should be designed in a way that all nodes select the same, regardless of the order in which they become available. The adopted solution is to sort the list of discovered nodes with the following criterias (in that order): 1. the size of the cluster a discovered node is part of; sorted from bigger to smaller clusters 2. the start time of a discovered node; sorted from older to younger nodes 3. the name of a discovered node; sorted alphabetically The first node in that list will not join anyone and simply proceed with its boot process. Other nodes will try to join the first node. 3. To reduce the chance of incorrectly having multiple standalone nodes because the discovery backend returned only a single node, we want to apply the following constraints to the list of nodes after it is filtered and sorted (see property 2 above): * The list must contain `node()` (i.e. the node running peer discovery itself). * If the RabbitMQ's cluster size hint is greater than 1, the list must have at least two nodes. The cluster size hint is the maximum between the configured target cluster size hint and the number of elements in the nodes list returned by the backend. If one of the constraint is not met, the entire peer discovery process is restarted after a delay. 4. The lock is acquired only to protect the actual join, not the discovery step where the backend is queried to get the list of peers. With the node selection described above, this will let the first node to start without acquiring the lock. 5. The cluster membership views queried as part of the algorithm to sort the list of nodes will be used to detect additional clusters or standalone nodes that did not cluster correctly. These nodes will be asked to re-evaluate peer discovery to increase the chance of forming a single cluster. 6. After some delay, peer discovery will be re-evaluated to further eliminate the chances of having multiple clusters instead of one. This commit covers properties from point 1 to point 4. Remaining properties will be the scope of additional pull requests after this one works. If there is a failure at any point during discovery, filtering/sorting, locking or joining, the entire process is restarted after a delay. This is configured using the following parameters: * cluster_formation.discovery_retry_limit * cluster_formation.discovery_retry_interval The default parameters were bumped to 30 retries with a delay of 1 second between each. The locking retries/interval parameters are not used by the new algorithm anymore. There are extra minor changes that come with the rewrite: * The configured backend is cached in a persistent term. The goal is to make sure we use the same backend throughout the entire process and when we call `maybe_unregister/0` even if the configuration changed for whatever reason in between. * `maybe_register/0` is called from `rabbit_db_cluster` instead of at the end of a successful peer discovery process. `rabbit_db_cluster` had to call `maybe_register/0` if the node was not virgin anyway. So make it simpler and always call it in `rabbit_db_cluster` regardless of the state of the node. * `log_configured_backend/0` is gone. `maybe_init/0` can log the backend directly. There is no need to explicitly call another function for that. * Messages are logged using `?LOG_*()` macros instead of the old `rabbit_log` module.
1 parent 8a59ae3 commit 2a0a0df

10 files changed

+916
-340
lines changed

deps/rabbit/BUILD.bazel

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -958,6 +958,14 @@ rabbitmq_suite(
958958
],
959959
)
960960

961+
rabbitmq_suite(
962+
name = "unit_cluster_formation_sort_nodes_SUITE",
963+
size = "small",
964+
deps = [
965+
"@meck//:erlang_app",
966+
],
967+
)
968+
961969
rabbitmq_suite(
962970
name = "unit_collections_SUITE",
963971
size = "small",

deps/rabbit/app.bzl

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1735,6 +1735,14 @@ def test_suite_beam_files(name = "test_suite_beam_files"):
17351735
app_name = "rabbit",
17361736
erlc_opts = "//:test_erlc_opts",
17371737
)
1738+
erlang_bytecode(
1739+
name = "unit_cluster_formation_sort_nodes_SUITE_beam_files",
1740+
testonly = True,
1741+
srcs = ["test/unit_cluster_formation_sort_nodes_SUITE.erl"],
1742+
outs = ["test/unit_cluster_formation_sort_nodes_SUITE.beam"],
1743+
app_name = "rabbit",
1744+
erlc_opts = "//:test_erlc_opts",
1745+
)
17381746
erlang_bytecode(
17391747
name = "unit_collections_SUITE_beam_files",
17401748
testonly = True,

deps/rabbit/src/rabbit_db.erl

Lines changed: 1 addition & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -51,7 +51,6 @@ init() ->
5151
#{domain => ?RMQLOG_DOMAIN_DB}),
5252

5353
ensure_dir_exists(),
54-
rabbit_peer_discovery:log_configured_backend(),
5554
rabbit_peer_discovery:maybe_init(),
5655

5756
pre_init(IsVirgin),
@@ -66,8 +65,8 @@ init() ->
6665
"DB: initialization successeful",
6766
#{domain => ?RMQLOG_DOMAIN_DB}),
6867

68+
rabbit_peer_discovery:maybe_register(),
6969
init_finished(),
70-
post_init(IsVirgin),
7170

7271
ok;
7372
Error ->
@@ -82,12 +81,6 @@ pre_init(IsVirgin) ->
8281
OtherMembers = rabbit_nodes:nodes_excl_me(Members),
8382
rabbit_db_cluster:ensure_feature_flags_are_in_sync(OtherMembers, IsVirgin).
8483

85-
post_init(false = _IsVirgin) ->
86-
rabbit_peer_discovery:maybe_register();
87-
post_init(true = _IsVirgin) ->
88-
%% Registration handled by rabbit_peer_discovery.
89-
ok.
90-
9184
init_using_mnesia() ->
9285
?LOG_DEBUG(
9386
"DB: initialize Mnesia",

deps/rabbit/src/rabbit_mnesia.erl

Lines changed: 13 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -114,13 +114,24 @@
114114
init() ->
115115
ensure_mnesia_running(),
116116
ensure_mnesia_dir(),
117+
%% If this node is virgin, we call peer discovery to see if this node
118+
%% should start as a standalone node or if it should join a cluster.
117119
case is_virgin_node() of
118120
true ->
119121
rabbit_log:info("Node database directory at ~ts is empty. "
120122
"Assuming we need to join an existing cluster or initialise from scratch...",
121123
[dir()]),
122-
rabbit_peer_discovery:maybe_create_cluster(
123-
fun create_cluster_callback/2);
124+
rabbit_peer_discovery:sync_desired_cluster();
125+
false ->
126+
ok
127+
end,
128+
%% Peer discovery may have been a no-op if it decided that all other nodes
129+
%% should join this one. Therefore, we need to look at if this node is
130+
%% still virgin and finish our use of Mnesia accordingly. In particular,
131+
%% this second part crates all our Mnesia tables.
132+
case is_virgin_node() of
133+
true ->
134+
init_db_and_upgrade([node()], disc, true, _Retry = true);
124135
false ->
125136
NodeType = node_type(),
126137
case is_node_type_permitted(NodeType) of
@@ -141,23 +152,6 @@ init() ->
141152
ok = rabbit_node_monitor:global_sync(),
142153
ok.
143154

144-
create_cluster_callback(none, NodeType) ->
145-
DiscNodes = [node()],
146-
NodeType1 = case is_node_type_permitted(NodeType) of
147-
false -> disc;
148-
true -> NodeType
149-
end,
150-
init_db_and_upgrade(DiscNodes, NodeType1, true, _Retry = true),
151-
ok;
152-
create_cluster_callback(RemoteNode, NodeType) ->
153-
{ok, {_, DiscNodes, _}} = discover_cluster0(RemoteNode),
154-
NodeType1 = case is_node_type_permitted(NodeType) of
155-
false -> disc;
156-
true -> NodeType
157-
end,
158-
init_db_and_upgrade(DiscNodes, NodeType1, true, _Retry = true),
159-
ok.
160-
161155
%% Make the node join a cluster. The node will be reset automatically
162156
%% before we actually cluster it. The nodes provided will be used to
163157
%% find out about the nodes in the cluster.

0 commit comments

Comments
 (0)