From 7fa38c0b97252d1854d3f59421cb0da5f2834665 Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Tue, 2 Jul 2019 16:11:54 +0100 Subject: [PATCH 01/18] Add marker rabbit_queue_type behaviour 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. --- include/amqqueue.hrl | 6 +++--- src/amqqueue.erl | 6 +++--- src/amqqueue_v1.erl | 5 +++-- src/rabbit_amqqueue.erl | 21 +++++++++++++++------ src/rabbit_classic_queue.erl | 5 +++++ src/rabbit_queue_type.erl | 8 ++++++++ src/rabbit_quorum_queue.erl | 13 ++++++++----- test/quorum_queue_SUITE.erl | 6 +++--- 8 files changed, 48 insertions(+), 22 deletions(-) create mode 100644 src/rabbit_classic_queue.erl create mode 100644 src/rabbit_queue_type.erl diff --git a/include/amqqueue.hrl b/include/amqqueue.hrl index 3a9ac45cce65..5a8a9bad03ec 100644 --- a/include/amqqueue.hrl +++ b/include/amqqueue.hrl @@ -51,16 +51,16 @@ (?is_amqqueue_v1(Q) andalso ?amqqueue_v1_field_state(Q) =:= State))). --define(amqqueue_v1_type, classic). +-define(amqqueue_v1_type, rabbit_classic_queue). -define(amqqueue_is_classic(Q), ((?is_amqqueue_v2(Q) andalso - ?amqqueue_v2_field_type(Q) =:= classic) orelse + ?amqqueue_v2_field_type(Q) =:= rabbit_classic_queue) orelse ?is_amqqueue_v1(Q))). -define(amqqueue_is_quorum(Q), (?is_amqqueue_v2(Q) andalso - ?amqqueue_v2_field_type(Q) =:= quorum) orelse + ?amqqueue_v2_field_type(Q) =:= rabbit_quorum_queue) orelse false). -define(amqqueue_has_valid_pid(Q), diff --git a/src/amqqueue.erl b/src/amqqueue.erl index 35e7f0c4c44e..2a12d11c26b5 100644 --- a/src/amqqueue.erl +++ b/src/amqqueue.erl @@ -118,7 +118,7 @@ slave_pids_pending_shutdown = [] :: [pid()] | '_', vhost :: rabbit_types:vhost() | undefined | '_', %% secondary index options = #{} :: map() | '_', - type = ?amqqueue_v1_type :: atom() | '_', + type = ?amqqueue_v1_type :: module() | '_', quorum_nodes = [] :: [node()] | '_' }). @@ -451,7 +451,7 @@ set_gm_pids(Queue, GMPids) -> -spec get_leader(amqqueue_v2()) -> node(). -get_leader(#amqqueue{type = quorum, pid = {_, Leader}}) -> Leader. +get_leader(#amqqueue{type = rabbit_quorum_queue, pid = {_, Leader}}) -> Leader. % operator_policy @@ -660,7 +660,7 @@ is_classic(Queue) -> -spec is_quorum(amqqueue()) -> boolean(). is_quorum(Queue) -> - get_type(Queue) =:= quorum. + get_type(Queue) =:= rabbit_quorum_queue. fields() -> case record_version_to_use() of diff --git a/src/amqqueue_v1.erl b/src/amqqueue_v1.erl index 9b739026c617..6323cbbff2e0 100644 --- a/src/amqqueue_v1.erl +++ b/src/amqqueue_v1.erl @@ -525,8 +525,9 @@ is_classic(Queue) -> -spec is_quorum(amqqueue()) -> boolean(). -is_quorum(Queue) -> - get_type(Queue) =:= quorum. +is_quorum(Queue) when ?is_amqqueue(Queue) -> + false. + % get_type(Queue) =:= rabbit_quorum_queue. fields() -> fields(?record_version). diff --git a/src/rabbit_amqqueue.erl b/src/rabbit_amqqueue.erl index cbe8738c5ae3..83f4d07deb21 100644 --- a/src/rabbit_amqqueue.erl +++ b/src/rabbit_amqqueue.erl @@ -274,7 +274,7 @@ declare(QueueName = #resource{virtual_host = VHost}, Durable, AutoDelete, Args, ok = check_declare_arguments(QueueName, Args), Type = get_queue_type(Args), TypeIsAllowed = - Type =:= classic orelse + Type =:= rabbit_classic_queue orelse rabbit_feature_flags:is_enabled(quorum_queue), case TypeIsAllowed of true -> @@ -325,9 +325,16 @@ declare_classic_queue(Q, Node) -> get_queue_type(Args) -> case rabbit_misc:table_lookup(Args, <<"x-queue-type">>) of undefined -> - classic; + rabbit_classic_queue; {_, V} -> - erlang:binary_to_existing_atom(V, utf8) + %% 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 end. -spec internal_declare(amqqueue:amqqueue(), boolean()) -> @@ -1555,9 +1562,11 @@ forget_node_for_queue(DeadNode, [H|T], Q) when ?is_amqqueue(Q) -> Type = amqqueue:get_type(Q), case {node_permits_offline_promotion(H), Type} of {false, _} -> forget_node_for_queue(DeadNode, T, Q); - {true, classic} -> Q1 = amqqueue:set_pid(Q, rabbit_misc:node_to_fake_pid(H)), + {true, rabbit_classic_queue} -> + Q1 = amqqueue:set_pid(Q, rabbit_misc:node_to_fake_pid(H)), ok = mnesia:write(rabbit_durable_queue, Q1, write); - {true, quorum} -> ok + {true, rabbit_quorum_queue} -> + ok end. node_permits_offline_promotion(Node) -> @@ -1755,7 +1764,7 @@ pseudo_queue(#resource{kind = queue} = QueueName, Pid, Durable) [], undefined, % VHost, #{user => undefined}, % ActingUser - classic % Type + rabbit_classic_queue % Type ). -spec immutable(amqqueue:amqqueue()) -> amqqueue:amqqueue(). diff --git a/src/rabbit_classic_queue.erl b/src/rabbit_classic_queue.erl new file mode 100644 index 000000000000..1de8018529c4 --- /dev/null +++ b/src/rabbit_classic_queue.erl @@ -0,0 +1,5 @@ +-module(rabbit_classic_queue). +-behaviour(rabbit_queue_type). + +-export([ + ]). diff --git a/src/rabbit_queue_type.erl b/src/rabbit_queue_type.erl new file mode 100644 index 000000000000..a2560a5419d0 --- /dev/null +++ b/src/rabbit_queue_type.erl @@ -0,0 +1,8 @@ +-module(rabbit_queue_type). + +-export([ + ]). + +-optional_callbacks([init/0]). + +-callback init() -> #{}. diff --git a/src/rabbit_quorum_queue.erl b/src/rabbit_quorum_queue.erl index eb34b9db9fea..d7d0a65e552c 100644 --- a/src/rabbit_quorum_queue.erl +++ b/src/rabbit_quorum_queue.erl @@ -16,6 +16,8 @@ -module(rabbit_quorum_queue). +-behaviour(rabbit_queue_type). + -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, infos/1]). @@ -597,7 +599,7 @@ cleanup_data_dir() -> {Name, _} = amqqueue:get_pid(Q), Name end - || Q <- rabbit_amqqueue:list_by_type(quorum), + || Q <- rabbit_amqqueue:list_by_type(?MODULE), lists:member(node(), amqqueue:get_quorum_nodes(Q))], Registered = ra_directory:list_registered(), _ = [maybe_delete_data_dir(UId) || {Name, UId} <- Registered, @@ -631,7 +633,8 @@ cluster_state(Name) -> end end. --spec status(rabbit_types:vhost(), Name :: rabbit_misc:resource_name()) -> rabbit_types:infos() | {error, term()}. +-spec status(rabbit_types:vhost(), Name :: rabbit_misc:resource_name()) -> + rabbit_types:infos() | {error, term()}. status(Vhost, QueueName) -> %% Handle not found queues QName = #resource{virtual_host = Vhost, name = QueueName, kind = queue}, @@ -808,7 +811,7 @@ shrink_all(Node) -> {QName, {error, Size, Err}} end end || Q <- rabbit_amqqueue:list(), - amqqueue:get_type(Q) == quorum, + amqqueue:get_type(Q) == ?MODULE, lists:member(Node, amqqueue:get_quorum_nodes(Q))]. -spec grow(node(), binary(), binary(), all | even) -> @@ -832,7 +835,7 @@ grow(Node, VhostSpec, QueueSpec, Strategy) -> end end || Q <- rabbit_amqqueue:list(), - amqqueue:get_type(Q) == quorum, + amqqueue:get_type(Q) == ?MODULE, %% don't add a member if there is already one on the node not lists:member(Node, amqqueue:get_quorum_nodes(Q)), %% node needs to be running @@ -991,7 +994,7 @@ i(leader, Q) -> leader(Q); i(open_files, Q) when ?is_amqqueue(Q) -> {Name, _} = amqqueue:get_pid(Q), Nodes = amqqueue:get_quorum_nodes(Q), - {Data, _} = rpc:multicall(Nodes, rabbit_quorum_queue, open_files, [Name]), + {Data, _} = rpc:multicall(Nodes, ?MODULE, open_files, [Name]), lists:flatten(Data); i(single_active_consumer_pid, Q) when ?is_amqqueue(Q) -> QPid = amqqueue:get_pid(Q), diff --git a/test/quorum_queue_SUITE.erl b/test/quorum_queue_SUITE.erl index cc0e7aa75a2a..7ef38895eb79 100644 --- a/test/quorum_queue_SUITE.erl +++ b/test/quorum_queue_SUITE.erl @@ -264,15 +264,15 @@ declare_args(Config) -> declare(Ch, LQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}, {<<"x-max-length">>, long, 2000}, {<<"x-max-length-bytes">>, long, 2000}]), - assert_queue_type(Server, LQ, quorum), + assert_queue_type(Server, LQ, rabbit_quorum_queue), DQ = <<"classic-declare-args-q">>, declare(Ch, DQ, [{<<"x-queue-type">>, longstr, <<"classic">>}]), - assert_queue_type(Server, DQ, classic), + assert_queue_type(Server, DQ, rabbit_classic_queue), DQ2 = <<"classic-q2">>, declare(Ch, DQ2), - assert_queue_type(Server, DQ2, classic). + assert_queue_type(Server, DQ2, rabbit_classic_queue). declare_invalid_properties(Config) -> Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), From e5dff9f951ce2bc227bf1b4fa05d0836b02bc094 Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Wed, 3 Jul 2019 10:59:00 +0100 Subject: [PATCH 02/18] Move queue declare into rabbit_queue_type First step to a greater abstraction! --- src/rabbit_amqqueue.erl | 44 ++++-------------------------------- src/rabbit_classic_queue.erl | 24 ++++++++++++++++++++ src/rabbit_queue_type.erl | 38 +++++++++++++++++++++++++++++-- src/rabbit_quorum_queue.erl | 16 +++++++++---- 4 files changed, 76 insertions(+), 46 deletions(-) diff --git a/src/rabbit_amqqueue.erl b/src/rabbit_amqqueue.erl index 83f4d07deb21..5f0821b262ec 100644 --- a/src/rabbit_amqqueue.erl +++ b/src/rabbit_amqqueue.erl @@ -265,18 +265,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, @@ -289,7 +284,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, @@ -298,43 +293,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 rabbit_queue_master_location_misc:get_location(Q) of - {ok, Node0} -> Node0; - {error, _} -> Node - 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()) -> diff --git a/src/rabbit_classic_queue.erl b/src/rabbit_classic_queue.erl index 1de8018529c4..a77a3ff360f3 100644 --- a/src/rabbit_classic_queue.erl +++ b/src/rabbit_classic_queue.erl @@ -1,5 +1,29 @@ -module(rabbit_classic_queue). -behaviour(rabbit_queue_type). +-include("amqqueue.hrl"). -export([ + is_enabled/0, + declare/2 ]). + +is_enabled() -> true. + +declare(Q, Node) when ?amqqueue_is_classic(Q) -> + QName = amqqueue:get_name(Q), + VHost = amqqueue:get_vhost(Q), + Node1 = case rabbit_queue_master_location_misc:get_location(Q) of + {ok, Node0} -> Node0; + {error, _} -> Node + 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. diff --git a/src/rabbit_queue_type.erl b/src/rabbit_queue_type.erl index a2560a5419d0..3cea76c24387 100644 --- a/src/rabbit_queue_type.erl +++ b/src/rabbit_queue_type.erl @@ -1,8 +1,42 @@ -module(rabbit_queue_type). -export([ + discover/1, + default/0, + is_enabled/1, + declare/2 ]). --optional_callbacks([init/0]). +% copied from rabbit_amqqueue +-type absent_reason() :: 'nodedown' | 'crashed' | stopped | timeout. + +%% 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(). + + +%% TODO: this should be controlled by a registry that is populated on boot +discover(<<"quorum">>) -> + rabbit_quorum_queue; +discover(<<"classic">>) -> + rabbit_classic_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). --callback init() -> #{}. diff --git a/src/rabbit_quorum_queue.erl b/src/rabbit_quorum_queue.erl index d7d0a65e552c..b6eec1b2e933 100644 --- a/src/rabbit_quorum_queue.erl +++ b/src/rabbit_quorum_queue.erl @@ -19,7 +19,7 @@ -behaviour(rabbit_queue_type). -export([init_state/2, handle_event/2]). --export([declare/1, recover/1, stop/1, delete/4, delete_immediately/2]). +-export([recover/1, stop/1, delete/4, delete_immediately/2]). -export([info/1, info/2, stat/1, infos/1]). -export([ack/3, reject/4, basic_get/4, basic_consume/10, basic_cancel/4]). -export([credit/4]). @@ -42,6 +42,9 @@ -export([shrink_all/1, grow/4]). +-export([is_enabled/0, + declare/2]). + %%-include_lib("rabbit_common/include/rabbit.hrl"). -include_lib("rabbit.hrl"). -include_lib("stdlib/include/qlc.hrl"). @@ -73,6 +76,12 @@ -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()) -> @@ -100,10 +109,9 @@ init_state({Name, _}, QName = #resource{}) -> handle_event({ra_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), From 8aa9d9fd2f00f7c9dc3adaf2d841244c599d6c81 Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Wed, 3 Jul 2019 14:31:38 +0100 Subject: [PATCH 03/18] Move queue delete into queue type implementation --- src/rabbit_amqqueue.erl | 82 ++---------------------------------- src/rabbit_channel.erl | 20 +++++---- src/rabbit_classic_queue.erl | 82 +++++++++++++++++++++++++++++++++++- src/rabbit_queue_type.erl | 18 +++++++- src/rabbit_quorum_queue.erl | 4 +- 5 files changed, 115 insertions(+), 91 deletions(-) diff --git a/src/rabbit_amqqueue.erl b/src/rabbit_amqqueue.erl index 5f0821b262ec..1823603b7a81 100644 --- a/src/rabbit_amqqueue.erl +++ b/src/rabbit_amqqueue.erl @@ -19,8 +19,7 @@ -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, not_found_or_absent/1, with/2, with/3, with_or_die/2, assert_equivalence/5, @@ -1123,82 +1122,8 @@ 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. - --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 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]). - --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). + rabbit_queue_type:delete(Q, IfUnused, IfEmpty, ActingUser). -spec purge(amqqueue:amqqueue()) -> {ok, qlen()}. @@ -1437,7 +1362,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). diff --git a/src/rabbit_channel.erl b/src/rabbit_channel.erl index d16929d962a1..1f10e6f6de59 100644 --- a/src/rabbit_channel.erl +++ b/src/rabbit_channel.erl @@ -2608,20 +2608,24 @@ handle_method(#'queue.delete'{queue = QueueNameBin, StrippedQueueNameBin = strip_cr_lf(QueueNameBin), QueueName = qbin_to_resource(StrippedQueueNameBin, VHostPath), - check_configure_permitted(QueueName, User, extract_authz_context(ConnPid, ChSrc)), + check_configure_permitted(QueueName, User, + extract_authz_context(ConnPid, ChSrc)), case rabbit_amqqueue:with( QueueName, fun (Q) -> rabbit_amqqueue:check_exclusive_access(Q, ConnPid), rabbit_amqqueue:delete(Q, IfUnused, IfEmpty, Username) end, - fun (not_found) -> {ok, 0}; - %% TODO delete crashed should clean up fifo states? - ({absent, Q, crashed}) -> rabbit_amqqueue:delete_crashed(Q, Username), - {ok, 0}; - ({absent, Q, stopped}) -> rabbit_amqqueue:delete_crashed(Q, Username), - {ok, 0}; - ({absent, Q, Reason}) -> rabbit_amqqueue:absent(Q, Reason) + fun (not_found) -> + {ok, 0}; + ({absent, Q, crashed}) -> + _ = rabbit_classic_queue:delete_crashed(Q, Username), + {ok, 0}; + ({absent, Q, stopped}) -> + _ = rabbit_classic_queue:delete_crashed(Q, Username), + {ok, 0}; + ({absent, Q, Reason}) -> + rabbit_amqqueue:absent(Q, Reason) end) of {error, in_use} -> precondition_failed("~s in use", [rabbit_misc:rs(QueueName)]); diff --git a/src/rabbit_classic_queue.erl b/src/rabbit_classic_queue.erl index a77a3ff360f3..b5cf93c32082 100644 --- a/src/rabbit_classic_queue.erl +++ b/src/rabbit_classic_queue.erl @@ -2,11 +2,18 @@ -behaviour(rabbit_queue_type). -include("amqqueue.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + -export([ is_enabled/0, - declare/2 + declare/2, + delete/4 ]). +-export([delete_crashed/1, + delete_crashed/2, + delete_crashed_internal/2]). + is_enabled() -> true. declare(Q, Node) when ?amqqueue_is_classic(Q) -> @@ -27,3 +34,76 @@ declare(Q, Node) when ?amqqueue_is_classic(Q) -> "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. + +-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). diff --git a/src/rabbit_queue_type.erl b/src/rabbit_queue_type.erl index 3cea76c24387..985b93b652fd 100644 --- a/src/rabbit_queue_type.erl +++ b/src/rabbit_queue_type.erl @@ -4,7 +4,8 @@ discover/1, default/0, is_enabled/1, - declare/2 + declare/2, + delete/4 ]). % copied from rabbit_amqqueue @@ -18,6 +19,12 @@ {'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). %% TODO: this should be controlled by a registry that is populated on boot discover(<<"quorum">>) -> @@ -40,3 +47,12 @@ 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). diff --git a/src/rabbit_quorum_queue.erl b/src/rabbit_quorum_queue.erl index b6eec1b2e933..125eb08fa044 100644 --- a/src/rabbit_quorum_queue.erl +++ b/src/rabbit_quorum_queue.erl @@ -371,9 +371,7 @@ stop(VHost) -> boolean(), boolean(), rabbit_types:username()) -> {ok, QLen :: non_neg_integer()}. - -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 = amqqueue:get_quorum_nodes(Q), From 1e8500b70a6d9e5d6e0e73abf8c9f1333ab2598a Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Wed, 3 Jul 2019 15:42:18 +0100 Subject: [PATCH 04/18] First cut stateful queue type --- src/rabbit_amqqueue.erl | 336 ++++++++++++++++++----------------- src/rabbit_channel.erl | 254 ++++++++++++-------------- src/rabbit_classic_queue.erl | 118 +++++++++++- src/rabbit_fifo_client.erl | 148 +++++++++------ src/rabbit_queue_type.erl | 243 ++++++++++++++++++++++++- src/rabbit_quorum_queue.erl | 99 ++++++----- test/queue_type_SUITE.erl | 216 ++++++++++++++++++++++ test/quorum_queue_SUITE.erl | 13 +- 8 files changed, 1019 insertions(+), 408 deletions(-) create mode 100644 test/queue_type_SUITE.erl diff --git a/src/rabbit_amqqueue.erl b/src/rabbit_amqqueue.erl index 1823603b7a81..5c3ef7ad7bfe 100644 --- a/src/rabbit_amqqueue.erl +++ b/src/rabbit_amqqueue.erl @@ -24,7 +24,7 @@ -export([lookup/1, not_found_or_absent/1, with/2, with/3, with_or_die/2, assert_equivalence/5, check_exclusive_access/2, with_exclusive_access_or_die/3, - stat/1, deliver/2, deliver/3, requeue/4, ack/4, reject/5]). + stat/1, deliver/2, deliver/3, requeue/4, ack/4, 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, @@ -37,7 +37,7 @@ -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([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]). @@ -1071,9 +1071,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()) -> {'ok', pid()} | rabbit_types:error('not_found'). @@ -1134,67 +1133,29 @@ 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(pid() | atom(), {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. + quorum_states()) -> ok. +requeue(QRef, {CTag, MsgIds}, _ChPid, QStates) -> + reject(QRef, true, {CTag, MsgIds}, QStates). -spec ack(pid(), {rabbit_fifo:consumer_tag(), [msg_id()]}, pid(), quorum_states()) -> quorum_states(). +ack(QPid, {CTag, MsgIds}, ChPid, QueueStates) -> + rabbit_queue_type:settle(QPid, CTag, MsgIds, ChPid, 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(pid() | atom(), 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. +reject(QRef, Requeue, {CTag, MsgIds}, QStates) -> + rabbit_queue_type:reject(QRef, CTag, Requeue, MsgIds, QStates). -spec notify_down_all(qpids(), pid()) -> ok_or_errors(). @@ -1229,27 +1190,29 @@ 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()}. - -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)}. + quorum_states(). +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) -> +% {QRef, _} = amqqueue:get_pid(Q), +% rabbit_queue_type:with( +% QRef, +% fun(S) -> +% rabbit_quorum_queue:credit(CTag, Credit, Drain, S) +% end, +% QStates). -spec basic_get(amqqueue:amqqueue(), pid(), boolean(), pid(), rabbit_types:ctag(), #{Name :: atom() => rabbit_fifo_client:state()}) -> @@ -1262,91 +1225,139 @@ basic_get(Q, ChPid, NoAck, LimiterPid, _CTag, _) 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) +basic_get(Q, _ChPid, NoAck, _LimiterPid, CTag, QStates0) 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]) + case rabbit_queue_type:with( + Q, + fun(S0) -> + case rabbit_quorum_queue:basic_get(Q, NoAck, CTag, S0) of + {ok, empty, S} -> + {empty, S}; + {ok, Count, Msg, S} -> + {{ok, Count, Msg}, S}; + {error, Reason} -> + QName = rabbit_quorum_queue:queue_name(S0), + rabbit_misc:protocol_error(internal_error, + "Cannot get a message from quorum queue '~s': ~p", + [rabbit_misc:rs(QName), Reason]) + end + end, + QStates0) of + {empty, QStates} -> + {empty, QStates}; + {{ok, Count, Msg}, QStates} -> + {ok, Count, Msg, QStates} 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'). +-type queue_ref() :: pid() | atom(). %% pid or registered name +-spec basic_consume(amqqueue:amqqueue(), boolean(), pid(), pid(), boolean(), + non_neg_integer(), rabbit_types:ctag(), boolean(), + rabbit_framing:amqp_table(), any(), rabbit_types:username(), + #{Ref :: queue_ref() => rabbit_queue_type:ctx()}) -> + {ok, rabbit_queue_type:ctxs(), 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), - {ok, QState} = rabbit_quorum_queue:basic_consume(Q, NoAck, ChPid, - ConsumerPrefetchCount, - ConsumerTag, - ExclusiveConsume, Args, - ActingUser, - OkMsg, QState0), - {ok, maps:put(Name, QState, QStates)}. - --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 = #{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). + +% get_ctx(Q, Contexts) +% when ?is_amqqueue(Q) andalso is_map(Contexts) -> +% QPid = amqqueue:get_pid(Q), +% Ref = qpid_to_ref(QPid), +% case Contexts of +% #{Ref := Ctx} -> +% Ctx; +% _ -> +% %% not found - initialize +% rabbit_queue_type:init(Q) +% end; +% get_ctx(QPid, Contexts) when is_map(Contexts) -> +% Ref = qpid_to_ref(QPid), +% %% if we use a QPid it should always be initialised +% maps:get(Ref, Contexts). + + +% basic_consume(Q, NoAck, ChPid, LimiterPid, +% LimiterActive, ConsumerPrefetchCount, ConsumerTag, +% ExclusiveConsume, Args, OkMsg, ActingUser, QState) +% when ?amqqueue_is_classic(Q) -> +% QPid = amqqueue:get_pid(Q), +% QName = amqqueue:get_name(Q), +% 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), +% {ok, QState} = rabbit_quorum_queue:basic_consume(Q, NoAck, ChPid, +% ConsumerPrefetchCount, +% ConsumerTag, +% ExclusiveConsume, Args, +% ActingUser, +% OkMsg, QState0), +% {ok, maps:put(Name, QState, QStates)}. + +-spec basic_cancel(amqqueue:amqqueue(), pid(), rabbit_types:ctag(), any(), + rabbit_types:username(), + #{Name :: atom() => rabbit_fifo_client:state()}) -> + {ok, #{Name :: atom() => rabbit_fifo_client:state()}}. +basic_cancel(Q, ChPid, ConsumerTag, OkMsg, ActingUser, QStates) -> + rabbit_queue_type:cancel(Q, ChPid, ConsumerTag, + OkMsg, ActingUser, QStates). +% 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 notify_decorators(amqqueue:amqqueue()) -> 'ok'. @@ -1715,15 +1726,14 @@ deliver(Qs, Delivery = #delivery{flow = Flow, 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 + fun({{QRef, _}, _QName}, S0) -> + rabbit_queue_type:with( + QRef, + fun (S) -> + rabbit_quorum_queue:deliver(Confirm, + Delivery, + S) + end, S0) end, QueueState0, Quorum) end, {QuorumPids, _} = lists:unzip(Quorum), @@ -1754,12 +1764,12 @@ qpids(Qs) -> 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, _} = 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_state({Name, _}, Map) -> +% maps:get(Name, Map). diff --git a/src/rabbit_channel.erl b/src/rabbit_channel.erl index 1f10e6f6de59..770ede144c18 100644 --- a/src/rabbit_channel.erl +++ b/src/rabbit_channel.erl @@ -288,7 +288,7 @@ send_command(Pid, Msg) -> (pid(), rabbit_types:ctag(), boolean(), rabbit_amqqueue:qmsg()) -> 'ok'. deliver(Pid, ConsumerTag, AckRequired, Msg) -> - gen_server2:cast(Pid, {deliver, ConsumerTag, AckRequired, Msg}). + gen_server2:cast(Pid, {deliver, ConsumerTag, AckRequired, [Msg]}). -spec deliver_reply(binary(), rabbit_types:delivery()) -> 'ok'. @@ -664,6 +664,7 @@ handle_cast({deliver, _CTag, _AckReq, _Msg}, State = #ch{cfg = #conf{state = closing}}) -> noreply(State); handle_cast({deliver, ConsumerTag, AckRequired, Msg}, State) -> + % TODO: handle as action noreply(handle_deliver(ConsumerTag, AckRequired, Msg, State)); handle_cast({deliver_reply, _K, _Del}, @@ -737,72 +738,26 @@ handle_cast({reject_publish, MsgSeqNo, _QPid}, State = #ch{unconfirmed = UC}) -> handle_cast({confirm, MsgSeqNos, QPid}, State) -> noreply_coalesce(confirm(MsgSeqNos, QPid, State)). -handle_info({ra_event, {Name, _} = From, _} = Evt, - #ch{queue_states = QueueStates, - queue_names = QNames, - consumer_mapping = ConsumerMapping} = State0) -> - case QueueStates of - #{Name := QState0} -> - QName = rabbit_quorum_queue:queue_name(QState0), - case rabbit_quorum_queue:handle_event(Evt, QState0) of - {{delivery, CTag, Msgs}, QState1} -> - AckRequired = case maps:find(CTag, ConsumerMapping) of - error -> - true; - {ok, {_, {NoAck, _, _, _}}} -> - not NoAck - end, - QState2 = case AckRequired of - false -> - {MsgIds, _} = lists:unzip(Msgs), - {ok, FS} = rabbit_quorum_queue:ack(CTag, MsgIds, QState1), - FS; - true -> - QState1 - end, - State = lists:foldl( - fun({MsgId, {MsgHeader, Msg}}, Acc) -> - IsDelivered = maps:is_key(delivery_count, MsgHeader), - Msg1 = add_delivery_count_header(MsgHeader, Msg), - handle_deliver(CTag, AckRequired, - {QName, From, MsgId, IsDelivered, Msg1}, - Acc) - end, State0#ch{queue_states = maps:put(Name, QState2, QueueStates)}, Msgs), - noreply(State); - {internal, MsgSeqNos, Actions, QState1} -> - State = State0#ch{queue_states = maps:put(Name, QState1, QueueStates)}, - %% execute actions - WriterPid = State#ch.cfg#conf.writer_pid, - lists:foreach(fun ({send_credit_reply, Avail}) -> - ok = rabbit_writer:send_command( - WriterPid, - #'basic.credit_ok'{available = - Avail}); - ({send_drained, {CTag, Credit}}) -> - ok = rabbit_writer:send_command( - WriterPid, - #'basic.credit_drained'{consumer_tag = CTag, - credit_drained = Credit}) - end, Actions), - noreply_coalesce(confirm(MsgSeqNos, Name, State)); - eol -> - State1 = handle_consuming_queue_down_or_eol(Name, State0), - State2 = handle_delivering_queue_down(Name, State1), - {ConfirmMXs, RejectMXs, UC1} = - unconfirmed_messages:forget_ref(Name, State2#ch.unconfirmed), - %% Deleted queue is a special case. - %% Do not nack the "rejected" messages. - State3 = record_confirms(ConfirmMXs ++ RejectMXs, - State2#ch{unconfirmed = UC1}), - erase_queue_stats(QName), - noreply_coalesce( - State3#ch{queue_states = maps:remove(Name, QueueStates), - queue_names = maps:remove(Name, QNames)}) - end; - _ -> - %% the assumption here is that the queue state has been cleaned up and - %% this is a residual Ra notification - noreply_coalesce(State0) +handle_info({ra_event, From, _} = Evt, + #ch{queue_states = QueueStates0} = State0) -> + QRef = qpid_to_ref(From), + case rabbit_queue_type:handle_event(QRef, Evt, QueueStates0) of + {ok, QState1, Actions} -> + State1 = State0#ch{queue_states = QState1}, + State = handle_queue_actions(Actions, State1), + noreply_coalesce(State); + eol -> + State1 = handle_consuming_queue_down_or_eol(QRef, State0), + State2 = handle_delivering_queue_down(QRef, State1), + {ConfirmMXs, RejectMXs, UC1} = + unconfirmed_messages:forget_ref(QRef, State2#ch.unconfirmed), + %% Deleted queue is a special case. + %% Do not nack the "rejected" messages. + State3 = record_confirms(ConfirmMXs ++ RejectMXs, + State2#ch{unconfirmed = UC1}), + erase_queue_stats(rabbit_queue_type:name(QRef, QueueStates0)), + noreply_coalesce( + State3#ch{queue_states = maps:remove(QRef, QueueStates0)}) end; handle_info({bump_credit, Msg}, State) -> @@ -859,10 +814,13 @@ handle_info(tick, State0 = #ch{queue_states = QueueStates0}) -> _ -> ok end, QueueStates1 = - maps:filter(fun(_, QS) -> - QName = rabbit_quorum_queue:queue_name(QS), + maps:filter(fun(QRef, _) -> + QName = rabbit_queue_type:name(QRef, QueueStates0), + rabbit_log:info("tick QName ~w", [QName]), [] /= rabbit_amqqueue:lookup([QName]) end, QueueStates0), + rabbit_log:info("channel tick pre ~w post ~w", [maps:size(QueueStates0), + maps:size(QueueStates1)]), case evaluate_consumer_timeout(State0#ch{queue_states = QueueStates1}) of {noreply, State} -> noreply(init_tick_timer(reset_tick_timer(State))); @@ -1246,6 +1204,7 @@ record_rejects(MXs, State = #ch{rejected = R, tx = Tx}) -> record_confirms([], State) -> State; record_confirms(MXs, State = #ch{confirmed = C}) -> + rabbit_log:info("recording confirms ~w", [MXs]), State#ch{confirmed = [MXs | C]}. handle_method({Method, Content}, State) -> @@ -1789,11 +1748,11 @@ basic_consume(QueueName, NoAck, ConsumerPrefetch, ActualConsumerTag, ConsumerPrefetch, ActualConsumerTag, ExclusiveConsume, Args, ok_msg(NoWait, #'basic.consume_ok'{ - consumer_tag = ActualConsumerTag}), + consumer_tag = ActualConsumerTag}), Username, QueueStates0), Q} end) of - {{ok, QueueStates}, Q} when ?is_amqqueue(Q) -> + {{ok, QueueStates, Actions}, Q} when ?is_amqqueue(Q) -> QPid = amqqueue:get_pid(Q), QName = amqqueue:get_name(Q), CM1 = maps:put( @@ -1804,23 +1763,10 @@ basic_consume(QueueName, NoAck, ConsumerPrefetch, ActualConsumerTag, NoAck, QPid, QName, State#ch{consumer_mapping = CM1, queue_states = QueueStates}), + State2 = handle_queue_actions(Actions, State1), {ok, case NoWait of - true -> consumer_monitor(ActualConsumerTag, State1); - false -> State1 - end}; - {ok, Q} when ?is_amqqueue(Q) -> - QPid = amqqueue:get_pid(Q), - QName = amqqueue:get_name(Q), - CM1 = maps:put( - ActualConsumerTag, - {Q, {NoAck, ConsumerPrefetch, ExclusiveConsume, Args}}, - ConsumerMapping), - State1 = track_delivering_queue( - NoAck, QPid, QName, - State#ch{consumer_mapping = CM1}), - {ok, case NoWait of - true -> consumer_monitor(ActualConsumerTag, State1); - false -> State1 + true -> consumer_monitor(ActualConsumerTag, State2); + false -> State2 end}; {{error, exclusive_consume_unavailable} = E, _Q} -> E; @@ -1848,11 +1794,11 @@ consumer_monitor(ConsumerTag, track_delivering_queue(NoAck, QPid, QName, State = #ch{queue_names = QNames, - queue_monitors = QMons, + % queue_monitors = QMons, delivering_queues = DQ}) -> QRef = qpid_to_ref(QPid), State#ch{queue_names = maps:put(QRef, QName, QNames), - queue_monitors = maybe_monitor(QRef, QMons), + % queue_monitors = maybe_monitor(QRef, QMons), delivering_queues = case NoAck of true -> DQ; false -> sets:add_element(QRef, DQ) @@ -1860,11 +1806,12 @@ track_delivering_queue(NoAck, QPid, QName, handle_publishing_queue_down(QPid, Reason, State = #ch{unconfirmed = UC, - queue_names = QNames}) + queue_states = QStates}) when ?IS_CLASSIC(QPid) -> - case maps:get(QPid, QNames, none) of + case rabbit_queue_type:name(QPid, QStates) of %% The queue is unknown, the confirm must have been processed already - none -> State; + undefined -> + State; _QName -> case {rabbit_misc:is_abnormal_exit(Reason), Reason} of {true, _} -> @@ -2019,7 +1966,7 @@ internal_reject(Requeue, Acked, Limiter, QueueStates = foreach_per_queue( fun({QPid, CTag}, MsgIds, Acc0) -> rabbit_amqqueue:reject(QPid, Requeue, {CTag, MsgIds}, - self(), Acc0) + Acc0) end, Acked, QueueStates0), ok = notify_limiter(Limiter, Acked), State#ch{queue_states = QueueStates}. @@ -2046,6 +1993,8 @@ record_sent(Type, Tag, AckRequired, end, DeliveredAt = os:system_time(millisecond), rabbit_trace:tap_out(Msg, ConnName, ChannelNum, Username, TraceState), + rabbit_log:info("channel recording sent ~w ~w ~w", + [Tag, DeliveryTag, AckRequired]), UAMQ1 = case AckRequired of true -> ?QUEUE:in({DeliveryTag, Tag, DeliveredAt, {QPid, MsgId}}, UAMQ); @@ -2180,13 +2129,14 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ mandatory = Mandatory, confirm = Confirm, msg_seq_no = MsgSeqNo}, - DelQNames}, State = #ch{queue_names = QNames, - queue_monitors = QMons, - queue_states = QueueStates0}) -> + DelQNames}, State0 = #ch{queue_names = _QNames, + queue_monitors = _QMons, + queue_states = QueueStates0}) -> Qs = rabbit_amqqueue:lookup(DelQNames), - {DeliveredQPids, DeliveredQQPids, QueueStates} = - rabbit_amqqueue:deliver(Qs, Delivery, QueueStates0), - AllDeliveredQRefs = DeliveredQPids ++ [N || {N, _} <- DeliveredQQPids], + {QueueStates, Actions} = + rabbit_queue_type:deliver(Qs, Delivery, QueueStates0), + State1 = handle_queue_actions(Actions, State0#ch{queue_states = QueueStates}), + % AllDeliveredQRefs = DeliveredQPids ++ [N || {N, _} <- DeliveredQQPids], %% The maybe_monitor_all/2 monitors all queues to which we %% delivered. But we want to monitor even queues we didn't deliver %% to, since we need their 'DOWN' messages to clean @@ -2197,46 +2147,36 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ %% ...and we need to add even non-delivered queues to queue_names %% since alternative algorithms to update queue_names less %% frequently would in fact be more expensive in the common case. - {QNames1, QMons1} = - lists:foldl(fun (Q, {QNames0, QMons0}) when ?is_amqqueue(Q) -> - QPid = amqqueue:get_pid(Q), - QRef = qpid_to_ref(QPid), - QName = amqqueue:get_name(Q), - case ?IS_CLASSIC(QRef) of - true -> - SPids = amqqueue:get_slave_pids(Q), - NewQNames = - maps:from_list([{Ref, QName} || Ref <- [QRef | SPids]]), - {maps:merge(NewQNames, QNames0), - maybe_monitor_all([QPid | SPids], QMons0)}; - false -> - {maps:put(QRef, QName, QNames0), QMons0} - end - end, - {QNames, QMons}, Qs), - State1 = State#ch{queue_names = QNames1, - queue_monitors = QMons1}, %% NB: the order here is important since basic.returns must be %% sent before confirms. + %% TODO: fix - HACK TO WORK OUT ALL QREFS + AllDeliveredQRefs = lists:foldl(fun (Q, Acc) -> + QRef = qpid_to_ref( + amqqueue:get_pid(Q)), + %% slave pids are always pids and thus refs + SPids = amqqueue:get_slave_pids(Q), + Acc ++ [QRef | SPids] + end, [], Qs), ok = process_routing_mandatory(Mandatory, AllDeliveredQRefs, Message, State1), - AllDeliveredQNames = [ QName || QRef <- AllDeliveredQRefs, - {ok, QName} <- [maps:find(QRef, QNames1)]], - State2 = process_routing_confirm(Confirm, - AllDeliveredQRefs, - AllDeliveredQNames, - MsgSeqNo, - XName, State1), - case rabbit_event:stats_level(State, #ch.stats_timer) of + QNames1 = State1#ch.queue_names, + AllDeliveredQNames = [QName || QRef <- AllDeliveredQRefs, + {ok, QName} <- [maps:find(QRef, QNames1)]], + State = process_routing_confirm(Confirm, + AllDeliveredQRefs, + AllDeliveredQNames, + MsgSeqNo, + XName, State1), + case rabbit_event:stats_level(State1, #ch.stats_timer) of fine -> ?INCR_STATS(exchange_stats, XName, 1, publish), [?INCR_STATS(queue_exchange_stats, {QName, XName}, 1, publish) || - QRef <- AllDeliveredQRefs, - {ok, QName} <- [maps:find(QRef, QNames1)]]; + QRef <- AllDeliveredQRefs, + {ok, QName} <- [maps:find(QRef, QNames1)]]; _ -> ok end, - State2#ch{queue_states = QueueStates}. + State. process_routing_mandatory(_Mandatory = true, _RoutedToQs = [], @@ -2256,15 +2196,23 @@ process_routing_confirm(false, _, _, _, _, State) -> process_routing_confirm(true, [], _, MsgSeqNo, XName, State) -> record_confirms([{MsgSeqNo, XName}], State); process_routing_confirm(true, QRefs, QNames, MsgSeqNo, XName, State) -> + rabbit_log:info("recording unconfirmed ~w ~w", [QRefs, MsgSeqNo]), State#ch{unconfirmed = unconfirmed_messages:insert(MsgSeqNo, QNames, QRefs, XName, State#ch.unconfirmed)}. -confirm(MsgSeqNos, QRef, State = #ch{queue_names = QNames, unconfirmed = UC}) -> +confirm(MsgSeqNos, QRef, State = #ch{queue_states = QStates, unconfirmed = UC}) -> %% NOTE: if queue name does not exist here it's likely that the ref also %% does not exist in unconfirmed messages. %% Neither does the 'ignore' atom, so it's a reasonable fallback. - QName = maps:get(QRef, QNames, ignore), + % QName = maps:get(QRef, QNames, ignore), + QName = case rabbit_queue_type:name(QRef, QStates) of + undefined -> + ignore; + N -> + N + end, + rabbit_log:info("confirm queue name ~w ~w ~w", [QName, QRef, MsgSeqNos]), {ConfirmMXs, RejectMXs, UC1} = unconfirmed_messages:confirm_multiple_msg_ref(MsgSeqNos, QName, QRef, UC), %% NB: don't call noreply/1 since we don't want to send confirms. @@ -2272,8 +2220,10 @@ confirm(MsgSeqNos, QRef, State = #ch{queue_names = QNames, unconfirmed = UC}) -> record_rejects(RejectMXs, State1). send_confirms_and_nacks(State = #ch{tx = none, confirmed = [], rejected = []}) -> + rabbit_log:info("sending no confirms ~n", []), State; send_confirms_and_nacks(State = #ch{tx = none, confirmed = C, rejected = R}) -> + rabbit_log:info("sending confirms ~w", [C]), case rabbit_node_monitor:pause_partition_guard() of ok -> Confirms = lists:append(C), @@ -2704,7 +2654,12 @@ handle_method(#'exchange.declare'{exchange = ExchangeNameBin, check_not_default_exchange(ExchangeName), _ = rabbit_exchange:lookup_or_die(ExchangeName). -handle_deliver(ConsumerTag, AckRequired, +handle_deliver(CTag, Ack, Msgs, State) -> + lists:foldl(fun(Msg, S) -> + handle_deliver0(CTag, Ack, Msg, S) + end, State, Msgs). + +handle_deliver0(ConsumerTag, AckRequired, Msg = {_QName, QPid, _MsgId, Redelivered, #basic_message{exchange_name = ExchangeName, routing_keys = [RoutingKey | _CcRoutes], @@ -2773,14 +2728,14 @@ maybe_monitor(QPid, QMons) when ?IS_CLASSIC(QPid) -> maybe_monitor(_, QMons) -> QMons. -maybe_monitor_all([], S) -> S; %% optimisation -maybe_monitor_all([Item], S) -> maybe_monitor(Item, S); %% optimisation -maybe_monitor_all(Items, S) -> lists:foldl(fun maybe_monitor/2, S, Items). +% maybe_monitor_all([], S) -> S; %% optimisation +% maybe_monitor_all([Item], S) -> maybe_monitor(Item, S); %% optimisation +% maybe_monitor_all(Items, S) -> lists:foldl(fun maybe_monitor/2, S, Items). -add_delivery_count_header(#{delivery_count := Count}, Msg) -> - rabbit_basic:add_header(<<"x-delivery-count">>, long, Count, Msg); -add_delivery_count_header(_, Msg) -> - Msg. +% add_delivery_count_header(#{delivery_count := Count}, Msg) -> +% rabbit_basic:add_header(<<"x-delivery-count">>, long, Count, Msg); +% add_delivery_count_header(_, Msg) -> +% Msg. qpid_to_ref(Pid) when is_pid(Pid) -> Pid; qpid_to_ref({Name, _}) -> Name; @@ -2822,3 +2777,24 @@ evaluate_consumer_timeout(State0 = #ch{cfg = #conf{channel = Channel, _ -> {noreply, State0} end. + +handle_queue_actions(Actions, #ch{} = State0) -> + WriterPid = State0#ch.cfg#conf.writer_pid, + lists:foldl( + fun ({send_credit_reply, Avail}, S0) -> + ok = rabbit_writer:send_command(WriterPid, + #'basic.credit_ok'{available = Avail}), + S0; + ({send_drained, {CTag, Credit}}, S0) -> + ok = rabbit_writer:send_command( + WriterPid, + #'basic.credit_drained'{consumer_tag = CTag, + credit_drained = Credit}), + S0; + ({monitor, Pid, _QRef}, #ch{queue_monitors = Mons} = S0) -> + S0#ch{queue_monitors = pmon:monitor(Pid, Mons)}; + ({settled, QRef, MsgSeqNos}, S0) -> + confirm(MsgSeqNos, QRef, S0); + ({deliver, CTag, AckRequired, Msgs}, S0) -> + handle_deliver(CTag, AckRequired, Msgs, S0) + end, State0, Actions). diff --git a/src/rabbit_classic_queue.erl b/src/rabbit_classic_queue.erl index b5cf93c32082..4f33c1143f9a 100644 --- a/src/rabbit_classic_queue.erl +++ b/src/rabbit_classic_queue.erl @@ -4,10 +4,26 @@ -include("amqqueue.hrl"). -include_lib("rabbit_common/include/rabbit.hrl"). +-record(?MODULE, {pid :: pid()}). +-define(STATE, ?MODULE). + +-opaque state() :: #?STATE{}. + +-export_type([state/0]). + -export([ is_enabled/0, declare/2, - delete/4 + delete/4, + stat/1, + init/1, + consume/3, + cancel/6, + handle_event/2, + deliver/2, + settle/4, + reject/4, + credit/4 ]). -export([delete_crashed/1, @@ -65,6 +81,106 @@ delete(Q, IfUnused, IfEmpty, ActingUser) when ?amqqueue_is_classic(Q) -> {ok, 0} end. +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) -> + #?STATE{pid = amqqueue:get_pid(Q)}. + +consume(Q, Spec, State) when ?amqqueue_is_classic(Q) -> + QPid = amqqueue:get_pid(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 + {ok, State, [{monitor, QPid, QPid}]}; + Err -> + Err + end. + +cancel(Q, ChPid, ConsumerTag, OkMsg, ActingUser, State) -> + QPid = amqqueue:get_pid(Q), + case delegate:invoke(QPid, {gen_server2, call, + [{basic_cancel, ChPid, ConsumerTag, + OkMsg, ActingUser}, infinity]}) of + ok -> + {ok, State}; + Err -> Err + end. + +-spec settle(rabbit_types:ctag(), [non_neg_integer()], + ChPid :: pid(), state()) -> + state(). +settle(_CTag, MsgIds, ChPid, State) -> + delegate:invoke_no_result(State#?STATE.pid, + {gen_server2, cast, [{ack, MsgIds, ChPid}]}), + State. + +reject(_CTag, Requeue, MsgIds, State) -> + ChPid = self(), + ok = delegate:invoke_no_result(State#?STATE.pid, + {gen_server2, cast, + [{reject, 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(_Evt, State) -> + {ok, State, []}. + +-spec deliver([{amqqueue:amqqueue(), state()}], + Delivery :: term()) -> + {[{amqqueue:amqqueue(), state()}], rabbit_queue_type:actions()}. +deliver(Qs, #delivery{flow = Flow, + confirm = _Confirm} = Delivery) -> + {MPids, SPids, Actions} = qpids(Qs), + 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}, + rabbit_log:info("rabbit_classic_queue delivery confirm ~w", [_Confirm, MMsg]), + delegate:invoke_no_result(MPids, {gen_server2, cast, [MMsg]}), + delegate:invoke_no_result(SPids, {gen_server2, cast, [SMsg]}), + %% TODO: monitors + {Qs, Actions}. + +qpids(Qs) -> + lists:foldl(fun ({Q, _}, {MPidAcc, SPidAcc, Actions0}) -> + QPid = amqqueue:get_pid(Q), + SPids = amqqueue:get_slave_pids(Q), + Actions = [{monitor, QPid, QPid} + | [{monitor, P, QPid} || P <- SPids]] ++ Actions0, + {[QPid | MPidAcc], SPidAcc ++ SPids, Actions} + end, {[], [], []}, Qs). +%% internal-ish -spec wait_for_promoted_or_stopped(amqqueue:amqqueue()) -> {promoted, amqqueue:amqqueue()} | {stopped, amqqueue:amqqueue()} | diff --git a/src/rabbit_fifo_client.erl b/src/rabbit_fifo_client.erl index 136800cc9992..f41fbe80a5df 100644 --- a/src/rabbit_fifo_client.erl +++ b/src/rabbit_fifo_client.erl @@ -60,6 +60,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(state, {cluster_name :: cluster_name(), @@ -233,7 +234,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. @@ -246,17 +247,14 @@ settle(ConsumerTag, [_|_] = MsgIds, %% the sending rate. %% -spec return(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) -> - {ok, state()}. + state(). return(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) -> Node = pick_node(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) -> @@ -267,7 +265,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. @@ -280,16 +278,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(). discard(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) -> Node = pick_node(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) -> @@ -300,7 +296,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 @@ -343,13 +339,24 @@ 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), + rabbit_log:info("checkout ~w AckRequired: ~w", [ConsumerTag, Ack]), + + 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 %% @@ -363,22 +370,20 @@ 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), %% the last received msgid provides us with the delivery count if we %% add one as it is 0 indexed - C = maps:get(ConsumerTag, CDels, #consumer{last_msg_id = -1}), + C = maps:get(ConsumerTag, CDels), Node = pick_node(State0), 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 @@ -480,14 +485,15 @@ update_machine_state(Node, Conf) -> %% used to {@link settle/3.} (roughly: AMQP 0.9.1 ack) message once finished %% with them. -spec handle_ra_event(ra_server_id(), ra_server_proc:ra_event_body(), state()) -> - {internal, Correlators :: [term()], actions(), state()} | - {rabbit_fifo:client_msg(), state()} | eol. + {internal, actions(), state()} | + {deliver, rabbit_types:ctag(), boolean(), [rabbit_amqqueue:qmsg()]} | eol. handle_ra_event(From, {applied, Seqs}, #state{soft_limit = SftLmt, unblock_handler = UnblockFun} = State0) -> - {Corrs, Actions, State1} = lists:foldl(fun seq_applied/2, + {Corrs, Actions0, State1} = lists:foldl(fun seq_applied/2, {[], [], State0#state{leader = From}}, Seqs), + Actions = [{settled, qref(From), Corrs} | lists:reverse(Actions0)], case maps:size(State1#state.pending) < SftLmt of true when State1#state.slow == true -> % we have exited soft limit state @@ -517,38 +523,38 @@ 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(Leader, {machine, {delivery, _ConsumerTag, _} = Del}, State0) -> handle_delivery(Leader, Del, State0); 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) -> % TODO: how should these be handled? re-sent on timer or try random - {internal, [], [], State0}; + {ok, State0, []}; handle_ra_event(_From, {rejected, {not_leader, Leader, Seq}}, State0) -> % ?INFO("rabbit_fifo_client: rejected ~b not leader ~w leader: ~w~n", % [Seq, From, Leader]), State1 = State0#state{leader = Leader}, State = resend(Seq, State1), - {internal, [], [], State}; + {ok, State, []}; handle_ra_event(_, timeout, #state{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. @@ -645,16 +651,33 @@ resend_all_pending(#state{pending = Pend} = State) -> Seqs = lists:sort(maps:keys(Pend)), lists:foldl(fun resend/2, State, Seqs). -handle_delivery(Leader, {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], + {ok, State} = settle(Tag, MsgIds, State0), + {ok, State, [Deliver]}. + + +handle_delivery(Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs}, + #state{cluster_name = QName, + consumer_deliveries = CDels0} = State0) -> + QRef = qref(Leader), {LastId, _} = lists:last(IdMsgs), + 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, @@ -665,33 +688,48 @@ handle_delivery(Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs} = Del0, %% out messages to the main queue to ensure they don't get stuck in %% case the node never comes back. Missing = get_missing_deliveries(Leader, Prev+1, FstId-1, Tag), - Del = {delivery, Tag, Missing ++ IdMsgs}, - {Del, State0#state{consumer_deliveries = - update_consumer(Tag, LastId, - length(IdMsgs) + NumMissing, - C, CDels0)}}; + 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(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}}) -> + IsDelivered = maps:is_key(delivery_count, MsgHeader), + Msg = add_delivery_count_header(MsgHeader, Msg0), + {QName, QRef, MsgId, IsDelivered, Msg} + end, Msgs). + +add_delivery_count_header(#{delivery_count := Count}, Msg) -> + rabbit_basic:add_header(<<"x-delivery-count">>, long, Count, Msg); +add_delivery_count_header(_, Msg) -> + Msg. + 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). @@ -773,3 +811,5 @@ find_leader([Server | Servers]) -> find_leader(Servers) end. +qref({Ref, _}) -> Ref; +qref(Ref) -> Ref. diff --git a/src/rabbit_queue_type.erl b/src/rabbit_queue_type.erl index 985b93b652fd..4062881060e0 100644 --- a/src/rabbit_queue_type.erl +++ b/src/rabbit_queue_type.erl @@ -1,16 +1,70 @@ -module(rabbit_queue_type). +-include("amqqueue.hrl"). -export([ discover/1, default/0, is_enabled/1, declare/2, - delete/4 + delete/4, + stat/1, + new/2, + consume/3, + cancel/6, + handle_event/3, + deliver/3, + settle/5, + reject/5, + credit/5, + + name/2 ]). +%% temporary +-export([with/3]). + +-type queue_ref() :: pid() | atom(). +-type queue_name() :: rabbit_types:r(queue). +-type queue_state() :: term(). + +%% 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()} | + {deliver, rabbit_type:ctag(), boolean(), [rabbit_amqqueue:qmsg()]}. + +-type actions() :: [action()]. + +-record(ctx, {module :: module(), + name :: queue_name(), + state :: queue_state()}). + +-opaque ctxs() :: #{queue_ref() => #ctx{}}. + +-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()}. + + +-export_type([ctxs/0, + consume_spec/0, + action/0, + actions/0]). + + % copied from rabbit_amqqueue -type absent_reason() :: 'nodedown' | 'crashed' | stopped | timeout. +%% intitialise and return a queue type specific session context +-callback init(amqqueue:amqqueue()) -> term(). + %% is the queue type feature enabled -callback is_enabled() -> boolean(). @@ -26,6 +80,38 @@ rabbit_types:ok(non_neg_integer()) | rabbit_types:error(in_use | not_empty). +-callback consume(amqqueue:amqqueue(), + consume_spec(), + queue_state()) -> + {ok, queue_state(), actions()} | {error, term()}. + +-callback cancel(amqqueue:amqqueue(), pid(), + rabbit_types:ctag(), + term(), + rabbit_types:username(), + queue_state()) -> + {ok, queue_state(), actions()} | {error, term()}. + +-callback handle_event(Event :: term(), + queue_state()) -> + {ok, queue_state(), actions()} | {error, term()} | eol. + +-callback deliver([{amqqueue:amqqueue(), queue_state()}], + Delivery :: term()) -> + {[{amqqueue:amqqueue(), queue_state()}], actions()}. + +-callback settle(rabbit_types:ctag(), [non_neg_integer()], + ChPid :: pid(), queue_state()) -> + queue_state(). + +-callback reject(rabbit_types:ctag(), Requeue :: boolean(), + MsgIds :: [non_neg_integer()], queue_state()) -> + queue_state(). + +-callback credit(rabbit_types:ctag(), + non_neg_integer(), Drain :: boolean(), queue_state()) -> + queue_state(). + %% TODO: this should be controlled by a registry that is populated on boot discover(<<"quorum">>) -> rabbit_quorum_queue; @@ -47,12 +133,159 @@ declare(Q, Node) -> Mod = amqqueue:get_type(Q), Mod:declare(Q, Node). --spec delete(amqqueue:amqqueue(), - boolean(), - boolean(), - rabbit_types:username()) -> +-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 stat(amqqueue:amqqueue()) -> + {'ok', non_neg_integer(), non_neg_integer()}. +stat(Q) -> + Mod = amqqueue:get_type(Q), + Mod:stat(Q). + +-spec new(amqqueue:amqqueue(), ctxs()) -> ctxs(). +new(Q, Ctxs) when ?is_amqqueue(Q) -> + Mod = amqqueue:get_type(Q), + Name = amqqueue:get_name(Q), + Ctx = #ctx{module = Mod, + name = Name, + state = Mod:init(Q)}, + Ctxs#{qref(Q) => Ctx}. + +-spec consume(amqqueue:amqqueue(), consume_spec(), ctxs()) -> + {ok, ctxs(), actions()} | {error, term()}. +consume(Q, Spec, Ctxs) -> + #ctx{state = State0} = Ctx = get_ctx(Q, Ctxs), + Mod = amqqueue:get_type(Q), + case Mod:consume(Q, Spec, State0) of + {ok, State, Actions} -> + {ok, set_ctx(Q, Ctx#ctx{state = State}, Ctxs), Actions}; + Err -> + Err + end. + +%% TODO switch to cancel spec api +-spec cancel(amqqueue:amqqueue(), pid(), + rabbit_types:ctag(), + term(), + rabbit_types:username(), + ctxs()) -> + {ok, ctxs(), actions()} | {error, term()}. +cancel(Q, ChPid, Tag, OkMsg, ActiveUser, Ctxs) -> + #ctx{state = State0} = Ctx = get_ctx(Q, Ctxs), + Mod = amqqueue:get_type(Q), + case Mod:cancel(Q, ChPid, Tag, OkMsg, ActiveUser, State0) of + {ok, State} -> + {ok, set_ctx(Q, Ctx#ctx{state = State}, Ctxs)}; + Err -> + Err + end. + +%% messages sent from queues +-spec handle_event(queue_ref(), term(), ctxs()) -> + {ok, ctxs(), actions()} | {error, term()}. +handle_event(QRef, Evt, Ctxs) -> + #ctx{module = Mod, + state = State0} = Ctx = get_ctx(QRef, Ctxs), + case Mod:handle_event(Evt, State0) of + {ok, State, Actions} -> + {ok, set_ctx(QRef, Ctx#ctx{state = State}, Ctxs), Actions}; + Err -> + Err + end. + + +deliver(Qs, Delivery, Ctxs) -> + %% sort by queue type - then dispatch each group + ByType = lists:foldl(fun (Q, Acc) -> + T = amqqueue:get_type(Q), + Ctx = get_ctx(Q, Ctxs), + maps:update_with( + T, fun (A) -> + Ctx = get_ctx(Q, Ctxs), + [{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), + {lists:foldl( + fun({Q, S}, Acc) -> + Ctx = get_ctx(Q, Acc), + set_ctx(qref(Q), Ctx#ctx{state = S}, Acc) + end, Ctxs, Xs), Actions}. + + +settle(QRef, CTag, MsgIds, ChPid, Ctxs) -> + #ctx{state = State0, + module = Mod} = Ctx = get_ctx(QRef, Ctxs), + State = Mod:settle(CTag, MsgIds, ChPid, State0), + set_ctx(QRef, Ctx#ctx{state = State}, Ctxs). + +reject(QRef, CTag, Requeue, MsgIds, Ctxs) -> + #ctx{state = State0, + module = Mod} = Ctx = get_ctx(QRef, Ctxs), + State = Mod:reject(CTag, Requeue, MsgIds, State0), + set_ctx(QRef, Ctx#ctx{state = State}, Ctxs). + +credit(Q, CTag, Credit, Drain, Ctxs) -> + #ctx{state = State0, + module = Mod} = Ctx = get_ctx(Q, Ctxs), + State = Mod:settle(CTag, Credit, Drain, State0), + set_ctx(Q, Ctx#ctx{state = State}, Ctxs). + +name(QRef, Ctxs) -> + case Ctxs of + #{QRef := Ctx} -> + Ctx#ctx.name; + _ -> + undefined + 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, Contexts) when ?is_amqqueue(Q) -> + Ref = qref(Q), + case Contexts of + #{Ref := Ctx} -> + Ctx; + _ -> + %% 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(QPid, Contexts) when is_map(Contexts) -> + Ref = qref(QPid), + %% if we use a QPid it should always be initialised + maps:get(Ref, Contexts). + +set_ctx(Q, Ctx, Contexts) + when ?is_amqqueue(Q) andalso is_map(Contexts) -> + Ref = qref(Q), + maps:put(Ref, Ctx, Contexts); +set_ctx(QPid, Ctx, Contexts) when is_map(Contexts) -> + Ref = qref(QPid), + maps:put(Ref, Ctx, Contexts). + +qref(Pid) when is_pid(Pid) -> + Pid; +qref(Q) when ?is_amqqueue(Q) -> + qref(amqqueue:get_pid(Q)); +qref({Name, _}) -> Name; +%% assume it already is a ref +qref(Ref) -> Ref. diff --git a/src/rabbit_quorum_queue.erl b/src/rabbit_quorum_queue.erl index 125eb08fa044..3f1dc201b935 100644 --- a/src/rabbit_quorum_queue.erl +++ b/src/rabbit_quorum_queue.erl @@ -18,13 +18,13 @@ -behaviour(rabbit_queue_type). --export([init_state/2, handle_event/2]). +-export([init/1, handle_event/2]). -export([recover/1, stop/1, delete/4, delete_immediately/2]). -export([info/1, info/2, stat/1, infos/1]). --export([ack/3, reject/4, basic_get/4, basic_consume/10, basic_cancel/4]). +-export([settle/4, reject/4, basic_get/4, consume/3, cancel/6]). -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]). @@ -45,6 +45,7 @@ -export([is_enabled/0, declare/2]). + %%-include_lib("rabbit_common/include/rabbit.hrl"). -include_lib("rabbit.hrl"). -include_lib("stdlib/include/qlc.hrl"). @@ -84,15 +85,14 @@ is_enabled() -> %%---------------------------------------------------------------------------- --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), + {Name, _LeaderNode} = Leader = amqqueue:get_pid(Q), Nodes = amqqueue:get_quorum_nodes(Q), + QName = amqqueue:get_name(Q), %% Ensure the leader is listed first Servers0 = [{Name, N} || N <- Nodes], Servers = [Leader | lists:delete(Leader, Servers0)], @@ -102,9 +102,7 @@ init_state({Name, _}, QName = #resource{}) -> -spec handle_event({'ra_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) -> rabbit_fifo_client:handle_ra_event(From, Evt, QState). @@ -440,18 +438,16 @@ 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()}. +settle(CTag, MsgIds, _ChPid, QState) -> + {_, S} = rabbit_fifo_client:settle(quorum_ctag(CTag), MsgIds, QState), + S. -reject(true, CTag, MsgIds, QState) -> +-spec reject(rabbit_types:ctag(), Confirm :: boolean(), [msg_id()], + rabbit_fifo_client:state()) -> + rabbit_fifo_client:state(). +reject(CTag, true, MsgIds, QState) -> rabbit_fifo_client:return(quorum_ctag(CTag), MsgIds, QState); -reject(false, CTag, MsgIds, QState) -> +reject(CTag, false, MsgIds, QState) -> rabbit_fifo_client:discard(quorum_ctag(CTag), MsgIds, QState). credit(CTag, Credit, Drain, QState) -> @@ -462,7 +458,6 @@ credit(CTag, Credit, Drain, QState) -> {'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), @@ -487,16 +482,23 @@ basic_get(Q, NoAck, CTag0, QState0) when ?amqqueue_is_quorum(Q) -> {error, timeout} end. --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()}. - -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, State0) 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), @@ -514,12 +516,13 @@ basic_consume(Q, NoAck, ChPid, prefetch => ConsumerPrefetchCount, args => Args, username => ActingUser}, - {ok, QState} = rabbit_fifo_client:checkout(ConsumerTag, - Prefetch, - ConsumerMeta, - QState0), - {ok, {_, SacResult}, _} = ra:local_query(QPid, - fun rabbit_fifo:query_single_active_consumer/1), + {ok, State} = rabbit_fifo_client:checkout(ConsumerTag, + Prefetch, + ConsumerMeta, + State0), + {ok, {_, SacResult}, _} = ra:local_query( + QPid, + fun rabbit_fifo:query_single_active_consumer/1), SingleActiveConsumerOn = single_active_consumer_on(Q), {IsSingleActiveConsumer, ActivityStatus} = case {SingleActiveConsumerOn, SacResult} of @@ -534,16 +537,17 @@ basic_consume(Q, NoAck, ChPid, %% TODO: emit as rabbit_fifo effect rabbit_core_metrics:consumer_created(ChPid, ConsumerTag, ExclusiveConsume, not NoAck, QName, - ConsumerPrefetchCount, IsSingleActiveConsumer, + ConsumerPrefetchCount, + IsSingleActiveConsumer, ActivityStatus, Args), - {ok, QState}. + {ok, State, []}. --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) -> +cancel(_Q, ChPid, ConsumerTag, OkMsg, _ActingUser, State) -> maybe_send_reply(ChPid, OkMsg), - rabbit_fifo_client:cancel_checkout(quorum_ctag(ConsumerTag), QState0). + rabbit_fifo_client:cancel_checkout(quorum_ctag(ConsumerTag), State). -spec stateless_deliver(amqqueue:ra_server_id(), rabbit_types:delivery()) -> 'ok'. @@ -561,6 +565,13 @@ deliver(true, Delivery, QState0) -> rabbit_fifo_client:enqueue(Delivery#delivery.msg_seq_no, Delivery#delivery.message, QState0). +deliver(QSs, #delivery{confirm = Confirm} = Delivery) -> + lists:foldl( + fun({Q, S0}, {Qs, Actions}) -> + {_, S} = deliver(Confirm, Delivery, S0), + {[{Q, S} | Qs], Actions} + end, {[], []}, QSs). + -spec info(amqqueue:amqqueue()) -> rabbit_types:infos(). info(Q) -> diff --git a/test/queue_type_SUITE.erl b/test/queue_type_SUITE.erl new file mode 100644 index 000000000000..a77a5f2d50c6 --- /dev/null +++ b/test/queue_type_SUITE.erl @@ -0,0 +1,216 @@ +-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 = 1, + 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()), + 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. + +merge_app_env(Config) -> + rabbit_ct_helpers:merge_app_env( + rabbit_ct_helpers:merge_app_env(Config, + {rabbit, [{core_metrics_gc_interval, 100}]}), + {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, <>} + ]), + 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 -> + 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 -> + 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}). diff --git a/test/quorum_queue_SUITE.erl b/test/quorum_queue_SUITE.erl index 7ef38895eb79..c8250e6ac56c 100644 --- a/test/quorum_queue_SUITE.erl +++ b/test/quorum_queue_SUITE.erl @@ -837,8 +837,8 @@ cleanup_queue_state_on_channel_after_publish(Config) -> [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 @@ -1525,6 +1525,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 @@ -1535,6 +1537,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 @@ -1544,8 +1548,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) -> From 7a7bf158e10820467a52b130d73fcbade4da30a6 Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Mon, 15 Jul 2019 12:33:59 +0100 Subject: [PATCH 05/18] Queue type: dequeue/basic_get --- src/rabbit_amqqueue.erl | 62 ++++++++++++++++++----------------- src/rabbit_channel.erl | 23 ++++++------- src/rabbit_classic_queue.erl | 17 +++++++++- src/rabbit_fifo_client.erl | 23 ++++++++----- src/rabbit_queue_type.erl | 38 ++++++++++++++++----- src/rabbit_quorum_queue.erl | 40 +++++++++++----------- test/dead_lettering_SUITE.erl | 6 ++-- 7 files changed, 128 insertions(+), 81 deletions(-) diff --git a/src/rabbit_amqqueue.erl b/src/rabbit_amqqueue.erl index 5c3ef7ad7bfe..a917414f7bad 100644 --- a/src/rabbit_amqqueue.erl +++ b/src/rabbit_amqqueue.erl @@ -35,7 +35,7 @@ -export([list_by_type/1]). -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/6, notify_decorators/1]). -export([notify_sent/2, notify_sent_queue_down/1, resume/2]). -export([notify_down_all/2, notify_down_all/3, activate_limit_all/2, credit/5]). -export([on_node_up/1, on_node_down/1]). @@ -1214,40 +1214,42 @@ credit(Q, CTag, Credit, Drain, QStates) -> % end, % QStates). --spec basic_get(amqqueue:amqqueue(), pid(), boolean(), pid(), rabbit_types:ctag(), +-spec basic_get(amqqueue:amqqueue(), boolean(), pid(), rabbit_types:ctag(), #{Name :: atom() => rabbit_fifo_client:state()}) -> {'ok', non_neg_integer(), qmsg(), quorum_states()} | {'empty', quorum_states()} | 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, QStates0) - when ?amqqueue_is_quorum(Q) -> - case rabbit_queue_type:with( - Q, - fun(S0) -> - case rabbit_quorum_queue:basic_get(Q, NoAck, CTag, S0) of - {ok, empty, S} -> - {empty, S}; - {ok, Count, Msg, S} -> - {{ok, Count, Msg}, S}; - {error, Reason} -> - QName = rabbit_quorum_queue:queue_name(S0), - rabbit_misc:protocol_error(internal_error, - "Cannot get a message from quorum queue '~s': ~p", - [rabbit_misc:rs(QName), Reason]) - end - end, - QStates0) of - {empty, QStates} -> - {empty, QStates}; - {{ok, Count, Msg}, QStates} -> - {ok, Count, Msg, QStates} - end. +% 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, QStates0) +% when ?amqqueue_is_quorum(Q) -> +% case rabbit_queue_type:with( +% Q, +% fun(S0) -> +% case rabbit_quorum_queue:basic_get(Q, NoAck, CTag, S0) of +% {ok, empty, S} -> +% {empty, S}; +% {ok, Count, Msg, S} -> +% {{ok, Count, Msg}, S}; +% {error, Reason} -> +% QName = rabbit_quorum_queue:queue_name(S0), +% rabbit_misc:protocol_error(internal_error, +% "Cannot get a message from quorum queue '~s': ~p", +% [rabbit_misc:rs(QName), Reason]) +% end +% end, +% QStates0) of +% {empty, QStates} -> +% {empty, QStates}; +% {{ok, Count, Msg}, QStates} -> +% {ok, Count, Msg, QStates} +% end. -type queue_ref() :: pid() | atom(). %% pid or registered name diff --git a/src/rabbit_channel.erl b/src/rabbit_channel.erl index 770ede144c18..7cc55092f85d 100644 --- a/src/rabbit_channel.erl +++ b/src/rabbit_channel.erl @@ -1351,12 +1351,11 @@ handle_method(#'basic.get'{queue = QueueNameBin, no_ack = NoAck}, case rabbit_amqqueue:with_exclusive_access_or_die( QueueName, ConnPid, %% Use the delivery tag as consumer tag for quorum queues - fun (Q) -> rabbit_amqqueue:basic_get( - Q, self(), NoAck, rabbit_limiter:pid(Limiter), - DeliveryTag, QueueStates0) + fun (Q) -> + rabbit_amqqueue:basic_get( + Q, NoAck, rabbit_limiter:pid(Limiter), + DeliveryTag, QueueStates0) end) of - {ok, MessageCount, Msg} -> - handle_basic_get(WriterPid, DeliveryTag, NoAck, MessageCount, Msg, State); {ok, MessageCount, Msg, QueueStates} -> handle_basic_get(WriterPid, DeliveryTag, NoAck, MessageCount, Msg, State#ch{queue_states = QueueStates}); @@ -1955,18 +1954,20 @@ reject(DeliveryTag, Requeue, Multiple, {Acked, Remaining} = collect_acks(UAMQ, DeliveryTag, Multiple), State1 = State#ch{unacked_message_q = Remaining}, {noreply, case Tx of - none -> internal_reject(Requeue, Acked, State1#ch.limiter, State1); - {Msgs, Acks} -> Acks1 = ack_cons(Requeue, Acked, Acks), - State1#ch{tx = {Msgs, Acks1}} + none -> + internal_reject(Requeue, Acked, State1#ch.limiter, State1); + {Msgs, Acks} -> + Acks1 = ack_cons(Requeue, Acked, Acks), + State1#ch{tx = {Msgs, Acks1}} end}. %% NB: Acked is in youngest-first order internal_reject(Requeue, Acked, Limiter, State = #ch{queue_states = QueueStates0}) -> QueueStates = foreach_per_queue( - fun({QPid, CTag}, MsgIds, Acc0) -> - rabbit_amqqueue:reject(QPid, Requeue, {CTag, MsgIds}, - Acc0) + fun({QRef, CTag}, MsgIds, Acc0) -> + rabbit_queue_type:reject(QRef, CTag, Requeue, + MsgIds, Acc0) end, Acked, QueueStates0), ok = notify_limiter(Limiter, Acked), State#ch{queue_states = QueueStates}. diff --git a/src/rabbit_classic_queue.erl b/src/rabbit_classic_queue.erl index 4f33c1143f9a..d68216c01bf1 100644 --- a/src/rabbit_classic_queue.erl +++ b/src/rabbit_classic_queue.erl @@ -23,7 +23,8 @@ deliver/2, settle/4, reject/4, - credit/4 + credit/4, + dequeue/4 ]). -export([delete_crashed/1, @@ -172,6 +173,20 @@ deliver(Qs, #delivery{flow = Flow, %% TODO: monitors {Qs, Actions}. + +-spec dequeue(NoAck :: boolean(), LimiterPid :: pid(), + rabbit_types:ctag(), state()) -> + {ok, Count :: non_neg_integer(), empty | rabbit_amqqueue:qmsg(), 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. + qpids(Qs) -> lists:foldl(fun ({Q, _}, {MPidAcc, SPidAcc, Actions0}) -> QPid = amqqueue:get_pid(Q), diff --git a/src/rabbit_fifo_client.erl b/src/rabbit_fifo_client.erl index f41fbe80a5df..62f585b0ada7 100644 --- a/src/rabbit_fifo_client.erl +++ b/src/rabbit_fifo_client.erl @@ -184,11 +184,12 @@ enqueue(Msg, State) -> %% @param State The {@module} state. %% %% @returns `{ok, IdMsg, State}' or `{error | timeout, term()}' --spec dequeue(rabbit_fifo:consumer_tag(), - Settlement :: settled | unsettled, state()) -> - {ok, {rabbit_fifo:delivery_msg(), non_neg_integer()} - | empty, state()} | {error | timeout, term()}. -dequeue(ConsumerTag, Settlement, #state{timeout = Timeout} = State0) -> +% -spec dequeue(rabbit_fifo:consumer_tag(), +% Settlement :: settled | unsettled, state()) -> +% {ok, {rabbit_fifo:delivery_msg(), non_neg_integer()} +% | empty, state()} | {error | timeout, term()}. +dequeue(ConsumerTag, Settlement, #state{timeout = Timeout, + cluster_name = QName} = State0) -> Node = pick_node(State0), ConsumerId = consumer_id(ConsumerTag), case ra:process_command(Node, @@ -197,9 +198,15 @@ dequeue(ConsumerTag, Settlement, #state{timeout = Timeout} = State0) -> #{}), Timeout) of {ok, {dequeue, empty}, Leader} -> - {ok, empty, State0#state{leader = Leader}}; - {ok, {dequeue, Msg, NumReady}, Leader} -> - {ok, {Msg, NumReady}, State0#state{leader = Leader}}; + {empty, State0#state{leader = Leader}}; + {ok, {dequeue, {MsgId, {MsgHeader, Msg0}}, MsgsReady}, Leader} -> + Count = maps:get(delivery_count, MsgHeader, 0), + IsDelivered = Count > 0, + Msg = rabbit_basic:add_header(<<"x-delivery-count">>, long, Count, + Msg0), + {ok, MsgsReady, {QName, Leader, MsgId, IsDelivered, Msg}, + State0#state{leader = Leader}}; + % {ok, {Msg, NumReady}, State0#state{leader = Leader}}; Err -> Err end. diff --git a/src/rabbit_queue_type.erl b/src/rabbit_queue_type.erl index 4062881060e0..4a7edb2561f3 100644 --- a/src/rabbit_queue_type.erl +++ b/src/rabbit_queue_type.erl @@ -16,6 +16,7 @@ settle/5, reject/5, credit/5, + dequeue/5, name/2 ]). @@ -112,6 +113,11 @@ 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(), empty | rabbit_amqqueue:qmsg(), + queue_state()}. + %% TODO: this should be controlled by a registry that is populated on boot discover(<<"quorum">>) -> rabbit_quorum_queue; @@ -190,13 +196,19 @@ cancel(Q, ChPid, Tag, OkMsg, ActiveUser, Ctxs) -> -spec handle_event(queue_ref(), term(), ctxs()) -> {ok, ctxs(), actions()} | {error, term()}. handle_event(QRef, Evt, Ctxs) -> - #ctx{module = Mod, - state = State0} = Ctx = get_ctx(QRef, Ctxs), - case Mod:handle_event(Evt, State0) of - {ok, State, Actions} -> - {ok, set_ctx(QRef, Ctx#ctx{state = State}, Ctxs), Actions}; - Err -> - Err + %% events can arrive after a queue state has been cleared up + %% so need to be defensive here + case get_ctx(QRef, Ctxs) of + #ctx{module = Mod, + state = State0} = Ctx -> + case Mod:handle_event(Evt, State0) of + {ok, State, Actions} -> + {ok, set_ctx(QRef, Ctx#ctx{state = State}, Ctxs), Actions}; + Err -> + Err + end; + undefined -> + {ok, Ctxs, []} end. @@ -241,6 +253,16 @@ credit(Q, CTag, Credit, Drain, Ctxs) -> State = Mod:settle(CTag, Credit, Drain, State0), set_ctx(Q, Ctx#ctx{state = State}, Ctxs). +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)} + end. + name(QRef, Ctxs) -> case Ctxs of #{QRef := Ctx} -> @@ -272,7 +294,7 @@ get_ctx(Q, Contexts) when ?is_amqqueue(Q) -> get_ctx(QPid, Contexts) when is_map(Contexts) -> Ref = qref(QPid), %% if we use a QPid it should always be initialised - maps:get(Ref, Contexts). + maps:get(Ref, Contexts, undefined). set_ctx(Q, Ctx, Contexts) when ?is_amqqueue(Q) andalso is_map(Contexts) -> diff --git a/src/rabbit_quorum_queue.erl b/src/rabbit_quorum_queue.erl index 3f1dc201b935..ce4136da60be 100644 --- a/src/rabbit_quorum_queue.erl +++ b/src/rabbit_quorum_queue.erl @@ -21,7 +21,7 @@ -export([init/1, handle_event/2]). -export([recover/1, stop/1, delete/4, delete_immediately/2]). -export([info/1, info/2, stat/1, infos/1]). --export([settle/4, reject/4, basic_get/4, consume/3, cancel/6]). +-export([settle/4, reject/4, dequeue/4, consume/3, cancel/6]). -export([credit/4]). -export([purge/1]). -export([stateless_deliver/2, deliver/3, deliver/2]). @@ -453,14 +453,12 @@ reject(CTag, false, 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()} | +-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, timeout | term()}. -basic_get(Q, NoAck, CTag0, QState0) when ?amqqueue_is_quorum(Q) -> - QName = amqqueue:get_name(Q), - Id = amqqueue:get_pid(Q), +dequeue(NoAck, _LimiterPid, CTag0, QState0) -> CTag = quorum_ctag(CTag0), Settlement = case NoAck of true -> @@ -468,19 +466,19 @@ 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 = maps:get(delivery_count, MsgHeader, 0), - IsDelivered = Count > 0, - Msg = rabbit_basic:add_header(<<"x-delivery-count">>, long, Count, Msg0), - {ok, MsgsReady, {QName, Id, MsgId, IsDelivered, Msg}, QState}; - {error, _} = Err -> - Err; - {timeout, _} -> - {error, timeout} - end. + rabbit_fifo_client:dequeue(CTag, Settlement, QState0). + % {ok, empty, QState} -> + % {ok, empty, QState}; + % {ok, {{MsgId, {MsgHeader, Msg0}}, MsgsReady}, QState} -> + % Count = maps:get(delivery_count, MsgHeader, 0), + % IsDelivered = Count > 0, + % Msg = rabbit_basic:add_header(<<"x-delivery-count">>, long, Count, Msg0), + % {ok, MsgsReady, {QName, Id, MsgId, IsDelivered, Msg}, QState}; + % {error, _} = Err -> + % Err; + % {timeout, _} -> + % {error, timeout} + % end. -spec consume(amqqueue:amqqueue(), rabbit_queue_type:consume_spec(), diff --git a/test/dead_lettering_SUITE.erl b/test/dead_lettering_SUITE.erl index fe5e91c8ed0c..d1895988113a 100644 --- a/test/dead_lettering_SUITE.erl +++ b/test/dead_lettering_SUITE.erl @@ -1068,9 +1068,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), From 7d347633e8ca1c8ec23aabc8372f6e1a0ce51718 Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Tue, 16 Jul 2019 14:53:16 +0100 Subject: [PATCH 06/18] Move info inside queue type abstraction --- src/rabbit_amqqueue.erl | 22 ++-------- src/rabbit_channel.erl | 81 ++++++++++++++++-------------------- src/rabbit_classic_queue.erl | 24 ++++++++++- src/rabbit_queue_type.erl | 67 ++++++++++++++++++++++++----- src/rabbit_quorum_queue.erl | 20 +++++---- src/unconfirmed_messages.erl | 2 + 6 files changed, 135 insertions(+), 81 deletions(-) diff --git a/src/rabbit_amqqueue.erl b/src/rabbit_amqqueue.erl index a917414f7bad..a8868948c261 100644 --- a/src/rabbit_amqqueue.erl +++ b/src/rabbit_amqqueue.erl @@ -913,28 +913,14 @@ format(_) -> []. -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). diff --git a/src/rabbit_channel.erl b/src/rabbit_channel.erl index 7cc55092f85d..0a2bf7626539 100644 --- a/src/rabbit_channel.erl +++ b/src/rabbit_channel.erl @@ -133,8 +133,6 @@ next_tag, %% messages pending consumer acknowledgement unacked_message_q, - %% a map of queue ref to queue name - queue_names, %% queue processes are monitored to update %% queue names queue_monitors, @@ -525,7 +523,6 @@ init([Channel, ReaderPid, WriterPid, ConnPid, ConnName, Protocol, User, VHost, tx = none, next_tag = 1, unacked_message_q = ?QUEUE:new(), - queue_names = #{}, queue_monitors = pmon:new(), consumer_mapping = #{}, queue_consumers = #{}, @@ -790,12 +787,15 @@ handle_info({'DOWN', _MRef, process, QPid, Reason}, State) -> %% any credit that was deferred will be sent to the rabbit_reader %% processs that might be blocked by this particular channel. credit_flow:peer_down(QPid), - #ch{queue_names = QNames, queue_monitors = QMons} = State4, - case maps:find(QPid, QNames) of - {ok, QName} -> erase_queue_stats(QName); - error -> ok + #ch{queue_states = QStates, queue_monitors = QMons} = State4, + case rabbit_queue_type:name(QPid, QStates) of + undefined -> + ok; + QName -> + erase_queue_stats(QName) end, - noreply(State4#ch{queue_names = maps:remove(QPid, QNames), + %% TODO: provide api function for remove from queue states? + noreply(State4#ch{queue_states = maps:remove(QPid, QStates), queue_monitors = pmon:erase(QPid, QMons)}); handle_info({'EXIT', _Pid, Reason}, State) -> @@ -816,11 +816,8 @@ handle_info(tick, State0 = #ch{queue_states = QueueStates0}) -> QueueStates1 = maps:filter(fun(QRef, _) -> QName = rabbit_queue_type:name(QRef, QueueStates0), - rabbit_log:info("tick QName ~w", [QName]), [] /= rabbit_amqqueue:lookup([QName]) end, QueueStates0), - rabbit_log:info("channel tick pre ~w post ~w", [maps:size(QueueStates0), - maps:size(QueueStates1)]), case evaluate_consumer_timeout(State0#ch{queue_states = QueueStates1}) of {noreply, State} -> noreply(init_tick_timer(reset_tick_timer(State))); @@ -1204,7 +1201,6 @@ record_rejects(MXs, State = #ch{rejected = R, tx = Tx}) -> record_confirms([], State) -> State; record_confirms(MXs, State = #ch{confirmed = C}) -> - rabbit_log:info("recording confirms ~w", [MXs]), State#ch{confirmed = [MXs | C]}. handle_method({Method, Content}, State) -> @@ -1753,13 +1749,12 @@ basic_consume(QueueName, NoAck, ConsumerPrefetch, ActualConsumerTag, end) of {{ok, QueueStates, Actions}, Q} when ?is_amqqueue(Q) -> QPid = amqqueue:get_pid(Q), - QName = amqqueue:get_name(Q), CM1 = maps:put( ActualConsumerTag, {Q, {NoAck, ConsumerPrefetch, ExclusiveConsume, Args}}, ConsumerMapping), State1 = track_delivering_queue( - NoAck, QPid, QName, + NoAck, QPid, State#ch{consumer_mapping = CM1, queue_states = QueueStates}), State2 = handle_queue_actions(Actions, State1), @@ -1791,12 +1786,12 @@ consumer_monitor(ConsumerTag, State#ch{queue_monitors = maybe_monitor(QRef, QMons), queue_consumers = QCons1}. -track_delivering_queue(NoAck, QPid, QName, - State = #ch{queue_names = QNames, +track_delivering_queue(NoAck, QPid, + State = #ch{ % queue_monitors = QMons, delivering_queues = DQ}) -> QRef = qpid_to_ref(QPid), - State#ch{queue_names = maps:put(QRef, QName, QNames), + State#ch{ % queue_monitors = maybe_monitor(QRef, QMons), delivering_queues = case NoAck of true -> DQ; @@ -1816,6 +1811,7 @@ handle_publishing_queue_down(QPid, Reason, {true, _} -> {RejectMXs, UC1} = unconfirmed_messages:reject_all_for_queue(QPid, UC), + record_rejects(RejectMXs, State#ch{unconfirmed = UC1}); {false, normal} -> {ConfirmMXs, RejectMXs, UC1} = @@ -1837,14 +1833,14 @@ handle_publishing_queue_down(QPid, _Reason, _State) when ?IS_QUORUM(QPid) -> handle_consuming_queue_down_or_eol(QRef, State = #ch{queue_consumers = QCons, - queue_names = QNames}) -> + queue_states = QStates}) -> ConsumerTags = case maps:find(QRef, QCons) of error -> gb_sets:new(); {ok, CTags} -> CTags end, gb_sets:fold( fun (CTag, StateN = #ch{consumer_mapping = CMap}) -> - QName = maps:get(QRef, QNames), + QName = rabbit_queue_type:name(QRef, QStates), case queue_down_consumer_action(CTag, CMap) of remove -> cancel_consumer(CTag, QName, StateN); @@ -1994,8 +1990,6 @@ record_sent(Type, Tag, AckRequired, end, DeliveredAt = os:system_time(millisecond), rabbit_trace:tap_out(Msg, ConnName, ChannelNum, Username, TraceState), - rabbit_log:info("channel recording sent ~w ~w ~w", - [Tag, DeliveryTag, AckRequired]), UAMQ1 = case AckRequired of true -> ?QUEUE:in({DeliveryTag, Tag, DeliveredAt, {QPid, MsgId}}, UAMQ); @@ -2033,23 +2027,24 @@ collect_acks(ToAcc, PrefixAcc, Q, DeliveryTag, Multiple) -> end. %% NB: Acked is in youngest-first order -ack(Acked, State = #ch{queue_names = QNames, - queue_states = QueueStates0}) -> +ack(Acked, State = #ch{queue_states = QueueStates0}) -> QueueStates = foreach_per_queue( fun ({QPid, CTag}, MsgIds, Acc0) -> Acc = rabbit_amqqueue:ack(QPid, {CTag, MsgIds}, self(), Acc0), - incr_queue_stats(QPid, QNames, MsgIds, State), + incr_queue_stats(QPid, MsgIds, State), Acc end, Acked, QueueStates0), ok = notify_limiter(State#ch.limiter, Acked), State#ch{queue_states = QueueStates}. -incr_queue_stats(QPid, QNames, MsgIds, State) -> - case maps:find(qpid_to_ref(QPid), QNames) of - {ok, QName} -> Count = length(MsgIds), - ?INCR_STATS(queue_stats, QName, Count, ack, State); - error -> ok +incr_queue_stats(QPid, MsgIds, #ch{queue_states = QStates} = State) -> + case rabbit_queue_type:name(qpid_to_ref(QPid), QStates) of + undefined -> + ok; + QName -> + Count = length(MsgIds), + ?INCR_STATS(queue_stats, QName, Count, ack, State) end. %% {Msgs, Acks} @@ -2130,8 +2125,7 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ mandatory = Mandatory, confirm = Confirm, msg_seq_no = MsgSeqNo}, - DelQNames}, State0 = #ch{queue_names = _QNames, - queue_monitors = _QMons, + DelQNames}, State0 = #ch{queue_monitors = _QMons, queue_states = QueueStates0}) -> Qs = rabbit_amqqueue:lookup(DelQNames), {QueueStates, Actions} = @@ -2160,9 +2154,8 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ end, [], Qs), ok = process_routing_mandatory(Mandatory, AllDeliveredQRefs, Message, State1), - QNames1 = State1#ch.queue_names, - AllDeliveredQNames = [QName || QRef <- AllDeliveredQRefs, - {ok, QName} <- [maps:find(QRef, QNames1)]], + AllDeliveredQNames = [rabbit_queue_type:name(QRef, QueueStates) + || QRef <- AllDeliveredQRefs], State = process_routing_confirm(Confirm, AllDeliveredQRefs, AllDeliveredQNames, @@ -2171,9 +2164,8 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ case rabbit_event:stats_level(State1, #ch.stats_timer) of fine -> ?INCR_STATS(exchange_stats, XName, 1, publish), - [?INCR_STATS(queue_exchange_stats, {QName, XName}, 1, publish) || - QRef <- AllDeliveredQRefs, - {ok, QName} <- [maps:find(QRef, QNames1)]]; + [?INCR_STATS(queue_exchange_stats, {QName, XName}, 1, publish) + || QName <- AllDeliveredQNames]; _ -> ok end, @@ -2197,7 +2189,6 @@ process_routing_confirm(false, _, _, _, _, State) -> process_routing_confirm(true, [], _, MsgSeqNo, XName, State) -> record_confirms([{MsgSeqNo, XName}], State); process_routing_confirm(true, QRefs, QNames, MsgSeqNo, XName, State) -> - rabbit_log:info("recording unconfirmed ~w ~w", [QRefs, MsgSeqNo]), State#ch{unconfirmed = unconfirmed_messages:insert(MsgSeqNo, QNames, QRefs, XName, State#ch.unconfirmed)}. @@ -2213,7 +2204,6 @@ confirm(MsgSeqNos, QRef, State = #ch{queue_states = QStates, unconfirmed = UC}) N -> N end, - rabbit_log:info("confirm queue name ~w ~w ~w", [QName, QRef, MsgSeqNos]), {ConfirmMXs, RejectMXs, UC1} = unconfirmed_messages:confirm_multiple_msg_ref(MsgSeqNos, QName, QRef, UC), %% NB: don't call noreply/1 since we don't want to send confirms. @@ -2221,10 +2211,8 @@ confirm(MsgSeqNos, QRef, State = #ch{queue_states = QStates, unconfirmed = UC}) record_rejects(RejectMXs, State1). send_confirms_and_nacks(State = #ch{tx = none, confirmed = [], rejected = []}) -> - rabbit_log:info("sending no confirms ~n", []), State; send_confirms_and_nacks(State = #ch{tx = none, confirmed = C, rejected = R}) -> - rabbit_log:info("sending confirms ~w", [C]), case rabbit_node_monitor:pause_partition_guard() of ok -> Confirms = lists:append(C), @@ -2366,7 +2354,12 @@ i(Item, _) -> pending_raft_commands(QStates) -> maps:fold(fun (_, V, Acc) -> - Acc + rabbit_fifo_client:pending_size(V) + case rabbit_queue_type:state_info(V) of + #{pending_raft_commands := P} -> + Acc + P; + _ -> + Acc + end end, 0, QStates). name(#ch{cfg = #conf{conn_name = ConnName, channel = Channel}}) -> @@ -2683,7 +2676,7 @@ handle_deliver0(ConsumerTag, AckRequired, record_sent(deliver, ConsumerTag, AckRequired, Msg, State). handle_basic_get(WriterPid, DeliveryTag, NoAck, MessageCount, - Msg = {QName, QPid, _MsgId, Redelivered, + Msg = {_QName, QPid, _MsgId, Redelivered, #basic_message{exchange_name = ExchangeName, routing_keys = [RoutingKey | _CcRoutes], content = Content}}, State) -> @@ -2695,7 +2688,7 @@ handle_basic_get(WriterPid, DeliveryTag, NoAck, MessageCount, routing_key = RoutingKey, message_count = MessageCount}, Content), - State1 = track_delivering_queue(NoAck, QPid, QName, State), + State1 = track_delivering_queue(NoAck, QPid, State), {noreply, record_sent(get, DeliveryTag, not(NoAck), Msg, State1)}. init_tick_timer(State = #ch{tick_timer = undefined}) -> diff --git a/src/rabbit_classic_queue.erl b/src/rabbit_classic_queue.erl index d68216c01bf1..1104796787c5 100644 --- a/src/rabbit_classic_queue.erl +++ b/src/rabbit_classic_queue.erl @@ -24,7 +24,9 @@ settle/4, reject/4, credit/4, - dequeue/4 + dequeue/4, + info/2, + state_info/1 ]). -export([delete_crashed/1, @@ -187,6 +189,26 @@ dequeue(NoAck, LimiterPid, _CTag, State) -> {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. + qpids(Qs) -> lists:foldl(fun ({Q, _}, {MPidAcc, SPidAcc, Actions0}) -> QPid = amqqueue:get_pid(Q), diff --git a/src/rabbit_queue_type.erl b/src/rabbit_queue_type.erl index 4a7edb2561f3..35ead2721ab5 100644 --- a/src/rabbit_queue_type.erl +++ b/src/rabbit_queue_type.erl @@ -8,6 +8,11 @@ declare/2, delete/4, stat/1, + name/2, + info/2, + state_info/1, + info_down/3, + %% stateful client API new/2, consume/3, cancel/6, @@ -16,9 +21,8 @@ settle/5, reject/5, credit/5, - dequeue/5, + dequeue/5 - name/2 ]). %% temporary @@ -118,6 +122,14 @@ {ok, Count :: non_neg_integer(), empty | rabbit_amqqueue:qmsg(), queue_state()}. +%% 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(). + %% TODO: this should be controlled by a registry that is populated on boot discover(<<"quorum">>) -> rabbit_quorum_queue; @@ -154,6 +166,49 @@ stat(Q) -> Mod = amqqueue:get_type(Q), Mod:stat(Q). +-spec name(queue_ref(), ctxs()) -> + undefined | queue_name(). +name(QRef, Ctxs) -> + case Ctxs of + #{QRef := Ctx} -> + Ctx#ctx.name; + _ -> + undefined + 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). + +state_info(#ctx{state = S, + module = Mod}) -> + Mod:state_info(S). + +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. + -spec new(amqqueue:amqqueue(), ctxs()) -> ctxs(). new(Q, Ctxs) when ?is_amqqueue(Q) -> Mod = amqqueue:get_type(Q), @@ -263,14 +318,6 @@ dequeue(Q, NoAck, LimiterPid, CTag, Ctxs) -> {empty, set_ctx(Q, Ctx#ctx{state = State}, Ctxs)} end. -name(QRef, Ctxs) -> - case Ctxs of - #{QRef := Ctx} -> - Ctx#ctx.name; - _ -> - undefined - end. - %% temporary with(QRef, Fun, Ctxs) -> #ctx{state = State0} = Ctx = get_ctx(QRef, Ctxs), diff --git a/src/rabbit_quorum_queue.erl b/src/rabbit_quorum_queue.erl index ce4136da60be..44e6462359ed 100644 --- a/src/rabbit_quorum_queue.erl +++ b/src/rabbit_quorum_queue.erl @@ -20,7 +20,7 @@ -export([init/1, handle_event/2]). -export([recover/1, stop/1, delete/4, delete_immediately/2]). --export([info/1, info/2, stat/1, infos/1]). +-export([state_info/1, info/2, stat/1, infos/1]). -export([settle/4, reject/4, dequeue/4, consume/3, cancel/6]). -export([credit/4]). -export([purge/1]). @@ -570,14 +570,21 @@ deliver(QSs, #delivery{confirm = Confirm} = Delivery) -> {[{Q, S} | Qs], Actions} end, {[], []}, QSs). --spec info(amqqueue:amqqueue()) -> rabbit_types:infos(). -info(Q) -> +-spec info(amqqueue:amqqueue(), rabbit_types:info_keys()) -> + rabbit_types:infos(). +info(Q, all_keys) -> info(Q, [name, durable, auto_delete, arguments, pid, state, messages, - messages_ready, messages_unacknowledged]). + messages_ready, messages_unacknowledged]); +info(Q, Items) -> + [{Item, i(Item, Q)} || Item <- Items]. + +state_info(S) -> + #{pending_raft_commands => rabbit_fifo_client:pending_size(S)}. --spec infos(rabbit_types:r('queue')) -> rabbit_types:infos(). + +-spec infos(rabbit_types:r('queue')) -> rabbit_types:infos(). infos(QName) -> case rabbit_amqqueue:lookup(QName) of {ok, Q} -> @@ -586,10 +593,7 @@ infos(QName) -> [] end. --spec info(amqqueue:amqqueue(), rabbit_types:info_keys()) -> rabbit_types:infos(). -info(Q, Items) -> - [{Item, i(Item, Q)} || Item <- Items]. -spec stat(amqqueue:amqqueue()) -> {'ok', non_neg_integer(), non_neg_integer()}. diff --git a/src/unconfirmed_messages.erl b/src/unconfirmed_messages.erl index 0a4b533448fd..f9f3b2d36cff 100644 --- a/src/unconfirmed_messages.erl +++ b/src/unconfirmed_messages.erl @@ -244,6 +244,8 @@ remove_msg_ref(Confirm, MsgId, QueueName, QueueRef, #msg_status{refs = #{QueueRef := ?SET_VALUE} = Refs, queue_status = QStatus, exchange = XName} = MsgStatus -> + rabbit_log:info("~w QStatus ~w Confirm ~w", [?FUNCTION_NAME, + QStatus, Confirm]), QStatus1 = case {Confirm, QueueName} of {no_confirm, _} -> QStatus; {_, ignore} -> QStatus; From f0aaa63336426eca4257e5bfdc3c71071fa254fb Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Thu, 18 Jul 2019 09:23:49 +0100 Subject: [PATCH 07/18] Move policy change into queue type interface --- src/rabbit_amqqueue.erl | 287 ++---------------- src/rabbit_amqqueue_process.erl | 2 +- src/rabbit_basic.erl | 2 +- src/rabbit_channel.erl | 14 +- src/rabbit_classic_queue.erl | 29 +- src/rabbit_dead_letter.erl | 4 +- src/rabbit_fifo_client.erl | 58 ++-- src/rabbit_queue_type.erl | 67 +++- src/rabbit_quorum_queue.erl | 47 ++- .../amqqueue_backward_compatibility_SUITE.erl | 6 +- test/backing_queue_SUITE.erl | 2 +- test/rabbit_fifo_int_SUITE.erl | 180 +++++------ test/unit_inbroker_parallel_SUITE.erl | 2 +- 13 files changed, 256 insertions(+), 444 deletions(-) diff --git a/src/rabbit_amqqueue.erl b/src/rabbit_amqqueue.erl index a8868948c261..25f40c5c2553 100644 --- a/src/rabbit_amqqueue.erl +++ b/src/rabbit_amqqueue.erl @@ -24,7 +24,9 @@ -export([lookup/1, not_found_or_absent/1, with/2, with/3, with_or_die/2, assert_equivalence/5, check_exclusive_access/2, with_exclusive_access_or_die/3, - stat/1, deliver/2, deliver/3, requeue/4, ack/4, reject/4]). + stat/1, + % deliver/2, deliver/3, + 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, @@ -35,7 +37,7 @@ -export([list_by_type/1]). -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/5, 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/5]). -export([on_node_up/1, on_node_down/1]). @@ -76,7 +78,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()}]}. @@ -84,7 +87,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()}. %%---------------------------------------------------------------------------- @@ -407,7 +409,7 @@ 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). -spec lookup (name()) -> @@ -1000,14 +1002,9 @@ force_event_refresh(Ref) -> ok. -spec notify_policy_changed(amqqueue:amqqueue()) -> 'ok'. +notify_policy_changed(Q) when ?is_amqqueue(Q) -> -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). + rabbit_queue_type:policy_changed(Q). -spec consumers(amqqueue:amqqueue()) -> [{pid(), rabbit_types:ctag(), boolean(), non_neg_integer(), @@ -1121,36 +1118,33 @@ purge(Q) when ?amqqueue_is_quorum(Q) -> -spec requeue(pid() | atom(), {rabbit_fifo:consumer_tag(), [msg_id()]}, - pid(), - quorum_states()) -> ok. -requeue(QRef, {CTag, MsgIds}, _ChPid, QStates) -> + rabbit_queue_type:ctxs()) -> + rabbit_queue_type:ctxs(). +requeue(QRef, {CTag, MsgIds}, QStates) -> reject(QRef, true, {CTag, MsgIds}, QStates). -spec ack(pid(), {rabbit_fifo:consumer_tag(), [msg_id()]}, - pid(), - quorum_states()) -> - quorum_states(). -ack(QPid, {CTag, MsgIds}, ChPid, QueueStates) -> - rabbit_queue_type:settle(QPid, CTag, MsgIds, ChPid, QueueStates). + rabbit_queue_type:ctxs()) -> + rabbit_queue_type:ctxs(). +ack(QPid, {CTag, MsgIds}, QueueStates) -> + rabbit_queue_type:settle(QPid, CTag, MsgIds, QueueStates). -spec reject(pid() | atom(), boolean(), {rabbit_fifo:consumer_tag(), [msg_id()]}, - quorum_states()) -> - quorum_states(). + rabbit_queue_type:ctxs()) -> + rabbit_queue_type:ctxs(). reject(QRef, Requeue, {CTag, MsgIds}, QStates) -> rabbit_queue_type:reject(QRef, CTag, Requeue, 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 @@ -1179,70 +1173,24 @@ activate_limit_all(QRefs, ChPid) -> rabbit_types:ctag(), non_neg_integer(), boolean(), - quorum_states()) -> - quorum_states(). + rabbit_queue_type:ctxs()) -> + rabbit_queue_type:ctxs(). 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) -> -% {QRef, _} = amqqueue:get_pid(Q), -% rabbit_queue_type:with( -% QRef, -% fun(S) -> -% rabbit_quorum_queue:credit(CTag, Credit, Drain, S) -% end, -% QStates). - -spec basic_get(amqqueue:amqqueue(), boolean(), pid(), rabbit_types:ctag(), - #{Name :: atom() => rabbit_fifo_client:state()}) -> - {'ok', non_neg_integer(), qmsg(), quorum_states()} | - {'empty', quorum_states()} | + rabbit_queue_type:ctxs()) -> + {'ok', non_neg_integer(), qmsg(), rabbit_queue_type:ctxs()} | + {'empty', rabbit_queue_type:ctxs()} | 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, QStates0) -% when ?amqqueue_is_quorum(Q) -> -% case rabbit_queue_type:with( -% Q, -% fun(S0) -> -% case rabbit_quorum_queue:basic_get(Q, NoAck, CTag, S0) of -% {ok, empty, S} -> -% {empty, S}; -% {ok, Count, Msg, S} -> -% {{ok, Count, Msg}, S}; -% {error, Reason} -> -% QName = rabbit_quorum_queue:queue_name(S0), -% rabbit_misc:protocol_error(internal_error, -% "Cannot get a message from quorum queue '~s': ~p", -% [rabbit_misc:rs(QName), Reason]) -% end -% end, -% QStates0) of -% {empty, QStates} -> -% {empty, QStates}; -% {{ok, Count, Msg}, QStates} -> -% {ok, Count, Msg, QStates} -% end. - --type queue_ref() :: pid() | atom(). %% pid or registered name -spec basic_consume(amqqueue:amqqueue(), boolean(), pid(), pid(), boolean(), non_neg_integer(), rabbit_types:ctag(), boolean(), rabbit_framing:amqp_table(), any(), rabbit_types:username(), - #{Ref :: queue_ref() => rabbit_queue_type:ctx()}) -> + rabbit_queue_type:ctxs()) -> {ok, rabbit_queue_type:ctxs(), rabbit_queue_type:actions()} | {error, term()}. basic_consume(Q, NoAck, ChPid, LimiterPid, @@ -1264,88 +1212,13 @@ basic_consume(Q, NoAck, ChPid, LimiterPid, acting_user => ActingUser}, rabbit_queue_type:consume(Q, Spec, Contexts). -% get_ctx(Q, Contexts) -% when ?is_amqqueue(Q) andalso is_map(Contexts) -> -% QPid = amqqueue:get_pid(Q), -% Ref = qpid_to_ref(QPid), -% case Contexts of -% #{Ref := Ctx} -> -% Ctx; -% _ -> -% %% not found - initialize -% rabbit_queue_type:init(Q) -% end; -% get_ctx(QPid, Contexts) when is_map(Contexts) -> -% Ref = qpid_to_ref(QPid), -% %% if we use a QPid it should always be initialised -% maps:get(Ref, Contexts). - - -% basic_consume(Q, NoAck, ChPid, LimiterPid, -% LimiterActive, ConsumerPrefetchCount, ConsumerTag, -% ExclusiveConsume, Args, OkMsg, ActingUser, QState) -% when ?amqqueue_is_classic(Q) -> -% QPid = amqqueue:get_pid(Q), -% QName = amqqueue:get_name(Q), -% 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), -% {ok, QState} = rabbit_quorum_queue:basic_consume(Q, NoAck, ChPid, -% ConsumerPrefetchCount, -% ConsumerTag, -% ExclusiveConsume, Args, -% ActingUser, -% OkMsg, QState0), -% {ok, maps:put(Name, QState, QStates)}. - --spec basic_cancel(amqqueue:amqqueue(), pid(), rabbit_types:ctag(), any(), +-spec basic_cancel(amqqueue:amqqueue(), rabbit_types:ctag(), any(), rabbit_types:username(), - #{Name :: atom() => rabbit_fifo_client:state()}) -> - {ok, #{Name :: atom() => rabbit_fifo_client:state()}}. -basic_cancel(Q, ChPid, ConsumerTag, OkMsg, ActingUser, QStates) -> - rabbit_queue_type:cancel(Q, ChPid, ConsumerTag, + rabbit_queue_type:ctxs()) -> + {ok, rabbit_queue_type:ctxs()} | {error, term()}. +basic_cancel(Q, ConsumerTag, OkMsg, ActingUser, QStates) -> + rabbit_queue_type:cancel(Q, ConsumerTag, OkMsg, ActingUser, QStates). -% 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 notify_decorators(amqqueue:amqqueue()) -> 'ok'. @@ -1659,105 +1532,3 @@ pseudo_queue(#resource{kind = queue} = QueueName, Pid, Durable) -spec immutable(amqqueue:amqqueue()) -> amqqueue:amqqueue(). 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 slave since the message - %% arrives at the slave from two directions. We will ack one when - %% the slave receives the message direct from the channel, and the - %% other when it receives it via GM. - - case Flow of - %% Here we are tracking messages sent by the rabbit_channel - %% process. We are accessing the rabbit_channel process - %% dictionary. - flow -> [credit_flow:send(QPid) || QPid <- QPids], - [credit_flow:send(QPid) || QPid <- SPids]; - noflow -> ok - end, - - %% We let slaves know that they were being addressed as slaves 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({{QRef, _}, _QName}, S0) -> - rabbit_queue_type:with( - QRef, - fun (S) -> - rabbit_quorum_queue:deliver(Confirm, - Delivery, - S) - end, S0) - 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). diff --git a/src/rabbit_amqqueue_process.erl b/src/rabbit_amqqueue_process.erl index 46db98ba5a42..4b005da9948d 100644 --- a/src/rabbit_amqqueue_process.erl +++ b/src/rabbit_amqqueue_process.erl @@ -1545,7 +1545,7 @@ handle_cast({deliver, noreply(maybe_deliver_or_enqueue(Delivery, SlaveWhenPublished, State1)); %% [0] The second ack is since the channel thought we were a slave 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)); diff --git a/src/rabbit_basic.erl b/src/rabbit_basic.erl index 42b76218dfd1..82d225c52f2b 100644 --- a/src/rabbit_basic.erl +++ b/src/rabbit_basic.erl @@ -76,7 +76,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()) -> diff --git a/src/rabbit_channel.erl b/src/rabbit_channel.erl index 0a2bf7626539..52b629e7b935 100644 --- a/src/rabbit_channel.erl +++ b/src/rabbit_channel.erl @@ -754,7 +754,7 @@ handle_info({ra_event, From, _} = Evt, State2#ch{unconfirmed = UC1}), erase_queue_stats(rabbit_queue_type:name(QRef, QueueStates0)), noreply_coalesce( - State3#ch{queue_states = maps:remove(QRef, QueueStates0)}) + State3#ch{queue_states = rabbit_queue_type:remove(QRef, QueueStates0)}) end; handle_info({bump_credit, Msg}, State) -> @@ -795,7 +795,7 @@ handle_info({'DOWN', _MRef, process, QPid, Reason}, State) -> erase_queue_stats(QName) end, %% TODO: provide api function for remove from queue states? - noreply(State4#ch{queue_states = maps:remove(QPid, QStates), + noreply(State4#ch{queue_states = rabbit_queue_type:remove(QPid, QStates), queue_monitors = pmon:erase(QPid, QMons)}); handle_info({'EXIT', _Pid, Reason}, State) -> @@ -1490,7 +1490,7 @@ handle_method(#'basic.cancel'{consumer_tag = ConsumerTag, nowait = NoWait}, fun () -> {error, not_found} end, fun () -> rabbit_amqqueue:basic_cancel( - Q, self(), ConsumerTag, ok_msg(NoWait, OkMsg), + Q, ConsumerTag, ok_msg(NoWait, OkMsg), Username, QueueStates0) end) of {ok, QueueStates} -> @@ -1546,8 +1546,7 @@ handle_method(#'basic.recover_async'{requeue = true}, rabbit_misc:with_exit_handler( OkFun, fun () -> - rabbit_amqqueue:requeue(QPid, {CTag, MsgIds}, - self(), Acc0) + rabbit_amqqueue:requeue(QPid, {CTag, MsgIds}, Acc0) end) end, lists:reverse(UAMQL), QueueStates0), ok = notify_limiter(Limiter, UAMQL), @@ -1710,8 +1709,7 @@ handle_method(#'basic.credit'{consumer_tag = CTag, queue_states = QStates0}) -> case maps:find(CTag, Consumers) of {ok, {Q, _CParams}} -> - {ok, QStates} = rabbit_amqqueue:credit( - Q, self(), CTag, Credit, Drain, QStates0), + QStates = rabbit_amqqueue:credit(Q, CTag, Credit, Drain, QStates0), {noreply, State#ch{queue_states = QStates}}; error -> precondition_failed( "unknown consumer tag '~s'", [CTag]) @@ -2031,7 +2029,7 @@ ack(Acked, State = #ch{queue_states = QueueStates0}) -> QueueStates = foreach_per_queue( fun ({QPid, CTag}, MsgIds, Acc0) -> - Acc = rabbit_amqqueue:ack(QPid, {CTag, MsgIds}, self(), Acc0), + Acc = rabbit_amqqueue:ack(QPid, {CTag, MsgIds}, Acc0), incr_queue_stats(QPid, MsgIds, State), Acc end, Acked, QueueStates0), diff --git a/src/rabbit_classic_queue.erl b/src/rabbit_classic_queue.erl index 1104796787c5..93830ec9f75b 100644 --- a/src/rabbit_classic_queue.erl +++ b/src/rabbit_classic_queue.erl @@ -15,13 +15,14 @@ is_enabled/0, declare/2, delete/4, + policy_changed/1, stat/1, init/1, consume/3, - cancel/6, + cancel/5, handle_event/2, deliver/2, - settle/4, + settle/3, reject/4, credit/4, dequeue/4, @@ -84,6 +85,11 @@ delete(Q, IfUnused, IfEmpty, ActingUser) when ?amqqueue_is_classic(Q) -> {ok, 0} 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]}). @@ -117,22 +123,21 @@ consume(Q, Spec, State) when ?amqqueue_is_classic(Q) -> Err end. -cancel(Q, ChPid, ConsumerTag, OkMsg, ActingUser, State) -> +cancel(Q, ConsumerTag, OkMsg, ActingUser, State) -> QPid = amqqueue:get_pid(Q), case delegate:invoke(QPid, {gen_server2, call, - [{basic_cancel, ChPid, ConsumerTag, + [{basic_cancel, self(), ConsumerTag, OkMsg, ActingUser}, infinity]}) of ok -> {ok, State}; Err -> Err end. --spec settle(rabbit_types:ctag(), [non_neg_integer()], - ChPid :: pid(), state()) -> +-spec settle(rabbit_types:ctag(), [non_neg_integer()], state()) -> state(). -settle(_CTag, MsgIds, ChPid, State) -> +settle(_CTag, MsgIds, State) -> delegate:invoke_no_result(State#?STATE.pid, - {gen_server2, cast, [{ack, MsgIds, ChPid}]}), + {gen_server2, cast, [{ack, MsgIds, self()}]}), State. reject(_CTag, Requeue, MsgIds, State) -> @@ -153,12 +158,13 @@ handle_event(_Evt, State) -> {ok, State, []}. -spec deliver([{amqqueue:amqqueue(), state()}], - Delivery :: term()) -> + Delivery :: term()) -> {[{amqqueue:amqqueue(), state()}], rabbit_queue_type:actions()}. deliver(Qs, #delivery{flow = Flow, confirm = _Confirm} = Delivery) -> {MPids, SPids, Actions} = qpids(Qs), QPids = MPids ++ SPids, + rabbit_log:info("classic deliver to ~w", [QPids]), case Flow of %% Here we are tracking messages sent by the rabbit_channel %% process. We are accessing the rabbit_channel process @@ -169,16 +175,15 @@ deliver(Qs, #delivery{flow = Flow, end, MMsg = {deliver, Delivery, false}, SMsg = {deliver, Delivery, true}, - rabbit_log:info("rabbit_classic_queue delivery confirm ~w", [_Confirm, MMsg]), delegate:invoke_no_result(MPids, {gen_server2, cast, [MMsg]}), delegate:invoke_no_result(SPids, {gen_server2, cast, [SMsg]}), - %% TODO: monitors {Qs, Actions}. -spec dequeue(NoAck :: boolean(), LimiterPid :: pid(), rabbit_types:ctag(), state()) -> - {ok, Count :: non_neg_integer(), empty | rabbit_amqqueue:qmsg(), 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, diff --git a/src/rabbit_dead_letter.erl b/src/rabbit_dead_letter.erl index 3a400061a4e0..5b3bae1f0da5 100644 --- a/src/rabbit_dead_letter.erl +++ b/src/rabbit_dead_letter.erl @@ -29,14 +29,14 @@ -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, diff --git a/src/rabbit_fifo_client.erl b/src/rabbit_fifo_client.erl index 62f585b0ada7..98c3e2734118 100644 --- a/src/rabbit_fifo_client.erl +++ b/src/rabbit_fifo_client.erl @@ -184,10 +184,10 @@ enqueue(Msg, State) -> %% @param State The {@module} state. %% %% @returns `{ok, IdMsg, State}' or `{error | timeout, term()}' -% -spec dequeue(rabbit_fifo:consumer_tag(), -% Settlement :: settled | unsettled, state()) -> -% {ok, {rabbit_fifo:delivery_msg(), non_neg_integer()} -% | empty, state()} | {error | timeout, term()}. +-spec dequeue(rabbit_fifo:consumer_tag(), + Settlement :: settled | unsettled, state()) -> + {ok, non_neg_integer(), term(), non_neg_integer()} + | {empty, state()} | {error | timeout, term()}. dequeue(ConsumerTag, Settlement, #state{timeout = Timeout, cluster_name = QName} = State0) -> Node = pick_node(State0), @@ -202,15 +202,21 @@ dequeue(ConsumerTag, Settlement, #state{timeout = Timeout, {ok, {dequeue, {MsgId, {MsgHeader, Msg0}}, MsgsReady}, Leader} -> Count = maps:get(delivery_count, MsgHeader, 0), IsDelivered = Count > 0, - Msg = rabbit_basic:add_header(<<"x-delivery-count">>, long, Count, - Msg0), - {ok, MsgsReady, {QName, Leader, MsgId, IsDelivered, Msg}, + Msg = add_delivery_count_header(Msg0, Count), + {ok, MsgsReady, + {QName, Leader, MsgId, IsDelivered, Msg}, State0#state{leader = Leader}}; - % {ok, {Msg, NumReady}, State0#state{leader = Leader}}; Err -> 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.} @@ -221,16 +227,14 @@ dequeue(ConsumerTag, Settlement, #state{timeout = Timeout, %% the sending rate. %% -spec settle(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) -> - {ok, state()}. + state(). settle(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) -> Node = pick_node(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) -> @@ -322,7 +326,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 @@ -492,15 +497,21 @@ update_machine_state(Node, Conf) -> %% used to {@link settle/3.} (roughly: AMQP 0.9.1 ack) message once finished %% with them. -spec handle_ra_event(ra_server_id(), ra_server_proc:ra_event_body(), state()) -> - {internal, actions(), state()} | - {deliver, rabbit_types:ctag(), boolean(), [rabbit_amqqueue:qmsg()]} | eol. + {ok, state(), actions()}. + % {deliver, rabbit_types:ctag(), boolean(), [rabbit_amqqueue:qmsg()]} | eol. handle_ra_event(From, {applied, Seqs}, #state{soft_limit = SftLmt, unblock_handler = UnblockFun} = State0) -> {Corrs, Actions0, State1} = lists:foldl(fun seq_applied/2, {[], [], State0#state{leader = From}}, Seqs), - Actions = [{settled, qref(From), Corrs} | lists:reverse(Actions0)], + Actions = case Corrs of + [] -> + lists:reverse(Actions0); + _ -> + [{settled, qref(From), 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 @@ -664,7 +675,7 @@ maybe_auto_ack(true, Deliver, State0) -> maybe_auto_ack(false, {deliver, Tag, _Ack, Msgs} = Deliver, State0) -> %% we have to auto ack these deliveries MsgIds = [I || {_, _, I, _, _} <- Msgs], - {ok, State} = settle(Tag, MsgIds, State0), + State = settle(Tag, MsgIds, State0), {ok, State, [Deliver]}. @@ -722,16 +733,11 @@ handle_delivery(Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs}, transform_msgs(QName, QRef, Msgs) -> lists:map(fun({MsgId, {MsgHeader, Msg0}}) -> - IsDelivered = maps:is_key(delivery_count, MsgHeader), - Msg = add_delivery_count_header(MsgHeader, Msg0), - {QName, QRef, MsgId, IsDelivered, Msg} + Count = maps:get(delivery_count, MsgHeader, not_found), + Msg = add_delivery_count_header(Msg0, Count), + {QName, QRef, MsgId, is_integer(Count), Msg} end, Msgs). -add_delivery_count_header(#{delivery_count := Count}, Msg) -> - rabbit_basic:add_header(<<"x-delivery-count">>, long, Count, Msg); -add_delivery_count_header(_, Msg) -> - Msg. - update_consumer(Tag, LastId, DelCntIncr, #consumer{delivery_count = D} = C, Consumers) -> maps:put(Tag, diff --git a/src/rabbit_queue_type.erl b/src/rabbit_queue_type.erl index 35ead2721ab5..d3edd02501d8 100644 --- a/src/rabbit_queue_type.erl +++ b/src/rabbit_queue_type.erl @@ -7,18 +7,20 @@ is_enabled/1, declare/2, delete/4, + policy_changed/1, stat/1, name/2, + remove/2, info/2, state_info/1, info_down/3, %% stateful client API new/2, consume/3, - cancel/6, + cancel/5, handle_event/3, deliver/3, - settle/5, + settle/4, reject/5, credit/5, dequeue/5 @@ -85,12 +87,14 @@ rabbit_types:ok(non_neg_integer()) | rabbit_types:error(in_use | not_empty). +-callback policy_changed(amqqueue:amqqueue()) -> ok. + -callback consume(amqqueue:amqqueue(), consume_spec(), queue_state()) -> {ok, queue_state(), actions()} | {error, term()}. --callback cancel(amqqueue:amqqueue(), pid(), +-callback cancel(amqqueue:amqqueue(), rabbit_types:ctag(), term(), rabbit_types:username(), @@ -105,8 +109,7 @@ Delivery :: term()) -> {[{amqqueue:amqqueue(), queue_state()}], actions()}. --callback settle(rabbit_types:ctag(), [non_neg_integer()], - ChPid :: pid(), queue_state()) -> +-callback settle(rabbit_types:ctag(), [non_neg_integer()], queue_state()) -> queue_state(). -callback reject(rabbit_types:ctag(), Requeue :: boolean(), @@ -119,8 +122,12 @@ -callback dequeue(NoAck :: boolean(), LimiterPid :: pid(), rabbit_types:ctag(), queue_state()) -> - {ok, Count :: non_neg_integer(), empty | rabbit_amqqueue:qmsg(), - 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()) -> @@ -160,6 +167,11 @@ delete(Q, IfUnused, IfEmpty, ActingUser) -> Mod:delete(Q, IfUnused, IfEmpty, ActingUser). +-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) -> @@ -176,6 +188,10 @@ name(QRef, Ctxs) -> undefined end. +-spec remove(queue_ref(), ctxs()) -> ctxs(). +remove(QRef, Ctxs) -> + maps:remove(QRef, Ctxs). + -spec info(amqqueue:amqqueue(), all_keys | rabbit_types:info_keys()) -> rabbit_types:infos(). info(Q, Items) when ?amqqueue_state_is(Q, crashed) -> @@ -231,16 +247,16 @@ consume(Q, Spec, Ctxs) -> end. %% TODO switch to cancel spec api --spec cancel(amqqueue:amqqueue(), pid(), +-spec cancel(amqqueue:amqqueue(), rabbit_types:ctag(), term(), rabbit_types:username(), ctxs()) -> - {ok, ctxs(), actions()} | {error, term()}. -cancel(Q, ChPid, Tag, OkMsg, ActiveUser, Ctxs) -> + {ok, ctxs()} | {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, ChPid, Tag, OkMsg, ActiveUser, State0) of + case Mod:cancel(Q, Tag, OkMsg, ActiveUser, State0) of {ok, State} -> {ok, set_ctx(Q, Ctx#ctx{state = State}, Ctxs)}; Err -> @@ -249,7 +265,7 @@ cancel(Q, ChPid, Tag, OkMsg, ActiveUser, Ctxs) -> %% messages sent from queues -spec handle_event(queue_ref(), term(), ctxs()) -> - {ok, ctxs(), actions()} | {error, term()}. + {ok, ctxs(), 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 @@ -267,6 +283,15 @@ handle_event(QRef, Evt, Ctxs) -> end. +-spec deliver([amqqueue:amqqueue()], Delivery :: term(), + stateless | ctxs()) -> + {ctxs(), actions()}. +deliver(Qs, Delivery, stateless) -> + _ = lists:map(fun(Q) -> + Mod = amqqueue:get_type(Q), + _ = Mod:deliver([{Q, stateless}], Delivery) + end, Qs), + {stateless, []}; deliver(Qs, Delivery, Ctxs) -> %% sort by queue type - then dispatch each group ByType = lists:foldl(fun (Q, Acc) -> @@ -290,24 +315,36 @@ deliver(Qs, Delivery, Ctxs) -> end, Ctxs, Xs), Actions}. -settle(QRef, CTag, MsgIds, ChPid, Ctxs) -> +-spec settle(queue_ref(), rabbit_types:ctag(), + [non_neg_integer()], ctxs()) -> ctxs(). +settle(QRef, CTag, MsgIds, Ctxs) -> #ctx{state = State0, module = Mod} = Ctx = get_ctx(QRef, Ctxs), - State = Mod:settle(CTag, MsgIds, ChPid, State0), + State = Mod:settle(CTag, MsgIds, State0), set_ctx(QRef, Ctx#ctx{state = State}, Ctxs). +-spec reject(queue_ref(), rabbit_types:ctag(), + boolean(), [non_neg_integer()], ctxs()) -> ctxs(). reject(QRef, CTag, Requeue, MsgIds, Ctxs) -> #ctx{state = State0, module = Mod} = Ctx = get_ctx(QRef, Ctxs), State = Mod:reject(CTag, Requeue, MsgIds, State0), set_ctx(QRef, Ctx#ctx{state = State}, Ctxs). +-spec credit(amqqueue:amqqueue() | queue_ref(), + rabbit_types:ctag(), non_neg_integer(), + boolean(), ctxs()) -> ctxs(). credit(Q, CTag, Credit, Drain, Ctxs) -> #ctx{state = State0, module = Mod} = Ctx = get_ctx(Q, Ctxs), - State = Mod:settle(CTag, Credit, Drain, State0), + State = Mod:credit(CTag, Credit, Drain, State0), set_ctx(Q, Ctx#ctx{state = State}, Ctxs). +-spec dequeue(amqqueue:amqqueue(), boolean(), + pid(), rabbit_types:ctag(), + ctxs()) -> + {ok, non_neg_integer(), term(), ctxs()} | + {empty, ctxs()}. dequeue(Q, NoAck, LimiterPid, CTag, Ctxs) -> #ctx{state = State0} = Ctx = get_ctx(Q, Ctxs), Mod = amqqueue:get_type(Q), diff --git a/src/rabbit_quorum_queue.erl b/src/rabbit_quorum_queue.erl index 44e6462359ed..42b240f52758 100644 --- a/src/rabbit_quorum_queue.erl +++ b/src/rabbit_quorum_queue.erl @@ -21,7 +21,7 @@ -export([init/1, handle_event/2]). -export([recover/1, stop/1, delete/4, delete_immediately/2]). -export([state_info/1, info/2, stat/1, infos/1]). --export([settle/4, reject/4, dequeue/4, consume/3, cancel/6]). +-export([settle/3, reject/4, dequeue/4, consume/3, cancel/5]). -export([credit/4]). -export([purge/1]). -export([stateless_deliver/2, deliver/3, deliver/2]). @@ -37,7 +37,7 @@ -export([add_member/4]). -export([delete_member/3]). -export([requeue/3]). --export([policy_changed/2]). +-export([policy_changed/1]). -export([cleanup_data_dir/0]). -export([shrink_all/1, grow/4]). @@ -412,7 +412,6 @@ delete(Q, _IfUnused, _IfEmpty, ActingUser) when ?amqqueue_is_quorum(Q) -> end end. - force_delete_queue(Servers) -> [begin case catch(ra:force_delete_server(S)) of @@ -438,9 +437,8 @@ delete_immediately(Resource, {_Name, _} = QPid) -> rabbit_core_metrics:queue_deleted(Resource), ok. -settle(CTag, MsgIds, _ChPid, QState) -> - {_, S} = rabbit_fifo_client:settle(quorum_ctag(CTag), MsgIds, QState), - S. +settle(CTag, MsgIds, QState) -> + rabbit_fifo_client:settle(quorum_ctag(CTag), MsgIds, QState). -spec reject(rabbit_types:ctag(), Confirm :: boolean(), [msg_id()], rabbit_fifo_client:state()) -> @@ -457,7 +455,7 @@ credit(CTag, Credit, Drain, QState) -> rabbit_types:ctag(), rabbit_fifo_client:state()) -> {empty, rabbit_fifo_client:state()} | {ok, QLen :: non_neg_integer(), qmsg(), rabbit_fifo_client:state()} | - {error, timeout | term()}. + {error, term()}. dequeue(NoAck, _LimiterPid, CTag0, QState0) -> CTag = quorum_ctag(CTag0), Settlement = case NoAck of @@ -467,18 +465,6 @@ dequeue(NoAck, _LimiterPid, CTag0, QState0) -> unsettled end, rabbit_fifo_client:dequeue(CTag, Settlement, QState0). - % {ok, empty, QState} -> - % {ok, empty, QState}; - % {ok, {{MsgId, {MsgHeader, Msg0}}, MsgsReady}, QState} -> - % Count = maps:get(delivery_count, MsgHeader, 0), - % IsDelivered = Count > 0, - % Msg = rabbit_basic:add_header(<<"x-delivery-count">>, long, Count, Msg0), - % {ok, MsgsReady, {QName, Id, MsgId, IsDelivered, Msg}, QState}; - % {error, _} = Err -> - % Err; - % {timeout, _} -> - % {error, timeout} - % end. -spec consume(amqqueue:amqqueue(), rabbit_queue_type:consume_spec(), @@ -543,8 +529,8 @@ consume(Q, Spec, State0) when ?amqqueue_is_quorum(Q) -> % -spec basic_cancel(rabbit_types:ctag(), ChPid :: pid(), any(), rabbit_fifo_client:state()) -> % {'ok', rabbit_fifo_client:state()}. -cancel(_Q, ChPid, ConsumerTag, OkMsg, _ActingUser, State) -> - maybe_send_reply(ChPid, OkMsg), +cancel(_Q, ConsumerTag, OkMsg, _ActingUser, State) -> + maybe_send_reply(self(), OkMsg), rabbit_fifo_client:cancel_checkout(quorum_ctag(ConsumerTag), State). -spec stateless_deliver(amqqueue:ra_server_id(), rabbit_types:delivery()) -> 'ok'. @@ -556,7 +542,6 @@ stateless_deliver(ServerId, Delivery) -> -spec deliver(Confirm :: boolean(), rabbit_types:delivery(), rabbit_fifo_client:state()) -> {ok | slow, rabbit_fifo_client:state()}. - deliver(false, Delivery, QState0) -> rabbit_fifo_client:enqueue(Delivery#delivery.message, QState0); deliver(true, Delivery, QState0) -> @@ -565,7 +550,12 @@ deliver(true, Delivery, QState0) -> deliver(QSs, #delivery{confirm = Confirm} = Delivery) -> lists:foldl( - fun({Q, S0}, {Qs, Actions}) -> + 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). @@ -636,9 +626,10 @@ maybe_delete_data_dir(UId) -> ok end. -policy_changed(QName, Node) -> - {ok, Q} = rabbit_amqqueue:lookup(QName), - rabbit_fifo_client:update_machine_state(Node, ra_machine_config(Q)). +policy_changed(Q) -> + rabbit_log:info("PLICY CHANGE ~w", [ra_machine_config(Q)]), + QPid = amqqueue:get_pid(Q), + rabbit_fifo_client:update_machine_state(QPid, ra_machine_config(Q)). -spec cluster_state(Name :: atom()) -> 'down' | 'recovering' | 'running'. @@ -652,8 +643,8 @@ cluster_state(Name) -> end end. --spec status(rabbit_types:vhost(), Name :: rabbit_misc:resource_name()) -> - rabbit_types:infos() | {error, term()}. +-spec status(rabbit_types:vhost(), Name :: binary()) -> + [rabbit_types:infos()] | {error, term()}. status(Vhost, QueueName) -> %% Handle not found queues QName = #resource{virtual_host = Vhost, name = QueueName, kind = queue}, diff --git a/test/amqqueue_backward_compatibility_SUITE.erl b/test/amqqueue_backward_compatibility_SUITE.erl index 05a049c9bbd9..a02c4721bc56 100644 --- a/test/amqqueue_backward_compatibility_SUITE.erl +++ b/test/amqqueue_backward_compatibility_SUITE.erl @@ -104,7 +104,7 @@ new_amqqueue_v2_is_amqqueue(_) -> [], VHost, #{}, - classic), + rabbit_classic_queue), ?assert(?is_amqqueue(Queue)), ?assert(?is_amqqueue_v2(Queue)), ?assert(not ?is_amqqueue_v1(Queue)), @@ -253,7 +253,7 @@ amqqueue_v2_type_matching(_) -> [], VHost, #{}, - classic), + rabbit_classic_queue), ?assert(?amqqueue_is_classic(ClassicQueue)), ?assert(amqqueue:is_classic(ClassicQueue)), ?assert(not ?amqqueue_is_quorum(ClassicQueue)), @@ -267,7 +267,7 @@ amqqueue_v2_type_matching(_) -> [], VHost, #{}, - quorum), + rabbit_quorum_queue), ?assert(not ?amqqueue_is_classic(QuorumQueue)), ?assert(not amqqueue:is_classic(QuorumQueue)), ?assert(?amqqueue_is_quorum(QuorumQueue)), diff --git a/test/backing_queue_SUITE.erl b/test/backing_queue_SUITE.erl index e2250a616b9b..8c201ac58c74 100644 --- a/test/backing_queue_SUITE.erl +++ b/test/backing_queue_SUITE.erl @@ -1382,7 +1382,7 @@ publish_and_confirm(Q, Payload, Count) -> Delivery = #delivery{mandatory = false, sender = self(), confirm = true, message = Msg, msg_seq_no = Seq, flow = noflow}, - _QPids = rabbit_amqqueue:deliver([Q], Delivery) + _QPids = rabbit_queue_type:deliver([Q], Delivery, #{}) end || Seq <- Seqs], wait_for_confirms(gb_sets:from_list(Seqs)). diff --git a/test/rabbit_fifo_int_SUITE.erl b/test/rabbit_fifo_int_SUITE.erl index d4ae417a789d..8a585b2bb8e5 100644 --- a/test/rabbit_fifo_int_SUITE.erl +++ b/test/rabbit_fifo_int_SUITE.erl @@ -82,7 +82,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 @@ -95,12 +95,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), - S + {ok, FState4, + [{deliver, C, true, + [{_Qname, _QRef, MsgId, _SomBool, _Msg}]}]} -> + rabbit_fifo_client:settle(C, [MsgId], FState4) end after 5000 -> exit(await_msg_timeout) @@ -125,10 +123,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}]}]} -> + _S = rabbit_fifo_client:return(Ctag, [Mid], FState7), ok end after 2000 -> @@ -147,8 +144,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(F1, 100), - {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. @@ -162,9 +159,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 -> @@ -178,23 +175,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 @@ -212,7 +210,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(F3, 50), @@ -239,9 +237,9 @@ resends_lost_command(Config) -> meck:unload(ra), {ok, F3} = rabbit_fifo_client:enqueue(msg3, F2), {_, _, F4} = process_ra_events(F3, 500), - {ok, {{_, {_, msg1}}, _}, F5} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F4), - {ok, {{_, {_, msg2}}, _}, F6} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F5), - {ok, {{_, {_, msg3}}, _}, _F7} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F6), + {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. @@ -265,7 +263,7 @@ detects_lost_delivery(Config) -> F000 = rabbit_fifo_client:init(ClusterName, [ServerId]), {ok, F00} = rabbit_fifo_client:enqueue(msg1, F000), {_, _, F0} = process_ra_events(F00, 100), - {ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, 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 @@ -295,13 +293,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. @@ -324,9 +322,9 @@ resends_after_lost_applied(Config) -> % send another message {ok, F3} = rabbit_fifo_client:enqueue(msg3, F2), {_, _, F4} = process_ra_events(F3, 500), - {ok, {{_, {_, msg1}}, _}, F5} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F4), - {ok, {{_, {_, msg2}}, _}, F6} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F5), - {ok, {{_, {_, msg3}}, _}, _F7} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F6), + {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. @@ -374,15 +372,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), + {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), @@ -394,11 +393,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_events0(F2, [], [], 250, 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) -> @@ -410,20 +409,20 @@ credit(Config) -> {ok, F2} = rabbit_fifo_client:enqueue(m2, F1), {_, _, F3} = process_ra_events(F2, [], 250), %% checkout with 0 prefetch - {ok, F4} = rabbit_fifo_client:checkout(<<"tag">>, 0, credited, undefined, F3), + {ok, F4} = rabbit_fifo_client:checkout(<<"tag">>, 0, credited, #{}, F3), %% assert no deliveries {_, _, F5} = process_ra_events0(F4, [], [], 250, fun (D, _) -> error({unexpected_delivery, D}) end), %% provide some credit - {ok, F6} = rabbit_fifo_client:credit(<<"tag">>, 1, false, F5), - {[{_, {_, m1}}], [{send_credit_reply, _}], F7} = + F6 = rabbit_fifo_client:credit(<<"tag">>, 1, false, F5), + {[{_, _, _, _, m1}], [{send_credit_reply, _}], F7} = process_ra_events(F6, [], 250), %% credit and drain - {ok, F8} = rabbit_fifo_client:credit(<<"tag">>, 4, true, F7), - {[{_, {_, m2}}], [{send_credit_reply, _}, {send_drained, _}], F9} = + F8 = rabbit_fifo_client:credit(<<"tag">>, 4, true, F7), + {[{_, _, _, _, m2}], [{send_credit_reply, _}, {send_drained, _}], F9} = process_ra_events(F8, [], 250), flush(), @@ -436,8 +435,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}}], _, _} = process_ra_events(F12, [], 250), + F12 = rabbit_fifo_client:credit(<<"tag">>, 10, false, F11), + {[{_, _, _, _, m3}], _, _} = process_ra_events(F12, [], 250), ok. untracked_enqueue(Config) -> @@ -448,7 +447,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. @@ -479,7 +478,7 @@ test_queries(Config) -> receive stop -> ok end end), F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4), - {ok, _} = rabbit_fifo_client:checkout(<<"tag">>, 1, undefined, F0), + {ok, _} = rabbit_fifo_client:checkout(<<"tag">>, 1, #{}, F0), {ok, {_, Ready}, _} = ra:local_query(ServerId, fun rabbit_fifo:query_messages_ready/1), ?assertEqual(1, Ready), @@ -503,15 +502,17 @@ 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(F2_, 100), - {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(F4_, 100), - {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), + {ok, _, {_, _, MsgId, _, msg2}, F5} = rabbit_fifo_client:dequeue(Tag, settled, F4), + _F6 = rabbit_fifo_client:settle(Tag, [MsgId], F5), ra:stop_server(ServerId), ok. @@ -524,8 +525,8 @@ enq_deq_n(0, F0, Acc) -> enq_deq_n(N, F, Acc) -> {ok, F1} = rabbit_fifo_client:enqueue(N, F), {_, _, F2} = process_ra_events(F1, 10), - {ok, {{_, {_, Deq}}, _}, F3} = rabbit_fifo_client:dequeue(term_to_binary(N), settled, F2), - + {ok, _, {_, _, _, _, Deq}, F3} = rabbit_fifo_client:dequeue(term_to_binary(N), settled, F2), + % {ok, {{_, {_, Deq}}, _}, F3} = rabbit_fifo_client:dequeue(term_to_binary(N), settled, F2), {_, _, F4} = process_ra_events(F3, 5), enq_deq_n(N-1, F4, [Deq | Acc]). @@ -541,8 +542,8 @@ process_ra_event(State, Wait) -> receive {ra_event, From, Evt} -> ct:pal("processed ra event ~p~n", [Evt]), - {internal, _, _, S} = - rabbit_fifo_client:handle_ra_event(From, Evt, State), + {ok, S, _Actions} = + rabbit_fifo_client:handle_ra_event(From, Evt, State), S after Wait -> exit(ra_event_timeout) @@ -552,23 +553,27 @@ process_ra_events(State0, Wait) -> process_ra_events(State0, [], Wait). process_ra_events(State, Acc, Wait) -> - 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 + rabbit_fifo_client:settle(Tag, MsgIds, S) end, process_ra_events0(State, Acc, [], Wait, DeliveryFun). process_ra_events0(State0, Acc, Actions0, Wait, DeliveryFun) -> receive {ra_event, From, Evt} -> + ct:pal("Ra event ~w", [Evt]), case rabbit_fifo_client:handle_ra_event(From, Evt, State0) of - {internal, _, Actions, State} -> - process_ra_events0(State, Acc, Actions0 ++ Actions, + {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_events0(State, Msgs, Actions0 ++ Actions, Wait, DeliveryFun); - {{delivery, _Tag, Msgs} = Del, State1} -> - State = DeliveryFun(Del, State1), - process_ra_events0(State, Acc ++ Msgs, Actions0, Wait, DeliveryFun); eol -> eol end @@ -577,36 +582,35 @@ process_ra_events0(State0, Acc, Actions0, Wait, DeliveryFun) -> 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 - after Wait -> - State0 - end. + element(3, + process_ra_events0(State0, [], [], Wait, + fun ({deliver, Tag, _, Msgs}, S) -> + MsgIds = [element(3, M) || M <- Msgs], + ct:pal("discarding ~w", [MsgIds]), + rabbit_fifo_client:discard(Tag, MsgIds, S) + 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 - after Wait -> - State0 - end. + element(3, + process_ra_events0(State0, [], [], Wait, + fun ({deliver, Tag, _, Msgs}, S) -> + MsgIds = [element(3, M) || M <- Msgs], + rabbit_fifo_client:return(Tag, MsgIds, S) + end)). + % 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 + % after Wait -> + % State0 + % end. validate_process_down(Name, 0) -> exit({process_not_down, Name}); diff --git a/test/unit_inbroker_parallel_SUITE.erl b/test/unit_inbroker_parallel_SUITE.erl index a14d8236431c..3d0bb511f5b5 100644 --- a/test/unit_inbroker_parallel_SUITE.erl +++ b/test/unit_inbroker_parallel_SUITE.erl @@ -257,7 +257,7 @@ publish_and_confirm(Q, Payload, Count) -> Delivery = #delivery{mandatory = false, sender = self(), confirm = true, message = Msg, msg_seq_no = Seq, flow = noflow}, - _QPids = rabbit_amqqueue:deliver([Q], Delivery) + _QPids = rabbit_queue_type:deliver([Q], Delivery, #{}) end || Seq <- Seqs], wait_for_confirms(gb_sets:from_list(Seqs)). From 4a91b5d516f2bfc147de32f86acf012552be7e41 Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Thu, 18 Jul 2019 17:37:09 +0100 Subject: [PATCH 08/18] Add purge to queue type --- src/rabbit_amqqueue.erl | 11 ++++------- src/rabbit_classic_queue.erl | 7 +++++++ src/rabbit_queue_type.erl | 9 +++++++++ src/rabbit_quorum_queue.erl | 10 ++++++---- 4 files changed, 26 insertions(+), 11 deletions(-) diff --git a/src/rabbit_amqqueue.erl b/src/rabbit_amqqueue.erl index 25f40c5c2553..8e810c48e035 100644 --- a/src/rabbit_amqqueue.erl +++ b/src/rabbit_amqqueue.erl @@ -129,7 +129,8 @@ recover_classic_queues(VHost, Queues) -> 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)), + 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)], @@ -1109,12 +1110,8 @@ delete(Q, IfUnused, IfEmpty, 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). +purge(Q) when ?is_amqqueue(Q) -> + rabbit_queue_type:purge(Q). -spec requeue(pid() | atom(), {rabbit_fifo:consumer_tag(), [msg_id()]}, diff --git a/src/rabbit_classic_queue.erl b/src/rabbit_classic_queue.erl index 93830ec9f75b..ee038ba8ff21 100644 --- a/src/rabbit_classic_queue.erl +++ b/src/rabbit_classic_queue.erl @@ -15,6 +15,7 @@ is_enabled/0, declare/2, delete/4, + purge/1, policy_changed/1, stat/1, init/1, @@ -214,6 +215,12 @@ info(Q, Items) -> [] 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) -> lists:foldl(fun ({Q, _}, {MPidAcc, SPidAcc, Actions0}) -> QPid = amqqueue:get_pid(Q), diff --git a/src/rabbit_queue_type.erl b/src/rabbit_queue_type.erl index d3edd02501d8..f7131e86863d 100644 --- a/src/rabbit_queue_type.erl +++ b/src/rabbit_queue_type.erl @@ -7,6 +7,7 @@ is_enabled/1, declare/2, delete/4, + purge/1, policy_changed/1, stat/1, name/2, @@ -87,6 +88,9 @@ rabbit_types:ok(non_neg_integer()) | rabbit_types:error(in_use | not_empty). +-callback purge(amqqueue:amqqueue()) -> + {ok, non_neg_integer()} | {error, term()}. + -callback policy_changed(amqqueue:amqqueue()) -> ok. -callback consume(amqqueue:amqqueue(), @@ -166,6 +170,11 @@ 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) -> diff --git a/src/rabbit_quorum_queue.erl b/src/rabbit_quorum_queue.erl index 42b240f52758..47bfbed31e9e 100644 --- a/src/rabbit_quorum_queue.erl +++ b/src/rabbit_quorum_queue.erl @@ -585,8 +585,8 @@ infos(QName) -> --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) -> Leader = amqqueue:get_pid(Q), try @@ -597,7 +597,10 @@ stat(Q) 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) -> @@ -627,7 +630,6 @@ maybe_delete_data_dir(UId) -> end. policy_changed(Q) -> - rabbit_log:info("PLICY CHANGE ~w", [ra_machine_config(Q)]), QPid = amqqueue:get_pid(Q), rabbit_fifo_client:update_machine_state(QPid, ra_machine_config(Q)). From 5277cbba655219d29e30866ef08d0586558e6162 Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Mon, 22 Jul 2019 12:32:39 +0100 Subject: [PATCH 09/18] Add recovery to the queue type interface --- src/rabbit_amqqueue.erl | 96 +++++++----------------------------- src/rabbit_channel.erl | 9 ---- src/rabbit_classic_queue.erl | 42 +++++++++++++++- src/rabbit_queue_type.erl | 55 +++++++++++++++++---- src/rabbit_quorum_queue.erl | 89 ++++++++++++++++++--------------- src/rabbit_vhost.erl | 6 +-- 6 files changed, 155 insertions(+), 142 deletions(-) diff --git a/src/rabbit_amqqueue.erl b/src/rabbit_amqqueue.erl index 8e810c48e035..e36606452948 100644 --- a/src/rabbit_amqqueue.erl +++ b/src/rabbit_amqqueue.erl @@ -78,8 +78,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()}]}. @@ -110,35 +110,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). @@ -152,7 +128,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), @@ -179,67 +154,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(), - 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, amqqueue:get_quorum_nodes(Q)))])) - end). - -find_local_durable_classic_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_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, amqqueue: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(), @@ -250,7 +190,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()). @@ -1108,8 +1047,7 @@ delete_immediately_by_resource(Resources) -> delete(Q, IfUnused, IfEmpty, ActingUser) -> rabbit_queue_type:delete(Q, IfUnused, IfEmpty, ActingUser). --spec purge(amqqueue:amqqueue()) -> {ok, qlen()}. - +-spec purge(amqqueue:amqqueue()) -> qlen(). purge(Q) when ?is_amqqueue(Q) -> rabbit_queue_type:purge(Q). diff --git a/src/rabbit_channel.erl b/src/rabbit_channel.erl index 52b629e7b935..72a112ff5435 100644 --- a/src/rabbit_channel.erl +++ b/src/rabbit_channel.erl @@ -2720,15 +2720,6 @@ maybe_monitor(QPid, QMons) when ?IS_CLASSIC(QPid) -> maybe_monitor(_, QMons) -> QMons. -% maybe_monitor_all([], S) -> S; %% optimisation -% maybe_monitor_all([Item], S) -> maybe_monitor(Item, S); %% optimisation -% maybe_monitor_all(Items, S) -> lists:foldl(fun maybe_monitor/2, S, Items). - -% add_delivery_count_header(#{delivery_count := Count}, Msg) -> -% rabbit_basic:add_header(<<"x-delivery-count">>, long, Count, Msg); -% add_delivery_count_header(_, Msg) -> -% Msg. - qpid_to_ref(Pid) when is_pid(Pid) -> Pid; qpid_to_ref({Name, _}) -> Name; %% assume it already is a ref diff --git a/src/rabbit_classic_queue.erl b/src/rabbit_classic_queue.erl index ee038ba8ff21..6fa468ae5b55 100644 --- a/src/rabbit_classic_queue.erl +++ b/src/rabbit_classic_queue.erl @@ -15,6 +15,8 @@ is_enabled/0, declare/2, delete/4, + is_recoverable/1, + recover/2, purge/1, policy_changed/1, stat/1, @@ -86,6 +88,36 @@ delete(Q, IfUnused, IfEmpty, ActingUser) when ?amqqueue_is_classic(Q) -> {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), @@ -165,7 +197,6 @@ deliver(Qs, #delivery{flow = Flow, confirm = _Confirm} = Delivery) -> {MPids, SPids, Actions} = qpids(Qs), QPids = MPids ++ SPids, - rabbit_log:info("classic deliver to ~w", [QPids]), case Flow of %% Here we are tracking messages sent by the rabbit_channel %% process. We are accessing the rabbit_channel process @@ -272,3 +303,12 @@ delete_crashed_internal(Q, ActingUser) -> {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]. diff --git a/src/rabbit_queue_type.erl b/src/rabbit_queue_type.erl index f7131e86863d..1f7db656c2bb 100644 --- a/src/rabbit_queue_type.erl +++ b/src/rabbit_queue_type.erl @@ -7,6 +7,8 @@ is_enabled/1, declare/2, delete/4, + is_recoverable/1, + recover/2, purge/1, policy_changed/1, stat/1, @@ -70,9 +72,6 @@ % copied from rabbit_amqqueue -type absent_reason() :: 'nodedown' | 'crashed' | stopped | timeout. -%% intitialise and return a queue type specific session context --callback init(amqqueue:amqqueue()) -> term(). - %% is the queue type feature enabled -callback is_enabled() -> boolean(). @@ -88,23 +87,37 @@ 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 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()) -> + rabbit_types:ctag(), + term(), + rabbit_types:username(), + queue_state()) -> {ok, queue_state(), actions()} | {error, term()}. +%% any async events returned from the queue system should be processed through +%% this -callback handle_event(Event :: term(), queue_state()) -> {ok, queue_state(), actions()} | {error, term()} | eol. @@ -130,9 +143,6 @@ {empty, queue_state()} | {error, term()}. - - - %% return a map of state summary information -callback state_info(queue_state()) -> #{atom() := term()}. @@ -272,6 +282,31 @@ cancel(Q, Tag, OkMsg, ActiveUser, Ctxs) -> 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 => []}, Qs), + maps:fold(fun (Mod, Queues, {R0, F0}) -> + {R, F} = Mod:recover(VHost, Queues), + {R0 ++ R, F0 ++ F} + end, {[], []}, ByType). + %% messages sent from queues -spec handle_event(queue_ref(), term(), ctxs()) -> {ok, ctxs(), actions()} | eol | {error, term()}. diff --git a/src/rabbit_quorum_queue.erl b/src/rabbit_quorum_queue.erl index 47bfbed31e9e..a0334284c61f 100644 --- a/src/rabbit_quorum_queue.erl +++ b/src/rabbit_quorum_queue.erl @@ -19,7 +19,7 @@ -behaviour(rabbit_queue_type). -export([init/1, handle_event/2]). --export([recover/1, stop/1, delete/4, delete_immediately/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/3, reject/4, dequeue/4, consume/3, cancel/5]). -export([credit/4]). @@ -313,51 +313,60 @@ reductions(Name) -> 0 end. --spec recover([amqqueue:amqqueue()]) -> [amqqueue:amqqueue()]. +is_recoverable(Q) -> + Node = node(), + lists:member(Node, amqqueue:get_quorum_nodes(Q)). -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, + Nodes = amqqueue:get_quorum_nodes(Q0), + Res = 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. + 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), diff --git a/src/rabbit_vhost.erl b/src/rabbit_vhost.erl index 8ab5a09725f0..40e6059462dd 100644 --- a/src/rabbit_vhost.erl +++ b/src/rabbit_vhost.erl @@ -52,11 +52,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. From 3603048c89e18ea26cc0746abcfd368f50ed9f94 Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Wed, 24 Jul 2019 15:28:24 +0100 Subject: [PATCH 10/18] Rename amqqueue quorum_nodes field To a more generic an extensible opaque queue type specific map. --- src/amqqueue.erl | 30 ++++++++--------- src/amqqueue_v1.erl | 18 +++++----- src/rabbit_amqqueue.erl | 14 ++++---- src/rabbit_channel.erl | 10 +++--- src/rabbit_classic_queue.erl | 2 +- src/rabbit_quorum_queue.erl | 64 +++++++++++++++++++++--------------- src/unconfirmed_messages.erl | 46 +++++++++++++------------- 7 files changed, 97 insertions(+), 87 deletions(-) diff --git a/src/amqqueue.erl b/src/amqqueue.erl index 2a12d11c26b5..2e1fee5cded1 100644 --- a/src/amqqueue.erl +++ b/src/amqqueue.erl @@ -57,9 +57,9 @@ % policy_version get_policy_version/1, set_policy_version/2, - % quorum_nodes - get_quorum_nodes/1, - set_quorum_nodes/2, + % type_state + get_type_state/1, + set_type_state/2, % recoverable_slaves get_recoverable_slaves/1, set_recoverable_slaves/2, @@ -119,7 +119,7 @@ vhost :: rabbit_types:vhost() | undefined | '_', %% secondary index options = #{} :: map() | '_', type = ?amqqueue_v1_type :: module() | '_', - quorum_nodes = [] :: [node()] | '_' + type_state = #{} :: map() | '_' }). -type amqqueue() :: amqqueue_v1:amqqueue_v1() | amqqueue_v2(). @@ -143,7 +143,7 @@ vhost :: rabbit_types:vhost() | undefined, options :: map(), type :: atom(), - quorum_nodes :: [node()] + type_state :: #{} }. -type ra_server_id() :: {Name :: atom(), Node :: node()}. @@ -170,7 +170,7 @@ vhost :: '_', options :: '_', type :: atom() | '_', - quorum_nodes :: '_' + type_state :: '_' }. -export_type([amqqueue/0, @@ -551,18 +551,16 @@ set_recoverable_slaves(#amqqueue{} = Queue, Slaves) -> set_recoverable_slaves(Queue, Slaves) -> amqqueue_v1:set_recoverable_slaves(Queue, Slaves). -% quorum_nodes (new in v2) +% type_state (new in v2) --spec get_quorum_nodes(amqqueue()) -> [node()]. +-spec get_type_state(amqqueue()) -> map(). +get_type_state(#amqqueue{type_state = TState}) -> TState; +get_type_state(_) -> []. -get_quorum_nodes(#amqqueue{quorum_nodes = Nodes}) -> Nodes; -get_quorum_nodes(_) -> []. - --spec set_quorum_nodes(amqqueue(), [node()]) -> amqqueue(). - -set_quorum_nodes(#amqqueue{} = Queue, Nodes) -> - Queue#amqqueue{quorum_nodes = Nodes}; -set_quorum_nodes(Queue, _Nodes) -> +-spec set_type_state(amqqueue(), map()) -> amqqueue(). +set_type_state(#amqqueue{} = Queue, TState) -> + Queue#amqqueue{type_state = TState}; +set_type_state(Queue, _TState) -> Queue. % slave_pids diff --git a/src/amqqueue_v1.erl b/src/amqqueue_v1.erl index 6323cbbff2e0..55569e7d2d60 100644 --- a/src/amqqueue_v1.erl +++ b/src/amqqueue_v1.erl @@ -57,9 +57,9 @@ % policy_version get_policy_version/1, set_policy_version/2, - % quorum_nodes - get_quorum_nodes/1, - set_quorum_nodes/2, + % type_state + get_type_state/1, + set_type_state/2, % recoverable_slaves get_recoverable_slaves/1, set_recoverable_slaves/2, @@ -449,16 +449,16 @@ get_recoverable_slaves(#amqqueue{recoverable_slaves = Slaves}) -> set_recoverable_slaves(#amqqueue{} = Queue, Slaves) -> Queue#amqqueue{recoverable_slaves = Slaves}. -% quorum_nodes (new in v2) +% type_state (new in v2) --spec get_quorum_nodes(amqqueue()) -> no_return(). +-spec get_type_state(amqqueue()) -> no_return(). -get_quorum_nodes(_) -> throw({unsupported, ?record_version, get_quorum_nodes}). +get_type_state(_) -> throw({unsupported, ?record_version, get_type_state}). --spec set_quorum_nodes(amqqueue(), [node()]) -> no_return(). +-spec set_type_state(amqqueue(), [node()]) -> no_return(). -set_quorum_nodes(_, _) -> - throw({unsupported, ?record_version, set_quorum_nodes}). +set_type_state(_, _) -> + throw({unsupported, ?record_version, set_type_state}). % slave_pids diff --git a/src/rabbit_amqqueue.erl b/src/rabbit_amqqueue.erl index e36606452948..6afe8c2ea8de 100644 --- a/src/rabbit_amqqueue.erl +++ b/src/rabbit_amqqueue.erl @@ -733,10 +733,14 @@ list_by_type(Type) -> Qs. list_local_followers() -> + %% TODO: all this shoudl really be beind the queue type interface [ amqqueue:get_name(Q) || Q <- list(), amqqueue:is_quorum(Q), - amqqueue:get_state(Q) =/= crashed, amqqueue:get_leader(Q) =/= node(), lists:member(node(), amqqueue:get_quorum_nodes(Q))]. + amqqueue:get_state(Q) =/= crashed, + amqqueue:get_leader(Q) =/= node(), + rabbit_quorum_queue:is_recoverable(Q) + ]. is_local_to_node(QPid, Node) when ?IS_CLASSIC(QPid) -> Node =:= node(QPid); @@ -943,7 +947,6 @@ force_event_refresh(Ref) -> -spec notify_policy_changed(amqqueue:amqqueue()) -> 'ok'. notify_policy_changed(Q) when ?is_amqqueue(Q) -> - rabbit_queue_type:policy_changed(Q). -spec consumers(amqqueue:amqqueue()) -> @@ -1236,10 +1239,9 @@ forget_all_durable(Node) -> %% Try to promote a slave while down - it should recover as a %% master. We try to take the oldest slave here for best chance of %% recovery. -forget_node_for_queue(DeadNode, Q) +forget_node_for_queue(_DeadNode, Q) when ?amqqueue_is_quorum(Q) -> - QN = amqqueue: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). @@ -1261,7 +1263,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. diff --git a/src/rabbit_channel.erl b/src/rabbit_channel.erl index 72a112ff5435..1657b4efabe4 100644 --- a/src/rabbit_channel.erl +++ b/src/rabbit_channel.erl @@ -794,7 +794,6 @@ handle_info({'DOWN', _MRef, process, QPid, Reason}, State) -> QName -> erase_queue_stats(QName) end, - %% TODO: provide api function for remove from queue states? noreply(State4#ch{queue_states = rabbit_queue_type:remove(QPid, QStates), queue_monitors = pmon:erase(QPid, QMons)}); @@ -2128,8 +2127,8 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ Qs = rabbit_amqqueue:lookup(DelQNames), {QueueStates, Actions} = rabbit_queue_type:deliver(Qs, Delivery, QueueStates0), - State1 = handle_queue_actions(Actions, State0#ch{queue_states = QueueStates}), - % AllDeliveredQRefs = DeliveredQPids ++ [N || {N, _} <- DeliveredQQPids], + State1 = handle_queue_actions(Actions, + State0#ch{queue_states = QueueStates}), %% The maybe_monitor_all/2 monitors all queues to which we %% delivered. But we want to monitor even queues we didn't deliver %% to, since we need their 'DOWN' messages to clean @@ -2162,8 +2161,9 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ case rabbit_event:stats_level(State1, #ch.stats_timer) of fine -> ?INCR_STATS(exchange_stats, XName, 1, publish), - [?INCR_STATS(queue_exchange_stats, {QName, XName}, 1, publish) - || QName <- AllDeliveredQNames]; + [?INCR_STATS(queue_exchange_stats, + {amqqueue:get_name(Q), XName}, 1, publish) + || Q <- Qs]; _ -> ok end, diff --git a/src/rabbit_classic_queue.erl b/src/rabbit_classic_queue.erl index 6fa468ae5b55..1f4b867134c9 100644 --- a/src/rabbit_classic_queue.erl +++ b/src/rabbit_classic_queue.erl @@ -290,7 +290,7 @@ wait_for_promoted_or_stopped(Q0) -> {error, not_found} end. --spec delete_crashed(amqqueue:amqqueue()) -> 'ok'. +-spec delete_crashed(amqqueue:amqqueue()) -> ok. delete_crashed(Q) -> delete_crashed(Q, ?INTERNAL_USER). diff --git a/src/rabbit_quorum_queue.erl b/src/rabbit_quorum_queue.erl index a0334284c61f..270010464e3f 100644 --- a/src/rabbit_quorum_queue.erl +++ b/src/rabbit_quorum_queue.erl @@ -91,7 +91,7 @@ init(Q) when ?is_amqqueue(Q) -> %% 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. {Name, _LeaderNode} = Leader = amqqueue:get_pid(Q), - Nodes = amqqueue:get_quorum_nodes(Q), + Nodes = get_nodes(Q), QName = amqqueue:get_name(Q), %% Ensure the leader is listed first Servers0 = [{Name, N} || N <- Nodes], @@ -125,7 +125,7 @@ declare(Q, _Node) when ?amqqueue_is_quorum(Q) -> Id = {RaName, node()}, Nodes = select_quorum_nodes(QuorumSize, rabbit_mnesia:cluster_nodes(all)), NewQ0 = amqqueue:set_pid(Q, Id), - NewQ1 = amqqueue:set_quorum_nodes(NewQ0, Nodes), + NewQ1 = amqqueue:set_type_state(NewQ0, #{nodes => Nodes}), case rabbit_amqqueue:internal_declare(NewQ1, false) of {created, NewQ} -> TickTimeout = application:get_env(rabbit, quorum_tick_interval, ?TICK_TIMEOUT), @@ -230,7 +230,7 @@ become_leader(QName, Name) -> end), case rabbit_amqqueue:lookup(QName) of {ok, Q0} when ?is_amqqueue(Q0) -> - Nodes = amqqueue:get_quorum_nodes(Q0), + Nodes = get_nodes(Q0), [rpc:call(Node, ?MODULE, rpc_delete_metrics, [QName], ?RPC_TIMEOUT) || Node <- Nodes, Node =/= node()]; @@ -315,7 +315,8 @@ reductions(Name) -> is_recoverable(Q) -> Node = node(), - lists:member(Node, amqqueue:get_quorum_nodes(Q)). + Nodes = get_nodes(Q), + lists:member(Node, Nodes). -spec recover(binary(), [amqqueue:amqqueue()]) -> {[amqqueue:amqqueue()], [amqqueue:amqqueue()]}. @@ -323,7 +324,7 @@ recover(_Vhost, Queues) -> lists:foldl( fun (Q0, {R0, F0}) -> {Name, _} = amqqueue:get_pid(Q0), - Nodes = amqqueue:get_quorum_nodes(Q0), + Nodes = get_nodes(Q0), Res = case ra:restart_server({Name, node()}) of ok -> % queue was restarted, good @@ -381,7 +382,7 @@ stop(VHost) -> delete(Q, _IfUnused, _IfEmpty, ActingUser) when ?amqqueue_is_quorum(Q) -> {Name, _} = amqqueue:get_pid(Q), QName = amqqueue:get_name(Q), - QNodes = amqqueue:get_quorum_nodes(Q), + QNodes = get_nodes(Q), %% TODO Quorum queue needs to support consumer tracking for IfUnused Timeout = ?DELETE_TIMEOUT, {ok, ReadyMsgs, _} = stat(Q), @@ -621,7 +622,7 @@ cleanup_data_dir() -> Name end || Q <- rabbit_amqqueue:list_by_type(?MODULE), - lists:member(node(), amqqueue:get_quorum_nodes(Q))], + lists:member(node(), get_nodes(Q))], Registered = ra_directory:list_registered(), _ = [maybe_delete_data_dir(UId) || {Name, UId} <- Registered, not lists:member(Name, Names)], @@ -664,7 +665,7 @@ status(Vhost, QueueName) -> {ok, Q} when ?amqqueue_is_classic(Q) -> {error, classic_queue_not_supported}; {ok, Q} when ?amqqueue_is_quorum(Q) -> - Nodes = amqqueue:get_quorum_nodes(Q), + Nodes = get_nodes(Q), [begin case get_sys_status({RName, N}) of {ok, Sys} -> @@ -716,7 +717,7 @@ add_member(VHost, Name, Node, Timeout) -> {ok, Q} when ?amqqueue_is_classic(Q) -> {error, classic_queue_not_supported}; {ok, Q} when ?amqqueue_is_quorum(Q) -> - QNodes = amqqueue:get_quorum_nodes(Q), + QNodes = get_nodes(Q), case lists:member(Node, rabbit_mnesia:cluster_nodes(running)) of false -> {error, node_not_running}; @@ -747,9 +748,10 @@ add_member(Q, Node, Timeout) when ?amqqueue_is_quorum(Q) -> case ra:add_member(Members, ServerId, Timeout) of {ok, _, Leader} -> Fun = fun(Q1) -> - Q2 = amqqueue:set_quorum_nodes( - Q1, - [Node | amqqueue:get_quorum_nodes(Q1)]), + Q2 = update_type_state( + Q1, fun(#{nodes := Nodes} = Ts) -> + Ts#{nodes => [Node | Nodes]} + end), amqqueue:set_pid(Q2, Leader) end, rabbit_misc:execute_mnesia_transaction( @@ -773,7 +775,7 @@ delete_member(VHost, Name, Node) -> {ok, Q} when ?amqqueue_is_classic(Q) -> {error, classic_queue_not_supported}; {ok, Q} when ?amqqueue_is_quorum(Q) -> - QNodes = amqqueue:get_quorum_nodes(Q), + QNodes = get_nodes(Q), case lists:member(Node, QNodes) of false -> %% idempotent by design @@ -799,10 +801,11 @@ delete_member(Q, Node) when ?amqqueue_is_quorum(Q) -> case ra:leave_and_delete_server(Members, ServerId) of ok -> Fun = fun(Q1) -> - amqqueue:set_quorum_nodes( + update_type_state( Q1, - lists:delete(Node, - amqqueue:get_quorum_nodes(Q1))) + fun(#{nodes := Nodes} = Ts) -> + Ts#{nodes => lists:delete(Node, Nodes)} + end) end, rabbit_misc:execute_mnesia_transaction( fun() -> rabbit_amqqueue:update(QName, Fun) end), @@ -822,7 +825,7 @@ shrink_all(Node) -> QName = amqqueue:get_name(Q), rabbit_log:info("~s: removing member (replica) on node ~w", [rabbit_misc:rs(QName), Node]), - Size = length(amqqueue:get_quorum_nodes(Q)), + Size = length(get_nodes(Q)), case delete_member(Q, Node) of ok -> {QName, {ok, Size-1}}; @@ -833,7 +836,7 @@ shrink_all(Node) -> end end || Q <- rabbit_amqqueue:list(), amqqueue:get_type(Q) == ?MODULE, - lists:member(Node, amqqueue:get_quorum_nodes(Q))]. + lists:member(Node, get_nodes(Q))]. -spec grow(node(), binary(), binary(), all | even) -> [{rabbit_amqqueue:name(), @@ -841,7 +844,7 @@ shrink_all(Node) -> grow(Node, VhostSpec, QueueSpec, Strategy) -> Running = rabbit_mnesia:cluster_nodes(running), [begin - Size = length(amqqueue:get_quorum_nodes(Q)), + Size = length(get_nodes(Q)), QName = amqqueue:get_name(Q), rabbit_log:info("~s: adding a new member (replica) on node ~w", [rabbit_misc:rs(QName), Node]), @@ -858,10 +861,10 @@ grow(Node, VhostSpec, QueueSpec, Strategy) -> || Q <- rabbit_amqqueue:list(), amqqueue:get_type(Q) == ?MODULE, %% don't add a member if there is already one on the node - not lists:member(Node, amqqueue:get_quorum_nodes(Q)), + not lists:member(Node, get_nodes(Q)), %% node needs to be running lists:member(Node, Running), - matches_strategy(Strategy, amqqueue:get_quorum_nodes(Q)), + matches_strategy(Strategy, get_nodes(Q)), is_match(amqqueue:get_vhost(Q), VhostSpec) andalso is_match(get_resource_name(amqqueue:get_name(Q)), QueueSpec) ]. @@ -1009,12 +1012,12 @@ i(garbage_collection, Q) when ?is_amqqueue(Q) -> [] end; i(members, Q) when ?is_amqqueue(Q) -> - amqqueue:get_quorum_nodes(Q); + get_nodes(Q); i(online, Q) -> online(Q); i(leader, Q) -> leader(Q); i(open_files, Q) when ?is_amqqueue(Q) -> {Name, _} = amqqueue:get_pid(Q), - Nodes = amqqueue:get_quorum_nodes(Q), + Nodes = get_nodes(Q), {Data, _} = rpc:multicall(Nodes, ?MODULE, open_files, [Name]), lists:flatten(Data); i(single_active_consumer_pid, Q) when ?is_amqqueue(Q) -> @@ -1067,12 +1070,12 @@ leader(Q) when ?is_amqqueue(Q) -> end. online(Q) when ?is_amqqueue(Q) -> - Nodes = amqqueue:get_quorum_nodes(Q), + Nodes = get_nodes(Q), {Name, _} = amqqueue:get_pid(Q), [Node || Node <- Nodes, is_process_alive(Name, Node)]. format(Q) when ?is_amqqueue(Q) -> - Nodes = amqqueue:get_quorum_nodes(Q), + Nodes = get_nodes(Q), [{members, Nodes}, {online, online(Q)}, {leader, leader(Q)}]. is_process_alive(Name, Node) -> @@ -1164,7 +1167,7 @@ select_quorum_nodes(Size, Rest, Selected) -> %% member with the current leader first members(Q) when ?amqqueue_is_quorum(Q) -> {RaName, LeaderNode} = amqqueue:get_pid(Q), - Nodes = lists:delete(LeaderNode, amqqueue:get_quorum_nodes(Q)), + Nodes = lists:delete(LeaderNode, get_nodes(Q)), [{RaName, N} || N <- [LeaderNode | Nodes]]. make_ra_conf(Q, ServerId, TickTimeout) -> @@ -1183,3 +1186,12 @@ make_ra_conf(Q, ServerId, TickTimeout) -> tick_timeout => TickTimeout, machine => RaMachine}. +get_nodes(Q) when ?is_amqqueue(Q) -> + #{nodes := Nodes} = amqqueue:get_type_state(Q), + Nodes. + +update_type_state(Q, Fun) when ?is_amqqueue(Q) -> + Ts = amqqueue:get_type_state(Q), + amqqueue:set_type_state(Q, Fun(Ts)). + + diff --git a/src/unconfirmed_messages.erl b/src/unconfirmed_messages.erl index f9f3b2d36cff..cb1461d9ef79 100644 --- a/src/unconfirmed_messages.erl +++ b/src/unconfirmed_messages.erl @@ -85,29 +85,29 @@ insert(MsgId, QueueNames, QueueRefs, XName, #unconfirmed{ordered = Ordered, index = Index, reverse = Reverse} = UC) -> - case maps:get(MsgId, Index, none) of - none -> + case maps:is_key(MsgId, Index) of + false -> 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) - }; - _ -> + 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) + }; + true -> error({message_already_exists, MsgId, QueueNames, QueueRefs, XName, UC}) end. @@ -244,8 +244,6 @@ remove_msg_ref(Confirm, MsgId, QueueName, QueueRef, #msg_status{refs = #{QueueRef := ?SET_VALUE} = Refs, queue_status = QStatus, exchange = XName} = MsgStatus -> - rabbit_log:info("~w QStatus ~w Confirm ~w", [?FUNCTION_NAME, - QStatus, Confirm]), QStatus1 = case {Confirm, QueueName} of {no_confirm, _} -> QStatus; {_, ignore} -> QStatus; From 2e3aaf4b852467eb7fff155c45b13cfffcc92ea4 Mon Sep 17 00:00:00 2001 From: Diana Corbacho Date: Wed, 24 Jul 2019 17:03:27 +0100 Subject: [PATCH 11/18] Fix tests and handle classic API response --- src/rabbit_amqqueue_process.erl | 2 +- test/backing_queue_SUITE.erl | 10 +++++----- test/dynamic_ha_SUITE.erl | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/rabbit_amqqueue_process.erl b/src/rabbit_amqqueue_process.erl index 4b005da9948d..d46bb67e5fd2 100644 --- a/src/rabbit_amqqueue_process.erl +++ b/src/rabbit_amqqueue_process.erl @@ -1271,7 +1271,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 diff --git a/test/backing_queue_SUITE.erl b/test/backing_queue_SUITE.erl index 8c201ac58c74..6261d095a973 100644 --- a/test/backing_queue_SUITE.erl +++ b/test/backing_queue_SUITE.erl @@ -700,8 +700,8 @@ bq_variable_queue_delete_msg_store_files_callback1(Config) -> {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">>, #{}), {ok, CountMinusOne} = rabbit_amqqueue:purge(Q), @@ -733,7 +733,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( @@ -741,8 +741,8 @@ 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, + {ok, CountMinusOne, {QName, QPid1, _AckTag, true, _Msg}, _} = + rabbit_amqqueue:basic_get(Q1, false, Limiter, <<"bq_queue_recover1">>, #{}), exit(QPid1, shutdown), VQ1 = variable_queue_init(Q, true), diff --git a/test/dynamic_ha_SUITE.erl b/test/dynamic_ha_SUITE.erl index 01fbc55e225d..0293135dc7af 100644 --- a/test/dynamic_ha_SUITE.erl +++ b/test/dynamic_ha_SUITE.erl @@ -701,7 +701,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 From 277840b83b69103adadbd9c5ffe2d367411e1de1 Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Thu, 25 Jul 2019 12:31:54 +0100 Subject: [PATCH 12/18] 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. --- src/rabbit_channel.erl | 13 +++++-- src/rabbit_classic_queue.erl | 3 +- src/rabbit_queue_type.erl | 68 ++++++++++++++++++++++++++++-------- 3 files changed, 66 insertions(+), 18 deletions(-) diff --git a/src/rabbit_channel.erl b/src/rabbit_channel.erl index 1657b4efabe4..0403fa72ff88 100644 --- a/src/rabbit_channel.erl +++ b/src/rabbit_channel.erl @@ -2147,11 +2147,11 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ amqqueue:get_pid(Q)), %% slave pids are always pids and thus refs SPids = amqqueue:get_slave_pids(Q), - Acc ++ [QRef | SPids] + [QRef | SPids] ++ Acc end, [], Qs), ok = process_routing_mandatory(Mandatory, AllDeliveredQRefs, Message, State1), - AllDeliveredQNames = [rabbit_queue_type:name(QRef, QueueStates) + AllDeliveredQNames = [rabbit_queue_type:name(QRef, State1#ch.queue_states) || QRef <- AllDeliveredQRefs], State = process_routing_confirm(Confirm, AllDeliveredQRefs, @@ -2778,6 +2778,15 @@ handle_queue_actions(Actions, #ch{} = State0) -> S0#ch{queue_monitors = pmon:monitor(Pid, Mons)}; ({settled, QRef, MsgSeqNos}, S0) -> confirm(MsgSeqNos, QRef, S0); + ({link_names, QRef, QRefs}, S0) -> + link_names(QRef, QRefs, S0); ({deliver, CTag, AckRequired, Msgs}, S0) -> handle_deliver(CTag, AckRequired, Msgs, S0) end, State0, Actions). + +link_names(QRef, QRefs, #ch{queue_states = Qs0} = State) -> + %% linking typicaly slave pids to their master process + Qs = lists:foldl(fun (R, Acc) -> + rabbit_queue_type:link_name(QRef, R, Acc) + end, Qs0, QRefs), + State#ch{queue_states = Qs}. diff --git a/src/rabbit_classic_queue.erl b/src/rabbit_classic_queue.erl index 1f4b867134c9..eaa465b72c4d 100644 --- a/src/rabbit_classic_queue.erl +++ b/src/rabbit_classic_queue.erl @@ -256,7 +256,8 @@ qpids(Qs) -> lists:foldl(fun ({Q, _}, {MPidAcc, SPidAcc, Actions0}) -> QPid = amqqueue:get_pid(Q), SPids = amqqueue:get_slave_pids(Q), - Actions = [{monitor, QPid, QPid} + Actions = [{link_names, QPid, SPids}, + {monitor, QPid, QPid} | [{monitor, P, QPid} || P <- SPids]] ++ Actions0, {[QPid | MPidAcc], SPidAcc ++ SPids, Actions} end, {[], [], []}, Qs). diff --git a/src/rabbit_queue_type.erl b/src/rabbit_queue_type.erl index 1f7db656c2bb..2c88bfa51116 100644 --- a/src/rabbit_queue_type.erl +++ b/src/rabbit_queue_type.erl @@ -13,6 +13,7 @@ policy_changed/1, stat/1, name/2, + link_name/3, remove/2, info/2, state_info/1, @@ -37,10 +38,13 @@ -type queue_name() :: rabbit_types:r(queue). -type queue_state() :: term(). +-define(QREF(QueueReference), + is_pid(QueueReference) 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()} | + {link_name, queue_ref(), [queue_ref()]} | {deliver, rabbit_type:ctag(), boolean(), [rabbit_amqqueue:qmsg()]}. -type actions() :: [action()]. @@ -49,7 +53,7 @@ name :: queue_name(), state :: queue_state()}). --opaque ctxs() :: #{queue_ref() => #ctx{}}. +-opaque ctxs() :: #{queue_ref() => #ctx{} | queue_ref()}. -type consume_spec() :: #{no_ack := boolean(), channel_pid := pid(), @@ -201,15 +205,41 @@ stat(Q) -> undefined | queue_name(). name(QRef, Ctxs) -> case Ctxs of - #{QRef := Ctx} -> + #{QRef := #ctx{} = Ctx} -> Ctx#ctx.name; + #{QRef := Parent} -> + name(Parent, Ctxs); _ -> undefined end. +link_name(QRef, QRef, _Ctxs) -> + %% should this be lenient? + exit({cannot_link_queue_type_name_to_itself, QRef}); +link_name(ParentQRef, QRef, Ctxs) -> + case Ctxs of + #{QRef := ParentQRef} when ?QREF(ParentQRef) -> + %% already registered + Ctxs; + #{ParentQRef := #ctx{}} -> + %% link together + Ctxs#{QRef => ParentQRef}; + _ -> + exit(parent_queue_ref_not_found) + end. + -spec remove(queue_ref(), ctxs()) -> ctxs(). -remove(QRef, Ctxs) -> - maps:remove(QRef, Ctxs). +remove(QRef, Ctxs0) -> + case maps:take(QRef, Ctxs0) of + error -> + Ctxs0; + {_, Ctxs} -> + %% remove all linked queue refs + maps:filter(fun (_, V) -> + V == QRef + end, Ctxs) + end. + -spec info(amqqueue:amqqueue(), all_keys | rabbit_types:info_keys()) -> rabbit_types:infos(). @@ -338,15 +368,16 @@ deliver(Qs, Delivery, stateless) -> {stateless, []}; deliver(Qs, Delivery, Ctxs) -> %% sort by queue type - then dispatch each group - ByType = lists:foldl(fun (Q, Acc) -> - T = amqqueue:get_type(Q), - Ctx = get_ctx(Q, Ctxs), - maps:update_with( - T, fun (A) -> - Ctx = get_ctx(Q, Ctxs), - [{Q, Ctx#ctx.state} | A] - end, [{Q, Ctx#ctx.state}], Acc) - end, #{}, Qs), + ByType = lists:foldl( + fun (Q, Acc) -> + T = amqqueue:get_type(Q), + Ctx = get_ctx(Q, Ctxs), + maps:update_with( + T, fun (A) -> + Ctx = get_ctx(Q, Ctxs), + [{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), @@ -409,7 +440,7 @@ with(QRef, Fun, Ctxs) -> get_ctx(Q, Contexts) when ?is_amqqueue(Q) -> Ref = qref(Q), case Contexts of - #{Ref := Ctx} -> + #{Ref := #ctx{} = Ctx} -> Ctx; _ -> %% not found - initialize @@ -422,7 +453,14 @@ get_ctx(Q, Contexts) when ?is_amqqueue(Q) -> get_ctx(QPid, Contexts) when is_map(Contexts) -> Ref = qref(QPid), %% if we use a QPid it should always be initialised - maps:get(Ref, Contexts, undefined). + case maps:get(Ref, Contexts) of + #ctx{} = Ctx -> + Ctx; + R when ?QREF(R) -> + exit({cannot_get_linked_context_for, QPid}); + _ -> + exit({queue_type_context_not_found, QPid}) + end. set_ctx(Q, Ctx, Contexts) when ?is_amqqueue(Q) andalso is_map(Contexts) -> From 4f6ebd8a9666dbaf06832af07095135f477cf331 Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Thu, 25 Jul 2019 14:26:45 +0100 Subject: [PATCH 13/18] Fix issue with events of deleted queues Also update queue type smoke test to use a cluster by default. --- src/rabbit_queue_type.erl | 20 ++++++++++++++------ test/queue_type_SUITE.erl | 10 ++++++++-- 2 files changed, 22 insertions(+), 8 deletions(-) diff --git a/src/rabbit_queue_type.erl b/src/rabbit_queue_type.erl index 2c88bfa51116..57287349536c 100644 --- a/src/rabbit_queue_type.erl +++ b/src/rabbit_queue_type.erl @@ -343,7 +343,7 @@ recover(VHost, Qs) -> 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) of + case get_ctx(QRef, Ctxs, undefined) of #ctx{module = Mod, state = State0} = Ctx -> case Mod:handle_event(Evt, State0) of @@ -450,16 +450,24 @@ get_ctx(Q, Contexts) when ?is_amqqueue(Q) -> name = Name, state = Mod:init(Q)} end; -get_ctx(QPid, Contexts) when is_map(Contexts) -> +get_ctx(QPid, Contexts) when ?QREF(QPid) andalso is_map(Contexts) -> + case get_ctx(QPid, Contexts, undefined) of + undefined -> + exit({queue_context_not_found, QPid}); + Ctx -> + Ctx + end. + +get_ctx(QPid, Contexts, Default) when is_map(Contexts) -> Ref = qref(QPid), %% if we use a QPid it should always be initialised - case maps:get(Ref, Contexts) of + case maps:get(Ref, Contexts, undefined) of #ctx{} = Ctx -> Ctx; + undefined -> + Default; R when ?QREF(R) -> - exit({cannot_get_linked_context_for, QPid}); - _ -> - exit({queue_type_context_not_found, QPid}) + exit({cannot_get_linked_context_for, QPid}) end. set_ctx(Q, Ctx, Contexts) diff --git a/test/queue_type_SUITE.erl b/test/queue_type_SUITE.erl index a77a5f2d50c6..3970ffe349c6 100644 --- a/test/queue_type_SUITE.erl +++ b/test/queue_type_SUITE.erl @@ -42,7 +42,7 @@ end_per_suite(Config) -> ok. init_per_group(Group, Config) -> - ClusterSize = 1, + ClusterSize = 3, Config1 = rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, ClusterSize}, {rmq_nodename_suffix, Group}, @@ -53,6 +53,7 @@ init_per_group(Group, Config) -> 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( @@ -70,7 +71,12 @@ init_per_group(Group, Config) -> Skip -> end_per_group(Group, Config2), Skip - end. + 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( From e48965533d813d565d7bbba43d46367cebba9954 Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Thu, 25 Jul 2019 15:07:01 +0100 Subject: [PATCH 14/18] be defensive around state_info --- src/rabbit_queue_type.erl | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/rabbit_queue_type.erl b/src/rabbit_queue_type.erl index 57287349536c..7273d241885b 100644 --- a/src/rabbit_queue_type.erl +++ b/src/rabbit_queue_type.erl @@ -253,7 +253,9 @@ info(Q, Items) -> state_info(#ctx{state = S, module = Mod}) -> - Mod:state_info(S). + Mod:state_info(S); +state_info(_) -> + #{}. info_down(Q, all_keys, DownReason) -> info_down(Q, rabbit_amqqueue_process:info_keys(), DownReason); From 550b9806eb1c1c4a3f8e701b5dc752d3689cbdf2 Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Thu, 25 Jul 2019 16:25:15 +0100 Subject: [PATCH 15/18] fix basic.get reject bug --- src/rabbit_fifo_client.erl | 3 +-- src/rabbit_queue_type.erl | 6 ++++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/rabbit_fifo_client.erl b/src/rabbit_fifo_client.erl index 98c3e2734118..0df0b9609e82 100644 --- a/src/rabbit_fifo_client.erl +++ b/src/rabbit_fifo_client.erl @@ -204,7 +204,7 @@ dequeue(ConsumerTag, Settlement, #state{timeout = Timeout, IsDelivered = Count > 0, Msg = add_delivery_count_header(Msg0, Count), {ok, MsgsReady, - {QName, Leader, MsgId, IsDelivered, Msg}, + {QName, qref(Leader), MsgId, IsDelivered, Msg}, State0#state{leader = Leader}}; Err -> Err @@ -498,7 +498,6 @@ update_machine_state(Node, Conf) -> %% with them. -spec handle_ra_event(ra_server_id(), ra_server_proc:ra_event_body(), state()) -> {ok, state(), actions()}. - % {deliver, rabbit_types:ctag(), boolean(), [rabbit_amqqueue:qmsg()]} | eol. handle_ra_event(From, {applied, Seqs}, #state{soft_limit = SftLmt, unblock_handler = UnblockFun} = State0) -> diff --git a/src/rabbit_queue_type.erl b/src/rabbit_queue_type.erl index 7273d241885b..73fa5d34c1b0 100644 --- a/src/rabbit_queue_type.erl +++ b/src/rabbit_queue_type.erl @@ -394,7 +394,8 @@ deliver(Qs, Delivery, Ctxs) -> -spec settle(queue_ref(), rabbit_types:ctag(), [non_neg_integer()], ctxs()) -> ctxs(). -settle(QRef, CTag, MsgIds, Ctxs) -> +settle(QRef, CTag, MsgIds, Ctxs) + when ?QREF(QRef) -> #ctx{state = State0, module = Mod} = Ctx = get_ctx(QRef, Ctxs), State = Mod:settle(CTag, MsgIds, State0), @@ -402,7 +403,8 @@ settle(QRef, CTag, MsgIds, Ctxs) -> -spec reject(queue_ref(), rabbit_types:ctag(), boolean(), [non_neg_integer()], ctxs()) -> ctxs(). -reject(QRef, CTag, Requeue, MsgIds, Ctxs) -> +reject(QRef, CTag, Requeue, MsgIds, Ctxs) + when ?QREF(QRef) -> #ctx{state = State0, module = Mod} = Ctx = get_ctx(QRef, Ctxs), State = Mod:reject(CTag, Requeue, MsgIds, State0), From 5b6bcec4ce86d34b3386b3b9d77a08e34456be8b Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Thu, 25 Jul 2019 16:38:49 +0100 Subject: [PATCH 16/18] correct default value of amqqueue getter --- src/amqqueue.erl | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/amqqueue.erl b/src/amqqueue.erl index 2e1fee5cded1..fbaf32541f3d 100644 --- a/src/amqqueue.erl +++ b/src/amqqueue.erl @@ -554,8 +554,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) -> From 899beb36dc24e2d9f2b2aedeb9dd158a68472d50 Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Fri, 26 Jul 2019 16:39:52 +0100 Subject: [PATCH 17/18] wip --- src/rabbit_channel.erl | 88 +++++++++++++++++++++++++----------------- 1 file changed, 52 insertions(+), 36 deletions(-) diff --git a/src/rabbit_channel.erl b/src/rabbit_channel.erl index 0403fa72ff88..304eba99b941 100644 --- a/src/rabbit_channel.erl +++ b/src/rabbit_channel.erl @@ -733,6 +733,9 @@ handle_cast({reject_publish, MsgSeqNo, _QPid}, State = #ch{unconfirmed = UC}) -> end; handle_cast({confirm, MsgSeqNos, QPid}, State) -> + + % rabbit_log:info("handle_cast({confirm, ~w ~w", + % [QPid, MsgSeqNos]), noreply_coalesce(confirm(MsgSeqNos, QPid, State)). handle_info({ra_event, From, _} = Evt, @@ -744,7 +747,8 @@ handle_info({ra_event, From, _} = Evt, State = handle_queue_actions(Actions, State1), noreply_coalesce(State); eol -> - State1 = handle_consuming_queue_down_or_eol(QRef, State0), + QName = rabbit_queue_type:name(QRef, QueueStates0), + State1 = handle_consuming_queue_down_or_eol(QRef, QName, State0), State2 = handle_delivering_queue_down(QRef, State1), {ConfirmMXs, RejectMXs, UC1} = unconfirmed_messages:forget_ref(QRef, State2#ch.unconfirmed), @@ -777,9 +781,15 @@ handle_info(emit_stats, State) -> %% stats timer. {noreply, send_confirms_and_nacks(State1), hibernate}; -handle_info({'DOWN', _MRef, process, QPid, Reason}, State) -> - State1 = handle_publishing_queue_down(QPid, Reason, State), - State3 = handle_consuming_queue_down_or_eol(QPid, State1), +handle_info({'DOWN', _MRef, process, QPid, Reason}, + #ch{queue_states = QStates, queue_monitors = QMons} = State0) -> + %% need to remove classic queue type state before handling consuming + %% queue down as it may want to recover and thus create a new state + QName = rabbit_queue_type:name(QPid, QStates), + State = State0#ch{queue_monitors = pmon:erase(QPid, QMons), + queue_states = rabbit_queue_type:remove(QPid, QStates)}, + State1 = handle_publishing_queue_down(QPid, QName, Reason, State), + State3 = handle_consuming_queue_down_or_eol(QPid, QName, State1), State4 = handle_delivering_queue_down(QPid, State3), %% A rabbit_amqqueue_process has died. If our channel was being %% blocked by this process, and no other process is blocking our @@ -787,15 +797,15 @@ handle_info({'DOWN', _MRef, process, QPid, Reason}, State) -> %% any credit that was deferred will be sent to the rabbit_reader %% processs that might be blocked by this particular channel. credit_flow:peer_down(QPid), - #ch{queue_states = QStates, queue_monitors = QMons} = State4, - case rabbit_queue_type:name(QPid, QStates) of + %% check if the queue state has be re-added and only delete queue stats + %% if it has + case rabbit_queue_type:name(QPid, State4#ch.queue_states) of undefined -> ok; - QName -> - erase_queue_stats(QName) + QN -> + erase_queue_stats(QN) end, - noreply(State4#ch{queue_states = rabbit_queue_type:remove(QPid, QStates), - queue_monitors = pmon:erase(QPid, QMons)}); + noreply(State4); handle_info({'EXIT', _Pid, Reason}, State) -> {stop, Reason, State}; @@ -1191,6 +1201,7 @@ maybe_set_fast_reply_to(C, _State) -> record_rejects([], State) -> State; record_rejects(MXs, State = #ch{rejected = R, tx = Tx}) -> + rabbit_log:info("record_rejects ~w", [MXs]), Tx1 = case Tx of none -> none; _ -> failed @@ -1770,7 +1781,6 @@ maybe_stat(true, _Q) -> {ok, 0, 0}. consumer_monitor(ConsumerTag, State = #ch{consumer_mapping = ConsumerMapping, - queue_monitors = QMons, queue_consumers = QCons}) -> {Q, _} = maps:get(ConsumerTag, ConsumerMapping), QPid = amqqueue:get_pid(Q), @@ -1780,35 +1790,30 @@ consumer_monitor(ConsumerTag, error -> gb_sets:singleton(ConsumerTag) end, QCons1 = maps:put(QRef, CTags1, QCons), - State#ch{queue_monitors = maybe_monitor(QRef, QMons), - queue_consumers = QCons1}. + State#ch{queue_consumers = QCons1}. track_delivering_queue(NoAck, QPid, - State = #ch{ - % queue_monitors = QMons, - delivering_queues = DQ}) -> + State = #ch{delivering_queues = DQ}) -> QRef = qpid_to_ref(QPid), - State#ch{ - % queue_monitors = maybe_monitor(QRef, QMons), - delivering_queues = case NoAck of + State#ch{delivering_queues = case NoAck of true -> DQ; false -> sets:add_element(QRef, DQ) end}. -handle_publishing_queue_down(QPid, Reason, - State = #ch{unconfirmed = UC, - queue_states = QStates}) +handle_publishing_queue_down(QPid, QName, Reason, + State = #ch{unconfirmed = UC}) when ?IS_CLASSIC(QPid) -> - case rabbit_queue_type:name(QPid, QStates) of + case QName of %% The queue is unknown, the confirm must have been processed already undefined -> State; _QName -> + rabbit_log:info("handle_publishing_queue_down ~w ~w", + [QPid, Reason]), case {rabbit_misc:is_abnormal_exit(Reason), Reason} of {true, _} -> {RejectMXs, UC1} = unconfirmed_messages:reject_all_for_queue(QPid, UC), - record_rejects(RejectMXs, State#ch{unconfirmed = UC1}); {false, normal} -> {ConfirmMXs, RejectMXs, UC1} = @@ -1822,31 +1827,38 @@ handle_publishing_queue_down(QPid, Reason, unconfirmed_messages:forget_ref(QPid, UC), State1 = record_confirms(ConfirmMXs, State#ch{unconfirmed = UC1}), + rabbit_log:info("confirms / rejects ~w ~w", + [ConfirmMXs, RejectMXs]), record_rejects(RejectMXs, State1) end end; -handle_publishing_queue_down(QPid, _Reason, _State) when ?IS_QUORUM(QPid) -> +handle_publishing_queue_down(QPid, _QName, _Reason, _State) + when ?IS_QUORUM(QPid) -> error(quorum_queues_should_never_be_monitored). -handle_consuming_queue_down_or_eol(QRef, - State = #ch{queue_consumers = QCons, - queue_states = QStates}) -> +handle_consuming_queue_down_or_eol(_QRef, undefined, State) -> + State; +handle_consuming_queue_down_or_eol(QRef, QName, + State = #ch{queue_consumers = QCons}) -> ConsumerTags = case maps:find(QRef, QCons) of error -> gb_sets:new(); {ok, CTags} -> CTags end, gb_sets:fold( fun (CTag, StateN = #ch{consumer_mapping = CMap}) -> - QName = rabbit_queue_type:name(QRef, QStates), case queue_down_consumer_action(CTag, CMap) of remove -> cancel_consumer(CTag, QName, StateN); {recover, {NoAck, ConsumerPrefetch, Exclusive, Args}} -> - case catch basic_consume( %% [0] + rabbit_log:info("recovering consumer ~p", [CTag]), + case catch basic_consume( QName, NoAck, ConsumerPrefetch, CTag, Exclusive, Args, true, StateN) of {ok, StateN1} -> StateN1; - _ -> cancel_consumer(CTag, QName, StateN) + Err -> + rabbit_log:info("re-consume failed with ~p", + [Err]), + cancel_consumer(CTag, QName, StateN) end end end, State#ch{queue_consumers = maps:remove(QRef, QCons)}, ConsumerTags). @@ -1859,6 +1871,7 @@ handle_consuming_queue_down_or_eol(QRef, cancel_consumer(CTag, QName, State = #ch{cfg = #conf{capabilities = Capabilities}, consumer_mapping = CMap}) -> + rabbit_log:info("cancelling consumer ~p", [CTag]), case rabbit_misc:table_lookup( Capabilities, <<"consumer_cancel_notify">>) of {bool, true} -> ok = send(#'basic.cancel'{consumer_tag = CTag, @@ -2062,7 +2075,7 @@ notify_queues(State = #ch{cfg = #conf{state = closing}}) -> {ok, State}; notify_queues(State = #ch{consumer_mapping = Consumers, cfg = Cfg, - delivering_queues = DQ }) -> + delivering_queues = DQ}) -> QRefs0 = sets:to_list( sets:union(sets:from_list(consumer_queue_refs(Consumers)), DQ)), %% filter to only include pids to avoid trying to notify quorum queues @@ -2198,6 +2211,8 @@ confirm(MsgSeqNos, QRef, State = #ch{queue_states = QStates, unconfirmed = UC}) % QName = maps:get(QRef, QNames, ignore), QName = case rabbit_queue_type:name(QRef, QStates) of undefined -> + rabbit_log:info("confirm( queue not found ~w", + [QRef, MsgSeqNos]), ignore; N -> N @@ -2715,10 +2730,10 @@ maybe_cancel_tick_timer(#ch{tick_timer = TRef, %% only classic queues need monitoring so rather than special casing %% everywhere monitors are set up we wrap it here for this module -maybe_monitor(QPid, QMons) when ?IS_CLASSIC(QPid) -> - pmon:monitor(QPid, QMons); -maybe_monitor(_, QMons) -> - QMons. +% maybe_monitor(QPid, QMons) when ?IS_CLASSIC(QPid) -> +% pmon:monitor(QPid, QMons); +% maybe_monitor(_, QMons) -> +% QMons. qpid_to_ref(Pid) when is_pid(Pid) -> Pid; qpid_to_ref({Name, _}) -> Name; @@ -2775,6 +2790,7 @@ handle_queue_actions(Actions, #ch{} = State0) -> credit_drained = Credit}), S0; ({monitor, Pid, _QRef}, #ch{queue_monitors = Mons} = S0) -> + % rabbit_log:info("monitoriing ~w", [Pid]), S0#ch{queue_monitors = pmon:monitor(Pid, Mons)}; ({settled, QRef, MsgSeqNos}, S0) -> confirm(MsgSeqNos, QRef, S0); From 1465bb6a9264451a5db444cf56d72cdbb255db60 Mon Sep 17 00:00:00 2001 From: kjnilsson Date: Mon, 5 Aug 2019 14:15:37 +0100 Subject: [PATCH 18/18] wip --- src/rabbit_channel.erl | 9 +-------- src/rabbit_classic_queue.erl | 1 + src/rabbit_queue_type.erl | 17 +++++++++++++++-- 3 files changed, 17 insertions(+), 10 deletions(-) diff --git a/src/rabbit_channel.erl b/src/rabbit_channel.erl index 304eba99b941..01883eee30f6 100644 --- a/src/rabbit_channel.erl +++ b/src/rabbit_channel.erl @@ -1808,8 +1808,6 @@ handle_publishing_queue_down(QPid, QName, Reason, undefined -> State; _QName -> - rabbit_log:info("handle_publishing_queue_down ~w ~w", - [QPid, Reason]), case {rabbit_misc:is_abnormal_exit(Reason), Reason} of {true, _} -> {RejectMXs, UC1} = @@ -1827,8 +1825,6 @@ handle_publishing_queue_down(QPid, QName, Reason, unconfirmed_messages:forget_ref(QPid, UC), State1 = record_confirms(ConfirmMXs, State#ch{unconfirmed = UC1}), - rabbit_log:info("confirms / rejects ~w ~w", - [ConfirmMXs, RejectMXs]), record_rejects(RejectMXs, State1) end end; @@ -2135,8 +2131,7 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ mandatory = Mandatory, confirm = Confirm, msg_seq_no = MsgSeqNo}, - DelQNames}, State0 = #ch{queue_monitors = _QMons, - queue_states = QueueStates0}) -> + DelQNames}, State0 = #ch{queue_states = QueueStates0}) -> Qs = rabbit_amqqueue:lookup(DelQNames), {QueueStates, Actions} = rabbit_queue_type:deliver(Qs, Delivery, QueueStates0), @@ -2211,8 +2206,6 @@ confirm(MsgSeqNos, QRef, State = #ch{queue_states = QStates, unconfirmed = UC}) % QName = maps:get(QRef, QNames, ignore), QName = case rabbit_queue_type:name(QRef, QStates) of undefined -> - rabbit_log:info("confirm( queue not found ~w", - [QRef, MsgSeqNos]), ignore; N -> N diff --git a/src/rabbit_classic_queue.erl b/src/rabbit_classic_queue.erl index eaa465b72c4d..4a82570be8e2 100644 --- a/src/rabbit_classic_queue.erl +++ b/src/rabbit_classic_queue.erl @@ -195,6 +195,7 @@ handle_event(_Evt, State) -> {[{amqqueue:amqqueue(), state()}], rabbit_queue_type:actions()}. deliver(Qs, #delivery{flow = Flow, confirm = _Confirm} = Delivery) -> + %% TODO: record master and slaves for confirm processing {MPids, SPids, Actions} = qpids(Qs), QPids = MPids ++ SPids, case Flow of diff --git a/src/rabbit_queue_type.erl b/src/rabbit_queue_type.erl index 73fa5d34c1b0..4871d25d26ab 100644 --- a/src/rabbit_queue_type.erl +++ b/src/rabbit_queue_type.erl @@ -34,9 +34,11 @@ %% temporary -export([with/3]). --type queue_ref() :: pid() | atom(). +%% gah what is a good identity of a classic queue including all replicas -type queue_name() :: rabbit_types:r(queue). +-type queue_ref() :: pid() | atom(). -type queue_state() :: term(). +-type msg_tag() :: term(). -define(QREF(QueueReference), is_pid(QueueReference) orelse is_atom(QueueReference)). @@ -44,13 +46,24 @@ %% session, like knowing when to notify on monitor down -type action() :: {monitor, Pid :: pid(), queue_ref()} | - {link_name, queue_ref(), [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()]. -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()}). -opaque ctxs() :: #{queue_ref() => #ctx{} | queue_ref()}.