Skip to content

Commit

Permalink
WIP Decrease memory usage of queue_type state
Browse files Browse the repository at this point in the history
Prior to this commit, 1 MQTT publisher publishing to 1 Million target
classic queues requires around 680 MB of process memory.

After this commit, it requires around 290 MB of process memory.

This commit requires feature flag classic_queue_type_delivery_support
and introduces a new one called no_queue_name_in_classic_queue_client.

Instead of storing the binary queue name 4 times, this commit now stores
it only 1 time.

The monitor_registry is removed since only classic queue clients monitor
their classic queue server processes.

The classic queue client does not store the queue name anymore. Instead
the queue name is included in messages handled by the classic queue
client.

Storing the queue name in the record ctx was unnecessary.

More potential future memory optimisations:
* Use ets continuations when looking up destination queues
* Only fetch ETS columns that are necessary instead of whole queue
  records
* Do not hold the same vhost binary in memory many times. Instead,
  maintain a mapping.
* Remove unnecessary tuple fields.
  • Loading branch information
ansd committed Oct 19, 2022
1 parent f83bb0d commit 8493487
Show file tree
Hide file tree
Showing 16 changed files with 165 additions and 281 deletions.
2 changes: 1 addition & 1 deletion deps/rabbit/src/rabbit_amqqueue.erl
Original file line number Diff line number Diff line change
Expand Up @@ -1745,7 +1745,7 @@ basic_get(Q, NoAck, LimiterPid, CTag, QStates) ->
non_neg_integer(), rabbit_types:ctag(), boolean(),
rabbit_framing:amqp_table(), any(), rabbit_types:username(),
rabbit_queue_type:state()) ->
{ok, rabbit_queue_type:state(), rabbit_queue_type:actions()} |
{ok, rabbit_queue_type:state()} |
{error, term()} |
{protocol_error, Type :: atom(), Reason :: string(), Args :: term()}.
basic_consume(Q, NoAck, ChPid, LimiterPid,
Expand Down
51 changes: 5 additions & 46 deletions deps/rabbit/src/rabbit_channel.erl
Original file line number Diff line number Diff line change
Expand Up @@ -743,27 +743,6 @@ handle_cast({mandatory_received, _MsgSeqNo}, State) ->
%% NB: don't call noreply/1 since we don't want to send confirms.
noreply_coalesce(State);

handle_cast({reject_publish, _MsgSeqNo, QPid} = Evt, State) ->
%% For backwards compatibility
QRef = rabbit_queue_type:find_name_from_pid(QPid, State#ch.queue_states),
case QRef of
undefined ->
%% ignore if no queue could be found for the given pid
noreply(State);
_ ->
handle_cast({queue_event, QRef, Evt}, State)
end;

handle_cast({confirm, _MsgSeqNo, QPid} = Evt, State) ->
%% For backwards compatibility
QRef = rabbit_queue_type:find_name_from_pid(QPid, State#ch.queue_states),
case QRef of
undefined ->
%% ignore if no queue could be found for the given pid
noreply(State);
_ ->
handle_cast({queue_event, QRef, Evt}, State)
end;
handle_cast({queue_event, QRef, Evt},
#ch{queue_states = QueueStates0} = State0) ->
case rabbit_queue_type:handle_event(QRef, Evt, QueueStates0) of
Expand All @@ -786,11 +765,6 @@ handle_cast({queue_event, QRef, Evt},
rabbit_misc:protocol_error(Type, Reason, ReasonArgs)
end.

handle_info({ra_event, {Name, _} = From, Evt}, State) ->
%% For backwards compatibility
QRef = find_queue_name_from_quorum_name(Name, State#ch.queue_states),
handle_cast({queue_event, QRef, {From, Evt}}, State);

handle_info({bump_credit, Msg}, State) ->
%% A rabbit_amqqueue_process is granting credit to our channel. If
%% our channel was being blocked by this process, and no other
Expand All @@ -811,11 +785,11 @@ handle_info(emit_stats, State) ->
%% stats timer.
{noreply, send_confirms_and_nacks(State1), hibernate};

handle_info({'DOWN', _MRef, process, QPid, Reason},
handle_info({{'DOWN', QName}, _MRef, process, QPid, Reason},
#ch{queue_states = QStates0,
queue_monitors = _QMons} = State0) ->
credit_flow:peer_down(QPid),
case rabbit_queue_type:handle_down(QPid, Reason, QStates0) of
case rabbit_queue_type:handle_down(QPid, QName, Reason, QStates0) of
{ok, QState1, Actions} ->
State1 = State0#ch{queue_states = QState1},
State = handle_queue_actions(Actions, State1),
Expand Down Expand Up @@ -1802,7 +1776,7 @@ basic_consume(QueueName, NoAck, ConsumerPrefetch, ActualConsumerTag,
Username, QueueStates0),
Q}
end) of
{{ok, QueueStates, Actions}, Q} when ?is_amqqueue(Q) ->
{{ok, QueueStates}, Q} when ?is_amqqueue(Q) ->
rabbit_global_counters:consumer_created(amqp091),
CM1 = maps:put(
ActualConsumerTag,
Expand All @@ -1811,10 +1785,9 @@ basic_consume(QueueName, NoAck, ConsumerPrefetch, ActualConsumerTag,

State1 = State#ch{consumer_mapping = CM1,
queue_states = QueueStates},
State2 = handle_queue_actions(Actions, State1),
{ok, case NoWait of
true -> consumer_monitor(ActualConsumerTag, State2);
false -> State2
true -> consumer_monitor(ActualConsumerTag, State1);
false -> State1
end};
{{error, exclusive_consume_unavailable} = E, _Q} ->
E;
Expand Down Expand Up @@ -2885,20 +2858,6 @@ handle_queue_actions(Actions, #ch{} = State0) ->

end, State0, Actions).

find_queue_name_from_quorum_name(Name, QStates) ->
Fun = fun(K, _V, undefined) ->
{ok, Q} = rabbit_amqqueue:lookup(K),
case amqqueue:get_pid(Q) of
{Name, _} ->
amqqueue:get_name(Q);
_ ->
undefined
end;
(_, _, Acc) ->
Acc
end,
rabbit_queue_type:fold_state(Fun, undefined, QStates).

maybe_increase_global_publishers(#ch{publishing_mode = true} = State0) ->
State0;
maybe_increase_global_publishers(State0) ->
Expand Down
152 changes: 77 additions & 75 deletions deps/rabbit/src/rabbit_classic_queue.erl
Original file line number Diff line number Diff line change
Expand Up @@ -4,14 +4,20 @@
-include("amqqueue.hrl").
-include_lib("rabbit_common/include/rabbit.hrl").

%% TODO possible to use sets / maps instead of lists?
%% Check performance with QoS 1 and 1 million target queues.
-record(msg_status, {pending :: [pid()],
confirmed = [] :: [pid()]}).

-define(STATE, ?MODULE).
-record(?STATE, {pid :: undefined | pid(), %% the current master pid
qref :: term(), %% TODO
unconfirmed = #{} ::
#{non_neg_integer() => #msg_status{}}}).
-record(?STATE, {
%% the current master pid
pid :: undefined | pid(),
%% undefined if feature flag no_queue_name_in_classic_queue_client enabled
qref :: term(),
unconfirmed = #{} :: #{non_neg_integer() => #msg_status{}},
monitored = #{} :: #{pid() => ok}
}).


-opaque state() :: #?STATE{}.
Expand Down Expand Up @@ -156,9 +162,14 @@ stat(Q) ->

-spec init(amqqueue:amqqueue()) -> {ok, state()}.
init(Q) when ?amqqueue_is_classic(Q) ->
QName = amqqueue:get_name(Q),
QRef = case rabbit_feature_flags:is_enabled(no_queue_name_in_classic_queue_client) of
true ->
undefined;
false ->
amqqueue:get_name(Q)
end,
{ok, #?STATE{pid = amqqueue:get_pid(Q),
qref = QName}}.
qref = QRef}}.

-spec close(state()) -> ok.
close(_State) ->
Expand All @@ -174,7 +185,7 @@ update(Q, #?STATE{pid = Pid} = State) when ?amqqueue_is_classic(Q) ->
State#?STATE{pid = NewPid}
end.

consume(Q, Spec, State) when ?amqqueue_is_classic(Q) ->
consume(Q, Spec, State0) when ?amqqueue_is_classic(Q) ->
QPid = amqqueue:get_pid(Q),
QRef = amqqueue:get_name(Q),
#{no_ack := NoAck,
Expand All @@ -194,9 +205,9 @@ consume(Q, Spec, State) when ?amqqueue_is_classic(Q) ->
ExclusiveConsume, Args, OkMsg, ActingUser},
infinity]}) of
ok ->
%% ask the host process to monitor this pid
%% TODO: track pids as they change
{ok, State#?STATE{pid = QPid}, [{monitor, QPid, QRef}]};
State = ensure_monitor(QPid, QRef, State0),
{ok, State#?STATE{pid = QPid}};
Err ->
Err
end.
Expand Down Expand Up @@ -233,8 +244,10 @@ credit(CTag, Credit, Drain, State) ->
[{credit, ChPid, CTag, Credit, Drain}]}),
{State, []}.

handle_event({confirm, MsgSeqNos, Pid}, #?STATE{qref = QRef,
unconfirmed = U0} = State) ->
handle_event({confirm, MsgSeqNos, Pid}, #?STATE{qref = QRef} = State) ->
%% backwards compatibility when feature flag no_queue_name_in_classic_queue_client disabled
handle_event({confirm, MsgSeqNos, Pid, QRef}, State);
handle_event({confirm, MsgSeqNos, Pid, QRef}, #?STATE{unconfirmed = U0} = State) ->
%% confirms should never result in rejections
{Unconfirmed, ConfirmedSeqNos, []} =
settle_seq_nos(MsgSeqNos, Pid, U0, confirm),
Expand All @@ -247,17 +260,20 @@ handle_event({confirm, MsgSeqNos, Pid}, #?STATE{qref = QRef,
{ok, State#?STATE{unconfirmed = Unconfirmed}, Actions};
handle_event({deliver, _, _, _} = Delivery, #?STATE{} = State) ->
{ok, State, [Delivery]};
handle_event({reject_publish, SeqNo, _QPid},
#?STATE{qref = QRef,
unconfirmed = U0} = State) ->
handle_event({reject_publish, SeqNo, QPid}, #?STATE{qref = QRef} = State) ->
%% backwards compatibility when feature flag no_queue_name_in_classic_queue_client disabled
handle_event({reject_publish, SeqNo, QPid, QRef}, State);
handle_event({reject_publish, SeqNo, _QPid, QRef},
#?STATE{unconfirmed = U0} = State) ->
%% It does not matter which queue rejected the message,
%% if any queue did, it should not be confirmed.
{U, Rejected} = reject_seq_no(SeqNo, U0),
Actions = [{rejected, QRef, Rejected}],
{ok, State#?STATE{unconfirmed = U}, Actions};
handle_event({down, Pid, Info}, #?STATE{qref = QRef,
pid = MasterPid,
unconfirmed = U0} = State0) ->
handle_event({down, Pid, QRef, Info}, #?STATE{monitored = Monitored,
pid = MasterPid,
unconfirmed = U0} = State0) ->
State = State0#?STATE{monitored = maps:remove(Pid, Monitored)},
Actions0 = case Pid =:= MasterPid of
true ->
[{queue_down, QRef}];
Expand All @@ -279,7 +295,7 @@ handle_event({down, Pid, Info}, #?STATE{qref = QRef,
Actions = settlement_action(
settled, QRef, Settled,
settlement_action(rejected, QRef, Rejected, Actions0)),
{ok, State0#?STATE{unconfirmed = Unconfirmed}, Actions};
{ok, State#?STATE{unconfirmed = Unconfirmed}, Actions};
true ->
%% any abnormal exit should be considered a full reject of the
%% oustanding message ids - If the message didn't get to all
Expand All @@ -294,7 +310,7 @@ handle_event({down, Pid, Info}, #?STATE{qref = QRef,
end
end, [], U0),
U = maps:without(MsgIds, U0),
{ok, State0#?STATE{unconfirmed = U},
{ok, State#?STATE{unconfirmed = U},
[{rejected, QRef, MsgIds} | Actions0]}
end;
handle_event({send_drained, _} = Action, State) ->
Expand All @@ -319,7 +335,7 @@ deliver(Qs0, #delivery{flow = Flow,
Msg = Msg0#basic_message{id = rabbit_guid:gen()},
Delivery = Delivery0#delivery{message = Msg},

{MPids, SPids, Qs, Actions} = qpids(Qs0, Confirm, MsgNo),
{MPids, SPids, Qs} = qpids(Qs0, Confirm, MsgNo),
case Flow of
%% Here we are tracking messages sent by the rabbit_channel
%% process. We are accessing the rabbit_channel process
Expand All @@ -333,7 +349,7 @@ deliver(Qs0, #delivery{flow = Flow,
SMsg = {deliver, Delivery, true},
delegate:invoke_no_result(MPids, {gen_server2, cast, [MMsg]}),
delegate:invoke_no_result(SPids, {gen_server2, cast, [SMsg]}),
{Qs, Actions}.
{Qs, []}.


-spec dequeue(NoAck :: boolean(), LimiterPid :: pid(),
Expand Down Expand Up @@ -381,14 +397,16 @@ purge(Q) when ?is_amqqueue(Q) ->

qpids(Qs, Confirm, MsgNo) ->
lists:foldl(
fun ({Q, S0}, {MPidAcc, SPidAcc, Qs0, Actions0}) ->
fun ({Q, S0}, {MPidAcc, SPidAcc, Qs0}) ->
QPid = amqqueue:get_pid(Q),
SPids = amqqueue:get_slave_pids(Q),
QRef = amqqueue:get_name(Q),
Actions = [{monitor, QPid, QRef}
| [{monitor, P, QRef} || P <- SPids]] ++ Actions0,
S1 = ensure_monitor(QPid, QRef, S0),
S2 = lists:foldl(fun(SPid, Acc) ->
ensure_monitor(SPid, QRef, Acc)
end, S1, SPids),
%% confirm record only if necessary
S = case S0 of
S = case S2 of
#?STATE{unconfirmed = U0} ->
Rec = [QPid | SPids],
U = case Confirm of
Expand All @@ -397,14 +415,14 @@ qpids(Qs, Confirm, MsgNo) ->
true ->
U0#{MsgNo => #msg_status{pending = Rec}}
end,
S0#?STATE{pid = QPid,
S2#?STATE{pid = QPid,
unconfirmed = U};
stateless ->
S0
S2
end,
{[QPid | MPidAcc], SPidAcc ++ SPids,
[{Q, S} | Qs0], Actions}
end, {[], [], [], []}, Qs).
[{Q, S} | Qs0]}
end, {[], [], []}, Qs).

%% internal-ish
-spec wait_for_promoted_or_stopped(amqqueue:amqqueue()) ->
Expand Down Expand Up @@ -521,59 +539,43 @@ update_msg_status(confirm, Pid, #msg_status{pending = P,
update_msg_status(down, Pid, #msg_status{pending = P} = S) ->
S#msg_status{pending = lists:delete(Pid, P)}.

ensure_monitor(_, _, State = stateless) ->
State;
ensure_monitor(Pid, _, State = #?STATE{monitored = Monitored})
when is_map_key(Pid, Monitored) ->
State;
ensure_monitor(Pid, QName, State = #?STATE{monitored = Monitored}) ->
_ = erlang:monitor(process, Pid, [{tag, {'DOWN', QName}}]),
State#?STATE{monitored = Monitored#{Pid => ok}}.

%% part of channel <-> queue api
confirm_to_sender(Pid, QName, MsgSeqNos) ->
%% the stream queue included the queue type refactoring and thus requires
%% a different message format
case rabbit_queue_type:is_supported() of
true ->
gen_server:cast(Pid,
{queue_event, QName,
{confirm, MsgSeqNos, self()}});
false ->
gen_server2:cast(Pid, {confirm, MsgSeqNos, self()})
end.
Msg = case rabbit_feature_flags:is_enabled(no_queue_name_in_classic_queue_client) of
true ->
{confirm, MsgSeqNos, self(), QName};
false ->
{confirm, MsgSeqNos, self()}
end,
gen_server:cast(Pid, {queue_event, QName, Msg}).

send_rejection(Pid, QName, MsgSeqNo) ->
case rabbit_queue_type:is_supported() of
true ->
gen_server:cast(Pid, {queue_event, QName,
{reject_publish, MsgSeqNo, self()}});
false ->
gen_server2:cast(Pid, {reject_publish, MsgSeqNo, self()})
end.
Msg = case rabbit_feature_flags:is_enabled(no_queue_name_in_classic_queue_client) of
true ->
{reject_publish, MsgSeqNo, self(), QName};
false ->
{reject_publish, MsgSeqNo, self()}
end,
gen_server:cast(Pid, {queue_event, QName, Msg}).

deliver_to_consumer(Pid, QName, CTag, AckRequired, Message) ->
case has_classic_queue_type_delivery_support() of
true ->
Deliver = {deliver, CTag, AckRequired, [Message]},
Evt = {queue_event, QName, Deliver},
gen_server:cast(Pid, Evt);
false ->
Deliver = {deliver, CTag, AckRequired, Message},
gen_server2:cast(Pid, Deliver)
end.
Deliver = {deliver, CTag, AckRequired, [Message]},
Evt = {queue_event, QName, Deliver},
gen_server:cast(Pid, Evt).

send_drained(Pid, QName, CTagCredits) ->
case has_classic_queue_type_delivery_support() of
true ->
gen_server:cast(Pid, {queue_event, QName,
{send_drained, CTagCredits}});
false ->
gen_server2:cast(Pid, {send_drained, CTagCredits})
end.
gen_server:cast(Pid, {queue_event, QName,
{send_drained, CTagCredits}}).

send_credit_reply(Pid, QName, Len) when is_integer(Len) ->
case rabbit_queue_type:is_supported() of
true ->
gen_server:cast(Pid, {queue_event, QName,
{send_credit_reply, Len}});
false ->
gen_server2:cast(Pid, {send_credit_reply, Len})
end.

has_classic_queue_type_delivery_support() ->
%% some queue_events were missed in the initial queue_type implementation
%% this feature flag enables those and completes the initial queue type
%% API for classic queues
rabbit_feature_flags:is_enabled(classic_queue_type_delivery_support).
gen_server:cast(Pid, {queue_event, QName,
{send_credit_reply, Len}}).
10 changes: 9 additions & 1 deletion deps/rabbit/src/rabbit_core_ff.erl
Original file line number Diff line number Diff line change
Expand Up @@ -108,10 +108,18 @@
{classic_queue_type_delivery_support,
#{desc => "Bug fix for classic queue deliveries using mixed versions",
doc_url => "https://github.com/rabbitmq/rabbitmq-server/issues/5931",
stability => stable,
%%TODO remove compatibility code
stability => required,
depends_on => [stream_queue]
}}).

-rabbit_feature_flag(
{no_queue_name_in_classic_queue_client,
#{desc => "Remove queue name from classic queue type client to save memory",
stability => stable,
depends_on => [classic_queue_type_delivery_support]
}}).

%% -------------------------------------------------------------------
%% Direct exchange routing v2.
%% -------------------------------------------------------------------
Expand Down
Loading

0 comments on commit 8493487

Please sign in to comment.