Merge pull request #14125 from rabbitmq/rabbitmq-server-14087-take-2
Re-submit #14087 by @SimonUnge: introduce an opinionated, opt-in way to prevent a node from booting if it's been reset in the past
This commit is contained in:
commit
b75fc23770
|
@ -264,7 +264,7 @@ PARALLEL_CT_SET_1_D = amqqueue_backward_compatibility channel_interceptor channe
|
|||
|
||||
PARALLEL_CT_SET_2_A = cluster confirms_rejects consumer_timeout rabbit_access_control rabbit_confirms rabbit_core_metrics_gc rabbit_cuttlefish rabbit_db_binding rabbit_db_exchange
|
||||
PARALLEL_CT_SET_2_B = clustering_recovery crashing_queues deprecated_features direct_exchange_routing_v2 disconnect_detected_during_alarm exchanges unit_gen_server2
|
||||
PARALLEL_CT_SET_2_C = disk_monitor dynamic_qq unit_disk_monitor unit_file_handle_cache unit_log_management unit_operator_policy
|
||||
PARALLEL_CT_SET_2_C = disk_monitor dynamic_qq unit_disk_monitor unit_file_handle_cache unit_log_management unit_operator_policy prevent_startup_if_node_was_reset
|
||||
PARALLEL_CT_SET_2_D = queue_length_limits queue_parallel quorum_queue_member_reconciliation rabbit_fifo rabbit_fifo_dlx rabbit_stream_coordinator
|
||||
|
||||
PARALLEL_CT_SET_3_A = definition_import per_user_connection_channel_limit_partitions per_vhost_connection_limit_partitions policy priority_queue_recovery rabbit_fifo_v0 rabbit_stream_sac_coordinator_v4 rabbit_stream_sac_coordinator unit_credit_flow unit_queue_consumers unit_queue_location unit_quorum_queue
|
||||
|
|
|
@ -43,6 +43,7 @@
|
|||
, disk_monitor_SUITE
|
||||
, dynamic_qq_SUITE
|
||||
, exchanges_SUITE
|
||||
, prevent_startup_if_node_was_reset_SUITE
|
||||
, rabbit_stream_queue_SUITE
|
||||
]}.
|
||||
|
||||
|
|
|
@ -1646,6 +1646,16 @@ end}.
|
|||
{datatype, string}
|
||||
]}.
|
||||
|
||||
|
||||
%% Whether to verify if this is the first time a node starts.
|
||||
%% When enabled, nodes will create a marker file on first startup
|
||||
%% and refuse to start if the marker exists but tables are empty.
|
||||
%%
|
||||
|
||||
{mapping, "prevent_startup_if_node_was_reset", "rabbit.prevent_startup_if_node_was_reset", [
|
||||
{datatype, {enum, [true, false]}}
|
||||
]}.
|
||||
|
||||
% ==========================
|
||||
% Logging section
|
||||
% ==========================
|
||||
|
|
|
@ -40,7 +40,7 @@
|
|||
%% Boot steps.
|
||||
-export([update_cluster_tags/0, maybe_insert_default_data/0, boot_delegate/0, recover/0,
|
||||
pg_local_amqp_session/0,
|
||||
pg_local_amqp_connection/0]).
|
||||
pg_local_amqp_connection/0, prevent_startup_if_node_was_reset/0]).
|
||||
|
||||
-rabbit_boot_step({pre_boot, [{description, "rabbit boot start"}]}).
|
||||
|
||||
|
@ -199,10 +199,16 @@
|
|||
{requires, [core_initialized]},
|
||||
{enables, routing_ready}]}).
|
||||
|
||||
-rabbit_boot_step({prevent_startup_if_node_was_reset,
|
||||
[{description, "prevents node boot if a prior boot marker file exists but the database is not seeded (requires opt-in configuration in rabbitmq.conf)"},
|
||||
{mfa, {?MODULE, prevent_startup_if_node_was_reset, []}},
|
||||
{requires, recovery},
|
||||
{enables, empty_db_check}]}).
|
||||
|
||||
-rabbit_boot_step({empty_db_check,
|
||||
[{description, "empty DB check"},
|
||||
{mfa, {?MODULE, maybe_insert_default_data, []}},
|
||||
{requires, recovery},
|
||||
{requires, prevent_startup_if_node_was_reset},
|
||||
{enables, routing_ready}]}).
|
||||
|
||||
|
||||
|
@ -235,6 +241,7 @@
|
|||
{requires, [core_initialized, recovery]},
|
||||
{enables, routing_ready}]}).
|
||||
|
||||
|
||||
-rabbit_boot_step({pre_flight,
|
||||
[{description, "ready to communicate with peers and clients"},
|
||||
{requires, [core_initialized, recovery, routing_ready]}]}).
|
||||
|
@ -1151,6 +1158,44 @@ update_cluster_tags() ->
|
|||
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
|
||||
rabbit_runtime_parameters:set_global(cluster_tags, Tags, <<"internal_user">>).
|
||||
|
||||
|
||||
-spec prevent_startup_if_node_was_reset() -> 'ok' | no_return().
|
||||
|
||||
prevent_startup_if_node_was_reset() ->
|
||||
case application:get_env(rabbit, prevent_startup_if_node_was_reset, false) of
|
||||
false ->
|
||||
%% Feature is disabled, skip the check
|
||||
?LOG_DEBUG("prevent_startup_if_node_was_reset is disabled",
|
||||
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
|
||||
ok;
|
||||
true ->
|
||||
%% Feature is enabled, perform the check
|
||||
DataDir = data_dir(),
|
||||
MarkerFile = filename:join(DataDir, "node_initialized.marker"),
|
||||
case filelib:is_file(MarkerFile) of
|
||||
true ->
|
||||
%% Not the first run, check if tables need default data
|
||||
case rabbit_table:needs_default_data() of
|
||||
true ->
|
||||
?LOG_ERROR("Node has already been initialized, but database appears empty. "
|
||||
"This could indicate data loss or a split-brain scenario.",
|
||||
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
|
||||
throw({error, cluster_already_initialized_but_tables_empty});
|
||||
false ->
|
||||
?LOG_INFO("Node has already been initialized, proceeding with normal startup",
|
||||
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
|
||||
ok
|
||||
end;
|
||||
false ->
|
||||
%% First time starting, create the marker file
|
||||
?LOG_INFO("First node startup detected, creating initialization marker",
|
||||
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
|
||||
ok = filelib:ensure_dir(MarkerFile),
|
||||
ok = file:write_file(MarkerFile, <<>>, [exclusive]), % Empty file.
|
||||
ok
|
||||
end
|
||||
end.
|
||||
|
||||
-spec maybe_insert_default_data() -> 'ok'.
|
||||
|
||||
maybe_insert_default_data() ->
|
||||
|
|
|
@ -0,0 +1,168 @@
|
|||
%% This Source Code Form is subject to the terms of the Mozilla Public
|
||||
%% License, v. 2.0. If a copy of the MPL was not distributed with this
|
||||
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
|
||||
%%
|
||||
%% Copyright (c) 2007-2024 Broadcom. All Rights Reserved. The term "Broadcom" refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
|
||||
%%
|
||||
|
||||
%% Test suite for the prevent_startup_if_node_was_reset feature.
|
||||
%% This feature helps detect potential data loss scenarios by maintaining
|
||||
%% a marker file to track if a node has been initialized before.
|
||||
|
||||
-module(prevent_startup_if_node_was_reset_SUITE).
|
||||
|
||||
-include_lib("common_test/include/ct.hrl").
|
||||
-include_lib("eunit/include/eunit.hrl").
|
||||
|
||||
-compile(export_all).
|
||||
|
||||
all() ->
|
||||
[
|
||||
{group, single_node_mnesia},
|
||||
{group, single_node_khepri}
|
||||
].
|
||||
|
||||
groups() ->
|
||||
[
|
||||
{single_node_mnesia, [], [
|
||||
prevent_startup_if_node_was_reset_disabled,
|
||||
prevent_startup_if_node_was_reset_enabled
|
||||
]},
|
||||
{single_node_khepri, [], [
|
||||
prevent_startup_if_node_was_reset_disabled,
|
||||
prevent_startup_if_node_was_reset_enabled
|
||||
]}
|
||||
].
|
||||
|
||||
%% -------------------------------------------------------------------
|
||||
%% Testsuite setup/teardown.
|
||||
%% -------------------------------------------------------------------
|
||||
|
||||
init_per_suite(Config) ->
|
||||
rabbit_ct_helpers:log_environment(),
|
||||
rabbit_ct_helpers:run_setup_steps(Config).
|
||||
|
||||
end_per_suite(Config) ->
|
||||
rabbit_ct_helpers:run_teardown_steps(Config).
|
||||
|
||||
init_per_group(Groupname, Config) ->
|
||||
Config0 = rabbit_ct_helpers:set_config(Config, [
|
||||
{metadata_store, meta_store(Groupname)},
|
||||
{rmq_nodes_clustered, false},
|
||||
{rmq_nodename_suffix, Groupname},
|
||||
{rmq_nodes_count, 1}
|
||||
]),
|
||||
rabbit_ct_helpers:run_steps(
|
||||
Config0,
|
||||
rabbit_ct_broker_helpers:setup_steps() ++
|
||||
rabbit_ct_client_helpers:setup_steps()
|
||||
).
|
||||
|
||||
end_per_group(_, Config) ->
|
||||
rabbit_ct_helpers:run_steps(
|
||||
Config,
|
||||
rabbit_ct_client_helpers:teardown_steps() ++
|
||||
rabbit_ct_broker_helpers:teardown_steps()
|
||||
).
|
||||
|
||||
init_per_testcase(Testcase, Config) ->
|
||||
rabbit_ct_helpers:testcase_started(Config, Testcase),
|
||||
Config.
|
||||
|
||||
end_per_testcase(Testcase, Config) ->
|
||||
rabbit_ct_helpers:testcase_finished(Config, Testcase).
|
||||
|
||||
%% -------------------------------------------------------------------
|
||||
%% Test cases
|
||||
%% -------------------------------------------------------------------
|
||||
|
||||
prevent_startup_if_node_was_reset_disabled(Config) ->
|
||||
% When feature is disabled (default), node should start normally
|
||||
DataDir = rabbit_ct_broker_helpers:get_node_config(Config, 0, data_dir),
|
||||
MarkerFile = filename:join(DataDir, "node_initialized.marker"),
|
||||
% Setting is disabled so no marker file should be present
|
||||
?assertNot(filelib:is_file(MarkerFile)),
|
||||
|
||||
% Restarting the node should work fine
|
||||
ok = stop_app(Config),
|
||||
set_env(Config, false),
|
||||
ok = start_app(Config),
|
||||
% Still no marker file
|
||||
?assertNot(filelib:is_file(MarkerFile)),
|
||||
ok.
|
||||
|
||||
prevent_startup_if_node_was_reset_enabled(Config) ->
|
||||
DataDir = rabbit_ct_broker_helpers:get_node_config(Config, 0, data_dir),
|
||||
MarkerFile = filename:join(DataDir, "node_initialized.marker"),
|
||||
|
||||
ok = stop_app(Config),
|
||||
set_env(Config, true),
|
||||
ok = start_app(Config),
|
||||
% Setting is enabled so marker file should be present after initial startup
|
||||
?assert(filelib:is_file(MarkerFile)),
|
||||
|
||||
% Restarting the node should be fine, as there is a marker file
|
||||
% and corresponding schema data (consistent state)
|
||||
|
||||
ok = stop_app(Config),
|
||||
ok = start_app(Config),
|
||||
|
||||
SchemaFile = schema_file(Config),
|
||||
|
||||
?assert(filelib:is_file(MarkerFile)),
|
||||
|
||||
% Stop the node and remove the present schema to simulate data loss
|
||||
ok = stop_app(Config),
|
||||
file:delete(SchemaFile),
|
||||
% Node should fail to start because marker exists but schema is missing,
|
||||
% indicating potential data loss or corruption
|
||||
?assertMatch(
|
||||
{error, 69, _},
|
||||
start_app(Config)
|
||||
),
|
||||
ok.
|
||||
|
||||
%% -------------------------------------------------------------------
|
||||
%% Internal helpers
|
||||
%% -------------------------------------------------------------------
|
||||
|
||||
stop_app(Config) ->
|
||||
Node = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
|
||||
case rabbit_ct_broker_helpers:rabbitmqctl(Config, Node, ["stop_app"]) of
|
||||
{ok, _} -> ok;
|
||||
Error -> Error
|
||||
end.
|
||||
|
||||
start_app(Config) ->
|
||||
Node = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
|
||||
case rabbit_ct_broker_helpers:rabbitmqctl(Config, Node, ["start_app"]) of
|
||||
{ok, _} -> ok;
|
||||
Error -> Error
|
||||
end.
|
||||
|
||||
maybe_enable_prevent_startup_if_node_was_reset(Config, prevent_startup_if_node_was_reset_enabled) ->
|
||||
rabbit_ct_helpers:merge_app_env(
|
||||
Config, {rabbit, [{prevent_startup_if_node_was_reset, true}]}
|
||||
);
|
||||
maybe_enable_prevent_startup_if_node_was_reset(Config, _) ->
|
||||
Config.
|
||||
|
||||
meta_store(single_node_mnesia) ->
|
||||
mnesia;
|
||||
meta_store(single_node_khepri) ->
|
||||
khepri.
|
||||
|
||||
schema_file(Config) ->
|
||||
DataDir = rabbit_ct_broker_helpers:get_node_config(Config, 0, data_dir),
|
||||
MetaStore = rabbit_ct_helpers:get_config(Config, metadata_store),
|
||||
case MetaStore of
|
||||
mnesia ->
|
||||
filename:join(DataDir, "schema.DAT");
|
||||
khepri ->
|
||||
NodeName = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
|
||||
filename:join([DataDir, "coordination", NodeName, "names.dets"])
|
||||
end.
|
||||
|
||||
set_env(Config, Bool) ->
|
||||
Node = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
|
||||
ok = rpc:call(Node, application, set_env, [rabbit, prevent_startup_if_node_was_reset, Bool]).
|
Loading…
Reference in New Issue