Skip to content

Commit 2acffca

Browse files
committed
rabbit_feature_flags: Use required feature flages enable callback
[Why] Before this patch, required feature flags were basically checked during boot: they must have been enabled when they were mere stable feature flags. If they were not, the node refused to boot. This was easy for the developer because making a feature flag required allowed to remove the entire compatibility code. Very satisfying. Unfortunately, this was a pain point to end users, especially those who did not pay attention to RabbitMQ and the release notes and were just asking their package manager to update everything. They could end up with a node that refuse to boot. The only solution was to downgrade, enable the disabled stabte feature flags, upgrade again. [How] This patch removes the check and handle required feature flags like any other stable feature flags, except that they are forcibly enabled during boot. This is done when the cluster feature flags states are verified and synchronized. If a required feature flag is not enabled yet, it is enabled at that time. This means that as developers, we will have to keep compatibility code forever, like the required feature flag definition itself.
1 parent 0a337d3 commit 2acffca

File tree

4 files changed

+63
-149
lines changed

4 files changed

+63
-149
lines changed

deps/rabbit/src/rabbit_feature_flags.erl

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1363,7 +1363,7 @@ run_feature_flags_mod_on_remote_node(Node, Function, Args, Timeout) ->
13631363
sync_feature_flags_with_cluster([] = _Nodes, true = _NodeIsVirgin) ->
13641364
rabbit_ff_controller:enable_default();
13651365
sync_feature_flags_with_cluster([] = _Nodes, false = _NodeIsVirgin) ->
1366-
ok;
1366+
rabbit_ff_controller:enable_required();
13671367
sync_feature_flags_with_cluster(Nodes, _NodeIsVirgin) ->
13681368
%% We don't use `rabbit_nodes:filter_running()' here because the given
13691369
%% `Nodes' list may contain nodes which are not members yet (the cluster

deps/rabbit/src/rabbit_ff_controller.erl

Lines changed: 58 additions & 104 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,7 @@
3838
-export([is_supported/1, is_supported/2,
3939
enable/1,
4040
enable_default/0,
41+
enable_required/0,
4142
check_node_compatibility/2,
4243
sync_cluster/1,
4344
refresh_after_app_load/0,
@@ -136,6 +137,24 @@ enable_default() ->
136137
Ret
137138
end.
138139

140+
enable_required() ->
141+
?LOG_DEBUG(
142+
"Feature flags: enable required feature flags",
143+
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
144+
case erlang:whereis(?LOCAL_NAME) of
145+
Pid when is_pid(Pid) ->
146+
%% The function is called while `rabbit' is running.
147+
gen_statem:call(?LOCAL_NAME, enable_required);
148+
undefined ->
149+
%% The function is called while `rabbit' is stopped. We need to
150+
%% start a one-off controller, again to make sure concurrent
151+
%% changes are blocked.
152+
{ok, Pid} = start_link(),
153+
Ret = gen_statem:call(Pid, enable_required),
154+
gen_statem:stop(Pid),
155+
Ret
156+
end.
157+
139158
check_node_compatibility(RemoteNode, LocalNodeAsVirgin) ->
140159
ThisNode = node(),
141160
case LocalNodeAsVirgin of
@@ -304,6 +323,8 @@ proceed_with_task({enable, FeatureNames}) ->
304323
enable_task(FeatureNames);
305324
proceed_with_task(enable_default) ->
306325
enable_default_task();
326+
proceed_with_task(enable_required) ->
327+
enable_required_task();
307328
proceed_with_task({sync_cluster, Nodes}) ->
308329
sync_cluster_task(Nodes);
309330
proceed_with_task(refresh_after_app_load) ->
@@ -841,6 +862,18 @@ get_forced_feature_flag_names_from_config() ->
841862
_ when is_list(Value) -> {ok, Value}
842863
end.
843864

865+
-spec enable_required_task() -> Ret when
866+
Ret :: ok | {error, Reason},
867+
Reason :: term().
868+
869+
enable_required_task() ->
870+
?LOG_DEBUG(
871+
"Feature flags: enabling required feature flags on this node",
872+
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
873+
{ok, Inventory} = collect_inventory_on_nodes([node()]),
874+
RequiredFeatureNames = list_required_feature_flags(Inventory),
875+
enable_many(Inventory, RequiredFeatureNames).
876+
844877
-spec sync_cluster_task() -> Ret when
845878
Ret :: ok | {error, Reason},
846879
Reason :: term().
@@ -855,23 +888,6 @@ sync_cluster_task() ->
855888
Reason :: term().
856889

857890
sync_cluster_task(Nodes) ->
858-
%% We assume that a feature flag can only be enabled, not disabled.
859-
%% Therefore this synchronization searches for feature flags enabled on
860-
%% some nodes but not all, and make sure they are enabled everywhere.
861-
%%
862-
%% This happens when a node joins a cluster and that node has a different
863-
%% set of enabled feature flags.
864-
%%
865-
%% FIXME: `enable_task()' requires that all nodes in the cluster run to
866-
%% enable anything. Should we require the same here? On one hand, this
867-
%% would make sure a feature flag isn't enabled while there is a network
868-
%% partition. On the other hand, this would require that all nodes are
869-
%% running before we can expand the cluster...
870-
?LOG_DEBUG(
871-
"Feature flags: synchronizing feature flags on nodes: ~tp",
872-
[Nodes],
873-
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
874-
875891
case collect_inventory_on_nodes(Nodes) of
876892
{ok, Inventory} ->
877893
CantEnable = list_deprecated_features_that_cant_be_denied(
@@ -880,7 +896,17 @@ sync_cluster_task(Nodes) ->
880896
[] ->
881897
FeatureNames = list_feature_flags_enabled_somewhere(
882898
Inventory, false),
883-
enable_many(Inventory, FeatureNames);
899+
900+
%% In addition to feature flags enabled somewhere, we also
901+
%% ensure required feature flags are enabled accross the
902+
%% board.
903+
RequiredFeatureNames = list_required_feature_flags(
904+
Inventory),
905+
906+
FeatureNamesToEnable = lists:usort(
907+
FeatureNames ++
908+
RequiredFeatureNames),
909+
enable_many(Inventory, FeatureNamesToEnable);
884910
_ ->
885911
?LOG_ERROR(
886912
"Feature flags: the following deprecated features "
@@ -998,7 +1024,7 @@ enable_with_registry_locked(
9981024
[FeatureName],
9991025
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
10001026

1001-
case check_required_and_enable(Inventory, FeatureName) of
1027+
case update_feature_state_and_enable(Inventory, FeatureName) of
10021028
{ok, _Inventory} = Ok ->
10031029
?LOG_NOTICE(
10041030
"Feature flags: `~ts` enabled",
@@ -1014,91 +1040,6 @@ enable_with_registry_locked(
10141040
end
10151041
end.
10161042

1017-
-spec check_required_and_enable(Inventory, FeatureName) -> Ret when
1018-
Inventory :: rabbit_feature_flags:cluster_inventory(),
1019-
FeatureName :: rabbit_feature_flags:feature_name(),
1020-
Ret :: {ok, Inventory} | {error, Reason},
1021-
Reason :: term().
1022-
1023-
check_required_and_enable(
1024-
#{feature_flags := FeatureFlags,
1025-
states_per_node := _} = Inventory,
1026-
FeatureName) ->
1027-
%% Required feature flags vs. virgin nodes.
1028-
FeatureProps = maps:get(FeatureName, FeatureFlags),
1029-
Stability = rabbit_feature_flags:get_stability(FeatureProps),
1030-
ProvidedBy = maps:get(provided_by, FeatureProps),
1031-
NodesWhereDisabled = list_nodes_where_feature_flag_is_disabled(
1032-
Inventory, FeatureName),
1033-
1034-
MarkDirectly = case Stability of
1035-
required when ProvidedBy =:= rabbit ->
1036-
?LOG_DEBUG(
1037-
"Feature flags: `~s`: the feature flag is "
1038-
"required on some nodes; list virgin nodes "
1039-
"to determine if the feature flag can simply "
1040-
"be marked as enabled",
1041-
[FeatureName],
1042-
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
1043-
VirginNodesWhereDisabled =
1044-
lists:filter(
1045-
fun(Node) ->
1046-
case rabbit_db:is_virgin_node(Node) of
1047-
IsVirgin when is_boolean(IsVirgin) ->
1048-
IsVirgin;
1049-
undefined ->
1050-
false
1051-
end
1052-
end, NodesWhereDisabled),
1053-
VirginNodesWhereDisabled =:= NodesWhereDisabled;
1054-
required when ProvidedBy =/= rabbit ->
1055-
%% A plugin can be enabled/disabled at runtime and
1056-
%% between restarts. Thus we have no way to
1057-
%% distinguish a newly enabled plugin from a plugin
1058-
%% which was enabled in the past.
1059-
%%
1060-
%% Therefore, we always mark required feature flags
1061-
%% from plugins directly as enabled. However, the
1062-
%% plugin is responsible for checking that its
1063-
%% possibly existing data is as it expects it or
1064-
%% perform any cleanup/conversion!
1065-
?LOG_DEBUG(
1066-
"Feature flags: `~s`: the feature flag is "
1067-
"required on some nodes; it comes from a "
1068-
"plugin which can be enabled at runtime, "
1069-
"so it can be marked as enabled",
1070-
[FeatureName],
1071-
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
1072-
true;
1073-
_ ->
1074-
false
1075-
end,
1076-
1077-
case MarkDirectly of
1078-
false ->
1079-
case Stability of
1080-
required ->
1081-
?LOG_DEBUG(
1082-
"Feature flags: `~s`: some nodes where the feature "
1083-
"flag is disabled are not virgin, we need to perform "
1084-
"a regular sync",
1085-
[FeatureName],
1086-
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS});
1087-
_ ->
1088-
ok
1089-
end,
1090-
update_feature_state_and_enable(Inventory, FeatureName);
1091-
true ->
1092-
?LOG_DEBUG(
1093-
"Feature flags: `~s`: all nodes where the feature flag is "
1094-
"disabled are virgin, we can directly mark it as enabled "
1095-
"there",
1096-
[FeatureName],
1097-
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
1098-
mark_as_enabled_on_nodes(
1099-
NodesWhereDisabled, Inventory, FeatureName, true)
1100-
end.
1101-
11021043
-spec update_feature_state_and_enable(Inventory, FeatureName) -> Ret when
11031044
Inventory :: rabbit_feature_flags:cluster_inventory(),
11041045
FeatureName :: rabbit_feature_flags:feature_name(),
@@ -1445,6 +1386,19 @@ list_feature_flags_enabled_somewhere(
14451386
end, #{}, StatesPerNode),
14461387
lists:sort(maps:keys(MergedStates)).
14471388

1389+
list_required_feature_flags(#{feature_flags := FeatureFlags}) ->
1390+
RequiredFeatureNames = maps:fold(
1391+
fun(FeatureName, FeatureProps, Acc) ->
1392+
Stability = (
1393+
rabbit_feature_flags:get_stability(
1394+
FeatureProps)),
1395+
case Stability of
1396+
required -> [FeatureName | Acc];
1397+
_ -> Acc
1398+
end
1399+
end, [], FeatureFlags),
1400+
lists:sort(RequiredFeatureNames).
1401+
14481402
-spec list_deprecated_features_that_cant_be_denied(Inventory) ->
14491403
Ret when
14501404
Inventory :: rabbit_feature_flags:cluster_inventory(),

deps/rabbit/src/rabbit_ff_registry_factory.erl

Lines changed: 1 addition & 34 deletions
Original file line numberDiff line numberDiff line change
@@ -247,8 +247,6 @@ maybe_initialize_registry(NewSupportedFeatureFlags,
247247
%% return an error (and RabbitMQ start will abort). RabbitMQ won't be
248248
%% able to work, especially if the feature flag needed some
249249
%% migration, because the corresponding code was removed.
250-
NewNode =
251-
not rabbit_feature_flags:does_enabled_feature_flags_list_file_exist(),
252250
FeatureStates0 = case RegistryInitialized of
253251
true ->
254252
maps:merge(
@@ -261,42 +259,11 @@ maybe_initialize_registry(NewSupportedFeatureFlags,
261259
maps:map(
262260
fun
263261
(FeatureName, FeatureProps) when ?IS_FEATURE_FLAG(FeatureProps) ->
264-
Stability = rabbit_feature_flags:get_stability(FeatureProps),
265-
ProvidedBy = maps:get(provided_by, FeatureProps),
266262
State = case FeatureStates0 of
267263
#{FeatureName := FeatureState} -> FeatureState;
268264
_ -> false
269265
end,
270-
case Stability of
271-
required when State =:= true ->
272-
%% The required feature flag is already enabled, we keep
273-
%% it this way.
274-
State;
275-
required when NewNode ->
276-
%% This is the very first time the node starts, we
277-
%% already mark the required feature flag as enabled.
278-
?assertNotEqual(state_changing, State),
279-
true;
280-
required when ProvidedBy =/= rabbit ->
281-
?assertNotEqual(state_changing, State),
282-
true;
283-
required ->
284-
%% This is not a new node and the required feature flag
285-
%% is disabled. This is an error and RabbitMQ must be
286-
%% downgraded to enable the feature flag.
287-
?assertNotEqual(state_changing, State),
288-
?LOG_ERROR(
289-
"Feature flags: `~ts`: required feature flag not "
290-
"enabled! It must be enabled before upgrading "
291-
"RabbitMQ.",
292-
[FeatureName],
293-
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
294-
throw({error,
295-
{disabled_required_feature_flag,
296-
FeatureName}});
297-
_ ->
298-
State
299-
end;
266+
State;
300267
(FeatureName, FeatureProps) when ?IS_DEPRECATION(FeatureProps) ->
301268
case FeatureStates0 of
302269
#{FeatureName := FeatureState} ->

deps/rabbit/test/feature_flags_v2_SUITE.erl

Lines changed: 3 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -1480,15 +1480,8 @@ have_required_feature_flag_in_cluster_and_add_member_without_it(
14801480
ok = run_on_node(
14811481
NewNode,
14821482
fun() ->
1483-
?assertMatch(
1484-
{error,
1485-
{exception,
1486-
{assertNotEqual,
1487-
[{module, rabbit_ff_registry_factory},
1488-
{line, _},
1489-
{expression, "State"},
1490-
{value, state_changing}]},
1491-
_}},
1483+
?assertEqual(
1484+
ok,
14921485
rabbit_feature_flags:sync_feature_flags_with_cluster(
14931486
Nodes, false)),
14941487
ok
@@ -1500,7 +1493,7 @@ have_required_feature_flag_in_cluster_and_add_member_without_it(
15001493
Node,
15011494
fun() ->
15021495
?assertEqual(
1503-
Node =/= NewNode,
1496+
true,
15041497
rabbit_feature_flags:is_enabled(FeatureName)),
15051498
ok
15061499
end,

0 commit comments

Comments
 (0)