Merge pull request #11513 from rabbitmq/loic-remove-ram-durations
CQ: Remove rabbit_memory_monitor and RAM durations
This commit is contained in:
commit
7c2d29d1e4
|
|
@ -164,7 +164,6 @@ def all_beam_files(name = "all_beam_files"):
|
|||
"src/rabbit_logger_exchange_h.erl",
|
||||
"src/rabbit_looking_glass.erl",
|
||||
"src/rabbit_maintenance.erl",
|
||||
"src/rabbit_memory_monitor.erl",
|
||||
"src/rabbit_message_interceptor.erl",
|
||||
"src/rabbit_metrics.erl",
|
||||
"src/rabbit_mirror_queue_misc.erl",
|
||||
|
|
@ -424,7 +423,6 @@ def all_test_beam_files(name = "all_test_beam_files"):
|
|||
"src/rabbit_logger_exchange_h.erl",
|
||||
"src/rabbit_looking_glass.erl",
|
||||
"src/rabbit_maintenance.erl",
|
||||
"src/rabbit_memory_monitor.erl",
|
||||
"src/rabbit_message_interceptor.erl",
|
||||
"src/rabbit_metrics.erl",
|
||||
"src/rabbit_mirror_queue_misc.erl",
|
||||
|
|
@ -703,7 +701,6 @@ def all_srcs(name = "all_srcs"):
|
|||
"src/rabbit_logger_exchange_h.erl",
|
||||
"src/rabbit_looking_glass.erl",
|
||||
"src/rabbit_maintenance.erl",
|
||||
"src/rabbit_memory_monitor.erl",
|
||||
"src/rabbit_message_interceptor.erl",
|
||||
"src/rabbit_metrics.erl",
|
||||
"src/rabbit_mirror_queue_misc.erl",
|
||||
|
|
|
|||
|
|
@ -148,13 +148,6 @@
|
|||
[{description, "kernel ready"},
|
||||
{requires, external_infrastructure}]}).
|
||||
|
||||
-rabbit_boot_step({rabbit_memory_monitor,
|
||||
[{description, "memory monitor"},
|
||||
{mfa, {rabbit_sup, start_restartable_child,
|
||||
[rabbit_memory_monitor]}},
|
||||
{requires, rabbit_alarm},
|
||||
{enables, core_initialized}]}).
|
||||
|
||||
-rabbit_boot_step({guid_generator,
|
||||
[{description, "guid generator"},
|
||||
{mfa, {rabbit_sup, start_restartable_child,
|
||||
|
|
|
|||
|
|
@ -74,7 +74,7 @@
|
|||
|
||||
%% internal
|
||||
-export([internal_declare/2, internal_delete/2, run_backing_queue/3,
|
||||
set_ram_duration_target/2, set_maximum_since_use/2,
|
||||
set_maximum_since_use/2,
|
||||
emit_consumers_local/3, internal_delete/3]).
|
||||
|
||||
-include_lib("rabbit_common/include/rabbit.hrl").
|
||||
|
|
@ -1790,11 +1790,6 @@ forget_node_for_queue(Q) ->
|
|||
run_backing_queue(QPid, Mod, Fun) ->
|
||||
gen_server2:cast(QPid, {run_backing_queue, Mod, Fun}).
|
||||
|
||||
-spec set_ram_duration_target(pid(), number() | 'infinity') -> 'ok'.
|
||||
|
||||
set_ram_duration_target(QPid, Duration) ->
|
||||
gen_server2:cast(QPid, {set_ram_duration_target, Duration}).
|
||||
|
||||
-spec set_maximum_since_use(pid(), non_neg_integer()) -> 'ok'.
|
||||
|
||||
set_maximum_since_use(QPid, Age) ->
|
||||
|
|
|
|||
|
|
@ -11,9 +11,9 @@
|
|||
|
||||
-behaviour(gen_server2).
|
||||
|
||||
-define(SYNC_INTERVAL, 200). %% milliseconds
|
||||
-define(RAM_DURATION_UPDATE_INTERVAL, 5000).
|
||||
-define(CONSUMER_BIAS_RATIO, 2.0). %% i.e. consume 100% faster
|
||||
-define(SYNC_INTERVAL, 200). %% milliseconds
|
||||
-define(UPDATE_RATES_INTERVAL, 5000).
|
||||
-define(CONSUMER_BIAS_RATIO, 2.0). %% i.e. consume 100% faster
|
||||
|
||||
-export([info_keys/0]).
|
||||
|
||||
|
|
@ -48,7 +48,7 @@
|
|||
expires,
|
||||
%% timer used to periodically sync (flush) queue index
|
||||
sync_timer_ref,
|
||||
%% timer used to update ingress/egress rates and queue RAM duration target
|
||||
%% timer used to update ingress/egress rates
|
||||
rate_timer_ref,
|
||||
%% timer used to clean up this queue due to TTL (on when unused)
|
||||
expiry_timer_ref,
|
||||
|
|
@ -212,9 +212,6 @@ init_it2(Recover, From, State = #q{q = Q,
|
|||
true ->
|
||||
ok = file_handle_cache:register_callback(
|
||||
rabbit_amqqueue, set_maximum_since_use, [self()]),
|
||||
ok = rabbit_memory_monitor:register(
|
||||
self(), {rabbit_amqqueue,
|
||||
set_ram_duration_target, [self()]}),
|
||||
BQ = backing_queue_module(),
|
||||
BQS = bq_init(BQ, Q, TermsOrNew),
|
||||
send_reply(From, {new, Q}),
|
||||
|
|
@ -364,8 +361,7 @@ terminate_shutdown(Fun, #q{status = Status} = State) ->
|
|||
fun stop_ttl_timer/1]),
|
||||
case BQS of
|
||||
undefined -> State1;
|
||||
_ -> ok = rabbit_memory_monitor:deregister(self()),
|
||||
QName = qname(State),
|
||||
_ -> QName = qname(State),
|
||||
notify_decorators(shutdown, State),
|
||||
[emit_consumer_deleted(Ch, CTag, QName, ActingUser) ||
|
||||
{Ch, CTag, _, _, _, _, _, _} <-
|
||||
|
|
@ -529,8 +525,8 @@ stop_sync_timer(State) -> rabbit_misc:stop_timer(State, #q.sync_timer_ref).
|
|||
|
||||
ensure_rate_timer(State) ->
|
||||
rabbit_misc:ensure_timer(State, #q.rate_timer_ref,
|
||||
?RAM_DURATION_UPDATE_INTERVAL,
|
||||
update_ram_duration).
|
||||
?UPDATE_RATES_INTERVAL,
|
||||
update_rates).
|
||||
|
||||
stop_rate_timer(State) -> rabbit_misc:stop_timer(State, #q.rate_timer_ref).
|
||||
|
||||
|
|
@ -1193,7 +1189,6 @@ prioritise_cast(Msg, _Len, State) ->
|
|||
case Msg of
|
||||
delete_immediately -> 8;
|
||||
{delete_exclusive, _Pid} -> 8;
|
||||
{set_ram_duration_target, _Duration} -> 8;
|
||||
{set_maximum_since_use, _Age} -> 8;
|
||||
{run_backing_queue, _Mod, _Fun} -> 6;
|
||||
{ack, _AckTags, _ChPid} -> 4; %% [1]
|
||||
|
|
@ -1221,13 +1216,12 @@ consumer_bias(#q{backing_queue = BQ, backing_queue_state = BQS}, Low, High) ->
|
|||
prioritise_info(Msg, _Len, #q{q = Q}) ->
|
||||
DownPid = amqqueue:get_exclusive_owner(Q),
|
||||
case Msg of
|
||||
{'DOWN', _, process, DownPid, _} -> 8;
|
||||
update_ram_duration -> 8;
|
||||
{maybe_expire, _Version} -> 8;
|
||||
{drop_expired, _Version} -> 8;
|
||||
emit_stats -> 7;
|
||||
sync_timeout -> 6;
|
||||
_ -> 0
|
||||
{'DOWN', _, process, DownPid, _} -> 8;
|
||||
{maybe_expire, _Version} -> 8;
|
||||
{drop_expired, _Version} -> 8;
|
||||
emit_stats -> 7;
|
||||
sync_timeout -> 6;
|
||||
_ -> 0
|
||||
end.
|
||||
|
||||
handle_call({init, Recover}, From, State) ->
|
||||
|
|
@ -1505,11 +1499,6 @@ handle_cast({deactivate_limit, ChPid}, State) ->
|
|||
noreply(possibly_unblock(rabbit_queue_consumers:deactivate_limit_fun(),
|
||||
ChPid, State));
|
||||
|
||||
handle_cast({set_ram_duration_target, Duration},
|
||||
State = #q{backing_queue = BQ, backing_queue_state = BQS}) ->
|
||||
BQS1 = BQ:set_ram_duration_target(Duration, BQS),
|
||||
noreply(State#q{backing_queue_state = BQS1});
|
||||
|
||||
handle_cast({set_maximum_since_use, Age}, State) ->
|
||||
ok = file_handle_cache:set_maximum_since_use(Age),
|
||||
noreply(State);
|
||||
|
|
@ -1659,15 +1648,12 @@ handle_info({'DOWN', _MonitorRef, process, DownPid, _Reason}, State) ->
|
|||
{stop, State1} -> stop(State1)
|
||||
end;
|
||||
|
||||
handle_info(update_ram_duration, State = #q{backing_queue = BQ,
|
||||
backing_queue_state = BQS}) ->
|
||||
{RamDuration, BQS1} = BQ:ram_duration(BQS),
|
||||
DesiredDuration =
|
||||
rabbit_memory_monitor:report_ram_duration(self(), RamDuration),
|
||||
BQS2 = BQ:set_ram_duration_target(DesiredDuration, BQS1),
|
||||
handle_info(update_rates, State = #q{backing_queue = BQ,
|
||||
backing_queue_state = BQS}) ->
|
||||
BQS1 = BQ:update_rates(BQS),
|
||||
%% Don't call noreply/1, we don't want to set timers
|
||||
{State1, Timeout} = next_state(State#q{rate_timer_ref = undefined,
|
||||
backing_queue_state = BQS2}),
|
||||
backing_queue_state = BQS1}),
|
||||
{noreply, State1, Timeout};
|
||||
|
||||
handle_info(sync_timeout, State) ->
|
||||
|
|
@ -1697,11 +1683,8 @@ handle_pre_hibernate(State = #q{backing_queue_state = undefined}) ->
|
|||
{hibernate, State};
|
||||
handle_pre_hibernate(State = #q{backing_queue = BQ,
|
||||
backing_queue_state = BQS}) ->
|
||||
{RamDuration, BQS1} = BQ:ram_duration(BQS),
|
||||
DesiredDuration =
|
||||
rabbit_memory_monitor:report_ram_duration(self(), RamDuration),
|
||||
BQS2 = BQ:set_ram_duration_target(DesiredDuration, BQS1),
|
||||
BQS3 = BQ:handle_pre_hibernate(BQS2),
|
||||
BQS1 = BQ:update_rates(BQS),
|
||||
BQS3 = BQ:handle_pre_hibernate(BQS1),
|
||||
rabbit_event:if_enabled(
|
||||
State, #q.stats_timer,
|
||||
fun () -> emit_stats(State,
|
||||
|
|
|
|||
|
|
@ -33,7 +33,6 @@
|
|||
-type purged_msg_count() :: non_neg_integer().
|
||||
-type async_callback() ::
|
||||
fun ((atom(), fun ((atom(), state()) -> state())) -> 'ok').
|
||||
-type duration() :: ('undefined' | 'infinity' | number()).
|
||||
|
||||
-type msg_fun(A) :: fun ((mc:state(), ack(), A) -> A).
|
||||
-type msg_pred() :: fun ((rabbit_types:message_properties()) -> boolean()).
|
||||
|
|
@ -190,21 +189,8 @@
|
|||
%% What's the queue depth, where depth = length + number of pending acks
|
||||
-callback depth(state()) -> non_neg_integer().
|
||||
|
||||
%% For the next three functions, the assumption is that you're
|
||||
%% monitoring something like the ingress and egress rates of the
|
||||
%% queue. The RAM duration is thus the length of time represented by
|
||||
%% the messages held in RAM given the current rates. If you want to
|
||||
%% ignore all of this stuff, then do so, and return 0 in
|
||||
%% ram_duration/1.
|
||||
|
||||
%% The target is to have no more messages in RAM than indicated by the
|
||||
%% duration and the current queue rates.
|
||||
-callback set_ram_duration_target(duration(), state()) -> state().
|
||||
|
||||
%% Optionally recalculate the duration internally (likely to be just
|
||||
%% update your internal rates), and report how many seconds the
|
||||
%% messages in RAM represent given the current rates of the queue.
|
||||
-callback ram_duration(state()) -> {duration(), state()}.
|
||||
%% Update the internal message rates.
|
||||
-callback update_rates(state()) -> state().
|
||||
|
||||
%% Should 'timeout' be called as soon as the queue process can manage
|
||||
%% (either on an empty mailbox, or when a timer fires)?
|
||||
|
|
|
|||
|
|
@ -1,262 +0,0 @@
|
|||
%% 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.
|
||||
%%
|
||||
|
||||
|
||||
%% This module handles the node-wide memory statistics.
|
||||
%% It receives statistics from all queues, counts the desired
|
||||
%% queue length (in seconds), and sends this information back to
|
||||
%% queues.
|
||||
|
||||
-module(rabbit_memory_monitor).
|
||||
|
||||
-behaviour(gen_server2).
|
||||
|
||||
-export([start_link/0, register/2, deregister/1,
|
||||
report_ram_duration/2, stop/0, conserve_resources/3, memory_use/1]).
|
||||
|
||||
-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
|
||||
terminate/2, code_change/3]).
|
||||
|
||||
-record(process, {pid, reported, sent, callback, monitor}).
|
||||
|
||||
-record(state, {timer, %% 'internal_update' timer
|
||||
queue_durations, %% ets #process
|
||||
queue_duration_sum, %% sum of all queue_durations
|
||||
queue_duration_count, %% number of elements in sum
|
||||
desired_duration, %% the desired queue duration
|
||||
disk_alarm %% disable paging, disk alarm has fired
|
||||
}).
|
||||
|
||||
-define(SERVER, ?MODULE).
|
||||
-define(TABLE_NAME, ?MODULE).
|
||||
|
||||
%% If all queues are pushed to disk (duration 0), then the sum of
|
||||
%% their reported lengths will be 0. If memory then becomes available,
|
||||
%% unless we manually intervene, the sum will remain 0, and the queues
|
||||
%% will never get a non-zero duration. Thus when the mem use is <
|
||||
%% SUM_INC_THRESHOLD, increase the sum artificially by SUM_INC_AMOUNT.
|
||||
-define(SUM_INC_THRESHOLD, 0.95).
|
||||
-define(SUM_INC_AMOUNT, 1.0).
|
||||
|
||||
-define(EPSILON, 0.000001). %% less than this and we clamp to 0
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
%% Public API
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
-spec start_link() -> rabbit_types:ok_pid_or_error().
|
||||
|
||||
start_link() ->
|
||||
gen_server2:start_link({local, ?SERVER}, ?MODULE, [], []).
|
||||
|
||||
-spec register(pid(), {atom(),atom(),[any()]}) -> 'ok'.
|
||||
|
||||
register(Pid, MFA = {_M, _F, _A}) ->
|
||||
gen_server2:call(?SERVER, {register, Pid, MFA}, infinity).
|
||||
|
||||
-spec deregister(pid()) -> 'ok'.
|
||||
|
||||
deregister(Pid) ->
|
||||
gen_server2:cast(?SERVER, {deregister, Pid}).
|
||||
|
||||
-spec report_ram_duration
|
||||
(pid(), float() | 'infinity') -> number() | 'infinity'.
|
||||
|
||||
report_ram_duration(Pid, QueueDuration) ->
|
||||
gen_server2:call(?SERVER,
|
||||
{report_ram_duration, Pid, QueueDuration}, infinity).
|
||||
|
||||
-spec stop() -> 'ok'.
|
||||
|
||||
stop() ->
|
||||
gen_server2:cast(?SERVER, stop).
|
||||
|
||||
-spec conserve_resources(pid(),
|
||||
rabbit_alarm:resource_alarm_source(),
|
||||
rabbit_alarm:resource_alert()) -> ok.
|
||||
%% Paging should be enabled/disabled only in response to disk resource alarms
|
||||
%% for the current node.
|
||||
conserve_resources(Pid, disk, {_, Conserve, Node}) when node(Pid) =:= Node ->
|
||||
gen_server2:cast(Pid, {disk_alarm, Conserve});
|
||||
conserve_resources(_Pid, _Source, _Conserve) ->
|
||||
ok.
|
||||
|
||||
memory_use(Type) ->
|
||||
vm_memory_monitor:get_memory_use(Type).
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
%% Gen_server callbacks
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
init([]) ->
|
||||
{ok, Interval} = application:get_env(rabbit, memory_monitor_interval),
|
||||
{ok, TRef} = timer:send_interval(Interval, update),
|
||||
|
||||
Ets = ets:new(?TABLE_NAME, [set, private, {keypos, #process.pid}]),
|
||||
Alarms = rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}),
|
||||
{ok, internal_update(
|
||||
#state { timer = TRef,
|
||||
queue_durations = Ets,
|
||||
queue_duration_sum = 0.0,
|
||||
queue_duration_count = 0,
|
||||
desired_duration = infinity,
|
||||
disk_alarm = lists:member(disk, Alarms)})}.
|
||||
|
||||
handle_call({report_ram_duration, Pid, QueueDuration}, From,
|
||||
State = #state { queue_duration_sum = Sum,
|
||||
queue_duration_count = Count,
|
||||
queue_durations = Durations,
|
||||
desired_duration = SendDuration }) ->
|
||||
|
||||
[Proc = #process { reported = PrevQueueDuration }] =
|
||||
ets:lookup(Durations, Pid),
|
||||
|
||||
gen_server2:reply(From, SendDuration),
|
||||
|
||||
{Sum1, Count1} =
|
||||
case {PrevQueueDuration, QueueDuration} of
|
||||
{infinity, infinity} -> {Sum, Count};
|
||||
{infinity, _} -> {Sum + QueueDuration, Count + 1};
|
||||
{_, infinity} -> {Sum - PrevQueueDuration, Count - 1};
|
||||
{_, _} -> {Sum - PrevQueueDuration + QueueDuration,
|
||||
Count}
|
||||
end,
|
||||
true = ets:insert(Durations, Proc #process { reported = QueueDuration,
|
||||
sent = SendDuration }),
|
||||
{noreply, State #state { queue_duration_sum = zero_clamp(Sum1),
|
||||
queue_duration_count = Count1 }};
|
||||
|
||||
handle_call({register, Pid, MFA}, _From,
|
||||
State = #state { queue_durations = Durations }) ->
|
||||
MRef = erlang:monitor(process, Pid),
|
||||
true = ets:insert(Durations, #process { pid = Pid, reported = infinity,
|
||||
sent = infinity, callback = MFA,
|
||||
monitor = MRef }),
|
||||
{reply, ok, State};
|
||||
|
||||
handle_call(_Request, _From, State) ->
|
||||
{noreply, State}.
|
||||
|
||||
handle_cast({disk_alarm, Alarm}, State = #state{disk_alarm = Alarm}) ->
|
||||
{noreply, State};
|
||||
|
||||
handle_cast({disk_alarm, Alarm}, State) ->
|
||||
{noreply, internal_update(State#state{disk_alarm = Alarm})};
|
||||
|
||||
handle_cast({deregister, Pid}, State) ->
|
||||
{noreply, internal_deregister(Pid, true, State)};
|
||||
|
||||
handle_cast(stop, State) ->
|
||||
{stop, normal, State};
|
||||
|
||||
handle_cast(_Request, State) ->
|
||||
{noreply, State}.
|
||||
|
||||
handle_info(update, State) ->
|
||||
{noreply, internal_update(State)};
|
||||
|
||||
handle_info({'DOWN', _MRef, process, Pid, _Reason}, State) ->
|
||||
{noreply, internal_deregister(Pid, false, State)};
|
||||
|
||||
handle_info(_Info, State) ->
|
||||
{noreply, State}.
|
||||
|
||||
terminate(_Reason, #state { timer = TRef }) ->
|
||||
_ = timer:cancel(TRef),
|
||||
ok.
|
||||
|
||||
code_change(_OldVsn, State, _Extra) ->
|
||||
{ok, State}.
|
||||
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
%% Internal functions
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
zero_clamp(Sum) when Sum < ?EPSILON -> 0.0;
|
||||
zero_clamp(Sum) -> Sum.
|
||||
|
||||
internal_deregister(Pid, Demonitor,
|
||||
State = #state { queue_duration_sum = Sum,
|
||||
queue_duration_count = Count,
|
||||
queue_durations = Durations }) ->
|
||||
case ets:lookup(Durations, Pid) of
|
||||
[] -> State;
|
||||
[#process { reported = PrevQueueDuration, monitor = MRef }] ->
|
||||
true = case Demonitor of
|
||||
true -> erlang:demonitor(MRef);
|
||||
false -> true
|
||||
end,
|
||||
{Sum1, Count1} =
|
||||
case PrevQueueDuration of
|
||||
infinity -> {Sum, Count};
|
||||
_ -> {zero_clamp(Sum - PrevQueueDuration),
|
||||
Count - 1}
|
||||
end,
|
||||
true = ets:delete(Durations, Pid),
|
||||
State #state { queue_duration_sum = Sum1,
|
||||
queue_duration_count = Count1 }
|
||||
end.
|
||||
|
||||
internal_update(State = #state{queue_durations = Durations,
|
||||
desired_duration = DesiredDurationAvg,
|
||||
disk_alarm = DiskAlarm}) ->
|
||||
DesiredDurationAvg1 = desired_duration_average(State),
|
||||
ShouldInform = should_inform_predicate(DiskAlarm),
|
||||
case ShouldInform(DesiredDurationAvg, DesiredDurationAvg1) of
|
||||
true -> inform_queues(ShouldInform, DesiredDurationAvg1, Durations);
|
||||
false -> ok
|
||||
end,
|
||||
State#state{desired_duration = DesiredDurationAvg1}.
|
||||
|
||||
desired_duration_average(#state{disk_alarm = true}) ->
|
||||
infinity;
|
||||
desired_duration_average(#state{disk_alarm = false,
|
||||
queue_duration_sum = Sum,
|
||||
queue_duration_count = Count}) ->
|
||||
{ok, LimitThreshold} =
|
||||
application:get_env(rabbit, vm_memory_high_watermark_paging_ratio),
|
||||
MemoryRatio = memory_use(ratio),
|
||||
if MemoryRatio =:= infinity ->
|
||||
0.0;
|
||||
MemoryRatio < LimitThreshold orelse Count == 0 ->
|
||||
infinity;
|
||||
MemoryRatio < ?SUM_INC_THRESHOLD ->
|
||||
((Sum + ?SUM_INC_AMOUNT) / Count) / MemoryRatio;
|
||||
true ->
|
||||
(Sum / Count) / MemoryRatio
|
||||
end.
|
||||
|
||||
inform_queues(ShouldInform, DesiredDurationAvg, Durations) ->
|
||||
true =
|
||||
ets:foldl(
|
||||
fun (Proc = #process{reported = QueueDuration,
|
||||
sent = PrevSendDuration,
|
||||
callback = {M, F, A}}, true) ->
|
||||
case ShouldInform(PrevSendDuration, DesiredDurationAvg)
|
||||
andalso ShouldInform(QueueDuration, DesiredDurationAvg) of
|
||||
true -> ok = erlang:apply(
|
||||
M, F, A ++ [DesiredDurationAvg]),
|
||||
ets:insert(
|
||||
Durations,
|
||||
Proc#process{sent = DesiredDurationAvg});
|
||||
false -> true
|
||||
end
|
||||
end, true, Durations).
|
||||
|
||||
%% In normal use, we only inform queues immediately if the desired
|
||||
%% duration has decreased, we want to ensure timely paging.
|
||||
should_inform_predicate(false) -> fun greater_than/2;
|
||||
%% When the disk alarm has gone off though, we want to inform queues
|
||||
%% immediately if the desired duration has *increased* - we want to
|
||||
%% ensure timely stopping paging.
|
||||
should_inform_predicate(true) -> fun (D1, D2) -> greater_than(D2, D1) end.
|
||||
|
||||
greater_than(infinity, infinity) -> false;
|
||||
greater_than(infinity, _D2) -> true;
|
||||
greater_than(_D1, infinity) -> false;
|
||||
greater_than(D1, D2) -> D1 > D2.
|
||||
|
|
@ -29,7 +29,7 @@
|
|||
publish/5, publish_delivered/4, discard/3, drain_confirmed/1,
|
||||
dropwhile/2, fetchwhile/4, fetch/2, drop/2, ack/2, requeue/2,
|
||||
ackfold/4, fold/3, len/1, is_empty/1, depth/1,
|
||||
set_ram_duration_target/2, ram_duration/1, needs_timeout/1, timeout/1,
|
||||
update_rates/1, needs_timeout/1, timeout/1,
|
||||
handle_pre_hibernate/1, resume/1, msg_rates/1,
|
||||
info/2, invoke/3, is_duplicate/2, set_queue_mode/2,
|
||||
set_queue_version/2,
|
||||
|
|
@ -326,18 +326,10 @@ depth(#state{bq = BQ, bqss = BQSs}) ->
|
|||
depth(#passthrough{bq = BQ, bqs = BQS}) ->
|
||||
BQ:depth(BQS).
|
||||
|
||||
set_ram_duration_target(DurationTarget, State = #state{bq = BQ}) ->
|
||||
foreach1(fun (_P, BQSN) ->
|
||||
BQ:set_ram_duration_target(DurationTarget, BQSN)
|
||||
end, State);
|
||||
set_ram_duration_target(DurationTarget,
|
||||
State = #passthrough{bq = BQ, bqs = BQS}) ->
|
||||
?passthrough1(set_ram_duration_target(DurationTarget, BQS)).
|
||||
|
||||
ram_duration(State = #state{bq = BQ}) ->
|
||||
fold_min2(fun (_P, BQSN) -> BQ:ram_duration(BQSN) end, State);
|
||||
ram_duration(State = #passthrough{bq = BQ, bqs = BQS}) ->
|
||||
?passthrough2(ram_duration(BQS)).
|
||||
update_rates(State = #state{bq = BQ}) ->
|
||||
fold_min2(fun (_P, BQSN) -> BQ:update_rates(BQSN) end, State);
|
||||
update_rates(State = #passthrough{bq = BQ, bqs = BQS}) ->
|
||||
?passthrough1(update_rates(BQS)).
|
||||
|
||||
needs_timeout(#state{bq = BQ, bqss = BQSs}) ->
|
||||
fold0(fun (_P, _BQSN, timed) -> timed;
|
||||
|
|
|
|||
|
|
@ -13,7 +13,7 @@
|
|||
discard/3, drain_confirmed/1,
|
||||
dropwhile/2, fetchwhile/4, fetch/2, drop/2, ack/2, requeue/2,
|
||||
ackfold/4, fold/3, len/1, is_empty/1, depth/1,
|
||||
set_ram_duration_target/2, ram_duration/1, needs_timeout/1, timeout/1,
|
||||
update_rates/1, needs_timeout/1, timeout/1,
|
||||
handle_pre_hibernate/1, resume/1, msg_rates/1,
|
||||
info/2, invoke/3, is_duplicate/2, set_queue_mode/2,
|
||||
set_queue_version/2, zip_msgs_and_acks/4]).
|
||||
|
|
@ -350,12 +350,12 @@
|
|||
|
||||
-define(MICROS_PER_SECOND, 1000000.0).
|
||||
|
||||
%% We're sampling every 5s for RAM duration; a half life that is of
|
||||
%% We're updating rates every 5s at most; a half life that is of
|
||||
%% the same order of magnitude is probably about right.
|
||||
-define(RATE_AVG_HALF_LIFE, 5.0).
|
||||
|
||||
%% We will recalculate the #rates{} every time we get asked for our
|
||||
%% RAM duration, or every N messages published, whichever is
|
||||
%% We will recalculate the #rates{} every 5 seconds,
|
||||
%% or every N messages published, whichever is
|
||||
%% sooner. We do this since the priority calculations in
|
||||
%% rabbit_amqqueue_process need fairly fresh rates.
|
||||
-define(MSGS_PER_RATE_CALC, 100).
|
||||
|
|
@ -670,9 +670,6 @@ is_empty(State) -> 0 == len(State).
|
|||
depth(State) ->
|
||||
len(State) + count_pending_acks(State).
|
||||
|
||||
set_ram_duration_target(_DurationTarget, State) ->
|
||||
State.
|
||||
|
||||
maybe_update_rates(State = #vqstate{ in_counter = InCount,
|
||||
out_counter = OutCount })
|
||||
when InCount + OutCount > ?MSGS_PER_RATE_CALC ->
|
||||
|
|
@ -712,12 +709,6 @@ update_rate(Now, TS, Count, Rate) ->
|
|||
Count / Time, Rate)
|
||||
end.
|
||||
|
||||
%% @todo Should be renamed since it's only used to update_rates.
|
||||
%% Can do this after mirroring gets removed.
|
||||
ram_duration(State) ->
|
||||
State1 = update_rates(State),
|
||||
{infinity, State1}.
|
||||
|
||||
needs_timeout(#vqstate { index_state = IndexState,
|
||||
unconfirmed_simple = UCS }) ->
|
||||
case {rabbit_classic_queue_index_v2:needs_sync(IndexState), sets:is_empty(UCS)} of
|
||||
|
|
|
|||
|
|
@ -27,10 +27,8 @@
|
|||
variable_queue_drop,
|
||||
variable_queue_fold_msg_on_disk,
|
||||
variable_queue_dropfetchwhile,
|
||||
variable_queue_dropwhile_varying_ram_duration,
|
||||
variable_queue_dropwhile_restart,
|
||||
variable_queue_dropwhile_sync_restart,
|
||||
variable_queue_fetchwhile_varying_ram_duration,
|
||||
variable_queue_ack_limiting,
|
||||
variable_queue_purge,
|
||||
variable_queue_requeue,
|
||||
|
|
@ -1018,8 +1016,6 @@ bq_variable_queue_delete_msg_store_files_callback1(Config) ->
|
|||
Count = 30,
|
||||
QTState = publish_and_confirm(Q, Payload, Count),
|
||||
|
||||
rabbit_amqqueue:set_ram_duration_target(QPid, 0),
|
||||
|
||||
{ok, Limiter} = rabbit_limiter:start_link(no_id),
|
||||
|
||||
CountMinusOne = Count - 1,
|
||||
|
|
@ -1111,17 +1107,16 @@ variable_queue_partial_segments_delta_thing2(VQ0, _QName) ->
|
|||
HalfSegment = SegmentSize div 2,
|
||||
OneAndAHalfSegment = SegmentSize + HalfSegment,
|
||||
VQ1 = variable_queue_publish(true, OneAndAHalfSegment, VQ0),
|
||||
{_Duration, VQ2} = rabbit_variable_queue:ram_duration(VQ1),
|
||||
VQ2 = rabbit_variable_queue:update_rates(VQ1),
|
||||
VQ3 = check_variable_queue_status(
|
||||
variable_queue_set_ram_duration_target(0, VQ2),
|
||||
VQ2,
|
||||
%% We only have one message in memory because the amount in memory
|
||||
%% depends on the consume rate, which is nil in this test.
|
||||
[{delta, {delta, 1, OneAndAHalfSegment - 1, 0, OneAndAHalfSegment}},
|
||||
{q3, 1},
|
||||
{len, OneAndAHalfSegment}]),
|
||||
VQ4 = variable_queue_set_ram_duration_target(infinity, VQ3),
|
||||
VQ5 = check_variable_queue_status(
|
||||
variable_queue_publish(true, 1, VQ4),
|
||||
variable_queue_publish(true, 1, VQ3),
|
||||
%% one alpha, but it's in the same segment as the deltas
|
||||
%% @todo That's wrong now! v1/v2
|
||||
[{delta, {delta, 1, OneAndAHalfSegment, 0, OneAndAHalfSegment + 1}},
|
||||
|
|
@ -1158,9 +1153,8 @@ variable_queue_all_the_bits_not_covered_elsewhere_A2(VQ0, QName) ->
|
|||
Count = 2 * IndexMod:next_segment_boundary(0),
|
||||
VQ1 = variable_queue_publish(true, Count, VQ0),
|
||||
VQ2 = variable_queue_publish(false, Count, VQ1),
|
||||
VQ3 = variable_queue_set_ram_duration_target(0, VQ2),
|
||||
{VQ4, _AckTags} = variable_queue_fetch(Count, true, false,
|
||||
Count + Count, VQ3),
|
||||
Count + Count, VQ2),
|
||||
{VQ5, _AckTags1} = variable_queue_fetch(Count, false, false,
|
||||
Count, VQ4),
|
||||
_VQ6 = rabbit_variable_queue:terminate(shutdown, VQ5),
|
||||
|
|
@ -1168,8 +1162,7 @@ variable_queue_all_the_bits_not_covered_elsewhere_A2(VQ0, QName) ->
|
|||
{{_Msg1, true, _AckTag1}, VQ8} = rabbit_variable_queue:fetch(true, VQ7),
|
||||
Count1 = rabbit_variable_queue:len(VQ8),
|
||||
VQ9 = variable_queue_publish(false, 1, VQ8),
|
||||
VQ10 = variable_queue_set_ram_duration_target(0, VQ9),
|
||||
{VQ11, _AckTags2} = variable_queue_fetch(Count1, true, true, Count, VQ10),
|
||||
{VQ11, _AckTags2} = variable_queue_fetch(Count1, true, true, Count, VQ9),
|
||||
{VQ12, _AckTags3} = variable_queue_fetch(1, false, false, 1, VQ11),
|
||||
VQ12.
|
||||
|
||||
|
|
@ -1182,8 +1175,7 @@ variable_queue_all_the_bits_not_covered_elsewhere_B1(Config) ->
|
|||
fun variable_queue_all_the_bits_not_covered_elsewhere_B2/2,
|
||||
?config(variable_queue_type, Config)).
|
||||
|
||||
variable_queue_all_the_bits_not_covered_elsewhere_B2(VQ0, QName) ->
|
||||
VQ1 = variable_queue_set_ram_duration_target(0, VQ0),
|
||||
variable_queue_all_the_bits_not_covered_elsewhere_B2(VQ1, QName) ->
|
||||
VQ2 = variable_queue_publish(false, 4, VQ1),
|
||||
{VQ3, AckTags} = variable_queue_fetch(2, false, false, 4, VQ2),
|
||||
{_Guids, VQ4} =
|
||||
|
|
@ -1364,51 +1356,6 @@ variable_queue_dropwhile_sync_restart2(VQ0, QName) ->
|
|||
|
||||
VQ5.
|
||||
|
||||
variable_queue_dropwhile_varying_ram_duration(Config) ->
|
||||
passed = rabbit_ct_broker_helpers:rpc(Config, 0,
|
||||
?MODULE, variable_queue_dropwhile_varying_ram_duration1, [Config]).
|
||||
|
||||
variable_queue_dropwhile_varying_ram_duration1(Config) ->
|
||||
with_fresh_variable_queue(
|
||||
fun variable_queue_dropwhile_varying_ram_duration2/2,
|
||||
?config(variable_queue_type, Config)).
|
||||
|
||||
variable_queue_dropwhile_varying_ram_duration2(VQ0, _QName) ->
|
||||
test_dropfetchwhile_varying_ram_duration(
|
||||
fun (VQ1) ->
|
||||
{_, VQ2} = rabbit_variable_queue:dropwhile(
|
||||
fun (_) -> false end, VQ1),
|
||||
VQ2
|
||||
end, VQ0).
|
||||
|
||||
variable_queue_fetchwhile_varying_ram_duration(Config) ->
|
||||
passed = rabbit_ct_broker_helpers:rpc(Config, 0,
|
||||
?MODULE, variable_queue_fetchwhile_varying_ram_duration1, [Config]).
|
||||
|
||||
variable_queue_fetchwhile_varying_ram_duration1(Config) ->
|
||||
with_fresh_variable_queue(
|
||||
fun variable_queue_fetchwhile_varying_ram_duration2/2,
|
||||
?config(variable_queue_type, Config)).
|
||||
|
||||
variable_queue_fetchwhile_varying_ram_duration2(VQ0, _QName) ->
|
||||
test_dropfetchwhile_varying_ram_duration(
|
||||
fun (VQ1) ->
|
||||
{_, ok, VQ2} = rabbit_variable_queue:fetchwhile(
|
||||
fun (_) -> false end,
|
||||
fun (_, _, A) -> A end,
|
||||
ok, VQ1),
|
||||
VQ2
|
||||
end, VQ0).
|
||||
|
||||
test_dropfetchwhile_varying_ram_duration(Fun, VQ0) ->
|
||||
VQ1 = variable_queue_publish(false, 1, VQ0),
|
||||
VQ2 = variable_queue_set_ram_duration_target(0, VQ1),
|
||||
VQ3 = Fun(VQ2),
|
||||
VQ4 = variable_queue_set_ram_duration_target(infinity, VQ3),
|
||||
VQ5 = variable_queue_publish(false, 1, VQ4),
|
||||
VQ6 = Fun(VQ5),
|
||||
VQ6.
|
||||
|
||||
variable_queue_ack_limiting(Config) ->
|
||||
passed = rabbit_ct_broker_helpers:rpc(Config, 0,
|
||||
?MODULE, variable_queue_ack_limiting1, [Config]).
|
||||
|
|
@ -1427,8 +1374,8 @@ variable_queue_ack_limiting2(VQ0, _Config) ->
|
|||
Churn = Len div 32,
|
||||
VQ2 = publish_fetch_and_ack(Churn, Len, VQ1),
|
||||
|
||||
%% update stats for duration
|
||||
{_Duration, VQ3} = rabbit_variable_queue:ram_duration(VQ2),
|
||||
%% update stats
|
||||
VQ3 = rabbit_variable_queue:update_rates(VQ2),
|
||||
|
||||
%% fetch half the messages
|
||||
{VQ4, _AckTags} = variable_queue_fetch(Len div 2, false, false, Len, VQ3),
|
||||
|
|
@ -1437,9 +1384,7 @@ variable_queue_ack_limiting2(VQ0, _Config) ->
|
|||
%% that's the only predictable stats we got.
|
||||
VQ5 = check_variable_queue_status(VQ4, [{len, Len div 2}]),
|
||||
|
||||
VQ6 = variable_queue_set_ram_duration_target(0, VQ5),
|
||||
|
||||
VQ6.
|
||||
VQ5.
|
||||
|
||||
variable_queue_purge(Config) ->
|
||||
passed = rabbit_ct_broker_helpers:rpc(Config, 0,
|
||||
|
|
@ -1509,8 +1454,7 @@ variable_queue_requeue_ram_beta2(VQ0, _Config) ->
|
|||
{VQ2, AcksR} = variable_queue_fetch(Count, false, false, Count, VQ1),
|
||||
{Back, Front} = lists:split(Count div 2, AcksR),
|
||||
{_, VQ3} = rabbit_variable_queue:requeue(erlang:tl(Back), VQ2),
|
||||
VQ4 = variable_queue_set_ram_duration_target(0, VQ3),
|
||||
{_, VQ5} = rabbit_variable_queue:requeue([erlang:hd(Back)], VQ4),
|
||||
{_, VQ5} = rabbit_variable_queue:requeue([erlang:hd(Back)], VQ3),
|
||||
VQ6 = requeue_one_by_one(Front, VQ5),
|
||||
{VQ7, AcksAll} = variable_queue_fetch(Count, false, true, Count, VQ6),
|
||||
{_, VQ8} = rabbit_variable_queue:ack(AcksAll, VQ7),
|
||||
|
|
@ -1570,8 +1514,7 @@ variable_queue_mode_change2(VQ0, _Config) ->
|
|||
{Back, Front} = lists:split(Count div 2, AcksR),
|
||||
{_, VQ5} = rabbit_variable_queue:requeue(erlang:tl(Back), VQ4),
|
||||
VQ6 = maybe_switch_queue_mode(VQ5),
|
||||
VQ7 = variable_queue_set_ram_duration_target(0, VQ6),
|
||||
VQ8 = maybe_switch_queue_mode(VQ7),
|
||||
VQ8 = maybe_switch_queue_mode(VQ6),
|
||||
{_, VQ9} = rabbit_variable_queue:requeue([erlang:hd(Back)], VQ8),
|
||||
VQ10 = maybe_switch_queue_mode(VQ9),
|
||||
VQ11 = requeue_one_by_one(Front, VQ10),
|
||||
|
|
@ -1827,10 +1770,6 @@ assert_props(List, PropVals) ->
|
|||
Error -> error(Error -- [ok])
|
||||
end.
|
||||
|
||||
variable_queue_set_ram_duration_target(Duration, VQ) ->
|
||||
variable_queue_wait_for_shuffling_end(
|
||||
rabbit_variable_queue:set_ram_duration_target(Duration, VQ)).
|
||||
|
||||
publish_fetch_and_ack(0, _Len, VQ0) ->
|
||||
VQ0;
|
||||
publish_fetch_and_ack(N, Len, VQ0) ->
|
||||
|
|
@ -1883,8 +1822,7 @@ variable_queue_with_holes(VQ0) ->
|
|||
VQ1 = variable_queue_publish(
|
||||
false, 1, Count,
|
||||
fun (_, P) -> P end, fun erlang:term_to_binary/1, VQ0),
|
||||
VQ2 = variable_queue_set_ram_duration_target(0, VQ1),
|
||||
{VQ3, AcksR} = variable_queue_fetch(Count, false, false, Count, VQ2),
|
||||
{VQ3, AcksR} = variable_queue_fetch(Count, false, false, Count, VQ1),
|
||||
Acks = lists:reverse(AcksR),
|
||||
AckSeqs = lists:zip(Acks, Seq),
|
||||
[{Subset1, _Seq1}, {Subset2, _Seq2}, {Subset3, Seq3}] =
|
||||
|
|
@ -1896,11 +1834,10 @@ variable_queue_with_holes(VQ0) ->
|
|||
VQ5 = requeue_one_by_one(Subset1, VQ4),
|
||||
%% by now we have some messages (and holes) in delta
|
||||
VQ6 = requeue_one_by_one(Subset2, VQ5),
|
||||
VQ7 = variable_queue_set_ram_duration_target(infinity, VQ6),
|
||||
%% add the q1 tail
|
||||
VQ8 = variable_queue_publish(
|
||||
true, Count + 1, Interval,
|
||||
fun (_, P) -> P end, fun erlang:term_to_binary/1, VQ7),
|
||||
fun (_, P) -> P end, fun erlang:term_to_binary/1, VQ6),
|
||||
%% assertions
|
||||
vq_with_holes_assertions(VQ8),
|
||||
Depth = Count + Interval,
|
||||
|
|
|
|||
|
|
@ -14,7 +14,7 @@
|
|||
discard/3, drain_confirmed/1,
|
||||
dropwhile/2, fetchwhile/4, fetch/2, drop/2, ack/2, requeue/2,
|
||||
ackfold/4, fold/3, len/1, is_empty/1, depth/1,
|
||||
set_ram_duration_target/2, ram_duration/1, needs_timeout/1, timeout/1,
|
||||
update_rates/1, needs_timeout/1, timeout/1,
|
||||
handle_pre_hibernate/1, resume/1, msg_rates/1,
|
||||
info/2, invoke/3, is_duplicate/2, set_queue_mode/2,
|
||||
set_queue_version/2,
|
||||
|
|
@ -278,11 +278,8 @@ is_empty(State) -> 0 == len(State).
|
|||
depth(State) ->
|
||||
rabbit_variable_queue:depth(State).
|
||||
|
||||
set_ram_duration_target(DurationTarget, State) ->
|
||||
rabbit_variable_queue:set_ram_duration_target(DurationTarget, State).
|
||||
|
||||
ram_duration(State) ->
|
||||
rabbit_variable_queue:ram_duration(State).
|
||||
update_rates(State) ->
|
||||
rabbit_variable_queue:update_rates(State).
|
||||
|
||||
needs_timeout(State) ->
|
||||
rabbit_variable_queue:needs_timeout(State).
|
||||
|
|
|
|||
|
|
@ -120,7 +120,7 @@ end_per_testcase(Testcase, Config) ->
|
|||
%% * len/1, is_empty/1 - info items
|
||||
%% * handle_pre_hibernate/1 - hibernation
|
||||
%%
|
||||
%% * set_ram_duration_target/2, ram_duration/1, status/1
|
||||
%% * status/1
|
||||
%% - maybe need unit testing?
|
||||
%%
|
||||
%% [0] publish enough to get credit flow from msg store
|
||||
|
|
@ -473,19 +473,6 @@ unknown_info_key(Config) ->
|
|||
rabbit_ct_client_helpers:close_connection(Conn),
|
||||
passed.
|
||||
|
||||
ram_duration(_Config) ->
|
||||
QName = rabbit_misc:r(<<"/">>, queue, <<"ram_duration-queue">>),
|
||||
Q0 = rabbit_amqqueue:pseudo_queue(QName, self()),
|
||||
Q1 = amqqueue:set_arguments(Q0, [{<<"x-max-priority">>, long, 5}]),
|
||||
PQ = rabbit_priority_queue,
|
||||
BQS1 = PQ:init(Q1, new, fun(_, _) -> ok end),
|
||||
{_Duration1, BQS2} = PQ:ram_duration(BQS1),
|
||||
BQS3 = PQ:set_ram_duration_target(infinity, BQS2),
|
||||
BQS4 = PQ:set_ram_duration_target(1, BQS3),
|
||||
{_Duration2, BQS5} = PQ:ram_duration(BQS4),
|
||||
PQ:delete_and_terminate(a_whim, BQS5),
|
||||
passed.
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
declare(Ch, Q, Args) when is_list(Args) ->
|
||||
|
|
|
|||
|
|
@ -680,7 +680,6 @@ rabbit:
|
|||
- rabbit_logger_exchange_h
|
||||
- rabbit_looking_glass
|
||||
- rabbit_maintenance
|
||||
- rabbit_memory_monitor
|
||||
- rabbit_message_interceptor
|
||||
- rabbit_metrics
|
||||
- rabbit_mirror_queue_misc
|
||||
|
|
|
|||
Loading…
Reference in New Issue