Skip to content

Commit 2ceacb5

Browse files
authored
Merge pull request #13096 from rabbitmq/mergify/bp/v4.0.x/pr-13069
Remove deprecated/unused/old startup checks (backport #13069)
2 parents 0fb3634 + 690dba0 commit 2ceacb5

File tree

7 files changed

+4
-174
lines changed

7 files changed

+4
-174
lines changed

deps/rabbit/BUILD.bazel

Lines changed: 0 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -95,7 +95,6 @@ _APP_ENV = """[
9595
%% writing to the queue index. See the setting
9696
%% queue_index_embed_msgs_below above.
9797
{msg_store_credit_disc_bound, {4000, 800}},
98-
{msg_store_io_batch_size, 4096},
9998
%% see rabbitmq-server#143,
10099
%% rabbitmq-server#949, rabbitmq-server#1098
101100
{credit_flow_default_credit, {400, 200}},
@@ -525,14 +524,6 @@ rabbitmq_integration_suite(
525524
],
526525
)
527526

528-
rabbitmq_suite(
529-
name = "msg_store_SUITE",
530-
size = "small",
531-
deps = [
532-
"//deps/rabbit_common:erlang_app",
533-
],
534-
)
535-
536527
rabbitmq_integration_suite(
537528
name = "peer_discovery_classic_config_SUITE",
538529
size = "large",

deps/rabbit/Makefile

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -78,7 +78,6 @@ define PROJECT_ENV
7878
%% writing to the queue index. See the setting
7979
%% queue_index_embed_msgs_below above.
8080
{msg_store_credit_disc_bound, {4000, 800}},
81-
{msg_store_io_batch_size, 4096},
8281
%% see rabbitmq-server#143,
8382
%% rabbitmq-server#949, rabbitmq-server#1098
8483
{credit_flow_default_credit, {400, 200}},
@@ -192,7 +191,6 @@ SLOW_CT_SUITES := backing_queue \
192191
health_check \
193192
many_node_ha \
194193
metrics \
195-
msg_store \
196194
partitions \
197195
per_user_connection_tracking \
198196
per_vhost_connection_limit \
@@ -272,7 +270,7 @@ PARALLEL_CT_SET_2_D = queue_length_limits queue_parallel quorum_queue_member_rec
272270
PARALLEL_CT_SET_3_A = definition_import per_user_connection_channel_limit_partitions per_vhost_connection_limit_partitions policy priority_queue_recovery rabbit_fifo_prop rabbit_fifo_v0 rabbit_stream_sac_coordinator unit_credit_flow unit_queue_consumers unit_queue_location unit_quorum_queue
273271
PARALLEL_CT_SET_3_B = cluster_upgrade list_consumers_sanity_check list_queues_online_and_offline logging lqueue maintenance_mode rabbit_fifo_q
274272
PARALLEL_CT_SET_3_C = cli_forget_cluster_node feature_flags_v2 mc_unit message_containers_deaths_v2 message_size_limit metadata_store_migration
275-
PARALLEL_CT_SET_3_D = metadata_store_phase1 metrics mirrored_supervisor msg_store peer_discovery_classic_config proxy_protocol runtime_parameters unit_stats_and_metrics unit_supervisor2 unit_vm_memory_monitor
273+
PARALLEL_CT_SET_3_D = metadata_store_phase1 metrics mirrored_supervisor peer_discovery_classic_config proxy_protocol runtime_parameters unit_stats_and_metrics unit_supervisor2 unit_vm_memory_monitor
276274

277275
PARALLEL_CT_SET_4_A = clustering_events rabbit_local_random_exchange rabbit_message_interceptor rabbitmq_4_0_deprecations unit_pg_local unit_plugin_directories unit_plugin_versioning unit_policy_validators unit_priority_queue
278276
PARALLEL_CT_SET_4_B = per_user_connection_tracking per_vhost_connection_limit rabbit_fifo_dlx_integration rabbit_fifo_int

deps/rabbit/app.bzl

Lines changed: 0 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -1066,15 +1066,6 @@ def test_suite_beam_files(name = "test_suite_beam_files"):
10661066
beam = ["ebin/mirrored_supervisor.beam"],
10671067
erlc_opts = "//:test_erlc_opts",
10681068
)
1069-
erlang_bytecode(
1070-
name = "msg_store_SUITE_beam_files",
1071-
testonly = True,
1072-
srcs = ["test/msg_store_SUITE.erl"],
1073-
outs = ["test/msg_store_SUITE.beam"],
1074-
app_name = "rabbit",
1075-
erlc_opts = "//:test_erlc_opts",
1076-
deps = ["//deps/rabbit_common:erlang_app"],
1077-
)
10781069
erlang_bytecode(
10791070
name = "peer_discovery_classic_config_SUITE_beam_files",
10801071
testonly = True,

deps/rabbit/ct.test.spec

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -60,7 +60,6 @@
6060
, metadata_store_phase1_SUITE
6161
, metrics_SUITE
6262
, mirrored_supervisor_SUITE
63-
, msg_store_SUITE
6463
, peer_discovery_classic_config_SUITE
6564
]}.
6665

deps/rabbit/src/rabbit.erl

Lines changed: 0 additions & 95 deletions
Original file line numberDiff line numberDiff line change
@@ -42,9 +42,6 @@
4242
pg_local_amqp_session/0,
4343
pg_local_amqp_connection/0]).
4444

45-
%% for tests
46-
-export([validate_msg_store_io_batch_size_and_credit_disc_bound/2]).
47-
4845
-rabbit_boot_step({pre_boot, [{description, "rabbit boot start"}]}).
4946

5047
-rabbit_boot_step({codec_correctness_check,
@@ -291,8 +288,6 @@
291288

292289
-define(APPS, [os_mon, mnesia, rabbit_common, rabbitmq_prelaunch, ra, sysmon_handler, rabbit, osiris]).
293290

294-
-define(DIRTY_IO_SCHEDULERS_WARNING_THRESHOLD, 10).
295-
296291
%% 1 minute
297292
-define(BOOT_START_TIMEOUT, 1 * 60 * 1000).
298293
%% 12 hours
@@ -971,7 +966,6 @@ start(normal, []) ->
971966
print_banner(),
972967
log_banner(),
973968
warn_if_kernel_config_dubious(),
974-
warn_if_disc_io_options_dubious(),
975969

976970
?LOG_DEBUG(""),
977971
?LOG_DEBUG("== Plugins (prelaunch phase) =="),
@@ -1434,14 +1428,6 @@ warn_if_kernel_config_dubious() ->
14341428
#{domain => ?RMQLOG_DOMAIN_GLOBAL})
14351429
end
14361430
end,
1437-
DirtyIOSchedulers = erlang:system_info(dirty_io_schedulers),
1438-
case DirtyIOSchedulers < ?DIRTY_IO_SCHEDULERS_WARNING_THRESHOLD of
1439-
true -> ?LOG_WARNING(
1440-
"Erlang VM is running with ~b dirty I/O schedulers, "
1441-
"file I/O performance may worsen", [DirtyIOSchedulers],
1442-
#{domain => ?RMQLOG_DOMAIN_GLOBAL});
1443-
false -> ok
1444-
end,
14451431
IDCOpts = case application:get_env(kernel, inet_default_connect_options) of
14461432
undefined -> [];
14471433
{ok, Val} -> Val
@@ -1453,87 +1439,6 @@ warn_if_kernel_config_dubious() ->
14531439
true -> ok
14541440
end.
14551441

1456-
warn_if_disc_io_options_dubious() ->
1457-
%% if these values are not set, it doesn't matter since
1458-
%% rabbit_variable_queue will pick up the values defined in the
1459-
%% IO_BATCH_SIZE and CREDIT_DISC_BOUND constants.
1460-
CreditDiscBound = rabbit_misc:get_env(rabbit, msg_store_credit_disc_bound,
1461-
undefined),
1462-
IoBatchSize = rabbit_misc:get_env(rabbit, msg_store_io_batch_size,
1463-
undefined),
1464-
case catch validate_msg_store_io_batch_size_and_credit_disc_bound(
1465-
CreditDiscBound, IoBatchSize) of
1466-
ok -> ok;
1467-
{error, {Reason, Vars}} ->
1468-
?LOG_WARNING(Reason, Vars,
1469-
#{domain => ?RMQLOG_DOMAIN_GLOBAL})
1470-
end.
1471-
1472-
validate_msg_store_io_batch_size_and_credit_disc_bound(CreditDiscBound,
1473-
IoBatchSize) ->
1474-
case IoBatchSize of
1475-
undefined ->
1476-
ok;
1477-
IoBatchSize when is_integer(IoBatchSize) ->
1478-
if IoBatchSize < ?IO_BATCH_SIZE ->
1479-
throw({error,
1480-
{"io_batch_size of ~b lower than recommended value ~b, "
1481-
"paging performance may worsen",
1482-
[IoBatchSize, ?IO_BATCH_SIZE]}});
1483-
true ->
1484-
ok
1485-
end;
1486-
IoBatchSize ->
1487-
throw({error,
1488-
{"io_batch_size should be an integer, but ~b given",
1489-
[IoBatchSize]}})
1490-
end,
1491-
1492-
%% CreditDiscBound = {InitialCredit, MoreCreditAfter}
1493-
{RIC, RMCA} = ?CREDIT_DISC_BOUND,
1494-
case CreditDiscBound of
1495-
undefined ->
1496-
ok;
1497-
{IC, MCA} when is_integer(IC), is_integer(MCA) ->
1498-
if IC < RIC; MCA < RMCA ->
1499-
throw({error,
1500-
{"msg_store_credit_disc_bound {~b, ~b} lower than"
1501-
"recommended value {~b, ~b},"
1502-
" paging performance may worsen",
1503-
[IC, MCA, RIC, RMCA]}});
1504-
true ->
1505-
ok
1506-
end;
1507-
{IC, MCA} ->
1508-
throw({error,
1509-
{"both msg_store_credit_disc_bound values should be integers, but ~tp given",
1510-
[{IC, MCA}]}});
1511-
CreditDiscBound ->
1512-
throw({error,
1513-
{"invalid msg_store_credit_disc_bound value given: ~tp",
1514-
[CreditDiscBound]}})
1515-
end,
1516-
1517-
case {CreditDiscBound, IoBatchSize} of
1518-
{undefined, undefined} ->
1519-
ok;
1520-
{_CDB, undefined} ->
1521-
ok;
1522-
{undefined, _IBS} ->
1523-
ok;
1524-
{{InitialCredit, _MCA}, IoBatchSize} ->
1525-
if IoBatchSize < InitialCredit ->
1526-
throw(
1527-
{error,
1528-
{"msg_store_io_batch_size ~b should be bigger than the initial "
1529-
"credit value from msg_store_credit_disc_bound ~b,"
1530-
" paging performance may worsen",
1531-
[IoBatchSize, InitialCredit]}});
1532-
true ->
1533-
ok
1534-
end
1535-
end.
1536-
15371442
-spec product_name() -> string().
15381443

15391444
product_name() ->

deps/rabbit/src/rabbit_variable_queue.erl

Lines changed: 3 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -210,7 +210,7 @@
210210
disk_read_count,
211211
disk_write_count,
212212

213-
io_batch_size,
213+
io_batch_size, %% Unused.
214214

215215
%% default queue or lazy queue
216216
mode, %% Unused.
@@ -334,7 +334,7 @@
334334
disk_read_count :: non_neg_integer(),
335335
disk_write_count :: non_neg_integer(),
336336

337-
io_batch_size :: pos_integer(),
337+
io_batch_size :: 0,
338338
mode :: 'default' | 'lazy',
339339
version :: 2,
340340
unconfirmed_simple :: sets:set()}.
@@ -1195,8 +1195,6 @@ init(IsDurable, IndexState, StoreState, DeltaCount, DeltaBytes, Terms,
11951195
end_seq_id = NextSeqId })
11961196
end,
11971197
Now = erlang:monotonic_time(),
1198-
IoBatchSize = rabbit_misc:get_env(rabbit, msg_store_io_batch_size,
1199-
?IO_BATCH_SIZE),
12001198

12011199
{ok, IndexMaxSize} = application:get_env(
12021200
rabbit, queue_index_embed_msgs_below),
@@ -1242,7 +1240,7 @@ init(IsDurable, IndexState, StoreState, DeltaCount, DeltaBytes, Terms,
12421240
disk_read_count = 0,
12431241
disk_write_count = 0,
12441242

1245-
io_batch_size = IoBatchSize,
1243+
io_batch_size = 0,
12461244

12471245
mode = default,
12481246
virtual_host = VHost},

deps/rabbit/test/msg_store_SUITE.erl

Lines changed: 0 additions & 52 deletions
This file was deleted.

0 commit comments

Comments
 (0)