diff --git a/Makefile b/Makefile index 31d10759848f..cc0b26ca0b46 100644 --- a/Makefile +++ b/Makefile @@ -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 diff --git a/scripts/rabbitmq-env b/scripts/rabbitmq-env index ea4d0c88325b..c4d1d9333517 100755 --- a/scripts/rabbitmq-env +++ b/scripts/rabbitmq-env @@ -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 diff --git a/scripts/rabbitmq-queues b/scripts/rabbitmq-queues new file mode 100755 index 000000000000..13a3aca8a63e --- /dev/null +++ b/scripts/rabbitmq-queues @@ -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 "$@" diff --git a/scripts/rabbitmq-queues.bat b/scripts/rabbitmq-queues.bat new file mode 100644 index 000000000000..359faf76f5cc --- /dev/null +++ b/scripts/rabbitmq-queues.bat @@ -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_) +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 diff --git a/scripts/rabbitmq-server b/scripts/rabbitmq-server index 36dca388829a..d66e2517c67f 100755 --- a/scripts/rabbitmq-server +++ b/scripts/rabbitmq-server @@ -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} \ "$@" diff --git a/scripts/rabbitmq-server.bat b/scripts/rabbitmq-server.bat index f29c743b4847..4d12d2f31a64 100644 --- a/scripts/rabbitmq-server.bat +++ b/scripts/rabbitmq-server.bat @@ -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! diff --git a/scripts/rabbitmq-service.bat b/scripts/rabbitmq-service.bat index 8a38a42a23bc..73a91b9a1e5f 100644 --- a/scripts/rabbitmq-service.bat +++ b/scripts/rabbitmq-service.bat @@ -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! diff --git a/src/rabbit.erl b/src/rabbit.erl index 0feb8f3dac6f..3401391b095c 100644 --- a/src/rabbit.erl +++ b/src/rabbit.erl @@ -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 diff --git a/src/rabbit_amqqueue.erl b/src/rabbit_amqqueue.erl index ca4fb1c0194f..7c4386ba57d6 100644 --- a/src/rabbit_amqqueue.erl +++ b/src/rabbit_amqqueue.erl @@ -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]}). - -requeue(QPid, MsgIds, ChPid) -> - delegate:invoke(QPid, {gen_server2, call, [{requeue, MsgIds, ChPid}, 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, 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. -ack(QPid, MsgIds, ChPid) -> - delegate:invoke_no_result(QPid, {gen_server2, cast, [{ack, 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) -> - delegate:invoke_no_result(QPid, {gen_server2, cast, [{reject, Requeue, MsgIds, ChPid}]}). +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}]}). - -credit(#amqqueue{pid = QPid}, ChPid, CTag, Credit, Drain) -> - delegate:invoke_no_result(QPid, {gen_server2, cast, [{credit, ChPid, CTag, Credit, Drain}]}). - -basic_get(#amqqueue{pid = QPid}, ChPid, NoAck, LimiterPid) -> - delegate:invoke(QPid, {gen_server2, call, [{basic_get, ChPid, NoAck, LimiterPid}, infinity]}). + delegate:invoke_no_result(QPids, {gen_server2, cast, + [{activate_limit, ChPid}]}). + +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, 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}, NoAck, ChPid, LimiterPid, +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]}). - -basic_cancel(#amqqueue{pid = QPid}, ChPid, ConsumerTag, OkMsg, ActingUser) -> - delegate:invoke(QPid, {gen_server2, call, - [{basic_cancel, ChPid, ConsumerTag, 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, 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) -> +deliver(Qs, Delivery) -> + deliver(Qs, Delivery, untracked), + ok. + +deliver([], _Delivery, QueueState) -> %% /dev/null optimisation - []; + {[], [], QueueState}; -deliver(Qs, Delivery = #delivery{flow = Flow}) -> - {MPids, SPids} = qpids(Qs), +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. - -qpids([]) -> {[], []}; %% optimisation -qpids([#amqqueue{pid = QPid, slave_pids = SPids}]) -> {[QPid], SPids}; %% opt + 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 = {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). diff --git a/src/rabbit_amqqueue_process.erl b/src/rabbit_amqqueue_process.erl index 334980bf497a..d9f17114013d 100644 --- a/src/rabbit_amqqueue_process.erl +++ b/src/rabbit_amqqueue_process.erl @@ -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); diff --git a/src/rabbit_channel.erl b/src/rabbit_channel.erl index ddcc04e70b8f..a8f3f88c2663 100644 --- a/src/rabbit_channel.erl +++ b/src/rabbit_channel.erl @@ -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). diff --git a/src/rabbit_core_metrics_gc.erl b/src/rabbit_core_metrics_gc.erl index 5435f403eeeb..375fe5196755 100644 --- a/src/rabbit_core_metrics_gc.erl +++ b/src/rabbit_core_metrics_gc.erl @@ -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()), diff --git a/src/rabbit_dead_letter.erl b/src/rabbit_dead_letter.erl index 5ec1da0e3439..06691a29ad80 100644 --- a/src/rabbit_dead_letter.erl +++ b/src/rabbit_dead_letter.erl @@ -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, diff --git a/src/rabbit_direct.erl b/src/rabbit_direct.erl index c3d8e70efd02..9db543e19ac9 100644 --- a/src/rabbit_direct.erl +++ b/src/rabbit_direct.erl @@ -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). diff --git a/src/rabbit_fifo.erl b/src/rabbit_fifo.erl new file mode 100644 index 000000000000..178642dc9fb9 --- /dev/null +++ b/src/rabbit_fifo.erl @@ -0,0 +1,1577 @@ +-module(rabbit_fifo). + +-behaviour(ra_machine). + +-compile(inline_list_funcs). +-compile(inline). + +-include_lib("ra/include/ra.hrl"). + +-export([ + init/1, + apply/4, + state_enter/2, + tick/2, + overview/1, + get_checked_out/4, + %% aux + init_aux/1, + handle_aux/6, + % queries + query_messages_ready/1, + query_messages_checked_out/1, + query_processes/1, + query_ra_indexes/1, + query_consumer_count/1, + usage/1, + + %% misc + dehydrate_state/1 + ]). + +-ifdef(TEST). +-export([ + metrics_handler/1 + ]). +-endif. + +-type raw_msg() :: term(). +%% The raw message. It is opaque to rabbit_fifo. + +-type msg_in_id() :: non_neg_integer(). +% a queue scoped monotonically incrementing integer used to enforce order +% in the unassigned messages map + +-type msg_id() :: non_neg_integer(). +%% A consumer-scoped monotonically incrementing integer included with a +%% {@link delivery/0.}. Used to settle deliveries using +%% {@link rabbit_fifo_client:settle/3.} + +-type msg_seqno() :: non_neg_integer(). +%% A sender process scoped monotonically incrementing integer included +%% in enqueue messages. Used to ensure ordering of messages send from the +%% same process + +-type msg_header() :: #{delivery_count => non_neg_integer()}. +%% The message header map: +%% delivery_count: the number of unsuccessful delivery attempts. +%% A non-zero value indicates a previous attempt. + +-type msg() :: {msg_header(), raw_msg()}. +%% message with a header map. + +-type indexed_msg() :: {ra_index(), msg()}. + +-type delivery_msg() :: {msg_id(), msg()}. +%% A tuple consisting of the message id and the headered message. + +-type consumer_tag() :: binary(). +%% An arbitrary binary tag used to distinguish between different consumers +%% set up by the same process. See: {@link rabbit_fifo_client:checkout/3.} + +-type delivery() :: {delivery, consumer_tag(), [delivery_msg()]}. +%% Represents the delivery of one or more rabbit_fifo messages. + +-type consumer_id() :: {consumer_tag(), pid()}. +%% The entity that receives messages. Uniquely identifies a consumer. + +-type credit_mode() :: simple_prefetch | credited. +%% determines how credit is replenished + +-type checkout_spec() :: {once | auto, Num :: non_neg_integer(), + credit_mode()} | + {dequeue, settled | unsettled} | + cancel. + +-type protocol() :: + {enqueue, Sender :: maybe(pid()), MsgSeq :: maybe(msg_seqno()), + Msg :: raw_msg()} | + {checkout, Spec :: checkout_spec(), Consumer :: consumer_id()} | + {settle, MsgIds :: [msg_id()], Consumer :: consumer_id()} | + {return, MsgIds :: [msg_id()], Consumer :: consumer_id()} | + {discard, MsgIds :: [msg_id()], Consumer :: consumer_id()} | + {credit, + Credit :: non_neg_integer(), + DeliveryCount :: non_neg_integer(), + Drain :: boolean(), + Consumer :: consumer_id()} | + purge. + +-type command() :: protocol() | ra_machine:builtin_command(). +%% all the command types suppored by ra fifo + +-type client_msg() :: delivery(). +%% the messages `rabbit_fifo' can send to consumers. + +-type applied_mfa() :: {module(), atom(), list()}. +% represents a partially applied module call + +-define(SHADOW_COPY_INTERVAL, 4096). +-define(USE_AVG_HALF_LIFE, 10000.0). + +-record(consumer, + {checked_out = #{} :: #{msg_id() => {msg_in_id(), indexed_msg()}}, + next_msg_id = 0 :: msg_id(), % part of snapshot data + %% max number of messages that can be sent + %% decremented for each delivery + credit = 0 : non_neg_integer(), + %% total number of checked out messages - ever + %% incremented for each delivery + delivery_count = 0 :: non_neg_integer(), + %% the mode of how credit is incremented + %% simple_prefetch: credit is re-filled as deliveries are settled + %% or returned. + %% credited: credit can only be changed by receiving a consumer_credit + %% command: `{consumer_credit, ReceiverDeliveryCount, Credit}' + credit_mode = simple_prefetch :: credit_mode(), % part of snapshot data + lifetime = once :: once | auto, + suspected_down = false :: boolean() + }). + +-record(enqueuer, + {next_seqno = 1 :: msg_seqno(), + % out of order enqueues - sorted list + pending = [] :: [{msg_seqno(), ra_index(), raw_msg()}], + suspected_down = false :: boolean() + }). + +-record(state, + {name :: atom(), + shadow_copy_interval = ?SHADOW_COPY_INTERVAL :: non_neg_integer(), + % unassigned messages + messages = #{} :: #{msg_in_id() => indexed_msg()}, + % defines the lowest message in id available in the messages map + % that isn't a return + low_msg_num :: msg_in_id() | undefined, + % defines the next message in id to be added to the messages map + next_msg_num = 1 :: msg_in_id(), + % list of returned msg_in_ids - when checking out it picks from + % this list first before taking low_msg_num + returns = queue:new() :: queue:queue(msg_in_id()), + % a counter of enqueues - used to trigger shadow copy points + enqueue_count = 0 :: non_neg_integer(), + % a map containing all the live processes that have ever enqueued + % a message to this queue as well as a cached value of the smallest + % ra_index of all pending enqueues + enqueuers = #{} :: #{pid() => #enqueuer{}}, + % master index of all enqueue raft indexes including pending + % enqueues + % rabbit_fifo_index can be slow when calculating the smallest + % index when there are large gaps but should be faster than gb_trees + % for normal appending operations - backed by a map + ra_indexes = rabbit_fifo_index:empty() :: rabbit_fifo_index:state(), + % consumers need to reflect consumer state at time of snapshot + % needs to be part of snapshot + consumers = #{} :: #{consumer_id() => #consumer{}}, + % consumers that require further service are queued here + % needs to be part of snapshot + service_queue = queue:new() :: queue:queue(consumer_id()), + dead_letter_handler :: maybe(applied_mfa()), + cancel_consumer_handler :: maybe(applied_mfa()), + become_leader_handler :: maybe(applied_mfa()), + metrics_handler :: maybe(applied_mfa()), + prefix_msg_count = 0 :: non_neg_integer() + }). + +-opaque state() :: #state{}. + +-type config() :: #{name := atom(), + dead_letter_handler => applied_mfa(), + become_leader_handler => applied_mfa(), + cancel_consumer_handler => applied_mfa(), + metrics_handler => applied_mfa(), + shadow_copy_interval => non_neg_integer()}. + +-export_type([protocol/0, + delivery/0, + command/0, + consumer_tag/0, + consumer_id/0, + client_msg/0, + msg/0, + msg_id/0, + msg_seqno/0, + delivery_msg/0, + state/0, + config/0]). + +-spec init(config()) -> {state(), ra_machine:effects()}. +init(#{name := Name} = Conf) -> + DLH = maps:get(dead_letter_handler, Conf, undefined), + CCH = maps:get(cancel_consumer_handler, Conf, undefined), + BLH = maps:get(become_leader_handler, Conf, undefined), + MH = maps:get(metrics_handler, Conf, undefined), + SHI = maps:get(shadow_copy_interval, Conf, ?SHADOW_COPY_INTERVAL), + #state{name = Name, + dead_letter_handler = DLH, + cancel_consumer_handler = CCH, + become_leader_handler = BLH, + metrics_handler = MH, + shadow_copy_interval = SHI}. + + + +% msg_ids are scoped per consumer +% ra_indexes holds all raft indexes for enqueues currently on queue +-spec apply(ra_machine:command_meta_data(), command(), + ra_machine:effects(), state()) -> + {state(), ra_machine:effects(), Reply :: term()}. +apply(#{index := RaftIdx}, {enqueue, From, Seq, RawMsg}, Effects0, State00) -> + case maybe_enqueue(RaftIdx, From, Seq, RawMsg, Effects0, State00) of + {ok, State0, Effects} -> + State = append_to_master_index(RaftIdx, State0), + checkout(State, Effects); + {duplicate, State, Effects} -> + {State, Effects, ok} + end; +apply(#{index := RaftIdx}, {settle, MsgIds, ConsumerId}, Effects0, + #state{consumers = Cons0} = State) -> + case Cons0 of + #{ConsumerId := Con0} -> + % need to increment metrics before completing as any snapshot + % states taken need to includ them + complete_and_checkout(RaftIdx, MsgIds, ConsumerId, + Con0, Effects0, State); + _ -> + {State, Effects0, ok} + end; +apply(#{index := RaftIdx}, {discard, MsgIds, ConsumerId}, Effects0, + #state{consumers = Cons0} = State0) -> + case Cons0 of + #{ConsumerId := Con0} -> + {State, Effects, Res} = complete_and_checkout(RaftIdx, MsgIds, + ConsumerId, Con0, + Effects0, State0), + Discarded = maps:with(MsgIds, Con0#consumer.checked_out), + {State, dead_letter_effects(Discarded, State, Effects), Res}; + _ -> + {State0, Effects0, ok} + end; +apply(_, {return, MsgIds, ConsumerId}, Effects0, + #state{consumers = Cons0} = State) -> + case Cons0 of + #{ConsumerId := Con0 = #consumer{checked_out = Checked0}} -> + Checked = maps:without(MsgIds, Checked0), + Returned = maps:with(MsgIds, Checked0), + MsgNumMsgs = [M || M <- maps:values(Returned)], + return(ConsumerId, MsgNumMsgs, Con0, Checked, Effects0, State); + _ -> + {State, Effects0, ok} + end; +apply(_, {credit, NewCredit, RemoteDelCnt, Drain, ConsumerId}, Effects0, + #state{consumers = Cons0, + service_queue = ServiceQueue0} = State0) -> + case Cons0 of + #{ConsumerId := #consumer{delivery_count = DelCnt} = Con0} -> + %% this can go below 0 when credit is reduced + C = max(0, RemoteDelCnt + NewCredit - DelCnt), + %% grant the credit + Con1 = Con0#consumer{credit = C}, + ServiceQueue = maybe_queue_consumer(ConsumerId, Con1, + ServiceQueue0), + Cons = maps:put(ConsumerId, Con1, Cons0), + {State1, Effects, ok} = + checkout(State0#state{service_queue = ServiceQueue, + consumers = Cons}, Effects0), + Response = {send_credit_reply, maps:size(State1#state.messages)}, + %% by this point all checkouts for the updated credit value + %% should be processed so we can evaluate the drain + case Drain of + false -> + %% just return the result of the checkout + {State1, Effects, Response}; + true -> + Con = #consumer{credit = PostCred} = + maps:get(ConsumerId, State1#state.consumers), + %% add the outstanding credit to the delivery count + DeliveryCount = Con#consumer.delivery_count + PostCred, + Consumers = maps:put(ConsumerId, + Con#consumer{delivery_count = DeliveryCount, + credit = 0}, + State1#state.consumers), + Drained = Con#consumer.credit, + {CTag, _} = ConsumerId, + {State1#state{consumers = Consumers}, + Effects, + %% returning a multi response with two client actions + %% for the channel to execute + {multi, [Response, {send_drained, [{CTag, Drained}]}]}} + end; + _ -> + %% credit for unknown consumer - just ignore + {State0, Effects0, ok} + end; +apply(_, {checkout, {dequeue, _}, {_Tag, _Pid}}, Effects0, + #state{messages = M, + prefix_msg_count = 0} = State0) when map_size(M) == 0 -> + %% TODO do we need metric visibility of empty get requests? + {State0, Effects0, {dequeue, empty}}; +apply(Meta, {checkout, {dequeue, settled}, ConsumerId}, + Effects0, State0) -> + % TODO: this clause could probably be optimised + State1 = update_consumer(ConsumerId, {once, 1, simple_prefetch}, State0), + % turn send msg effect into reply + {success, _, MsgId, Msg, State2} = checkout_one(State1), + % immediately settle + {State, Effects, _} = apply(Meta, {settle, [MsgId], ConsumerId}, + Effects0, State2), + {State, Effects, {dequeue, {MsgId, Msg}}}; +apply(_, {checkout, {dequeue, unsettled}, {_Tag, Pid} = Consumer}, + Effects0, State0) -> + State1 = update_consumer(Consumer, {once, 1, simple_prefetch}, State0), + Effects1 = [{monitor, process, Pid} | Effects0], + {State, Reply, Effects} = case checkout_one(State1) of + {success, _, MsgId, Msg, S} -> + {S, {MsgId, Msg}, Effects1}; + {inactive, S} -> + {S, empty, [{aux, inactive} | Effects1]}; + S -> + {S, empty, Effects1} + end, + {State, Effects, {dequeue, Reply}}; +apply(_, {checkout, cancel, ConsumerId}, Effects0, State0) -> + {CancelEffects, State1} = cancel_consumer(ConsumerId, {Effects0, State0}), + % TODO: here we should really demonitor the pid but _only_ if it has no + % other consumers or enqueuers. + checkout(State1, CancelEffects); +apply(_, {checkout, Spec, {_Tag, Pid} = ConsumerId}, Effects0, State0) -> + State1 = update_consumer(ConsumerId, Spec, State0), + {State, Effects, Res} = checkout(State1, Effects0), + {State, [{monitor, process, Pid} | Effects], Res}; +apply(#{index := RaftIdx}, purge, Effects0, + #state{consumers = Cons0, ra_indexes = Indexes } = State0) -> + Total = rabbit_fifo_index:size(Indexes), + {State1, Effects1, _} = + maps:fold( + fun(ConsumerId, C = #consumer{checked_out = Checked0}, + {StateAcc0, EffectsAcc0, ok}) -> + MsgRaftIdxs = [RIdx || {_MsgInId, {RIdx, _}} + <- maps:values(Checked0)], + complete(ConsumerId, MsgRaftIdxs, C, + #{}, EffectsAcc0, StateAcc0) + end, {State0, Effects0, ok}, Cons0), + {State, Effects, _} = + update_smallest_raft_index( + RaftIdx, Indexes, + State1#state{ra_indexes = rabbit_fifo_index:empty(), + messages = #{}, + returns = queue:new(), + low_msg_num = undefined}, Effects1), + {State, [garbage_collection | Effects], {purge, Total}}; +apply(_, {down, ConsumerPid, noconnection}, + Effects0, #state{consumers = Cons0, + enqueuers = Enqs0} = State0) -> + Node = node(ConsumerPid), + % mark all consumers and enqueuers as suspect + % and monitor the node + Cons = maps:map(fun({_, P}, C) when node(P) =:= Node -> + C#consumer{suspected_down = true}; + (_, C) -> C + end, Cons0), + Enqs = maps:map(fun(P, E) when node(P) =:= Node -> + E#enqueuer{suspected_down = true}; + (_, E) -> E + end, Enqs0), + Effects = case maps:size(Cons) of + 0 -> + [{aux, inactive}, {monitor, node, Node} | Effects0]; + _ -> + [{monitor, node, Node} | Effects0] + end, + {State0#state{consumers = Cons, enqueuers = Enqs}, Effects, ok}; +apply(_, {down, Pid, _Info}, Effects0, + #state{consumers = Cons0, + enqueuers = Enqs0} = State0) -> + % remove any enqueuer for the same pid + % TODO: if there are any pending enqueuers these should be enqueued + % This should be ok as we won't see any more enqueues from this pid + State1 = case maps:take(Pid, Enqs0) of + {#enqueuer{pending = Pend}, Enqs} -> + lists:foldl(fun ({_, RIdx, RawMsg}, S) -> + enqueue(RIdx, RawMsg, S) + end, State0#state{enqueuers = Enqs}, Pend); + error -> + State0 + end, + % return checked out messages to main queue + % Find the consumers for the down pid + DownConsumers = maps:keys( + maps:filter(fun({_, P}, _) -> P =:= Pid end, Cons0)), + {Effects1, State2} = lists:foldl(fun cancel_consumer/2, {Effects0, State1}, + DownConsumers), + checkout(State2, Effects1); +apply(_, {nodeup, Node}, Effects0, + #state{consumers = Cons0, + enqueuers = Enqs0} = State0) -> + Cons = maps:fold(fun({_, P}, #consumer{suspected_down = true}, Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, _, Acc) -> Acc + end, [], Cons0), + Enqs = maps:fold(fun(P, #enqueuer{suspected_down = true}, Acc) + when node(P) =:= Node -> + [P | Acc]; + (_, _, Acc) -> Acc + end, [], Enqs0), + Monitors = [{monitor, process, P} || P <- Cons ++ Enqs], + % TODO: should we unsuspect these processes here? + % TODO: avoid list concat + {State0, Monitors ++ Effects0, ok}; +apply(_, {nodedown, _Node}, Effects, State) -> + {State, Effects, ok}. + +-spec state_enter(ra_server:ra_state(), state()) -> ra_machine:effects(). +state_enter(leader, #state{consumers = Custs, + name = Name, + become_leader_handler = BLH}) -> + % return effects to monitor all current consumerss + Effects = [{monitor, process, P} || {_, P} <- maps:keys(Custs)], + case BLH of + undefined -> + Effects; + {Mod, Fun, Args} -> + [{mod_call, Mod, Fun, Args ++ [Name]} | Effects] + end; +state_enter(eol, #state{enqueuers = Enqs, consumers = Custs0}) -> + Custs = maps:fold(fun({_, P}, V, S) -> S#{P => V} end, #{}, Custs0), + [{send_msg, P, eol, ra_event} || P <- maps:keys(maps:merge(Enqs, Custs))]; +state_enter(_, _) -> + %% catch all as not handling all states + []. + + +-spec tick(non_neg_integer(), state()) -> ra_machine:effects(). +tick(_Ts, #state{name = Name, + messages = Messages, + ra_indexes = Indexes, + metrics_handler = MH, + consumers = Cons} = State) -> + Metrics = {Name, + maps:size(Messages), % Ready + num_checked_out(State), % checked out + rabbit_fifo_index:size(Indexes), %% Total + maps:size(Cons)}, % Consumers + case MH of + undefined -> + [{aux, emit}]; + {Mod, Fun, Args} -> + [{mod_call, Mod, Fun, Args ++ [Metrics]}, {aux, emit}] + end. + +-spec overview(state()) -> map(). +overview(#state{consumers = Cons, + enqueuers = Enqs, + messages = Messages, + ra_indexes = Indexes} = State) -> + #{type => ?MODULE, + num_consumers => maps:size(Cons), + num_checked_out => num_checked_out(State), + num_enqueuers => maps:size(Enqs), + num_ready_messages => maps:size(Messages), + num_messages => rabbit_fifo_index:size(Indexes)}. + +-spec get_checked_out(consumer_id(), msg_id(), msg_id(), state()) -> + [delivery_msg()]. +get_checked_out(Cid, From, To, #state{consumers = Consumers}) -> + case Consumers of + #{Cid := #consumer{checked_out = Checked}} -> + [{K, snd(snd(maps:get(K, Checked)))} || K <- lists:seq(From, To)]; + _ -> + [] + end. + +init_aux(Name) when is_atom(Name) -> + %% TODO: catch specific exeption throw if table already exists + ok = ra_machine_ets:create_table(rabbit_fifo_usage, + [named_table, set, public, + {write_concurrency, true}]), + Now = erlang:monotonic_time(micro_seconds), + {Name, {inactive, Now, 1, 1.0}}. + +handle_aux(_, cast, Cmd, {Name, Use0}, Log, _) -> + Use = case Cmd of + _ when Cmd == active orelse Cmd == inactive -> + update_use(Use0, Cmd); + emit -> + true = ets:insert(rabbit_fifo_usage, + {Name, utilisation(Use0)}), + Use0 + end, + {no_reply, {Name, Use}, Log}. + +%%% Queries + +query_messages_ready(#state{messages = M}) -> + M. + +query_messages_checked_out(#state{consumers = Consumers}) -> + maps:fold(fun (_, #consumer{checked_out = C}, S) -> + maps:merge(S, maps:from_list(maps:values(C))) + end, #{}, Consumers). + +query_processes(#state{enqueuers = Enqs, consumers = Cons0}) -> + Cons = maps:fold(fun({_, P}, V, S) -> S#{P => V} end, #{}, Cons0), + maps:keys(maps:merge(Enqs, Cons)). + + +query_ra_indexes(#state{ra_indexes = RaIndexes}) -> + RaIndexes. + +query_consumer_count(#state{consumers = Consumers}) -> + maps:size(Consumers). + +%% other + +-spec usage(atom()) -> float(). +usage(Name) when is_atom(Name) -> + case ets:lookup(rabbit_fifo_usage, Name) of + [] -> 0.0; + [{_, Use}] -> Use + end. + +%%% Internal + +update_use({inactive, _, _, _} = CUInfo, inactive) -> + CUInfo; +update_use({active, _, _} = CUInfo, active) -> + CUInfo; +update_use({active, Since, Avg}, inactive) -> + Now = erlang:monotonic_time(micro_seconds), + {inactive, Now, Now - Since, Avg}; +update_use({inactive, Since, Active, Avg}, active) -> + Now = erlang:monotonic_time(micro_seconds), + {active, Now, use_avg(Active, Now - Since, Avg)}. + +utilisation({active, Since, Avg}) -> + use_avg(erlang:monotonic_time(micro_seconds) - Since, 0, Avg); +utilisation({inactive, Since, Active, Avg}) -> + use_avg(Active, erlang:monotonic_time(micro_seconds) - Since, Avg). + +use_avg(0, 0, Avg) -> + Avg; +use_avg(Active, Inactive, Avg) -> + Time = Inactive + Active, + moving_average(Time, ?USE_AVG_HALF_LIFE, Active / Time, Avg). + +moving_average(_Time, _, Next, undefined) -> + Next; +moving_average(Time, HalfLife, Next, Current) -> + Weight = math:exp(Time * math:log(0.5) / HalfLife), + Next * (1 - Weight) + Current * Weight. + +num_checked_out(#state{consumers = Cons}) -> + lists:foldl(fun (#consumer{checked_out = C}, Acc) -> + maps:size(C) + Acc + end, 0, maps:values(Cons)). + +cancel_consumer(ConsumerId, + {Effects0, #state{consumers = C0, name = Name} = S0}) -> + case maps:take(ConsumerId, C0) of + {#consumer{checked_out = Checked0}, Cons} -> + S = maps:fold(fun (_, {MsgNum, Msg}, S) -> + return_one(MsgNum, Msg, S) + end, S0, Checked0), + Effects = cancel_consumer_effects(ConsumerId, Name, S, Effects0), + case maps:size(Cons) of + 0 -> + {[{aux, inactive} | Effects], S#state{consumers = Cons}}; + _ -> + {Effects, S#state{consumers = Cons}} + end; + error -> + % already removed - do nothing + {Effects0, S0} + end. + +incr_enqueue_count(#state{enqueue_count = C, + shadow_copy_interval = C} = State0) -> + % time to stash a dehydrated state version + State = State0#state{enqueue_count = 0}, + {State, dehydrate_state(State)}; +incr_enqueue_count(#state{enqueue_count = C} = State) -> + {State#state{enqueue_count = C + 1}, undefined}. + +enqueue(RaftIdx, RawMsg, #state{messages = Messages, + low_msg_num = LowMsgNum, + next_msg_num = NextMsgNum} = State0) -> + Msg = {RaftIdx, {#{}, RawMsg}}, % indexed message with header map + State0#state{messages = Messages#{NextMsgNum => Msg}, + % this is probably only done to record it when low_msg_num + % is undefined + low_msg_num = min(LowMsgNum, NextMsgNum), + next_msg_num = NextMsgNum + 1}. + +append_to_master_index(RaftIdx, + #state{ra_indexes = Indexes0} = State0) -> + {State, Shadow} = incr_enqueue_count(State0), + Indexes = rabbit_fifo_index:append(RaftIdx, Shadow, Indexes0), + State#state{ra_indexes = Indexes}. + +enqueue_pending(From, + #enqueuer{next_seqno = Next, + pending = [{Next, RaftIdx, RawMsg} | Pending]} = Enq0, + State0) -> + State = enqueue(RaftIdx, RawMsg, State0), + Enq = Enq0#enqueuer{next_seqno = Next + 1, pending = Pending}, + enqueue_pending(From, Enq, State); +enqueue_pending(From, Enq, #state{enqueuers = Enqueuers0} = State) -> + State#state{enqueuers = Enqueuers0#{From => Enq}}. + +maybe_enqueue(RaftIdx, undefined, undefined, RawMsg, Effects, + State0) -> + % direct enqueue without tracking + {ok, enqueue(RaftIdx, RawMsg, State0), Effects}; +maybe_enqueue(RaftIdx, From, MsgSeqNo, RawMsg, Effects0, + #state{enqueuers = Enqueuers0} = State0) -> + case maps:get(From, Enqueuers0, undefined) of + undefined -> + State1 = State0#state{enqueuers = Enqueuers0#{From => #enqueuer{}}}, + {ok, State, Effects} = maybe_enqueue(RaftIdx, From, MsgSeqNo, + RawMsg, Effects0, State1), + {ok, State, [{monitor, process, From} | Effects]}; + #enqueuer{next_seqno = MsgSeqNo} = Enq0 -> + % it is the next expected seqno + State1 = enqueue(RaftIdx, RawMsg, State0), + Enq = Enq0#enqueuer{next_seqno = MsgSeqNo + 1}, + State = enqueue_pending(From, Enq, State1), + {ok, State, Effects0}; + #enqueuer{next_seqno = Next, + pending = Pending0} = Enq0 + when MsgSeqNo > Next -> + % out of order delivery + Pending = [{MsgSeqNo, RaftIdx, RawMsg} | Pending0], + Enq = Enq0#enqueuer{pending = lists:sort(Pending)}, + {ok, State0#state{enqueuers = Enqueuers0#{From => Enq}}, Effects0}; + #enqueuer{next_seqno = Next} when MsgSeqNo =< Next -> + % duplicate delivery - remove the raft index from the ra_indexes + % map as it was added earlier + {duplicate, State0, Effects0} + end. + +snd(T) -> + element(2, T). + +return(ConsumerId, MsgNumMsgs, #consumer{lifetime = Life} = Con0, Checked, + Effects0, #state{consumers = Cons0, service_queue = SQ0} = State0) -> + Con = case Life of + auto -> + Num = length(MsgNumMsgs), + Con0#consumer{checked_out = Checked, + credit = increase_credit(Con0, Num)}; + once -> + Con0#consumer{checked_out = Checked} + end, + {Cons, SQ, Effects} = update_or_remove_sub(ConsumerId, Con, Cons0, + SQ0, Effects0), + State1 = lists:foldl(fun(dummy, #state{prefix_msg_count = MsgCount} = S0) -> + S0#state{prefix_msg_count = MsgCount + 1}; + ({MsgNum, Msg}, S0) -> + return_one(MsgNum, Msg, S0) + end, State0, MsgNumMsgs), + checkout(State1#state{consumers = Cons, + service_queue = SQ}, + Effects). + +% used to processes messages that are finished +complete(ConsumerId, MsgRaftIdxs, + Con0, Checked, Effects0, + #state{consumers = Cons0, service_queue = SQ0, + ra_indexes = Indexes0} = State0) -> + %% credit_mode = simple_prefetch should automatically top-up credit as messages + %% are simple_prefetch or otherwise returned + Con = Con0#consumer{checked_out = Checked, + credit = increase_credit(Con0, length(MsgRaftIdxs))}, + {Cons, SQ, Effects} = update_or_remove_sub(ConsumerId, Con, Cons0, + SQ0, Effects0), + Indexes = lists:foldl(fun rabbit_fifo_index:delete/2, Indexes0, MsgRaftIdxs), + {State0#state{consumers = Cons, + ra_indexes = Indexes, + service_queue = SQ}, Effects, ok}. + +increase_credit(#consumer{lifetime = once, + credit = Credit}, _) -> + %% once consumers cannot increment credit + Credit; +increase_credit(#consumer{lifetime = auto, + credit_mode = credited, + credit = Credit}, _) -> + %% credit_mode: credit also doens't automatically increment credit + Credit; +increase_credit(#consumer{credit = Current}, Credit) -> + Current + Credit. + +complete_and_checkout(IncomingRaftIdx, MsgIds, ConsumerId, + #consumer{checked_out = Checked0} = Con0, + Effects0, #state{ra_indexes = Indexes0} = State0) -> + Checked = maps:without(MsgIds, Checked0), + Discarded = maps:with(MsgIds, Checked0), + MsgRaftIdxs = [RIdx || {_, {RIdx, _}} <- maps:values(Discarded)], + {State1, Effects1, _} = complete(ConsumerId, MsgRaftIdxs, + Con0, Checked, Effects0, State0), + {State, Effects, _} = checkout(State1, Effects1), + % settle metrics are incremented separately + update_smallest_raft_index(IncomingRaftIdx, Indexes0, State, Effects). + +dead_letter_effects(_Discarded, + #state{dead_letter_handler = undefined}, + Effects) -> + Effects; +dead_letter_effects(Discarded, + #state{dead_letter_handler = {Mod, Fun, Args}}, Effects) -> + DeadLetters = maps:fold(fun(_, {_, {_, {_, Msg}}}, + % MsgId, MsgIdID, RaftId, Header + Acc) -> [{rejected, Msg} | Acc] + end, [], Discarded), + [{mod_call, Mod, Fun, Args ++ [DeadLetters]} | Effects]. + +cancel_consumer_effects(_, _, #state{cancel_consumer_handler = undefined}, + Effects) -> + Effects; +cancel_consumer_effects(Pid, Name, + #state{cancel_consumer_handler = {Mod, Fun, Args}}, + Effects) -> + [{mod_call, Mod, Fun, Args ++ [Pid, Name]} | Effects]. + +update_smallest_raft_index(IncomingRaftIdx, OldIndexes, + #state{ra_indexes = Indexes, + messages = Messages} = State, Effects) -> + case rabbit_fifo_index:size(Indexes) of + 0 when map_size(Messages) =:= 0 -> + % there are no messages on queue anymore and no pending enqueues + % we can forward release_cursor all the way until + % the last received command + {State, [{release_cursor, IncomingRaftIdx, State} | Effects], ok}; + _ -> + NewSmallest = rabbit_fifo_index:smallest(Indexes), + % Take the smallest raft index available in the index when starting + % to process this command + case {NewSmallest, rabbit_fifo_index:smallest(OldIndexes)} of + {{Smallest, _}, {Smallest, _}} -> + % smallest has not changed, do not issue release cursor + % effects + {State, Effects, ok}; + {_, {Smallest, Shadow}} when Shadow =/= undefined -> + % ?INFO("RELEASE ~w ~w ~w~n", [IncomingRaftIdx, Smallest, + % Shadow]), + {State, [{release_cursor, Smallest, Shadow} | Effects], ok}; + _ -> % smallest + % no shadow taken for this index, + % no release cursor increase + {State, Effects, ok} + end + end. + +% TODO update message then update messages and returns in single operations +return_one(MsgNum, {RaftId, {Header0, RawMsg}}, + #state{messages = Messages, + returns = Returns} = State0) -> + Header = maps:update_with(delivery_count, + fun (C) -> C+1 end, + 1, Header0), + Msg = {RaftId, {Header, RawMsg}}, + % this should not affect the release cursor in any way + State0#state{messages = maps:put(MsgNum, Msg, Messages), + returns = queue:in(MsgNum, Returns)}. + + +checkout(State, Effects) -> + checkout0(checkout_one(State), Effects, #{}). + +checkout0({success, ConsumerId, MsgId, Msg, State}, Effects, Acc0) -> + DelMsg = {MsgId, Msg}, + Acc = maps:update_with(ConsumerId, + fun (M) -> [DelMsg | M] end, + [DelMsg], Acc0), + checkout0(checkout_one(State), Effects, Acc); +checkout0({inactive, State}, Effects0, Acc) -> + Effects = append_send_msg_effects(Effects0, Acc), + {State, [{aux, inactive} | Effects], ok}; +checkout0(State, Effects0, Acc) -> + Effects = append_send_msg_effects(Effects0, Acc), + {State, Effects, ok}. + +append_send_msg_effects(Effects, AccMap) when map_size(AccMap) == 0 -> + Effects; +append_send_msg_effects(Effects0, AccMap) -> + Effects = maps:fold(fun (C, Msgs, Ef) -> + [send_msg_effect(C, lists:reverse(Msgs)) | Ef] + end, Effects0, AccMap), + [{aux, active} | Effects]. + +next_checkout_message(#state{returns = Returns, + low_msg_num = Low0, + next_msg_num = NextMsgNum} = State) -> + %% use peek rather than out there as the most likely case is an empty + %% queue + case queue:peek(Returns) of + empty -> + case Low0 of + undefined -> + {undefined, State}; + _ -> + case Low0 + 1 of + NextMsgNum -> + %% the map will be empty after this item is removed + {Low0, State#state{low_msg_num = undefined}}; + Low -> + {Low0, State#state{low_msg_num = Low}} + end + end; + {value, Next} -> + {Next, State#state{returns = queue:drop(Returns)}} + end. + +take_next_msg(#state{prefix_msg_count = 0, + messages = Messages0} = State0) -> + {NextMsgInId, State} = next_checkout_message(State0), + %% messages are available + case maps:take(NextMsgInId, Messages0) of + {IdxMsg, Messages} -> + {{NextMsgInId, IdxMsg}, State, Messages, 0}; + error -> + error + end; +take_next_msg(#state{prefix_msg_count = MsgCount, + messages = Messages} = State) -> + {dummy, State, Messages, MsgCount - 1}. + +send_msg_effect({CTag, CPid}, Msgs) -> + {send_msg, CPid, {delivery, CTag, Msgs}, ra_event}. + +checkout_one(#state{service_queue = SQ0, + messages = Messages0, + consumers = Cons0} = InitState) -> + case queue:peek(SQ0) of + {value, ConsumerId} -> + case take_next_msg(InitState) of + {ConsumerMsg, State0, Messages, PrefMsgC} -> + SQ1 = queue:drop(SQ0), + %% there are consumers waiting to be serviced + %% process consumer checkout + case maps:find(ConsumerId, Cons0) of + {ok, #consumer{credit = 0}} -> + %% no credit but was still on queue + %% can happen when draining + %% recurse without consumer on queue + checkout_one(InitState#state{service_queue = SQ1}); + {ok, #consumer{checked_out = Checked0, + next_msg_id = Next, + credit = Credit, + delivery_count = DelCnt} = Con0} -> + Checked = maps:put(Next, ConsumerMsg, Checked0), + Con = Con0#consumer{checked_out = Checked, + next_msg_id = Next + 1, + credit = Credit - 1, + delivery_count = DelCnt + 1}, + {Cons, SQ, []} = % we expect no effects + update_or_remove_sub(ConsumerId, Con, + Cons0, SQ1, []), + State = State0#state{service_queue = SQ, + messages = Messages, + prefix_msg_count = PrefMsgC, + consumers = Cons}, + Msg = case ConsumerMsg of + dummy -> dummy; + {_, {_, M}} -> M + end, + {success, ConsumerId, Next, Msg, State}; + error -> + %% consumer did not exist but was queued, recurse + checkout_one(InitState#state{service_queue = SQ1}) + end; + error -> + InitState + end; + empty -> + case maps:size(Messages0) of + 0 -> InitState; + _ -> {inactive, InitState} + end + end. + + +update_or_remove_sub(ConsumerId, #consumer{lifetime = auto, + credit = 0} = Con, + Cons, ServiceQueue, Effects) -> + {maps:put(ConsumerId, Con, Cons), ServiceQueue, Effects}; +update_or_remove_sub(ConsumerId, #consumer{lifetime = auto} = Con, + Cons, ServiceQueue, Effects) -> + {maps:put(ConsumerId, Con, Cons), + uniq_queue_in(ConsumerId, ServiceQueue), Effects}; +update_or_remove_sub(ConsumerId, #consumer{lifetime = once, + checked_out = Checked, + credit = 0} = Con, + Cons, ServiceQueue, Effects) -> + case maps:size(Checked) of + 0 -> + % we're done with this consumer + {maps:remove(ConsumerId, Cons), ServiceQueue, + [{demonitor, process, ConsumerId} | Effects]}; + _ -> + % there are unsettled items so need to keep around + {maps:put(ConsumerId, Con, Cons), ServiceQueue, Effects} + end; +update_or_remove_sub(ConsumerId, #consumer{lifetime = once} = Con, + Cons, ServiceQueue, Effects) -> + {maps:put(ConsumerId, Con, Cons), + uniq_queue_in(ConsumerId, ServiceQueue), Effects}. + +uniq_queue_in(Key, Queue) -> + % TODO: queue:member could surely be quite expensive, however the practical + % number of unique consumers may not be large enough for it to matter + case queue:member(Key, Queue) of + true -> + Queue; + false -> + queue:in(Key, Queue) + end. + + +update_consumer(ConsumerId, {Life, Credit, Mode}, + #state{consumers = Cons0, + service_queue = ServiceQueue0} = State0) -> + %% TODO: this logic may not be correct for updating a pre-existing consumer + Init = #consumer{lifetime = Life, credit = Credit, credit_mode = Mode}, + Cons = maps:update_with(ConsumerId, + fun(S) -> + %% remove any in-flight messages from + %% the credit update + N = maps:size(S#consumer.checked_out), + C = max(0, Credit - N), + S#consumer{lifetime = Life, + credit = C} + end, Init, Cons0), + ServiceQueue = maybe_queue_consumer(ConsumerId, maps:get(ConsumerId, Cons), + ServiceQueue0), + + State0#state{consumers = Cons, service_queue = ServiceQueue}. + +maybe_queue_consumer(ConsumerId, #consumer{credit = Credit}, + ServiceQueue0) -> + case Credit > 0 of + true -> + % consumerect needs service - check if already on service queue + uniq_queue_in(ConsumerId, ServiceQueue0); + false -> + ServiceQueue0 + end. + + +dehydrate_state(#state{messages = Messages0, + consumers = Consumers, + prefix_msg_count = MsgCount} = State) -> + State#state{messages = #{}, + ra_indexes = rabbit_fifo_index:empty(), + low_msg_num = undefined, + consumers = maps:map(fun (_, C) -> + C#consumer{checked_out = #{}} + end, Consumers), + returns = queue:new(), + prefix_msg_count = maps:size(Messages0) + MsgCount}. + + +-ifdef(TEST). +-include_lib("eunit/include/eunit.hrl"). + +-define(ASSERT_EFF(EfxPat, Effects), + ?ASSERT_EFF(EfxPat, true, Effects)). + +-define(ASSERT_EFF(EfxPat, Guard, Effects), + ?assert(lists:any(fun (EfxPat) when Guard -> true; + (_) -> false + end, Effects))). + +-define(ASSERT_NO_EFF(EfxPat, Effects), + ?assert(not lists:any(fun (EfxPat) -> true; + (_) -> false + end, Effects))). + +-define(assertNoEffect(EfxPat, Effects), + ?assert(not lists:any(fun (EfxPat) -> true; + (_) -> false + end, Effects))). + +test_init(Name) -> + init(#{name => Name, + shadow_copy_interval => 0, + metrics_handler => {?MODULE, metrics_handler, []}}). + +metrics_handler(_) -> + ok. + +enq_enq_checkout_test() -> + Cid = {<<"enq_enq_checkout_test">>, self()}, + {State1, _} = enq(1, 1, first, test_init(test)), + {State2, _} = enq(2, 2, second, State1), + {_State3, Effects, _} = + apply(meta(3), {checkout, {once, 2, simple_prefetch}, Cid}, [], State2), + ?ASSERT_EFF({monitor, _, _}, Effects), + ?ASSERT_EFF({send_msg, _, {delivery, _, _}, _}, Effects), + ok. + +credit_enq_enq_checkout_settled_credit_test() -> + Cid = {?FUNCTION_NAME, self()}, + {State1, _} = enq(1, 1, first, test_init(test)), + {State2, _} = enq(2, 2, second, State1), + {State3, Effects, _} = + apply(meta(3), {checkout, {auto, 1, credited}, Cid}, [], State2), + ?ASSERT_EFF({monitor, _, _}, Effects), + Deliveries = lists:filter(fun ({send_msg, _, {delivery, _, _}, _}) -> true; + (_) -> false + end, Effects), + ?assertEqual(1, length(Deliveries)), + %% settle the delivery this should _not_ result in further messages being + %% delivered + {State4, SettledEffects} = settle(Cid, 4, 1, State3), + ?assertEqual(false, lists:any(fun ({send_msg, _, {delivery, _, _}, _}) -> + true; + (_) -> false + end, SettledEffects)), + %% granting credit (3) should deliver the second msg if the receivers + %% delivery count is (1) + {State5, CreditEffects} = credit(Cid, 5, 1, 1, false, State4), + % ?debugFmt("CreditEffects ~p ~n~p", [CreditEffects, State4]), + ?ASSERT_EFF({send_msg, _, {delivery, _, _}, _}, CreditEffects), + {_State6, FinalEffects} = enq(6, 3, third, State5), + ?assertEqual(false, lists:any(fun ({send_msg, _, {delivery, _, _}, _}) -> + true; + (_) -> false + end, FinalEffects)), + ok. + +credit_with_drained_test() -> + Cid = {?FUNCTION_NAME, self()}, + State0 = test_init(test), + {State1, _, _} = + apply(meta(1), {checkout, {auto, 1, credited}, Cid}, [], State0), + {State2, _} = credit(Cid, 2, 0, 5, false, State1), + {State, DrainedEffs} = credit(Cid, 3, 0, 5, true, State2), + ?assertMatch(#state{consumers = #{Cid := #consumer{credit = 0, + delivery_count = 5}}}, + State), + ?ASSERT_EFF({send_msg, _, {send_drained, [{?FUNCTION_NAME, 5}]}, cast}, + DrainedEffs), + ok. + +credit_and_drain_test() -> + Cid = {?FUNCTION_NAME, self()}, + {State1, _} = enq(1, 1, first, test_init(test)), + {State2, _} = enq(2, 2, second, State1), + %% checkout without any initial credit (like AMQP 1.0 would) + {State3, CheckEffs, _} = + apply(meta(3), {checkout, {auto, 0, credited}, Cid}, [], State2), + + ?ASSERT_NO_EFF({send_msg, _, {delivery, _, _}}, CheckEffs), + {State4, Effects, {send_credit_reply, 0}} = + apply(meta(4), {credit, 4, 0, true, Cid}, [], State3), + ?assertMatch(#state{consumers = #{Cid := #consumer{credit = 0, + delivery_count = 4}}}, + State4), + + ?ASSERT_EFF({send_msg, _, {delivery, _, [{_, {_, first}}, + {_, {_, second}}]}, _}, Effects), + ?ASSERT_EFF({send_msg, _, {send_drained, [{?FUNCTION_NAME, 2}]}, cast}, + Effects), + {_State5, EnqEffs} = enq(5, 2, third, State4), + ?ASSERT_NO_EFF({send_msg, _, {delivery, _, _}}, EnqEffs), + ok. + + + +enq_enq_deq_test() -> + Cid = {?FUNCTION_NAME, self()}, + {State1, _} = enq(1, 1, first, test_init(test)), + {State2, _} = enq(2, 2, second, State1), + % get returns a reply value + {_State3, [{monitor, _, _}], {dequeue, {0, {_, first}}}} = + apply(meta(3), {checkout, {dequeue, unsettled}, Cid}, [], State2), + ok. + +enq_enq_deq_deq_settle_test() -> + Cid = {?FUNCTION_NAME, self()}, + {State1, _} = enq(1, 1, first, test_init(test)), + {State2, _} = enq(2, 2, second, State1), + % get returns a reply value + {State3, [{monitor, _, _}], {dequeue, {0, {_, first}}}} = + apply(meta(3), {checkout, {dequeue, unsettled}, Cid}, [], State2), + {_State4, _Effects4, {dequeue, empty}} = + apply(meta(4), {checkout, {dequeue, unsettled}, Cid}, [], State3), + ok. + +enq_enq_checkout_get_settled_test() -> + Cid = {?FUNCTION_NAME, self()}, + {State1, _} = enq(1, 1, first, test_init(test)), + % get returns a reply value + {_State2, _Effects, {dequeue, {0, {_, first}}}} = + apply(meta(3), {checkout, {dequeue, settled}, Cid}, [], State1), + ok. + +checkout_get_empty_test() -> + Cid = {?FUNCTION_NAME, self()}, + State = test_init(test), + {_State2, [], {dequeue, empty}} = + apply(meta(1), {checkout, {dequeue, unsettled}, Cid}, [], State), + ok. + +untracked_enq_deq_test() -> + Cid = {?FUNCTION_NAME, self()}, + State0 = test_init(test), + {State1, _, _} = apply(meta(1), {enqueue, undefined, undefined, first}, [], State0), + {_State2, _, {dequeue, {0, {_, first}}}} = + apply(meta(3), {checkout, {dequeue, settled}, Cid}, [], State1), + ok. +release_cursor_test() -> + Cid = {?FUNCTION_NAME, self()}, + {State1, _} = enq(1, 1, first, test_init(test)), + {State2, _} = enq(2, 2, second, State1), + {State3, _} = check(Cid, 3, 10, State2), + % no release cursor effect at this point + {State4, _} = settle(Cid, 4, 1, State3), + {_Final, Effects1} = settle(Cid, 5, 0, State4), + % empty queue forwards release cursor all the way + ?ASSERT_EFF({release_cursor, 5, _}, Effects1), + ok. + +checkout_enq_settle_test() -> + Cid = {?FUNCTION_NAME, self()}, + {State1, [{monitor, _, _}]} = check(Cid, 1, test_init(test)), + {State2, Effects0} = enq(2, 1, first, State1), + ?ASSERT_EFF({send_msg, _, + {delivery, ?FUNCTION_NAME, + [{0, {_, first}}]}, _}, + Effects0), + {State3, [_Inactive]} = enq(3, 2, second, State2), + {_, _Effects} = settle(Cid, 4, 0, State3), + % the release cursor is the smallest raft index that does not + % contribute to the state of the application + % ?ASSERT_EFF({release_cursor, 2, _}, Effects), + ok. + +out_of_order_enqueue_test() -> + Cid = {?FUNCTION_NAME, self()}, + {State1, [{monitor, _, _}]} = check_n(Cid, 5, 5, test_init(test)), + {State2, Effects2} = enq(2, 1, first, State1), + ?ASSERT_EFF({send_msg, _, {delivery, _, [{_, {_, first}}]}, _}, Effects2), + % assert monitor was set up + ?ASSERT_EFF({monitor, _, _}, Effects2), + % enqueue seq num 3 and 4 before 2 + {State3, Effects3} = enq(3, 3, third, State2), + ?assertNoEffect({send_msg, _, {delivery, _, _}, _}, Effects3), + {State4, Effects4} = enq(4, 4, fourth, State3), + % assert no further deliveries where made + ?assertNoEffect({send_msg, _, {delivery, _, _}, _}, Effects4), + {_State5, Effects5} = enq(5, 2, second, State4), + % assert two deliveries were now made + ?ASSERT_EFF({send_msg, _, {delivery, _, [{_, {_, second}}, + {_, {_, third}}, + {_, {_, fourth}}]}, _}, + Effects5), + ok. + +out_of_order_first_enqueue_test() -> + Cid = {?FUNCTION_NAME, self()}, + {State1, _} = check_n(Cid, 5, 5, test_init(test)), + {_State2, Effects2} = enq(2, 10, first, State1), + ?ASSERT_EFF({monitor, process, _}, Effects2), + ?assertNoEffect({send_msg, _, {delivery, _, [{_, {_, first}}]}, _}, + Effects2), + ok. + +duplicate_enqueue_test() -> + Cid = {<<"duplicate_enqueue_test">>, self()}, + {State1, [{monitor, _, _}]} = check_n(Cid, 5, 5, test_init(test)), + {State2, Effects2} = enq(2, 1, first, State1), + ?ASSERT_EFF({send_msg, _, {delivery, _, [{_, {_, first}}]}, _}, Effects2), + {_State3, Effects3} = enq(3, 1, first, State2), + ?assertNoEffect({send_msg, _, {delivery, _, [{_, {_, first}}]}, _}, Effects3), + ok. + +return_non_existent_test() -> + Cid = {<<"cid">>, self()}, + {State0, [_, _Inactive]} = enq(1, 1, second, test_init(test)), + % return non-existent + {_State2, [], _} = apply(meta(3), {return, [99], Cid}, [], State0), + ok. + +return_checked_out_test() -> + Cid = {<<"cid">>, self()}, + {State0, [_, _]} = enq(1, 1, first, test_init(test)), + {State1, [_Monitor, {aux, active}, + {send_msg, _, {delivery, _, [{MsgId, _}]}, _}]} = + check(Cid, 2, State0), + % return + {_State2, [_, _], _} = apply(meta(3), {return, [MsgId], Cid}, [], State1), + ok. + +return_auto_checked_out_test() -> + Cid = {<<"cid">>, self()}, + {State00, [_, _]} = enq(1, 1, first, test_init(test)), + {State0, [_]} = enq(2, 2, second, State00), + % it first active then inactive as the consumer took on but cannot take + % any more + {State1, [_Monitor, {aux, inactive}, {aux, active}, + {send_msg, _, {delivery, _, [{MsgId, _}]}, _} | _]} = + check_auto(Cid, 2, State0), + % return should include another delivery + {_State2, Effects, _} = apply(meta(3), {return, [MsgId], Cid}, [], State1), + ?ASSERT_EFF({send_msg, _, + {delivery, _, [{_, {#{delivery_count := 1}, first}}]}, _}, + Effects), + ok. + + +cancelled_checkout_out_test() -> + Cid = {<<"cid">>, self()}, + {State00, [_, _]} = enq(1, 1, first, test_init(test)), + {State0, [_]} = enq(2, 2, second, State00), + {State1, _} = check_auto(Cid, 2, State0), + % cancelled checkout should return all pending messages to queue + {State2, _, _} = apply(meta(3), {checkout, cancel, Cid}, [], State1), + + {State3, _, {dequeue, {0, {_, first}}}} = + apply(meta(3), {checkout, {dequeue, settled}, Cid}, [], State2), + {_State, _, {dequeue, {_, {_, second}}}} = + apply(meta(3), {checkout, {dequeue, settled}, Cid}, [], State3), + ok. + +down_with_noproc_consumer_returns_unsettled_test() -> + Cid = {<<"down_consumer_returns_unsettled_test">>, self()}, + {State0, [_, _]} = enq(1, 1, second, test_init(test)), + {State1, [{monitor, process, Pid} | _]} = check(Cid, 2, State0), + {State2, [_, _], _} = apply(meta(3), {down, Pid, noproc}, [], State1), + {_State, Effects} = check(Cid, 4, State2), + ?ASSERT_EFF({monitor, process, _}, Effects), + ok. + +down_with_noconnection_marks_suspect_and_node_is_monitored_test() -> + Pid = spawn(fun() -> ok end), + Cid = {<<"down_with_noconnect">>, Pid}, + Self = self(), + Node = node(Pid), + {State0, Effects0} = enq(1, 1, second, test_init(test)), + ?ASSERT_EFF({monitor, process, P}, P =:= Self, Effects0), + {State1, Effects1} = check(Cid, 2, State0), + ?ASSERT_EFF({monitor, process, P}, P =:= Pid, Effects1), + % monitor both enqueuer and consumer + % because we received a noconnection we now need to monitor the node + {State2a, _Effects2a, _} = apply(meta(3), {down, Pid, noconnection}, [], State1), + {State2, Effects2, _} = apply(meta(3), {down, Self, noconnection}, [], State2a), + ?ASSERT_EFF({monitor, node, _}, Effects2), + ?assertNoEffect({demonitor, process, _}, Effects2), + % when the node comes up we need to retry the process monitors for the + % disconnected processes + {_State3, Effects3, _} = apply(meta(3), {nodeup, Node}, [], State2), + % try to re-monitor the suspect processes + ?ASSERT_EFF({monitor, process, P}, P =:= Pid, Effects3), + ?ASSERT_EFF({monitor, process, P}, P =:= Self, Effects3), + ok. + +down_with_noproc_enqueuer_is_cleaned_up_test() -> + State00 = test_init(test), + Pid = spawn(fun() -> ok end), + {State0, Effects0, _} = apply(meta(1), {enqueue, Pid, 1, first}, [], State00), + ?ASSERT_EFF({monitor, process, _}, Effects0), + {State1, _Effects1, _} = apply(meta(3), {down, Pid, noproc}, [], State0), + % ensure there are no enqueuers + ?assert(0 =:= maps:size(State1#state.enqueuers)), + ok. + +completed_consumer_yields_demonitor_effect_test() -> + Cid = {<<"completed_consumer_yields_demonitor_effect_test">>, self()}, + {State0, [_, _]} = enq(1, 1, second, test_init(test)), + {State1, [{monitor, process, _} | _]} = check(Cid, 2, State0), + {_, Effects} = settle(Cid, 3, 0, State1), + ?ASSERT_EFF({demonitor, _, _}, Effects), + % release cursor for empty queue + ?ASSERT_EFF({release_cursor, 3, _}, Effects), + ok. + +discarded_message_without_dead_letter_handler_is_removed_test() -> + Cid = {<<"completed_consumer_yields_demonitor_effect_test">>, self()}, + {State0, [_, _]} = enq(1, 1, first, test_init(test)), + {State1, Effects1} = check_n(Cid, 2, 10, State0), + ?ASSERT_EFF({send_msg, _, + {delivery, _, [{0, {#{}, first}}]}, _}, + Effects1), + {_State2, Effects2, _} = apply(meta(1), {discard, [0], Cid}, [], State1), + ?assertNoEffect({send_msg, _, + {delivery, _, [{0, {#{}, first}}]}, _}, + Effects2), + ok. + +discarded_message_with_dead_letter_handler_emits_mod_call_effect_test() -> + Cid = {<<"completed_consumer_yields_demonitor_effect_test">>, self()}, + State00 = init(#{name => test, + dead_letter_handler => + {somemod, somefun, [somearg]}}), + {State0, [_, _]} = enq(1, 1, first, State00), + {State1, Effects1} = check_n(Cid, 2, 10, State0), + ?ASSERT_EFF({send_msg, _, + {delivery, _, [{0, {#{}, first}}]}, _}, + Effects1), + {_State2, Effects2, _} = apply(meta(1), {discard, [0], Cid}, [], State1), + % assert mod call effect with appended reason and message + ?ASSERT_EFF({mod_call, somemod, somefun, [somearg, [{rejected, first}]]}, + Effects2), + ok. + +tick_test() -> + Cid = {<<"c">>, self()}, + Cid2 = {<<"c2">>, self()}, + {S0, _} = enq(1, 1, fst, test_init(test)), + {S1, _} = enq(2, 2, snd, S0), + {S2, {MsgId, _}} = deq(3, Cid, unsettled, S1), + {S3, {_, _}} = deq(4, Cid2, unsettled, S2), + {S4, _, _} = apply(meta(5), {return, [MsgId], Cid}, [], S3), + + [{mod_call, _, _, [{test, 1, 1, 2, 1}]}, {aux, emit}] = tick(1, S4), + ok. + +enq_deq_snapshot_recover_test() -> + Tag = <<"release_cursor_snapshot_state_test">>, + Cid = {Tag, self()}, + % OthPid = spawn(fun () -> ok end), + % Oth = {<<"oth">>, OthPid}, + Commands = [ + {enqueue, self(), 1, one}, + {enqueue, self(), 2, two}, + {checkout, {dequeue, settled}, Cid}, + {enqueue, self(), 3, three}, + {enqueue, self(), 4, four}, + {checkout, {dequeue, settled}, Cid}, + {enqueue, self(), 5, five}, + {checkout, {dequeue, settled}, Cid} + ], + run_snapshot_test(?FUNCTION_NAME, Commands). + +enq_deq_settle_snapshot_recover_test() -> + Tag = atom_to_binary(?FUNCTION_NAME, utf8), + Cid = {Tag, self()}, + % OthPid = spawn(fun () -> ok end), + % Oth = {<<"oth">>, OthPid}, + Commands = [ + {enqueue, self(), 1, one}, + {enqueue, self(), 2, two}, + {checkout, {dequeue, unsettled}, Cid}, + {settle, [0], Cid} + ], + run_snapshot_test(?FUNCTION_NAME, Commands). + +enq_deq_settle_snapshot_recover_2_test() -> + Tag = atom_to_binary(?FUNCTION_NAME, utf8), + Cid = {Tag, self()}, + OthPid = spawn(fun () -> ok end), + Oth = {<<"oth">>, OthPid}, + Commands = [ + {enqueue, self(), 1, one}, + {enqueue, self(), 2, two}, + {checkout, {dequeue, unsettled}, Cid}, + {settle, [0], Cid}, + {enqueue, self(), 3, two}, + {checkout, {dequeue, unsettled}, Oth}, + {settle, [0], Oth} + ], + run_snapshot_test(?FUNCTION_NAME, Commands). + +snapshot_recover_test() -> + Tag = atom_to_binary(?FUNCTION_NAME, utf8), + Cid = {Tag, self()}, + Commands = [ + {checkout, {auto, 2, simple_prefetch}, Cid}, + {enqueue, self(), 1, one}, + {enqueue, self(), 2, two}, + {enqueue, self(), 3, three}, + purge + ], + run_snapshot_test(?FUNCTION_NAME, Commands). + +enq_deq_return_snapshot_recover_test() -> + Tag = atom_to_binary(?FUNCTION_NAME, utf8), + Cid = {Tag, self()}, + OthPid = spawn(fun () -> ok end), + Oth = {<<"oth">>, OthPid}, + Commands = [ + {enqueue, self(), 1, one}, + {enqueue, self(), 2, two}, + {checkout, {dequeue, unsettled}, Oth}, + {checkout, {dequeue, unsettled}, Cid}, + {settle, [0], Oth}, + {return, [0], Cid}, + {enqueue, self(), 3, three}, + purge + ], + run_snapshot_test(?FUNCTION_NAME, Commands). + +enq_check_settle_snapshot_recover_test() -> + Tag = atom_to_binary(?FUNCTION_NAME, utf8), + Cid = {Tag, self()}, + Commands = [ + {checkout, {auto, 2, simple_prefetch}, Cid}, + {enqueue, self(), 1, one}, + {enqueue, self(), 2, two}, + {settle, [1], Cid}, + {settle, [0], Cid}, + {enqueue, self(), 3, three}, + {settle, [2], Cid} + + ], + % ?debugFmt("~w running commands ~w~n", [?FUNCTION_NAME, C]), + run_snapshot_test(?FUNCTION_NAME, Commands). + + +run_snapshot_test(Name, Commands) -> + %% create every incremental permuation of the commands lists + %% and run the snapshot tests against that + [begin + % ?debugFmt("~w running commands ~w~n", [?FUNCTION_NAME, C]), + run_snapshot_test0(Name, C) + end || C <- prefixes(Commands, 1, [])]. + +run_snapshot_test0(Name, Commands) -> + Indexes = lists:seq(1, length(Commands)), + Entries = lists:zip(Indexes, Commands), + {State, Effects} = run_log(test_init(Name), Entries), + + [begin + Filtered = lists:dropwhile(fun({X, _}) when X =< SnapIdx -> true; + (_) -> false + end, Entries), + {S, _} = run_log(SnapState, Filtered), + % assert log can be restored from any release cursor index + % ?debugFmt("Name ~p Idx ~p S~p~nState~p~nSnapState ~p~nFiltered ~p~n", + % [Name, SnapIdx, S, State, SnapState, Filtered]), + ?assertEqual(State, S) + end || {release_cursor, SnapIdx, SnapState} <- Effects], + ok. + +prefixes(Source, N, Acc) when N > length(Source) -> + lists:reverse(Acc); +prefixes(Source, N, Acc) -> + {X, _} = lists:split(N, Source), + prefixes(Source, N+1, [X | Acc]). + +delivery_query_returns_deliveries_test() -> + Tag = <<"release_cursor_snapshot_state_test">>, + Cid = {Tag, self()}, + Commands = [ + {checkout, {auto, 5, simple_prefetch}, Cid}, + {enqueue, self(), 1, one}, + {enqueue, self(), 2, two}, + {enqueue, self(), 3, tre}, + {enqueue, self(), 4, for} + ], + Indexes = lists:seq(1, length(Commands)), + Entries = lists:zip(Indexes, Commands), + {State, _Effects} = run_log(test_init(help), Entries), + % 3 deliveries are returned + [{0, {#{}, one}}] = get_checked_out(Cid, 0, 0, State), + [_, _, _] = get_checked_out(Cid, 1, 3, State), + ok. + +pending_enqueue_is_enqueued_on_down_test() -> + Cid = {<<"cid">>, self()}, + Pid = self(), + {State0, _} = enq(1, 2, first, test_init(test)), + {State1, _, _} = apply(meta(2), {down, Pid, noproc}, [], State0), + {_State2, _, {dequeue, {0, {_, first}}}} = + apply(meta(3), {checkout, {dequeue, settled}, Cid}, [], State1), + ok. + +duplicate_delivery_test() -> + {State0, _} = enq(1, 1, first, test_init(test)), + {#state{ra_indexes = RaIdxs, + messages = Messages}, _} = enq(2, 1, first, State0), + ?assertEqual(1, rabbit_fifo_index:size(RaIdxs)), + ?assertEqual(1, maps:size(Messages)), + ok. + +state_enter_test() -> + S0 = init(#{name => the_name, + become_leader_handler => {m, f, [a]}}), + [{mod_call, m, f, [a, the_name]}] = state_enter(leader, S0), + ok. + +purge_test() -> + Cid = {<<"purge_test">>, self()}, + {State1, _} = enq(1, 1, first, test_init(test)), + {State2, _, {purge, 1}} = apply(meta(2), purge, [], State1), + {State3, _} = enq(3, 2, second, State2), + % get returns a reply value + {_State4, [{monitor, _, _}], {dequeue, {0, {_, second}}}} = + apply(meta(4), {checkout, {dequeue, unsettled}, Cid}, [], State3), + ok. + +purge_with_checkout_test() -> + Cid = {<<"purge_test">>, self()}, + {State0, _} = check_auto(Cid, 1, test_init(?FUNCTION_NAME)), + {State1, _} = enq(2, 1, first, State0), + {State2, _} = enq(3, 2, second, State1), + {State3, _, {purge, 2}} = apply(meta(2), purge, [], State2), + #consumer{checked_out = Checked} = maps:get(Cid, State3#state.consumers), + ?assertEqual(0, maps:size(Checked)), + ok. + +meta(Idx) -> + #{index => Idx, term => 1}. + +enq(Idx, MsgSeq, Msg, State) -> + strip_reply( + apply(meta(Idx), {enqueue, self(), MsgSeq, Msg}, [], State)). + +deq(Idx, Cid, Settlement, State0) -> + {State, _, {dequeue, Msg}} = + apply(meta(Idx), {checkout, {dequeue, Settlement}, Cid}, [], State0), + {State, Msg}. + +check_n(Cid, Idx, N, State) -> + strip_reply(apply(meta(Idx), + {checkout, {auto, N, simple_prefetch}, Cid}, [], State)). + +check(Cid, Idx, State) -> + strip_reply(apply(meta(Idx), + {checkout, {once, 1, simple_prefetch}, Cid}, [], State)). + +check_auto(Cid, Idx, State) -> + strip_reply(apply(meta(Idx), + {checkout, {auto, 1, simple_prefetch}, Cid}, [], State)). + +check(Cid, Idx, Num, State) -> + strip_reply(apply(meta(Idx), + {checkout, {once, Num, simple_prefetch}, Cid}, [], State)). + +settle(Cid, Idx, MsgId, State) -> + strip_reply(apply(meta(Idx), {settle, [MsgId], Cid}, [], State)). + +credit(Cid, Idx, Credit, DelCnt, Drain, State) -> + strip_reply(apply(meta(Idx), {credit, Credit, DelCnt, Drain, Cid}, [], State)). + +strip_reply({State, Effects, _Replu}) -> + {State, Effects}. + +run_log(InitState, Entries) -> + lists:foldl(fun ({Idx, E}, {Acc0, Efx0}) -> + case apply(meta(Idx), E, Efx0, Acc0) of + {Acc, Efx, _} -> + {Acc, Efx} + end + end, {InitState, []}, Entries). + + +%% AUX Tests + +aux_test() -> + _ = ra_machine_ets:start_link(), + Aux0 = init_aux(aux_test), + MacState = init(#{name => aux_test}), + Log = undefined, + {no_reply, Aux, undefined} = handle_aux(leader, cast, active, Aux0, + Log, MacState), + {no_reply, _Aux, undefined} = handle_aux(leader, cast, emit, Aux, + Log, MacState), + [X] = ets:lookup(rabbit_fifo_usage, aux_test), + ?assert(X > 0.0), + ok. + + +-endif. + diff --git a/src/rabbit_fifo_client.erl b/src/rabbit_fifo_client.erl new file mode 100644 index 000000000000..c087e35fb286 --- /dev/null +++ b/src/rabbit_fifo_client.erl @@ -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}}]}' +%% +%%
  • `ConsumerTag' the binary tag passed to {@link checkout/3.}
  • +%%
  • `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.
  • +-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]. diff --git a/src/rabbit_fifo_index.erl b/src/rabbit_fifo_index.erl new file mode 100644 index 000000000000..e1848862febc --- /dev/null +++ b/src/rabbit_fifo_index.erl @@ -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. diff --git a/src/rabbit_mirror_queue_misc.erl b/src/rabbit_mirror_queue_misc.erl index 16324abba06d..d2453726035c 100644 --- a/src/rabbit_mirror_queue_misc.erl +++ b/src/rabbit_mirror_queue_misc.erl @@ -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) -> diff --git a/src/rabbit_mnesia.erl b/src/rabbit_mnesia.erl index 793d365bf0fc..401b4a5545ea 100644 --- a/src/rabbit_mnesia.erl +++ b/src/rabbit_mnesia.erl @@ -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. diff --git a/src/rabbit_networking.erl b/src/rabbit_networking.erl index f6f8d226d03a..256d424740cc 100644 --- a/src/rabbit_networking.erl +++ b/src/rabbit_networking.erl @@ -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 diff --git a/src/rabbit_node_monitor.erl b/src/rabbit_node_monitor.erl index 808fea8e2378..7ecf6e33bfc6 100644 --- a/src/rabbit_node_monitor.erl +++ b/src/rabbit_node_monitor.erl @@ -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(), diff --git a/src/rabbit_queue_consumers.erl b/src/rabbit_queue_consumers.erl index 0fe3065fe8a9..e3322f3f647d 100644 --- a/src/rabbit_queue_consumers.erl +++ b/src/rabbit_queue_consumers.erl @@ -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; diff --git a/src/rabbit_quorum_queue.erl b/src/rabbit_quorum_queue.erl new file mode 100644 index 000000000000..795465855b01 --- /dev/null +++ b/src/rabbit_quorum_queue.erl @@ -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(<>, 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]). diff --git a/src/rabbit_reader.erl b/src/rabbit_reader.erl index 83f5ddaccb6e..91002d0b94f9 100644 --- a/src/rabbit_reader.erl +++ b/src/rabbit_reader.erl @@ -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) -> diff --git a/src/rabbit_upgrade_functions.erl b/src/rabbit_upgrade_functions.erl index 498db6e01c44..5c91ed7c2b1a 100644 --- a/src/rabbit_upgrade_functions.erl +++ b/src/rabbit_upgrade_functions.erl @@ -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 diff --git a/src/rabbit_variable_queue.erl b/src/rabbit_variable_queue.erl index 83738b7e5291..f91f8a648e4e 100644 --- a/src/rabbit_variable_queue.erl +++ b/src/rabbit_variable_queue.erl @@ -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. diff --git a/src/rabbit_vhost.erl b/src/rabbit_vhost.erl index bf89805d13e8..c460b02e5b38 100644 --- a/src/rabbit_vhost.erl +++ b/src/rabbit_vhost.erl @@ -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 diff --git a/src/rabbit_vhost_dead_letter.erl b/src/rabbit_vhost_dead_letter.erl new file mode 100644 index 000000000000..b4dede5e19f1 --- /dev/null +++ b/src/rabbit_vhost_dead_letter.erl @@ -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)}. diff --git a/src/rabbit_vhost_sup_sup.erl b/src/rabbit_vhost_sup_sup.erl index be892a6fbaff..19b86cb53c72 100644 --- a/src/rabbit_vhost_sup_sup.erl +++ b/src/rabbit_vhost_sup_sup.erl @@ -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, diff --git a/src/rabbit_vhost_sup_wrapper.erl b/src/rabbit_vhost_sup_wrapper.erl index 4ae68cdd75d9..e98b1a45aec1 100644 --- a/src/rabbit_vhost_sup_wrapper.erl +++ b/src/rabbit_vhost_sup_wrapper.erl @@ -63,4 +63,4 @@ start_vhost_sup(VHost) -> {ok, Pid}; Other -> Other - end. \ No newline at end of file + end. diff --git a/test/backing_queue_SUITE.erl b/test/backing_queue_SUITE.erl index 60f86e05423d..94cbd48e8c4b 100644 --- a/test/backing_queue_SUITE.erl +++ b/test/backing_queue_SUITE.erl @@ -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} = diff --git a/test/cluster_SUITE.erl b/test/cluster_SUITE.erl index 4864989b6a75..62928aae9f07 100644 --- a/test/cluster_SUITE.erl +++ b/test/cluster_SUITE.erl @@ -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(), diff --git a/test/proxy_protocol_SUITE.erl b/test/proxy_protocol_SUITE.erl index 136d2bb98080..84b94d72a88f 100644 --- a/test/proxy_protocol_SUITE.erl +++ b/test/proxy_protocol_SUITE.erl @@ -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. \ No newline at end of file + ConnectionName. diff --git a/test/quorum_queue_SUITE.erl b/test/quorum_queue_SUITE.erl new file mode 100644 index 000000000000..d983a9d396e2 --- /dev/null +++ b/test/quorum_queue_SUITE.erl @@ -0,0 +1,1783 @@ +%% 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(quorum_queue_SUITE). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). + +-compile(export_all). + +all() -> + [ + {group, single_node}, + {group, unclustered}, + {group, clustered} + ]. + +groups() -> + [ + {single_node, [], all_tests()}, + {unclustered, [], [ + {cluster_size_2, [], [add_member]} + ]}, + {clustered, [], [ + {cluster_size_2, [], [add_member_not_running, + add_member_classic, + add_member_already_a_member, + add_member_not_found, + delete_member_not_running, + delete_member_classic, + delete_member_not_found, + delete_member] + ++ all_tests()}, + {cluster_size_3, [], [ + declare_during_node_down, + recover_from_single_failure, + recover_from_multiple_failures, + leadership_takeover, + delete_declare, + metrics_cleanup_on_leadership_takeover, + metrics_cleanup_on_leader_crash, + consume_in_minority + ]}, + {cluster_size_5, [], [start_queue, + start_queue_concurrent, + quorum_cluster_size_3, + quorum_cluster_size_7 + ]} + ]} + ]. + +all_tests() -> + [ + declare_args, + declare_invalid_args, + declare_invalid_properties, + start_queue, + stop_queue, + restart_queue, + restart_all_types, + stop_start_rabbit_app, + publish, + publish_and_restart, + consume, + consume_first_empty, + consume_from_empty_queue, + consume_and_autoack, + subscribe, + subscribe_with_autoack, + consume_and_ack, + consume_and_multiple_ack, + subscribe_and_ack, + subscribe_and_multiple_ack, + consume_and_requeue_nack, + consume_and_requeue_multiple_nack, + subscribe_and_requeue_nack, + subscribe_and_requeue_multiple_nack, + consume_and_nack, + consume_and_multiple_nack, + subscribe_and_nack, + subscribe_and_multiple_nack, + subscribe_should_fail_when_global_qos_true, + publisher_confirms, + publisher_confirms_with_deleted_queue, + dead_letter_to_classic_queue, + dead_letter_to_quorum_queue, + dead_letter_from_classic_to_quorum_queue, + cleanup_queue_state_on_channel_after_publish, + cleanup_queue_state_on_channel_after_subscribe, + basic_cancel, + purge, + sync_queue, + cancel_sync_queue, + basic_recover, + idempotent_recover, + vhost_with_quorum_queue_is_deleted + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(clustered, Config) -> + rabbit_ct_helpers:set_config(Config, [{rmq_nodes_clustered, true}]); +init_per_group(unclustered, Config) -> + rabbit_ct_helpers:set_config(Config, [{rmq_nodes_clustered, false}]); +init_per_group(Group, Config) -> + ClusterSize = case Group of + single_node -> 1; + cluster_size_2 -> 2; + cluster_size_3 -> 3; + cluster_size_5 -> 5 + end, + Config1 = rabbit_ct_helpers:set_config(Config, + [{rmq_nodes_count, ClusterSize}, + {rmq_nodename_suffix, Group}, + {tcp_ports_base}]), + Config2 = rabbit_ct_helpers:run_steps(Config1, + [fun merge_app_env/1 ] ++ + rabbit_ct_broker_helpers:setup_steps()), + ok = rabbit_ct_broker_helpers:rpc( + Config2, 0, application, set_env, + [rabbit, channel_queue_cleanup_interval, 100]), + %% HACK: the larger cluster sizes benefit for a bit more time + %% after clustering before running the tests. + case Group of + cluster_size_5 -> + timer:sleep(5000), + Config2; + _ -> + Config2 + end. + +end_per_group(clustered, Config) -> + Config; +end_per_group(unclustered, Config) -> + Config; +end_per_group(_, Config) -> + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_testcase(Testcase, Config) -> + Config1 = rabbit_ct_helpers:testcase_started(Config, Testcase), + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_queues, []), + Q = rabbit_data_coercion:to_binary(Testcase), + Config2 = rabbit_ct_helpers:set_config(Config1, + [{queue_name, Q} + ]), + rabbit_ct_helpers:run_steps(Config2, + rabbit_ct_client_helpers:setup_steps()). + +merge_app_env(Config) -> + rabbit_ct_helpers:merge_app_env(Config, {rabbit, [{core_metrics_gc_interval, 100}]}). + +end_per_testcase(Testcase, Config) -> + catch delete_queues(), + Config1 = rabbit_ct_helpers:run_steps( + Config, + rabbit_ct_client_helpers:teardown_steps()), + rabbit_ct_helpers:testcase_finished(Config1, Testcase). + +%% ------------------------------------------------------------------- +%% Testcases. +%% ------------------------------------------------------------------- + +declare_args(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + LQ = ?config(queue_name, Config), + declare(Ch, LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}]), + assert_queue_type(Server, LQ, quorum), + + DQ = <<"classic-declare-args-q">>, + declare(Ch, DQ, [{<<"x-queue-type">>, longstr, <<"classic">>}]), + assert_queue_type(Server, DQ, classic), + + DQ2 = <<"classic-q2">>, + declare(Ch, DQ2), + assert_queue_type(Server, DQ2, classic). + +declare_invalid_properties(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + LQ = ?config(queue_name, Config), + + ?assertExit( + {{shutdown, {server_initiated_close, 406, _}}, _}, + amqp_channel:call( + rabbit_ct_client_helpers:open_channel(Config, Server), + #'queue.declare'{queue = LQ, + auto_delete = true, + durable = true, + arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]})), + ?assertExit( + {{shutdown, {server_initiated_close, 406, _}}, _}, + amqp_channel:call( + rabbit_ct_client_helpers:open_channel(Config, Server), + #'queue.declare'{queue = LQ, + exclusive = true, + durable = true, + arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]})), + ?assertExit( + {{shutdown, {server_initiated_close, 406, _}}, _}, + amqp_channel:call( + rabbit_ct_client_helpers:open_channel(Config, Server), + #'queue.declare'{queue = LQ, + durable = false, + arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]})). + +declare_invalid_args(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + LQ = ?config(queue_name, Config), + + ?assertExit( + {{shutdown, {server_initiated_close, 406, _}}, _}, + declare(rabbit_ct_client_helpers:open_channel(Config, Server), + LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-expires">>, long, 2000}])), + ?assertExit( + {{shutdown, {server_initiated_close, 406, _}}, _}, + declare(rabbit_ct_client_helpers:open_channel(Config, Server), + LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-message-ttl">>, long, 2000}])), + ?assertExit( + {{shutdown, {server_initiated_close, 406, _}}, _}, + declare(rabbit_ct_client_helpers:open_channel(Config, Server), + LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-max-length">>, long, 2000}])), + ?assertExit( + {{shutdown, {server_initiated_close, 406, _}}, _}, + declare(rabbit_ct_client_helpers:open_channel(Config, Server), + LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-max-length-bytes">>, long, 2000}])), + + ?assertExit( + {{shutdown, {server_initiated_close, 406, _}}, _}, + declare(rabbit_ct_client_helpers:open_channel(Config, Server), + LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-max-priority">>, long, 2000}])), + + ?assertExit( + {{shutdown, {server_initiated_close, 406, _}}, _}, + declare(rabbit_ct_client_helpers:open_channel(Config, Server), + LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-overflow">>, longstr, <<"drop-head">>}])), + + ?assertExit( + {{shutdown, {server_initiated_close, 406, _}}, _}, + declare(rabbit_ct_client_helpers:open_channel(Config, Server), + LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-queue-mode">>, longstr, <<"lazy">>}])), + + ?assertExit( + {{shutdown, {server_initiated_close, 406, _}}, _}, + declare(rabbit_ct_client_helpers:open_channel(Config, Server), + LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-quorum-initial-group-size">>, longstr, <<"hop">>}])), + + ?assertExit( + {{shutdown, {server_initiated_close, 406, _}}, _}, + declare(rabbit_ct_client_helpers:open_channel(Config, Server), + LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-quorum-initial-group-size">>, long, 0}])). + +start_queue(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + LQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', LQ, 0, 0}, + declare(Ch, LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + %% Check that the application and one ra node are up + ?assertMatch({ra, _, _}, lists:keyfind(ra, 1, + rpc:call(Server, application, which_applications, []))), + ?assertMatch([_], rpc:call(Server, supervisor, which_children, [ra_server_sup])), + + %% Test declare an existing queue + ?assertEqual({'queue.declare_ok', LQ, 0, 0}, + declare(Ch, LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + %% Test declare with same arguments + ?assertEqual({'queue.declare_ok', LQ, 0, 0}, + declare(Ch, LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + %% Test declare an existing queue with different arguments + ?assertExit(_, declare(Ch, LQ, [])), + + %% Check that the application and process are still up + ?assertMatch({ra, _, _}, lists:keyfind(ra, 1, + rpc:call(Server, application, which_applications, []))), + ?assertMatch([_], rpc:call(Server, supervisor, which_children, [ra_server_sup])). + +start_queue_concurrent(Config) -> + Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + LQ = ?config(queue_name, Config), + Self = self(), + [begin + _ = spawn_link(fun () -> + {_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, Server), + %% Test declare an existing queue + ?assertEqual({'queue.declare_ok', LQ, 0, 0}, + declare(Ch, LQ, + [{<<"x-queue-type">>, + longstr, + <<"quorum">>}])), + Self ! {done, Server} + end) + end || Server <- Servers], + + [begin + receive {done, Server} -> ok + after 5000 -> exit({await_done_timeout, Server}) + end + end || Server <- Servers], + + + ok. + +quorum_cluster_size_3(Config) -> + quorum_cluster_size_x(Config, 3, 3). + +quorum_cluster_size_7(Config) -> + quorum_cluster_size_x(Config, 7, 5). + +quorum_cluster_size_x(Config, Max, Expected) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + RaName = ra_name(QQ), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-quorum-initial-group-size">>, long, Max}])), + {ok, Members, _} = ra:members({RaName, Server}), + ?assertEqual(Expected, length(Members)), + Info = rpc:call(Server, rabbit_quorum_queue, infos, + [rabbit_misc:r(<<"/">>, queue, QQ)]), + MembersQ = proplists:get_value(members, Info), + ?assertEqual(Expected, length(MembersQ)). + +stop_queue(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + LQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', LQ, 0, 0}, + declare(Ch, LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + %% Check that the application and one ra node are up + ?assertMatch({ra, _, _}, lists:keyfind(ra, 1, + rpc:call(Server, application, which_applications, []))), + ?assertMatch([_], rpc:call(Server, supervisor, which_children, [ra_server_sup])), + + %% Delete the quorum queue + ?assertMatch(#'queue.delete_ok'{}, amqp_channel:call(Ch, #'queue.delete'{queue = LQ})), + %% Check that the application and process are down + wait_until(fun() -> + [] == rpc:call(Server, supervisor, which_children, [ra_server_sup]) + end), + ?assertMatch({ra, _, _}, lists:keyfind(ra, 1, + rpc:call(Server, application, which_applications, []))). + +restart_queue(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + LQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', LQ, 0, 0}, + declare(Ch, LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Server), + ok = rabbit_ct_broker_helpers:start_node(Config, Server), + + %% Check that the application and one ra node are up + ?assertMatch({ra, _, _}, lists:keyfind(ra, 1, + rpc:call(Server, application, which_applications, []))), + ?assertMatch([_], rpc:call(Server, supervisor, which_children, [ra_server_sup])). + +idempotent_recover(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + LQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', LQ, 0, 0}, + declare(Ch, LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + %% kill default vhost to trigger recovery + [{_, SupWrapperPid, _, _} | _] = rpc:call(Server, supervisor, + which_children, + [rabbit_vhost_sup_sup]), + [{_, Pid, _, _} | _] = rpc:call(Server, supervisor, + which_children, + [SupWrapperPid]), + %% kill the vhost process to trigger recover + rpc:call(Server, erlang, exit, [Pid, kill]), + + timer:sleep(1000), + %% validate quorum queue is still functional + RaName = ra_name(LQ), + {ok, _, _} = ra:members({RaName, Server}), + %% validate vhosts are running - or rather validate that at least one + %% vhost per cluster is running + [begin + #{cluster_state := ServerStatuses} = maps:from_list(I), + ?assertMatch(#{Server := running}, maps:from_list(ServerStatuses)) + end || I <- rpc:call(Server, rabbit_vhost,info_all, [])], + ok. + +vhost_with_quorum_queue_is_deleted(Config) -> + Node = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + VHost = <<"vhost2">>, + QName = atom_to_binary(?FUNCTION_NAME, utf8), + RaName = binary_to_atom(<>, utf8), + User = ?config(rmq_username, Config), + ok = rabbit_ct_broker_helpers:add_vhost(Config, Node, VHost, User), + ok = rabbit_ct_broker_helpers:set_full_permissions(Config, User, VHost), + Conn = rabbit_ct_client_helpers:open_unmanaged_connection(Config, Node, + VHost), + {ok, Ch} = amqp_connection:open_channel(Conn), + ?assertEqual({'queue.declare_ok', QName, 0, 0}, + declare(Ch, QName, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + UId = rpc:call(Node, ra_directory, where_is, [RaName]), + ?assert(UId =/= undefined), + ok = rabbit_ct_broker_helpers:delete_vhost(Config, VHost), + %% validate quorum queues got deleted + undefined = rpc:call(Node, ra_directory, where_is, [RaName]), + ok. + +restart_all_types(Config) -> + %% Test the node restart with both types of queues (quorum and classic) to + %% ensure there are no regressions + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ1 = <<"restart_all_types-qq1">>, + ?assertEqual({'queue.declare_ok', QQ1, 0, 0}, + declare(Ch, QQ1, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + QQ2 = <<"restart_all_types-qq2">>, + ?assertEqual({'queue.declare_ok', QQ2, 0, 0}, + declare(Ch, QQ2, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + CQ1 = <<"restart_all_types-classic1">>, + ?assertEqual({'queue.declare_ok', CQ1, 0, 0}, declare(Ch, CQ1, [])), + rabbit_ct_client_helpers:publish(Ch, CQ1, 1), + CQ2 = <<"restart_all_types-classic2">>, + ?assertEqual({'queue.declare_ok', CQ2, 0, 0}, declare(Ch, CQ2, [])), + rabbit_ct_client_helpers:publish(Ch, CQ2, 1), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Server), + ok = rabbit_ct_broker_helpers:start_node(Config, Server), + + %% Check that the application and two ra nodes are up + ?assertMatch({ra, _, _}, lists:keyfind(ra, 1, + rpc:call(Server, application, which_applications, []))), + ?assertMatch([_,_], rpc:call(Server, supervisor, which_children, [ra_server_sup])), + %% Check the classic queues restarted correctly + Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server), + {#'basic.get_ok'{}, #amqp_msg{}} = + amqp_channel:call(Ch2, #'basic.get'{queue = CQ1, no_ack = false}), + {#'basic.get_ok'{}, #amqp_msg{}} = + amqp_channel:call(Ch2, #'basic.get'{queue = CQ2, no_ack = false}), + delete_queues(Ch2, [QQ1, QQ2, CQ1, CQ2]). + +delete_queues(Ch, Queues) -> + [amqp_channel:call(Ch, #'queue.delete'{queue = Q}) || Q <- Queues]. + +stop_start_rabbit_app(Config) -> + %% Test start/stop of rabbit app with both types of queues (quorum and + %% classic) to ensure there are no regressions + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ1 = <<"stop_start_rabbit_app-qq">>, + ?assertEqual({'queue.declare_ok', QQ1, 0, 0}, + declare(Ch, QQ1, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + QQ2 = <<"quorum-q2">>, + ?assertEqual({'queue.declare_ok', QQ2, 0, 0}, + declare(Ch, QQ2, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + CQ1 = <<"stop_start_rabbit_app-classic">>, + ?assertEqual({'queue.declare_ok', CQ1, 0, 0}, declare(Ch, CQ1, [])), + rabbit_ct_client_helpers:publish(Ch, CQ1, 1), + CQ2 = <<"stop_start_rabbit_app-classic2">>, + ?assertEqual({'queue.declare_ok', CQ2, 0, 0}, declare(Ch, CQ2, [])), + rabbit_ct_client_helpers:publish(Ch, CQ2, 1), + + rabbit_control_helper:command(stop_app, Server), + %% Check the ra application has stopped (thus its supervisor and queues) + ?assertMatch(false, lists:keyfind(ra, 1, + rpc:call(Server, application, which_applications, []))), + + rabbit_control_helper:command(start_app, Server), + + %% Check that the application and two ra nodes are up + ?assertMatch({ra, _, _}, lists:keyfind(ra, 1, + rpc:call(Server, application, which_applications, []))), + ?assertMatch([_,_], rpc:call(Server, supervisor, which_children, [ra_server_sup])), + %% Check the classic queues restarted correctly + Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server), + {#'basic.get_ok'{}, #amqp_msg{}} = + amqp_channel:call(Ch2, #'basic.get'{queue = CQ1, no_ack = false}), + {#'basic.get_ok'{}, #amqp_msg{}} = + amqp_channel:call(Ch2, #'basic.get'{queue = CQ2, no_ack = false}), + delete_queues(Ch2, [QQ1, QQ2, CQ1, CQ2]). + +publish(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + publish(Ch, QQ), + Name = ra_name(QQ), + wait_for_messages_ready(Servers, Name, 1), + wait_for_messages_pending_ack(Servers, Name, 0). + +ra_name(Q) -> + binary_to_atom(<<"%2F_", Q/binary>>, utf8). + +publish_and_restart(Config) -> + %% Test the node restart with both types of queues (quorum and classic) to + %% ensure there are no regressions + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + RaName = ra_name(QQ), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Server), + ok = rabbit_ct_broker_helpers:start_node(Config, Server), + + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + publish(rabbit_ct_client_helpers:open_channel(Config, Server), QQ), + wait_for_messages_ready(Servers, RaName, 2), + wait_for_messages_pending_ack(Servers, RaName, 0). + +consume(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + consume(Ch, QQ, false), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 1), + rabbit_ct_client_helpers:close_channel(Ch), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0). + +consume_first_empty(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + consume_empty(Ch, QQ, false), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + consume(Ch, QQ, false), + rabbit_ct_client_helpers:close_channel(Ch). + +consume_in_minority(Config) -> + [Server0, Server1, Server2] = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server0), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Server1), + ok = rabbit_ct_broker_helpers:stop_node(Config, Server2), + + ?assertExit({{shutdown, {connection_closing, {server_initiated_close, 541, _}}}, _}, + amqp_channel:call(Ch, #'basic.get'{queue = QQ, + no_ack = false})). + +consume_and_autoack(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + RaName = ra_name(QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + consume(Ch, QQ, true), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0), + rabbit_ct_client_helpers:close_channel(Ch), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0). + +consume_from_empty_queue(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + consume_empty(Ch, QQ, false). + +subscribe(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + subscribe(Ch, QQ, false), + receive_basic_deliver(false), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 1), + rabbit_ct_client_helpers:close_channel(Ch), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0). + +subscribe_should_fail_when_global_qos_true(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + qos(Ch, 10, true), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + try subscribe(Ch, QQ, false) of + _ -> exit(subscribe_should_not_pass) + catch + _:_ = Err -> + ct:pal("Err ~p", [Err]) + end, + ok. + +subscribe_with_autoack(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 2), + wait_for_messages_pending_ack(Servers, RaName, 0), + subscribe(Ch, QQ, true), + receive_basic_deliver(false), + receive_basic_deliver(false), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0), + rabbit_ct_client_helpers:close_channel(Ch), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0). + +consume_and_ack(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + DeliveryTag = consume(Ch, QQ, false), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 1), + amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DeliveryTag}), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0). + +consume_and_multiple_ack(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + publish(Ch, QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 3), + wait_for_messages_pending_ack(Servers, RaName, 0), + _ = consume(Ch, QQ, false), + _ = consume(Ch, QQ, false), + DeliveryTag = consume(Ch, QQ, false), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 3), + amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DeliveryTag, + multiple = true}), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0). + +subscribe_and_ack(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + subscribe(Ch, QQ, false), + receive + {#'basic.deliver'{delivery_tag = DeliveryTag}, _} -> + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 1), + amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DeliveryTag}), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0) + end. + +subscribe_and_multiple_ack(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + publish(Ch, QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 3), + wait_for_messages_pending_ack(Servers, RaName, 0), + subscribe(Ch, QQ, false), + receive_basic_deliver(false), + receive_basic_deliver(false), + receive + {#'basic.deliver'{delivery_tag = DeliveryTag}, _} -> + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 3), + amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DeliveryTag, + multiple = true}), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0) + end. + +consume_and_requeue_nack(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 2), + wait_for_messages_pending_ack(Servers, RaName, 0), + DeliveryTag = consume(Ch, QQ, false), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 1), + amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, + multiple = false, + requeue = true}), + wait_for_messages_ready(Servers, RaName, 2), + wait_for_messages_pending_ack(Servers, RaName, 0). + +consume_and_requeue_multiple_nack(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + publish(Ch, QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 3), + wait_for_messages_pending_ack(Servers, RaName, 0), + _ = consume(Ch, QQ, false), + _ = consume(Ch, QQ, false), + DeliveryTag = consume(Ch, QQ, false), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 3), + amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, + multiple = true, + requeue = true}), + wait_for_messages_ready(Servers, RaName, 3), + wait_for_messages_pending_ack(Servers, RaName, 0). + +consume_and_nack(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + DeliveryTag = consume(Ch, QQ, false), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 1), + amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, + multiple = false, + requeue = false}), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0). + +consume_and_multiple_nack(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + publish(Ch, QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 3), + wait_for_messages_pending_ack(Servers, RaName, 0), + _ = consume(Ch, QQ, false), + _ = consume(Ch, QQ, false), + DeliveryTag = consume(Ch, QQ, false), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 3), + amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, + multiple = true, + requeue = false}), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0). + +subscribe_and_requeue_multiple_nack(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + publish(Ch, QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 3), + wait_for_messages_pending_ack(Servers, RaName, 0), + subscribe(Ch, QQ, false), + receive_basic_deliver(false), + receive_basic_deliver(false), + receive + {#'basic.deliver'{delivery_tag = DeliveryTag, + redelivered = false}, _} -> + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 3), + amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, + multiple = true, + requeue = true}), + receive_basic_deliver(true), + receive_basic_deliver(true), + receive + {#'basic.deliver'{delivery_tag = DeliveryTag1, + redelivered = true}, _} -> + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 3), + amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DeliveryTag1, + multiple = true}), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0) + end + end. + +subscribe_and_requeue_nack(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + subscribe(Ch, QQ, false), + receive + {#'basic.deliver'{delivery_tag = DeliveryTag, + redelivered = false}, _} -> + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 1), + amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, + multiple = false, + requeue = true}), + receive + {#'basic.deliver'{delivery_tag = DeliveryTag1, + redelivered = true}, _} -> + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 1), + amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DeliveryTag1}), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0) + end + end. + +subscribe_and_nack(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + subscribe(Ch, QQ, false), + receive + {#'basic.deliver'{delivery_tag = DeliveryTag, + redelivered = false}, _} -> + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 1), + amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, + multiple = false, + requeue = false}), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0) + end. + +subscribe_and_multiple_nack(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + publish(Ch, QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 3), + wait_for_messages_pending_ack(Servers, RaName, 0), + subscribe(Ch, QQ, false), + receive_basic_deliver(false), + receive_basic_deliver(false), + receive + {#'basic.deliver'{delivery_tag = DeliveryTag, + redelivered = false}, _} -> + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 3), + amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, + multiple = true, + requeue = false}), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0) + end. + +publisher_confirms(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + amqp_channel:call(Ch, #'confirm.select'{}), + amqp_channel:register_confirm_handler(Ch, self()), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + ct:pal("WAIT FOR CONFIRMS ~n", []), + amqp_channel:wait_for_confirms(Ch, 5000), + amqp_channel:unregister_confirm_handler(Ch), + ok. + +publisher_confirms_with_deleted_queue(Config) -> + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + amqp_channel:call(Ch, #'confirm.select'{}), + amqp_channel:register_confirm_handler(Ch, self()), + % subscribe(Ch, QQ, false), + publish(Ch, QQ), + delete_queues(Ch, [QQ]), + ct:pal("WAIT FOR CONFIRMS ~n", []), + amqp_channel:wait_for_confirms_or_die(Ch, 5000), + amqp_channel:unregister_confirm_handler(Ch), + ok. + +dead_letter_to_classic_queue(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + CQ = <<"classic-dead_letter_to_classic_queue">>, + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-dead-letter-exchange">>, longstr, <<>>}, + {<<"x-dead-letter-routing-key">>, longstr, CQ} + ])), + ?assertEqual({'queue.declare_ok', CQ, 0, 0}, declare(Ch, CQ, [])), + RaName = ra_name(QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + wait_for_messages(Config, [[CQ, <<"0">>, <<"0">>, <<"0">>]]), + DeliveryTag = consume(Ch, QQ, false), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 1), + wait_for_messages(Config, [[CQ, <<"0">>, <<"0">>, <<"0">>]]), + amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, + multiple = false, + requeue = false}), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0), + wait_for_messages(Config, [[CQ, <<"1">>, <<"1">>, <<"0">>]]), + _ = consume(Ch, CQ, false). + +dead_letter_to_quorum_queue(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + QQ2 = <<"dead_letter_to_quorum_queue-q2">>, + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-dead-letter-exchange">>, longstr, <<>>}, + {<<"x-dead-letter-routing-key">>, longstr, QQ2} + ])), + ?assertEqual({'queue.declare_ok', QQ2, 0, 0}, + declare(Ch, QQ2, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + RaName = ra_name(QQ), + RaName2 = ra_name(QQ2), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + wait_for_messages_ready(Servers, RaName2, 0), + wait_for_messages_pending_ack(Servers, RaName2, 0), + DeliveryTag = consume(Ch, QQ, false), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 1), + wait_for_messages_ready(Servers, RaName2, 0), + wait_for_messages_pending_ack(Servers, RaName2, 0), + amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, + multiple = false, + requeue = false}), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0), + wait_for_messages_ready(Servers, RaName2, 1), + wait_for_messages_pending_ack(Servers, RaName2, 0), + _ = consume(Ch, QQ2, false). + +dead_letter_from_classic_to_quorum_queue(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + CQ = <<"classic-q-dead_letter_from_classic_to_quorum_queue">>, + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', CQ, 0, 0}, + declare(Ch, CQ, [{<<"x-dead-letter-exchange">>, longstr, <<>>}, + {<<"x-dead-letter-routing-key">>, longstr, QQ} + ])), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + RaName = ra_name(QQ), + publish(Ch, CQ), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0), + wait_for_messages(Config, [[CQ, <<"1">>, <<"1">>, <<"0">>]]), + DeliveryTag = consume(Ch, CQ, false), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0), + wait_for_messages(Config, [[CQ, <<"1">>, <<"0">>, <<"1">>]]), + amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag, + multiple = false, + requeue = false}), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + wait_for_messages(Config, [[CQ, <<"0">>, <<"0">>, <<"0">>]]), + _ = consume(Ch, QQ, false), + rabbit_ct_client_helpers:close_channel(Ch). + +cleanup_queue_state_on_channel_after_publish(Config) -> + %% Declare/delete the queue in one channel and publish on a different one, + %% to verify that the cleanup is propagated through channels + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch1 = rabbit_ct_client_helpers:open_channel(Config, Server), + Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch1, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + RaName = ra_name(QQ), + publish(Ch2, QQ), + Res = dirty_query(Servers, RaName, fun rabbit_fifo:query_consumer_count/1), + ct:pal ("Res ~p", [Res]), + wait_for_messages_pending_ack(Servers, RaName, 0), + wait_for_messages_ready(Servers, RaName, 1), + [NCh1, NCh2] = rpc:call(Server, rabbit_channel, list, []), + %% Check the channel state contains the state for the quorum queue on + %% channel 1 and 2 + wait_for_cleanup(Server, NCh1, 0), + wait_for_cleanup(Server, NCh2, 1), + %% then delete the queue and wait for the process to terminate + ?assertMatch(#'queue.delete_ok'{}, + amqp_channel:call(Ch1, #'queue.delete'{queue = QQ})), + wait_until(fun() -> + [] == rpc:call(Server, supervisor, which_children, + [ra_server_sup]) + end), + %% Check that all queue states have been cleaned + wait_for_cleanup(Server, NCh1, 0), + wait_for_cleanup(Server, NCh2, 0). + +cleanup_queue_state_on_channel_after_subscribe(Config) -> + %% Declare/delete the queue and publish in one channel, while consuming on a + %% different one to verify that the cleanup is propagated through channels + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch1 = rabbit_ct_client_helpers:open_channel(Config, Server), + Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch1, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + RaName = ra_name(QQ), + publish(Ch1, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + subscribe(Ch2, QQ, false), + receive + {#'basic.deliver'{delivery_tag = DeliveryTag, + redelivered = false}, _} -> + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 1), + amqp_channel:cast(Ch2, #'basic.ack'{delivery_tag = DeliveryTag, + multiple = true}), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0) + end, + [NCh1, NCh2] = rpc:call(Server, rabbit_channel, list, []), + %% Check the channel state contains the state for the quorum queue on channel 1 and 2 + wait_for_cleanup(Server, NCh1, 1), + wait_for_cleanup(Server, NCh2, 1), + ?assertMatch(#'queue.delete_ok'{}, amqp_channel:call(Ch1, #'queue.delete'{queue = QQ})), + wait_until(fun() -> + [] == rpc:call(Server, supervisor, which_children, [ra_server_sup]) + end), + %% Check that all queue states have been cleaned + wait_for_cleanup(Server, NCh1, 0), + wait_for_cleanup(Server, NCh2, 0). + +recover_from_single_failure(Config) -> + [Server, Server1, Server2] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Server2), + RaName = ra_name(QQ), + + publish(Ch, QQ), + publish(Ch, QQ), + publish(Ch, QQ), + wait_for_messages_ready([Server, Server1], RaName, 3), + wait_for_messages_pending_ack([Server, Server1], RaName, 0), + + ok = rabbit_ct_broker_helpers:start_node(Config, Server2), + wait_for_messages_ready(Servers, RaName, 3), + wait_for_messages_pending_ack(Servers, RaName, 0). + +recover_from_multiple_failures(Config) -> + [Server, Server1, Server2] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Server1), + RaName = ra_name(QQ), + + publish(Ch, QQ), + publish(Ch, QQ), + publish(Ch, QQ), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Server2), + + publish(Ch, QQ), + publish(Ch, QQ), + publish(Ch, QQ), + + wait_for_messages_ready([Server], RaName, 3), + wait_for_messages_pending_ack([Server], RaName, 0), + + ok = rabbit_ct_broker_helpers:start_node(Config, Server1), + ok = rabbit_ct_broker_helpers:start_node(Config, Server2), + + %% there is an assumption here that the messages were not lost and were + %% recovered when a quorum was restored. Not the best test perhaps. + wait_for_messages_ready(Servers, RaName, 6), + wait_for_messages_pending_ack(Servers, RaName, 0). + +leadership_takeover(Config) -> + %% Kill nodes in succession forcing the takeover of leadership, and all messages that + %% are in the queue. + [Server, Server1, Server2] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Server1), + RaName = ra_name(QQ), + + publish(Ch, QQ), + publish(Ch, QQ), + publish(Ch, QQ), + + wait_for_messages_ready([Server], RaName, 3), + wait_for_messages_pending_ack([Server], RaName, 0), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Server2), + + ok = rabbit_ct_broker_helpers:start_node(Config, Server1), + ok = rabbit_ct_broker_helpers:stop_node(Config, Server), + ok = rabbit_ct_broker_helpers:start_node(Config, Server2), + ok = rabbit_ct_broker_helpers:stop_node(Config, Server1), + ok = rabbit_ct_broker_helpers:start_node(Config, Server), + + wait_for_messages_ready([Server2, Server], RaName, 3), + wait_for_messages_pending_ack([Server2, Server], RaName, 0), + + ok = rabbit_ct_broker_helpers:start_node(Config, Server1), + wait_for_messages_ready(Servers, RaName, 3), + wait_for_messages_pending_ack(Servers, RaName, 0). + +metrics_cleanup_on_leadership_takeover(Config) -> + %% Queue core metrics should be deleted from a node once the leadership is transferred + %% to another follower + [Server, _, _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + publish(Ch, QQ), + publish(Ch, QQ), + + wait_for_messages_ready([Server], RaName, 3), + wait_for_messages_pending_ack([Server], RaName, 0), + {ok, _, {_, Leader}} = ra:members({RaName, Server}), + QRes = rabbit_misc:r(<<"/">>, queue, QQ), + wait_until( + fun() -> + case rpc:call(Leader, ets, lookup, [queue_coarse_metrics, QRes]) of + [{QRes, 3, 0, 3, _}] -> true; + _ -> false + end + end), + force_leader_change(Leader, Servers, QQ), + wait_until(fun () -> + [] =:= rpc:call(Leader, ets, lookup, [queue_coarse_metrics, QRes]) andalso + [] =:= rpc:call(Leader, ets, lookup, [queue_metrics, QRes]) + end), + ok. + +metrics_cleanup_on_leader_crash(Config) -> + %% Queue core metrics should be deleted from a node once the leadership is transferred + %% to another follower + [Server | _] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + publish(Ch, QQ), + publish(Ch, QQ), + + wait_for_messages_ready([Server], RaName, 3), + wait_for_messages_pending_ack([Server], RaName, 0), + {ok, _, {Name, Leader}} = ra:members({RaName, Server}), + QRes = rabbit_misc:r(<<"/">>, queue, QQ), + wait_until( + fun() -> + case rpc:call(Leader, ets, lookup, [queue_coarse_metrics, QRes]) of + [{QRes, 3, 0, 3, _}] -> true; + _ -> false + end + end), + Pid = rpc:call(Leader, erlang, whereis, [Name]), + rpc:call(Leader, erlang, exit, [Pid, kill]), + [Other | _] = lists:delete(Leader, Servers), + catch ra:trigger_election(Other), + %% kill it again just in case it came straight back up again + catch rpc:call(Leader, erlang, exit, [Pid, kill]), + + %% this isn't a reliable test as the leader can be restarted so quickly + %% after a crash it is elected leader of the next term as well. + wait_until( + fun() -> + [] == rpc:call(Leader, ets, lookup, [queue_coarse_metrics, QRes]) + end), + ok. + +delete_declare(Config) -> + %% Delete cluster in ra is asynchronous, we have to ensure that we handle that in rmq + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, + nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + publish(Ch, QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 3), + + ?assertMatch(#'queue.delete_ok'{}, + amqp_channel:call(Ch, #'queue.delete'{queue = QQ})), + %% the actual data deletions happen after the call has returned as a quorum + %% queue leader waits for all nodes to confirm they replicated the poison + %% pill before terminating itself. + timer:sleep(1000), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + %% Ensure that is a new queue and it's empty + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 0). + +basic_cancel(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + subscribe(Ch, QQ, false), + receive + {#'basic.deliver'{}, _} -> + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 1), + amqp_channel:call(Ch, #'basic.cancel'{consumer_tag = <<"ctag">>}), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0) + end. + +purge(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 2), + wait_for_messages_pending_ack(Servers, RaName, 0), + _DeliveryTag = consume(Ch, QQ, false), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 1), + {'queue.purge_ok', 2} = amqp_channel:call(Ch, #'queue.purge'{queue = QQ}), + wait_for_messages_pending_ack(Servers, RaName, 0), + wait_for_messages_ready(Servers, RaName, 0). + +sync_queue(Config) -> + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + {error, _, _} = + rabbit_ct_broker_helpers:rabbitmqctl(Config, 0, [<<"sync_queue">>, QQ]), + ok. + +cancel_sync_queue(Config) -> + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + {error, _, _} = + rabbit_ct_broker_helpers:rabbitmqctl(Config, 0, [<<"cancel_sync_queue">>, QQ]), + ok. + +declare_during_node_down(Config) -> + [Server, DownServer, _] = Servers = rabbit_ct_broker_helpers:get_node_configs( + Config, nodename), + + stop_node(Config, DownServer), + % rabbit_ct_broker_helpers:stop_node(Config, DownServer), + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + timer:sleep(2000), + rabbit_ct_broker_helpers:start_node(Config, DownServer), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + ok. + +add_member_not_running(Config) -> + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + ct:pal("add_member_not_running config ~p", [Config]), + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + ?assertEqual({error, node_not_running}, + rpc:call(Server, rabbit_quorum_queue, add_member, + [<<"/">>, QQ, 'rabbit@burrow'])). + +add_member_classic(Config) -> + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + CQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', CQ, 0, 0}, declare(Ch, CQ, [])), + ?assertEqual({error, classic_queue_not_supported}, + rpc:call(Server, rabbit_quorum_queue, add_member, + [<<"/">>, CQ, Server])). + +add_member_already_a_member(Config) -> + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + ?assertEqual({error, already_a_member}, + rpc:call(Server, rabbit_quorum_queue, add_member, + [<<"/">>, QQ, Server])). + +add_member_not_found(Config) -> + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + QQ = ?config(queue_name, Config), + ?assertEqual({error, not_found}, + rpc:call(Server, rabbit_quorum_queue, add_member, + [<<"/">>, QQ, Server])). + +add_member(Config) -> + [Server0, Server1] = Servers0 = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Ch = rabbit_ct_client_helpers:open_channel(Config, Server0), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + ?assertEqual({error, node_not_running}, + rpc:call(Server0, rabbit_quorum_queue, add_member, + [<<"/">>, QQ, Server1])), + ok = rabbit_control_helper:command(stop_app, Server1), + ok = rabbit_control_helper:command(join_cluster, Server1, [atom_to_list(Server0)], []), + rabbit_control_helper:command(start_app, Server1), + ?assertEqual(ok, rpc:call(Server0, rabbit_quorum_queue, add_member, + [<<"/">>, QQ, Server1])), + Info = rpc:call(Server0, rabbit_quorum_queue, infos, + [rabbit_misc:r(<<"/">>, queue, QQ)]), + Servers = lists:sort(Servers0), + ?assertEqual(Servers, lists:sort(proplists:get_value(online, Info, []))). + +delete_member_not_running(Config) -> + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + ?assertEqual({error, node_not_running}, + rpc:call(Server, rabbit_quorum_queue, delete_member, + [<<"/">>, QQ, 'rabbit@burrow'])). + +delete_member_classic(Config) -> + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + CQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', CQ, 0, 0}, declare(Ch, CQ, [])), + ?assertEqual({error, classic_queue_not_supported}, + rpc:call(Server, rabbit_quorum_queue, delete_member, + [<<"/">>, CQ, Server])). + +delete_member_not_found(Config) -> + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + QQ = ?config(queue_name, Config), + ?assertEqual({error, not_found}, + rpc:call(Server, rabbit_quorum_queue, delete_member, + [<<"/">>, QQ, Server])). + +delete_member(Config) -> + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + timer:sleep(100), + ?assertEqual(ok, + rpc:call(Server, rabbit_quorum_queue, delete_member, + [<<"/">>, QQ, Server])), + ?assertEqual({error, not_a_member}, + rpc:call(Server, rabbit_quorum_queue, delete_member, + [<<"/">>, QQ, Server])). + +basic_recover(Config) -> + [Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + QQ = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', QQ, 0, 0}, + declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), + + RaName = ra_name(QQ), + publish(Ch, QQ), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0), + _ = consume(Ch, QQ, false), + wait_for_messages_ready(Servers, RaName, 0), + wait_for_messages_pending_ack(Servers, RaName, 1), + amqp_channel:cast(Ch, #'basic.recover'{requeue = true}), + wait_for_messages_ready(Servers, RaName, 1), + wait_for_messages_pending_ack(Servers, RaName, 0). +%%---------------------------------------------------------------------------- + +declare(Ch, Q) -> + declare(Ch, Q, []). + +declare(Ch, Q, Args) -> + amqp_channel:call(Ch, #'queue.declare'{queue = Q, + durable = true, + auto_delete = false, + arguments = Args}). + +assert_queue_type(Server, Q, Expected) -> + Actual = get_queue_type(Server, Q), + Expected = Actual. + +get_queue_type(Server, Q) -> + QNameRes = rabbit_misc:r(<<"/">>, queue, Q), + {ok, AMQQueue} = + rpc:call(Server, rabbit_amqqueue, lookup, [QNameRes]), + AMQQueue#amqqueue.type. + +wait_for_messages(Config, Stats) -> + wait_for_messages(Config, lists:sort(Stats), 60). + +wait_for_messages(Config, Stats, 0) -> + ?assertEqual(Stats, + lists:sort( + filter_queues(Stats, + rabbit_ct_broker_helpers:rabbitmqctl_list( + Config, 0, ["list_queues", "name", "messages", "messages_ready", + "messages_unacknowledged"])))); +wait_for_messages(Config, Stats, N) -> + case lists:sort( + filter_queues(Stats, + rabbit_ct_broker_helpers:rabbitmqctl_list( + Config, 0, ["list_queues", "name", "messages", "messages_ready", + "messages_unacknowledged"]))) of + Stats0 when Stats0 == Stats -> + ok; + _ -> + timer:sleep(500), + wait_for_messages(Config, Stats, N - 1) + end. + +filter_queues(Expected, Got) -> + Keys = [K || [K, _, _, _] <- Expected], + lists:filter(fun([K, _, _, _]) -> + lists:member(K, Keys) + end, Got). + +publish(Ch, Queue) -> + ok = amqp_channel:call(Ch, + #'basic.publish'{routing_key = Queue}, + #amqp_msg{props = #'P_basic'{delivery_mode = 2}, + payload = <<"msg">>}). + +consume(Ch, Queue, NoAck) -> + {GetOk, _} = Reply = amqp_channel:call(Ch, #'basic.get'{queue = Queue, + no_ack = NoAck}), + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg">>}}, Reply), + GetOk#'basic.get_ok'.delivery_tag. + +consume_empty(Ch, Queue, NoAck) -> + ?assertMatch(#'basic.get_empty'{}, + amqp_channel:call(Ch, #'basic.get'{queue = Queue, + no_ack = NoAck})). + +subscribe(Ch, Queue, NoAck) -> + amqp_channel:subscribe(Ch, #'basic.consume'{queue = Queue, + no_ack = NoAck, + consumer_tag = <<"ctag">>}, + self()), + receive + #'basic.consume_ok'{consumer_tag = <<"ctag">>} -> + ok + end. + +qos(Ch, Prefetch, Global) -> + ?assertMatch(#'basic.qos_ok'{}, + amqp_channel:call(Ch, #'basic.qos'{global = Global, + prefetch_count = Prefetch})). + +receive_basic_deliver(Redelivered) -> + receive + {#'basic.deliver'{redelivered = R}, _} when R == Redelivered -> + ok + end. + +wait_for_cleanup(Server, Channel, Number) -> + wait_for_cleanup(Server, Channel, Number, 60). + +wait_for_cleanup(Server, Channel, Number, 0) -> + ?assertEqual(Number, length(rpc:call(Server, rabbit_channel, list_queue_states, [Channel]))); +wait_for_cleanup(Server, Channel, Number, N) -> + case length(rpc:call(Server, rabbit_channel, list_queue_states, [Channel])) of + Length when Number == Length -> + ok; + _ -> + timer:sleep(500), + wait_for_cleanup(Server, Channel, Number, N - 1) + end. + + +wait_for_messages_ready(Servers, QName, Ready) -> + wait_for_messages(Servers, QName, Ready, + fun rabbit_fifo:query_messages_ready/1, 60). + +wait_for_messages_pending_ack(Servers, QName, Ready) -> + wait_for_messages(Servers, QName, Ready, + fun rabbit_fifo:query_messages_checked_out/1, 60). + +wait_for_messages(Servers, QName, Number, Fun, 0) -> + Msgs = dirty_query(Servers, QName, Fun), + Totals = lists:map(fun(M) when is_map(M) -> + maps:size(M); + (_) -> + -1 + end, Msgs), + ?assertEqual(Totals, [Number || _ <- lists:seq(1, length(Servers))]); +wait_for_messages(Servers, QName, Number, Fun, N) -> + Msgs = dirty_query(Servers, QName, Fun), + case lists:all(fun(M) when is_map(M) -> + maps:size(M) == Number; + (_) -> + false + end, Msgs) of + true -> + ok; + _ -> + timer:sleep(500), + wait_for_messages(Servers, QName, Number, Fun, N - 1) + end. + +dirty_query(Servers, QName, Fun) -> + lists:map( + fun(N) -> + case rpc:call(N, ra, local_query, [{QName, N}, Fun]) of + {ok, {_, Msgs}, _} -> + Msgs; + _ -> + undefined + end + end, Servers). + +wait_until(Condition) -> + wait_until(Condition, 60). + +wait_until(Condition, 0) -> + ?assertEqual(true, Condition()); +wait_until(Condition, N) -> + case Condition() of + true -> + ok; + _ -> + timer:sleep(500), + wait_until(Condition, N - 1) + end. + +force_leader_change(Leader, Servers, Q) -> + RaName = ra_name(Q), + [F1, _] = Servers -- [Leader], + ok = rpc:call(F1, ra, trigger_election, [{RaName, F1}]), + case ra:members({RaName, Leader}) of + {ok, _, {_, Leader}} -> + %% Leader has been re-elected + force_leader_change(Leader, Servers, Q); + {ok, _, _} -> + %% Leader has changed + ok + end. + +delete_queues() -> + [rabbit_amqqueue:delete(Q, false, false, <<"dummy">>) + || Q <- rabbit_amqqueue:list()]. + +stop_node(Config, Server) -> + rabbit_ct_broker_helpers:rabbitmqctl(Config, Server, ["stop"]). diff --git a/test/rabbit_fifo_SUITE.erl b/test/rabbit_fifo_SUITE.erl new file mode 100644 index 000000000000..a2e22afc2e52 --- /dev/null +++ b/test/rabbit_fifo_SUITE.erl @@ -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.