Stream Queue
This is an aggregated commit of all changes related to the initial implementation of queue types and on top of that the stream queue type. The varios commit messages have simply been included mostly un-edited below. Make rabbit_amqqueue:not_found_or_absent_dirty/1 visible For use in the stream plugin. Use bigger retention policy on max-age test Set coordinator timeout to 30s Handle coordinator unavailable error Handle operator policies as maps when checking if is applicable Add is_policy_applicable/2 to classic queues Ignore restart commands if the stream has been deleted It could happen that after termination some of the monitors are still up and trigger writer/replica restarts Policy support on stream queues Remove subscription events on stream coordinator Ensure old leaders are removed from monitors Introduce delay when retrying a failed phase Note that this ensures monitor is setup, there was a bug where no monitor was really started when re-trying the same phase Restart replicas after leader election instead of relying on old monitors Use timer for stream coordinator retries Fix stream stats for members/online Multiple fixes for replica monitoring and restart Ensure pending commands are appended at the end and re-run Ensure phase is reset with the state Remove duplicates from replica list Restart current phase on state_enter Remove unused import Ensure rabbit is running when checking for stream quorum Restart replicas Add a close/1 function to queue types So that we can get a chance of cleaning up resources if needed. Stream queues close their osiris logs at this point. fix compiler errors stream-queue: take retention into account When calculating ready messages metrics. Add osiris to the list of rabbit deps Retry restart of replicas Do not restart replicas or leaders after receiving a delete cluster command Add more logging to the stream coordinator Monitor subscribed processes on the stream coordinator Memory breakdown for stream queues Update quorum queue event formatter rabbit_msg_record fixes Refactor channel confirms Remove old unconfirmed_messages module that was designed to handle multiple queue fan in logic including all ha mirrors etc. Replaced with simpler rabbit_confirms module that handles the fan out and leaves any queue specific logic (such as confirms from mirrors) to the queue type implemention. Also this module has a dedicated test module. Which is nice. Backward compatibility with 3.8.x events Supports mixed version cluster upgrades Match specification when stream queue already exists Max age retention for stream queues Stop all replicas before starting leader election stream: disallow global qos remove IS_CLASSIC|QUORUM macros Ensure only classic queues are notified on channel down This also removes the delivering_queues map in the channel state as it should not be needed for this and just cause additional unecessary accounting. Polish AMQP 1.0/0.9.1 properties conversion Support byte in application properties, handle 1-bit representation for booleans. Use binary in header for long AMQP 1.0 ID Fix AMQP 1.0 to 0.9.1 conversion Fix test due to incorrect type Convert timestamp application properties to/from seconds AMQP 1.0 uses milliseconds for timestamp and AMQP 0.9.1 uses seconds, so conversion needed. Dialyzer fixes Handle all message-id types AMQP 1.0 is more liberal in it's allowed types of message-id and correlation-id - this adds headers to describe the type of the data in the message_id / correlation_id properties and also handles the case where the data cannot fit by again using headers. Resize stream coordinator cluster when broker configuration changes convert timestamp to and fro seconds user_id should be a binary message annotations keys need to be symbols stream-queue: default exchange and routing key As these won't be present for data written using the rabbitmq-stream plugin. Add exchange, routing key as message annotations To the AMQP 1.0 formatted data to enable roundtrip. Add osiris logging module config And update logging config test suite. Restart election when start of new leader fails The node might have just gone down so we need to try another one Only aux keeps track of phase now, as it might change if the leader election fails Stream coordinator refactor - all state is kept on the ra machine Ensure any ra cluster not a qq is not cleaned up Fixes to recovery and monitoring Add AMQP 1.0 common to dependencies Add rabbit_msg_record module To handle conversions into internal stream storage format. Use rabbitmq-common stream-queue branch Use SSH for osiris dependency Stream coordinator: delete replica Stream coordinator: add replica Stream coordinator: leader failover Stream coordinator: declare and delete Test consuming from a random offset Previous offsets should not be delivered to consumers Consume from stream replicas and multiple test fixes Use max-length-bytes and add new max-segment-size Use SSH for osiris dependency Basic cancel for stream queues Publish stream queues and settle/reject/requeue refactor Consume from stream queues Fix recovery Publish stream messages Add/delete stream replicas Use safe queue names Set retention policy for stream queues Required by the ctl command [#171207092] Stream queue delete queue fix missing callback impl Stream queue declare Queue type abstraction And use the implementing module as the value of the amqqueue record `type` field. This will allow for easy dispatch to the queue type implementation. Queue type abstraction Move queue declare into rabbit_queue_type Move queue delete into queue type implementation Queue type: dequeue/basic_get Move info inside queue type abstraction Move policy change into queue type interface Add purge to queue type Add recovery to the queue type interface Rename amqqueue quorum_nodes field To a more generic an extensible opaque queue type specific map. Fix tests and handle classic API response Fix HA queue confirm bug All mirrors need to be present as queue names. This introduces context linking allowing additional queue refs to be linked to a single "master" queue ref contining the actual queue context. Fix issue with events of deleted queues Also update queue type smoke test to use a cluster by default. correct default value of amqqueue getter Move classic queues further inside queue type interface why [TrackerId] Dialyzer fixes
This commit is contained in:
parent
bdb6f9b508
commit
f20fa273e9
6
Makefile
6
Makefile
|
@ -117,7 +117,8 @@ define PROJECT_ENV
|
|||
%% Socket writer will run GC every 1 GB of outgoing data
|
||||
{writer_gc_threshold, 1000000000},
|
||||
%% interval at which connection/channel tracking executes post operations
|
||||
{tracking_execution_timeout, 15000}
|
||||
{tracking_execution_timeout, 15000},
|
||||
{stream_messages_soft_limit, 256}
|
||||
]
|
||||
endef
|
||||
|
||||
|
@ -130,11 +131,12 @@ APPS_DIR := $(CURDIR)/apps
|
|||
|
||||
LOCAL_DEPS = sasl rabbitmq_prelaunch os_mon inets compiler public_key crypto ssl syntax_tools xmerl
|
||||
BUILD_DEPS = rabbitmq_cli syslog
|
||||
DEPS = cuttlefish ranch lager rabbit_common ra sysmon_handler stdout_formatter recon observer_cli
|
||||
DEPS = cuttlefish ranch lager rabbit_common ra sysmon_handler stdout_formatter recon observer_cli osiris amqp10_common
|
||||
TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers amqp_client meck proper
|
||||
|
||||
dep_cuttlefish = hex 2.4.1
|
||||
dep_syslog = git https://github.com/schlagert/syslog 3.4.5
|
||||
dep_osiris = git https://github.com/rabbitmq/osiris master
|
||||
|
||||
define usage_xml_to_erl
|
||||
$(subst __,_,$(patsubst $(DOCS_DIR)/rabbitmq%.1.xml, src/rabbit_%_usage.erl, $(subst -,_,$(1))))
|
||||
|
|
|
@ -360,7 +360,9 @@ override_with_hard_coded_critical_config() ->
|
|||
{ra,
|
||||
%% Make Ra use a custom logger that dispatches to lager
|
||||
%% instead of the default OTP logger
|
||||
[{logger_module, rabbit_log_ra_shim}]}
|
||||
[{logger_module, rabbit_log_ra_shim}]},
|
||||
{osiris,
|
||||
[{logger_module, rabbit_log_osiris_shim}]}
|
||||
],
|
||||
apply_erlang_term_based_config(Config).
|
||||
|
||||
|
|
|
@ -54,6 +54,11 @@
|
|||
?amqqueue_v2_field_type(Q) =:= rabbit_quorum_queue) orelse
|
||||
false).
|
||||
|
||||
-define(amqqueue_is_stream(Q),
|
||||
(?is_amqqueue_v2(Q) andalso
|
||||
?amqqueue_v2_field_type(Q) =:= rabbit_stream_queue) orelse
|
||||
false).
|
||||
|
||||
-define(amqqueue_has_valid_pid(Q),
|
||||
((?is_amqqueue_v2(Q) andalso
|
||||
is_pid(?amqqueue_v2_field_pid(Q))) orelse
|
||||
|
|
|
@ -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 https://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-2020 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-streams "$@"
|
|
@ -0,0 +1,63 @@
|
|||
@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 https://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-2020 Pivotal Software, Inc. All rights reserved.
|
||||
REM
|
||||
|
||||
REM Scopes the variables to the current batch file
|
||||
setlocal
|
||||
|
||||
rem Preserve values that might contain exclamation marks before
|
||||
rem enabling delayed expansion
|
||||
set TDP0=%~dp0
|
||||
set STAR=%*
|
||||
setlocal enabledelayedexpansion
|
||||
|
||||
REM Get default settings with user overrides for (RABBITMQ_)<var_name>
|
||||
REM Non-empty defaults should be set in rabbitmq-env
|
||||
call "%TDP0%\rabbitmq-env.bat" %~n0
|
||||
|
||||
if not exist "!ERLANG_HOME!\bin\erl.exe" (
|
||||
echo.
|
||||
echo ******************************
|
||||
echo ERLANG_HOME not set correctly.
|
||||
echo ******************************
|
||||
echo.
|
||||
echo Please either set ERLANG_HOME to point to your Erlang installation or place the
|
||||
echo RabbitMQ server distribution in the Erlang lib folder.
|
||||
echo.
|
||||
exit /B 1
|
||||
)
|
||||
|
||||
REM Disable erl_crash.dump by default for control scripts.
|
||||
if not defined ERL_CRASH_DUMP_SECONDS (
|
||||
set ERL_CRASH_DUMP_SECONDS=0
|
||||
)
|
||||
|
||||
"!ERLANG_HOME!\bin\erl.exe" +B ^
|
||||
-boot !CLEAN_BOOT_FILE! ^
|
||||
-noinput -noshell -hidden -smp enable ^
|
||||
!RABBITMQ_CTL_ERL_ARGS! ^
|
||||
-run escript start ^
|
||||
-escript main rabbitmqctl_escript ^
|
||||
-extra "%RABBITMQ_HOME%\escript\rabbitmq-streams" !STAR!
|
||||
|
||||
if ERRORLEVEL 1 (
|
||||
exit /B %ERRORLEVEL%
|
||||
)
|
||||
|
||||
EXIT /B 0
|
||||
|
||||
endlocal
|
||||
endlocal
|
|
@ -548,8 +548,10 @@ set_recoverable_slaves(Queue, Slaves) ->
|
|||
% type_state (new in v2)
|
||||
|
||||
-spec get_type_state(amqqueue()) -> map().
|
||||
get_type_state(#amqqueue{type_state = TState}) -> TState;
|
||||
get_type_state(_) -> [].
|
||||
get_type_state(#amqqueue{type_state = TState}) ->
|
||||
TState;
|
||||
get_type_state(_) ->
|
||||
#{}.
|
||||
|
||||
-spec set_type_state(amqqueue(), map()) -> amqqueue().
|
||||
set_type_state(#amqqueue{} = Queue, TState) ->
|
||||
|
|
|
@ -114,6 +114,20 @@
|
|||
{requires, pre_boot},
|
||||
{enables, external_infrastructure}]}).
|
||||
|
||||
-rabbit_boot_step({rabbit_osiris_metrics,
|
||||
[{description, "osiris metrics scraper"},
|
||||
{mfa, {rabbit_sup, start_child,
|
||||
[rabbit_osiris_metrics]}},
|
||||
{requires, pre_boot},
|
||||
{enables, external_infrastructure}]}).
|
||||
|
||||
%% -rabbit_boot_step({rabbit_stream_coordinator,
|
||||
%% [{description, "stream queues coordinator"},
|
||||
%% {mfa, {rabbit_stream_coordinator, start,
|
||||
%% []}},
|
||||
%% {requires, pre_boot},
|
||||
%% {enables, external_infrastructure}]}).
|
||||
|
||||
-rabbit_boot_step({rabbit_event,
|
||||
[{description, "statistics event manager"},
|
||||
{mfa, {rabbit_sup, start_restartable_child,
|
||||
|
@ -255,7 +269,7 @@
|
|||
-include("rabbit_framing.hrl").
|
||||
-include("rabbit.hrl").
|
||||
|
||||
-define(APPS, [os_mon, mnesia, rabbit_common, rabbitmq_prelaunch, ra, sysmon_handler, rabbit]).
|
||||
-define(APPS, [os_mon, mnesia, rabbit_common, rabbitmq_prelaunch, ra, sysmon_handler, rabbit, osiris]).
|
||||
|
||||
-define(ASYNC_THREADS_WARNING_THRESHOLD, 8).
|
||||
|
||||
|
|
|
@ -10,13 +10,14 @@
|
|||
-export([warn_file_limit/0]).
|
||||
-export([recover/1, stop/1, start/1, declare/6, declare/7,
|
||||
delete_immediately/1, delete_exclusive/2, delete/4, purge/1,
|
||||
forget_all_durable/1, delete_crashed/1, delete_crashed/2,
|
||||
delete_crashed_internal/2]).
|
||||
forget_all_durable/1]).
|
||||
-export([pseudo_queue/2, pseudo_queue/3, immutable/1]).
|
||||
-export([lookup/1, lookup_many/1, not_found_or_absent/1, with/2, with/3, with_or_die/2,
|
||||
-export([lookup/1, lookup_many/1, not_found_or_absent/1, not_found_or_absent_dirty/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, deliver/3, requeue/4, ack/4, reject/5]).
|
||||
stat/1,
|
||||
requeue/3, ack/3, reject/4]).
|
||||
-export([not_found/1, absent/2]).
|
||||
-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,
|
||||
|
@ -27,9 +28,9 @@
|
|||
-export([list_by_type/1, sample_local_queues/0, sample_n_by_name/2, sample_n/2]).
|
||||
-export([force_event_refresh/1, notify_policy_changed/1]).
|
||||
-export([consumers/1, consumers_all/1, emit_consumers_all/4, consumer_info_keys/0]).
|
||||
-export([basic_get/6, basic_consume/12, basic_cancel/6, notify_decorators/1]).
|
||||
-export([basic_get/5, basic_consume/12, basic_cancel/5, 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/6]).
|
||||
-export([notify_down_all/2, notify_down_all/3, activate_limit_all/2, credit/5]).
|
||||
-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]).
|
||||
|
@ -53,6 +54,8 @@
|
|||
|
||||
-export([is_policy_applicable/2]).
|
||||
|
||||
-export([check_max_age/1]).
|
||||
|
||||
%% internal
|
||||
-export([internal_declare/2, internal_delete/2, run_backing_queue/3,
|
||||
set_ram_duration_target/2, set_maximum_since_use/2,
|
||||
|
@ -78,7 +81,8 @@
|
|||
-type qpids() :: [pid()].
|
||||
-type qlen() :: rabbit_types:ok(non_neg_integer()).
|
||||
-type qfun(A) :: fun ((amqqueue:amqqueue()) -> A | no_return()).
|
||||
-type qmsg() :: {name(), pid() | {atom(), pid()}, msg_id(), boolean(), rabbit_types:message()}.
|
||||
-type qmsg() :: {name(), pid() | {atom(), pid()}, msg_id(),
|
||||
boolean(), rabbit_types:message()}.
|
||||
-type msg_id() :: non_neg_integer().
|
||||
-type ok_or_errors() ::
|
||||
'ok' | {'error', [{'error' | 'exit' | 'throw', any()}]}.
|
||||
|
@ -86,7 +90,6 @@
|
|||
-type queue_not_found() :: not_found.
|
||||
-type queue_absent() :: {'absent', amqqueue:amqqueue(), absent_reason()}.
|
||||
-type not_found_or_absent() :: queue_not_found() | queue_absent().
|
||||
-type quorum_states() :: #{Name :: atom() => rabbit_fifo_client:state()}.
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
|
@ -110,34 +113,11 @@ warn_file_limit() ->
|
|||
end.
|
||||
|
||||
-spec recover(rabbit_types:vhost()) ->
|
||||
{RecoveredClassic :: [amqqueue:amqqueue()],
|
||||
FailedClassic :: [amqqueue:amqqueue()],
|
||||
Quorum :: [amqqueue:amqqueue()]}.
|
||||
|
||||
{Recovered :: [amqqueue:amqqueue()],
|
||||
Failed :: [amqqueue:amqqueue()]}.
|
||||
recover(VHost) ->
|
||||
AllClassic = find_local_durable_classic_queues(VHost),
|
||||
Quorum = find_local_quorum_queues(VHost),
|
||||
{RecoveredClassic, FailedClassic} = recover_classic_queues(VHost, AllClassic),
|
||||
{RecoveredClassic, FailedClassic, 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
|
||||
%% for further processing in recover_durable_queues.
|
||||
{ok, OrderedRecoveryTerms} =
|
||||
BQ:start(VHost, [amqqueue:get_name(Q) || Q <- Queues]),
|
||||
case rabbit_amqqueue_sup_sup:start_for_vhost(VHost) of
|
||||
{ok, _} ->
|
||||
RecoveredQs = recover_durable_queues(lists:zip(Queues, OrderedRecoveryTerms)),
|
||||
RecoveredNames = [amqqueue:get_name(Q) || Q <- RecoveredQs],
|
||||
FailedQueues = [Q || Q <- Queues,
|
||||
not lists:member(amqqueue:get_name(Q), RecoveredNames)],
|
||||
{RecoveredQs, FailedQueues};
|
||||
{error, Reason} ->
|
||||
rabbit_log:error("Failed to start queue supervisor for vhost '~s': ~s", [VHost, Reason]),
|
||||
throw({error, Reason})
|
||||
end.
|
||||
AllDurable = find_local_durable_queues(VHost),
|
||||
rabbit_queue_type:recover(VHost, AllDurable).
|
||||
|
||||
filter_pid_per_type(QPids) ->
|
||||
lists:partition(fun(QPid) -> ?IS_CLASSIC(QPid) end, QPids).
|
||||
|
@ -151,7 +131,6 @@ filter_resource_per_type(Resources) ->
|
|||
lists:partition(fun({_Resource, QPid}) -> ?IS_CLASSIC(QPid) end, Queues).
|
||||
|
||||
-spec stop(rabbit_types:vhost()) -> 'ok'.
|
||||
|
||||
stop(VHost) ->
|
||||
%% Classic queues
|
||||
ok = rabbit_amqqueue_sup_sup:stop_for_vhost(VHost),
|
||||
|
@ -178,67 +157,32 @@ mark_local_durable_queues_stopped(VHost) ->
|
|||
do_mark_local_durable_queues_stopped(VHost)).
|
||||
|
||||
do_mark_local_durable_queues_stopped(VHost) ->
|
||||
Qs = find_local_durable_classic_queues(VHost),
|
||||
Qs = find_local_durable_queues(VHost),
|
||||
rabbit_misc:execute_mnesia_transaction(
|
||||
fun() ->
|
||||
[ store_queue(amqqueue:set_state(Q, stopped))
|
||||
|| Q <- Qs,
|
||||
|| Q <- Qs, amqqueue:get_type(Q) =:= rabbit_classic_queue,
|
||||
amqqueue:get_state(Q) =/= stopped ]
|
||||
end).
|
||||
|
||||
find_local_quorum_queues(VHost) ->
|
||||
Node = node(),
|
||||
find_local_durable_queues(VHost) ->
|
||||
mnesia:async_dirty(
|
||||
fun () ->
|
||||
qlc:e(qlc:q([Q || Q <- mnesia:table(rabbit_durable_queue),
|
||||
amqqueue:get_vhost(Q) =:= VHost,
|
||||
amqqueue:is_quorum(Q) andalso
|
||||
(lists:member(Node, get_quorum_nodes(Q)))]))
|
||||
end).
|
||||
|
||||
find_local_durable_classic_queues(VHost) ->
|
||||
Node = node(),
|
||||
mnesia:async_dirty(
|
||||
fun () ->
|
||||
qlc:e(qlc:q([Q || Q <- mnesia:table(rabbit_durable_queue),
|
||||
amqqueue:get_vhost(Q) =:= VHost,
|
||||
amqqueue:is_classic(Q) andalso
|
||||
(is_local_to_node(amqqueue:get_pid(Q), 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, amqqueue:get_name(Q), read) =:= []
|
||||
orelse not rabbit_mnesia:is_process_alive(amqqueue:get_pid(Q))))
|
||||
]))
|
||||
qlc:e(
|
||||
qlc:q(
|
||||
[Q || Q <- mnesia:table(rabbit_durable_queue),
|
||||
amqqueue:get_vhost(Q) =:= VHost andalso
|
||||
rabbit_queue_type:is_recoverable(Q)
|
||||
]))
|
||||
end).
|
||||
|
||||
find_recoverable_queues() ->
|
||||
Node = node(),
|
||||
mnesia:async_dirty(
|
||||
fun () ->
|
||||
qlc:e(qlc:q([Q || Q <- mnesia:table(rabbit_durable_queue),
|
||||
(amqqueue:is_classic(Q) andalso
|
||||
(is_local_to_node(amqqueue:get_pid(Q), 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, amqqueue:get_name(Q), read) =:= []
|
||||
orelse not rabbit_mnesia:is_process_alive(amqqueue:get_pid(Q)))))
|
||||
orelse (amqqueue:is_quorum(Q) andalso lists:member(Node, get_quorum_nodes(Q)))
|
||||
]))
|
||||
rabbit_queue_type:is_recoverable(Q)]))
|
||||
end).
|
||||
|
||||
recover_durable_queues(QueuesAndRecoveryTerms) ->
|
||||
{Results, Failures} =
|
||||
gen_server2:mcall(
|
||||
[{rabbit_amqqueue_sup_sup:start_queue_process(node(), Q, recovery),
|
||||
{init, {self(), Terms}}} || {Q, Terms} <- QueuesAndRecoveryTerms]),
|
||||
[rabbit_log:error("Queue ~p failed to initialise: ~p~n",
|
||||
[Pid, Error]) || {Pid, Error} <- Failures],
|
||||
[Q || {_, {new, Q}} <- Results].
|
||||
|
||||
-spec declare(name(),
|
||||
boolean(),
|
||||
boolean(),
|
||||
|
@ -249,7 +193,6 @@ recover_durable_queues(QueuesAndRecoveryTerms) ->
|
|||
{'new', amqqueue:amqqueue(), rabbit_fifo_client:state()} |
|
||||
{'absent', amqqueue:amqqueue(), absent_reason()} |
|
||||
rabbit_types:channel_exit().
|
||||
|
||||
declare(QueueName, Durable, AutoDelete, Args, Owner, ActingUser) ->
|
||||
declare(QueueName, Durable, AutoDelete, Args, Owner, ActingUser, node()).
|
||||
|
||||
|
@ -266,18 +209,13 @@ declare(QueueName, Durable, AutoDelete, Args, Owner, ActingUser) ->
|
|||
rabbit_types:username(),
|
||||
node()) ->
|
||||
{'new' | 'existing' | 'owner_died', amqqueue:amqqueue()} |
|
||||
{'new', amqqueue:amqqueue(), rabbit_fifo_client:state()} |
|
||||
{'absent', amqqueue:amqqueue(), absent_reason()} |
|
||||
rabbit_types:channel_exit().
|
||||
|
||||
declare(QueueName = #resource{virtual_host = VHost}, Durable, AutoDelete, Args,
|
||||
Owner, ActingUser, Node) ->
|
||||
ok = check_declare_arguments(QueueName, Args),
|
||||
Type = get_queue_type(Args),
|
||||
TypeIsAllowed =
|
||||
Type =:= rabbit_classic_queue orelse
|
||||
rabbit_feature_flags:is_enabled(quorum_queue),
|
||||
case TypeIsAllowed of
|
||||
case rabbit_queue_type:is_enabled(Type) of
|
||||
true ->
|
||||
Q0 = amqqueue:new(QueueName,
|
||||
none,
|
||||
|
@ -290,7 +228,7 @@ declare(QueueName = #resource{virtual_host = VHost}, Durable, AutoDelete, Args,
|
|||
Type),
|
||||
Q = rabbit_queue_decorator:set(
|
||||
rabbit_policy:set(Q0)),
|
||||
do_declare(Q, Node);
|
||||
rabbit_queue_type:declare(Q, Node);
|
||||
false ->
|
||||
rabbit_misc:protocol_error(
|
||||
internal_error,
|
||||
|
@ -299,49 +237,12 @@ declare(QueueName = #resource{virtual_host = VHost}, Durable, AutoDelete, Args,
|
|||
[rabbit_misc:rs(QueueName), Type, Node])
|
||||
end.
|
||||
|
||||
do_declare(Q, Node) when ?amqqueue_is_classic(Q) ->
|
||||
declare_classic_queue(Q, Node);
|
||||
do_declare(Q, _Node) when ?amqqueue_is_quorum(Q) ->
|
||||
rabbit_quorum_queue:declare(Q).
|
||||
|
||||
declare_classic_queue(Q, Node) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
VHost = amqqueue:get_vhost(Q),
|
||||
Node1 = case Node of
|
||||
{ignore_location, Node0} ->
|
||||
Node0;
|
||||
_ ->
|
||||
case rabbit_queue_master_location_misc:get_location(Q) of
|
||||
{ok, Node0} -> Node0;
|
||||
undefined -> Node;
|
||||
{error, _} -> Node
|
||||
end
|
||||
end,
|
||||
Node1 = rabbit_mirror_queue_misc:initial_queue_node(Q, Node1),
|
||||
case rabbit_vhost_sup_sup:get_vhost_sup(VHost, Node1) of
|
||||
{ok, _} ->
|
||||
gen_server2:call(
|
||||
rabbit_amqqueue_sup_sup:start_queue_process(Node1, Q, declare),
|
||||
{init, new}, infinity);
|
||||
{error, Error} ->
|
||||
rabbit_misc:protocol_error(internal_error,
|
||||
"Cannot declare a queue '~s' on node '~s': ~255p",
|
||||
[rabbit_misc:rs(QName), Node1, Error])
|
||||
end.
|
||||
|
||||
get_queue_type(Args) ->
|
||||
case rabbit_misc:table_lookup(Args, <<"x-queue-type">>) of
|
||||
undefined ->
|
||||
rabbit_classic_queue;
|
||||
rabbit_queue_type:default();
|
||||
{_, V} ->
|
||||
%% TODO: this mapping of "friendly" queue type name to the
|
||||
%% implementing module should be part of some kind of registry
|
||||
case V of
|
||||
<<"quorum">> ->
|
||||
rabbit_quorum_queue;
|
||||
<<"classic">> ->
|
||||
rabbit_classic_queue
|
||||
end
|
||||
rabbit_queue_type:discover(V)
|
||||
end.
|
||||
|
||||
-spec internal_declare(amqqueue:amqqueue(), boolean()) ->
|
||||
|
@ -451,14 +352,12 @@ policy_changed(Q1, Q2) ->
|
|||
[ok = M:policy_changed(Q1, Q2) || M <- lists:usort(D1 ++ D2)],
|
||||
%% Make sure we emit a stats event even if nothing
|
||||
%% mirroring-related has changed - the policy may have changed anyway.
|
||||
notify_policy_changed(Q1).
|
||||
notify_policy_changed(Q2).
|
||||
|
||||
is_policy_applicable(QName, Policy) ->
|
||||
case lookup(QName) of
|
||||
{ok, Q} when ?amqqueue_is_quorum(Q) ->
|
||||
rabbit_quorum_queue:is_policy_applicable(Q, Policy);
|
||||
{ok, Q} when ?amqqueue_is_classic(Q) ->
|
||||
rabbit_amqqueue_process:is_policy_applicable(Q, Policy);
|
||||
{ok, Q} ->
|
||||
rabbit_queue_type:is_policy_applicable(Q, Policy);
|
||||
_ ->
|
||||
%% Defaults to previous behaviour. Apply always
|
||||
true
|
||||
|
@ -872,7 +771,9 @@ declare_args() ->
|
|||
{<<"x-queue-mode">>, fun check_queue_mode/2},
|
||||
{<<"x-single-active-consumer">>, fun check_single_active_consumer_arg/2},
|
||||
{<<"x-queue-type">>, fun check_queue_type/2},
|
||||
{<<"x-quorum-initial-group-size">>, fun check_default_quorum_initial_group_size_arg/2}].
|
||||
{<<"x-quorum-initial-group-size">>, fun check_default_quorum_initial_group_size_arg/2},
|
||||
{<<"x-max-age">>, fun check_max_age_arg/2},
|
||||
{<<"x-max-segment-size">>, fun check_non_neg_int_arg/2}].
|
||||
|
||||
consume_args() -> [{<<"x-priority">>, fun check_int_arg/2},
|
||||
{<<"x-cancel-on-ha-failover">>, fun check_bool_arg/2}].
|
||||
|
@ -926,6 +827,48 @@ check_default_quorum_initial_group_size_arg({Type, Val}, Args) ->
|
|||
Error -> Error
|
||||
end.
|
||||
|
||||
check_max_age_arg({longstr, Val}, _Args) ->
|
||||
case check_max_age(Val) of
|
||||
{error, _} = E ->
|
||||
E;
|
||||
_ ->
|
||||
ok
|
||||
end;
|
||||
check_max_age_arg({Type, _}, _Args) ->
|
||||
{error, {unacceptable_type, Type}}.
|
||||
|
||||
check_max_age(MaxAge) ->
|
||||
case re:run(MaxAge, "(^[0-9]*)(.*)", [{capture, all_but_first, list}]) of
|
||||
{match, [Value, Unit]} ->
|
||||
case list_to_integer(Value) of
|
||||
I when I > 0 ->
|
||||
case lists:member(Unit, ["Y", "M", "D", "h", "m", "s"]) of
|
||||
true ->
|
||||
Int = list_to_integer(Value),
|
||||
Int * unit_value_in_ms(Unit);
|
||||
false ->
|
||||
{error, invalid_max_age}
|
||||
end;
|
||||
_ ->
|
||||
{error, invalid_max_age}
|
||||
end;
|
||||
_ ->
|
||||
{error, invalid_max_age}
|
||||
end.
|
||||
|
||||
unit_value_in_ms("Y") ->
|
||||
365 * unit_value_in_ms("D");
|
||||
unit_value_in_ms("M") ->
|
||||
30 * unit_value_in_ms("D");
|
||||
unit_value_in_ms("D") ->
|
||||
24 * unit_value_in_ms("h");
|
||||
unit_value_in_ms("h") ->
|
||||
3600 * unit_value_in_ms("s");
|
||||
unit_value_in_ms("m") ->
|
||||
60 * unit_value_in_ms("s");
|
||||
unit_value_in_ms("s") ->
|
||||
1000.
|
||||
|
||||
%% Note that the validity of x-dead-letter-exchange is already verified
|
||||
%% by rabbit_channel's queue.declare handler.
|
||||
check_dlxname_arg({longstr, _}, _) -> ok;
|
||||
|
@ -958,7 +901,7 @@ check_queue_mode({Type, _}, _Args) ->
|
|||
{error, {unacceptable_type, Type}}.
|
||||
|
||||
check_queue_type({longstr, Val}, _Args) ->
|
||||
case lists:member(Val, [<<"classic">>, <<"quorum">>]) of
|
||||
case lists:member(Val, [<<"classic">>, <<"quorum">>, <<"stream">>]) of
|
||||
true -> ok;
|
||||
false -> {error, invalid_queue_type}
|
||||
end;
|
||||
|
@ -1051,7 +994,7 @@ list_by_type(Type) ->
|
|||
list_local_quorum_queue_names() ->
|
||||
[ amqqueue:get_name(Q) || Q <- list_by_type(quorum),
|
||||
amqqueue:get_state(Q) =/= crashed,
|
||||
lists:member(node(), get_quorum_nodes(Q))].
|
||||
lists:member(node(), get_quorum_nodes(Q))].
|
||||
|
||||
-spec list_local_quorum_queues() -> [amqqueue:amqqueue()].
|
||||
list_local_quorum_queues() ->
|
||||
|
@ -1067,10 +1010,13 @@ list_local_leaders() ->
|
|||
|
||||
-spec list_local_followers() -> [amqqueue:amqqueue()].
|
||||
list_local_followers() ->
|
||||
[ Q || Q <- list(),
|
||||
[Q
|
||||
|| Q <- list(),
|
||||
amqqueue:is_quorum(Q),
|
||||
amqqueue:get_state(Q) =/= crashed, amqqueue:get_leader(Q) =/= node(),
|
||||
lists:member(node(), get_quorum_nodes(Q))].
|
||||
amqqueue:get_state(Q) =/= crashed,
|
||||
amqqueue:get_leader(Q) =/= node(),
|
||||
rabbit_quorum_queue:is_recoverable(Q)
|
||||
].
|
||||
|
||||
-spec list_local_mirrored_classic_queues() -> [amqqueue:amqqueue()].
|
||||
list_local_mirrored_classic_queues() ->
|
||||
|
@ -1243,28 +1189,14 @@ format(Q) -> rabbit_amqqueue_process:format(Q).
|
|||
|
||||
-spec info(amqqueue:amqqueue()) -> rabbit_types:infos().
|
||||
|
||||
info(Q) when ?amqqueue_is_quorum(Q) -> rabbit_quorum_queue:info(Q);
|
||||
info(Q) when ?amqqueue_state_is(Q, crashed) -> info_down(Q, crashed);
|
||||
info(Q) when ?amqqueue_state_is(Q, stopped) -> info_down(Q, stopped);
|
||||
info(Q) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
delegate:invoke(QPid, {gen_server2, call, [info, infinity]}).
|
||||
info(Q) when ?is_amqqueue(Q) -> rabbit_queue_type:info(Q, all_keys).
|
||||
|
||||
|
||||
-spec info(amqqueue:amqqueue(), rabbit_types:info_keys()) ->
|
||||
rabbit_types:infos().
|
||||
|
||||
info(Q, Items) when ?amqqueue_is_quorum(Q) ->
|
||||
rabbit_quorum_queue:info(Q, Items);
|
||||
info(Q, Items) when ?amqqueue_state_is(Q, crashed) ->
|
||||
info_down(Q, Items, crashed);
|
||||
info(Q, Items) when ?amqqueue_state_is(Q, stopped) ->
|
||||
info_down(Q, Items, stopped);
|
||||
info(Q, Items) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
case delegate:invoke(QPid, {gen_server2, call, [{info, Items}, infinity]}) of
|
||||
{ok, Res} -> Res;
|
||||
{error, Error} -> throw(Error)
|
||||
end.
|
||||
info(Q, Items) when ?is_amqqueue(Q) ->
|
||||
rabbit_queue_type:info(Q, Items).
|
||||
|
||||
info_down(Q, DownReason) ->
|
||||
info_down(Q, rabbit_amqqueue_process:info_keys(), DownReason).
|
||||
|
@ -1367,14 +1299,8 @@ force_event_refresh(Ref) ->
|
|||
ok.
|
||||
|
||||
-spec notify_policy_changed(amqqueue:amqqueue()) -> 'ok'.
|
||||
|
||||
notify_policy_changed(Q) when ?amqqueue_is_classic(Q) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
gen_server2:cast(QPid, policy_changed);
|
||||
notify_policy_changed(Q) when ?amqqueue_is_quorum(Q) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
QName = amqqueue:get_name(Q),
|
||||
rabbit_quorum_queue:policy_changed(QName, QPid).
|
||||
notify_policy_changed(Q) when ?is_amqqueue(Q) ->
|
||||
rabbit_queue_type:policy_changed(Q).
|
||||
|
||||
-spec consumers(amqqueue:amqqueue()) ->
|
||||
[{pid(), rabbit_types:ctag(), boolean(), non_neg_integer(),
|
||||
|
@ -1389,7 +1315,12 @@ consumers(Q) when ?amqqueue_is_quorum(Q) ->
|
|||
case ra:local_query(QPid, fun rabbit_fifo:query_consumers/1) of
|
||||
{ok, {_, Result}, _} -> maps:values(Result);
|
||||
_ -> []
|
||||
end.
|
||||
end;
|
||||
consumers(Q) when ?amqqueue_is_stream(Q) ->
|
||||
%% TODO how??? they only exist on the channel
|
||||
%% we could list the offset listener on the writer but we don't even have a consumer tag,
|
||||
%% only a (channel) pid and offset
|
||||
[].
|
||||
|
||||
-spec consumer_info_keys() -> rabbit_types:info_keys().
|
||||
|
||||
|
@ -1425,9 +1356,8 @@ get_queue_consumer_info(Q, ConsumerInfoKeys) ->
|
|||
|
||||
-spec stat(amqqueue:amqqueue()) ->
|
||||
{'ok', non_neg_integer(), non_neg_integer()}.
|
||||
|
||||
stat(Q) when ?amqqueue_is_quorum(Q) -> rabbit_quorum_queue:stat(Q);
|
||||
stat(Q) -> delegate:invoke(amqqueue:get_pid(Q), {gen_server2, call, [stat, infinity]}).
|
||||
stat(Q) ->
|
||||
rabbit_queue_type:stat(Q).
|
||||
|
||||
-spec pid_of(amqqueue:amqqueue()) ->
|
||||
pid().
|
||||
|
@ -1476,162 +1406,46 @@ delete_immediately_by_resource(Resources) ->
|
|||
qlen() |
|
||||
rabbit_types:error('in_use') |
|
||||
rabbit_types:error('not_empty').
|
||||
|
||||
delete(Q,
|
||||
IfUnused, IfEmpty, ActingUser) when ?amqqueue_is_quorum(Q) ->
|
||||
rabbit_quorum_queue:delete(Q, IfUnused, IfEmpty, ActingUser);
|
||||
delete(Q, IfUnused, IfEmpty, ActingUser) ->
|
||||
case wait_for_promoted_or_stopped(Q) of
|
||||
{promoted, Q1} ->
|
||||
QPid = amqqueue:get_pid(Q1),
|
||||
delegate:invoke(QPid, {gen_server2, call, [{delete, IfUnused, IfEmpty, ActingUser}, infinity]});
|
||||
{stopped, Q1} ->
|
||||
#resource{name = Name, virtual_host = Vhost} = amqqueue:get_name(Q1),
|
||||
case IfEmpty of
|
||||
true ->
|
||||
rabbit_log:error("Queue ~s in vhost ~s has its master node down and "
|
||||
"no mirrors available or eligible for promotion. "
|
||||
"The queue may be non-empty. "
|
||||
"Refusing to force-delete.",
|
||||
[Name, Vhost]),
|
||||
{error, not_empty};
|
||||
false ->
|
||||
rabbit_log:warning("Queue ~s in vhost ~s has its master node is down and "
|
||||
"no mirrors available or eligible for promotion. "
|
||||
"Forcing queue deletion.",
|
||||
[Name, Vhost]),
|
||||
delete_crashed_internal(Q1, ActingUser),
|
||||
{ok, 0}
|
||||
end;
|
||||
{error, not_found} ->
|
||||
%% Assume the queue was deleted
|
||||
{ok, 0}
|
||||
end.
|
||||
rabbit_queue_type:delete(Q, IfUnused, IfEmpty, ActingUser).
|
||||
|
||||
-spec wait_for_promoted_or_stopped(amqqueue:amqqueue()) ->
|
||||
{promoted, amqqueue:amqqueue()} |
|
||||
{stopped, amqqueue:amqqueue()} |
|
||||
{error, not_found}.
|
||||
wait_for_promoted_or_stopped(Q0) ->
|
||||
QName = amqqueue:get_name(Q0),
|
||||
case lookup(QName) of
|
||||
{ok, Q} ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
SPids = amqqueue:get_slave_pids(Q),
|
||||
case rabbit_mnesia:is_process_alive(QPid) of
|
||||
true -> {promoted, Q};
|
||||
false ->
|
||||
case lists:any(fun(Pid) ->
|
||||
rabbit_mnesia:is_process_alive(Pid)
|
||||
end, SPids) of
|
||||
%% There is a live mirror. May be promoted
|
||||
true ->
|
||||
timer:sleep(100),
|
||||
wait_for_promoted_or_stopped(Q);
|
||||
%% All mirror pids are stopped.
|
||||
%% No process left for the queue
|
||||
false -> {stopped, Q}
|
||||
end
|
||||
end;
|
||||
{error, not_found} ->
|
||||
{error, not_found}
|
||||
end.
|
||||
-spec purge(amqqueue:amqqueue()) -> qlen().
|
||||
purge(Q) when ?is_amqqueue(Q) ->
|
||||
rabbit_queue_type:purge(Q).
|
||||
|
||||
-spec delete_crashed(amqqueue:amqqueue()) -> 'ok'.
|
||||
|
||||
delete_crashed(Q) ->
|
||||
delete_crashed(Q, ?INTERNAL_USER).
|
||||
|
||||
delete_crashed(Q, ActingUser) ->
|
||||
ok = rpc:call(amqqueue:qnode(Q), ?MODULE, delete_crashed_internal, [Q, ActingUser]).
|
||||
|
||||
-spec delete_crashed_internal(amqqueue:amqqueue(), rabbit_types:username()) -> 'ok'.
|
||||
|
||||
delete_crashed_internal(Q, ActingUser) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
{ok, BQ} = application:get_env(rabbit, backing_queue_module),
|
||||
BQ:delete_crashed(Q),
|
||||
ok = internal_delete(QName, ActingUser).
|
||||
|
||||
-spec purge(amqqueue:amqqueue()) -> {ok, qlen()}.
|
||||
|
||||
purge(Q) when ?amqqueue_is_classic(Q) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
delegate:invoke(QPid, {gen_server2, call, [purge, infinity]});
|
||||
purge(Q) when ?amqqueue_is_quorum(Q) ->
|
||||
NodeId = amqqueue:get_pid(Q),
|
||||
rabbit_quorum_queue:purge(NodeId).
|
||||
|
||||
-spec requeue(pid() | amqqueue:ra_server_id(),
|
||||
-spec requeue(name(),
|
||||
{rabbit_fifo:consumer_tag(), [msg_id()]},
|
||||
pid(),
|
||||
quorum_states()) ->
|
||||
'ok'.
|
||||
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.
|
||||
rabbit_queue_type:state()) ->
|
||||
{ok, rabbit_queue_type:state(), rabbit_queue_type:actions()}.
|
||||
requeue(QRef, {CTag, MsgIds}, QStates) ->
|
||||
reject(QRef, true, {CTag, MsgIds}, QStates).
|
||||
|
||||
-spec ack(pid(),
|
||||
-spec ack(name(),
|
||||
{rabbit_fifo:consumer_tag(), [msg_id()]},
|
||||
pid(),
|
||||
quorum_states()) ->
|
||||
quorum_states().
|
||||
rabbit_queue_type:state()) ->
|
||||
{ok, rabbit_queue_type:state(), rabbit_queue_type:actions()}.
|
||||
ack(QPid, {CTag, MsgIds}, QueueStates) ->
|
||||
rabbit_queue_type:settle(QPid, complete, CTag, MsgIds, QueueStates).
|
||||
|
||||
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.
|
||||
|
||||
-spec reject(pid() | amqqueue:ra_server_id(),
|
||||
-spec reject(name(),
|
||||
boolean(),
|
||||
{rabbit_fifo:consumer_tag(), [msg_id()]},
|
||||
pid(),
|
||||
quorum_states()) ->
|
||||
quorum_states().
|
||||
|
||||
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.
|
||||
rabbit_queue_type:state()) ->
|
||||
{ok, rabbit_queue_type:state(), rabbit_queue_type:actions()}.
|
||||
reject(QRef, Requeue, {CTag, MsgIds}, QStates) ->
|
||||
Op = case Requeue of
|
||||
true -> requeue;
|
||||
false -> discard
|
||||
end,
|
||||
rabbit_queue_type:settle(QRef, Op, CTag, MsgIds, QStates).
|
||||
|
||||
-spec notify_down_all(qpids(), pid()) -> ok_or_errors().
|
||||
|
||||
notify_down_all(QPids, ChPid) ->
|
||||
notify_down_all(QPids, ChPid, ?CHANNEL_OPERATION_TIMEOUT).
|
||||
|
||||
-spec notify_down_all(qpids(), pid(), non_neg_integer()) ->
|
||||
ok_or_errors().
|
||||
|
||||
notify_down_all(QPids, ChPid, Timeout) ->
|
||||
case rpc:call(node(), delegate, invoke,
|
||||
[QPids, {gen_server2, call, [{notify_down, ChPid}, infinity]}], Timeout) of
|
||||
|
@ -1657,130 +1471,55 @@ activate_limit_all(QRefs, ChPid) ->
|
|||
[{activate_limit, ChPid}]}).
|
||||
|
||||
-spec credit(amqqueue:amqqueue(),
|
||||
pid(),
|
||||
rabbit_types:ctag(),
|
||||
non_neg_integer(),
|
||||
boolean(),
|
||||
quorum_states()) ->
|
||||
{'ok', quorum_states()}.
|
||||
rabbit_queue_type:state()) ->
|
||||
rabbit_queue_type:state().
|
||||
credit(Q, CTag, Credit, Drain, QStates) ->
|
||||
rabbit_queue_type:credit(Q, CTag, Credit, Drain, QStates).
|
||||
|
||||
credit(Q, ChPid, CTag, Credit,
|
||||
Drain, QStates) when ?amqqueue_is_classic(Q) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
delegate:invoke_no_result(QPid, {gen_server2, cast,
|
||||
[{credit, ChPid, CTag, Credit, Drain}]}),
|
||||
{ok, QStates};
|
||||
credit(Q,
|
||||
_ChPid, CTag, Credit,
|
||||
Drain, QStates) when ?amqqueue_is_quorum(Q) ->
|
||||
{Name, _} = Id = amqqueue:get_pid(Q),
|
||||
QName = amqqueue:get_name(Q),
|
||||
QState0 = get_quorum_state(Id, QName, QStates),
|
||||
{ok, QState} = rabbit_quorum_queue:credit(CTag, Credit, Drain, QState0),
|
||||
{ok, maps:put(Name, QState, QStates)}.
|
||||
|
||||
-spec basic_get(amqqueue:amqqueue(), pid(), boolean(), pid(), rabbit_types:ctag(),
|
||||
#{Name :: atom() => rabbit_fifo_client:state()}) ->
|
||||
{'ok', non_neg_integer(), qmsg(), quorum_states()} |
|
||||
{'empty', quorum_states()} |
|
||||
-spec basic_get(amqqueue:amqqueue(), boolean(), pid(), rabbit_types:ctag(),
|
||||
rabbit_queue_type:state()) ->
|
||||
{'ok', non_neg_integer(), qmsg(), rabbit_queue_type:state()} |
|
||||
{'empty', rabbit_queue_type:state()} |
|
||||
rabbit_types:channel_exit().
|
||||
basic_get(Q, NoAck, LimiterPid, CTag, QStates0) ->
|
||||
rabbit_queue_type:dequeue(Q, NoAck, LimiterPid, CTag, QStates0).
|
||||
|
||||
basic_get(Q, ChPid, NoAck, LimiterPid, _CTag, _)
|
||||
when ?amqqueue_is_classic(Q) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
delegate:invoke(QPid, {gen_server2, call,
|
||||
[{basic_get, ChPid, NoAck, LimiterPid}, infinity]});
|
||||
basic_get(Q, _ChPid, NoAck, _LimiterPid, CTag, QStates)
|
||||
when ?amqqueue_is_quorum(Q) ->
|
||||
{Name, _} = Id = amqqueue:get_pid(Q),
|
||||
QName = amqqueue:get_name(Q),
|
||||
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.
|
||||
|
||||
-spec basic_consume
|
||||
(amqqueue:amqqueue(), boolean(), pid(), pid(), boolean(),
|
||||
non_neg_integer(), rabbit_types:ctag(), boolean(),
|
||||
rabbit_framing:amqp_table(), any(), rabbit_types:username(),
|
||||
#{Name :: atom() => rabbit_fifo_client:state()}) ->
|
||||
rabbit_types:ok_or_error('exclusive_consume_unavailable').
|
||||
|
||||
-spec basic_consume(amqqueue:amqqueue(), boolean(), pid(), pid(), boolean(),
|
||||
non_neg_integer(), rabbit_types:ctag(), boolean(),
|
||||
rabbit_framing:amqp_table(), any(), rabbit_types:username(),
|
||||
rabbit_queue_type:state()) ->
|
||||
{ok, rabbit_queue_type:state(), rabbit_queue_type:actions()} |
|
||||
{error, term()}.
|
||||
basic_consume(Q, NoAck, ChPid, LimiterPid,
|
||||
LimiterActive, ConsumerPrefetchCount, ConsumerTag,
|
||||
ExclusiveConsume, Args, OkMsg, ActingUser, QState)
|
||||
when ?amqqueue_is_classic(Q) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
ExclusiveConsume, Args, OkMsg, ActingUser, Contexts) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
%% first phase argument validation
|
||||
%% each queue type may do further validations
|
||||
ok = check_consume_arguments(QName, Args),
|
||||
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(Q, _NoAck, _ChPid,
|
||||
_LimiterPid, true, _ConsumerPrefetchCount, _ConsumerTag,
|
||||
_ExclusiveConsume, _Args, _OkMsg, _ActingUser, _QStates)
|
||||
when ?amqqueue_is_quorum(Q) ->
|
||||
{error, global_qos_not_supported_for_queue_type};
|
||||
basic_consume(Q,
|
||||
NoAck, ChPid, _LimiterPid, _LimiterActive, ConsumerPrefetchCount,
|
||||
ConsumerTag, ExclusiveConsume, Args, OkMsg,
|
||||
ActingUser, QStates)
|
||||
when ?amqqueue_is_quorum(Q) ->
|
||||
{Name, _} = Id = amqqueue:get_pid(Q),
|
||||
QName = amqqueue:get_name(Q),
|
||||
ok = check_consume_arguments(QName, Args),
|
||||
QState0 = get_quorum_state(Id, QName, QStates),
|
||||
case rabbit_quorum_queue:basic_consume(Q, NoAck, ChPid,
|
||||
ConsumerPrefetchCount,
|
||||
ConsumerTag,
|
||||
ExclusiveConsume, Args,
|
||||
ActingUser,
|
||||
OkMsg, QState0) of
|
||||
{ok, QState} ->
|
||||
{ok, maps:put(Name, QState, QStates)};
|
||||
{error, Reason} ->
|
||||
rabbit_misc:protocol_error(internal_error,
|
||||
"Cannot consume a message from quorum queue '~s': ~w",
|
||||
[rabbit_misc:rs(QName), Reason])
|
||||
end.
|
||||
Spec = #{no_ack => NoAck,
|
||||
channel_pid => ChPid,
|
||||
limiter_pid => LimiterPid,
|
||||
limiter_active => LimiterActive,
|
||||
prefetch_count => ConsumerPrefetchCount,
|
||||
consumer_tag => ConsumerTag,
|
||||
exclusive_consume => ExclusiveConsume,
|
||||
args => Args,
|
||||
ok_msg => OkMsg,
|
||||
acting_user => ActingUser},
|
||||
rabbit_queue_type:consume(Q, Spec, Contexts).
|
||||
|
||||
-spec basic_cancel
|
||||
(amqqueue:amqqueue(), pid(), rabbit_types:ctag(), any(),
|
||||
rabbit_types:username(), #{Name :: atom() => rabbit_fifo_client:state()}) ->
|
||||
'ok' | {'ok', #{Name :: atom() => rabbit_fifo_client:state()}}.
|
||||
|
||||
basic_cancel(Q, ChPid, ConsumerTag, OkMsg, ActingUser,
|
||||
QState)
|
||||
when ?amqqueue_is_classic(Q) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
case delegate:invoke(QPid, {gen_server2, call,
|
||||
[{basic_cancel, ChPid, ConsumerTag, OkMsg, ActingUser},
|
||||
infinity]}) of
|
||||
ok ->
|
||||
{ok, QState};
|
||||
Err -> Err
|
||||
end;
|
||||
basic_cancel(Q, ChPid,
|
||||
ConsumerTag, OkMsg, _ActingUser, QStates)
|
||||
when ?amqqueue_is_quorum(Q) ->
|
||||
{Name, _} = Id = amqqueue:get_pid(Q),
|
||||
QState0 = get_quorum_state(Id, QStates),
|
||||
{ok, QState} = rabbit_quorum_queue:basic_cancel(ConsumerTag, ChPid, OkMsg, QState0),
|
||||
{ok, maps:put(Name, QState, QStates)}.
|
||||
-spec basic_cancel(amqqueue:amqqueue(), rabbit_types:ctag(), any(),
|
||||
rabbit_types:username(),
|
||||
rabbit_queue_type:state()) ->
|
||||
{ok, rabbit_queue_type:state()} | {error, term()}.
|
||||
basic_cancel(Q, ConsumerTag, OkMsg, ActingUser, QStates) ->
|
||||
rabbit_queue_type:cancel(Q, ConsumerTag,
|
||||
OkMsg, ActingUser, QStates).
|
||||
|
||||
-spec notify_decorators(amqqueue:amqqueue()) -> 'ok'.
|
||||
|
||||
|
@ -1796,7 +1535,8 @@ notify_sent_queue_down(QPid) ->
|
|||
|
||||
-spec resume(pid(), pid()) -> 'ok'.
|
||||
|
||||
resume(QPid, ChPid) -> delegate:invoke_no_result(QPid, {gen_server2, cast, [{resume, ChPid}]}).
|
||||
resume(QPid, ChPid) -> delegate:invoke_no_result(QPid, {gen_server2, cast,
|
||||
[{resume, ChPid}]}).
|
||||
|
||||
internal_delete1(QueueName, OnlyDurable) ->
|
||||
internal_delete1(QueueName, OnlyDurable, normal).
|
||||
|
@ -1862,10 +1602,9 @@ forget_all_durable(Node) ->
|
|||
%% Try to promote a mirror while down - it should recover as a
|
||||
%% master. We try to take the oldest mirror here for best chance of
|
||||
%% recovery.
|
||||
forget_node_for_queue(DeadNode, Q)
|
||||
forget_node_for_queue(_DeadNode, Q)
|
||||
when ?amqqueue_is_quorum(Q) ->
|
||||
QN = get_quorum_nodes(Q),
|
||||
forget_node_for_queue(DeadNode, QN, Q);
|
||||
ok;
|
||||
forget_node_for_queue(DeadNode, Q) ->
|
||||
RS = amqqueue:get_recoverable_slaves(Q),
|
||||
forget_node_for_queue(DeadNode, RS, Q).
|
||||
|
@ -1887,7 +1626,7 @@ forget_node_for_queue(DeadNode, [H|T], Q) when ?is_amqqueue(Q) ->
|
|||
{false, _} -> forget_node_for_queue(DeadNode, T, Q);
|
||||
{true, rabbit_classic_queue} ->
|
||||
Q1 = amqqueue:set_pid(Q, rabbit_misc:node_to_fake_pid(H)),
|
||||
ok = mnesia:write(rabbit_durable_queue, Q1, write);
|
||||
ok = mnesia:write(rabbit_durable_queue, Q1, write);
|
||||
{true, rabbit_quorum_queue} ->
|
||||
ok
|
||||
end.
|
||||
|
@ -2109,110 +1848,7 @@ pseudo_queue(#resource{kind = queue} = QueueName, Pid, Durable)
|
|||
|
||||
immutable(Q) -> amqqueue:set_immutable(Q).
|
||||
|
||||
-spec deliver([amqqueue:amqqueue()], rabbit_types:delivery()) -> 'ok'.
|
||||
|
||||
deliver(Qs, Delivery) ->
|
||||
deliver(Qs, Delivery, untracked),
|
||||
ok.
|
||||
|
||||
-spec deliver([amqqueue:amqqueue()],
|
||||
rabbit_types:delivery(),
|
||||
quorum_states() | 'untracked') ->
|
||||
{qpids(),
|
||||
[{amqqueue:ra_server_id(), name()}],
|
||||
quorum_states()}.
|
||||
|
||||
deliver([], _Delivery, QueueState) ->
|
||||
%% /dev/null optimisation
|
||||
{[], [], QueueState};
|
||||
|
||||
deliver(Qs, Delivery = #delivery{flow = Flow,
|
||||
confirm = Confirm}, QueueState0) ->
|
||||
{Quorum, MPids, SPids} = qpids(Qs),
|
||||
QPids = MPids ++ SPids,
|
||||
%% We use up two credits to send to a mirror since the message
|
||||
%% arrives at the mirror from two directions. We will ack one when
|
||||
%% the mirror 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
|
||||
%% dictionary.
|
||||
flow -> [credit_flow:send(QPid) || QPid <- QPids],
|
||||
[credit_flow:send(QPid) || QPid <- SPids];
|
||||
noflow -> ok
|
||||
end,
|
||||
|
||||
%% We let mirrors know that they were being addressed as mirrors at
|
||||
%% the time - if they receive such a message from the channel
|
||||
%% after they have become master they should mark the message as
|
||||
%% 'delivered' since they do not know what the master may have
|
||||
%% done with it.
|
||||
MMsg = {deliver, Delivery, false},
|
||||
SMsg = {deliver, Delivery, true},
|
||||
delegate:invoke_no_result(MPids, {gen_server2, cast, [MMsg]}),
|
||||
delegate:invoke_no_result(SPids, {gen_server2, cast, [SMsg]}),
|
||||
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([Q]) when ?amqqueue_is_quorum(Q) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
{LocalName, LeaderNode} = amqqueue:get_pid(Q),
|
||||
{[{{LocalName, LeaderNode}, QName}], [], []}; %% opt
|
||||
qpids([Q]) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
SPids = amqqueue:get_slave_pids(Q),
|
||||
{[], [QPid], SPids}; %% opt
|
||||
qpids(Qs) ->
|
||||
{QuoPids, MPids, SPids} =
|
||||
lists:foldl(fun (Q,
|
||||
{QuoPidAcc, MPidAcc, SPidAcc})
|
||||
when ?amqqueue_is_quorum(Q) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
QName = amqqueue:get_name(Q),
|
||||
{[{QPid, QName} | QuoPidAcc], MPidAcc, SPidAcc};
|
||||
(Q,
|
||||
{QuoPidAcc, MPidAcc, SPidAcc}) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
SPids = amqqueue:get_slave_pids(Q),
|
||||
{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).
|
||||
|
||||
get_quorum_nodes(Q) when ?is_amqqueue(Q) ->
|
||||
get_quorum_nodes(Q) ->
|
||||
case amqqueue:get_type_state(Q) of
|
||||
#{nodes := Nodes} ->
|
||||
Nodes;
|
||||
|
|
|
@ -425,19 +425,9 @@ process_args_policy(State = #q{q = Q,
|
|||
{<<"queue-mode">>, fun res_arg/2, fun init_queue_mode/2}],
|
||||
drop_expired_msgs(
|
||||
lists:foldl(fun({Name, Resolve, Fun}, StateN) ->
|
||||
Fun(args_policy_lookup(Name, Resolve, Q), StateN)
|
||||
Fun(rabbit_queue_type_util:args_policy_lookup(Name, Resolve, Q), StateN)
|
||||
end, State#q{args_policy_version = N + 1}, ArgsTable)).
|
||||
|
||||
args_policy_lookup(Name, Resolve, Q) ->
|
||||
Args = amqqueue:get_arguments(Q),
|
||||
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.
|
||||
|
||||
res_arg(_PolVal, ArgVal) -> ArgVal.
|
||||
res_min(PolVal, ArgVal) -> erlang:min(PolVal, ArgVal).
|
||||
|
||||
|
@ -498,12 +488,13 @@ noreply(NewState) ->
|
|||
{NewState1, Timeout} = next_state(NewState),
|
||||
{noreply, ensure_stats_timer(ensure_rate_timer(NewState1)), Timeout}.
|
||||
|
||||
next_state(State = #q{backing_queue = BQ,
|
||||
next_state(State = #q{q = Q,
|
||||
backing_queue = BQ,
|
||||
backing_queue_state = BQS,
|
||||
msg_id_to_channel = MTC}) ->
|
||||
assert_invariant(State),
|
||||
{MsgIds, BQS1} = BQ:drain_confirmed(BQS),
|
||||
MTC1 = confirm_messages(MsgIds, MTC),
|
||||
MTC1 = confirm_messages(MsgIds, MTC, amqqueue:get_name(Q)),
|
||||
State1 = State#q{backing_queue_state = BQS1, msg_id_to_channel = MTC1},
|
||||
case BQ:needs_timeout(BQS1) of
|
||||
false -> {stop_sync_timer(State1), hibernate };
|
||||
|
@ -586,9 +577,9 @@ maybe_send_drained(WasEmpty, State) ->
|
|||
end,
|
||||
State.
|
||||
|
||||
confirm_messages([], MTC) ->
|
||||
confirm_messages([], MTC, _QName) ->
|
||||
MTC;
|
||||
confirm_messages(MsgIds, MTC) ->
|
||||
confirm_messages(MsgIds, MTC, QName) ->
|
||||
{CMs, MTC1} =
|
||||
lists:foldl(
|
||||
fun(MsgId, {CMs, MTC0}) ->
|
||||
|
@ -608,7 +599,7 @@ confirm_messages(MsgIds, MTC) ->
|
|||
end, {#{}, MTC}, MsgIds),
|
||||
maps:fold(
|
||||
fun(Pid, MsgSeqNos, _) ->
|
||||
rabbit_misc:confirm_to_sender(Pid, MsgSeqNos)
|
||||
rabbit_misc:confirm_to_sender(Pid, QName, MsgSeqNos)
|
||||
end,
|
||||
ok,
|
||||
CMs),
|
||||
|
@ -629,8 +620,9 @@ send_or_record_confirm(#delivery{confirm = true,
|
|||
{eventually, State#q{msg_id_to_channel = MTC1}};
|
||||
send_or_record_confirm(#delivery{confirm = true,
|
||||
sender = SenderPid,
|
||||
msg_seq_no = MsgSeqNo}, State) ->
|
||||
rabbit_misc:confirm_to_sender(SenderPid, [MsgSeqNo]),
|
||||
msg_seq_no = MsgSeqNo},
|
||||
#q{q = Q} = State) ->
|
||||
rabbit_misc:confirm_to_sender(SenderPid, amqqueue:get_name(Q), [MsgSeqNo]),
|
||||
{immediately, State}.
|
||||
|
||||
%% This feature was used by `rabbit_amqqueue_process` and
|
||||
|
@ -648,9 +640,9 @@ send_mandatory(#delivery{mandatory = true,
|
|||
discard(#delivery{confirm = Confirm,
|
||||
sender = SenderPid,
|
||||
flow = Flow,
|
||||
message = #basic_message{id = MsgId}}, BQ, BQS, MTC) ->
|
||||
message = #basic_message{id = MsgId}}, BQ, BQS, MTC, QName) ->
|
||||
MTC1 = case Confirm of
|
||||
true -> confirm_messages([MsgId], MTC);
|
||||
true -> confirm_messages([MsgId], MTC, QName);
|
||||
false -> MTC
|
||||
end,
|
||||
BQS1 = BQ:discard(MsgId, SenderPid, Flow, BQS),
|
||||
|
@ -679,7 +671,8 @@ run_message_queue(ActiveConsumersChanged, State) ->
|
|||
attempt_delivery(Delivery = #delivery{sender = SenderPid,
|
||||
flow = Flow,
|
||||
message = Message},
|
||||
Props, Delivered, State = #q{backing_queue = BQ,
|
||||
Props, Delivered, State = #q{q = Q,
|
||||
backing_queue = BQ,
|
||||
backing_queue_state = BQS,
|
||||
msg_id_to_channel = MTC}) ->
|
||||
case rabbit_queue_consumers:deliver(
|
||||
|
@ -689,7 +682,7 @@ attempt_delivery(Delivery = #delivery{sender = SenderPid,
|
|||
Message, Props, SenderPid, Flow, BQS),
|
||||
{{Message, Delivered, AckTag}, {BQS1, MTC}};
|
||||
(false) -> {{Message, Delivered, undefined},
|
||||
discard(Delivery, BQ, BQS, MTC)}
|
||||
discard(Delivery, BQ, BQS, MTC, amqqueue:get_name(Q))}
|
||||
end, qname(State), State#q.consumers, State#q.single_active_consumer_on, State#q.active_consumer) of
|
||||
{delivered, ActiveConsumersChanged, {BQS1, MTC1}, Consumers} ->
|
||||
{delivered, maybe_notify_decorators(
|
||||
|
@ -745,7 +738,7 @@ deliver_or_enqueue(Delivery = #delivery{message = Message,
|
|||
sender = SenderPid,
|
||||
flow = Flow},
|
||||
Delivered,
|
||||
State = #q{backing_queue = BQ}) ->
|
||||
State = #q{q = Q, backing_queue = BQ}) ->
|
||||
{Confirm, State1} = send_or_record_confirm(Delivery, State),
|
||||
Props = message_properties(Message, Confirm, State1),
|
||||
case attempt_delivery(Delivery, Props, Delivered, State1) of
|
||||
|
@ -755,7 +748,7 @@ deliver_or_enqueue(Delivery = #delivery{message = Message,
|
|||
{undelivered, State2 = #q{ttl = 0, dlx = undefined,
|
||||
backing_queue_state = BQS,
|
||||
msg_id_to_channel = MTC}} ->
|
||||
{BQS1, MTC1} = discard(Delivery, BQ, BQS, MTC),
|
||||
{BQS1, MTC1} = discard(Delivery, BQ, BQS, MTC, amqqueue:get_name(Q)),
|
||||
State2#q{backing_queue_state = BQS1, msg_id_to_channel = MTC1};
|
||||
{undelivered, State2 = #q{backing_queue_state = BQS}} ->
|
||||
|
||||
|
@ -809,10 +802,11 @@ send_reject_publish(#delivery{confirm = true,
|
|||
msg_seq_no = MsgSeqNo,
|
||||
message = #basic_message{id = MsgId}},
|
||||
_Delivered,
|
||||
State = #q{ backing_queue = BQ,
|
||||
State = #q{ q = Q,
|
||||
backing_queue = BQ,
|
||||
backing_queue_state = BQS,
|
||||
msg_id_to_channel = MTC}) ->
|
||||
gen_server2:cast(SenderPid, {reject_publish, MsgSeqNo, self()}),
|
||||
gen_server2:cast(SenderPid, {queue_event, Q, {reject_publish, MsgSeqNo, self()}}),
|
||||
|
||||
MTC1 = maps:remove(MsgId, MTC),
|
||||
BQS1 = BQ:discard(MsgId, SenderPid, Flow, BQS),
|
||||
|
@ -1273,7 +1267,7 @@ handle_call({init, Recover}, From, State) ->
|
|||
end;
|
||||
|
||||
handle_call(info, _From, State) ->
|
||||
reply(infos(info_keys(), State), State);
|
||||
reply({ok, infos(info_keys(), State)}, State);
|
||||
|
||||
handle_call({info, Items}, _From, State) ->
|
||||
try
|
||||
|
@ -1547,7 +1541,7 @@ handle_cast({deliver,
|
|||
noreply(maybe_deliver_or_enqueue(Delivery, SlaveWhenPublished, State1));
|
||||
%% [0] The second ack is since the channel thought we were a mirror at
|
||||
%% the time it published this message, so it used two credits (see
|
||||
%% rabbit_amqqueue:deliver/2).
|
||||
%% rabbit_queue_type:deliver/2).
|
||||
|
||||
handle_cast({ack, AckTags, ChPid}, State) ->
|
||||
noreply(ack(AckTags, ChPid, State));
|
||||
|
|
|
@ -69,7 +69,7 @@ publish(Delivery = #delivery{
|
|||
|
||||
publish(X, Delivery) ->
|
||||
Qs = rabbit_amqqueue:lookup(rabbit_exchange:route(X, Delivery)),
|
||||
rabbit_amqqueue:deliver(Qs, Delivery).
|
||||
rabbit_queue_type:deliver(Qs, Delivery, stateless).
|
||||
|
||||
-spec delivery
|
||||
(boolean(), boolean(), rabbit_types:message(), undefined | integer()) ->
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,452 @@
|
|||
-module(rabbit_classic_queue).
|
||||
-behaviour(rabbit_queue_type).
|
||||
|
||||
-include("amqqueue.hrl").
|
||||
-include_lib("rabbit_common/include/rabbit.hrl").
|
||||
|
||||
-record(?MODULE, {pid :: undefined | pid(), %% the current master pid
|
||||
qref :: term(), %% TODO
|
||||
unconfirmed = #{} :: #{non_neg_integer() => [pid()]}}).
|
||||
-define(STATE, ?MODULE).
|
||||
|
||||
-opaque state() :: #?STATE{}.
|
||||
|
||||
-export_type([state/0]).
|
||||
|
||||
-export([
|
||||
is_enabled/0,
|
||||
declare/2,
|
||||
delete/4,
|
||||
is_recoverable/1,
|
||||
recover/2,
|
||||
purge/1,
|
||||
policy_changed/1,
|
||||
stat/1,
|
||||
init/1,
|
||||
close/1,
|
||||
update/2,
|
||||
consume/3,
|
||||
cancel/5,
|
||||
handle_event/2,
|
||||
deliver/2,
|
||||
settle/4,
|
||||
credit/4,
|
||||
dequeue/4,
|
||||
info/2,
|
||||
state_info/1,
|
||||
is_policy_applicable/2
|
||||
]).
|
||||
|
||||
-export([delete_crashed/1,
|
||||
delete_crashed/2,
|
||||
delete_crashed_internal/2]).
|
||||
|
||||
is_enabled() -> true.
|
||||
|
||||
declare(Q, Node) when ?amqqueue_is_classic(Q) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
VHost = amqqueue:get_vhost(Q),
|
||||
Node1 = case Node of
|
||||
{ignore_location, Node0} ->
|
||||
Node0;
|
||||
_ ->
|
||||
case rabbit_queue_master_location_misc:get_location(Q) of
|
||||
{ok, Node0} -> Node0;
|
||||
{error, _} -> Node
|
||||
end
|
||||
end,
|
||||
Node1 = rabbit_mirror_queue_misc:initial_queue_node(Q, Node1),
|
||||
case rabbit_vhost_sup_sup:get_vhost_sup(VHost, Node1) of
|
||||
{ok, _} ->
|
||||
gen_server2:call(
|
||||
rabbit_amqqueue_sup_sup:start_queue_process(Node1, Q, declare),
|
||||
{init, new}, infinity);
|
||||
{error, Error} ->
|
||||
rabbit_misc:protocol_error(internal_error,
|
||||
"Cannot declare a queue '~s' on node '~s': ~255p",
|
||||
[rabbit_misc:rs(QName), Node1, Error])
|
||||
end.
|
||||
|
||||
delete(Q, IfUnused, IfEmpty, ActingUser) when ?amqqueue_is_classic(Q) ->
|
||||
case wait_for_promoted_or_stopped(Q) of
|
||||
{promoted, Q1} ->
|
||||
QPid = amqqueue:get_pid(Q1),
|
||||
delegate:invoke(QPid, {gen_server2, call,
|
||||
[{delete, IfUnused, IfEmpty, ActingUser},
|
||||
infinity]});
|
||||
{stopped, Q1} ->
|
||||
#resource{name = Name, virtual_host = Vhost} = amqqueue:get_name(Q1),
|
||||
case IfEmpty of
|
||||
true ->
|
||||
rabbit_log:error("Queue ~s in vhost ~s has its master node down and "
|
||||
"no mirrors available or eligible for promotion. "
|
||||
"The queue may be non-empty. "
|
||||
"Refusing to force-delete.",
|
||||
[Name, Vhost]),
|
||||
{error, not_empty};
|
||||
false ->
|
||||
rabbit_log:warning("Queue ~s in vhost ~s has its master node is down and "
|
||||
"no mirrors available or eligible for promotion. "
|
||||
"Forcing queue deletion.",
|
||||
[Name, Vhost]),
|
||||
delete_crashed_internal(Q1, ActingUser),
|
||||
{ok, 0}
|
||||
end;
|
||||
{error, not_found} ->
|
||||
%% Assume the queue was deleted
|
||||
{ok, 0}
|
||||
end.
|
||||
|
||||
is_recoverable(Q) when ?is_amqqueue(Q) ->
|
||||
Node = node(),
|
||||
Node =:= node(amqqueue:get_pid(Q)) 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, amqqueue:get_name(Q), read) =:= []
|
||||
orelse not rabbit_mnesia:is_process_alive(amqqueue:get_pid(Q))).
|
||||
|
||||
recover(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
|
||||
%% for further processing in recover_durable_queues.
|
||||
{ok, OrderedRecoveryTerms} =
|
||||
BQ:start(VHost, [amqqueue:get_name(Q) || Q <- Queues]),
|
||||
case rabbit_amqqueue_sup_sup:start_for_vhost(VHost) of
|
||||
{ok, _} ->
|
||||
RecoveredQs = recover_durable_queues(lists:zip(Queues,
|
||||
OrderedRecoveryTerms)),
|
||||
RecoveredNames = [amqqueue:get_name(Q) || Q <- RecoveredQs],
|
||||
FailedQueues = [Q || Q <- Queues,
|
||||
not lists:member(amqqueue:get_name(Q), RecoveredNames)],
|
||||
{RecoveredQs, FailedQueues};
|
||||
{error, Reason} ->
|
||||
rabbit_log:error("Failed to start queue supervisor for vhost '~s': ~s", [VHost, Reason]),
|
||||
throw({error, Reason})
|
||||
end.
|
||||
|
||||
-spec policy_changed(amqqueue:amqqueue()) -> ok.
|
||||
policy_changed(Q) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
gen_server2:cast(QPid, policy_changed).
|
||||
|
||||
stat(Q) ->
|
||||
delegate:invoke(amqqueue:get_pid(Q),
|
||||
{gen_server2, call, [stat, infinity]}).
|
||||
|
||||
-spec init(amqqueue:amqqueue()) -> state().
|
||||
init(Q) when ?amqqueue_is_classic(Q) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
#?STATE{pid = amqqueue:get_pid(Q),
|
||||
qref = QName}.
|
||||
|
||||
-spec close(state()) -> ok.
|
||||
close(_State) ->
|
||||
ok.
|
||||
|
||||
-spec update(amqqueue:amqqueue(), state()) -> state().
|
||||
update(Q, #?STATE{pid = Pid} = State) when ?amqqueue_is_classic(Q) ->
|
||||
case amqqueue:get_pid(Q) of
|
||||
Pid ->
|
||||
State;
|
||||
NewPid ->
|
||||
%% master pid is different, update
|
||||
State#?STATE{pid = NewPid}
|
||||
end.
|
||||
|
||||
consume(Q, Spec, State) when ?amqqueue_is_classic(Q) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
QRef = amqqueue:get_name(Q),
|
||||
#{no_ack := NoAck,
|
||||
channel_pid := ChPid,
|
||||
limiter_pid := LimiterPid,
|
||||
limiter_active := LimiterActive,
|
||||
prefetch_count := ConsumerPrefetchCount,
|
||||
consumer_tag := ConsumerTag,
|
||||
exclusive_consume := ExclusiveConsume,
|
||||
args := Args,
|
||||
ok_msg := OkMsg,
|
||||
acting_user := ActingUser} = Spec,
|
||||
case delegate:invoke(QPid,
|
||||
{gen_server2, call,
|
||||
[{basic_consume, NoAck, ChPid, LimiterPid,
|
||||
LimiterActive, ConsumerPrefetchCount, ConsumerTag,
|
||||
ExclusiveConsume, Args, OkMsg, ActingUser},
|
||||
infinity]}) of
|
||||
ok ->
|
||||
%% ask the host process to monitor this pid
|
||||
%% TODO: track pids as they change
|
||||
{ok, State#?STATE{pid = QPid}, [{monitor, QPid, QRef}]};
|
||||
Err ->
|
||||
Err
|
||||
end.
|
||||
|
||||
cancel(Q, ConsumerTag, OkMsg, ActingUser, State) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
case delegate:invoke(QPid, {gen_server2, call,
|
||||
[{basic_cancel, self(), ConsumerTag,
|
||||
OkMsg, ActingUser}, infinity]}) of
|
||||
ok ->
|
||||
{ok, State};
|
||||
Err -> Err
|
||||
end.
|
||||
|
||||
-spec settle(rabbit_queue_type:settle_op(), rabbit_types:ctag(),
|
||||
[non_neg_integer()], state()) ->
|
||||
{state(), rabbit_queue_type:actions()}.
|
||||
settle(complete, _CTag, MsgIds, State) ->
|
||||
delegate:invoke_no_result(State#?STATE.pid,
|
||||
{gen_server2, cast, [{ack, MsgIds, self()}]}),
|
||||
{State, []};
|
||||
settle(Op, _CTag, MsgIds, State) ->
|
||||
ChPid = self(),
|
||||
ok = delegate:invoke_no_result(State#?STATE.pid,
|
||||
{gen_server2, cast,
|
||||
[{reject, Op == requeue, MsgIds, ChPid}]}),
|
||||
{State, []}.
|
||||
|
||||
credit(CTag, Credit, Drain, State) ->
|
||||
ChPid = self(),
|
||||
delegate:invoke_no_result(State#?STATE.pid,
|
||||
{gen_server2, cast,
|
||||
[{credit, ChPid, CTag, Credit, Drain}]}),
|
||||
State.
|
||||
|
||||
handle_event({confirm, MsgSeqNos, Pid}, #?STATE{qref = QRef,
|
||||
unconfirmed = U0} = State) ->
|
||||
{Unconfirmed, ConfirmedSeqNos} = confirm_seq_nos(MsgSeqNos, Pid, U0),
|
||||
Actions = [{settled, QRef, ConfirmedSeqNos}],
|
||||
%% handle confirm event from queues
|
||||
%% in this case the classic queue should track each individual publish and
|
||||
%% the processes involved and only emit a settle action once they have all
|
||||
%% been received (or DOWN has been received).
|
||||
%% Hence this part of the confirm logic is queue specific.
|
||||
{ok, State#?STATE{unconfirmed = Unconfirmed}, Actions};
|
||||
handle_event({reject_publish, SeqNo, _QPid},
|
||||
#?STATE{qref = QRef,
|
||||
unconfirmed = U0} = State) ->
|
||||
%% It does not matter which queue rejected the message,
|
||||
%% if any queue did, it should not be confirmed.
|
||||
{U, Rejected} = reject_seq_no(SeqNo, U0),
|
||||
Actions = [{rejected, QRef, Rejected}],
|
||||
{ok, State#?STATE{unconfirmed = U}, Actions};
|
||||
handle_event({down, Pid, Info}, #?STATE{qref = QRef,
|
||||
pid = MasterPid,
|
||||
unconfirmed = U0} = State0) ->
|
||||
case rabbit_misc:is_abnormal_exit(Info) of
|
||||
false when Info =:= normal andalso Pid == MasterPid ->
|
||||
%% queue was deleted and masterpid is down
|
||||
eol;
|
||||
false ->
|
||||
%% this assumes the mirror isn't part of the active set
|
||||
%% so we can confirm this particular pid
|
||||
MsgSeqNos = maps:keys(
|
||||
maps:filter(fun (_, Pids) ->
|
||||
lists:member(Pid, Pids)
|
||||
end, U0)),
|
||||
%% if the exit is normal, treat it as a "confirm"
|
||||
{Unconfirmed, ConfirmedSeqNos} = confirm_seq_nos(MsgSeqNos, Pid, U0),
|
||||
Actions = [{settled, QRef, ConfirmedSeqNos}],
|
||||
{ok, State0#?STATE{unconfirmed = Unconfirmed}, Actions};
|
||||
true ->
|
||||
%% any abnormal exit should be considered a full reject of the
|
||||
%% oustanding message ids - If the message didn't get to all
|
||||
%% mirrors we have to assume it will never get there
|
||||
MsgIds = maps:fold(
|
||||
fun (SeqNo, Pids, Acc) ->
|
||||
case lists:member(Pid, Pids) of
|
||||
true ->
|
||||
[SeqNo | Acc];
|
||||
false ->
|
||||
Acc
|
||||
end
|
||||
end, [], U0),
|
||||
U = maps:without(MsgIds, U0),
|
||||
{ok, State0#?STATE{unconfirmed = U}, [{rejected, QRef, MsgIds}]}
|
||||
end.
|
||||
|
||||
-spec deliver([{amqqueue:amqqueue(), state()}],
|
||||
Delivery :: term()) ->
|
||||
{[{amqqueue:amqqueue(), state()}], rabbit_queue_type:actions()}.
|
||||
deliver(Qs0, #delivery{flow = Flow,
|
||||
msg_seq_no = MsgNo,
|
||||
message = #basic_message{exchange_name = _Ex},
|
||||
confirm = _Confirm} = Delivery) ->
|
||||
%% TODO: record master and slaves for confirm processing
|
||||
{MPids, SPids, Qs, Actions} = qpids(Qs0, MsgNo),
|
||||
QPids = MPids ++ SPids,
|
||||
case Flow of
|
||||
%% Here we are tracking messages sent by the rabbit_channel
|
||||
%% process. We are accessing the rabbit_channel process
|
||||
%% dictionary.
|
||||
flow -> [credit_flow:send(QPid) || QPid <- QPids],
|
||||
[credit_flow:send(QPid) || QPid <- SPids];
|
||||
noflow -> ok
|
||||
end,
|
||||
MMsg = {deliver, Delivery, false},
|
||||
SMsg = {deliver, Delivery, true},
|
||||
delegate:invoke_no_result(MPids, {gen_server2, cast, [MMsg]}),
|
||||
delegate:invoke_no_result(SPids, {gen_server2, cast, [SMsg]}),
|
||||
{Qs, Actions}.
|
||||
|
||||
|
||||
-spec dequeue(NoAck :: boolean(), LimiterPid :: pid(),
|
||||
rabbit_types:ctag(), state()) ->
|
||||
{ok, Count :: non_neg_integer(), rabbit_amqqueue:qmsg(), state()} |
|
||||
{empty, state()}.
|
||||
dequeue(NoAck, LimiterPid, _CTag, State) ->
|
||||
QPid = State#?STATE.pid,
|
||||
case delegate:invoke(QPid, {gen_server2, call,
|
||||
[{basic_get, self(), NoAck, LimiterPid}, infinity]}) of
|
||||
empty ->
|
||||
{empty, State};
|
||||
{ok, Count, Msg} ->
|
||||
{ok, Count, Msg, State}
|
||||
end.
|
||||
|
||||
-spec state_info(state()) -> #{atom() := term()}.
|
||||
state_info(_State) ->
|
||||
#{}.
|
||||
|
||||
%% general queue info
|
||||
-spec info(amqqueue:amqqueue(), all_keys | rabbit_types:info_keys()) ->
|
||||
rabbit_types:infos().
|
||||
info(Q, Items) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
Req = case Items of
|
||||
all_keys -> info;
|
||||
_ -> {info, Items}
|
||||
end,
|
||||
case delegate:invoke(QPid, {gen_server2, call, [Req, infinity]}) of
|
||||
{ok, Result} ->
|
||||
Result;
|
||||
{error, _Err} ->
|
||||
[]
|
||||
end.
|
||||
|
||||
-spec purge(amqqueue:amqqueue()) ->
|
||||
{ok, non_neg_integer()}.
|
||||
purge(Q) when ?is_amqqueue(Q) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
delegate:invoke(QPid, {gen_server2, call, [purge, infinity]}).
|
||||
|
||||
qpids(Qs, MsgNo) ->
|
||||
lists:foldl(
|
||||
fun ({Q, S0}, {MPidAcc, SPidAcc, Qs0, Actions0}) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
SPids = amqqueue:get_slave_pids(Q),
|
||||
QRef = amqqueue:get_name(Q),
|
||||
Actions = [{monitor, QPid, QRef}
|
||||
| [{monitor, P, QRef} || P <- SPids]] ++ Actions0,
|
||||
%% confirm record
|
||||
S = case S0 of
|
||||
#?STATE{unconfirmed = U0} ->
|
||||
Rec = [QPid | SPids],
|
||||
U = U0#{MsgNo => Rec},
|
||||
S0#?STATE{unconfirmed = U};
|
||||
stateless ->
|
||||
S0
|
||||
end,
|
||||
{[QPid | MPidAcc],
|
||||
SPidAcc ++ SPids,
|
||||
[{Q, S} | Qs0],
|
||||
Actions}
|
||||
end, {[], [], [], []}, Qs).
|
||||
|
||||
%% internal-ish
|
||||
-spec wait_for_promoted_or_stopped(amqqueue:amqqueue()) ->
|
||||
{promoted, amqqueue:amqqueue()} |
|
||||
{stopped, amqqueue:amqqueue()} |
|
||||
{error, not_found}.
|
||||
wait_for_promoted_or_stopped(Q0) ->
|
||||
QName = amqqueue:get_name(Q0),
|
||||
case rabbit_amqqueue:lookup(QName) of
|
||||
{ok, Q} ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
SPids = amqqueue:get_slave_pids(Q),
|
||||
case rabbit_mnesia:is_process_alive(QPid) of
|
||||
true -> {promoted, Q};
|
||||
false ->
|
||||
case lists:any(fun(Pid) ->
|
||||
rabbit_mnesia:is_process_alive(Pid)
|
||||
end, SPids) of
|
||||
%% There is a live slave. May be promoted
|
||||
true ->
|
||||
timer:sleep(100),
|
||||
wait_for_promoted_or_stopped(Q);
|
||||
%% All slave pids are stopped.
|
||||
%% No process left for the queue
|
||||
false -> {stopped, Q}
|
||||
end
|
||||
end;
|
||||
{error, not_found} ->
|
||||
{error, not_found}
|
||||
end.
|
||||
|
||||
-spec delete_crashed(amqqueue:amqqueue()) -> ok.
|
||||
delete_crashed(Q) ->
|
||||
delete_crashed(Q, ?INTERNAL_USER).
|
||||
|
||||
delete_crashed(Q, ActingUser) ->
|
||||
ok = rpc:call(amqqueue:qnode(Q), ?MODULE, delete_crashed_internal,
|
||||
[Q, ActingUser]).
|
||||
|
||||
delete_crashed_internal(Q, ActingUser) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
{ok, BQ} = application:get_env(rabbit, backing_queue_module),
|
||||
BQ:delete_crashed(Q),
|
||||
ok = rabbit_amqqueue:internal_delete(QName, ActingUser).
|
||||
|
||||
recover_durable_queues(QueuesAndRecoveryTerms) ->
|
||||
{Results, Failures} =
|
||||
gen_server2:mcall(
|
||||
[{rabbit_amqqueue_sup_sup:start_queue_process(node(), Q, recovery),
|
||||
{init, {self(), Terms}}} || {Q, Terms} <- QueuesAndRecoveryTerms]),
|
||||
[rabbit_log:error("Queue ~p failed to initialise: ~p~n",
|
||||
[Pid, Error]) || {Pid, Error} <- Failures],
|
||||
[Q || {_, {new, Q}} <- Results].
|
||||
|
||||
-spec is_policy_applicable(amqqueue:amqqueue(), any()) -> boolean().
|
||||
is_policy_applicable(_Q, Policy) ->
|
||||
Applicable = [<<"expires">>, <<"message-ttl">>, <<"dead-letter-exchange">>,
|
||||
<<"dead-letter-routing-key">>, <<"max-length">>,
|
||||
<<"max-length-bytes">>, <<"max-in-memory-length">>, <<"max-in-memory-bytes">>,
|
||||
<<"max-priority">>, <<"overflow">>, <<"queue-mode">>,
|
||||
<<"single-active-consumer">>, <<"delivery-limit">>,
|
||||
<<"ha-mode">>, <<"ha-params">>, <<"ha-sync-mode">>,
|
||||
<<"ha-promote-on-shutdown">>, <<"ha-promote-on-failure">>,
|
||||
<<"queue-master-locator">>],
|
||||
lists:all(fun({P, _}) ->
|
||||
lists:member(P, Applicable)
|
||||
end, Policy).
|
||||
|
||||
reject_seq_no(SeqNo, U0) ->
|
||||
reject_seq_no(SeqNo, U0, []).
|
||||
|
||||
reject_seq_no(SeqNo, U0, Acc) ->
|
||||
case maps:take(SeqNo, U0) of
|
||||
{_, U} ->
|
||||
{U, [SeqNo | Acc]};
|
||||
error ->
|
||||
{U0, Acc}
|
||||
end.
|
||||
|
||||
confirm_seq_nos(MsgSeqNos, Pid, U0) ->
|
||||
lists:foldl(
|
||||
fun (SeqNo, {U, A0}) ->
|
||||
case U of
|
||||
#{SeqNo := Pids0} ->
|
||||
case lists:delete(Pid, Pids0) of
|
||||
[] ->
|
||||
%% the updated unconfirmed state
|
||||
%% and the seqnos to settle
|
||||
{maps:remove(SeqNo, U), [SeqNo | A0]};
|
||||
Pids ->
|
||||
{U#{SeqNo => Pids}, A0}
|
||||
end;
|
||||
_ ->
|
||||
{U, A0}
|
||||
end
|
||||
end, {U0, []}, MsgSeqNos).
|
|
@ -0,0 +1,151 @@
|
|||
-module(rabbit_confirms).
|
||||
|
||||
-compile({no_auto_import, [size/1]}).
|
||||
|
||||
-include_lib("rabbit_common/include/rabbit.hrl").
|
||||
|
||||
-export([init/0,
|
||||
insert/4,
|
||||
confirm/3,
|
||||
reject/2,
|
||||
|
||||
remove_queue/2,
|
||||
|
||||
smallest/1,
|
||||
size/1,
|
||||
is_empty/1]).
|
||||
|
||||
-type seq_no() :: non_neg_integer().
|
||||
-type queue_name() :: rabbit_amqqueue:name().
|
||||
-type exchange_name() :: rabbit_exchange:name().
|
||||
|
||||
-record(?MODULE, {smallest :: undefined | seq_no(),
|
||||
unconfirmed = #{} :: #{seq_no() =>
|
||||
{exchange_name(),
|
||||
#{queue_name() => ok}}}
|
||||
}).
|
||||
|
||||
-type mx() :: {seq_no(), exchange_name()}.
|
||||
|
||||
-opaque state() :: #?MODULE{}.
|
||||
|
||||
-export_type([
|
||||
state/0
|
||||
]).
|
||||
|
||||
-spec init() -> state().
|
||||
init() ->
|
||||
#?MODULE{}.
|
||||
|
||||
-spec insert(seq_no(), [queue_name()], exchange_name(), state()) ->
|
||||
state().
|
||||
insert(SeqNo, QNames, #resource{kind = exchange} = XName,
|
||||
#?MODULE{smallest = S0,
|
||||
unconfirmed = U0} = State)
|
||||
when is_integer(SeqNo)
|
||||
andalso is_list(QNames)
|
||||
andalso is_map_key(SeqNo, U0) == false ->
|
||||
U = U0#{SeqNo => {XName, maps:from_list([{Q, ok} || Q <- QNames])}},
|
||||
S = case S0 of
|
||||
undefined -> SeqNo;
|
||||
_ -> S0
|
||||
end,
|
||||
State#?MODULE{smallest = S,
|
||||
unconfirmed = U}.
|
||||
|
||||
-spec confirm([seq_no()], queue_name(), state()) ->
|
||||
{[mx()], state()}.
|
||||
confirm(SeqNos, QName, #?MODULE{smallest = Smallest0,
|
||||
unconfirmed = U0} = State)
|
||||
when is_list(SeqNos) ->
|
||||
{Confirmed, U} = lists:foldr(
|
||||
fun (SeqNo, Acc) ->
|
||||
confirm_one(SeqNo, QName, Acc)
|
||||
end, {[], U0}, SeqNos),
|
||||
%% check if smallest is in Confirmed
|
||||
Smallest =
|
||||
case lists:any(fun ({S, _}) -> S == Smallest0 end, Confirmed) of
|
||||
true ->
|
||||
%% work out new smallest
|
||||
next_smallest(Smallest0, U);
|
||||
false ->
|
||||
Smallest0
|
||||
end,
|
||||
{Confirmed, State#?MODULE{smallest = Smallest,
|
||||
unconfirmed = U}}.
|
||||
|
||||
-spec reject(seq_no(), state()) ->
|
||||
{ok, mx(), state()} | {error, not_found}.
|
||||
reject(SeqNo, #?MODULE{smallest = Smallest0,
|
||||
unconfirmed = U0} = State)
|
||||
when is_integer(SeqNo) ->
|
||||
case maps:take(SeqNo, U0) of
|
||||
{{XName, _QS}, U} ->
|
||||
Smallest = case SeqNo of
|
||||
Smallest0 ->
|
||||
%% need to scan as the smallest was removed
|
||||
next_smallest(Smallest0, U);
|
||||
_ ->
|
||||
Smallest0
|
||||
end,
|
||||
{ok, {SeqNo, XName}, State#?MODULE{unconfirmed = U,
|
||||
smallest = Smallest}};
|
||||
error ->
|
||||
{error, not_found}
|
||||
end.
|
||||
|
||||
%% idempotent
|
||||
-spec remove_queue(queue_name(), state()) ->
|
||||
{[mx()], state()}.
|
||||
remove_queue(QName, #?MODULE{unconfirmed = U} = State) ->
|
||||
SeqNos = maps:fold(
|
||||
fun (SeqNo, {_XName, QS0}, Acc) ->
|
||||
case maps:is_key(QName, QS0) of
|
||||
true ->
|
||||
[SeqNo | Acc];
|
||||
false ->
|
||||
Acc
|
||||
end
|
||||
end, [], U),
|
||||
confirm(lists:sort(SeqNos), QName,State).
|
||||
|
||||
-spec smallest(state()) -> seq_no() | undefined.
|
||||
smallest(#?MODULE{smallest = Smallest}) ->
|
||||
Smallest.
|
||||
|
||||
-spec size(state()) -> non_neg_integer().
|
||||
size(#?MODULE{unconfirmed = U}) ->
|
||||
maps:size(U).
|
||||
|
||||
-spec is_empty(state()) -> boolean().
|
||||
is_empty(State) ->
|
||||
size(State) == 0.
|
||||
|
||||
%% INTERNAL
|
||||
|
||||
confirm_one(SeqNo, QName, {Acc, U0}) ->
|
||||
case maps:take(SeqNo, U0) of
|
||||
{{XName, QS}, U1}
|
||||
when is_map_key(QName, QS)
|
||||
andalso map_size(QS) == 1 ->
|
||||
%% last queue confirm
|
||||
{[{SeqNo, XName} | Acc], U1};
|
||||
{{XName, QS}, U1} ->
|
||||
{Acc, U1#{SeqNo => {XName, maps:remove(QName, QS)}}};
|
||||
error ->
|
||||
{Acc, U0}
|
||||
end.
|
||||
|
||||
next_smallest(_S, U) when map_size(U) == 0 ->
|
||||
undefined;
|
||||
next_smallest(S, U) when is_map_key(S, U) ->
|
||||
S;
|
||||
next_smallest(S, U) ->
|
||||
%% TODO: this is potentially infinitely recursive if called incorrectly
|
||||
next_smallest(S+1, U).
|
||||
|
||||
|
||||
|
||||
-ifdef(TEST).
|
||||
-include_lib("eunit/include/eunit.hrl").
|
||||
-endif.
|
|
@ -8,6 +8,7 @@
|
|||
-module(rabbit_core_ff).
|
||||
|
||||
-export([quorum_queue_migration/3,
|
||||
stream_queue_migration/3,
|
||||
implicit_default_bindings_migration/3,
|
||||
virtual_host_metadata_migration/3,
|
||||
maintenance_mode_status_migration/3,
|
||||
|
@ -21,6 +22,14 @@
|
|||
migration_fun => {?MODULE, quorum_queue_migration}
|
||||
}}).
|
||||
|
||||
-rabbit_feature_flag(
|
||||
{stream_queue,
|
||||
#{desc => "Support queues of type `stream`",
|
||||
doc_url => "https://www.rabbitmq.com/stream-queues.html",
|
||||
stability => stable,
|
||||
migration_fun => {?MODULE, stream_queue_migration}
|
||||
}}).
|
||||
|
||||
-rabbit_feature_flag(
|
||||
{implicit_default_bindings,
|
||||
#{desc => "Default bindings are now implicit, instead of "
|
||||
|
@ -69,6 +78,9 @@ quorum_queue_migration(_FeatureName, _FeatureProps, is_enabled) ->
|
|||
mnesia:table_info(rabbit_queue, attributes) =:= Fields andalso
|
||||
mnesia:table_info(rabbit_durable_queue, attributes) =:= Fields.
|
||||
|
||||
stream_queue_migration(_FeatureName, _FeatureProps, _Enable) ->
|
||||
ok.
|
||||
|
||||
migrate_to_amqqueue_with_type(FeatureName, [Table | Rest], Fields) ->
|
||||
rabbit_log_feature_flags:info(
|
||||
"Feature flag `~s`: migrating Mnesia table ~s...",
|
||||
|
|
|
@ -20,14 +20,15 @@
|
|||
|
||||
-spec publish(rabbit_types:message(), reason(), rabbit_types:exchange(),
|
||||
'undefined' | binary(), rabbit_amqqueue:name()) -> 'ok'.
|
||||
|
||||
publish(Msg, Reason, X, RK, QName) ->
|
||||
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).
|
||||
_ = rabbit_queue_type:deliver(rabbit_amqqueue:lookup(Queues),
|
||||
Delivery, stateless),
|
||||
ok.
|
||||
|
||||
make_msg(Msg = #basic_message{content = Content,
|
||||
exchange_name = Exchange,
|
||||
|
|
|
@ -661,6 +661,7 @@ queue_definition(Q) ->
|
|||
Type = case amqqueue:get_type(Q) of
|
||||
rabbit_classic_queue -> classic;
|
||||
rabbit_quorum_queue -> quorum;
|
||||
rabbit_stream_queue -> stream;
|
||||
T -> T
|
||||
end,
|
||||
#{
|
||||
|
|
|
@ -286,7 +286,7 @@ apply(Meta, #credit{credit = NewCredit, delivery_count = RemoteDelCnt,
|
|||
end;
|
||||
apply(_, #checkout{spec = {dequeue, _}},
|
||||
#?MODULE{cfg = #cfg{consumer_strategy = single_active}} = State0) ->
|
||||
{State0, {error, unsupported}};
|
||||
{State0, {error, {unsupported, single_active_consumer}}};
|
||||
apply(#{index := Index,
|
||||
system_time := Ts,
|
||||
from := From} = Meta, #checkout{spec = {dequeue, Settlement},
|
||||
|
@ -968,7 +968,6 @@ usage(Name) when is_atom(Name) ->
|
|||
messages_ready(#?MODULE{messages = M,
|
||||
prefix_msgs = {RCnt, _R, PCnt, _P},
|
||||
returns = R}) ->
|
||||
|
||||
%% prefix messages will rarely have anything in them during normal
|
||||
%% operations so length/1 is fine here
|
||||
lqueue:len(M) + lqueue:len(R) + RCnt + PCnt.
|
||||
|
|
|
@ -51,6 +51,7 @@
|
|||
-type cluster_name() :: rabbit_types:r(queue).
|
||||
|
||||
-record(consumer, {last_msg_id :: seq() | -1,
|
||||
ack = false :: boolean(),
|
||||
delivery_count = 0 :: non_neg_integer()}).
|
||||
|
||||
-record(cfg, {cluster_name :: cluster_name(),
|
||||
|
@ -219,10 +220,11 @@ enqueue(Msg, State) ->
|
|||
%% @returns `{ok, IdMsg, State}' or `{error | timeout, term()}'
|
||||
-spec dequeue(rabbit_fifo:consumer_tag(),
|
||||
Settlement :: settled | unsettled, state()) ->
|
||||
{ok, {rabbit_fifo:delivery_msg(), non_neg_integer()}
|
||||
| empty, state()} | {error | timeout, term()}.
|
||||
{ok, non_neg_integer(), term(), non_neg_integer()}
|
||||
| {empty, state()} | {error | timeout, term()}.
|
||||
dequeue(ConsumerTag, Settlement,
|
||||
#state{cfg = #cfg{timeout = Timeout}} = State0) ->
|
||||
#state{cfg = #cfg{timeout = Timeout,
|
||||
cluster_name = QName}} = State0) ->
|
||||
Node = pick_server(State0),
|
||||
ConsumerId = consumer_id(ConsumerTag),
|
||||
case ra:process_command(Node,
|
||||
|
@ -231,9 +233,16 @@ dequeue(ConsumerTag, Settlement,
|
|||
#{}),
|
||||
Timeout) of
|
||||
{ok, {dequeue, empty}, Leader} ->
|
||||
{ok, empty, State0#state{leader = Leader}};
|
||||
{ok, {dequeue, Msg, NumReady}, Leader} ->
|
||||
{ok, {Msg, NumReady},
|
||||
{empty, State0#state{leader = Leader}};
|
||||
{ok, {dequeue, {MsgId, {MsgHeader, Msg0}}, MsgsReady}, Leader} ->
|
||||
Count = case MsgHeader of
|
||||
#{delivery_count := C} -> C;
|
||||
_ -> 0
|
||||
end,
|
||||
IsDelivered = Count > 0,
|
||||
Msg = add_delivery_count_header(Msg0, Count),
|
||||
{ok, MsgsReady,
|
||||
{QName, qref(Leader), MsgId, IsDelivered, Msg},
|
||||
State0#state{leader = Leader}};
|
||||
{ok, {error, _} = Err, _Leader} ->
|
||||
Err;
|
||||
|
@ -241,6 +250,13 @@ dequeue(ConsumerTag, Settlement,
|
|||
Err
|
||||
end.
|
||||
|
||||
add_delivery_count_header(#basic_message{} = Msg0, Count)
|
||||
when is_integer(Count) ->
|
||||
rabbit_basic:add_header(<<"x-delivery-count">>, long, Count, Msg0);
|
||||
add_delivery_count_header(Msg, _Count) ->
|
||||
Msg.
|
||||
|
||||
|
||||
%% @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.}
|
||||
|
@ -251,16 +267,14 @@ dequeue(ConsumerTag, Settlement,
|
|||
%% the sending rate.
|
||||
%%
|
||||
-spec settle(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) ->
|
||||
{ok, state()}.
|
||||
{state(), list()}.
|
||||
settle(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) ->
|
||||
Node = pick_server(State0),
|
||||
Cmd = rabbit_fifo:make_settle(consumer_id(ConsumerTag), MsgIds),
|
||||
case send_command(Node, undefined, Cmd, normal, State0) of
|
||||
{slow, S} ->
|
||||
{_, S} ->
|
||||
% turn slow into ok for this function
|
||||
{ok, S};
|
||||
{ok, _} = Ret ->
|
||||
Ret
|
||||
{S, []}
|
||||
end;
|
||||
settle(ConsumerTag, [_|_] = MsgIds,
|
||||
#state{unsent_commands = Unsent0} = State0) ->
|
||||
|
@ -271,7 +285,7 @@ settle(ConsumerTag, [_|_] = MsgIds,
|
|||
fun ({Settles, Returns, Discards}) ->
|
||||
{Settles ++ MsgIds, Returns, Discards}
|
||||
end, {MsgIds, [], []}, Unsent0),
|
||||
{ok, State0#state{unsent_commands = Unsent}}.
|
||||
{State0#state{unsent_commands = Unsent}, []}.
|
||||
|
||||
%% @doc Return a message to the queue.
|
||||
%% @param ConsumerTag the tag uniquely identifying the consumer.
|
||||
|
@ -284,17 +298,14 @@ settle(ConsumerTag, [_|_] = MsgIds,
|
|||
%% the sending rate.
|
||||
%%
|
||||
-spec return(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) ->
|
||||
{ok, state()}.
|
||||
{state(), list()}.
|
||||
return(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) ->
|
||||
Node = pick_server(State0),
|
||||
% TODO: make rabbit_fifo return support lists of message ids
|
||||
Cmd = rabbit_fifo:make_return(consumer_id(ConsumerTag), MsgIds),
|
||||
case send_command(Node, undefined, Cmd, normal, State0) of
|
||||
{slow, S} ->
|
||||
% turn slow into ok for this function
|
||||
{ok, S};
|
||||
{ok, _} = Ret ->
|
||||
Ret
|
||||
{_, S} ->
|
||||
{S, []}
|
||||
end;
|
||||
return(ConsumerTag, [_|_] = MsgIds,
|
||||
#state{unsent_commands = Unsent0} = State0) ->
|
||||
|
@ -305,7 +316,7 @@ return(ConsumerTag, [_|_] = MsgIds,
|
|||
fun ({Settles, Returns, Discards}) ->
|
||||
{Settles, Returns ++ MsgIds, Discards}
|
||||
end, {[], MsgIds, []}, Unsent0),
|
||||
{ok, State0#state{unsent_commands = Unsent}}.
|
||||
{State0#state{unsent_commands = Unsent}, []}.
|
||||
|
||||
%% @doc Discards a checked out message.
|
||||
%% If the queue has a dead_letter_handler configured this will be called.
|
||||
|
@ -318,16 +329,14 @@ return(ConsumerTag, [_|_] = MsgIds,
|
|||
%% 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()}.
|
||||
{state(), list()}.
|
||||
discard(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) ->
|
||||
Node = pick_server(State0),
|
||||
Cmd = rabbit_fifo:make_discard(consumer_id(ConsumerTag), MsgIds),
|
||||
case send_command(Node, undefined, Cmd, normal, State0) of
|
||||
{slow, S} ->
|
||||
{_, S} ->
|
||||
% turn slow into ok for this function
|
||||
{ok, S};
|
||||
{ok, _} = Ret ->
|
||||
Ret
|
||||
{S, []}
|
||||
end;
|
||||
discard(ConsumerTag, [_|_] = MsgIds,
|
||||
#state{unsent_commands = Unsent0} = State0) ->
|
||||
|
@ -338,7 +347,7 @@ discard(ConsumerTag, [_|_] = MsgIds,
|
|||
fun ({Settles, Returns, Discards}) ->
|
||||
{Settles, Returns, Discards ++ MsgIds}
|
||||
end, {[], [], MsgIds}, Unsent0),
|
||||
{ok, State0#state{unsent_commands = Unsent}}.
|
||||
{State0#state{unsent_commands = Unsent}, []}.
|
||||
|
||||
|
||||
%% @doc Register with the rabbit_fifo queue to "checkout" messages as they
|
||||
|
@ -357,7 +366,8 @@ discard(ConsumerTag, [_|_] = MsgIds,
|
|||
-spec checkout(rabbit_fifo:consumer_tag(), NumUnsettled :: non_neg_integer(),
|
||||
rabbit_fifo:consumer_meta(),
|
||||
state()) -> {ok, state()} | {error | timeout, term()}.
|
||||
checkout(ConsumerTag, NumUnsettled, ConsumerInfo, State0) ->
|
||||
checkout(ConsumerTag, NumUnsettled, ConsumerInfo, State0)
|
||||
when is_map(ConsumerInfo) ->
|
||||
checkout(ConsumerTag, NumUnsettled, simple_prefetch, ConsumerInfo, State0).
|
||||
|
||||
%% @doc Register with the rabbit_fifo queue to "checkout" messages as they
|
||||
|
@ -381,13 +391,23 @@ checkout(ConsumerTag, NumUnsettled, ConsumerInfo, State0) ->
|
|||
CreditMode :: rabbit_fifo:credit_mode(),
|
||||
Meta :: rabbit_fifo:consumer_meta(),
|
||||
state()) -> {ok, state()} | {error | timeout, term()}.
|
||||
checkout(ConsumerTag, NumUnsettled, CreditMode, Meta, State0) ->
|
||||
checkout(ConsumerTag, NumUnsettled, CreditMode, Meta,
|
||||
#state{consumer_deliveries = CDels0} = State0) ->
|
||||
Servers = sorted_servers(State0),
|
||||
ConsumerId = {ConsumerTag, self()},
|
||||
Cmd = rabbit_fifo:make_checkout(ConsumerId,
|
||||
{auto, NumUnsettled, CreditMode},
|
||||
Meta),
|
||||
try_process_command(Servers, Cmd, State0).
|
||||
%% ???
|
||||
Ack = maps:get(ack, Meta, true),
|
||||
|
||||
SDels = maps:update_with(ConsumerTag,
|
||||
fun (V) ->
|
||||
V#consumer{ack = Ack}
|
||||
end,
|
||||
#consumer{last_msg_id = -1,
|
||||
ack = Ack}, CDels0),
|
||||
try_process_command(Servers, Cmd, State0#state{consumer_deliveries = SDels}).
|
||||
|
||||
%% @doc Provide credit to the queue
|
||||
%%
|
||||
|
@ -401,7 +421,7 @@ checkout(ConsumerTag, NumUnsettled, CreditMode, Meta, State0) ->
|
|||
Credit :: non_neg_integer(),
|
||||
Drain :: boolean(),
|
||||
state()) ->
|
||||
{ok, state()}.
|
||||
state().
|
||||
credit(ConsumerTag, Credit, Drain,
|
||||
#state{consumer_deliveries = CDels} = State0) ->
|
||||
ConsumerId = consumer_id(ConsumerTag),
|
||||
|
@ -412,11 +432,9 @@ credit(ConsumerTag, Credit, Drain,
|
|||
Cmd = rabbit_fifo:make_credit(ConsumerId, Credit,
|
||||
C#consumer.last_msg_id + 1, Drain),
|
||||
case send_command(Node, undefined, Cmd, normal, State0) of
|
||||
{slow, S} ->
|
||||
{_, S} ->
|
||||
% turn slow into ok for this function
|
||||
{ok, S};
|
||||
{ok, _} = Ret ->
|
||||
Ret
|
||||
S
|
||||
end.
|
||||
|
||||
%% @doc Cancels a checkout with the rabbit_fifo queue for the consumer tag
|
||||
|
@ -532,12 +550,20 @@ update_machine_state(Server, Conf) ->
|
|||
{internal, Correlators :: [term()], actions(), state()} |
|
||||
{rabbit_fifo:client_msg(), state()} | eol.
|
||||
handle_ra_event(From, {applied, Seqs},
|
||||
#state{cfg = #cfg{soft_limit = SftLmt,
|
||||
unblock_handler = UnblockFun}} = State00) ->
|
||||
State0 = State00#state{leader = From},
|
||||
{Corrs, Actions, State1} = lists:foldl(fun seq_applied/2,
|
||||
{[], [], State0},
|
||||
#state{cfg = #cfg{cluster_name = QRef,
|
||||
soft_limit = SftLmt,
|
||||
unblock_handler = UnblockFun}} = State0) ->
|
||||
|
||||
{Corrs, Actions0, State1} = lists:foldl(fun seq_applied/2,
|
||||
{[], [], State0#state{leader = From}},
|
||||
Seqs),
|
||||
Actions = case Corrs of
|
||||
[] ->
|
||||
lists:reverse(Actions0);
|
||||
_ ->
|
||||
[{settled, QRef, Corrs}
|
||||
| lists:reverse(Actions0)]
|
||||
end,
|
||||
case maps:size(State1#state.pending) < SftLmt of
|
||||
true when State1#state.slow == true ->
|
||||
% we have exited soft limit state
|
||||
|
@ -567,9 +593,9 @@ handle_ra_event(From, {applied, Seqs},
|
|||
end
|
||||
end, State2, Commands),
|
||||
UnblockFun(),
|
||||
{internal, lists:reverse(Corrs), lists:reverse(Actions), State};
|
||||
{ok, State, Actions};
|
||||
_ ->
|
||||
{internal, lists:reverse(Corrs), lists:reverse(Actions), State1}
|
||||
{ok, State1, Actions}
|
||||
end;
|
||||
handle_ra_event(From, {machine, {delivery, _ConsumerTag, _} = Del}, State0) ->
|
||||
handle_delivery(From, Del, State0);
|
||||
|
@ -580,27 +606,27 @@ handle_ra_event(_, {machine, {queue_status, Status}},
|
|||
handle_ra_event(Leader, {machine, leader_change},
|
||||
#state{leader = Leader} = State) ->
|
||||
%% leader already known
|
||||
{internal, [], [], State};
|
||||
{ok, State, []};
|
||||
handle_ra_event(Leader, {machine, leader_change}, State0) ->
|
||||
%% we need to update leader
|
||||
%% and resend any pending commands
|
||||
State = resend_all_pending(State0#state{leader = Leader}),
|
||||
{internal, [], [], State};
|
||||
{ok, State, []};
|
||||
handle_ra_event(_From, {rejected, {not_leader, undefined, _Seq}}, State0) ->
|
||||
% set timer to try find leder and resend
|
||||
{internal, [], [], set_timer(State0)};
|
||||
% TODO: how should these be handled? re-sent on timer or try random
|
||||
{ok, State0, []};
|
||||
handle_ra_event(_From, {rejected, {not_leader, Leader, Seq}}, State0) ->
|
||||
State1 = State0#state{leader = Leader},
|
||||
State = resend(Seq, State1),
|
||||
{internal, [], [], State};
|
||||
{ok, State, []};
|
||||
handle_ra_event(_, timeout, #state{cfg = #cfg{servers = Servers}} = State0) ->
|
||||
case find_leader(Servers) of
|
||||
undefined ->
|
||||
%% still no leader, set the timer again
|
||||
{internal, [], [], set_timer(State0)};
|
||||
{ok, set_timer(State0), []};
|
||||
Leader ->
|
||||
State = resend_all_pending(State0#state{leader = Leader}),
|
||||
{internal, [], [], State}
|
||||
{ok, State, []}
|
||||
end;
|
||||
handle_ra_event(_Leader, {machine, eol}, _State0) ->
|
||||
eol.
|
||||
|
@ -695,21 +721,35 @@ resend(OldSeq, #state{pending = Pending0, leader = Leader} = State) ->
|
|||
|
||||
resend_all_pending(#state{pending = Pend} = State) ->
|
||||
Seqs = lists:sort(maps:keys(Pend)),
|
||||
rabbit_log:info("rabbit_fifo_client: resend all pending ~w", [Seqs]),
|
||||
lists:foldl(fun resend/2, State, Seqs).
|
||||
|
||||
handle_delivery(From, {delivery, Tag, [{FstId, _} | _] = IdMsgs} = Del0,
|
||||
#state{consumer_deliveries = CDels0} = State0) ->
|
||||
maybe_auto_ack(true, Deliver, State0) ->
|
||||
%% manual ack is enabled
|
||||
{ok, State0, [Deliver]};
|
||||
maybe_auto_ack(false, {deliver, Tag, _Ack, Msgs} = Deliver, State0) ->
|
||||
%% we have to auto ack these deliveries
|
||||
MsgIds = [I || {_, _, I, _, _} <- Msgs],
|
||||
{State, Actions} = settle(Tag, MsgIds, State0),
|
||||
{ok, State, [Deliver] ++ Actions}.
|
||||
|
||||
|
||||
handle_delivery(Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs},
|
||||
#state{cfg = #cfg{cluster_name = QName},
|
||||
consumer_deliveries = CDels0} = State0) ->
|
||||
QRef = qref(Leader),
|
||||
{LastId, _} = lists:last(IdMsgs),
|
||||
%% NB: deliveries may not be from the leader so we will not update the
|
||||
%% tracked leader id here
|
||||
Consumer = #consumer{ack = Ack} = maps:get(Tag, CDels0),
|
||||
%% format as a deliver action
|
||||
Del = {deliver, Tag, Ack, transform_msgs(QName, QRef, IdMsgs)},
|
||||
%% TODO: remove potential default allocation
|
||||
case maps:get(Tag, CDels0, #consumer{last_msg_id = -1}) of
|
||||
case Consumer of
|
||||
#consumer{last_msg_id = Prev} = C
|
||||
when FstId =:= Prev+1 ->
|
||||
{Del0, State0#state{consumer_deliveries =
|
||||
update_consumer(Tag, LastId, length(IdMsgs), C,
|
||||
CDels0)}};
|
||||
maybe_auto_ack(Ack, Del,
|
||||
State0#state{consumer_deliveries =
|
||||
update_consumer(Tag, LastId,
|
||||
length(IdMsgs), C,
|
||||
CDels0)});
|
||||
#consumer{last_msg_id = Prev} = C
|
||||
when FstId > Prev+1 ->
|
||||
NumMissing = FstId - Prev + 1,
|
||||
|
@ -719,34 +759,49 @@ handle_delivery(From, {delivery, Tag, [{FstId, _} | _] = IdMsgs} = Del0,
|
|||
%% When the node is disconnected the leader will return all checked
|
||||
%% out messages to the main queue to ensure they don't get stuck in
|
||||
%% case the node never comes back.
|
||||
Missing = get_missing_deliveries(From, Prev+1, FstId-1, Tag),
|
||||
Del = {delivery, Tag, Missing ++ IdMsgs},
|
||||
{Del, State0#state{consumer_deliveries =
|
||||
update_consumer(Tag, LastId,
|
||||
length(IdMsgs) + NumMissing,
|
||||
C, CDels0)}};
|
||||
Missing = get_missing_deliveries(Leader, Prev+1, FstId-1, Tag),
|
||||
XDel = {deliver, Tag, Ack, transform_msgs(QName, QRef,
|
||||
Missing ++ IdMsgs)},
|
||||
maybe_auto_ack(Ack, XDel,
|
||||
State0#state{consumer_deliveries =
|
||||
update_consumer(Tag, LastId,
|
||||
length(IdMsgs) + NumMissing,
|
||||
C, CDels0)});
|
||||
#consumer{last_msg_id = Prev}
|
||||
when FstId =< Prev ->
|
||||
case lists:dropwhile(fun({Id, _}) -> Id =< Prev end, IdMsgs) of
|
||||
[] ->
|
||||
{internal, [], [], State0};
|
||||
{ok, State0, []};
|
||||
IdMsgs2 ->
|
||||
handle_delivery(From, {delivery, Tag, IdMsgs2}, State0)
|
||||
handle_delivery(Leader, {delivery, Tag, IdMsgs2}, State0)
|
||||
end;
|
||||
_ when FstId =:= 0 ->
|
||||
C when FstId =:= 0 ->
|
||||
% the very first delivery
|
||||
{Del0, State0#state{consumer_deliveries =
|
||||
update_consumer(Tag, LastId,
|
||||
length(IdMsgs),
|
||||
#consumer{last_msg_id = LastId},
|
||||
CDels0)}}
|
||||
maybe_auto_ack(Ack, Del,
|
||||
State0#state{consumer_deliveries =
|
||||
update_consumer(Tag, LastId,
|
||||
length(IdMsgs),
|
||||
C#consumer{last_msg_id = LastId},
|
||||
CDels0)})
|
||||
end.
|
||||
|
||||
transform_msgs(QName, QRef, Msgs) ->
|
||||
lists:map(
|
||||
fun({MsgId, {MsgHeader, Msg0}}) ->
|
||||
{Msg, Redelivered} = case MsgHeader of
|
||||
#{delivery_count := C} ->
|
||||
{add_delivery_count_header(Msg0, C), true};
|
||||
_ ->
|
||||
{Msg0, false}
|
||||
end,
|
||||
{QName, QRef, MsgId, Redelivered, Msg}
|
||||
end, Msgs).
|
||||
|
||||
update_consumer(Tag, LastId, DelCntIncr,
|
||||
#consumer{delivery_count = D}, Consumers) ->
|
||||
#consumer{delivery_count = D} = C, Consumers) ->
|
||||
maps:put(Tag,
|
||||
#consumer{last_msg_id = LastId,
|
||||
delivery_count = D + DelCntIncr},
|
||||
C#consumer{last_msg_id = LastId,
|
||||
delivery_count = D + DelCntIncr},
|
||||
Consumers).
|
||||
|
||||
|
||||
|
@ -821,9 +876,17 @@ add_command(Cid, return, MsgIds, Acc) ->
|
|||
add_command(Cid, discard, MsgIds, Acc) ->
|
||||
[rabbit_fifo:make_discard(Cid, MsgIds) | Acc].
|
||||
|
||||
set_timer(#state{cfg = #cfg{servers = [Server | _]}} = State) ->
|
||||
set_timer(#state{leader = Leader0,
|
||||
cfg = #cfg{servers = [Server | _],
|
||||
cluster_name = QName}} = State) ->
|
||||
Leader = case Leader0 of
|
||||
undefined -> Server;
|
||||
_ ->
|
||||
Leader0
|
||||
end,
|
||||
Ref = erlang:send_after(?TIMER_TIME, self(),
|
||||
{ra_event, Server, timeout}),
|
||||
{'$gen_cast',
|
||||
{queue_event, QName, {Leader, timeout}}}),
|
||||
State#state{timer_state = Ref}.
|
||||
|
||||
cancel_timer(#state{timer_state = undefined} = State) ->
|
||||
|
@ -840,3 +903,6 @@ find_leader([Server | Servers]) ->
|
|||
_ ->
|
||||
find_leader(Servers)
|
||||
end.
|
||||
|
||||
qref({Ref, _}) -> Ref;
|
||||
qref(Ref) -> Ref.
|
||||
|
|
|
@ -11,7 +11,7 @@
|
|||
|
||||
-export([start_link/0]).
|
||||
-export([filename/0]).
|
||||
-export([gen/0, gen_secure/0, string/2, binary/2]).
|
||||
-export([gen/0, gen_secure/0, string/2, binary/2, to_string/1]).
|
||||
|
||||
-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,
|
||||
code_change/3]).
|
||||
|
@ -151,6 +151,12 @@ string(G, Prefix) when is_binary(Prefix) ->
|
|||
binary(G, Prefix) ->
|
||||
list_to_binary(string(G, Prefix)).
|
||||
|
||||
%% copied from https://stackoverflow.com/questions/1657204/erlang-uuid-generator
|
||||
to_string(<<TL:32, TM:16, THV:16, CSR:8, CSL:8, N:48>>) ->
|
||||
lists:flatten(
|
||||
io_lib:format("~8.16.0b-~4.16.0b-~4.16.0b-~2.16.0b~2.16.0b-~12.16.0b",
|
||||
[TL, TM, THV, CSR, CSL, N])).
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
init([Serial]) ->
|
||||
|
|
|
@ -578,11 +578,12 @@ send_or_record_confirm(published, #delivery { sender = ChPid,
|
|||
send_or_record_confirm(_Status, #delivery { sender = ChPid,
|
||||
confirm = true,
|
||||
msg_seq_no = MsgSeqNo },
|
||||
MS, _State) ->
|
||||
ok = rabbit_misc:confirm_to_sender(ChPid, [MsgSeqNo]),
|
||||
MS, #state{q = Q} = _State) ->
|
||||
ok = rabbit_misc:confirm_to_sender(ChPid, amqqueue:get_name(Q), [MsgSeqNo]),
|
||||
MS.
|
||||
|
||||
confirm_messages(MsgIds, State = #state { msg_id_status = MS }) ->
|
||||
confirm_messages(MsgIds, State = #state{q = Q, msg_id_status = MS}) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
{CMs, MS1} =
|
||||
lists:foldl(
|
||||
fun (MsgId, {CMsN, MSN} = Acc) ->
|
||||
|
@ -610,7 +611,10 @@ confirm_messages(MsgIds, State = #state { msg_id_status = MS }) ->
|
|||
Acc
|
||||
end
|
||||
end, {gb_trees:empty(), MS}, MsgIds),
|
||||
rabbit_misc:gb_trees_foreach(fun rabbit_misc:confirm_to_sender/2, CMs),
|
||||
Fun = fun (Pid, MsgSeqNos) ->
|
||||
rabbit_misc:confirm_to_sender(Pid, QName, MsgSeqNos)
|
||||
end,
|
||||
rabbit_misc:gb_trees_foreach(Fun, CMs),
|
||||
State #state { msg_id_status = MS1 }.
|
||||
|
||||
handle_process_result({ok, State}) -> noreply(State);
|
||||
|
|
|
@ -0,0 +1,399 @@
|
|||
-module(rabbit_msg_record).
|
||||
|
||||
-export([
|
||||
init/1,
|
||||
to_iodata/1,
|
||||
from_amqp091/2,
|
||||
to_amqp091/1,
|
||||
add_message_annotations/2,
|
||||
message_annotation/2,
|
||||
message_annotation/3
|
||||
]).
|
||||
|
||||
-include("rabbit.hrl").
|
||||
-include("rabbit_framing.hrl").
|
||||
-include_lib("amqp10_common/include/amqp10_framing.hrl").
|
||||
|
||||
-type maybe(T) :: T | undefined.
|
||||
-type amqp10_data() :: #'v1_0.data'{} |
|
||||
[#'v1_0.amqp_sequence'{} | #'v1_0.data'{}] |
|
||||
#'v1_0.amqp_value'{}.
|
||||
-record(msg,
|
||||
{
|
||||
% header :: maybe(#'v1_0.header'{}),
|
||||
% delivery_annotations :: maybe(#'v1_0.delivery_annotations'{}),
|
||||
message_annotations :: maybe(#'v1_0.message_annotations'{}),
|
||||
properties :: maybe(#'v1_0.properties'{}),
|
||||
application_properties :: maybe(#'v1_0.application_properties'{}),
|
||||
data :: maybe(amqp10_data())
|
||||
% footer :: maybe(#'v1_0.footer'{})
|
||||
}).
|
||||
|
||||
%% holds static or rarely changing fields
|
||||
-record(cfg, {}).
|
||||
-record(?MODULE, {cfg :: #cfg{},
|
||||
msg :: #msg{},
|
||||
%% holds a list of modifications to various sections
|
||||
changes = [] :: list()}).
|
||||
|
||||
-opaque state() :: #?MODULE{}.
|
||||
|
||||
-export_type([
|
||||
state/0
|
||||
]).
|
||||
|
||||
%% this module acts as a wrapper / converter for the internal binar storage format
|
||||
%% (AMQP 1.0) and any format it needs to be converted to / from.
|
||||
%% Efficiency is key. No unnecessary allocations or work should be done until it
|
||||
%% is absolutely needed
|
||||
|
||||
%% init from an AMQP 1.0 encoded binary
|
||||
-spec init(binary()) -> state().
|
||||
init(Bin) when is_binary(Bin) ->
|
||||
%% TODO: delay parsing until needed
|
||||
{MA, P, AP, D} = decode(amqp10_framing:decode_bin(Bin),
|
||||
{undefined, undefined, undefined, undefined}),
|
||||
#?MODULE{cfg = #cfg{},
|
||||
msg = #msg{properties = P,
|
||||
application_properties = AP,
|
||||
message_annotations = MA,
|
||||
data = D}}.
|
||||
|
||||
decode([], Acc) ->
|
||||
Acc;
|
||||
decode([#'v1_0.message_annotations'{} = MA | Rem], {_, P, AP, D}) ->
|
||||
decode(Rem, {MA, P, AP, D});
|
||||
decode([#'v1_0.properties'{} = P | Rem], {MA, _, AP, D}) ->
|
||||
decode(Rem, {MA, P, AP, D});
|
||||
decode([#'v1_0.application_properties'{} = AP | Rem], {MA, P, _, D}) ->
|
||||
decode(Rem, {MA, P, AP, D});
|
||||
decode([#'v1_0.data'{} = D | Rem], {MA, P, AP, _}) ->
|
||||
decode(Rem, {MA, P, AP, D}).
|
||||
|
||||
amqp10_properties_empty(#'v1_0.properties'{message_id = undefined,
|
||||
user_id = undefined,
|
||||
to = undefined,
|
||||
% subject = wrap(utf8, RKey),
|
||||
reply_to = undefined,
|
||||
correlation_id = undefined,
|
||||
content_type = undefined,
|
||||
content_encoding = undefined,
|
||||
creation_time = undefined}) ->
|
||||
true;
|
||||
amqp10_properties_empty(_) ->
|
||||
false.
|
||||
|
||||
%% to realise the final binary data representation
|
||||
-spec to_iodata(state()) -> iodata().
|
||||
to_iodata(#?MODULE{msg = #msg{properties = P,
|
||||
application_properties = AP,
|
||||
message_annotations = MA,
|
||||
data = Data}}) ->
|
||||
[
|
||||
case MA of
|
||||
#'v1_0.message_annotations'{content = []} ->
|
||||
<<>>;
|
||||
_ ->
|
||||
amqp10_framing:encode_bin(MA)
|
||||
end,
|
||||
case amqp10_properties_empty(P) of
|
||||
true -> <<>>;
|
||||
false ->
|
||||
amqp10_framing:encode_bin(P)
|
||||
end,
|
||||
case AP of
|
||||
#'v1_0.application_properties'{content = []} ->
|
||||
<<>>;
|
||||
_ ->
|
||||
amqp10_framing:encode_bin(AP)
|
||||
end,
|
||||
amqp10_framing:encode_bin(Data)
|
||||
].
|
||||
|
||||
%% TODO: refine type spec here
|
||||
-spec add_message_annotations(#{binary() => {atom(), term()}}, state()) ->
|
||||
state().
|
||||
add_message_annotations(Anns,
|
||||
#?MODULE{msg =
|
||||
#msg{message_annotations = MA0} = Msg} = State) ->
|
||||
Content = maps:fold(
|
||||
fun (K, {T, V}, Acc) ->
|
||||
map_add(symbol, K, T, V, Acc)
|
||||
end,
|
||||
case MA0 of
|
||||
undefined -> [];
|
||||
#'v1_0.message_annotations'{content = C} -> C
|
||||
end,
|
||||
Anns),
|
||||
|
||||
State#?MODULE{msg =
|
||||
Msg#msg{message_annotations =
|
||||
#'v1_0.message_annotations'{content = Content}}}.
|
||||
|
||||
%% TODO: refine
|
||||
-type amqp10_term() :: {atom(), term()}.
|
||||
|
||||
-spec message_annotation(binary(), state()) -> undefined | amqp10_term().
|
||||
message_annotation(Key, State) ->
|
||||
message_annotation(Key, State, undefined).
|
||||
|
||||
-spec message_annotation(binary(), state(), undefined | amqp10_term()) ->
|
||||
undefined | amqp10_term().
|
||||
message_annotation(_Key, #?MODULE{msg = #msg{message_annotations = undefined}},
|
||||
Default) ->
|
||||
Default;
|
||||
message_annotation(Key,
|
||||
#?MODULE{msg =
|
||||
#msg{message_annotations =
|
||||
#'v1_0.message_annotations'{content = Content}}},
|
||||
Default)
|
||||
when is_binary(Key) ->
|
||||
case lists:search(fun ({{symbol, K}, _}) -> K == Key end, Content) of
|
||||
{value, {_K, V}} ->
|
||||
V;
|
||||
false ->
|
||||
Default
|
||||
end.
|
||||
|
||||
|
||||
%% take a binary AMQP 1.0 input function,
|
||||
%% parses it and returns the current parse state
|
||||
%% this is the input function from storage and from, e.g. socket input
|
||||
-spec from_amqp091(#'P_basic'{}, iodata()) -> state().
|
||||
from_amqp091(#'P_basic'{message_id = MsgId,
|
||||
expiration = Expiration,
|
||||
delivery_mode = DelMode,
|
||||
headers = Headers,
|
||||
user_id = UserId,
|
||||
reply_to = ReplyTo,
|
||||
type = Type,
|
||||
priority = Priority,
|
||||
app_id = AppId,
|
||||
correlation_id = CorrId,
|
||||
content_type = ContentType,
|
||||
content_encoding = ContentEncoding,
|
||||
timestamp = Timestamp
|
||||
}, Data) ->
|
||||
%% TODO: support parsing properties bin directly?
|
||||
ConvertedTs = case Timestamp of
|
||||
undefined ->
|
||||
undefined;
|
||||
_ ->
|
||||
Timestamp * 1000
|
||||
end,
|
||||
P = #'v1_0.properties'{message_id = wrap(utf8, MsgId),
|
||||
user_id = wrap(binary, UserId),
|
||||
to = undefined,
|
||||
% subject = wrap(utf8, RKey),
|
||||
reply_to = wrap(utf8, ReplyTo),
|
||||
correlation_id = wrap(utf8, CorrId),
|
||||
content_type = wrap(symbol, ContentType),
|
||||
content_encoding = wrap(symbol, ContentEncoding),
|
||||
creation_time = wrap(timestamp, ConvertedTs)},
|
||||
|
||||
APC0 = [{wrap(utf8, K), from_091(T, V)} || {K, T, V}
|
||||
<- case Headers of
|
||||
undefined -> [];
|
||||
_ -> Headers
|
||||
end],
|
||||
%% properties that do not map directly to AMQP 1.0 properties are stored
|
||||
%% in application properties
|
||||
APC = map_add(utf8, <<"x-basic-type">>, utf8, Type,
|
||||
map_add(utf8, <<"x-basic-app-id">>, utf8, AppId, APC0)),
|
||||
|
||||
MAC = map_add(symbol, <<"x-basic-priority">>, ubyte, Priority,
|
||||
map_add(symbol, <<"x-basic-delivery-mode">>, ubyte, DelMode,
|
||||
map_add(symbol, <<"x-basic-expiration">>, utf8, Expiration, []))),
|
||||
|
||||
AP = #'v1_0.application_properties'{content = APC},
|
||||
MA = #'v1_0.message_annotations'{content = MAC},
|
||||
#?MODULE{cfg = #cfg{},
|
||||
msg = #msg{properties = P,
|
||||
application_properties = AP,
|
||||
message_annotations = MA,
|
||||
data = #'v1_0.data'{content = Data}}}.
|
||||
|
||||
map_add(_T, _Key, _Type, undefined, Acc) ->
|
||||
Acc;
|
||||
map_add(KeyType, Key, Type, Value, Acc) ->
|
||||
[{wrap(KeyType, Key), wrap(Type, Value)} | Acc].
|
||||
|
||||
-spec to_amqp091(state()) -> {#'P_basic'{}, iodata()}.
|
||||
to_amqp091(#?MODULE{msg = #msg{properties = P,
|
||||
application_properties = APR,
|
||||
message_annotations = MAR,
|
||||
data = #'v1_0.data'{content = Payload}}}) ->
|
||||
#'v1_0.properties'{message_id = MsgId,
|
||||
user_id = UserId,
|
||||
reply_to = ReplyTo0,
|
||||
correlation_id = CorrId,
|
||||
content_type = ContentType,
|
||||
content_encoding = ContentEncoding,
|
||||
creation_time = Timestamp} = case P of
|
||||
undefined ->
|
||||
#'v1_0.properties'{};
|
||||
_ ->
|
||||
P
|
||||
end,
|
||||
|
||||
AP0 = case APR of
|
||||
#'v1_0.application_properties'{content = AC} -> AC;
|
||||
_ -> []
|
||||
end,
|
||||
MA0 = case MAR of
|
||||
#'v1_0.message_annotations'{content = MC} -> MC;
|
||||
_ -> []
|
||||
end,
|
||||
|
||||
{Type, AP1} = amqp10_map_get(utf8(<<"x-basic-type">>), AP0),
|
||||
{AppId, AP} = amqp10_map_get(utf8(<<"x-basic-app-id">>), AP1),
|
||||
|
||||
{Priority, MA1} = amqp10_map_get(symbol(<<"x-basic-priority">>), MA0),
|
||||
{DelMode, MA2} = amqp10_map_get(symbol(<<"x-basic-delivery-mode">>), MA1),
|
||||
{Expiration, _MA} = amqp10_map_get(symbol(<<"x-basic-expiration">>), MA2),
|
||||
|
||||
Headers0 = [to_091(unwrap(K), V) || {K, V} <- AP],
|
||||
{Headers1, MsgId091} = message_id(MsgId, <<"x-message-id-type">>, Headers0),
|
||||
{Headers, CorrId091} = message_id(CorrId, <<"x-correlation-id-type">>, Headers1),
|
||||
|
||||
BP = #'P_basic'{message_id = MsgId091,
|
||||
delivery_mode = DelMode,
|
||||
expiration = Expiration,
|
||||
user_id = unwrap(UserId),
|
||||
headers = case Headers of
|
||||
[] -> undefined;
|
||||
_ -> Headers
|
||||
end,
|
||||
reply_to = unwrap(ReplyTo0),
|
||||
type = Type,
|
||||
app_id = AppId,
|
||||
priority = Priority,
|
||||
correlation_id = CorrId091,
|
||||
content_type = unwrap(ContentType),
|
||||
content_encoding = unwrap(ContentEncoding),
|
||||
timestamp = case unwrap(Timestamp) of
|
||||
undefined ->
|
||||
undefined;
|
||||
Ts ->
|
||||
Ts div 1000
|
||||
end
|
||||
},
|
||||
{BP, Payload}.
|
||||
|
||||
%%% Internal
|
||||
|
||||
amqp10_map_get(K, AP0) ->
|
||||
case lists:keytake(K, 1, AP0) of
|
||||
false ->
|
||||
{undefined, AP0};
|
||||
{value, {_, V}, AP} ->
|
||||
{unwrap(V), AP}
|
||||
end.
|
||||
|
||||
wrap(_Type, undefined) ->
|
||||
undefined;
|
||||
wrap(Type, Val) ->
|
||||
{Type, Val}.
|
||||
|
||||
unwrap(undefined) ->
|
||||
undefined;
|
||||
unwrap({_Type, V}) ->
|
||||
V.
|
||||
|
||||
% symbol_for(#'v1_0.properties'{}) ->
|
||||
% {symbol, <<"amqp:properties:list">>};
|
||||
|
||||
% number_for(#'v1_0.properties'{}) ->
|
||||
% {ulong, 115};
|
||||
% encode(Frame = #'v1_0.properties'{}) ->
|
||||
% amqp10_framing:encode_described(list, 115, Frame);
|
||||
|
||||
% encode_described(list, CodeNumber, Frame) ->
|
||||
% {described, {ulong, CodeNumber},
|
||||
% {list, lists:map(fun encode/1, tl(tuple_to_list(Frame)))}};
|
||||
|
||||
% -spec generate(amqp10_type()) -> iolist().
|
||||
% generate({described, Descriptor, Value}) ->
|
||||
% DescBin = generate(Descriptor),
|
||||
% ValueBin = generate(Value),
|
||||
% [ ?DESCRIBED_BIN, DescBin, ValueBin ].
|
||||
|
||||
to_091(Key, {utf8, V}) when is_binary(V) -> {Key, longstr, V};
|
||||
to_091(Key, {long, V}) -> {Key, long, V};
|
||||
to_091(Key, {byte, V}) -> {Key, byte, V};
|
||||
to_091(Key, {ubyte, V}) -> {Key, unsignedbyte, V};
|
||||
to_091(Key, {short, V}) -> {Key, short, V};
|
||||
to_091(Key, {ushort, V}) -> {Key, unsignedshort, V};
|
||||
to_091(Key, {uint, V}) -> {Key, unsignedint, V};
|
||||
to_091(Key, {int, V}) -> {Key, signedint, V};
|
||||
to_091(Key, {double, V}) -> {Key, double, V};
|
||||
to_091(Key, {float, V}) -> {Key, float, V};
|
||||
%% NB: header values can never be shortstr!
|
||||
to_091(Key, {timestamp, V}) -> {Key, timestamp, V div 1000};
|
||||
to_091(Key, {binary, V}) -> {Key, binary, V};
|
||||
to_091(Key, {boolean, V}) -> {Key, bool, V};
|
||||
to_091(Key, true) -> {Key, bool, true};
|
||||
to_091(Key, false) -> {Key, bool, false}.
|
||||
|
||||
from_091(longstr, V) when is_binary(V) -> {utf8, V};
|
||||
from_091(long, V) -> {long, V};
|
||||
from_091(unsignedbyte, V) -> {ubyte, V};
|
||||
from_091(short, V) -> {short, V};
|
||||
from_091(unsignedshort, V) -> {ushort, V};
|
||||
from_091(unsignedint, V) -> {uint, V};
|
||||
from_091(signedint, V) -> {int, V};
|
||||
from_091(double, V) -> {double, V};
|
||||
from_091(float, V) -> {float, V};
|
||||
from_091(bool, V) -> {boolean, V};
|
||||
from_091(binary, V) -> {binary, V};
|
||||
from_091(timestamp, V) -> {timestamp, V * 1000}.
|
||||
|
||||
% convert_header(signedint, V) -> [$I, <<V:32/signed>>];
|
||||
% convert_header(decimal, V) -> {Before, After} = V,
|
||||
% [$D, Before, <<After:32>>];
|
||||
% convert_header(timestamp, V) -> [$T, <<V:64>>];
|
||||
% % convert_header(table, V) -> [$F | table_to_binary(V)];
|
||||
% % convert_header(array, V) -> [$A | array_to_binary(V)];
|
||||
% convert_header(byte, V) -> [$b, <<V:8/signed>>];
|
||||
% convert_header(double, V) -> [$d, <<V:64/float>>];
|
||||
% convert_header(float, V) -> [$f, <<V:32/float>>];
|
||||
% convert_header(short, V) -> [$s, <<V:16/signed>>];
|
||||
% convert_header(binary, V) -> [$x | long_string_to_binary(V)];
|
||||
% convert_header(unsignedbyte, V) -> [$B, <<V:8/unsigned>>];
|
||||
% convert_header(unsignedshort, V) -> [$u, <<V:16/unsigned>>];
|
||||
% convert_header(unsignedint, V) -> [$i, <<V:32/unsigned>>];
|
||||
% convert_header(void, _V) -> [$V].
|
||||
|
||||
utf8(T) -> {utf8, T}.
|
||||
symbol(T) -> {symbol, T}.
|
||||
|
||||
message_id({uuid, UUID}, HKey, H0) ->
|
||||
H = [{HKey, longstr, <<"uuid">>} | H0],
|
||||
{H, rabbit_data_coercion:to_binary(rabbit_guid:to_string(UUID))};
|
||||
message_id({ulong, N}, HKey, H0) ->
|
||||
H = [{HKey, longstr, <<"ulong">>} | H0],
|
||||
{H, erlang:integer_to_binary(N)};
|
||||
message_id({binary, B}, HKey, H0) ->
|
||||
E = base64:encode(B),
|
||||
case byte_size(E) > 256 of
|
||||
true ->
|
||||
K = binary:replace(HKey, <<"-type">>, <<>>),
|
||||
{[{K, longstr, B} | H0], undefined};
|
||||
false ->
|
||||
H = [{HKey, longstr, <<"binary">>} | H0],
|
||||
{H, E}
|
||||
end;
|
||||
message_id({utf8, S}, HKey, H0) ->
|
||||
case byte_size(S) > 256 of
|
||||
true ->
|
||||
K = binary:replace(HKey, <<"-type">>, <<>>),
|
||||
{[{K, longstr, S} | H0], undefined};
|
||||
false ->
|
||||
{H0, S}
|
||||
end;
|
||||
message_id(MsgId, _, H) ->
|
||||
{H, unwrap(MsgId)}.
|
||||
|
||||
-ifdef(TEST).
|
||||
-include_lib("eunit/include/eunit.hrl").
|
||||
-endif.
|
|
@ -0,0 +1,103 @@
|
|||
%% 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 https://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.
|
||||
%%
|
||||
%% Copyright (c) 2012-2020 VMware, Inc. or its affiliates. All rights reserved.
|
||||
%%
|
||||
|
||||
-module(rabbit_osiris_metrics).
|
||||
|
||||
-behaviour(gen_server).
|
||||
|
||||
-export([start_link/0]).
|
||||
|
||||
-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,
|
||||
code_change/3]).
|
||||
|
||||
-define(TICK_TIMEOUT, 5000).
|
||||
-define(SERVER, ?MODULE).
|
||||
|
||||
-define(STATISTICS_KEYS,
|
||||
[policy,
|
||||
operator_policy,
|
||||
effective_policy_definition,
|
||||
state,
|
||||
leader,
|
||||
online,
|
||||
members
|
||||
]).
|
||||
|
||||
-record(state, {timeout :: non_neg_integer()}).
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
%% Starts the raw metrics storage and owns the ETS tables.
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
-spec start_link() -> rabbit_types:ok_pid_or_error().
|
||||
|
||||
start_link() ->
|
||||
gen_server:start_link({local, ?SERVER}, ?MODULE, [], []).
|
||||
|
||||
init([]) ->
|
||||
Timeout = application:get_env(rabbit, stream_tick_interval,
|
||||
?TICK_TIMEOUT),
|
||||
erlang:send_after(Timeout, self(), tick),
|
||||
{ok, #state{timeout = Timeout}}.
|
||||
|
||||
handle_call(_Request, _From, State) ->
|
||||
{noreply, State}.
|
||||
|
||||
handle_cast(_Request, State) ->
|
||||
{noreply, State}.
|
||||
|
||||
handle_info(tick, #state{timeout = Timeout} = State) ->
|
||||
Data = osiris_counters:overview(),
|
||||
maps:map(
|
||||
fun ({osiris_writer, QName}, #{offset := Offs,
|
||||
first_offset := FstOffs}) ->
|
||||
COffs = Offs + 1 - FstOffs,
|
||||
rabbit_core_metrics:queue_stats(QName, COffs, 0, COffs, 0),
|
||||
Infos = try
|
||||
%% TODO complete stats!
|
||||
case rabbit_amqqueue:lookup(QName) of
|
||||
{ok, Q} ->
|
||||
rabbit_stream_queue:info(Q, ?STATISTICS_KEYS);
|
||||
_ ->
|
||||
[]
|
||||
end
|
||||
catch
|
||||
_:_ ->
|
||||
%% It's possible that the writer has died but
|
||||
%% it's still on the amqqueue record, so the
|
||||
%% `erlang:process_info/2` calls will return
|
||||
%% `undefined` and crash with a badmatch.
|
||||
%% At least for now, skipping the metrics might
|
||||
%% be the best option. Otherwise this brings
|
||||
%% down `rabbit_sup` and the whole `rabbit` app.
|
||||
[]
|
||||
end,
|
||||
rabbit_core_metrics:queue_stats(QName, Infos),
|
||||
rabbit_event:notify(queue_stats, Infos ++ [{name, QName},
|
||||
{messages, COffs},
|
||||
{messages_ready, COffs},
|
||||
{messages_unacknowledged, 0}]),
|
||||
ok;
|
||||
(_, _V) ->
|
||||
ok
|
||||
end, Data),
|
||||
erlang:send_after(Timeout, self(), tick),
|
||||
{noreply, State}.
|
||||
|
||||
terminate(_Reason, _State) ->
|
||||
ok.
|
||||
|
||||
code_change(_OldVsn, State, _Extra) ->
|
||||
{ok, State}.
|
|
@ -39,6 +39,8 @@ register() ->
|
|||
{policy_validator, <<"queue-mode">>},
|
||||
{policy_validator, <<"overflow">>},
|
||||
{policy_validator, <<"delivery-limit">>},
|
||||
{policy_validator, <<"max-age">>},
|
||||
{policy_validator, <<"max-segment-size">>},
|
||||
{operator_policy_validator, <<"expires">>},
|
||||
{operator_policy_validator, <<"message-ttl">>},
|
||||
{operator_policy_validator, <<"max-length">>},
|
||||
|
@ -135,7 +137,21 @@ validate_policy0(<<"delivery-limit">>, Value)
|
|||
when is_integer(Value), Value >= 0 ->
|
||||
ok;
|
||||
validate_policy0(<<"delivery-limit">>, Value) ->
|
||||
{error, "~p is not a valid delivery limit", [Value]}.
|
||||
{error, "~p is not a valid delivery limit", [Value]};
|
||||
|
||||
validate_policy0(<<"max-age">>, Value) ->
|
||||
case rabbit_amqqueue:check_max_age(Value) of
|
||||
{error, _} ->
|
||||
{error, "~p is not a valid max age", [Value]};
|
||||
_ ->
|
||||
ok
|
||||
end;
|
||||
|
||||
validate_policy0(<<"max-segment-size">>, Value)
|
||||
when is_integer(Value), Value >= 0 ->
|
||||
ok;
|
||||
validate_policy0(<<"max-segment-size">>, Value) ->
|
||||
{error, "~p is not a valid segment size", [Value]}.
|
||||
|
||||
merge_policy_value(<<"message-ttl">>, Val, OpVal) -> min(Val, OpVal);
|
||||
merge_policy_value(<<"max-length">>, Val, OpVal) -> min(Val, OpVal);
|
||||
|
|
|
@ -472,10 +472,15 @@ matches_type(_, _) -> false.
|
|||
sort_pred(A, B) -> pget(priority, A) >= pget(priority, B).
|
||||
|
||||
is_applicable(#resource{kind = queue} = Resource, Policy) ->
|
||||
rabbit_amqqueue:is_policy_applicable(Resource, Policy);
|
||||
rabbit_amqqueue:is_policy_applicable(Resource, to_list(Policy));
|
||||
is_applicable(_, _) ->
|
||||
true.
|
||||
|
||||
to_list(L) when is_list(L) ->
|
||||
L;
|
||||
to_list(M) when is_map(M) ->
|
||||
maps:to_list(M).
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
operator_policy_validation() ->
|
||||
|
|
|
@ -0,0 +1,560 @@
|
|||
-module(rabbit_queue_type).
|
||||
-include("amqqueue.hrl").
|
||||
-include_lib("rabbit_common/include/resource.hrl").
|
||||
|
||||
-export([
|
||||
init/0,
|
||||
close/1,
|
||||
discover/1,
|
||||
default/0,
|
||||
is_enabled/1,
|
||||
declare/2,
|
||||
delete/4,
|
||||
is_recoverable/1,
|
||||
recover/2,
|
||||
purge/1,
|
||||
policy_changed/1,
|
||||
stat/1,
|
||||
remove/2,
|
||||
info/2,
|
||||
state_info/1,
|
||||
info_down/3,
|
||||
%% stateful client API
|
||||
new/2,
|
||||
consume/3,
|
||||
cancel/5,
|
||||
handle_down/3,
|
||||
handle_event/3,
|
||||
module/2,
|
||||
deliver/3,
|
||||
settle/5,
|
||||
credit/5,
|
||||
dequeue/5,
|
||||
fold_state/3,
|
||||
is_policy_applicable/2
|
||||
]).
|
||||
|
||||
%% temporary
|
||||
-export([with/3]).
|
||||
|
||||
%% gah what is a good identity of a classic queue including all replicas
|
||||
-type queue_name() :: rabbit_types:r(queue).
|
||||
-type queue_ref() :: queue_name() | atom().
|
||||
-type queue_state() :: term().
|
||||
-type msg_tag() :: term().
|
||||
|
||||
-define(STATE, ?MODULE).
|
||||
|
||||
-define(QREF(QueueReference),
|
||||
(is_tuple(QueueReference) andalso element(1, QueueReference) == resource)
|
||||
orelse is_atom(QueueReference)).
|
||||
%% anything that the host process needs to do on behalf of the queue type
|
||||
%% session, like knowing when to notify on monitor down
|
||||
-type action() ::
|
||||
{monitor, Pid :: pid(), queue_ref()} |
|
||||
%% indicate to the queue type module that a message has been delivered
|
||||
%% fully to the queue
|
||||
{settled, Success :: boolean(), [msg_tag()]} |
|
||||
{deliver, rabbit_type:ctag(), boolean(), [rabbit_amqqueue:qmsg()]}.
|
||||
|
||||
-type actions() :: [action()].
|
||||
|
||||
-type event() ::
|
||||
{down, pid(), Info :: term()} |
|
||||
term().
|
||||
|
||||
-record(ctx, {module :: module(),
|
||||
name :: queue_name(),
|
||||
%% "publisher confirm queue accounting"
|
||||
%% queue type implementation should emit a:
|
||||
%% {settle, Success :: boolean(), msg_tag()}
|
||||
%% to either settle or reject the delivery of a
|
||||
%% message to the queue instance
|
||||
%% The queue type module will then emit a {confirm | reject, [msg_tag()}
|
||||
%% action to the channel or channel like process when a msg_tag
|
||||
%% has reached its conclusion
|
||||
% unsettled = #{} :: #{msg_tag() => [queue_ref()]},
|
||||
state :: queue_state()}).
|
||||
|
||||
|
||||
-record(?STATE, {ctxs = #{} :: #{queue_ref() => #ctx{} | queue_ref()},
|
||||
monitor_registry = #{} :: #{pid() => queue_ref()}
|
||||
}).
|
||||
|
||||
-opaque state() :: #?STATE{}.
|
||||
|
||||
-type consume_spec() :: #{no_ack := boolean(),
|
||||
channel_pid := pid(),
|
||||
limiter_pid => pid(),
|
||||
limiter_active => boolean(),
|
||||
prefetch_count => non_neg_integer(),
|
||||
consumer_tag := rabbit_types:ctag(),
|
||||
exclusive_consume => boolean(),
|
||||
args => rabbit_framing:amqp_table(),
|
||||
ok_msg := term(),
|
||||
acting_user := rabbit_types:username()}.
|
||||
|
||||
|
||||
|
||||
% copied from rabbit_amqqueue
|
||||
-type absent_reason() :: 'nodedown' | 'crashed' | stopped | timeout.
|
||||
|
||||
-type settle_op() :: 'complete' | 'requeue' | 'discard'.
|
||||
|
||||
-export_type([state/0,
|
||||
consume_spec/0,
|
||||
action/0,
|
||||
actions/0,
|
||||
settle_op/0]).
|
||||
|
||||
%% is the queue type feature enabled
|
||||
-callback is_enabled() -> boolean().
|
||||
|
||||
-callback declare(amqqueue:amqqueue(), node()) ->
|
||||
{'new' | 'existing' | 'owner_died', amqqueue:amqqueue()} |
|
||||
{'absent', amqqueue:amqqueue(), absent_reason()} |
|
||||
rabbit_types:channel_exit().
|
||||
|
||||
-callback delete(amqqueue:amqqueue(),
|
||||
boolean(),
|
||||
boolean(),
|
||||
rabbit_types:username()) ->
|
||||
rabbit_types:ok(non_neg_integer()) |
|
||||
rabbit_types:error(in_use | not_empty).
|
||||
|
||||
-callback recover(rabbit_types:vhost(), [amqqueue:amqqueue()]) ->
|
||||
{Recovered :: [amqqueue:amqqueue()],
|
||||
Failed :: [amqqueue:amqqueue()]}.
|
||||
|
||||
%% checks if the queue should be recovered
|
||||
-callback is_recoverable(amqqueue:amqqueue()) ->
|
||||
boolean().
|
||||
|
||||
-callback purge(amqqueue:amqqueue()) ->
|
||||
{ok, non_neg_integer()} | {error, term()}.
|
||||
|
||||
-callback policy_changed(amqqueue:amqqueue()) -> ok.
|
||||
|
||||
%% stateful
|
||||
%% intitialise and return a queue type specific session context
|
||||
-callback init(amqqueue:amqqueue()) -> queue_state().
|
||||
|
||||
-callback close(queue_state()) -> ok.
|
||||
%% update the queue type state from amqqrecord
|
||||
-callback update(amqqueue:amqqueue(), queue_state()) -> queue_state().
|
||||
|
||||
-callback consume(amqqueue:amqqueue(),
|
||||
consume_spec(),
|
||||
queue_state()) ->
|
||||
{ok, queue_state(), actions()} | {error, term()}.
|
||||
|
||||
-callback cancel(amqqueue:amqqueue(),
|
||||
rabbit_types:ctag(),
|
||||
term(),
|
||||
rabbit_types:username(),
|
||||
queue_state()) ->
|
||||
{ok, queue_state()} | {error, term()}.
|
||||
|
||||
%% any async events returned from the queue system should be processed through
|
||||
%% this
|
||||
-callback handle_event(Event :: event(),
|
||||
queue_state()) ->
|
||||
{ok, queue_state(), actions()} | {error, term()} | eol.
|
||||
|
||||
-callback deliver([{amqqueue:amqqueue(), queue_state()}],
|
||||
Delivery :: term()) ->
|
||||
{[{amqqueue:amqqueue(), queue_state()}], actions()}.
|
||||
|
||||
-callback settle(settle_op(), rabbit_types:ctag(), [non_neg_integer()], queue_state()) ->
|
||||
{queue_state(), actions()}.
|
||||
|
||||
-callback credit(rabbit_types:ctag(),
|
||||
non_neg_integer(), Drain :: boolean(), queue_state()) ->
|
||||
queue_state().
|
||||
|
||||
-callback dequeue(NoAck :: boolean(), LimiterPid :: pid(),
|
||||
rabbit_types:ctag(), queue_state()) ->
|
||||
{ok, Count :: non_neg_integer(), rabbit_amqqueue:qmsg(), queue_state()} |
|
||||
{empty, queue_state()} |
|
||||
{error, term()}.
|
||||
|
||||
%% return a map of state summary information
|
||||
-callback state_info(queue_state()) ->
|
||||
#{atom() := term()}.
|
||||
|
||||
%% general queue info
|
||||
-callback info(amqqueue:amqqueue(), all_keys | rabbit_types:info_keys()) ->
|
||||
rabbit_types:infos().
|
||||
|
||||
-callback stat(amqqueue:amqqueue()) ->
|
||||
{'ok', non_neg_integer(), non_neg_integer()}.
|
||||
|
||||
-callback is_policy_applicable(amqqueue:amqqueue(), any()) ->
|
||||
boolean().
|
||||
|
||||
%% TODO: this should be controlled by a registry that is populated on boot
|
||||
discover(<<"quorum">>) ->
|
||||
rabbit_quorum_queue;
|
||||
discover(<<"classic">>) ->
|
||||
rabbit_classic_queue;
|
||||
discover(<<"stream">>) ->
|
||||
rabbit_stream_queue.
|
||||
|
||||
default() ->
|
||||
rabbit_classic_queue.
|
||||
|
||||
-spec is_enabled(module()) -> boolean().
|
||||
is_enabled(Type) ->
|
||||
Type:is_enabled().
|
||||
|
||||
-spec declare(amqqueue:amqqueue(), node()) ->
|
||||
{'new' | 'existing' | 'owner_died', amqqueue:amqqueue()} |
|
||||
{'absent', amqqueue:amqqueue(), absent_reason()} |
|
||||
rabbit_types:channel_exit().
|
||||
declare(Q, Node) ->
|
||||
Mod = amqqueue:get_type(Q),
|
||||
Mod:declare(Q, Node).
|
||||
|
||||
-spec delete(amqqueue:amqqueue(), boolean(),
|
||||
boolean(), rabbit_types:username()) ->
|
||||
rabbit_types:ok(non_neg_integer()) |
|
||||
rabbit_types:error(in_use | not_empty).
|
||||
delete(Q, IfUnused, IfEmpty, ActingUser) ->
|
||||
Mod = amqqueue:get_type(Q),
|
||||
Mod:delete(Q, IfUnused, IfEmpty, ActingUser).
|
||||
|
||||
-spec purge(amqqueue:amqqueue()) ->
|
||||
{'ok', non_neg_integer()}.
|
||||
purge(Q) ->
|
||||
Mod = amqqueue:get_type(Q),
|
||||
Mod:purge(Q).
|
||||
|
||||
-spec policy_changed(amqqueue:amqqueue()) -> 'ok'.
|
||||
policy_changed(Q) ->
|
||||
Mod = amqqueue:get_type(Q),
|
||||
Mod:policy_changed(Q).
|
||||
|
||||
-spec stat(amqqueue:amqqueue()) ->
|
||||
{'ok', non_neg_integer(), non_neg_integer()}.
|
||||
stat(Q) ->
|
||||
Mod = amqqueue:get_type(Q),
|
||||
Mod:stat(Q).
|
||||
|
||||
-spec remove(queue_ref(), state()) -> state().
|
||||
remove(QRef, #?STATE{ctxs = Ctxs0} = State) ->
|
||||
case maps:take(QRef, Ctxs0) of
|
||||
error ->
|
||||
State#?STATE{ctxs = Ctxs0};
|
||||
{_, Ctxs} ->
|
||||
%% remove all linked queue refs
|
||||
State#?STATE{ctxs = maps:filter(fun (_, V) ->
|
||||
V == QRef
|
||||
end, Ctxs)}
|
||||
end.
|
||||
|
||||
-spec info(amqqueue:amqqueue(), all_keys | rabbit_types:info_keys()) ->
|
||||
rabbit_types:infos().
|
||||
info(Q, Items) when ?amqqueue_state_is(Q, crashed) ->
|
||||
info_down(Q, Items, crashed);
|
||||
info(Q, Items) when ?amqqueue_state_is(Q, stopped) ->
|
||||
info_down(Q, Items, stopped);
|
||||
info(Q, Items) ->
|
||||
Mod = amqqueue:get_type(Q),
|
||||
Mod:info(Q, Items).
|
||||
|
||||
fold_state(Fun, Acc, #?STATE{ctxs = Ctxs}) ->
|
||||
maps:fold(Fun, Acc, Ctxs).
|
||||
|
||||
state_info(#ctx{state = S,
|
||||
module = Mod}) ->
|
||||
Mod:state_info(S);
|
||||
state_info(_) ->
|
||||
#{}.
|
||||
|
||||
info_down(Q, all_keys, DownReason) ->
|
||||
info_down(Q, rabbit_amqqueue_process:info_keys(), DownReason);
|
||||
info_down(Q, Items, DownReason) ->
|
||||
[{Item, i_down(Item, Q, DownReason)} || Item <- Items].
|
||||
|
||||
i_down(name, Q, _) -> amqqueue:get_name(Q);
|
||||
i_down(durable, Q, _) -> amqqueue:is_durable(Q);
|
||||
i_down(auto_delete, Q, _) -> amqqueue:is_auto_delete(Q);
|
||||
i_down(arguments, Q, _) -> amqqueue:get_arguments(Q);
|
||||
i_down(pid, Q, _) -> amqqueue:get_pid(Q);
|
||||
i_down(recoverable_slaves, Q, _) -> amqqueue:get_recoverable_slaves(Q);
|
||||
i_down(type, Q, _) -> amqqueue:get_type(Q);
|
||||
i_down(state, _Q, DownReason) -> DownReason;
|
||||
i_down(K, _Q, _DownReason) ->
|
||||
case lists:member(K, rabbit_amqqueue_process:info_keys()) of
|
||||
true -> '';
|
||||
false -> throw({bad_argument, K})
|
||||
end.
|
||||
|
||||
is_policy_applicable(Q, Policy) ->
|
||||
Mod = amqqueue:get_type(Q),
|
||||
Mod:is_policy_applicable(Q, Policy).
|
||||
|
||||
-spec init() -> state().
|
||||
init() ->
|
||||
#?STATE{}.
|
||||
|
||||
-spec close(state()) -> ok.
|
||||
close(#?STATE{ctxs = Contexts}) ->
|
||||
_ = maps:map(
|
||||
fun (_, #ctx{module = Mod,
|
||||
state = S}) ->
|
||||
ok = Mod:close(S)
|
||||
end, Contexts),
|
||||
ok.
|
||||
|
||||
-spec new(amqqueue:amqqueue(), state()) -> state().
|
||||
new(Q, State) when ?is_amqqueue(Q) ->
|
||||
Ctx = get_ctx(Q, State),
|
||||
set_ctx(Q, Ctx, State).
|
||||
|
||||
-spec consume(amqqueue:amqqueue(), consume_spec(), state()) ->
|
||||
{ok, state(), actions()} | {error, term()}.
|
||||
consume(Q, Spec, State) ->
|
||||
#ctx{state = State0} = Ctx = get_ctx(Q, State),
|
||||
Mod = amqqueue:get_type(Q),
|
||||
case Mod:consume(Q, Spec, State0) of
|
||||
{ok, CtxState, Actions} ->
|
||||
return_ok(set_ctx(Q, Ctx#ctx{state = CtxState}, State), Actions);
|
||||
Err ->
|
||||
Err
|
||||
end.
|
||||
|
||||
%% TODO switch to cancel spec api
|
||||
-spec cancel(amqqueue:amqqueue(),
|
||||
rabbit_types:ctag(),
|
||||
term(),
|
||||
rabbit_types:username(),
|
||||
state()) ->
|
||||
{ok, state()} | {error, term()}.
|
||||
cancel(Q, Tag, OkMsg, ActiveUser, Ctxs) ->
|
||||
#ctx{state = State0} = Ctx = get_ctx(Q, Ctxs),
|
||||
Mod = amqqueue:get_type(Q),
|
||||
case Mod:cancel(Q, Tag, OkMsg, ActiveUser, State0) of
|
||||
{ok, State} ->
|
||||
{ok, set_ctx(Q, Ctx#ctx{state = State}, Ctxs)};
|
||||
Err ->
|
||||
Err
|
||||
end.
|
||||
|
||||
-spec is_recoverable(amqqueue:amqqueue()) ->
|
||||
boolean().
|
||||
is_recoverable(Q) ->
|
||||
Mod = amqqueue:get_type(Q),
|
||||
Mod:is_recoverable(Q).
|
||||
|
||||
-spec recover(rabbit_types:vhost(), [amqqueue:amqqueue()]) ->
|
||||
{Recovered :: [amqqueue:amqqueue()],
|
||||
Failed :: [amqqueue:amqqueue()]}.
|
||||
recover(VHost, Qs) ->
|
||||
ByType = lists:foldl(
|
||||
fun (Q, Acc) ->
|
||||
T = amqqueue:get_type(Q),
|
||||
maps:update_with(T, fun (X) ->
|
||||
[Q | X]
|
||||
end, Acc)
|
||||
%% TODO resolve all registered queue types from registry
|
||||
end, #{rabbit_classic_queue => [],
|
||||
rabbit_quorum_queue => [],
|
||||
rabbit_stream_queue => []}, Qs),
|
||||
maps:fold(fun (Mod, Queues, {R0, F0}) ->
|
||||
{R, F} = Mod:recover(VHost, Queues),
|
||||
{R0 ++ R, F0 ++ F}
|
||||
end, {[], []}, ByType).
|
||||
|
||||
-spec handle_down(pid(), term(), state()) ->
|
||||
{ok, state(), actions()} | {eol, queue_ref()} | {error, term()}.
|
||||
handle_down(Pid, Info, #?STATE{monitor_registry = Reg} = State0) ->
|
||||
%% lookup queue ref in monitor registry
|
||||
case Reg of
|
||||
#{Pid := QRef} ->
|
||||
%% TODO: remove Pid from monitor_registry
|
||||
case handle_event(QRef, {down, Pid, Info}, State0) of
|
||||
{ok, State, Actions} ->
|
||||
{ok, State, [{queue_down, QRef} | Actions]};
|
||||
eol ->
|
||||
{eol, QRef};
|
||||
Err ->
|
||||
Err
|
||||
end;
|
||||
_ ->
|
||||
{ok, State0, []}
|
||||
end.
|
||||
|
||||
%% messages sent from queues
|
||||
-spec handle_event(queue_ref(), term(), state()) ->
|
||||
{ok, state(), actions()} | eol | {error, term()}.
|
||||
handle_event(QRef, Evt, Ctxs) ->
|
||||
%% events can arrive after a queue state has been cleared up
|
||||
%% so need to be defensive here
|
||||
case get_ctx(QRef, Ctxs, undefined) of
|
||||
#ctx{module = Mod,
|
||||
state = State0} = Ctx ->
|
||||
case Mod:handle_event(Evt, State0) of
|
||||
{ok, State, Actions} ->
|
||||
return_ok(set_ctx(QRef, Ctx#ctx{state = State}, Ctxs), Actions);
|
||||
Err ->
|
||||
Err
|
||||
end;
|
||||
undefined ->
|
||||
{ok, Ctxs, []}
|
||||
end.
|
||||
|
||||
-spec module(queue_ref(), state()) ->
|
||||
{ok, module()} | {error, not_found}.
|
||||
module(QRef, Ctxs) ->
|
||||
%% events can arrive after a queue state has been cleared up
|
||||
%% so need to be defensive here
|
||||
case get_ctx(QRef, Ctxs, undefined) of
|
||||
#ctx{module = Mod} ->
|
||||
{ok, Mod};
|
||||
undefined ->
|
||||
{error, not_found}
|
||||
end.
|
||||
|
||||
-spec deliver([amqqueue:amqqueue()], Delivery :: term(),
|
||||
stateless | state()) ->
|
||||
{ok, state(), actions()}.
|
||||
deliver(Qs, Delivery, stateless) ->
|
||||
_ = lists:map(fun(Q) ->
|
||||
Mod = amqqueue:get_type(Q),
|
||||
_ = Mod:deliver([{Q, stateless}], Delivery)
|
||||
end, Qs),
|
||||
{ok, stateless, []};
|
||||
deliver(Qs, Delivery, #?STATE{} = State0) ->
|
||||
%% sort by queue type - then dispatch each group
|
||||
ByType = lists:foldl(
|
||||
fun (Q, Acc) ->
|
||||
T = amqqueue:get_type(Q),
|
||||
Ctx = get_ctx(Q, State0),
|
||||
maps:update_with(
|
||||
T, fun (A) ->
|
||||
[{Q, Ctx#ctx.state} | A]
|
||||
end, [{Q, Ctx#ctx.state}], Acc)
|
||||
end, #{}, Qs),
|
||||
%%% dispatch each group to queue type interface?
|
||||
{Xs, Actions} = maps:fold(fun(Mod, QSs, {X0, A0}) ->
|
||||
{X, A} = Mod:deliver(QSs, Delivery),
|
||||
{X0 ++ X, A0 ++ A}
|
||||
end, {[], []}, ByType),
|
||||
State = lists:foldl(
|
||||
fun({Q, S}, Acc) ->
|
||||
Ctx = get_ctx(Q, Acc),
|
||||
set_ctx(qref(Q), Ctx#ctx{state = S}, Acc)
|
||||
end, State0, Xs),
|
||||
return_ok(State, Actions).
|
||||
|
||||
|
||||
-spec settle(queue_ref(), settle_op(), rabbit_types:ctag(),
|
||||
[non_neg_integer()], state()) -> {ok, state(), actions()}.
|
||||
settle(QRef, Op, CTag, MsgIds, Ctxs)
|
||||
when ?QREF(QRef) ->
|
||||
#ctx{state = State0,
|
||||
module = Mod} = Ctx = get_ctx(QRef, Ctxs),
|
||||
{State, Actions} = Mod:settle(Op, CTag, MsgIds, State0),
|
||||
{ok, set_ctx(QRef, Ctx#ctx{state = State}, Ctxs), Actions}.
|
||||
|
||||
-spec credit(amqqueue:amqqueue() | queue_ref(),
|
||||
rabbit_types:ctag(), non_neg_integer(),
|
||||
boolean(), state()) -> state().
|
||||
credit(Q, CTag, Credit, Drain, Ctxs) ->
|
||||
#ctx{state = State0,
|
||||
module = Mod} = Ctx = get_ctx(Q, Ctxs),
|
||||
State = Mod:credit(CTag, Credit, Drain, State0),
|
||||
set_ctx(Q, Ctx#ctx{state = State}, Ctxs).
|
||||
|
||||
-spec dequeue(amqqueue:amqqueue(), boolean(),
|
||||
pid(), rabbit_types:ctag(), state()) ->
|
||||
{ok, non_neg_integer(), term(), state()} |
|
||||
{empty, state()}.
|
||||
dequeue(Q, NoAck, LimiterPid, CTag, Ctxs) ->
|
||||
#ctx{state = State0} = Ctx = get_ctx(Q, Ctxs),
|
||||
Mod = amqqueue:get_type(Q),
|
||||
case Mod:dequeue(NoAck, LimiterPid, CTag, State0) of
|
||||
{ok, Num, Msg, State} ->
|
||||
{ok, Num, Msg, set_ctx(Q, Ctx#ctx{state = State}, Ctxs)};
|
||||
{empty, State} ->
|
||||
{empty, set_ctx(Q, Ctx#ctx{state = State}, Ctxs)};
|
||||
{error, _} = Err ->
|
||||
Err
|
||||
end.
|
||||
|
||||
%% temporary
|
||||
with(QRef, Fun, Ctxs) ->
|
||||
#ctx{state = State0} = Ctx = get_ctx(QRef, Ctxs),
|
||||
{Res, State} = Fun(State0),
|
||||
{Res, set_ctx(QRef, Ctx#ctx{state = State}, Ctxs)}.
|
||||
|
||||
|
||||
get_ctx(Q, #?STATE{ctxs = Contexts}) when ?is_amqqueue(Q) ->
|
||||
Ref = qref(Q),
|
||||
case Contexts of
|
||||
#{Ref := #ctx{module = Mod,
|
||||
state = State} = Ctx} ->
|
||||
Ctx#ctx{state = Mod:update(Q, State)};
|
||||
_ ->
|
||||
%% not found - initialize
|
||||
Mod = amqqueue:get_type(Q),
|
||||
Name = amqqueue:get_name(Q),
|
||||
#ctx{module = Mod,
|
||||
name = Name,
|
||||
state = Mod:init(Q)}
|
||||
end;
|
||||
get_ctx(QRef, Contexts) when ?QREF(QRef) ->
|
||||
case get_ctx(QRef, Contexts, undefined) of
|
||||
undefined ->
|
||||
exit({queue_context_not_found, QRef});
|
||||
Ctx ->
|
||||
Ctx
|
||||
end.
|
||||
|
||||
get_ctx(QRef, #?STATE{ctxs = Contexts}, Default) ->
|
||||
Ref = qref(QRef),
|
||||
%% if we use a QRef it should always be initialised
|
||||
case maps:get(Ref, Contexts, undefined) of
|
||||
#ctx{} = Ctx ->
|
||||
Ctx;
|
||||
undefined ->
|
||||
Default
|
||||
end.
|
||||
|
||||
set_ctx(Q, Ctx, #?STATE{ctxs = Contexts} = State)
|
||||
when ?is_amqqueue(Q) ->
|
||||
Ref = qref(Q),
|
||||
State#?STATE{ctxs = maps:put(Ref, Ctx, Contexts)};
|
||||
set_ctx(QRef, Ctx, #?STATE{ctxs = Contexts} = State) ->
|
||||
Ref = qref(QRef),
|
||||
State#?STATE{ctxs = maps:put(Ref, Ctx, Contexts)}.
|
||||
|
||||
qref(#resource{kind = queue} = QName) ->
|
||||
QName;
|
||||
qref(Q) when ?is_amqqueue(Q) ->
|
||||
amqqueue:get_name(Q).
|
||||
|
||||
return_ok(State0, []) ->
|
||||
{ok, State0, []};
|
||||
return_ok(State0, Actions0) ->
|
||||
{State, Actions} =
|
||||
lists:foldl(
|
||||
fun({monitor, Pid, QRef},
|
||||
{#?STATE{monitor_registry = M0} = S0, A0}) ->
|
||||
case M0 of
|
||||
#{Pid := QRef} ->
|
||||
%% already monitored by the qref
|
||||
{S0, A0};
|
||||
#{Pid := _} ->
|
||||
%% TODO: allow multiple Qrefs to monitor the same pid
|
||||
exit(return_ok_duplicate_montored_pid);
|
||||
_ ->
|
||||
_ = erlang:monitor(process, Pid),
|
||||
M = M0#{Pid => QRef},
|
||||
{S0#?STATE{monitor_registry = M}, A0}
|
||||
end;
|
||||
(Act, {S, A0}) ->
|
||||
{S, [Act | A0]}
|
||||
end, {State0, []}, Actions0),
|
||||
{ok, State, lists:reverse(Actions)}.
|
|
@ -0,0 +1,80 @@
|
|||
%% 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 https://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-2020 Pivotal Software, Inc. All rights reserved.
|
||||
%%
|
||||
|
||||
-module(rabbit_queue_type_util).
|
||||
|
||||
-export([check_invalid_arguments/3,
|
||||
args_policy_lookup/3,
|
||||
qname_to_internal_name/1,
|
||||
check_auto_delete/1,
|
||||
check_exclusive/1,
|
||||
check_non_durable/1]).
|
||||
|
||||
-include("rabbit.hrl").
|
||||
-include("amqqueue.hrl").
|
||||
|
||||
check_invalid_arguments(QueueName, Args, Keys) ->
|
||||
[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.
|
||||
|
||||
args_policy_lookup(Name, Resolve, Q) when ?is_amqqueue(Q) ->
|
||||
Args = amqqueue:get_arguments(Q),
|
||||
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.
|
||||
|
||||
%% TODO escape hack
|
||||
qname_to_internal_name(#resource{virtual_host = <<"/">>, name = Name}) ->
|
||||
erlang:binary_to_atom(<<"%2F_", Name/binary>>, utf8);
|
||||
qname_to_internal_name(#resource{virtual_host = VHost, name = Name}) ->
|
||||
erlang:binary_to_atom(<<VHost/binary, "_", Name/binary>>, utf8).
|
||||
|
||||
check_auto_delete(Q) when ?amqqueue_is_auto_delete(Q) ->
|
||||
Name = amqqueue:get_name(Q),
|
||||
rabbit_misc:protocol_error(
|
||||
precondition_failed,
|
||||
"invalid property 'auto-delete' for ~s",
|
||||
[rabbit_misc:rs(Name)]);
|
||||
check_auto_delete(_) ->
|
||||
ok.
|
||||
|
||||
check_exclusive(Q) when ?amqqueue_exclusive_owner_is(Q, none) ->
|
||||
ok;
|
||||
check_exclusive(Q) when ?is_amqqueue(Q) ->
|
||||
Name = amqqueue:get_name(Q),
|
||||
rabbit_misc:protocol_error(
|
||||
precondition_failed,
|
||||
"invalid property 'exclusive-owner' for ~s",
|
||||
[rabbit_misc:rs(Name)]).
|
||||
|
||||
check_non_durable(Q) when ?amqqueue_is_durable(Q) ->
|
||||
ok;
|
||||
check_non_durable(Q) when not ?amqqueue_is_durable(Q) ->
|
||||
Name = amqqueue:get_name(Q),
|
||||
rabbit_misc:protocol_error(
|
||||
precondition_failed,
|
||||
"invalid property 'non-durable' for ~s",
|
||||
[rabbit_misc:rs(Name)]).
|
|
@ -7,13 +7,18 @@
|
|||
|
||||
-module(rabbit_quorum_queue).
|
||||
|
||||
-export([init_state/2, handle_event/2]).
|
||||
-export([declare/1, recover/1, stop/1, delete/4, delete_immediately/2]).
|
||||
-export([info/1, info/2, stat/1, stat/2, infos/1]).
|
||||
-export([ack/3, reject/4, basic_get/4, basic_consume/10, basic_cancel/4]).
|
||||
-behaviour(rabbit_queue_type).
|
||||
|
||||
-export([init/1,
|
||||
close/1,
|
||||
update/2,
|
||||
handle_event/2]).
|
||||
-export([is_recoverable/1, recover/2, stop/1, delete/4, delete_immediately/2]).
|
||||
-export([state_info/1, info/2, stat/1, infos/1]).
|
||||
-export([settle/4, dequeue/4, consume/3, cancel/5]).
|
||||
-export([credit/4]).
|
||||
-export([purge/1]).
|
||||
-export([stateless_deliver/2, deliver/3]).
|
||||
-export([stateless_deliver/2, deliver/3, deliver/2]).
|
||||
-export([dead_letter_publish/4]).
|
||||
-export([queue_name/1]).
|
||||
-export([cluster_state/1, status/2]).
|
||||
|
@ -27,7 +32,8 @@
|
|||
-export([add_member/4]).
|
||||
-export([delete_member/3]).
|
||||
-export([requeue/3]).
|
||||
-export([policy_changed/2]).
|
||||
-export([policy_changed/1]).
|
||||
-export([format_ra_event/3]).
|
||||
-export([cleanup_data_dir/0]).
|
||||
-export([shrink_all/1,
|
||||
grow/4]).
|
||||
|
@ -43,6 +49,12 @@
|
|||
]).
|
||||
-export([reclaim_memory/2]).
|
||||
|
||||
-export([is_enabled/0,
|
||||
declare/2]).
|
||||
|
||||
-import(rabbit_queue_type_util, [args_policy_lookup/3,
|
||||
qname_to_internal_name/1]).
|
||||
|
||||
-include_lib("stdlib/include/qlc.hrl").
|
||||
-include("rabbit.hrl").
|
||||
-include("amqqueue.hrl").
|
||||
|
@ -73,17 +85,22 @@
|
|||
-define(DELETE_TIMEOUT, 5000).
|
||||
-define(ADD_MEMBER_TIMEOUT, 5000).
|
||||
|
||||
%%----------- rabbit_queue_type ---------------------------------------------
|
||||
|
||||
-spec is_enabled() -> boolean().
|
||||
is_enabled() ->
|
||||
rabbit_feature_flags:is_enabled(quorum_queue).
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
-spec init_state(amqqueue:ra_server_id(), rabbit_amqqueue:name()) ->
|
||||
rabbit_fifo_client:state().
|
||||
init_state({Name, _}, QName = #resource{}) ->
|
||||
-spec init(amqqueue:amqqueue()) -> rabbit_fifo_client:state().
|
||||
init(Q) when ?is_amqqueue(Q) ->
|
||||
{ok, SoftLimit} = application:get_env(rabbit, quorum_commands_soft_limit),
|
||||
%% This lookup could potentially return an {error, not_found}, but we do not
|
||||
%% know what to do if the queue has `disappeared`. Let it crash.
|
||||
{ok, Q} = rabbit_amqqueue:lookup(QName),
|
||||
Leader = amqqueue:get_pid(Q),
|
||||
Nodes = rabbit_amqqueue:get_quorum_nodes(Q),
|
||||
{Name, _LeaderNode} = Leader = amqqueue:get_pid(Q),
|
||||
Nodes = get_nodes(Q),
|
||||
QName = amqqueue:get_name(Q),
|
||||
%% Ensure the leader is listed first
|
||||
Servers0 = [{Name, N} || N <- Nodes],
|
||||
Servers = [Leader | lists:delete(Leader, Servers0)],
|
||||
|
@ -91,19 +108,26 @@ init_state({Name, _}, QName = #resource{}) ->
|
|||
fun() -> credit_flow:block(Name) end,
|
||||
fun() -> credit_flow:unblock(Name), ok end).
|
||||
|
||||
-spec handle_event({'ra_event', amqqueue:ra_server_id(), any()},
|
||||
-spec close(rabbit_fifo_client:state()) -> ok.
|
||||
close(_State) ->
|
||||
ok.
|
||||
|
||||
-spec update(amqqueue:amqqueue(), rabbit_fifo_client:state()) ->
|
||||
rabbit_fifo_client:state().
|
||||
update(Q, State) when ?amqqueue_is_quorum(Q) ->
|
||||
%% QQ state maintains it's own updates
|
||||
State.
|
||||
|
||||
-spec handle_event({amqqueue:ra_server_id(), any()},
|
||||
rabbit_fifo_client:state()) ->
|
||||
{internal, Correlators :: [term()], rabbit_fifo_client:actions(),
|
||||
rabbit_fifo_client:state()} |
|
||||
{rabbit_fifo:client_msg(), rabbit_fifo_client:state()} |
|
||||
{ok, rabbit_fifo_client:state(), rabbit_queue_type:actions()} |
|
||||
eol.
|
||||
handle_event({ra_event, From, Evt}, QState) ->
|
||||
handle_event({From, Evt}, QState) ->
|
||||
rabbit_fifo_client:handle_ra_event(From, Evt, QState).
|
||||
|
||||
-spec declare(amqqueue:amqqueue()) ->
|
||||
-spec declare(amqqueue:amqqueue(), node()) ->
|
||||
{new | existing, amqqueue:amqqueue()} | rabbit_types:channel_exit().
|
||||
|
||||
declare(Q) when ?amqqueue_is_quorum(Q) ->
|
||||
declare(Q, _Node) when ?amqqueue_is_quorum(Q) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
Durable = amqqueue:is_durable(Q),
|
||||
AutoDelete = amqqueue:is_auto_delete(Q),
|
||||
|
@ -111,11 +135,11 @@ declare(Q) when ?amqqueue_is_quorum(Q) ->
|
|||
Opts = amqqueue:get_options(Q),
|
||||
ActingUser = maps:get(user, Opts, ?UNKNOWN_USER),
|
||||
check_invalid_arguments(QName, Arguments),
|
||||
check_auto_delete(Q),
|
||||
check_exclusive(Q),
|
||||
check_non_durable(Q),
|
||||
rabbit_queue_type_util:check_auto_delete(Q),
|
||||
rabbit_queue_type_util:check_exclusive(Q),
|
||||
rabbit_queue_type_util:check_non_durable(Q),
|
||||
QuorumSize = get_default_quorum_initial_group_size(Arguments),
|
||||
RaName = qname_to_rname(QName),
|
||||
RaName = qname_to_internal_name(QName),
|
||||
Id = {RaName, node()},
|
||||
Nodes = select_quorum_nodes(QuorumSize, rabbit_mnesia:cluster_nodes(all)),
|
||||
NewQ0 = amqqueue:set_pid(Q, Id),
|
||||
|
@ -250,7 +274,8 @@ all_replica_states() ->
|
|||
|
||||
-spec list_with_minimum_quorum() -> [amqqueue:amqqueue()].
|
||||
list_with_minimum_quorum() ->
|
||||
filter_quorum_critical(rabbit_amqqueue:list_local_quorum_queues()).
|
||||
filter_quorum_critical(
|
||||
rabbit_amqqueue:list_local_quorum_queues()).
|
||||
|
||||
-spec list_with_minimum_quorum_for_cli() -> [#{binary() => term()}].
|
||||
list_with_minimum_quorum_for_cli() ->
|
||||
|
@ -436,51 +461,64 @@ reductions(Name) ->
|
|||
0
|
||||
end.
|
||||
|
||||
-spec recover([amqqueue:amqqueue()]) -> [amqqueue:amqqueue()].
|
||||
is_recoverable(Q) ->
|
||||
Node = node(),
|
||||
Nodes = get_nodes(Q),
|
||||
lists:member(Node, Nodes).
|
||||
|
||||
recover(Queues) ->
|
||||
[begin
|
||||
-spec recover(binary(), [amqqueue:amqqueue()]) ->
|
||||
{[amqqueue:amqqueue()], [amqqueue:amqqueue()]}.
|
||||
recover(_Vhost, Queues) ->
|
||||
lists:foldl(
|
||||
fun (Q0, {R0, F0}) ->
|
||||
{Name, _} = amqqueue:get_pid(Q0),
|
||||
case ra:restart_server({Name, node()}) of
|
||||
ok ->
|
||||
% queue was restarted, good
|
||||
ok;
|
||||
{error, Err1}
|
||||
when Err1 == not_started orelse
|
||||
Err1 == name_not_registered ->
|
||||
% queue was never started on this node
|
||||
% so needs to be started from scratch.
|
||||
TickTimeout = application:get_env(rabbit, quorum_tick_interval,
|
||||
?TICK_TIMEOUT),
|
||||
Conf = make_ra_conf(Q0, {Name, node()}, TickTimeout),
|
||||
case ra:start_server(Conf) of
|
||||
ok ->
|
||||
ok;
|
||||
Err2 ->
|
||||
rabbit_log:warning("recover: quorum queue ~w could not"
|
||||
" be started ~w", [Name, Err2]),
|
||||
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,
|
||||
QName = amqqueue:get_name(Q0),
|
||||
Nodes = get_nodes(Q0),
|
||||
Formatter = {?MODULE, format_ra_event, [QName]},
|
||||
Res = case ra:restart_server({Name, node()},
|
||||
#{ra_event_formatter => Formatter}) of
|
||||
ok ->
|
||||
% queue was restarted, good
|
||||
ok;
|
||||
{error, Err1}
|
||||
when Err1 == not_started orelse
|
||||
Err1 == 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;
|
||||
Err2 ->
|
||||
rabbit_log:warning("recover: quorum queue ~w could not"
|
||||
" be started ~w", [Name, Err2]),
|
||||
fail
|
||||
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]),
|
||||
fail
|
||||
end,
|
||||
%% we have to ensure the quorum queue is
|
||||
%% present in the rabbit_queue table and not just in rabbit_durable_queue
|
||||
%% 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 || Q0 <- Queues].
|
||||
|
||||
-spec stop(rabbit_types:vhost()) -> 'ok'.
|
||||
case Res of
|
||||
ok ->
|
||||
{[Q | R0], F0};
|
||||
fail ->
|
||||
{R0, [Q | F0]}
|
||||
end
|
||||
end, {[], []}, Queues).
|
||||
|
||||
-spec stop(rabbit_types:vhost()) -> ok.
|
||||
stop(VHost) ->
|
||||
_ = [begin
|
||||
Pid = amqqueue:get_pid(Q),
|
||||
|
@ -492,7 +530,6 @@ stop(VHost) ->
|
|||
boolean(), boolean(),
|
||||
rabbit_types:username()) ->
|
||||
{ok, QLen :: non_neg_integer()}.
|
||||
|
||||
delete(Q, true, _IfEmpty, _ActingUser) when ?amqqueue_is_quorum(Q) ->
|
||||
rabbit_misc:protocol_error(
|
||||
not_implemented,
|
||||
|
@ -503,8 +540,7 @@ delete(Q, _IfUnused, true, _ActingUser) when ?amqqueue_is_quorum(Q) ->
|
|||
not_implemented,
|
||||
"cannot delete ~s. queue.delete operations with if-empty flag set are not supported by quorum queues",
|
||||
[rabbit_misc:rs(amqqueue:get_name(Q))]);
|
||||
delete(Q,
|
||||
_IfUnused, _IfEmpty, ActingUser) when ?amqqueue_is_quorum(Q) ->
|
||||
delete(Q, _IfUnused, _IfEmpty, ActingUser) when ?amqqueue_is_quorum(Q) ->
|
||||
{Name, _} = amqqueue:get_pid(Q),
|
||||
QName = amqqueue:get_name(Q),
|
||||
QNodes = get_nodes(Q),
|
||||
|
@ -547,7 +583,6 @@ delete(Q,
|
|||
end
|
||||
end.
|
||||
|
||||
|
||||
force_delete_queue(Servers) ->
|
||||
[begin
|
||||
case catch(ra:force_delete_server(S)) of
|
||||
|
@ -573,32 +608,22 @@ delete_immediately(Resource, {_Name, _} = QPid) ->
|
|||
rabbit_core_metrics:queue_deleted(Resource),
|
||||
ok.
|
||||
|
||||
-spec ack(rabbit_types:ctag(), [msg_id()], rabbit_fifo_client:state()) ->
|
||||
{'ok', rabbit_fifo_client:state()}.
|
||||
|
||||
ack(CTag, MsgIds, QState) ->
|
||||
rabbit_fifo_client:settle(quorum_ctag(CTag), MsgIds, QState).
|
||||
|
||||
-spec reject(Confirm :: boolean(), rabbit_types:ctag(), [msg_id()], rabbit_fifo_client:state()) ->
|
||||
{'ok', rabbit_fifo_client:state()}.
|
||||
|
||||
reject(true, CTag, MsgIds, QState) ->
|
||||
settle(complete, CTag, MsgIds, QState) ->
|
||||
rabbit_fifo_client:settle(quorum_ctag(CTag), MsgIds, QState);
|
||||
settle(requeue, CTag, MsgIds, QState) ->
|
||||
rabbit_fifo_client:return(quorum_ctag(CTag), MsgIds, QState);
|
||||
reject(false, CTag, MsgIds, QState) ->
|
||||
settle(discard, CTag, MsgIds, QState) ->
|
||||
rabbit_fifo_client:discard(quorum_ctag(CTag), MsgIds, QState).
|
||||
|
||||
credit(CTag, Credit, Drain, QState) ->
|
||||
rabbit_fifo_client:credit(quorum_ctag(CTag), Credit, Drain, QState).
|
||||
|
||||
-spec basic_get(amqqueue: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()} |
|
||||
{error, timeout | term()}.
|
||||
|
||||
basic_get(Q, NoAck, CTag0, QState0) when ?amqqueue_is_quorum(Q) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
Id = amqqueue:get_pid(Q),
|
||||
-spec dequeue(NoAck :: boolean(), pid(),
|
||||
rabbit_types:ctag(), rabbit_fifo_client:state()) ->
|
||||
{empty, rabbit_fifo_client:state()} |
|
||||
{ok, QLen :: non_neg_integer(), qmsg(), rabbit_fifo_client:state()} |
|
||||
{error, term()}.
|
||||
dequeue(NoAck, _LimiterPid, CTag0, QState0) ->
|
||||
CTag = quorum_ctag(CTag0),
|
||||
Settlement = case NoAck of
|
||||
true ->
|
||||
|
@ -606,39 +631,25 @@ basic_get(Q, NoAck, CTag0, QState0) when ?amqqueue_is_quorum(Q) ->
|
|||
false ->
|
||||
unsettled
|
||||
end,
|
||||
case rabbit_fifo_client:dequeue(CTag, Settlement, QState0) of
|
||||
{ok, empty, QState} ->
|
||||
{ok, empty, QState};
|
||||
{ok, {{MsgId, {MsgHeader, Msg0}}, MsgsReady}, QState} ->
|
||||
Count = case MsgHeader of
|
||||
#{delivery_count := C} -> C;
|
||||
_ -> 0
|
||||
end,
|
||||
IsDelivered = Count > 0,
|
||||
Msg = rabbit_basic:add_header(<<"x-delivery-count">>, long, Count, Msg0),
|
||||
{ok, MsgsReady, {QName, Id, MsgId, IsDelivered, Msg}, QState};
|
||||
{error, unsupported} ->
|
||||
rabbit_misc:protocol_error(
|
||||
resource_locked,
|
||||
"cannot obtain access to locked ~s. basic.get operations are not supported by quorum queues with single active consumer",
|
||||
[rabbit_misc:rs(QName)]);
|
||||
{error, _} = Err ->
|
||||
Err;
|
||||
{timeout, _} ->
|
||||
{error, timeout}
|
||||
end.
|
||||
rabbit_fifo_client:dequeue(CTag, Settlement, QState0).
|
||||
|
||||
-spec basic_consume(amqqueue:amqqueue(), NoAck :: boolean(), ChPid :: pid(),
|
||||
ConsumerPrefetchCount :: non_neg_integer(),
|
||||
rabbit_types:ctag(), ExclusiveConsume :: boolean(),
|
||||
Args :: rabbit_framing:amqp_table(), ActingUser :: binary(),
|
||||
any(), rabbit_fifo_client:state()) ->
|
||||
{'ok', rabbit_fifo_client:state()} |
|
||||
{error, timeout | term()}.
|
||||
|
||||
basic_consume(Q, NoAck, ChPid,
|
||||
ConsumerPrefetchCount, ConsumerTag0, ExclusiveConsume, Args,
|
||||
ActingUser, OkMsg, QState0) when ?amqqueue_is_quorum(Q) ->
|
||||
-spec consume(amqqueue:amqqueue(),
|
||||
rabbit_queue_type:consume_spec(),
|
||||
rabbit_fifo_client:state()) ->
|
||||
{ok, rabbit_fifo_client:state(), rabbit_queue_type:actions()} |
|
||||
{error, global_qos_not_supported_for_queue_type}.
|
||||
consume(Q, #{limiter_active := true}, _State)
|
||||
when ?amqqueue_is_quorum(Q) ->
|
||||
{error, global_qos_not_supported_for_queue_type};
|
||||
consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) ->
|
||||
#{no_ack := NoAck,
|
||||
channel_pid := ChPid,
|
||||
prefetch_count := ConsumerPrefetchCount,
|
||||
consumer_tag := ConsumerTag0,
|
||||
exclusive_consume := ExclusiveConsume,
|
||||
args := Args,
|
||||
ok_msg := OkMsg,
|
||||
acting_user := ActingUser} = Spec,
|
||||
%% TODO: validate consumer arguments
|
||||
%% currently quorum queues do not support any arguments
|
||||
QName = amqqueue:get_name(Q),
|
||||
|
@ -681,19 +692,19 @@ basic_consume(Q, NoAck, ChPid,
|
|||
emit_consumer_created(ChPid, ConsumerTag, ExclusiveConsume,
|
||||
AckRequired, QName, Prefetch,
|
||||
Args, none, ActingUser),
|
||||
{ok, QState};
|
||||
{ok, QState, []};
|
||||
{error, Error} ->
|
||||
Error;
|
||||
{timeout, _} ->
|
||||
{error, timeout}
|
||||
end.
|
||||
|
||||
-spec basic_cancel(rabbit_types:ctag(), ChPid :: pid(), 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()}.
|
||||
|
||||
basic_cancel(ConsumerTag, ChPid, OkMsg, QState0) ->
|
||||
maybe_send_reply(ChPid, OkMsg),
|
||||
rabbit_fifo_client:cancel_checkout(quorum_ctag(ConsumerTag), QState0).
|
||||
cancel(_Q, ConsumerTag, OkMsg, _ActingUser, State) ->
|
||||
maybe_send_reply(self(), OkMsg),
|
||||
rabbit_fifo_client:cancel_checkout(quorum_ctag(ConsumerTag), State).
|
||||
|
||||
emit_consumer_created(ChPid, CTag, Exclusive, AckRequired, QName, PrefetchCount, Args, Ref, ActingUser) ->
|
||||
rabbit_event:notify(consumer_created,
|
||||
|
@ -746,14 +757,25 @@ deliver(true, Delivery, QState0) ->
|
|||
{ok, State}
|
||||
end.
|
||||
|
||||
-spec info(amqqueue:amqqueue()) -> rabbit_types:infos().
|
||||
deliver(QSs, #delivery{confirm = Confirm} = Delivery) ->
|
||||
lists:foldl(
|
||||
fun({Q, stateless}, {Qs, Actions}) ->
|
||||
QRef = amqqueue:get_pid(Q),
|
||||
ok = rabbit_fifo_client:untracked_enqueue(
|
||||
[QRef], Delivery#delivery.message),
|
||||
{Qs, Actions};
|
||||
({Q, S0}, {Qs, Actions}) ->
|
||||
{_, S} = deliver(Confirm, Delivery, S0),
|
||||
{[{Q, S} | Qs], Actions}
|
||||
end, {[], []}, QSs).
|
||||
|
||||
|
||||
state_info(S) ->
|
||||
#{pending_raft_commands => rabbit_fifo_client:pending_size(S)}.
|
||||
|
||||
|
||||
info(Q) ->
|
||||
info(Q, [name, durable, auto_delete, arguments, pid, state, messages,
|
||||
messages_ready, messages_unacknowledged]).
|
||||
|
||||
-spec infos(rabbit_types:r('queue')) -> rabbit_types:infos().
|
||||
|
||||
infos(QName) ->
|
||||
infos(QName, ?STATISTICS_KEYS).
|
||||
|
||||
|
@ -765,7 +787,6 @@ infos(QName, Keys) ->
|
|||
[]
|
||||
end.
|
||||
|
||||
-spec info(amqqueue:amqqueue(), rabbit_types:info_keys()) -> rabbit_types:infos().
|
||||
|
||||
info(Q, Items) ->
|
||||
lists:foldr(fun(totals, Acc) ->
|
||||
|
@ -776,8 +797,8 @@ info(Q, Items) ->
|
|||
[{Item, i(Item, Q)} | Acc]
|
||||
end, [], Items).
|
||||
|
||||
-spec stat(amqqueue:amqqueue()) -> {'ok', non_neg_integer(), non_neg_integer()}.
|
||||
|
||||
-spec stat(amqqueue:amqqueue()) ->
|
||||
{'ok', non_neg_integer(), non_neg_integer()}.
|
||||
stat(Q) when ?is_amqqueue(Q) ->
|
||||
%% same short default timeout as in rabbit_fifo_client:stat/1
|
||||
stat(Q, 250).
|
||||
|
@ -798,7 +819,10 @@ stat(Q, Timeout) when ?is_amqqueue(Q) ->
|
|||
{ok, 0, 0}
|
||||
end.
|
||||
|
||||
purge(Node) ->
|
||||
-spec purge(amqqueue:amqqueue()) ->
|
||||
{ok, non_neg_integer()}.
|
||||
purge(Q) when ?is_amqqueue(Q) ->
|
||||
Node = amqqueue:get_pid(Q),
|
||||
rabbit_fifo_client:purge(Node).
|
||||
|
||||
requeue(ConsumerTag, MsgIds, QState) ->
|
||||
|
@ -811,9 +835,11 @@ cleanup_data_dir() ->
|
|||
end
|
||||
|| Q <- rabbit_amqqueue:list_by_type(?MODULE),
|
||||
lists:member(node(), get_nodes(Q))],
|
||||
NoQQClusters = rabbit_ra_registry:list_not_quorum_clusters(),
|
||||
Registered = ra_directory:list_registered(),
|
||||
Running = Names ++ NoQQClusters,
|
||||
_ = [maybe_delete_data_dir(UId) || {Name, UId} <- Registered,
|
||||
not lists:member(Name, Names)],
|
||||
not lists:member(Name, Running)],
|
||||
ok.
|
||||
|
||||
maybe_delete_data_dir(UId) ->
|
||||
|
@ -827,9 +853,10 @@ maybe_delete_data_dir(UId) ->
|
|||
ok
|
||||
end.
|
||||
|
||||
policy_changed(QName, Server) ->
|
||||
{ok, Q} = rabbit_amqqueue:lookup(QName),
|
||||
rabbit_fifo_client:update_machine_state(Server, ra_machine_config(Q)).
|
||||
policy_changed(Q) ->
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
_ = rabbit_fifo_client:update_machine_state(QPid, ra_machine_config(Q)),
|
||||
ok.
|
||||
|
||||
-spec cluster_state(Name :: atom()) -> 'down' | 'recovering' | 'running'.
|
||||
|
||||
|
@ -848,7 +875,7 @@ cluster_state(Name) ->
|
|||
status(Vhost, QueueName) ->
|
||||
%% Handle not found queues
|
||||
QName = #resource{virtual_host = Vhost, name = QueueName, kind = queue},
|
||||
RName = qname_to_rname(QName),
|
||||
RName = qname_to_internal_name(QName),
|
||||
case rabbit_amqqueue:lookup(QName) of
|
||||
{ok, Q} when ?amqqueue_is_classic(Q) ->
|
||||
{error, classic_queue_not_supported};
|
||||
|
@ -1147,16 +1174,6 @@ init_dlx(DLX, Q) when ?is_amqqueue(Q) ->
|
|||
|
||||
res_arg(_PolVal, ArgVal) -> ArgVal.
|
||||
|
||||
args_policy_lookup(Name, Resolve, Q) when ?is_amqqueue(Q) ->
|
||||
Args = amqqueue:get_arguments(Q),
|
||||
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(X, RK, QName, ReasonMsgs) ->
|
||||
|
@ -1168,12 +1185,6 @@ dead_letter_publish(X, RK, QName, ReasonMsgs) ->
|
|||
ok
|
||||
end.
|
||||
|
||||
%% TODO escape hack
|
||||
qname_to_rname(#resource{virtual_host = <<"/">>, name = Name}) ->
|
||||
erlang:binary_to_atom(<<"%2F_", Name/binary>>, utf8);
|
||||
qname_to_rname(#resource{virtual_host = VHost, name = Name}) ->
|
||||
erlang:binary_to_atom(<<VHost/binary, "_", Name/binary>>, utf8).
|
||||
|
||||
find_quorum_queues(VHost) ->
|
||||
Node = node(),
|
||||
mnesia:async_dirty(
|
||||
|
@ -1413,52 +1424,7 @@ maybe_send_reply(ChPid, Msg) -> ok = rabbit_channel:send_command(ChPid, Msg).
|
|||
check_invalid_arguments(QueueName, Args) ->
|
||||
Keys = [<<"x-message-ttl">>,
|
||||
<<"x-max-priority">>, <<"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],
|
||||
|
||||
case rabbit_misc:table_lookup(Args, <<"x-overflow">>) of
|
||||
undefined -> ok;
|
||||
{_, <<"reject-publish-dlx">>} ->
|
||||
rabbit_misc:protocol_error(
|
||||
precondition_failed,
|
||||
"invalid arg 'x-overflow' with value 'reject-publish-dlx' for ~s",
|
||||
[rabbit_misc:rs(QueueName)]);
|
||||
_ ->
|
||||
ok
|
||||
end,
|
||||
ok.
|
||||
|
||||
check_auto_delete(Q) when ?amqqueue_is_auto_delete(Q) ->
|
||||
Name = amqqueue:get_name(Q),
|
||||
rabbit_misc:protocol_error(
|
||||
precondition_failed,
|
||||
"invalid property 'auto-delete' for ~s",
|
||||
[rabbit_misc:rs(Name)]);
|
||||
check_auto_delete(_) ->
|
||||
ok.
|
||||
|
||||
check_exclusive(Q) when ?amqqueue_exclusive_owner_is(Q, none) ->
|
||||
ok;
|
||||
check_exclusive(Q) when ?is_amqqueue(Q) ->
|
||||
Name = amqqueue:get_name(Q),
|
||||
rabbit_misc:protocol_error(
|
||||
precondition_failed,
|
||||
"invalid property 'exclusive-owner' for ~s",
|
||||
[rabbit_misc:rs(Name)]).
|
||||
|
||||
check_non_durable(Q) when ?amqqueue_is_durable(Q) ->
|
||||
ok;
|
||||
check_non_durable(Q) when not ?amqqueue_is_durable(Q) ->
|
||||
Name = amqqueue:get_name(Q),
|
||||
rabbit_misc:protocol_error(
|
||||
precondition_failed,
|
||||
"invalid property 'non-durable' for ~s",
|
||||
[rabbit_misc:rs(Name)]).
|
||||
rabbit_queue_type_util:check_invalid_arguments(QueueName, Args, Keys).
|
||||
|
||||
queue_name(RaFifoState) ->
|
||||
rabbit_fifo_client:cluster_name(RaFifoState).
|
||||
|
@ -1492,12 +1458,16 @@ members(Q) when ?amqqueue_is_quorum(Q) ->
|
|||
Nodes = lists:delete(LeaderNode, get_nodes(Q)),
|
||||
[{RaName, N} || N <- [LeaderNode | Nodes]].
|
||||
|
||||
format_ra_event(ServerId, Evt, QRef) ->
|
||||
{'$gen_cast', {queue_event, QRef, {ServerId, Evt}}}.
|
||||
|
||||
make_ra_conf(Q, ServerId, TickTimeout) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
RaMachine = ra_machine(Q),
|
||||
[{ClusterName, _} | _] = Members = members(Q),
|
||||
UId = ra:new_uid(ra_lib:to_binary(ClusterName)),
|
||||
FName = rabbit_misc:rs(QName),
|
||||
Formatter = {?MODULE, format_ra_event, [QName]},
|
||||
#{cluster_name => ClusterName,
|
||||
id => ServerId,
|
||||
uid => UId,
|
||||
|
@ -1506,7 +1476,8 @@ make_ra_conf(Q, ServerId, TickTimeout) ->
|
|||
initial_members => Members,
|
||||
log_init_args => #{uid => UId},
|
||||
tick_timeout => TickTimeout,
|
||||
machine => RaMachine}.
|
||||
machine => RaMachine,
|
||||
ra_event_formatter => Formatter}.
|
||||
|
||||
get_nodes(Q) when ?is_amqqueue(Q) ->
|
||||
#{nodes := Nodes} = amqqueue:get_type_state(Q),
|
||||
|
|
|
@ -0,0 +1,25 @@
|
|||
%% 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 https://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-2020 VMware, Inc. or its affiliates. All rights reserved.
|
||||
%%
|
||||
|
||||
-module(rabbit_ra_registry).
|
||||
|
||||
-export([list_not_quorum_clusters/0]).
|
||||
|
||||
%% Not all ra clusters are quorum queues. We need to keep a list of these so we don't
|
||||
%% take them into account in operations such as memory calculation and data cleanup.
|
||||
%% Hardcoded atm
|
||||
list_not_quorum_clusters() ->
|
||||
[rabbit_stream_coordinator].
|
|
@ -0,0 +1,906 @@
|
|||
%% 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 https://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.
|
||||
%%
|
||||
%% Copyright (c) 2012-2020 VMware, Inc. or its affiliates. All rights reserved.
|
||||
%%
|
||||
-module(rabbit_stream_coordinator).
|
||||
|
||||
-behaviour(ra_machine).
|
||||
|
||||
-export([start/0]).
|
||||
-export([format_ra_event/2]).
|
||||
|
||||
-export([init/1,
|
||||
apply/3,
|
||||
state_enter/2,
|
||||
init_aux/1,
|
||||
handle_aux/6,
|
||||
tick/2]).
|
||||
|
||||
-export([recover/0,
|
||||
start_cluster/1,
|
||||
delete_cluster/2,
|
||||
add_replica/2,
|
||||
delete_replica/2]).
|
||||
|
||||
-export([policy_changed/1]).
|
||||
|
||||
-export([phase_repair_mnesia/2,
|
||||
phase_start_cluster/1,
|
||||
phase_delete_cluster/2,
|
||||
phase_check_quorum/1,
|
||||
phase_start_new_leader/1,
|
||||
phase_stop_replicas/1,
|
||||
phase_start_replica/3,
|
||||
phase_delete_replica/2]).
|
||||
|
||||
-export([log_overview/1]).
|
||||
|
||||
-define(STREAM_COORDINATOR_STARTUP, {stream_coordinator_startup, self()}).
|
||||
-define(TICK_TIMEOUT, 60000).
|
||||
-define(RESTART_TIMEOUT, 1000).
|
||||
-define(PHASE_RETRY_TIMEOUT, 10000).
|
||||
-define(CMD_TIMEOUT, 30000).
|
||||
|
||||
-record(?MODULE, {streams, monitors}).
|
||||
|
||||
start() ->
|
||||
Nodes = rabbit_mnesia:cluster_nodes(all),
|
||||
ServerId = {?MODULE, node()},
|
||||
case ra:restart_server(ServerId) of
|
||||
{error, Reason} when Reason == not_started orelse
|
||||
Reason == name_not_registered ->
|
||||
case ra:start_server(make_ra_conf(node(), Nodes)) of
|
||||
ok ->
|
||||
global:set_lock(?STREAM_COORDINATOR_STARTUP),
|
||||
case find_members(Nodes) of
|
||||
[] ->
|
||||
%% We're the first (and maybe only) one
|
||||
ra:trigger_election(ServerId);
|
||||
Members ->
|
||||
%% What to do if we get a timeout?
|
||||
{ok, _, _} = ra:add_member(Members, ServerId, 30000)
|
||||
end,
|
||||
global:del_lock(?STREAM_COORDINATOR_STARTUP),
|
||||
_ = ra:members(ServerId),
|
||||
ok;
|
||||
Error ->
|
||||
exit(Error)
|
||||
end;
|
||||
ok ->
|
||||
ok;
|
||||
Error ->
|
||||
exit(Error)
|
||||
end.
|
||||
|
||||
find_members([]) ->
|
||||
[];
|
||||
find_members([Node | Nodes]) ->
|
||||
case ra:members({?MODULE, Node}) of
|
||||
{_, Members, _} ->
|
||||
Members;
|
||||
{error, noproc} ->
|
||||
find_members(Nodes);
|
||||
{timeout, _} ->
|
||||
%% not sure what to do here
|
||||
find_members(Nodes)
|
||||
end.
|
||||
|
||||
recover() ->
|
||||
ra:restart_server({?MODULE, node()}).
|
||||
|
||||
start_cluster(Q) ->
|
||||
process_command({start_cluster, #{queue => Q}}).
|
||||
|
||||
delete_cluster(StreamId, ActingUser) ->
|
||||
process_command({delete_cluster, #{stream_id => StreamId, acting_user => ActingUser}}).
|
||||
|
||||
add_replica(StreamId, Node) ->
|
||||
process_command({start_replica, #{stream_id => StreamId, node => Node,
|
||||
retries => 1}}).
|
||||
|
||||
policy_changed(StreamId) ->
|
||||
process_command({policy_changed, #{stream_id => StreamId}}).
|
||||
|
||||
delete_replica(StreamId, Node) ->
|
||||
process_command({delete_replica, #{stream_id => StreamId, node => Node}}).
|
||||
|
||||
process_command(Cmd) ->
|
||||
Servers = ensure_coordinator_started(),
|
||||
process_command(Servers, Cmd).
|
||||
|
||||
process_command([], _Cmd) ->
|
||||
{error, coordinator_unavailable};
|
||||
process_command([Server | Servers], {CmdName, _} = Cmd) ->
|
||||
case ra:process_command(Server, Cmd, ?CMD_TIMEOUT) of
|
||||
{timeout, _} ->
|
||||
rabbit_log:warning("Coordinator timeout on server ~p when processing command ~p",
|
||||
[Server, CmdName]),
|
||||
process_command(Servers, Cmd);
|
||||
{error, noproc} ->
|
||||
process_command(Servers, Cmd);
|
||||
Reply ->
|
||||
Reply
|
||||
end.
|
||||
|
||||
ensure_coordinator_started() ->
|
||||
Local = {?MODULE, node()},
|
||||
AllNodes = all_nodes(),
|
||||
case ra:restart_server(Local) of
|
||||
{error, Reason} when Reason == not_started orelse
|
||||
Reason == name_not_registered ->
|
||||
OtherNodes = all_nodes() -- [Local],
|
||||
%% We can't use find_members/0 here as a process that timeouts means the cluster is up
|
||||
case lists:filter(fun(N) -> global:whereis_name(N) =/= undefined end, OtherNodes) of
|
||||
[] ->
|
||||
start_coordinator_cluster();
|
||||
_ ->
|
||||
OtherNodes
|
||||
end;
|
||||
ok ->
|
||||
AllNodes;
|
||||
{error, {already_started, _}} ->
|
||||
AllNodes;
|
||||
_ ->
|
||||
AllNodes
|
||||
end.
|
||||
|
||||
start_coordinator_cluster() ->
|
||||
Nodes = rabbit_mnesia:cluster_nodes(running),
|
||||
case ra:start_cluster([make_ra_conf(Node, Nodes) || Node <- Nodes]) of
|
||||
{ok, Started, _} ->
|
||||
Started;
|
||||
{error, cluster_not_formed} ->
|
||||
rabbit_log:warning("Stream coordinator cluster not formed", []),
|
||||
[]
|
||||
end.
|
||||
|
||||
all_nodes() ->
|
||||
Nodes = rabbit_mnesia:cluster_nodes(running) -- [node()],
|
||||
[{?MODULE, Node} || Node <- [node() | Nodes]].
|
||||
|
||||
init(_Conf) ->
|
||||
#?MODULE{streams = #{},
|
||||
monitors = #{}}.
|
||||
|
||||
apply(#{from := From}, {policy_changed, #{stream_id := StreamId}} = Cmd,
|
||||
#?MODULE{streams = Streams0} = State) ->
|
||||
case maps:get(StreamId, Streams0, undefined) of
|
||||
undefined ->
|
||||
{State, ok, []};
|
||||
#{conf := Conf,
|
||||
state := running} ->
|
||||
case rabbit_stream_queue:update_stream_conf(Conf) of
|
||||
Conf ->
|
||||
%% No changes, ensure we only trigger an election if it's a must
|
||||
{State, ok, []};
|
||||
_ ->
|
||||
{State, ok, [{mod_call, osiris_writer, stop, [Conf]}]}
|
||||
end;
|
||||
SState0 ->
|
||||
Streams = maps:put(StreamId, add_pending_cmd(From, Cmd, SState0), Streams0),
|
||||
{State#?MODULE{streams = Streams}, '$ra_no_reply', []}
|
||||
|
||||
end;
|
||||
apply(#{from := From}, {start_cluster, #{queue := Q}}, #?MODULE{streams = Streams} = State) ->
|
||||
#{name := StreamId} = Conf = amqqueue:get_type_state(Q),
|
||||
case maps:is_key(StreamId, Streams) of
|
||||
true ->
|
||||
{State, '$ra_no_reply', wrap_reply(From, {error, already_started})};
|
||||
false ->
|
||||
Phase = phase_start_cluster,
|
||||
PhaseArgs = [Q],
|
||||
SState = #{state => start_cluster,
|
||||
phase => Phase,
|
||||
phase_args => PhaseArgs,
|
||||
conf => Conf,
|
||||
reply_to => From,
|
||||
pending_cmds => [],
|
||||
pending_replicas => []},
|
||||
rabbit_log:debug("rabbit_stream_coordinator: ~p entering phase_start_cluster", [StreamId]),
|
||||
{State#?MODULE{streams = maps:put(StreamId, SState, Streams)}, '$ra_no_reply',
|
||||
[{aux, {phase, StreamId, Phase, PhaseArgs}}]}
|
||||
end;
|
||||
apply(_Meta, {start_cluster_reply, Q}, #?MODULE{streams = Streams,
|
||||
monitors = Monitors0} = State) ->
|
||||
#{name := StreamId,
|
||||
leader_pid := LeaderPid,
|
||||
replica_pids := ReplicaPids} = Conf = amqqueue:get_type_state(Q),
|
||||
SState0 = maps:get(StreamId, Streams),
|
||||
Phase = phase_repair_mnesia,
|
||||
PhaseArgs = [new, Q],
|
||||
SState = SState0#{conf => Conf,
|
||||
phase => Phase,
|
||||
phase_args => PhaseArgs},
|
||||
Monitors = lists:foldl(fun(Pid, M) ->
|
||||
maps:put(Pid, {StreamId, follower}, M)
|
||||
end, maps:put(LeaderPid, {StreamId, leader}, Monitors0), ReplicaPids),
|
||||
MonitorActions = [{monitor, process, Pid} || Pid <- ReplicaPids ++ [LeaderPid]],
|
||||
rabbit_log:debug("rabbit_stream_coordinator: ~p entering ~p "
|
||||
"after start_cluster_reply", [StreamId, Phase]),
|
||||
{State#?MODULE{streams = maps:put(StreamId, SState, Streams),
|
||||
monitors = Monitors}, ok,
|
||||
MonitorActions ++ [{aux, {phase, StreamId, Phase, PhaseArgs}}]};
|
||||
apply(_Meta, {start_replica_failed, StreamId, Node, Retries, Reply},
|
||||
#?MODULE{streams = Streams0} = State) ->
|
||||
rabbit_log:debug("rabbit_stream_coordinator: ~p start replica failed", [StreamId]),
|
||||
case maps:get(StreamId, Streams0, undefined) of
|
||||
undefined ->
|
||||
{State, {error, not_found}, []};
|
||||
#{pending_replicas := Pending,
|
||||
reply_to := From} = SState ->
|
||||
Streams = Streams0#{StreamId => clear_stream_state(SState#{pending_replicas =>
|
||||
add_unique(Node, Pending)})},
|
||||
reply_and_run_pending(
|
||||
From, StreamId, ok, Reply,
|
||||
[{timer, {pipeline,
|
||||
[{start_replica, #{stream_id => StreamId,
|
||||
node => Node,
|
||||
from => undefined,
|
||||
retries => Retries + 1}}]},
|
||||
?RESTART_TIMEOUT * Retries}],
|
||||
State#?MODULE{streams = Streams})
|
||||
end;
|
||||
apply(_Meta, {phase_finished, StreamId, Reply}, #?MODULE{streams = Streams0} = State) ->
|
||||
rabbit_log:debug("rabbit_stream_coordinator: ~p phase finished", [StreamId]),
|
||||
case maps:get(StreamId, Streams0, undefined) of
|
||||
undefined ->
|
||||
{State, {error, not_found}, []};
|
||||
#{reply_to := From} = SState ->
|
||||
Streams = Streams0#{StreamId => clear_stream_state(SState)},
|
||||
reply_and_run_pending(From, StreamId, ok, Reply, [], State#?MODULE{streams = Streams})
|
||||
end;
|
||||
apply(#{from := From}, {start_replica, #{stream_id := StreamId, node := Node,
|
||||
retries := Retries}} = Cmd,
|
||||
#?MODULE{streams = Streams0} = State) ->
|
||||
case maps:get(StreamId, Streams0, undefined) of
|
||||
undefined ->
|
||||
case From of
|
||||
undefined ->
|
||||
{State, ok, []};
|
||||
_ ->
|
||||
{State, '$ra_no_reply', wrap_reply(From, {error, not_found})}
|
||||
end;
|
||||
#{conf := Conf,
|
||||
state := running} = SState0 ->
|
||||
Phase = phase_start_replica,
|
||||
PhaseArgs = [Node, Conf, Retries],
|
||||
SState = update_stream_state(From, start_replica, Phase, PhaseArgs, SState0),
|
||||
rabbit_log:debug("rabbit_stream_coordinator: ~p entering ~p on node ~p",
|
||||
[StreamId, Phase, Node]),
|
||||
{State#?MODULE{streams = Streams0#{StreamId => SState}}, '$ra_no_reply',
|
||||
[{aux, {phase, StreamId, Phase, PhaseArgs}}]};
|
||||
SState0 ->
|
||||
Streams = maps:put(StreamId, add_pending_cmd(From, Cmd, SState0), Streams0),
|
||||
{State#?MODULE{streams = Streams}, '$ra_no_reply', []}
|
||||
end;
|
||||
apply(_Meta, {start_replica_reply, StreamId, Pid},
|
||||
#?MODULE{streams = Streams, monitors = Monitors0} = State) ->
|
||||
case maps:get(StreamId, Streams, undefined) of
|
||||
undefined ->
|
||||
{State, {error, not_found}, []};
|
||||
#{conf := Conf0} = SState0 ->
|
||||
#{replica_nodes := Replicas0,
|
||||
replica_pids := ReplicaPids0} = Conf0,
|
||||
{ReplicaPids, MaybePid} = delete_replica_pid(node(Pid), ReplicaPids0),
|
||||
Conf = Conf0#{replica_pids => [Pid | ReplicaPids],
|
||||
replica_nodes => add_unique(node(Pid), Replicas0)},
|
||||
Phase = phase_repair_mnesia,
|
||||
PhaseArgs = [update, Conf],
|
||||
rabbit_log:debug("rabbit_stream_coordinator: ~p entering ~p after start replica", [StreamId, Phase]),
|
||||
#{pending_replicas := Pending} = SState0 = maps:get(StreamId, Streams),
|
||||
SState = SState0#{conf => Conf,
|
||||
phase => Phase,
|
||||
phase_args => PhaseArgs,
|
||||
pending_replicas => lists:delete(node(Pid), Pending)},
|
||||
Monitors1 = Monitors0#{Pid => {StreamId, follower}},
|
||||
Monitors = case MaybePid of
|
||||
[P] -> maps:remove(P, Monitors1);
|
||||
_ -> Monitors1
|
||||
end,
|
||||
{State#?MODULE{streams = Streams#{StreamId => SState},
|
||||
monitors = Monitors}, ok,
|
||||
[{monitor, process, Pid}, {aux, {phase, StreamId, Phase, PhaseArgs}}]}
|
||||
end;
|
||||
apply(#{from := From}, {delete_replica, #{stream_id := StreamId, node := Node}} = Cmd,
|
||||
#?MODULE{streams = Streams0,
|
||||
monitors = Monitors0} = State) ->
|
||||
case maps:get(StreamId, Streams0, undefined) of
|
||||
undefined ->
|
||||
{State, '$ra_no_reply', wrap_reply(From, {error, not_found})};
|
||||
#{conf := Conf0,
|
||||
state := running,
|
||||
pending_replicas := Pending0} = SState0 ->
|
||||
Replicas0 = maps:get(replica_nodes, Conf0),
|
||||
ReplicaPids0 = maps:get(replica_pids, Conf0),
|
||||
case lists:member(Node, Replicas0) of
|
||||
false ->
|
||||
reply_and_run_pending(From, StreamId, '$ra_no_reply', ok, [], State);
|
||||
true ->
|
||||
[Pid] = lists:filter(fun(P) -> node(P) == Node end, ReplicaPids0),
|
||||
ReplicaPids = lists:delete(Pid, ReplicaPids0),
|
||||
Replicas = lists:delete(Node, Replicas0),
|
||||
Pending = lists:delete(Node, Pending0),
|
||||
Conf = Conf0#{replica_pids => ReplicaPids,
|
||||
replica_nodes => Replicas},
|
||||
Phase = phase_delete_replica,
|
||||
PhaseArgs = [Node, Conf],
|
||||
SState = update_stream_state(From, delete_replica,
|
||||
Phase, PhaseArgs,
|
||||
SState0#{conf => Conf0,
|
||||
pending_replicas => Pending}),
|
||||
Monitors = maps:remove(Pid, Monitors0),
|
||||
rabbit_log:debug("rabbit_stream_coordinator: ~p entering ~p on node ~p", [StreamId, Phase, Node]),
|
||||
{State#?MODULE{monitors = Monitors,
|
||||
streams = Streams0#{StreamId => SState}},
|
||||
'$ra_no_reply',
|
||||
[{demonitor, process, Pid},
|
||||
{aux, {phase, StreamId, Phase, PhaseArgs}}]}
|
||||
end;
|
||||
SState0 ->
|
||||
Streams = maps:put(StreamId, add_pending_cmd(From, Cmd, SState0), Streams0),
|
||||
{State#?MODULE{streams = Streams}, '$ra_no_reply', []}
|
||||
end;
|
||||
apply(#{from := From}, {delete_cluster, #{stream_id := StreamId,
|
||||
acting_user := ActingUser}} = Cmd,
|
||||
#?MODULE{streams = Streams0, monitors = Monitors0} = State) ->
|
||||
case maps:get(StreamId, Streams0, undefined) of
|
||||
undefined ->
|
||||
{State, '$ra_no_reply', wrap_reply(From, {ok, 0})};
|
||||
#{conf := Conf,
|
||||
state := running} = SState0 ->
|
||||
ReplicaPids = maps:get(replica_pids, Conf),
|
||||
LeaderPid = maps:get(leader_pid, Conf),
|
||||
Monitors = lists:foldl(fun(Pid, M) ->
|
||||
maps:remove(Pid, M)
|
||||
end, Monitors0, ReplicaPids ++ [LeaderPid]),
|
||||
Phase = phase_delete_cluster,
|
||||
PhaseArgs = [Conf, ActingUser],
|
||||
SState = update_stream_state(From, delete_cluster, Phase, PhaseArgs, SState0),
|
||||
Demonitors = [{demonitor, process, Pid} || Pid <- [LeaderPid | ReplicaPids]],
|
||||
rabbit_log:debug("rabbit_stream_coordinator: ~p entering ~p",
|
||||
[StreamId, Phase]),
|
||||
{State#?MODULE{monitors = Monitors,
|
||||
streams = Streams0#{StreamId => SState}}, '$ra_no_reply',
|
||||
Demonitors ++ [{aux, {phase, StreamId, Phase, PhaseArgs}}]};
|
||||
SState0 ->
|
||||
Streams = maps:put(StreamId, add_pending_cmd(From, Cmd, SState0), Streams0),
|
||||
{State#?MODULE{streams = Streams}, '$ra_no_reply', []}
|
||||
end;
|
||||
apply(_Meta, {delete_cluster_reply, StreamId}, #?MODULE{streams = Streams} = State0) ->
|
||||
#{reply_to := From,
|
||||
pending_cmds := Pending} = maps:get(StreamId, Streams),
|
||||
State = State0#?MODULE{streams = maps:remove(StreamId, Streams)},
|
||||
rabbit_log:debug("rabbit_stream_coordinator: ~p finished delete_cluster_reply",
|
||||
[StreamId]),
|
||||
Actions = [{ra, pipeline_command, [{?MODULE, node()}, Cmd]} || Cmd <- Pending],
|
||||
{State, ok, Actions ++ wrap_reply(From, {ok, 0})};
|
||||
apply(_Meta, {down, Pid, _Reason} = Cmd, #?MODULE{streams = Streams,
|
||||
monitors = Monitors0} = State) ->
|
||||
case maps:get(Pid, Monitors0, undefined) of
|
||||
{StreamId, Role} ->
|
||||
Monitors = maps:remove(Pid, Monitors0),
|
||||
case maps:get(StreamId, Streams, undefined) of
|
||||
#{state := delete_cluster} ->
|
||||
{State#?MODULE{monitors = Monitors}, ok, []};
|
||||
undefined ->
|
||||
{State#?MODULE{monitors = Monitors}, ok, []};
|
||||
#{state := running,
|
||||
conf := #{replica_pids := Pids} = Conf0,
|
||||
pending_cmds := Pending0} = SState0 ->
|
||||
case Role of
|
||||
leader ->
|
||||
rabbit_log:info("rabbit_stream_coordinator: ~p leader is down, starting election", [StreamId]),
|
||||
Phase = phase_stop_replicas,
|
||||
PhaseArgs = [Conf0],
|
||||
SState = update_stream_state(undefined, leader_election, Phase, PhaseArgs, SState0),
|
||||
Events = [{demonitor, process, P} || P <- Pids],
|
||||
Monitors1 = lists:foldl(fun(P, M) ->
|
||||
maps:remove(P, M)
|
||||
end, Monitors, Pids),
|
||||
rabbit_log:debug("rabbit_stream_coordinator: ~p entering ~p", [StreamId, Phase]),
|
||||
{State#?MODULE{monitors = Monitors1,
|
||||
streams = Streams#{StreamId => SState}},
|
||||
ok, Events ++ [{aux, {phase, StreamId, Phase, PhaseArgs}}]};
|
||||
follower ->
|
||||
case rabbit_misc:is_process_alive(maps:get(leader_pid, Conf0)) of
|
||||
true ->
|
||||
Phase = phase_start_replica,
|
||||
PhaseArgs = [node(Pid), Conf0, 1],
|
||||
SState = update_stream_state(undefined,
|
||||
replica_restart,
|
||||
Phase, PhaseArgs,
|
||||
SState0),
|
||||
rabbit_log:debug("rabbit_stream_coordinator: ~p replica on node ~p is down, entering ~p", [StreamId, node(Pid), Phase]),
|
||||
{State#?MODULE{monitors = Monitors,
|
||||
streams = Streams#{StreamId => SState}},
|
||||
ok, [{aux, {phase, StreamId, Phase, PhaseArgs}}]};
|
||||
false ->
|
||||
SState = SState0#{pending_cmds => Pending0 ++ [Cmd]},
|
||||
reply_and_run_pending(undefined, StreamId, ok, ok, [], State#?MODULE{streams = Streams#{StreamId => SState}})
|
||||
end
|
||||
end;
|
||||
#{pending_cmds := Pending0} = SState0 ->
|
||||
SState = SState0#{pending_cmds => Pending0 ++ [Cmd]},
|
||||
{State#?MODULE{streams = Streams#{StreamId => SState}}, ok, []}
|
||||
end;
|
||||
undefined ->
|
||||
{State, ok, []}
|
||||
end;
|
||||
apply(_Meta, {start_leader_election, StreamId, NewEpoch, Offsets},
|
||||
#?MODULE{streams = Streams} = State) ->
|
||||
#{conf := Conf0} = SState0 = maps:get(StreamId, Streams),
|
||||
#{leader_node := Leader,
|
||||
replica_nodes := Replicas,
|
||||
replica_pids := ReplicaPids0} = Conf0,
|
||||
NewLeader = find_max_offset(Offsets),
|
||||
rabbit_log:info("rabbit_stream_coordinator: ~p starting new leader on node ~p",
|
||||
[StreamId, NewLeader]),
|
||||
{ReplicaPids, _} = delete_replica_pid(NewLeader, ReplicaPids0),
|
||||
Conf = rabbit_stream_queue:update_stream_conf(
|
||||
Conf0#{epoch => NewEpoch,
|
||||
leader_node => NewLeader,
|
||||
replica_nodes => lists:delete(NewLeader, Replicas ++ [Leader]),
|
||||
replica_pids => ReplicaPids}),
|
||||
Phase = phase_start_new_leader,
|
||||
PhaseArgs = [Conf],
|
||||
SState = SState0#{conf => Conf,
|
||||
phase => Phase,
|
||||
phase_args => PhaseArgs},
|
||||
rabbit_log:debug("rabbit_stream_coordinator: ~p entering phase_start_new_leader",
|
||||
[StreamId]),
|
||||
{State#?MODULE{streams = Streams#{StreamId => SState}}, ok,
|
||||
[{aux, {phase, StreamId, Phase, PhaseArgs}}]};
|
||||
apply(_Meta, {leader_elected, StreamId, NewLeaderPid},
|
||||
#?MODULE{streams = Streams, monitors = Monitors0} = State) ->
|
||||
rabbit_log:info("rabbit_stream_coordinator: ~p leader elected", [StreamId]),
|
||||
#{conf := Conf0,
|
||||
pending_cmds := Pending0} = SState0 = maps:get(StreamId, Streams),
|
||||
#{leader_pid := LeaderPid,
|
||||
replica_nodes := Replicas} = Conf0,
|
||||
Conf = Conf0#{leader_pid => NewLeaderPid},
|
||||
Phase = phase_repair_mnesia,
|
||||
PhaseArgs = [update, Conf],
|
||||
Pending = Pending0 ++ [{start_replica, #{stream_id => StreamId, node => R,
|
||||
retries => 1, from => undefined}}
|
||||
|| R <- Replicas],
|
||||
SState = SState0#{conf => Conf,
|
||||
phase => Phase,
|
||||
phase_args => PhaseArgs,
|
||||
pending_replicas => Replicas,
|
||||
pending_cmds => Pending},
|
||||
Monitors = maps:put(NewLeaderPid, {StreamId, leader}, maps:remove(LeaderPid, Monitors0)),
|
||||
rabbit_log:debug("rabbit_stream_coordinator: ~p entering ~p after "
|
||||
"leader election", [StreamId, Phase]),
|
||||
{State#?MODULE{streams = Streams#{StreamId => SState},
|
||||
monitors = Monitors}, ok,
|
||||
[{monitor, process, NewLeaderPid},
|
||||
{aux, {phase, StreamId, Phase, PhaseArgs}}]};
|
||||
apply(_Meta, {replicas_stopped, StreamId}, #?MODULE{streams = Streams} = State) ->
|
||||
case maps:get(StreamId, Streams, undefined) of
|
||||
undefined ->
|
||||
{State, {error, not_found}, []};
|
||||
#{conf := Conf0} = SState0 ->
|
||||
Phase = phase_check_quorum,
|
||||
Conf = Conf0#{replica_pids => []},
|
||||
PhaseArgs = [Conf],
|
||||
SState = SState0#{conf => Conf,
|
||||
phase => Phase,
|
||||
phase_args => PhaseArgs},
|
||||
rabbit_log:info("rabbit_stream_coordinator: ~p all replicas have been stopped, "
|
||||
"checking quorum available", [StreamId]),
|
||||
{State#?MODULE{streams = Streams#{StreamId => SState}}, ok,
|
||||
[{aux, {phase, StreamId, Phase, PhaseArgs}}]}
|
||||
end;
|
||||
apply(_Meta, {stream_updated, #{name := StreamId} = Conf}, #?MODULE{streams = Streams} = State) ->
|
||||
SState0 = maps:get(StreamId, Streams),
|
||||
Phase = phase_repair_mnesia,
|
||||
PhaseArgs = [update, Conf],
|
||||
SState = SState0#{conf => Conf,
|
||||
phase => Phase,
|
||||
phase_args => PhaseArgs},
|
||||
rabbit_log:debug("rabbit_stream_coordinator: ~p entering ~p after"
|
||||
" stream_updated", [StreamId, Phase]),
|
||||
{State#?MODULE{streams = Streams#{StreamId => SState}}, ok,
|
||||
[{aux, {phase, StreamId, Phase, PhaseArgs}}]};
|
||||
apply(_, {timeout, {pipeline, Cmds}}, State) ->
|
||||
Actions = [{mod_call, ra, pipeline_command, [{?MODULE, node()}, Cmd]} || Cmd <- Cmds],
|
||||
{State, ok, Actions};
|
||||
apply(_, {timeout, {aux, Cmd}}, State) ->
|
||||
{State, ok, [{aux, Cmd}]};
|
||||
apply(Meta, {_, #{from := From}} = Cmd, State) ->
|
||||
?MODULE:apply(Meta#{from => From}, Cmd, State).
|
||||
|
||||
state_enter(leader, #?MODULE{streams = Streams, monitors = Monitors}) ->
|
||||
maps:fold(fun(_, #{conf := #{name := StreamId},
|
||||
pending_replicas := Pending,
|
||||
state := State,
|
||||
phase := Phase,
|
||||
phase_args := PhaseArgs}, Acc) ->
|
||||
restart_aux_phase(State, Phase, PhaseArgs, StreamId) ++
|
||||
pipeline_restart_replica_cmds(StreamId, Pending) ++
|
||||
Acc
|
||||
end, [{monitor, process, P} || P <- maps:keys(Monitors)], Streams);
|
||||
state_enter(follower, #?MODULE{monitors = Monitors}) ->
|
||||
[{monitor, process, P} || P <- maps:keys(Monitors)];
|
||||
state_enter(recover, _) ->
|
||||
put('$rabbit_vm_category', ?MODULE),
|
||||
[];
|
||||
state_enter(_, _) ->
|
||||
[].
|
||||
|
||||
restart_aux_phase(running, _, _, _) ->
|
||||
[];
|
||||
restart_aux_phase(_State, Phase, PhaseArgs, StreamId) ->
|
||||
[{aux, {phase, StreamId, Phase, PhaseArgs}}].
|
||||
|
||||
pipeline_restart_replica_cmds(StreamId, Pending) ->
|
||||
[{timer, {pipeline, [{start_replica, #{stream_id => StreamId,
|
||||
node => Node,
|
||||
from => undefined,
|
||||
retries => 1}}
|
||||
|| Node <- Pending]}, ?RESTART_TIMEOUT}].
|
||||
|
||||
tick(_Ts, _State) ->
|
||||
[{aux, maybe_resize_coordinator_cluster}].
|
||||
|
||||
maybe_resize_coordinator_cluster() ->
|
||||
spawn(fun() ->
|
||||
case ra:members({?MODULE, node()}) of
|
||||
{_, Members, _} ->
|
||||
MemberNodes = [Node || {_, Node} <- Members],
|
||||
Running = rabbit_mnesia:cluster_nodes(running),
|
||||
All = rabbit_mnesia:cluster_nodes(all),
|
||||
case Running -- MemberNodes of
|
||||
[] ->
|
||||
ok;
|
||||
New ->
|
||||
rabbit_log:warning("New rabbit node(s) detected, "
|
||||
"adding stream coordinator in: ~p", [New]),
|
||||
add_members(Members, New)
|
||||
end,
|
||||
case MemberNodes -- All of
|
||||
[] ->
|
||||
ok;
|
||||
Old ->
|
||||
rabbit_log:warning("Rabbit node(s) removed from the cluster, "
|
||||
"deleting stream coordinator in: ~p", [Old]),
|
||||
remove_members(Members, Old)
|
||||
end;
|
||||
_ ->
|
||||
ok
|
||||
end
|
||||
end).
|
||||
|
||||
add_members(_, []) ->
|
||||
ok;
|
||||
add_members(Members, [Node | Nodes]) ->
|
||||
Conf = make_ra_conf(Node, [N || {_, N} <- Members]),
|
||||
case ra:start_server(Conf) of
|
||||
ok ->
|
||||
case ra:add_member(Members, {?MODULE, Node}) of
|
||||
{ok, NewMembers, _} ->
|
||||
add_members(NewMembers, Nodes);
|
||||
_ ->
|
||||
add_members(Members, Nodes)
|
||||
end;
|
||||
Error ->
|
||||
rabbit_log:warning("Stream coordinator failed to start on node ~p : ~p",
|
||||
[Node, Error]),
|
||||
add_members(Members, Nodes)
|
||||
end.
|
||||
|
||||
remove_members(_, []) ->
|
||||
ok;
|
||||
remove_members(Members, [Node | Nodes]) ->
|
||||
case ra:remove_member(Members, {?MODULE, Node}) of
|
||||
{ok, NewMembers, _} ->
|
||||
remove_members(NewMembers, Nodes);
|
||||
_ ->
|
||||
remove_members(Members, Nodes)
|
||||
end.
|
||||
|
||||
init_aux(_Name) ->
|
||||
{#{}, undefined}.
|
||||
|
||||
%% TODO ensure the dead writer is restarted as a replica at some point in time, increasing timeout?
|
||||
handle_aux(leader, _, maybe_resize_coordinator_cluster, {Monitors, undefined}, LogState, _) ->
|
||||
Pid = maybe_resize_coordinator_cluster(),
|
||||
{no_reply, {Monitors, Pid}, LogState, [{monitor, process, aux, Pid}]};
|
||||
handle_aux(leader, _, maybe_resize_coordinator_cluster, AuxState, LogState, _) ->
|
||||
%% Coordinator resizing is still happening, let's ignore this tick event
|
||||
{no_reply, AuxState, LogState};
|
||||
handle_aux(leader, _, {down, Pid, _}, {Monitors, Pid}, LogState, _) ->
|
||||
%% Coordinator resizing has finished
|
||||
{no_reply, {Monitors, undefined}, LogState};
|
||||
handle_aux(leader, _, {phase, _, Fun, Args} = Cmd, {Monitors, Coordinator}, LogState, _) ->
|
||||
Pid = erlang:apply(?MODULE, Fun, Args),
|
||||
Actions = [{monitor, process, aux, Pid}],
|
||||
{no_reply, {maps:put(Pid, Cmd, Monitors), Coordinator}, LogState, Actions};
|
||||
handle_aux(leader, _, {down, Pid, normal}, {Monitors, Coordinator}, LogState, _) ->
|
||||
{no_reply, {maps:remove(Pid, Monitors), Coordinator}, LogState};
|
||||
handle_aux(leader, _, {down, Pid, Reason}, {Monitors0, Coordinator}, LogState, _) ->
|
||||
%% The phase has failed, let's retry it
|
||||
case maps:get(Pid, Monitors0) of
|
||||
{phase, StreamId, phase_start_new_leader, Args} ->
|
||||
rabbit_log:warning("Error while starting new leader for stream queue ~p, "
|
||||
"restarting election: ~p", [StreamId, Reason]),
|
||||
Monitors = maps:remove(Pid, Monitors0),
|
||||
Cmd = {phase, StreamId, phase_check_quorum, Args},
|
||||
{no_reply, {Monitors, Coordinator}, LogState, [{timer, {aux, Cmd}, ?PHASE_RETRY_TIMEOUT}]};
|
||||
{phase, StreamId, Fun, _} = Cmd ->
|
||||
rabbit_log:warning("Error while executing coordinator phase ~p for stream queue ~p ~p",
|
||||
[Fun, StreamId, Reason]),
|
||||
Monitors = maps:remove(Pid, Monitors0),
|
||||
{no_reply, {Monitors, Coordinator}, LogState, [{timer, {aux, Cmd}, ?PHASE_RETRY_TIMEOUT}]}
|
||||
end;
|
||||
handle_aux(_, _, _, AuxState, LogState, _) ->
|
||||
{no_reply, AuxState, LogState}.
|
||||
|
||||
reply_and_run_pending(From, StreamId, Reply, WrapReply, Actions0, #?MODULE{streams = Streams} = State) ->
|
||||
#{pending_cmds := Pending} = SState0 = maps:get(StreamId, Streams),
|
||||
AuxActions = [{mod_call, ra, pipeline_command, [{?MODULE, node()}, Cmd]}
|
||||
|| Cmd <- Pending],
|
||||
SState = maps:put(pending_cmds, [], SState0),
|
||||
Actions = case From of
|
||||
undefined ->
|
||||
AuxActions ++ Actions0;
|
||||
_ ->
|
||||
wrap_reply(From, WrapReply) ++ AuxActions ++ Actions0
|
||||
end,
|
||||
{State#?MODULE{streams = Streams#{StreamId => SState}}, Reply, Actions}.
|
||||
|
||||
wrap_reply(From, Reply) ->
|
||||
[{reply, From, {wrap_reply, Reply}}].
|
||||
|
||||
add_pending_cmd(From, {CmdName, CmdMap}, #{pending_cmds := Pending0} = StreamState) ->
|
||||
%% Remove from pending the leader election and automatic replica restart when
|
||||
%% the command is delete_cluster
|
||||
Pending = case CmdName of
|
||||
delete_cluster ->
|
||||
lists:filter(fun({down, _, _}) ->
|
||||
false;
|
||||
(_) ->
|
||||
true
|
||||
end, Pending0);
|
||||
_ ->
|
||||
Pending0
|
||||
end,
|
||||
maps:put(pending_cmds, Pending ++ [{CmdName, maps:put(from, From, CmdMap)}],
|
||||
StreamState).
|
||||
|
||||
clear_stream_state(StreamState) ->
|
||||
StreamState#{reply_to => undefined,
|
||||
state => running,
|
||||
phase => undefined,
|
||||
phase_args => undefined}.
|
||||
|
||||
update_stream_state(From, State, Phase, PhaseArgs, StreamState) ->
|
||||
StreamState#{reply_to => From,
|
||||
state => State,
|
||||
phase => Phase,
|
||||
phase_args => PhaseArgs}.
|
||||
|
||||
phase_start_replica(Node, #{name := StreamId} = Conf0,
|
||||
Retries) ->
|
||||
spawn(
|
||||
fun() ->
|
||||
%% If a new leader hasn't yet been elected, this will fail with a badmatch
|
||||
%% as get_reader_context returns a no proc. An unhandled failure will
|
||||
%% crash this monitored process and restart it later.
|
||||
%% TODO However, do we want that crash in the log? We might need to try/catch
|
||||
%% to provide a log message instead as it's 'expected'. We could try to
|
||||
%% verify first that the leader is alive, but there would still be potential
|
||||
%% for a race condition in here.
|
||||
try
|
||||
case osiris_replica:start(Node, Conf0) of
|
||||
{ok, Pid} ->
|
||||
ra:pipeline_command({?MODULE, node()},
|
||||
{start_replica_reply, StreamId, Pid});
|
||||
{error, already_present} ->
|
||||
ra:pipeline_command({?MODULE, node()}, {phase_finished, StreamId, ok});
|
||||
{error, {already_started, _}} ->
|
||||
ra:pipeline_command({?MODULE, node()}, {phase_finished, StreamId, ok});
|
||||
{error, Reason} = Error ->
|
||||
rabbit_log:warning("Error while starting replica for ~p : ~p",
|
||||
[maps:get(name, Conf0), Reason]),
|
||||
ra:pipeline_command({?MODULE, node()},
|
||||
{start_replica_failed, StreamId, Node, Retries, Error})
|
||||
end
|
||||
catch _:E->
|
||||
rabbit_log:warning("Error while starting replica for ~p : ~p",
|
||||
[maps:get(name, Conf0), E]),
|
||||
ra:pipeline_command({?MODULE, node()},
|
||||
{start_replica_failed, StreamId, Node, Retries, {error, E}})
|
||||
end
|
||||
end).
|
||||
|
||||
phase_delete_replica(Node, Conf) ->
|
||||
spawn(
|
||||
fun() ->
|
||||
ok = osiris_replica:delete(Node, Conf),
|
||||
ra:pipeline_command({?MODULE, node()}, {stream_updated, Conf})
|
||||
end).
|
||||
|
||||
phase_stop_replicas(#{replica_nodes := Replicas,
|
||||
name := StreamId} = Conf) ->
|
||||
spawn(
|
||||
fun() ->
|
||||
[try
|
||||
osiris_replica:stop(Node, Conf)
|
||||
catch _:{{nodedown, _}, _} ->
|
||||
%% It could be the old leader that is still down, it's normal.
|
||||
ok
|
||||
end || Node <- Replicas],
|
||||
ra:pipeline_command({?MODULE, node()}, {replicas_stopped, StreamId})
|
||||
end).
|
||||
|
||||
phase_start_new_leader(#{name := StreamId, leader_node := Node, leader_pid := LPid} = Conf) ->
|
||||
spawn(fun() ->
|
||||
osiris_replica:stop(Node, Conf),
|
||||
%% If the start fails, the monitor will capture the crash and restart it
|
||||
case osiris_writer:start(Conf) of
|
||||
{ok, Pid} ->
|
||||
ra:pipeline_command({?MODULE, node()},
|
||||
{leader_elected, StreamId, Pid});
|
||||
{error, already_present} ->
|
||||
ra:pipeline_command({?MODULE, node()},
|
||||
{leader_elected, StreamId, LPid});
|
||||
{error, {already_started, Pid}} ->
|
||||
ra:pipeline_command({?MODULE, node()},
|
||||
{leader_elected, StreamId, Pid})
|
||||
end
|
||||
end).
|
||||
|
||||
phase_check_quorum(#{name := StreamId,
|
||||
epoch := Epoch,
|
||||
replica_nodes := Nodes} = Conf) ->
|
||||
spawn(fun() ->
|
||||
Offsets = find_replica_offsets(Conf),
|
||||
case is_quorum(length(Nodes) + 1, length(Offsets)) of
|
||||
true ->
|
||||
ra:pipeline_command({?MODULE, node()},
|
||||
{start_leader_election, StreamId, Epoch + 1, Offsets});
|
||||
false ->
|
||||
%% Let's crash this process so the monitor will restart it
|
||||
exit({not_enough_quorum, StreamId})
|
||||
end
|
||||
end).
|
||||
|
||||
find_replica_offsets(#{replica_nodes := Nodes,
|
||||
leader_node := Leader} = Conf) ->
|
||||
lists:foldl(
|
||||
fun(Node, Acc) ->
|
||||
try
|
||||
%% osiris_log:overview/1 needs the directory - last item of the list
|
||||
case rpc:call(Node, rabbit, is_running, []) of
|
||||
false ->
|
||||
Acc;
|
||||
true ->
|
||||
case rpc:call(Node, ?MODULE, log_overview, [Conf]) of
|
||||
{badrpc, nodedown} ->
|
||||
Acc;
|
||||
{_Range, Offsets} ->
|
||||
[{Node, select_highest_offset(Offsets)} | Acc]
|
||||
end
|
||||
end
|
||||
catch
|
||||
_:_ ->
|
||||
Acc
|
||||
end
|
||||
end, [], Nodes ++ [Leader]).
|
||||
|
||||
select_highest_offset([]) ->
|
||||
empty;
|
||||
select_highest_offset(Offsets) ->
|
||||
lists:last(Offsets).
|
||||
|
||||
log_overview(Config) ->
|
||||
Dir = osiris_log:directory(Config),
|
||||
osiris_log:overview(Dir).
|
||||
|
||||
find_max_offset(Offsets) ->
|
||||
[{Node, _} | _] = lists:sort(fun({_, {Ao, E}}, {_, {Bo, E}}) ->
|
||||
Ao >= Bo;
|
||||
({_, {_, Ae}}, {_, {_, Be}}) ->
|
||||
Ae >= Be;
|
||||
({_, empty}, _) ->
|
||||
false;
|
||||
(_, {_, empty}) ->
|
||||
true
|
||||
end, Offsets),
|
||||
Node.
|
||||
|
||||
is_quorum(1, 1) ->
|
||||
true;
|
||||
is_quorum(NumReplicas, NumAlive) ->
|
||||
NumAlive >= ((NumReplicas div 2) + 1).
|
||||
|
||||
phase_repair_mnesia(new, Q) ->
|
||||
spawn(fun() ->
|
||||
Reply = rabbit_amqqueue:internal_declare(Q, false),
|
||||
#{name := StreamId} = amqqueue:get_type_state(Q),
|
||||
ra:pipeline_command({?MODULE, node()}, {phase_finished, StreamId, Reply})
|
||||
end);
|
||||
|
||||
phase_repair_mnesia(update, #{reference := QName,
|
||||
leader_pid := LeaderPid,
|
||||
name := StreamId} = Conf) ->
|
||||
Fun = fun (Q) ->
|
||||
amqqueue:set_type_state(amqqueue:set_pid(Q, LeaderPid), Conf)
|
||||
end,
|
||||
spawn(fun() ->
|
||||
case rabbit_misc:execute_mnesia_transaction(
|
||||
fun() ->
|
||||
rabbit_amqqueue:update(QName, Fun)
|
||||
end) of
|
||||
not_found ->
|
||||
%% This can happen during recovery
|
||||
[Q] = mnesia:dirty_read(rabbit_durable_queue, QName),
|
||||
rabbit_amqqueue:ensure_rabbit_queue_record_is_initialized(Fun(Q));
|
||||
_ ->
|
||||
ok
|
||||
end,
|
||||
ra:pipeline_command({?MODULE, node()}, {phase_finished, StreamId, ok})
|
||||
end).
|
||||
|
||||
phase_start_cluster(Q0) ->
|
||||
spawn(
|
||||
fun() ->
|
||||
case osiris:start_cluster(amqqueue:get_type_state(Q0)) of
|
||||
{ok, #{leader_pid := Pid} = Conf} ->
|
||||
Q = amqqueue:set_type_state(amqqueue:set_pid(Q0, Pid), Conf),
|
||||
ra:pipeline_command({?MODULE, node()}, {start_cluster_reply, Q});
|
||||
{error, {already_started, _}} ->
|
||||
ra:pipeline_command({?MODULE, node()}, {start_cluster_finished, {error, already_started}})
|
||||
end
|
||||
end).
|
||||
|
||||
phase_delete_cluster(#{name := StreamId,
|
||||
reference := QName} = Conf, ActingUser) ->
|
||||
spawn(
|
||||
fun() ->
|
||||
ok = osiris:delete_cluster(Conf),
|
||||
_ = rabbit_amqqueue:internal_delete(QName, ActingUser),
|
||||
ra:pipeline_command({?MODULE, node()}, {delete_cluster_reply, StreamId})
|
||||
end).
|
||||
|
||||
format_ra_event(ServerId, Evt) ->
|
||||
{stream_coordinator_event, ServerId, Evt}.
|
||||
|
||||
make_ra_conf(Node, Nodes) ->
|
||||
UId = ra:new_uid(ra_lib:to_binary(?MODULE)),
|
||||
Formatter = {?MODULE, format_ra_event, []},
|
||||
Members = [{?MODULE, N} || N <- Nodes],
|
||||
TickTimeout = application:get_env(rabbit, stream_tick_interval,
|
||||
?TICK_TIMEOUT),
|
||||
#{cluster_name => ?MODULE,
|
||||
id => {?MODULE, Node},
|
||||
uid => UId,
|
||||
friendly_name => atom_to_list(?MODULE),
|
||||
metrics_key => ?MODULE,
|
||||
initial_members => Members,
|
||||
log_init_args => #{uid => UId},
|
||||
tick_timeout => TickTimeout,
|
||||
machine => {module, ?MODULE, #{}},
|
||||
ra_event_formatter => Formatter}.
|
||||
|
||||
add_unique(Node, Nodes) ->
|
||||
case lists:member(Node, Nodes) of
|
||||
true ->
|
||||
Nodes;
|
||||
_ ->
|
||||
[Node | Nodes]
|
||||
end.
|
||||
|
||||
delete_replica_pid(Node, ReplicaPids) ->
|
||||
lists:partition(fun(P) -> node(P) =/= Node end, ReplicaPids).
|
|
@ -0,0 +1,665 @@
|
|||
%% 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 https://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.
|
||||
%%
|
||||
%% Copyright (c) 2012-2020 VMware, Inc. or its affiliates. All rights reserved.
|
||||
%%
|
||||
|
||||
-module(rabbit_stream_queue).
|
||||
|
||||
-behaviour(rabbit_queue_type).
|
||||
|
||||
-export([is_enabled/0,
|
||||
declare/2,
|
||||
delete/4,
|
||||
purge/1,
|
||||
policy_changed/1,
|
||||
recover/2,
|
||||
is_recoverable/1,
|
||||
consume/3,
|
||||
cancel/5,
|
||||
handle_event/2,
|
||||
deliver/2,
|
||||
settle/4,
|
||||
credit/4,
|
||||
dequeue/4,
|
||||
info/2,
|
||||
init/1,
|
||||
close/1,
|
||||
update/2,
|
||||
state_info/1,
|
||||
stat/1,
|
||||
is_policy_applicable/2]).
|
||||
|
||||
-export([set_retention_policy/3]).
|
||||
-export([add_replica/3,
|
||||
delete_replica/3]).
|
||||
-export([format_osiris_event/2]).
|
||||
-export([update_stream_conf/1]).
|
||||
|
||||
-include("rabbit.hrl").
|
||||
-include("amqqueue.hrl").
|
||||
|
||||
-type appender_seq() :: non_neg_integer().
|
||||
|
||||
-record(stream, {name :: rabbit_types:r('queue'),
|
||||
credit :: integer(),
|
||||
max :: non_neg_integer(),
|
||||
start_offset = 0 :: non_neg_integer(),
|
||||
listening_offset = 0 :: non_neg_integer(),
|
||||
log :: undefined | orisis_log:state()}).
|
||||
|
||||
-record(stream_client, {name :: term(),
|
||||
leader :: pid(),
|
||||
next_seq = 1 :: non_neg_integer(),
|
||||
correlation = #{} :: #{appender_seq() => term()},
|
||||
soft_limit :: non_neg_integer(),
|
||||
slow = false :: boolean(),
|
||||
readers = #{} :: #{term() => #stream{}}
|
||||
}).
|
||||
|
||||
-import(rabbit_queue_type_util, [args_policy_lookup/3]).
|
||||
|
||||
-type client() :: #stream_client{}.
|
||||
|
||||
-spec is_enabled() -> boolean().
|
||||
is_enabled() ->
|
||||
rabbit_feature_flags:is_enabled(stream_queue).
|
||||
|
||||
-spec declare(amqqueue:amqqueue(), node()) ->
|
||||
{'new' | 'existing', amqqueue:amqqueue()} |
|
||||
rabbit_types:channel_exit().
|
||||
declare(Q0, Node) when ?amqqueue_is_stream(Q0) ->
|
||||
Arguments = amqqueue:get_arguments(Q0),
|
||||
QName = amqqueue:get_name(Q0),
|
||||
check_invalid_arguments(QName, Arguments),
|
||||
rabbit_queue_type_util:check_auto_delete(Q0),
|
||||
rabbit_queue_type_util:check_exclusive(Q0),
|
||||
rabbit_queue_type_util:check_non_durable(Q0),
|
||||
Opts = amqqueue:get_options(Q0),
|
||||
ActingUser = maps:get(user, Opts, ?UNKNOWN_USER),
|
||||
Conf0 = make_stream_conf(Node, Q0),
|
||||
case rabbit_stream_coordinator:start_cluster(
|
||||
amqqueue:set_type_state(Q0, Conf0)) of
|
||||
{ok, {error, already_started}, _} ->
|
||||
rabbit_misc:protocol_error(precondition_failed,
|
||||
"safe queue name already in use '~s'",
|
||||
[Node]);
|
||||
{ok, {created, Q}, _} ->
|
||||
rabbit_event:notify(queue_created,
|
||||
[{name, QName},
|
||||
{durable, true},
|
||||
{auto_delete, false},
|
||||
{arguments, Arguments},
|
||||
{user_who_performed_action,
|
||||
ActingUser}]),
|
||||
{new, Q};
|
||||
{ok, {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]);
|
||||
{ok, {existing, Q}, _} ->
|
||||
{existing, Q};
|
||||
{error, coordinator_unavailable} ->
|
||||
_ = rabbit_amqqueue:internal_delete(QName, ActingUser),
|
||||
rabbit_misc:protocol_error(
|
||||
internal_error,
|
||||
"Cannot declare a queue '~s' on node '~s': coordinator unavailable",
|
||||
[rabbit_misc:rs(QName), node()])
|
||||
end.
|
||||
|
||||
-spec delete(amqqueue:amqqueue(), boolean(),
|
||||
boolean(), rabbit_types:username()) ->
|
||||
rabbit_types:ok(non_neg_integer()) |
|
||||
rabbit_types:error(in_use | not_empty).
|
||||
delete(Q, _IfUnused, _IfEmpty, ActingUser) ->
|
||||
Name = maps:get(name, amqqueue:get_type_state(Q)),
|
||||
{ok, Reply, _} = rabbit_stream_coordinator:delete_cluster(Name, ActingUser),
|
||||
Reply.
|
||||
|
||||
-spec purge(amqqueue:amqqueue()) ->
|
||||
{'ok', non_neg_integer()}.
|
||||
purge(_Q) ->
|
||||
{ok, 0}.
|
||||
|
||||
-spec policy_changed(amqqueue:amqqueue()) -> 'ok'.
|
||||
policy_changed(Q) ->
|
||||
Name = maps:get(name, amqqueue:get_type_state(Q)),
|
||||
_ = rabbit_stream_coordinator:policy_changed(Name),
|
||||
ok.
|
||||
|
||||
stat(_) ->
|
||||
{ok, 0, 0}.
|
||||
|
||||
consume(Q, #{prefetch_count := 0}, _)
|
||||
when ?amqqueue_is_stream(Q) ->
|
||||
rabbit_misc:protocol_error(precondition_failed,
|
||||
"consumer prefetch count is not set for '~s'",
|
||||
[rabbit_misc:rs(amqqueue:get_name(Q))]);
|
||||
consume(Q, #{no_ack := true}, _)
|
||||
when ?amqqueue_is_stream(Q) ->
|
||||
rabbit_misc:protocol_error(
|
||||
not_implemented,
|
||||
"automatic acknowledgement not supported by stream queues ~s",
|
||||
[rabbit_misc:rs(amqqueue:get_name(Q))]);
|
||||
consume(Q, #{limiter_active := true}, _State)
|
||||
when ?amqqueue_is_stream(Q) ->
|
||||
{error, global_qos_not_supported_for_queue_type};
|
||||
consume(Q, Spec, QState0) when ?amqqueue_is_stream(Q) ->
|
||||
%% Messages should include the offset as a custom header.
|
||||
check_queue_exists_in_local_node(Q),
|
||||
#{no_ack := NoAck,
|
||||
channel_pid := ChPid,
|
||||
prefetch_count := ConsumerPrefetchCount,
|
||||
consumer_tag := ConsumerTag,
|
||||
exclusive_consume := ExclusiveConsume,
|
||||
args := Args,
|
||||
ok_msg := OkMsg} = Spec,
|
||||
QName = amqqueue:get_name(Q),
|
||||
Offset = case rabbit_misc:table_lookup(Args, <<"x-stream-offset">>) of
|
||||
undefined ->
|
||||
next;
|
||||
{_, <<"first">>} ->
|
||||
first;
|
||||
{_, <<"last">>} ->
|
||||
last;
|
||||
{_, <<"next">>} ->
|
||||
next;
|
||||
{_, V} ->
|
||||
V
|
||||
end,
|
||||
rabbit_core_metrics:consumer_created(ChPid, ConsumerTag, ExclusiveConsume,
|
||||
not NoAck, QName,
|
||||
ConsumerPrefetchCount, false,
|
||||
up, Args),
|
||||
%% FIXME: reply needs to be sent before the stream begins sending
|
||||
%% really it should be sent by the stream queue process like classic queues
|
||||
%% do
|
||||
maybe_send_reply(ChPid, OkMsg),
|
||||
QState = begin_stream(QState0, Q, ConsumerTag, Offset,
|
||||
ConsumerPrefetchCount),
|
||||
{ok, QState, []}.
|
||||
|
||||
get_local_pid(#{leader_pid := Pid}) when node(Pid) == node() ->
|
||||
Pid;
|
||||
get_local_pid(#{replica_pids := ReplicaPids}) ->
|
||||
[Local | _] = lists:filter(fun(Pid) ->
|
||||
node(Pid) == node()
|
||||
end, ReplicaPids),
|
||||
Local.
|
||||
|
||||
begin_stream(#stream_client{readers = Readers0} = State,
|
||||
Q, Tag, Offset, Max) ->
|
||||
LocalPid = get_local_pid(amqqueue:get_type_state(Q)),
|
||||
{ok, Seg0} = osiris:init_reader(LocalPid, Offset),
|
||||
NextOffset = osiris_log:next_offset(Seg0) - 1,
|
||||
osiris:register_offset_listener(LocalPid, NextOffset),
|
||||
%% TODO: avoid double calls to the same process
|
||||
StartOffset = case Offset of
|
||||
last -> NextOffset;
|
||||
next -> NextOffset;
|
||||
_ -> Offset
|
||||
end,
|
||||
Str0 = #stream{name = amqqueue:get_name(Q),
|
||||
credit = Max,
|
||||
start_offset = StartOffset,
|
||||
listening_offset = NextOffset,
|
||||
log = Seg0,
|
||||
max = Max},
|
||||
State#stream_client{readers = Readers0#{Tag => Str0}}.
|
||||
|
||||
cancel(_Q, ConsumerTag, OkMsg, ActingUser, #stream_client{readers = Readers0,
|
||||
name = QName} = State) ->
|
||||
Readers = maps:remove(ConsumerTag, Readers0),
|
||||
rabbit_core_metrics:consumer_deleted(self(), ConsumerTag, QName),
|
||||
rabbit_event:notify(consumer_deleted, [{consumer_tag, ConsumerTag},
|
||||
{channel, self()},
|
||||
{queue, QName},
|
||||
{user_who_performed_action, ActingUser}]),
|
||||
maybe_send_reply(self(), OkMsg),
|
||||
{ok, State#stream_client{readers = Readers}}.
|
||||
|
||||
credit(_, _, _, _) ->
|
||||
ok.
|
||||
|
||||
deliver(QSs, #delivery{confirm = Confirm} = Delivery) ->
|
||||
lists:foldl(
|
||||
fun({_Q, stateless}, {Qs, Actions}) ->
|
||||
%% TODO what do we do with stateless?
|
||||
%% QRef = amqqueue:get_pid(Q),
|
||||
%% ok = rabbit_fifo_client:untracked_enqueue(
|
||||
%% [QRef], Delivery#delivery.message),
|
||||
{Qs, Actions};
|
||||
({Q, S0}, {Qs, Actions}) ->
|
||||
S = deliver(Confirm, Delivery, S0),
|
||||
{[{Q, S} | Qs], Actions}
|
||||
end, {[], []}, QSs).
|
||||
|
||||
deliver(_Confirm, #delivery{message = Msg, msg_seq_no = MsgId},
|
||||
#stream_client{name = Name,
|
||||
leader = LeaderPid,
|
||||
next_seq = Seq,
|
||||
correlation = Correlation0,
|
||||
soft_limit = SftLmt,
|
||||
slow = Slow0} = State) ->
|
||||
ok = osiris:write(LeaderPid, Seq, msg_to_iodata(Msg)),
|
||||
Correlation = case MsgId of
|
||||
undefined ->
|
||||
Correlation0;
|
||||
_ when is_number(MsgId) ->
|
||||
Correlation0#{Seq => MsgId}
|
||||
end,
|
||||
Slow = case maps:size(Correlation) >= SftLmt of
|
||||
true when not Slow0 ->
|
||||
credit_flow:block(Name),
|
||||
true;
|
||||
Bool ->
|
||||
Bool
|
||||
end,
|
||||
State#stream_client{next_seq = Seq + 1,
|
||||
correlation = Correlation,
|
||||
slow = Slow}.
|
||||
-spec dequeue(_, _, _, client()) -> no_return().
|
||||
dequeue(_, _, _, #stream_client{name = Name}) ->
|
||||
rabbit_misc:protocol_error(
|
||||
not_implemented,
|
||||
"basic.get not supported by stream queues ~s",
|
||||
[rabbit_misc:rs(Name)]).
|
||||
|
||||
handle_event({osiris_written, From, Corrs}, State = #stream_client{correlation = Correlation0,
|
||||
soft_limit = SftLmt,
|
||||
slow = Slow0,
|
||||
name = Name}) ->
|
||||
MsgIds = maps:values(maps:with(Corrs, Correlation0)),
|
||||
Correlation = maps:without(Corrs, Correlation0),
|
||||
Slow = case maps:size(Correlation) < SftLmt of
|
||||
true when Slow0 ->
|
||||
credit_flow:unblock(Name),
|
||||
false;
|
||||
_ ->
|
||||
Slow0
|
||||
end,
|
||||
{ok, State#stream_client{correlation = Correlation,
|
||||
slow = Slow}, [{settled, From, MsgIds}]};
|
||||
handle_event({osiris_offset, _From, _Offs}, State = #stream_client{leader = Leader,
|
||||
readers = Readers0,
|
||||
name = Name}) ->
|
||||
%% offset isn't actually needed as we use the atomic to read the
|
||||
%% current committed
|
||||
{Readers, TagMsgs} = maps:fold(
|
||||
fun (Tag, Str0, {Acc, TM}) ->
|
||||
{Str, Msgs} = stream_entries(Name, Leader, Str0),
|
||||
%% HACK for now, better to just return but
|
||||
%% tricky with acks credits
|
||||
%% that also evaluate the stream
|
||||
% gen_server:cast(self(), {stream_delivery, Tag, Msgs}),
|
||||
{Acc#{Tag => Str}, [{Tag, Leader, Msgs} | TM]}
|
||||
end, {#{}, []}, Readers0),
|
||||
Ack = true,
|
||||
Deliveries = [{deliver, Tag, Ack, OffsetMsg}
|
||||
|| {Tag, _LeaderPid, OffsetMsg} <- TagMsgs],
|
||||
{ok, State#stream_client{readers = Readers}, Deliveries}.
|
||||
|
||||
is_recoverable(Q) ->
|
||||
Node = node(),
|
||||
#{replica_nodes := Nodes,
|
||||
leader_node := Leader} = amqqueue:get_type_state(Q),
|
||||
lists:member(Node, Nodes ++ [Leader]).
|
||||
|
||||
recover(_VHost, Queues) ->
|
||||
lists:foldl(
|
||||
fun (Q0, {R0, F0}) ->
|
||||
{ok, Q} = recover(Q0),
|
||||
{[Q | R0], F0}
|
||||
end, {[], []}, Queues).
|
||||
|
||||
settle(complete, CTag, MsgIds, #stream_client{readers = Readers0,
|
||||
name = Name,
|
||||
leader = Leader} = State) ->
|
||||
Credit = length(MsgIds),
|
||||
{Readers, Msgs} = case Readers0 of
|
||||
#{CTag := #stream{credit = Credit0} = Str0} ->
|
||||
Str1 = Str0#stream{credit = Credit0 + Credit},
|
||||
{Str, Msgs0} = stream_entries(Name, Leader, Str1),
|
||||
{Readers0#{CTag => Str}, Msgs0};
|
||||
_ ->
|
||||
{Readers0, []}
|
||||
end,
|
||||
{State#stream_client{readers = Readers}, [{deliver, CTag, true, Msgs}]};
|
||||
settle(_, _, _, #stream_client{name = Name}) ->
|
||||
rabbit_misc:protocol_error(
|
||||
not_implemented,
|
||||
"basic.nack and basic.reject not supported by stream queues ~s",
|
||||
[rabbit_misc:rs(Name)]).
|
||||
|
||||
info(Q, Items) ->
|
||||
lists:foldr(fun(Item, Acc) ->
|
||||
[{Item, i(Item, Q)} | Acc]
|
||||
end, [], Items).
|
||||
|
||||
i(name, Q) when ?is_amqqueue(Q) -> amqqueue:get_name(Q);
|
||||
i(durable, Q) when ?is_amqqueue(Q) -> amqqueue:is_durable(Q);
|
||||
i(auto_delete, Q) when ?is_amqqueue(Q) -> amqqueue:is_auto_delete(Q);
|
||||
i(arguments, Q) when ?is_amqqueue(Q) -> amqqueue:get_arguments(Q);
|
||||
i(leader, Q) when ?is_amqqueue(Q) ->
|
||||
#{leader_node := Leader} = amqqueue:get_type_state(Q),
|
||||
Leader;
|
||||
i(members, Q) when ?is_amqqueue(Q) ->
|
||||
#{replica_nodes := Nodes} = amqqueue:get_type_state(Q),
|
||||
Nodes;
|
||||
i(online, Q) ->
|
||||
#{replica_pids := ReplicaPids,
|
||||
leader_pid := LeaderPid} = amqqueue:get_type_state(Q),
|
||||
[node(P) || P <- ReplicaPids ++ [LeaderPid], rabbit_misc:is_process_alive(P)];
|
||||
i(state, Q) when ?is_amqqueue(Q) ->
|
||||
%% TODO the coordinator should answer this, I guess??
|
||||
running;
|
||||
i(messages, Q) when ?is_amqqueue(Q) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
case ets:lookup(queue_coarse_metrics, QName) of
|
||||
[{_, _, _, M, _}] ->
|
||||
M;
|
||||
[] ->
|
||||
0
|
||||
end;
|
||||
i(messages_ready, Q) when ?is_amqqueue(Q) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
case ets:lookup(queue_coarse_metrics, QName) of
|
||||
[{_, MR, _, _, _}] ->
|
||||
MR;
|
||||
[] ->
|
||||
0
|
||||
end;
|
||||
i(messages_unacknowledged, Q) when ?is_amqqueue(Q) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
case ets:lookup(queue_coarse_metrics, QName) of
|
||||
[{_, _, MU, _, _}] ->
|
||||
MU;
|
||||
[] ->
|
||||
0
|
||||
end;
|
||||
i(committed_offset, Q) ->
|
||||
%% TODO should it be on a metrics table?
|
||||
Data = osiris_counters:overview(),
|
||||
maps:get(committed_offset,
|
||||
maps:get({osiris_writer, amqqueue:get_name(Q)}, Data));
|
||||
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(type, _) ->
|
||||
stream.
|
||||
|
||||
init(Q) when ?is_amqqueue(Q) ->
|
||||
Leader = amqqueue:get_pid(Q),
|
||||
{ok, SoftLimit} = application:get_env(rabbit, stream_messages_soft_limit),
|
||||
#stream_client{name = amqqueue:get_name(Q),
|
||||
leader = Leader,
|
||||
soft_limit = SoftLimit}.
|
||||
|
||||
close(#stream_client{readers = Readers}) ->
|
||||
_ = maps:map(fun (_, #stream{log = Log}) ->
|
||||
osiris_log:close(Log)
|
||||
end, Readers),
|
||||
ok.
|
||||
|
||||
update(_, State) ->
|
||||
State.
|
||||
|
||||
state_info(_) ->
|
||||
#{}.
|
||||
|
||||
set_retention_policy(Name, VHost, Policy) ->
|
||||
case rabbit_amqqueue:check_max_age(Policy) of
|
||||
{error, _} = E ->
|
||||
E;
|
||||
MaxAge ->
|
||||
QName = rabbit_misc:r(VHost, queue, Name),
|
||||
Fun = fun(Q) ->
|
||||
Conf = amqqueue:get_type_state(Q),
|
||||
amqqueue:set_type_state(Q, Conf#{max_age => MaxAge})
|
||||
end,
|
||||
case rabbit_misc:execute_mnesia_transaction(
|
||||
fun() -> rabbit_amqqueue:update(QName, Fun) end) of
|
||||
not_found ->
|
||||
{error, not_found};
|
||||
_ ->
|
||||
ok
|
||||
end
|
||||
end.
|
||||
|
||||
add_replica(VHost, Name, Node) ->
|
||||
QName = rabbit_misc:r(VHost, queue, Name),
|
||||
case rabbit_amqqueue:lookup(QName) of
|
||||
{ok, Q} when ?amqqueue_is_classic(Q) ->
|
||||
{error, classic_queue_not_supported};
|
||||
{ok, Q} when ?amqqueue_is_quorum(Q) ->
|
||||
{error, quorum_queue_not_supported};
|
||||
{ok, Q} when ?amqqueue_is_stream(Q) ->
|
||||
case lists:member(Node, rabbit_mnesia:cluster_nodes(running)) of
|
||||
false ->
|
||||
{error, node_not_running};
|
||||
true ->
|
||||
#{name := StreamId} = amqqueue:get_type_state(Q),
|
||||
{ok, Reply, _} = rabbit_stream_coordinator:add_replica(StreamId, Node),
|
||||
Reply
|
||||
end;
|
||||
E ->
|
||||
E
|
||||
end.
|
||||
|
||||
delete_replica(VHost, Name, Node) ->
|
||||
QName = rabbit_misc:r(VHost, queue, Name),
|
||||
case rabbit_amqqueue:lookup(QName) of
|
||||
{ok, Q} when ?amqqueue_is_classic(Q) ->
|
||||
{error, classic_queue_not_supported};
|
||||
{ok, Q} when ?amqqueue_is_quorum(Q) ->
|
||||
{error, quorum_queue_not_supported};
|
||||
{ok, Q} when ?amqqueue_is_stream(Q) ->
|
||||
case lists:member(Node, rabbit_mnesia:cluster_nodes(running)) of
|
||||
false ->
|
||||
{error, node_not_running};
|
||||
true ->
|
||||
#{name := StreamId} = amqqueue:get_type_state(Q),
|
||||
{ok, Reply, _} = rabbit_stream_coordinator:delete_replica(StreamId, Node),
|
||||
Reply
|
||||
end;
|
||||
E ->
|
||||
E
|
||||
end.
|
||||
|
||||
make_stream_conf(Node, Q) ->
|
||||
QName = amqqueue:get_name(Q),
|
||||
Name = queue_name(QName),
|
||||
%% MaxLength = args_policy_lookup(<<"max-length">>, fun min/2, Q),
|
||||
MaxBytes = args_policy_lookup(<<"max-length-bytes">>, fun min/2, Q),
|
||||
MaxAge = max_age(args_policy_lookup(<<"max-age">>, fun max_age/2, Q)),
|
||||
MaxSegmentSize = args_policy_lookup(<<"max-segment-size">>, fun min/2, Q),
|
||||
Replicas = rabbit_mnesia:cluster_nodes(all) -- [Node],
|
||||
Formatter = {?MODULE, format_osiris_event, [QName]},
|
||||
Retention = lists:filter(fun({_, R}) ->
|
||||
R =/= undefined
|
||||
end, [{max_bytes, MaxBytes},
|
||||
{max_age, MaxAge}]),
|
||||
add_if_defined(max_segment_size, MaxSegmentSize, #{reference => QName,
|
||||
name => Name,
|
||||
retention => Retention,
|
||||
leader_node => Node,
|
||||
replica_nodes => Replicas,
|
||||
event_formatter => Formatter,
|
||||
epoch => 1}).
|
||||
|
||||
update_stream_conf(#{reference := QName} = Conf) ->
|
||||
case rabbit_amqqueue:lookup(QName) of
|
||||
{ok, Q} ->
|
||||
MaxBytes = args_policy_lookup(<<"max-length-bytes">>, fun min/2, Q),
|
||||
MaxAge = max_age(args_policy_lookup(<<"max-age">>, fun max_age/2, Q)),
|
||||
MaxSegmentSize = args_policy_lookup(<<"max-segment-size">>, fun min/2, Q),
|
||||
Retention = lists:filter(fun({_, R}) ->
|
||||
R =/= undefined
|
||||
end, [{max_bytes, MaxBytes},
|
||||
{max_age, MaxAge}]),
|
||||
add_if_defined(max_segment_size, MaxSegmentSize, Conf#{retention => Retention});
|
||||
_ ->
|
||||
Conf
|
||||
end.
|
||||
|
||||
add_if_defined(_, undefined, Map) ->
|
||||
Map;
|
||||
add_if_defined(Key, Value, Map) ->
|
||||
maps:put(Key, Value, Map).
|
||||
|
||||
format_osiris_event(Evt, QRef) ->
|
||||
{'$gen_cast', {queue_event, QRef, Evt}}.
|
||||
|
||||
max_age(undefined) ->
|
||||
undefined;
|
||||
max_age(Bin) when is_binary(Bin) ->
|
||||
rabbit_amqqueue:check_max_age(Bin);
|
||||
max_age(Age) ->
|
||||
Age.
|
||||
|
||||
max_age(Age1, Age2) ->
|
||||
min(rabbit_amqqueue:check_max_age(Age1), rabbit_amqqueue:check_max_age(Age2)).
|
||||
|
||||
check_invalid_arguments(QueueName, Args) ->
|
||||
Keys = [<<"x-expires">>, <<"x-message-ttl">>,
|
||||
<<"x-max-priority">>, <<"x-queue-mode">>, <<"x-overflow">>,
|
||||
<<"x-max-in-memory-length">>, <<"x-max-in-memory-bytes">>,
|
||||
<<"x-quorum-initial-group-size">>, <<"x-cancel-on-ha-failover">>],
|
||||
rabbit_queue_type_util:check_invalid_arguments(QueueName, Args, Keys).
|
||||
|
||||
queue_name(#resource{virtual_host = VHost, name = Name}) ->
|
||||
Timestamp = erlang:integer_to_binary(erlang:system_time()),
|
||||
osiris_util:to_base64uri(erlang:binary_to_list(<<VHost/binary, "_", Name/binary, "_",
|
||||
Timestamp/binary>>)).
|
||||
|
||||
recover(Q) ->
|
||||
rabbit_stream_coordinator:recover(),
|
||||
{ok, Q}.
|
||||
|
||||
check_queue_exists_in_local_node(Q) ->
|
||||
Conf = amqqueue:get_type_state(Q),
|
||||
AllNodes = [maps:get(leader_node, Conf) | maps:get(replica_nodes, Conf)],
|
||||
case lists:member(node(), AllNodes) of
|
||||
true ->
|
||||
ok;
|
||||
false ->
|
||||
rabbit_misc:protocol_error(precondition_failed,
|
||||
"queue '~s' does not a have a replica on the local node",
|
||||
[rabbit_misc:rs(amqqueue:get_name(Q))])
|
||||
end.
|
||||
|
||||
maybe_send_reply(_ChPid, undefined) -> ok;
|
||||
maybe_send_reply(ChPid, Msg) -> ok = rabbit_channel:send_command(ChPid, Msg).
|
||||
|
||||
stream_entries(Name, Id, Str) ->
|
||||
stream_entries(Name, Id, Str, []).
|
||||
|
||||
stream_entries(Name, LeaderPid,
|
||||
#stream{name = QName,
|
||||
credit = Credit,
|
||||
start_offset = StartOffs,
|
||||
listening_offset = LOffs,
|
||||
log = Seg0} = Str0, MsgIn)
|
||||
when Credit > 0 ->
|
||||
case osiris_log:read_chunk_parsed(Seg0) of
|
||||
{end_of_stream, Seg} ->
|
||||
NextOffset = osiris_log:next_offset(Seg),
|
||||
case NextOffset > LOffs of
|
||||
true ->
|
||||
osiris:register_offset_listener(LeaderPid, NextOffset),
|
||||
{Str0#stream{log = Seg,
|
||||
listening_offset = NextOffset}, MsgIn};
|
||||
false ->
|
||||
{Str0#stream{log = Seg}, MsgIn}
|
||||
end;
|
||||
{Records, Seg} ->
|
||||
Msgs = [begin
|
||||
Msg0 = binary_to_msg(QName, B),
|
||||
Msg = rabbit_basic:add_header(<<"x-stream-offset">>,
|
||||
long, O, Msg0),
|
||||
{Name, LeaderPid, O, false, Msg}
|
||||
end || {O, B} <- Records,
|
||||
O >= StartOffs],
|
||||
|
||||
NumMsgs = length(Msgs),
|
||||
|
||||
Str = Str0#stream{credit = Credit - NumMsgs,
|
||||
log = Seg},
|
||||
case Str#stream.credit < 1 of
|
||||
true ->
|
||||
%% we are done here
|
||||
{Str, MsgIn ++ Msgs};
|
||||
false ->
|
||||
%% if there are fewer Msgs than Entries0 it means there were non-events
|
||||
%% in the log and we should recurse and try again
|
||||
stream_entries(Name, LeaderPid, Str, MsgIn ++ Msgs)
|
||||
end
|
||||
end;
|
||||
stream_entries(_Name, _Id, Str, Msgs) ->
|
||||
{Str, Msgs}.
|
||||
|
||||
binary_to_msg(#resource{virtual_host = VHost,
|
||||
kind = queue,
|
||||
name = QName}, Data) ->
|
||||
R0 = rabbit_msg_record:init(Data),
|
||||
%% if the message annotation isn't present the data most likely came from
|
||||
%% the rabbitmq-stream plugin so we'll choose defaults that simulate use
|
||||
%% of the direct exchange
|
||||
{utf8, Exchange} = rabbit_msg_record:message_annotation(<<"x-exchange">>,
|
||||
R0, {utf8, <<>>}),
|
||||
{utf8, RoutingKey} = rabbit_msg_record:message_annotation(<<"x-routing-key">>,
|
||||
R0, {utf8, QName}),
|
||||
{Props, Payload} = rabbit_msg_record:to_amqp091(R0),
|
||||
XName = #resource{kind = exchange,
|
||||
virtual_host = VHost,
|
||||
name = Exchange},
|
||||
Content = #content{class_id = 60,
|
||||
properties = Props,
|
||||
properties_bin = none,
|
||||
payload_fragments_rev = [Payload]},
|
||||
{ok, Msg} = rabbit_basic:message(XName, RoutingKey, Content),
|
||||
Msg.
|
||||
|
||||
|
||||
msg_to_iodata(#basic_message{exchange_name = #resource{name = Exchange},
|
||||
routing_keys = [RKey | _],
|
||||
content = Content}) ->
|
||||
#content{properties = Props,
|
||||
payload_fragments_rev = Payload} =
|
||||
rabbit_binary_parser:ensure_content_decoded(Content),
|
||||
R0 = rabbit_msg_record:from_amqp091(Props, lists:reverse(Payload)),
|
||||
%% TODO durable?
|
||||
R = rabbit_msg_record:add_message_annotations(
|
||||
#{<<"x-exchange">> => {utf8, Exchange},
|
||||
<<"x-routing-key">> => {utf8, RKey}}, R0),
|
||||
rabbit_msg_record:to_iodata(R).
|
||||
|
||||
-spec is_policy_applicable(amqqueue:amqqueue(), any()) -> boolean().
|
||||
is_policy_applicable(_Q, Policy) ->
|
||||
Applicable = [<<"max-length-bytes">>, <<"max-age">>, <<"max-segment-size">>],
|
||||
lists:all(fun({P, _}) ->
|
||||
lists:member(P, Applicable)
|
||||
end, Policy).
|
|
@ -48,11 +48,11 @@ recover(VHost) ->
|
|||
VHostStubFile = filename:join(VHostDir, ".vhost"),
|
||||
ok = rabbit_file:ensure_dir(VHostStubFile),
|
||||
ok = file:write_file(VHostStubFile, VHost),
|
||||
{RecoveredClassic, FailedClassic, Quorum} = rabbit_amqqueue:recover(VHost),
|
||||
AllQs = RecoveredClassic ++ FailedClassic ++ Quorum,
|
||||
{Recovered, Failed} = rabbit_amqqueue:recover(VHost),
|
||||
AllQs = Recovered ++ Failed,
|
||||
QNames = [amqqueue:get_name(Q) || Q <- AllQs],
|
||||
ok = rabbit_binding:recover(rabbit_exchange:recover(VHost), QNames),
|
||||
ok = rabbit_amqqueue:start(RecoveredClassic),
|
||||
ok = rabbit_amqqueue:start(Recovered),
|
||||
%% Start queue mirrors.
|
||||
ok = rabbit_mirror_queue_misc:on_vhost_up(VHost),
|
||||
ok.
|
||||
|
|
|
@ -20,8 +20,8 @@ memory() ->
|
|||
{Sums, _Other} = sum_processes(
|
||||
lists:append(All), distinguishers(), [memory]),
|
||||
|
||||
[Qs, QsSlave, Qqs, ConnsReader, ConnsWriter, ConnsChannel, ConnsOther,
|
||||
MsgIndexProc, MgmtDbProc, Plugins] =
|
||||
[Qs, QsSlave, Qqs, Ssqs, Srqs, SCoor, ConnsReader, ConnsWriter, ConnsChannel,
|
||||
ConnsOther, MsgIndexProc, MgmtDbProc, Plugins] =
|
||||
[aggregate(Names, Sums, memory, fun (X) -> X end)
|
||||
|| Names <- distinguished_interesting_sups()],
|
||||
|
||||
|
@ -55,7 +55,8 @@ memory() ->
|
|||
|
||||
OtherProc = Processes
|
||||
- ConnsReader - ConnsWriter - ConnsChannel - ConnsOther
|
||||
- Qs - QsSlave - Qqs - MsgIndexProc - Plugins - MgmtDbProc - MetricsProc,
|
||||
- Qs - QsSlave - Qqs - Ssqs - Srqs - SCoor - MsgIndexProc - Plugins
|
||||
- MgmtDbProc - MetricsProc,
|
||||
|
||||
[
|
||||
%% Connections
|
||||
|
@ -68,6 +69,9 @@ memory() ->
|
|||
{queue_procs, Qs},
|
||||
{queue_slave_procs, QsSlave},
|
||||
{quorum_queue_procs, Qqs},
|
||||
{stream_queue_procs, Ssqs},
|
||||
{stream_queue_replica_reader_procs, Srqs},
|
||||
{stream_queue_coordinator_procs, SCoor},
|
||||
|
||||
%% Processes
|
||||
{plugins, Plugins},
|
||||
|
@ -114,8 +118,8 @@ binary() ->
|
|||
sets:add_element({Ptr, Sz}, Acc0)
|
||||
end, Acc, Info)
|
||||
end, distinguishers(), [{binary, sets:new()}]),
|
||||
[Other, Qs, QsSlave, Qqs, ConnsReader, ConnsWriter, ConnsChannel, ConnsOther,
|
||||
MsgIndexProc, MgmtDbProc, Plugins] =
|
||||
[Other, Qs, QsSlave, Qqs, Ssqs, Srqs, Scoor, ConnsReader, ConnsWriter,
|
||||
ConnsChannel, ConnsOther, MsgIndexProc, MgmtDbProc, Plugins] =
|
||||
[aggregate(Names, [{other, Rest} | Sums], binary, fun sum_binary/1)
|
||||
|| Names <- [[other] | distinguished_interesting_sups()]],
|
||||
[{connection_readers, ConnsReader},
|
||||
|
@ -125,6 +129,9 @@ binary() ->
|
|||
{queue_procs, Qs},
|
||||
{queue_slave_procs, QsSlave},
|
||||
{quorum_queue_procs, Qqs},
|
||||
{stream_queue_procs, Ssqs},
|
||||
{stream_queue_replica_reader_procs, Srqs},
|
||||
{stream_queue_coordinator_procs, Scoor},
|
||||
{plugins, Plugins},
|
||||
{mgmt_db, MgmtDbProc},
|
||||
{msg_index, MsgIndexProc},
|
||||
|
@ -168,7 +175,8 @@ bytes(Words) -> try
|
|||
end.
|
||||
|
||||
interesting_sups() ->
|
||||
[queue_sups(), quorum_sups(), conn_sups() | interesting_sups0()].
|
||||
[queue_sups(), quorum_sups(), stream_server_sups(), stream_reader_sups(),
|
||||
conn_sups() | interesting_sups0()].
|
||||
|
||||
queue_sups() ->
|
||||
all_vhosts_children(rabbit_amqqueue_sup_sup).
|
||||
|
@ -184,6 +192,9 @@ quorum_sups() ->
|
|||
supervisor:which_children(ra_server_sup_sup)]
|
||||
end.
|
||||
|
||||
stream_server_sups() -> [osiris_server_sup].
|
||||
stream_reader_sups() -> [osiris_replica_reader_sup].
|
||||
|
||||
msg_stores() ->
|
||||
all_vhosts_children(msg_store_transient)
|
||||
++
|
||||
|
@ -229,13 +240,17 @@ ranch_server_sups() ->
|
|||
with(Sups, With) -> [{Sup, With} || Sup <- Sups].
|
||||
|
||||
distinguishers() -> with(queue_sups(), fun queue_type/1) ++
|
||||
with(conn_sups(), fun conn_type/1).
|
||||
with(conn_sups(), fun conn_type/1) ++
|
||||
with(quorum_sups(), fun ra_type/1).
|
||||
|
||||
distinguished_interesting_sups() ->
|
||||
[
|
||||
with(queue_sups(), master),
|
||||
with(queue_sups(), slave),
|
||||
quorum_sups(),
|
||||
with(quorum_sups(), quorum),
|
||||
stream_server_sups(),
|
||||
stream_reader_sups(),
|
||||
with(quorum_sups(), stream),
|
||||
with(conn_sups(), reader),
|
||||
with(conn_sups(), writer),
|
||||
with(conn_sups(), channel),
|
||||
|
@ -292,6 +307,12 @@ conn_type(PDict) ->
|
|||
_ -> other
|
||||
end.
|
||||
|
||||
ra_type(PDict) ->
|
||||
case keyfind('$rabbit_vm_category', PDict) of
|
||||
{value, rabbit_stream_coordinator} -> stream;
|
||||
_ -> quorum
|
||||
end.
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
%% NB: this code is non-rabbit specific.
|
||||
|
|
|
@ -1,266 +0,0 @@
|
|||
%% This Source Code Form is subject to the terms of the Mozilla Public
|
||||
%% License, v. 2.0. If a copy of the MPL was not distributed with this
|
||||
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
|
||||
%%
|
||||
%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
|
||||
%%
|
||||
|
||||
%% Unconfirmed messages tracking.
|
||||
%%
|
||||
%% A message should be confirmed to the publisher only when all queues confirm.
|
||||
%%
|
||||
%% Messages are published to multiple queues while each queue may be
|
||||
%% represented by several processes (queue refs).
|
||||
%%
|
||||
%% Queue refs return confirmations, rejections, can fail or disconnect.
|
||||
%% If a queue ref fails, messgae should be rejected.
|
||||
%% If all queue refs for a queue disconnect (not fail) without confirmation,
|
||||
%% messge should be rejected.
|
||||
%%
|
||||
%% For simplicity, disconnects do not return a reject until all message refs
|
||||
%% confirm or disconnect.
|
||||
|
||||
-module(unconfirmed_messages).
|
||||
|
||||
-export([new/0,
|
||||
insert/5,
|
||||
confirm_multiple_msg_ref/4,
|
||||
forget_ref/2,
|
||||
|
||||
reject_msg/2,
|
||||
reject_all_for_queue/2,
|
||||
|
||||
smallest/1,
|
||||
size/1,
|
||||
is_empty/1]).
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
-export_type([?MODULE/0]).
|
||||
-define(SET_VALUE, []).
|
||||
|
||||
-type queue_ref() :: term().
|
||||
-type msg_id() :: term().
|
||||
-type queue_name() :: rabbit_amqqueue:name().
|
||||
-type exchange_name() :: rabbit_exchange:name().
|
||||
-type map_set(Type) :: #{Type => ?SET_VALUE}.
|
||||
|
||||
-record(msg_status, {
|
||||
%% a set of refs waiting for confirm
|
||||
refs = #{} :: map_set(queue_ref()),
|
||||
%% shows which queues had at least one confirmation
|
||||
queue_status = #{} :: #{queue_name() => confirmed | rejected},
|
||||
exchange :: exchange_name()
|
||||
}).
|
||||
|
||||
-record(unconfirmed, {
|
||||
%% needed to get unconfirmed cutoff
|
||||
ordered = gb_sets:new() :: gb_sets:set(msg_id()),
|
||||
%% contains message statuses of all message IDs
|
||||
index = #{} :: #{msg_id() => #msg_status{}},
|
||||
%% needed to look up message IDs for a queue ref
|
||||
reverse = #{} :: #{queue_ref() => #{msg_id() => ?SET_VALUE}}
|
||||
}).
|
||||
|
||||
-opaque ?MODULE() :: #unconfirmed{}.
|
||||
|
||||
%%----------------------------------------------------------------------------
|
||||
|
||||
-spec new() -> ?MODULE().
|
||||
new() -> #unconfirmed{}.
|
||||
|
||||
%% Insert an entry for the message ID. Fails if there already is
|
||||
%% an entry with the given ID.
|
||||
-spec insert(msg_id(), [queue_name()], [queue_ref()], exchange_name(), ?MODULE()) -> ?MODULE().
|
||||
insert(MsgId, QueueNames, QueueRefs, XName,
|
||||
#unconfirmed{ordered = Ordered,
|
||||
index = Index,
|
||||
reverse = Reverse} = UC) ->
|
||||
case maps:get(MsgId, Index, none) of
|
||||
none ->
|
||||
UC#unconfirmed{
|
||||
ordered = gb_sets:add(MsgId, Ordered),
|
||||
index =
|
||||
Index#{MsgId =>
|
||||
#msg_status{
|
||||
refs = maps:from_list([{QR, ?SET_VALUE} || QR <- QueueRefs]),
|
||||
queue_status = maps:from_list([{QN, rejected} || QN <- QueueNames]),
|
||||
exchange = XName}},
|
||||
reverse = lists:foldl(
|
||||
fun
|
||||
(Ref, R) ->
|
||||
case R of
|
||||
#{Ref := MsgIdsSet} ->
|
||||
R#{Ref => MsgIdsSet#{MsgId => ?SET_VALUE}};
|
||||
_ ->
|
||||
R#{Ref => #{MsgId => ?SET_VALUE}}
|
||||
end
|
||||
end,
|
||||
Reverse, QueueRefs)
|
||||
};
|
||||
_ ->
|
||||
error({message_already_exists, MsgId, QueueNames, QueueRefs, XName, UC})
|
||||
end.
|
||||
|
||||
%% Confirms messages on behalf of the given queue. If it was the last queue (ref)
|
||||
%% on the waiting list, returns message id and excahnge name
|
||||
%% and performs the necessary cleanup.
|
||||
-spec confirm_multiple_msg_ref(msg_id(), queue_name(), queue_ref(), ?MODULE()) ->
|
||||
{[{msg_id(), exchange_name()}], [{msg_id(), exchange_name()}], ?MODULE()}.
|
||||
confirm_multiple_msg_ref(MsgIds, QueueName, QueueRef,
|
||||
#unconfirmed{reverse = Reverse} = UC0) ->
|
||||
lists:foldl(
|
||||
fun(MsgId, {C, R, UC}) ->
|
||||
case remove_msg_ref(confirm, MsgId, QueueName, QueueRef, UC) of
|
||||
{{confirmed, V}, UC1} -> {[V | C], R, UC1};
|
||||
{{rejected, V}, UC1} -> {C, [V | R], UC1};
|
||||
{not_confirmed, UC1} -> {C, R, UC1}
|
||||
end
|
||||
end,
|
||||
{[], [], UC0#unconfirmed{reverse = remove_from_reverse(QueueRef, MsgIds, Reverse)}},
|
||||
MsgIds).
|
||||
|
||||
%% Removes all messages for a queue.
|
||||
%% Returns lists of confirmed and rejected messages.
|
||||
%%
|
||||
%% If there are no more refs left for the message, either
|
||||
%% 'confirmed' or 'rejected'.
|
||||
%% 'confirmed' is returned if all queues have confirmed the message.
|
||||
-spec forget_ref(queue_ref(), ?MODULE()) ->
|
||||
{Confirmed :: [{msg_id(), exchange_name()}],
|
||||
Rejected :: [{msg_id(), exchange_name()}],
|
||||
?MODULE()}.
|
||||
forget_ref(QueueRef, #unconfirmed{reverse = Reverse0} = UC0) ->
|
||||
MsgIds = maps:keys(maps:get(QueueRef, Reverse0, #{})),
|
||||
lists:foldl(fun(MsgId, {C, R, UC}) ->
|
||||
case remove_msg_ref(no_confirm, MsgId, ignore, QueueRef, UC) of
|
||||
{not_confirmed, UC1} -> {C, R, UC1};
|
||||
{{confirmed, V}, UC1} -> {[V | C], R, UC1};
|
||||
{{rejected, V}, UC1} -> {C, [V | R], UC1}
|
||||
end
|
||||
end,
|
||||
{[], [], UC0#unconfirmed{reverse = maps:remove(QueueRef, Reverse0)}},
|
||||
MsgIds).
|
||||
|
||||
%% Rejects a single message with the given ID.
|
||||
%% Returns 'rejected' if there was a message with
|
||||
%% such ID.
|
||||
-spec reject_msg(msg_id(), ?MODULE()) ->
|
||||
{{rejected, {msg_id(), exchange_name()}} | not_confirmed, ?MODULE()}.
|
||||
reject_msg(MsgId, #unconfirmed{ordered = Ordered, index = Index, reverse = Reverse} = UC) ->
|
||||
case maps:get(MsgId, Index, none) of
|
||||
none ->
|
||||
{not_confirmed, UC};
|
||||
#msg_status{exchange = XName,
|
||||
refs = Refs} ->
|
||||
{{rejected, {MsgId, XName}},
|
||||
UC#unconfirmed{ordered = gb_sets:del_element(MsgId, Ordered),
|
||||
index = maps:remove(MsgId, Index),
|
||||
reverse = remove_multiple_from_reverse(maps:keys(Refs), [MsgId], Reverse)}}
|
||||
end.
|
||||
|
||||
%% Rejects all pending messages for a queue.
|
||||
-spec reject_all_for_queue(queue_ref(), ?MODULE()) ->
|
||||
{Rejected :: [{msg_id(), exchange_name()}], ?MODULE()}.
|
||||
reject_all_for_queue(QueueRef, #unconfirmed{reverse = Reverse0} = UC0) ->
|
||||
MsgIds = maps:keys(maps:get(QueueRef, Reverse0, #{})),
|
||||
lists:foldl(
|
||||
fun(MsgId, {R, UC}) ->
|
||||
case reject_msg(MsgId, UC) of
|
||||
{not_confirmed, UC1} -> {R, UC1};
|
||||
{{rejected, V}, UC1} -> {[V | R], UC1}
|
||||
end
|
||||
end,
|
||||
{[], UC0#unconfirmed{reverse = maps:remove(QueueRef, Reverse0)}},
|
||||
MsgIds).
|
||||
|
||||
%% Returns a smallest message id.
|
||||
-spec smallest(?MODULE()) -> msg_id().
|
||||
smallest(#unconfirmed{ordered = Ordered}) ->
|
||||
gb_sets:smallest(Ordered).
|
||||
|
||||
-spec size(?MODULE()) -> msg_id().
|
||||
size(#unconfirmed{index = Index}) -> maps:size(Index).
|
||||
|
||||
-spec is_empty(?MODULE()) -> boolean().
|
||||
is_empty(#unconfirmed{index = Index, reverse = Reverse, ordered = Ordered} = UC) ->
|
||||
case maps:size(Index) == 0 of
|
||||
true ->
|
||||
%% Assertion
|
||||
case maps:size(Reverse) == gb_sets:size(Ordered)
|
||||
andalso
|
||||
maps:size(Reverse) == 0 of
|
||||
true -> ok;
|
||||
false -> error({size_mismatch, UC})
|
||||
end,
|
||||
true;
|
||||
_ ->
|
||||
false
|
||||
end.
|
||||
|
||||
-spec remove_from_reverse(queue_ref(), [msg_id()],
|
||||
#{queue_ref() => #{msg_id() => ?SET_VALUE}}) ->
|
||||
#{queue_ref() => #{msg_id() => ?SET_VALUE}}.
|
||||
remove_from_reverse(QueueRef, MsgIds, Reverse) when is_list(MsgIds) ->
|
||||
case maps:get(QueueRef, Reverse, none) of
|
||||
none ->
|
||||
Reverse;
|
||||
MsgIdsSet ->
|
||||
NewMsgIdsSet = maps:without(MsgIds, MsgIdsSet),
|
||||
case maps:size(NewMsgIdsSet) > 0 of
|
||||
true -> Reverse#{QueueRef => NewMsgIdsSet};
|
||||
false -> maps:remove(QueueRef, Reverse)
|
||||
end
|
||||
end.
|
||||
|
||||
-spec remove_multiple_from_reverse([queue_ref()], [msg_id()],
|
||||
#{queue_ref() => #{msg_id() => ?SET_VALUE}}) ->
|
||||
#{queue_ref() => #{msg_id() => ?SET_VALUE}}.
|
||||
remove_multiple_from_reverse(Refs, MsgIds, Reverse0) ->
|
||||
lists:foldl(
|
||||
fun(Ref, Reverse) ->
|
||||
remove_from_reverse(Ref, MsgIds, Reverse)
|
||||
end,
|
||||
Reverse0,
|
||||
Refs).
|
||||
|
||||
-spec remove_msg_ref(confirm | no_confirm, msg_id(), queue_name() | 'ignore', queue_ref(), ?MODULE()) ->
|
||||
{{confirmed | rejected, {msg_id(), exchange_name()}} | not_confirmed,
|
||||
?MODULE()}.
|
||||
remove_msg_ref(Confirm, MsgId, QueueName, QueueRef,
|
||||
#unconfirmed{ordered = Ordered, index = Index} = UC) ->
|
||||
case maps:get(MsgId, Index, none) of
|
||||
none ->
|
||||
{not_confirmed, UC};
|
||||
#msg_status{refs = #{QueueRef := ?SET_VALUE} = Refs,
|
||||
queue_status = QStatus,
|
||||
exchange = XName} = MsgStatus ->
|
||||
QStatus1 = case {Confirm, QueueName} of
|
||||
{no_confirm, _} -> QStatus;
|
||||
{_, ignore} -> QStatus;
|
||||
{confirm, _} -> QStatus#{QueueName => confirmed}
|
||||
end,
|
||||
case maps:size(Refs) == 1 of
|
||||
true ->
|
||||
{{confirm_status(QStatus1), {MsgId, XName}},
|
||||
UC#unconfirmed{
|
||||
ordered = gb_sets:del_element(MsgId, Ordered),
|
||||
index = maps:remove(MsgId, Index)}};
|
||||
false ->
|
||||
{not_confirmed,
|
||||
UC#unconfirmed{
|
||||
index = Index#{MsgId =>
|
||||
MsgStatus#msg_status{
|
||||
refs = maps:remove(QueueRef, Refs),
|
||||
queue_status = QStatus1}}}}
|
||||
end;
|
||||
_ -> {not_confirmed, UC}
|
||||
end.
|
||||
|
||||
-spec confirm_status(#{queue_name() => confirmed | rejected}) -> confirmed | rejected.
|
||||
confirm_status(QueueStatus) ->
|
||||
case lists:all(fun(confirmed) -> true; (_) -> false end,
|
||||
maps:values(QueueStatus)) of
|
||||
true -> confirmed;
|
||||
false -> rejected
|
||||
end.
|
|
@ -684,17 +684,17 @@ bq_variable_queue_delete_msg_store_files_callback1(Config) ->
|
|||
QPid = amqqueue:get_pid(Q),
|
||||
Payload = <<0:8388608>>, %% 1MB
|
||||
Count = 30,
|
||||
publish_and_confirm(Q, Payload, Count),
|
||||
QTState = publish_and_confirm(Q, Payload, Count),
|
||||
|
||||
rabbit_amqqueue:set_ram_duration_target(QPid, 0),
|
||||
|
||||
{ok, Limiter} = rabbit_limiter:start_link(no_id),
|
||||
|
||||
CountMinusOne = Count - 1,
|
||||
{ok, CountMinusOne, {QName, QPid, _AckTag, false, _Msg}} =
|
||||
rabbit_amqqueue:basic_get(Q, self(), true, Limiter,
|
||||
{ok, CountMinusOne, {QName, QPid, _AckTag, false, _Msg}, _} =
|
||||
rabbit_amqqueue:basic_get(Q, true, Limiter,
|
||||
<<"bq_variable_queue_delete_msg_store_files_callback1">>,
|
||||
#{}),
|
||||
QTState),
|
||||
{ok, CountMinusOne} = rabbit_amqqueue:purge(Q),
|
||||
|
||||
%% give the queue a second to receive the close_fds callback msg
|
||||
|
@ -713,8 +713,7 @@ bq_queue_recover1(Config) ->
|
|||
{new, Q} = rabbit_amqqueue:declare(QName0, true, false, [], none, <<"acting-user">>),
|
||||
QName = amqqueue:get_name(Q),
|
||||
QPid = amqqueue:get_pid(Q),
|
||||
publish_and_confirm(Q, <<>>, Count),
|
||||
|
||||
QT = publish_and_confirm(Q, <<>>, Count),
|
||||
SupPid = get_queue_sup_pid(Q),
|
||||
true = is_pid(SupPid),
|
||||
exit(SupPid, kill),
|
||||
|
@ -724,7 +723,7 @@ bq_queue_recover1(Config) ->
|
|||
after 10000 -> exit(timeout_waiting_for_queue_death)
|
||||
end,
|
||||
rabbit_amqqueue:stop(?VHOST),
|
||||
{Recovered, [], []} = rabbit_amqqueue:recover(?VHOST),
|
||||
{Recovered, []} = rabbit_amqqueue:recover(?VHOST),
|
||||
rabbit_amqqueue:start(Recovered),
|
||||
{ok, Limiter} = rabbit_limiter:start_link(no_id),
|
||||
rabbit_amqqueue:with_or_die(
|
||||
|
@ -732,9 +731,9 @@ bq_queue_recover1(Config) ->
|
|||
fun (Q1) when ?is_amqqueue(Q1) ->
|
||||
QPid1 = amqqueue:get_pid(Q1),
|
||||
CountMinusOne = Count - 1,
|
||||
{ok, CountMinusOne, {QName, QPid1, _AckTag, true, _Msg}} =
|
||||
rabbit_amqqueue:basic_get(Q1, self(), false, Limiter,
|
||||
<<"bq_queue_recover1">>, #{}),
|
||||
{ok, CountMinusOne, {QName, QPid1, _AckTag, true, _Msg}, _} =
|
||||
rabbit_amqqueue:basic_get(Q1, false, Limiter,
|
||||
<<"bq_queue_recover1">>, QT),
|
||||
exit(QPid1, shutdown),
|
||||
VQ1 = variable_queue_init(Q, true),
|
||||
{{_Msg1, true, _AckTag1}, VQ2} =
|
||||
|
@ -1366,25 +1365,34 @@ variable_queue_init(Q, Recover) ->
|
|||
|
||||
publish_and_confirm(Q, Payload, Count) ->
|
||||
Seqs = lists:seq(1, Count),
|
||||
[begin
|
||||
Msg = rabbit_basic:message(rabbit_misc:r(<<>>, exchange, <<>>),
|
||||
<<>>, #'P_basic'{delivery_mode = 2},
|
||||
Payload),
|
||||
Delivery = #delivery{mandatory = false, sender = self(),
|
||||
confirm = true, message = Msg, msg_seq_no = Seq,
|
||||
flow = noflow},
|
||||
_QPids = rabbit_amqqueue:deliver([Q], Delivery)
|
||||
end || Seq <- Seqs],
|
||||
wait_for_confirms(gb_sets:from_list(Seqs)).
|
||||
QTState0 = rabbit_queue_type:new(Q, rabbit_queue_type:init()),
|
||||
QTState =
|
||||
lists:foldl(
|
||||
fun (Seq, Acc0) ->
|
||||
Msg = rabbit_basic:message(rabbit_misc:r(<<>>, exchange, <<>>),
|
||||
<<>>, #'P_basic'{delivery_mode = 2},
|
||||
Payload),
|
||||
Delivery = #delivery{mandatory = false, sender = self(),
|
||||
confirm = true, message = Msg, msg_seq_no = Seq,
|
||||
flow = noflow},
|
||||
{ok, Acc, _Actions} = rabbit_queue_type:deliver([Q], Delivery, Acc0),
|
||||
Acc
|
||||
end, QTState0, Seqs),
|
||||
wait_for_confirms(gb_sets:from_list(Seqs)),
|
||||
QTState.
|
||||
|
||||
wait_for_confirms(Unconfirmed) ->
|
||||
case gb_sets:is_empty(Unconfirmed) of
|
||||
true -> ok;
|
||||
false -> receive {'$gen_cast', {confirm, Confirmed, _}} ->
|
||||
false -> receive {'$gen_cast',
|
||||
{queue_event, _QName,
|
||||
{confirm, Confirmed, _}}} ->
|
||||
wait_for_confirms(
|
||||
rabbit_misc:gb_sets_difference(
|
||||
Unconfirmed, gb_sets:from_list(Confirmed)))
|
||||
after ?TIMEOUT -> exit(timeout_waiting_for_confirm)
|
||||
after ?TIMEOUT ->
|
||||
flush(),
|
||||
exit(timeout_waiting_for_confirm)
|
||||
end
|
||||
end.
|
||||
|
||||
|
@ -1436,6 +1444,7 @@ variable_queue_publish(IsPersistent, Start, Count, PropFun, PayloadFun, VQ) ->
|
|||
variable_queue_wait_for_shuffling_end(
|
||||
lists:foldl(
|
||||
fun (N, VQN) ->
|
||||
|
||||
rabbit_variable_queue:publish(
|
||||
rabbit_basic:message(
|
||||
rabbit_misc:r(<<>>, exchange, <<>>),
|
||||
|
@ -1526,12 +1535,13 @@ variable_queue_status(VQ) ->
|
|||
variable_queue_wait_for_shuffling_end(VQ) ->
|
||||
case credit_flow:blocked() of
|
||||
false -> VQ;
|
||||
true -> receive
|
||||
{bump_credit, Msg} ->
|
||||
credit_flow:handle_bump_msg(Msg),
|
||||
variable_queue_wait_for_shuffling_end(
|
||||
rabbit_variable_queue:resume(VQ))
|
||||
end
|
||||
true ->
|
||||
receive
|
||||
{bump_credit, Msg} ->
|
||||
credit_flow:handle_bump_msg(Msg),
|
||||
variable_queue_wait_for_shuffling_end(
|
||||
rabbit_variable_queue:resume(VQ))
|
||||
end
|
||||
end.
|
||||
|
||||
msg2int(#basic_message{content = #content{ payload_fragments_rev = P}}) ->
|
||||
|
@ -1576,11 +1586,13 @@ variable_queue_with_holes(VQ0) ->
|
|||
fun (_, P) -> P end, fun erlang:term_to_binary/1, VQ7),
|
||||
%% assertions
|
||||
Status = variable_queue_status(VQ8),
|
||||
|
||||
vq_with_holes_assertions(VQ8, proplists:get_value(mode, Status)),
|
||||
Depth = Count + Interval,
|
||||
Depth = rabbit_variable_queue:depth(VQ8),
|
||||
Len = Depth - length(Subset3),
|
||||
Len = rabbit_variable_queue:len(VQ8),
|
||||
|
||||
{Seq3, Seq -- Seq3, lists:seq(Count + 1, Count + Interval), VQ8}.
|
||||
|
||||
vq_with_holes_assertions(VQ, default) ->
|
||||
|
@ -1604,3 +1616,12 @@ check_variable_queue_status(VQ0, Props) ->
|
|||
S = variable_queue_status(VQ1),
|
||||
assert_props(S, Props),
|
||||
VQ1.
|
||||
|
||||
flush() ->
|
||||
receive
|
||||
Any ->
|
||||
ct:pal("flush ~p", [Any]),
|
||||
flush()
|
||||
after 0 ->
|
||||
ok
|
||||
end.
|
||||
|
|
|
@ -71,11 +71,13 @@ notify_down_all(Config) ->
|
|||
RabbitCh = rabbit_ct_client_helpers:open_channel(Config, 0),
|
||||
HareCh = rabbit_ct_client_helpers:open_channel(Config, 1),
|
||||
|
||||
ct:pal("one"),
|
||||
%% success
|
||||
set_channel_operation_timeout_config(Config, 1000),
|
||||
configure_bq(Config),
|
||||
QCfg0 = qconfig(RabbitCh, <<"q0">>, <<"ex0">>, true, false),
|
||||
declare(QCfg0),
|
||||
ct:pal("two"),
|
||||
%% Testing rabbit_amqqueue:notify_down_all via rabbit_channel.
|
||||
%% Consumer count = 0 after correct channel termination and
|
||||
%% notification of queues via delegate:call/3
|
||||
|
@ -83,6 +85,7 @@ notify_down_all(Config) ->
|
|||
rabbit_ct_client_helpers:close_channel(RabbitCh),
|
||||
0 = length(get_consumers(Config, Rabbit, ?DEFAULT_VHOST)),
|
||||
false = is_process_alive(RabbitCh),
|
||||
ct:pal("three"),
|
||||
|
||||
%% fail
|
||||
set_channel_operation_timeout_config(Config, 10),
|
||||
|
|
|
@ -388,9 +388,12 @@ kill_the_queue(QueueName) ->
|
|||
[begin
|
||||
{ok, Q} = rabbit_amqqueue:lookup({resource, <<"/">>, queue, QueueName}),
|
||||
Pid = amqqueue:get_pid(Q),
|
||||
ct:pal("~w killed", [Pid]),
|
||||
timer:sleep(1),
|
||||
exit(Pid, kill)
|
||||
end
|
||||
|| _ <- lists:seq(1, 11)],
|
||||
|| _ <- lists:seq(1, 50)],
|
||||
timer:sleep(1),
|
||||
{ok, Q} = rabbit_amqqueue:lookup({resource, <<"/">>, queue, QueueName}),
|
||||
Pid = amqqueue:get_pid(Q),
|
||||
case is_process_alive(Pid) of
|
||||
|
@ -399,7 +402,11 @@ kill_the_queue(QueueName) ->
|
|||
false -> ok
|
||||
end.
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
flush() ->
|
||||
receive
|
||||
Any ->
|
||||
ct:pal("flush ~p", [Any]),
|
||||
flush()
|
||||
after 0 ->
|
||||
ok
|
||||
end.
|
||||
|
|
|
@ -1059,9 +1059,11 @@ dead_letter_headers_BCC(Config) ->
|
|||
?assertMatch({array, _}, rabbit_misc:table_lookup(Headers3, <<"x-death">>)).
|
||||
|
||||
|
||||
%% Three top-level headers are added for the very first dead-lettering event. They are
|
||||
%% Three top-level headers are added for the very first dead-lettering event.
|
||||
%% They are
|
||||
%% x-first-death-reason, x-first-death-queue, x-first-death-exchange
|
||||
%% They have the same values as the reason, queue, and exchange fields of the original
|
||||
%% They have the same values as the reason, queue, and exchange fields of the
|
||||
%% original
|
||||
%% dead lettering event. Once added, these headers are never modified.
|
||||
dead_letter_headers_first_death(Config) ->
|
||||
{_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
|
||||
|
|
|
@ -424,8 +424,7 @@ nodes_policy_should_pick_master_from_its_params(Config) ->
|
|||
nodename),
|
||||
|
||||
Ch = rabbit_ct_client_helpers:open_channel(Config, A),
|
||||
?assertEqual(true, apply_policy_to_declared_queue(Config, Ch, [A],
|
||||
[all])),
|
||||
?assertEqual(true, apply_policy_to_declared_queue(Config, Ch, [A], [all])),
|
||||
%% --> Master: A
|
||||
%% Slaves: [B, C] or [C, B]
|
||||
SSPids = ?awaitMatch(SSPids when is_list(SSPids),
|
||||
|
@ -450,7 +449,7 @@ nodes_policy_should_pick_master_from_its_params(Config) ->
|
|||
%% should instead use an existing synchronised mirror as the new master,
|
||||
%% even though that isn't in the policy.
|
||||
?assertEqual(true, apply_policy_to_declared_queue(Config, Ch, [A],
|
||||
[{nodes, [LastSlave, A]}])),
|
||||
[{nodes, [LastSlave, A]}])),
|
||||
%% --> Master: B or C (same as previous policy)
|
||||
%% Slaves: [A]
|
||||
|
||||
|
@ -931,6 +930,7 @@ apply_in_parallel(Config, Nodes, Policies) ->
|
|||
Self = self(),
|
||||
[spawn_link(fun() ->
|
||||
[begin
|
||||
|
||||
apply_policy(Config, N, Policy)
|
||||
end || Policy <- Policies],
|
||||
Self ! parallel_task_done
|
||||
|
@ -969,7 +969,7 @@ wait_for_last_policy(QueueName, NodeA, TestedPolicies, Tries) ->
|
|||
%% Let's wait a bit longer.
|
||||
timer:sleep(1000),
|
||||
wait_for_last_policy(QueueName, NodeA, TestedPolicies, Tries - 1);
|
||||
FinalInfo ->
|
||||
{ok, FinalInfo} ->
|
||||
%% The last policy is the final state
|
||||
LastPolicy = lists:last(TestedPolicies),
|
||||
case verify_policy(LastPolicy, FinalInfo) of
|
||||
|
|
|
@ -57,7 +57,8 @@ groups() ->
|
|||
trigger_message_store_compaction]},
|
||||
{quorum_queue, [parallel], AllTests ++ [delete_immediately_by_pid_fails]},
|
||||
{quorum_queue_in_memory_limit, [parallel], AllTests ++ [delete_immediately_by_pid_fails]},
|
||||
{quorum_queue_in_memory_bytes, [parallel], AllTests ++ [delete_immediately_by_pid_fails]}
|
||||
{quorum_queue_in_memory_bytes, [parallel], AllTests ++ [delete_immediately_by_pid_fails]},
|
||||
{stream_queue, [parallel], AllTests}
|
||||
]}
|
||||
].
|
||||
|
||||
|
@ -122,13 +123,24 @@ init_per_group(mirrored_queue, Config) ->
|
|||
{queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]},
|
||||
{queue_durable, true}]),
|
||||
rabbit_ct_helpers:run_steps(Config1, []);
|
||||
init_per_group(stream_queue, Config) ->
|
||||
case rabbit_ct_broker_helpers:enable_feature_flag(Config, stream_queue) of
|
||||
ok ->
|
||||
rabbit_ct_helpers:set_config(
|
||||
Config,
|
||||
[{queue_args, [{<<"x-queue-type">>, longstr, <<"stream">>}]},
|
||||
{queue_durable, true}]);
|
||||
Skip ->
|
||||
Skip
|
||||
end;
|
||||
init_per_group(Group, Config0) ->
|
||||
case lists:member({group, Group}, all()) of
|
||||
true ->
|
||||
ClusterSize = 3,
|
||||
Config = rabbit_ct_helpers:merge_app_env(
|
||||
Config0, {rabbit, [{channel_tick_interval, 1000},
|
||||
{quorum_tick_interval, 1000}]}),
|
||||
{quorum_tick_interval, 1000},
|
||||
{stream_tick_interval, 1000}]}),
|
||||
Config1 = rabbit_ct_helpers:set_config(
|
||||
Config, [ {rmq_nodename_suffix, Group},
|
||||
{rmq_nodes_count, ClusterSize}
|
||||
|
@ -514,6 +526,11 @@ basic_cancel(Config) ->
|
|||
publish(Ch, QName, [<<"msg1">>]),
|
||||
wait_for_messages(Config, [[QName, <<"1">>, <<"1">>, <<"0">>]]),
|
||||
CTag = atom_to_binary(?FUNCTION_NAME, utf8),
|
||||
|
||||
%% Let's set consumer prefetch so it works with stream queues
|
||||
?assertMatch(#'basic.qos_ok'{},
|
||||
amqp_channel:call(Ch, #'basic.qos'{global = false,
|
||||
prefetch_count = 1})),
|
||||
subscribe(Ch, QName, false, CTag),
|
||||
receive
|
||||
{#'basic.deliver'{delivery_tag = DeliveryTag}, _} ->
|
||||
|
|
|
@ -0,0 +1,234 @@
|
|||
-module(queue_type_SUITE).
|
||||
|
||||
-compile(export_all).
|
||||
|
||||
-export([
|
||||
]).
|
||||
|
||||
-include_lib("common_test/include/ct.hrl").
|
||||
-include_lib("eunit/include/eunit.hrl").
|
||||
-include_lib("amqp_client/include/amqp_client.hrl").
|
||||
|
||||
%%%===================================================================
|
||||
%%% Common Test callbacks
|
||||
%%%===================================================================
|
||||
|
||||
all() ->
|
||||
[
|
||||
{group, classic},
|
||||
{group, quorum}
|
||||
].
|
||||
|
||||
|
||||
all_tests() ->
|
||||
[
|
||||
smoke
|
||||
].
|
||||
|
||||
groups() ->
|
||||
[
|
||||
{classic, [], all_tests()},
|
||||
{quorum, [], all_tests()}
|
||||
].
|
||||
|
||||
init_per_suite(Config0) ->
|
||||
rabbit_ct_helpers:log_environment(),
|
||||
Config = rabbit_ct_helpers:merge_app_env(
|
||||
Config0, {rabbit, [{quorum_tick_interval, 1000}]}),
|
||||
rabbit_ct_helpers:run_setup_steps(Config).
|
||||
|
||||
end_per_suite(Config) ->
|
||||
rabbit_ct_helpers:run_teardown_steps(Config),
|
||||
ok.
|
||||
|
||||
init_per_group(Group, Config) ->
|
||||
ClusterSize = 3,
|
||||
Config1 = rabbit_ct_helpers:set_config(Config,
|
||||
[{rmq_nodes_count, ClusterSize},
|
||||
{rmq_nodename_suffix, Group},
|
||||
{tcp_ports_base}]),
|
||||
Config1b = rabbit_ct_helpers:set_config(Config1,
|
||||
[{queue_type, atom_to_binary(Group, utf8)},
|
||||
{net_ticktime, 10}]),
|
||||
Config2 = rabbit_ct_helpers:run_steps(Config1b,
|
||||
[fun merge_app_env/1 ] ++
|
||||
rabbit_ct_broker_helpers:setup_steps()),
|
||||
Config3 =
|
||||
case rabbit_ct_broker_helpers:enable_feature_flag(Config2, quorum_queue) of
|
||||
ok ->
|
||||
ok = rabbit_ct_broker_helpers:rpc(
|
||||
Config2, 0, application, set_env,
|
||||
[rabbit, channel_tick_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;
|
||||
Skip ->
|
||||
end_per_group(Group, Config2),
|
||||
Skip
|
||||
end,
|
||||
rabbit_ct_broker_helpers:set_policy(
|
||||
Config3, 0,
|
||||
<<"ha-policy">>, <<".*">>, <<"queues">>,
|
||||
[{<<"ha-mode">>, <<"all">>}]),
|
||||
Config3.
|
||||
|
||||
merge_app_env(Config) ->
|
||||
rabbit_ct_helpers:merge_app_env(
|
||||
rabbit_ct_helpers:merge_app_env(Config,
|
||||
{rabbit,
|
||||
[{core_metrics_gc_interval, 100},
|
||||
{log, [{file, [{level, debug}]}]}]}),
|
||||
{ra, [{min_wal_roll_over_interval, 30000}]}).
|
||||
|
||||
end_per_group(_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},
|
||||
{alt_queue_name, <<Q/binary, "_alt">>}
|
||||
]),
|
||||
rabbit_ct_helpers:run_steps(Config2,
|
||||
rabbit_ct_client_helpers:setup_steps()).
|
||||
|
||||
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).
|
||||
|
||||
%%%===================================================================
|
||||
%%% Test cases
|
||||
%%%===================================================================
|
||||
|
||||
smoke(Config) ->
|
||||
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
|
||||
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
|
||||
QName = ?config(queue_name, Config),
|
||||
?assertEqual({'queue.declare_ok', QName, 0, 0},
|
||||
declare(Ch, QName, [{<<"x-queue-type">>, longstr,
|
||||
?config(queue_type, Config)}])),
|
||||
#'confirm.select_ok'{} = amqp_channel:call(Ch, #'confirm.select'{}),
|
||||
amqp_channel:register_confirm_handler(Ch, self()),
|
||||
publish(Ch, QName, <<"msg1">>),
|
||||
ct:pal("waiting for confirms from ~s", [QName]),
|
||||
ok = receive
|
||||
#'basic.ack'{} -> ok;
|
||||
#'basic.nack'{} -> fail
|
||||
after 2500 ->
|
||||
flush(),
|
||||
exit(confirm_timeout)
|
||||
end,
|
||||
DTag = basic_get(Ch, QName),
|
||||
|
||||
basic_ack(Ch, DTag),
|
||||
basic_get_empty(Ch, QName),
|
||||
|
||||
%% consume
|
||||
publish(Ch, QName, <<"msg2">>),
|
||||
ConsumerTag1 = <<"ctag1">>,
|
||||
ok = subscribe(Ch, QName, ConsumerTag1),
|
||||
%% receive and ack
|
||||
receive
|
||||
{#'basic.deliver'{delivery_tag = DeliveryTag,
|
||||
redelivered = false},
|
||||
#amqp_msg{}} ->
|
||||
basic_ack(Ch, DeliveryTag)
|
||||
after 5000 ->
|
||||
flush(),
|
||||
exit(basic_deliver_timeout)
|
||||
end,
|
||||
basic_cancel(Ch, ConsumerTag1),
|
||||
|
||||
%% assert empty
|
||||
basic_get_empty(Ch, QName),
|
||||
|
||||
%% consume and nack
|
||||
ConsumerTag2 = <<"ctag2">>,
|
||||
ok = subscribe(Ch, QName, ConsumerTag2),
|
||||
publish(Ch, QName, <<"msg3">>),
|
||||
receive
|
||||
{#'basic.deliver'{delivery_tag = T,
|
||||
redelivered = false},
|
||||
#amqp_msg{}} ->
|
||||
basic_cancel(Ch, ConsumerTag2),
|
||||
basic_nack(Ch, T)
|
||||
after 5000 ->
|
||||
exit(basic_deliver_timeout)
|
||||
end,
|
||||
%% get and ack
|
||||
basic_ack(Ch, basic_get(Ch, QName)),
|
||||
ok.
|
||||
|
||||
%% Utility
|
||||
delete_queues() ->
|
||||
[rabbit_amqqueue:delete(Q, false, false, <<"dummy">>)
|
||||
|| Q <- rabbit_amqqueue:list()].
|
||||
|
||||
declare(Ch, Q, Args) ->
|
||||
amqp_channel:call(Ch, #'queue.declare'{queue = Q,
|
||||
durable = true,
|
||||
auto_delete = false,
|
||||
arguments = Args}).
|
||||
|
||||
publish(Ch, Queue, Msg) ->
|
||||
ok = amqp_channel:cast(Ch,
|
||||
#'basic.publish'{routing_key = Queue},
|
||||
#amqp_msg{props = #'P_basic'{delivery_mode = 2},
|
||||
payload = Msg}).
|
||||
|
||||
basic_get(Ch, Queue) ->
|
||||
{GetOk, _} = Reply = amqp_channel:call(Ch, #'basic.get'{queue = Queue,
|
||||
no_ack = false}),
|
||||
?assertMatch({#'basic.get_ok'{}, #amqp_msg{}}, Reply),
|
||||
GetOk#'basic.get_ok'.delivery_tag.
|
||||
|
||||
basic_get_empty(Ch, Queue) ->
|
||||
?assertMatch(#'basic.get_empty'{},
|
||||
amqp_channel:call(Ch, #'basic.get'{queue = Queue,
|
||||
no_ack = false})).
|
||||
|
||||
subscribe(Ch, Queue, CTag) ->
|
||||
amqp_channel:subscribe(Ch, #'basic.consume'{queue = Queue,
|
||||
no_ack = false,
|
||||
consumer_tag = CTag},
|
||||
self()),
|
||||
receive
|
||||
#'basic.consume_ok'{consumer_tag = CTag} ->
|
||||
ok
|
||||
after 5000 ->
|
||||
exit(basic_consume_timeout)
|
||||
end.
|
||||
|
||||
basic_ack(Ch, DTag) ->
|
||||
amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DTag,
|
||||
multiple = false}).
|
||||
|
||||
basic_cancel(Ch, CTag) ->
|
||||
#'basic.cancel_ok'{} =
|
||||
amqp_channel:call(Ch, #'basic.cancel'{consumer_tag = CTag}).
|
||||
|
||||
basic_nack(Ch, DTag) ->
|
||||
amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DTag,
|
||||
requeue = true,
|
||||
multiple = false}).
|
||||
|
||||
flush() ->
|
||||
receive
|
||||
Any ->
|
||||
ct:pal("flush ~p", [Any]),
|
||||
flush()
|
||||
after 0 ->
|
||||
ok
|
||||
end.
|
|
@ -383,13 +383,19 @@ start_queue(Config) ->
|
|||
|
||||
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
|
||||
LQ = ?config(queue_name, Config),
|
||||
%% The stream coordinator is also a ra process, we need to ensure the quorum tests
|
||||
%% are not affected by any other ra cluster that could be added in the future
|
||||
Children = length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup])),
|
||||
|
||||
?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_sup])),
|
||||
Expected = Children + 1,
|
||||
?assertMatch(Expected,
|
||||
length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup]))),
|
||||
|
||||
%% Test declare an existing queue
|
||||
?assertEqual({'queue.declare_ok', LQ, 0, 0},
|
||||
|
@ -405,7 +411,8 @@ start_queue(Config) ->
|
|||
%% 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_sup])).
|
||||
?assertMatch(Expected,
|
||||
length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup]))).
|
||||
|
||||
start_queue_concurrent(Config) ->
|
||||
Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
|
||||
|
@ -463,6 +470,10 @@ quorum_cluster_size_x(Config, Max, Expected) ->
|
|||
stop_queue(Config) ->
|
||||
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
|
||||
|
||||
%% The stream coordinator is also a ra process, we need to ensure the quorum tests
|
||||
%% are not affected by any other ra cluster that could be added in the future
|
||||
Children = length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup])),
|
||||
|
||||
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
|
||||
LQ = ?config(queue_name, Config),
|
||||
?assertEqual({'queue.declare_ok', LQ, 0, 0},
|
||||
|
@ -471,13 +482,15 @@ stop_queue(Config) ->
|
|||
%% 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_sup])),
|
||||
Expected = Children + 1,
|
||||
?assertMatch(Expected,
|
||||
length(rpc:call(Server, supervisor, which_children, [ra_server_sup_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_sup])
|
||||
Children == length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup]))
|
||||
end),
|
||||
?assertMatch({ra, _, _}, lists:keyfind(ra, 1,
|
||||
rpc:call(Server, application, which_applications, []))).
|
||||
|
@ -485,6 +498,10 @@ stop_queue(Config) ->
|
|||
restart_queue(Config) ->
|
||||
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
|
||||
|
||||
%% The stream coordinator is also a ra process, we need to ensure the quorum tests
|
||||
%% are not affected by any other ra cluster that could be added in the future
|
||||
Children = length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup])),
|
||||
|
||||
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
|
||||
LQ = ?config(queue_name, Config),
|
||||
?assertEqual({'queue.declare_ok', LQ, 0, 0},
|
||||
|
@ -496,7 +513,9 @@ restart_queue(Config) ->
|
|||
%% 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_sup])).
|
||||
Expected = Children + 1,
|
||||
?assertMatch(Expected,
|
||||
length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup]))).
|
||||
|
||||
idempotent_recover(Config) ->
|
||||
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
|
||||
|
@ -554,6 +573,10 @@ restart_all_types(Config) ->
|
|||
%% ensure there are no regressions
|
||||
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
|
||||
|
||||
%% The stream coordinator is also a ra process, we need to ensure the quorum tests
|
||||
%% are not affected by any other ra cluster that could be added in the future
|
||||
Children = rpc:call(Server, supervisor, which_children, [ra_server_sup_sup]),
|
||||
|
||||
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
|
||||
QQ1 = <<"restart_all_types-qq1">>,
|
||||
?assertEqual({'queue.declare_ok', QQ1, 0, 0},
|
||||
|
@ -575,7 +598,9 @@ restart_all_types(Config) ->
|
|||
%% 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_sup])),
|
||||
Expected = length(Children) + 2,
|
||||
Got = length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup])),
|
||||
?assertMatch(Expected, Got),
|
||||
%% Check the classic queues restarted correctly
|
||||
Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server),
|
||||
{#'basic.get_ok'{}, #amqp_msg{}} =
|
||||
|
@ -592,6 +617,10 @@ stop_start_rabbit_app(Config) ->
|
|||
%% classic) to ensure there are no regressions
|
||||
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
|
||||
|
||||
%% The stream coordinator is also a ra process, we need to ensure the quorum tests
|
||||
%% are not affected by any other ra cluster that could be added in the future
|
||||
Children = length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup])),
|
||||
|
||||
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
|
||||
QQ1 = <<"stop_start_rabbit_app-qq">>,
|
||||
?assertEqual({'queue.declare_ok', QQ1, 0, 0},
|
||||
|
@ -617,7 +646,9 @@ stop_start_rabbit_app(Config) ->
|
|||
%% 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_sup])),
|
||||
Expected = Children + 2,
|
||||
?assertMatch(Expected,
|
||||
length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup]))),
|
||||
%% Check the classic queues restarted correctly
|
||||
Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server),
|
||||
{#'basic.get_ok'{}, #amqp_msg{}} =
|
||||
|
@ -935,6 +966,10 @@ cleanup_queue_state_on_channel_after_publish(Config) ->
|
|||
%% to verify that the cleanup is propagated through channels
|
||||
[Server | _] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
|
||||
|
||||
%% The stream coordinator is also a ra process, we need to ensure the quorum tests
|
||||
%% are not affected by any other ra cluster that could be added in the future
|
||||
Children = length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup])),
|
||||
|
||||
Ch1 = rabbit_ct_client_helpers:open_channel(Config, Server),
|
||||
Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server),
|
||||
QQ = ?config(queue_name, Config),
|
||||
|
@ -955,18 +990,22 @@ cleanup_queue_state_on_channel_after_publish(Config) ->
|
|||
?assertMatch(#'queue.delete_ok'{},
|
||||
amqp_channel:call(Ch1, #'queue.delete'{queue = QQ})),
|
||||
wait_until(fun() ->
|
||||
[] == rpc:call(Server, supervisor, which_children,
|
||||
[ra_server_sup_sup])
|
||||
Children == length(rpc:call(Server, supervisor, which_children,
|
||||
[ra_server_sup_sup]))
|
||||
end),
|
||||
%% Check that all queue states have been cleaned
|
||||
wait_for_cleanup(Server, NCh1, 0),
|
||||
wait_for_cleanup(Server, NCh2, 0).
|
||||
wait_for_cleanup(Server, NCh2, 0),
|
||||
wait_for_cleanup(Server, NCh1, 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),
|
||||
|
||||
%% The stream coordinator is also a ra process, we need to ensure the quorum tests
|
||||
%% are not affected by any other ra cluster that could be added in the future
|
||||
Children = length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup])),
|
||||
|
||||
Ch1 = rabbit_ct_client_helpers:open_channel(Config, Server),
|
||||
Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server),
|
||||
QQ = ?config(queue_name, Config),
|
||||
|
@ -993,7 +1032,7 @@ cleanup_queue_state_on_channel_after_subscribe(Config) ->
|
|||
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_sup])
|
||||
Children == length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup]))
|
||||
end),
|
||||
%% Check that all queue states have been cleaned
|
||||
wait_for_cleanup(Server, NCh1, 0),
|
||||
|
@ -1596,8 +1635,8 @@ cleanup_data_dir(Config) ->
|
|||
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])),
|
||||
timer:sleep(100),
|
||||
|
||||
[{_, UId1}] = rpc:call(Server1, ra_directory, list_registered, []),
|
||||
[{_, UId2}] = rpc:call(Server2, ra_directory, list_registered, []),
|
||||
UId1 = proplists:get_value(ra_name(QQ), rpc:call(Server1, ra_directory, list_registered, [])),
|
||||
UId2 = proplists:get_value(ra_name(QQ), rpc:call(Server2, ra_directory, list_registered, [])),
|
||||
DataDir1 = rpc:call(Server1, ra_env, server_data_dir, [UId1]),
|
||||
DataDir2 = rpc:call(Server2, ra_env, server_data_dir, [UId2]),
|
||||
?assert(filelib:is_dir(DataDir1)),
|
||||
|
@ -1748,6 +1787,11 @@ reconnect_consumer_and_wait_channel_down(Config) ->
|
|||
delete_immediately_by_resource(Config) ->
|
||||
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
|
||||
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
|
||||
|
||||
%% The stream coordinator is also a ra process, we need to ensure the quorum tests
|
||||
%% are not affected by any other ra cluster that could be added in the future
|
||||
Children = length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup])),
|
||||
|
||||
QQ = ?config(queue_name, Config),
|
||||
?assertEqual({'queue.declare_ok', QQ, 0, 0},
|
||||
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])),
|
||||
|
@ -1756,7 +1800,7 @@ delete_immediately_by_resource(Config) ->
|
|||
|
||||
%% Check that the application and process are down
|
||||
wait_until(fun() ->
|
||||
[] == rpc:call(Server, supervisor, which_children, [ra_server_sup_sup])
|
||||
Children == length(rpc:call(Server, supervisor, which_children, [ra_server_sup_sup]))
|
||||
end),
|
||||
?assertMatch({ra, _, _}, lists:keyfind(ra, 1,
|
||||
rpc:call(Server, application, which_applications, []))).
|
||||
|
@ -1784,6 +1828,8 @@ subscribe_redelivery_count(Config) ->
|
|||
amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag,
|
||||
multiple = false,
|
||||
requeue = true})
|
||||
after 5000 ->
|
||||
exit(basic_deliver_timeout)
|
||||
end,
|
||||
|
||||
receive
|
||||
|
@ -1794,6 +1840,8 @@ subscribe_redelivery_count(Config) ->
|
|||
amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = DeliveryTag1,
|
||||
multiple = false,
|
||||
requeue = true})
|
||||
after 5000 ->
|
||||
exit(basic_deliver_timeout_2)
|
||||
end,
|
||||
|
||||
receive
|
||||
|
@ -1803,8 +1851,13 @@ subscribe_redelivery_count(Config) ->
|
|||
?assertMatch({DCHeader, _, 2}, rabbit_basic:header(DCHeader, H2)),
|
||||
amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DeliveryTag2,
|
||||
multiple = false}),
|
||||
ct:pal("wait_for_messages_ready", []),
|
||||
wait_for_messages_ready(Servers, RaName, 0),
|
||||
ct:pal("wait_for_messages_pending_ack", []),
|
||||
wait_for_messages_pending_ack(Servers, RaName, 0)
|
||||
after 5000 ->
|
||||
flush(500),
|
||||
exit(basic_deliver_timeout_3)
|
||||
end.
|
||||
|
||||
subscribe_redelivery_limit(Config) ->
|
||||
|
|
|
@ -28,15 +28,10 @@ wait_for_messages_total(Servers, QName, Total) ->
|
|||
|
||||
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))]);
|
||||
?assertEqual(Msgs, [Number || _ <- lists:seq(1, length(Servers))]);
|
||||
wait_for_messages(Servers, QName, Number, Fun, N) ->
|
||||
Msgs = dirty_query(Servers, QName, Fun),
|
||||
ct:pal("Got messages ~p", [Msgs]),
|
||||
ct:pal("Got messages ~p ~p", [QName, Msgs]),
|
||||
%% hack to allow the check to succeed in mixed versions clusters if at
|
||||
%% least one node matches the criteria rather than all nodes for
|
||||
F = case is_mixed_versions() of
|
||||
|
|
|
@ -0,0 +1,154 @@
|
|||
-module(rabbit_confirms_SUITE).
|
||||
|
||||
-compile(export_all).
|
||||
|
||||
-export([
|
||||
]).
|
||||
|
||||
-include_lib("common_test/include/ct.hrl").
|
||||
-include_lib("eunit/include/eunit.hrl").
|
||||
|
||||
%%%===================================================================
|
||||
%%% Common Test callbacks
|
||||
%%%===================================================================
|
||||
|
||||
all() ->
|
||||
[
|
||||
{group, tests}
|
||||
].
|
||||
|
||||
|
||||
all_tests() ->
|
||||
[
|
||||
confirm,
|
||||
reject,
|
||||
remove_queue
|
||||
].
|
||||
|
||||
groups() ->
|
||||
[
|
||||
{tests, [], all_tests()}
|
||||
].
|
||||
|
||||
init_per_suite(Config) ->
|
||||
Config.
|
||||
|
||||
end_per_suite(_Config) ->
|
||||
ok.
|
||||
|
||||
init_per_group(_Group, Config) ->
|
||||
Config.
|
||||
|
||||
end_per_group(_Group, _Config) ->
|
||||
ok.
|
||||
|
||||
init_per_testcase(_TestCase, Config) ->
|
||||
Config.
|
||||
|
||||
end_per_testcase(_TestCase, _Config) ->
|
||||
ok.
|
||||
|
||||
%%%===================================================================
|
||||
%%% Test cases
|
||||
%%%===================================================================
|
||||
|
||||
confirm(_Config) ->
|
||||
XName = rabbit_misc:r(<<"/">>, exchange, <<"X">>),
|
||||
QName = rabbit_misc:r(<<"/">>, queue, <<"Q">>),
|
||||
QName2 = rabbit_misc:r(<<"/">>, queue, <<"Q2">>),
|
||||
U0 = rabbit_confirms:init(),
|
||||
?assertEqual(0, rabbit_confirms:size(U0)),
|
||||
?assertEqual(undefined, rabbit_confirms:smallest(U0)),
|
||||
?assertEqual(true, rabbit_confirms:is_empty(U0)),
|
||||
|
||||
U1 = rabbit_confirms:insert(1, [QName], XName, U0),
|
||||
?assertEqual(1, rabbit_confirms:size(U1)),
|
||||
?assertEqual(1, rabbit_confirms:smallest(U1)),
|
||||
?assertEqual(false, rabbit_confirms:is_empty(U1)),
|
||||
|
||||
{[{1, XName}], U2} = rabbit_confirms:confirm([1], QName, U1),
|
||||
?assertEqual(0, rabbit_confirms:size(U2)),
|
||||
?assertEqual(undefined, rabbit_confirms:smallest(U2)),
|
||||
?assertEqual(true, rabbit_confirms:is_empty(U2)),
|
||||
|
||||
U3 = rabbit_confirms:insert(2, [QName], XName, U1),
|
||||
?assertEqual(2, rabbit_confirms:size(U3)),
|
||||
?assertEqual(1, rabbit_confirms:smallest(U3)),
|
||||
?assertEqual(false, rabbit_confirms:is_empty(U3)),
|
||||
|
||||
{[{1, XName}], U4} = rabbit_confirms:confirm([1], QName, U3),
|
||||
?assertEqual(1, rabbit_confirms:size(U4)),
|
||||
?assertEqual(2, rabbit_confirms:smallest(U4)),
|
||||
?assertEqual(false, rabbit_confirms:is_empty(U4)),
|
||||
|
||||
U5 = rabbit_confirms:insert(2, [QName, QName2], XName, U1),
|
||||
?assertEqual(2, rabbit_confirms:size(U5)),
|
||||
?assertEqual(1, rabbit_confirms:smallest(U5)),
|
||||
?assertEqual(false, rabbit_confirms:is_empty(U5)),
|
||||
|
||||
{[{1, XName}], U6} = rabbit_confirms:confirm([1, 2], QName, U5),
|
||||
?assertEqual(2, rabbit_confirms:smallest(U6)),
|
||||
|
||||
{[{2, XName}], U7} = rabbit_confirms:confirm([2], QName2, U6),
|
||||
?assertEqual(0, rabbit_confirms:size(U7)),
|
||||
?assertEqual(undefined, rabbit_confirms:smallest(U7)),
|
||||
|
||||
|
||||
U8 = rabbit_confirms:insert(2, [QName], XName, U1),
|
||||
{[{1, XName}, {2, XName}], _U9} = rabbit_confirms:confirm([1, 2], QName, U8),
|
||||
ok.
|
||||
|
||||
|
||||
reject(_Config) ->
|
||||
XName = rabbit_misc:r(<<"/">>, exchange, <<"X">>),
|
||||
QName = rabbit_misc:r(<<"/">>, queue, <<"Q">>),
|
||||
QName2 = rabbit_misc:r(<<"/">>, queue, <<"Q2">>),
|
||||
U0 = rabbit_confirms:init(),
|
||||
?assertEqual(0, rabbit_confirms:size(U0)),
|
||||
?assertEqual(undefined, rabbit_confirms:smallest(U0)),
|
||||
?assertEqual(true, rabbit_confirms:is_empty(U0)),
|
||||
|
||||
U1 = rabbit_confirms:insert(1, [QName], XName, U0),
|
||||
|
||||
{ok, {1, XName}, U2} = rabbit_confirms:reject(1, U1),
|
||||
{error, not_found} = rabbit_confirms:reject(1, U2),
|
||||
?assertEqual(0, rabbit_confirms:size(U2)),
|
||||
?assertEqual(undefined, rabbit_confirms:smallest(U2)),
|
||||
|
||||
U3 = rabbit_confirms:insert(2, [QName, QName2], XName, U1),
|
||||
|
||||
{ok, {1, XName}, U4} = rabbit_confirms:reject(1, U3),
|
||||
{error, not_found} = rabbit_confirms:reject(1, U4),
|
||||
?assertEqual(1, rabbit_confirms:size(U4)),
|
||||
?assertEqual(2, rabbit_confirms:smallest(U4)),
|
||||
|
||||
{ok, {2, XName}, U5} = rabbit_confirms:reject(2, U3),
|
||||
{error, not_found} = rabbit_confirms:reject(2, U5),
|
||||
?assertEqual(1, rabbit_confirms:size(U5)),
|
||||
?assertEqual(1, rabbit_confirms:smallest(U5)),
|
||||
|
||||
ok.
|
||||
|
||||
remove_queue(_Config) ->
|
||||
XName = rabbit_misc:r(<<"/">>, exchange, <<"X">>),
|
||||
QName = rabbit_misc:r(<<"/">>, queue, <<"Q">>),
|
||||
QName2 = rabbit_misc:r(<<"/">>, queue, <<"Q2">>),
|
||||
U0 = rabbit_confirms:init(),
|
||||
|
||||
U1 = rabbit_confirms:insert(1, [QName, QName2], XName, U0),
|
||||
U2 = rabbit_confirms:insert(2, [QName2], XName, U1),
|
||||
{[{2, XName}], U3} = rabbit_confirms:remove_queue(QName2, U2),
|
||||
?assertEqual(1, rabbit_confirms:size(U3)),
|
||||
?assertEqual(1, rabbit_confirms:smallest(U3)),
|
||||
{[{1, XName}], U4} = rabbit_confirms:remove_queue(QName, U3),
|
||||
?assertEqual(0, rabbit_confirms:size(U4)),
|
||||
?assertEqual(undefined, rabbit_confirms:smallest(U4)),
|
||||
|
||||
U5 = rabbit_confirms:insert(1, [QName], XName, U0),
|
||||
U6 = rabbit_confirms:insert(2, [QName], XName, U5),
|
||||
{[{1, XName}, {2, XName}], _U} = rabbit_confirms:remove_queue(QName, U6),
|
||||
|
||||
ok.
|
||||
|
||||
|
||||
%% Utility
|
|
@ -674,7 +674,7 @@ single_active_consumer_basic_get_test(_) ->
|
|||
?assertEqual(single_active, State0#rabbit_fifo.cfg#cfg.consumer_strategy),
|
||||
?assertEqual(0, map_size(State0#rabbit_fifo.consumers)),
|
||||
{State1, _} = enq(1, 1, first, State0),
|
||||
{_State, {error, unsupported}} =
|
||||
{_State, {error, {unsupported, single_active_consumer}}} =
|
||||
apply(meta(2), rabbit_fifo:make_checkout(Cid, {dequeue, unsettled}, #{}),
|
||||
State1),
|
||||
ok.
|
||||
|
|
|
@ -86,7 +86,7 @@ basics(Config) ->
|
|||
CustomerTag = UId,
|
||||
ok = start_cluster(ClusterName, [ServerId]),
|
||||
FState0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
|
||||
{ok, FState1} = rabbit_fifo_client:checkout(CustomerTag, 1, undefined, FState0),
|
||||
{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
|
||||
|
@ -99,11 +99,10 @@ basics(Config) ->
|
|||
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),
|
||||
{ok, FState4,
|
||||
[{deliver, C, true,
|
||||
[{_Qname, _QRef, MsgId, _SomBool, _Msg}]}]} ->
|
||||
{S, _A} = rabbit_fifo_client:settle(C, [MsgId], FState4),
|
||||
S
|
||||
end
|
||||
after 5000 ->
|
||||
|
@ -129,10 +128,9 @@ basics(Config) ->
|
|||
receive
|
||||
{ra_event, Frm, E} ->
|
||||
case rabbit_fifo_client:handle_ra_event(Frm, E, FState6b) of
|
||||
{internal, _, _, _FState7} ->
|
||||
exit({unexpected_internal_event, E});
|
||||
{{delivery, Ctag, [{Mid, {_, two}}]}, FState7} ->
|
||||
{ok, _S} = rabbit_fifo_client:return(Ctag, [Mid], FState7),
|
||||
{ok, FState7, [{deliver, Ctag, true,
|
||||
[{_, _, Mid, _, two}]}]} ->
|
||||
{_, _} = rabbit_fifo_client:return(Ctag, [Mid], FState7),
|
||||
ok
|
||||
end
|
||||
after 2000 ->
|
||||
|
@ -150,8 +148,8 @@ return(Config) ->
|
|||
{ok, F0} = rabbit_fifo_client:enqueue(1, msg1, F00),
|
||||
{ok, F1} = rabbit_fifo_client:enqueue(2, msg2, F0),
|
||||
{_, _, F2} = process_ra_events(receive_ra_events(2, 0), F1),
|
||||
{ok, {{MsgId, _}, _}, F} = rabbit_fifo_client:dequeue(<<"tag">>, unsettled, F2),
|
||||
{ok, _F2} = rabbit_fifo_client:return(<<"tag">>, [MsgId], F),
|
||||
{ok, _, {_, _, MsgId, _, _}, F} = rabbit_fifo_client:dequeue(<<"tag">>, unsettled, F2),
|
||||
_F2 = rabbit_fifo_client:return(<<"tag">>, [MsgId], F),
|
||||
|
||||
ra:stop_server(ServerId),
|
||||
ok.
|
||||
|
@ -165,9 +163,9 @@ rabbit_fifo_returns_correlation(Config) ->
|
|||
receive
|
||||
{ra_event, Frm, E} ->
|
||||
case rabbit_fifo_client:handle_ra_event(Frm, E, F1) of
|
||||
{internal, [corr1], [], _F2} ->
|
||||
{ok, _F2, [{settled, _, _}]} ->
|
||||
ok;
|
||||
{Del, _} ->
|
||||
Del ->
|
||||
exit({unexpected, Del})
|
||||
end
|
||||
after 2000 ->
|
||||
|
@ -181,23 +179,24 @@ duplicate_delivery(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, undefined, F0),
|
||||
{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} ->
|
||||
{ok, S1, [{settled, _, _}]} ->
|
||||
Loop(S1);
|
||||
{_Del, S1} ->
|
||||
{ok, 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} ->
|
||||
case rabbit_fifo_client:handle_ra_event(
|
||||
F, E1, S1) of
|
||||
{ok, S2, _} ->
|
||||
S2
|
||||
end
|
||||
end
|
||||
|
@ -215,7 +214,7 @@ usage(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, undefined, F0),
|
||||
{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(receive_ra_events(2, 2), F3),
|
||||
|
@ -242,9 +241,9 @@ resends_lost_command(Config) ->
|
|||
meck:unload(ra),
|
||||
{ok, F3} = rabbit_fifo_client:enqueue(msg3, F2),
|
||||
{_, _, F4} = process_ra_events(receive_ra_events(2, 0), F3),
|
||||
{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),
|
||||
{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.
|
||||
|
||||
|
@ -268,7 +267,7 @@ detects_lost_delivery(Config) ->
|
|||
F000 = rabbit_fifo_client:init(ClusterName, [ServerId]),
|
||||
{ok, F00} = rabbit_fifo_client:enqueue(msg1, F000),
|
||||
{_, _, F0} = process_ra_events(receive_ra_events(1, 0), F00),
|
||||
{ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, undefined, F0),
|
||||
{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
|
||||
|
@ -298,13 +297,13 @@ returns_after_down(Config) ->
|
|||
_Pid = spawn(fun () ->
|
||||
F = rabbit_fifo_client:init(ClusterName, [ServerId]),
|
||||
{ok, _} = rabbit_fifo_client:checkout(<<"tag">>, 10,
|
||||
undefined, F),
|
||||
#{}, F),
|
||||
Self ! checkout_done
|
||||
end),
|
||||
receive checkout_done -> ok after 1000 -> exit(checkout_done_timeout) end,
|
||||
timer:sleep(1000),
|
||||
% message should be available for dequeue
|
||||
{ok, {{_, {_, msg1}}, _}, _} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F2),
|
||||
{ok, _, {_, _, _, _, msg1}, _} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F2),
|
||||
ra:stop_server(ServerId),
|
||||
ok.
|
||||
|
||||
|
@ -327,9 +326,9 @@ resends_after_lost_applied(Config) ->
|
|||
% send another message
|
||||
{ok, F4} = rabbit_fifo_client:enqueue(msg3, F3),
|
||||
{_, _, F5} = process_ra_events(receive_ra_events(1, 0), F4),
|
||||
{ok, {{_, {_, msg1}}, _}, F6} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F5),
|
||||
{ok, {{_, {_, msg2}}, _}, F7} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F6),
|
||||
{ok, {{_, {_, msg3}}, _}, _F8} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F7),
|
||||
{ok, _, {_, _, _, _, msg1}, F6} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F5),
|
||||
{ok, _, {_, _, _, _, msg2}, F7} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F6),
|
||||
{ok, _, {_, _, _, _, msg3}, _F8} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F7),
|
||||
ra:stop_server(ServerId),
|
||||
ok.
|
||||
|
||||
|
@ -377,15 +376,16 @@ discard(Config) ->
|
|||
_ = ra:members(ServerId),
|
||||
|
||||
F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
|
||||
{ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, undefined, F0),
|
||||
{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),
|
||||
F3 = discard_next_delivery(F2, 5000),
|
||||
{empty, _F4} = rabbit_fifo_client:dequeue(<<"tag1">>, settled, F3),
|
||||
receive
|
||||
{dead_letter, Letters} ->
|
||||
[{_, msg1}] = Letters,
|
||||
ok
|
||||
after 500 ->
|
||||
flush(),
|
||||
exit(dead_letter_timeout)
|
||||
end,
|
||||
ra:stop_server(ServerId),
|
||||
|
@ -397,11 +397,11 @@ cancel_checkout(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, undefined, F1),
|
||||
{ok, F2} = rabbit_fifo_client:checkout(<<"tag">>, 10, #{}, F1),
|
||||
{_, _, F3} = process_ra_events(receive_ra_events(1, 1), F2, [], [], fun (_, S) -> S end),
|
||||
{ok, F4} = rabbit_fifo_client:cancel_checkout(<<"tag">>, F3),
|
||||
{ok, F5} = rabbit_fifo_client:return(<<"tag">>, [0], F4),
|
||||
{ok, {{_, {_, m1}}, _}, _} = rabbit_fifo_client:dequeue(<<"d1">>, settled, F5),
|
||||
{F5, _} = rabbit_fifo_client:return(<<"tag">>, [0], F4),
|
||||
{ok, _, {_, _, _, _, m1}, F5} = rabbit_fifo_client:dequeue(<<"d1">>, settled, F5),
|
||||
ok.
|
||||
|
||||
credit(Config) ->
|
||||
|
@ -413,20 +413,20 @@ credit(Config) ->
|
|||
{ok, F2} = rabbit_fifo_client:enqueue(m2, F1),
|
||||
{_, _, F3} = process_ra_events(receive_ra_events(2, 0), F2),
|
||||
%% checkout with 0 prefetch
|
||||
{ok, F4} = rabbit_fifo_client:checkout(<<"tag">>, 0, credited, undefined, F3),
|
||||
{ok, F4} = rabbit_fifo_client:checkout(<<"tag">>, 0, credited, #{}, F3),
|
||||
%% assert no deliveries
|
||||
{_, _, F5} = process_ra_events(receive_ra_events(), F4, [], [],
|
||||
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} =
|
||||
F6 = rabbit_fifo_client:credit(<<"tag">>, 1, false, F5),
|
||||
{[{_, _, _, _, m1}], [{send_credit_reply, _}], F7} =
|
||||
process_ra_events(receive_ra_events(1, 1), F6),
|
||||
|
||||
%% credit and drain
|
||||
{ok, F8} = rabbit_fifo_client:credit(<<"tag">>, 4, true, F7),
|
||||
{[{_, {_, m2}}], [{send_credit_reply, _}, {send_drained, _}], F9} =
|
||||
F8 = rabbit_fifo_client:credit(<<"tag">>, 4, true, F7),
|
||||
{[{_, _, _, _, m2}], [{send_credit_reply, _}, {send_drained, _}], F9} =
|
||||
process_ra_events(receive_ra_events(1, 1), F8),
|
||||
flush(),
|
||||
|
||||
|
@ -439,9 +439,8 @@ credit(Config) ->
|
|||
(D, _) -> error({unexpected_delivery, D})
|
||||
end),
|
||||
%% credit again and receive the last message
|
||||
{ok, F12} = rabbit_fifo_client:credit(<<"tag">>, 10, false, F11),
|
||||
{[{_, {_, m3}}], [{send_credit_reply, _}], _} =
|
||||
process_ra_events(receive_ra_events(1, 1), F12),
|
||||
F12 = rabbit_fifo_client:credit(<<"tag">>, 10, false, F11),
|
||||
{[{_, _, _, _, m3}], _, _} = process_ra_events(receive_ra_events(1, 1), F12),
|
||||
ok.
|
||||
|
||||
untracked_enqueue(Config) ->
|
||||
|
@ -452,7 +451,7 @@ untracked_enqueue(Config) ->
|
|||
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),
|
||||
{ok, _, {_, _, _, _, msg1}, _F5} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F0),
|
||||
ra:stop_server(ServerId),
|
||||
ok.
|
||||
|
||||
|
@ -472,6 +471,7 @@ flow(Config) ->
|
|||
ok.
|
||||
|
||||
test_queries(Config) ->
|
||||
% ok = logger:set_primary_config(level, all),
|
||||
ClusterName = ?config(cluster_name, Config),
|
||||
ServerId = ?config(node_id, Config),
|
||||
ok = start_cluster(ClusterName, [ServerId]),
|
||||
|
@ -484,20 +484,23 @@ test_queries(Config) ->
|
|||
Self ! ready,
|
||||
receive stop -> ok end
|
||||
end),
|
||||
receive
|
||||
ready -> ok
|
||||
after 5000 ->
|
||||
exit(ready_timeout)
|
||||
end,
|
||||
F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4),
|
||||
ok = receive ready -> ok after 5000 -> timeout end,
|
||||
{ok, _} = rabbit_fifo_client:checkout(<<"tag">>, 1, undefined, F0),
|
||||
?assertMatch({ok, {_RaIdxTerm, 1}, _Leader},
|
||||
ra:local_query(ServerId,
|
||||
fun rabbit_fifo:query_messages_ready/1)),
|
||||
?assertMatch({ok, {_RaIdxTerm, 1}, _Leader},
|
||||
ra:local_query(ServerId,
|
||||
fun rabbit_fifo:query_messages_checked_out/1)),
|
||||
?assertMatch({ok, {_RaIdxTerm, Processes}, _Leader}
|
||||
when length(Processes) == 2,
|
||||
ra:local_query(ServerId,
|
||||
fun rabbit_fifo:query_processes/1)),
|
||||
P ! stop,
|
||||
{ok, _} = rabbit_fifo_client:checkout(<<"tag">>, 1, #{}, F0),
|
||||
{ok, {_, Ready}, _} = ra:local_query(ServerId,
|
||||
fun rabbit_fifo:query_messages_ready/1),
|
||||
?assertEqual(1, Ready),
|
||||
{ok, {_, Checked}, _} = ra:local_query(ServerId,
|
||||
fun rabbit_fifo:query_messages_checked_out/1),
|
||||
?assertEqual(1, Checked),
|
||||
{ok, {_, Processes}, _} = ra:local_query(ServerId,
|
||||
fun rabbit_fifo:query_processes/1),
|
||||
?assertEqual(2, length(Processes)),
|
||||
P ! stop,
|
||||
ra:stop_server(ServerId),
|
||||
ok.
|
||||
|
||||
|
@ -511,15 +514,16 @@ dequeue(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),
|
||||
{empty, F1b} = rabbit_fifo_client:dequeue(Tag, settled, F1),
|
||||
{ok, F2_} = rabbit_fifo_client:enqueue(msg1, F1b),
|
||||
{_, _, F2} = process_ra_events(receive_ra_events(1, 0), F2_),
|
||||
|
||||
{ok, {{0, {_, msg1}}, _}, F3} = rabbit_fifo_client:dequeue(Tag, settled, F2),
|
||||
% {ok, {{0, {_, msg1}}, _}, F3} = rabbit_fifo_client:dequeue(Tag, settled, F2),
|
||||
{ok, _, {_, _, 0, _, msg1}, F3} = rabbit_fifo_client:dequeue(Tag, settled, F2),
|
||||
{ok, F4_} = rabbit_fifo_client:enqueue(msg2, F3),
|
||||
{_, _, F4} = process_ra_events(receive_ra_events(1, 0), F4_),
|
||||
{ok, {{MsgId, {_, msg2}}, _}, F5} = rabbit_fifo_client:dequeue(Tag, unsettled, F4),
|
||||
{ok, _F6} = rabbit_fifo_client:settle(Tag, [MsgId], F5),
|
||||
{ok, _, {_, _, MsgId, _, msg2}, F5} = rabbit_fifo_client:dequeue(Tag, unsettled, F4),
|
||||
{_F6, _A} = rabbit_fifo_client:settle(Tag, [MsgId], F5),
|
||||
ra:stop_server(ServerId),
|
||||
ok.
|
||||
|
||||
|
@ -534,8 +538,8 @@ conf(ClusterName, UId, ServerId, _, Peers) ->
|
|||
process_ra_event(State, Wait) ->
|
||||
receive
|
||||
{ra_event, From, Evt} ->
|
||||
{internal, _, _, S} =
|
||||
rabbit_fifo_client:handle_ra_event(From, Evt, State),
|
||||
{ok, S, _Actions} =
|
||||
rabbit_fifo_client:handle_ra_event(From, Evt, State),
|
||||
S
|
||||
after Wait ->
|
||||
exit(ra_event_timeout)
|
||||
|
@ -572,10 +576,10 @@ receive_ra_events(Acc) ->
|
|||
end.
|
||||
|
||||
process_ra_events(Events, State) ->
|
||||
DeliveryFun = fun ({delivery, Tag, Msgs}, S) ->
|
||||
DeliveryFun = fun ({deliver, _, Tag, Msgs}, S) ->
|
||||
MsgIds = [element(1, M) || M <- Msgs],
|
||||
{ok, S2} = rabbit_fifo_client:settle(Tag, MsgIds, S),
|
||||
S2
|
||||
{S0, _} = rabbit_fifo_client:settle(Tag, MsgIds, S),
|
||||
S0
|
||||
end,
|
||||
process_ra_events(Events, State, [], [], DeliveryFun).
|
||||
|
||||
|
@ -583,43 +587,41 @@ process_ra_events([], State0, Acc, Actions0, _DeliveryFun) ->
|
|||
{Acc, Actions0, State0};
|
||||
process_ra_events([{ra_event, From, Evt} | Events], State0, Acc, Actions0, DeliveryFun) ->
|
||||
case rabbit_fifo_client:handle_ra_event(From, Evt, State0) of
|
||||
{internal, _, Actions, State} ->
|
||||
process_ra_events(Events, State, Acc, Actions0 ++ Actions, DeliveryFun);
|
||||
{{delivery, _Tag, Msgs} = Del, State1} ->
|
||||
State = DeliveryFun(Del, State1),
|
||||
process_ra_events(Events, State, Acc ++ Msgs, Actions0, DeliveryFun);
|
||||
{ok, State1, Actions1} ->
|
||||
{Msgs, Actions, State} =
|
||||
lists:foldl(
|
||||
fun ({deliver, _, _, Msgs} = Del, {M, A, S}) ->
|
||||
{M ++ Msgs, A, DeliveryFun(Del, S)};
|
||||
(Ac, {M, A, S}) ->
|
||||
{M, A ++ [Ac], S}
|
||||
end, {Acc, [], State1}, Actions1),
|
||||
process_ra_events(Events, State, Msgs, Actions0 ++ Actions, DeliveryFun);
|
||||
eol ->
|
||||
eol
|
||||
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],
|
||||
{ok, State} = rabbit_fifo_client:discard(Tag, MsgIds,
|
||||
State1),
|
||||
State
|
||||
end
|
||||
{ra_event, _, {machine, {delivery, _, _}}} = Evt ->
|
||||
element(3, process_ra_events([Evt], State0, [], [],
|
||||
fun ({deliver, Tag, _, Msgs}, S) ->
|
||||
MsgIds = [element(3, M) || M <- Msgs],
|
||||
{S0, _} = rabbit_fifo_client:discard(Tag, MsgIds, S),
|
||||
S0
|
||||
end))
|
||||
after Wait ->
|
||||
State0
|
||||
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],
|
||||
{ok, State} = rabbit_fifo_client:return(Tag, MsgIds,
|
||||
State1),
|
||||
State
|
||||
end
|
||||
{ra_event, _, {machine, {delivery, _, _}}} = Evt ->
|
||||
element(3, process_ra_events([Evt], State0, [], [],
|
||||
fun ({deliver, Tag, _, Msgs}, S) ->
|
||||
MsgIds = [element(3, M) || M <- Msgs],
|
||||
{S0, _} = rabbit_fifo_client:return(Tag, MsgIds, S),
|
||||
S0
|
||||
end))
|
||||
after Wait ->
|
||||
State0
|
||||
end.
|
||||
|
|
|
@ -51,12 +51,15 @@ run(TestPid, Channel, Queue, CancelOnFailover, LowestSeen, MsgsToConsume) ->
|
|||
%% counter.
|
||||
if
|
||||
MsgNum + 1 == LowestSeen ->
|
||||
error_logger:info_msg("recording ~w left ~w",
|
||||
[MsgNum, MsgsToConsume]),
|
||||
run(TestPid, Channel, Queue,
|
||||
CancelOnFailover, MsgNum, MsgsToConsume - 1);
|
||||
MsgNum >= LowestSeen ->
|
||||
error_logger:info_msg(
|
||||
"consumer ~p on ~p ignoring redelivered msg ~p~n",
|
||||
[self(), Channel, MsgNum]),
|
||||
"consumer ~p on ~p ignoring redelivered msg ~p"
|
||||
"lowest seen ~w~n",
|
||||
[self(), Channel, MsgNum, LowestSeen]),
|
||||
true = Redelivered, %% ASSERTION
|
||||
run(TestPid, Channel, Queue,
|
||||
CancelOnFailover, LowestSeen, MsgsToConsume);
|
||||
|
|
|
@ -0,0 +1,213 @@
|
|||
-module(rabbit_msg_record_SUITE).
|
||||
|
||||
-compile(export_all).
|
||||
|
||||
-export([
|
||||
]).
|
||||
|
||||
-include("rabbit.hrl").
|
||||
-include("rabbit_framing.hrl").
|
||||
-include_lib("common_test/include/ct.hrl").
|
||||
-include_lib("eunit/include/eunit.hrl").
|
||||
-include_lib("amqp10_common/include/amqp10_framing.hrl").
|
||||
|
||||
%%%===================================================================
|
||||
%%% Common Test callbacks
|
||||
%%%===================================================================
|
||||
|
||||
all() ->
|
||||
[
|
||||
{group, tests}
|
||||
].
|
||||
|
||||
|
||||
all_tests() ->
|
||||
[
|
||||
ampq091_roundtrip,
|
||||
message_id_ulong,
|
||||
message_id_uuid,
|
||||
message_id_binary,
|
||||
message_id_large_binary,
|
||||
message_id_large_string
|
||||
].
|
||||
|
||||
groups() ->
|
||||
[
|
||||
{tests, [], all_tests()}
|
||||
].
|
||||
|
||||
init_per_suite(Config) ->
|
||||
Config.
|
||||
|
||||
end_per_suite(_Config) ->
|
||||
ok.
|
||||
|
||||
init_per_group(_Group, Config) ->
|
||||
Config.
|
||||
|
||||
end_per_group(_Group, _Config) ->
|
||||
ok.
|
||||
|
||||
init_per_testcase(_TestCase, Config) ->
|
||||
Config.
|
||||
|
||||
end_per_testcase(_TestCase, _Config) ->
|
||||
ok.
|
||||
|
||||
%%%===================================================================
|
||||
%%% Test cases
|
||||
%%%===================================================================
|
||||
|
||||
ampq091_roundtrip(_Config) ->
|
||||
Props = #'P_basic'{content_type = <<"text/plain">>,
|
||||
content_encoding = <<"gzip">>,
|
||||
headers = [{<<"x-stream-offset">>, long, 99},
|
||||
{<<"x-string">>, longstr, <<"a string">>},
|
||||
{<<"x-bool">>, bool, false},
|
||||
{<<"x-unsignedbyte">>, unsignedbyte, 1},
|
||||
{<<"x-unsignedshort">>, unsignedshort, 1},
|
||||
{<<"x-unsignedint">>, unsignedint, 1},
|
||||
{<<"x-signedint">>, signedint, 1},
|
||||
{<<"x-timestamp">>, timestamp, 1},
|
||||
{<<"x-double">>, double, 1.0},
|
||||
{<<"x-float">>, float, 1.0},
|
||||
{<<"x-binary">>, binary, <<"data">>}
|
||||
],
|
||||
delivery_mode = 2,
|
||||
priority = 99,
|
||||
correlation_id = <<"corr">> ,
|
||||
reply_to = <<"reply-to">>,
|
||||
expiration = <<"1">>,
|
||||
message_id = <<"msg-id">>,
|
||||
timestamp = 99,
|
||||
type = <<"45">>,
|
||||
user_id = <<"banana">>,
|
||||
app_id = <<"rmq">>
|
||||
% cluster_id = <<"adf">>
|
||||
},
|
||||
Payload = [<<"data">>],
|
||||
test_amqp091_roundtrip(Props, Payload),
|
||||
test_amqp091_roundtrip(#'P_basic'{}, Payload),
|
||||
ok.
|
||||
|
||||
message_id_ulong(_Config) ->
|
||||
Num = 9876789,
|
||||
ULong = erlang:integer_to_binary(Num),
|
||||
P = #'v1_0.properties'{message_id = {ulong, Num},
|
||||
correlation_id = {ulong, Num}},
|
||||
D = #'v1_0.data'{content = <<"data">>},
|
||||
Bin = [amqp10_framing:encode_bin(P),
|
||||
amqp10_framing:encode_bin(D)],
|
||||
R = rabbit_msg_record:init(iolist_to_binary(Bin)),
|
||||
{Props, _} = rabbit_msg_record:to_amqp091(R),
|
||||
?assertMatch(#'P_basic'{message_id = ULong,
|
||||
correlation_id = ULong,
|
||||
headers =
|
||||
[
|
||||
%% ordering shouldn't matter
|
||||
{<<"x-correlation-id-type">>, longstr, <<"ulong">>},
|
||||
{<<"x-message-id-type">>, longstr, <<"ulong">>}
|
||||
]},
|
||||
Props),
|
||||
ok.
|
||||
|
||||
message_id_uuid(_Config) ->
|
||||
%% fake a uuid
|
||||
UUId = erlang:md5(term_to_binary(make_ref())),
|
||||
TextUUId = rabbit_data_coercion:to_binary(rabbit_guid:to_string(UUId)),
|
||||
P = #'v1_0.properties'{message_id = {uuid, UUId},
|
||||
correlation_id = {uuid, UUId}},
|
||||
D = #'v1_0.data'{content = <<"data">>},
|
||||
Bin = [amqp10_framing:encode_bin(P),
|
||||
amqp10_framing:encode_bin(D)],
|
||||
R = rabbit_msg_record:init(iolist_to_binary(Bin)),
|
||||
{Props, _} = rabbit_msg_record:to_amqp091(R),
|
||||
?assertMatch(#'P_basic'{message_id = TextUUId,
|
||||
correlation_id = TextUUId,
|
||||
headers =
|
||||
[
|
||||
%% ordering shouldn't matter
|
||||
{<<"x-correlation-id-type">>, longstr, <<"uuid">>},
|
||||
{<<"x-message-id-type">>, longstr, <<"uuid">>}
|
||||
]},
|
||||
Props),
|
||||
ok.
|
||||
|
||||
message_id_binary(_Config) ->
|
||||
%% fake a uuid
|
||||
Orig = <<"asdfasdf">>,
|
||||
Text = base64:encode(Orig),
|
||||
P = #'v1_0.properties'{message_id = {binary, Orig},
|
||||
correlation_id = {binary, Orig}},
|
||||
D = #'v1_0.data'{content = <<"data">>},
|
||||
Bin = [amqp10_framing:encode_bin(P),
|
||||
amqp10_framing:encode_bin(D)],
|
||||
R = rabbit_msg_record:init(iolist_to_binary(Bin)),
|
||||
{Props, _} = rabbit_msg_record:to_amqp091(R),
|
||||
?assertMatch(#'P_basic'{message_id = Text,
|
||||
correlation_id = Text,
|
||||
headers =
|
||||
[
|
||||
%% ordering shouldn't matter
|
||||
{<<"x-correlation-id-type">>, longstr, <<"binary">>},
|
||||
{<<"x-message-id-type">>, longstr, <<"binary">>}
|
||||
]},
|
||||
Props),
|
||||
ok.
|
||||
|
||||
message_id_large_binary(_Config) ->
|
||||
%% cannot fit in a shortstr
|
||||
Orig = crypto:strong_rand_bytes(500),
|
||||
P = #'v1_0.properties'{message_id = {binary, Orig},
|
||||
correlation_id = {binary, Orig}},
|
||||
D = #'v1_0.data'{content = <<"data">>},
|
||||
Bin = [amqp10_framing:encode_bin(P),
|
||||
amqp10_framing:encode_bin(D)],
|
||||
R = rabbit_msg_record:init(iolist_to_binary(Bin)),
|
||||
{Props, _} = rabbit_msg_record:to_amqp091(R),
|
||||
?assertMatch(#'P_basic'{message_id = undefined,
|
||||
correlation_id = undefined,
|
||||
headers =
|
||||
[
|
||||
%% ordering shouldn't matter
|
||||
{<<"x-correlation-id">>, longstr, Orig},
|
||||
{<<"x-message-id">>, longstr, Orig}
|
||||
]},
|
||||
Props),
|
||||
ok.
|
||||
|
||||
message_id_large_string(_Config) ->
|
||||
%% cannot fit in a shortstr
|
||||
Orig = base64:encode(crypto:strong_rand_bytes(500)),
|
||||
P = #'v1_0.properties'{message_id = {utf8, Orig},
|
||||
correlation_id = {utf8, Orig}},
|
||||
D = #'v1_0.data'{content = <<"data">>},
|
||||
Bin = [amqp10_framing:encode_bin(P),
|
||||
amqp10_framing:encode_bin(D)],
|
||||
R = rabbit_msg_record:init(iolist_to_binary(Bin)),
|
||||
{Props, _} = rabbit_msg_record:to_amqp091(R),
|
||||
?assertMatch(#'P_basic'{message_id = undefined,
|
||||
correlation_id = undefined,
|
||||
headers =
|
||||
[
|
||||
%% ordering shouldn't matter
|
||||
{<<"x-correlation-id">>, longstr, Orig},
|
||||
{<<"x-message-id">>, longstr, Orig}
|
||||
]},
|
||||
Props),
|
||||
ok.
|
||||
|
||||
%% Utility
|
||||
|
||||
test_amqp091_roundtrip(Props, Payload) ->
|
||||
MsgRecord0 = rabbit_msg_record:from_amqp091(Props, Payload),
|
||||
MsgRecord = rabbit_msg_record:init(
|
||||
iolist_to_binary(rabbit_msg_record:to_iodata(MsgRecord0))),
|
||||
% meck:unload(),
|
||||
{PropsOut, PayloadOut} = rabbit_msg_record:to_amqp091(MsgRecord),
|
||||
?assertEqual(Props, PropsOut),
|
||||
?assertEqual(iolist_to_binary(Payload),
|
||||
iolist_to_binary(PayloadOut)),
|
||||
ok.
|
||||
|
||||
|
File diff suppressed because it is too large
Load Diff
|
@ -234,8 +234,10 @@ consume_survives(Config,
|
|||
DeathFun(Config, A),
|
||||
%% verify that the consumer got all msgs, or die - the await_response
|
||||
%% calls throw an exception if anything goes wrong....
|
||||
rabbit_ha_test_consumer:await_response(ConsumerPid),
|
||||
ct:pal("awaiting produce ~w", [ProducerPid]),
|
||||
rabbit_ha_test_producer:await_response(ProducerPid),
|
||||
ct:pal("awaiting consumer ~w", [ConsumerPid]),
|
||||
rabbit_ha_test_consumer:await_response(ConsumerPid),
|
||||
ok.
|
||||
|
||||
confirms_survive(Config, DeathFun) ->
|
||||
|
|
|
@ -126,6 +126,10 @@ sink_rewrite_sinks() ->
|
|||
{rabbit_log_mirroring_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
|
||||
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
|
||||
{rabbit_log_osiris_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
|
||||
{rabbit_handlers,
|
||||
[{lager_forwarder_backend,[lager_event,info]}]}]},
|
||||
{rabbit_log_prelaunch_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
|
||||
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
|
||||
|
@ -226,6 +230,10 @@ sink_handlers_merged_with_lager_extra_sinks_handlers(_) ->
|
|||
{rabbit_log_mirroring_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
|
||||
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
|
||||
{rabbit_log_osiris_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
|
||||
{rabbit_handlers,
|
||||
[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
|
||||
{rabbit_log_prelaunch_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
|
||||
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
|
||||
|
@ -317,6 +325,10 @@ level_sinks() ->
|
|||
{rabbit_log_mirroring_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,error]}]},
|
||||
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,error]}]}]},
|
||||
{rabbit_log_osiris_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
|
||||
{rabbit_handlers,
|
||||
[{lager_forwarder_backend,[lager_event,info]}]}]},
|
||||
{rabbit_log_prelaunch_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
|
||||
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
|
||||
|
@ -427,6 +439,10 @@ file_sinks(DefaultLevel) ->
|
|||
{rabbit_log_mirroring_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
|
||||
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
|
||||
{rabbit_log_osiris_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
|
||||
{rabbit_handlers,
|
||||
[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
|
||||
{rabbit_log_prelaunch_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
|
||||
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
|
||||
|
@ -674,6 +690,10 @@ default_expected_sinks(UpgradeFile) ->
|
|||
{rabbit_log_mirroring_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
|
||||
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
|
||||
{rabbit_log_osiris_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
|
||||
{rabbit_handlers,
|
||||
[{lager_forwarder_backend,[lager_event,info]}]}]},
|
||||
{rabbit_log_prelaunch_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
|
||||
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
|
||||
|
@ -761,6 +781,10 @@ tty_expected_sinks() ->
|
|||
{rabbit_log_mirroring_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
|
||||
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
|
||||
{rabbit_log_osiris_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
|
||||
{rabbit_handlers,
|
||||
[{lager_forwarder_backend,[lager_event,info]}]}]},
|
||||
{rabbit_log_prelaunch_lager_event,
|
||||
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
|
||||
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
|
||||
|
|
Loading…
Reference in New Issue