Skip to content

Commit 690dba0

Browse files
mkuratczykmergify[bot]
authored andcommitted
Remove msg_store_io_batch_size and msg_store_credit_disc_bound checks
msg_store_io_batch_size is no longer used msg_store_credit_disc_bound appears to be used in the code, but I don't see any impact of that value on the performance. It should be properly investigated and either removed completely or fixed, because there's hardly any point in warning about the values configured (plus, this settings is hopefully almost never used anyway) (cherry picked from commit 14171fb)
1 parent 0f2fed1 commit 690dba0

File tree

7 files changed

+4
-164
lines changed

7 files changed

+4
-164
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 & 85 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,
@@ -969,7 +966,6 @@ start(normal, []) ->
969966
print_banner(),
970967
log_banner(),
971968
warn_if_kernel_config_dubious(),
972-
warn_if_disc_io_options_dubious(),
973969

974970
?LOG_DEBUG(""),
975971
?LOG_DEBUG("== Plugins (prelaunch phase) =="),
@@ -1443,87 +1439,6 @@ warn_if_kernel_config_dubious() ->
14431439
true -> ok
14441440
end.
14451441

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

15291444
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)