Quorum queues (#1706)

* Test queue.declare method with quorum type

[#154472130]

* Cosmetics

[#154472130]

* Start quorum queue

Includes ra as a rabbit dependency

[#154472152]

* Update info and list operations to use quorum queues

Basic implementation. Might need an update when more functionality
is added to the quorum queues.

[#154472152]

* Stop quorum queue

[#154472158]

* Restart quorum queue

[#154472164]

* Introduce UId in ra config to support newer version of ra

Improved ra stop

[#154472158]

* Put data inside VHost specific subdirs

[#154472164]

* Include ra in rabbit deps to support stop_app/start_app command

[#154472164]

* Stop quorum queues in `rabbit_amqqueue:stop/1`

[#154472158]

* Revert creation of fifo ets table inside rabbit

Now supported by ra

[#154472158]

* Filter quorum queues

[#154472158]

* Test restart node with quorum queues

[#154472164]

* Publish to quorum queues

[#154472174]

* Use `ra:restart_node/1`

[#154472164]

* Wait for stats to be published when querying quorum queues

[#154472174]

* Test publish and queue length after restart

[#154472174]

* Consume messages from quorum queues with basic.get

[#154472211]

* Autoack messages from quorum queues on basic.get

[#154472211]

* Fix no_ack meaning

no_ack = true is equivalent to autoack

[#154472211]

* Use data_dir as provided in the config

If we modify the data_dir, ra is not able to delete the data
when a queue is deleted

[#154472158]

* Remove unused code/variables

[#154472158]

* Subscribe to a quorum queue

Supports auto-ack

[#154472215]

* Ack messages consumed from quorum queues

[#154472221]

* Nack messages consumed from quorum queues

[#154804608]

* Use delivery tag as consumer tag for basic.get in quorum queues

[#154472221]

* Support for publisher confirms in quorum queues

[#154472198]

* Integrate with ra_fifo_client

* Clear queue state on queue.delete

[#154472158]

* Fix quorum nack

[#154804608]

* Test redelivery after nack

[#154804608]

* Nack without requeueing

[#154472225]

* Test multiple acks

[#154804208]

* Test multiple nacks

[#154804314]

* Configure dead letter exchange with queue declare

[#155076661]

* Use a per-vhost process to handle dead-lettering

Needs to hold state for quorum queues

[#155401802]

* Implement dead-lettering on nack'ed messages

[#154804620]

* Use queue name as a resource on message delivery

Fixes a previously introduced bug

[#154804608]

* Handle ra events on dead letter process

[#155401802]

* Pass empty queue states to queue delete

Queue deletion on vhost deletion calls directly to rabbit_amqqueue.
Queue states are not available, but we can provide an empty map as
in deletion the states are only needed for cleanup.

* Generate quorum queue stats and events

Consumer delete events are still pending, as depend on basic.cancel
(not implemented yet), ra terminating or ra detecting channel down

[#154472241]

* Ensure quorum mapping entries are available before metric emission

[#154472241]

* Configure data_dir, uses new RABBITMQ_QUORUM_BASE env var

[#154472152]

* Use untracked enqueues when sending wihtout channel

Updated several other calls missed during the quorum implementation

* Revert "Configure data_dir, uses new RABBITMQ_QUORUM_BASE env var"

This reverts commit f226121241.

* Configure data_dir, uses new RABBITMQ_QUORUM_DIR based on mnesia dir

[#154472152]

* Fix get_quorum_state

* Fix calculation of quorum pids

* Move all quorum queues code to its own module

[#154472241]

* Return an error when declaring a quorum queue with an incompatible argument

[#154521696]

* Cleanup of quorum queue state after queue delete

Also fixes some existing problems where the state wasn't properly
stored

[#155458625]

* Revert Revert "Declare a quorum queue using the queue.declare method"

* Remove duplicated state info

[#154472241]

* Start/stop multi-node quorum queue

[#154472231]
[#154472236]

* Restart nodes in a multi-node quorum cluster

[#154472238]

* Test restart and leadership takeover on multiple nodes

[#154472238]

* Wait for leader down after deleting a quorum cluster

It ensures an smooth delete-declare sequence without race
conditions. The test included here detected the situation before
the fix.

[#154472236]

* Populate quorum_mapping from mnesia when not available

Ensures that leader nodes that don't have direct requests can get
the mapping ra name -> queue name

* Cosmetics

* Do not emit core metrics if queue has just been deleted

* Use rabbit_mnesia:is_process_alive

Fixes bug introduced by cac9583e1b
[#154472231]

* Only try to report stats if quorum process is alive

* Implement cancel consumer callback

Deletes metrics and sends consumer deleted event

* Remove unnecessary trigger election call

ra:restart_node has already been called during the recovery

* Apply cancellation callback on node hosting the channel

* Cosmetics

* Read new fifo metrics which store directly total, ready and unack

* Implement basic.cancel for quorum queues

* Store leader in amqqueue record, report all in stats

[#154472407]

* Declare quorum queue in mnesia before starting the ra cluster

Record needs to be stored first to update the leader on ra effects

* Revert

* Purge quorum queues

[#154472182]

* Improve use of untracked_enqueue

Choose the persisted leader id instead of just using the id of the
leader at point of creation.

* Store quorum leader in the pid field of amqqueue record

Same as mirrored queues, no real need for an additional field

* Improve recovery

When a ra node has never been started on a rabbit node ensure it doesn't
fail but instead rebuilds the config and starts the node as a new node.

Also fix issue when a quorum queue is declared when one of it's rabbit
nodes are unavailable.

[#157054606]

* Cleanup core metrics after leader change

[#157054473]

* Return an error on sync_queue on quorum queues

[#154472334]

* Return an error on cancel_sync_queue on quorum queues

[#154472337]

* Fix basic_cancel and basic_consume return values

Ensure the quorum queue state is always returned by these functions.

* Restore arity of amqqeueu delete and purge functions.

This avoids some breaking changes in the cli.

* Fix bug returning consumers.

* remove rogue debug log

* Integrate ingress flow control with quorum queues

[#157000583]

* Configure commands soft limit

[#157000583]

* Support quorum pids on rabbit_mnesia:is_process_alive

* Publish consumers metric for quorum queues

* Whitelist quorum directory in is_virgin_node

Allow the quorum directoy to exist without affecting the status of the
Rabbit node.

* Delete queue_metrics on leader change.

Also run the become_leader handler in a separate process to avoid
blocking.

[#157424225]

* Report cluster status in quorum queue infos. New per node status command.

Related to
[#157146500]

* Remove quorum_mapping table

As we can store the full queue name resource as the cluster id of the
ra_fifo_client state we can avoid needed the quorum_mapping table.

* Fix xref issue

* Provide quorum members information in stats

[#157146500]

* fix unused variable

* quorum queue multiple declare handling

Extend rabbit_amqqueue:internal_declare/2 to indicate if the queue
record was created or exisiting. From this we can then provide a code
path that should handle concurrent queue declares of the same quorum
queue.

* Return an error when declaring exclusive/auto-delete quorum queue

[#157472160]

* Restore lost changes

from 79c9bd201e

* recover another part of commit

* fixup cherry pick

* Ra io/file metrics handler and stats publishing

[#157193081]

* Revert "Ra io/file metrics handler and stats publishing"

This reverts commit 05d15c7865.

* Do not issue confirms on node down for quorum queues.

Only a ra_event should be used to issue positive confirms for a quorum
queue.

* Ra stats publishing

[#157193081]

* Pick consumer utilisation from ra data

[#155402726]

* Handle error when deleting a quorum queue and all nodes are already down

This is in fact a successful deletion as all raft nodes are already 'stopped'

[#158656366]

* Return an error when declaring non-durable quorum queues

[#158656454]

* Rename dirty_query to committed_query

* Delete stats on leader node

[#158661152]

* Give full list of nodes to fifo client

* Handle timeout in quorum basic_get

* Fix unused variable error

* Handle timeout in basic get

[#158656366]

* Force GC after purge

[#158789389]

* Increase `ra:delete_cluster` timeout to 120s

* Revert "Force GC after purge"

This reverts commit 5c98bf2299.

* Add quorum member command

[#157481599]

* Delete quorum member command

[#157481599]

* Implement basic.recover for quorum queues

[#157597411]

* Change concumer utilisation

to use the new ra_fifo table and api.

* Set max quorum queue size limit

Defaults to 7, can be configured per queue on queue.declare
Nodes are selected randomly from the list of nodes, but the one
that is executing the queue.declare command

[#159338081]

* remove potentially unrelated changes to rabbit_networking

* Move ra_fifo to rabbit

Copied ra_fifo to rabbit and renamed it rabbit_fifo.

[#159338031]

* rabbit_fifo tidy up

* rabbit_fifo tidy up

* rabbit_fifo: customer -> consumer rename

* Move ra_fifo tests

[#159338031]

* Tweak quorum_queue defaults

* quorum_queue test reliability

* Optimise quorum_queue test suite.

By only starting a rabbit cluster per group rather than test.

[#160612638]

* Renamings in line with ra API changes

* rabbit_fifo fixes

* Update with ra API changes

Ra has consolidated and simplified it's api. These changes update to
confirm to that.

* Update rabbit_fifo with latest ra changes

* Clean up out of date comment

* Return map of states

* Add test case for basic.get on an empty queue

Before the previous patch, any subsequent basic.get would crash as
the map of states had been replaced by a single state.

* Clarify use of deliver tags on record_sent

* Clean up queues after testcase

* Remove erlang monitor of quorum queues in rabbit_channel

The eol event can be used instead

* Use macros to make clearer distinctions between quorum/classic queues

Cosmetic only

* Erase queue stats on 'eol' event

* Update to follow Ra's cluster_id -> cluster_name rename.

* Rename qourum-cluster-size

To quorum-initial-group-size

* Issue confirms on quorum queue eol

Also avoid creating quorum queue session state on queue operation
methods.

* Only classic queues should be notified on channel down

* Quorum queues do not support global qos

Exit with protocol error of a basic.consume for a quorum queue is issued
on a channel with global qos enabled.

* unused variable name

* Refactoring

Strictly enfornce that channels do not monitor quorum queues.

* Refactor foreach_per_queue in the channel.

To make it call classic and quorum queues the same way.

[#161314899]

* rename function

* Query classic and quorum queues separately

during recovery as they should not be marked as stopped during failed
vhost recovery.

* Remove force_event_refresh function

As the only user of this function, the management API no longer requires
it.

* fix errors

* Remove created_at from amqqueue record

[#161343680]

* rabbit_fifo: support AMQP 1.0 consumer credit

This change implements an alternative consumer credit mechanism similar
to AMQP 1.0 link credit where the credit (prefetch) isn't automatically
topped up as deliveries are settled and instead needs to be manually
increased using a credit command. This is to be integrated with the AMQP
1.0 plugin.

[#161256187]

* Add basic.credit support for quorum queues.

Added support for AMQP 1.0 transfer flow control.

[#161256187]

* Make quorum queue recover idempotent

So that if a vhost crashes and runs the recover steps it doesn't fail
because ra servers are still running.

[#161343651]

* Add tests for vhost deletion

To ensure quorum queues are cleaned up on vhost removal.

Also fix xref issue.

[#161343673]

* remove unused clause

* always return latest value of queue

* Add rabbitmq-queues scripts. Remove ra config from .bat scripts.

* Return error if trying to get quorum status of a classic queue.
This commit is contained in:
Karl Nilsson 2018-10-29 09:47:29 +00:00 committed by GitHub
parent 8ed6b4477f
commit 3f3702514e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
35 changed files with 6619 additions and 391 deletions

View File

@ -96,6 +96,8 @@ define PROJECT_ENV
%% see rabbitmq-server#143,
%% rabbitmq-server#949, rabbitmq-server#1098
{credit_flow_default_credit, {400, 200}},
{quorum_commands_soft_limit, 256},
{quorum_cluster_size, 5},
%% see rabbitmq-server#248
%% and rabbitmq-server#667
{channel_operation_timeout, 15000},
@ -127,13 +129,14 @@ define PROJECT_ENV
%% vhost had to shut down, see server#1158 and server#1280
{vhost_restart_strategy, continue},
%% {global, prefetch count}
{default_consumer_prefetch, {false, 0}}
{default_consumer_prefetch, {false, 0}},
{channel_queue_cleanup_interval, 60000}
]
endef
LOCAL_DEPS = sasl mnesia os_mon inets
BUILD_DEPS = rabbitmq_cli syslog
DEPS = ranch lager rabbit_common
DEPS = ranch syslog lager rabbit_common ra
TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers amqp_client meck proper
dep_syslog = git https://github.com/schlagert/syslog 3.4.5

View File

@ -245,6 +245,7 @@ DEFAULT_NODE_PORT=5672
[ "x" = "x$RABBITMQ_SERVER_CODE_PATH" ] && RABBITMQ_SERVER_CODE_PATH=${SERVER_CODE_PATH}
[ "x" = "x$RABBITMQ_MNESIA_DIR" ] && RABBITMQ_MNESIA_DIR=${MNESIA_DIR}
[ "x" = "x$RABBITMQ_MNESIA_DIR" ] && RABBITMQ_MNESIA_DIR=${RABBITMQ_MNESIA_BASE}/${RABBITMQ_NODENAME}
[ "x" = "x$RABBITMQ_QUORUM_DIR" ] && RABBITMQ_QUORUM_DIR=${RABBITMQ_MNESIA_DIR}/quorum
[ "x" = "x$RABBITMQ_GENERATED_CONFIG_DIR" ] && RABBITMQ_GENERATED_CONFIG_DIR=${GENERATED_CONFIG_DIR}
[ "x" = "x$RABBITMQ_ADVANCED_CONFIG_FILE" ] && RABBITMQ_ADVANCED_CONFIG_FILE=${ADVANCED_CONFIG_FILE}
[ "x" = "x$RABBITMQ_SCHEMA_DIR" ] && RABBITMQ_SCHEMA_DIR=${SCHEMA_DIR}
@ -255,7 +256,8 @@ rmq_normalize_path_var \
RABBITMQ_CONFIG_FILE \
RABBITMQ_LOG_BASE \
RABBITMQ_MNESIA_BASE \
RABBITMQ_MNESIA_DIR
RABBITMQ_MNESIA_DIR \
RABBITMQ_QUORUM_DIR
[ "x" = "x$RABBITMQ_PID_FILE" ] && RABBITMQ_PID_FILE="$PID_FILE"
@ -349,6 +351,10 @@ if [ "${RABBITMQ_DEV_ENV}" ]; then
"$RABBITMQ_MNESIA_DIR_source" != 'environment' ]; then
RABBITMQ_MNESIA_DIR="${mnesia_dir}"
fi
if [ "${mnesia_dir}" -a \
"$RABBITMQ_QUORUM_DIR_source" != 'environment' ]; then
RABBITMQ_QUORUM_DIR="${mnesia_dir}/quorum"
fi
fi
if path_contains_existing_directory "${RABBITMQ_PLUGINS_DIR}" ; then

32
scripts/rabbitmq-queues Executable file
View File

@ -0,0 +1,32 @@
#!/bin/sh
## The contents of this file are subject to the Mozilla Public License
## Version 1.1 (the "License"); you may not use this file except in
## compliance with the License. You may obtain a copy of the License
## at http://www.mozilla.org/MPL/
##
## Software distributed under the License is distributed on an "AS IS"
## basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
## the License for the specific language governing rights and
## limitations under the License.
##
## The Original Code is RabbitMQ.
##
## The Initial Developer of the Original Code is GoPivotal, Inc.
## Copyright (c) 2007-2017 Pivotal Software, Inc. All rights reserved.
##
# Exit immediately if a pipeline, which may consist of a single simple command,
# a list, or a compound command returns a non-zero status
set -e
# Each variable or function that is created or modified is given the export
# attribute and marked for export to the environment of subsequent commands.
set -a
# shellcheck source=/dev/null
#
# TODO: when shellcheck adds support for relative paths, change to
# shellcheck source=./rabbitmq-env
. "${0%/*}"/rabbitmq-env
run_escript rabbitmqctl_escript "${ESCRIPT_DIR:?must be defined}"/rabbitmq-queues "$@"

View File

@ -0,0 +1,66 @@
@echo off
REM The contents of this file are subject to the Mozilla Public License
REM Version 1.1 (the "License"); you may not use this file except in
REM compliance with the License. You may obtain a copy of the License
REM at http://www.mozilla.org/MPL/
REM
REM Software distributed under the License is distributed on an "AS IS"
REM basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
REM the License for the specific language governing rights and
REM limitations under the License.
REM
REM The Original Code is RabbitMQ.
REM
REM The Initial Developer of the Original Code is GoPivotal, Inc.
REM Copyright (c) 2007-2015 Pivotal Software, Inc. All rights reserved.
REM
REM Scopes the variables to the current batch file
setlocal
rem Preserve values that might contain exclamation marks before
rem enabling delayed expansion
set TDP0=%~dp0
set STAR=%*
setlocal enabledelayedexpansion
REM Get default settings with user overrides for (RABBITMQ_)<var_name>
REM Non-empty defaults should be set in rabbitmq-env
call "%TDP0%\rabbitmq-env.bat" %~n0
if not exist "!ERLANG_HOME!\bin\erl.exe" (
echo.
echo ******************************
echo ERLANG_HOME not set correctly.
echo ******************************
echo.
echo Please either set ERLANG_HOME to point to your Erlang installation or place the
echo RabbitMQ server distribution in the Erlang lib folder.
echo.
exit /B 1
)
REM Disable erl_crash.dump by default for control scripts.
if not defined ERL_CRASH_DUMP_SECONDS (
set ERL_CRASH_DUMP_SECONDS=0
)
"!ERLANG_HOME!\bin\erl.exe" +B ^
-boot !CLEAN_BOOT_FILE! ^
-noinput -noshell -hidden -smp enable ^
!RABBITMQ_CTL_ERL_ARGS! ^
-kernel inet_dist_listen_min !RABBITMQ_CTL_DIST_PORT_MIN! ^
-kernel inet_dist_listen_max !RABBITMQ_CTL_DIST_PORT_MAX! ^
-sasl errlog_type error ^
-mnesia dir \""!RABBITMQ_MNESIA_DIR:\=/!"\" ^
-nodename !RABBITMQ_NODENAME! ^
-run escript start ^
-escript main rabbitmqctl_escript ^
-extra "%RABBITMQ_HOME%\escript\rabbitmq-queues" !STAR!
if ERRORLEVEL 1 (
exit /B 1
)
endlocal
endlocal

View File

@ -311,6 +311,7 @@ start_rabbitmq_server() {
-os_mon start_disksup false \
-os_mon start_memsup false \
-mnesia dir "\"${RABBITMQ_MNESIA_DIR}\"" \
-ra data_dir "\"${RABBITMQ_QUORUM_DIR}\"" \
${RABBITMQ_SERVER_START_ARGS} \
${RABBITMQ_DIST_ARG} \
"$@"

View File

@ -256,6 +256,7 @@ if "!ENV_OK!"=="false" (
-os_mon start_disksup false ^
-os_mon start_memsup false ^
-mnesia dir \""!RABBITMQ_MNESIA_DIR:\=/!"\" ^
-ra data_dir \""!RABBITMQ_QUORUM_DIR:\=/!"\" ^
!RABBITMQ_SERVER_START_ARGS! ^
!RABBITMQ_DIST_ARG! ^
!STAR!

View File

@ -330,6 +330,7 @@ set ERLANG_SERVICE_ARGUMENTS= ^
-os_mon start_disksup false ^
-os_mon start_memsup false ^
-mnesia dir \""!RABBITMQ_MNESIA_DIR:\=/!"\" ^
-ra data_dir \""!RABBITMQ_QUORUM_DIR:\=/!"\" ^
!RABBITMQ_SERVER_START_ARGS! ^
!RABBITMQ_DIST_ARG! ^
!STARVAR!

View File

@ -25,7 +25,7 @@
-export([start/0, boot/0, stop/0,
stop_and_halt/0, await_startup/0, await_startup/1,
status/0, is_running/0, alarms/0,
is_running/1, environment/0, rotate_logs/0, force_event_refresh/1,
is_running/1, environment/0, rotate_logs/0,
start_fhc/0]).
-export([start/2, stop/1, prep_stop/1]).
@ -225,7 +225,7 @@
-include("rabbit_framing.hrl").
-include("rabbit.hrl").
-define(APPS, [os_mon, mnesia, rabbit_common, rabbit]).
-define(APPS, [os_mon, mnesia, rabbit_common, ra, rabbit]).
-define(ASYNC_THREADS_WARNING_THRESHOLD, 8).
@ -252,7 +252,6 @@
-spec is_running(node()) -> boolean().
-spec environment() -> [{param(), term()}].
-spec rotate_logs() -> rabbit_types:ok_or_error(any()).
-spec force_event_refresh(reference()) -> 'ok'.
-spec log_locations() -> [log_location()].
@ -941,12 +940,6 @@ start_logger() ->
log_locations() ->
rabbit_lager:log_locations().
force_event_refresh(Ref) ->
rabbit_direct:force_event_refresh(Ref),
rabbit_networking:force_connection_event_refresh(Ref),
rabbit_channel:force_event_refresh(Ref),
rabbit_amqqueue:force_event_refresh(Ref).
%%---------------------------------------------------------------------------
%% misc

View File

@ -25,21 +25,24 @@
-export([lookup/1, not_found_or_absent/1, with/2, with/3, with_or_die/2,
assert_equivalence/5,
check_exclusive_access/2, with_exclusive_access_or_die/3,
stat/1, deliver/2, requeue/3, ack/3, reject/4]).
stat/1, deliver/2, deliver/3, requeue/4, ack/4, reject/5]).
-export([list/0, list/1, info_keys/0, info/1, info/2, info_all/1, info_all/2,
emit_info_all/5, list_local/1, info_local/1,
emit_info_local/4, emit_info_down/4]).
-export([list_down/1, count/1, list_names/0, list_local_names/0]).
-export([force_event_refresh/1, notify_policy_changed/1]).
-export([notify_policy_changed/1]).
-export([consumers/1, consumers_all/1, emit_consumers_all/4, consumer_info_keys/0]).
-export([basic_get/4, basic_consume/11, basic_cancel/5, notify_decorators/1]).
-export([basic_get/6, basic_consume/12, basic_cancel/6, notify_decorators/1]).
-export([notify_sent/2, notify_sent_queue_down/1, resume/2]).
-export([notify_down_all/2, notify_down_all/3, activate_limit_all/2, credit/5]).
-export([notify_down_all/2, notify_down_all/3, activate_limit_all/2, credit/6]).
-export([on_node_up/1, on_node_down/1]).
-export([update/2, store_queue/1, update_decorators/1, policy_changed/2]).
-export([update_mirroring/1, sync_mirrors/1, cancel_sync_mirrors/1]).
-export([emit_unresponsive/6, emit_unresponsive_local/5, is_unresponsive/2]).
-export([is_mirrored/1, is_dead_exclusive/1]). % Note: exported due to use in qlc expression.
-export([is_replicated/1, is_dead_exclusive/1]). % Note: exported due to use in qlc expression.
-export([list_local_followers/0]).
-export([ensure_rabbit_queue_record_is_initialized/1]).
-export([format/1]).
-export([pid_of/1, pid_of/2]).
-export([mark_local_durable_queues_stopped/1]).
@ -47,9 +50,9 @@
%% internal
-export([internal_declare/2, internal_delete/2, run_backing_queue/3,
set_ram_duration_target/2, set_maximum_since_use/2,
emit_consumers_local/3, internal_delete/3]).
emit_consumers_local/3, internal_delete/3]).
-include("rabbit.hrl").
-include_lib("rabbit_common/include/rabbit.hrl").
-include_lib("stdlib/include/qlc.hrl").
-define(INTEGER_ARG_TYPES, [byte, short, signedint, long,
@ -57,6 +60,8 @@
-define(MORE_CONSUMER_CREDIT_AFTER, 50).
-define(IS_CLASSIC(QPid), is_pid(QPid)).
-define(IS_QUORUM(QPid), is_tuple(QPid)).
%%----------------------------------------------------------------------------
-export_type([name/0, qmsg/0, absent_reason/0]).
@ -82,11 +87,13 @@
rabbit_types:maybe(pid()), rabbit_types:username()) ->
{'new' | 'existing' | 'absent' | 'owner_died',
rabbit_types:amqqueue()} |
{'new', rabbit_types:amqqueue(), rabbit_fifo_client:state()} |
rabbit_types:channel_exit().
-spec declare
(name(), boolean(), boolean(), rabbit_framing:amqp_table(),
rabbit_types:maybe(pid()), rabbit_types:username(), node()) ->
{'new' | 'existing' | 'owner_died', rabbit_types:amqqueue()} |
{'new', rabbit_types:amqqueue(), rabbit_fifo_client:state()} |
{'absent', rabbit_types:amqqueue(), absent_reason()} |
rabbit_types:channel_exit().
-spec internal_declare(rabbit_types:amqqueue(), boolean()) ->
@ -124,7 +131,6 @@
-spec info_all(rabbit_types:vhost()) -> [rabbit_types:infos()].
-spec info_all(rabbit_types:vhost(), rabbit_types:info_keys()) ->
[rabbit_types:infos()].
-spec force_event_refresh(reference()) -> 'ok'.
-spec notify_policy_changed(rabbit_types:amqqueue()) -> 'ok'.
-spec consumers(rabbit_types:amqqueue()) ->
[{pid(), rabbit_types:ctag(), boolean(), non_neg_integer(),
@ -150,31 +156,36 @@
rabbit_types:error('not_empty').
-spec delete_crashed(rabbit_types:amqqueue()) -> 'ok'.
-spec delete_crashed_internal(rabbit_types:amqqueue(), rabbit_types:username()) -> 'ok'.
-spec purge(rabbit_types:amqqueue()) -> qlen().
-spec purge(rabbit_types:amqqueue()) -> {ok, qlen()}.
-spec forget_all_durable(node()) -> 'ok'.
-spec deliver([rabbit_types:amqqueue()], rabbit_types:delivery()) ->
qpids().
-spec requeue(pid(), [msg_id()], pid()) -> 'ok'.
-spec ack(pid(), [msg_id()], pid()) -> 'ok'.
-spec reject(pid(), [msg_id()], boolean(), pid()) -> 'ok'.
-spec deliver([rabbit_types:amqqueue()], rabbit_types:delivery(), #{Name :: atom() => rabbit_fifo_client:state()} | 'untracked') ->
{qpids(), #{Name :: atom() => rabbit_fifo_client:state()}}.
-spec deliver([rabbit_types:amqqueue()], rabbit_types:delivery()) -> 'ok'.
-spec requeue(pid(), [msg_id()], pid(), #{Name :: atom() => rabbit_fifo_client:state()}) -> 'ok'.
-spec ack(pid(), [msg_id()], pid(), #{Name :: atom() => rabbit_fifo_client:state()}) -> 'ok'.
-spec reject(pid() | {atom(), node()}, [msg_id()], boolean(), pid(),
#{Name :: atom() => rabbit_fifo_client:state()}) -> 'ok'.
-spec notify_down_all(qpids(), pid()) -> ok_or_errors().
-spec notify_down_all(qpids(), pid(), non_neg_integer()) ->
ok_or_errors().
-spec activate_limit_all(qpids(), pid()) -> ok_or_errors().
-spec basic_get(rabbit_types:amqqueue(), pid(), boolean(), pid()) ->
-spec basic_get(rabbit_types:amqqueue(), pid(), boolean(), pid(), rabbit_types:ctag(),
#{Name :: atom() => rabbit_fifo_client:state()}) ->
{'ok', non_neg_integer(), qmsg()} | 'empty'.
-spec credit
(rabbit_types:amqqueue(), pid(), rabbit_types:ctag(), non_neg_integer(),
boolean()) ->
boolean(), #{Name :: atom() => rabbit_fifo_client:state()}) ->
'ok'.
-spec basic_consume
(rabbit_types:amqqueue(), boolean(), pid(), pid(), boolean(),
non_neg_integer(), rabbit_types:ctag(), boolean(),
rabbit_framing:amqp_table(), any(), rabbit_types:username()) ->
rabbit_framing:amqp_table(), any(), rabbit_types:username(),
#{Name :: atom() => rabbit_fifo_client:state()}) ->
rabbit_types:ok_or_error('exclusive_consume_unavailable').
-spec basic_cancel
(rabbit_types:amqqueue(), pid(), rabbit_types:ctag(), any(),
rabbit_types:username()) -> 'ok'.
rabbit_types:username(), #{Name :: atom() => rabbit_fifo_client:state()}) ->
'ok' | {'ok', #{Name :: atom() => rabbit_fifo_client:state()}}.
-spec notify_decorators(rabbit_types:amqqueue()) -> 'ok'.
-spec resume(pid(), pid()) -> 'ok'.
-spec internal_delete(name(), rabbit_types:username()) ->
@ -199,7 +210,7 @@
'ok' | rabbit_types:error('not_mirrored').
-spec cancel_sync_mirrors(rabbit_types:amqqueue() | pid()) ->
'ok' | {'ok', 'not_syncing'}.
-spec is_mirrored(rabbit_types:amqqueue()) -> boolean().
-spec is_replicated(rabbit_types:amqqueue()) -> boolean().
-spec pid_of(rabbit_types:amqqueue()) ->
{'ok', pid()} | rabbit_types:error('not_found').
@ -213,7 +224,7 @@
arguments]).
warn_file_limit() ->
DurableQueues = find_durable_queues(),
DurableQueues = find_recoverable_queues(),
L = length(DurableQueues),
%% if there are not enough file handles, the server might hang
@ -228,7 +239,11 @@ warn_file_limit() ->
end.
recover(VHost) ->
Queues = find_durable_queues(VHost),
Classic = find_local_durable_classic_queues(VHost),
Quorum = find_local_quorum_queues(VHost),
recover_classic_queues(VHost, Classic) ++ rabbit_quorum_queue:recover(Quorum).
recover_classic_queues(VHost, Queues) ->
{ok, BQ} = application:get_env(rabbit, backing_queue_module),
%% We rely on BQ:start/1 returning the recovery terms in the same
%% order as the supplied queue names, so that we can zip them together
@ -243,21 +258,30 @@ recover(VHost) ->
throw({error, Reason})
end.
filter_per_type(Queues) ->
lists:partition(fun(#amqqueue{type = Type}) -> Type == classic end, Queues).
filter_pid_per_type(QPids) ->
lists:partition(fun(QPid) -> ?IS_CLASSIC(QPid) end, QPids).
stop(VHost) ->
%% Classic queues
ok = rabbit_amqqueue_sup_sup:stop_for_vhost(VHost),
{ok, BQ} = application:get_env(rabbit, backing_queue_module),
ok = BQ:stop(VHost).
ok = BQ:stop(VHost),
rabbit_quorum_queue:stop(VHost).
start(Qs) ->
{Classic, _Quorum} = filter_per_type(Qs),
%% At this point all recovered queues and their bindings are
%% visible to routing, so now it is safe for them to complete
%% their initialisation (which may involve interacting with other
%% queues).
[Pid ! {self(), go} || #amqqueue{pid = Pid} <- Qs],
_ = [Pid ! {self(), go} || #amqqueue{pid = Pid} <- Classic],
ok.
mark_local_durable_queues_stopped(VHost) ->
Qs = find_durable_queues(VHost),
Qs = find_local_durable_classic_queues(VHost),
rabbit_misc:execute_mnesia_transaction(
fun() ->
[ store_queue(Q#amqqueue{ state = stopped })
@ -265,38 +289,57 @@ mark_local_durable_queues_stopped(VHost) ->
State =/= stopped ]
end).
find_durable_queues(VHost) ->
find_local_quorum_queues(VHost) ->
Node = node(),
mnesia:async_dirty(
fun () ->
qlc:e(qlc:q([Q || Q = #amqqueue{vhost = VH,
type = quorum,
quorum_nodes = QuorumNodes}
<- mnesia:table(rabbit_durable_queue),
VH =:= VHost,
(lists:member(Node, QuorumNodes))]))
end).
find_local_durable_classic_queues(VHost) ->
Node = node(),
mnesia:async_dirty(
fun () ->
qlc:e(qlc:q([Q || Q = #amqqueue{name = Name,
vhost = VH,
pid = Pid}
pid = Pid,
type = classic}
<- mnesia:table(rabbit_durable_queue),
VH =:= VHost,
node(Pid) == Node andalso
%% Terminations on node down will not remove the rabbit_queue
%% record if it is a mirrored queue (such info is now obtained from
%% the policy). Thus, we must check if the local pid is alive
%% - if the record is present - in order to restart.
(mnesia:read(rabbit_queue, Name, read) =:= []
orelse not erlang:is_process_alive(Pid))]))
(is_local_to_node(Pid, Node) andalso
%% Terminations on node down will not remove the rabbit_queue
%% record if it is a mirrored queue (such info is now obtained from
%% the policy). Thus, we must check if the local pid is alive
%% - if the record is present - in order to restart.
(mnesia:read(rabbit_queue, Name, read) =:= []
orelse not rabbit_mnesia:is_process_alive(Pid)))
]))
end).
find_durable_queues() ->
find_recoverable_queues() ->
Node = node(),
mnesia:async_dirty(
fun () ->
qlc:e(qlc:q([Q || Q = #amqqueue{name = Name,
pid = Pid}
pid = Pid,
type = Type,
quorum_nodes = QuorumNodes}
<- mnesia:table(rabbit_durable_queue),
node(Pid) == Node andalso
%% Terminations on node down will not remove the rabbit_queue
%% record if it is a mirrored queue (such info is now obtained from
%% the policy). Thus, we must check if the local pid is alive
%% - if the record is present - in order to restart.
(mnesia:read(rabbit_queue, Name, read) =:= []
orelse not erlang:is_process_alive(Pid))]))
(Type == classic andalso
(is_local_to_node(Pid, Node) andalso
%% Terminations on node down will not remove the rabbit_queue
%% record if it is a mirrored queue (such info is now obtained from
%% the policy). Thus, we must check if the local pid is alive
%% - if the record is present - in order to restart.
(mnesia:read(rabbit_queue, Name, read) =:= []
orelse not rabbit_mnesia:is_process_alive(Pid))))
orelse (Type == quorum andalso lists:member(Node, QuorumNodes))
]))
end).
recover_durable_queues(QueuesAndRecoveryTerms) ->
@ -318,6 +361,7 @@ declare(QueueName, Durable, AutoDelete, Args, Owner, ActingUser) ->
declare(QueueName = #resource{virtual_host = VHost}, Durable, AutoDelete, Args,
Owner, ActingUser, Node) ->
ok = check_declare_arguments(QueueName, Args),
Type = get_queue_type(Args),
Q = rabbit_queue_decorator:set(
rabbit_policy:set(#amqqueue{name = QueueName,
durable = Durable,
@ -333,8 +377,17 @@ declare(QueueName = #resource{virtual_host = VHost}, Durable, AutoDelete, Args,
policy_version = 0,
slave_pids_pending_shutdown = [],
vhost = VHost,
options = #{user => ActingUser}})),
options = #{user => ActingUser},
type = Type})),
case Type of
classic ->
declare_classic_queue(Q, Node);
quorum ->
rabbit_quorum_queue:declare(Q)
end.
declare_classic_queue(#amqqueue{name = QName, vhost = VHost} = Q, Node) ->
Node1 = case rabbit_queue_master_location_misc:get_location(Q) of
{ok, Node0} -> Node0;
{error, _} -> Node
@ -348,14 +401,22 @@ declare(QueueName = #resource{virtual_host = VHost}, Durable, AutoDelete, Args,
{error, Error} ->
rabbit_misc:protocol_error(internal_error,
"Cannot declare a queue '~s' on node '~s': ~255p",
[rabbit_misc:rs(QueueName), Node1, Error])
[rabbit_misc:rs(QName), Node1, Error])
end.
get_queue_type(Args) ->
case rabbit_misc:table_lookup(Args, <<"x-queue-type">>) of
undefined ->
classic;
{_, V} ->
erlang:binary_to_existing_atom(V, utf8)
end.
internal_declare(Q, true) ->
rabbit_misc:execute_mnesia_tx_with_tail(
fun () ->
ok = store_queue(Q#amqqueue{state = live}),
rabbit_misc:const(Q)
rabbit_misc:const({created, Q})
end);
internal_declare(Q = #amqqueue{name = QueueName}, false) ->
rabbit_misc:execute_mnesia_tx_with_tail(
@ -366,12 +427,12 @@ internal_declare(Q = #amqqueue{name = QueueName}, false) ->
not_found -> Q1 = rabbit_policy:set(Q),
Q2 = Q1#amqqueue{state = live},
ok = store_queue(Q2),
B = add_default_binding(Q1),
fun () -> B(), Q1 end;
B = add_default_binding(Q2),
fun () -> B(), {created, Q2} end;
{absent, _Q, _} = R -> rabbit_misc:const(R)
end;
[ExistingQ] ->
rabbit_misc:const(ExistingQ)
rabbit_misc:const({existing, ExistingQ})
end
end).
@ -389,6 +450,15 @@ update(Name, Fun) ->
not_found
end.
%% only really used for quorum queues to ensure the rabbit_queue record
%% is initialised
ensure_rabbit_queue_record_is_initialized(Q) ->
rabbit_misc:execute_mnesia_tx_with_tail(
fun () ->
ok = store_queue(Q),
rabbit_misc:const({ok, Q})
end).
store_queue(Q = #amqqueue{durable = true}) ->
ok = mnesia:write(rabbit_durable_queue,
Q#amqqueue{slave_pids = [],
@ -498,7 +568,7 @@ with(Name, F, E, RetriesLeft) ->
end.
retry_wait(Q = #amqqueue{pid = QPid, name = Name, state = QState}, F, E, RetriesLeft) ->
case {QState, is_mirrored(Q)} of
case {QState, is_replicated(Q)} of
%% We don't want to repeat an operation if
%% there are no slaves to migrate to
{stopped, false} ->
@ -575,7 +645,9 @@ declare_args() ->
{<<"x-max-length-bytes">>, fun check_non_neg_int_arg/2},
{<<"x-max-priority">>, fun check_max_priority_arg/2},
{<<"x-overflow">>, fun check_overflow/2},
{<<"x-queue-mode">>, fun check_queue_mode/2}].
{<<"x-queue-mode">>, fun check_queue_mode/2},
{<<"x-queue-type">>, fun check_queue_type/2},
{<<"x-quorum-initial-group-size">>, fun check_default_quorum_initial_group_size_arg/2}].
consume_args() -> [{<<"x-priority">>, fun check_int_arg/2},
{<<"x-cancel-on-ha-failover">>, fun check_bool_arg/2}].
@ -616,6 +688,13 @@ check_max_priority_arg({Type, Val}, Args) ->
Error -> Error
end.
check_default_quorum_initial_group_size_arg({Type, Val}, Args) ->
case check_non_neg_int_arg({Type, Val}, Args) of
ok when Val == 0 -> {error, {value_zero, Val}};
ok -> ok;
Error -> Error
end.
%% Note that the validity of x-dead-letter-exchange is already verified
%% by rabbit_channel's queue.declare handler.
check_dlxname_arg({longstr, _}, _) -> ok;
@ -645,14 +724,38 @@ check_queue_mode({longstr, Val}, _Args) ->
check_queue_mode({Type, _}, _Args) ->
{error, {unacceptable_type, Type}}.
check_queue_type({longstr, Val}, _Args) ->
case lists:member(Val, [<<"classic">>, <<"quorum">>]) of
true -> ok;
false -> {error, invalid_queue_type}
end;
check_queue_type({Type, _}, _Args) ->
{error, {unacceptable_type, Type}}.
list() -> mnesia:dirty_match_object(rabbit_queue, #amqqueue{_ = '_'}).
list_names() -> mnesia:dirty_all_keys(rabbit_queue).
list_local_names() ->
[ Q#amqqueue.name || #amqqueue{state = State, pid = QPid} = Q <- list(),
State =/= crashed,
node() =:= node(QPid) ].
State =/= crashed, is_local_to_node(QPid, node())].
list_local_followers() ->
[ Q#amqqueue.name
|| #amqqueue{state = State, type = quorum, pid = {_, Leader},
quorum_nodes = Nodes} = Q <- list(),
State =/= crashed, Leader =/= node(), lists:member(node(), Nodes)].
is_local_to_node(QPid, Node) when ?IS_CLASSIC(QPid) ->
Node =:= node(QPid);
is_local_to_node({_, Leader} = QPid, Node) when ?IS_QUORUM(QPid) ->
Node =:= Leader.
qnode(QPid) when ?IS_CLASSIC(QPid) ->
node(QPid);
qnode({_, Node} = QPid) when ?IS_QUORUM(QPid) ->
Node.
list(VHostPath) ->
list(VHostPath, rabbit_queue).
@ -710,10 +813,16 @@ is_unresponsive(#amqqueue{ pid = QPid }, Timeout) ->
true
end.
format(Q = #amqqueue{ type = quorum }) -> rabbit_quorum_queue:format(Q);
format(_) -> [].
info(Q = #amqqueue{ type = quorum }) -> rabbit_quorum_queue:info(Q);
info(Q = #amqqueue{ state = crashed }) -> info_down(Q, crashed);
info(Q = #amqqueue{ state = stopped }) -> info_down(Q, stopped);
info(#amqqueue{ pid = QPid }) -> delegate:invoke(QPid, {gen_server2, call, [info, infinity]}).
info(Q = #amqqueue{ type = quorum }, Items) ->
rabbit_quorum_queue:info(Q, Items);
info(Q = #amqqueue{ state = crashed }, Items) ->
info_down(Q, Items, crashed);
info(Q = #amqqueue{ state = stopped }, Items) ->
@ -782,14 +891,8 @@ info_local(VHostPath) ->
map(list_local(VHostPath), fun (Q) -> info(Q, [name]) end).
list_local(VHostPath) ->
[ Q || #amqqueue{state = State, pid=QPid} = Q <- list(VHostPath),
State =/= crashed,
node() =:= node(QPid) ].
force_event_refresh(Ref) ->
[gen_server2:cast(Q#amqqueue.pid,
{force_event_refresh, Ref}) || Q <- list()],
ok.
[ Q || #amqqueue{state = State, pid = QPid} = Q <- list(VHostPath),
State =/= crashed, is_local_to_node(QPid, node()) ].
notify_policy_changed(#amqqueue{pid = QPid}) ->
gen_server2:cast(QPid, policy_changed).
@ -823,6 +926,7 @@ get_queue_consumer_info(Q, ConsumerInfoKeys) ->
AckRequired, Prefetch, Args]) ||
{ChPid, CTag, AckRequired, Prefetch, Args, _} <- consumers(Q)].
stat(#amqqueue{type = quorum} = Q) -> rabbit_quorum_queue:stat(Q);
stat(#amqqueue{pid = QPid}) -> delegate:invoke(QPid, {gen_server2, call, [stat, infinity]}).
pid_of(#amqqueue{pid = Pid}) -> Pid.
@ -837,9 +941,14 @@ delete_exclusive(QPids, ConnId) ->
ok.
delete_immediately(QPids) ->
[gen_server2:cast(QPid, delete_immediately) || QPid <- QPids],
{Classic, Quorum} = filter_pid_per_type(QPids),
[gen_server2:cast(QPid, delete_immediately) || QPid <- Classic],
[rabbit_quorum_queue:delete_immediately(QPid) || QPid <- Quorum],
ok.
delete(#amqqueue{ type = quorum} = Q,
IfUnused, IfEmpty, ActingUser) ->
rabbit_quorum_queue:delete(Q, IfUnused, IfEmpty, ActingUser);
delete(Q, IfUnused, IfEmpty, ActingUser) ->
case wait_for_promoted_or_stopped(Q) of
{promoted, #amqqueue{pid = QPid}} ->
@ -894,24 +1003,62 @@ delete_crashed(Q) ->
delete_crashed(Q, ?INTERNAL_USER).
delete_crashed(#amqqueue{ pid = QPid } = Q, ActingUser) ->
ok = rpc:call(node(QPid), ?MODULE, delete_crashed_internal, [Q, ActingUser]).
ok = rpc:call(qnode(QPid), ?MODULE, delete_crashed_internal, [Q, ActingUser]).
delete_crashed_internal(Q = #amqqueue{ name = QName }, ActingUser) ->
{ok, BQ} = application:get_env(rabbit, backing_queue_module),
BQ:delete_crashed(Q),
ok = internal_delete(QName, ActingUser).
purge(#amqqueue{ pid = QPid }) ->
delegate:invoke(QPid, {gen_server2, call, [purge, infinity]}).
purge(#amqqueue{ pid = QPid, type = classic}) ->
delegate:invoke(QPid, {gen_server2, call, [purge, infinity]});
purge(#amqqueue{ pid = NodeId, type = quorum}) ->
rabbit_quorum_queue:purge(NodeId).
requeue(QPid, MsgIds, ChPid) ->
delegate:invoke(QPid, {gen_server2, call, [{requeue, MsgIds, ChPid}, infinity]}).
ack(QPid, MsgIds, ChPid) ->
delegate:invoke_no_result(QPid, {gen_server2, cast, [{ack, MsgIds, ChPid}]}).
requeue(QPid, {_, MsgIds}, ChPid, QuorumStates) when ?IS_CLASSIC(QPid) ->
ok = delegate:invoke(QPid, {gen_server2, call, [{requeue, MsgIds, ChPid}, infinity]}),
QuorumStates;
requeue({Name, _} = QPid, {CTag, MsgIds}, _ChPid, QuorumStates)
when ?IS_QUORUM(QPid) ->
case QuorumStates of
#{Name := QState0} ->
{ok, QState} = rabbit_quorum_queue:requeue(CTag, MsgIds, QState0),
maps:put(Name, QState, QuorumStates);
_ ->
% queue was not found
QuorumStates
end.
reject(QPid, Requeue, MsgIds, ChPid) ->
delegate:invoke_no_result(QPid, {gen_server2, cast, [{reject, Requeue, MsgIds, ChPid}]}).
ack(QPid, {_, MsgIds}, ChPid, QueueStates) when ?IS_CLASSIC(QPid) ->
delegate:invoke_no_result(QPid, {gen_server2, cast, [{ack, MsgIds, ChPid}]}),
QueueStates;
ack({Name, _} = QPid, {CTag, MsgIds}, _ChPid, QuorumStates)
when ?IS_QUORUM(QPid) ->
case QuorumStates of
#{Name := QState0} ->
{ok, QState} = rabbit_quorum_queue:ack(CTag, MsgIds, QState0),
maps:put(Name, QState, QuorumStates);
_ ->
%% queue was not found
QuorumStates
end.
reject(QPid, Requeue, {_, MsgIds}, ChPid, QStates) when ?IS_CLASSIC(QPid) ->
ok = delegate:invoke_no_result(QPid, {gen_server2, cast,
[{reject, Requeue, MsgIds, ChPid}]}),
QStates;
reject({Name, _} = QPid, Requeue, {CTag, MsgIds}, _ChPid, QuorumStates)
when ?IS_QUORUM(QPid) ->
case QuorumStates of
#{Name := QState0} ->
{ok, QState} = rabbit_quorum_queue:reject(Requeue, CTag,
MsgIds, QState0),
maps:put(Name, QState, QuorumStates);
_ ->
%% queue was not found
QuorumStates
end.
notify_down_all(QPids, ChPid) ->
notify_down_all(QPids, ChPid, ?CHANNEL_OPERATION_TIMEOUT).
@ -934,26 +1081,83 @@ notify_down_all(QPids, ChPid, Timeout) ->
end.
activate_limit_all(QPids, ChPid) ->
delegate:invoke_no_result(QPids, {gen_server2, cast, [{activate_limit, ChPid}]}).
delegate:invoke_no_result(QPids, {gen_server2, cast,
[{activate_limit, ChPid}]}).
credit(#amqqueue{pid = QPid}, ChPid, CTag, Credit, Drain) ->
delegate:invoke_no_result(QPid, {gen_server2, cast, [{credit, ChPid, CTag, Credit, Drain}]}).
credit(#amqqueue{pid = QPid, type = classic}, ChPid, CTag, Credit,
Drain, QStates) ->
delegate:invoke_no_result(QPid, {gen_server2, cast,
[{credit, ChPid, CTag, Credit, Drain}]}),
{ok, QStates};
credit(#amqqueue{pid = {Name, _} = Id, name = QName, type = quorum},
_ChPid, CTag, Credit,
Drain, QStates) ->
QState0 = get_quorum_state(Id, QName, QStates),
{ok, QState} = rabbit_quorum_queue:credit(CTag, Credit, Drain, QState0),
{ok, maps:put(Name, QState, QStates)}.
basic_get(#amqqueue{pid = QPid}, ChPid, NoAck, LimiterPid) ->
delegate:invoke(QPid, {gen_server2, call, [{basic_get, ChPid, NoAck, LimiterPid}, infinity]}).
basic_consume(#amqqueue{pid = QPid, name = QName}, NoAck, ChPid, LimiterPid,
basic_get(#amqqueue{pid = QPid, type = classic}, ChPid, NoAck, LimiterPid,
_CTag, _) ->
delegate:invoke(QPid, {gen_server2, call,
[{basic_get, ChPid, NoAck, LimiterPid}, infinity]});
basic_get(#amqqueue{pid = {Name, _} = Id, type = quorum, name = QName} = Q, _ChPid, NoAck,
_LimiterPid, CTag, QStates) ->
QState0 = get_quorum_state(Id, QName, QStates),
case rabbit_quorum_queue:basic_get(Q, NoAck, CTag, QState0) of
{ok, empty, QState} ->
{empty, maps:put(Name, QState, QStates)};
{ok, Count, Msg, QState} ->
{ok, Count, Msg, maps:put(Name, QState, QStates)};
{error, Reason} ->
rabbit_misc:protocol_error(internal_error,
"Cannot get a message from quorum queue '~s': ~p",
[rabbit_misc:rs(QName), Reason])
end.
basic_consume(#amqqueue{pid = QPid, name = QName, type = classic}, NoAck, ChPid, LimiterPid,
LimiterActive, ConsumerPrefetchCount, ConsumerTag,
ExclusiveConsume, Args, OkMsg, ActingUser) ->
ExclusiveConsume, Args, OkMsg, ActingUser, QState) ->
ok = check_consume_arguments(QName, Args),
delegate:invoke(QPid, {gen_server2, call,
[{basic_consume, NoAck, ChPid, LimiterPid, LimiterActive,
ConsumerPrefetchCount, ConsumerTag, ExclusiveConsume,
Args, OkMsg, ActingUser}, infinity]}).
case delegate:invoke(QPid, {gen_server2, call,
[{basic_consume, NoAck, ChPid, LimiterPid, LimiterActive,
ConsumerPrefetchCount, ConsumerTag, ExclusiveConsume,
Args, OkMsg, ActingUser}, infinity]}) of
ok ->
{ok, QState};
Err ->
Err
end;
basic_consume(#amqqueue{type = quorum}, _NoAck, _ChPid,
_LimiterPid, true, _ConsumerPrefetchCount, _ConsumerTag,
_ExclusiveConsume, _Args, _OkMsg, _ActingUser, _QStates) ->
{error, global_qos_not_supported_for_queue_type};
basic_consume(#amqqueue{pid = {Name, _} = Id, name = QName, type = quorum} = Q, NoAck, ChPid,
_LimiterPid, _LimiterActive, ConsumerPrefetchCount, ConsumerTag,
ExclusiveConsume, Args, OkMsg, _ActingUser, QStates) ->
ok = check_consume_arguments(QName, Args),
QState0 = get_quorum_state(Id, QName, QStates),
{ok, QState} = rabbit_quorum_queue:basic_consume(Q, NoAck, ChPid,
ConsumerPrefetchCount,
ConsumerTag,
ExclusiveConsume, Args,
OkMsg, QState0),
{ok, maps:put(Name, QState, QStates)}.
basic_cancel(#amqqueue{pid = QPid}, ChPid, ConsumerTag, OkMsg, ActingUser) ->
delegate:invoke(QPid, {gen_server2, call,
[{basic_cancel, ChPid, ConsumerTag, OkMsg, ActingUser}, infinity]}).
basic_cancel(#amqqueue{pid = QPid, type = classic}, ChPid, ConsumerTag, OkMsg, ActingUser,
QState) ->
case delegate:invoke(QPid, {gen_server2, call,
[{basic_cancel, ChPid, ConsumerTag, OkMsg, ActingUser},
infinity]}) of
ok ->
{ok, QState};
Err -> Err
end;
basic_cancel(#amqqueue{pid = {Name, _} = Id, type = quorum}, ChPid,
ConsumerTag, OkMsg, _ActingUser, QStates) ->
QState0 = get_quorum_state(Id, QStates),
{ok, QState} = rabbit_quorum_queue:basic_cancel(ConsumerTag, ChPid, OkMsg, QState0),
{ok, maps:put(Name, QState, QStates)}.
notify_decorators(#amqqueue{pid = QPid}) ->
delegate:invoke_no_result(QPid, {gen_server2, cast, [notify_decorators]}).
@ -1018,7 +1222,7 @@ forget_all_durable(Node) ->
#amqqueue{_ = '_'}, write),
[forget_node_for_queue(Node, Q) ||
#amqqueue{pid = Pid} = Q <- Qs,
node(Pid) =:= Node],
is_local_to_node(Pid, Node)],
ok
end),
ok.
@ -1082,7 +1286,9 @@ cancel_sync_mirrors(#amqqueue{pid = QPid}) ->
cancel_sync_mirrors(QPid) ->
delegate:invoke(QPid, {gen_server2, call, [cancel_sync_mirrors, infinity]}).
is_mirrored(Q) ->
is_replicated(#amqqueue{type = quorum}) ->
true;
is_replicated(Q) ->
rabbit_mirror_queue_misc:is_mirrored(Q).
is_dead_exclusive(#amqqueue{exclusive_owner = none}) ->
@ -1165,9 +1371,9 @@ queues_to_delete_when_node_down(NodeDown) ->
rabbit_misc:execute_mnesia_transaction(fun () ->
qlc:e(qlc:q([QName ||
#amqqueue{name = QName, pid = Pid} = Q <- mnesia:table(rabbit_queue),
node(Pid) == NodeDown andalso
qnode(Pid) == NodeDown andalso
not rabbit_mnesia:is_process_alive(Pid) andalso
(not rabbit_amqqueue:is_mirrored(Q) orelse
(not rabbit_amqqueue:is_replicated(Q) orelse
rabbit_amqqueue:is_dead_exclusive(Q))]
))
end).
@ -1212,17 +1418,23 @@ immutable(Q) -> Q#amqqueue{pid = none,
decorators = none,
state = none}.
deliver([], _Delivery) ->
%% /dev/null optimisation
[];
deliver(Qs, Delivery) ->
deliver(Qs, Delivery, untracked),
ok.
deliver(Qs, Delivery = #delivery{flow = Flow}) ->
{MPids, SPids} = qpids(Qs),
deliver([], _Delivery, QueueState) ->
%% /dev/null optimisation
{[], [], QueueState};
deliver(Qs, Delivery = #delivery{flow = Flow,
confirm = Confirm}, QueueState0) ->
{Quorum, MPids, SPids} = qpids(Qs),
QPids = MPids ++ SPids,
%% We use up two credits to send to a slave since the message
%% arrives at the slave from two directions. We will ack one when
%% the slave receives the message direct from the channel, and the
%% other when it receives it via GM.
case Flow of
%% Here we are tracking messages sent by the rabbit_channel
%% process. We are accessing the rabbit_channel process
@ -1241,13 +1453,51 @@ deliver(Qs, Delivery = #delivery{flow = Flow}) ->
SMsg = {deliver, Delivery, true},
delegate:invoke_no_result(MPids, {gen_server2, cast, [MMsg]}),
delegate:invoke_no_result(SPids, {gen_server2, cast, [SMsg]}),
QPids.
QueueState =
case QueueState0 of
untracked ->
lists:foreach(
fun({Pid, _QName}) ->
rabbit_quorum_queue:stateless_deliver(Pid, Delivery)
end, Quorum),
untracked;
_ ->
lists:foldl(
fun({{Name, _} = Pid, QName}, QStates) ->
QState0 = get_quorum_state(Pid, QName, QStates),
case rabbit_quorum_queue:deliver(Confirm, Delivery, QState0) of
{ok, QState} ->
maps:put(Name, QState, QStates);
{slow, QState} ->
maps:put(Name, QState, QStates)
end
end, QueueState0, Quorum)
end,
{QuorumPids, _} = lists:unzip(Quorum),
{QPids, QuorumPids, QueueState}.
qpids([]) -> {[], []}; %% optimisation
qpids([#amqqueue{pid = QPid, slave_pids = SPids}]) -> {[QPid], SPids}; %% opt
qpids([]) -> {[], [], []}; %% optimisation
qpids([#amqqueue{pid = {LocalName, LeaderNode}, type = quorum, name = QName}]) ->
{[{{LocalName, LeaderNode}, QName}], [], []}; %% opt
qpids([#amqqueue{pid = QPid, slave_pids = SPids}]) ->
{[], [QPid], SPids}; %% opt
qpids(Qs) ->
{MPids, SPids} = lists:foldl(fun (#amqqueue{pid = QPid, slave_pids = SPids},
{MPidAcc, SPidAcc}) ->
{[QPid | MPidAcc], [SPids | SPidAcc]}
end, {[], []}, Qs),
{MPids, lists:append(SPids)}.
{QuoPids, MPids, SPids} =
lists:foldl(fun (#amqqueue{pid = QPid, type = quorum, name = QName},
{QuoPidAcc, MPidAcc, SPidAcc}) ->
{[{QPid, QName} | QuoPidAcc], MPidAcc, SPidAcc};
(#amqqueue{pid = QPid, slave_pids = SPids},
{QuoPidAcc, MPidAcc, SPidAcc}) ->
{QuoPidAcc, [QPid | MPidAcc], [SPids | SPidAcc]}
end, {[], [], []}, Qs),
{QuoPids, MPids, lists:append(SPids)}.
get_quorum_state({Name, _} = Id, QName, Map) ->
case maps:find(Name, Map) of
{ok, S} -> S;
error ->
rabbit_quorum_queue:init_state(Id, QName)
end.
get_quorum_state({Name, _}, Map) ->
maps:get(Name, Map).

View File

@ -195,7 +195,7 @@ init_it2(Recover, From, State = #q{q = Q,
backing_queue_state = undefined}) ->
{Barrier, TermsOrNew} = recovery_status(Recover),
case rabbit_amqqueue:internal_declare(Q, Recover /= new) of
#amqqueue{} = Q1 ->
{Res, #amqqueue{} = Q1} when Res == created orelse Res == existing ->
case matches(Recover, Q, Q1) of
true ->
ok = file_handle_cache:register_callback(
@ -954,11 +954,13 @@ dead_letter_maxlen_msg(X, State = #q{backing_queue = BQ}) ->
dead_letter_msgs(Fun, Reason, X, State = #q{dlx_routing_key = RK,
backing_queue_state = BQS,
backing_queue = BQ}) ->
backing_queue = BQ,
q = #amqqueue{ name = Resource } }) ->
#resource{virtual_host = VHost} = Resource,
QName = qname(State),
{Res, Acks1, BQS1} =
Fun(fun (Msg, AckTag, Acks) ->
rabbit_dead_letter:publish(Msg, Reason, X, RK, QName),
rabbit_vhost_dead_letter:publish(VHost, X, RK, QName, [{Reason, Msg}]),
[AckTag | Acks]
end, [], BQS),
{_Guids, BQS2} = BQ:ack(Acks1, BQS1),
@ -1424,26 +1426,6 @@ handle_cast({credit, ChPid, CTag, Credit, Drain},
run_message_queue(true, State1)
end);
handle_cast({force_event_refresh, Ref},
State = #q{consumers = Consumers,
exclusive_consumer = Exclusive}) ->
rabbit_event:notify(queue_created, infos(?CREATION_EVENT_KEYS, State), Ref),
QName = qname(State),
AllConsumers = rabbit_queue_consumers:all(Consumers),
case Exclusive of
none ->
[emit_consumer_created(
Ch, CTag, false, AckRequired, QName, Prefetch,
Args, Ref, ActingUser) ||
{Ch, CTag, AckRequired, Prefetch, Args, ActingUser}
<- AllConsumers];
{Ch, CTag} ->
[{Ch, CTag, AckRequired, Prefetch, Args, ActingUser}] = AllConsumers,
emit_consumer_created(
Ch, CTag, true, AckRequired, QName, Prefetch, Args, Ref, ActingUser)
end,
noreply(rabbit_event:init_stats_timer(State, #q.stats_timer));
handle_cast(notify_decorators, State) ->
notify_decorators(State),
noreply(State);

View File

@ -51,8 +51,8 @@
%% When a queue is declared as exclusive on a channel, the channel
%% will notify queue collector of that queue.
-include("rabbit_framing.hrl").
-include("rabbit.hrl").
-include_lib("rabbit_common/include/rabbit_framing.hrl").
-include_lib("rabbit_common/include/rabbit.hrl").
-behaviour(gen_server2).
@ -63,7 +63,6 @@
emit_info_all/4, info_local/1]).
-export([refresh_config_local/0, ready_for_close/1]).
-export([refresh_interceptors/0]).
-export([force_event_refresh/1]).
-export([init/1, terminate/2, code_change/3, handle_call/3, handle_cast/2,
handle_info/2, handle_pre_hibernate/1, prioritise_call/4,
@ -73,6 +72,7 @@
-export([get_vhost/1, get_user/1]).
%% For testing
-export([build_topic_variable_map/3]).
-export([list_queue_states/1]).
%% Mgmt HTTP API refactor
-export([handle_method/5]).
@ -156,7 +156,9 @@
reply_consumer,
%% flow | noflow, see rabbitmq-server#114
delivery_flow,
interceptor_state
interceptor_state,
queue_states,
queue_cleanup_timer
}).
@ -166,7 +168,7 @@
-define(STATISTICS_KEYS,
[reductions,
pid,
pid,
transactional,
confirm,
consumer_count,
@ -207,6 +209,9 @@
put({Type, Key}, none)
end).
-define(IS_CLASSIC(QPid), is_pid(QPid)).
-define(IS_QUORUM(QPid), is_tuple(QPid)).
%%----------------------------------------------------------------------------
-export_type([channel_number/0]).
@ -249,7 +254,6 @@
-spec info_all(rabbit_types:info_keys()) -> [rabbit_types:infos()].
-spec refresh_config_local() -> 'ok'.
-spec ready_for_close(pid()) -> 'ok'.
-spec force_event_refresh(reference()) -> 'ok'.
%%----------------------------------------------------------------------------
@ -394,9 +398,8 @@ refresh_interceptors() ->
ready_for_close(Pid) ->
rabbit_channel_common:ready_for_close(Pid).
force_event_refresh(Ref) ->
[gen_server2:cast(C, {force_event_refresh, Ref}) || C <- list()],
ok.
list_queue_states(Pid) ->
gen_server2:call(Pid, list_queue_states).
%%---------------------------------------------------------------------------
@ -451,7 +454,8 @@ init([Channel, ReaderPid, WriterPid, ConnPid, ConnName, Protocol, User, VHost,
consumer_prefetch = Prefetch,
reply_consumer = none,
delivery_flow = Flow,
interceptor_state = undefined},
interceptor_state = undefined,
queue_states = #{}},
State1 = State#ch{
interceptor_state = rabbit_channel_interceptor:init(State)},
State2 = rabbit_event:init_stats_timer(State1, #ch.stats_timer),
@ -461,7 +465,8 @@ init([Channel, ReaderPid, WriterPid, ConnPid, ConnName, Protocol, User, VHost,
rabbit_event:if_enabled(State2, #ch.stats_timer,
fun() -> emit_stats(State2) end),
put_operation_timeout(),
{ok, State2, hibernate,
State3 = init_queue_cleanup_timer(State2),
{ok, State3, hibernate,
{backoff, ?HIBERNATE_AFTER_MIN, ?HIBERNATE_AFTER_MIN, ?DESIRED_HIBERNATE}}.
prioritise_call(Msg, _From, _Len, _State) ->
@ -511,6 +516,10 @@ handle_call({declare_fast_reply_to, Key}, _From,
_ -> not_found
end, State);
handle_call(list_queue_states, _From, State = #ch{queue_states = QueueStates}) ->
%% For testing of cleanup only
{reply, maps:keys(QueueStates), State};
handle_call(_Request, _From, State) ->
noreply(State).
@ -562,23 +571,8 @@ handle_cast({command, Msg}, State) ->
handle_cast({deliver, _CTag, _AckReq, _Msg}, State = #ch{state = closing}) ->
noreply(State);
handle_cast({deliver, ConsumerTag, AckRequired,
Msg = {_QName, QPid, _MsgId, Redelivered,
#basic_message{exchange_name = ExchangeName,
routing_keys = [RoutingKey | _CcRoutes],
content = Content}}},
State = #ch{writer_pid = WriterPid,
next_tag = DeliveryTag}) ->
ok = rabbit_writer:send_command_and_notify(
WriterPid, QPid, self(),
#'basic.deliver'{consumer_tag = ConsumerTag,
delivery_tag = DeliveryTag,
redelivered = Redelivered,
exchange = ExchangeName#resource.name,
routing_key = RoutingKey},
Content),
rabbit_basic:maybe_gc_large_msg(Content),
noreply(record_sent(ConsumerTag, AckRequired, Msg, State));
handle_cast({deliver, ConsumerTag, AckRequired, Msg}, State) ->
noreply(handle_deliver(ConsumerTag, AckRequired, Msg, State));
handle_cast({deliver_reply, _K, _Del}, State = #ch{state = closing}) ->
noreply(State);
@ -615,11 +609,6 @@ handle_cast({send_drained, CTagCredit}, State = #ch{writer_pid = WriterPid}) ->
|| {ConsumerTag, CreditDrained} <- CTagCredit],
noreply(State);
handle_cast({force_event_refresh, Ref}, State) ->
rabbit_event:notify(channel_created, infos(?CREATION_EVENT_KEYS, State),
Ref),
noreply(rabbit_event:init_stats_timer(State, #ch.stats_timer));
handle_cast({mandatory_received, MsgSeqNo}, State = #ch{mandatory = Mand}) ->
%% NB: don't call noreply/1 since we don't want to send confirms.
noreply_coalesce(State#ch{mandatory = dtree:drop(MsgSeqNo, Mand)});
@ -631,10 +620,74 @@ handle_cast({reject_publish, MsgSeqNo, _QPid}, State = #ch{unconfirmed = UC}) ->
%% NB: don't call noreply/1 since we don't want to send confirms.
noreply_coalesce(record_rejects(MXs, State#ch{unconfirmed = UC1}));
handle_cast({confirm, MsgSeqNos, QPid}, State = #ch{unconfirmed = UC}) ->
{MXs, UC1} = dtree:take(MsgSeqNos, QPid, UC),
%% NB: don't call noreply/1 since we don't want to send confirms.
noreply_coalesce(record_confirms(MXs, State#ch{unconfirmed = UC1})).
handle_cast({confirm, MsgSeqNos, QPid}, State) ->
noreply_coalesce(confirm(MsgSeqNos, QPid, State)).
handle_info({ra_event, {Name, _} = From, _} = Evt,
#ch{queue_states = QueueStates,
queue_names = QNames,
consumer_mapping = ConsumerMapping} = State0) ->
case QueueStates of
#{Name := QState0} ->
case rabbit_quorum_queue:handle_event(Evt, QState0) of
{{delivery, CTag, Msgs}, QState1} ->
AckRequired = case maps:find(CTag, ConsumerMapping) of
error ->
true;
{ok, {_, {NoAck, _, _, _}}} ->
not NoAck
end,
QState2 = case AckRequired of
false ->
{MsgIds, _} = lists:unzip(Msgs),
{ok, FS} = rabbit_quorum_queue:ack(CTag, MsgIds, QState1),
FS;
true ->
QState1
end,
QName = rabbit_quorum_queue:queue_name(QState2),
State = lists:foldl(
fun({MsgId, {MsgHeader, Msg}}, Acc) ->
IsDelivered = maps:is_key(delivery_count, MsgHeader),
handle_deliver(CTag, AckRequired,
{QName, From, MsgId, IsDelivered, Msg},
Acc)
end, State0#ch{queue_states = maps:put(Name, QState2, QueueStates)}, Msgs),
noreply(State);
{internal, MsgSeqNos, Actions, QState1} ->
State = State0#ch{queue_states = maps:put(Name, QState1, QueueStates)},
%% execute actions
WriterPid = State#ch.writer_pid,
lists:foreach(fun ({send_credit_reply, Avail}) ->
ok = rabbit_writer:send_command(
WriterPid,
#'basic.credit_ok'{available =
Avail});
({send_drained, {CTag, Credit}}) ->
ok = rabbit_writer:send_command(
WriterPid,
#'basic.credit_drained'{consumer_tag = CTag,
credit_drained = Credit})
end, Actions),
noreply_coalesce(confirm(MsgSeqNos, From, State));
eol ->
State1 = handle_consuming_queue_down_or_eol(From, State0),
State2 = handle_delivering_queue_down(From, State1),
{MXs, UC1} = dtree:take(From, State2#ch.unconfirmed),
State3 = record_confirms(MXs, State1#ch{unconfirmed = UC1}),
case maps:find(From, QNames) of
{ok, QName} -> erase_queue_stats(QName);
error -> ok
end,
noreply_coalesce(
State3#ch{queue_states = maps:remove(Name, QueueStates),
queue_names = maps:remove(From, QNames)})
end;
_ ->
%% the assumption here is that the queue state has been cleaned up and
%% this is a residual ra notification
noreply_coalesce(State0)
end;
handle_info({bump_credit, Msg}, State) ->
%% A rabbit_amqqueue_process is granting credit to our channel. If
@ -658,7 +711,7 @@ handle_info(emit_stats, State) ->
handle_info({'DOWN', _MRef, process, QPid, Reason}, State) ->
State1 = handle_publishing_queue_down(QPid, Reason, State),
State3 = handle_consuming_queue_down(QPid, State1),
State3 = handle_consuming_queue_down_or_eol(QPid, State1),
State4 = handle_delivering_queue_down(QPid, State3),
%% A rabbit_amqqueue_process has died. If our channel was being
%% blocked by this process, and no other process is blocking our
@ -681,7 +734,15 @@ handle_info({{Ref, Node}, LateAnswer}, State = #ch{channel = Channel})
when is_reference(Ref) ->
rabbit_log_channel:warning("Channel ~p ignoring late answer ~p from ~p",
[Channel, LateAnswer, Node]),
noreply(State).
noreply(State);
handle_info(queue_cleanup, State = #ch{queue_states = QueueStates0}) ->
QueueStates =
maps:filter(fun(_, QS) ->
QName = rabbit_quorum_queue:queue_name(QS),
[] /= rabbit_amqqueue:lookup(QName)
end, QueueStates0),
noreply(init_queue_cleanup_timer(State#ch{queue_states = QueueStates})).
handle_pre_hibernate(State) ->
ok = clear_permission_cache(),
@ -1136,8 +1197,7 @@ handle_method(#'basic.ack'{delivery_tag = DeliveryTag,
{Acked, Remaining} = collect_acks(UAMQ, DeliveryTag, Multiple),
State1 = State#ch{unacked_message_q = Remaining},
{noreply, case Tx of
none -> ack(Acked, State1),
State1;
none -> ack(Acked, State1);
{Msgs, Acks} -> Acks1 = ack_cons(ack, Acked, Acks),
State1#ch{tx = {Msgs, Acks1}}
end};
@ -1148,29 +1208,24 @@ handle_method(#'basic.get'{queue = QueueNameBin, no_ack = NoAck},
limiter = Limiter,
next_tag = DeliveryTag,
user = User,
virtual_host = VHostPath}) ->
virtual_host = VHostPath,
queue_states = QueueStates0}) ->
QueueName = qbin_to_resource(QueueNameBin, VHostPath),
check_read_permitted(QueueName, User),
case rabbit_amqqueue:with_exclusive_access_or_die(
QueueName, ConnPid,
%% Use the delivery tag as consumer tag for quorum queues
fun (Q) -> rabbit_amqqueue:basic_get(
Q, self(), NoAck, rabbit_limiter:pid(Limiter))
Q, self(), NoAck, rabbit_limiter:pid(Limiter),
DeliveryTag, QueueStates0)
end) of
{ok, MessageCount,
Msg = {QName, QPid, _MsgId, Redelivered,
#basic_message{exchange_name = ExchangeName,
routing_keys = [RoutingKey | _CcRoutes],
content = Content}}} ->
ok = rabbit_writer:send_command(
WriterPid,
#'basic.get_ok'{delivery_tag = DeliveryTag,
redelivered = Redelivered,
exchange = ExchangeName#resource.name,
routing_key = RoutingKey,
message_count = MessageCount},
Content),
State1 = monitor_delivering_queue(NoAck, QPid, QName, State),
{noreply, record_sent(none, not(NoAck), Msg, State1)};
{ok, MessageCount, Msg} ->
handle_basic_get(WriterPid, DeliveryTag, NoAck, MessageCount, Msg, State);
{ok, MessageCount, Msg, QueueStates} ->
handle_basic_get(WriterPid, DeliveryTag, NoAck, MessageCount, Msg,
State#ch{queue_states = QueueStates});
{empty, QueueStates} ->
{reply, #'basic.get_empty'{}, State#ch{queue_states = QueueStates}};
empty ->
?INCR_STATS(queue_stats, QueueName, 1, get_empty, State),
{reply, #'basic.get_empty'{}, State}
@ -1255,6 +1310,10 @@ handle_method(#'basic.consume'{queue = QueueNameBin,
{error, exclusive_consume_unavailable} ->
rabbit_misc:protocol_error(
access_refused, "~s in exclusive use",
[rabbit_misc:rs(QueueName)]);
{error, global_qos_not_supported_for_queue_type} ->
rabbit_misc:protocol_error(
not_implemented, "~s does not support global qos",
[rabbit_misc:rs(QueueName)])
end;
{ok, _} ->
@ -1266,7 +1325,8 @@ handle_method(#'basic.consume'{queue = QueueNameBin,
handle_method(#'basic.cancel'{consumer_tag = ConsumerTag, nowait = NoWait},
_, State = #ch{consumer_mapping = ConsumerMapping,
queue_consumers = QCons,
user = #user{username = Username}}) ->
user = #user{username = Username},
queue_states = QueueStates0}) ->
OkMsg = #'basic.cancel_ok'{consumer_tag = ConsumerTag},
case maps:find(ConsumerTag, ConsumerMapping) of
error ->
@ -1295,10 +1355,10 @@ handle_method(#'basic.cancel'{consumer_tag = ConsumerTag, nowait = NoWait},
fun () ->
rabbit_amqqueue:basic_cancel(
Q, self(), ConsumerTag, ok_msg(NoWait, OkMsg),
Username)
Username, QueueStates0)
end) of
ok ->
{noreply, NewState};
{ok, QueueStates} ->
{noreply, NewState#ch{queue_states = QueueStates}};
{error, not_found} ->
%% Spec requires we ignore this situation.
return_ok(NewState, NoWait, OkMsg)
@ -1339,19 +1399,25 @@ handle_method(#'basic.qos'{global = true,
{reply, #'basic.qos_ok'{}, State#ch{limiter = Limiter1}};
handle_method(#'basic.recover_async'{requeue = true},
_, State = #ch{unacked_message_q = UAMQ, limiter = Limiter}) ->
_, State = #ch{unacked_message_q = UAMQ, limiter = Limiter,
queue_states = QueueStates0}) ->
OkFun = fun () -> ok end,
UAMQL = queue:to_list(UAMQ),
foreach_per_queue(
fun (QPid, MsgIds) ->
rabbit_misc:with_exit_handler(
OkFun,
fun () -> rabbit_amqqueue:requeue(QPid, MsgIds, self()) end)
end, lists:reverse(UAMQL)),
QueueStates =
foreach_per_queue(
fun ({QPid, CTag}, MsgIds, Acc0) ->
rabbit_misc:with_exit_handler(
OkFun,
fun () ->
rabbit_amqqueue:requeue(QPid, {CTag, MsgIds},
self(), Acc0)
end)
end, lists:reverse(UAMQL), QueueStates0),
ok = notify_limiter(Limiter, UAMQL),
%% No answer required - basic.recover is the newer, synchronous
%% variant of this method
{noreply, State#ch{unacked_message_q = queue:new()}};
{noreply, State#ch{unacked_message_q = queue:new(),
queue_states = QueueStates}};
handle_method(#'basic.recover_async'{requeue = false}, _, _State) ->
rabbit_misc:protocol_error(not_implemented, "requeue=false", []);
@ -1412,8 +1478,8 @@ handle_method(#'queue.delete'{nowait = NoWait} = Method, _,
virtual_host = VHostPath,
queue_collector_pid = CollectorPid,
user = User}) ->
{ok, PurgedMessageCount} = handle_method(Method, ConnPid, CollectorPid,
VHostPath, User),
{ok, PurgedMessageCount} =
handle_method(Method, ConnPid, CollectorPid, VHostPath, User),
return_ok(State, NoWait,
#'queue.delete_ok'{message_count = PurgedMessageCount});
@ -1438,10 +1504,12 @@ handle_method(#'queue.purge'{nowait = NoWait} = Method,
user = User,
queue_collector_pid = CollectorPid,
virtual_host = VHostPath}) ->
{ok, PurgedMessageCount} = handle_method(Method, ConnPid, CollectorPid,
VHostPath, User),
return_ok(State, NoWait,
#'queue.purge_ok'{message_count = PurgedMessageCount});
case handle_method(Method, ConnPid, CollectorPid,
VHostPath, User) of
{ok, PurgedMessageCount} ->
return_ok(State, NoWait,
#'queue.purge_ok'{message_count = PurgedMessageCount})
end;
handle_method(#'tx.select'{}, _, #ch{confirm_enabled = true}) ->
precondition_failed("cannot switch from confirm to tx mode");
@ -1459,10 +1527,12 @@ handle_method(#'tx.commit'{}, _, State = #ch{tx = {Msgs, Acks},
limiter = Limiter}) ->
State1 = rabbit_misc:queue_fold(fun deliver_to_queues/2, State, Msgs),
Rev = fun (X) -> lists:reverse(lists:sort(X)) end,
lists:foreach(fun ({ack, A}) -> ack(Rev(A), State1);
({Requeue, A}) -> reject(Requeue, Rev(A), Limiter)
end, lists:reverse(Acks)),
{noreply, maybe_complete_tx(State1#ch{tx = committing})};
State2 = lists:foldl(fun ({ack, A}, Acc) ->
ack(Rev(A), Acc);
({Requeue, A}, Acc) ->
internal_reject(Requeue, Rev(A), Limiter, Acc)
end, State1, lists:reverse(Acks)),
{noreply, maybe_complete_tx(State2#ch{tx = committing})};
handle_method(#'tx.rollback'{}, _, #ch{tx = none}) ->
precondition_failed("channel is not transactional");
@ -1490,13 +1560,15 @@ handle_method(#'channel.flow'{active = false}, _, _State) ->
handle_method(#'basic.credit'{consumer_tag = CTag,
credit = Credit,
drain = Drain},
_, State = #ch{consumer_mapping = Consumers}) ->
_, State = #ch{consumer_mapping = Consumers,
queue_states = QStates0}) ->
case maps:find(CTag, Consumers) of
{ok, {Q, _CParams}} -> ok = rabbit_amqqueue:credit(
Q, self(), CTag, Credit, Drain),
{noreply, State};
error -> precondition_failed(
"unknown consumer tag '~s'", [CTag])
{ok, {Q, _CParams}} ->
{ok, QStates} = rabbit_amqqueue:credit(
Q, self(), CTag, Credit, Drain, QStates0),
{noreply, State#ch{queue_states = QStates}};
error -> precondition_failed(
"unknown consumer tag '~s'", [CTag])
end;
handle_method(_MethodRecord, _Content, _State) ->
@ -1513,7 +1585,8 @@ basic_consume(QueueName, NoAck, ConsumerPrefetch, ActualConsumerTag,
State = #ch{conn_pid = ConnPid,
limiter = Limiter,
consumer_mapping = ConsumerMapping,
user = #user{username = Username}}) ->
user = #user{username = Username},
queue_states = QueueStates0}) ->
case rabbit_amqqueue:with_exclusive_access_or_die(
QueueName, ConnPid,
fun (Q) ->
@ -1525,15 +1598,28 @@ basic_consume(QueueName, NoAck, ConsumerPrefetch, ActualConsumerTag,
ExclusiveConsume, Args,
ok_msg(NoWait, #'basic.consume_ok'{
consumer_tag = ActualConsumerTag}),
Username),
Username, QueueStates0),
Q}
end) of
{{ok, QueueStates}, Q = #amqqueue{pid = QPid, name = QName}} ->
CM1 = maps:put(
ActualConsumerTag,
{Q, {NoAck, ConsumerPrefetch, ExclusiveConsume, Args}},
ConsumerMapping),
State1 = track_delivering_queue(
NoAck, QPid, QName,
State#ch{consumer_mapping = CM1,
queue_states = QueueStates}),
{ok, case NoWait of
true -> consumer_monitor(ActualConsumerTag, State1);
false -> State1
end};
{ok, Q = #amqqueue{pid = QPid, name = QName}} ->
CM1 = maps:put(
ActualConsumerTag,
{Q, {NoAck, ConsumerPrefetch, ExclusiveConsume, Args}},
ConsumerMapping),
State1 = monitor_delivering_queue(
State1 = track_delivering_queue(
NoAck, QPid, QName,
State#ch{consumer_mapping = CM1}),
{ok, case NoWait of
@ -1541,6 +1627,8 @@ basic_consume(QueueName, NoAck, ConsumerPrefetch, ActualConsumerTag,
false -> State1
end};
{{error, exclusive_consume_unavailable} = E, _Q} ->
E;
{{error, global_qos_not_supported_for_queue_type} = E, _Q} ->
E
end.
@ -1551,29 +1639,30 @@ consumer_monitor(ConsumerTag,
State = #ch{consumer_mapping = ConsumerMapping,
queue_monitors = QMons,
queue_consumers = QCons}) ->
{#amqqueue{pid = QPid}, _CParams} =
{#amqqueue{pid = QPid}, _} =
maps:get(ConsumerTag, ConsumerMapping),
CTags1 = case maps:find(QPid, QCons) of
{ok, CTags} -> gb_sets:insert(ConsumerTag, CTags);
error -> gb_sets:singleton(ConsumerTag)
end,
{ok, CTags} -> gb_sets:insert(ConsumerTag, CTags);
error -> gb_sets:singleton(ConsumerTag)
end,
QCons1 = maps:put(QPid, CTags1, QCons),
State#ch{queue_monitors = pmon:monitor(QPid, QMons),
State#ch{queue_monitors = maybe_monitor(QPid, QMons),
queue_consumers = QCons1}.
monitor_delivering_queue(NoAck, QPid, QName,
State = #ch{queue_names = QNames,
queue_monitors = QMons,
delivering_queues = DQ}) ->
State#ch{queue_names = maps:put(QPid, QName, QNames),
queue_monitors = pmon:monitor(QPid, QMons),
track_delivering_queue(NoAck, QPid, QName,
State = #ch{queue_names = QNames,
queue_monitors = QMons,
delivering_queues = DQ}) ->
State#ch{queue_names = maps:put(QPid, QName, QNames),
queue_monitors = maybe_monitor(QPid, QMons),
delivering_queues = case NoAck of
true -> DQ;
false -> sets:add_element(QPid, DQ)
end}.
handle_publishing_queue_down(QPid, Reason, State = #ch{unconfirmed = UC,
mandatory = Mand}) ->
mandatory = Mand})
when ?IS_CLASSIC(QPid) ->
{MMsgs, Mand1} = dtree:take(QPid, Mand),
[basic_return(Msg, State, no_route) || {_, Msg} <- MMsgs],
State1 = State#ch{mandatory = Mand1},
@ -1583,10 +1672,13 @@ handle_publishing_queue_down(QPid, Reason, State = #ch{unconfirmed = UC,
false -> {MXs, UC1} = dtree:take(QPid, UC),
record_confirms(MXs, State1#ch{unconfirmed = UC1})
end.
end;
handle_publishing_queue_down(QPid, _Reason, _State) when ?IS_QUORUM(QPid) ->
error(quorum_queues_should_never_be_monitored).
handle_consuming_queue_down(QPid, State = #ch{queue_consumers = QCons,
queue_names = QNames}) ->
handle_consuming_queue_down_or_eol(QPid,
State = #ch{queue_consumers = QCons,
queue_names = QNames}) ->
ConsumerTags = case maps:find(QPid, QCons) of
error -> gb_sets:new();
{ok, CTags} -> CTags
@ -1703,19 +1795,21 @@ reject(DeliveryTag, Requeue, Multiple,
{Acked, Remaining} = collect_acks(UAMQ, DeliveryTag, Multiple),
State1 = State#ch{unacked_message_q = Remaining},
{noreply, case Tx of
none -> reject(Requeue, Acked, State1#ch.limiter),
State1;
none -> internal_reject(Requeue, Acked, State1#ch.limiter, State1);
{Msgs, Acks} -> Acks1 = ack_cons(Requeue, Acked, Acks),
State1#ch{tx = {Msgs, Acks1}}
end}.
%% NB: Acked is in youngest-first order
reject(Requeue, Acked, Limiter) ->
foreach_per_queue(
fun (QPid, MsgIds) ->
rabbit_amqqueue:reject(QPid, Requeue, MsgIds, self())
end, Acked),
ok = notify_limiter(Limiter, Acked).
internal_reject(Requeue, Acked, Limiter,
State = #ch{queue_states = QueueStates0}) ->
QueueStates = foreach_per_queue(
fun({QPid, CTag}, MsgIds, Acc0) ->
rabbit_amqqueue:reject(QPid, Requeue, {CTag, MsgIds},
self(), Acc0)
end, Acked, QueueStates0),
ok = notify_limiter(Limiter, Acked),
State#ch{queue_states = QueueStates}.
record_sent(ConsumerTag, AckRequired,
Msg = {QName, QPid, MsgId, Redelivered, _Message},
@ -1726,8 +1820,9 @@ record_sent(ConsumerTag, AckRequired,
conn_name = ConnName,
channel = ChannelNum}) ->
?INCR_STATS(queue_stats, QName, 1, case {ConsumerTag, AckRequired} of
{none, true} -> get;
{none, false} -> get_no_ack;
{_, true} when is_integer(ConsumerTag) -> get;
{_, false} when is_integer(ConsumerTag) -> get_no_ack;
%% Authentic consumer tag, this is a delivery
{_ , true} -> deliver;
{_ , false} -> deliver_no_ack
end, State),
@ -1773,17 +1868,24 @@ collect_acks(ToAcc, PrefixAcc, Q, DeliveryTag, Multiple) ->
end.
%% NB: Acked is in youngest-first order
ack(Acked, State = #ch{queue_names = QNames}) ->
foreach_per_queue(
fun (QPid, MsgIds) ->
ok = rabbit_amqqueue:ack(QPid, MsgIds, self()),
case maps:find(QPid, QNames) of
{ok, QName} -> Count = length(MsgIds),
?INCR_STATS(queue_stats, QName, Count, ack, State);
error -> ok
end
end, Acked),
ok = notify_limiter(State#ch.limiter, Acked).
ack(Acked, State = #ch{queue_names = QNames,
queue_states = QueueStates0}) ->
QueueStates =
foreach_per_queue(
fun ({QPid, CTag}, MsgIds, Acc0) ->
Acc = rabbit_amqqueue:ack(QPid, {CTag, MsgIds}, self(), Acc0),
incr_queue_stats(QPid, QNames, MsgIds, State),
Acc
end, Acked, QueueStates0),
ok = notify_limiter(State#ch.limiter, Acked),
State#ch{queue_states = QueueStates}.
incr_queue_stats(QPid, QNames, MsgIds, State) ->
case maps:find(QPid, QNames) of
{ok, QName} -> Count = length(MsgIds),
?INCR_STATS(queue_stats, QName, Count, ack, State);
error -> ok
end.
%% {Msgs, Acks}
%%
@ -1803,23 +1905,24 @@ notify_queues(State = #ch{state = closing}) ->
{ok, State};
notify_queues(State = #ch{consumer_mapping = Consumers,
delivering_queues = DQ }) ->
QPids = sets:to_list(
sets:union(sets:from_list(consumer_queues(Consumers)), DQ)),
QPids0 = sets:to_list(
sets:union(sets:from_list(consumer_queues(Consumers)), DQ)),
%% filter to only include pids to avoid trying to notify quorum queues
QPids = [P || P <- QPids0, ?IS_CLASSIC(P)],
Timeout = get_operation_timeout(),
{rabbit_amqqueue:notify_down_all(QPids, self(), Timeout),
State#ch{state = closing}}.
foreach_per_queue(_F, []) ->
ok;
foreach_per_queue(F, [{_DTag, _CTag, {QPid, MsgId}}]) -> %% common case
F(QPid, [MsgId]);
%% NB: UAL should be in youngest-first order; the tree values will
%% then be in oldest-first order
foreach_per_queue(F, UAL) ->
T = lists:foldl(fun ({_DTag, _CTag, {QPid, MsgId}}, T) ->
rabbit_misc:gb_trees_cons(QPid, MsgId, T)
foreach_per_queue(_F, [], Acc) ->
Acc;
foreach_per_queue(F, [{_DTag, CTag, {QPid, MsgId}}], Acc) ->
%% quorum queue, needs the consumer tag
F({QPid, CTag}, [MsgId], Acc);
foreach_per_queue(F, UAL, Acc) ->
T = lists:foldl(fun ({_DTag, CTag, {QPid, MsgId}}, T) ->
rabbit_misc:gb_trees_cons({QPid, CTag}, MsgId, T)
end, gb_trees:empty(), UAL),
rabbit_misc:gb_trees_foreach(F, T).
rabbit_misc:gb_trees_fold(fun (Key, Val, Acc0) -> F(Key, Val, Acc0) end, Acc, T).
consumer_queues(Consumers) ->
lists:usort([QPid || {_Key, {#amqqueue{pid = QPid}, _CParams}}
@ -1828,13 +1931,19 @@ consumer_queues(Consumers) ->
%% tell the limiter about the number of acks that have been received
%% for messages delivered to subscribed consumers, but not acks for
%% messages sent in a response to a basic.get (identified by their
%% 'none' consumer tag)
%% consumer tag as an integer (the same as the delivery tag, required
%% quorum queues))
notify_limiter(Limiter, Acked) ->
%% optimisation: avoid the potentially expensive 'foldl' in the
%% common case.
case rabbit_limiter:is_active(Limiter) of
false -> ok;
true -> case lists:foldl(fun ({_, none, _}, Acc) -> Acc;
true -> case lists:foldl(fun ({_, CTag, _}, Acc) when is_integer(CTag) ->
%% Quorum queues use integer CTags
%% classic queues use binaries
%% Quorum queues do not interact
%% with limiters
Acc;
({_, _, _}, Acc) -> Acc + 1
end, 0, Acked) of
0 -> ok;
@ -1854,10 +1963,13 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{
confirm = Confirm,
msg_seq_no = MsgSeqNo},
DelQNames}, State = #ch{queue_names = QNames,
queue_monitors = QMons}) ->
queue_monitors = QMons,
queue_states = QueueStates0}) ->
Qs = rabbit_amqqueue:lookup(DelQNames),
DeliveredQPids = rabbit_amqqueue:deliver(Qs, Delivery),
%% The pmon:monitor_all/2 monitors all queues to which we
{DeliveredQPids, DeliveredQQPids, QueueStates} =
rabbit_amqqueue:deliver(Qs, Delivery, QueueStates0),
AllDeliveredQPids = DeliveredQPids ++ DeliveredQQPids,
%% The maybe_monitor_all/2 monitors all queues to which we
%% delivered. But we want to monitor even queues we didn't deliver
%% to, since we need their 'DOWN' messages to clean
%% queue_names. So we also need to monitor each QPid from
@ -1873,26 +1985,26 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{
{case maps:is_key(QPid, QNames0) of
true -> QNames0;
false -> maps:put(QPid, QName, QNames0)
end, pmon:monitor(QPid, QMons0)}
end, {QNames, pmon:monitor_all(DeliveredQPids, QMons)}, Qs),
end, maybe_monitor(QPid, QMons0)}
end, {QNames, maybe_monitor_all(DeliveredQPids, QMons)}, Qs),
State1 = State#ch{queue_names = QNames1,
queue_monitors = QMons1},
%% NB: the order here is important since basic.returns must be
%% sent before confirms.
State2 = process_routing_mandatory(Mandatory, DeliveredQPids, MsgSeqNo,
State2 = process_routing_mandatory(Mandatory, AllDeliveredQPids, MsgSeqNo,
Message, State1),
State3 = process_routing_confirm( Confirm, DeliveredQPids, MsgSeqNo,
XName, State2),
State3 = process_routing_confirm( Confirm, AllDeliveredQPids, MsgSeqNo,
XName, State2),
case rabbit_event:stats_level(State3, #ch.stats_timer) of
fine ->
?INCR_STATS(exchange_stats, XName, 1, publish),
[?INCR_STATS(queue_exchange_stats, {QName, XName}, 1, publish) ||
QPid <- DeliveredQPids,
QPid <- AllDeliveredQPids,
{ok, QName} <- [maps:find(QPid, QNames1)]];
_ ->
ok
end,
State3.
State3#ch{queue_states = QueueStates}.
process_routing_mandatory(false, _, _MsgSeqNo, _Msg, State) ->
State;
@ -1911,6 +2023,11 @@ process_routing_confirm(true, QPids, MsgSeqNo, XName, State) ->
State#ch{unconfirmed = dtree:insert(MsgSeqNo, QPids, XName,
State#ch.unconfirmed)}.
confirm(MsgSeqNos, QPid, State = #ch{unconfirmed = UC}) ->
{MXs, UC1} = dtree:take(MsgSeqNos, QPid, UC),
%% NB: don't call noreply/1 since we don't want to send confirms.
record_confirms(MXs, State#ch{unconfirmed = UC1}).
send_confirms_and_nacks(State = #ch{tx = none, confirmed = [], rejected = []}) ->
State;
send_confirms_and_nacks(State = #ch{tx = none, confirmed = C, rejected = R}) ->
@ -2088,6 +2205,7 @@ get_operation_timeout() ->
%% Refactored and exported to allow direct calls from the HTTP API,
%% avoiding the usage of AMQP 0-9-1 from the management.
handle_method(#'exchange.bind'{destination = DestinationNameBin,
source = SourceNameBin,
routing_key = RoutingKey,
@ -2138,7 +2256,8 @@ handle_method(#'queue.declare'{queue = QueueNameBin,
auto_delete = AutoDelete,
nowait = NoWait,
arguments = Args} = Declare,
ConnPid, CollectorPid, VHostPath, #user{username = Username} = User) ->
ConnPid, CollectorPid, VHostPath,
#user{username = Username} = User) ->
Owner = case ExclusiveDeclare of
true -> ConnPid;
false -> none
@ -2195,7 +2314,8 @@ handle_method(#'queue.declare'{queue = QueueNameBin,
{existing, _Q} ->
%% must have been created between the stat and the
%% declare. Loop around again.
handle_method(Declare, ConnPid, CollectorPid, VHostPath, User);
handle_method(Declare, ConnPid, CollectorPid, VHostPath,
User);
{absent, Q, Reason} ->
rabbit_misc:absent(Q, Reason);
{owner_died, _Q} ->
@ -2221,7 +2341,8 @@ handle_method(#'queue.declare'{queue = QueueNameBin,
handle_method(#'queue.delete'{queue = QueueNameBin,
if_unused = IfUnused,
if_empty = IfEmpty},
ConnPid, _CollectorPid, VHostPath, User = #user{username = Username}) ->
ConnPid, _CollectorPid, VHostPath,
User = #user{username = Username}) ->
StrippedQueueNameBin = strip_cr_lf(QueueNameBin),
QueueName = qbin_to_resource(StrippedQueueNameBin, VHostPath),
@ -2233,6 +2354,7 @@ handle_method(#'queue.delete'{queue = QueueNameBin,
rabbit_amqqueue:delete(Q, IfUnused, IfEmpty, Username)
end,
fun (not_found) -> {ok, 0};
%% TODO delete crashed should clean up fifo states?
({absent, Q, crashed}) -> rabbit_amqqueue:delete_crashed(Q, Username),
{ok, 0};
({absent, Q, stopped}) -> rabbit_amqqueue:delete_crashed(Q, Username),
@ -2243,12 +2365,13 @@ handle_method(#'queue.delete'{queue = QueueNameBin,
precondition_failed("~s in use", [rabbit_misc:rs(QueueName)]);
{error, not_empty} ->
precondition_failed("~s not empty", [rabbit_misc:rs(QueueName)]);
{ok, _Count} = OK ->
OK
{ok, Count} ->
{ok, Count}
end;
handle_method(#'exchange.delete'{exchange = ExchangeNameBin,
if_unused = IfUnused},
_ConnPid, _CollectorPid, VHostPath, User = #user{username = Username}) ->
_ConnPid, _CollectorPid, VHostPath,
User = #user{username = Username}) ->
StrippedExchangeNameBin = strip_cr_lf(ExchangeNameBin),
ExchangeName = rabbit_misc:r(VHostPath, exchange, StrippedExchangeNameBin),
check_not_default_exchange(ExchangeName),
@ -2276,7 +2399,8 @@ handle_method(#'exchange.declare'{exchange = ExchangeNameBin,
auto_delete = AutoDelete,
internal = Internal,
arguments = Args},
_ConnPid, _CollectorPid, VHostPath, #user{username = Username} = User) ->
_ConnPid, _CollectorPid, VHostPath,
#user{username = Username} = User) ->
CheckedType = rabbit_exchange:check_type(TypeNameBin),
ExchangeName = rabbit_misc:r(VHostPath, exchange, strip_cr_lf(ExchangeNameBin)),
check_not_default_exchange(ExchangeName),
@ -2312,3 +2436,56 @@ handle_method(#'exchange.declare'{exchange = ExchangeNameBin,
ExchangeName = rabbit_misc:r(VHostPath, exchange, strip_cr_lf(ExchangeNameBin)),
check_not_default_exchange(ExchangeName),
_ = rabbit_exchange:lookup_or_die(ExchangeName).
handle_deliver(ConsumerTag, AckRequired,
Msg = {_QName, QPid, _MsgId, Redelivered,
#basic_message{exchange_name = ExchangeName,
routing_keys = [RoutingKey | _CcRoutes],
content = Content}},
State = #ch{writer_pid = WriterPid,
next_tag = DeliveryTag}) ->
Deliver = #'basic.deliver'{consumer_tag = ConsumerTag,
delivery_tag = DeliveryTag,
redelivered = Redelivered,
exchange = ExchangeName#resource.name,
routing_key = RoutingKey},
case ?IS_CLASSIC(QPid) of
true ->
ok = rabbit_writer:send_command_and_notify(
WriterPid, QPid, self(), Deliver, Content);
false ->
ok = rabbit_writer:send_command(WriterPid, Deliver, Content)
end,
rabbit_basic:maybe_gc_large_msg(Content),
record_sent(ConsumerTag, AckRequired, Msg, State).
handle_basic_get(WriterPid, DeliveryTag, NoAck, MessageCount,
Msg = {QName, QPid, _MsgId, Redelivered,
#basic_message{exchange_name = ExchangeName,
routing_keys = [RoutingKey | _CcRoutes],
content = Content}}, State) ->
ok = rabbit_writer:send_command(
WriterPid,
#'basic.get_ok'{delivery_tag = DeliveryTag,
redelivered = Redelivered,
exchange = ExchangeName#resource.name,
routing_key = RoutingKey,
message_count = MessageCount},
Content),
State1 = track_delivering_queue(NoAck, QPid, QName, State),
{noreply, record_sent(DeliveryTag, not(NoAck), Msg, State1)}.
init_queue_cleanup_timer(State) ->
{ok, Interval} = application:get_env(rabbit, channel_queue_cleanup_interval),
State#ch{queue_cleanup_timer = erlang:send_after(Interval, self(), queue_cleanup)}.
%% only classic queues need monitoring so rather than special casing
%% everywhere monitors are set up we wrap it here for this module
maybe_monitor(QPid, QMons) when ?IS_CLASSIC(QPid) ->
pmon:monitor(QPid, QMons);
maybe_monitor(_, QMons) ->
QMons.
maybe_monitor_all([], S) -> S; %% optimisation
maybe_monitor_all([Item], S) -> maybe_monitor(Item, S); %% optimisation
maybe_monitor_all(Items, S) -> lists:foldl(fun maybe_monitor/2, S, Items).

View File

@ -77,7 +77,23 @@ gc_local_queues() ->
Queues = rabbit_amqqueue:list_local_names(),
GbSet = gb_sets:from_list(Queues),
gc_entity(queue_metrics, GbSet),
gc_entity(queue_coarse_metrics, GbSet).
gc_entity(queue_coarse_metrics, GbSet),
Followers = gb_sets:from_list(rabbit_amqqueue:list_local_followers()),
gc_leader_data(Followers).
gc_leader_data(Followers) ->
ets:foldl(fun({Id, _, _, _, _}, none) ->
gc_leader_data(Id, queue_coarse_metrics, Followers)
end, none, queue_coarse_metrics).
gc_leader_data(Id, Table, GbSet) ->
case gb_sets:is_member(Id, GbSet) of
true ->
ets:delete(Table, Id),
none;
false ->
none
end.
gc_global_queues() ->
GbSet = gb_sets:from_list(rabbit_amqqueue:list_names()),

View File

@ -16,7 +16,7 @@
-module(rabbit_dead_letter).
-export([publish/5]).
-export([publish/6]).
-include("rabbit.hrl").
-include("rabbit_framing.hrl").
@ -26,18 +26,20 @@
-type reason() :: 'expired' | 'rejected' | 'maxlen'.
-spec publish(rabbit_types:message(), reason(), rabbit_types:exchange(),
'undefined' | binary(), rabbit_amqqueue:name()) -> 'ok'.
'undefined' | binary(), rabbit_amqqueue:name(),
#{Name :: atom() => rabbit_fifo_client:state()}) -> 'ok'.
%%----------------------------------------------------------------------------
publish(Msg, Reason, X, RK, QName) ->
publish(Msg, Reason, X, RK, QName, QueueStates0) ->
DLMsg = make_msg(Msg, Reason, X#exchange.name, RK, QName),
Delivery = rabbit_basic:delivery(false, false, DLMsg, undefined),
{Queues, Cycles} = detect_cycles(Reason, DLMsg,
rabbit_exchange:route(X, Delivery)),
lists:foreach(fun log_cycle_once/1, Cycles),
rabbit_amqqueue:deliver(rabbit_amqqueue:lookup(Queues), Delivery),
ok.
{_, _, QueueStates} = rabbit_amqqueue:deliver(rabbit_amqqueue:lookup(Queues),
Delivery, QueueStates0),
QueueStates.
make_msg(Msg = #basic_message{content = Content,
exchange_name = Exchange,

View File

@ -16,7 +16,7 @@
-module(rabbit_direct).
-export([boot/0, force_event_refresh/1, list/0, connect/5,
-export([boot/0, list/0, connect/5,
start_channel/9, disconnect/2]).
%% Internal
-export([list_local/0]).
@ -29,7 +29,6 @@
%%----------------------------------------------------------------------------
-spec boot() -> 'ok'.
-spec force_event_refresh(reference()) -> 'ok'.
-spec list() -> [pid()].
-spec list_local() -> [pid()].
-spec connect
@ -55,10 +54,6 @@ boot() -> rabbit_sup:start_supervisor_child(
[{local, rabbit_direct_client_sup},
{rabbit_channel_sup, start_link, []}]).
force_event_refresh(Ref) ->
[Pid ! {force_event_refresh, Ref} || Pid <- list()],
ok.
list_local() ->
pg_local:get_members(rabbit_direct).

1577
src/rabbit_fifo.erl Normal file

File diff suppressed because it is too large Load Diff

667
src/rabbit_fifo_client.erl Normal file
View File

@ -0,0 +1,667 @@
%% @doc Provides an easy to consume API for interacting with the {@link rabbit_fifo.}
%% state machine implementation running inside a `ra' raft system.
%%
%% Handles command tracking and other non-functional concerns.
-module(rabbit_fifo_client).
-export([
init/2,
init/3,
init/5,
checkout/3,
checkout/4,
cancel_checkout/2,
enqueue/2,
enqueue/3,
dequeue/3,
settle/3,
return/3,
discard/3,
credit/4,
handle_ra_event/3,
untracked_enqueue/2,
purge/1,
cluster_name/1
]).
-include_lib("ra/include/ra.hrl").
-define(SOFT_LIMIT, 256).
-type seq() :: non_neg_integer().
-type action() :: {send_credit_reply, Available :: non_neg_integer()} |
{send_drained, CTagCredit ::
{rabbit_fifo:consumer_tag(), non_neg_integer()}}.
-type actions() :: [action()].
-record(consumer, {last_msg_id :: seq(),
delivery_count = 0 :: non_neg_integer()}).
-record(state, {cluster_name :: ra_cluster_name(),
servers = [] :: [ra_server_id()],
leader :: maybe(ra_server_id()),
next_seq = 0 :: seq(),
last_applied :: maybe(seq()),
next_enqueue_seq = 1 :: seq(),
%% indicates that we've exceeded the soft limit
slow = false :: boolean(),
unsent_commands = #{} :: #{rabbit_fifo:consumer_id() =>
{[seq()], [seq()], [seq()]}},
soft_limit = ?SOFT_LIMIT :: non_neg_integer(),
pending = #{} :: #{seq() =>
{maybe(term()), rabbit_fifo:command()}},
consumer_deliveries = #{} :: #{rabbit_fifo:consumer_tag() =>
#consumer{}},
priority = normal :: normal | low,
block_handler = fun() -> ok end :: fun(() -> ok),
unblock_handler = fun() -> ok end :: fun(() -> ok),
timeout :: non_neg_integer()
}).
-opaque state() :: #state{}.
-export_type([
state/0
]).
%% @doc Create the initial state for a new rabbit_fifo sessions. A state is needed
%% to interact with a rabbit_fifo queue using @module.
%% @param ClusterName the id of the cluster to interact with
%% @param Servers The known servers of the queue. If the current leader is known
%% ensure the leader node is at the head of the list.
-spec init(ra_cluster_name(), [ra_server_id()]) -> state().
init(ClusterName, Servers) ->
init(ClusterName, Servers, ?SOFT_LIMIT).
%% @doc Create the initial state for a new rabbit_fifo sessions. A state is needed
%% to interact with a rabbit_fifo queue using @module.
%% @param ClusterName the id of the cluster to interact with
%% @param Servers The known servers of the queue. If the current leader is known
%% ensure the leader node is at the head of the list.
%% @param MaxPending size defining the max number of pending commands.
-spec init(ra_cluster_name(), [ra_server_id()], non_neg_integer()) -> state().
init(ClusterName, Servers, SoftLimit) ->
Timeout = application:get_env(kernel, net_ticktime, 60000) + 5000,
#state{cluster_name = ClusterName,
servers = Servers,
soft_limit = SoftLimit,
timeout = Timeout}.
-spec init(ra_cluster_name(), [ra_server_id()], non_neg_integer(), fun(() -> ok),
fun(() -> ok)) -> state().
init(ClusterName, Servers, SoftLimit, BlockFun, UnblockFun) ->
Timeout = application:get_env(kernel, net_ticktime, 60000) + 5000,
#state{cluster_name = ClusterName,
servers = Servers,
block_handler = BlockFun,
unblock_handler = UnblockFun,
soft_limit = SoftLimit,
timeout = Timeout}.
%% @doc Enqueues a message.
%% @param Correlation an arbitrary erlang term used to correlate this
%% command when it has been applied.
%% @param Msg an arbitrary erlang term representing the message.
%% @param State the current {@module} state.
%% @returns
%% `{ok | slow, State}' if the command was successfully sent. If the return
%% tag is `slow' it means the limit is approaching and it is time to slow down
%% the sending rate.
%% {@module} assigns a sequence number to every raft command it issues. The
%% SequenceNumber can be correlated to the applied sequence numbers returned
%% by the {@link handle_ra_event/2. handle_ra_event/2} function.
-spec enqueue(Correlation :: term(), Msg :: term(), State :: state()) ->
{ok | slow, state()}.
enqueue(Correlation, Msg, State0 = #state{slow = Slow,
block_handler = BlockFun}) ->
Node = pick_node(State0),
{Next, State1} = next_enqueue_seq(State0),
% by default there is no correlation id
Cmd = {enqueue, self(), Next, Msg},
case send_command(Node, Correlation, Cmd, low, State1) of
{slow, _} = Ret when not Slow ->
BlockFun(),
Ret;
Any ->
Any
end.
%% @doc Enqueues a message.
%% @param Msg an arbitrary erlang term representing the message.
%% @param State the current {@module} state.
%% @returns
%% `{ok | slow, State}' if the command was successfully sent. If the return
%% tag is `slow' it means the limit is approaching and it is time to slow down
%% the sending rate.
%% {@module} assigns a sequence number to every raft command it issues. The
%% SequenceNumber can be correlated to the applied sequence numbers returned
%% by the {@link handle_ra_event/2. handle_ra_event/2} function.
%%
-spec enqueue(Msg :: term(), State :: state()) ->
{ok | slow, state()}.
enqueue(Msg, State) ->
enqueue(undefined, Msg, State).
%% @doc Dequeue a message from the queue.
%%
%% This is a syncronous call. I.e. the call will block until the command
%% has been accepted by the ra process or it times out.
%%
%% @param ConsumerTag a unique tag to identify this particular consumer.
%% @param Settlement either `settled' or `unsettled'. When `settled' no
%% further settlement needs to be done.
%% @param State The {@module} state.
%%
%% @returns `{ok, IdMsg, State}' or `{error | timeout, term()}'
-spec dequeue(rabbit_fifo:consumer_tag(),
Settlement :: settled | unsettled, state()) ->
{ok, rabbit_fifo:delivery_msg() | empty, state()} | {error | timeout, term()}.
dequeue(ConsumerTag, Settlement, #state{timeout = Timeout} = State0) ->
Node = pick_node(State0),
ConsumerId = consumer_id(ConsumerTag),
case ra:process_command(Node, {checkout, {dequeue, Settlement},
ConsumerId}, Timeout) of
{ok, {dequeue, Reply}, Leader} ->
{ok, Reply, State0#state{leader = Leader}};
Err ->
Err
end.
%% @doc Settle a message. Permanently removes message from the queue.
%% @param ConsumerTag the tag uniquely identifying the consumer.
%% @param MsgIds the message ids received with the {@link rabbit_fifo:delivery/0.}
%% @param State the {@module} state
%% @returns
%% `{ok | slow, State}' if the command was successfully sent. If the return
%% tag is `slow' it means the limit is approaching and it is time to slow down
%% the sending rate.
%%
-spec settle(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) ->
{ok, state()}.
settle(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) ->
Node = pick_node(State0),
Cmd = {settle, MsgIds, consumer_id(ConsumerTag)},
case send_command(Node, undefined, Cmd, normal, State0) of
{slow, S} ->
% turn slow into ok for this function
{ok, S};
{ok, _} = Ret ->
Ret
end;
settle(ConsumerTag, [_|_] = MsgIds,
#state{unsent_commands = Unsent0} = State0) ->
ConsumerId = consumer_id(ConsumerTag),
%% we've reached the soft limit so will stash the command to be
%% sent once we have seen enough notifications
Unsent = maps:update_with(ConsumerId,
fun ({Settles, Returns, Discards}) ->
{Settles ++ MsgIds, Returns, Discards}
end, {MsgIds, [], []}, Unsent0),
{ok, State0#state{unsent_commands = Unsent}}.
%% @doc Return a message to the queue.
%% @param ConsumerTag the tag uniquely identifying the consumer.
%% @param MsgIds the message ids to return received
%% from {@link rabbit_fifo:delivery/0.}
%% @param State the {@module} state
%% @returns
%% `{ok | slow, State}' if the command was successfully sent. If the return
%% tag is `slow' it means the limit is approaching and it is time to slow down
%% the sending rate.
%%
-spec return(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) ->
{ok, state()}.
return(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) ->
Node = pick_node(State0),
% TODO: make rabbit_fifo return support lists of message ids
Cmd = {return, MsgIds, consumer_id(ConsumerTag)},
case send_command(Node, undefined, Cmd, normal, State0) of
{slow, S} ->
% turn slow into ok for this function
{ok, S};
{ok, _} = Ret ->
Ret
end;
return(ConsumerTag, [_|_] = MsgIds,
#state{unsent_commands = Unsent0} = State0) ->
ConsumerId = consumer_id(ConsumerTag),
%% we've reached the soft limit so will stash the command to be
%% sent once we have seen enough notifications
Unsent = maps:update_with(ConsumerId,
fun ({Settles, Returns, Discards}) ->
{Settles, Returns ++ MsgIds, Discards}
end, {[], MsgIds, []}, Unsent0),
{ok, State0#state{unsent_commands = Unsent}}.
%% @doc Discards a checked out message.
%% If the queue has a dead_letter_handler configured this will be called.
%% @param ConsumerTag the tag uniquely identifying the consumer.
%% @param MsgIds the message ids to discard
%% from {@link rabbit_fifo:delivery/0.}
%% @param State the {@module} state
%% @returns
%% `{ok | slow, State}' if the command was successfully sent. If the return
%% tag is `slow' it means the limit is approaching and it is time to slow down
%% the sending rate.
-spec discard(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) ->
{ok | slow, state()}.
discard(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) ->
Node = pick_node(State0),
Cmd = {discard, MsgIds, consumer_id(ConsumerTag)},
case send_command(Node, undefined, Cmd, normal, State0) of
{slow, S} ->
% turn slow into ok for this function
{ok, S};
{ok, _} = Ret ->
Ret
end;
discard(ConsumerTag, [_|_] = MsgIds,
#state{unsent_commands = Unsent0} = State0) ->
ConsumerId = consumer_id(ConsumerTag),
%% we've reached the soft limit so will stash the command to be
%% sent once we have seen enough notifications
Unsent = maps:update_with(ConsumerId,
fun ({Settles, Returns, Discards}) ->
{Settles, Returns, Discards ++ MsgIds}
end, {[], [], MsgIds}, Unsent0),
{ok, State0#state{unsent_commands = Unsent}}.
%% @doc Register with the rabbit_fifo queue to "checkout" messages as they
%% become available.
%%
%% This is a syncronous call. I.e. the call will block until the command
%% has been accepted by the ra process or it times out.
%%
%% @param ConsumerTag a unique tag to identify this particular consumer.
%% @param NumUnsettled the maximum number of in-flight messages. Once this
%% number of messages has been received but not settled no further messages
%% will be delivered to the consumer.
%% @param State The {@module} state.
%%
%% @returns `{ok, State}' or `{error | timeout, term()}'
-spec checkout(rabbit_fifo:consumer_tag(), NumUnsettled :: non_neg_integer(),
state()) -> {ok, state()} | {error | timeout, term()}.
checkout(ConsumerTag, NumUnsettled, State0) ->
checkout(ConsumerTag, NumUnsettled, simple_prefetch, State0).
%% @doc Register with the rabbit_fifo queue to "checkout" messages as they
%% become available.
%%
%% This is a syncronous call. I.e. the call will block until the command
%% has been accepted by the ra process or it times out.
%%
%% @param ConsumerTag a unique tag to identify this particular consumer.
%% @param NumUnsettled the maximum number of in-flight messages. Once this
%% number of messages has been received but not settled no further messages
%% will be delivered to the consumer.
%% @param CreditMode The credit mode to use for the checkout.
%% simple_prefetch: credit is auto topped up as deliveries are settled
%% credited: credit is only increased by sending credit to the queue
%% @param State The {@module} state.
%%
%% @returns `{ok, State}' or `{error | timeout, term()}'
-spec checkout(rabbit_fifo:consumer_tag(), NumUnsettled :: non_neg_integer(),
CreditMode :: rabbit_fifo:credit_mode(),
state()) -> {ok, state()} | {error | timeout, term()}.
checkout(ConsumerTag, NumUnsettled, CreditMode, State0) ->
Servers = sorted_servers(State0),
ConsumerId = {ConsumerTag, self()},
Cmd = {checkout, {auto, NumUnsettled, CreditMode}, ConsumerId},
try_process_command(Servers, Cmd, State0).
%% @doc Provide credit to the queue
%%
%% This only has an effect if the consumer uses credit mode: credited
%% @param ConsumerTag a unique tag to identify this particular consumer.
%% @param Credit the amount of credit to provide to theq queue
%% @param Drain tells the queue to use up any credit that cannot be immediately
%% fulfilled. (i.e. there are not enough messages on queue to use up all the
%% provided credit).
-spec credit(rabbit_fifo:consumer_tag(),
Credit :: non_neg_integer(),
Drain :: boolean(),
state()) ->
{ok, state()}.
credit(ConsumerTag, Credit, Drain,
#state{consumer_deliveries = CDels} = State0) ->
ConsumerId = consumer_id(ConsumerTag),
%% the last received msgid provides us with the delivery count if we
%% add one as it is 0 indexed
C = maps:get(ConsumerTag, CDels, #consumer{last_msg_id = -1}),
Node = pick_node(State0),
Cmd = {credit, Credit, C#consumer.last_msg_id + 1, Drain, ConsumerId},
ct:pal("sending credit ~w", [Cmd]),
case send_command(Node, undefined, Cmd, normal, State0) of
{slow, S} ->
% turn slow into ok for this function
{ok, S};
{ok, _} = Ret ->
Ret
end.
%% @doc Cancels a checkout with the rabbit_fifo queue for the consumer tag
%%
%% This is a syncronous call. I.e. the call will block until the command
%% has been accepted by the ra process or it times out.
%%
%% @param ConsumerTag a unique tag to identify this particular consumer.
%% @param State The {@module} state.
%%
%% @returns `{ok, State}' or `{error | timeout, term()}'
-spec cancel_checkout(rabbit_fifo:consumer_tag(), state()) ->
{ok, state()} | {error | timeout, term()}.
cancel_checkout(ConsumerTag, #state{consumer_deliveries = CDels} = State0) ->
Servers = sorted_servers(State0),
ConsumerId = {ConsumerTag, self()},
Cmd = {checkout, cancel, ConsumerId},
State = State0#state{consumer_deliveries = maps:remove(ConsumerTag, CDels)},
try_process_command(Servers, Cmd, State).
%% @doc Purges all the messages from a rabbit_fifo queue and returns the number
%% of messages purged.
-spec purge(ra_server_id()) -> {ok, non_neg_integer()} | {error | timeout, term()}.
purge(Node) ->
case ra:process_command(Node, purge) of
{ok, {purge, Reply}, _} ->
{ok, Reply};
Err ->
Err
end.
%% @doc returns the cluster name
-spec cluster_name(state()) -> ra_cluster_name().
cluster_name(#state{cluster_name = ClusterName}) ->
ClusterName.
%% @doc Handles incoming `ra_events'. Events carry both internal "bookeeping"
%% events emitted by the `ra' leader as well as `rabbit_fifo' emitted events such
%% as message deliveries. All ra events need to be handled by {@module}
%% to ensure bookeeping, resends and flow control is correctly handled.
%%
%% If the `ra_event' contains a `rabbit_fifo' generated message it will be returned
%% for further processing.
%%
%% Example:
%%
%% ```
%% receive
%% {ra_event, From, Evt} ->
%% case rabbit_fifo_client:handle_ra_event(From, Evt, State0) of
%% {internal, _Seq, State} -> State;
%% {{delivery, _ConsumerTag, Msgs}, State} ->
%% handle_messages(Msgs),
%% ...
%% end
%% end
%% '''
%%
%% @param From the {@link ra_server_id().} of the sending process.
%% @param Event the body of the `ra_event'.
%% @param State the current {@module} state.
%%
%% @returns
%% `{internal, AppliedCorrelations, State}' if the event contained an internally
%% handled event such as a notification and a correlation was included with
%% the command (e.g. in a call to `enqueue/3' the correlation terms are returned
%% here.
%%
%% `{RaFifoEvent, State}' if the event contained a client message generated by
%% the `rabbit_fifo' state machine such as a delivery.
%%
%% The type of `rabbit_fifo' client messages that can be received are:
%%
%% `{delivery, ConsumerTag, [{MsgId, {MsgHeader, Msg}}]}'
%%
%% <li>`ConsumerTag' the binary tag passed to {@link checkout/3.}</li>
%% <li>`MsgId' is a consumer scoped monotonically incrementing id that can be
%% used to {@link settle/3.} (roughly: AMQP 0.9.1 ack) message once finished
%% with them.</li>
-spec handle_ra_event(ra_server_id(), ra_server_proc:ra_event_body(), state()) ->
{internal, Correlators :: [term()], actions(), state()} |
{rabbit_fifo:client_msg(), state()} | eol.
handle_ra_event(From, {applied, Seqs},
#state{soft_limit = SftLmt,
unblock_handler = UnblockFun} = State0) ->
{Corrs, Actions, State1} = lists:foldl(fun seq_applied/2,
{[], [], State0#state{leader = From}},
Seqs),
case maps:size(State1#state.pending) < SftLmt of
true when State1#state.slow == true ->
% we have exited soft limit state
% send any unsent commands
State2 = State1#state{slow = false,
unsent_commands = #{}},
% build up a list of commands to issue
Commands = maps:fold(
fun (Cid, {Settled, Returns, Discards}, Acc) ->
add_command(Cid, settle, Settled,
add_command(Cid, return, Returns,
add_command(Cid, discard, Discards, Acc)))
end, [], State1#state.unsent_commands),
Node = pick_node(State2),
%% send all the settlements and returns
State = lists:foldl(fun (C, S0) ->
case send_command(Node, undefined,
C, normal, S0) of
{T, S} when T =/= error ->
S
end
end, State2, Commands),
UnblockFun(),
{internal, lists:reverse(Corrs), lists:reverse(Actions), State};
_ ->
{internal, lists:reverse(Corrs), lists:reverse(Actions), State1}
end;
handle_ra_event(Leader, {machine, {delivery, _ConsumerTag, _} = Del}, State0) ->
handle_delivery(Leader, Del, State0);
handle_ra_event(_From, {rejected, {not_leader, undefined, _Seq}}, State0) ->
% TODO: how should these be handled? re-sent on timer or try random
{internal, [], [], State0};
handle_ra_event(_From, {rejected, {not_leader, Leader, Seq}}, State0) ->
State1 = State0#state{leader = Leader},
State = resend(Seq, State1),
{internal, [], [], State};
handle_ra_event(_Leader, {machine, eol}, _State0) ->
eol.
%% @doc Attempts to enqueue a message using cast semantics. This provides no
%% guarantees or retries if the message fails to achieve consensus or if the
%% servers sent to happens not to be available. If the message is sent to a
%% follower it will attempt the deliver it to the leader, if known. Else it will
%% drop the messages.
%%
%% NB: only use this for non-critical enqueues where a full rabbit_fifo_client state
%% cannot be maintained.
%%
%% @param CusterId the cluster id.
%% @param Servers the known servers in the cluster.
%% @param Msg the message to enqueue.
%%
%% @returns `ok'
-spec untracked_enqueue([ra_server_id()], term()) ->
ok.
untracked_enqueue([Node | _], Msg) ->
Cmd = {enqueue, undefined, undefined, Msg},
ok = ra:pipeline_command(Node, Cmd),
ok.
%% Internal
try_process_command([Server | Rem], Cmd, State) ->
case ra:process_command(Server, Cmd, 30000) of
{ok, _, Leader} ->
{ok, State#state{leader = Leader}};
Err when length(Rem) =:= 0 ->
Err;
_ ->
try_process_command(Rem, Cmd, State)
end.
seq_applied({Seq, MaybeAction},
{Corrs, Actions0, #state{last_applied = Last} = State0})
when Seq > Last orelse Last =:= undefined ->
State1 = case Last of
undefined -> State0;
_ ->
do_resends(Last+1, Seq-1, State0)
end,
{Actions, State} = maybe_add_action(MaybeAction, Actions0, State1),
case maps:take(Seq, State#state.pending) of
{{undefined, _}, Pending} ->
{Corrs, Actions, State#state{pending = Pending,
last_applied = Seq}};
{{Corr, _}, Pending} ->
{[Corr | Corrs], Actions, State#state{pending = Pending,
last_applied = Seq}};
error ->
% must have already been resent or removed for some other reason
{Corrs, Actions, State}
end;
seq_applied(_Seq, Acc) ->
Acc.
maybe_add_action(ok, Acc, State) ->
{Acc, State};
maybe_add_action({multi, Actions}, Acc0, State0) ->
lists:foldl(fun (Act, {Acc, State}) ->
maybe_add_action(Act, Acc, State)
end, {Acc0, State0}, Actions);
maybe_add_action({send_drained, {Tag, Credit}} = Action, Acc,
#state{consumer_deliveries = CDels} = State) ->
%% add credit to consumer delivery_count
C = maps:get(Tag, CDels),
{[Action | Acc],
State#state{consumer_deliveries =
update_consumer(Tag, C#consumer.last_msg_id,
Credit, C, CDels)}};
maybe_add_action(Action, Acc, State) ->
%% anything else is assumed to be an action
{[Action | Acc], State}.
do_resends(From, To, State) when From =< To ->
?INFO("doing resends From ~w To ~w~n", [From, To]),
lists:foldl(fun resend/2, State, lists:seq(From, To));
do_resends(_, _, State) ->
State.
% resends a command with a new sequence number
resend(OldSeq, #state{pending = Pending0, leader = Leader} = State) ->
case maps:take(OldSeq, Pending0) of
{{Corr, Cmd}, Pending} ->
%% resends aren't subject to flow control here
resend_command(Leader, Corr, Cmd, State#state{pending = Pending});
error ->
State
end.
handle_delivery(Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs} = Del0,
#state{consumer_deliveries = CDels0} = State0) ->
{LastId, _} = lists:last(IdMsgs),
%% TODO: remove potential default allocation
case maps:get(Tag, CDels0, #consumer{last_msg_id = -1}) of
#consumer{last_msg_id = Prev} = C
when FstId =:= Prev+1 ->
{Del0, State0#state{consumer_deliveries =
update_consumer(Tag, LastId, length(IdMsgs), C,
CDels0)}};
#consumer{last_msg_id = Prev} = C
when FstId > Prev+1 ->
Missing = get_missing_deliveries(Leader, Prev+1, FstId-1, Tag),
Del = {delivery, Tag, Missing ++ IdMsgs},
{Del, State0#state{consumer_deliveries =
update_consumer(Tag, LastId,
length(IdMsgs) + length(Missing),
C, CDels0)}};
#consumer{last_msg_id = Prev}
when FstId =< Prev ->
case lists:dropwhile(fun({Id, _}) -> Id =< Prev end, IdMsgs) of
[] ->
{internal, [], [], State0};
IdMsgs2 ->
handle_delivery(Leader, {delivery, Tag, IdMsgs2}, State0)
end;
_ when FstId =:= 0 ->
% the very first delivery
{Del0, State0#state{consumer_deliveries =
update_consumer(Tag, LastId,
length(IdMsgs),
#consumer{last_msg_id = LastId},
CDels0)}}
end.
update_consumer(Tag, LastId, DelCntIncr,
#consumer{delivery_count = D}, Consumers) ->
maps:put(Tag,
#consumer{last_msg_id = LastId,
delivery_count = D + DelCntIncr},
Consumers).
get_missing_deliveries(Leader, From, To, ConsumerTag) ->
ConsumerId = consumer_id(ConsumerTag),
% ?INFO("get_missing_deliveries for ~w from ~b to ~b",
% [ConsumerId, From, To]),
Query = fun (State) ->
rabbit_fifo:get_checked_out(ConsumerId, From, To, State)
end,
{ok, {_, Missing}, _} = ra:local_query(Leader, Query),
Missing.
pick_node(#state{leader = undefined, servers = [N | _]}) ->
N;
pick_node(#state{leader = Leader}) ->
Leader.
% servers sorted by last known leader
sorted_servers(#state{leader = undefined, servers = Servers}) ->
Servers;
sorted_servers(#state{leader = Leader, servers = Servers}) ->
[Leader | lists:delete(Leader, Servers)].
next_seq(#state{next_seq = Seq} = State) ->
{Seq, State#state{next_seq = Seq + 1}}.
next_enqueue_seq(#state{next_enqueue_seq = Seq} = State) ->
{Seq, State#state{next_enqueue_seq = Seq + 1}}.
consumer_id(ConsumerTag) ->
{ConsumerTag, self()}.
send_command(Server, Correlation, Command, Priority,
#state{pending = Pending,
priority = Priority,
soft_limit = SftLmt} = State0) ->
{Seq, State} = next_seq(State0),
ok = ra:pipeline_command(Server, Command, Seq, Priority),
Tag = case maps:size(Pending) >= SftLmt of
true -> slow;
false -> ok
end,
{Tag, State#state{pending = Pending#{Seq => {Correlation, Command}},
priority = Priority,
slow = Tag == slow}};
%% once a low priority command has been sent it's not possible to then
%% send a normal priority command without risking that commands are
%% re-ordered. From an AMQP 0.9.1 point of view this should only affect
%% channels that _both_ publish and consume as the enqueue operation is the
%% only low priority one that is sent.
send_command(Node, Correlation, Command, normal,
#state{priority = low} = State) ->
send_command(Node, Correlation, Command, low, State);
send_command(Node, Correlation, Command, low,
#state{priority = normal} = State) ->
send_command(Node, Correlation, Command, low,
State#state{priority = low}).
resend_command(Node, Correlation, Command,
#state{pending = Pending} = State0) ->
{Seq, State} = next_seq(State0),
ok = ra:pipeline_command(Node, Command, Seq),
State#state{pending = Pending#{Seq => {Correlation, Command}}}.
add_command(_Cid, _Tag, [], Acc) ->
Acc;
add_command(Cid, Tag, MsgIds, Acc) ->
[{Tag, MsgIds, Cid} | Acc].

165
src/rabbit_fifo_index.erl Normal file
View File

@ -0,0 +1,165 @@
-module(rabbit_fifo_index).
-export([
empty/0,
fetch/2,
append/3,
return/3,
delete/2,
size/1,
smallest/1,
next_key_after/2,
map/2
]).
-include_lib("ra/include/ra.hrl").
-compile({no_auto_import, [size/1]}).
-record(state, {data = #{} :: #{integer() => term()},
smallest :: undefined | non_neg_integer(),
largest :: undefined | non_neg_integer()
}).
-opaque state() :: #state{}.
-export_type([state/0]).
-spec empty() -> state().
empty() ->
#state{}.
-spec fetch(integer(), state()) -> undefined | term().
fetch(Key, #state{data = Data}) ->
maps:get(Key, Data, undefined).
% only integer keys are supported
-spec append(integer(), term(), state()) -> state().
append(Key, Value,
#state{data = Data,
smallest = Smallest,
largest = Largest} = State)
when Key > Largest orelse Largest =:= undefined ->
State#state{data = maps:put(Key, Value, Data),
smallest = ra_lib:default(Smallest, Key),
largest = Key}.
-spec return(integer(), term(), state()) -> state().
return(Key, Value, #state{data = Data, smallest = Smallest} = State)
when is_integer(Key) andalso Key < Smallest ->
% TODO: this could potentially result in very large gaps which would
% result in poor performance of smallest/1
% We could try to persist a linked list of "smallests" to make it quicker
% to skip from one to the other - needs measurement
State#state{data = maps:put(Key, Value, Data),
smallest = Key};
return(Key, Value, #state{data = Data} = State)
when is_integer(Key) ->
State#state{data = maps:put(Key, Value, Data)}.
-spec delete(integer(), state()) -> state().
delete(Smallest, #state{data = Data0,
largest = Largest,
smallest = Smallest} = State) ->
Data = maps:remove(Smallest, Data0),
case find_next(Smallest + 1, Largest, Data) of
undefined ->
State#state{data = Data,
smallest = undefined,
largest = undefined};
Next ->
State#state{data = Data, smallest = Next}
end;
delete(Key, #state{data = Data} = State) ->
State#state{data = maps:remove(Key, Data)}.
-spec size(state()) -> non_neg_integer().
size(#state{data = Data}) ->
maps:size(Data).
-spec smallest(state()) -> undefined | {integer(), term()}.
smallest(#state{smallest = undefined}) ->
undefined;
smallest(#state{smallest = Smallest, data = Data}) ->
{Smallest, maps:get(Smallest, Data)}.
-spec next_key_after(non_neg_integer(), state()) -> undefined | integer().
next_key_after(_Idx, #state{smallest = undefined}) ->
% map must be empty
undefined;
next_key_after(Idx, #state{smallest = Smallest,
largest = Largest})
when Idx+1 < Smallest orelse Idx+1 > Largest ->
undefined;
next_key_after(Idx, #state{data = Data} = State) ->
Next = Idx+1,
case maps:is_key(Next, Data) of
true ->
Next;
false ->
next_key_after(Next, State)
end.
-spec map(fun(), state()) -> state().
map(F, #state{data = Data} = State) ->
State#state{data = maps:map(F, Data)}.
%% internal
find_next(Next, Last, _Map) when Next > Last ->
undefined;
find_next(Next, Last, Map) ->
case Map of
#{Next := _} ->
Next;
_ ->
% in degenerate cases the range here could be very large
% and hence this could be very slow
% the typical case should idealy be better
% assuming fifo-ish deletion of entries
find_next(Next+1, Last, Map)
end.
-ifdef(TEST).
-include_lib("eunit/include/eunit.hrl").
append_test() ->
S0 = empty(),
undefined = fetch(99, S0),
undefined = smallest(S0),
0 = size(S0),
S1 = append(1, one, S0),
undefined = fetch(99, S1),
one = fetch(1, S1),
1 = size(S1),
{1, one} = smallest(S1),
S2 = append(2, two, S1),
two = fetch(2, S2),
2 = size(S2),
{1, one} = smallest(S2),
S3 = delete(1, S2),
{2, two} = smallest(S3),
1 = size(S3),
S4 = return(1, one, S3),
one = fetch(1, S4),
2 = size(S4),
{1, one} = smallest(S4),
S5 = delete(2, delete(1, S4)),
undefined = smallest(S5),
0 = size(S0),
ok.
next_after_test() ->
S = append(3, three,
append(2, two,
append(1, one,
empty()))),
1 = next_key_after(0, S),
2 = next_key_after(1, S),
3 = next_key_after(2, S),
undefined = next_key_after(3, S),
undefined = next_key_after(4, S),
ok.
-endif.

View File

@ -190,7 +190,8 @@ on_vhost_up(VHost) ->
QNames0;
(Q = #amqqueue{name = QName,
pid = Pid,
slave_pids = SPids}, QNames0) ->
slave_pids = SPids,
type = classic}, QNames0) ->
%% We don't want to pass in the whole
%% cluster - we don't want a situation
%% where starting one node causes us to
@ -206,7 +207,9 @@ on_vhost_up(VHost) ->
case lists:member(node(), SNodes) of
true -> [QName | QNames0];
false -> QNames0
end
end;
(_, QNames0) ->
QNames0
end, [], rabbit_queue)
end),
[add_mirror(QName, node(), async) || QName <- QNames],
@ -446,12 +449,18 @@ maybe_auto_sync(Q = #amqqueue{pid = QPid}) ->
sync_queue(Q) ->
rabbit_amqqueue:with(
Q, fun(#amqqueue{pid = QPid}) -> rabbit_amqqueue:sync_mirrors(QPid) end).
Q, fun(#amqqueue{pid = QPid, type = classic}) ->
rabbit_amqqueue:sync_mirrors(QPid);
(#amqqueue{type = quorum}) ->
{error, quorum_queue_not_supported}
end).
cancel_sync_queue(Q) ->
rabbit_amqqueue:with(
Q, fun(#amqqueue{pid = QPid}) ->
rabbit_amqqueue:cancel_sync_mirrors(QPid)
Q, fun(#amqqueue{pid = QPid, type = classic}) ->
rabbit_amqqueue:cancel_sync_mirrors(QPid);
(#amqqueue{type = quorum}) ->
{error, quorum_queue_not_supported}
end).
sync_batch_size(#amqqueue{} = Q) ->

View File

@ -29,6 +29,7 @@
is_clustered/0,
on_running_node/1,
is_process_alive/1,
is_registered_process_alive/1,
cluster_nodes/1,
node_type/0,
dir/0,
@ -77,7 +78,8 @@
{'partitions', [{node(), [node()]}]}].
-spec is_clustered() -> boolean().
-spec on_running_node(pid()) -> boolean().
-spec is_process_alive(pid()) -> boolean().
-spec is_process_alive(pid() | {atom(), node()}) -> boolean().
-spec is_registered_process_alive(atom()) -> boolean().
-spec cluster_nodes('all' | 'disc' | 'ram' | 'running') -> [node()].
-spec node_type() -> node_type().
-spec dir() -> file:filename().
@ -434,9 +436,15 @@ on_running_node(Pid) -> lists:member(node(Pid), cluster_nodes(running)).
%% (i.e. not partitioned or some random node).
%%
%% See also rabbit_misc:is_process_alive/1 which does not.
is_process_alive(Pid) ->
is_process_alive(Pid) when is_pid(Pid) ->
on_running_node(Pid) andalso
rpc:call(node(Pid), erlang, is_process_alive, [Pid]) =:= true.
rpc:call(node(Pid), erlang, is_process_alive, [Pid]) =:= true;
is_process_alive({Name, Node}) ->
lists:member(Node, cluster_nodes(running)) andalso
rpc:call(Node, rabbit_mnesia, is_registered_process_alive, [Name]) =:= true.
is_registered_process_alive(Name) ->
is_pid(whereis(Name)).
cluster_nodes(WhichNodes) -> cluster_status(WhichNodes).
@ -933,10 +941,13 @@ is_virgin_node() ->
true;
{ok, []} ->
true;
{ok, [File1, File2]} ->
lists:usort([dir() ++ "/" ++ File1, dir() ++ "/" ++ File2]) =:=
{ok, [File1, File2, File3]} ->
lists:usort([filename:join(dir(), File1),
filename:join(dir(), File2),
filename:join(dir(), File3)]) =:=
lists:usort([rabbit_node_monitor:cluster_status_filename(),
rabbit_node_monitor:running_nodes_filename()]);
rabbit_node_monitor:running_nodes_filename(),
rabbit_node_monitor:quorum_filename()]);
{ok, _} ->
false
end.

View File

@ -35,7 +35,7 @@
connection_info/1, connection_info/2,
connection_info_all/0, connection_info_all/1,
emit_connection_info_all/4, emit_connection_info_local/3,
close_connection/2, force_connection_event_refresh/1, accept_ack/2,
close_connection/2, accept_ack/2,
tcp_host/1]).
%% Used by TCP-based transports, e.g. STOMP adapter
@ -87,7 +87,6 @@
-spec connection_info_all(rabbit_types:info_keys()) ->
[rabbit_types:infos()].
-spec close_connection(pid(), string()) -> 'ok'.
-spec force_connection_event_refresh(reference()) -> 'ok'.
-spec accept_ack(any(), rabbit_net:socket()) -> ok.
-spec on_node_down(node()) -> 'ok'.
@ -163,8 +162,8 @@ boot_tls(NumAcceptors) ->
ensure_ssl() ->
{ok, SslAppsConfig} = application:get_env(rabbit, ssl_apps),
ok = app_utils:start_applications(SslAppsConfig),
{ok, SslOptsConfig} = application:get_env(rabbit, ssl_options),
rabbit_ssl_options:fix(SslOptsConfig).
{ok, SslOptsConfig0} = application:get_env(rabbit, ssl_options),
rabbit_ssl_options:fix(SslOptsConfig0).
poodle_check(Context) ->
{ok, Vsn} = application:get_key(ssl, vsn),
@ -369,10 +368,6 @@ close_connection(Pid, Explanation) ->
ok
end.
force_connection_event_refresh(Ref) ->
[rabbit_reader:force_event_refresh(C, Ref) || C <- connections()],
ok.
accept_ack(Ref, Sock) ->
ok = ranch:accept_ack(Ref),
case tune_buffer_size(Sock) of

View File

@ -24,7 +24,8 @@
-export([start_link/0]).
-export([running_nodes_filename/0,
cluster_status_filename/0, prepare_cluster_status_files/0,
cluster_status_filename/0, quorum_filename/0,
prepare_cluster_status_files/0,
write_cluster_status/1, read_cluster_status/0,
update_cluster_status/0, reset_cluster_status/0]).
-export([notify_node_up/0, notify_joined_cluster/0, notify_left_cluster/1]).
@ -100,7 +101,10 @@ running_nodes_filename() ->
filename:join(rabbit_mnesia:dir(), "nodes_running_at_shutdown").
cluster_status_filename() ->
rabbit_mnesia:dir() ++ "/cluster_nodes.config".
filename:join(rabbit_mnesia:dir(), "cluster_nodes.config").
quorum_filename() ->
filename:join(rabbit_mnesia:dir(), "quorum").
prepare_cluster_status_files() ->
rabbit_mnesia:ensure_mnesia_dir(),

View File

@ -210,8 +210,9 @@ deliver(FetchFun, QName, ConsumersChanged,
deliver_to_consumer(FetchFun, E = {ChPid, Consumer}, QName) ->
C = lookup_ch(ChPid),
case is_ch_blocked(C) of
true -> block_consumer(C, E),
undelivered;
true ->
block_consumer(C, E),
undelivered;
false -> case rabbit_limiter:can_send(C#cr.limiter,
Consumer#consumer.ack_required,
Consumer#consumer.tag) of
@ -330,6 +331,7 @@ activate_limit_fun() ->
end.
credit(IsEmpty, Credit, Drain, ChPid, CTag, State) ->
case lookup_ch(ChPid) of
not_found ->
unchanged;

733
src/rabbit_quorum_queue.erl Normal file
View File

@ -0,0 +1,733 @@
%% The contents of this file are subject to the Mozilla Public License
%% Version 1.1 (the "License"); you may not use this file except in
%% compliance with the License. You may obtain a copy of the License
%% at http://www.mozilla.org/MPL/
%%
%% Software distributed under the License is distributed on an "AS IS"
%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
%% the License for the specific language governing rights and
%% limitations under the License.
%%
%% The Original Code is RabbitMQ.
%%
%% The Initial Developer of the Original Code is GoPivotal, Inc.
%% Copyright (c) 2018 Pivotal Software, Inc. All rights reserved.
%%
-module(rabbit_quorum_queue).
-export([init_state/2, handle_event/2]).
-export([declare/1, recover/1, stop/1, delete/4, delete_immediately/1]).
-export([info/1, info/2, stat/1, infos/1]).
-export([ack/3, reject/4, basic_get/4, basic_consume/9, basic_cancel/4]).
-export([credit/4]).
-export([purge/1]).
-export([stateless_deliver/2, deliver/3]).
-export([dead_letter_publish/5]).
-export([queue_name/1]).
-export([cluster_state/1, status/2]).
-export([cancel_consumer_handler/3, cancel_consumer/3]).
-export([become_leader/2, update_metrics/2]).
-export([rpc_delete_metrics/1]).
-export([format/1]).
-export([open_files/1]).
-export([add_member/3]).
-export([delete_member/3]).
-export([requeue/3]).
-include_lib("rabbit_common/include/rabbit.hrl").
-include_lib("stdlib/include/qlc.hrl").
-type ra_server_id() :: {Name :: atom(), Node :: node()}.
-type msg_id() :: non_neg_integer().
-type qmsg() :: {rabbit_types:r('queue'), pid(), msg_id(), boolean(), rabbit_types:message()}.
-spec handle_event({'ra_event', ra_server_id(), any()}, rabbit_fifo_client:state()) ->
{'internal', Correlators :: [term()], rabbit_fifo_client:state()} |
{rabbit_fifo:client_msg(), rabbit_fifo_client:state()}.
-spec declare(rabbit_types:amqqueue()) -> {'new', rabbit_types:amqqueue(), rabbit_fifo_client:state()}.
-spec recover([rabbit_types:amqqueue()]) -> [rabbit_types:amqqueue() |
{'absent', rabbit_types:amqqueue(), atom()}].
-spec stop(rabbit_types:vhost()) -> 'ok'.
-spec delete(rabbit_types:amqqueue(), boolean(), boolean(), rabbit_types:username()) ->
{'ok', QLen :: non_neg_integer()}.
-spec ack(rabbit_types:ctag(), [msg_id()], rabbit_fifo_client:state()) ->
{'ok', rabbit_fifo_client:state()}.
-spec reject(Confirm :: boolean(), rabbit_types:ctag(), [msg_id()], rabbit_fifo_client:state()) ->
{'ok', rabbit_fifo_client:state()}.
-spec basic_get(rabbit_types:amqqueue(), NoAck :: boolean(), rabbit_types:ctag(),
rabbit_fifo_client:state()) ->
{'ok', 'empty', rabbit_fifo_client:state()} |
{'ok', QLen :: non_neg_integer(), qmsg(), rabbit_fifo_client:state()}.
-spec basic_consume(rabbit_types:amqqueue(), NoAck :: boolean(), ChPid :: pid(),
ConsumerPrefetchCount :: non_neg_integer(), rabbit_types:ctag(),
ExclusiveConsume :: boolean(), Args :: rabbit_framing:amqp_table(),
any(), rabbit_fifo_client:state()) -> {'ok', rabbit_fifo_client:state()}.
-spec basic_cancel(rabbit_types:ctag(), ChPid :: pid(), any(), rabbit_fifo_client:state()) ->
{'ok', rabbit_fifo_client:state()}.
-spec stateless_deliver(ra_server_id(), rabbit_types:delivery()) -> 'ok'.
-spec deliver(Confirm :: boolean(), rabbit_types:delivery(), rabbit_fifo_client:state()) ->
rabbit_fifo_client:state().
-spec info(rabbit_types:amqqueue()) -> rabbit_types:infos().
-spec info(rabbit_types:amqqueue(), rabbit_types:info_keys()) -> rabbit_types:infos().
-spec infos(rabbit_types:r('queue')) -> rabbit_types:infos().
-spec stat(rabbit_types:amqqueue()) -> {'ok', non_neg_integer(), non_neg_integer()}.
-spec cluster_state(Name :: atom()) -> 'down' | 'recovering' | 'running'.
-spec status(rabbit_types:vhost(), Name :: atom()) -> rabbit_types:infos() | {error, term()}.
-define(STATISTICS_KEYS,
[policy,
operator_policy,
effective_policy_definition,
consumers,
memory,
state,
garbage_collection,
leader,
online,
members,
open_files
]).
%%----------------------------------------------------------------------------
-spec init_state(ra_server_id(), rabbit_types:r('queue')) ->
rabbit_fifo_client:state().
init_state({Name, _}, QName) ->
{ok, SoftLimit} = application:get_env(rabbit, quorum_commands_soft_limit),
{ok, #amqqueue{pid = Leader, quorum_nodes = Nodes0}} =
rabbit_amqqueue:lookup(QName),
%% Ensure the leader is listed first
Nodes = [Leader | lists:delete(Leader, Nodes0)],
rabbit_fifo_client:init(QName, Nodes, SoftLimit,
fun() -> credit_flow:block(Name), ok end,
fun() -> credit_flow:unblock(Name), ok end).
handle_event({ra_event, From, Evt}, FState) ->
rabbit_fifo_client:handle_ra_event(From, Evt, FState).
declare(#amqqueue{name = QName,
durable = Durable,
auto_delete = AutoDelete,
arguments = Arguments,
options = Opts} = Q) ->
ActingUser = maps:get(user, Opts, ?UNKNOWN_USER),
check_invalid_arguments(QName, Arguments),
check_auto_delete(Q),
check_exclusive(Q),
check_non_durable(Q),
QuorumSize = get_default_quorum_initial_group_size(Arguments),
RaName = qname_to_rname(QName),
Id = {RaName, node()},
Nodes = select_quorum_nodes(QuorumSize, rabbit_mnesia:cluster_nodes(all)),
NewQ0 = Q#amqqueue{pid = Id,
quorum_nodes = Nodes},
case rabbit_amqqueue:internal_declare(NewQ0, false) of
{created, NewQ} ->
RaMachine = ra_machine(NewQ),
case ra:start_cluster(RaName, RaMachine,
[{RaName, Node} || Node <- Nodes]) of
{ok, _, _} ->
rabbit_event:notify(queue_created,
[{name, QName},
{durable, Durable},
{auto_delete, AutoDelete},
{arguments, Arguments},
{user_who_performed_action, ActingUser}]),
{new, NewQ};
{error, Error} ->
_ = rabbit_amqqueue:internal_delete(QName, ActingUser),
rabbit_misc:protocol_error(internal_error,
"Cannot declare a queue '~s' on node '~s': ~255p",
[rabbit_misc:rs(QName), node(), Error])
end;
{existing, _} = Ex ->
Ex
end.
ra_machine(Q = #amqqueue{name = QName}) ->
{module, rabbit_fifo,
#{dead_letter_handler => dlx_mfa(Q),
cancel_consumer_handler => {?MODULE, cancel_consumer, [QName]},
become_leader_handler => {?MODULE, become_leader, [QName]},
metrics_handler => {?MODULE, update_metrics, [QName]}}}.
cancel_consumer_handler(QName, {ConsumerTag, ChPid}, _Name) ->
Node = node(ChPid),
% QName = queue_name(Name),
case Node == node() of
true -> cancel_consumer(QName, ChPid, ConsumerTag);
false -> rabbit_misc:rpc_call(Node, rabbit_quorum_queue,
cancel_consumer,
[QName, ChPid, ConsumerTag])
end.
cancel_consumer(QName, ChPid, ConsumerTag) ->
rabbit_core_metrics:consumer_deleted(ChPid, ConsumerTag, QName),
rabbit_event:notify(consumer_deleted,
[{consumer_tag, ConsumerTag},
{channel, ChPid},
{queue, QName},
{user_who_performed_action, ?INTERNAL_USER}]).
become_leader(QName, Name) ->
Fun = fun(Q1) ->
Q1#amqqueue{pid = {Name, node()},
state = live}
end,
%% as this function is called synchronously when a ra node becomes leader
%% we need to ensure there is no chance of blocking as else the ra node
%% may not be able to establish it's leadership
spawn(fun() ->
rabbit_misc:execute_mnesia_transaction(
fun() ->
rabbit_amqqueue:update(QName, Fun)
end),
case rabbit_amqqueue:lookup(QName) of
{ok, #amqqueue{quorum_nodes = Nodes}} ->
[rpc:call(Node, ?MODULE, rpc_delete_metrics, [QName])
|| Node <- Nodes, Node =/= node()];
_ ->
ok
end
end).
rpc_delete_metrics(QName) ->
ets:delete(queue_coarse_metrics, QName),
ets:delete(queue_metrics, QName),
ok.
update_metrics(QName, {Name, MR, MU, M, C}) ->
R = reductions(Name),
rabbit_core_metrics:queue_stats(QName, MR, MU, M, R),
Util = case C of
0 -> 0;
_ -> rabbit_fifo:usage(Name)
end,
Infos = [{consumers, C}, {consumer_utilisation, Util} | infos(QName)],
rabbit_core_metrics:queue_stats(QName, Infos),
rabbit_event:notify(queue_stats, Infos ++ [{name, QName},
{messages, M},
{messages_ready, MR},
{messages_unacknowledged, MU},
{reductions, R}]).
reductions(Name) ->
try
{reductions, R} = process_info(whereis(Name), reductions),
R
catch
error:badarg ->
0
end.
recover(Queues) ->
[begin
case ra:restart_server({Name, node()}) of
ok ->
% queue was restarted, good
ok;
{error, Err}
when Err == not_started orelse
Err == name_not_registered ->
% queue was never started on this node
% so needs to be started from scratch.
Machine = ra_machine(Q0),
RaNodes = [{Name, Node} || Node <- Nodes],
case ra:start_server(Name, {Name, node()}, Machine, RaNodes) of
ok -> ok;
Err ->
rabbit_log:warning("recover: Quorum queue ~w could not"
" be started ~w", [Name, Err]),
ok
end;
{error, {already_started, _}} ->
%% this is fine and can happen if a vhost crashes and performs
%% recovery whilst the ra application and servers are still
%% running
ok;
Err ->
%% catch all clause to avoid causing the vhost not to start
rabbit_log:warning("recover: Quorum queue ~w could not be "
"restarted ~w", [Name, Err]),
ok
end,
%% we have to ensure the quorum queue is
%% present in the rabbit_queue table and not just in rabbit_durable_queue
%% So many code paths are dependent on this.
{ok, Q} = rabbit_amqqueue:ensure_rabbit_queue_record_is_initialized(Q0),
Q
end || #amqqueue{pid = {Name, _},
quorum_nodes = Nodes} = Q0 <- Queues].
stop(VHost) ->
_ = [ra:stop_server(Pid) || #amqqueue{pid = Pid} <- find_quorum_queues(VHost)],
ok.
delete(#amqqueue{ type = quorum, pid = {Name, _}, name = QName, quorum_nodes = QNodes},
_IfUnused, _IfEmpty, ActingUser) ->
%% TODO Quorum queue needs to support consumer tracking for IfUnused
Msgs = quorum_messages(Name),
_ = rabbit_amqqueue:internal_delete(QName, ActingUser),
case ra:delete_cluster([{Name, Node} || Node <- QNodes], 120000) of
{ok, {_, LeaderNode} = Leader} ->
MRef = erlang:monitor(process, Leader),
receive
{'DOWN', MRef, process, _, _} ->
ok
end,
rpc:call(LeaderNode, rabbit_core_metrics, queue_deleted, [QName]),
{ok, Msgs};
{error, {no_more_nodes_to_try, Errs}} = Err ->
case lists:all(fun({{error, noproc}, _}) -> true;
(_) -> false
end, Errs) of
true ->
%% If all ra nodes were already down, the delete
%% has succeed
rabbit_core_metrics:queue_deleted(QName),
{ok, Msgs};
false ->
Err
end
end.
delete_immediately({Name, _} = QPid) ->
QName = queue_name(Name),
_ = rabbit_amqqueue:internal_delete(QName, ?INTERNAL_USER),
ok = ra:delete_cluster([QPid]),
rabbit_core_metrics:queue_deleted(QName),
ok.
ack(CTag, MsgIds, FState) ->
rabbit_fifo_client:settle(quorum_ctag(CTag), MsgIds, FState).
reject(true, CTag, MsgIds, FState) ->
rabbit_fifo_client:return(quorum_ctag(CTag), MsgIds, FState);
reject(false, CTag, MsgIds, FState) ->
rabbit_fifo_client:discard(quorum_ctag(CTag), MsgIds, FState).
credit(CTag, Credit, Drain, QState) ->
rabbit_fifo_client:credit(quorum_ctag(CTag), Credit, Drain, QState).
basic_get(#amqqueue{name = QName, pid = {Name, _} = Id, type = quorum}, NoAck,
CTag0, FState0) ->
CTag = quorum_ctag(CTag0),
Settlement = case NoAck of
true ->
settled;
false ->
unsettled
end,
case rabbit_fifo_client:dequeue(CTag, Settlement, FState0) of
{ok, empty, FState} ->
{ok, empty, FState};
{ok, {MsgId, {MsgHeader, Msg}}, FState} ->
IsDelivered = maps:is_key(delivery_count, MsgHeader),
{ok, quorum_messages(Name), {QName, Id, MsgId, IsDelivered, Msg}, FState};
{timeout, _} ->
{error, timeout}
end.
basic_consume(#amqqueue{name = QName, type = quorum}, NoAck, ChPid,
ConsumerPrefetchCount, ConsumerTag, ExclusiveConsume, Args, OkMsg,
QState0) ->
maybe_send_reply(ChPid, OkMsg),
%% A prefetch count of 0 means no limitation, let's make it into something large for ra
Prefetch = case ConsumerPrefetchCount of
0 -> 2000;
Other -> Other
end,
{ok, QState} = rabbit_fifo_client:checkout(quorum_ctag(ConsumerTag),
Prefetch, QState0),
rabbit_core_metrics:consumer_created(ChPid, ConsumerTag, ExclusiveConsume,
not NoAck, QName,
ConsumerPrefetchCount, Args),
{ok, QState}.
basic_cancel(ConsumerTag, ChPid, OkMsg, FState0) ->
maybe_send_reply(ChPid, OkMsg),
rabbit_fifo_client:cancel_checkout(quorum_ctag(ConsumerTag), FState0).
stateless_deliver(ServerId, Delivery) ->
ok = rabbit_fifo_client:untracked_enqueue([ServerId],
Delivery#delivery.message).
deliver(false, Delivery, FState0) ->
rabbit_fifo_client:enqueue(Delivery#delivery.message, FState0);
deliver(true, Delivery, FState0) ->
rabbit_fifo_client:enqueue(Delivery#delivery.msg_seq_no,
Delivery#delivery.message, FState0).
info(Q) ->
info(Q, [name, durable, auto_delete, arguments, pid, state, messages,
messages_ready, messages_unacknowledged]).
infos(QName) ->
case rabbit_amqqueue:lookup(QName) of
{ok, Q} ->
info(Q, ?STATISTICS_KEYS);
{error, not_found} ->
[]
end.
info(Q, Items) ->
[{Item, i(Item, Q)} || Item <- Items].
stat(_Q) ->
{ok, 0, 0}. %% TODO length, consumers count
purge(Node) ->
rabbit_fifo_client:purge(Node).
requeue(ConsumerTag, MsgIds, FState) ->
rabbit_fifo_client:return(quorum_ctag(ConsumerTag), MsgIds, FState).
cluster_state(Name) ->
case whereis(Name) of
undefined -> down;
_ ->
case ets:lookup(ra_state, Name) of
[{_, recover}] -> recovering;
_ -> running
end
end.
status(Vhost, QueueName) ->
%% Handle not found queues
QName = #resource{virtual_host = Vhost, name = QueueName, kind = queue},
RName = qname_to_rname(QName),
case rabbit_amqqueue:lookup(QName) of
{ok, #amqqueue{type = classic}} ->
{error, classic_queue_not_supported};
{ok, #amqqueue{pid = {_, Leader}, quorum_nodes = Nodes}} ->
Info = [{leader, Leader}, {members, Nodes}],
case ets:lookup(ra_state, RName) of
[{_, State}] ->
[{local_state, State} | Info];
[] ->
Info
end;
{error, not_found} = E ->
E
end.
add_member(VHost, Name, Node) ->
QName = #resource{virtual_host = VHost, name = Name, kind = queue},
case rabbit_amqqueue:lookup(QName) of
{ok, #amqqueue{type = classic}} ->
{error, classic_queue_not_supported};
{ok, #amqqueue{quorum_nodes = QNodes} = Q} ->
case lists:member(Node, rabbit_mnesia:cluster_nodes(running)) of
false ->
{error, node_not_running};
true ->
case lists:member(Node, QNodes) of
true ->
{error, already_a_member};
false ->
add_member(Q, Node)
end
end;
{error, not_found} = E ->
E
end.
add_member(#amqqueue{pid = {RaName, _} = ServerRef, name = QName,
quorum_nodes = QNodes} = Q, Node) ->
%% TODO parallel calls might crash this, or add a duplicate in quorum_nodes
ServerId = {RaName, Node},
case ra:start_server(RaName, ServerId, ra_machine(Q),
[{RaName, N} || N <- QNodes]) of
ok ->
case ra:add_member(ServerRef, ServerId) of
{ok, _, Leader} ->
Fun = fun(Q1) ->
Q1#amqqueue{quorum_nodes =
[Node | Q1#amqqueue.quorum_nodes],
pid = Leader}
end,
rabbit_misc:execute_mnesia_transaction(
fun() -> rabbit_amqqueue:update(QName, Fun) end),
ok;
E ->
%% TODO should we stop the ra process here?
E
end;
{error, _} = E ->
E
end.
delete_member(VHost, Name, Node) ->
QName = #resource{virtual_host = VHost, name = Name, kind = queue},
case rabbit_amqqueue:lookup(QName) of
{ok, #amqqueue{type = classic}} ->
{error, classic_queue_not_supported};
{ok, #amqqueue{quorum_nodes = QNodes} = Q} ->
case lists:member(Node, rabbit_mnesia:cluster_nodes(running)) of
false ->
{error, node_not_running};
true ->
case lists:member(Node, QNodes) of
false ->
{error, not_a_member};
true ->
delete_member(Q, Node)
end
end;
{error, not_found} = E ->
E
end.
delete_member(#amqqueue{pid = {RaName, _}, name = QName}, Node) ->
ServerId = {RaName, Node},
case ra:leave_and_delete_server(ServerId) of
ok ->
Fun = fun(Q1) ->
Q1#amqqueue{quorum_nodes =
lists:delete(Node, Q1#amqqueue.quorum_nodes)}
end,
rabbit_misc:execute_mnesia_transaction(
fun() -> rabbit_amqqueue:update(QName, Fun) end),
ok;
E ->
E
end.
%%----------------------------------------------------------------------------
dlx_mfa(#amqqueue{name = Resource} = Q) ->
#resource{virtual_host = VHost} = Resource,
DLX = init_dlx(args_policy_lookup(<<"dead-letter-exchange">>, fun res_arg/2, Q), Q),
DLXRKey = args_policy_lookup(<<"dead-letter-routing-key">>, fun res_arg/2, Q),
{?MODULE, dead_letter_publish, [VHost, DLX, DLXRKey, Q#amqqueue.name]}.
init_dlx(undefined, _Q) ->
undefined;
init_dlx(DLX, #amqqueue{name = QName}) ->
rabbit_misc:r(QName, exchange, DLX).
res_arg(_PolVal, ArgVal) -> ArgVal.
args_policy_lookup(Name, Resolve, Q = #amqqueue{arguments = Args}) ->
AName = <<"x-", Name/binary>>,
case {rabbit_policy:get(Name, Q), rabbit_misc:table_lookup(Args, AName)} of
{undefined, undefined} -> undefined;
{undefined, {_Type, Val}} -> Val;
{Val, undefined} -> Val;
{PolVal, {_Type, ArgVal}} -> Resolve(PolVal, ArgVal)
end.
dead_letter_publish(_, undefined, _, _, _) ->
ok;
dead_letter_publish(VHost, X, RK, QName, ReasonMsgs) ->
rabbit_vhost_dead_letter:publish(VHost, X, RK, QName, ReasonMsgs).
%% TODO escape hack
qname_to_rname(#resource{virtual_host = <<"/">>, name = Name}) ->
erlang:binary_to_atom(<<"%2F_", Name/binary>>, utf8);
qname_to_rname(#resource{virtual_host = VHost, name = Name}) ->
erlang:binary_to_atom(<<VHost/binary, "_", Name/binary>>, utf8).
find_quorum_queues(VHost) ->
Node = node(),
mnesia:async_dirty(
fun () ->
qlc:e(qlc:q([Q || Q = #amqqueue{vhost = VH,
pid = Pid,
type = quorum}
<- mnesia:table(rabbit_durable_queue),
VH =:= VHost,
qnode(Pid) == Node]))
end).
i(name, #amqqueue{name = Name}) -> Name;
i(durable, #amqqueue{durable = Dur}) -> Dur;
i(auto_delete, #amqqueue{auto_delete = AD}) -> AD;
i(arguments, #amqqueue{arguments = Args}) -> Args;
i(pid, #amqqueue{pid = {Name, _}}) -> whereis(Name);
i(messages, #amqqueue{pid = {Name, _}}) ->
quorum_messages(Name);
i(messages_ready, #amqqueue{name = QName}) ->
case ets:lookup(queue_coarse_metrics, QName) of
[{_, MR, _, _, _}] ->
MR;
[] ->
0
end;
i(messages_unacknowledged, #amqqueue{name = QName}) ->
case ets:lookup(queue_coarse_metrics, QName) of
[{_, _, MU, _, _}] ->
MU;
[] ->
0
end;
i(policy, Q) ->
case rabbit_policy:name(Q) of
none -> '';
Policy -> Policy
end;
i(operator_policy, Q) ->
case rabbit_policy:name_op(Q) of
none -> '';
Policy -> Policy
end;
i(effective_policy_definition, Q) ->
case rabbit_policy:effective_definition(Q) of
undefined -> [];
Def -> Def
end;
i(consumers, #amqqueue{name = QName}) ->
case ets:lookup(queue_metrics, QName) of
[{_, M, _}] ->
proplists:get_value(consumers, M, 0);
[] ->
0
end;
i(memory, #amqqueue{pid = {Name, _}}) ->
try
{memory, M} = process_info(whereis(Name), memory),
M
catch
error:badarg ->
0
end;
i(state, #amqqueue{pid = {Name, Node}}) ->
%% Check against the leader or last known leader
case rpc:call(Node, ?MODULE, cluster_state, [Name]) of
{badrpc, _} -> down;
State -> State
end;
i(local_state, #amqqueue{pid = {Name, _}}) ->
case ets:lookup(ra_state, Name) of
[{_, State}] -> State;
_ -> not_member
end;
i(garbage_collection, #amqqueue{pid = {Name, _}}) ->
try
rabbit_misc:get_gc_info(whereis(Name))
catch
error:badarg ->
[]
end;
i(members, #amqqueue{quorum_nodes = Nodes}) ->
Nodes;
i(online, Q) -> online(Q);
i(leader, Q) -> leader(Q);
i(open_files, #amqqueue{pid = {Name, _},
quorum_nodes = Nodes}) ->
{Data, _} = rpc:multicall(Nodes, rabbit_quorum_queue, open_files, [Name]),
lists:flatten(Data);
i(_K, _Q) -> ''.
open_files(Name) ->
case whereis(Name) of
undefined -> {node(), 0};
Pid -> case ets:lookup(ra_open_file_metrics, Pid) of
[] -> {node(), 0};
[{_, Count}] -> {node(), Count}
end
end.
leader(#amqqueue{pid = {Name, Leader}}) ->
case is_process_alive(Name, Leader) of
true -> Leader;
false -> ''
end.
online(#amqqueue{quorum_nodes = Nodes,
pid = {Name, _Leader}}) ->
[Node || Node <- Nodes, is_process_alive(Name, Node)].
format(#amqqueue{quorum_nodes = Nodes} = Q) ->
[{members, Nodes}, {online, online(Q)}, {leader, leader(Q)}].
is_process_alive(Name, Node) ->
erlang:is_pid(rpc:call(Node, erlang, whereis, [Name])).
quorum_messages(QName) ->
case ets:lookup(queue_coarse_metrics, QName) of
[{_, _, _, M, _}] ->
M;
[] ->
0
end.
quorum_ctag(Int) when is_integer(Int) ->
integer_to_binary(Int);
quorum_ctag(Other) ->
Other.
maybe_send_reply(_ChPid, undefined) -> ok;
maybe_send_reply(ChPid, Msg) -> ok = rabbit_channel:send_command(ChPid, Msg).
qnode(QPid) when is_pid(QPid) ->
node(QPid);
qnode({_, Node}) ->
Node.
check_invalid_arguments(QueueName, Args) ->
Keys = [<<"x-expires">>, <<"x-message-ttl">>, <<"x-max-length">>,
<<"x-max-length-bytes">>, <<"x-max-priority">>, <<"x-overflow">>,
<<"x-queue-mode">>],
[case rabbit_misc:table_lookup(Args, Key) of
undefined -> ok;
_TypeVal -> rabbit_misc:protocol_error(
precondition_failed,
"invalid arg '~s' for ~s",
[Key, rabbit_misc:rs(QueueName)])
end || Key <- Keys],
ok.
check_auto_delete(#amqqueue{auto_delete = true, name = Name}) ->
rabbit_misc:protocol_error(
precondition_failed,
"invalid property 'auto-delete' for ~s",
[rabbit_misc:rs(Name)]);
check_auto_delete(_) ->
ok.
check_exclusive(#amqqueue{exclusive_owner = none}) ->
ok;
check_exclusive(#amqqueue{name = Name}) ->
rabbit_misc:protocol_error(
precondition_failed,
"invalid property 'exclusive-owner' for ~s",
[rabbit_misc:rs(Name)]).
check_non_durable(#amqqueue{durable = true}) ->
ok;
check_non_durable(#amqqueue{name = Name,
durable = false}) ->
rabbit_misc:protocol_error(
precondition_failed,
"invalid property 'non-durable' for ~s",
[rabbit_misc:rs(Name)]).
queue_name(RaFifoState) ->
rabbit_fifo_client:cluster_name(RaFifoState).
get_default_quorum_initial_group_size(Arguments) ->
case rabbit_misc:table_lookup(Arguments, <<"x-quorum-initial-group-size">>) of
undefined -> application:get_env(rabbit, default_quorum_initial_group_size);
{_Type, Val} -> Val
end.
select_quorum_nodes(Size, All) when length(All) =< Size ->
All;
select_quorum_nodes(Size, All) ->
Node = node(),
case lists:member(Node, All) of
true ->
select_quorum_nodes(Size - 1, lists:delete(Node, All), [Node]);
false ->
select_quorum_nodes(Size, All, [])
end.
select_quorum_nodes(0, _, Selected) ->
Selected;
select_quorum_nodes(Size, Rest, Selected) ->
S = lists:nth(rand:uniform(length(Rest)), Rest),
select_quorum_nodes(Size - 1, lists:delete(S, Rest), [S | Selected]).

View File

@ -57,7 +57,7 @@
-include("rabbit_framing.hrl").
-include("rabbit.hrl").
-export([start_link/3, info_keys/0, info/1, info/2, force_event_refresh/2,
-export([start_link/3, info_keys/0, info/1, info/2,
shutdown/2]).
-export([system_continue/3, system_terminate/4, system_code_change/4]).
@ -161,7 +161,6 @@
-spec info_keys() -> rabbit_types:info_keys().
-spec info(pid()) -> rabbit_types:infos().
-spec info(pid(), rabbit_types:info_keys()) -> rabbit_types:infos().
-spec force_event_refresh(pid(), reference()) -> 'ok'.
-spec shutdown(pid(), string()) -> 'ok'.
-type resource_alert() :: {WasAlarmSetForNode :: boolean(),
IsThereAnyAlarmsWithSameSourceInTheCluster :: boolean(),
@ -217,9 +216,6 @@ info(Pid, Items) ->
{error, Error} -> throw(Error)
end.
force_event_refresh(Pid, Ref) ->
gen_server:cast(Pid, {force_event_refresh, Ref}).
conserve_resources(Pid, Source, {_, Conserve, _}) ->
Pid ! {conserve_resources, Source, Conserve},
ok.
@ -619,17 +615,6 @@ handle_other({'$gen_call', From, {info, Items}}, State) ->
catch Error -> {error, Error}
end),
State;
handle_other({'$gen_cast', {force_event_refresh, Ref}}, State)
when ?IS_RUNNING(State) ->
rabbit_event:notify(
connection_created,
augment_infos_with_user_provided_connection_name(
[{type, network} | infos(?CREATION_EVENT_KEYS, State)], State),
Ref),
rabbit_event:init_stats_timer(State, #v1.stats_timer);
handle_other({'$gen_cast', {force_event_refresh, _Ref}}, State) ->
%% Ignore, we will emit a created event once we start running.
State;
handle_other(ensure_stats, State) ->
ensure_stats_timer(State);
handle_other(emit_stats, State) ->

View File

@ -60,6 +60,8 @@
-rabbit_upgrade({queue_vhost_field, mnesia, [operator_policies]}).
-rabbit_upgrade({topic_permission, mnesia, []}).
-rabbit_upgrade({queue_options, mnesia, [queue_vhost_field]}).
-rabbit_upgrade({queue_type, mnesia, [queue_options]}).
-rabbit_upgrade({queue_quorum_nodes, mnesia, [queue_type]}).
-rabbit_upgrade({exchange_options, mnesia, [operator_policies]}).
%% -------------------------------------------------------------------
@ -98,6 +100,8 @@
-spec operator_policies() -> 'ok'.
-spec queue_vhost_field() -> 'ok'.
-spec queue_options() -> 'ok'.
-spec queue_type() -> 'ok'.
-spec queue_quorum_nodes() -> 'ok'.
-spec exchange_options() -> 'ok'.
@ -576,6 +580,47 @@ queue_options(Table) ->
sync_slave_pids, recoverable_slaves, policy, operator_policy,
gm_pids, decorators, state, policy_version, slave_pids_pending_shutdown, vhost, options]).
queue_type() ->
ok = queue_type(rabbit_queue),
ok = queue_type(rabbit_durable_queue),
ok.
queue_type(Table) ->
transform(
Table,
fun ({amqqueue, Name, Durable, AutoDelete, ExclusiveOwner, Arguments,
Pid, SlavePids, SyncSlavePids, DSN, Policy, OperatorPolicy, GmPids, Decorators,
State, PolicyVersion, SlavePidsPendingShutdown, VHost, Options}) ->
{amqqueue, Name, Durable, AutoDelete, ExclusiveOwner, Arguments,
Pid, SlavePids, SyncSlavePids, DSN, Policy, OperatorPolicy, GmPids, Decorators,
State, PolicyVersion, SlavePidsPendingShutdown, VHost, Options, classic}
end,
[name, durable, auto_delete, exclusive_owner, arguments, pid, slave_pids,
sync_slave_pids, recoverable_slaves, policy, operator_policy,
gm_pids, decorators, state, policy_version, slave_pids_pending_shutdown, vhost, options,
type]).
queue_quorum_nodes() ->
ok = queue_quorum_nodes(rabbit_queue),
ok = queue_quorum_nodes(rabbit_durable_queue),
ok.
queue_quorum_nodes(Table) ->
transform(
Table,
fun ({amqqueue, Name, Durable, AutoDelete, ExclusiveOwner, Arguments,
Pid, SlavePids, SyncSlavePids, DSN, Policy, OperatorPolicy, GmPids, Decorators,
State, PolicyVersion, SlavePidsPendingShutdown, VHost, Options, Type}) ->
{amqqueue, Name, Durable, AutoDelete, ExclusiveOwner, Arguments,
Pid, SlavePids, SyncSlavePids, DSN, Policy, OperatorPolicy, GmPids, Decorators,
State, PolicyVersion, SlavePidsPendingShutdown, VHost, Options, Type,
undefined}
end,
[name, durable, auto_delete, exclusive_owner, arguments, pid, slave_pids,
sync_slave_pids, recoverable_slaves, policy, operator_policy,
gm_pids, decorators, state, policy_version, slave_pids_pending_shutdown, vhost, options,
type, quorum_nodes]).
%% Prior to 3.6.0, passwords were hashed using MD5, this populates
%% existing records with said default. Users created with 3.6.0+ will
%% have internal_user.hashing_algorithm populated by the internal

View File

@ -488,10 +488,12 @@ start(VHost, DurableQueues) ->
Ref = proplists:get_value(persistent_ref, Terms),
Ref =/= undefined
end],
start_dead_letter_process(VHost),
start_msg_store(VHost, ClientRefs, StartFunState),
{ok, AllTerms}.
stop(VHost) ->
ok = rabbit_vhost_dead_letter:stop(VHost),
ok = stop_msg_store(VHost),
ok = rabbit_queue_index:stop(VHost).
@ -515,6 +517,14 @@ do_start_msg_store(VHost, Type, Refs, StartFunState) ->
exit({error, Error})
end.
start_dead_letter_process(VHost) ->
case rabbit_vhost_dead_letter:start(VHost) of
{ok, _} ->
rabbit_log:info("Started dead letter process for vhost '~s'~n", [VHost]);
Err ->
exit(Err)
end.
abbreviated_type(?TRANSIENT_MSG_STORE) -> transient;
abbreviated_type(?PERSISTENT_MSG_STORE) -> persistent.

View File

@ -212,6 +212,7 @@ delete_storage(VHost) ->
assert_benign(ok, _) -> ok;
assert_benign({ok, _}, _) -> ok;
assert_benign({ok, _, _}, _) -> ok;
assert_benign({error, not_found}, _) -> ok;
assert_benign({error, {absent, Q, _}}, ActingUser) ->
%% Removing the mnesia entries here is safe. If/when the down node

View File

@ -0,0 +1,132 @@
%% The contents of this file are subject to the Mozilla Public License
%% Version 1.1 (the "License"); you may not use this file except in
%% compliance with the License. You may obtain a copy of the License
%% at http://www.mozilla.org/MPL/
%%
%% Software distributed under the License is distributed on an "AS IS"
%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
%% the License for the specific language governing rights and
%% limitations under the License.
%%
%% The Original Code is RabbitMQ.
%%
%% The Initial Developer of the Original Code is GoPivotal, Inc.
%% Copyright (c) 2018 Pivotal Software, Inc. All rights reserved.
%%
-module(rabbit_vhost_dead_letter).
-include("rabbit.hrl").
-behaviour(gen_server).
-export([start/1, start_link/0]).
-export([stop/1]).
-export([publish/5]).
-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,
code_change/3]).
-record(state, {queue_states,
queue_cleanup_timer}).
start(VHost) ->
case rabbit_vhost_sup_sup:get_vhost_sup(VHost) of
{ok, VHostSup} ->
supervisor2:start_child(VHostSup,
{rabbit_vhost_dead_letter,
{rabbit_vhost_dead_letter, start_link, []},
transient, ?WORKER_WAIT, worker, [rabbit_vhost_dead_letter]});
{error, {no_such_vhost, VHost}} = E ->
rabbit_log:error("Failed to start a dead letter process for vhost ~s: vhost no"
" longer exists!", [VHost]),
E
end.
stop(VHost) ->
case rabbit_vhost_sup_sup:get_vhost_sup(VHost) of
{ok, VHostSup} ->
ok = supervisor2:terminate_child(VHostSup, rabbit_vhost_dead_letter),
ok = supervisor2:delete_child(VHostSup, rabbit_vhost_dead_letter);
{error, {no_such_vhost, VHost}} ->
rabbit_log:error("Failed to stop a dead letter process for vhost ~s: "
"vhost no longer exists!", [VHost]),
ok
end.
publish(VHost, X, RK, QName, ReasonMsgs) ->
case vhost_dead_letter_pid(VHost) of
no_pid ->
%% TODO what to do???
ok;
Pid ->
gen_server:cast(Pid, {publish, X, RK, QName, ReasonMsgs})
end.
vhost_dead_letter_pid(VHost) ->
{ok, VHostSup} = rabbit_vhost_sup_sup:get_vhost_sup(VHost),
case supervisor2:find_child(VHostSup, rabbit_vhost_dead_letter) of
[Pid] -> Pid;
[] -> no_pid
end.
start_link() ->
gen_server:start_link(?MODULE, [], []).
init([]) ->
{ok, init_queue_cleanup_timer(#state{queue_states = #{}})}.
handle_call(_Req, _From, State) ->
{reply, ok, State}.
handle_cast({publish, X, RK, QName, ReasonMsgs}, #state{queue_states = QueueStates0} = State)
when is_record(X, exchange) ->
QueueStates = batch_publish(X, RK, QName, ReasonMsgs, QueueStates0),
{noreply, State#state{queue_states = QueueStates}};
handle_cast({publish, DLX, RK, QName, ReasonMsgs}, #state{queue_states = QueueStates0} = State) ->
QueueStates =
case rabbit_exchange:lookup(DLX) of
{ok, X} ->
batch_publish(X, RK, QName, ReasonMsgs, QueueStates0);
{error, not_found} ->
QueueStates0
end,
{noreply, State#state{queue_states = QueueStates}}.
handle_info({ra_event, {Name, _}, _} = Evt,
#state{queue_states = QueueStates} = State0) ->
FState0 = maps:get(Name, QueueStates),
case rabbit_quorum_queue:handle_event(Evt, FState0) of
{_, _, _, FState1} ->
{noreply,
State0#state{queue_states = maps:put(Name, FState1, QueueStates)}};
eol ->
{noreply,
State0#state{queue_states = maps:remove(Name, QueueStates)}}
end;
handle_info(queue_cleanup, State = #state{queue_states = QueueStates0}) ->
QueueStates = maps:filter(fun(Name, _) ->
QName = rabbit_quorum_queue:queue_name(Name),
case rabbit_amqqueue:lookup(QName) of
[] ->
false;
_ ->
true
end
end, QueueStates0),
{noreply, init_queue_cleanup_timer(State#state{queue_states = QueueStates})};
handle_info(_I, State) ->
{noreply, State}.
terminate(_, _) -> ok.
code_change(_, State, _) -> {ok, State}.
batch_publish(X, RK, QName, ReasonMsgs, QueueStates) ->
lists:foldl(fun({Reason, Msg}, Acc) ->
rabbit_dead_letter:publish(Msg, Reason, X, RK, QName, Acc)
end, QueueStates, ReasonMsgs).
init_queue_cleanup_timer(State) ->
{ok, Interval} = application:get_env(rabbit, channel_queue_cleanup_interval),
State#state{queue_cleanup_timer = erlang:send_after(Interval, self(), queue_cleanup)}.

View File

@ -53,7 +53,6 @@ init([]) ->
%% unless the operator opts in.
RestartStrategy = vhost_restart_strategy(),
ets:new(?MODULE, [named_table, public, {keypos, #vhost_sup.vhost}]),
{ok, {{simple_one_for_one, 0, 5},
[{rabbit_vhost, {rabbit_vhost_sup_wrapper, start_link, []},
RestartStrategy, ?SUPERVISOR_WAIT, supervisor,

View File

@ -63,4 +63,4 @@ start_vhost_sup(VHost) ->
{ok, Pid};
Other ->
Other
end.
end.

View File

@ -701,7 +701,9 @@ bq_variable_queue_delete_msg_store_files_callback1(Config) ->
CountMinusOne = Count - 1,
{ok, CountMinusOne, {QName, QPid, _AckTag, false, _Msg}} =
rabbit_amqqueue:basic_get(Q, self(), true, Limiter),
rabbit_amqqueue:basic_get(Q, self(), true, Limiter,
<<"bq_variable_queue_delete_msg_store_files_callback1">>,
#{}),
{ok, CountMinusOne} = rabbit_amqqueue:purge(Q),
%% give the queue a second to receive the close_fds callback msg
@ -737,7 +739,8 @@ bq_queue_recover1(Config) ->
fun (Q1 = #amqqueue { pid = QPid1 }) ->
CountMinusOne = Count - 1,
{ok, CountMinusOne, {QName, QPid1, _AckTag, true, _Msg}} =
rabbit_amqqueue:basic_get(Q1, self(), false, Limiter),
rabbit_amqqueue:basic_get(Q1, self(), false, Limiter,
<<"bq_queue_recover1">>, #{}),
exit(QPid1, shutdown),
VQ1 = variable_queue_init(Q, true),
{{_Msg1, true, _AckTag1}, VQ2} =

View File

@ -29,8 +29,7 @@
delegates_async,
delegates_sync,
queue_cleanup,
declare_on_dead_queue,
refresh_events
declare_on_dead_queue
]).
all() ->
@ -240,34 +239,6 @@ declare_on_dead_queue1(_Config, SecondaryNode) ->
after ?TIMEOUT -> throw(failed_to_create_and_kill_queue)
end.
refresh_events(Config) ->
{I, J} = ?config(test_direction, Config),
From = rabbit_ct_broker_helpers:get_node_config(Config, I, nodename),
To = rabbit_ct_broker_helpers:get_node_config(Config, J, nodename),
rabbit_ct_broker_helpers:add_code_path_to_node(To, ?MODULE),
passed = rabbit_ct_broker_helpers:rpc(Config,
From, ?MODULE, refresh_events1, [Config, To]).
refresh_events1(Config, SecondaryNode) ->
dummy_event_receiver:start(self(), [node(), SecondaryNode],
[channel_created, queue_created]),
{_Writer, Ch} = test_spawn(),
expect_events(pid, Ch, channel_created),
rabbit_channel:shutdown(Ch),
{_Writer2, Ch2} = test_spawn(SecondaryNode),
expect_events(pid, Ch2, channel_created),
rabbit_channel:shutdown(Ch2),
{new, #amqqueue{name = QName} = Q} =
rabbit_amqqueue:declare(queue_name(Config, <<"refresh_events-q">>),
false, false, [], none, <<"acting-user">>),
expect_events(name, QName, queue_created),
rabbit_amqqueue:delete(Q, false, false, <<"acting-user">>),
dummy_event_receiver:stop(),
passed.
make_responder(FMsg) -> make_responder(FMsg, timeout).
make_responder(FMsg, Throw) ->
@ -307,19 +278,6 @@ dead_queue_loop(QueueName, OldPid) ->
Q
end.
expect_events(Tag, Key, Type) ->
expect_event(Tag, Key, Type),
rabbit:force_event_refresh(make_ref()),
expect_event(Tag, Key, Type).
expect_event(Tag, Key, Type) ->
receive #event{type = Type, props = Props} ->
case rabbit_misc:pget(Tag, Props) of
Key -> ok;
_ -> expect_event(Tag, Key, Type)
end
after ?TIMEOUT -> throw({failed_to_receive_event, Type})
end.
test_spawn() ->
{Writer, _Limiter, Ch} = rabbit_ct_broker_helpers:test_channel(),

View File

@ -97,4 +97,4 @@ connection_name() ->
Pid = lists:nth(1, Pids),
{dictionary, Dict} = process_info(Pid, dictionary),
{process_name, {rabbit_reader, ConnectionName}} = lists:keyfind(process_name, 1, Dict),
ConnectionName.
ConnectionName.

1783
test/quorum_queue_SUITE.erl Normal file

File diff suppressed because it is too large Load Diff

624
test/rabbit_fifo_SUITE.erl Normal file
View File

@ -0,0 +1,624 @@
-module(rabbit_fifo_SUITE).
-compile(export_all).
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
all() ->
[
{group, tests}
].
all_tests() ->
[
basics,
return,
rabbit_fifo_returns_correlation,
resends_lost_command,
returns_after_down,
resends_after_lost_applied,
handles_reject_notification,
two_quick_enqueues,
detects_lost_delivery,
dequeue,
discard,
cancel_checkout,
credit,
untracked_enqueue,
flow,
test_queries,
duplicate_delivery,
usage
].
groups() ->
[
{tests, [], all_tests()}
].
init_per_group(_, Config) ->
PrivDir = ?config(priv_dir, Config),
_ = application:load(ra),
ok = application:set_env(ra, data_dir, PrivDir),
application:ensure_all_started(ra),
application:ensure_all_started(lg),
Config.
end_per_group(_, Config) ->
_ = application:stop(ra),
Config.
init_per_testcase(TestCase, Config) ->
ra_server_sup:remove_all(),
ServerName2 = list_to_atom(atom_to_list(TestCase) ++ "2"),
ServerName3 = list_to_atom(atom_to_list(TestCase) ++ "3"),
[
{cluster_name, TestCase},
{uid, atom_to_binary(TestCase, utf8)},
{node_id, {TestCase, node()}},
{uid2, atom_to_binary(ServerName2, utf8)},
{node_id2, {ServerName2, node()}},
{uid3, atom_to_binary(ServerName3, utf8)},
{node_id3, {ServerName3, node()}}
| Config].
basics(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
UId = ?config(uid, Config),
CustomerTag = UId,
ok = start_cluster(ClusterName, [ServerId]),
FState0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
{ok, FState1} = rabbit_fifo_client:checkout(CustomerTag, 1, FState0),
ra_log_wal:force_roll_over(ra_log_wal),
% create segment the segment will trigger a snapshot
timer:sleep(1000),
{ok, FState2} = rabbit_fifo_client:enqueue(one, FState1),
% process ra events
FState3 = process_ra_event(FState2, 250),
FState5 = receive
{ra_event, From, Evt} ->
case rabbit_fifo_client:handle_ra_event(From, Evt, FState3) of
{internal, _AcceptedSeqs, _Actions, _FState4} ->
exit(unexpected_internal_event);
{{delivery, C, [{MsgId, _Msg}]}, FState4} ->
{ok, S} = rabbit_fifo_client:settle(C, [MsgId],
FState4),
S
end
after 5000 ->
exit(await_msg_timeout)
end,
% process settle applied notificaiton
FState5b = process_ra_event(FState5, 250),
_ = ra:stop_server(ServerId),
_ = ra:restart_server(ServerId),
% give time to become leader
timer:sleep(500),
{ok, FState6} = rabbit_fifo_client:enqueue(two, FState5b),
% process applied event
FState6b = process_ra_event(FState6, 250),
receive
{ra_event, Frm, E} ->
case rabbit_fifo_client:handle_ra_event(Frm, E, FState6b) of
{internal, _, _, _FState7} ->
ct:pal("unexpected event ~p~n", [E]),
exit({unexpected_internal_event, E});
{{delivery, Ctag, [{Mid, {_, two}}]}, FState7} ->
{ok, _S} = rabbit_fifo_client:return(Ctag, [Mid], FState7),
ok
end
after 2000 ->
exit(await_msg_timeout)
end,
ra:stop_server(ServerId),
ok.
return(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ServerId2 = ?config(node_id2, Config),
ok = start_cluster(ClusterName, [ServerId, ServerId2]),
F00 = rabbit_fifo_client:init(ClusterName, [ServerId, ServerId2]),
{ok, F0} = rabbit_fifo_client:enqueue(1, msg1, F00),
{ok, F1} = rabbit_fifo_client:enqueue(2, msg2, F0),
{_, _, F2} = process_ra_events(F1, 100),
{ok, {MsgId, _}, F} = rabbit_fifo_client:dequeue(<<"tag">>, unsettled, F2),
{ok, _F2} = rabbit_fifo_client:return(<<"tag">>, [MsgId], F),
ra:stop_server(ServerId),
ok.
rabbit_fifo_returns_correlation(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
{ok, F1} = rabbit_fifo_client:enqueue(corr1, msg1, F0),
receive
{ra_event, Frm, E} ->
case rabbit_fifo_client:handle_ra_event(Frm, E, F1) of
{internal, [corr1], [], _F2} ->
ok;
{Del, _} ->
exit({unexpected, Del})
end
after 2000 ->
exit(await_msg_timeout)
end,
ra:stop_server(ServerId),
ok.
duplicate_delivery(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
{ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, F0),
{ok, F2} = rabbit_fifo_client:enqueue(corr1, msg1, F1),
Fun = fun Loop(S0) ->
receive
{ra_event, Frm, E} = Evt ->
case rabbit_fifo_client:handle_ra_event(Frm, E, S0) of
{internal, [corr1], [], S1} ->
Loop(S1);
{_Del, S1} ->
%% repeat event delivery
self() ! Evt,
%% check that then next received delivery doesn't
%% repeat or crash
receive
{ra_event, F, E1} ->
case rabbit_fifo_client:handle_ra_event(F, E1, S1) of
{internal, [], [], S2} ->
S2
end
end
end
after 2000 ->
exit(await_msg_timeout)
end
end,
Fun(F2),
ra:stop_server(ServerId),
ok.
usage(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
{ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, F0),
{ok, F2} = rabbit_fifo_client:enqueue(corr1, msg1, F1),
{ok, F3} = rabbit_fifo_client:enqueue(corr2, msg2, F2),
{_, _, _} = process_ra_events(F3, 50),
% force tick and usage stats emission
ServerId ! tick_timeout,
timer:sleep(50),
% ct:pal("ets ~w ~w ~w", [ets:tab2list(rabbit_fifo_usage), ServerId, UId]),
Use = rabbit_fifo:usage(element(1, ServerId)),
ct:pal("Use ~w~n", [Use]),
ra:stop_server(ServerId),
?assert(Use > 0.0),
ok.
resends_lost_command(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
ok = meck:new(ra, [passthrough]),
F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
{ok, F1} = rabbit_fifo_client:enqueue(msg1, F0),
% lose the enqueue
meck:expect(ra, pipeline_command, fun (_, _, _) -> ok end),
{ok, F2} = rabbit_fifo_client:enqueue(msg2, F1),
meck:unload(ra),
{ok, F3} = rabbit_fifo_client:enqueue(msg3, F2),
{_, _, F4} = process_ra_events(F3, 500),
{ok, {_, {_, msg1}}, F5} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F4),
{ok, {_, {_, msg2}}, F6} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F5),
{ok, {_, {_, msg3}}, _F7} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F6),
ra:stop_server(ServerId),
ok.
two_quick_enqueues(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
F1 = element(2, rabbit_fifo_client:enqueue(msg1, F0)),
{ok, F2} = rabbit_fifo_client:enqueue(msg2, F1),
_ = process_ra_events(F2, 500),
ra:stop_server(ServerId),
ok.
detects_lost_delivery(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
F000 = rabbit_fifo_client:init(ClusterName, [ServerId]),
{ok, F00} = rabbit_fifo_client:enqueue(msg1, F000),
{_, _, F0} = process_ra_events(F00, 100),
{ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, F0),
{ok, F2} = rabbit_fifo_client:enqueue(msg2, F1),
{ok, F3} = rabbit_fifo_client:enqueue(msg3, F2),
% lose first delivery
receive
{ra_event, _, {machine, {delivery, _, [{_, {_, msg1}}]}}} ->
ok
after 500 ->
exit(await_delivery_timeout)
end,
% assert three deliveries were received
{[_, _, _], _, _} = process_ra_events(F3, 500),
ra:stop_server(ServerId),
ok.
returns_after_down(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
{ok, F1} = rabbit_fifo_client:enqueue(msg1, F0),
{_, _, F2} = process_ra_events(F1, 500),
% start a customer in a separate processes
% that exits after checkout
Self = self(),
_Pid = spawn(fun () ->
F = rabbit_fifo_client:init(ClusterName, [ServerId]),
{ok, _} = rabbit_fifo_client:checkout(<<"tag">>, 10, F),
Self ! checkout_done
end),
receive checkout_done -> ok after 1000 -> exit(checkout_done_timeout) end,
% message should be available for dequeue
{ok, {_, {_, msg1}}, _} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F2),
ra:stop_server(ServerId),
ok.
resends_after_lost_applied(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
{_, _, F1} = process_ra_events(element(2, rabbit_fifo_client:enqueue(msg1, F0)),
500),
{ok, F2} = rabbit_fifo_client:enqueue(msg2, F1),
% lose an applied event
receive
{ra_event, _, {applied, _}} ->
ok
after 500 ->
exit(await_ra_event_timeout)
end,
% send another message
{ok, F3} = rabbit_fifo_client:enqueue(msg3, F2),
{_, _, F4} = process_ra_events(F3, 500),
{ok, {_, {_, msg1}}, F5} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F4),
{ok, {_, {_, msg2}}, F6} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F5),
{ok, {_, {_, msg3}}, _F7} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F6),
ra:stop_server(ServerId),
ok.
handles_reject_notification(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId1 = ?config(node_id, Config),
ServerId2 = ?config(node_id2, Config),
UId1 = ?config(uid, Config),
CId = {UId1, self()},
ok = start_cluster(ClusterName, [ServerId1, ServerId2]),
_ = ra:process_command(ServerId1, {checkout,
{auto, 10, simple_prefetch}, CId}),
% reverse order - should try the first node in the list first
F0 = rabbit_fifo_client:init(ClusterName, [ServerId2, ServerId1]),
{ok, F1} = rabbit_fifo_client:enqueue(one, F0),
timer:sleep(500),
% the applied notification
_F2 = process_ra_event(F1, 250),
ra:stop_server(ServerId1),
ra:stop_server(ServerId2),
ok.
discard(Config) ->
PrivDir = ?config(priv_dir, Config),
ServerId = ?config(node_id, Config),
UId = ?config(uid, Config),
ClusterName = ?config(cluster_name, Config),
Conf = #{cluster_name => ClusterName,
id => ServerId,
uid => UId,
log_init_args => #{data_dir => PrivDir, uid => UId},
initial_member => [],
machine => {module, rabbit_fifo,
#{dead_letter_handler =>
{?MODULE, dead_letter_handler, [self()]}}}},
_ = ra:start_server(Conf),
ok = ra:trigger_election(ServerId),
_ = ra:members(ServerId),
F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
{ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, F0),
{ok, F2} = rabbit_fifo_client:enqueue(msg1, F1),
F3 = discard_next_delivery(F2, 500),
{ok, empty, _F4} = rabbit_fifo_client:dequeue(<<"tag1">>, settled, F3),
receive
{dead_letter, Letters} ->
ct:pal("dead letters ~p~n", [Letters]),
[{_, msg1}] = Letters,
ok
after 500 ->
exit(dead_letter_timeout)
end,
ra:stop_server(ServerId),
ok.
cancel_checkout(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4),
{ok, F1} = rabbit_fifo_client:enqueue(m1, F0),
{ok, F2} = rabbit_fifo_client:checkout(<<"tag">>, 10, F1),
{_, _, F3} = process_ra_events0(F2, [], [], 250, fun (_, S) -> S end),
{ok, F4} = rabbit_fifo_client:cancel_checkout(<<"tag">>, F3),
{ok, {_, {_, m1}}, _} = rabbit_fifo_client:dequeue(<<"d1">>, settled, F4),
ok.
credit(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4),
{ok, F1} = rabbit_fifo_client:enqueue(m1, F0),
{ok, F2} = rabbit_fifo_client:enqueue(m2, F1),
{_, _, F3} = process_ra_events(F2, [], 250),
%% checkout with 0 prefetch
{ok, F4} = rabbit_fifo_client:checkout(<<"tag">>, 0, credited, F3),
%% assert no deliveries
{_, _, F5} = process_ra_events0(F4, [], [], 250,
fun
(D, _) -> error({unexpected_delivery, D})
end),
%% provide some credit
{ok, F6} = rabbit_fifo_client:credit(<<"tag">>, 1, false, F5),
{[{_, {_, m1}}], [{send_credit_reply, _}], F7} =
process_ra_events(F6, [], 250),
%% credit and drain
{ok, F8} = rabbit_fifo_client:credit(<<"tag">>, 4, true, F7),
{[{_, {_, m2}}], [{send_credit_reply, _}, {send_drained, _}], F9} =
process_ra_events(F8, [], 250),
flush(),
%% enqueue another message - at this point the consumer credit should be
%% all used up due to the drain
{ok, F10} = rabbit_fifo_client:enqueue(m3, F9),
%% assert no deliveries
{_, _, F11} = process_ra_events0(F10, [], [], 250,
fun
(D, _) -> error({unexpected_delivery, D})
end),
%% credit again and receive the last message
{ok, F12} = rabbit_fifo_client:credit(<<"tag">>, 10, false, F11),
{[{_, {_, m3}}], _, _} = process_ra_events(F12, [], 250),
ok.
untracked_enqueue(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
ok = rabbit_fifo_client:untracked_enqueue([ServerId], msg1),
timer:sleep(100),
F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
{ok, {_, {_, msg1}}, _} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F0),
ra:stop_server(ServerId),
ok.
flow(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 3),
{ok, F1} = rabbit_fifo_client:enqueue(m1, F0),
{ok, F2} = rabbit_fifo_client:enqueue(m2, F1),
{ok, F3} = rabbit_fifo_client:enqueue(m3, F2),
{slow, F4} = rabbit_fifo_client:enqueue(m4, F3),
{_, _, F5} = process_ra_events(F4, 500),
{ok, _} = rabbit_fifo_client:enqueue(m5, F5),
ra:stop_server(ServerId),
ok.
test_queries(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
P = spawn(fun () ->
F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4),
{ok, F1} = rabbit_fifo_client:enqueue(m1, F0),
{ok, F2} = rabbit_fifo_client:enqueue(m2, F1),
process_ra_events(F2, 100),
receive stop -> ok end
end),
F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4),
{ok, _} = rabbit_fifo_client:checkout(<<"tag">>, 1, F0),
{ok, {_, Ready}, _} = ra:local_query(ServerId,
fun rabbit_fifo:query_messages_ready/1),
?assertEqual(1, maps:size(Ready)),
ct:pal("Ready ~w~n", [Ready]),
{ok, {_, Checked}, _} = ra:local_query(ServerId,
fun rabbit_fifo:query_messages_checked_out/1),
?assertEqual(1, maps:size(Checked)),
ct:pal("Checked ~w~n", [Checked]),
{ok, {_, Processes}, _} = ra:local_query(ServerId,
fun rabbit_fifo:query_processes/1),
ct:pal("Processes ~w~n", [Processes]),
?assertEqual(2, length(Processes)),
P ! stop,
ra:stop_server(ServerId),
ok.
dead_letter_handler(Pid, Msgs) ->
Pid ! {dead_letter, Msgs}.
dequeue(Config) ->
ClusterName = ?config(priv_dir, Config),
ServerId = ?config(node_id, Config),
UId = ?config(uid, Config),
Tag = UId,
ok = start_cluster(ClusterName, [ServerId]),
F1 = rabbit_fifo_client:init(ClusterName, [ServerId]),
{ok, empty, F1b} = rabbit_fifo_client:dequeue(Tag, settled, F1),
{ok, F2_} = rabbit_fifo_client:enqueue(msg1, F1b),
{_, _, F2} = process_ra_events(F2_, 100),
{ok, {0, {_, msg1}}, F3} = rabbit_fifo_client:dequeue(Tag, settled, F2),
{ok, F4_} = rabbit_fifo_client:enqueue(msg2, F3),
{_, _, F4} = process_ra_events(F4_, 100),
{ok, {MsgId, {_, msg2}}, F5} = rabbit_fifo_client:dequeue(Tag, unsettled, F4),
{ok, _F6} = rabbit_fifo_client:settle(Tag, [MsgId], F5),
ra:stop_server(ServerId),
ok.
enq_deq_n(N, F0) ->
enq_deq_n(N, F0, []).
enq_deq_n(0, F0, Acc) ->
{_, _, F} = process_ra_events(F0, 100),
{F, Acc};
enq_deq_n(N, F, Acc) ->
{ok, F1} = rabbit_fifo_client:enqueue(N, F),
{_, _, F2} = process_ra_events(F1, 10),
{ok, {_, {_, Deq}}, F3} = rabbit_fifo_client:dequeue(term_to_binary(N), settled, F2),
{_, _, F4} = process_ra_events(F3, 5),
enq_deq_n(N-1, F4, [Deq | Acc]).
conf(ClusterName, UId, ServerId, _, Peers) ->
#{cluster_name => ClusterName,
id => ServerId,
uid => UId,
log_init_args => #{uid => UId},
initial_members => Peers,
machine => {module, rabbit_fifo, #{}}}.
process_ra_event(State, Wait) ->
receive
{ra_event, From, Evt} ->
% ct:pal("processed ra event ~p~n", [Evt]),
{internal, _, _, S} = rabbit_fifo_client:handle_ra_event(From, Evt, State),
S
after Wait ->
exit(ra_event_timeout)
end.
process_ra_events(State0, Wait) ->
process_ra_events(State0, [], Wait).
process_ra_events(State, Acc, Wait) ->
DeliveryFun = fun ({delivery, Tag, Msgs}, S) ->
MsgIds = [element(1, M) || M <- Msgs],
{ok, S2} = rabbit_fifo_client:settle(Tag, MsgIds, S),
S2
end,
process_ra_events0(State, Acc, [], Wait, DeliveryFun).
process_ra_events0(State0, Acc, Actions0, Wait, DeliveryFun) ->
receive
{ra_event, From, Evt} ->
% ct:pal("ra event ~w~n", [Evt]),
case rabbit_fifo_client:handle_ra_event(From, Evt, State0) of
{internal, _, Actions, State} ->
process_ra_events0(State, Acc, Actions0 ++ Actions,
Wait, DeliveryFun);
{{delivery, _Tag, Msgs} = Del, State1} ->
State = DeliveryFun(Del, State1),
process_ra_events0(State, Acc ++ Msgs, Actions0, Wait, DeliveryFun);
eol ->
eol
end
after Wait ->
{Acc, Actions0, State0}
end.
discard_next_delivery(State0, Wait) ->
receive
{ra_event, From, Evt} ->
case rabbit_fifo_client:handle_ra_event(From, Evt, State0) of
{internal, _, _Actions, State} ->
discard_next_delivery(State, Wait);
{{delivery, Tag, Msgs}, State1} ->
MsgIds = [element(1, M) || M <- Msgs],
ct:pal("discarding ~p", [Msgs]),
{ok, State} = rabbit_fifo_client:discard(Tag, MsgIds,
State1),
State
end
after Wait ->
State0
end.
return_next_delivery(State0, Wait) ->
receive
{ra_event, From, Evt} ->
case rabbit_fifo_client:handle_ra_event(From, Evt, State0) of
{internal, _, _, State} ->
return_next_delivery(State, Wait);
{{delivery, Tag, Msgs}, State1} ->
MsgIds = [element(1, M) || M <- Msgs],
ct:pal("returning ~p", [Msgs]),
{ok, State} = rabbit_fifo_client:return(Tag, MsgIds,
State1),
State
end
after Wait ->
State0
end.
validate_process_down(Name, 0) ->
exit({process_not_down, Name});
validate_process_down(Name, Num) ->
case whereis(Name) of
undefined ->
ok;
_ ->
timer:sleep(100),
validate_process_down(Name, Num-1)
end.
start_cluster(ClusterName, ServerIds, RaFifoConfig) ->
{ok, Started, _} = ra:start_cluster(ClusterName,
{module, rabbit_fifo, RaFifoConfig},
ServerIds),
?assertEqual(length(Started), length(ServerIds)),
ok.
start_cluster(ClusterName, ServerIds) ->
start_cluster(ClusterName, ServerIds, #{}).
flush() ->
receive
Msg ->
ct:pal("flushed: ~w~n", [Msg]),
flush()
after 10 ->
ok
end.