Skip to content

Commit

Permalink
rabbit_feature_flags: Use required feature flages enable callback
Browse files Browse the repository at this point in the history
[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.
  • Loading branch information
dumbbell committed Oct 15, 2024
1 parent 0a337d3 commit 2acffca
Show file tree
Hide file tree
Showing 4 changed files with 63 additions and 149 deletions.
2 changes: 1 addition & 1 deletion deps/rabbit/src/rabbit_feature_flags.erl
Original file line number Diff line number Diff line change
Expand Up @@ -1363,7 +1363,7 @@ run_feature_flags_mod_on_remote_node(Node, Function, Args, Timeout) ->
sync_feature_flags_with_cluster([] = _Nodes, true = _NodeIsVirgin) ->
rabbit_ff_controller:enable_default();
sync_feature_flags_with_cluster([] = _Nodes, false = _NodeIsVirgin) ->
ok;
rabbit_ff_controller:enable_required();
sync_feature_flags_with_cluster(Nodes, _NodeIsVirgin) ->
%% We don't use `rabbit_nodes:filter_running()' here because the given
%% `Nodes' list may contain nodes which are not members yet (the cluster
Expand Down
162 changes: 58 additions & 104 deletions deps/rabbit/src/rabbit_ff_controller.erl
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
-export([is_supported/1, is_supported/2,
enable/1,
enable_default/0,
enable_required/0,
check_node_compatibility/2,
sync_cluster/1,
refresh_after_app_load/0,
Expand Down Expand Up @@ -136,6 +137,24 @@ enable_default() ->
Ret
end.

enable_required() ->
?LOG_DEBUG(
"Feature flags: enable required feature flags",
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
case erlang:whereis(?LOCAL_NAME) of
Pid when is_pid(Pid) ->
%% The function is called while `rabbit' is running.
gen_statem:call(?LOCAL_NAME, enable_required);
undefined ->
%% The function is called while `rabbit' is stopped. We need to
%% start a one-off controller, again to make sure concurrent
%% changes are blocked.
{ok, Pid} = start_link(),
Ret = gen_statem:call(Pid, enable_required),
gen_statem:stop(Pid),
Ret
end.

check_node_compatibility(RemoteNode, LocalNodeAsVirgin) ->
ThisNode = node(),
case LocalNodeAsVirgin of
Expand Down Expand Up @@ -304,6 +323,8 @@ proceed_with_task({enable, FeatureNames}) ->
enable_task(FeatureNames);
proceed_with_task(enable_default) ->
enable_default_task();
proceed_with_task(enable_required) ->
enable_required_task();
proceed_with_task({sync_cluster, Nodes}) ->
sync_cluster_task(Nodes);
proceed_with_task(refresh_after_app_load) ->
Expand Down Expand Up @@ -841,6 +862,18 @@ get_forced_feature_flag_names_from_config() ->
_ when is_list(Value) -> {ok, Value}
end.
-spec enable_required_task() -> Ret when
Ret :: ok | {error, Reason},
Reason :: term().
enable_required_task() ->
?LOG_DEBUG(
"Feature flags: enabling required feature flags on this node",
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
{ok, Inventory} = collect_inventory_on_nodes([node()]),
RequiredFeatureNames = list_required_feature_flags(Inventory),
enable_many(Inventory, RequiredFeatureNames).
-spec sync_cluster_task() -> Ret when
Ret :: ok | {error, Reason},
Reason :: term().
Expand All @@ -855,23 +888,6 @@ sync_cluster_task() ->
Reason :: term().
sync_cluster_task(Nodes) ->
%% We assume that a feature flag can only be enabled, not disabled.
%% Therefore this synchronization searches for feature flags enabled on
%% some nodes but not all, and make sure they are enabled everywhere.
%%
%% This happens when a node joins a cluster and that node has a different
%% set of enabled feature flags.
%%
%% FIXME: `enable_task()' requires that all nodes in the cluster run to
%% enable anything. Should we require the same here? On one hand, this
%% would make sure a feature flag isn't enabled while there is a network
%% partition. On the other hand, this would require that all nodes are
%% running before we can expand the cluster...
?LOG_DEBUG(
"Feature flags: synchronizing feature flags on nodes: ~tp",
[Nodes],
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
case collect_inventory_on_nodes(Nodes) of
{ok, Inventory} ->
CantEnable = list_deprecated_features_that_cant_be_denied(
Expand All @@ -880,7 +896,17 @@ sync_cluster_task(Nodes) ->
[] ->
FeatureNames = list_feature_flags_enabled_somewhere(
Inventory, false),
enable_many(Inventory, FeatureNames);
%% In addition to feature flags enabled somewhere, we also
%% ensure required feature flags are enabled accross the
%% board.
RequiredFeatureNames = list_required_feature_flags(
Inventory),
FeatureNamesToEnable = lists:usort(
FeatureNames ++
RequiredFeatureNames),
enable_many(Inventory, FeatureNamesToEnable);
_ ->
?LOG_ERROR(
"Feature flags: the following deprecated features "
Expand Down Expand Up @@ -998,7 +1024,7 @@ enable_with_registry_locked(
[FeatureName],
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),

case check_required_and_enable(Inventory, FeatureName) of
case update_feature_state_and_enable(Inventory, FeatureName) of
{ok, _Inventory} = Ok ->
?LOG_NOTICE(
"Feature flags: `~ts` enabled",
Expand All @@ -1014,91 +1040,6 @@ enable_with_registry_locked(
end
end.

-spec check_required_and_enable(Inventory, FeatureName) -> Ret when
Inventory :: rabbit_feature_flags:cluster_inventory(),
FeatureName :: rabbit_feature_flags:feature_name(),
Ret :: {ok, Inventory} | {error, Reason},
Reason :: term().

check_required_and_enable(
#{feature_flags := FeatureFlags,
states_per_node := _} = Inventory,
FeatureName) ->
%% Required feature flags vs. virgin nodes.
FeatureProps = maps:get(FeatureName, FeatureFlags),
Stability = rabbit_feature_flags:get_stability(FeatureProps),
ProvidedBy = maps:get(provided_by, FeatureProps),
NodesWhereDisabled = list_nodes_where_feature_flag_is_disabled(
Inventory, FeatureName),

MarkDirectly = case Stability of
required when ProvidedBy =:= rabbit ->
?LOG_DEBUG(
"Feature flags: `~s`: the feature flag is "
"required on some nodes; list virgin nodes "
"to determine if the feature flag can simply "
"be marked as enabled",
[FeatureName],
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
VirginNodesWhereDisabled =
lists:filter(
fun(Node) ->
case rabbit_db:is_virgin_node(Node) of
IsVirgin when is_boolean(IsVirgin) ->
IsVirgin;
undefined ->
false
end
end, NodesWhereDisabled),
VirginNodesWhereDisabled =:= NodesWhereDisabled;
required when ProvidedBy =/= rabbit ->
%% A plugin can be enabled/disabled at runtime and
%% between restarts. Thus we have no way to
%% distinguish a newly enabled plugin from a plugin
%% which was enabled in the past.
%%
%% Therefore, we always mark required feature flags
%% from plugins directly as enabled. However, the
%% plugin is responsible for checking that its
%% possibly existing data is as it expects it or
%% perform any cleanup/conversion!
?LOG_DEBUG(
"Feature flags: `~s`: the feature flag is "
"required on some nodes; it comes from a "
"plugin which can be enabled at runtime, "
"so it can be marked as enabled",
[FeatureName],
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
true;
_ ->
false
end,

case MarkDirectly of
false ->
case Stability of
required ->
?LOG_DEBUG(
"Feature flags: `~s`: some nodes where the feature "
"flag is disabled are not virgin, we need to perform "
"a regular sync",
[FeatureName],
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS});
_ ->
ok
end,
update_feature_state_and_enable(Inventory, FeatureName);
true ->
?LOG_DEBUG(
"Feature flags: `~s`: all nodes where the feature flag is "
"disabled are virgin, we can directly mark it as enabled "
"there",
[FeatureName],
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
mark_as_enabled_on_nodes(
NodesWhereDisabled, Inventory, FeatureName, true)
end.

-spec update_feature_state_and_enable(Inventory, FeatureName) -> Ret when
Inventory :: rabbit_feature_flags:cluster_inventory(),
FeatureName :: rabbit_feature_flags:feature_name(),
Expand Down Expand Up @@ -1445,6 +1386,19 @@ list_feature_flags_enabled_somewhere(
end, #{}, StatesPerNode),
lists:sort(maps:keys(MergedStates)).

list_required_feature_flags(#{feature_flags := FeatureFlags}) ->
RequiredFeatureNames = maps:fold(
fun(FeatureName, FeatureProps, Acc) ->
Stability = (
rabbit_feature_flags:get_stability(
FeatureProps)),
case Stability of
required -> [FeatureName | Acc];
_ -> Acc
end
end, [], FeatureFlags),
lists:sort(RequiredFeatureNames).

-spec list_deprecated_features_that_cant_be_denied(Inventory) ->
Ret when
Inventory :: rabbit_feature_flags:cluster_inventory(),
Expand Down
35 changes: 1 addition & 34 deletions deps/rabbit/src/rabbit_ff_registry_factory.erl
Original file line number Diff line number Diff line change
Expand Up @@ -247,8 +247,6 @@ maybe_initialize_registry(NewSupportedFeatureFlags,
%% return an error (and RabbitMQ start will abort). RabbitMQ won't be
%% able to work, especially if the feature flag needed some
%% migration, because the corresponding code was removed.
NewNode =
not rabbit_feature_flags:does_enabled_feature_flags_list_file_exist(),
FeatureStates0 = case RegistryInitialized of
true ->
maps:merge(
Expand All @@ -261,42 +259,11 @@ maybe_initialize_registry(NewSupportedFeatureFlags,
maps:map(
fun
(FeatureName, FeatureProps) when ?IS_FEATURE_FLAG(FeatureProps) ->
Stability = rabbit_feature_flags:get_stability(FeatureProps),
ProvidedBy = maps:get(provided_by, FeatureProps),
State = case FeatureStates0 of
#{FeatureName := FeatureState} -> FeatureState;
_ -> false
end,
case Stability of
required when State =:= true ->
%% The required feature flag is already enabled, we keep
%% it this way.
State;
required when NewNode ->
%% This is the very first time the node starts, we
%% already mark the required feature flag as enabled.
?assertNotEqual(state_changing, State),
true;
required when ProvidedBy =/= rabbit ->
?assertNotEqual(state_changing, State),
true;
required ->
%% This is not a new node and the required feature flag
%% is disabled. This is an error and RabbitMQ must be
%% downgraded to enable the feature flag.
?assertNotEqual(state_changing, State),
?LOG_ERROR(
"Feature flags: `~ts`: required feature flag not "
"enabled! It must be enabled before upgrading "
"RabbitMQ.",
[FeatureName],
#{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}),
throw({error,
{disabled_required_feature_flag,
FeatureName}});
_ ->
State
end;
State;
(FeatureName, FeatureProps) when ?IS_DEPRECATION(FeatureProps) ->
case FeatureStates0 of
#{FeatureName := FeatureState} ->
Expand Down
13 changes: 3 additions & 10 deletions deps/rabbit/test/feature_flags_v2_SUITE.erl
Original file line number Diff line number Diff line change
Expand Up @@ -1480,15 +1480,8 @@ have_required_feature_flag_in_cluster_and_add_member_without_it(
ok = run_on_node(
NewNode,
fun() ->
?assertMatch(
{error,
{exception,
{assertNotEqual,
[{module, rabbit_ff_registry_factory},
{line, _},
{expression, "State"},
{value, state_changing}]},
_}},
?assertEqual(
ok,
rabbit_feature_flags:sync_feature_flags_with_cluster(
Nodes, false)),
ok
Expand All @@ -1500,7 +1493,7 @@ have_required_feature_flag_in_cluster_and_add_member_without_it(
Node,
fun() ->
?assertEqual(
Node =/= NewNode,
true,
rabbit_feature_flags:is_enabled(FeatureName)),
ok
end,
Expand Down

0 comments on commit 2acffca

Please sign in to comment.