rabbitmq-server/deps/rabbit/src/rabbit_variable_queue.erl

3047 lines
133 KiB
Erlang
Raw Normal View History

%% 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/.
2009-10-07 20:53:03 +08:00
%%
2021-01-22 14:00:14 +08:00
%% Copyright (c) 2007-2021 VMware, Inc. or its affiliates. All rights reserved.
2009-10-07 20:53:03 +08:00
%%
-module(rabbit_variable_queue).
-export([init/3, terminate/2, delete_and_terminate/2, delete_crashed/1,
purge/1, purge_acks/1,
publish/6, publish_delivered/5,
batch_publish/4, batch_publish_delivered/4,
discard/4, drain_confirmed/1,
2014-03-04 00:29:55 +08:00
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,
handle_pre_hibernate/1, resume/1, msg_rates/1,
2015-10-10 23:40:23 +08:00
info/2, invoke/3, is_duplicate/2, set_queue_mode/2,
zip_msgs_and_acks/4, multiple_routing_keys/0, handle_info/2]).
-export([start/2, stop/1]).
%% exported for testing only
-export([start_msg_store/3, stop_msg_store/1, init/6]).
-export([move_messages_to_vhost_store/0]).
-export([migrate_queue/3, migrate_message/3, get_per_vhost_store_client/2,
get_global_store_client/1, log_upgrade_verbose/1,
log_upgrade_verbose/2]).
-include_lib("stdlib/include/qlc.hrl").
-define(QUEUE_MIGRATION_BATCH_SIZE, 100).
-define(EMPTY_START_FUN_STATE, {fun (ok) -> finished end, ok}).
2010-01-21 23:07:06 +08:00
%%----------------------------------------------------------------------------
%% Messages, and their position in the queue, can be in memory or on
%% disk, or both. Persistent messages will have both message and
%% position pushed to disk as soon as they arrive; transient messages
%% can be written to disk (and thus both types can be evicted from
%% memory) under memory pressure. The question of whether a message is
%% in RAM and whether it is persistent are orthogonal.
%%
%% Messages are persisted using the queue index and the message
%% store. Normally the queue index holds the position of the message
%% *within this queue* along with a couple of small bits of metadata,
%% while the message store holds the message itself (including headers
%% and other properties).
%%
%% However, as an optimisation, small messages can be embedded
%% directly in the queue index and bypass the message store
%% altogether.
%%
2010-01-21 23:07:06 +08:00
%% Definitions:
%%
2010-01-21 23:07:06 +08:00
%% alpha: this is a message where both the message itself, and its
%% position within the queue are held in RAM
%%
%% beta: this is a message where the message itself is only held on
%% disk (if persisted to the message store) but its position
%% within the queue is held in RAM.
2010-01-21 23:07:06 +08:00
%%
%% gamma: this is a message where the message itself is only held on
%% disk, but its position is both in RAM and on disk.
%%
%% delta: this is a collection of messages, represented by a single
%% term, where the messages and their position are only held on
%% disk.
%%
%% Note that for persistent messages, the message and its position
%% within the queue are always held on disk, *in addition* to being in
%% one of the above classifications.
%%
2011-10-12 23:24:35 +08:00
%% Also note that within this code, the term gamma seldom
%% appears. It's frequently the case that gammas are defined by betas
%% who have had their queue position recorded on disk.
%%
2010-01-21 23:07:06 +08:00
%% In general, messages move q1 -> q2 -> delta -> q3 -> q4, though
%% many of these steps are frequently skipped. q1 and q4 only hold
2011-10-12 23:24:35 +08:00
%% alphas, q2 and q3 hold both betas and gammas. When a message
%% arrives, its classification is determined. It is then added to the
%% rightmost appropriate queue.
%%
2010-01-21 23:07:06 +08:00
%% If a new message is determined to be a beta or gamma, q1 is
%% empty. If a new message is determined to be a delta, q1 and q2 are
%% empty (and actually q4 too).
%%
2010-01-21 23:07:06 +08:00
%% When removing messages from a queue, if q4 is empty then q3 is read
%% directly. If q3 becomes empty then the next segment's worth of
%% messages from delta are read into q3, reducing the size of
%% delta. If the queue is non empty, either q4 or q3 contain
%% entries. It is never permitted for delta to hold all the messages
%% in the queue.
%%
2010-01-21 23:07:06 +08:00
%% The duration indicated to us by the memory_monitor is used to
%% calculate, given our current ingress and egress rates, how many
2011-10-12 23:24:35 +08:00
%% messages we should hold in RAM (i.e. as alphas). We track the
%% ingress and egress rates for both messages and pending acks and
%% rates for both are considered when calculating the number of
%% messages to hold in RAM. When we need to push alphas to betas or
%% betas to gammas, we favour writing out messages that are further
%% from the head of the queue. This minimises writes to disk, as the
%% messages closer to the tail of the queue stay in the queue for
%% longer, thus do not need to be replaced as quickly by sending other
%% messages to disk.
%%
2010-01-21 23:07:06 +08:00
%% Whilst messages are pushed to disk and forgotten from RAM as soon
%% as requested by a new setting of the queue RAM duration, the
%% inverse is not true: we only load messages back into RAM as
%% demanded as the queue is read from. Thus only publishes to the
%% queue will take up available spare capacity.
%%
2010-07-19 21:47:59 +08:00
%% When we report our duration to the memory monitor, we calculate
%% average ingress and egress rates over the last two samples, and
%% then calculate our duration based on the sum of the ingress and
%% egress rates. More than two samples could be used, but it's a
%% balance between responding quickly enough to changes in
%% producers/consumers versus ignoring temporary blips. The problem
%% with temporary blips is that with just a few queues, they can have
%% substantial impact on the calculation of the average duration and
%% hence cause unnecessary I/O. Another alternative is to increase the
%% amqqueue_process:RAM_DURATION_UPDATE_PERIOD to beyond 5
%% seconds. However, that then runs the risk of being too slow to
%% inform the memory monitor of changes. Thus a 5 second interval,
%% plus a rolling average over the last two samples seems to work
%% well in practice.
%%
%% The sum of the ingress and egress rates is used because the egress
%% rate alone is not sufficient. Adding in the ingress rate means that
%% queues which are being flooded by messages are given more memory,
%% resulting in them being able to process the messages faster (by
%% doing less I/O, or at least deferring it) and thus helping keep
%% their mailboxes empty and thus the queue as a whole is more
%% responsive. If such a queue also has fast but previously idle
%% consumers, the consumer can then start to be driven as fast as it
%% can go, whereas if only egress rate was being used, the incoming
%% messages may have to be written to disk and then read back in,
%% resulting in the hard disk being a bottleneck in driving the
%% consumers. Generally, we want to give Rabbit every chance of
%% getting rid of messages as fast as possible and remaining
%% responsive, and using only the egress rate impacts that goal.
%%
2011-10-12 23:24:35 +08:00
%% Once the queue has more alphas than the target_ram_count, the
%% surplus must be converted to betas, if not gammas, if not rolled
%% into delta. The conditions under which these transitions occur
%% reflect the conflicting goals of minimising RAM cost per msg, and
%% minimising CPU cost per msg. Once the msg has become a beta, its
%% payload is no longer in RAM, thus a read from the msg_store must
%% occur before the msg can be delivered, but the RAM cost of a beta
%% is the same as a gamma, so converting a beta to gamma will not free
%% up any further RAM. To reduce the RAM cost further, the gamma must
%% be rolled into delta. Whilst recovering a beta or a gamma to an
%% alpha requires only one disk read (from the msg_store), recovering
%% a msg from within delta will require two reads (queue_index and
%% then msg_store). But delta has a near-0 per-msg RAM cost. So the
2011-10-15 22:11:01 +08:00
%% conflict is between using delta more, which will free up more
2011-10-12 23:24:35 +08:00
%% memory, but require additional CPU and disk ops, versus using delta
%% less and gammas and betas more, which will cost more memory, but
%% require fewer disk ops and less CPU overhead.
%%
%% In the case of a persistent msg published to a durable queue, the
%% msg is immediately written to the msg_store and queue_index. If
%% then additionally converted from an alpha, it'll immediately go to
%% a gamma (as it's already in queue_index), and cannot exist as a
%% beta. Thus a durable queue with a mixture of persistent and
%% transient msgs in it which has more messages than permitted by the
%% target_ram_count may contain an interspersed mixture of betas and
%% gammas in q2 and q3.
2011-10-12 23:24:35 +08:00
%%
%% There is then a ratio that controls how many betas and gammas there
%% can be. This is based on the target_ram_count and thus expresses
%% the fact that as the number of permitted alphas in the queue falls,
%% so should the number of betas and gammas fall (i.e. delta
%% grows). If q2 and q3 contain more than the permitted number of
%% betas and gammas, then the surplus are forcibly converted to gammas
%% (as necessary) and then rolled into delta. The ratio is that
%% delta/(betas+gammas+delta) equals
2011-10-15 22:11:01 +08:00
%% (betas+gammas+delta)/(target_ram_count+betas+gammas+delta). I.e. as
%% the target_ram_count shrinks to 0, so must betas and gammas.
%%
%% The conversion of betas to deltas is done if there are at least
%% ?IO_BATCH_SIZE betas in q2 & q3. This value should not be too small,
%% otherwise the frequent operations on the queues of q2 and q3 will not be
%% effectively amortised (switching the direction of queue access defeats
%% amortisation). Note that there is a natural upper bound due to credit_flow
%% limits on the alpha to beta conversion.
%%
%% The conversion from alphas to betas is chunked due to the
%% credit_flow limits of the msg_store. This further smooths the
%% effects of changes to the target_ram_count and ensures the queue
%% remains responsive even when there is a large amount of IO work to
%% do. The 'resume' callback is utilised to ensure that conversions
%% are done as promptly as possible whilst ensuring the queue remains
%% responsive.
%%
%% In the queue we keep track of both messages that are pending
2011-03-22 20:39:04 +08:00
%% delivery and messages that are pending acks. In the event of a
%% queue purge, we only need to load qi segments if the queue has
%% elements in deltas (i.e. it came under significant memory
%% pressure). In the event of a queue deletion, in addition to the
%% preceding, by keeping track of pending acks in RAM, we do not need
%% to search through qi segments looking for messages that are yet to
%% be acknowledged.
%%
%% Pending acks are recorded in memory by storing the message itself.
%% If the message has been sent to disk, we do not store the message
%% content. During memory reduction, pending acks containing message
%% content have that content removed and the corresponding messages
%% are pushed out to disk.
%%
%% Messages from pending acks are returned to q4, q3 and delta during
%% requeue, based on the limits of seq_id contained in each. Requeued
%% messages retain their original seq_id, maintaining order
%% when requeued.
%%
%% The order in which alphas are pushed to betas and pending acks
%% are pushed to disk is determined dynamically. We always prefer to
%% push messages for the source (alphas or acks) that is growing the
%% fastest (with growth measured as avg. ingress - avg. egress).
%%
%% Notes on Clean Shutdown
%% (This documents behaviour in variable_queue, queue_index and
%% msg_store.)
%%
%% In order to try to achieve as fast a start-up as possible, if a
%% clean shutdown occurs, we try to save out state to disk to reduce
%% work on startup. In the msg_store this takes the form of the
%% index_module's state, plus the file_summary ets table, and client
%% refs. In the VQ, this takes the form of the count of persistent
%% messages in the queue and references into the msg_stores. The
%% queue_index adds to these terms the details of its segments and
%% stores the terms in the queue directory.
%%
%% Two message stores are used. One is created for persistent messages
%% to durable queues that must survive restarts, and the other is used
%% for all other messages that just happen to need to be written to
%% disk. On start up we can therefore nuke the transient message
%% store, and be sure that the messages in the persistent store are
%% all that we need.
%%
%% The references to the msg_stores are there so that the msg_store
%% knows to only trust its saved state if all of the queues it was
%% previously talking to come up cleanly. Likewise, the queues
%% themselves (esp queue_index) skips work in init if all the queues
%% and msg_store were shutdown cleanly. This gives both good speed
%% improvements and also robustness so that if anything possibly went
%% wrong in shutdown (or there was subsequent manual tampering), all
%% messages and queues that can be recovered are recovered, safely.
%%
%% To delete transient messages lazily, the variable_queue, on
%% startup, stores the next_seq_id reported by the queue_index as the
%% transient_threshold. From that point on, whenever it's reading a
%% message off disk via the queue_index, if the seq_id is below this
%% threshold and the message is transient then it drops the message
%% (the message itself won't exist on disk because it would have been
%% stored in the transient msg_store which would have had its saved
%% state nuked on startup). This avoids the expensive operation of
%% scanning the entire queue on startup in order to delete transient
%% messages that were only pushed to disk to save memory.
%%
%%----------------------------------------------------------------------------
2009-10-07 20:53:03 +08:00
-behaviour(rabbit_backing_queue).
2010-04-08 23:05:08 +08:00
2009-10-07 20:53:03 +08:00
-record(vqstate,
{ q1,
q2,
delta,
2009-10-07 20:53:03 +08:00
q3,
q4,
next_seq_id,
ram_pending_ack, %% msgs using store, still in RAM
disk_pending_ack, %% msgs in store, paged out
qi_pending_ack, %% msgs using qi, *can't* be paged out
index_state,
msg_store_clients,
durable,
transient_threshold,
qi_embed_msgs_below,
len, %% w/o unacked
bytes, %% w/o unacked
unacked_bytes,
persistent_count, %% w unacked
persistent_bytes, %% w unacked
delta_transient_bytes, %%
2010-12-06 03:31:22 +08:00
target_ram_count,
ram_msg_count, %% w/o unacked
ram_msg_count_prev,
ram_ack_count_prev,
ram_bytes, %% w unacked
2009-10-07 23:27:23 +08:00
out_counter,
in_counter,
rates,
msgs_on_disk,
2010-09-04 00:17:58 +08:00
msg_indices_on_disk,
2010-11-24 23:04:14 +08:00
unconfirmed,
confirmed,
2010-11-03 00:34:35 +08:00
ack_out_counter,
ack_in_counter,
%% Unlike the other counters these two do not feed into
%% #rates{} and get reset
disk_read_count,
2015-08-27 23:11:08 +08:00
disk_write_count,
2015-10-10 23:40:23 +08:00
io_batch_size,
%% default queue or lazy queue
mode,
2016-09-29 21:53:06 +08:00
%% number of reduce_memory_usage executions, once it
%% reaches a threshold the queue will manually trigger a runtime GC
%% see: maybe_execute_gc/1
memory_reduction_run_count,
%% Queue data is grouped by VHost. We need to store it
%% to work with queue index.
virtual_host,
waiting_bump = false
}).
2009-10-07 20:53:03 +08:00
2014-01-27 21:59:16 +08:00
-record(rates, { in, out, ack_in, ack_out, timestamp }).
-record(msg_status,
{ seq_id,
msg_id,
msg,
is_persistent,
is_delivered,
msg_in_store,
index_on_disk,
persist_to,
msg_props
}).
2010-04-08 23:05:08 +08:00
-record(delta,
{ start_seq_id, %% start_seq_id is inclusive
2010-04-08 23:05:08 +08:00
count,
transient,
end_seq_id %% end_seq_id is exclusive
2011-03-05 03:32:39 +08:00
}).
2010-04-08 23:05:08 +08:00
-define(HEADER_GUESS_SIZE, 100). %% see determine_persist_to/2
2010-06-03 21:33:47 +08:00
-define(PERSISTENT_MSG_STORE, msg_store_persistent).
-define(TRANSIENT_MSG_STORE, msg_store_transient).
-define(INDEX, rabbit_classic_queue_index_v2).
-define(QUEUE, lqueue).
Make #amqqueue{} a private record + add a feature flag The #amqqueue{} record is now isolated in the `amqqueue` module: all accesses and modifications to the record are made through this helper. The type `rabbit_types:amqqueue()` is now `amqqueue:amqqueue()`. `amqqueue` knows about the new #amqqueue{} record introduced with quorum queues. If the actual record is the old one (as defined in RabbitMQ 3.7.x), it calls `amqqueue_v1` which knows about the old definition. When it needs to produce an #amqqueue{} record, it verifies if it is allowed to, using the new Feature flags subsystem and the `quorum_queue` feature flag specifically, and proceeds. If an old format #amqqueue{} is required, the creation is deferred to `amqqueue_v1`. The new modules come with a couple headers: they provide macros to replace pattern matching with `when` conditions. The `amqqueue_v*.hrl` headers are generated using the `macros()` functions in `amqqueue` and `amqqueue_v1` modules. Some operations now depend on the state of the `quorum_queue` feature flag. In particular, creating a quorum queue is denied if the feature flag is disabled. In the process, the following modules were moved from rabbitmq-common to rabbitmq-server: * rabbit_backing_queue * rabbit_queue_master_locator Likewise, the few functions were copied from `rabbit_misc` to `rabbit_amqqueue` because they depend on `amqqueue`. They don't really need to live in rabbitmq-common and they prevent further progress with the feature flags: feature flags need informations about the cluster. The `quorum_queue` feature flag comes with a migration function which is responsible of converting the #amqqueue{} records in Mnesia tables `rabbit_queue` and `rabbit_durable_queue`. Therefore, the upgrade function which did this is removed in this commit. [#159298729]
2018-10-11 18:12:39 +08:00
-include_lib("rabbit_common/include/rabbit.hrl").
-include_lib("rabbit_common/include/rabbit_framing.hrl").
-include("amqqueue.hrl").
%%----------------------------------------------------------------------------
2011-03-09 19:17:27 +08:00
-rabbit_upgrade({multiple_routing_keys, local, []}).
-rabbit_upgrade({move_messages_to_vhost_store, message_store, []}).
-type seq_id() :: non_neg_integer().
-type rates() :: #rates { in :: float(),
2014-01-27 21:59:16 +08:00
out :: float(),
ack_in :: float(),
ack_out :: float(),
timestamp :: rabbit_types:timestamp()}.
-type delta() :: #delta { start_seq_id :: non_neg_integer(),
2010-12-13 20:19:26 +08:00
count :: non_neg_integer(),
end_seq_id :: non_neg_integer() }.
2010-04-08 23:05:08 +08:00
2012-05-04 01:20:53 +08:00
%% The compiler (rightfully) complains that ack() and state() are
%% unused. For this reason we duplicate a -spec from
%% rabbit_backing_queue with the only intent being to remove
%% warnings. The problem here is that we can't parameterise the BQ
%% behaviour by these two types as we would like to. We still leave
%% these here for documentation purposes.
-type ack() :: seq_id().
-type state() :: #vqstate {
q1 :: ?QUEUE:?QUEUE(),
q2 :: ?QUEUE:?QUEUE(),
delta :: delta(),
q3 :: ?QUEUE:?QUEUE(),
q4 :: ?QUEUE:?QUEUE(),
next_seq_id :: seq_id(),
ram_pending_ack :: gb_trees:tree(),
disk_pending_ack :: gb_trees:tree(),
qi_pending_ack :: gb_trees:tree(),
index_state :: any(),
msg_store_clients :: 'undefined' | {{any(), binary()},
{any(), binary()}},
durable :: boolean(),
transient_threshold :: non_neg_integer(),
qi_embed_msgs_below :: non_neg_integer(),
len :: non_neg_integer(),
bytes :: non_neg_integer(),
unacked_bytes :: non_neg_integer(),
persistent_count :: non_neg_integer(),
persistent_bytes :: non_neg_integer(),
2010-12-06 03:31:22 +08:00
target_ram_count :: non_neg_integer() | 'infinity',
ram_msg_count :: non_neg_integer(),
ram_msg_count_prev :: non_neg_integer(),
ram_ack_count_prev :: non_neg_integer(),
ram_bytes :: non_neg_integer(),
out_counter :: non_neg_integer(),
in_counter :: non_neg_integer(),
2010-11-24 23:04:14 +08:00
rates :: rates(),
msgs_on_disk :: gb_sets:set(),
msg_indices_on_disk :: gb_sets:set(),
unconfirmed :: gb_sets:set(),
confirmed :: gb_sets:set(),
ack_out_counter :: non_neg_integer(),
ack_in_counter :: non_neg_integer(),
disk_read_count :: non_neg_integer(),
2015-08-27 23:11:08 +08:00
disk_write_count :: non_neg_integer(),
2015-10-10 23:40:23 +08:00
io_batch_size :: pos_integer(),
mode :: 'default' | 'lazy',
2016-09-29 21:53:06 +08:00
memory_reduction_run_count :: non_neg_integer()}.
-define(BLANK_DELTA, #delta { start_seq_id = undefined,
2010-06-04 05:16:23 +08:00
count = 0,
transient = 0,
2010-06-04 05:16:23 +08:00
end_seq_id = undefined }).
-define(BLANK_DELTA_PATTERN(Z), #delta { start_seq_id = Z,
2010-06-04 05:16:23 +08:00
count = 0,
transient = 0,
2010-06-04 05:16:23 +08:00
end_seq_id = Z }).
2014-01-27 21:59:16 +08:00
-define(MICROS_PER_SECOND, 1000000.0).
2014-01-27 23:05:01 +08:00
%% We're sampling every 5s for RAM duration; a half life that is of
%% the same order of magnitude is probably about right.
-define(RATE_AVG_HALF_LIFE, 5.0).
2014-01-27 21:59:16 +08:00
%% We will recalculate the #rates{} every time we get asked for our
%% RAM duration, 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).
2014-01-27 21:59:16 +08:00
%% we define the garbage collector threshold
%% it needs to tune the `reduce_memory_use` calls. Thus, the garbage collection.
%% see: rabbitmq-server-973 and rabbitmq-server-964
-define(DEFAULT_EXPLICIT_GC_RUN_OP_THRESHOLD, 1000).
-define(EXPLICIT_GC_RUN_OP_THRESHOLD(Mode),
2016-09-29 21:53:06 +08:00
case get(explicit_gc_run_operation_threshold) of
undefined ->
2017-01-02 19:48:26 +08:00
Val = explicit_gc_run_operation_threshold_for_mode(Mode),
2016-09-29 21:53:06 +08:00
put(explicit_gc_run_operation_threshold, Val),
Val;
Val -> Val
end).
2017-01-02 19:48:26 +08:00
explicit_gc_run_operation_threshold_for_mode(Mode) ->
{Key, Fallback} = case Mode of
lazy -> {lazy_queue_explicit_gc_run_operation_threshold,
?DEFAULT_EXPLICIT_GC_RUN_OP_THRESHOLD};
_ -> {queue_explicit_gc_run_operation_threshold,
?DEFAULT_EXPLICIT_GC_RUN_OP_THRESHOLD}
end,
rabbit_misc:get_env(rabbit, Key, Fallback).
%%----------------------------------------------------------------------------
%% Public API
%%----------------------------------------------------------------------------
start(VHost, DurableQueues) ->
{AllTerms, StartFunState} = ?INDEX:start(VHost, DurableQueues),
2016-12-21 02:38:40 +08:00
%% Group recovery terms by vhost.
ClientRefs = [Ref || Terms <- AllTerms,
Terms /= non_clean_shutdown,
begin
Ref = proplists:get_value(persistent_ref, Terms),
Ref =/= undefined
end],
start_msg_store(VHost, ClientRefs, StartFunState),
{ok, AllTerms}.
stop(VHost) ->
ok = stop_msg_store(VHost),
ok = ?INDEX:stop(VHost).
start_msg_store(VHost, Refs, StartFunState) when is_list(Refs); Refs == undefined ->
rabbit_log:info("Starting message stores for vhost '~s'", [VHost]),
2017-07-06 20:12:06 +08:00
do_start_msg_store(VHost, ?TRANSIENT_MSG_STORE, undefined, ?EMPTY_START_FUN_STATE),
do_start_msg_store(VHost, ?PERSISTENT_MSG_STORE, Refs, StartFunState),
ok.
2017-07-06 20:12:06 +08:00
do_start_msg_store(VHost, Type, Refs, StartFunState) ->
case rabbit_vhost_msg_store:start(VHost, Type, Refs, StartFunState) of
{ok, _} ->
rabbit_log:info("Started message store of type ~s for vhost '~s'", [abbreviated_type(Type), VHost]);
{error, {no_such_vhost, VHost}} = Err ->
rabbit_log:error("Failed to start message store of type ~s for vhost '~s': the vhost no longer exists!",
[Type, VHost]),
exit(Err);
{error, Error} ->
rabbit_log:error("Failed to start message store of type ~s for vhost '~s': ~p",
[Type, VHost, Error]),
exit({error, Error})
end.
abbreviated_type(?TRANSIENT_MSG_STORE) -> transient;
abbreviated_type(?PERSISTENT_MSG_STORE) -> persistent.
stop_msg_store(VHost) ->
rabbit_vhost_msg_store:stop(VHost, ?TRANSIENT_MSG_STORE),
rabbit_vhost_msg_store:stop(VHost, ?PERSISTENT_MSG_STORE),
ok.
2014-12-04 01:27:16 +08:00
init(Queue, Recover, Callback) ->
init(
Queue, Recover, Callback,
fun (MsgIds, ActionTaken) ->
msgs_written_to_disk(Callback, MsgIds, ActionTaken)
end,
fun (MsgIds) -> msg_indices_written_to_disk(Callback, MsgIds) end,
fun (MsgIds) -> msgs_and_indices_written_to_disk(Callback, MsgIds) end).
2010-09-27 22:48:01 +08:00
Make #amqqueue{} a private record + add a feature flag The #amqqueue{} record is now isolated in the `amqqueue` module: all accesses and modifications to the record are made through this helper. The type `rabbit_types:amqqueue()` is now `amqqueue:amqqueue()`. `amqqueue` knows about the new #amqqueue{} record introduced with quorum queues. If the actual record is the old one (as defined in RabbitMQ 3.7.x), it calls `amqqueue_v1` which knows about the old definition. When it needs to produce an #amqqueue{} record, it verifies if it is allowed to, using the new Feature flags subsystem and the `quorum_queue` feature flag specifically, and proceeds. If an old format #amqqueue{} is required, the creation is deferred to `amqqueue_v1`. The new modules come with a couple headers: they provide macros to replace pattern matching with `when` conditions. The `amqqueue_v*.hrl` headers are generated using the `macros()` functions in `amqqueue` and `amqqueue_v1` modules. Some operations now depend on the state of the `quorum_queue` feature flag. In particular, creating a quorum queue is denied if the feature flag is disabled. In the process, the following modules were moved from rabbitmq-common to rabbitmq-server: * rabbit_backing_queue * rabbit_queue_master_locator Likewise, the few functions were copied from `rabbit_misc` to `rabbit_amqqueue` because they depend on `amqqueue`. They don't really need to live in rabbitmq-common and they prevent further progress with the feature flags: feature flags need informations about the cluster. The `quorum_queue` feature flag comes with a migration function which is responsible of converting the #amqqueue{} records in Mnesia tables `rabbit_queue` and `rabbit_durable_queue`. Therefore, the upgrade function which did this is removed in this commit. [#159298729]
2018-10-11 18:12:39 +08:00
init(Q, new, AsyncCallback, MsgOnDiskFun, MsgIdxOnDiskFun, MsgAndIdxOnDiskFun) when ?is_amqqueue(Q) ->
QueueName = amqqueue:get_name(Q),
IsDurable = amqqueue:is_durable(Q),
IndexState = ?INDEX:init(QueueName,
2014-12-04 01:27:16 +08:00
MsgIdxOnDiskFun, MsgAndIdxOnDiskFun),
2016-03-10 21:29:08 +08:00
VHost = QueueName#resource.virtual_host,
init(IsDurable, IndexState, 0, 0, [],
2010-10-21 07:21:37 +08:00
case IsDurable of
true -> msg_store_client_init(?PERSISTENT_MSG_STORE,
2016-03-10 21:29:08 +08:00
MsgOnDiskFun, AsyncCallback, VHost);
2010-10-21 07:21:37 +08:00
false -> undefined
end,
msg_store_client_init(?TRANSIENT_MSG_STORE, undefined,
AsyncCallback, VHost), VHost);
2014-08-20 21:34:35 +08:00
%% We can be recovering a transient queue if it crashed
Make #amqqueue{} a private record + add a feature flag The #amqqueue{} record is now isolated in the `amqqueue` module: all accesses and modifications to the record are made through this helper. The type `rabbit_types:amqqueue()` is now `amqqueue:amqqueue()`. `amqqueue` knows about the new #amqqueue{} record introduced with quorum queues. If the actual record is the old one (as defined in RabbitMQ 3.7.x), it calls `amqqueue_v1` which knows about the old definition. When it needs to produce an #amqqueue{} record, it verifies if it is allowed to, using the new Feature flags subsystem and the `quorum_queue` feature flag specifically, and proceeds. If an old format #amqqueue{} is required, the creation is deferred to `amqqueue_v1`. The new modules come with a couple headers: they provide macros to replace pattern matching with `when` conditions. The `amqqueue_v*.hrl` headers are generated using the `macros()` functions in `amqqueue` and `amqqueue_v1` modules. Some operations now depend on the state of the `quorum_queue` feature flag. In particular, creating a quorum queue is denied if the feature flag is disabled. In the process, the following modules were moved from rabbitmq-common to rabbitmq-server: * rabbit_backing_queue * rabbit_queue_master_locator Likewise, the few functions were copied from `rabbit_misc` to `rabbit_amqqueue` because they depend on `amqqueue`. They don't really need to live in rabbitmq-common and they prevent further progress with the feature flags: feature flags need informations about the cluster. The `quorum_queue` feature flag comes with a migration function which is responsible of converting the #amqqueue{} records in Mnesia tables `rabbit_queue` and `rabbit_durable_queue`. Therefore, the upgrade function which did this is removed in this commit. [#159298729]
2018-10-11 18:12:39 +08:00
init(Q, Terms, AsyncCallback, MsgOnDiskFun, MsgIdxOnDiskFun, MsgAndIdxOnDiskFun) when ?is_amqqueue(Q) ->
QueueName = amqqueue:get_name(Q),
IsDurable = amqqueue:is_durable(Q),
{PRef, RecoveryTerms} = process_recovery_terms(Terms),
2016-03-10 21:29:08 +08:00
VHost = QueueName#resource.virtual_host,
2014-08-20 21:34:35 +08:00
{PersistentClient, ContainsCheckFun} =
case IsDurable of
true -> C = msg_store_client_init(?PERSISTENT_MSG_STORE, PRef,
2016-03-10 21:29:08 +08:00
MsgOnDiskFun, AsyncCallback,
VHost),
2014-12-04 01:38:49 +08:00
{C, fun (MsgId) when is_binary(MsgId) ->
rabbit_msg_store:contains(MsgId, C);
2014-12-06 00:18:53 +08:00
(#basic_message{is_persistent = Persistent}) ->
Persistent
2014-12-04 01:38:49 +08:00
end};
2014-08-20 21:34:35 +08:00
false -> {undefined, fun(_MsgId) -> false end}
end,
TransientClient = msg_store_client_init(?TRANSIENT_MSG_STORE,
undefined, AsyncCallback,
2016-03-10 21:29:08 +08:00
VHost),
{DeltaCount, DeltaBytes, IndexState} =
?INDEX:recover(
QueueName, RecoveryTerms,
rabbit_vhost_msg_store:successfully_recovered_state(
VHost,
?PERSISTENT_MSG_STORE),
2014-12-04 01:27:16 +08:00
ContainsCheckFun, MsgIdxOnDiskFun, MsgAndIdxOnDiskFun),
2014-08-20 21:34:35 +08:00
init(IsDurable, IndexState, DeltaCount, DeltaBytes, RecoveryTerms,
PersistentClient, TransientClient, VHost).
2009-10-07 20:53:03 +08:00
process_recovery_terms(Terms=non_clean_shutdown) ->
{rabbit_guid:gen(), Terms};
process_recovery_terms(Terms) ->
case proplists:get_value(persistent_ref, Terms) of
undefined -> {rabbit_guid:gen(), []};
2014-01-16 03:14:23 +08:00
PRef -> {PRef, Terms}
end.
terminate(_Reason, State) ->
State1 = #vqstate { virtual_host = VHost,
next_seq_id = NextSeqId,
persistent_count = PCount,
persistent_bytes = PBytes,
2010-06-04 05:16:23 +08:00
index_state = IndexState,
2010-10-21 07:21:37 +08:00
msg_store_clients = {MSCStateP, MSCStateT} } =
purge_pending_ack(true, State),
2010-10-21 07:21:37 +08:00
PRef = case MSCStateP of
undefined -> undefined;
_ -> ok = maybe_client_terminate(MSCStateP),
2010-10-21 07:21:37 +08:00
rabbit_msg_store:client_ref(MSCStateP)
end,
ok = rabbit_msg_store:client_delete_and_terminate(MSCStateT),
Terms = [{next_seq_id, NextSeqId},
{persistent_ref, PRef},
{persistent_count, PCount},
{persistent_bytes, PBytes}],
a(State1#vqstate {
index_state = ?INDEX:terminate(VHost, Terms, IndexState),
msg_store_clients = undefined }).
%% the only difference between purge and delete is that delete also
%% needs to delete everything that's been delivered and not ack'd.
delete_and_terminate(_Reason, State) ->
2015-09-08 00:32:13 +08:00
%% Normally when we purge messages we interact with the qi by
%% issues delivers and acks for every purged message. In this case
%% we don't need to do that, so we just delete the qi.
2015-09-07 23:11:40 +08:00
State1 = purge_and_index_reset(State),
State2 = #vqstate { msg_store_clients = {MSCStateP, MSCStateT} } =
2015-09-08 00:23:19 +08:00
purge_pending_ack_delete_and_terminate(State1),
case MSCStateP of
undefined -> ok;
2010-10-21 07:21:37 +08:00
_ -> rabbit_msg_store:client_delete_and_terminate(MSCStateP)
end,
2010-10-21 07:21:37 +08:00
rabbit_msg_store:client_delete_and_terminate(MSCStateT),
a(State2 #vqstate { msg_store_clients = undefined }).
Make #amqqueue{} a private record + add a feature flag The #amqqueue{} record is now isolated in the `amqqueue` module: all accesses and modifications to the record are made through this helper. The type `rabbit_types:amqqueue()` is now `amqqueue:amqqueue()`. `amqqueue` knows about the new #amqqueue{} record introduced with quorum queues. If the actual record is the old one (as defined in RabbitMQ 3.7.x), it calls `amqqueue_v1` which knows about the old definition. When it needs to produce an #amqqueue{} record, it verifies if it is allowed to, using the new Feature flags subsystem and the `quorum_queue` feature flag specifically, and proceeds. If an old format #amqqueue{} is required, the creation is deferred to `amqqueue_v1`. The new modules come with a couple headers: they provide macros to replace pattern matching with `when` conditions. The `amqqueue_v*.hrl` headers are generated using the `macros()` functions in `amqqueue` and `amqqueue_v1` modules. Some operations now depend on the state of the `quorum_queue` feature flag. In particular, creating a quorum queue is denied if the feature flag is disabled. In the process, the following modules were moved from rabbitmq-common to rabbitmq-server: * rabbit_backing_queue * rabbit_queue_master_locator Likewise, the few functions were copied from `rabbit_misc` to `rabbit_amqqueue` because they depend on `amqqueue`. They don't really need to live in rabbitmq-common and they prevent further progress with the feature flags: feature flags need informations about the cluster. The `quorum_queue` feature flag comes with a migration function which is responsible of converting the #amqqueue{} records in Mnesia tables `rabbit_queue` and `rabbit_durable_queue`. Therefore, the upgrade function which did this is removed in this commit. [#159298729]
2018-10-11 18:12:39 +08:00
delete_crashed(Q) when ?is_amqqueue(Q) ->
QName = amqqueue:get_name(Q),
ok = ?INDEX:erase(QName).
2015-09-06 18:45:14 +08:00
purge(State = #vqstate { len = Len }) ->
case is_pending_ack_empty(State) and is_unconfirmed_empty(State) of
2015-09-06 18:45:14 +08:00
true ->
2015-09-07 23:11:40 +08:00
{Len, purge_and_index_reset(State)};
2015-09-06 18:45:14 +08:00
false ->
{Len, purge_when_pending_acks(State)}
end.
2013-01-19 22:20:51 +08:00
purge_acks(State) -> a(purge_pending_ack(false, State)).
publish(Msg, MsgProps, IsDelivered, ChPid, Flow, State) ->
State1 =
publish1(Msg, MsgProps, IsDelivered, ChPid, Flow,
fun maybe_write_to_disk/4,
State),
a(maybe_reduce_memory_use(maybe_update_rates(State1))).
2009-10-07 20:53:03 +08:00
batch_publish(Publishes, ChPid, Flow, State) ->
{ChPid, Flow, State1} =
lists:foldl(fun batch_publish1/2, {ChPid, Flow, State}, Publishes),
State2 = ui(State1),
a(maybe_reduce_memory_use(maybe_update_rates(State2))).
publish_delivered(Msg, MsgProps, ChPid, Flow, State) ->
{SeqId, State1} =
publish_delivered1(Msg, MsgProps, ChPid, Flow,
fun maybe_write_to_disk/4,
State),
{SeqId, a(maybe_reduce_memory_use(maybe_update_rates(State1)))}.
batch_publish_delivered(Publishes, ChPid, Flow, State) ->
{ChPid, Flow, SeqIds, State1} =
lists:foldl(fun batch_publish_delivered1/2,
{ChPid, Flow, [], State}, Publishes),
State2 = ui(State1),
{lists:reverse(SeqIds), a(maybe_reduce_memory_use(maybe_update_rates(State2)))}.
discard(_MsgId, _ChPid, _Flow, State) -> State.
drain_confirmed(State = #vqstate { confirmed = C }) ->
case gb_sets:is_empty(C) of
true -> {[], State}; %% common case
false -> {gb_sets:to_list(C), State #vqstate {
confirmed = gb_sets:new() }}
end.
dropwhile(Pred, State) ->
{MsgProps, State1} =
remove_by_predicate(Pred, State),
{MsgProps, a(State1)}.
fetchwhile(Pred, Fun, Acc, State) ->
{MsgProps, Acc1, State1} =
fetch_by_predicate(Pred, Fun, Acc, State),
{MsgProps, Acc1, a(State1)}.
fetch(AckRequired, State) ->
case queue_out(State) of
{empty, State1} ->
{empty, a(State1)};
{{value, MsgStatus}, State1} ->
%% it is possible that the message wasn't read from disk
%% at this point, so read it in.
2013-01-12 18:18:28 +08:00
{Msg, State2} = read_msg(MsgStatus, State1),
{AckTag, State3} = remove(AckRequired, MsgStatus, State2),
{{Msg, MsgStatus#msg_status.is_delivered, AckTag}, a(State3)}
end.
drop(AckRequired, State) ->
case queue_out(State) of
{empty, State1} ->
{empty, a(State1)};
{{value, MsgStatus}, State1} ->
2013-01-02 22:54:51 +08:00
{AckTag, State2} = remove(AckRequired, MsgStatus, State1),
{{MsgStatus#msg_status.msg_id, AckTag}, a(State2)}
end.
2018-11-30 18:30:36 +08:00
%% Duplicated from rabbit_backing_queue
-spec ack([ack()], state()) -> {[rabbit_guid:guid()], state()}.
ack([], State) ->
{[], State};
2014-02-03 23:26:43 +08:00
%% optimisation: this head is essentially a partial evaluation of the
%% general case below, for the single-ack case.
ack([SeqId], State) ->
2016-10-26 21:59:12 +08:00
case remove_pending_ack(true, SeqId, State) of
{none, _} ->
2020-01-28 03:33:11 +08:00
{[], State};
2016-10-26 21:59:12 +08:00
{#msg_status { msg_id = MsgId,
is_persistent = IsPersistent,
msg_in_store = MsgInStore,
index_on_disk = IndexOnDisk },
State1 = #vqstate { index_state = IndexState,
msg_store_clients = MSCState,
ack_out_counter = AckOutCount }} ->
IndexState1 = case IndexOnDisk of
true -> ?INDEX:ack([SeqId], IndexState);
2016-10-26 21:59:12 +08:00
false -> IndexState
end,
case MsgInStore of
true -> ok = msg_store_remove(MSCState, IsPersistent, [MsgId]);
false -> ok
end,
{[MsgId],
a(State1 #vqstate { index_state = IndexState1,
ack_out_counter = AckOutCount + 1 })}
end;
ack(AckTags, State) ->
{{IndexOnDiskSeqIds, MsgIdsByStore, AllMsgIds},
State1 = #vqstate { index_state = IndexState,
msg_store_clients = MSCState,
ack_out_counter = AckOutCount }} =
lists:foldl(
fun (SeqId, {Acc, State2}) ->
2016-10-26 21:59:12 +08:00
case remove_pending_ack(true, SeqId, State2) of
{none, _} ->
{Acc, State2};
{MsgStatus, State3} ->
{accumulate_ack(MsgStatus, Acc), State3}
end
end, {accumulate_ack_init(), State}, AckTags),
IndexState1 = ?INDEX:ack(IndexOnDiskSeqIds, IndexState),
remove_msgs_by_id(MsgIdsByStore, MSCState),
{lists:reverse(AllMsgIds),
a(State1 #vqstate { index_state = IndexState1,
ack_out_counter = AckOutCount + length(AckTags) })}.
2015-10-13 22:05:22 +08:00
requeue(AckTags, #vqstate { mode = default,
delta = Delta,
q3 = Q3,
q4 = Q4,
in_counter = InCounter,
len = Len } = State) ->
{SeqIds, Q4a, MsgIds, State1} = queue_merge(lists:sort(AckTags), Q4, [],
beta_limit(Q3),
fun publish_alpha/2, State),
2011-09-30 04:15:15 +08:00
{SeqIds1, Q3a, MsgIds1, State2} = queue_merge(SeqIds, Q3, MsgIds,
delta_limit(Delta),
fun publish_beta/2, State1),
{Delta1, MsgIds2, State3} = delta_merge(SeqIds1, Delta, MsgIds1,
State2),
2011-09-30 04:15:15 +08:00
MsgCount = length(MsgIds2),
{MsgIds2, a(maybe_reduce_memory_use(
2016-01-27 23:59:21 +08:00
maybe_update_rates(ui(
State3 #vqstate { delta = Delta1,
q3 = Q3a,
q4 = Q4a,
in_counter = InCounter + MsgCount,
2016-01-27 23:59:21 +08:00
len = Len + MsgCount }))))};
2015-10-13 22:05:22 +08:00
requeue(AckTags, #vqstate { mode = lazy,
delta = Delta,
q3 = Q3,
in_counter = InCounter,
len = Len } = State) ->
{SeqIds, Q3a, MsgIds, State1} = queue_merge(lists:sort(AckTags), Q3, [],
delta_limit(Delta),
fun publish_beta/2, State),
{Delta1, MsgIds1, State2} = delta_merge(SeqIds, Delta, MsgIds,
State1),
MsgCount = length(MsgIds1),
{MsgIds1, a(maybe_reduce_memory_use(
2016-01-27 23:59:21 +08:00
maybe_update_rates(ui(
2015-10-13 22:05:22 +08:00
State2 #vqstate { delta = Delta1,
q3 = Q3a,
in_counter = InCounter + MsgCount,
2016-01-27 23:59:21 +08:00
len = Len + MsgCount }))))}.
ackfold(MsgFun, Acc, State, AckTags) ->
{AccN, StateN} =
lists:foldl(fun(SeqId, {Acc0, State0}) ->
MsgStatus = lookup_pending_ack(SeqId, State0),
2013-01-12 18:18:28 +08:00
{Msg, State1} = read_msg(MsgStatus, State0),
{MsgFun(Msg, SeqId, Acc0), State1}
end, {Acc, State}, AckTags),
{AccN, a(StateN)}.
fold(Fun, Acc, State = #vqstate{index_state = IndexState}) ->
2013-01-13 18:59:59 +08:00
{Its, IndexState1} = lists:foldl(fun inext/2, {[], IndexState},
[msg_iterator(State),
disk_ack_iterator(State),
ram_ack_iterator(State),
qi_ack_iterator(State)]),
ifold(Fun, Acc, Its, State#vqstate{index_state = IndexState1}).
2012-11-22 01:53:18 +08:00
2010-06-15 15:29:55 +08:00
len(#vqstate { len = Len }) -> Len.
2010-06-15 15:29:55 +08:00
is_empty(State) -> 0 == len(State).
depth(State) ->
len(State) + count_pending_acks(State).
2010-12-06 03:31:22 +08:00
set_ram_duration_target(
DurationTarget, State = #vqstate {
rates = #rates { in = AvgIngressRate,
out = AvgEgressRate,
ack_in = AvgAckIngressRate,
ack_out = AvgAckEgressRate },
2010-12-06 03:31:22 +08:00
target_ram_count = TargetRamCount }) ->
Rate =
AvgEgressRate + AvgIngressRate + AvgAckEgressRate + AvgAckIngressRate,
2010-12-06 03:31:22 +08:00
TargetRamCount1 =
case DurationTarget of
infinity -> infinity;
2010-06-03 15:09:07 +08:00
_ -> trunc(DurationTarget * Rate) %% msgs = sec * msgs/sec
end,
2010-12-06 03:31:22 +08:00
State1 = State #vqstate { target_ram_count = TargetRamCount1 },
a(case TargetRamCount1 == infinity orelse
(TargetRamCount =/= infinity andalso
TargetRamCount1 >= TargetRamCount) of
true -> State1;
false -> reduce_memory_use(State1)
end).
maybe_update_rates(State = #vqstate{ in_counter = InCount,
out_counter = OutCount })
when InCount + OutCount > ?MSGS_PER_RATE_CALC ->
update_rates(State);
maybe_update_rates(State) ->
State.
2014-02-06 02:55:45 +08:00
update_rates(State = #vqstate{ in_counter = InCount,
out_counter = OutCount,
ack_in_counter = AckInCount,
2014-01-27 21:59:16 +08:00
ack_out_counter = AckOutCount,
2014-02-06 02:55:45 +08:00
rates = #rates{ in = InRate,
out = OutRate,
ack_in = AckInRate,
2014-01-27 21:59:16 +08:00
ack_out = AckOutRate,
timestamp = TS }}) ->
2016-05-12 17:34:15 +08:00
Now = erlang:monotonic_time(),
2014-02-06 02:55:45 +08:00
Rates = #rates { in = update_rate(Now, TS, InCount, InRate),
out = update_rate(Now, TS, OutCount, OutRate),
ack_in = update_rate(Now, TS, AckInCount, AckInRate),
ack_out = update_rate(Now, TS, AckOutCount, AckOutRate),
2014-01-27 21:59:16 +08:00
timestamp = Now },
State#vqstate{ in_counter = 0,
out_counter = 0,
ack_in_counter = 0,
ack_out_counter = 0,
rates = Rates }.
2014-01-27 23:05:01 +08:00
update_rate(Now, TS, Count, Rate) ->
2016-05-12 17:34:15 +08:00
Time = erlang:convert_time_unit(Now - TS, native, micro_seconds) /
?MICROS_PER_SECOND,
if
Time == 0 -> Rate;
true -> rabbit_misc:moving_average(Time, ?RATE_AVG_HALF_LIFE,
Count / Time, Rate)
end.
2014-01-30 02:15:24 +08:00
ram_duration(State) ->
State1 = #vqstate { rates = #rates { in = AvgIngressRate,
out = AvgEgressRate,
ack_in = AvgAckIngressRate,
ack_out = AvgAckEgressRate },
2014-01-30 02:15:24 +08:00
ram_msg_count = RamMsgCount,
ram_msg_count_prev = RamMsgCountPrev,
ram_pending_ack = RPA,
qi_pending_ack = QPA,
2014-01-30 02:15:24 +08:00
ram_ack_count_prev = RamAckCountPrev } =
update_rates(State),
RamAckCount = gb_trees:size(RPA) + gb_trees:size(QPA),
Duration = %% msgs+acks / (msgs+acks/sec) == sec
case lists:all(fun (X) -> X < 0.01 end,
[AvgEgressRate, AvgIngressRate,
AvgAckEgressRate, AvgAckIngressRate]) of
true -> infinity;
false -> (RamMsgCountPrev + RamMsgCount +
RamAckCount + RamAckCountPrev) /
(4 * (AvgEgressRate + AvgIngressRate +
AvgAckEgressRate + AvgAckIngressRate))
end,
2014-01-30 02:15:24 +08:00
{Duration, State1}.
2014-03-04 00:29:55 +08:00
needs_timeout(#vqstate { index_state = IndexState }) ->
case ?INDEX:needs_sync(IndexState) of
2014-03-04 00:29:55 +08:00
confirms -> timed;
other -> idle;
false -> false
end.
timeout(State = #vqstate { index_state = IndexState }) ->
State #vqstate { index_state = ?INDEX:sync(IndexState) }.
handle_pre_hibernate(State = #vqstate { index_state = IndexState }) ->
State #vqstate { index_state = ?INDEX:flush(IndexState) }.
handle_info(bump_reduce_memory_use, State = #vqstate{ waiting_bump = true }) ->
State#vqstate{ waiting_bump = false };
handle_info(bump_reduce_memory_use, State) ->
State.
resume(State) -> a(reduce_memory_use(State)).
2014-03-04 00:29:55 +08:00
msg_rates(#vqstate { rates = #rates { in = AvgIngressRate,
out = AvgEgressRate } }) ->
2013-11-15 21:40:40 +08:00
{AvgIngressRate, AvgEgressRate}.
info(messages_ready_ram, #vqstate{ram_msg_count = RamMsgCount}) ->
RamMsgCount;
info(messages_unacknowledged_ram, #vqstate{ram_pending_ack = RPA,
qi_pending_ack = QPA}) ->
gb_trees:size(RPA) + gb_trees:size(QPA);
info(messages_ram, State) ->
info(messages_ready_ram, State) + info(messages_unacknowledged_ram, State);
info(messages_persistent, #vqstate{persistent_count = PersistentCount}) ->
PersistentCount;
info(messages_paged_out, #vqstate{delta = #delta{transient = Count}}) ->
Count;
info(message_bytes, #vqstate{bytes = Bytes,
unacked_bytes = UBytes}) ->
Bytes + UBytes;
info(message_bytes_ready, #vqstate{bytes = Bytes}) ->
Bytes;
info(message_bytes_unacknowledged, #vqstate{unacked_bytes = UBytes}) ->
UBytes;
info(message_bytes_ram, #vqstate{ram_bytes = RamBytes}) ->
RamBytes;
2014-07-29 20:26:43 +08:00
info(message_bytes_persistent, #vqstate{persistent_bytes = PersistentBytes}) ->
PersistentBytes;
info(message_bytes_paged_out, #vqstate{delta_transient_bytes = PagedOutBytes}) ->
PagedOutBytes;
info(head_message_timestamp, #vqstate{
2015-04-27 21:14:10 +08:00
q3 = Q3,
q4 = Q4,
ram_pending_ack = RPA,
qi_pending_ack = QPA}) ->
head_message_timestamp(Q3, Q4, RPA, QPA);
info(disk_reads, #vqstate{disk_read_count = Count}) ->
Count;
info(disk_writes, #vqstate{disk_write_count = Count}) ->
Count;
info(backing_queue_status, #vqstate {
2010-12-06 03:31:22 +08:00
q1 = Q1, q2 = Q2, delta = Delta, q3 = Q3, q4 = Q4,
2015-10-13 21:31:41 +08:00
mode = Mode,
2014-07-29 23:30:24 +08:00
len = Len,
2010-12-06 03:31:22 +08:00
target_ram_count = TargetRamCount,
next_seq_id = NextSeqId,
2014-01-27 21:59:16 +08:00
rates = #rates { in = AvgIngressRate,
out = AvgEgressRate,
ack_in = AvgAckIngressRate,
ack_out = AvgAckEgressRate }}) ->
2015-10-13 22:01:42 +08:00
[ {mode , Mode},
2015-10-13 21:31:41 +08:00
{q1 , ?QUEUE:len(Q1)},
{q2 , ?QUEUE:len(Q2)},
2010-12-06 03:31:22 +08:00
{delta , Delta},
{q3 , ?QUEUE:len(Q3)},
{q4 , ?QUEUE:len(Q4)},
2014-07-29 23:30:24 +08:00
{len , Len},
2010-12-06 03:31:22 +08:00
{target_ram_count , TargetRamCount},
{next_seq_id , NextSeqId},
{avg_ingress_rate , AvgIngressRate},
{avg_egress_rate , AvgEgressRate},
{avg_ack_ingress_rate, AvgAckIngressRate},
{avg_ack_egress_rate , AvgAckEgressRate} ];
info(_, _) ->
''.
invoke(?MODULE, Fun, State) -> Fun(?MODULE, State);
invoke( _, _, State) -> State.
2011-04-08 19:03:42 +08:00
is_duplicate(_Msg, State) -> {false, State}.
2011-04-08 19:03:42 +08:00
2015-10-10 23:40:23 +08:00
set_queue_mode(Mode, State = #vqstate { mode = Mode }) ->
State;
set_queue_mode(lazy, State = #vqstate {
target_ram_count = TargetRamCount }) ->
%% To become a lazy queue we need to page everything to disk first.
State1 = convert_to_lazy(State),
%% restore the original target_ram_count
a(State1 #vqstate { mode = lazy, target_ram_count = TargetRamCount });
set_queue_mode(default, State) ->
%% becoming a default queue means loading messages from disk like
2015-10-24 01:27:07 +08:00
%% when a queue is recovered.
2015-10-10 23:40:23 +08:00
a(maybe_deltas_to_betas(State #vqstate { mode = default }));
set_queue_mode(_, State) ->
State.
zip_msgs_and_acks(Msgs, AckTags, Accumulator, _State) ->
lists:foldl(fun ({{#basic_message{ id = Id }, _Props}, AckTag}, Acc) ->
[{Id, AckTag} | Acc]
end, Accumulator, lists:zip(Msgs, AckTags)).
2015-10-10 23:40:23 +08:00
convert_to_lazy(State) ->
State1 = #vqstate { delta = Delta, q3 = Q3, len = Len } =
2015-10-14 02:50:31 +08:00
set_ram_duration_target(0, State),
case Delta#delta.count + ?QUEUE:len(Q3) == Len of
2015-10-14 02:50:31 +08:00
true ->
State1;
false ->
%% When pushing messages to disk, we might have been
%% blocked by the msg_store, so we need to see if we have
2015-10-21 23:58:40 +08:00
%% to wait for more credit, and then keep paging messages.
2015-10-14 02:50:31 +08:00
%%
%% The amqqueue_process could have taken care of this, but
%% between the time it receives the bump_credit msg and
%% calls BQ:resume to keep paging messages to disk, some
%% other request may arrive to the BQ which at this moment
%% is not in a proper state for a lazy BQ (unless all
%% messages have been paged to disk already).
wait_for_msg_store_credit(),
2016-08-09 05:23:03 +08:00
convert_to_lazy(resume(State1))
2015-10-14 02:50:31 +08:00
end.
2015-10-10 23:40:23 +08:00
wait_for_msg_store_credit() ->
case credit_flow:blocked() of
true -> receive
{bump_credit, Msg} ->
credit_flow:handle_bump_msg(Msg)
end;
false -> ok
end.
2015-04-27 21:14:10 +08:00
%% Get the Timestamp property of the first msg, if present. This is
%% the one with the oldest timestamp among the heads of the pending
%% acks and unread queues. We can't check disk_pending_acks as these
%% are paged out - we assume some will soon be paged in rather than
%% forcing it to happen. Pending ack msgs are included as they are
%% regarded as unprocessed until acked, this also prevents the result
%% apparently oscillating during repeated rejects. Q3 is only checked
%% when Q4 is empty as any Q4 msg will be earlier.
head_message_timestamp(Q3, Q4, RPA, QPA) ->
HeadMsgs = [ HeadMsgStatus#msg_status.msg ||
HeadMsgStatus <-
[ get_qs_head([Q4, Q3]),
get_pa_head(RPA),
get_pa_head(QPA) ],
2015-10-14 13:19:48 +08:00
HeadMsgStatus /= undefined,
HeadMsgStatus#msg_status.msg /= undefined ],
2015-04-27 21:14:10 +08:00
Timestamps =
[Timestamp || HeadMsg <- HeadMsgs,
Timestamp <- [rabbit_basic:extract_timestamp(
HeadMsg#basic_message.content)],
Timestamp /= undefined
],
case Timestamps == [] of
true -> '';
2015-04-27 21:14:10 +08:00
false -> lists:min(Timestamps)
end.
2015-04-27 21:14:10 +08:00
get_qs_head(Qs) ->
catch lists:foldl(
fun (Q, Acc) ->
case get_q_head(Q) of
undefined -> Acc;
Val -> throw(Val)
end
end, undefined, Qs).
get_q_head(Q) ->
get_collection_head(Q, fun ?QUEUE:is_empty/1, fun ?QUEUE:peek/1).
get_pa_head(PA) ->
get_collection_head(PA, fun gb_trees:is_empty/1, fun gb_trees:smallest/1).
get_collection_head(Col, IsEmpty, GetVal) ->
case IsEmpty(Col) of
false ->
{_, MsgStatus} = GetVal(Col),
MsgStatus;
true -> undefined
end.
%%----------------------------------------------------------------------------
%% Minor helpers
%%----------------------------------------------------------------------------
a(State = #vqstate { q1 = Q1, q2 = Q2, delta = Delta, q3 = Q3, q4 = Q4,
mode = default,
len = Len,
bytes = Bytes,
unacked_bytes = UnackedBytes,
persistent_count = PersistentCount,
persistent_bytes = PersistentBytes,
ram_msg_count = RamMsgCount,
ram_bytes = RamBytes}) ->
E1 = ?QUEUE:is_empty(Q1),
E2 = ?QUEUE:is_empty(Q2),
ED = Delta#delta.count == 0,
E3 = ?QUEUE:is_empty(Q3),
E4 = ?QUEUE:is_empty(Q4),
LZ = Len == 0,
2015-10-24 01:27:07 +08:00
%% if q1 has messages then q3 cannot be empty. See publish/6.
true = E1 or not E3,
%% if q2 has messages then we have messages in delta (paged to
%% disk). See push_alphas_to_betas/2.
true = E2 or not ED,
%% if delta has messages then q3 cannot be empty. This is enforced
%% by paging, where min([segment_entry_count(), len(q3)]) messages
2015-10-14 05:43:30 +08:00
%% are always kept on RAM.
true = ED or not E3,
%% if the queue length is 0, then q3 and q4 must be empty.
true = LZ == (E3 and E4),
true = Len >= 0,
true = Bytes >= 0,
true = UnackedBytes >= 0,
true = PersistentCount >= 0,
true = PersistentBytes >= 0,
true = RamMsgCount >= 0,
2012-11-21 00:12:06 +08:00
true = RamMsgCount =< Len,
true = RamBytes >= 0,
true = RamBytes =< Bytes + UnackedBytes,
State;
a(State = #vqstate { q1 = Q1, q2 = Q2, delta = Delta, q3 = Q3, q4 = Q4,
mode = lazy,
len = Len,
bytes = Bytes,
unacked_bytes = UnackedBytes,
persistent_count = PersistentCount,
persistent_bytes = PersistentBytes,
ram_msg_count = RamMsgCount,
ram_bytes = RamBytes}) ->
E1 = ?QUEUE:is_empty(Q1),
E2 = ?QUEUE:is_empty(Q2),
ED = Delta#delta.count == 0,
E3 = ?QUEUE:is_empty(Q3),
E4 = ?QUEUE:is_empty(Q4),
LZ = Len == 0,
L3 = ?QUEUE:len(Q3),
2015-10-24 01:27:07 +08:00
%% q1 must always be empty, since q1 only gets messages during
%% publish, but for lazy queues messages go straight to delta.
true = E1,
%% q2 only gets messages from q1 when push_alphas_to_betas is
%% called for a non empty delta, which won't be the case for a
%% lazy queue. This means q2 must always be empty.
true = E2,
2015-10-24 01:27:07 +08:00
%% q4 must always be empty, since q1 only gets messages during
%% publish, but for lazy queues messages go straight to delta.
true = E4,
%% if the queue is empty, then delta is empty and q3 is empty.
true = LZ == (ED and E3),
%% There should be no messages in q1, q2, and q4
true = Delta#delta.count + L3 == Len,
true = Len >= 0,
true = Bytes >= 0,
true = UnackedBytes >= 0,
true = PersistentCount >= 0,
true = PersistentBytes >= 0,
true = RamMsgCount >= 0,
true = RamMsgCount =< Len,
true = RamBytes >= 0,
true = RamBytes =< Bytes + UnackedBytes,
State.
d(Delta = #delta { start_seq_id = Start, count = Count, end_seq_id = End })
when Start + Count =< End ->
Delta.
2014-12-12 18:51:17 +08:00
m(MsgStatus = #msg_status { is_persistent = IsPersistent,
msg_in_store = MsgInStore,
index_on_disk = IndexOnDisk }) ->
true = (not IsPersistent) or IndexOnDisk,
2014-12-12 18:51:17 +08:00
true = msg_in_ram(MsgStatus) or MsgInStore,
MsgStatus.
2010-06-13 21:47:17 +08:00
one_if(true ) -> 1;
one_if(false) -> 0.
2010-07-19 16:01:28 +08:00
cons_if(true, E, L) -> [E | L];
cons_if(false, _E, L) -> L.
gb_sets_maybe_insert(false, _Val, Set) -> Set;
2013-01-30 19:07:10 +08:00
gb_sets_maybe_insert(true, Val, Set) -> gb_sets:add(Val, Set).
msg_status(IsPersistent, IsDelivered, SeqId,
Msg = #basic_message {id = MsgId}, MsgProps, IndexMaxSize) ->
#msg_status{seq_id = SeqId,
msg_id = MsgId,
msg = Msg,
is_persistent = IsPersistent,
is_delivered = IsDelivered,
msg_in_store = false,
index_on_disk = false,
persist_to = determine_persist_to(Msg, MsgProps, IndexMaxSize),
msg_props = MsgProps}.
beta_msg_status({Msg = #basic_message{id = MsgId},
SeqId, MsgProps, IsPersistent, IsDelivered}) ->
MS0 = beta_msg_status0(SeqId, MsgProps, IsPersistent, IsDelivered),
MS0#msg_status{msg_id = MsgId,
msg = Msg,
persist_to = msg_store, % queue_index,
msg_in_store = false};
beta_msg_status({MsgId, SeqId, MsgProps, IsPersistent, IsDelivered}) ->
MS0 = beta_msg_status0(SeqId, MsgProps, IsPersistent, IsDelivered),
MS0#msg_status{msg_id = MsgId,
msg = undefined,
persist_to = msg_store,
msg_in_store = true}.
beta_msg_status0(SeqId, MsgProps, IsPersistent, IsDelivered) ->
#msg_status{seq_id = SeqId,
msg = undefined,
is_persistent = IsPersistent,
is_delivered = IsDelivered,
index_on_disk = true,
msg_props = MsgProps}.
trim_msg_status(MsgStatus) ->
case persist_to(MsgStatus) of
msg_store -> MsgStatus#msg_status{msg = undefined};
queue_index -> MsgStatus
end.
2010-10-21 07:21:37 +08:00
with_msg_store_state({MSCStateP, MSCStateT}, true, Fun) ->
{Result, MSCStateP1} = Fun(MSCStateP),
{Result, {MSCStateP1, MSCStateT}};
with_msg_store_state({MSCStateP, MSCStateT}, false, Fun) ->
{Result, MSCStateT1} = Fun(MSCStateT),
{Result, {MSCStateP, MSCStateT1}}.
with_immutable_msg_store_state(MSCState, IsPersistent, Fun) ->
{Res, MSCState} = with_msg_store_state(MSCState, IsPersistent,
fun (MSCState1) ->
{Fun(MSCState1), MSCState1}
end),
Res.
2010-06-16 15:01:53 +08:00
2016-03-10 21:29:08 +08:00
msg_store_client_init(MsgStore, MsgOnDiskFun, Callback, VHost) ->
msg_store_client_init(MsgStore, rabbit_guid:gen(), MsgOnDiskFun,
2016-03-10 21:29:08 +08:00
Callback, VHost).
2016-03-10 21:29:08 +08:00
msg_store_client_init(MsgStore, Ref, MsgOnDiskFun, Callback, VHost) ->
CloseFDsFun = msg_store_close_fds_fun(MsgStore =:= ?PERSISTENT_MSG_STORE),
rabbit_vhost_msg_store:client_init(VHost, MsgStore,
Ref, MsgOnDiskFun,
fun () ->
Callback(?MODULE, CloseFDsFun)
end).
2010-06-16 15:01:53 +08:00
msg_store_write(MSCState, IsPersistent, MsgId, Msg) ->
with_immutable_msg_store_state(
2010-06-16 15:01:53 +08:00
MSCState, IsPersistent,
fun (MSCState1) ->
rabbit_msg_store:write_flow(MsgId, Msg, MSCState1)
end).
msg_store_read(MSCState, IsPersistent, MsgId) ->
2010-10-21 07:21:37 +08:00
with_msg_store_state(
MSCState, IsPersistent,
fun (MSCState1) ->
rabbit_msg_store:read(MsgId, MSCState1)
end).
2010-06-16 15:01:53 +08:00
msg_store_remove(MSCState, IsPersistent, MsgIds) ->
2010-10-21 07:21:37 +08:00
with_immutable_msg_store_state(
MSCState, IsPersistent,
fun (MCSState1) ->
rabbit_msg_store:remove(MsgIds, MCSState1)
end).
2010-06-16 15:01:53 +08:00
msg_store_close_fds(MSCState, IsPersistent) ->
with_msg_store_state(
MSCState, IsPersistent,
fun (MSCState1) -> rabbit_msg_store:close_all_indicated(MSCState1) end).
msg_store_close_fds_fun(IsPersistent) ->
2011-04-08 19:03:42 +08:00
fun (?MODULE, State = #vqstate { msg_store_clients = MSCState }) ->
{ok, MSCState1} = msg_store_close_fds(MSCState, IsPersistent),
State #vqstate { msg_store_clients = MSCState1 }
end.
maybe_write_delivered(false, _SeqId, IndexState) ->
IndexState;
maybe_write_delivered(true, SeqId, IndexState) ->
?INDEX:deliver([SeqId], IndexState).
betas_from_index_entries(List, TransientThreshold, DelsAndAcksFun, State) ->
{Filtered, Delivers, Acks, RamReadyCount, RamBytes, TransientCount, TransientBytes} =
lists:foldr(
fun ({_MsgOrId, SeqId, _MsgProps, IsPersistent, IsDelivered} = M,
{Filtered1, Delivers1, Acks1, RRC, RB, TC, TB} = Acc) ->
case SeqId < TransientThreshold andalso not IsPersistent of
true -> {Filtered1,
2010-07-19 16:01:28 +08:00
cons_if(not IsDelivered, SeqId, Delivers1),
[SeqId | Acks1], RRC, RB, TC, TB};
false -> MsgStatus = m(beta_msg_status(M)),
2014-12-12 18:51:17 +08:00
HaveMsg = msg_in_ram(MsgStatus),
Size = msg_size(MsgStatus),
case is_msg_in_pending_acks(SeqId, State) of
false -> {?QUEUE:in_r(MsgStatus, Filtered1),
Delivers1, Acks1,
RRC + one_if(HaveMsg),
RB + one_if(HaveMsg) * Size,
TC + one_if(not IsPersistent),
TB + one_if(not IsPersistent) * Size};
true -> Acc %% [0]
end
end
end, {?QUEUE:new(), [], [], 0, 0, 0, 0}, List),
{Filtered, RamReadyCount, RamBytes, DelsAndAcksFun(Delivers, Acks, State),
TransientCount, TransientBytes}.
%% [0] We don't increase RamBytes here, even though it pertains to
%% unacked messages too, since if HaveMsg then the message must have
%% been stored in the QI, thus the message must have been in
%% qi_pending_ack, thus it must already have been in RAM.
is_msg_in_pending_acks(SeqId, #vqstate { ram_pending_ack = RPA,
2015-09-10 05:57:01 +08:00
disk_pending_ack = DPA,
qi_pending_ack = QPA }) ->
(gb_trees:is_defined(SeqId, RPA) orelse
gb_trees:is_defined(SeqId, DPA) orelse
gb_trees:is_defined(SeqId, QPA)).
expand_delta(SeqId, ?BLANK_DELTA_PATTERN(X), IsPersistent) ->
d(#delta { start_seq_id = SeqId, count = 1, end_seq_id = SeqId + 1,
transient = one_if(not IsPersistent)});
expand_delta(SeqId, #delta { start_seq_id = StartSeqId,
count = Count,
transient = Transient } = Delta,
IsPersistent )
when SeqId < StartSeqId ->
d(Delta #delta { start_seq_id = SeqId, count = Count + 1,
transient = Transient + one_if(not IsPersistent)});
2011-10-15 22:56:35 +08:00
expand_delta(SeqId, #delta { count = Count,
end_seq_id = EndSeqId,
transient = Transient } = Delta,
IsPersistent)
when SeqId >= EndSeqId ->
d(Delta #delta { count = Count + 1, end_seq_id = SeqId + 1,
transient = Transient + one_if(not IsPersistent)});
expand_delta(_SeqId, #delta { count = Count,
transient = Transient } = Delta,
IsPersistent ) ->
d(Delta #delta { count = Count + 1,
transient = Transient + one_if(not IsPersistent) }).
%%----------------------------------------------------------------------------
%% Internal major helpers for Public API
%%----------------------------------------------------------------------------
init(IsDurable, IndexState, DeltaCount, DeltaBytes, Terms,
PersistentClient, TransientClient, VHost) ->
{LowSeqId, HiSeqId, IndexState1} = ?INDEX:bounds(IndexState),
2010-10-21 07:21:37 +08:00
{NextSeqId, DeltaCount1, DeltaBytes1} =
case Terms of
non_clean_shutdown -> {HiSeqId, DeltaCount, DeltaBytes};
_ -> {proplists:get_value(next_seq_id,
Terms, 0),
proplists:get_value(persistent_count,
Terms, DeltaCount),
proplists:get_value(persistent_bytes,
Terms, DeltaBytes)}
end,
2010-10-21 07:21:37 +08:00
Delta = case DeltaCount1 == 0 andalso DeltaCount /= undefined of
true -> ?BLANK_DELTA;
false -> d(#delta { start_seq_id = LowSeqId,
count = DeltaCount1,
transient = 0,
end_seq_id = HiSeqId })
2010-10-21 07:21:37 +08:00
end,
2016-05-12 17:34:15 +08:00
Now = erlang:monotonic_time(),
2015-08-27 23:11:08 +08:00
IoBatchSize = rabbit_misc:get_env(rabbit, msg_store_io_batch_size,
?IO_BATCH_SIZE),
{ok, IndexMaxSize} = application:get_env(
rabbit, queue_index_embed_msgs_below),
2010-10-21 07:21:37 +08:00
State = #vqstate {
q1 = ?QUEUE:new(),
q2 = ?QUEUE:new(),
2010-12-06 03:31:22 +08:00
delta = Delta,
q3 = ?QUEUE:new(),
q4 = ?QUEUE:new(),
2010-12-06 03:31:22 +08:00
next_seq_id = NextSeqId,
ram_pending_ack = gb_trees:empty(),
disk_pending_ack = gb_trees:empty(),
qi_pending_ack = gb_trees:empty(),
2010-12-06 03:31:22 +08:00
index_state = IndexState1,
msg_store_clients = {PersistentClient, TransientClient},
durable = IsDurable,
transient_threshold = NextSeqId,
qi_embed_msgs_below = IndexMaxSize,
2010-12-06 03:31:22 +08:00
len = DeltaCount1,
persistent_count = DeltaCount1,
bytes = DeltaBytes1,
persistent_bytes = DeltaBytes1,
delta_transient_bytes = 0,
2010-12-06 03:31:22 +08:00
target_ram_count = infinity,
ram_msg_count = 0,
ram_msg_count_prev = 0,
ram_ack_count_prev = 0,
ram_bytes = 0,
unacked_bytes = 0,
2010-12-06 03:31:22 +08:00
out_counter = 0,
in_counter = 0,
2014-01-27 21:59:16 +08:00
rates = blank_rates(Now),
2010-12-06 03:31:22 +08:00
msgs_on_disk = gb_sets:new(),
msg_indices_on_disk = gb_sets:new(),
unconfirmed = gb_sets:new(),
confirmed = gb_sets:new(),
2010-12-06 03:31:22 +08:00
ack_out_counter = 0,
ack_in_counter = 0,
disk_read_count = 0,
2015-08-27 23:11:08 +08:00
disk_write_count = 0,
2015-10-10 23:40:23 +08:00
io_batch_size = IoBatchSize,
2016-09-26 21:07:27 +08:00
mode = default,
memory_reduction_run_count = 0,
virtual_host = VHost},
2010-10-21 07:21:37 +08:00
a(maybe_deltas_to_betas(State)).
2014-01-27 21:59:16 +08:00
blank_rates(Now) ->
#rates { in = 0.0,
out = 0.0,
ack_in = 0.0,
ack_out = 0.0,
timestamp = Now}.
in_r(MsgStatus = #msg_status { msg = undefined },
2015-10-10 23:52:53 +08:00
State = #vqstate { mode = default, q3 = Q3, q4 = Q4 }) ->
case ?QUEUE:is_empty(Q4) of
true -> State #vqstate { q3 = ?QUEUE:in_r(MsgStatus, Q3) };
false -> {Msg, State1 = #vqstate { q4 = Q4a }} =
read_msg(MsgStatus, State),
MsgStatus1 = MsgStatus#msg_status{msg = Msg},
stats(ready0, {MsgStatus, MsgStatus1}, 0,
2015-01-23 21:10:42 +08:00
State1 #vqstate { q4 = ?QUEUE:in_r(MsgStatus1, Q4a) })
end;
2015-10-14 05:43:43 +08:00
in_r(MsgStatus,
State = #vqstate { mode = default, q4 = Q4 }) ->
2015-10-10 23:52:53 +08:00
State #vqstate { q4 = ?QUEUE:in_r(MsgStatus, Q4) };
%% lazy queues
in_r(MsgStatus = #msg_status { seq_id = SeqId, is_persistent = IsPersistent },
2015-10-10 23:52:53 +08:00
State = #vqstate { mode = lazy, q3 = Q3, delta = Delta}) ->
case ?QUEUE:is_empty(Q3) of
true ->
{_MsgStatus1, State1} =
maybe_write_to_disk(true, true, MsgStatus, State),
State2 = stats(ready0, {MsgStatus, none}, 1, State1),
Delta1 = expand_delta(SeqId, Delta, IsPersistent),
State2 #vqstate{ delta = Delta1};
2015-10-10 23:52:53 +08:00
false ->
State #vqstate { q3 = ?QUEUE:in_r(MsgStatus, Q3) }
end.
2015-10-10 23:53:43 +08:00
queue_out(State = #vqstate { mode = default, q4 = Q4 }) ->
case ?QUEUE:out(Q4) of
{empty, _Q4} ->
case fetch_from_q3(State) of
2011-06-26 15:52:59 +08:00
{empty, _State1} = Result -> Result;
{loaded, {MsgStatus, State1}} -> {{value, MsgStatus}, State1}
end;
{{value, MsgStatus}, Q4a} ->
{{value, MsgStatus}, State #vqstate { q4 = Q4a }}
2015-10-10 23:53:43 +08:00
end;
%% lazy queues
queue_out(State = #vqstate { mode = lazy }) ->
case fetch_from_q3(State) of
{empty, _State1} = Result -> Result;
{loaded, {MsgStatus, State1}} -> {{value, MsgStatus}, State1}
end.
2013-01-12 18:18:28 +08:00
read_msg(#msg_status{msg = undefined,
msg_id = MsgId,
is_persistent = IsPersistent}, State) ->
read_msg(MsgId, IsPersistent, State);
2013-01-12 18:18:28 +08:00
read_msg(#msg_status{msg = Msg}, State) ->
{Msg, State}.
read_msg(MsgId, IsPersistent, State = #vqstate{msg_store_clients = MSCState,
disk_read_count = Count}) ->
{{ok, Msg = #basic_message {}}, MSCState1} =
2012-02-14 20:05:27 +08:00
msg_store_read(MSCState, IsPersistent, MsgId),
{Msg, State #vqstate {msg_store_clients = MSCState1,
disk_read_count = Count + 1}}.
stats(Signs, Statuses, DeltaPaged, State) ->
stats0(expand_signs(Signs), expand_statuses(Statuses), DeltaPaged, State).
2015-10-13 21:07:03 +08:00
expand_signs(ready0) -> {0, 0, true};
expand_signs(lazy_pub) -> {1, 0, true};
expand_signs({A, B}) -> {A, B, false}.
expand_statuses({none, A}) -> {false, msg_in_ram(A), A};
expand_statuses({B, none}) -> {msg_in_ram(B), false, B};
2015-10-13 21:07:03 +08:00
expand_statuses({lazy, A}) -> {false , false, A};
expand_statuses({B, A}) -> {msg_in_ram(B), msg_in_ram(A), B}.
%% In this function at least, we are religious: the variable name
%% contains "Ready" or "Unacked" iff that is what it counts. If
%% neither is present it counts both.
2015-01-23 21:10:42 +08:00
stats0({DeltaReady, DeltaUnacked, ReadyMsgPaged},
{InRamBefore, InRamAfter, MsgStatus}, DeltaPaged,
2015-01-23 21:10:42 +08:00
State = #vqstate{len = ReadyCount,
bytes = ReadyBytes,
ram_msg_count = RamReadyCount,
persistent_count = PersistentCount,
unacked_bytes = UnackedBytes,
ram_bytes = RamBytes,
delta_transient_bytes = DeltaBytes,
2015-01-23 21:10:42 +08:00
persistent_bytes = PersistentBytes}) ->
S = msg_size(MsgStatus),
2015-01-23 21:10:42 +08:00
DeltaTotal = DeltaReady + DeltaUnacked,
DeltaRam = case {InRamBefore, InRamAfter} of
{false, false} -> 0;
{false, true} -> 1;
{true, false} -> -1;
{true, true} -> 0
end,
DeltaRamReady = case DeltaReady of
1 -> one_if(InRamAfter);
-1 -> -one_if(InRamBefore);
0 when ReadyMsgPaged -> DeltaRam;
0 -> 0
end,
DeltaPersistent = DeltaTotal * one_if(MsgStatus#msg_status.is_persistent),
State#vqstate{len = ReadyCount + DeltaReady,
ram_msg_count = RamReadyCount + DeltaRamReady,
persistent_count = PersistentCount + DeltaPersistent,
bytes = ReadyBytes + DeltaReady * S,
unacked_bytes = UnackedBytes + DeltaUnacked * S,
ram_bytes = RamBytes + DeltaRam * S,
persistent_bytes = PersistentBytes + DeltaPersistent * S,
delta_transient_bytes = DeltaBytes + DeltaPaged * one_if(not MsgStatus#msg_status.is_persistent) * S}.
msg_size(#msg_status{msg_props = #message_properties{size = Size}}) -> Size.
2013-01-12 18:18:28 +08:00
2014-12-12 00:36:56 +08:00
msg_in_ram(#msg_status{msg = Msg}) -> Msg =/= undefined.
%% first param: AckRequired
remove(true, MsgStatus = #msg_status {
seq_id = SeqId,
is_delivered = IsDelivered,
index_on_disk = IndexOnDisk },
State = #vqstate {out_counter = OutCount,
index_state = IndexState}) ->
%% Mark it delivered if necessary
IndexState1 = maybe_write_delivered(
IndexOnDisk andalso not IsDelivered,
SeqId, IndexState),
State1 = record_pending_ack(
MsgStatus #msg_status {
is_delivered = true }, State),
State2 = stats({-1, 1}, {MsgStatus, MsgStatus}, 0, State1),
{SeqId, maybe_update_rates(
State2 #vqstate {out_counter = OutCount + 1,
index_state = IndexState1})};
%% This function body has the same behaviour as remove_queue_entries/3
%% but instead of removing messages based on a ?QUEUE, this removes
%% just one message, the one referenced by the MsgStatus provided.
remove(false, MsgStatus = #msg_status {
seq_id = SeqId,
msg_id = MsgId,
is_persistent = IsPersistent,
is_delivered = IsDelivered,
msg_in_store = MsgInStore,
index_on_disk = IndexOnDisk },
State = #vqstate {out_counter = OutCount,
index_state = IndexState,
msg_store_clients = MSCState}) ->
%% Mark it delivered if necessary
IndexState1 = maybe_write_delivered(
IndexOnDisk andalso not IsDelivered,
SeqId, IndexState),
%% Remove from msg_store and queue index, if necessary
case MsgInStore of
true -> ok = msg_store_remove(MSCState, IsPersistent, [MsgId]);
false -> ok
end,
IndexState2 =
case IndexOnDisk of
true -> ?INDEX:ack([SeqId], IndexState1);
false -> IndexState1
end,
State1 = stats({-1, 0}, {MsgStatus, none}, 0, State),
{undefined, maybe_update_rates(
State1 #vqstate {out_counter = OutCount + 1,
index_state = IndexState2})}.
%% This function exists as a way to improve dropwhile/2
%% performance. The idea of having this function is to optimise calls
%% to rabbit_queue_index by batching delivers and acks, instead of
%% sending them one by one.
%%
%% Instead of removing every message as their are popped from the
%% queue, it first accumulates them and then removes them by calling
%% remove_queue_entries/3, since the behaviour of
%% remove_queue_entries/3 when used with
%% process_delivers_and_acks_fun(deliver_and_ack) is the same as
%% calling remove(false, MsgStatus, State).
%%
%% remove/3 also updates the out_counter in every call, but here we do
%% it just once at the end.
%%
%% @todo This function is really bad. If there are 1 million messages
%% expired, it will first collect the 1 million messages and then
%% process them. It should probably limit the number of messages
%% it removes at once and loop until satisfied instead. It could
%% also let the index first figure out until what seq_id() to read
%% (since the index has expiration encoded, it could use binary
%% search to find where it should stop reading) and then in a second
%% step do the reading with a limit for each read and drop only that.
remove_by_predicate(Pred, State = #vqstate {out_counter = OutCount}) ->
{MsgProps, QAcc, State1} =
collect_by_predicate(Pred, ?QUEUE:new(), State),
State2 =
remove_queue_entries(
QAcc, process_delivers_and_acks_fun(deliver_and_ack), State1),
%% maybe_update_rates/1 is called in remove/2 for every
%% message. Since we update out_counter only once, we call it just
%% there.
{MsgProps, maybe_update_rates(
State2 #vqstate {
out_counter = OutCount + ?QUEUE:len(QAcc)})}.
%% This function exists as a way to improve fetchwhile/4
%% performance. The idea of having this function is to optimise calls
%% to rabbit_queue_index by batching delivers, instead of sending them
%% one by one.
%%
%% Fun is the function passed to fetchwhile/4 that's
%% applied to every fetched message and used to build the fetchwhile/4
%% result accumulator FetchAcc.
%%
%% @todo See todo in remove_by_predicate/2 function.
fetch_by_predicate(Pred, Fun, FetchAcc,
State = #vqstate {
index_state = IndexState,
out_counter = OutCount}) ->
{MsgProps, QAcc, State1} =
collect_by_predicate(Pred, ?QUEUE:new(), State),
{Delivers, FetchAcc1, State2} =
process_queue_entries(QAcc, Fun, FetchAcc, State1),
IndexState1 = ?INDEX:deliver(Delivers, IndexState),
{MsgProps, FetchAcc1, maybe_update_rates(
State2 #vqstate {
index_state = IndexState1,
out_counter = OutCount + ?QUEUE:len(QAcc)})}.
%% We try to do here the same as what remove(true, State) does but
%% processing several messages at the same time. The idea is to
%% optimize rabbit_queue_index:deliver/2 calls by sending a list of
%% SeqIds instead of one by one, thus process_queue_entries1 will
%% accumulate the required deliveries, will record_pending_ack for
%% each message, and will update stats, like remove/2 does.
%%
%% For the meaning of Fun and FetchAcc arguments see
%% fetch_by_predicate/4 above.
process_queue_entries(Q, Fun, FetchAcc, State) ->
?QUEUE:foldl(fun (MsgStatus, Acc) ->
process_queue_entries1(MsgStatus, Fun, Acc)
end,
{[], FetchAcc, State}, Q).
process_queue_entries1(
#msg_status { seq_id = SeqId, is_delivered = IsDelivered,
index_on_disk = IndexOnDisk} = MsgStatus,
Fun,
{Delivers, FetchAcc, State}) ->
{Msg, State1} = read_msg(MsgStatus, State),
State2 = record_pending_ack(
MsgStatus #msg_status {
is_delivered = true }, State1),
{cons_if(IndexOnDisk andalso not IsDelivered, SeqId, Delivers),
Fun(Msg, SeqId, FetchAcc),
stats({-1, 1}, {MsgStatus, MsgStatus}, 0, State2)}.
collect_by_predicate(Pred, QAcc, State) ->
case queue_out(State) of
{empty, State1} ->
{undefined, QAcc, State1};
{{value, MsgStatus = #msg_status { msg_props = MsgProps }}, State1} ->
case Pred(MsgProps) of
true -> collect_by_predicate(Pred, ?QUEUE:in(MsgStatus, QAcc),
State1);
false -> {MsgProps, QAcc, in_r(MsgStatus, State1)}
end
end.
2015-09-07 19:30:33 +08:00
%%----------------------------------------------------------------------------
%% Helpers for Public API purge/1 function
%%----------------------------------------------------------------------------
2015-09-07 20:01:52 +08:00
%% The difference between purge_when_pending_acks/1
2015-09-07 23:11:40 +08:00
%% vs. purge_and_index_reset/1 is that the first one issues a deliver
%% and an ack to the queue index for every message that's being
%% removed, while the later just resets the queue index state.
2015-09-07 19:30:33 +08:00
purge_when_pending_acks(State) ->
2015-09-09 00:27:22 +08:00
State1 = purge1(process_delivers_and_acks_fun(deliver_and_ack), State),
2015-09-07 19:30:33 +08:00
a(State1).
2015-09-07 23:11:40 +08:00
purge_and_index_reset(State) ->
2015-09-09 00:27:22 +08:00
State1 = purge1(process_delivers_and_acks_fun(none), State),
2015-09-07 19:52:57 +08:00
a(reset_qi_state(State1)).
2015-09-07 19:30:33 +08:00
2015-09-07 20:01:52 +08:00
%% This function removes messages from each of {q1, q2, q3, q4}.
%%
%% With remove_queue_entries/3 q1 and q4 are emptied, while q2 and q3
%% are specially handled by purge_betas_and_deltas/2.
%%
%% purge_betas_and_deltas/2 loads messages from the queue index,
%% filling up q3 and in some cases moving messages form q2 to q3 while
2019-02-13 01:24:18 +08:00
%% resetting q2 to an empty queue (see maybe_deltas_to_betas/2). The
2015-09-07 20:01:52 +08:00
%% messages loaded into q3 are removed by calling
%% remove_queue_entries/3 until there are no more messages to be read
2015-09-08 00:51:59 +08:00
%% from the queue index. Messages are read in batches from the queue
2015-09-07 20:01:52 +08:00
%% index.
2015-09-07 19:30:33 +08:00
purge1(AfterFun, State = #vqstate { q4 = Q4}) ->
State1 = remove_queue_entries(Q4, AfterFun, State),
2015-09-09 00:35:53 +08:00
State2 = #vqstate {q1 = Q1} =
purge_betas_and_deltas(AfterFun, State1#vqstate{q4 = ?QUEUE:new()}),
2015-09-07 19:30:33 +08:00
State3 = remove_queue_entries(Q1, AfterFun, State2),
2015-09-09 00:35:53 +08:00
a(State3#vqstate{q1 = ?QUEUE:new()}).
2015-09-07 19:30:33 +08:00
2015-09-09 00:35:53 +08:00
reset_qi_state(State = #vqstate{index_state = IndexState}) ->
State#vqstate{index_state =
?INDEX:reset_state(IndexState)}.
2015-09-07 19:30:33 +08:00
is_pending_ack_empty(State) ->
count_pending_acks(State) =:= 0.
is_unconfirmed_empty(#vqstate { unconfirmed = UC }) ->
gb_sets:is_empty(UC).
2015-09-07 19:30:33 +08:00
count_pending_acks(#vqstate { ram_pending_ack = RPA,
disk_pending_ack = DPA,
qi_pending_ack = QPA }) ->
gb_trees:size(RPA) + gb_trees:size(DPA) + gb_trees:size(QPA).
2015-12-23 21:01:32 +08:00
purge_betas_and_deltas(DelsAndAcksFun, State = #vqstate { mode = Mode }) ->
State0 = #vqstate { q3 = Q3 } =
case Mode of
lazy -> maybe_deltas_to_betas(DelsAndAcksFun, State);
_ -> State
end,
case ?QUEUE:is_empty(Q3) of
2015-12-23 21:01:32 +08:00
true -> State0;
false -> State1 = remove_queue_entries(Q3, DelsAndAcksFun, State0),
2015-09-06 18:45:14 +08:00
purge_betas_and_deltas(DelsAndAcksFun,
maybe_deltas_to_betas(
DelsAndAcksFun,
State1#vqstate{q3 = ?QUEUE:new()}))
end.
2015-09-06 18:45:14 +08:00
remove_queue_entries(Q, DelsAndAcksFun,
State = #vqstate{msg_store_clients = MSCState}) ->
{MsgIdsByStore, Delivers, Acks, State1} =
?QUEUE:foldl(fun remove_queue_entries1/2,
{maps:new(), [], [], State}, Q),
remove_msgs_by_id(MsgIdsByStore, MSCState),
DelsAndAcksFun(Delivers, Acks, State1).
remove_queue_entries1(
#msg_status { msg_id = MsgId, seq_id = SeqId, is_delivered = IsDelivered,
msg_in_store = MsgInStore, index_on_disk = IndexOnDisk,
is_persistent = IsPersistent} = MsgStatus,
{MsgIdsByStore, Delivers, Acks, State}) ->
{case MsgInStore of
2017-10-30 23:33:35 +08:00
true -> rabbit_misc:maps_cons(IsPersistent, MsgId, MsgIdsByStore);
false -> MsgIdsByStore
end,
cons_if(IndexOnDisk andalso not IsDelivered, SeqId, Delivers),
cons_if(IndexOnDisk, SeqId, Acks),
stats({-1, 0}, {MsgStatus, none}, 0, State)}.
2015-09-07 19:30:33 +08:00
process_delivers_and_acks_fun(deliver_and_ack) ->
fun (Delivers, Acks, State = #vqstate { index_state = IndexState }) ->
2015-09-08 01:05:54 +08:00
IndexState1 =
?INDEX:ack(
Acks, ?INDEX:deliver(Delivers, IndexState)),
2015-09-07 19:30:33 +08:00
State #vqstate { index_state = IndexState1 }
end;
process_delivers_and_acks_fun(_) ->
fun (_, _, State) ->
State
end.
%%----------------------------------------------------------------------------
%% Internal gubbins for publishing
%%----------------------------------------------------------------------------
2015-10-13 18:00:39 +08:00
publish1(Msg = #basic_message { is_persistent = IsPersistent, id = MsgId },
MsgProps = #message_properties { needs_confirming = NeedsConfirming },
IsDelivered, _ChPid, _Flow, PersistFun,
State = #vqstate { q1 = Q1, q3 = Q3, q4 = Q4,
2015-10-13 18:00:39 +08:00
mode = default,
qi_embed_msgs_below = IndexMaxSize,
next_seq_id = SeqId,
in_counter = InCount,
durable = IsDurable,
unconfirmed = UC }) ->
IsPersistent1 = IsDurable andalso IsPersistent,
MsgStatus = msg_status(IsPersistent1, IsDelivered, SeqId, Msg, MsgProps, IndexMaxSize),
2021-05-17 20:38:09 +08:00
{MsgStatus1, State1} = PersistFun(false, false, MsgStatus, State),
State2 = case ?QUEUE:is_empty(Q3) of
false -> State1 #vqstate { q1 = ?QUEUE:in(m(MsgStatus1), Q1) };
true -> State1 #vqstate { q4 = ?QUEUE:in(m(MsgStatus1), Q4) }
end,
InCount1 = InCount + 1,
UC1 = gb_sets_maybe_insert(NeedsConfirming, MsgId, UC),
stats({1, 0}, {none, MsgStatus1}, 0,
State2#vqstate{ next_seq_id = SeqId + 1,
in_counter = InCount1,
2015-10-13 18:00:39 +08:00
unconfirmed = UC1 });
publish1(Msg = #basic_message { is_persistent = IsPersistent, id = MsgId },
2015-10-10 23:54:14 +08:00
MsgProps = #message_properties { needs_confirming = NeedsConfirming },
IsDelivered, _ChPid, _Flow, PersistFun,
2015-10-13 18:00:39 +08:00
State = #vqstate { mode = lazy,
qi_embed_msgs_below = IndexMaxSize,
2015-10-10 23:54:14 +08:00
next_seq_id = SeqId,
in_counter = InCount,
durable = IsDurable,
unconfirmed = UC,
delta = Delta}) ->
2015-10-10 23:54:14 +08:00
IsPersistent1 = IsDurable andalso IsPersistent,
MsgStatus = msg_status(IsPersistent1, IsDelivered, SeqId, Msg, MsgProps, IndexMaxSize),
{MsgStatus1, State1} = PersistFun(true, true, MsgStatus, State),
Delta1 = expand_delta(SeqId, Delta, IsPersistent),
2015-10-13 21:07:03 +08:00
UC1 = gb_sets_maybe_insert(NeedsConfirming, MsgId, UC),
stats(lazy_pub, {lazy, m(MsgStatus1)}, 1,
2015-10-10 23:54:14 +08:00
State1#vqstate{ delta = Delta1,
next_seq_id = SeqId + 1,
in_counter = InCount + 1,
unconfirmed = UC1}).
2009-10-07 23:27:23 +08:00
batch_publish1({Msg, MsgProps, IsDelivered}, {ChPid, Flow, State}) ->
{ChPid, Flow, publish1(Msg, MsgProps, IsDelivered, ChPid, Flow,
fun maybe_prepare_write_to_disk/4, State)}.
publish_delivered1(Msg = #basic_message { is_persistent = IsPersistent,
id = MsgId },
MsgProps = #message_properties {
needs_confirming = NeedsConfirming },
_ChPid, _Flow, PersistFun,
2015-10-13 21:07:03 +08:00
State = #vqstate { mode = default,
qi_embed_msgs_below = IndexMaxSize,
next_seq_id = SeqId,
out_counter = OutCount,
in_counter = InCount,
durable = IsDurable,
unconfirmed = UC }) ->
IsPersistent1 = IsDurable andalso IsPersistent,
MsgStatus = msg_status(IsPersistent1, true, SeqId, Msg, MsgProps, IndexMaxSize),
2021-05-17 20:38:09 +08:00
{MsgStatus1, State1} = PersistFun(false, false, MsgStatus, State),
State2 = record_pending_ack(m(MsgStatus1), State1),
UC1 = gb_sets_maybe_insert(NeedsConfirming, MsgId, UC),
State3 = stats({0, 1}, {none, MsgStatus1}, 0,
State2 #vqstate { next_seq_id = SeqId + 1,
out_counter = OutCount + 1,
in_counter = InCount + 1,
unconfirmed = UC1 }),
2015-10-13 21:07:03 +08:00
{SeqId, State3};
publish_delivered1(Msg = #basic_message { is_persistent = IsPersistent,
id = MsgId },
MsgProps = #message_properties {
needs_confirming = NeedsConfirming },
_ChPid, _Flow, PersistFun,
State = #vqstate { mode = lazy,
qi_embed_msgs_below = IndexMaxSize,
next_seq_id = SeqId,
out_counter = OutCount,
in_counter = InCount,
durable = IsDurable,
unconfirmed = UC }) ->
2015-10-13 21:07:03 +08:00
IsPersistent1 = IsDurable andalso IsPersistent,
MsgStatus = msg_status(IsPersistent1, true, SeqId, Msg, MsgProps, IndexMaxSize),
{MsgStatus1, State1} = PersistFun(true, true, MsgStatus, State),
State2 = record_pending_ack(m(MsgStatus1), State1),
UC1 = gb_sets_maybe_insert(NeedsConfirming, MsgId, UC),
State3 = stats({0, 1}, {none, MsgStatus1}, 0,
State2 #vqstate { next_seq_id = SeqId + 1,
2015-10-13 21:07:03 +08:00
out_counter = OutCount + 1,
in_counter = InCount + 1,
unconfirmed = UC1 }),
{SeqId, State3}.
batch_publish_delivered1({Msg, MsgProps}, {ChPid, Flow, SeqIds, State}) ->
{SeqId, State1} =
publish_delivered1(Msg, MsgProps, ChPid, Flow,
fun maybe_prepare_write_to_disk/4,
State),
{ChPid, Flow, [SeqId | SeqIds], State1}.
2009-10-07 23:27:23 +08:00
2010-06-16 15:01:53 +08:00
maybe_write_msg_to_disk(_Force, MsgStatus = #msg_status {
msg_in_store = true }, State) ->
{MsgStatus, State};
maybe_write_msg_to_disk(Force, MsgStatus = #msg_status {
msg = Msg, msg_id = MsgId,
is_persistent = IsPersistent },
State = #vqstate{ msg_store_clients = MSCState,
disk_write_count = Count})
when Force orelse IsPersistent ->
case persist_to(MsgStatus) of
msg_store -> ok = msg_store_write(MSCState, IsPersistent, MsgId,
prepare_to_store(Msg)),
{MsgStatus#msg_status{msg_in_store = true},
State#vqstate{disk_write_count = Count + 1}};
queue_index -> {MsgStatus, State}
end;
maybe_write_msg_to_disk(_Force, MsgStatus, State) ->
{MsgStatus, State}.
2019-02-13 01:18:14 +08:00
%% Due to certain optimisations made inside
%% rabbit_queue_index:pre_publish/7 we need to have two separate
%% functions for index persistence. This one is only used when paging
2015-08-28 01:53:16 +08:00
%% during memory pressure. We didn't want to modify
%% maybe_write_index_to_disk/3 because that function is used in other
%% places.
maybe_batch_write_index_to_disk(_Force,
MsgStatus = #msg_status {
index_on_disk = true }, State) ->
{MsgStatus, State};
maybe_batch_write_index_to_disk(Force,
MsgStatus = #msg_status {
msg = Msg,
msg_id = MsgId,
seq_id = SeqId,
is_persistent = IsPersistent,
is_delivered = IsDelivered,
msg_props = MsgProps},
State = #vqstate {
target_ram_count = TargetRamCount,
disk_write_count = DiskWriteCount,
index_state = IndexState})
when Force orelse IsPersistent ->
{MsgOrId, DiskWriteCount1} =
case persist_to(MsgStatus) of
msg_store -> {MsgId, DiskWriteCount};
queue_index -> {prepare_to_store(Msg), DiskWriteCount + 1}
end,
IndexState1 = ?INDEX:pre_publish(
MsgOrId, SeqId, MsgProps, IsPersistent, IsDelivered,
TargetRamCount, IndexState),
{MsgStatus#msg_status{index_on_disk = true},
State#vqstate{index_state = IndexState1,
disk_write_count = DiskWriteCount1}};
maybe_batch_write_index_to_disk(_Force, MsgStatus, State) ->
{MsgStatus, State}.
2021-05-17 20:38:09 +08:00
maybe_write_index_to_disk(_Force, MsgStatus = #msg_status {
index_on_disk = true }, State) ->
{MsgStatus, State};
maybe_write_index_to_disk(Force, MsgStatus = #msg_status {
msg = Msg,
msg_id = MsgId,
seq_id = SeqId,
is_persistent = IsPersistent,
is_delivered = IsDelivered,
msg_props = MsgProps},
State = #vqstate{target_ram_count = TargetRamCount,
disk_write_count = DiskWriteCount,
index_state = IndexState})
when Force orelse IsPersistent ->
{MsgOrId, DiskWriteCount1} =
case persist_to(MsgStatus) of
msg_store -> {MsgId, DiskWriteCount};
queue_index -> {prepare_to_store(Msg), DiskWriteCount + 1}
end,
IndexState2 = ?INDEX:publish(
MsgOrId, SeqId, MsgProps, IsPersistent, IsDelivered, TargetRamCount,
IndexState),
{MsgStatus#msg_status{index_on_disk = true},
State#vqstate{index_state = IndexState2,
disk_write_count = DiskWriteCount1}};
maybe_write_index_to_disk(_Force, MsgStatus, State) ->
{MsgStatus, State}.
maybe_write_to_disk(ForceMsg, ForceIndex, MsgStatus, State) ->
{MsgStatus1, State1} = maybe_write_msg_to_disk(ForceMsg, MsgStatus, State),
maybe_write_index_to_disk(ForceIndex, MsgStatus1, State1).
2010-06-03 21:07:28 +08:00
maybe_prepare_write_to_disk(ForceMsg, ForceIndex, MsgStatus, State) ->
{MsgStatus1, State1} = maybe_write_msg_to_disk(ForceMsg, MsgStatus, State),
maybe_batch_write_index_to_disk(ForceIndex, MsgStatus1, State1).
determine_persist_to(_, _, _) -> msg_store.
%determine_persist_to(#basic_message{
% content = #content{properties = Props,
% properties_bin = PropsBin}},
% #message_properties{size = BodySize},
% IndexMaxSize) ->
% %% The >= is so that you can set the env to 0 and never persist
% %% to the index.
% %%
% %% We want this to be fast, so we avoid size(term_to_binary())
% %% here, or using the term size estimation from truncate.erl, both
% %% of which are too slow. So instead, if the message body size
% %% goes over the limit then we avoid any other checks.
% %%
% %% If it doesn't we need to decide if the properties will push
% %% it past the limit. If we have the encoded properties (usual
% %% case) we can just check their size. If we don't (message came
% %% via the direct client), we make a guess based on the number of
% %% headers.
% case BodySize >= IndexMaxSize of
% true -> msg_store;
% false -> Est = case is_binary(PropsBin) of
% true -> BodySize + size(PropsBin);
% false -> #'P_basic'{headers = Hs} = Props,
% case Hs of
% undefined -> 0;
% _ -> length(Hs)
% end * ?HEADER_GUESS_SIZE + BodySize
% end,
% case Est >= IndexMaxSize of
% true -> msg_store;
% false -> queue_index
% end
% end.
persist_to(#msg_status{persist_to = To}) -> To.
prepare_to_store(Msg) ->
Msg#basic_message{
%% don't persist any recoverable decoded properties
content = rabbit_binary_parser:clear_decoded_content(
Msg #basic_message.content)}.
2010-07-15 14:54:25 +08:00
%%----------------------------------------------------------------------------
%% Internal gubbins for acks
%%----------------------------------------------------------------------------
record_pending_ack(#msg_status { seq_id = SeqId } = MsgStatus,
State = #vqstate { ram_pending_ack = RPA,
disk_pending_ack = DPA,
qi_pending_ack = QPA,
ack_in_counter = AckInCount}) ->
Insert = fun (Tree) -> gb_trees:insert(SeqId, MsgStatus, Tree) end,
{RPA1, DPA1, QPA1} =
case {msg_in_ram(MsgStatus), persist_to(MsgStatus)} of
2014-12-12 18:51:17 +08:00
{false, _} -> {RPA, Insert(DPA), QPA};
{_, queue_index} -> {RPA, DPA, Insert(QPA)};
{_, msg_store} -> {Insert(RPA), DPA, QPA}
end,
State #vqstate { ram_pending_ack = RPA1,
disk_pending_ack = DPA1,
qi_pending_ack = QPA1,
ack_in_counter = AckInCount + 1}.
lookup_pending_ack(SeqId, #vqstate { ram_pending_ack = RPA,
disk_pending_ack = DPA,
qi_pending_ack = QPA}) ->
case gb_trees:lookup(SeqId, RPA) of
{value, V} -> V;
none -> case gb_trees:lookup(SeqId, DPA) of
{value, V} -> V;
none -> gb_trees:get(SeqId, QPA)
end
end.
2010-07-15 14:54:25 +08:00
2015-01-23 21:10:42 +08:00
%% First parameter = UpdateStats
2014-08-11 23:04:11 +08:00
remove_pending_ack(true, SeqId, State) ->
2016-10-26 21:59:12 +08:00
case remove_pending_ack(false, SeqId, State) of
{none, _} ->
{none, State};
{MsgStatus, State1} ->
{MsgStatus, stats({0, -1}, {MsgStatus, none}, 0, State1)}
2016-10-26 21:59:12 +08:00
end;
remove_pending_ack(false, SeqId, State = #vqstate{ram_pending_ack = RPA,
disk_pending_ack = DPA,
qi_pending_ack = QPA}) ->
case gb_trees:lookup(SeqId, RPA) of
{value, V} -> RPA1 = gb_trees:delete(SeqId, RPA),
{V, State #vqstate { ram_pending_ack = RPA1 }};
none -> case gb_trees:lookup(SeqId, DPA) of
{value, V} ->
DPA1 = gb_trees:delete(SeqId, DPA),
{V, State#vqstate{disk_pending_ack = DPA1}};
none ->
2016-10-26 21:59:12 +08:00
case gb_trees:lookup(SeqId, QPA) of
{value, V} ->
QPA1 = gb_trees:delete(SeqId, QPA),
{V, State#vqstate{qi_pending_ack = QPA1}};
none ->
{none, State}
end
end
end.
purge_pending_ack(KeepPersistent,
State = #vqstate { index_state = IndexState,
msg_store_clients = MSCState }) ->
{IndexOnDiskSeqIds, MsgIdsByStore, State1} = purge_pending_ack1(State),
case KeepPersistent of
true -> remove_transient_msgs_by_id(MsgIdsByStore, MSCState),
State1;
false -> IndexState1 =
?INDEX:ack(IndexOnDiskSeqIds, IndexState),
remove_msgs_by_id(MsgIdsByStore, MSCState),
State1 #vqstate { index_state = IndexState1 }
end.
2015-09-08 00:23:19 +08:00
purge_pending_ack_delete_and_terminate(
State = #vqstate { index_state = IndexState,
msg_store_clients = MSCState }) ->
{_, MsgIdsByStore, State1} = purge_pending_ack1(State),
IndexState1 = ?INDEX:delete_and_terminate(IndexState),
remove_msgs_by_id(MsgIdsByStore, MSCState),
State1 #vqstate { index_state = IndexState1 }.
purge_pending_ack1(State = #vqstate { ram_pending_ack = RPA,
disk_pending_ack = DPA,
qi_pending_ack = QPA }) ->
F = fun (_SeqId, MsgStatus, Acc) -> accumulate_ack(MsgStatus, Acc) end,
2011-09-29 17:14:36 +08:00
{IndexOnDiskSeqIds, MsgIdsByStore, _AllMsgIds} =
rabbit_misc:gb_trees_fold(
F, rabbit_misc:gb_trees_fold(
F, rabbit_misc:gb_trees_fold(
F, accumulate_ack_init(), RPA), DPA), QPA),
State1 = State #vqstate { ram_pending_ack = gb_trees:empty(),
disk_pending_ack = gb_trees:empty(),
qi_pending_ack = gb_trees:empty()},
{IndexOnDiskSeqIds, MsgIdsByStore, State1}.
2017-10-30 23:33:35 +08:00
%% MsgIdsByStore is an map with two keys:
%%
%% true: holds a list of Persistent Message Ids.
%% false: holds a list of Transient Message Ids.
%%
2017-10-30 23:33:35 +08:00
%% When we call maps:to_list/1 we get two sets of msg ids, where
%% IsPersistent is either true for persistent messages or false for
%% transient ones. The msg_store_remove/3 function takes this boolean
%% flag to determine from which store the messages should be removed
%% from.
remove_msgs_by_id(MsgIdsByStore, MSCState) ->
[ok = msg_store_remove(MSCState, IsPersistent, MsgIds)
2017-10-30 23:33:35 +08:00
|| {IsPersistent, MsgIds} <- maps:to_list(MsgIdsByStore)].
remove_transient_msgs_by_id(MsgIdsByStore, MSCState) ->
2017-10-30 23:33:35 +08:00
case maps:find(false, MsgIdsByStore) of
error -> ok;
{ok, MsgIds} -> ok = msg_store_remove(MSCState, false, MsgIds)
2010-07-15 14:54:25 +08:00
end.
2017-10-30 23:33:35 +08:00
accumulate_ack_init() -> {[], maps:new(), []}.
2010-07-15 14:54:25 +08:00
accumulate_ack(#msg_status { seq_id = SeqId,
msg_id = MsgId,
is_persistent = IsPersistent,
msg_in_store = MsgInStore,
index_on_disk = IndexOnDisk },
2011-09-29 17:14:36 +08:00
{IndexOnDiskSeqIdsAcc, MsgIdsByStore, AllMsgIds}) ->
{cons_if(IndexOnDisk, SeqId, IndexOnDiskSeqIdsAcc),
case MsgInStore of
2017-10-30 23:33:35 +08:00
true -> rabbit_misc:maps_cons(IsPersistent, MsgId, MsgIdsByStore);
false -> MsgIdsByStore
end,
2011-04-08 19:03:42 +08:00
[MsgId | AllMsgIds]}.
2010-07-15 14:54:25 +08:00
%%----------------------------------------------------------------------------
%% Internal plumbing for confirms (aka publisher acks)
%%----------------------------------------------------------------------------
2011-03-08 00:15:40 +08:00
record_confirms(MsgIdSet, State = #vqstate { msgs_on_disk = MOD,
msg_indices_on_disk = MIOD,
unconfirmed = UC,
confirmed = C }) ->
2012-04-12 00:44:43 +08:00
State #vqstate {
msgs_on_disk = rabbit_misc:gb_sets_difference(MOD, MsgIdSet),
msg_indices_on_disk = rabbit_misc:gb_sets_difference(MIOD, MsgIdSet),
unconfirmed = rabbit_misc:gb_sets_difference(UC, MsgIdSet),
confirmed = gb_sets:union(C, MsgIdSet) }.
2011-01-12 20:04:24 +08:00
msgs_written_to_disk(Callback, MsgIdSet, ignored) ->
2012-10-17 00:41:24 +08:00
Callback(?MODULE,
fun (?MODULE, State) -> record_confirms(MsgIdSet, State) end);
2011-03-08 00:15:40 +08:00
msgs_written_to_disk(Callback, MsgIdSet, written) ->
2011-04-08 19:03:42 +08:00
Callback(?MODULE,
fun (?MODULE, State = #vqstate { msgs_on_disk = MOD,
msg_indices_on_disk = MIOD,
unconfirmed = UC }) ->
2011-03-08 00:15:40 +08:00
Confirmed = gb_sets:intersection(UC, MsgIdSet),
record_confirms(gb_sets:intersection(MsgIdSet, MIOD),
State #vqstate {
msgs_on_disk =
2011-03-08 00:15:40 +08:00
gb_sets:union(MOD, Confirmed) })
end).
2011-03-08 00:15:40 +08:00
msg_indices_written_to_disk(Callback, MsgIdSet) ->
2011-04-08 19:03:42 +08:00
Callback(?MODULE,
fun (?MODULE, State = #vqstate { msgs_on_disk = MOD,
msg_indices_on_disk = MIOD,
unconfirmed = UC }) ->
2011-03-08 00:15:40 +08:00
Confirmed = gb_sets:intersection(UC, MsgIdSet),
record_confirms(gb_sets:intersection(MsgIdSet, MOD),
State #vqstate {
msg_indices_on_disk =
2011-03-08 00:15:40 +08:00
gb_sets:union(MIOD, Confirmed) })
end).
2010-09-29 23:57:48 +08:00
2014-12-04 01:27:16 +08:00
msgs_and_indices_written_to_disk(Callback, MsgIdSet) ->
Callback(?MODULE,
fun (?MODULE, State) -> record_confirms(MsgIdSet, State) end).
%%----------------------------------------------------------------------------
%% Internal plumbing for requeue
%%----------------------------------------------------------------------------
2011-10-03 04:06:02 +08:00
publish_alpha(#msg_status { msg = undefined } = MsgStatus, State) ->
2013-01-12 18:18:28 +08:00
{Msg, State1} = read_msg(MsgStatus, State),
MsgStatus1 = MsgStatus#msg_status { msg = Msg },
{MsgStatus1, stats({1, -1}, {MsgStatus, MsgStatus1}, 0, State1)};
2013-01-12 18:18:28 +08:00
publish_alpha(MsgStatus, State) ->
{MsgStatus, stats({1, -1}, {MsgStatus, MsgStatus}, 0, State)}.
2011-10-03 04:06:02 +08:00
publish_beta(MsgStatus, State) ->
2016-01-27 23:59:21 +08:00
{MsgStatus1, State1} = maybe_prepare_write_to_disk(true, false, MsgStatus, State),
MsgStatus2 = m(trim_msg_status(MsgStatus1)),
{MsgStatus2, stats({1, -1}, {MsgStatus, MsgStatus2}, 0, State1)}.
2011-09-30 04:15:15 +08:00
%% Rebuild queue, inserting sequence ids to maintain ordering
queue_merge(SeqIds, Q, MsgIds, Limit, PubFun, State) ->
2011-10-03 04:06:02 +08:00
queue_merge(SeqIds, Q, ?QUEUE:new(), MsgIds,
Limit, PubFun, State).
2011-09-30 04:15:15 +08:00
2011-10-03 04:06:02 +08:00
queue_merge([SeqId | Rest] = SeqIds, Q, Front, MsgIds,
Limit, PubFun, State)
2011-09-30 04:15:15 +08:00
when Limit == undefined orelse SeqId < Limit ->
2011-10-03 04:06:02 +08:00
case ?QUEUE:out(Q) of
2011-09-30 04:15:15 +08:00
{{value, #msg_status { seq_id = SeqIdQ } = MsgStatus}, Q1}
when SeqIdQ < SeqId ->
%% enqueue from the remaining queue
2011-10-03 04:06:02 +08:00
queue_merge(SeqIds, Q1, ?QUEUE:in(MsgStatus, Front), MsgIds,
Limit, PubFun, State);
2011-09-30 04:15:15 +08:00
{_, _Q1} ->
%% enqueue from the remaining list of sequence ids
2016-10-26 21:59:12 +08:00
case msg_from_pending_ack(SeqId, State) of
{none, _} ->
queue_merge(Rest, Q, Front, MsgIds, Limit, PubFun, State);
{MsgStatus, State1} ->
{#msg_status { msg_id = MsgId } = MsgStatus1, State2} =
PubFun(MsgStatus, State1),
queue_merge(Rest, Q, ?QUEUE:in(MsgStatus1, Front), [MsgId | MsgIds],
Limit, PubFun, State2)
end
2011-09-30 04:15:15 +08:00
end;
2011-10-03 04:06:02 +08:00
queue_merge(SeqIds, Q, Front, MsgIds,
_Limit, _PubFun, State) ->
2011-10-03 04:06:02 +08:00
{SeqIds, ?QUEUE:join(Front, Q), MsgIds, State}.
delta_merge([], Delta, MsgIds, State) ->
{Delta, MsgIds, State};
delta_merge(SeqIds, Delta, MsgIds, State) ->
2016-10-26 21:59:12 +08:00
lists:foldl(fun (SeqId, {Delta0, MsgIds0, State0} = Acc) ->
case msg_from_pending_ack(SeqId, State0) of
{none, _} ->
Acc;
{#msg_status { msg_id = MsgId,
is_persistent = IsPersistent } = MsgStatus, State1} ->
2016-10-26 21:59:12 +08:00
{_MsgStatus, State2} =
maybe_prepare_write_to_disk(true, true, MsgStatus, State1),
{expand_delta(SeqId, Delta0, IsPersistent), [MsgId | MsgIds0],
stats({1, -1}, {MsgStatus, none}, 1, State2)}
2016-10-26 21:59:12 +08:00
end
end, {Delta, MsgIds, State}, SeqIds).
2011-09-30 00:48:29 +08:00
%% Mostly opposite of record_pending_ack/2
msg_from_pending_ack(SeqId, State) ->
2016-10-26 21:59:12 +08:00
case remove_pending_ack(false, SeqId, State) of
{none, _} ->
{none, State};
{#msg_status { msg_props = MsgProps } = MsgStatus, State1} ->
{MsgStatus #msg_status {
msg_props = MsgProps #message_properties { needs_confirming = false } },
State1}
end.
2011-09-30 22:02:13 +08:00
beta_limit(Q) ->
case ?QUEUE:peek(Q) of
{value, #msg_status { seq_id = SeqId }} -> SeqId;
empty -> undefined
end.
2021-02-25 06:06:41 +08:00
delta_limit(?BLANK_DELTA_PATTERN(_)) -> undefined;
delta_limit(#delta { start_seq_id = StartSeqId }) -> StartSeqId.
%%----------------------------------------------------------------------------
%% Iterator
%%----------------------------------------------------------------------------
ram_ack_iterator(State) ->
{ack, gb_trees:iterator(State#vqstate.ram_pending_ack)}.
disk_ack_iterator(State) ->
{ack, gb_trees:iterator(State#vqstate.disk_pending_ack)}.
qi_ack_iterator(State) ->
{ack, gb_trees:iterator(State#vqstate.qi_pending_ack)}.
msg_iterator(State) -> istate(start, State).
istate(start, State) -> {q4, State#vqstate.q4, State};
istate(q4, State) -> {q3, State#vqstate.q3, State};
istate(q3, State) -> {delta, State#vqstate.delta, State};
istate(delta, State) -> {q2, State#vqstate.q2, State};
istate(q2, State) -> {q1, State#vqstate.q1, State};
istate(q1, _State) -> done.
next({ack, It}, IndexState) ->
case gb_trees:next(It) of
none -> {empty, IndexState};
{_SeqId, MsgStatus, It1} -> Next = {ack, It1},
{value, MsgStatus, true, Next, IndexState}
end;
next(done, IndexState) -> {empty, IndexState};
next({delta, #delta{start_seq_id = SeqId,
end_seq_id = SeqId}, State}, IndexState) ->
next(istate(delta, State), IndexState);
next({delta, #delta{start_seq_id = SeqId,
end_seq_id = SeqIdEnd} = Delta, State}, IndexState) ->
SeqIdB = ?INDEX:next_segment_boundary(SeqId),
SeqId1 = lists:min([SeqIdB,
%% We must limit the number of messages read at once
%% otherwise the queue will attempt to read up to 65536
%% messages from the new index each time. The value
%% chosen here is arbitrary.
SeqId + 2048,
SeqIdEnd]),
{List, IndexState1} = ?INDEX:read(SeqId, SeqId1, IndexState),
next({delta, Delta#delta{start_seq_id = SeqId1}, List, State}, IndexState1);
next({delta, Delta, [], State}, IndexState) ->
next({delta, Delta, State}, IndexState);
next({delta, Delta, [{_, SeqId, _, _, _} = M | Rest], State}, IndexState) ->
case is_msg_in_pending_acks(SeqId, State) of
false -> Next = {delta, Delta, Rest, State},
{value, beta_msg_status(M), false, Next, IndexState};
true -> next({delta, Delta, Rest, State}, IndexState)
end;
next({Key, Q, State}, IndexState) ->
case ?QUEUE:out(Q) of
{empty, _Q} -> next(istate(Key, State), IndexState);
2013-01-13 18:21:13 +08:00
{{value, MsgStatus}, QN} -> Next = {Key, QN, State},
{value, MsgStatus, false, Next, IndexState}
end.
2013-01-13 18:59:59 +08:00
inext(It, {Its, IndexState}) ->
case next(It, IndexState) of
{empty, IndexState1} ->
2013-01-13 18:59:59 +08:00
{Its, IndexState1};
{value, MsgStatus1, Unacked, It1, IndexState1} ->
{[{MsgStatus1, Unacked, It1} | Its], IndexState1}
2013-01-13 18:59:59 +08:00
end.
rabbit_variable_queue: Ensure modified state is not discarded in ifold Commit d12b4d8e08005f97c5557f0948b3d03c15ef8c1b introduced a bug where the old value of `State` is returned in the `stop` case. The problem was sometimes visible in the backing_queue_SUITE's `variable_queue_fold` testcase. The testcase crashed with the following exception: === Location: [{rabbit_ct_broker_helpers,rpc,1509}, {backing_queue_SUITE,variable_queue_fold,1144}, {test_server,ts_tc,1754}, {test_server,run_test_case_eval1,1263}, {test_server,run_test_case_eval,1195}] === === Reason: {error, {case_clause,undefined}, [{file_handle_cache,'-partition_handles/1-fun-0-',2, [{file,"src/file_handle_cache.erl"},{line,804}]}, {file_handle_cache,get_or_reopen,1, [{file,"src/file_handle_cache.erl"},{line,743}]}, {file_handle_cache_stats,timer_tc,1, [{file,"src/file_handle_cache_stats.erl"}, {line,54}]}, {file_handle_cache_stats,update,2, [{file,"src/file_handle_cache_stats.erl"}, {line,40}]}, {file_handle_cache,with_handles,3, [{file,"src/file_handle_cache.erl"},{line,700}]}, {rabbit_msg_store,read_from_disk,2, [{file,"src/rabbit_msg_store.erl"},{line,1259}]}, {rabbit_msg_store,client_read3,3, [{file,"src/rabbit_msg_store.erl"},{line,671}]}, {rabbit_msg_store,safe_ets_update_counter,5, [{file,"src/rabbit_msg_store.erl"},{line,1314}]}]} The `State` (a #vqstate{} record) contains the rabbit_msg_store's state as well (a #client_msstate{} record). This msg store client state is updated after most calls to the msg store. In particular, the msg store client state contains a map in the `file_handle_cache` field of that record to store all file handles returned by the file_handle_cache. So each time the msg store opens or closes files as part of its operation, the client state is updated with a modifed `file_handle_cache` map. In addition to that, the file_handle_cache module uses the process dictionary to store even more data about the various file handles used by that process. Therefore, by discarding the updated #client_msstate{}, we loose the up-to-date `file_handle_cache` map. However, the process dictionary is still updated by the file_handle_cache module. This means that the map and the process dictionary are out-of-sync at this point. That's what causes the crash in the file_handle_cache module: it is called with arguments which don't work with the data in the process dictionary. This commit fixes that and re-names some variables to clearly show the progression of modified data. Fixes #2488 Major props to @dumbbell for figuring this out, and @pjk25 for assistance!
2020-10-30 23:27:30 +08:00
ifold(_Fun, Acc, [], State0) ->
{Acc, State0};
ifold(Fun, Acc, Its0, State0) ->
[{MsgStatus, Unacked, It} | Rest] =
lists:sort(fun ({#msg_status{seq_id = SeqId1}, _, _},
{#msg_status{seq_id = SeqId2}, _, _}) ->
SeqId1 =< SeqId2
rabbit_variable_queue: Ensure modified state is not discarded in ifold Commit d12b4d8e08005f97c5557f0948b3d03c15ef8c1b introduced a bug where the old value of `State` is returned in the `stop` case. The problem was sometimes visible in the backing_queue_SUITE's `variable_queue_fold` testcase. The testcase crashed with the following exception: === Location: [{rabbit_ct_broker_helpers,rpc,1509}, {backing_queue_SUITE,variable_queue_fold,1144}, {test_server,ts_tc,1754}, {test_server,run_test_case_eval1,1263}, {test_server,run_test_case_eval,1195}] === === Reason: {error, {case_clause,undefined}, [{file_handle_cache,'-partition_handles/1-fun-0-',2, [{file,"src/file_handle_cache.erl"},{line,804}]}, {file_handle_cache,get_or_reopen,1, [{file,"src/file_handle_cache.erl"},{line,743}]}, {file_handle_cache_stats,timer_tc,1, [{file,"src/file_handle_cache_stats.erl"}, {line,54}]}, {file_handle_cache_stats,update,2, [{file,"src/file_handle_cache_stats.erl"}, {line,40}]}, {file_handle_cache,with_handles,3, [{file,"src/file_handle_cache.erl"},{line,700}]}, {rabbit_msg_store,read_from_disk,2, [{file,"src/rabbit_msg_store.erl"},{line,1259}]}, {rabbit_msg_store,client_read3,3, [{file,"src/rabbit_msg_store.erl"},{line,671}]}, {rabbit_msg_store,safe_ets_update_counter,5, [{file,"src/rabbit_msg_store.erl"},{line,1314}]}]} The `State` (a #vqstate{} record) contains the rabbit_msg_store's state as well (a #client_msstate{} record). This msg store client state is updated after most calls to the msg store. In particular, the msg store client state contains a map in the `file_handle_cache` field of that record to store all file handles returned by the file_handle_cache. So each time the msg store opens or closes files as part of its operation, the client state is updated with a modifed `file_handle_cache` map. In addition to that, the file_handle_cache module uses the process dictionary to store even more data about the various file handles used by that process. Therefore, by discarding the updated #client_msstate{}, we loose the up-to-date `file_handle_cache` map. However, the process dictionary is still updated by the file_handle_cache module. This means that the map and the process dictionary are out-of-sync at this point. That's what causes the crash in the file_handle_cache module: it is called with arguments which don't work with the data in the process dictionary. This commit fixes that and re-names some variables to clearly show the progression of modified data. Fixes #2488 Major props to @dumbbell for figuring this out, and @pjk25 for assistance!
2020-10-30 23:27:30 +08:00
end, Its0),
{Msg, State1} = read_msg(MsgStatus, State0),
case Fun(Msg, MsgStatus#msg_status.msg_props, Unacked, Acc) of
{stop, Acc1} ->
rabbit_variable_queue: Ensure modified state is not discarded in ifold Commit d12b4d8e08005f97c5557f0948b3d03c15ef8c1b introduced a bug where the old value of `State` is returned in the `stop` case. The problem was sometimes visible in the backing_queue_SUITE's `variable_queue_fold` testcase. The testcase crashed with the following exception: === Location: [{rabbit_ct_broker_helpers,rpc,1509}, {backing_queue_SUITE,variable_queue_fold,1144}, {test_server,ts_tc,1754}, {test_server,run_test_case_eval1,1263}, {test_server,run_test_case_eval,1195}] === === Reason: {error, {case_clause,undefined}, [{file_handle_cache,'-partition_handles/1-fun-0-',2, [{file,"src/file_handle_cache.erl"},{line,804}]}, {file_handle_cache,get_or_reopen,1, [{file,"src/file_handle_cache.erl"},{line,743}]}, {file_handle_cache_stats,timer_tc,1, [{file,"src/file_handle_cache_stats.erl"}, {line,54}]}, {file_handle_cache_stats,update,2, [{file,"src/file_handle_cache_stats.erl"}, {line,40}]}, {file_handle_cache,with_handles,3, [{file,"src/file_handle_cache.erl"},{line,700}]}, {rabbit_msg_store,read_from_disk,2, [{file,"src/rabbit_msg_store.erl"},{line,1259}]}, {rabbit_msg_store,client_read3,3, [{file,"src/rabbit_msg_store.erl"},{line,671}]}, {rabbit_msg_store,safe_ets_update_counter,5, [{file,"src/rabbit_msg_store.erl"},{line,1314}]}]} The `State` (a #vqstate{} record) contains the rabbit_msg_store's state as well (a #client_msstate{} record). This msg store client state is updated after most calls to the msg store. In particular, the msg store client state contains a map in the `file_handle_cache` field of that record to store all file handles returned by the file_handle_cache. So each time the msg store opens or closes files as part of its operation, the client state is updated with a modifed `file_handle_cache` map. In addition to that, the file_handle_cache module uses the process dictionary to store even more data about the various file handles used by that process. Therefore, by discarding the updated #client_msstate{}, we loose the up-to-date `file_handle_cache` map. However, the process dictionary is still updated by the file_handle_cache module. This means that the map and the process dictionary are out-of-sync at this point. That's what causes the crash in the file_handle_cache module: it is called with arguments which don't work with the data in the process dictionary. This commit fixes that and re-names some variables to clearly show the progression of modified data. Fixes #2488 Major props to @dumbbell for figuring this out, and @pjk25 for assistance!
2020-10-30 23:27:30 +08:00
{Acc1, State1};
{cont, Acc1} ->
rabbit_variable_queue: Ensure modified state is not discarded in ifold Commit d12b4d8e08005f97c5557f0948b3d03c15ef8c1b introduced a bug where the old value of `State` is returned in the `stop` case. The problem was sometimes visible in the backing_queue_SUITE's `variable_queue_fold` testcase. The testcase crashed with the following exception: === Location: [{rabbit_ct_broker_helpers,rpc,1509}, {backing_queue_SUITE,variable_queue_fold,1144}, {test_server,ts_tc,1754}, {test_server,run_test_case_eval1,1263}, {test_server,run_test_case_eval,1195}] === === Reason: {error, {case_clause,undefined}, [{file_handle_cache,'-partition_handles/1-fun-0-',2, [{file,"src/file_handle_cache.erl"},{line,804}]}, {file_handle_cache,get_or_reopen,1, [{file,"src/file_handle_cache.erl"},{line,743}]}, {file_handle_cache_stats,timer_tc,1, [{file,"src/file_handle_cache_stats.erl"}, {line,54}]}, {file_handle_cache_stats,update,2, [{file,"src/file_handle_cache_stats.erl"}, {line,40}]}, {file_handle_cache,with_handles,3, [{file,"src/file_handle_cache.erl"},{line,700}]}, {rabbit_msg_store,read_from_disk,2, [{file,"src/rabbit_msg_store.erl"},{line,1259}]}, {rabbit_msg_store,client_read3,3, [{file,"src/rabbit_msg_store.erl"},{line,671}]}, {rabbit_msg_store,safe_ets_update_counter,5, [{file,"src/rabbit_msg_store.erl"},{line,1314}]}]} The `State` (a #vqstate{} record) contains the rabbit_msg_store's state as well (a #client_msstate{} record). This msg store client state is updated after most calls to the msg store. In particular, the msg store client state contains a map in the `file_handle_cache` field of that record to store all file handles returned by the file_handle_cache. So each time the msg store opens or closes files as part of its operation, the client state is updated with a modifed `file_handle_cache` map. In addition to that, the file_handle_cache module uses the process dictionary to store even more data about the various file handles used by that process. Therefore, by discarding the updated #client_msstate{}, we loose the up-to-date `file_handle_cache` map. However, the process dictionary is still updated by the file_handle_cache module. This means that the map and the process dictionary are out-of-sync at this point. That's what causes the crash in the file_handle_cache module: it is called with arguments which don't work with the data in the process dictionary. This commit fixes that and re-names some variables to clearly show the progression of modified data. Fixes #2488 Major props to @dumbbell for figuring this out, and @pjk25 for assistance!
2020-10-30 23:27:30 +08:00
IndexState0 = State1#vqstate.index_state,
{Its1, IndexState1} = inext(It, {Rest, IndexState0}),
State2 = State1#vqstate{index_state = IndexState1},
ifold(Fun, Acc1, Its1, State2)
end.
2012-11-23 20:28:48 +08:00
%%----------------------------------------------------------------------------
%% Phase changes
%%----------------------------------------------------------------------------
maybe_reduce_memory_use(State = #vqstate {memory_reduction_run_count = MRedRunCount,
mode = Mode}) ->
case MRedRunCount >= ?EXPLICIT_GC_RUN_OP_THRESHOLD(Mode) of
true -> State1 = reduce_memory_use(State),
State1#vqstate{memory_reduction_run_count = 0};
false -> State#vqstate{memory_reduction_run_count = MRedRunCount + 1}
end.
reduce_memory_use(State = #vqstate { target_ram_count = infinity }) ->
State;
reduce_memory_use(State = #vqstate {
2015-10-10 23:51:32 +08:00
mode = default,
ram_pending_ack = RPA,
2010-12-06 03:31:22 +08:00
ram_msg_count = RamMsgCount,
target_ram_count = TargetRamCount,
2015-08-27 23:11:08 +08:00
io_batch_size = IoBatchSize,
2014-01-30 02:15:24 +08:00
rates = #rates { in = AvgIngress,
out = AvgEgress,
ack_in = AvgAckIngress,
ack_out = AvgAckEgress } }) ->
{CreditDiscBound, _} =rabbit_misc:get_env(rabbit,
msg_store_credit_disc_bound,
?CREDIT_DISC_BOUND),
{NeedResumeA2B, State1} = {_, #vqstate { q2 = Q2, q3 = Q3 }} =
case chunk_size(RamMsgCount + gb_trees:size(RPA), TargetRamCount) of
0 -> {false, State};
2010-12-06 02:57:19 +08:00
%% Reduce memory of pending acks and alphas. The order is
%% determined based on which is growing faster. Whichever
%% comes second may very well get a quota of 0 if the
%% first manages to push out the max number of messages.
A2BChunk ->
%% In case there are few messages to be sent to a message store
%% and many messages to be embedded to the queue index,
%% we should limit the number of messages to be flushed
%% to avoid blocking the process.
A2BChunkActual = case A2BChunk > CreditDiscBound * 2 of
true -> CreditDiscBound * 2;
false -> A2BChunk
end,
Funs = case ((AvgAckIngress - AvgAckEgress) >
2012-05-26 18:01:24 +08:00
(AvgIngress - AvgEgress)) of
true -> [fun limit_ram_acks/2,
fun push_alphas_to_betas/2];
false -> [fun push_alphas_to_betas/2,
fun limit_ram_acks/2]
2012-05-26 18:01:24 +08:00
end,
{Quota, State2} = lists:foldl(fun (ReduceFun, {QuotaN, StateN}) ->
2012-05-26 18:01:24 +08:00
ReduceFun(QuotaN, StateN)
end, {A2BChunkActual, State}, Funs),
{(Quota == 0) andalso (A2BChunk > A2BChunkActual), State2}
end,
Permitted = permitted_beta_count(State1),
{NeedResumeB2D, State3} =
%% If there are more messages with their queue position held in RAM,
%% a.k.a. betas, in Q2 & Q3 than IoBatchSize,
%% write their queue position to disk, a.k.a. push_betas_to_deltas
case chunk_size(?QUEUE:len(Q2) + ?QUEUE:len(Q3),
Permitted) of
B2DChunk when B2DChunk >= IoBatchSize ->
%% Same as for alphas to betas. Limit a number of messages
%% to be flushed to disk at once to avoid blocking the process.
B2DChunkActual = case B2DChunk > CreditDiscBound * 2 of
true -> CreditDiscBound * 2;
false -> B2DChunk
end,
StateBD = push_betas_to_deltas(B2DChunkActual, State1),
{B2DChunk > B2DChunkActual, StateBD};
_ ->
{false, State1}
end,
#vqstate{ index_state = IndexState } = State3,
State4 = State3#vqstate{ index_state = ?INDEX:flush(IndexState) },
%% We can be blocked by the credit flow, or limited by a batch size,
%% or finished with flushing.
%% If blocked by the credit flow - the credit grant will resume processing,
%% if limited by a batch - the batch continuation message should be sent.
%% The continuation message will be prioritised over publishes,
2018-04-24 00:37:57 +08:00
%% but not consumptions, so the queue can make progess.
Blocked = credit_flow:blocked(),
case {Blocked, NeedResumeA2B orelse NeedResumeB2D} of
%% Credit bump will continue paging
{true, _} -> State4;
%% Finished with paging
{false, false} -> State4;
%% Planning next batch
{false, true} ->
%% We don't want to use self-credit-flow, because it's harder to
%% reason about. So the process sends a (prioritised) message to
%% itself and sets a waiting_bump value to keep the message box clean
maybe_bump_reduce_memory_use(State4)
end;
2015-10-10 23:51:32 +08:00
%% When using lazy queues, there are no alphas, so we don't need to
%% call push_alphas_to_betas/2.
reduce_memory_use(State = #vqstate {
mode = lazy,
ram_pending_ack = RPA,
ram_msg_count = RamMsgCount,
target_ram_count = TargetRamCount }) ->
State1 = #vqstate { q3 = Q3 } =
case chunk_size(RamMsgCount + gb_trees:size(RPA), TargetRamCount) of
0 -> State;
S1 -> {_, State2} = limit_ram_acks(S1, State),
State2
end,
State3 =
case chunk_size(?QUEUE:len(Q3),
permitted_beta_count(State1)) of
0 ->
State1;
S2 ->
push_betas_to_deltas(S2, State1)
end,
#vqstate{ index_state = IndexState } = State3,
State4 = State3#vqstate{ index_state = ?INDEX:flush(IndexState) },
garbage_collect(),
State4.
2010-06-23 01:56:00 +08:00
maybe_bump_reduce_memory_use(State = #vqstate{ waiting_bump = true }) ->
State;
maybe_bump_reduce_memory_use(State) ->
self() ! bump_reduce_memory_use,
State#vqstate{ waiting_bump = true }.
2010-10-14 21:19:21 +08:00
limit_ram_acks(0, State) ->
2015-08-28 22:12:26 +08:00
{0, ui(State)};
limit_ram_acks(Quota, State = #vqstate { ram_pending_ack = RPA,
disk_pending_ack = DPA }) ->
case gb_trees:is_empty(RPA) of
true ->
2015-08-28 22:12:26 +08:00
{Quota, ui(State)};
false ->
{SeqId, MsgStatus, RPA1} = gb_trees:take_largest(RPA),
{MsgStatus1, State1} =
maybe_prepare_write_to_disk(true, false, MsgStatus, State),
MsgStatus2 = m(trim_msg_status(MsgStatus1)),
DPA1 = gb_trees:insert(SeqId, MsgStatus2, DPA),
limit_ram_acks(Quota - 1,
stats({0, 0}, {MsgStatus, MsgStatus2}, 0,
2015-01-23 21:10:42 +08:00
State1 #vqstate { ram_pending_ack = RPA1,
disk_pending_ack = DPA1 }))
end.
2010-10-14 21:19:21 +08:00
permitted_beta_count(#vqstate { len = 0 }) ->
2010-06-23 01:03:11 +08:00
infinity;
2015-10-10 23:51:32 +08:00
permitted_beta_count(#vqstate { mode = lazy,
target_ram_count = TargetRamCount}) ->
TargetRamCount;
2011-10-17 20:02:02 +08:00
permitted_beta_count(#vqstate { target_ram_count = 0, q3 = Q3 }) ->
lists:min([?QUEUE:len(Q3), ?INDEX:next_segment_boundary(0)]);
permitted_beta_count(#vqstate { q1 = Q1,
q4 = Q4,
target_ram_count = TargetRamCount,
len = Len }) ->
BetaDelta = Len - ?QUEUE:len(Q1) - ?QUEUE:len(Q4),
lists:max([?INDEX:next_segment_boundary(0),
BetaDelta - ((BetaDelta * BetaDelta) div
(BetaDelta + TargetRamCount))]).
2010-06-23 01:03:11 +08:00
chunk_size(Current, Permitted)
2010-07-07 02:18:25 +08:00
when Permitted =:= infinity orelse Permitted >= Current ->
0;
chunk_size(Current, Permitted) ->
Current - Permitted.
2010-07-07 02:18:25 +08:00
2015-10-10 23:53:43 +08:00
fetch_from_q3(State = #vqstate { mode = default,
q1 = Q1,
q2 = Q2,
delta = #delta { count = DeltaCount },
q3 = Q3,
q4 = Q4 }) ->
case ?QUEUE:out(Q3) of
2010-07-15 19:59:59 +08:00
{empty, _Q3} ->
{empty, State};
{{value, MsgStatus}, Q3a} ->
State1 = State #vqstate { q3 = Q3a },
State2 = case {?QUEUE:is_empty(Q3a), 0 == DeltaCount} of
{true, true} ->
%% q3 is now empty, it wasn't before;
%% delta is still empty. So q2 must be
%% empty, and we know q4 is empty
%% otherwise we wouldn't be loading from
%% q3. As such, we can just set q4 to Q1.
true = ?QUEUE:is_empty(Q2), %% ASSERTION
true = ?QUEUE:is_empty(Q4), %% ASSERTION
State1 #vqstate { q1 = ?QUEUE:new(), q4 = Q1 };
{true, false} ->
maybe_deltas_to_betas(State1);
{false, _} ->
%% q3 still isn't empty, we've not
%% touched delta, so the invariants
%% between q1, q2, delta and q3 are
%% maintained
State1
end,
{loaded, {MsgStatus, State2}}
2015-10-10 23:53:43 +08:00
end;
%% lazy queues
fetch_from_q3(State = #vqstate { mode = lazy,
delta = #delta { count = DeltaCount },
q3 = Q3 }) ->
case ?QUEUE:out(Q3) of
{empty, _Q3} when DeltaCount =:= 0 ->
{empty, State};
{empty, _Q3} ->
fetch_from_q3(maybe_deltas_to_betas(State));
{{value, MsgStatus}, Q3a} ->
State1 = State #vqstate { q3 = Q3a },
{loaded, {MsgStatus, State1}}
2010-07-15 19:59:59 +08:00
end.
2015-09-06 18:45:14 +08:00
maybe_deltas_to_betas(State) ->
AfterFun = process_delivers_and_acks_fun(deliver_and_ack),
maybe_deltas_to_betas(AfterFun, State).
2015-09-06 18:45:14 +08:00
2015-09-07 20:03:15 +08:00
maybe_deltas_to_betas(_DelsAndAcksFun,
State = #vqstate {delta = ?BLANK_DELTA_PATTERN(X) }) ->
State;
2015-09-06 18:45:14 +08:00
maybe_deltas_to_betas(DelsAndAcksFun,
State = #vqstate {
2010-06-04 05:16:23 +08:00
q2 = Q2,
delta = Delta,
q3 = Q3,
index_state = IndexState,
ram_msg_count = RamMsgCount,
ram_bytes = RamBytes,
disk_read_count = DiskReadCount,
delta_transient_bytes = DeltaTransientBytes,
2010-06-04 05:16:23 +08:00
transient_threshold = TransientThreshold }) ->
#delta { start_seq_id = DeltaSeqId,
count = DeltaCount,
transient = Transient,
end_seq_id = DeltaSeqIdEnd } = Delta,
DeltaSeqId1 =
lists:min([?INDEX:next_segment_boundary(DeltaSeqId),
%% We must limit the number of messages read at once
%% otherwise the queue will attempt to read up to 65536
%% messages from the new index each time. The value
%% chosen here is arbitrary.
DeltaSeqId + 2048,
DeltaSeqIdEnd]),
{List, IndexState1} = ?INDEX:read(DeltaSeqId, DeltaSeqId1,
IndexState),
{Q3a, RamCountsInc, RamBytesInc, State1, TransientCount, TransientBytes} =
betas_from_index_entries(List, TransientThreshold,
DelsAndAcksFun,
State #vqstate { index_state = IndexState1 }),
2015-09-06 18:45:14 +08:00
State2 = State1 #vqstate { ram_msg_count = RamMsgCount + RamCountsInc,
ram_bytes = RamBytes + RamBytesInc,
2015-09-07 20:03:15 +08:00
disk_read_count = DiskReadCount + RamCountsInc },
case ?QUEUE:len(Q3a) of
0 ->
%% we ignored every message in the segment due to it being
%% transient and below the threshold
maybe_deltas_to_betas(
2015-09-06 18:45:14 +08:00
DelsAndAcksFun,
State2 #vqstate {
delta = d(Delta #delta { start_seq_id = DeltaSeqId1 })});
Q3aLen ->
Q3b = ?QUEUE:join(Q3, Q3a),
case DeltaCount - Q3aLen of
0 ->
%% delta is now empty, but it wasn't before, so
%% can now join q2 onto q3
2015-09-06 18:45:14 +08:00
State2 #vqstate { q2 = ?QUEUE:new(),
delta = ?BLANK_DELTA,
q3 = ?QUEUE:join(Q3b, Q2),
delta_transient_bytes = 0};
N when N > 0 ->
Delta1 = d(#delta { start_seq_id = DeltaSeqId1,
count = N,
transient = Transient - TransientCount,
end_seq_id = DeltaSeqIdEnd }),
2015-09-06 18:45:14 +08:00
State2 #vqstate { delta = Delta1,
q3 = Q3b,
delta_transient_bytes = DeltaTransientBytes - TransientBytes }
end
end.
push_alphas_to_betas(Quota, State) ->
2011-10-24 16:08:43 +08:00
{Quota1, State1} =
push_alphas_to_betas(
fun ?QUEUE:out/1,
fun (MsgStatus, Q1a,
State0 = #vqstate { q3 = Q3, delta = #delta { count = 0,
transient = 0 } }) ->
2011-10-24 16:08:43 +08:00
State0 #vqstate { q1 = Q1a, q3 = ?QUEUE:in(MsgStatus, Q3) };
(MsgStatus, Q1a, State0 = #vqstate { q2 = Q2 }) ->
State0 #vqstate { q1 = Q1a, q2 = ?QUEUE:in(MsgStatus, Q2) }
end, Quota, State #vqstate.q1, State),
{Quota2, State2} =
push_alphas_to_betas(
fun ?QUEUE:out_r/1,
fun (MsgStatus, Q4a, State0 = #vqstate { q3 = Q3 }) ->
State0 #vqstate { q3 = ?QUEUE:in_r(MsgStatus, Q3), q4 = Q4a }
end, Quota1, State1 #vqstate.q4, State1),
{Quota2, State2}.
2011-10-24 16:08:43 +08:00
push_alphas_to_betas(_Generator, _Consumer, Quota, _Q,
State = #vqstate { ram_msg_count = RamMsgCount,
target_ram_count = TargetRamCount })
2010-07-06 23:05:02 +08:00
when Quota =:= 0 orelse
2010-12-06 03:31:22 +08:00
TargetRamCount =:= infinity orelse
TargetRamCount >= RamMsgCount ->
2015-08-28 22:12:26 +08:00
{Quota, ui(State)};
2011-10-24 16:08:43 +08:00
push_alphas_to_betas(Generator, Consumer, Quota, Q, State) ->
%% We consume credits from the message_store whenever we need to
%% persist a message to disk. See:
%% rabbit_variable_queue:msg_store_write/4. So perhaps the
%% msg_store is trying to throttle down our queue.
case credit_flow:blocked() of
2015-08-28 22:12:26 +08:00
true -> {Quota, ui(State)};
false -> case Generator(Q) of
{empty, _Q} ->
2015-08-28 22:12:26 +08:00
{Quota, ui(State)};
{{value, MsgStatus}, Qa} ->
{MsgStatus1, State1} =
maybe_prepare_write_to_disk(true, false, MsgStatus,
State),
MsgStatus2 = m(trim_msg_status(MsgStatus1)),
2015-01-23 21:10:42 +08:00
State2 = stats(
ready0, {MsgStatus, MsgStatus2}, 0, State1),
State3 = Consumer(MsgStatus2, Qa, State2),
push_alphas_to_betas(Generator, Consumer, Quota - 1,
Qa, State3)
end
2009-10-07 23:27:23 +08:00
end.
2015-10-10 23:51:32 +08:00
push_betas_to_deltas(Quota, State = #vqstate { mode = default,
q2 = Q2,
delta = Delta,
q3 = Q3}) ->
PushState = {Quota, Delta, State},
{Q3a, PushState1} = push_betas_to_deltas(
fun ?QUEUE:out_r/1,
fun ?INDEX:next_segment_boundary/1,
Q3, PushState),
{Q2a, PushState2} = push_betas_to_deltas(
fun ?QUEUE:out/1,
fun (Q2MinSeqId) -> Q2MinSeqId end,
Q2, PushState1),
{_, Delta1, State1} = PushState2,
State1 #vqstate { q2 = Q2a,
delta = Delta1,
2015-10-10 23:51:32 +08:00
q3 = Q3a };
%% In the case of lazy queues we want to page as many messages as
%% possible from q3.
push_betas_to_deltas(Quota, State = #vqstate { mode = lazy,
delta = Delta,
q3 = Q3}) ->
PushState = {Quota, Delta, State},
{Q3a, PushState1} = push_betas_to_deltas(
fun ?QUEUE:out_r/1,
fun (Q2MinSeqId) -> Q2MinSeqId end,
Q3, PushState),
{_, Delta1, State1} = PushState1,
State1 #vqstate { delta = Delta1,
q3 = Q3a }.
2015-10-10 23:51:32 +08:00
push_betas_to_deltas(Generator, LimitFun, Q, PushState) ->
case ?QUEUE:is_empty(Q) of
true ->
{Q, PushState};
false ->
{value, #msg_status { seq_id = MinSeqId }} = ?QUEUE:peek(Q),
{value, #msg_status { seq_id = MaxSeqId }} = ?QUEUE:peek_r(Q),
Limit = LimitFun(MinSeqId),
case MaxSeqId < Limit of
true -> {Q, PushState};
false -> push_betas_to_deltas1(Generator, Limit, Q, PushState)
end
end.
2015-08-28 22:19:35 +08:00
push_betas_to_deltas1(_Generator, _Limit, Q, {0, Delta, State}) ->
2015-08-28 22:12:26 +08:00
{Q, {0, Delta, ui(State)}};
2015-08-28 22:19:35 +08:00
push_betas_to_deltas1(Generator, Limit, Q, {Quota, Delta, State}) ->
2009-10-07 23:27:23 +08:00
case Generator(Q) of
{empty, _Q} ->
2015-08-28 22:12:26 +08:00
{Q, {Quota, Delta, ui(State)}};
{{value, #msg_status { seq_id = SeqId }}, _Qa}
when SeqId < Limit ->
2015-08-28 22:12:26 +08:00
{Q, {Quota, Delta, ui(State)}};
2011-10-17 20:02:02 +08:00
{{value, MsgStatus = #msg_status { seq_id = SeqId }}, Qa} ->
{#msg_status { index_on_disk = true,
is_persistent = IsPersistent }, State1} =
maybe_batch_write_index_to_disk(true, MsgStatus, State),
State2 = stats(ready0, {MsgStatus, none}, 1, State1),
Delta1 = expand_delta(SeqId, Delta, IsPersistent),
push_betas_to_deltas1(Generator, Limit, Qa,
{Quota - 1, Delta1, State2})
2009-10-07 23:27:23 +08:00
end.
2011-02-11 20:51:50 +08:00
2015-08-28 22:12:26 +08:00
%% Flushes queue index batch caches and updates queue index state.
ui(#vqstate{index_state = IndexState,
target_ram_count = TargetRamCount} = State) ->
IndexState1 = ?INDEX:flush_pre_publish_cache(
2015-08-28 22:12:26 +08:00
TargetRamCount, IndexState),
State#vqstate{index_state = IndexState1}.
2011-02-23 00:57:39 +08:00
%%----------------------------------------------------------------------------
%% Upgrading
%%----------------------------------------------------------------------------
2018-11-30 18:30:36 +08:00
-spec multiple_routing_keys() -> 'ok'.
2011-02-23 00:57:39 +08:00
multiple_routing_keys() ->
transform_storage(
2011-03-05 04:31:09 +08:00
fun ({basic_message, ExchangeName, Routing_Key, Content,
2011-03-05 04:37:37 +08:00
MsgId, Persistent}) ->
2011-03-05 04:31:09 +08:00
{ok, {basic_message, ExchangeName, [Routing_Key], Content,
2011-03-05 04:37:37 +08:00
MsgId, Persistent}};
2011-03-05 04:31:09 +08:00
(_) -> {error, corrupt_message}
end),
2011-02-23 00:57:39 +08:00
ok.
%% Assumes message store is not running
transform_storage(TransformFun) ->
transform_store(?PERSISTENT_MSG_STORE, TransformFun),
transform_store(?TRANSIENT_MSG_STORE, TransformFun).
2011-02-23 00:57:39 +08:00
transform_store(Store, TransformFun) ->
rabbit_msg_store:force_recovery(rabbit_mnesia:dir(), Store),
rabbit_msg_store:transform_dir(rabbit_mnesia:dir(), Store, TransformFun).
2016-04-06 19:28:02 +08:00
move_messages_to_vhost_store() ->
case list_persistent_queues() of
[] ->
log_upgrade("No durable queues found."
" Skipping message store migration"),
ok;
Queues ->
move_messages_to_vhost_store(Queues)
end,
ok = delete_old_store(),
ok = rabbit_queue_index:cleanup_global_recovery_terms().
move_messages_to_vhost_store(Queues) ->
log_upgrade("Moving messages to per-vhost message store"),
%% Move the queue index for each persistent queue to the new store
lists:foreach(
fun(Queue) ->
Make #amqqueue{} a private record + add a feature flag The #amqqueue{} record is now isolated in the `amqqueue` module: all accesses and modifications to the record are made through this helper. The type `rabbit_types:amqqueue()` is now `amqqueue:amqqueue()`. `amqqueue` knows about the new #amqqueue{} record introduced with quorum queues. If the actual record is the old one (as defined in RabbitMQ 3.7.x), it calls `amqqueue_v1` which knows about the old definition. When it needs to produce an #amqqueue{} record, it verifies if it is allowed to, using the new Feature flags subsystem and the `quorum_queue` feature flag specifically, and proceeds. If an old format #amqqueue{} is required, the creation is deferred to `amqqueue_v1`. The new modules come with a couple headers: they provide macros to replace pattern matching with `when` conditions. The `amqqueue_v*.hrl` headers are generated using the `macros()` functions in `amqqueue` and `amqqueue_v1` modules. Some operations now depend on the state of the `quorum_queue` feature flag. In particular, creating a quorum queue is denied if the feature flag is disabled. In the process, the following modules were moved from rabbitmq-common to rabbitmq-server: * rabbit_backing_queue * rabbit_queue_master_locator Likewise, the few functions were copied from `rabbit_misc` to `rabbit_amqqueue` because they depend on `amqqueue`. They don't really need to live in rabbitmq-common and they prevent further progress with the feature flags: feature flags need informations about the cluster. The `quorum_queue` feature flag comes with a migration function which is responsible of converting the #amqqueue{} records in Mnesia tables `rabbit_queue` and `rabbit_durable_queue`. Therefore, the upgrade function which did this is removed in this commit. [#159298729]
2018-10-11 18:12:39 +08:00
QueueName = amqqueue:get_name(Queue),
2021-05-28 17:57:28 +08:00
rabbit_queue_index:move_to_per_vhost_stores(QueueName)
end,
Queues),
%% Legacy (global) msg_store may require recovery.
%% This upgrade step should only be started
%% if we are upgrading from a pre-3.7.0 version.
{QueuesWithTerms, RecoveryRefs, StartFunState} = read_old_recovery_terms(Queues),
OldStore = run_old_persistent_store(RecoveryRefs, StartFunState),
VHosts = rabbit_vhost:list_names(),
%% New store should not be recovered.
NewMsgStore = start_new_store(VHosts),
%% Recovery terms should be started for all vhosts for new store.
[ok = rabbit_recovery_terms:open_table(VHost) || VHost <- VHosts],
MigrationBatchSize = application:get_env(rabbit, queue_migration_batch_size,
?QUEUE_MIGRATION_BATCH_SIZE),
in_batches(MigrationBatchSize,
{rabbit_variable_queue, migrate_queue, [OldStore, NewMsgStore]},
QueuesWithTerms,
"message_store upgrades: Migrating batch ~p of ~p queues. Out of total ~p ",
"message_store upgrades: Batch ~p of ~p queues migrated ~n. ~p total left"),
log_upgrade("Message store migration finished"),
ok = rabbit_sup:stop_child(OldStore),
[ok= rabbit_recovery_terms:close_table(VHost) || VHost <- VHosts],
ok = stop_new_store(NewMsgStore).
2016-04-06 19:28:02 +08:00
in_batches(Size, MFA, List, MessageStart, MessageEnd) ->
in_batches(Size, 1, MFA, List, MessageStart, MessageEnd).
in_batches(_, _, _, [], _, _) -> ok;
in_batches(Size, BatchNum, MFA, List, MessageStart, MessageEnd) ->
Length = length(List),
{Batch, Tail} = case Size > Length of
true -> {List, []};
false -> lists:split(Size, List)
end,
ProcessedLength = (BatchNum - 1) * Size,
rabbit_log:info(MessageStart, [BatchNum, Size, ProcessedLength + Length]),
{M, F, A} = MFA,
Keys = [ rpc:async_call(node(), M, F, [El | A]) || El <- Batch ],
lists:foreach(fun(Key) ->
case rpc:yield(Key) of
{badrpc, Err} -> throw(Err);
_ -> ok
end
end,
Keys),
rabbit_log:info(MessageEnd, [BatchNum, Size, length(Tail)]),
in_batches(Size, BatchNum + 1, MFA, Tail, MessageStart, MessageEnd).
migrate_queue({QueueName = #resource{virtual_host = VHost, name = Name},
RecoveryTerm},
OldStore, NewStore) ->
log_upgrade_verbose(
"Migrating messages in queue ~s in vhost ~s to per-vhost message store",
2016-12-24 06:05:37 +08:00
[Name, VHost]),
2016-11-12 01:32:23 +08:00
OldStoreClient = get_global_store_client(OldStore),
NewStoreClient = get_per_vhost_store_client(QueueName, NewStore),
%% WARNING: During scan_queue_segments queue index state is being recovered
%% and terminated. This can cause side effects!
2021-05-28 17:57:28 +08:00
rabbit_queue_index:scan_queue_segments(
2016-12-24 06:05:37 +08:00
%% We migrate only persistent messages which are found in message store
%% and are not acked yet
fun (_SeqId, MsgId, _MsgProps, true, _IsDelivered, no_ack, OldC)
when is_binary(MsgId) ->
migrate_message(MsgId, OldC, NewStoreClient);
(_SeqId, _MsgId, _MsgProps,
_IsPersistent, _IsDelivered, _IsAcked, OldC) ->
OldC
2016-04-20 01:07:17 +08:00
end,
OldStoreClient,
QueueName),
rabbit_msg_store:client_terminate(OldStoreClient),
rabbit_msg_store:client_terminate(NewStoreClient),
NewClientRef = rabbit_msg_store:client_ref(NewStoreClient),
case RecoveryTerm of
non_clean_shutdown -> ok;
Term when is_list(Term) ->
NewRecoveryTerm = lists:keyreplace(persistent_ref, 1, RecoveryTerm,
{persistent_ref, NewClientRef}),
2021-05-28 17:57:28 +08:00
rabbit_queue_index:update_recovery_term(QueueName, NewRecoveryTerm)
end,
2016-12-24 06:05:37 +08:00
log_upgrade_verbose("Finished migrating queue ~s in vhost ~s", [Name, VHost]),
{QueueName, NewClientRef}.
migrate_message(MsgId, OldC, NewC) ->
case rabbit_msg_store:read(MsgId, OldC) of
{{ok, Msg}, OldC1} ->
ok = rabbit_msg_store:write(MsgId, Msg, NewC),
OldC1;
_ -> OldC
end.
2016-04-20 01:07:17 +08:00
get_per_vhost_store_client(#resource{virtual_host = VHost}, NewStore) ->
{VHost, StorePid} = lists:keyfind(VHost, 1, NewStore),
rabbit_msg_store:client_init(StorePid, rabbit_guid:gen(),
fun(_,_) -> ok end, fun() -> ok end).
2016-11-12 01:32:23 +08:00
get_global_store_client(OldStore) ->
rabbit_msg_store:client_init(OldStore,
rabbit_guid:gen(),
fun(_,_) -> ok end,
fun() -> ok end).
2016-04-06 19:28:02 +08:00
list_persistent_queues() ->
Node = node(),
mnesia:async_dirty(
fun () ->
Make #amqqueue{} a private record + add a feature flag The #amqqueue{} record is now isolated in the `amqqueue` module: all accesses and modifications to the record are made through this helper. The type `rabbit_types:amqqueue()` is now `amqqueue:amqqueue()`. `amqqueue` knows about the new #amqqueue{} record introduced with quorum queues. If the actual record is the old one (as defined in RabbitMQ 3.7.x), it calls `amqqueue_v1` which knows about the old definition. When it needs to produce an #amqqueue{} record, it verifies if it is allowed to, using the new Feature flags subsystem and the `quorum_queue` feature flag specifically, and proceeds. If an old format #amqqueue{} is required, the creation is deferred to `amqqueue_v1`. The new modules come with a couple headers: they provide macros to replace pattern matching with `when` conditions. The `amqqueue_v*.hrl` headers are generated using the `macros()` functions in `amqqueue` and `amqqueue_v1` modules. Some operations now depend on the state of the `quorum_queue` feature flag. In particular, creating a quorum queue is denied if the feature flag is disabled. In the process, the following modules were moved from rabbitmq-common to rabbitmq-server: * rabbit_backing_queue * rabbit_queue_master_locator Likewise, the few functions were copied from `rabbit_misc` to `rabbit_amqqueue` because they depend on `amqqueue`. They don't really need to live in rabbitmq-common and they prevent further progress with the feature flags: feature flags need informations about the cluster. The `quorum_queue` feature flag comes with a migration function which is responsible of converting the #amqqueue{} records in Mnesia tables `rabbit_queue` and `rabbit_durable_queue`. Therefore, the upgrade function which did this is removed in this commit. [#159298729]
2018-10-11 18:12:39 +08:00
qlc:e(qlc:q([Q || Q <- mnesia:table(rabbit_durable_queue),
?amqqueue_is_classic(Q),
amqqueue:qnode(Q) == Node,
mnesia:read(rabbit_queue, amqqueue:get_name(Q), read) =:= []]))
2016-04-06 19:28:02 +08:00
end).
read_old_recovery_terms([]) ->
{[], [], ?EMPTY_START_FUN_STATE};
read_old_recovery_terms(Queues) ->
Make #amqqueue{} a private record + add a feature flag The #amqqueue{} record is now isolated in the `amqqueue` module: all accesses and modifications to the record are made through this helper. The type `rabbit_types:amqqueue()` is now `amqqueue:amqqueue()`. `amqqueue` knows about the new #amqqueue{} record introduced with quorum queues. If the actual record is the old one (as defined in RabbitMQ 3.7.x), it calls `amqqueue_v1` which knows about the old definition. When it needs to produce an #amqqueue{} record, it verifies if it is allowed to, using the new Feature flags subsystem and the `quorum_queue` feature flag specifically, and proceeds. If an old format #amqqueue{} is required, the creation is deferred to `amqqueue_v1`. The new modules come with a couple headers: they provide macros to replace pattern matching with `when` conditions. The `amqqueue_v*.hrl` headers are generated using the `macros()` functions in `amqqueue` and `amqqueue_v1` modules. Some operations now depend on the state of the `quorum_queue` feature flag. In particular, creating a quorum queue is denied if the feature flag is disabled. In the process, the following modules were moved from rabbitmq-common to rabbitmq-server: * rabbit_backing_queue * rabbit_queue_master_locator Likewise, the few functions were copied from `rabbit_misc` to `rabbit_amqqueue` because they depend on `amqqueue`. They don't really need to live in rabbitmq-common and they prevent further progress with the feature flags: feature flags need informations about the cluster. The `quorum_queue` feature flag comes with a migration function which is responsible of converting the #amqqueue{} records in Mnesia tables `rabbit_queue` and `rabbit_durable_queue`. Therefore, the upgrade function which did this is removed in this commit. [#159298729]
2018-10-11 18:12:39 +08:00
QueueNames = [amqqueue:get_name(Q) || Q <- Queues],
2021-05-28 17:57:28 +08:00
{AllTerms, StartFunState} = rabbit_queue_index:read_global_recovery_terms(QueueNames),
Refs = [Ref || Terms <- AllTerms,
Terms /= non_clean_shutdown,
begin
Ref = proplists:get_value(persistent_ref, Terms),
Ref =/= undefined
end],
{lists:zip(QueueNames, AllTerms), Refs, StartFunState}.
run_old_persistent_store(Refs, StartFunState) ->
OldStoreName = ?PERSISTENT_MSG_STORE,
ok = rabbit_sup:start_child(OldStoreName, rabbit_msg_store, start_global_store_link,
[OldStoreName, rabbit_mnesia:dir(),
Refs, StartFunState]),
OldStoreName.
start_new_store(VHosts) ->
%% Ensure vhost supervisor is started, so we can add vhosts to it.
lists:map(fun(VHost) ->
VHostDir = rabbit_vhost:msg_store_dir_path(VHost),
{ok, Pid} = rabbit_msg_store:start_link(?PERSISTENT_MSG_STORE,
VHostDir,
undefined,
?EMPTY_START_FUN_STATE),
{VHost, Pid}
end,
VHosts).
stop_new_store(NewStore) ->
lists:foreach(fun({_VHost, StorePid}) ->
unlink(StorePid),
exit(StorePid, shutdown)
end,
NewStore),
ok.
delete_old_store() ->
log_upgrade("Removing the old message store data"),
rabbit_file:recursive_delete(
[filename:join([rabbit_mnesia:dir(), ?PERSISTENT_MSG_STORE])]),
%% Delete old transient store as well
rabbit_file:recursive_delete(
[filename:join([rabbit_mnesia:dir(), ?TRANSIENT_MSG_STORE])]),
ok.
2016-04-20 01:07:17 +08:00
log_upgrade(Msg) ->
log_upgrade(Msg, []).
log_upgrade(Msg, Args) ->
rabbit_log:info("message_store upgrades: " ++ Msg, Args).
log_upgrade_verbose(Msg) ->
log_upgrade_verbose(Msg, []).
log_upgrade_verbose(Msg, Args) ->
2016-12-24 06:05:37 +08:00
rabbit_log_upgrade:info(Msg, Args).
maybe_client_terminate(MSCStateP) ->
%% Queue might have been asked to stop by the supervisor, it needs a clean
%% shutdown in order for the supervising strategy to work - if it reaches max
%% restarts might bring the vhost down.
try
rabbit_msg_store:client_terminate(MSCStateP)
catch
_:_ ->
ok
end.