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:
kjnilsson 2020-09-29 11:43:24 +01:00
parent bdb6f9b508
commit f20fa273e9
50 changed files with 6567 additions and 1664 deletions

View File

@ -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))))

View File

@ -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).

View File

@ -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

32
scripts/rabbitmq-streams Executable file
View File

@ -0,0 +1,32 @@
#!/bin/sh
## The contents of this file are subject to the Mozilla Public License
## Version 1.1 (the "License"); you may not use this file except in
## compliance with the License. You may obtain a copy of the License
## at 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 "$@"

View File

@ -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

View File

@ -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) ->

View File

@ -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).

View File

@ -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;

View File

@ -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));

View File

@ -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

View File

@ -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).

151
src/rabbit_confirms.erl Normal file
View File

@ -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.

View File

@ -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...",

View File

@ -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,

View File

@ -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,
#{

View File

@ -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.

View File

@ -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.

View File

@ -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]) ->

View File

@ -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);

399
src/rabbit_msg_record.erl Normal file
View File

@ -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.

View File

@ -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}.

View File

@ -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);

View File

@ -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() ->

560
src/rabbit_queue_type.erl Normal file
View File

@ -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)}.

View File

@ -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)]).

View File

@ -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),

View File

@ -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].

View File

@ -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).

665
src/rabbit_stream_queue.erl Normal file
View File

@ -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).

View File

@ -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.

View File

@ -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.

View File

@ -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.

View File

@ -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.

View File

@ -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),

View File

@ -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.

View File

@ -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),

View File

@ -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

View File

@ -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}, _} ->

234
test/queue_type_SUITE.erl Normal file
View File

@ -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.

View File

@ -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) ->

View File

@ -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

View File

@ -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

View File

@ -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.

View File

@ -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.

View File

@ -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);

View File

@ -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

View File

@ -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) ->

View File

@ -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]}]}]},