Skip to content

Commit

Permalink
Adjust vheap sizes for message handling processes in OTP 27
Browse files Browse the repository at this point in the history
OTP 27 reset all assumptions on how the vm reacts to processes that
buffer and process a lot of large binaries.

Substantially increasing the vheap sizes for such process restores
most of the same performance by allowing processes to hold more binary
data before major garbage collections are triggered.

This introduces a new module to capture process flag configurations.

The new vheap sizes are only applied when running on OTP 27 or
above.
  • Loading branch information
kjnilsson committed Sep 9, 2024
1 parent f50e8b9 commit bebd21d
Show file tree
Hide file tree
Showing 7 changed files with 60 additions and 2 deletions.
3 changes: 3 additions & 0 deletions deps/rabbit/app.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -192,6 +192,7 @@ def all_beam_files(name = "all_beam_files"):
"src/rabbit_prelaunch_logging.erl",
"src/rabbit_priority_queue.erl",
"src/rabbit_process.erl",
"src/rabbit_process_flag.erl",
"src/rabbit_queue_consumers.erl",
"src/rabbit_queue_decorator.erl",
"src/rabbit_queue_index.erl",
Expand Down Expand Up @@ -448,6 +449,7 @@ def all_test_beam_files(name = "all_test_beam_files"):
"src/rabbit_prelaunch_logging.erl",
"src/rabbit_priority_queue.erl",
"src/rabbit_process.erl",
"src/rabbit_process_flag.erl",
"src/rabbit_queue_consumers.erl",
"src/rabbit_queue_decorator.erl",
"src/rabbit_queue_index.erl",
Expand Down Expand Up @@ -727,6 +729,7 @@ def all_srcs(name = "all_srcs"):
"src/rabbit_prelaunch_logging.erl",
"src/rabbit_priority_queue.erl",
"src/rabbit_process.erl",
"src/rabbit_process_flag.erl",
"src/rabbit_queue_consumers.erl",
"src/rabbit_queue_decorator.erl",
"src/rabbit_queue_index.erl",
Expand Down
2 changes: 1 addition & 1 deletion deps/rabbit/src/rabbit_amqp_session.erl
Original file line number Diff line number Diff line change
Expand Up @@ -384,7 +384,7 @@ init({ReaderPid, WriterPid, ChannelNum, MaxFrameSize, User, Vhost, ConnName,
outgoing_window = ?UINT(RemoteOutgoingWindow),
handle_max = ClientHandleMax}}) ->
process_flag(trap_exit, true),
process_flag(message_queue_data, off_heap),
rabbit_process_flag:adjust_for_message_handling_proc(),

ok = pg:join(pg_scope(), self(), self()),
Alarms0 = rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}),
Expand Down
15 changes: 15 additions & 0 deletions deps/rabbit/src/rabbit_channel.erl
Original file line number Diff line number Diff line change
Expand Up @@ -485,6 +485,8 @@ update_user_state(Pid, UserState) when is_pid(Pid) ->
init([Channel, ReaderPid, WriterPid, ConnPid, ConnName, Protocol, User, VHost,
Capabilities, CollectorPid, LimiterPid, AmqpParams]) ->
process_flag(trap_exit, true),
rabbit_process_flag:adjust_for_message_handling_proc(),

?LG_PROCESS_TYPE(channel),
?store_proc_name({ConnName, Channel}),
ok = pg_local:join(rabbit_channels, self()),
Expand Down Expand Up @@ -2784,3 +2786,16 @@ maybe_decrease_global_publishers(#ch{publishing_mode = true}) ->

is_global_qos_permitted() ->
rabbit_deprecated_features:is_permitted(global_qos).

adjust_vheap() ->
case code_version:get_otp_version() of
OtpMaj when OtpMaj >= 27 ->
%% 46422 is the default min_bin_vheap_size and for OTP 27 and above
%% we want to substantially increase it for processes that may buffer
%% messages. 32x has proven workable in testing whilst not being
%% rediculously large
process_flag(min_bin_vheap_size, 46422 * 32);
_ ->
ok
end.

32 changes: 32 additions & 0 deletions deps/rabbit/src/rabbit_process_flag.erl
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
%% 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.
%%

-module(rabbit_process_flag).


-export([adjust_for_message_handling_proc/0
]).

%% @doc Enqueues a message.
%% Adjust process flags for processes that handle RabbitMQ messages.
%% For example any process that uses the `rabbit_queue_type' module
%% may benefit from this tuning.
%% @returns `ok'
-spec adjust_for_message_handling_proc() -> ok.
adjust_for_message_handling_proc() ->
process_flag(message_queue_data, off_heap),
case code_version:get_otp_version() of
OtpMaj when OtpMaj >= 27 ->
%% 46422 is the default min_bin_vheap_size and for OTP 27 and above
%% we want to substantially increase it for processes that may buffer
%% messages. 32x has proven workable in testing whilst not being
%% rediculously large
process_flag(min_bin_vheap_size, 46422 * 32),
ok;
_ ->
ok
end.
8 changes: 7 additions & 1 deletion deps/rabbit/src/rabbit_ra_systems.erl
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,13 @@ get_config(quorum_queues = RaSystem) ->
AERBatchSize = application:get_env(rabbit, quorum_max_append_entries_rpc_batch_size,
?QUORUM_AER_MAX_RPC_SIZE),
CompressMemTables = application:get_env(rabbit, quorum_compress_mem_tables, true),
MinBinVheapSize = ?MIN_BIN_VHEAP_SIZE_DEFAULT * ?MIN_BIN_VHEAP_SIZE_MULT,
MinBinVheapSize = case code_version:get_otp_version() of
OtpMaj when OtpMaj >= 27 ->
?MIN_BIN_VHEAP_SIZE_DEFAULT * ?MIN_BIN_VHEAP_SIZE_MULT;
_ ->
?MIN_BIN_VHEAP_SIZE_DEFAULT
end,

DefaultConfig#{name => RaSystem,
wal_min_bin_vheap_size => MinBinVheapSize,
server_min_bin_vheap_size => MinBinVheapSize,
Expand Down
1 change: 1 addition & 0 deletions deps/rabbit_common/src/code_version.erl
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,7 @@ get_forms(Code) ->
throw({no_abstract_code, Reason})
end.

-spec get_otp_version() -> non_neg_integer().
get_otp_version() ->
Version = erlang:system_info(otp_release),
case re:run(Version, "^[0-9][0-9]", [{capture, first, list}]) of
Expand Down
1 change: 1 addition & 0 deletions moduleindex.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -694,6 +694,7 @@ rabbit:
- rabbit_prelaunch_logging
- rabbit_priority_queue
- rabbit_process
- rabbit_process_flag
- rabbit_queue_consumers
- rabbit_queue_decorator
- rabbit_queue_index
Expand Down

0 comments on commit bebd21d

Please sign in to comment.