From d8ae41119c518751283e438cdb93b40ca35616dc Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Mon, 9 Jan 2023 10:46:18 +0100 Subject: [PATCH 01/12] Move missing Mnesia-specific code to rabbit_db_topic_exchange module --- deps/rabbit/BUILD.bazel | 6 + deps/rabbit/src/rabbit_db_topic_exchange.erl | 298 +++++++++++++++++ .../rabbit/src/rabbit_exchange_type_topic.erl | 210 +----------- deps/rabbit/test/exchanges_SUITE.erl | 299 ++++++++++++++++++ 4 files changed, 608 insertions(+), 205 deletions(-) create mode 100644 deps/rabbit/src/rabbit_db_topic_exchange.erl create mode 100644 deps/rabbit/test/exchanges_SUITE.erl diff --git a/deps/rabbit/BUILD.bazel b/deps/rabbit/BUILD.bazel index 585aa2e70b30..4689ca4cc9eb 100644 --- a/deps/rabbit/BUILD.bazel +++ b/deps/rabbit/BUILD.bazel @@ -1093,6 +1093,12 @@ rabbitmq_integration_suite( size = "small", ) +rabbitmq_integration_suite( + name = "exchanges_SUITE", + size = "small", +) + + assert_suites() filegroup( diff --git a/deps/rabbit/src/rabbit_db_topic_exchange.erl b/deps/rabbit/src/rabbit_db_topic_exchange.erl new file mode 100644 index 000000000000..a5282c1a0944 --- /dev/null +++ b/deps/rabbit/src/rabbit_db_topic_exchange.erl @@ -0,0 +1,298 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_topic_exchange). + +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([insert/1, delete_all_for_exchange/1, delete/1, match/2]). + +%% ------------------------------------------------------------------- +%% insert(). +%% ------------------------------------------------------------------- + +-spec insert(Binding) -> ok when + Binding :: rabbit_types:binding(). +%% @doc Inserts a topic binding. +%% +%% @private + +insert(#binding{source = XName, key = RoutingKey, destination = Destination, args = Args}) -> + rabbit_db:run( + #{mnesia => fun() -> insert_in_mnesia(XName, RoutingKey, Destination, Args) end + }). + +%% ------------------------------------------------------------------- +%% delete_all_for_exchange(). +%% ------------------------------------------------------------------- + +-spec delete_all_for_exchange(ExchangeName) -> ok when + ExchangeName :: rabbit_types:r('exchange'). +%% @doc Deletes all topic bindings for the exchange named `ExchangeName' +%% +%% @private + +delete_all_for_exchange(XName) -> + rabbit_db:run( + #{mnesia => fun() -> delete_all_for_exchange_in_mnesia(XName) end + }). + +%% ------------------------------------------------------------------- +%% delete(). +%% ------------------------------------------------------------------- + +-spec delete([Binding]) -> ok when + Binding :: rabbit_types:binding(). +%% @doc Deletes all given topic bindings +%% +%% @private + +delete(Bs) when is_list(Bs) -> + rabbit_db:run( + #{mnesia => fun() -> delete_in_mnesia(Bs) end + }). + +%% ------------------------------------------------------------------- +%% match(). +%% ------------------------------------------------------------------- + +-spec match(ExchangeName, RoutingKey) -> ok when + ExchangeName :: rabbit_types:r('exchange'), + RoutingKey :: binary(). +%% @doc Finds the topic binding matching the given exchange and routing key and returns +%% the destination of the binding +%% +%% @returns a list of resources +%% +%% @private + +match(XName, RoutingKey) -> + rabbit_db:run( + #{mnesia => + fun() -> + match_in_mnesia(XName, RoutingKey) + end + }). + +%% Internal +%% -------------------------------------------------------------- + +split_topic_key(Key) -> + split_topic_key(Key, [], []). + +insert_in_mnesia(XName, RoutingKey, Destination, Args) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + FinalNode = follow_down_create(XName, split_topic_key(RoutingKey)), + trie_add_binding(XName, FinalNode, Destination, Args), + ok + end). + +delete_all_for_exchange_in_mnesia(XName) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + trie_remove_all_nodes(XName), + trie_remove_all_edges(XName), + trie_remove_all_bindings(XName), + ok + end). + +match_in_mnesia(XName, RoutingKey) -> + Words = split_topic_key(RoutingKey), + mnesia:async_dirty(fun trie_match/2, [XName, Words]). + +trie_remove_all_nodes(X) -> + remove_all(rabbit_topic_trie_node, + #topic_trie_node{trie_node = #trie_node{exchange_name = X, + _ = '_'}, + _ = '_'}). + +trie_remove_all_edges(X) -> + remove_all(rabbit_topic_trie_edge, + #topic_trie_edge{trie_edge = #trie_edge{exchange_name = X, + _ = '_'}, + _ = '_'}). + +trie_remove_all_bindings(X) -> + remove_all(rabbit_topic_trie_binding, + #topic_trie_binding{ + trie_binding = #trie_binding{exchange_name = X, _ = '_'}, + _ = '_'}). + +remove_all(Table, Pattern) -> + lists:foreach(fun (R) -> mnesia:delete_object(Table, R, write) end, + mnesia:match_object(Table, Pattern, write)). + +delete_in_mnesia_tx(Bs) -> + %% See rabbit_binding:lock_route_tables for the rationale for + %% taking table locks. + _ = case Bs of + [_] -> ok; + _ -> [mnesia:lock({table, T}, write) || + T <- [rabbit_topic_trie_node, + rabbit_topic_trie_edge, + rabbit_topic_trie_binding]] + end, + [case follow_down_get_path(X, split_topic_key(K)) of + {ok, Path = [{FinalNode, _} | _]} -> + trie_remove_binding(X, FinalNode, D, Args), + remove_path_if_empty(X, Path); + {error, _Node, _RestW} -> + %% We're trying to remove a binding that no longer exists. + %% That's unexpected, but shouldn't be a problem. + ok + end || #binding{source = X, key = K, destination = D, args = Args} <- Bs], + ok. + +delete_in_mnesia(Bs) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> delete_in_mnesia_tx(Bs) end). + +split_topic_key(<<>>, [], []) -> + []; +split_topic_key(<<>>, RevWordAcc, RevResAcc) -> + lists:reverse([lists:reverse(RevWordAcc) | RevResAcc]); +split_topic_key(<<$., Rest/binary>>, RevWordAcc, RevResAcc) -> + split_topic_key(Rest, [], [lists:reverse(RevWordAcc) | RevResAcc]); +split_topic_key(<>, RevWordAcc, RevResAcc) -> + split_topic_key(Rest, [C | RevWordAcc], RevResAcc). + +trie_match(X, Words) -> + trie_match(X, root, Words, []). + +trie_match(X, Node, [], ResAcc) -> + trie_match_part(X, Node, "#", fun trie_match_skip_any/4, [], + trie_bindings(X, Node) ++ ResAcc); +trie_match(X, Node, [W | RestW] = Words, ResAcc) -> + lists:foldl(fun ({WArg, MatchFun, RestWArg}, Acc) -> + trie_match_part(X, Node, WArg, MatchFun, RestWArg, Acc) + end, ResAcc, [{W, fun trie_match/4, RestW}, + {"*", fun trie_match/4, RestW}, + {"#", fun trie_match_skip_any/4, Words}]). + +trie_match_part(X, Node, Search, MatchFun, RestW, ResAcc) -> + case trie_child(X, Node, Search) of + {ok, NextNode} -> MatchFun(X, NextNode, RestW, ResAcc); + error -> ResAcc + end. + +trie_match_skip_any(X, Node, [], ResAcc) -> + trie_match(X, Node, [], ResAcc); +trie_match_skip_any(X, Node, [_ | RestW] = Words, ResAcc) -> + trie_match_skip_any(X, Node, RestW, + trie_match(X, Node, Words, ResAcc)). + +follow_down_create(X, Words) -> + case follow_down_last_node(X, Words) of + {ok, FinalNode} -> FinalNode; + {error, Node, RestW} -> lists:foldl( + fun (W, CurNode) -> + NewNode = new_node_id(), + trie_add_edge(X, CurNode, NewNode, W), + NewNode + end, Node, RestW) + end. + +new_node_id() -> + rabbit_guid:gen(). + +follow_down_last_node(X, Words) -> + follow_down(X, fun (_, Node, _) -> Node end, root, Words). + +follow_down_get_path(X, Words) -> + follow_down(X, fun (W, Node, PathAcc) -> [{Node, W} | PathAcc] end, + [{root, none}], Words). + +follow_down(X, AccFun, Acc0, Words) -> + follow_down(X, root, AccFun, Acc0, Words). + +follow_down(_X, _CurNode, _AccFun, Acc, []) -> + {ok, Acc}; +follow_down(X, CurNode, AccFun, Acc, Words = [W | RestW]) -> + case trie_child(X, CurNode, W) of + {ok, NextNode} -> follow_down(X, NextNode, AccFun, + AccFun(W, NextNode, Acc), RestW); + error -> {error, Acc, Words} + end. + +remove_path_if_empty(_, [{root, none}]) -> + ok; +remove_path_if_empty(X, [{Node, W} | [{Parent, _} | _] = RestPath]) -> + case mnesia:read(rabbit_topic_trie_node, + #trie_node{exchange_name = X, node_id = Node}, write) of + [] -> trie_remove_edge(X, Parent, Node, W), + remove_path_if_empty(X, RestPath); + _ -> ok + end. + +trie_child(X, Node, Word) -> + case mnesia:read({rabbit_topic_trie_edge, + #trie_edge{exchange_name = X, + node_id = Node, + word = Word}}) of + [#topic_trie_edge{node_id = NextNode}] -> {ok, NextNode}; + [] -> error + end. + +trie_bindings(X, Node) -> + MatchHead = #topic_trie_binding{ + trie_binding = #trie_binding{exchange_name = X, + node_id = Node, + destination = '$1', + arguments = '_'}}, + mnesia:select(rabbit_topic_trie_binding, [{MatchHead, [], ['$1']}]). + +trie_update_node_counts(X, Node, Field, Delta) -> + E = case mnesia:read(rabbit_topic_trie_node, + #trie_node{exchange_name = X, + node_id = Node}, write) of + [] -> #topic_trie_node{trie_node = #trie_node{ + exchange_name = X, + node_id = Node}, + edge_count = 0, + binding_count = 0}; + [E0] -> E0 + end, + case setelement(Field, E, element(Field, E) + Delta) of + #topic_trie_node{edge_count = 0, binding_count = 0} -> + ok = mnesia:delete_object(rabbit_topic_trie_node, E, write); + EN -> + ok = mnesia:write(rabbit_topic_trie_node, EN, write) + end. + +trie_add_edge(X, FromNode, ToNode, W) -> + trie_update_node_counts(X, FromNode, #topic_trie_node.edge_count, +1), + trie_edge_op(X, FromNode, ToNode, W, fun mnesia:write/3). + +trie_remove_edge(X, FromNode, ToNode, W) -> + trie_update_node_counts(X, FromNode, #topic_trie_node.edge_count, -1), + trie_edge_op(X, FromNode, ToNode, W, fun mnesia:delete_object/3). + +trie_edge_op(X, FromNode, ToNode, W, Op) -> + ok = Op(rabbit_topic_trie_edge, + #topic_trie_edge{trie_edge = #trie_edge{exchange_name = X, + node_id = FromNode, + word = W}, + node_id = ToNode}, + write). + +trie_add_binding(X, Node, D, Args) -> + trie_update_node_counts(X, Node, #topic_trie_node.binding_count, +1), + trie_binding_op(X, Node, D, Args, fun mnesia:write/3). + +trie_remove_binding(X, Node, D, Args) -> + trie_update_node_counts(X, Node, #topic_trie_node.binding_count, -1), + trie_binding_op(X, Node, D, Args, fun mnesia:delete_object/3). + +trie_binding_op(X, Node, D, Args, Op) -> + ok = Op(rabbit_topic_trie_binding, + #topic_trie_binding{ + trie_binding = #trie_binding{exchange_name = X, + node_id = Node, + destination = D, + arguments = Args}}, + write). diff --git a/deps/rabbit/src/rabbit_exchange_type_topic.erl b/deps/rabbit/src/rabbit_exchange_type_topic.erl index 041185562153..37568ed480d5 100644 --- a/deps/rabbit/src/rabbit_exchange_type_topic.erl +++ b/deps/rabbit/src/rabbit_exchange_type_topic.erl @@ -35,52 +35,28 @@ description() -> serialise_events() -> false. %% NB: This may return duplicate results in some situations (that's ok) -route(#exchange{name = X}, +route(#exchange{name = XName}, #delivery{message = #basic_message{routing_keys = Routes}}) -> - lists:append([begin - Words = split_topic_key(RKey), - mnesia:async_dirty(fun trie_match/2, [X, Words]) - end || RKey <- Routes]). + lists:append([rabbit_db_topic_exchange:match(XName, RKey) || RKey <- Routes]). validate(_X) -> ok. validate_binding(_X, _B) -> ok. create(_Tx, _X) -> ok. delete(transaction, #exchange{name = X}, _Bs) -> - trie_remove_all_nodes(X), - trie_remove_all_edges(X), - trie_remove_all_bindings(X), - ok; + rabbit_db_topic_exchange:delete_all_for_exchange(X); delete(none, _Exchange, _Bs) -> ok. policy_changed(_X1, _X2) -> ok. add_binding(transaction, _Exchange, Binding) -> - internal_add_binding(Binding); + rabbit_db_topic_exchange:insert(Binding); add_binding(none, _Exchange, _Binding) -> ok. remove_bindings(transaction, _X, Bs) -> - %% See rabbit_binding:lock_route_tables for the rationale for - %% taking table locks. - _ = case Bs of - [_] -> ok; - _ -> [mnesia:lock({table, T}, write) || - T <- [rabbit_topic_trie_node, - rabbit_topic_trie_edge, - rabbit_topic_trie_binding]] - end, - [case follow_down_get_path(X, split_topic_key(K)) of - {ok, Path = [{FinalNode, _} | _]} -> - trie_remove_binding(X, FinalNode, D, Args), - remove_path_if_empty(X, Path); - {error, _Node, _RestW} -> - %% We're trying to remove a binding that no longer exists. - %% That's unexpected, but shouldn't be a problem. - ok - end || #binding{source = X, key = K, destination = D, args = Args} <- Bs], - ok; + rabbit_db_topic_exchange:delete(Bs); remove_bindings(none, _X, _Bs) -> ok. @@ -88,179 +64,3 @@ assert_args_equivalence(X, Args) -> rabbit_exchange:assert_args_equivalence(X, Args). %%---------------------------------------------------------------------------- - -internal_add_binding(#binding{source = X, key = K, destination = D, - args = Args}) -> - FinalNode = follow_down_create(X, split_topic_key(K)), - trie_add_binding(X, FinalNode, D, Args), - ok. - -trie_match(X, Words) -> - trie_match(X, root, Words, []). - -trie_match(X, Node, [], ResAcc) -> - trie_match_part(X, Node, "#", fun trie_match_skip_any/4, [], - trie_bindings(X, Node) ++ ResAcc); -trie_match(X, Node, [W | RestW] = Words, ResAcc) -> - lists:foldl(fun ({WArg, MatchFun, RestWArg}, Acc) -> - trie_match_part(X, Node, WArg, MatchFun, RestWArg, Acc) - end, ResAcc, [{W, fun trie_match/4, RestW}, - {"*", fun trie_match/4, RestW}, - {"#", fun trie_match_skip_any/4, Words}]). - -trie_match_part(X, Node, Search, MatchFun, RestW, ResAcc) -> - case trie_child(X, Node, Search) of - {ok, NextNode} -> MatchFun(X, NextNode, RestW, ResAcc); - error -> ResAcc - end. - -trie_match_skip_any(X, Node, [], ResAcc) -> - trie_match(X, Node, [], ResAcc); -trie_match_skip_any(X, Node, [_ | RestW] = Words, ResAcc) -> - trie_match_skip_any(X, Node, RestW, - trie_match(X, Node, Words, ResAcc)). - -follow_down_create(X, Words) -> - case follow_down_last_node(X, Words) of - {ok, FinalNode} -> FinalNode; - {error, Node, RestW} -> lists:foldl( - fun (W, CurNode) -> - NewNode = new_node_id(), - trie_add_edge(X, CurNode, NewNode, W), - NewNode - end, Node, RestW) - end. - -follow_down_last_node(X, Words) -> - follow_down(X, fun (_, Node, _) -> Node end, root, Words). - -follow_down_get_path(X, Words) -> - follow_down(X, fun (W, Node, PathAcc) -> [{Node, W} | PathAcc] end, - [{root, none}], Words). - -follow_down(X, AccFun, Acc0, Words) -> - follow_down(X, root, AccFun, Acc0, Words). - -follow_down(_X, _CurNode, _AccFun, Acc, []) -> - {ok, Acc}; -follow_down(X, CurNode, AccFun, Acc, Words = [W | RestW]) -> - case trie_child(X, CurNode, W) of - {ok, NextNode} -> follow_down(X, NextNode, AccFun, - AccFun(W, NextNode, Acc), RestW); - error -> {error, Acc, Words} - end. - -remove_path_if_empty(_, [{root, none}]) -> - ok; -remove_path_if_empty(X, [{Node, W} | [{Parent, _} | _] = RestPath]) -> - case mnesia:read(rabbit_topic_trie_node, - #trie_node{exchange_name = X, node_id = Node}, write) of - [] -> trie_remove_edge(X, Parent, Node, W), - remove_path_if_empty(X, RestPath); - _ -> ok - end. - -trie_child(X, Node, Word) -> - case mnesia:read({rabbit_topic_trie_edge, - #trie_edge{exchange_name = X, - node_id = Node, - word = Word}}) of - [#topic_trie_edge{node_id = NextNode}] -> {ok, NextNode}; - [] -> error - end. - -trie_bindings(X, Node) -> - MatchHead = #topic_trie_binding{ - trie_binding = #trie_binding{exchange_name = X, - node_id = Node, - destination = '$1', - arguments = '_'}}, - mnesia:select(rabbit_topic_trie_binding, [{MatchHead, [], ['$1']}]). - -trie_update_node_counts(X, Node, Field, Delta) -> - E = case mnesia:read(rabbit_topic_trie_node, - #trie_node{exchange_name = X, - node_id = Node}, write) of - [] -> #topic_trie_node{trie_node = #trie_node{ - exchange_name = X, - node_id = Node}, - edge_count = 0, - binding_count = 0}; - [E0] -> E0 - end, - case setelement(Field, E, element(Field, E) + Delta) of - #topic_trie_node{edge_count = 0, binding_count = 0} -> - ok = mnesia:delete_object(rabbit_topic_trie_node, E, write); - EN -> - ok = mnesia:write(rabbit_topic_trie_node, EN, write) - end. - -trie_add_edge(X, FromNode, ToNode, W) -> - trie_update_node_counts(X, FromNode, #topic_trie_node.edge_count, +1), - trie_edge_op(X, FromNode, ToNode, W, fun mnesia:write/3). - -trie_remove_edge(X, FromNode, ToNode, W) -> - trie_update_node_counts(X, FromNode, #topic_trie_node.edge_count, -1), - trie_edge_op(X, FromNode, ToNode, W, fun mnesia:delete_object/3). - -trie_edge_op(X, FromNode, ToNode, W, Op) -> - ok = Op(rabbit_topic_trie_edge, - #topic_trie_edge{trie_edge = #trie_edge{exchange_name = X, - node_id = FromNode, - word = W}, - node_id = ToNode}, - write). - -trie_add_binding(X, Node, D, Args) -> - trie_update_node_counts(X, Node, #topic_trie_node.binding_count, +1), - trie_binding_op(X, Node, D, Args, fun mnesia:write/3). - -trie_remove_binding(X, Node, D, Args) -> - trie_update_node_counts(X, Node, #topic_trie_node.binding_count, -1), - trie_binding_op(X, Node, D, Args, fun mnesia:delete_object/3). - -trie_binding_op(X, Node, D, Args, Op) -> - ok = Op(rabbit_topic_trie_binding, - #topic_trie_binding{ - trie_binding = #trie_binding{exchange_name = X, - node_id = Node, - destination = D, - arguments = Args}}, - write). - -trie_remove_all_nodes(X) -> - remove_all(rabbit_topic_trie_node, - #topic_trie_node{trie_node = #trie_node{exchange_name = X, - _ = '_'}, - _ = '_'}). - -trie_remove_all_edges(X) -> - remove_all(rabbit_topic_trie_edge, - #topic_trie_edge{trie_edge = #trie_edge{exchange_name = X, - _ = '_'}, - _ = '_'}). - -trie_remove_all_bindings(X) -> - remove_all(rabbit_topic_trie_binding, - #topic_trie_binding{ - trie_binding = #trie_binding{exchange_name = X, _ = '_'}, - _ = '_'}). - -remove_all(Table, Pattern) -> - lists:foreach(fun (R) -> mnesia:delete_object(Table, R, write) end, - mnesia:match_object(Table, Pattern, write)). - -new_node_id() -> - rabbit_guid:gen(). - -split_topic_key(Key) -> - split_topic_key(Key, [], []). - -split_topic_key(<<>>, [], []) -> - []; -split_topic_key(<<>>, RevWordAcc, RevResAcc) -> - lists:reverse([lists:reverse(RevWordAcc) | RevResAcc]); -split_topic_key(<<$., Rest/binary>>, RevWordAcc, RevResAcc) -> - split_topic_key(Rest, [], [lists:reverse(RevWordAcc) | RevResAcc]); -split_topic_key(<>, RevWordAcc, RevResAcc) -> - split_topic_key(Rest, [C | RevWordAcc], RevResAcc). diff --git a/deps/rabbit/test/exchanges_SUITE.erl b/deps/rabbit/test/exchanges_SUITE.erl new file mode 100644 index 000000000000..7f7459cecbfc --- /dev/null +++ b/deps/rabbit/test/exchanges_SUITE.erl @@ -0,0 +1,299 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2018-2022 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(exchanges_SUITE). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). + +-compile([nowarn_export_all, export_all]). +-compile(export_all). + +suite() -> + [{timetrap, 5 * 60000}]. + +all() -> + [ + {group, mnesia_store} + ]. + +groups() -> + [ + {mnesia_store, [], all_tests()} + ]. + +all_tests() -> + [ + direct_exchange, + headers_exchange, + topic_exchange, + fanout_exchange, + invalid_exchange + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config, []). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(mnesia_store = Group, Config) -> + init_per_group_common(Group, Config, 1). + +init_per_group_common(Group, Config, Size) -> + Config1 = rabbit_ct_helpers:set_config(Config, + [{rmq_nodes_count, Size}, + {rmq_nodename_suffix, Group}, + {tcp_ports_base}]), + rabbit_ct_helpers:run_steps(Config1, rabbit_ct_broker_helpers:setup_steps()). + +end_per_group(_, Config) -> + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_testcase(Testcase, Config) -> + Config1 = rabbit_ct_helpers:testcase_started(Config, Testcase), + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_queues, []), + Name = rabbit_data_coercion:to_binary(Testcase), + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_exchange, [Name]), + Config2 = rabbit_ct_helpers:set_config(Config1, + [{queue_name, Name}, + {alt_queue_name, <>}, + {exchange_name, Name} + ]), + rabbit_ct_helpers:run_steps(Config2, rabbit_ct_client_helpers:setup_steps()). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_queues, []), + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_exchange, + [?config(exchange_name, Config)]), + Config1 = rabbit_ct_helpers:run_steps( + Config, + rabbit_ct_client_helpers:teardown_steps()), + rabbit_ct_helpers:testcase_finished(Config1, Testcase). + +%% ------------------------------------------------------------------- +%% Testcases. +%% ------------------------------------------------------------------- +direct_exchange(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + AltQ = ?config(alt_queue_name, Config), + ?assertEqual({'queue.declare_ok', AltQ, 0, 0}, declare(Ch, AltQ, [])), + + Direct = <<"amq.direct">>, + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = Direct, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = Direct, + queue = AltQ, + routing_key = AltQ}), + publish(Ch, Direct, Q, <<"msg1">>), + publish(Ch, Direct, <<"anyotherkey">>, <<"msg2">>), + + quorum_queue_utils:wait_for_messages(Config, [[Q, <<"1">>, <<"1">>, <<"0">>], + [AltQ, <<"0">>, <<"0">>, <<"0">>]]), + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg1">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + ?assertMatch(#'basic.get_empty'{}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + ?assertMatch(#'basic.get_empty'{}, + amqp_channel:call(Ch, #'basic.get'{queue = AltQ})), + ok. + +topic_exchange(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + + Topic = <<"amq.topic">>, + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = Topic, + queue = Q, + routing_key = <<"this.*.rules">>}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = Topic, + queue = Q, + routing_key = <<"*.for.*">>}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = Topic, + queue = Q, + routing_key = <<"simply#carrots">>}), + + publish(Ch, Topic, <<"this.queue.rules">>, <<"msg1">>), + publish(Ch, Topic, <<"this.exchange.rules">>, <<"msg2">>), + publish(Ch, Topic, <<"another.queue.rules">>, <<"msg3">>), + publish(Ch, Topic, <<"carrots.for.power">>, <<"msg4">>), + publish(Ch, Topic, <<"simplycarrots">>, <<"msg5">>), + publish(Ch, Topic, <<"*.queue.rules">>, <<"msg6">>), + + quorum_queue_utils:wait_for_messages(Config, [[Q, <<"3">>, <<"3">>, <<"0">>]]), + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg1">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg2">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg4">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + ?assertMatch(#'basic.get_empty'{}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = Topic, + queue = Q, + routing_key = <<"#.noclue">>}), + publish(Ch, Topic, <<"simplycarrots">>, <<"msg7">>), + quorum_queue_utils:wait_for_messages(Config, [[Q, <<"3">>, <<"0">>, <<"3">>]]), + publish(Ch, Topic, <<"#.bla">>, <<"msg8">>), + quorum_queue_utils:wait_for_messages(Config, [[Q, <<"3">>, <<"0">>, <<"3">>]]), + publish(Ch, Topic, <<"#.noclue">>, <<"msg9">>), + quorum_queue_utils:wait_for_messages(Config, [[Q, <<"4">>, <<"1">>, <<"3">>]]), + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg9">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + ?assertMatch(#'basic.get_empty'{}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = Topic, + queue = Q, + routing_key = <<"#">>}), + publish(Ch, Topic, <<"simplycarrots">>, <<"msg10">>), + quorum_queue_utils:wait_for_messages(Config, [[Q, <<"5">>, <<"1">>, <<"4">>]]), + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg10">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + ?assertMatch(#'basic.get_empty'{}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + ok. + +fanout_exchange(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + AltQ = ?config(alt_queue_name, Config), + ?assertEqual({'queue.declare_ok', AltQ, 0, 0}, declare(Ch, AltQ, [])), + + Fanout = <<"amq.fanout">>, + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = Fanout, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = Fanout, + queue = AltQ, + routing_key = AltQ}), + publish(Ch, Fanout, Q, <<"msg1">>), + publish(Ch, Fanout, <<"anyotherkey">>, <<"msg2">>), + + quorum_queue_utils:wait_for_messages(Config, [[Q, <<"2">>, <<"2">>, <<"0">>], + [AltQ, <<"2">>, <<"2">>, <<"0">>]]), + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg1">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg2">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + ?assertMatch(#'basic.get_empty'{}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg1">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = AltQ})), + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg2">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = AltQ})), + ?assertMatch(#'basic.get_empty'{}, + amqp_channel:call(Ch, #'basic.get'{queue = AltQ})), + ok. + +headers_exchange(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + AltQ = ?config(alt_queue_name, Config), + ?assertEqual({'queue.declare_ok', AltQ, 0, 0}, declare(Ch, AltQ, [])), + + Headers = <<"amq.headers">>, + #'queue.bind_ok'{} = + amqp_channel:call(Ch, + #'queue.bind'{exchange = Headers, + queue = Q, + arguments = [{<<"x-match">>, longstr, <<"all">>}, + {<<"foo">>, longstr, <<"bar">>}, + {<<"fuu">>, longstr, <<"ber">>}] + }), + #'queue.bind_ok'{} = + amqp_channel:call(Ch, + #'queue.bind'{exchange = Headers, + queue = AltQ, + arguments = [{<<"x-match">>, longstr, <<"any">>}, + {<<"foo">>, longstr, <<"bar">>}, + {<<"fuu">>, longstr, <<"ber">>}] + }), + + publish(Ch, Headers, <<>>, <<"msg1">>, [{<<"foo">>, longstr, <<"bar">>}, + {<<"fuu">>, longstr, <<"ber">>}]), + publish(Ch, Headers, <<>>, <<"msg2">>, [{<<"foo">>, longstr, <<"bar">>}]), + publish(Ch, Headers, <<>>, <<"msg3">>), + + quorum_queue_utils:wait_for_messages(Config, [[Q, <<"1">>, <<"1">>, <<"0">>], + [AltQ, <<"2">>, <<"2">>, <<"0">>]]), + + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg1">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + ?assertMatch(#'basic.get_empty'{}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg1">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = AltQ})), + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg2">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = AltQ})), + ?assertMatch(#'basic.get_empty'{}, + amqp_channel:call(Ch, #'basic.get'{queue = AltQ})), + ok. + +invalid_exchange(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + + ?assertExit( + {{shutdown, {server_initiated_close, 404, _}}, _}, + amqp_channel:call(Ch, #'queue.bind'{exchange = <<"invalid">>, + queue = Q, + routing_key = Q})). + +%% Internal + +delete_queues() -> + [{ok, _} = rabbit_amqqueue:delete(Q, false, false, <<"dummy">>) + || Q <- rabbit_amqqueue:list()]. + +delete_exchange(Name) -> + _ = rabbit_exchange:delete(rabbit_misc:r(<<"/">>, exchange, Name), false, <<"dummy">>). + +declare(Ch, Q, Args) -> + declare(Ch, Q, Args, true). + +declare(Ch, Q, Args, Durable) -> + amqp_channel:call(Ch, #'queue.declare'{queue = Q, + durable = Durable, + auto_delete = false, + arguments = Args}). + +publish(Ch, X, RoutingKey, Msg) -> + publish(Ch, X, RoutingKey, Msg, []). + +publish(Ch, X, RoutingKey, Msg, Headers) -> + ok = amqp_channel:cast(Ch, #'basic.publish'{exchange = X, + routing_key = RoutingKey}, + #amqp_msg{props = #'P_basic'{delivery_mode = 2, + headers = Headers}, + payload = Msg}). From d0ac99df5ecb927716172c8f67aca44a30bb9cab Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Thu, 12 Jan 2023 20:35:54 +0100 Subject: [PATCH 02/12] Move queue/exchange/binding/policy Mnesia-specific code to rabbit_db_* modules --- deps/rabbit/BUILD.bazel | 4 + deps/rabbit/src/rabbit_amqqueue.erl | 406 +++------ deps/rabbit/src/rabbit_amqqueue_process.erl | 24 +- deps/rabbit/src/rabbit_binding.erl | 587 +++---------- deps/rabbit/src/rabbit_classic_queue.erl | 2 +- deps/rabbit/src/rabbit_core_ff.erl | 12 +- deps/rabbit/src/rabbit_db.erl | 10 + deps/rabbit/src/rabbit_db_binding.erl | 699 +++++++++++++++ deps/rabbit/src/rabbit_db_exchange.erl | 516 +++++++++++ deps/rabbit/src/rabbit_db_policy.erl | 63 ++ deps/rabbit/src/rabbit_db_queue.erl | 540 ++++++++++++ deps/rabbit/src/rabbit_db_topic_exchange.erl | 2 +- deps/rabbit/src/rabbit_exchange.erl | 281 ++---- deps/rabbit/src/rabbit_exchange_decorator.erl | 18 +- .../src/rabbit_exchange_type_direct.erl | 4 +- .../src/rabbit_exchange_type_fanout.erl | 4 +- .../src/rabbit_exchange_type_headers.erl | 4 +- .../src/rabbit_exchange_type_invalid.erl | 4 +- .../rabbit/src/rabbit_exchange_type_topic.erl | 20 +- .../rabbit_observer_cli_classic_queues.erl | 9 +- deps/rabbit/src/rabbit_policy.erl | 152 ++-- deps/rabbit/src/rabbit_priority_queue.erl | 8 +- deps/rabbit/src/rabbit_queue_decorator.erl | 4 + deps/rabbit/src/rabbit_queue_location.erl | 7 +- deps/rabbit/src/rabbit_quorum_queue.erl | 26 +- deps/rabbit/src/rabbit_stream_coordinator.erl | 7 +- deps/rabbit/src/rabbit_stream_queue.erl | 10 +- deps/rabbit/test/bindings_SUITE.erl | 808 ++++++++++++++++++ .../src/rabbit_exchange_type.erl | 7 +- 29 files changed, 3071 insertions(+), 1167 deletions(-) create mode 100644 deps/rabbit/src/rabbit_db_binding.erl create mode 100644 deps/rabbit/src/rabbit_db_exchange.erl create mode 100644 deps/rabbit/src/rabbit_db_policy.erl create mode 100644 deps/rabbit/src/rabbit_db_queue.erl create mode 100644 deps/rabbit/test/bindings_SUITE.erl diff --git a/deps/rabbit/BUILD.bazel b/deps/rabbit/BUILD.bazel index 4689ca4cc9eb..da8aff391708 100644 --- a/deps/rabbit/BUILD.bazel +++ b/deps/rabbit/BUILD.bazel @@ -1098,6 +1098,10 @@ rabbitmq_integration_suite( size = "small", ) +rabbitmq_integration_suite( + name = "bindings_SUITE", + size = "small", +) assert_suites() diff --git a/deps/rabbit/src/rabbit_amqqueue.erl b/deps/rabbit/src/rabbit_amqqueue.erl index 9c14c2a29c72..98a27915fb6f 100644 --- a/deps/rabbit/src/rabbit_amqqueue.erl +++ b/deps/rabbit/src/rabbit_amqqueue.erl @@ -7,14 +7,13 @@ -module(rabbit_amqqueue). --export([store_queue_ram_dirty/1]). -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]). -export([pseudo_queue/2, pseudo_queue/3, immutable/1]). -export([exists/1, lookup/1, lookup/2, lookup_many/1, - not_found_or_absent/1, not_found_or_absent_dirty/1, + not_found_or_absent_dirty/1, with/2, with/3, with_or_die/2, assert_equivalence/5, augment_declare_args/5, @@ -22,12 +21,12 @@ stat/1, deliver/2, 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, +-export([list/0, list_durable/0, list/1, info_keys/0, info/1, info/2, info_all/1, info_all/2, emit_info_all/5, list_local/1, info_local/1, emit_info_local/4, emit_info_down/4]). -export([count/0]). -export([list_down/1, count/1, list_names/0, list_names/1, list_local_names/0, - list_local_names_down/0, list_with_possible_retry/1]). + list_local_names_down/0]). -export([list_by_type/1, sample_local_queues/0, sample_n_by_name/2, sample_n/2]). -export([force_event_refresh/1, notify_policy_changed/1]). -export([consumers/1, consumers_all/1, emit_consumers_all/4, consumer_info_keys/0]). @@ -164,31 +163,23 @@ start(Qs) -> ok. mark_local_durable_queues_stopped(VHost) -> - Qs = find_local_durable_queues(VHost), - rabbit_misc:execute_mnesia_transaction( - fun() -> - [ store_queue(amqqueue:set_state(Q, stopped)) - || Q <- Qs, amqqueue:get_type(Q) =:= rabbit_classic_queue, - amqqueue:get_state(Q) =/= stopped ] - end). + Qs0 = find_local_durable_queues(VHost), + Qs = [amqqueue:set_state(Q, stopped) + || Q <- Qs0, amqqueue:get_type(Q) =:= rabbit_classic_queue, + amqqueue:get_state(Q) =/= stopped ], + rabbit_db_queue:insert(Qs). 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 andalso - rabbit_queue_type:is_recoverable(Q) - ])) - end). + Qs = rabbit_db_queue:get_all_durable(VHost), + lists:filter(fun(Q) -> + rabbit_queue_type:is_recoverable(Q) + end, Qs). find_recoverable_queues() -> - mnesia:async_dirty( - fun () -> - qlc:e(qlc:q([Q || Q <- mnesia:table(rabbit_durable_queue), - rabbit_queue_type:is_recoverable(Q)])) - end). + Qs = rabbit_db_queue:get_all_durable(), + lists:filter(fun(Q) -> + rabbit_queue_type:is_recoverable(Q) + end, Qs). -spec declare(name(), boolean(), @@ -255,84 +246,39 @@ get_queue_type(Args) -> internal_declare(Q, Recover) -> do_internal_declare(Q, Recover). -do_internal_declare(Q, true) -> - rabbit_misc:execute_mnesia_tx_with_tail( - fun () -> - ok = store_queue(amqqueue:set_state(Q, live)), - rabbit_misc:const({created, Q}) - end); -do_internal_declare(Q, false) -> - QueueName = amqqueue:get_name(Q), - rabbit_misc:execute_mnesia_tx_with_tail( - fun () -> - case mnesia:wread({rabbit_queue, QueueName}) of - [] -> - case not_found_or_absent(QueueName) of - not_found -> Q1 = rabbit_policy:set(Q), - Q2 = amqqueue:set_state(Q1, live), - ok = store_queue(Q2), - fun () -> {created, Q2} end; - {absent, _Q, _} = R -> rabbit_misc:const(R) - end; - [ExistingQ] -> - rabbit_misc:const({existing, ExistingQ}) - end - end). +do_internal_declare(Q0, true) -> + Q = amqqueue:set_state(Q0, live), + store_queue(Q), + {created, Q0}; +do_internal_declare(Q0, false) -> + Q = rabbit_policy:set(amqqueue:set_state(Q0, live)), + Queue = rabbit_queue_decorator:set(Q), + DurableQueue = amqqueue:reset_mirroring_and_decorators(Q), + rabbit_db_queue:create_or_get(DurableQueue, Queue). -spec update (name(), fun((amqqueue:amqqueue()) -> amqqueue:amqqueue())) -> 'not_found' | amqqueue:amqqueue(). update(Name, Fun) -> - case mnesia:wread({rabbit_queue, Name}) of - [Q] -> - Durable = amqqueue:is_durable(Q), - Q1 = Fun(Q), - ok = mnesia:write(rabbit_queue, Q1, write), - case Durable of - true -> ok = mnesia:write(rabbit_durable_queue, Q1, write); - _ -> ok - end, - Q1; - [] -> - not_found - end. + rabbit_db_queue:update(Name, Fun). %% only really used for quorum queues to ensure the rabbit_queue record %% is initialised ensure_rabbit_queue_record_is_initialized(Q) -> - rabbit_misc:execute_mnesia_tx_with_tail( - fun () -> - ok = store_queue(Q), - rabbit_misc:const({ok, Q}) - end). + store_queue(Q). -spec store_queue(amqqueue:amqqueue()) -> 'ok'. -store_queue(Q) when ?amqqueue_is_durable(Q) -> - Q1 = amqqueue:reset_mirroring_and_decorators(Q), - ok = mnesia:write(rabbit_durable_queue, Q1, write), - store_queue_ram(Q); -store_queue(Q) when not ?amqqueue_is_durable(Q) -> - store_queue_ram(Q). - -store_queue_ram(Q) -> - ok = mnesia:write(rabbit_queue, rabbit_queue_decorator:set(Q), write). - -store_queue_ram_dirty(Q) -> - ok = mnesia:dirty_write(rabbit_queue, rabbit_queue_decorator:set(Q)). +store_queue(Q0) -> + Q = rabbit_queue_decorator:set(Q0), + DurableQ = amqqueue:reset_mirroring_and_decorators(Q0), + rabbit_db_queue:insert(DurableQ, Q). -spec update_decorators(name()) -> 'ok'. update_decorators(Name) -> - rabbit_misc:execute_mnesia_transaction( - fun() -> - case mnesia:wread({rabbit_queue, Name}) of - [Q] -> store_queue_ram(Q), - ok; - [] -> ok - end - end). + rabbit_db_queue:update_decorators(Name). -spec policy_changed(amqqueue:amqqueue(), amqqueue:amqqueue()) -> 'ok'. @@ -371,13 +317,8 @@ is_server_named_allowed(Args) -> [amqqueue:amqqueue()]. lookup([]) -> []; %% optimisation -lookup([Name]) -> ets:lookup(rabbit_queue, Name); %% optimisation -lookup(Names) when is_list(Names) -> - %% Normally we'd call mnesia:dirty_read/1 here, but that is quite - %% expensive for reasons explained in rabbit_misc:dirty_read/1. - lists:append([ets:lookup(rabbit_queue, Name) || Name <- Names]); -lookup(Name) -> - rabbit_misc:dirty_read({rabbit_queue, Name}). +lookup(Names) -> + rabbit_db_queue:get(Names). -spec lookup_many ([name()]) -> [amqqueue:amqqueue()]. @@ -395,28 +336,12 @@ lookup(Name, VHost) -spec exists(name()) -> boolean(). exists(Name) -> - ets:member(rabbit_queue, Name). - --spec not_found_or_absent(name()) -> not_found_or_absent(). - -not_found_or_absent(Name) -> - %% NB: we assume that the caller has already performed a lookup on - %% rabbit_queue and not found anything - case mnesia:read({rabbit_durable_queue, Name}) of - [] -> not_found; - [Q] -> {absent, Q, nodedown} %% Q exists on stopped node - end. + rabbit_db_queue:exists(Name). -spec not_found_or_absent_dirty(name()) -> not_found_or_absent(). not_found_or_absent_dirty(Name) -> - %% We should read from both tables inside a tx, to get a - %% consistent view. But the chances of an inconsistency are small, - %% and only affect the error kind. - case rabbit_misc:dirty_read({rabbit_durable_queue, Name}) of - {error, not_found} -> not_found; - {ok, Q} -> {absent, Q, nodedown} - end. + rabbit_db_queue:not_found_or_absent_queue_dirty(Name). -spec get_rebalance_lock(pid()) -> {true, {rebalance_queues, pid()}} | false. @@ -617,7 +542,7 @@ with(#resource{} = Name, F, E, RetriesLeft) -> fun () -> retry_wait(Q, F, E, RetriesLeft) end, fun () -> F(Q) end); {error, not_found} -> - E(not_found_or_absent_dirty(Name)) + E(rabbit_db_queue:not_found_or_absent_queue_dirty(Name)) end. -spec retry_wait(amqqueue:amqqueue(), @@ -1117,10 +1042,7 @@ check_queue_type(_Val, _Args) -> -spec list() -> [amqqueue:amqqueue()]. list() -> - list_with_possible_retry(fun do_list/0). - -do_list() -> - All = mnesia:dirty_match_object(rabbit_queue, amqqueue:pattern_match_all()), + All = rabbit_db_queue:get_all(), NodesRunning = rabbit_nodes:all_running(), lists:filter(fun (Q) -> Pid = amqqueue:get_pid(Q), @@ -1131,11 +1053,12 @@ do_list() -> -spec count() -> non_neg_integer(). count() -> - mnesia:table_info(rabbit_queue, size). + rabbit_db_queue:count(). -spec list_names() -> [rabbit_amqqueue:name()]. -list_names() -> mnesia:dirty_all_keys(rabbit_queue). +list_names() -> + rabbit_db_queue:list(). list_names(VHost) -> [amqqueue:get_name(Q) || Q <- list(VHost)]. @@ -1186,6 +1109,8 @@ sample_n(Queues, N) when is_list(Queues) andalso is_integer(N) andalso N > 0 -> Names = [amqqueue:get_name(Q) || Q <- Queues], sample_n_by_name(Names, N). +list_durable() -> + rabbit_db_queue:get_all_durable(). -spec list_by_type(atom()) -> [amqqueue:amqqueue()]. @@ -1193,14 +1118,7 @@ list_by_type(classic) -> list_by_type(rabbit_classic_queue); list_by_type(quorum) -> list_by_type(rabbit_quorum_queue); list_by_type(stream) -> list_by_type(rabbit_stream_queue); list_by_type(Type) -> - {atomic, Qs} = - mnesia:sync_transaction( - fun () -> - mnesia:match_object(rabbit_durable_queue, - amqqueue:pattern_match_on_type(Type), - read) - end), - Qs. + rabbit_db_queue:get_all_durable_by_type(Type). -spec list_local_quorum_queue_names() -> [rabbit_amqqueue:name()]. @@ -1306,7 +1224,7 @@ is_in_virtual_host(Q, VHostName) -> -spec list(vhost:name()) -> [amqqueue:amqqueue()]. list(VHostPath) -> - All = list(VHostPath, rabbit_queue), + All = rabbit_db_queue:get_all(VHostPath), NodesRunning = rabbit_nodes:all_running(), lists:filter(fun (Q) -> Pid = amqqueue:get_pid(Q), @@ -1314,52 +1232,6 @@ list(VHostPath) -> St =/= stopped orelse lists:member(node(Pid), NodesRunning) end, All). -list(VHostPath, TableName) -> - list_with_possible_retry(fun() -> do_list(VHostPath, TableName) end). - -%% Not dirty_match_object since that would not be transactional when used in a -%% tx context -do_list(VHostPath, TableName) -> - mnesia:async_dirty( - fun () -> - mnesia:match_object( - TableName, - amqqueue:pattern_match_on_name(rabbit_misc:r(VHostPath, queue)), - read) - end). - -list_with_possible_retry(Fun) -> - %% amqqueue migration: - %% The `rabbit_queue` or `rabbit_durable_queue` tables - %% might be migrated between the time we query the pattern - %% (with the `amqqueue` module) and the time we call - %% `mnesia:dirty_match_object()`. This would lead to an empty list - %% (no object matching the now incorrect pattern), not a Mnesia - %% error. - %% - %% So if the result is an empty list and the version of the - %% `amqqueue` record changed in between, we retry the operation. - %% - %% However, we don't do this if inside a Mnesia transaction: we - %% could end up with a live lock between this started transaction - %% and the Mnesia table migration which is blocked (but the - %% rabbit_feature_flags lock is held). - AmqqueueRecordVersion = amqqueue:record_version_to_use(), - case Fun() of - [] -> - case mnesia:is_transaction() of - true -> - []; - false -> - case amqqueue:record_version_to_use() of - AmqqueueRecordVersion -> []; - _ -> Fun() - end - end; - Ret -> - Ret - end. - -spec list_down(rabbit_types:vhost()) -> [amqqueue:amqqueue()]. list_down(VHostPath) -> @@ -1367,7 +1239,7 @@ list_down(VHostPath) -> false -> []; true -> Alive = sets:from_list([amqqueue:get_name(Q) || Q <- list(VHostPath)]), - Durable = list(VHostPath, rabbit_durable_queue), + Durable = rabbit_db_queue:get_all_durable(VHostPath), NodesRunning = rabbit_nodes:all_running(), lists:filter(fun (Q) -> N = amqqueue:get_name(Q), @@ -1380,26 +1252,7 @@ list_down(VHostPath) -> end. count(VHost) -> - try - %% this is certainly suboptimal but there is no way to count - %% things using a secondary index in Mnesia. Our counter-table-per-node - %% won't work here because with leader migration of mirrored queues - %% the "ownership" of queues by nodes becomes a non-trivial problem - %% that requires a proper consensus algorithm. - length(list_for_count(VHost)) - catch _:Err -> - rabbit_log:error("Failed to fetch number of queues in vhost ~tp:~n~tp", - [VHost, Err]), - 0 - end. - -list_for_count(VHost) -> - list_with_possible_retry( - fun() -> - mnesia:dirty_index_read(rabbit_queue, - VHost, - amqqueue:field_vhost()) - end). + rabbit_db_queue:count(VHost). -spec info_keys() -> rabbit_types:info_keys(). @@ -1808,66 +1661,34 @@ notify_sent_queue_down(QPid) -> resume(QPid, ChPid) -> delegate:invoke_no_result(QPid, {gen_server2, cast, [{resume, ChPid}]}). -internal_delete1(QueueName, OnlyDurable) -> - internal_delete1(QueueName, OnlyDurable, normal). - -internal_delete1(QueueName, OnlyDurable, Reason) -> - ok = mnesia:delete({rabbit_queue, QueueName}), - case Reason of - auto_delete -> - case mnesia:wread({rabbit_durable_queue, QueueName}) of - [] -> ok; - [_] -> ok = mnesia:delete({rabbit_durable_queue, QueueName}) - end; - _ -> - mnesia:delete({rabbit_durable_queue, QueueName}) - end, - %% we want to execute some things, as decided by rabbit_exchange, - %% after the transaction. - rabbit_binding:remove_for_destination(QueueName, OnlyDurable). - -spec internal_delete(name(), rabbit_types:username()) -> 'ok'. internal_delete(QueueName, ActingUser) -> internal_delete(QueueName, ActingUser, normal). internal_delete(QueueName, ActingUser, Reason) -> - rabbit_misc:execute_mnesia_tx_with_tail( - fun () -> - case {mnesia:wread({rabbit_queue, QueueName}), - mnesia:wread({rabbit_durable_queue, QueueName})} of - {[], []} -> - rabbit_misc:const(ok); - _ -> - Deletions = internal_delete1(QueueName, false, Reason), - T = rabbit_binding:process_deletions(Deletions, - ?INTERNAL_USER), - fun() -> - ok = T(), - rabbit_core_metrics:queue_deleted(QueueName), - ok = rabbit_event:notify(queue_deleted, - [{name, QueueName}, - {user_who_performed_action, ActingUser}]) - end - end - end). + case rabbit_db_queue:delete(QueueName, Reason) of + ok -> + ok; + Deletions -> + rabbit_binding:process_deletions(Deletions), + rabbit_binding:notify_deletions(Deletions, ?INTERNAL_USER), + rabbit_core_metrics:queue_deleted(QueueName), + ok = rabbit_event:notify(queue_deleted, + [{name, QueueName}, + {user_who_performed_action, ActingUser}]) + end. -spec forget_all_durable(node()) -> 'ok'. forget_all_durable(Node) -> - %% Note rabbit is not running so we avoid e.g. the worker pool. Also why - %% we don't invoke the return from rabbit_binding:process_deletions/1. - {atomic, ok} = - mnesia:sync_transaction( - fun () -> - Qs = mnesia:match_object(rabbit_durable_queue, - amqqueue:pattern_match_all(), write), - _ = [forget_node_for_queue(Node, Q) || - Q <- Qs, - is_local_to_node(amqqueue:get_pid(Q), Node)], - ok - end), - ok. + UpdateFun = fun(Q) -> + forget_node_for_queue(Node, Q) + end, + FilterFun = fun(Q) -> + is_local_to_node(amqqueue:get_pid(Q), Node) + end, + rabbit_db_queue:match_and_update(amqqueue:pattern_match_all(), UpdateFun, FilterFun). %% Try to promote a mirror while down - it should recover as a %% leader. We try to take the oldest mirror here for best chance of @@ -1884,7 +1705,7 @@ forget_node_for_queue(_DeadNode, [], Q) -> %% Don't process_deletions since that just calls callbacks and we %% are not really up. Name = amqqueue:get_name(Q), - internal_delete1(Name, true); + rabbit_db_queue:internal_delete(Name, true, normal); %% Should not happen, but let's be conservative. forget_node_for_queue(DeadNode, [DeadNode | T], Q) -> @@ -1896,7 +1717,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 = rabbit_db_queue:insert([Q1]); {true, rabbit_quorum_queue} -> ok end. @@ -1988,13 +1809,7 @@ has_synchronised_mirrors_online(Q) -> -spec on_node_up(node()) -> 'ok'. on_node_up(Node) -> - ok = rabbit_misc:execute_mnesia_transaction( - fun () -> - Qs = mnesia:match_object(rabbit_queue, - amqqueue:pattern_match_all(), write), - [maybe_clear_recoverable_node(Node, Q) || Q <- Qs], - ok - end). + rabbit_db_queue:on_node_up(Node, fun maybe_clear_recoverable_node/2). maybe_clear_recoverable_node(Node, Q) -> SPids = amqqueue:get_sync_slave_pids(Q), @@ -2030,65 +1845,36 @@ maybe_clear_recoverable_node(Node, Q) -> -spec on_node_down(node()) -> 'ok'. on_node_down(Node) -> - {Time, {QueueNames, QueueDeletions}} = timer:tc(fun() -> delete_queues_on_node_down(Node) end), - case length(QueueNames) of - 0 -> ok; - _ -> rabbit_log:info("~tp transient queues from an old incarnation of node ~tp deleted in ~fs", [length(QueueNames), Node, Time/1000000]) - end, - notify_queue_binding_deletions(QueueDeletions), - rabbit_core_metrics:queues_deleted(QueueNames), - notify_queues_deleted(QueueNames), - ok. - -delete_queues_on_node_down(Node) -> - lists:unzip(lists:flatten([ - rabbit_misc:execute_mnesia_transaction( - fun () -> [{Queue, delete_queue(Queue)} || Queue <- Queues] end - ) || Queues <- partition_queues(queues_to_delete_when_node_down(Node)) - ])). - -delete_queue(QueueName) -> - ok = mnesia:delete({rabbit_queue, QueueName}), - rabbit_binding:remove_transient_for_destination(QueueName). - -% If there are many queues and we delete them all in a single Mnesia transaction, -% this can block all other Mnesia operations for a really long time. -% In situations where a node wants to (re-)join a cluster, -% Mnesia won't be able to sync on the new node until this operation finishes. -% As a result, we want to have multiple Mnesia transactions so that other -% operations can make progress in between these queue delete transactions. -% -% 10 queues per Mnesia transaction is an arbitrary number, but it seems to work OK with 50k queues per node. -partition_queues([Q0,Q1,Q2,Q3,Q4,Q5,Q6,Q7,Q8,Q9 | T]) -> - [[Q0,Q1,Q2,Q3,Q4,Q5,Q6,Q7,Q8,Q9] | partition_queues(T)]; -partition_queues(T) -> - [T]. - -queues_to_delete_when_node_down(NodeDown) -> - rabbit_misc:execute_mnesia_transaction(fun () -> - qlc:e(qlc:q([amqqueue:get_name(Q) || - Q <- mnesia:table(rabbit_queue), - amqqueue:qnode(Q) == NodeDown andalso - not rabbit_mnesia:is_process_alive(amqqueue:get_pid(Q)) andalso - (not amqqueue:is_classic(Q) orelse not amqqueue:is_durable(Q)) andalso - (not rabbit_amqqueue:is_replicated(Q) orelse - rabbit_amqqueue:is_dead_exclusive(Q))] - )) - end). + {Time, Ret} = timer:tc(fun() -> rabbit_db_queue:on_node_down(Node, fun filter_transient_queues_to_delete/2) end), + case Ret of + ok -> ok; + {QueueNames, Deletions} -> + case length(QueueNames) of + 0 -> ok; + _ -> rabbit_log:info("~tp transient queues from an old incarnation of node ~tp deleted in ~fs", [length(QueueNames), Node, Time/1000000]) + end, + notify_queue_binding_deletions(Deletions), + rabbit_core_metrics:queues_deleted(QueueNames), + notify_queues_deleted(QueueNames), + ok + end. +filter_transient_queues_to_delete(Node, Q) -> + amqqueue:qnode(Q) == Node andalso + not rabbit_mnesia:is_process_alive(amqqueue:get_pid(Q)) + andalso (not amqqueue:is_classic(Q) orelse not amqqueue:is_durable(Q)) + andalso (not rabbit_amqqueue:is_replicated(Q) + orelse rabbit_amqqueue:is_dead_exclusive(Q)). + +notify_queue_binding_deletions(QueueDeletions) when is_list(QueueDeletions) -> + Deletions = rabbit_binding:process_deletions( + lists:foldl(fun rabbit_binding:combine_deletions/2, + rabbit_binding:new_deletions(), + QueueDeletions)), + rabbit_binding:notify_deletions(Deletions, ?INTERNAL_USER); notify_queue_binding_deletions(QueueDeletions) -> - rabbit_misc:execute_mnesia_tx_with_tail( - fun() -> - rabbit_binding:process_deletions( - lists:foldl( - fun rabbit_binding:combine_deletions/2, - rabbit_binding:new_deletions(), - QueueDeletions - ), - ?INTERNAL_USER - ) - end - ). + Deletions = rabbit_binding:process_deletions(QueueDeletions), + rabbit_binding:notify_deletions(Deletions, ?INTERNAL_USER). notify_queues_deleted(QueueDeletions) -> lists:foreach( diff --git a/deps/rabbit/src/rabbit_amqqueue_process.erl b/deps/rabbit/src/rabbit_amqqueue_process.erl index c4835a5238a0..091a3001253e 100644 --- a/deps/rabbit/src/rabbit_amqqueue_process.erl +++ b/deps/rabbit/src/rabbit_amqqueue_process.erl @@ -280,19 +280,10 @@ init_with_backing_queue_state(Q, BQ, BQS, State3. terminate(shutdown = R, State = #q{backing_queue = BQ, q = Q0}) -> - QName = amqqueue:get_name(Q0), rabbit_core_metrics:queue_deleted(qname(State)), terminate_shutdown( fun (BQS) -> - rabbit_misc:execute_mnesia_transaction( - fun() -> - [Q] = mnesia:read({rabbit_queue, QName}), - Q2 = amqqueue:set_state(Q, stopped), - %% amqqueue migration: - %% The amqqueue was read from this transaction, no need - %% to handle migration. - rabbit_amqqueue:store_queue(Q2) - end), + update_state(stopped, Q0), BQ:terminate(R, BQS) end, State); terminate({shutdown, missing_owner} = Reason, State) -> @@ -315,10 +306,7 @@ terminate(normal, State) -> %% delete case terminate(_Reason, State = #q{q = Q}) -> terminate_shutdown(fun (BQS) -> Q2 = amqqueue:set_state(Q, crashed), - rabbit_misc:execute_mnesia_transaction( - fun() -> - rabbit_amqqueue:store_queue(Q2) - end), + rabbit_amqqueue:store_queue(Q2), BQS end, State). @@ -1887,4 +1875,10 @@ update_ha_mode(State) -> confirm_to_sender(Pid, QName, MsgSeqNos) -> rabbit_classic_queue:confirm_to_sender(Pid, QName, MsgSeqNos). - +update_state(State, Q) -> + Decorators = rabbit_queue_decorator:active(Q), + rabbit_db_queue:update(amqqueue:get_name(Q), + fun(Q0) -> + Q1 = amqqueue:set_state(Q0, State), + amqqueue:set_decorators(Q1, Decorators) + end). diff --git a/deps/rabbit/src/rabbit_binding.erl b/deps/rabbit/src/rabbit_binding.erl index b1b9ba963129..52bdce4ebe2f 100644 --- a/deps/rabbit/src/rabbit_binding.erl +++ b/deps/rabbit/src/rabbit_binding.erl @@ -9,19 +9,18 @@ -include_lib("rabbit_common/include/rabbit.hrl"). -include("amqqueue.hrl"). --export([recover/0, recover/2, exists/1, add/2, add/3, remove/1, remove/2, remove/3, remove/4]). +-export([recover/0, recover/2, exists/1, add/2, add/3, remove/3]). -export([list/1, list_for_source/1, list_for_destination/1, list_for_source_and_destination/2, list_for_source_and_destination/3, - list_explicit/0, list_between/2, has_any_between/2]). + list_explicit/0]). -export([new_deletions/0, combine_deletions/2, add_deletion/3, - process_deletions/2, binding_action/3]). + process_deletions/1, notify_deletions/2, group_bindings_fold/3]). -export([info_keys/0, info/1, info/2, info_all/1, info_all/2, info_all/4]). -%% these must all be run inside a mnesia tx --export([has_for_source/1, remove_for_source/1, - remove_for_destination/2, remove_transient_for_destination/1]). --export([populate_index_route_table/0]). +-export([reverse_binding/1]). -export([new/4]). +-export([reverse_route/1, index_route/1]). +-export([binding_type/2]). -define(DEFAULT_EXCHANGE(VHostPath), #resource{virtual_host = VHostPath, kind = exchange, @@ -78,16 +77,7 @@ new(Src, RoutingKey, Dst, Arguments) -> %% Global table recovery recover() -> - rabbit_misc:execute_mnesia_transaction( - fun () -> - _ = mnesia:lock({table, rabbit_durable_route}, read), - _ = mnesia:lock({table, rabbit_semi_durable_route}, write), - Routes = rabbit_misc:dirty_read_all(rabbit_durable_route), - Fun = fun(Route) -> - mnesia:dirty_write(rabbit_semi_durable_route, Route) - end, - lists:foreach(Fun, Routes) - end). + rabbit_db_binding:recover(). %% Virtual host-specific recovery @@ -100,40 +90,26 @@ recover(XNames, QNames) -> (#resource{kind = queue}) -> QNameSet end, {ok, Gatherer} = gatherer:start_link(), - [recover_semi_durable_route(Gatherer, R, SelectSet(Dst)) || - R = #route{binding = #binding{destination = Dst}} <- - rabbit_misc:dirty_read_all(rabbit_semi_durable_route)], + rabbit_db_binding:recover( + fun(Binding, Src, Dst, Fun) -> + recover_semi_durable_route(Gatherer, Binding, Src, Dst, SelectSet(Dst), Fun) + end), empty = gatherer:out(Gatherer), ok = gatherer:stop(Gatherer), ok. -recover_semi_durable_route(Gatherer, R = #route{binding = B}, ToRecover) -> - #binding{source = Src, destination = Dst} = B, +recover_semi_durable_route(Gatherer, Binding, Src, Dst, ToRecover, Fun) -> case sets:is_element(Dst, ToRecover) of true -> {ok, X} = rabbit_exchange:lookup(Src), ok = gatherer:fork(Gatherer), ok = worker_pool:submit_async( fun () -> - recover_semi_durable_route_txn(R, X), + Fun(Binding, X), gatherer:finish(Gatherer) end); false -> ok end. -recover_semi_durable_route_txn(R = #route{binding = B}, X) -> - rabbit_misc:execute_mnesia_transaction( - fun () -> - case mnesia:read(rabbit_semi_durable_route, B, read) of - [] -> no_recover; - _ -> ok = sync_transient_route(R, should_index_table(X), fun mnesia:write/3), - rabbit_exchange:serial(X) - end - end, - fun (no_recover, _) -> ok; - (_Serial, true) -> x_callback(transaction, X, add_binding, B); - (Serial, false) -> x_callback(Serial, X, add_binding, B) - end). - -spec exists(rabbit_types:binding()) -> boolean() | bind_errors(). exists(#binding{source = ?DEFAULT_EXCHANGE(_), @@ -141,11 +117,9 @@ exists(#binding{source = ?DEFAULT_EXCHANGE(_), key = QName, args = []}) -> rabbit_amqqueue:exists(Queue); -exists(Binding) -> - binding_action( - Binding, fun (_Src, _Dst, B) -> - rabbit_misc:const(mnesia:read({rabbit_route, B}) /= []) - end, fun not_found_or_absent_errs/1). +exists(Binding0) -> + Binding = sort_args(Binding0), + rabbit_db_binding:exists(Binding). -spec add(rabbit_types:binding(), rabbit_types:username()) -> bind_res(). @@ -153,98 +127,52 @@ add(Binding, ActingUser) -> add(Binding, fun (_Src, _Dst) -> ok end, ActingUser) -spec add(rabbit_types:binding(), inner_fun(), rabbit_types:username()) -> bind_res(). -add(Binding, InnerFun, ActingUser) -> - binding_action( - Binding, - fun (Src, Dst, B) -> - case rabbit_exchange:validate_binding(Src, B) of - ok -> - lock_resource(Src, read), - lock_resource(Dst, read), - %% this argument is used to check queue exclusivity; - %% in general, we want to fail on that in preference to - %% anything else - case InnerFun(Src, Dst) of - ok -> - case mnesia:read({rabbit_route, B}) of - [] -> add(Src, Dst, B, ActingUser); - [_] -> fun () -> ok end - end; - {error, _} = Err -> - rabbit_misc:const(Err) - end; - {error, _} = Err -> - rabbit_misc:const(Err) - end - end, fun not_found_or_absent_errs/1). - -add(Src, Dst, B, ActingUser) -> - ok = sync_route(#route{binding = B}, durable(Src), durable(Dst), - should_index_table(Src), fun mnesia:write/3), - x_callback(transaction, Src, add_binding, B), - Serial = rabbit_exchange:serial(Src), - fun () -> - x_callback(Serial, Src, add_binding, B), - ok = rabbit_event:notify( - binding_created, - info(B) ++ [{user_who_performed_action, ActingUser}]) +add(Binding0, InnerFun, ActingUser) -> + Binding = sort_args(Binding0), + case + rabbit_db_binding:create(Binding, binding_checks(Binding, InnerFun)) + of + ok -> + ok = rabbit_event:notify( + binding_created, + info(Binding) ++ [{user_who_performed_action, ActingUser}]); + Err -> + Err end. --spec remove(rabbit_types:binding()) -> bind_res(). -remove(Binding) -> remove(Binding, fun (_Src, _Dst) -> ok end, ?INTERNAL_USER). - --spec remove(rabbit_types:binding(), rabbit_types:username()) -> bind_res(). -remove(Binding, ActingUser) -> remove(Binding, fun (_Src, _Dst) -> ok end, ActingUser). +binding_type(Src, Dst) -> + binding_type0(durable(Src), durable(Dst)). +binding_type0(true, true) -> + durable; +binding_type0(false, true) -> + semi_durable; +binding_type0(_, _) -> + transient. -spec remove(rabbit_types:binding(), inner_fun(), rabbit_types:username()) -> bind_res(). -remove(Binding, InnerFun, ActingUser) -> - binding_action( - Binding, - fun (Src, Dst, B) -> - lock_resource(Src, read), - lock_resource(Dst, read), - case mnesia:read(rabbit_route, B, write) of - [] -> case mnesia:read(rabbit_durable_route, B, write) of - [] -> rabbit_misc:const(ok); - %% We still delete the binding and run - %% all post-delete functions if there is only - %% a durable route in the database - _ -> remove(Src, Dst, B, ActingUser) - end; - _ -> case InnerFun(Src, Dst) of - ok -> remove(Src, Dst, B, ActingUser); - {error, _} = Err -> rabbit_misc:const(Err) - end - end - end, fun absent_errs_only/1). - -remove(Src, Dst, B, ActingUser) -> - ok = sync_route(#route{binding = B}, durable(Src), durable(Dst), - should_index_table(Src), fun delete/3), - Deletions = maybe_auto_delete( - B#binding.source, [B], new_deletions(), false), - process_deletions(Deletions, ActingUser). +remove(Binding0, InnerFun, ActingUser) -> + Binding = sort_args(Binding0), + case + rabbit_db_binding:delete(Binding, InnerFun) + of + ok -> + ok; + {error, _} = Err -> + Err; + Deletions -> + notify_deletions(Deletions, ActingUser) + end. -spec list_explicit() -> bindings(). list_explicit() -> - mnesia:async_dirty( - fun () -> - AllRoutes = mnesia:dirty_match_object(rabbit_route, #route{_ = '_'}), - [B || #route{binding = B} <- AllRoutes] - end). + rabbit_db_binding:get_all_explicit(). -spec list(rabbit_types:vhost()) -> bindings(). list(VHostPath) -> - VHostResource = rabbit_misc:r(VHostPath, '_'), - Route = #route{binding = #binding{source = VHostResource, - destination = VHostResource, - _ = '_'}, - _ = '_'}, - ExplicitBindings = [B || #route{binding = B} <- mnesia:dirty_match_object(rabbit_route, - Route)], + ExplicitBindings = rabbit_db_binding:get_all(VHostPath), implicit_bindings(VHostPath) ++ ExplicitBindings. -spec list_for_source @@ -252,44 +180,17 @@ list(VHostPath) -> list_for_source(?DEFAULT_EXCHANGE(VHostPath)) -> implicit_bindings(VHostPath); -list_for_source(SrcName) -> - Route = #route{binding = #binding{source = SrcName, _ = '_'}}, - Fun = list_for_route(Route, false), - mnesia:async_dirty(Fun). + +list_for_source(Resource) -> + rabbit_db_binding:get_all_for_source(Resource). -spec list_for_destination (rabbit_types:binding_destination()) -> bindings(). list_for_destination(DstName = #resource{}) -> - Route = #route{binding = #binding{destination = DstName, - _ = '_'}}, - Fun = list_for_route(Route, true), - ExplicitBindings = mnesia:async_dirty(Fun), + ExplicitBindings = rabbit_db_binding:get_all_for_destination(DstName), implicit_for_destination(DstName) ++ ExplicitBindings. --spec list_between( - rabbit_types:binding_source(), - rabbit_types:binding_destination()) -> bindings(). - -list_between(SrcName = #resource{}, DstName = #resource{}) -> - mnesia:async_dirty( - fun() -> - Route = #route{binding = #binding{ - source = SrcName, - destination = DstName, - _ = '_'} - }, - Durable = [B || #route{binding = B} <- mnesia:match_object(rabbit_durable_route, Route, read)], - Transient = [B || #route{binding = B} <- mnesia:match_object(rabbit_route, Route, read)], - SemiDurable = [B || #route{binding = B} <- mnesia:match_object(rabbit_semi_durable_route, Route, read)], - Durable ++ Transient ++ SemiDurable - end). - --spec has_any_between(rabbit_types:binding_source(), - rabbit_types:binding_destination()) -> boolean(). -has_any_between(Source, Destination) -> - list_between(Source, Destination) =/= []. - implicit_bindings(VHostPath) -> DstQueues = rabbit_amqqueue:list_names(VHostPath), [ #binding{source = ?DEFAULT_EXCHANGE(VHostPath), @@ -325,23 +226,7 @@ list_for_source_and_destination(?DEFAULT_EXCHANGE(VHostPath), key = QName, args = []}]; list_for_source_and_destination(SrcName, DstName, Reverse) -> - Route = #route{binding = #binding{source = SrcName, - destination = DstName, - _ = '_'}}, - Fun = list_for_route(Route, Reverse), - mnesia:async_dirty(Fun). - -list_for_route(Route, false) -> - fun() -> - [B || #route{binding = B} <- mnesia:match_object(rabbit_route, Route, read)] - end; -list_for_route(Route, true) -> - fun() -> - [reverse_binding(B) || - #reverse_route{reverse_binding = B} <- - mnesia:match_object(rabbit_reverse_route, - reverse_route(Route), read)] - end. + rabbit_db_binding:get_all(SrcName, DstName, Reverse). -spec info_keys() -> rabbit_types:info_keys(). @@ -388,233 +273,22 @@ info_all(VHostPath, Items, Ref, AggregatorPid) -> rabbit_control_misc:emitting_map( AggregatorPid, Ref, fun(B) -> info(B, Items) end, list(VHostPath)). --spec has_for_source(rabbit_types:binding_source()) -> boolean(). - -has_for_source(SrcName) -> - Match = #route{binding = #binding{source = SrcName, _ = '_'}}, - %% we need to check for semi-durable routes (which subsumes - %% durable routes) here too in case a bunch of routes to durable - %% queues have been removed temporarily as a result of a node - %% failure - contains(rabbit_route, Match) orelse - contains(rabbit_semi_durable_route, Match). - --spec remove_for_source(rabbit_types:binding_source() | - rabbit_types:exchange()) -> - bindings(). - -remove_for_source(#exchange{name = SrcName} = SrcX) -> - remove_for_source(SrcName, should_index_table(SrcX)); -remove_for_source(SrcName) -> - remove_for_source(SrcName, undefined). - --spec remove_for_source(rabbit_types:binding_source(), - boolean() | undefined) -> - bindings(). -remove_for_source(SrcName, ShouldIndexTable) -> - lock_resource(SrcName), - Match = #route{binding = #binding{source = SrcName, _ = '_'}}, - remove_routes( - lists:usort( - mnesia:dirty_match_object(rabbit_route, Match) ++ - mnesia:dirty_match_object(rabbit_semi_durable_route, Match)), - ShouldIndexTable). - --spec remove_for_destination - (rabbit_types:binding_destination(), boolean()) -> deletions(). - -remove_for_destination(DstName, OnlyDurable) -> - remove_for_destination(DstName, OnlyDurable, fun remove_routes/1). - --spec remove_transient_for_destination - (rabbit_types:binding_destination()) -> deletions(). - -remove_transient_for_destination(DstName) -> - remove_for_destination(DstName, false, fun remove_transient_routes/1). - %%---------------------------------------------------------------------------- durable(#exchange{durable = D}) -> D; durable(Q) when ?is_amqqueue(Q) -> amqqueue:is_durable(Q). -binding_action(Binding = #binding{source = SrcName, - destination = DstName, - args = Arguments}, Fun, ErrFun) -> - call_with_source_and_destination( - SrcName, DstName, - fun (Src, Dst) -> - SortedArgs = rabbit_misc:sort_field_table(Arguments), - Fun(Src, Dst, Binding#binding{args = SortedArgs}) - end, ErrFun). - -sync_route(Route, true, true, ShouldIndexTable, Fun) -> - ok = Fun(rabbit_durable_route, Route, write), - sync_route(Route, false, true, ShouldIndexTable, Fun); - -sync_route(Route, false, true, ShouldIndexTable, Fun) -> - ok = Fun(rabbit_semi_durable_route, Route, write), - sync_route(Route, false, false, ShouldIndexTable, Fun); - -sync_route(Route, _SrcDurable, false, ShouldIndexTable, Fun) -> - sync_transient_route(Route, ShouldIndexTable, Fun). - -sync_transient_route(Route, ShouldIndexTable, Fun) -> - ok = Fun(rabbit_route, Route, write), - ok = Fun(rabbit_reverse_route, reverse_route(Route), write), - sync_index_route(Route, ShouldIndexTable, Fun). - -sync_index_route(Route, true, Fun) -> - %% Do not block as 'blocking' will cause a deadlock when function - %% rabbit_binding:populate_index_route_table/0 (feature flag migration) runs in parallel. - case rabbit_feature_flags:is_enabled(direct_exchange_routing_v2, non_blocking) of - true -> - ok = Fun(rabbit_index_route, index_route(Route), write); - false -> - ok; - state_changing -> - case rabbit_table:exists(rabbit_index_route) of - true -> - ok = Fun(rabbit_index_route, index_route(Route), write); - false -> - ok - end - end; -sync_index_route(_, _, _) -> - ok. - -call_with_source_and_destination(SrcName, DstName, Fun, ErrFun) -> - SrcTable = table_for_resource(SrcName), - DstTable = table_for_resource(DstName), - rabbit_misc:execute_mnesia_tx_with_tail( - fun () -> - case {mnesia:read({SrcTable, SrcName}), - mnesia:read({DstTable, DstName})} of - {[Src], [Dst]} -> Fun(Src, Dst); - {[], [_] } -> ErrFun([SrcName]); - {[_], [] } -> ErrFun([DstName]); - {[], [] } -> ErrFun([SrcName, DstName]) - end - end). - -not_found_or_absent_errs(Names) -> - Errs = [not_found_or_absent(Name) || Name <- Names], - rabbit_misc:const({error, {resources_missing, Errs}}). - -absent_errs_only(Names) -> - Errs = [E || Name <- Names, - {absent, _Q, _Reason} = E <- [not_found_or_absent(Name)]], - rabbit_misc:const(case Errs of - [] -> ok; - _ -> {error, {resources_missing, Errs}} - end). - -table_for_resource(#resource{kind = exchange}) -> rabbit_exchange; -table_for_resource(#resource{kind = queue}) -> rabbit_queue. - -not_found_or_absent(#resource{kind = exchange} = Name) -> - {not_found, Name}; -not_found_or_absent(#resource{kind = queue} = Name) -> - case rabbit_amqqueue:not_found_or_absent(Name) of - not_found -> {not_found, Name}; - {absent, _Q, _Reason} = R -> R - end. - -contains(Table, MatchHead) -> - continue(mnesia:select(Table, [{MatchHead, [], ['$_']}], 1, read)). - -continue('$end_of_table') -> false; -continue({[_|_], _}) -> true; -continue({[], Continuation}) -> continue(mnesia:select(Continuation)). - -remove_routes(Routes) -> - remove_routes(Routes, undefined). - -remove_routes(Routes, ShouldIndexTable) -> - %% This partitioning allows us to suppress unnecessary delete - %% operations on disk tables, which require an fsync. - {RamRoutes, DiskRoutes} = - lists:partition(fun (R) -> mnesia:read( - rabbit_durable_route, R#route.binding, read) == [] end, - Routes), - {RamOnlyRoutes, SemiDurableRoutes} = - lists:partition(fun (R) -> mnesia:read( - rabbit_semi_durable_route, R#route.binding, read) == [] end, - RamRoutes), - %% Of course the destination might not really be durable but it's - %% just as easy to try to delete it from the semi-durable table - %% than check first - [ok = sync_route(R, true, true, ShouldIndexTable, fun delete/3) || - R <- DiskRoutes], - [ok = sync_route(R, false, true, ShouldIndexTable, fun delete/3) || - R <- SemiDurableRoutes], - [ok = sync_route(R, false, false, ShouldIndexTable, fun delete/3) || - R <- RamOnlyRoutes], - _ = case ShouldIndexTable of - B when is_boolean(B) -> - ok; - undefined -> - [begin - case rabbit_exchange:lookup(Src) of - {ok, X} -> - ok = sync_index_route(R, should_index_table(X), fun delete/3); - {error, not_found} -> - ok = sync_index_route(R, true, fun delete/3) - end - end || #route{binding = #binding{source = Src}} = R <- Routes] - end, - [R#route.binding || R <- Routes]. - -delete(Tab, #route{binding = B}, LockKind) -> - mnesia:delete(Tab, B, LockKind); -delete(Tab, #reverse_route{reverse_binding = B}, LockKind) -> - mnesia:delete(Tab, B, LockKind); -delete(Tab, #index_route{} = Record, LockKind) -> - mnesia:delete_object(Tab, Record, LockKind). - -remove_transient_routes(Routes) -> - lists:map(fun(#route{binding = #binding{source = Src} = Binding} = Route) -> - ShouldIndexTable = case rabbit_exchange:lookup(Src) of - {ok, X} -> - should_index_table(X); - {error, not_found} -> - true - end, - ok = sync_transient_route(Route, ShouldIndexTable, fun delete/3), - Binding - end, Routes). - -remove_for_destination(DstName, OnlyDurable, Fun) -> - lock_resource(DstName), - MatchFwd = #route{binding = #binding{destination = DstName, _ = '_'}}, - MatchRev = reverse_route(MatchFwd), - Routes = case OnlyDurable of - false -> - [reverse_route(R) || - R <- mnesia:dirty_match_object( - rabbit_reverse_route, MatchRev)]; - true -> lists:usort( - mnesia:dirty_match_object( - rabbit_durable_route, MatchFwd) ++ - mnesia:dirty_match_object( - rabbit_semi_durable_route, MatchFwd)) - end, - Bindings = Fun(Routes), - group_bindings_fold(fun maybe_auto_delete/4, new_deletions(), - lists:keysort(#binding.source, Bindings), OnlyDurable). - -%% Instead of locking entire table on remove operations we can lock the -%% affected resource only. -lock_resource(Name) -> lock_resource(Name, write). - -lock_resource(Name, LockKind) -> - _ = mnesia:lock({global, Name, mnesia:table_info(rabbit_route, where_to_write)}, - LockKind), - ok. +sort_args(#binding{args = Arguments} = Binding) -> + SortedArgs = rabbit_misc:sort_field_table(Arguments), + Binding#binding{args = SortedArgs}. %% Requires that its input binding list is sorted in exchange-name %% order, so that the grouping of bindings (for passing to %% group_bindings_and_auto_delete1) works properly. +group_bindings_fold(Fun, Bindings, OnlyDurable) -> + group_bindings_fold(Fun, new_deletions(), Bindings, OnlyDurable). + group_bindings_fold(_Fun, Acc, [], _OnlyDurable) -> Acc; group_bindings_fold(Fun, Acc, [B = #binding{source = SrcName} | Bs], @@ -630,23 +304,6 @@ group_bindings_fold(Fun, SrcName, Acc, Removed, Bindings, OnlyDurable) -> group_bindings_fold(Fun, Fun(SrcName, Bindings, Acc, OnlyDurable), Removed, OnlyDurable). -maybe_auto_delete(XName, Bindings, Deletions, OnlyDurable) -> - {Entry, Deletions1} = - case mnesia:read({case OnlyDurable of - true -> rabbit_durable_exchange; - false -> rabbit_exchange - end, XName}) of - [] -> {{undefined, not_deleted, Bindings}, Deletions}; - [X] -> case rabbit_exchange:maybe_auto_delete(X, OnlyDurable) of - not_deleted -> - {{X, not_deleted, Bindings}, Deletions}; - {deleted, Deletions2} -> - {{X, deleted, Bindings}, - combine_deletions(Deletions, Deletions2)} - end - end, - add_deletion(XName, Entry, Deletions1). - reverse_route(#route{binding = Binding}) -> #reverse_route{reverse_binding = reverse_binding(Binding)}; @@ -715,76 +372,56 @@ merge_entry({X1, Deleted1, Bindings1}, {X2, Deleted2, Bindings2}) -> anything_but(not_deleted, Deleted1, Deleted2), [Bindings1 | Bindings2]}. --spec process_deletions(deletions(), rabbit_types:username()) -> rabbit_misc:thunk('ok'). - -process_deletions(Deletions, ActingUser) -> - AugmentedDeletions = - dict:map(fun (_XName, {X, deleted, Bindings}) -> - Bs = lists:flatten(Bindings), - x_callback(transaction, X, delete, Bs), - {X, deleted, Bs, none}; - (_XName, {X, not_deleted, Bindings}) -> - Bs = lists:flatten(Bindings), - x_callback(transaction, X, remove_bindings, Bs), - {X, not_deleted, Bs, rabbit_exchange:serial(X)} - end, Deletions), - fun() -> - dict:fold(fun (XName, {X, deleted, Bs, Serial}, ok) -> - ok = rabbit_event:notify( - exchange_deleted, - [{name, XName}, - {user_who_performed_action, ActingUser}]), - del_notify(Bs, ActingUser), - x_callback(Serial, X, delete, Bs); - (_XName, {X, not_deleted, Bs, Serial}, ok) -> - del_notify(Bs, ActingUser), - x_callback(Serial, X, remove_bindings, Bs) - end, ok, AugmentedDeletions) - end. +notify_deletions({error, not_found}, _) -> + ok; +notify_deletions(Deletions, ActingUser) -> + dict:fold(fun (XName, {_X, deleted, Bs, _}, ok) -> + notify_exchange_deletion(XName, ActingUser), + notify_bindings_deletion(Bs, ActingUser); + (_XName, {_X, not_deleted, Bs, _}, ok) -> + notify_bindings_deletion(Bs, ActingUser); + (XName, {_X, deleted, Bs}, ok) -> + notify_exchange_deletion(XName, ActingUser), + notify_bindings_deletion(Bs, ActingUser); + (_XName, {_X, not_deleted, Bs}, ok) -> + notify_bindings_deletion(Bs, ActingUser) + end, ok, Deletions). + +notify_exchange_deletion(XName, ActingUser) -> + ok = rabbit_event:notify( + exchange_deleted, + [{name, XName}, + {user_who_performed_action, ActingUser}]). + +notify_bindings_deletion(Bs, ActingUser) -> + [rabbit_event:notify(binding_deleted, + info(B) ++ [{user_who_performed_action, ActingUser}]) + || B <- Bs], + ok. -del_notify(Bs, ActingUser) -> [rabbit_event:notify( - binding_deleted, - info(B) ++ [{user_who_performed_action, ActingUser}]) - || B <- Bs], - ok. - -x_callback(Serial, X, F, Bs) -> - ok = rabbit_exchange:callback(X, F, Serial, [X, Bs]). - --spec populate_index_route_table() -> ok. -populate_index_route_table() -> - rabbit_misc:execute_mnesia_transaction( - fun () -> - mnesia:read_lock_table(rabbit_route), - mnesia:read_lock_table(rabbit_exchange), - mnesia:write_lock_table(rabbit_index_route), - Routes = rabbit_misc:dirty_read_all(rabbit_route), - lists:foreach( - fun(#route{binding = #binding{source = Exchange}} = Route) -> - case rabbit_exchange:lookup(Exchange) of - {ok, X} -> - case should_index_table(X) of - true -> - ok = mnesia:dirty_write(rabbit_index_route, index_route(Route)); - false -> - ok - end; - {error, not_found} -> - mnesia:abort({source_exchange_not_found, Route}) - end - end, Routes) - end). - -%% Only the direct exchange type uses the rabbit_index_route table to store its -%% bindings by table key tuple {SourceExchange, RoutingKey}. -%% Other built-in exchange types lookup destinations by SourceExchange, and -%% therefore will not need to read from the rabbit_index_route index table. -%% Therefore, we avoid inserting and deleting into rabbit_index_route for other exchange -%% types. This reduces write lock conflicts on the same tuple {SourceExchange, RoutingKey} -%% reducing the number of restarted Mnesia transactions. -should_index_table(#exchange{name = ?DEFAULT_EXCHANGE(_)}) -> - false; -should_index_table(#exchange{type = direct}) -> - true; -should_index_table(_) -> - false. +-spec process_deletions(deletions()) -> rabbit_misc:thunk('ok'). +process_deletions(Deletions) -> + dict:map(fun (_XName, {X, deleted, Bindings}) -> + Bs = lists:flatten(Bindings), + Serial = rabbit_exchange:serial(X), + rabbit_exchange:callback(X, delete, Serial, [X]), + {X, deleted, Bs, none}; + (_XName, {X, not_deleted, Bindings}) -> + Bs = lists:flatten(Bindings), + Serial = rabbit_exchange:serial(X), + rabbit_exchange:callback(X, remove_bindings, Serial, [X, Bs]), + {X, not_deleted, Bs, none} + end, Deletions). + +binding_checks(Binding, InnerFun) -> + fun(Src, Dst) -> + case rabbit_exchange:validate_binding(Src, Binding) of + ok -> + %% this argument is used to check queue exclusivity; + %% in general, we want to fail on that in preference to + %% anything else + InnerFun(Src, Dst); + Err -> + Err + end + end. diff --git a/deps/rabbit/src/rabbit_classic_queue.erl b/deps/rabbit/src/rabbit_classic_queue.erl index a5b92bda94dc..153889ce4646 100644 --- a/deps/rabbit/src/rabbit_classic_queue.erl +++ b/deps/rabbit/src/rabbit_classic_queue.erl @@ -127,7 +127,7 @@ is_recoverable(Q) when ?is_amqqueue(Q) -> %% 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) =:= [] + (rabbit_amqqueue:exists(Q) orelse not rabbit_mnesia:is_process_alive(amqqueue:get_pid(Q))). recover(VHost, Queues) -> diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl index 825fb11c803d..b94c7ff8f2db 100644 --- a/deps/rabbit/src/rabbit_core_ff.erl +++ b/deps/rabbit/src/rabbit_core_ff.erl @@ -133,17 +133,11 @@ Args :: rabbit_feature_flags:enable_callback_args(), Ret :: rabbit_feature_flags:enable_callback_ret(). direct_exchange_routing_v2_enable(#{feature_name := FeatureName}) -> - DependantTables = [rabbit_route, rabbit_exchange], - ok = rabbit_table:wait(DependantTables, _Retry = true), - [ok = rabbit_table:create_local_copy(Tab, ram_copies) || Tab <- DependantTables], - - NewTable = rabbit_index_route, + TableName = rabbit_index_route, try - ok = rabbit_table:create( - NewTable, rabbit_table:rabbit_index_route_definition()), - case rabbit_table:ensure_table_copy(NewTable, node(), ram_copies) of + case rabbit_db_binding:create_index_route_table() of ok -> - ok = rabbit_binding:populate_index_route_table(); + ok; {error, Err} = Error -> ?LOG_ERROR( "Feature flags: `~ts`: failed to add copy of table ~ts to " diff --git a/deps/rabbit/src/rabbit_db.erl b/deps/rabbit/src/rabbit_db.erl index d9297342e097..0e60e2033e3e 100644 --- a/deps/rabbit/src/rabbit_db.erl +++ b/deps/rabbit/src/rabbit_db.erl @@ -18,6 +18,11 @@ ensure_dir_exists/0]). -export([run/1]). +%% Exported to be used by various rabbit_db_* modules +-export([ + list_in_mnesia/2 + ]). + %% Default timeout for operations on remote nodes. -define(TIMEOUT, 60000). @@ -137,3 +142,8 @@ run(Funs) run_with_mnesia(Fun) -> Fun(). + +list_in_mnesia(Table, Match) -> + %% Not dirty_match_object since that would not be transactional when used in a + %% tx context + mnesia:async_dirty(fun () -> mnesia:match_object(Table, Match, read) end). diff --git a/deps/rabbit/src/rabbit_db_binding.erl b/deps/rabbit/src/rabbit_db_binding.erl new file mode 100644 index 000000000000..95abc5bf48f1 --- /dev/null +++ b/deps/rabbit/src/rabbit_db_binding.erl @@ -0,0 +1,699 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_binding). + +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([exists/1, create/2, delete/2, get_all/1, get_all_for_source/1, + get_all_for_destination/1, get_all/3, get_all_explicit/0, + fold/2]). + +%% Routing. These functions are in the hot code path +-export([match/2, match_routing_key/3]). + +%% Exported to be used by various rabbit_db_* modules +-export([ + delete_for_destination_in_mnesia/2, + delete_all_for_exchange_in_mnesia/3, + delete_transient_for_destination_in_mnesia/1, + has_for_source_in_mnesia/1 + ]). + +-export([recover/0, recover/1]). + +-export([create_index_route_table/0]). + +%% ------------------------------------------------------------------- +%% exists(). +%% ------------------------------------------------------------------- + +-spec exists(Binding) -> Exists when + Binding :: rabbit_types:binding(), + Exists :: boolean(). +%% @doc Indicates if the binding `Binding' exists. +%% +%% @returns true if the binding exists, false otherwise. +%% +%% @private + +exists(Binding) -> + rabbit_db:run( + #{mnesia => fun() -> exists_in_mnesia(Binding) end + }). + +exists_in_mnesia(Binding) -> + binding_action_in_mnesia( + Binding, fun (_Src, _Dst) -> + rabbit_misc:const(mnesia:read({rabbit_route, Binding}) /= []) + end, fun not_found_or_absent_errs_in_mnesia/1). + +%% ------------------------------------------------------------------- +%% create(). +%% ------------------------------------------------------------------- + +-spec create(Binding, ChecksFun) -> Ret when + Binding :: rabbit_types:binding(), + Src :: rabbit_types:r('exchange'), + Dst :: rabbit_types:r('exchange') | rabbit_types:r('queue'), + BindingType :: durable | semi_durable | transient, + ChecksFun :: fun((Src, Dst) -> {ok, BindingType} | {error, Reason :: any()}), + Ret :: ok | {error, Reason :: any()}. +%% @doc Writes a binding if it doesn't exist already and passes the validation in +%% `ChecksFun` i.e. exclusive access +%% +%% @returns ok, or an error if the validation has failed. +%% +%% @private + +create(Binding, ChecksFun) -> + rabbit_db:run( + #{mnesia => fun() -> create_in_mnesia(Binding, ChecksFun) end + }). + +%% ------------------------------------------------------------------- +%% delete(). +%% ------------------------------------------------------------------- + +-spec delete(Binding, ChecksFun) -> Ret when + Binding :: rabbit_types:binding(), + Src :: rabbit_types:r('exchange'), + Dst :: rabbit_types:r('exchange') | rabbit_types:r('queue'), + BindingType :: durable | semi_durable | transient, + ChecksFun :: fun((Src, Dst) -> {ok, BindingType} | {error, Reason :: any()}), + Ret :: ok | {error, Reason :: any()}. +%% @doc Deletes a binding record from the database if it passes the validation in +%% `ChecksFun`. It also triggers the deletion of auto-delete exchanges if needed. +%% +%% @private + +delete(Binding, ChecksFun) -> + rabbit_db:run( + #{mnesia => fun() -> delete_in_mnesia(Binding, ChecksFun) end + }). + +%% ------------------------------------------------------------------- +%% get_all(). +%% ------------------------------------------------------------------- + +-spec get_all(VHostName) -> [Binding] when + VHostName :: vhost:name(), + Binding :: rabbit_types:binding(). +%% @doc Returns all binding records in the given virtual host. +%% +%% @returns the list of binding records. +%% +%% @private + +get_all(VHost) -> + rabbit_db:run( + #{mnesia => fun() -> get_all_in_mnesia(VHost) end + }). + +get_all_in_mnesia(VHost) -> + VHostResource = rabbit_misc:r(VHost, '_'), + Match = #route{binding = #binding{source = VHostResource, + destination = VHostResource, + _ = '_'}, + _ = '_'}, + [B || #route{binding = B} <- rabbit_db:list_in_mnesia(rabbit_route, Match)]. + +-spec get_all_for_source(Src) -> [Binding] when + Src :: rabbit_types:r('exchange'), + Binding :: rabbit_types:binding(). +%% @doc Returns all binding records for a given exchange in the given virtual host. +%% +%% @returns the list of binding records. +%% +%% @private + +get_all_for_source(Resource) -> + rabbit_db:run( + #{mnesia => fun() -> get_all_for_source_in_mnesia(Resource) end + }). + +get_all_for_source_in_mnesia(Resource) -> + Route = #route{binding = #binding{source = Resource, _ = '_'}}, + Fun = list_for_route(Route, false), + mnesia:async_dirty(Fun). + +list_for_route(Route, false) -> + fun() -> + [B || #route{binding = B} <- mnesia:match_object(rabbit_route, Route, read)] + end; +list_for_route(Route, true) -> + fun() -> + [rabbit_binding:reverse_binding(B) || + #reverse_route{reverse_binding = B} <- + mnesia:match_object(rabbit_reverse_route, + rabbit_binding:reverse_route(Route), read)] + end. + +-spec get_all_for_destination(Dst) -> [Binding] when + Dst :: rabbit_types:r('exchange') | rabbit_types:r('queue'), + Binding :: rabbit_types:binding(). +%% @doc Returns all binding records for a given exchange or queue destination +%% in the given virtual host. +%% +%% @returns the list of binding records. +%% +%% @private + +get_all_for_destination(Dst) -> + rabbit_db:run( + #{mnesia => fun() -> get_all_for_destination_in_mnesia(Dst) end + }). + +get_all_for_destination_in_mnesia(Dst) -> + Route = #route{binding = #binding{destination = Dst, + _ = '_'}}, + Fun = list_for_route(Route, true), + mnesia:async_dirty(Fun). + +-spec get_all(Src, Dst, Reverse) -> [Binding] when + Src :: rabbit_types:r('exchange'), + Dst :: rabbit_types:r('exchange') | rabbit_types:r('queue'), + Reverse :: boolean(), + Binding :: rabbit_types:binding(). +%% @doc Returns all binding records for a given source and destination +%% in the given virtual host. +%% +%% @returns the list of binding records. +%% +%% @private + +get_all(SrcName, DstName, Reverse) -> + rabbit_db:run( + #{mnesia => fun() -> get_all_in_mnesia(SrcName, DstName, Reverse) end + }). + +get_all_in_mnesia(SrcName, DstName, Reverse) -> + Route = #route{binding = #binding{source = SrcName, + destination = DstName, + _ = '_'}}, + Fun = list_for_route(Route, Reverse), + mnesia:async_dirty(Fun). + +-spec get_all_explicit() -> [Binding] when + Binding :: rabbit_types:binding(). +%% @doc Returns all explicit binding records, the bindings explicitly added and not +%% automatically generated to the default exchange. +%% +%% @returns the list of binding records. +%% +%% @private + +get_all_explicit() -> + rabbit_db:run( + #{mnesia => fun() -> get_all_explicit_in_mnesia() end + }). + +get_all_explicit_in_mnesia() -> + mnesia:async_dirty( + fun () -> + AllRoutes = mnesia:dirty_match_object(rabbit_route, #route{_ = '_'}), + [B || #route{binding = B} <- AllRoutes] + end). + +-spec fold(Fun, Acc) -> Acc when + Fun :: fun((Binding :: rabbit_types:binding(), Acc) -> Acc), + Acc :: any(). +%% @doc Folds over all the bindings, making it more efficient than `get_all()` and +%% folding over the returned binding list. +%% Just used by prometheus_rabbitmq_core_metrics_collector to iterate over the bindings. +%% +%% @returns the fold accumulator +%% +%% @private + +fold(Fun, Acc) -> + rabbit_db:run( + #{mnesia => fun() -> fold_in_mnesia(Fun, Acc) end + }). + +fold_in_mnesia(Fun, Acc) -> + ets:foldl(fun(#route{binding = Binding}, Acc0) -> + Fun(Binding, Acc0) + end, Acc, rabbit_route). + +recover() -> + rabbit_db:run( + #{mnesia => fun() -> recover_in_mnesia() end + }). + +recover(RecoverFun) -> + rabbit_db:run( + #{mnesia => fun() -> recover_in_mnesia(RecoverFun) end + }). + +recover_in_mnesia(RecoverFun) -> + [RecoverFun(Route, Src, Dst, fun recover_semi_durable_route/2) || + #route{binding = #binding{destination = Dst, + source = Src}} = Route <- + rabbit_misc:dirty_read_all(rabbit_semi_durable_route)]. + +create_index_route_table() -> + rabbit_db:run( + #{mnesia => fun() -> create_index_route_table_in_mnesia() end + }). + +create_index_route_table_in_mnesia() -> + TableName = rabbit_index_route, + DependantTables = [rabbit_route, rabbit_exchange], + ok = rabbit_table:wait(DependantTables, _Retry = true), + [ok = rabbit_table:create_local_copy(Tab, ram_copies) || Tab <- DependantTables], + ok = rabbit_table:create( + TableName, rabbit_table:rabbit_index_route_definition()), + case rabbit_table:ensure_table_copy(TableName, node(), ram_copies) of + ok -> + ok = populate_index_route_table_in_mnesia(); + Error -> + Error + end. + +%% Routing - HOT CODE PATH + +match(SrcName, Match) -> + rabbit_db:run( + #{mnesia => fun() -> match_in_mnesia(SrcName, Match) end + }). + +match_in_mnesia(SrcName, Match) -> + MatchHead = #route{binding = #binding{source = SrcName, + _ = '_'}}, + Routes = ets:select(rabbit_route, [{MatchHead, [], [['$_']]}]), + [Dest || [#route{binding = Binding = #binding{destination = Dest}}] <- + Routes, Match(Binding)]. + +match_routing_key(SrcName, RoutingKeys, UseIndex) -> + rabbit_db:run( + #{mnesia => fun() -> match_routing_key_in_mnesia(SrcName, RoutingKeys, UseIndex) end + }). + +match_routing_key_in_mnesia(SrcName, RoutingKeys, UseIndex) -> + case UseIndex of + true -> + route_v2(rabbit_index_route, SrcName, RoutingKeys); + _ -> + route_in_mnesia_v1(SrcName, RoutingKeys) + end. + +delete_all_for_exchange_in_mnesia(X = #exchange{name = XName}, OnlyDurable, RemoveBindingsForSource) -> + Bindings = case RemoveBindingsForSource of + true -> delete_for_source_in_mnesia(XName); + false -> [] + end, + {deleted, X, Bindings, delete_for_destination_in_mnesia(XName, OnlyDurable, fun delete_routes/1)}. + +delete_for_destination_in_mnesia(DstName, OnlyDurable) -> + delete_for_destination_in_mnesia(DstName, OnlyDurable, fun delete_routes/1). + +-spec delete_transient_for_destination_in_mnesia(rabbit_types:binding_destination()) -> rabbit_binding:deletions(). +delete_transient_for_destination_in_mnesia(DstName) -> + delete_for_destination_in_mnesia(DstName, false, fun delete_transient_routes/1). + +-spec has_for_source_in_mnesia(rabbit_types:binding_source()) -> boolean(). + +has_for_source_in_mnesia(SrcName) -> + Match = #route{binding = #binding{source = SrcName, _ = '_'}}, + %% we need to check for semi-durable routes (which subsumes + %% durable routes) here too in case a bunch of routes to durable + %% queues have been removed temporarily as a result of a node + %% failure + contains(rabbit_route, Match) orelse + contains(rabbit_semi_durable_route, Match). + +%% Internal +%% -------------------------------------------------------------- +binding_action_in_mnesia(#binding{source = SrcName, + destination = DstName}, Fun, ErrFun) -> + SrcTable = table_for_resource(SrcName), + DstTable = table_for_resource(DstName), + rabbit_misc:execute_mnesia_tx_with_tail( + fun () -> + case {mnesia:read({SrcTable, SrcName}), + mnesia:read({DstTable, DstName})} of + {[Src], [Dst]} -> Fun(Src, Dst); + {[], [_] } -> ErrFun([SrcName]); + {[_], [] } -> ErrFun([DstName]); + {[], [] } -> ErrFun([SrcName, DstName]) + end + end). + +table_for_resource(#resource{kind = exchange}) -> rabbit_exchange; +table_for_resource(#resource{kind = queue}) -> rabbit_queue. + +create_in_mnesia(Binding, ChecksFun) -> + binding_action_in_mnesia( + Binding, + fun (Src, Dst) -> + case ChecksFun(Src, Dst) of + ok -> + BindingType = rabbit_binding:binding_type(Src, Dst), + case mnesia:read({rabbit_route, Binding}) of + [] -> + ok = sync_route(#route{binding = Binding}, BindingType, + should_index_table(Src), fun mnesia:write/3), + MaybeSerial = rabbit_exchange:serialise_events(Src), + Serial = serial_in_mnesia(MaybeSerial, Src), + fun () -> + rabbit_exchange:callback(Src, add_binding, Serial, [Src, Binding]) + end; + [_] -> fun () -> ok end + end; + {error, _} = Err -> + rabbit_misc:const(Err) + end + end, fun not_found_or_absent_errs_in_mnesia/1). + +populate_index_route_table_in_mnesia() -> + rabbit_misc:execute_mnesia_transaction( + fun () -> + _ = mnesia:lock({table, rabbit_route}, read), + _ = mnesia:lock({table, rabbit_index_route}, write), + Routes = rabbit_misc:dirty_read_all(rabbit_route), + lists:foreach(fun(#route{binding = #binding{source = Exchange}} = Route) -> + case rabbit_db_exchange:get(Exchange) of + {ok, X} -> + case should_index_table(X) of + true -> + mnesia:dirty_write(rabbit_index_route, + rabbit_binding:index_route(Route)); + false -> + ok + end; + _ -> + ok + end + end, Routes) + end). + +delete_in_mnesia(Binding, ChecksFun) -> + binding_action_in_mnesia( + Binding, + fun (Src, Dst) -> + lock_resource(Src, read), + lock_resource(Dst, read), + case mnesia:read(rabbit_route, Binding, write) of + [] -> case mnesia:read(rabbit_durable_route, Binding, write) of + [] -> rabbit_misc:const(ok); + %% We still delete the binding and run + %% all post-delete functions if there is only + %% a durable route in the database + _ -> delete_in_mnesia(Src, Dst, Binding) + end; + _ -> case ChecksFun(Src, Dst) of + ok -> delete_in_mnesia(Src, Dst, Binding); + {error, _} = Err -> rabbit_misc:const(Err) + end + end + end, fun absent_errs_only_in_mnesia/1). + +delete_in_mnesia(Src, Dst, B) -> + ok = sync_route(#route{binding = B}, rabbit_binding:binding_type(Src, Dst), + should_index_table(Src), fun delete/3), + Deletions0 = maybe_auto_delete_exchange_in_mnesia( + B#binding.source, [B], rabbit_binding:new_deletions(), false), + fun() -> rabbit_binding:process_deletions(Deletions0) end. + +delete_for_destination_in_mnesia(DstName, OnlyDurable, Fun) -> + lock_resource(DstName), + MatchFwd = #route{binding = #binding{destination = DstName, _ = '_'}}, + MatchRev = rabbit_binding:reverse_route(MatchFwd), + Routes = case OnlyDurable of + false -> + [rabbit_binding:reverse_route(R) || + R <- mnesia:dirty_match_object( + rabbit_reverse_route, MatchRev)]; + true -> lists:usort( + mnesia:dirty_match_object( + rabbit_durable_route, MatchFwd) ++ + mnesia:dirty_match_object( + rabbit_semi_durable_route, MatchFwd)) + end, + Bindings = Fun(Routes), + rabbit_binding:group_bindings_fold(fun maybe_auto_delete_exchange_in_mnesia/4, + lists:keysort(#binding.source, Bindings), OnlyDurable). + +delete_for_source_in_mnesia(#exchange{name = SrcName} = SrcX) -> + delete_for_source_in_mnesia(SrcName, should_index_table(SrcX)); +delete_for_source_in_mnesia(SrcName) -> + delete_for_source_in_mnesia(SrcName, undefined). + +-spec delete_for_source_in_mnesia(rabbit_types:binding_source(), + boolean() | undefined) -> [rabbit_types:binding()]. +delete_for_source_in_mnesia(SrcName, ShouldIndexTable) -> + lock_resource(SrcName), + Match = #route{binding = #binding{source = SrcName, _ = '_'}}, + delete_routes( + lists:usort( + mnesia:dirty_match_object(rabbit_route, Match) ++ + mnesia:dirty_match_object(rabbit_semi_durable_route, Match)), + ShouldIndexTable). + +delete_routes(Routes) -> + delete_routes(Routes, undefined). + +delete_routes(Routes, ShouldIndexTable) -> + %% This partitioning allows us to suppress unnecessary delete + %% operations on disk tables, which require an fsync. + {RamRoutes, DiskRoutes} = + lists:partition(fun (R) -> mnesia:read( + rabbit_durable_route, R#route.binding, read) == [] end, + Routes), + {RamOnlyRoutes, SemiDurableRoutes} = + lists:partition(fun (R) -> mnesia:read( + rabbit_semi_durable_route, R#route.binding, read) == [] end, + RamRoutes), + %% Of course the destination might not really be durable but it's + %% just as easy to try to delete it from the semi-durable table + %% than check first + [ok = sync_route(R, durable, ShouldIndexTable, fun delete/3) || + R <- DiskRoutes], + [ok = sync_route(R, semi_durable, ShouldIndexTable, fun delete/3) || + R <- SemiDurableRoutes], + [ok = sync_route(R, transient, ShouldIndexTable, fun delete/3) || + R <- RamOnlyRoutes], + _ = case ShouldIndexTable of + B when is_boolean(B) -> + ok; + undefined -> + [begin + case rabbit_db_exchange:get(Src) of + {ok, X} -> + ok = sync_index_route(R, should_index_table(X), fun delete/3); + _ -> + ok + end + end || #route{binding = #binding{source = Src}} = R <- Routes] + end, + [R#route.binding || R <- Routes]. + +delete_transient_routes(Routes) -> + lists:map(fun(#route{binding = #binding{source = Src} = Binding} = Route) -> + {ok, X} = rabbit_db_exchange:get(Src), + ok = sync_transient_route(Route, should_index_table(X), fun delete/3), + Binding + end, Routes). + +delete(Tab, #route{binding = B}, LockKind) -> + mnesia:delete(Tab, B, LockKind); +delete(Tab, #reverse_route{reverse_binding = B}, LockKind) -> + mnesia:delete(Tab, B, LockKind); +delete(Tab, #index_route{} = Record, LockKind) -> + mnesia:delete_object(Tab, Record, LockKind). + +%% Only the direct exchange type uses the rabbit_index_route table to store its +%% bindings by table key tuple {SourceExchange, RoutingKey}. +%% Other built-in exchange types lookup destinations by SourceExchange, and +%% therefore will not need to read from the rabbit_index_route index table. +%% Therefore, we avoid inserting and deleting into rabbit_index_route for other exchange +%% types. This reduces write lock conflicts on the same tuple {SourceExchange, RoutingKey} +%% reducing the number of restarted Mnesia transactions. +should_index_table(#exchange{name = #resource{name = Name}, + type = direct}) + when Name =/= <<>> -> + true; +should_index_table(_) -> + false. + +not_found_or_absent_errs_in_mnesia(Names) -> + Errs = [not_found_or_absent_in_mnesia(Name) || Name <- Names], + rabbit_misc:const({error, {resources_missing, Errs}}). + +absent_errs_only_in_mnesia(Names) -> + Errs = [E || Name <- Names, + {absent, _Q, _Reason} = E <- [not_found_or_absent_in_mnesia(Name)]], + rabbit_misc:const(case Errs of + [] -> ok; + _ -> {error, {resources_missing, Errs}} + end). + +not_found_or_absent_in_mnesia(#resource{kind = exchange} = Name) -> + {not_found, Name}; +not_found_or_absent_in_mnesia(#resource{kind = queue} = Name) -> + case rabbit_db_queue:not_found_or_absent_queue_in_mnesia(Name) of + not_found -> {not_found, Name}; + {absent, _Q, _Reason} = R -> R + end. + +recover_in_mnesia() -> + rabbit_misc:execute_mnesia_transaction( + fun () -> + _ = mnesia:lock({table, rabbit_durable_route}, read), + _ = mnesia:lock({table, rabbit_semi_durable_route}, write), + Routes = rabbit_misc:dirty_read_all(rabbit_durable_route), + Fun = fun(Route) -> + mnesia:dirty_write(rabbit_semi_durable_route, Route) + end, + lists:foreach(Fun, Routes), + ok + end). + +recover_semi_durable_route(#route{binding = B} = Route, X) -> + MaybeSerial = rabbit_exchange:serialise_events(X), + rabbit_misc:execute_mnesia_transaction( + fun () -> + case mnesia:read(rabbit_semi_durable_route, B, read) of + [] -> no_recover; + _ -> ok = sync_transient_route(Route, should_index_table(X), fun mnesia:write/3), + serial_in_mnesia(MaybeSerial, X) + end + end, + fun (no_recover, _) -> ok; + (_Serial, true) -> rabbit_exchange:callback(X, add_binding, transaction, [X, B]); + (Serial, false) -> rabbit_exchange:callback(X, add_binding, Serial, [X, B]) + end). + +serial_in_mnesia(false, _) -> + none; +serial_in_mnesia(true, X) -> + rabbit_db_exchange:next_serial_in_mnesia_tx(X). + +sync_route(Route, durable, ShouldIndexTable, Fun) -> + ok = Fun(rabbit_durable_route, Route, write), + sync_route(Route, semi_durable, ShouldIndexTable, Fun); + +sync_route(Route, semi_durable, ShouldIndexTable, Fun) -> + ok = Fun(rabbit_semi_durable_route, Route, write), + sync_route(Route, transient, ShouldIndexTable, Fun); + +sync_route(Route, transient, ShouldIndexTable, Fun) -> + sync_transient_route(Route, ShouldIndexTable, Fun). + +sync_transient_route(Route, ShouldIndexTable, Fun) -> + ok = Fun(rabbit_route, Route, write), + ok = Fun(rabbit_reverse_route, rabbit_binding:reverse_route(Route), write), + sync_index_route(Route, ShouldIndexTable, Fun). + +sync_index_route(Route, true, Fun) -> + %% Do not block as blocking will cause a dead lock when + %% function rabbit_binding:populate_index_route_table/0 + %% (i.e. feature flag migration) runs in parallel. + case rabbit_feature_flags:is_enabled(direct_exchange_routing_v2, non_blocking) of + true -> + ok = Fun(rabbit_index_route, rabbit_binding:index_route(Route), write); + false -> + ok; + state_changing -> + case rabbit_table:exists(rabbit_index_route) of + true -> + ok = Fun(rabbit_index_route, rabbit_binding:index_route(Route), write); + false -> + ok + end + end; +sync_index_route(_, _, _) -> + ok. + +maybe_auto_delete_exchange_in_mnesia(XName, Bindings, Deletions, OnlyDurable) -> + {Entry, Deletions1} = + case rabbit_db_exchange:maybe_auto_delete_in_mnesia(XName, OnlyDurable) of + {not_deleted, X} -> + {{X, not_deleted, Bindings}, Deletions}; + {deleted, X, Deletions2} -> + {{X, deleted, Bindings}, + rabbit_binding:combine_deletions(Deletions, Deletions2)} + end, + rabbit_binding:add_deletion(XName, Entry, Deletions1). + +%% Instead of locking entire table on remove operations we can lock the +%% affected resource only. +lock_resource(Name) -> lock_resource(Name, write). + +lock_resource(Name, LockKind) -> + _ = mnesia:lock({global, Name, mnesia:table_info(rabbit_route, where_to_write)}, + LockKind), + ok. + +contains(Table, MatchHead) -> + continue(mnesia:select(Table, [{MatchHead, [], ['$_']}], 1, read)). + +continue('$end_of_table') -> false; +continue({[_|_], _}) -> true; +continue({[], Continuation}) -> continue(mnesia:select(Continuation)). + +%% Routing. Hot code path +%% ------------------------------------------------------------------------- +route_in_mnesia_v1(SrcName, [RoutingKey]) -> + MatchHead = #route{binding = #binding{source = SrcName, + destination = '$1', + key = RoutingKey, + _ = '_'}}, + ets:select(rabbit_route, [{MatchHead, [], ['$1']}]); +route_in_mnesia_v1(SrcName, [_|_] = RoutingKeys) -> + %% Normally we'd call mnesia:dirty_select/2 here, but that is quite + %% expensive for the same reasons as above, and, additionally, due to + %% mnesia 'fixing' the table with ets:safe_fixtable/2, which is wholly + %% unnecessary. According to the ets docs (and the code in erl_db.c), + %% 'select' is safe anyway ("Functions that internally traverse over a + %% table, like select and match, will give the same guarantee as + %% safe_fixtable.") and, furthermore, even the lower level iterators + %% ('first' and 'next') are safe on ordered_set tables ("Note that for + %% tables of the ordered_set type, safe_fixtable/2 is not necessary as + %% calls to first/1 and next/2 will always succeed."), which + %% rabbit_route is. + MatchHead = #route{binding = #binding{source = SrcName, + destination = '$1', + key = '$2', + _ = '_'}}, + Conditions = [list_to_tuple(['orelse' | [{'=:=', '$2', RKey} || + RKey <- RoutingKeys]])], + ets:select(rabbit_route, [{MatchHead, Conditions, ['$1']}]). + +%% rabbit_router:match_routing_key/2 uses ets:select/2 to get destinations. +%% ets:select/2 is expensive because it needs to compile the match spec every +%% time and lookup does not happen by a hash key. +%% +%% In contrast, route_v2/2 increases end-to-end message sending throughput +%% (i.e. from RabbitMQ client to the queue process) by up to 35% by using ets:lookup_element/3. +%% Only the direct exchange type uses the rabbit_index_route table to store its +%% bindings by table key tuple {SourceExchange, RoutingKey}. +-spec route_v2(ets:table(), rabbit_types:binding_source(), [rabbit_router:routing_key(), ...]) -> + rabbit_router:match_result(). +route_v2(Table, SrcName, [RoutingKey]) -> + %% optimization + destinations(Table, SrcName, RoutingKey); +route_v2(Table, SrcName, [_|_] = RoutingKeys) -> + lists:flatmap(fun(Key) -> + destinations(Table, SrcName, Key) + end, RoutingKeys). + +destinations(Table, SrcName, RoutingKey) -> + %% Prefer try-catch block over checking Key existence with ets:member/2. + %% The latter reduces throughput by a few thousand messages per second because + %% of function db_member_hash in file erl_db_hash.c. + %% We optimise for the happy path, that is the binding / table key is present. + try + ets:lookup_element(Table, + {SrcName, RoutingKey}, + #index_route.destination) + catch + error:badarg -> + [] + end. + diff --git a/deps/rabbit/src/rabbit_db_exchange.erl b/deps/rabbit/src/rabbit_db_exchange.erl new file mode 100644 index 000000000000..d048d28d3e67 --- /dev/null +++ b/deps/rabbit/src/rabbit_db_exchange.erl @@ -0,0 +1,516 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_exchange). + +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([ + get_all/0, + get_all/1, + get_all_durable/0, + list/0, + get/1, + get_many/1, + count/0, + update/2, + create_or_get/1, + insert/1, + peek_serial/1, + next_serial/1, + delete/2, + delete_serial/1, + recover/1, + match/1, + exists/1 + ]). + +%% Used by other rabbit_db_* modules +-export([ + maybe_auto_delete_in_mnesia/2, + next_serial_in_mnesia_tx/1, + delete_in_mnesia/3, + update_in_mnesia_tx/2 + ]). + +-type name() :: rabbit_types:r('exchange'). + +%% ------------------------------------------------------------------- +%% get_all(). +%% ------------------------------------------------------------------- + +-spec get_all() -> [Exchange] when + Exchange :: rabbit_types:exchange(). +%% @doc Returns all exchange records. +%% +%% @returns the list of exchange records. +%% +%% @private + +get_all() -> + rabbit_db:run( + #{mnesia => fun() -> get_all_in_mnesia() end}). + +get_all_in_mnesia() -> + rabbit_db:list_in_mnesia(rabbit_exchange, #exchange{_ = '_'}). + +-spec get_all(VHostName) -> [Exchange] when + VHostName :: vhost:name(), + Exchange :: rabbit_types:exchange(). +%% @doc Returns all exchange records in the given virtual host. +%% +%% @returns the list of exchange records. +%% +%% @private + +get_all(VHost) -> + rabbit_db:run( + #{mnesia => fun() -> get_all_in_mnesia(VHost) end + }). + +get_all_in_mnesia(VHost) -> + Match = #exchange{name = rabbit_misc:r(VHost, exchange), _ = '_'}, + rabbit_db:list_in_mnesia(rabbit_exchange, Match). + +-spec get_all_durable() -> [Exchange] when + Exchange :: rabbit_types:exchange(). +%% @doc Returns all durable exchange records. +%% +%% @returns the list of exchange records. +%% +%% @private + +get_all_durable() -> + rabbit_db:run( + #{mnesia => fun() -> get_all_durable_in_mnesia() end + }). + +get_all_durable_in_mnesia() -> + rabbit_db:list_in_mnesia(rabbit_durable_exchange, #exchange{_ = '_'}). + +%% ------------------------------------------------------------------- +%% list(). +%% ------------------------------------------------------------------- + +-spec list() -> [Exchange] when + Exchange :: rabbit_types:exchange(). +%% @doc Lists the names of all exchanges. +%% +%% @returns a list of exchange names. +%% +%% @private + +list() -> + rabbit_db:run( + #{mnesia => fun() -> list_in_mnesia() end + }). + +list_in_mnesia() -> + mnesia:dirty_all_keys(rabbit_exchange). + +%% ------------------------------------------------------------------- +%% get(). +%% ------------------------------------------------------------------- + +-spec get(ExchangeName) -> Ret when + ExchangeName :: name(), + Ret :: {ok, Exchange :: rabbit_types:exchange()} | {error, not_found}. +%% @doc Returns the record of the exchange named `Name'. +%% +%% @returns the exchange record or `{error, not_found}' if no exchange is named +%% `Name'. +%% +%% @private + +get(Name) -> + rabbit_db:run( + #{mnesia => fun() -> get_in_mnesia(Name) end + }). + +get_in_mnesia(Name) -> + rabbit_misc:dirty_read({rabbit_exchange, Name}). + +%% ------------------------------------------------------------------- +%% get_many(). +%% ------------------------------------------------------------------- + +-spec get_many([ExchangeName]) -> [Exchange] when + ExchangeName :: name(), + Exchange :: rabbit_types:exchange(). +%% @doc Returns the records of the exchanges named `Name'. +%% +%% @returns a list of exchange records. +%% +%% @private + +get_many(Names) when is_list(Names) -> + rabbit_db:run( + #{mnesia => fun() -> get_many_in_mnesia(rabbit_exchange, Names) end + }). + +%% ------------------------------------------------------------------- +%% count(). +%% ------------------------------------------------------------------- + +-spec count() -> Num :: integer(). +%% @doc Counts the number of exchanges. +%% +%% @returns the number of exchange records. +%% +%% @private + +count() -> + rabbit_db:run( + #{mnesia => fun() -> count_in_mnesia() end}). + +count_in_mnesia() -> + mnesia:table_info(rabbit_exchange, size). + +%% ------------------------------------------------------------------- +%% update(). +%% ------------------------------------------------------------------- + +-spec update(ExchangeName, UpdateFun) -> Ret when + ExchangeName :: name(), + UpdateFun :: fun((Exchange) -> Exchange), + Ret :: Exchange :: rabbit_types:exchange() | not_found. +%% @doc Updates an existing exchange record using the result of +%% `UpdateFun'. +%% +%% @returns the updated exchange record if the record existed and the +%% update succeeded. It returns `not_found` if the transaction fails. +%% +%% @private + +update(XName, Fun) -> + rabbit_db:run( + #{mnesia => fun() -> update_in_mnesia(XName, Fun) end + }). + +update_in_mnesia(XName, Fun) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + update_in_mnesia_tx(XName, Fun) + end). + +%% ------------------------------------------------------------------- +%% create_or_get(). +%% ------------------------------------------------------------------- + +-spec create_or_get(Exchange) -> Ret when + Exchange :: rabbit_types:exchange(), + Ret :: {new, Exchange} | {existing, Exchange} | {error, any()}. +%% @doc Writes an exchange record if it doesn't exist already or returns +%% the existing one. +%% +%% @returns the existing record if there is one in the database already, or +%% the newly created record. +%% +%% @private + +create_or_get(X) -> + rabbit_db:run( + #{mnesia => fun() -> create_or_get_in_mnesia(X) end + }). + +create_or_get_in_mnesia(#exchange{name = XName} = X) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + case mnesia:wread({rabbit_exchange, XName}) of + [] -> + {new, insert_in_mnesia_tx(X)}; + [ExistingX] -> + {existing, ExistingX} + end + end). + +%% ------------------------------------------------------------------- +%% insert(). +%% ------------------------------------------------------------------- + +-spec insert([Exchange]) -> ok when + Exchange :: rabbit_types:exchange(). +%% @doc Writes the exchange records. +%% +%% @returns ok. +%% +%% @private + +insert(Xs) -> + rabbit_db:run( + #{mnesia => fun() -> insert_in_mnesia(Xs) end + }). + +insert_in_mnesia(Xs) when is_list(Xs) -> + rabbit_misc:execute_mnesia_transaction( + fun () -> + [mnesia:write(rabbit_durable_exchange, X, write) || X <- Xs] + end), + ok. + +%% ------------------------------------------------------------------- +%% peek_serial(). +%% ------------------------------------------------------------------- + +-spec peek_serial(ExchangeName) -> Serial when + ExchangeName :: name(), + Serial :: integer(). +%% @doc Returns the next serial number without increasing it. +%% +%% @returns the next serial number +%% +%% @private + +peek_serial(XName) -> + rabbit_db:run( + #{mnesia => fun() -> peek_serial_in_mnesia(XName) end + }). + +peek_serial_in_mnesia(XName) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + peek_serial_in_mnesia_tx(XName, read) + end). + +%% ------------------------------------------------------------------- +%% next_serial(). +%% ------------------------------------------------------------------- + +-spec next_serial(ExchangeName) -> Serial when + ExchangeName :: name(), + Serial :: integer(). +%% @doc Returns the next serial number and increases it. +%% +%% @returns the next serial number +%% +%% @private + +next_serial(X) -> + rabbit_db:run( + #{mnesia => fun() -> next_serial_in_mnesia(X) end + }). + +next_serial_in_mnesia(X) -> + rabbit_misc:execute_mnesia_transaction(fun() -> + next_serial_in_mnesia_tx(X) + end). + +%% ------------------------------------------------------------------- +%% delete(). +%% ------------------------------------------------------------------- + +-spec delete(ExchangeName, IfUnused) -> Ret when + ExchangeName :: name(), + IfUnused :: boolean(), + Exchange :: rabbit_types:exchange(), + Binding :: rabbit_types:binding(), + Deletions :: dict:dict(), + Ret :: {error, not_found} | {deleted, Exchange, [Binding], Deletions}. +%% @doc Deletes an exchange record from the database. If `IfUnused` is set +%% to `true`, it is only deleted when there are no bindings present on the +%% exchange. +%% +%% @returns an error if the exchange does not exist or a tuple with the exchange, +%% bindings and deletions. Bindings need to be processed on the same transaction, and +%% are later used to generate notifications. Probably shouldn't be here, but not sure +%% how to split it while keeping it atomic. Maybe something about deletions could be +%% handled outside of the transaction. +%% +%% @private + +delete(XName, IfUnused) -> + rabbit_db:run( + #{mnesia => fun() -> delete_in_mnesia(XName, IfUnused) end + }). + +delete_in_mnesia(XName, IfUnused) -> + DeletionFun = case IfUnused of + true -> fun conditional_delete_in_mnesia/2; + false -> fun unconditional_delete_in_mnesia/2 + end, + rabbit_misc:execute_mnesia_transaction( + fun() -> + case mnesia:wread({rabbit_exchange, XName}) of + [X] -> DeletionFun(X, false); + [] -> {error, not_found} + end + end). + +%% ------------------------------------------------------------------- +%% delete_serial(). +%% ------------------------------------------------------------------- + +-spec delete_serial(ExchangeName) -> ok when + ExchangeName :: name(). +%% @doc Deletes an exchange serial record from the database. +%% +%% @returns ok +%% +%% @private + +delete_serial(XName) -> + rabbit_db:run( + #{mnesia => fun() -> delete_serial_in_mnesia(XName) end + }). + +delete_serial_in_mnesia(XName) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + mnesia:delete({rabbit_exchange_serial, XName}) + end). + +%% ------------------------------------------------------------------- +%% recover(). +%% ------------------------------------------------------------------- + +-spec recover(VHostName) -> ok when + VHostName :: vhost:name(). +%% @doc Recovers all exchanges for a given vhost +%% +%% @returns ok +%% +%% @private + +recover(VHost) -> + rabbit_db:run( + #{mnesia => fun() -> recover_in_mnesia(VHost) end + }). + +%% ------------------------------------------------------------------- +%% match(). +%% ------------------------------------------------------------------- + +-spec match(Pattern) -> [Exchange] when + Pattern :: #exchange{}, + Exchange :: rabbit_types:exchange(). +%% @doc Returns all exchanges that match a given pattern +%% +%% @returns a list of exchange records +%% +%% @private + +match(Pattern) -> + rabbit_db:run( + #{mnesia => fun() -> match_in_mnesia(Pattern) end + }). + +match_in_mnesia(Pattern) -> + case mnesia:transaction( + fun() -> + mnesia:match_object(rabbit_exchange, Pattern, read) + end) of + {atomic, Xs} -> Xs; + {aborted, Err} -> {error, Err} + end. + +%% ------------------------------------------------------------------- +%% exists(). +%% ------------------------------------------------------------------- + +-spec exists(ExchangeName) -> Exists when + ExchangeName :: name(), + Exists :: boolean(). +%% @doc Indicates if the exchange named `Name' exists. +%% +%% @returns true if the exchange exists, false otherwise. +%% +%% @private + +exists(Name) -> + rabbit_db:run( + #{mnesia => fun() -> exists_in_mnesia(Name) end}). + +exists_in_mnesia(Name) -> + ets:member(rabbit_exchange, Name). + +%% Internal +%% -------------------------------------------------------------- + +peek_serial_in_mnesia_tx(XName, LockType) -> + case mnesia:read(rabbit_exchange_serial, XName, LockType) of + [#exchange_serial{next = Serial}] -> Serial; + _ -> 1 + end. + +next_serial_in_mnesia_tx(#exchange{name = XName}) -> + Serial = peek_serial_in_mnesia_tx(XName, write), + ok = mnesia:write(rabbit_exchange_serial, + #exchange_serial{name = XName, next = Serial + 1}, write), + Serial. + +update_in_mnesia_tx(Name, Fun) -> + Table = {rabbit_exchange, Name}, + case mnesia:wread(Table) of + [X] -> X1 = Fun(X), + insert_in_mnesia_tx(X1); + [] -> not_found + end. + +delete_in_mnesia(X = #exchange{name = XName}, OnlyDurable, RemoveBindingsForSource) -> + ok = mnesia:delete({rabbit_exchange, XName}), + mnesia:delete({rabbit_durable_exchange, XName}), + rabbit_db_binding:delete_all_for_exchange_in_mnesia(X, OnlyDurable, RemoveBindingsForSource). + +get_many_in_mnesia(Table, [Name]) -> ets:lookup(Table, Name); +get_many_in_mnesia(Table, Names) when is_list(Names) -> + %% Normally we'd call mnesia:dirty_read/1 here, but that is quite + %% expensive for reasons explained in rabbit_misc:dirty_read/1. + lists:append([ets:lookup(Table, Name) || Name <- Names]). + +conditional_delete_in_mnesia(X = #exchange{name = XName}, OnlyDurable) -> + case rabbit_db_binding:has_for_source_in_mnesia(XName) of + false -> delete_in_mnesia(X, OnlyDurable, false); + true -> {error, in_use} + end. + +unconditional_delete_in_mnesia(X, OnlyDurable) -> + delete_in_mnesia(X, OnlyDurable, true). + +-spec maybe_auto_delete_in_mnesia + (rabbit_types:exchange(), boolean()) + -> 'not_deleted' | {'deleted', rabbit_binding:deletions()}. +maybe_auto_delete_in_mnesia(XName, OnlyDurable) -> + case mnesia:read({case OnlyDurable of + true -> rabbit_durable_exchange; + false -> rabbit_exchange + end, XName}) of + [] -> {not_deleted, undefined}; + [#exchange{auto_delete = false} = X] -> {not_deleted, X}; + [#exchange{auto_delete = true} = X] -> + case conditional_delete_in_mnesia(X, OnlyDurable) of + {error, in_use} -> {not_deleted, X}; + {deleted, X, [], Deletions} -> {deleted, X, Deletions} + end + end. + +insert_in_mnesia_tx(X = #exchange{durable = true}) -> + mnesia:write(rabbit_durable_exchange, X#exchange{decorators = undefined}, + write), + insert_ram_in_mnesia_tx(X); +insert_in_mnesia_tx(X = #exchange{durable = false}) -> + insert_ram_in_mnesia_tx(X). + +insert_ram_in_mnesia_tx(X) -> + X1 = rabbit_exchange_decorator:set(X), + ok = mnesia:write(rabbit_exchange, X1, write), + X1. + +recover_in_mnesia(VHost) -> + rabbit_misc:table_filter( + fun (#exchange{name = XName}) -> + XName#resource.virtual_host =:= VHost andalso + mnesia:read({rabbit_exchange, XName}) =:= [] + end, + fun (X, true) -> + X; + (X, false) -> + X1 = rabbit_misc:execute_mnesia_transaction( + fun() -> insert_in_mnesia_tx(X) end), + Serial = rabbit_exchange:serial(X1), + rabbit_exchange:callback(X1, create, Serial, [X1]) + end, + rabbit_durable_exchange). diff --git a/deps/rabbit/src/rabbit_db_policy.erl b/deps/rabbit/src/rabbit_db_policy.erl new file mode 100644 index 000000000000..50317381e1ab --- /dev/null +++ b/deps/rabbit/src/rabbit_db_policy.erl @@ -0,0 +1,63 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_policy). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include("amqqueue.hrl"). + +-export([update/3]). + +-spec update(VHostName, UpdateXFun, UpdateQFun) -> Ret when + VHostName :: vhost:name(), + Exchange :: rabbit_types:exchange(), + Queue :: amqqueue:amqqueue(), + UpdateXFun :: fun((Exchange) -> Exchange), + UpdateQFun :: fun((Queue) -> Queue), + Ret :: {[{Exchange, Exchange}], [{Queue, Queue}]}. + +update(VHost, GetUpdatedExchangeFun, GetUpdatedQueueFun) -> + rabbit_db:run( + #{mnesia => fun() -> update_in_mnesia(VHost, GetUpdatedExchangeFun, GetUpdatedQueueFun) end + }). + +%% [1] We need to prevent this from becoming O(n^2) in a similar +%% manner to rabbit_binding:remove_for_{source,destination}. So see +%% the comment in rabbit_binding:lock_route_tables/0 for more rationale. +update_in_mnesia(VHost, GetUpdatedExchangeFun, GetUpdatedQueueFun) -> + Tabs = [rabbit_queue, rabbit_durable_queue, + rabbit_exchange, rabbit_durable_exchange], + rabbit_misc:execute_mnesia_transaction( + fun() -> + [mnesia:lock({table, T}, write) || T <- Tabs], %% [1] + Exchanges0 = rabbit_db_exchange:get_all(VHost), + Queues0 = rabbit_db_queue:get_all(VHost), + Exchanges = [GetUpdatedExchangeFun(X) || X <- Exchanges0], + Queues = [GetUpdatedQueueFun(Q) || Q <- Queues0], + {[update_exchange_policies(Map, fun rabbit_db_exchange:update_in_mnesia_tx/2) + || Map <- Exchanges, is_map(Map)], + [update_queue_policies(Map, fun rabbit_db_queue:update_in_mnesia_tx/2) + || Map <- Queues, is_map(Map)]} + end). + +update_exchange_policies(#{exchange := X = #exchange{name = XName}, + update_function := UpdateFun}, StoreFun) -> + NewExchange = StoreFun(XName, UpdateFun), + case NewExchange of + #exchange{} = X1 -> {X, X1}; + not_found -> {X, X } + end. + +update_queue_policies(#{queue := Q0, update_function := UpdateFun}, StoreFun) -> + QName = amqqueue:get_name(Q0), + NewQueue = StoreFun(QName, UpdateFun), + case NewQueue of + Q1 when ?is_amqqueue(Q1) -> + {Q0, Q1}; + not_found -> + {Q0, Q0} + end. diff --git a/deps/rabbit/src/rabbit_db_queue.erl b/deps/rabbit/src/rabbit_db_queue.erl new file mode 100644 index 000000000000..811e0f0ae569 --- /dev/null +++ b/deps/rabbit/src/rabbit_db_queue.erl @@ -0,0 +1,540 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_queue). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("stdlib/include/qlc.hrl"). +-include("amqqueue.hrl"). + +-export([ + get/1, + get_all/0, + get_all/1, + get_all_by_type/1, + get_all_by_type_and_node/3, + list/0, + count/0, + count/1, + create_or_get/2, + insert/2, + insert/1, + delete/2, + update/2, + update_decorators/1, + exists/1 + ]). + +-export([ + get_all_durable/0, + get_all_durable/1, + get_all_durable_by_type/1, + get_durable/1 + ]). + +-export([delete_transient/1]). +-export([on_node_up/2, + on_node_down/2]). + +-export([match_and_update/3]). +-export([insert_dirty/1]). + +-export([not_found_or_absent_queue_dirty/1]). + +-export([internal_delete/3]). + +%% Used by other rabbit_db_* modules +-export([ + update_in_mnesia_tx/2, + not_found_or_absent_queue_in_mnesia/1 + ]). + +%% ------------------------------------------------------------------- +%% get_all(). +%% ------------------------------------------------------------------- + +-spec get_all() -> [Queue] when + Queue :: amqqueue:amqqueue(). + +%% @doc Returns all queue records. +%% +%% @returns the list of all queue records. +%% +%% @private + +get_all() -> + rabbit_db:run( + #{mnesia => fun() -> get_all_in_mnesia() end + }). + +get_all_in_mnesia() -> + list_with_possible_retry_in_mnesia( + fun() -> + rabbit_db:list_in_mnesia(rabbit_queue, amqqueue:pattern_match_all()) + end). + +-spec get_all(VHostName) -> [Queue] when + VHostName :: vhost:name(), + Queue :: amqqueue:amqqueue(). + +%% @doc Gets all queues belonging to the given virtual host +%% +%% @returns a list of queue records. +%% +%% @private + +get_all(VHost) -> + rabbit_db:run( + #{mnesia => fun() -> get_all_in_mnesia(VHost) end + }). + +%% ------------------------------------------------------------------- +%% get_all_durable(). +%% ------------------------------------------------------------------- + +-spec get_all_durable() -> [Queue] when + Queue :: amqqueue:amqqueue(). + +%% @doc Returns all durable queue records. +%% +%% @returns a list of queue records. +%% +%% @private + +get_all_durable() -> + rabbit_db:run( + #{mnesia => fun() -> get_all_durable_in_mnesia() end + }). + +get_all_durable_in_mnesia() -> + list_with_possible_retry_in_mnesia( + fun() -> + rabbit_db:list_in_mnesia(rabbit_durable_queue, amqqueue:pattern_match_all()) + end). + +-spec get_all_durable(VHostName) -> [Queue] when + VHostName :: vhost:name(), + Queue :: amqqueue:amqqueue(). + +%% @doc Gets all durable queues belonging to the given virtual host +%% +%% @returns a list of queue records. +%% +%% @private + +get_all_durable(VHost) -> + rabbit_db:run( + #{mnesia => fun() -> get_all_durable_in_mnesia(VHost) end + }). + +get_all_durable_in_mnesia(VHost) -> + list_with_possible_retry_in_mnesia( + fun() -> + Pattern = amqqueue:pattern_match_on_name(rabbit_misc:r(VHost, queue)), + rabbit_db:list_in_mnesia(rabbit_durable_queue, Pattern) + end). + +get_all_durable_by_type(Type) -> + rabbit_db:run( + #{mnesia => fun() -> get_all_durable_by_type_in_mnesia(Type) end + }). + +get_all_durable_by_type_in_mnesia(Type) -> + Pattern = amqqueue:pattern_match_on_type(Type), + rabbit_db:list_in_mnesia(rabbit_durable_queue, Pattern). + +list() -> + rabbit_db:run( + #{mnesia => fun() -> list_in_mnesia() end + }). + +list_in_mnesia() -> + mnesia:dirty_all_keys(rabbit_queue). + +count() -> + rabbit_db:run( + #{mnesia => fun() -> count_in_mnesia() end + }). + +count_in_mnesia() -> + mnesia:table_info(rabbit_queue, size). + +count(VHost) -> + try + list_for_count(VHost) + catch _:Err -> + rabbit_log:error("Failed to fetch number of queues in vhost ~p:~n~p", + [VHost, Err]), + 0 + end. + +delete(QueueName, Reason) -> + rabbit_db:run( + #{mnesia => fun() -> delete_in_mnesia(QueueName, Reason) end + }). + +internal_delete(QueueName, OnlyDurable, Reason) -> + %% Only used by rabbit_amqqueue:forget_node_for_queue, which is only called + %% by `rabbit_mnesia:remove_node_if_mnesia_running`. Thus, once mnesia and/or + %% HA queues are removed it can be removed. + rabbit_db:run( + #{mnesia => fun() -> internal_delete_in_mnesia(QueueName, OnlyDurable, Reason) end + }). + +get(Names) when is_list(Names) -> + rabbit_db:run( + #{mnesia => fun() -> get_many_in_mnesia(rabbit_queue, Names) end + }); +get(Name) -> + rabbit_db:run( + #{mnesia => fun() -> get_in_mnesia(Name) end + }). + +get_in_mnesia(Name) -> + rabbit_misc:dirty_read({rabbit_queue, Name}). + +get_durable(Names) when is_list(Names) -> + rabbit_db:run( + #{mnesia => fun() -> get_many_in_mnesia(rabbit_durable_queue, Names) end + }); +get_durable(Name) -> + rabbit_db:run( + #{mnesia => fun() -> get_durable_in_mnesia(Name) end + }). + +get_durable_in_mnesia(Name) -> + rabbit_misc:dirty_read({rabbit_durable_queue, Name}). + +delete_transient(Queues) -> + rabbit_db:run( + #{mnesia => fun() -> delete_transient_in_mnesia(Queues) end + }). + +delete_transient_in_mnesia(Queues) -> + rabbit_misc:execute_mnesia_transaction( + fun () -> + [{QName, delete_transient_in_mnesia_tx(QName)} + || QName <- Queues] + end). + +on_node_up(Node, Fun) -> + rabbit_db:run( + #{mnesia => fun() -> on_node_up_in_mnesia(Node, Fun) end + }). + +on_node_up_in_mnesia(Node, Fun) -> + rabbit_misc:execute_mnesia_transaction( + fun () -> + Qs = mnesia:match_object(rabbit_queue, + amqqueue:pattern_match_all(), write), + [Fun(Node, Q) || Q <- Qs], + ok + end). + +on_node_down(Node, Fun) -> + rabbit_db:run( + #{mnesia => fun() -> on_node_down_in_mnesia(Node, Fun) end + }). + +on_node_down_in_mnesia(Node, Fun) -> + Qs = rabbit_misc:execute_mnesia_transaction( + fun () -> + qlc:e(qlc:q([amqqueue:get_name(Q) || Q <- mnesia:table(rabbit_queue), + Fun(Node, Q) + ])) + end), + lists:unzip(lists:flatten( + [case delete_transient(Queues) of + {error, noproc} -> []; + {error, {timeout, _}} -> []; + Value -> Value + end || Queues <- partition_queues(Qs)] + )). + +% If there are many queues and we delete them all in a single Mnesia transaction, +% this can block all other Mnesia operations for a really long time. +% In situations where a node wants to (re-)join a cluster, +% Mnesia won't be able to sync on the new node until this operation finishes. +% As a result, we want to have multiple Mnesia transactions so that other +% operations can make progress in between these queue delete transactions. +% +% 10 queues per Mnesia transaction is an arbitrary number, but it seems to work OK with 50k queues per node. +partition_queues([Q0,Q1,Q2,Q3,Q4,Q5,Q6,Q7,Q8,Q9 | T]) -> + [[Q0,Q1,Q2,Q3,Q4,Q5,Q6,Q7,Q8,Q9] | partition_queues(T)]; +partition_queues(T) -> + [T]. + +update(QName, Fun) -> + rabbit_db:run( + #{mnesia => fun() -> update_in_mnesia(QName, Fun) end + }). + +update_in_mnesia(QName, Fun) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + update_in_mnesia_tx(QName, Fun) + end). + +update_decorators(Name) -> + rabbit_db:run( + #{mnesia => fun() -> update_decorators_in_mnesia(Name) end + }). + +not_found_or_absent_queue_dirty(Name) -> + rabbit_db:run( + #{mnesia => fun() -> not_found_or_absent_queue_dirty_in_mnesia(Name) end + }). + +exists(Name) -> + rabbit_db:run( + #{mnesia => fun() -> exists_in_mnesia(Name) end + }). + +exists_in_mnesia(Name) -> + ets:member(rabbit_queue, Name). + +get_all_by_type(Type) -> + Pattern = amqqueue:pattern_match_on_type(Type), + rabbit_db:run( + #{mnesia => fun() -> get_all_by_pattern_in_mnesia(Pattern) end + }). + +get_all_by_pattern_in_mnesia(Pattern) -> + rabbit_db:list_in_mnesia(rabbit_queue, Pattern). + +get_all_by_type_and_node(VHost, Type, Node) -> + rabbit_db:run( + #{mnesia => fun() -> get_all_by_type_and_node_in_mnesia(VHost, Type, Node) end + }). + +get_all_by_type_and_node_in_mnesia(VHost, Type, Node) -> + mnesia:async_dirty( + fun () -> + qlc:e(qlc:q([Q || Q <- mnesia:table(rabbit_durable_queue), + amqqueue:get_type(Q) =:= Type, + amqqueue:get_vhost(Q) =:= VHost, + amqqueue:qnode(Q) == Node])) + end). + +create_or_get(DurableQ, Q) -> + rabbit_db:run( + #{mnesia => fun() -> create_or_get_in_mnesia(DurableQ, Q) end + }). + +create_or_get_in_mnesia(DurableQ, Q) -> + QueueName = amqqueue:get_name(Q), + rabbit_misc:execute_mnesia_transaction( + fun () -> + case mnesia:wread({rabbit_queue, QueueName}) of + [] -> + case not_found_or_absent_queue_in_mnesia(QueueName) of + not_found -> + insert_in_mnesia_tx(DurableQ, Q), + {created, Q}; + {absent, _Q, _} = R -> + R + end; + [ExistingQ] -> + {existing, ExistingQ} + end + end). + +insert(DurableQ, Q) -> + rabbit_db:run( + #{mnesia => fun() -> insert_in_mnesia(DurableQ, Q) end + }). + +insert_in_mnesia(DurableQ, Q) -> + rabbit_misc:execute_mnesia_transaction( + fun () -> + insert_in_mnesia_tx(DurableQ, Q) + end). + +insert(Qs) -> + rabbit_db:run( + #{mnesia => fun() -> insert_many_in_mnesia(Qs) end + }). + +insert_many_in_mnesia(Qs) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + [ok = mnesia:write(rabbit_durable_queue, Q, write) || Q <- Qs] + end). + +match_and_update(Pattern, UpdateFun, FilterFun) -> + rabbit_db:run( + #{mnesia => + fun() -> match_and_update_in_mnesia(Pattern, UpdateFun, FilterFun) end + }). + +match_and_update_in_mnesia(Pattern, UpdateFun, FilterFun) -> + %% Note rabbit is not running so we avoid e.g. the worker pool. Also why + %% we don't invoke the return from rabbit_binding:process_deletions/1. + {atomic, ok} = + mnesia:sync_transaction( + fun () -> + Qs = mnesia:match_object(rabbit_durable_queue, Pattern, write), + _ = [UpdateFun(Q) || Q <- Qs, FilterFun(Q)], + ok + end), + ok. + +insert_dirty(Q) -> + rabbit_db:run( + #{mnesia => fun() -> insert_dirty_in_mnesia(Q) end + }). + +insert_dirty_in_mnesia(Q) -> + ok = mnesia:dirty_write(rabbit_queue, rabbit_queue_decorator:set(Q)). + +update_in_mnesia_tx(Name, Fun) -> + case mnesia:wread({rabbit_queue, Name}) of + [Q] -> + Durable = amqqueue:is_durable(Q), + Q1 = Fun(Q), + ok = mnesia:write(rabbit_queue, Q1, write), + case Durable of + true -> ok = mnesia:write(rabbit_durable_queue, Q1, write); + _ -> ok + end, + Q1; + [] -> + not_found + end. + +not_found_or_absent_queue_in_mnesia(Name) -> + %% NB: we assume that the caller has already performed a lookup on + %% rabbit_queue and not found anything + case mnesia:read({rabbit_durable_queue, Name}) of + [] -> not_found; + [Q] -> {absent, Q, nodedown} %% Q exists on stopped node + end. + +%% Internal +%% -------------------------------------------------------------- +get_many_in_mnesia(Table, [Name]) -> + ets:lookup(Table, Name); +get_many_in_mnesia(Table, Names) when is_list(Names) -> + %% Normally we'd call mnesia:dirty_read/1 here, but that is quite + %% expensive for reasons explained in rabbit_misc:dirty_read/1. + lists:append([ets:lookup(Table, Name) || Name <- Names]). + +delete_transient_in_mnesia_tx(QName) -> + ok = mnesia:delete({rabbit_queue, QName}), + rabbit_db_binding:delete_transient_for_destination_in_mnesia(QName). + +get_all_in_mnesia(VHost) -> + list_with_possible_retry_in_mnesia( + fun() -> + Pattern = amqqueue:pattern_match_on_name(rabbit_misc:r(VHost, queue)), + rabbit_db:list_in_mnesia(rabbit_queue, Pattern) + end). + +not_found_or_absent_queue_dirty_in_mnesia(Name) -> + %% We should read from both tables inside a tx, to get a + %% consistent view. But the chances of an inconsistency are small, + %% and only affect the error kind. + case rabbit_misc:dirty_read({rabbit_durable_queue, Name}) of + {error, not_found} -> not_found; + {ok, Q} -> {absent, Q, nodedown} + end. + +list_with_possible_retry_in_mnesia(Fun) -> + %% amqqueue migration: + %% The `rabbit_queue` or `rabbit_durable_queue` tables + %% might be migrated between the time we query the pattern + %% (with the `amqqueue` module) and the time we call + %% `mnesia:dirty_match_object()`. This would lead to an empty list + %% (no object matching the now incorrect pattern), not a Mnesia + %% error. + %% + %% So if the result is an empty list and the version of the + %% `amqqueue` record changed in between, we retry the operation. + %% + %% However, we don't do this if inside a Mnesia transaction: we + %% could end up with a live lock between this started transaction + %% and the Mnesia table migration which is blocked (but the + %% rabbit_feature_flags lock is held). + AmqqueueRecordVersion = amqqueue:record_version_to_use(), + case Fun() of + [] -> + case mnesia:is_transaction() of + true -> + []; + false -> + case amqqueue:record_version_to_use() of + AmqqueueRecordVersion -> []; + _ -> Fun() + end + end; + Ret -> + Ret + end. + +delete_in_mnesia(QueueName, Reason) -> + rabbit_misc:execute_mnesia_transaction( + fun () -> + case {mnesia:wread({rabbit_queue, QueueName}), + mnesia:wread({rabbit_durable_queue, QueueName})} of + {[], []} -> + ok; + _ -> + internal_delete_in_mnesia(QueueName, false, Reason) + end + end). + +internal_delete_in_mnesia(QueueName, OnlyDurable, Reason) -> + ok = mnesia:delete({rabbit_queue, QueueName}), + case Reason of + auto_delete -> + case mnesia:wread({rabbit_durable_queue, QueueName}) of + [] -> ok; + [_] -> ok = mnesia:delete({rabbit_durable_queue, QueueName}) + end; + _ -> + mnesia:delete({rabbit_durable_queue, QueueName}) + end, + %% we want to execute some things, as decided by rabbit_exchange, + %% after the transaction. + rabbit_db_binding:delete_for_destination_in_mnesia(QueueName, OnlyDurable). + +list_for_count(VHost) -> + rabbit_db:run( + #{mnesia => fun() -> list_for_count_in_mnesia(VHost) end + }). + +list_for_count_in_mnesia(VHost) -> + %% this is certainly suboptimal but there is no way to count + %% things using a secondary index in Mnesia. Our counter-table-per-node + %% won't work here because with master migration of mirrored queues + %% the "ownership" of queues by nodes becomes a non-trivial problem + %% that requires a proper consensus algorithm. + list_with_possible_retry_in_mnesia( + fun() -> + length(mnesia:dirty_index_read(rabbit_queue, + VHost, + amqqueue:field_vhost())) + end). + +update_decorators_in_mnesia(Name) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + case mnesia:wread({rabbit_queue, Name}) of + [Q] -> ok = mnesia:write(rabbit_queue, rabbit_queue_decorator:set(Q), + write); + [] -> ok + end + end). + +insert_in_mnesia_tx(DurableQ, Q) -> + case ?amqqueue_is_durable(Q) of + true -> + ok = mnesia:write(rabbit_durable_queue, DurableQ, write); + false -> + ok + end, + ok = mnesia:write(rabbit_queue, Q, write). diff --git a/deps/rabbit/src/rabbit_db_topic_exchange.erl b/deps/rabbit/src/rabbit_db_topic_exchange.erl index a5282c1a0944..d58e5597fe7e 100644 --- a/deps/rabbit/src/rabbit_db_topic_exchange.erl +++ b/deps/rabbit/src/rabbit_db_topic_exchange.erl @@ -2,7 +2,7 @@ %% License, v. 2.0. If a copy of the MPL was not distributed with this %% file, You can obtain one at https://mozilla.org/MPL/2.0/. %% -%% Copyright (c) 2023 VMware, Inc. or its affiliates. All rights reserved. +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. %% -module(rabbit_db_topic_exchange). diff --git a/deps/rabbit/src/rabbit_exchange.erl b/deps/rabbit/src/rabbit_exchange.erl index 5d153a0b028e..bcc04c99f45b 100644 --- a/deps/rabbit/src/rabbit_exchange.erl +++ b/deps/rabbit/src/rabbit_exchange.erl @@ -12,12 +12,13 @@ -export([recover/1, policy_changed/2, callback/4, declare/7, assert_equivalence/6, assert_args_equivalence/2, check_type/1, exists/1, lookup/1, lookup_many/1, lookup_or_die/1, list/0, list/1, lookup_scratch/2, - update_scratch/3, update_decorators/1, immutable/1, + update_scratch/3, update_decorators/2, immutable/1, info_keys/0, info/1, info/2, info_all/1, info_all/2, info_all/4, route/2, delete/3, validate_binding/2, count/0]). -export([list_names/0]). +-export([serialise_events/1]). %% these must be run inside a mnesia tx --export([maybe_auto_delete/2, serial/1, peek_serial/1, update/2]). +-export([serial/1, peek_serial/1]). %%---------------------------------------------------------------------------- @@ -35,32 +36,29 @@ -spec recover(rabbit_types:vhost()) -> [name()]. recover(VHost) -> - Xs = rabbit_misc:table_filter( - fun (#exchange{name = XName}) -> - XName#resource.virtual_host =:= VHost andalso - mnesia:read({rabbit_exchange, XName}) =:= [] - end, - fun (X, Tx) -> - X1 = case Tx of - true -> store_ram(X); - false -> rabbit_exchange_decorator:set(X) - end, - callback(X1, create, map_create_tx(Tx), [X1]) - end, - rabbit_durable_exchange), + Xs = rabbit_db_exchange:recover(VHost), [XName || #exchange{name = XName} <- Xs]. -spec callback - (rabbit_types:exchange(), fun_name(), - fun((boolean()) -> non_neg_integer()) | atom(), [any()]) -> 'ok'. + (rabbit_types:exchange(), fun_name(), atom(), [any()]) -> 'ok'. -callback(X = #exchange{type = XType, - decorators = Decorators}, Fun, Serial0, Args) -> - Serial = if is_function(Serial0) -> Serial0; - is_atom(Serial0) -> fun (_Bool) -> Serial0 end +callback(X = #exchange{decorators = Decorators, name = XName}, Fun, Serial, Args) -> + case Fun of + delete -> rabbit_db_exchange:delete_serial(XName); + _ -> ok + end, + Modules = rabbit_exchange_decorator:select(all, Decorators), + callback0(X, Fun, Serial, Modules, Args), + ok. + +callback0(#exchange{type = XType}, Fun, Serial, Modules0, Args) when is_atom(Serial) -> + Modules = Modules0 ++ [type_to_module(XType)], + [ok = apply(M, Fun, [Serial | Args]) || M <- Modules]; +callback0(#exchange{type = XType} = X, Fun, Serial0, Modules, Args) -> + Serial = fun(true) -> Serial0; + (false) -> none end, - [ok = apply(M, Fun, [Serial(M:serialise_events(X)) | Args]) || - M <- rabbit_exchange_decorator:select(all, Decorators)], + [ok = apply(M, Fun, [Serial(M:serialise_events(X)) | Args]) || M <- Modules], Module = type_to_module(XType), apply(Module, Fun, [Serial(Module:serialise_events()) | Args]). @@ -84,13 +82,10 @@ serialise_events(X = #exchange{type = Type, decorators = Decorators}) -> -spec serial(rabbit_types:exchange()) -> fun((boolean()) -> 'none' | pos_integer()). -serial(#exchange{name = XName} = X) -> - Serial = case serialise_events(X) of - true -> next_serial(XName); - false -> none - end, - fun (true) -> Serial; - (false) -> none +serial(X) -> + case serialise_events(X) of + false -> 'none'; + true -> rabbit_db_exchange:next_serial(X) end. -spec declare @@ -118,46 +113,23 @@ declare(XName, Type, Durable, AutoDelete, Internal, Args, Username) -> ?EXCHANGE_DELETE_IN_PROGRESS_COMPONENT, XName#resource.name) of not_found -> - rabbit_misc:execute_mnesia_transaction( - fun () -> - case mnesia:wread({rabbit_exchange, XName}) of - [] -> - {new, store(X)}; - [ExistingX] -> - {existing, ExistingX} - end - end, - fun ({new, Exchange}, Tx) -> - ok = callback(X, create, map_create_tx(Tx), [Exchange]), - rabbit_event:notify_if(not Tx, exchange_created, info(Exchange)), - Exchange; - ({existing, Exchange}, _Tx) -> - Exchange; - (Err, _Tx) -> - Err - end); + case rabbit_db_exchange:create_or_get(X) of + {new, Exchange} -> + Serial = serial(Exchange), + ok = callback(X, create, Serial, [Exchange]), + rabbit_event:notify(exchange_created, info(Exchange)), + Exchange; + {existing, Exchange} -> + Exchange; + Err -> + Err + end; _ -> rabbit_log:warning("ignoring exchange.declare for exchange ~tp, exchange.delete in progress~n.", [XName]), X end. -map_create_tx(true) -> transaction; -map_create_tx(false) -> none. - - -store(X = #exchange{durable = true}) -> - mnesia:write(rabbit_durable_exchange, X#exchange{decorators = undefined}, - write), - store_ram(X); -store(X = #exchange{durable = false}) -> - store_ram(X). - -store_ram(X) -> - X1 = rabbit_exchange_decorator:set(X), - ok = mnesia:write(rabbit_exchange, X1, write), - X1. - %% Used with binaries sent over the wire; the type may not exist. -spec check_type @@ -209,25 +181,21 @@ assert_args_equivalence(#exchange{ name = Name, arguments = Args }, -spec exists(name()) -> boolean(). exists(Name) -> - ets:member(rabbit_exchange, Name). + rabbit_db_exchange:exists(Name). -spec lookup (name()) -> rabbit_types:ok(rabbit_types:exchange()) | rabbit_types:error('not_found'). lookup(Name) -> - rabbit_misc:dirty_read({rabbit_exchange, Name}). - + rabbit_db_exchange:get(Name). -spec lookup_many([name()]) -> [rabbit_types:exchange()]. -lookup_many([]) -> []; -lookup_many([Name]) -> ets:lookup(rabbit_exchange, Name); -lookup_many(Names) when is_list(Names) -> - %% Normally we'd call mnesia:dirty_read/1 here, but that is quite - %% expensive for reasons explained in rabbit_misc:dirty_read/1. - lists:append([ets:lookup(rabbit_exchange, Name) || Name <- Names]). - +lookup_many([]) -> + []; +lookup_many(Names) -> + rabbit_db_exchange:get_many(Names). -spec lookup_or_die (name()) -> rabbit_types:exchange() | @@ -241,16 +209,18 @@ lookup_or_die(Name) -> -spec list() -> [rabbit_types:exchange()]. -list() -> mnesia:dirty_match_object(rabbit_exchange, #exchange{_ = '_'}). +list() -> + rabbit_db_exchange:get_all(). -spec count() -> non_neg_integer(). count() -> - mnesia:table_info(rabbit_exchange, size). + rabbit_db_exchange:get_all(). -spec list_names() -> [rabbit_exchange:name()]. -list_names() -> mnesia:dirty_all_keys(rabbit_exchange). +list_names() -> + rabbit_db_exchange:list(). %% Not dirty_match_object since that would not be transactional when used in a %% tx context @@ -258,13 +228,7 @@ list_names() -> mnesia:dirty_all_keys(rabbit_exchange). -spec list(rabbit_types:vhost()) -> [rabbit_types:exchange()]. list(VHostPath) -> - mnesia:async_dirty( - fun () -> - mnesia:match_object( - rabbit_exchange, - #exchange{name = rabbit_misc:r(VHostPath, exchange), _ = '_'}, - read) - end). + rabbit_db_exchange:get_all(VHostPath). -spec lookup_scratch(name(), atom()) -> rabbit_types:ok(term()) | @@ -286,49 +250,34 @@ lookup_scratch(Name, App) -> -spec update_scratch(name(), atom(), fun((any()) -> any())) -> 'ok'. update_scratch(Name, App, Fun) -> - rabbit_misc:execute_mnesia_transaction( - fun() -> - _ = update(Name, - fun(X = #exchange{scratches = Scratches0}) -> - Scratches1 = case Scratches0 of - undefined -> orddict:new(); - _ -> Scratches0 - end, - Scratch = case orddict:find(App, Scratches1) of - {ok, S} -> S; - error -> undefined - end, - Scratches2 = orddict:store( - App, Fun(Scratch), Scratches1), - X#exchange{scratches = Scratches2} - end), - ok - end). - --spec update_decorators(name()) -> 'ok'. - -update_decorators(Name) -> - rabbit_misc:execute_mnesia_transaction( - fun() -> - case mnesia:wread({rabbit_exchange, Name}) of - [X] -> _ = store_ram(X), - ok; - [] -> ok - end - end). - --spec update - (name(), - fun((rabbit_types:exchange()) -> rabbit_types:exchange())) - -> not_found | rabbit_types:exchange(). - -update(Name, Fun) -> - case mnesia:wread({rabbit_exchange, Name}) of - [X] -> X1 = Fun(X), - store(X1); - [] -> not_found + Decorators = case rabbit_db_exchange:get(Name) of + {ok, X} -> rabbit_exchange_decorator:active(X); + {error, not_found} -> [] + end, + rabbit_db_exchange:update(Name, update_scratch_fun(App, Fun, Decorators)), + ok. + +update_scratch_fun(App, Fun, Decorators) -> + fun(X = #exchange{scratches = Scratches0}) -> + Scratches1 = case Scratches0 of + undefined -> orddict:new(); + _ -> Scratches0 + end, + Scratch = case orddict:find(App, Scratches1) of + {ok, S} -> S; + error -> undefined + end, + Scratches2 = orddict:store(App, Fun(Scratch), Scratches1), + X#exchange{scratches = Scratches2, + decorators = Decorators} end. +-spec update_decorators(name(), [atom()] | none | undefined) -> 'ok'. + +update_decorators(Name, Decorators) -> + Fun = fun(X) -> X#exchange{decorators = Decorators} end, + rabbit_db_exchange:update(Name, Fun). + -spec immutable(rabbit_types:exchange()) -> rabbit_types:exchange(). immutable(X) -> X#exchange{scratches = none, @@ -467,14 +416,6 @@ cons_if_present(XName, L) -> {error, not_found} -> L end. -call_with_exchange(XName, Fun) -> - rabbit_misc:execute_mnesia_tx_with_tail( - fun () -> case mnesia:read({rabbit_exchange, XName}) of - [] -> rabbit_misc:const({error, not_found}); - [X] -> Fun(X) - end - end). - -spec delete (name(), 'true', rabbit_types:username()) -> 'ok'| rabbit_types:error('not_found' | 'in_use'); @@ -482,10 +423,6 @@ call_with_exchange(XName, Fun) -> 'ok' | rabbit_types:error('not_found'). delete(XName, IfUnused, Username) -> - Fun = case IfUnused of - true -> fun conditional_delete/2; - false -> fun unconditional_delete/2 - end, try %% guard exchange.declare operations from failing when there's %% a race condition between it and an exchange.delete. @@ -494,24 +431,23 @@ delete(XName, IfUnused, Username) -> _ = rabbit_runtime_parameters:set(XName#resource.virtual_host, ?EXCHANGE_DELETE_IN_PROGRESS_COMPONENT, XName#resource.name, true, Username), - call_with_exchange( - XName, - fun (X) -> - case Fun(X, false) of - {deleted, X, Bs, Deletions} -> - rabbit_binding:process_deletions( - rabbit_binding:add_deletion( - XName, {X, deleted, Bs}, Deletions), Username); - {error, _InUseOrNotFound} = E -> - rabbit_misc:const(E) - end - end) + Deletions = process_deletions(rabbit_db_exchange:delete(XName, IfUnused)), + rabbit_binding:notify_deletions(Deletions, Username) after rabbit_runtime_parameters:clear(XName#resource.virtual_host, ?EXCHANGE_DELETE_IN_PROGRESS_COMPONENT, XName#resource.name, Username) end. +process_deletions({error, _} = E) -> + E; +process_deletions({deleted, #exchange{name = XName} = X, Bs, Deletions}) -> + rabbit_binding:process_deletions( + rabbit_binding:add_deletion( + XName, {X, deleted, Bs}, Deletions)); +process_deletions(Deletions) -> + rabbit_binding:process_deletions(Deletions). + -spec validate_binding (rabbit_types:exchange(), rabbit_types:binding()) -> rabbit_types:ok_or_error({'binding_invalid', string(), [any()]}). @@ -520,53 +456,10 @@ validate_binding(X = #exchange{type = XType}, Binding) -> Module = type_to_module(XType), Module:validate_binding(X, Binding). --spec maybe_auto_delete - (rabbit_types:exchange(), boolean()) - -> 'not_deleted' | {'deleted', rabbit_binding:deletions()}. - -maybe_auto_delete(#exchange{auto_delete = false}, _OnlyDurable) -> - not_deleted; -maybe_auto_delete(#exchange{auto_delete = true} = X, OnlyDurable) -> - case conditional_delete(X, OnlyDurable) of - {error, in_use} -> not_deleted; - {deleted, X, [], Deletions} -> {deleted, Deletions} - end. - -conditional_delete(X = #exchange{name = XName}, OnlyDurable) -> - case rabbit_binding:has_for_source(XName) of - false -> internal_delete(X, OnlyDurable, false); - true -> {error, in_use} - end. - -unconditional_delete(X, OnlyDurable) -> - internal_delete(X, OnlyDurable, true). - -internal_delete(X = #exchange{name = XName}, OnlyDurable, RemoveBindingsForSource) -> - ok = mnesia:delete({rabbit_exchange, XName}), - ok = mnesia:delete({rabbit_exchange_serial, XName}), - mnesia:delete({rabbit_durable_exchange, XName}), - Bindings = case RemoveBindingsForSource of - true -> rabbit_binding:remove_for_source(X); - false -> [] - end, - {deleted, X, Bindings, rabbit_binding:remove_for_destination( - XName, OnlyDurable)}. - -next_serial(XName) -> - Serial = peek_serial(XName, write), - ok = mnesia:write(rabbit_exchange_serial, - #exchange_serial{name = XName, next = Serial + 1}, write), - Serial. - -spec peek_serial(name()) -> pos_integer() | 'undefined'. -peek_serial(XName) -> peek_serial(XName, read). - -peek_serial(XName, LockType) -> - case mnesia:read(rabbit_exchange_serial, XName, LockType) of - [#exchange_serial{next = Serial}] -> Serial; - _ -> 1 - end. +peek_serial(XName) -> + rabbit_db_exchange:peek_serial(XName). invalid_module(T) -> rabbit_log:warning("Could not find exchange type ~ts.", [T]), diff --git a/deps/rabbit/src/rabbit_exchange_decorator.erl b/deps/rabbit/src/rabbit_exchange_decorator.erl index c1035539d8ee..3cafc4ee843d 100644 --- a/deps/rabbit/src/rabbit_exchange_decorator.erl +++ b/deps/rabbit/src/rabbit_exchange_decorator.erl @@ -9,7 +9,7 @@ -include_lib("rabbit_common/include/rabbit.hrl"). --export([select/2, set/1]). +-export([select/2, set/1, active/1]). -behaviour(rabbit_registry_class). @@ -24,8 +24,7 @@ %% It's possible in the future we might make decorators %% able to manipulate messages as they are published. --type(tx() :: 'transaction' | 'none'). --type(serial() :: pos_integer() | tx()). +-type(serial() :: pos_integer() | 'none'). -callback description() -> [proplists:property()]. @@ -36,10 +35,10 @@ -callback serialise_events(rabbit_types:exchange()) -> boolean(). %% called after declaration and recovery --callback create(tx(), rabbit_types:exchange()) -> 'ok'. +-callback create(serial(), rabbit_types:exchange()) -> 'ok'. %% called after exchange (auto)deletion. --callback delete(tx(), rabbit_types:exchange(), [rabbit_types:binding()]) -> +-callback delete(serial(), rabbit_types:exchange()) -> 'ok'. %% called when the policy attached to this exchange changes. @@ -87,6 +86,15 @@ set(X) -> end, {[], []}, list()), X#exchange{decorators = Decs}. +%% TODO The list of decorators can probably be a parameter, to avoid multiple queries +%% when we're updating many exchanges +active(X) -> + lists:foldl(fun (D, {Route, NoRoute}) -> + ActiveFor = D:active_for(X), + {cons_if_eq(all, ActiveFor, D, Route), + cons_if_eq(noroute, ActiveFor, D, NoRoute)} + end, {[], []}, list()). + list() -> [M || {_, M} <- rabbit_registry:lookup_all(exchange_decorator)]. cons_if_eq(Select, Select, Item, List) -> [Item | List]; diff --git a/deps/rabbit/src/rabbit_exchange_type_direct.erl b/deps/rabbit/src/rabbit_exchange_type_direct.erl index 62a92ef1ffce..87413e323a95 100644 --- a/deps/rabbit/src/rabbit_exchange_type_direct.erl +++ b/deps/rabbit/src/rabbit_exchange_type_direct.erl @@ -12,7 +12,7 @@ -export([description/0, serialise_events/0, route/2]). -export([validate/1, validate_binding/2, - create/2, delete/3, policy_changed/2, add_binding/3, + create/2, delete/2, policy_changed/2, add_binding/3, remove_bindings/3, assert_args_equivalence/2]). -export([info/1, info/2]). @@ -43,7 +43,7 @@ route(#exchange{name = Name, type = Type}, validate(_X) -> ok. validate_binding(_X, _B) -> ok. create(_Tx, _X) -> ok. -delete(_Tx, _X, _Bs) -> ok. +delete(_Tx, _X) -> ok. policy_changed(_X1, _X2) -> ok. add_binding(_Tx, _X, _B) -> ok. remove_bindings(_Tx, _X, _Bs) -> ok. diff --git a/deps/rabbit/src/rabbit_exchange_type_fanout.erl b/deps/rabbit/src/rabbit_exchange_type_fanout.erl index c69657f26279..0da7a29ab324 100644 --- a/deps/rabbit/src/rabbit_exchange_type_fanout.erl +++ b/deps/rabbit/src/rabbit_exchange_type_fanout.erl @@ -12,7 +12,7 @@ -export([description/0, serialise_events/0, route/2]). -export([validate/1, validate_binding/2, - create/2, delete/3, policy_changed/2, add_binding/3, + create/2, delete/2, policy_changed/2, add_binding/3, remove_bindings/3, assert_args_equivalence/2]). -export([info/1, info/2]). @@ -37,7 +37,7 @@ route(#exchange{name = Name}, _Delivery) -> validate(_X) -> ok. validate_binding(_X, _B) -> ok. create(_Tx, _X) -> ok. -delete(_Tx, _X, _Bs) -> ok. +delete(_Tx, _X) -> ok. policy_changed(_X1, _X2) -> ok. add_binding(_Tx, _X, _B) -> ok. remove_bindings(_Tx, _X, _Bs) -> ok. diff --git a/deps/rabbit/src/rabbit_exchange_type_headers.erl b/deps/rabbit/src/rabbit_exchange_type_headers.erl index a8c12e3225d8..0a89064bec03 100644 --- a/deps/rabbit/src/rabbit_exchange_type_headers.erl +++ b/deps/rabbit/src/rabbit_exchange_type_headers.erl @@ -13,7 +13,7 @@ -export([description/0, serialise_events/0, route/2]). -export([validate/1, validate_binding/2, - create/2, delete/3, policy_changed/2, add_binding/3, + create/2, delete/2, policy_changed/2, add_binding/3, remove_bindings/3, assert_args_equivalence/2]). -export([info/1, info/2]). @@ -142,7 +142,7 @@ headers_match([{PK, _PT, _PV} | PRest], [{DK, _DT, _DV} | DRest], validate(_X) -> ok. create(_Tx, _X) -> ok. -delete(_Tx, _X, _Bs) -> ok. +delete(_Tx, _X) -> ok. policy_changed(_X1, _X2) -> ok. add_binding(_Tx, _X, _B) -> ok. remove_bindings(_Tx, _X, _Bs) -> ok. diff --git a/deps/rabbit/src/rabbit_exchange_type_invalid.erl b/deps/rabbit/src/rabbit_exchange_type_invalid.erl index dcee1c3dc91a..97bfca5c6945 100644 --- a/deps/rabbit/src/rabbit_exchange_type_invalid.erl +++ b/deps/rabbit/src/rabbit_exchange_type_invalid.erl @@ -12,7 +12,7 @@ -export([description/0, serialise_events/0, route/2]). -export([validate/1, validate_binding/2, - create/2, delete/3, policy_changed/2, add_binding/3, + create/2, delete/2, policy_changed/2, add_binding/3, remove_bindings/3, assert_args_equivalence/2]). -export([info/1, info/2]). @@ -37,7 +37,7 @@ route(#exchange{name = Name, type = Type}, _) -> validate(_X) -> ok. validate_binding(_X, _B) -> ok. create(_Tx, _X) -> ok. -delete(_Tx, _X, _Bs) -> ok. +delete(_Tx, _X) -> ok. policy_changed(_X1, _X2) -> ok. add_binding(_Tx, _X, _B) -> ok. remove_bindings(_Tx, _X, _Bs) -> ok. diff --git a/deps/rabbit/src/rabbit_exchange_type_topic.erl b/deps/rabbit/src/rabbit_exchange_type_topic.erl index 37568ed480d5..8ed3f7e1b508 100644 --- a/deps/rabbit/src/rabbit_exchange_type_topic.erl +++ b/deps/rabbit/src/rabbit_exchange_type_topic.erl @@ -13,7 +13,7 @@ -export([description/0, serialise_events/0, route/2]). -export([validate/1, validate_binding/2, - create/2, delete/3, policy_changed/2, add_binding/3, + create/2, delete/2, policy_changed/2, add_binding/3, remove_bindings/3, assert_args_equivalence/2]). -export([info/1, info/2]). @@ -43,22 +43,16 @@ validate(_X) -> ok. validate_binding(_X, _B) -> ok. create(_Tx, _X) -> ok. -delete(transaction, #exchange{name = X}, _Bs) -> - rabbit_db_topic_exchange:delete_all_for_exchange(X); -delete(none, _Exchange, _Bs) -> - ok. +delete(_Serial, #exchange{name = X}) -> + rabbit_db_topic_exchange:delete_all_for_exchange(X). policy_changed(_X1, _X2) -> ok. -add_binding(transaction, _Exchange, Binding) -> - rabbit_db_topic_exchange:insert(Binding); -add_binding(none, _Exchange, _Binding) -> - ok. +add_binding(_Serial, _Exchange, Binding) -> + rabbit_db_topic_exchange:insert(Binding). -remove_bindings(transaction, _X, Bs) -> - rabbit_db_topic_exchange:delete(Bs); -remove_bindings(none, _X, _Bs) -> - ok. +remove_bindings(_Serial, _X, Bs) -> + rabbit_db_topic_exchange:delete(Bs). assert_args_equivalence(X, Args) -> rabbit_exchange:assert_args_equivalence(X, Args). diff --git a/deps/rabbit/src/rabbit_observer_cli_classic_queues.erl b/deps/rabbit/src/rabbit_observer_cli_classic_queues.erl index b4aa34778f7d..505395f15adb 100644 --- a/deps/rabbit/src/rabbit_observer_cli_classic_queues.erl +++ b/deps/rabbit/src/rabbit_observer_cli_classic_queues.erl @@ -86,14 +86,7 @@ sheet_body(State) -> %% This function gets all classic queues regardless of durable/exclusive status. list_classic_queues() -> - {atomic, Qs} = - mnesia:sync_transaction( - fun () -> - mnesia:match_object(rabbit_queue, - amqqueue:pattern_match_on_type(rabbit_classic_queue), - read) - end), - Qs. + rabbit_db_queue:get_all_by_type(rabbit_classic_queue). format_int(N) when N >= 1_000_000_000 -> integer_to_list(N div 1_000_000_000) ++ "B"; diff --git a/deps/rabbit/src/rabbit_policy.erl b/deps/rabbit/src/rabbit_policy.erl index c20eaec34cd8..c09fd5b5a664 100644 --- a/deps/rabbit/src/rabbit_policy.erl +++ b/deps/rabbit/src/rabbit_policy.erl @@ -261,35 +261,24 @@ recover() -> %% recovery has not yet happened; we must work with the rabbit_durable_ %% variants. recover0() -> - Xs = mnesia:dirty_match_object(rabbit_durable_exchange, #exchange{_ = '_'}), - Qs = rabbit_amqqueue:list_with_possible_retry( - fun() -> - mnesia:dirty_match_object( - rabbit_durable_queue, amqqueue:pattern_match_all()) - end), + Xs0 = rabbit_db_exchange:get_all_durable(), Policies = list(), OpPolicies = list_op(), - _ = [rabbit_misc:execute_mnesia_transaction( - fun () -> - mnesia:write( - rabbit_durable_exchange, - rabbit_exchange_decorator:set( - X#exchange{policy = match(Name, Policies), - operator_policy = match(Name, OpPolicies)}), - write) - end) || X = #exchange{name = Name} <- Xs], - _ = [begin - QName = amqqueue:get_name(Q0), - Policy1 = match(QName, Policies), - Q1 = amqqueue:set_policy(Q0, Policy1), - OpPolicy1 = match(QName, OpPolicies), - Q2 = amqqueue:set_operator_policy(Q1, OpPolicy1), - Q3 = rabbit_queue_decorator:set(Q2), - rabbit_misc:execute_mnesia_transaction( - fun () -> - mnesia:write(rabbit_durable_queue, Q3, write) - end) - end || Q0 <- Qs], + Xs = [rabbit_exchange_decorator:set( + X#exchange{policy = match(Name, Policies), + operator_policy = match(Name, OpPolicies)}) + || X = #exchange{name = Name} <- Xs0], + Qs = rabbit_amqqueue:list_durable(), + _ = rabbit_db_exchange:insert(Xs), + Qs0 = [begin + QName = amqqueue:get_name(Q0), + Policy1 = match(QName, Policies), + Q1 = amqqueue:set_policy(Q0, Policy1), + OpPolicy1 = match(QName, OpPolicies), + Q2 = amqqueue:set_operator_policy(Q1, OpPolicy1), + rabbit_queue_decorator:set(Q2) + end || Q0 <- Qs], + _ = rabbit_db_queue:insert(Qs0), ok. invalid_file() -> @@ -441,68 +430,67 @@ notify_clear(VHost, <<"operator_policy">>, Name, ActingUser) -> %% [2] We could be here in a post-tx fun after the vhost has been %% deleted; in which case it's fine to do nothing. update_matched_objects(VHost, PolicyDef, ActingUser) -> - Tabs = [rabbit_queue, rabbit_durable_queue, - rabbit_exchange, rabbit_durable_exchange], - {XUpdateResults, QUpdateResults} = rabbit_misc:execute_mnesia_transaction( - fun() -> - _ = [mnesia:lock({table, T}, write) || T <- Tabs], %% [1] - case catch {list(VHost), list_op(VHost)} of - {'EXIT', {throw, {error, {no_such_vhost, _}}}} -> - {[], []}; %% [2] - {'EXIT', Exit} -> - exit(Exit); - {Policies, OpPolicies} -> - {[update_exchange(X, Policies, OpPolicies) || - X <- rabbit_exchange:list(VHost)], - [update_queue(Q, Policies, OpPolicies) || - Q <- rabbit_amqqueue:list(VHost)]} - end - end), + {XUpdateResults, QUpdateResults} = + case catch {list(VHost), list_op(VHost)} of + {'EXIT', {throw, {error, {no_such_vhost, _}}}} -> + {[], []}; %% [2] + {'EXIT', Exit} -> + exit(Exit); + {Policies, OpPolicies} -> + rabbit_db_policy:update(VHost, + get_updated_exchange(Policies, OpPolicies), + get_updated_queue(Policies, OpPolicies)) + end, _ = [catch maybe_notify_of_policy_change(XRes, PolicyDef, ActingUser) || XRes <- XUpdateResults], _ = [catch maybe_notify_of_policy_change(QRes, PolicyDef, ActingUser) || QRes <- QUpdateResults], ok. -update_exchange(X = #exchange{name = XName, - policy = OldPolicy, - operator_policy = OldOpPolicy}, - Policies, OpPolicies) -> - case {match(XName, Policies), match(XName, OpPolicies)} of - {OldPolicy, OldOpPolicy} -> no_change; - {NewPolicy, NewOpPolicy} -> - NewExchange = rabbit_exchange:update( - XName, - fun(X0) -> - rabbit_exchange_decorator:set( - X0 #exchange{policy = NewPolicy, - operator_policy = NewOpPolicy}) - end), - case NewExchange of - #exchange{} = X1 -> {X, X1}; - not_found -> {X, X } +get_updated_exchange(Policies, OpPolicies) -> + fun(X = #exchange{name = XName, + policy = OldPolicy, + operator_policy = OldOpPolicy}) -> + case {match(XName, Policies), match(XName, OpPolicies)} of + {OldPolicy, OldOpPolicy} -> no_change; + {NewPolicy, NewOpPolicy} -> + Decorators = rabbit_exchange_decorator:active( + X#exchange{policy = NewPolicy, + operator_policy = NewOpPolicy}), + #{ + exchange => X, + update_function => + fun(X0) -> + X0#exchange{policy = NewPolicy, + operator_policy = NewOpPolicy, + decorators = Decorators} + end + } end end. -update_queue(Q0, Policies, OpPolicies) when ?is_amqqueue(Q0) -> - QName = amqqueue:get_name(Q0), - OldPolicy = amqqueue:get_policy(Q0), - OldOpPolicy = amqqueue:get_operator_policy(Q0), - case {match(QName, Policies), match(QName, OpPolicies)} of - {OldPolicy, OldOpPolicy} -> no_change; - {NewPolicy, NewOpPolicy} -> - F = fun (QFun0) -> - QFun1 = amqqueue:set_policy(QFun0, NewPolicy), - QFun2 = amqqueue:set_operator_policy(QFun1, NewOpPolicy), - NewPolicyVersion = amqqueue:get_policy_version(QFun2) + 1, - QFun3 = amqqueue:set_policy_version(QFun2, NewPolicyVersion), - rabbit_queue_decorator:set(QFun3) - end, - NewQueue = rabbit_amqqueue:update(QName, F), - case NewQueue of - Q1 when ?is_amqqueue(Q1) -> - {Q0, Q1}; - not_found -> - {Q0, Q0} - end +get_updated_queue(Policies, OpPolicies) -> + fun(Q0) when ?is_amqqueue(Q0) -> + OldPolicy = amqqueue:get_policy(Q0), + OldOpPolicy = amqqueue:get_operator_policy(Q0), + case {match(Q0, Policies), match(Q0, OpPolicies)} of + {OldPolicy, OldOpPolicy} -> no_change; + {NewPolicy, NewOpPolicy} -> + Q = amqqueue:set_operator_policy(amqqueue:set_policy(Q0, NewPolicy), + NewOpPolicy), + Decorators = rabbit_queue_decorator:active(Q), + #{ + queue => Q0, + update_function => update_queue(NewPolicy, NewOpPolicy, Decorators) + } + end + end. + +update_queue(Policy, OpPolicy, Decorators) -> + fun (Queue0) -> + Queue1 = amqqueue:set_policy(Queue0, Policy), + Queue2 = amqqueue:set_operator_policy(Queue1, OpPolicy), + PolicyVersion = amqqueue:get_policy_version(Queue2) + 1, + Queue3 = amqqueue:set_policy_version(Queue2, PolicyVersion), + amqqueue:set_decorators(Queue3, Decorators) end. maybe_notify_of_policy_change(no_change, _PolicyDef, _ActingUser)-> diff --git a/deps/rabbit/src/rabbit_priority_queue.erl b/deps/rabbit/src/rabbit_priority_queue.erl index 6fb37eacb7cf..3747adc719fb 100644 --- a/deps/rabbit/src/rabbit_priority_queue.erl +++ b/deps/rabbit/src/rabbit_priority_queue.erl @@ -106,11 +106,11 @@ mutate_name_bin(P, NameBin) -> <>. expand_queues(QNames) -> - lists:unzip( - lists:append([expand_queue(QName) || QName <- QNames])). + Qs = rabbit_db_queue:get_durable(QNames), + lists:unzip(lists:append([expand_queue(Q) || Q <- Qs])). -expand_queue(QName = #resource{name = QNameBin}) -> - {ok, Q} = rabbit_misc:dirty_read({rabbit_durable_queue, QName}), +expand_queue(Q) -> + #resource{name = QNameBin} = QName = amqqueue:get_name(Q), case priorities(Q) of none -> [{QName, QName}]; Ps -> [{QName, QName#resource{name = mutate_name_bin(P, QNameBin)}} diff --git a/deps/rabbit/src/rabbit_queue_decorator.erl b/deps/rabbit/src/rabbit_queue_decorator.erl index c7fe6fe78960..4edc93d7e70c 100644 --- a/deps/rabbit/src/rabbit_queue_decorator.erl +++ b/deps/rabbit/src/rabbit_queue_decorator.erl @@ -11,6 +11,7 @@ -include("amqqueue.hrl"). -export([select/1, set/1, register/2, unregister/1]). +-export([active/1, list/0]). -behaviour(rabbit_registry_class). @@ -43,6 +44,9 @@ set(Q) when ?is_amqqueue(Q) -> Decorators = [D || D <- list(), D:active_for(Q)], amqqueue:set_decorators(Q, Decorators). +active(Q) when ?is_amqqueue(Q) -> + [D || D <- list(), D:active_for(Q)]. + list() -> [M || {_, M} <- rabbit_registry:lookup_all(queue_decorator)]. register(TypeName, ModuleName) -> diff --git a/deps/rabbit/src/rabbit_queue_location.erl b/deps/rabbit/src/rabbit_queue_location.erl index 7befca934135..1825efffc96a 100644 --- a/deps/rabbit/src/rabbit_queue_location.erl +++ b/deps/rabbit/src/rabbit_queue_location.erl @@ -163,12 +163,7 @@ potential_leaders(Replicas, RunningNodes) -> %% Return a function so that queues are fetched lazily (i.e. only when needed, %% and at most once when no amqqueue migration is going on). get_queues_for_type(QueueType) -> - fun() -> rabbit_amqqueue:list_with_possible_retry( - fun() -> - mnesia:dirty_match_object(rabbit_queue, - amqqueue:pattern_match_on_type(QueueType)) - end) - end. + fun () -> rabbit_amqqueue:list_by_type(QueueType) end. shuffle(L0) when is_list(L0) -> L1 = lists:map(fun(E) -> {rand:uniform(), E} end, L0), diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 7ede155cf5d0..962c1decbb5b 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -326,10 +326,7 @@ become_leader(QName, Name) -> %% we need to ensure there is no chance of blocking as else the ra node %% may not be able to establish its leadership spawn(fun() -> - rabbit_misc:execute_mnesia_transaction( - fun() -> - rabbit_amqqueue:update(QName, Fun) - end), + rabbit_amqqueue:update(QName, Fun), case rabbit_amqqueue:lookup(QName) of {ok, Q0} when ?is_amqqueue(Q0) -> Nodes = get_nodes(Q0), @@ -541,10 +538,7 @@ repair_amqqueue_nodes(Q0) -> TS = TS0#{nodes => RaNodes}, amqqueue:set_type_state(Q, TS) end, - rabbit_misc:execute_mnesia_transaction( - fun() -> - rabbit_amqqueue:update(QName, Fun) - end), + rabbit_amqqueue:update(QName, Fun), repaired end. @@ -606,7 +600,7 @@ recover(_Vhost, Queues) -> %% present in the rabbit_queue table and not just in %% rabbit_durable_queue %% So many code paths are dependent on this. - ok = rabbit_amqqueue:store_queue_ram_dirty(Q0), + ok = rabbit_db_queue:insert_dirty(Q0), Q = Q0, case Res of ok -> @@ -1105,8 +1099,7 @@ add_member(Q, Node, Timeout) when ?amqqueue_is_quorum(Q) -> end), amqqueue:set_pid(Q2, Leader) end, - rabbit_misc:execute_mnesia_transaction( - fun() -> rabbit_amqqueue:update(QName, Fun) end), + rabbit_amqqueue:update(QName, Fun), ok; {timeout, _} -> _ = ra:force_delete_server(?RA_SYSTEM, ServerId), @@ -1160,8 +1153,7 @@ delete_member(Q, Node) when ?amqqueue_is_quorum(Q) -> Ts#{nodes => lists:delete(Node, Nodes)} end) end, - rabbit_misc:execute_mnesia_transaction( - fun() -> rabbit_amqqueue:update(QName, Fun) end), + rabbit_amqqueue:update(QName, Fun), case ra:force_delete_server(?RA_SYSTEM, ServerId) of ok -> ok; @@ -1359,13 +1351,7 @@ dead_letter_publish(X, RK, QName, Reason, Msgs) -> find_quorum_queues(VHost) -> Node = node(), - mnesia:async_dirty( - fun () -> - qlc:e(qlc:q([Q || Q <- mnesia:table(rabbit_durable_queue), - ?amqqueue_is_quorum(Q), - amqqueue:get_vhost(Q) =:= VHost, - amqqueue:qnode(Q) == Node])) - end). + rabbit_db_queue:get_all_by_type_and_node(VHost, rabbit_quorum_queue, Node). i_totals(Q) when ?is_amqqueue(Q) -> QName = amqqueue:get_name(Q), diff --git a/deps/rabbit/src/rabbit_stream_coordinator.erl b/deps/rabbit/src/rabbit_stream_coordinator.erl index 8f746f9b9fbe..05e99547bc95 100644 --- a/deps/rabbit/src/rabbit_stream_coordinator.erl +++ b/deps/rabbit/src/rabbit_stream_coordinator.erl @@ -1080,10 +1080,7 @@ phase_update_mnesia(StreamId, Args, #{reference := QName, Q end end, - try rabbit_misc:execute_mnesia_transaction( - fun() -> - rabbit_amqqueue:update(QName, Fun) - end) of + try rabbit_amqqueue:update(QName, Fun) of not_found -> rabbit_log:debug("~ts: resource for stream id ~ts not found, " "recovering from rabbit_durable_queue", @@ -1091,7 +1088,7 @@ phase_update_mnesia(StreamId, Args, #{reference := QName, %% This can happen during recovery %% we need to re-initialise the queue record %% if the stream id is a match - case mnesia:dirty_read(rabbit_durable_queue, QName) of + case rabbit_amqqueue:lookup_durable_queue(QName) of [] -> %% queue not found at all, it must have been deleted ok; diff --git a/deps/rabbit/src/rabbit_stream_queue.erl b/deps/rabbit/src/rabbit_stream_queue.erl index 2672c0e3d4f5..846cccd7855a 100644 --- a/deps/rabbit/src/rabbit_stream_queue.erl +++ b/deps/rabbit/src/rabbit_stream_queue.erl @@ -798,8 +798,7 @@ set_retention_policy(Name, VHost, Policy) -> Conf = amqqueue:get_type_state(Q), amqqueue:set_type_state(Q, Conf#{max_age => MaxAge}) end, - case rabbit_misc:execute_mnesia_transaction( - fun() -> rabbit_amqqueue:update(QName, Fun) end) of + case rabbit_amqqueue:update(QName, Fun) of not_found -> {error, not_found}; _ -> @@ -1064,13 +1063,10 @@ set_leader_pid(Pid, QName) -> Fun = fun (Q) -> amqqueue:set_pid(Q, Pid) end, - case rabbit_misc:execute_mnesia_transaction( - fun() -> - rabbit_amqqueue:update(QName, Fun) - end) of + case rabbit_amqqueue:update(QName, Fun) of not_found -> %% This can happen during recovery - [Q] = mnesia:dirty_read(rabbit_durable_queue, QName), + {ok, Q} = rabbit_amqqueue:lookup_durable_queue(QName), rabbit_amqqueue:ensure_rabbit_queue_record_is_initialized(Fun(Q)); _ -> ok diff --git a/deps/rabbit/test/bindings_SUITE.erl b/deps/rabbit/test/bindings_SUITE.erl new file mode 100644 index 000000000000..536a1c5f84f4 --- /dev/null +++ b/deps/rabbit/test/bindings_SUITE.erl @@ -0,0 +1,808 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(bindings_SUITE). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). +-include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). + +-compile([nowarn_export_all, export_all]). +-compile(export_all). + +suite() -> + [{timetrap, 5 * 60000}]. + +all() -> + [ + {group, mnesia_store}, + {group, mnesia_cluster} + ]. + +groups() -> + [ + {mnesia_store, [], all_tests()}, + {mnesia_cluster, [], [ + transient_queue_on_node_down_mnesia + ]} + ]. + +all_tests() -> + [ + %% Queue bindings + bind_and_unbind, + bind_and_delete, + bind_and_delete_source_exchange, + list_bindings, + list_for_source, + list_for_destination, + list_for_source_and_destination, + list_explicit, + info_all, + list_with_multiple_vhosts, + list_with_multiple_arguments, + bind_to_unknown_queue, + %% Exchange bindings + bind_and_unbind_exchange, + bind_and_delete_exchange_source, + bind_and_delete_exchange_destination, + bind_to_unknown_exchange + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config, []). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(mnesia_store = Group, Config) -> + init_per_group_common(Group, Config, 1); +init_per_group(mnesia_cluster = Group, Config) -> + init_per_group_common(Group, Config, 3). + +init_per_group_common(Group, Config, Size) -> + Config1 = rabbit_ct_helpers:set_config(Config, + [{rmq_nodes_count, Size}, + {rmq_nodename_suffix, Group}, + {tcp_ports_base}]), + rabbit_ct_helpers:run_steps(Config1, rabbit_ct_broker_helpers:setup_steps()). + +end_per_group(_, Config) -> + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_testcase(Testcase, Config) -> + Config1 = rabbit_ct_helpers:testcase_started(Config, Testcase), + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_queues, []), + Name = rabbit_data_coercion:to_binary(Testcase), + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_exchange, [Name]), + Config2 = rabbit_ct_helpers:set_config(Config1, + [{queue_name, Name}, + {alt_queue_name, <>}, + {exchange_name, Name} + ]), + rabbit_ct_helpers:run_steps(Config2, rabbit_ct_client_helpers:setup_steps()). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_queues, []), + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_exchange, + [?config(exchange_name, Config)]), + Config1 = rabbit_ct_helpers:run_steps( + Config, + rabbit_ct_client_helpers:teardown_steps()), + rabbit_ct_helpers:testcase_finished(Config1, Testcase). + +%% ------------------------------------------------------------------- +%% Testcases. +%% ------------------------------------------------------------------- +bind_and_unbind(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + + DefaultExchange = rabbit_misc:r(<<"/">>, exchange, <<>>), + QResource = rabbit_misc:r(<<"/">>, queue, Q), + DefaultBinding = binding_record(DefaultExchange, QResource, Q, []), + + %% Binding to the default exchange, it's always present + ?assertEqual([DefaultBinding], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + + %% Let's bind to other exchange + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = Q, + routing_key = Q}), + + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + QResource, Q, []), + Bindings = lists:sort([DefaultBinding, DirectBinding]), + + ?assertEqual(Bindings, + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), + + #'queue.unbind_ok'{} = amqp_channel:call(Ch, #'queue.unbind'{exchange = <<"amq.direct">>, + queue = Q, + routing_key = Q}), + + ?assertEqual([DefaultBinding], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + ok. + +bind_and_delete(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + + DefaultExchange = rabbit_misc:r(<<"/">>, exchange, <<>>), + QResource = rabbit_misc:r(<<"/">>, queue, Q), + DefaultBinding = binding_record(DefaultExchange, QResource, Q, []), + + %% Binding to the default exchange, it's always present + ?assertEqual([DefaultBinding], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + + %% Let's bind to other exchange + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = Q, + routing_key = Q}), + + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + QResource, Q, []), + Bindings = lists:sort([DefaultBinding, DirectBinding]), + + ?assertEqual(Bindings, + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), + + ?assertMatch(#'queue.delete_ok'{}, + amqp_channel:call(Ch, #'queue.delete'{queue = Q})), + + ?assertEqual([], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + ok. + +bind_and_delete_source_exchange(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + X = ?config(exchange_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + #'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = X}), + + DefaultExchange = rabbit_misc:r(<<"/">>, exchange, <<>>), + QResource = rabbit_misc:r(<<"/">>, queue, Q), + DefaultBinding = binding_record(DefaultExchange, QResource, Q, []), + + %% Binding to the default exchange, it's always present + ?assertEqual([DefaultBinding], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + + %% Let's bind to other exchange + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = X, + queue = Q, + routing_key = Q}), + + XBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, X), QResource, Q, []), + Bindings = lists:sort([DefaultBinding, XBinding]), + + ?assertEqual(Bindings, + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), + + ?assertMatch(#'exchange.delete_ok'{}, + amqp_channel:call(Ch, #'exchange.delete'{exchange = X})), + + ?assertEqual([DefaultBinding], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + ok. + +list_bindings(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + + DefaultExchange = rabbit_misc:r(<<"/">>, exchange, <<>>), + QResource = rabbit_misc:r(<<"/">>, queue, Q), + DefaultBinding = binding_record(DefaultExchange, QResource, Q, []), + + %% Binding to the default exchange, it's always present + ?assertEqual([DefaultBinding], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + + %% Let's bind to all other exchanges + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.fanout">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.headers">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.match">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.rabbitmq.trace">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.topic">>, + queue = Q, + routing_key = Q}), + + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + QResource, Q, []), + FanoutBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.fanout">>), + QResource, Q, []), + HeadersBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.headers">>), + QResource, Q, []), + MatchBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.match">>), + QResource, Q, []), + TraceBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.rabbitmq.trace">>), + QResource, Q, []), + TopicBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.topic">>), + QResource, Q, []), + Bindings = lists:sort([DefaultBinding, DirectBinding, FanoutBinding, HeadersBinding, + MatchBinding, TraceBinding, TopicBinding]), + + ?assertEqual(Bindings, + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), + + ok. + +list_for_source(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + QAlt = ?config(alt_queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + ?assertEqual({'queue.declare_ok', QAlt, 0, 0}, declare(Ch, QAlt, [])), + + QResource = rabbit_misc:r(<<"/">>, queue, Q), + QAltResource = rabbit_misc:r(<<"/">>, queue, QAlt), + + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.topic">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = QAlt, + routing_key = QAlt}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.topic">>, + queue = QAlt, + routing_key = QAlt}), + + DirectExchange = rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + TopicExchange = rabbit_misc:r(<<"/">>, exchange, <<"amq.topic">>), + DirectBinding = binding_record(DirectExchange, QResource, Q, []), + DirectABinding = binding_record(DirectExchange, QAltResource, QAlt, []), + TopicBinding = binding_record(TopicExchange, QResource, Q, []), + TopicABinding = binding_record(TopicExchange, QAltResource, QAlt, []), + DirectBindings = lists:sort([DirectBinding, DirectABinding]), + TopicBindings = lists:sort([TopicBinding, TopicABinding]), + + ?assertEqual( + DirectBindings, + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_for_source, + [DirectExchange]))), + ?assertEqual( + TopicBindings, + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_for_source, + [TopicExchange]))). + +list_with_multiple_vhosts(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + VHost1 = <<"vhost1">>, + VHost2 = <<"vhost2">>, + ok = rabbit_ct_broker_helpers:add_vhost(Config, VHost1), + ok = rabbit_ct_broker_helpers:set_full_permissions(Config, <<"guest">>, VHost1), + ok = rabbit_ct_broker_helpers:add_vhost(Config, VHost2), + ok = rabbit_ct_broker_helpers:set_full_permissions(Config, <<"guest">>, VHost2), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Conn1 = rabbit_ct_client_helpers:open_unmanaged_connection(Config, 0, VHost1), + {ok, Ch1} = amqp_connection:open_channel(Conn1), + Conn2 = rabbit_ct_client_helpers:open_unmanaged_connection(Config, 0, VHost2), + {ok, Ch2} = amqp_connection:open_channel(Conn2), + + Q = ?config(queue_name, Config), + QAlt = ?config(alt_queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + ?assertEqual({'queue.declare_ok', QAlt, 0, 0}, declare(Ch, QAlt, [])), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch1, Q, [])), + ?assertEqual({'queue.declare_ok', QAlt, 0, 0}, declare(Ch1, QAlt, [])), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch2, Q, [])), + ?assertEqual({'queue.declare_ok', QAlt, 0, 0}, declare(Ch2, QAlt, [])), + + QResource = rabbit_misc:r(<<"/">>, queue, Q), + QAltResource = rabbit_misc:r(<<"/">>, queue, QAlt), + QAltResource1 = rabbit_misc:r(VHost1, queue, QAlt), + QResource2 = rabbit_misc:r(VHost2, queue, Q), + QAltResource2 = rabbit_misc:r(VHost2, queue, QAlt), + + %% Default vhost: + %% direct - queue + %% topic - altqueue + %% Vhost1: + %% direct - altqueue + %% Vhost2: + %% topic - queue + %% topic - altqueue + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.topic">>, + queue = QAlt, + routing_key = QAlt}), + #'queue.bind_ok'{} = amqp_channel:call(Ch1, #'queue.bind'{exchange = <<"amq.direct">>, + queue = QAlt, + routing_key = QAlt}), + #'queue.bind_ok'{} = amqp_channel:call(Ch2, #'queue.bind'{exchange = <<"amq.topic">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch2, #'queue.bind'{exchange = <<"amq.topic">>, + queue = QAlt, + routing_key = QAlt}), + + DirectExchange = rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + TopicExchange = rabbit_misc:r(<<"/">>, exchange, <<"amq.topic">>), + DirectExchange1 = rabbit_misc:r(VHost1, exchange, <<"amq.direct">>), + TopicExchange2 = rabbit_misc:r(VHost2, exchange, <<"amq.topic">>), + DefaultExchange1 = rabbit_misc:r(VHost1, exchange, <<>>), + DefaultExchange2 = rabbit_misc:r(VHost2, exchange, <<>>), + + %% Direct exchange on default vhost + ?assertEqual( + [binding_record(DirectExchange, QResource, Q, [])], + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_for_source, + [DirectExchange]))), + %% Direct exchange on vhost 1 + ?assertEqual( + [binding_record(DirectExchange1, QAltResource1, QAlt, [])], + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_for_source, + [DirectExchange1]))), + ?assertEqual( + lists:sort([binding_record(DefaultExchange1, QAltResource1, QAlt, []), + binding_record(DirectExchange1, QAltResource1, QAlt, [])]), + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_for_destination, + [QAltResource1]))), + %% Topic exchange on default vhost + ?assertEqual( + [binding_record(TopicExchange, QAltResource, QAlt, [])], + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_for_source, + [TopicExchange]))), + %% Topic exchange on vhost 2 + ?assertEqual( + lists:sort([binding_record(TopicExchange2, QAltResource2, QAlt, []), + binding_record(TopicExchange2, QResource2, Q, [])]), + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_for_source, + [TopicExchange2]))), + ?assertEqual( + lists:sort([binding_record(TopicExchange2, QAltResource2, QAlt, []), + binding_record(DefaultExchange2, QAltResource2, QAlt, [])]), + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_for_destination, + [QAltResource2]))). + +list_with_multiple_arguments(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + + DefaultExchange = rabbit_misc:r(<<"/">>, exchange, <<>>), + QResource = rabbit_misc:r(<<"/">>, queue, Q), + DefaultBinding = binding_record(DefaultExchange, QResource, Q, []), + + %% Binding to the default exchange, it's always present + ?assertEqual([DefaultBinding], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + + %% Let's bind with multiple arguments + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.headers">>, + queue = Q, + routing_key = Q, + arguments = [{<<"x-match">>, longstr, <<"all">>}]}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.headers">>, + queue = Q, + routing_key = Q, + arguments = [{<<"x-match">>, longstr, <<"any">>}]}), + + AllBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.headers">>), + QResource, Q, [{<<"x-match">>, longstr, <<"all">>}]), + AnyBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.headers">>), + QResource, Q, [{<<"x-match">>, longstr, <<"any">>}]), + Bindings = lists:sort([DefaultBinding, AllBinding, AnyBinding]), + + ?assertEqual(Bindings, + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), + + ok. + +list_for_destination(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + QAlt = ?config(alt_queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + ?assertEqual({'queue.declare_ok', QAlt, 0, 0}, declare(Ch, QAlt, [])), + + QResource = rabbit_misc:r(<<"/">>, queue, Q), + QAltResource = rabbit_misc:r(<<"/">>, queue, QAlt), + + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.topic">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = QAlt, + routing_key = QAlt}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.topic">>, + queue = QAlt, + routing_key = QAlt}), + + DirectExchange = rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + TopicExchange = rabbit_misc:r(<<"/">>, exchange, <<"amq.topic">>), + DefaultExchange = rabbit_misc:r(<<"/">>, exchange, <<>>), + DirectBinding = binding_record(DirectExchange, QResource, Q, []), + DirectABinding = binding_record(DirectExchange, QAltResource, QAlt, []), + TopicBinding = binding_record(TopicExchange, QResource, Q, []), + TopicABinding = binding_record(TopicExchange, QAltResource, QAlt, []), + DefaultBinding = binding_record(DefaultExchange, QResource, Q, []), + DefaultABinding = binding_record(DefaultExchange, QAltResource, QAlt, []), + + Bindings = lists:sort([DefaultBinding, DirectBinding, TopicBinding]), + AltBindings = lists:sort([DefaultABinding, DirectABinding, TopicABinding]), + + ?assertEqual( + Bindings, + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_for_destination, + [QResource]))), + ?assertEqual( + AltBindings, + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_for_destination, + [QAltResource]))). + +list_for_source_and_destination(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + QAlt = ?config(alt_queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + ?assertEqual({'queue.declare_ok', QAlt, 0, 0}, declare(Ch, QAlt, [])), + + QResource = rabbit_misc:r(<<"/">>, queue, Q), + QAltResource = rabbit_misc:r(<<"/">>, queue, QAlt), + + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.topic">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = QAlt, + routing_key = QAlt}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.topic">>, + queue = QAlt, + routing_key = QAlt}), + + DirectExchange = rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + TopicExchange = rabbit_misc:r(<<"/">>, exchange, <<"amq.topic">>), + DefaultExchange = rabbit_misc:r(<<"/">>, exchange, <<>>), + DirectBinding = binding_record(DirectExchange, QResource, Q, []), + TopicBinding = binding_record(TopicExchange, QResource, Q, []), + DefaultABinding = binding_record(DefaultExchange, QAltResource, QAlt, []), + + ?assertEqual( + [DirectBinding], + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, + list_for_source_and_destination, + [DirectExchange, QResource]))), + ?assertEqual( + [TopicBinding], + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, + list_for_source_and_destination, + [TopicExchange, QResource]))), + ?assertEqual( + [DefaultABinding], + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, + list_for_source_and_destination, + [DefaultExchange, QAltResource]))). + + list_explicit(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + + QResource = rabbit_misc:r(<<"/">>, queue, Q), + + ?assertEqual([], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_explicit, [])), + + %% Let's bind to other exchanges + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.fanout">>, + queue = Q, + routing_key = Q}), + + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + QResource, Q, []), + FanoutBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.fanout">>), + QResource, Q, []), + Bindings = lists:sort([DirectBinding, FanoutBinding]), + + ?assertEqual(Bindings, + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_explicit, []))), + + ok. + +info_all(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + + Default = [{source_name,<<>>}, + {source_kind,exchange}, + {destination_name,<<"info_all">>}, + {destination_kind,queue}, + {routing_key,<<"info_all">>}, + {arguments,[]}, + {vhost,<<"/">>}], + + ?assertEqual([Default], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, info_all, [<<"/">>])), + + Direct = [{source_name,<<"amq.direct">>}, + {source_kind,exchange}, + {destination_name,<<"info_all">>}, + {destination_kind,queue}, + {routing_key,<<"info_all">>}, + {arguments,[]}, + {vhost,<<"/">>}], + + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = Q, + routing_key = Q}), + + Infos = lists:sort([Default, Direct]), + ?assertEqual(Infos, + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, info_all, [<<"/">>]))), + + ok. + +bind_to_unknown_queue(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + + %% Let's bind to exchange + ?assertExit({{shutdown, {server_initiated_close,404, _}}, _}, + amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = Q, + routing_key = Q})), + ?assertEqual([], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + ok. + +bind_and_unbind_exchange(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + X = ?config(exchange_name, Config), + + ?assertEqual([], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + + #'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = X}), + %% Let's bind to other exchange + #'exchange.bind_ok'{} = amqp_channel:call(Ch, #'exchange.bind'{destination = X, + source = <<"amq.direct">>, + routing_key = <<"key">>}), + + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + rabbit_misc:r(<<"/">>, exchange, X), + <<"key">>, []), + + ?assertEqual([DirectBinding], + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), + + #'exchange.unbind_ok'{} = amqp_channel:call(Ch, + #'exchange.unbind'{destination = X, + source = <<"amq.direct">>, + routing_key = <<"key">>}), + + ?assertEqual([], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + ok. + +bind_to_unknown_exchange(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + X = ?config(exchange_name, Config), + + ?assertEqual([], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + + %% Let's bind to exchange + ?assertExit({{shutdown, {server_initiated_close,404, _}}, _}, + amqp_channel:call(Ch, #'exchange.bind'{destination = X, + source = <<"amq.direct">>, + routing_key = <<"key">>})), + ?assertEqual([], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + ok. + +bind_and_delete_exchange_destination(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + X = ?config(exchange_name, Config), + + ?assertEqual([], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + + #'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = X}), + %% Let's bind to other exchange + #'exchange.bind_ok'{} = amqp_channel:call(Ch, #'exchange.bind'{destination = X, + source = <<"amq.direct">>, + routing_key = <<"key">>}), + + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + rabbit_misc:r(<<"/">>, exchange, X), + <<"key">>, []), + + ?assertEqual([DirectBinding], + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), + + #'exchange.delete_ok'{} = amqp_channel:call(Ch, #'exchange.delete'{exchange = X}), + + ?assertEqual([], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + ok. + +bind_and_delete_exchange_source(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + X = ?config(exchange_name, Config), + + ?assertEqual([], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + + #'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = X}), + %% Let's bind to other exchange + #'exchange.bind_ok'{} = amqp_channel:call(Ch, #'exchange.bind'{destination = <<"amq.direct">>, + source = X, + routing_key = <<"key">>}), + + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, X), + rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + <<"key">>, []), + + ?assertEqual([DirectBinding], + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), + + #'exchange.delete_ok'{} = amqp_channel:call(Ch, #'exchange.delete'{exchange = X}), + + ?assertEqual([], + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + ok. + +transient_queue_on_node_down_mnesia(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + QAlt = ?config(alt_queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + ?assertEqual({'queue.declare_ok', QAlt, 0, 0}, declare(Ch, QAlt, [], false)), + + DefaultExchange = rabbit_misc:r(<<"/">>, exchange, <<>>), + QResource = rabbit_misc:r(<<"/">>, queue, Q), + QAltResource = rabbit_misc:r(<<"/">>, queue, QAlt), + DefaultBinding = binding_record(DefaultExchange, QResource, Q, []), + DefaultAltBinding = binding_record(DefaultExchange, QAltResource, QAlt, []), + + %% Binding to the default exchange, it's always present + ?assertEqual(lists:sort([DefaultBinding, DefaultAltBinding]), + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 1, rabbit_binding, list, [<<"/">>]))), + + %% Let's bind to other exchange + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = QAlt, + routing_key = QAlt}), + + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + QResource, Q, []), + DirectAltBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + QAltResource, QAlt, []), + Bindings = lists:sort([DefaultBinding, DirectBinding, DefaultAltBinding, DirectAltBinding]), + + ?assertEqual(Bindings, + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 1, rabbit_binding, list, [<<"/">>]))), + + rabbit_ct_broker_helpers:stop_node(Config, Server), + + Bindings1 = lists:sort([DefaultBinding, DirectBinding]), + ?assertEqual([DirectBinding], + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 1, rabbit_binding, list, [<<"/">>]))), + ?assertMatch([], + rabbit_ct_broker_helpers:rpc(Config, 1, rabbit_amqqueue, list, [<<"/">>])), + + rabbit_ct_broker_helpers:start_node(Config, Server), + + ?awaitMatch(Bindings1, + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 1, rabbit_binding, list, [<<"/">>])), + 30000), + ?awaitMatch([_], rabbit_ct_broker_helpers:rpc(Config, 1, rabbit_amqqueue, list, [<<"/">>]), + 30000), + ok. + +%% Internal + +delete_queues() -> + [{ok, _} = rabbit_amqqueue:delete(Q, false, false, <<"dummy">>) + || Q <- rabbit_amqqueue:list()]. + +delete_exchange(Name) -> + _ = rabbit_exchange:delete(rabbit_misc:r(<<"/">>, exchange, Name), false, <<"dummy">>). + +declare(Ch, Q, Args) -> + declare(Ch, Q, Args, true). + +declare(Ch, Q, Args, Durable) -> + amqp_channel:call(Ch, #'queue.declare'{queue = Q, + durable = Durable, + auto_delete = false, + arguments = Args}). + +binding_record(Src, Dst, Key, Args) -> + #binding{source = Src, + destination = Dst, + key = Key, + args = Args}. diff --git a/deps/rabbit_common/src/rabbit_exchange_type.erl b/deps/rabbit_common/src/rabbit_exchange_type.erl index f1481782bc22..ecf62729b684 100644 --- a/deps/rabbit_common/src/rabbit_exchange_type.erl +++ b/deps/rabbit_common/src/rabbit_exchange_type.erl @@ -11,8 +11,7 @@ -export([added_to_rabbit_registry/2, removed_from_rabbit_registry/1]). --type(tx() :: 'transaction' | 'none'). --type(serial() :: pos_integer() | tx()). +-type(serial() :: pos_integer() | 'none'). -callback description() -> [proplists:property()]. @@ -35,10 +34,10 @@ rabbit_types:ok_or_error({'binding_invalid', string(), [any()]}). %% called after declaration and recovery --callback create(tx(), rabbit_types:exchange()) -> 'ok'. +-callback create(serial(), rabbit_types:exchange()) -> 'ok'. %% called after exchange (auto)deletion. --callback delete(tx(), rabbit_types:exchange(), [rabbit_types:binding()]) -> +-callback delete(serial(), rabbit_types:exchange()) -> 'ok'. %% called when the policy attached to this exchange changes. From 452152469d5c4aee53bd0e577011a4c3f5f01949 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Thu, 12 Jan 2023 21:01:06 +0100 Subject: [PATCH 03/12] Move mirrored supervisor Mnesia-specific code to rabbit_db_* modules --- deps/rabbit/src/mirrored_supervisor.erl | 207 ++++-------- deps/rabbit/src/mirrored_supervisor_sups.erl | 6 +- deps/rabbit/src/rabbit_db_msup.erl | 176 ++++++++++ .../rabbit/test/mirrored_supervisor_SUITE.erl | 313 +++++++++++------- 4 files changed, 429 insertions(+), 273 deletions(-) create mode 100644 deps/rabbit/src/rabbit_db_msup.erl diff --git a/deps/rabbit/src/mirrored_supervisor.erl b/deps/rabbit/src/mirrored_supervisor.erl index a2c026ffcdf1..b1660d67d259 100644 --- a/deps/rabbit/src/mirrored_supervisor.erl +++ b/deps/rabbit/src/mirrored_supervisor.erl @@ -48,10 +48,10 @@ %% This is basically the same as for supervisor, except that: %% %% 1) start_link(Module, Args) becomes -%% start_link(Group, TxFun, Module, Args). +%% start_link(Group, Module, Args). %% %% 2) start_link({local, Name}, Module, Args) becomes -%% start_link({local, Name}, Group, TxFun, Module, Args). +%% start_link({local, Name}, Group, Module, Args). %% %% 3) start_link({global, Name}, Module, Args) is not available. %% @@ -61,19 +61,6 @@ %% application should invoke create_tables() (or table_definitions() %% if it wants to manage table creation itself). %% -%% The TxFun parameter to start_link/{4,5} is a function which the -%% mirrored supervisor can use to execute Mnesia transactions. In the -%% RabbitMQ server this goes via a worker pool; in other cases a -%% function like: -%% -%% tx_fun(Fun) -> -%% case mnesia:sync_transaction(Fun) of -%% {atomic, Result} -> Result; -%% {aborted, Reason} -> throw({error, Reason}) -%% end. -%% -%% could be used. -%% %% Internals %% --------- %% @@ -111,15 +98,7 @@ -define(GEN_SERVER, gen_server2). -define(SUP_MODULE, mirrored_supervisor_sups). --define(TABLE, mirrored_sup_childspec). --define(TABLE_DEF, - {?TABLE, - [{record_name, mirrored_sup_childspec}, - {type, ordered_set}, - {attributes, record_info(fields, mirrored_sup_childspec)}]}). --define(TABLE_MATCH, {match, #mirrored_sup_childspec{ _ = '_' }}). - --export([start_link/4, start_link/5, +-export([start_link/3, start_link/4, start_child/2, restart_child/2, delete_child/2, terminate_child/2, which_children/1, count_children/1, check_childspecs/1]). @@ -129,10 +108,9 @@ -export([init/1, handle_call/3, handle_info/2, terminate/2, code_change/3, handle_cast/2]). --export([start_internal/3]). +-export([start_internal/2]). -export([create_tables/0, table_definitions/0]). - --record(mirrored_sup_childspec, {key, mirroring_pid, childspec}). +-export([supervisor/1, child/2]). -record(state, {overall, delegate, @@ -161,25 +139,20 @@ -type group_name() :: any(). --type(tx_fun() :: fun((fun(() -> A)) -> A)). - --spec start_link(GroupName, TxFun, Module, Args) -> startlink_ret() when +-spec start_link(GroupName, Module, Args) -> startlink_ret() when GroupName :: group_name(), - TxFun :: tx_fun(), Module :: module(), Args :: term(). --spec start_link(SupName, GroupName, TxFun, Module, Args) -> +-spec start_link(SupName, GroupName, Module, Args) -> startlink_ret() when SupName :: ?SUPERVISOR:sup_name(), GroupName :: group_name(), - TxFun :: tx_fun(), Module :: module(), Args :: term(). --spec start_internal(Group, TxFun, ChildSpecs) -> Result when +-spec start_internal(Group, ChildSpecs) -> Result when Group :: group_name(), - TxFun :: tx_fun(), ChildSpecs :: [?SUPERVISOR:child_spec()], Result :: {'ok', pid()} | {'error', term()}. @@ -188,18 +161,18 @@ %%---------------------------------------------------------------------------- -start_link(Group, TxFun, Mod, Args) -> - start_link0([], Group, TxFun, init(Mod, Args)). +start_link(Group, Mod, Args) -> + start_link0([], Group, init(Mod, Args)). -start_link({local, SupName}, Group, TxFun, Mod, Args) -> - start_link0([{local, SupName}], Group, TxFun, init(Mod, Args)); +start_link({local, SupName}, Group, Mod, Args) -> + start_link0([{local, SupName}], Group, init(Mod, Args)); -start_link({global, _SupName}, _Group, _TxFun, _Mod, _Args) -> +start_link({global, _SupName}, _Group, _Mod, _Args) -> erlang:error(badarg). -start_link0(Prefix, Group, TxFun, Init) -> +start_link0(Prefix, Group, Init) -> case apply(?SUPERVISOR, start_link, - Prefix ++ [?SUP_MODULE, {overall, Group, TxFun, Init}]) of + Prefix ++ [?SUP_MODULE, {overall, Group, Init}]) of {ok, Pid} -> case catch call(Pid, {init, Pid}) of ok -> {ok, Pid}; E -> E @@ -230,17 +203,14 @@ cast(Sup, Msg) -> with_exit_handler( find_call(Sup, Id, Msg) -> Group = call(Sup, group), - MatchHead = #mirrored_sup_childspec{mirroring_pid = '$1', - key = {Group, Id}, - _ = '_'}, - %% If we did this inside a tx we could still have failover - %% immediately after the tx - we can't be 100% here. So we may as - %% well dirty_select. - case mnesia:dirty_select(?TABLE, [{MatchHead, [], ['$1']}]) of - [Mirror] -> call(Mirror, Msg); - [] -> {error, not_found} + case find_mirror(Group, Id) of + {ok, Mirror} -> call(Mirror, Msg); + Err -> Err end. +find_mirror(Group, Id) -> + rabbit_db_msup:find_mirror(Group, Id). + fold(FunAtom, Sup, AggFun) -> Group = call(Sup, group), lists:foldl(AggFun, [], @@ -258,15 +228,14 @@ mirroring(Sup) -> child(Sup, mirroring). %%---------------------------------------------------------------------------- -start_internal(Group, TxFun, ChildSpecs) -> - ?GEN_SERVER:start_link(?MODULE, {Group, TxFun, ChildSpecs}, +start_internal(Group, ChildSpecs) -> + ?GEN_SERVER:start_link(?MODULE, {Group, ChildSpecs}, [{timeout, infinity}]). %%---------------------------------------------------------------------------- -init({Group, TxFun, ChildSpecs}) -> +init({Group, ChildSpecs}) -> {ok, #state{group = Group, - tx_fun = TxFun, initial_childspecs = ChildSpecs, child_order = child_order_from(ChildSpecs)}}. @@ -274,7 +243,6 @@ handle_call({init, Overall}, _From, State = #state{overall = undefined, delegate = undefined, group = Group, - tx_fun = TxFun, initial_childspecs = ChildSpecs}) -> process_flag(trap_exit, true), LockId = mirrored_supervisor_locks:lock(Group), @@ -287,7 +255,7 @@ handle_call({init, Overall}, _From, case Rest of [] -> rabbit_log:debug("Mirrored supervisor: no known peer members in group ~tp, will delete all child records for it", [Group]), - TxFun(fun() -> delete_all(Group) end); + delete_all(Group); _ -> ok end, [begin @@ -297,7 +265,7 @@ handle_call({init, Overall}, _From, Delegate = delegate(Overall), erlang:monitor(process, Delegate), State1 = State#state{overall = Overall, delegate = Delegate}, - Results = [maybe_start(Group, TxFun, Overall, Delegate, S) || S <- ChildSpecs], + Results = [maybe_start(Group, Overall, Delegate, S) || S <- ChildSpecs], mirrored_supervisor_locks:unlock(LockId), case errors(Results) of [] -> {reply, ok, State1}; @@ -307,19 +275,18 @@ handle_call({init, Overall}, _From, handle_call({start_child, ChildSpec}, _From, State = #state{overall = Overall, delegate = Delegate, - group = Group, - tx_fun = TxFun}) -> + group = Group}) -> LockId = mirrored_supervisor_locks:lock(Group), maybe_log_lock_acquisition_failure(LockId, Group), rabbit_log:debug("Mirrored supervisor: asked to consider starting a child, group: ~tp", [Group]), - Result = case maybe_start(Group, TxFun, Overall, Delegate, ChildSpec) of - already_in_mnesia -> + Result = case maybe_start(Group, Overall, Delegate, ChildSpec) of + already_in_store -> rabbit_log:debug("Mirrored supervisor: maybe_start for group ~tp," - " overall ~tp returned 'record already present'", [Group, Overall]), + " overall ~p returned 'record already present'", [Group, Overall]), {error, already_present}; - {already_in_mnesia, Pid} -> + {already_in_store, Pid} -> rabbit_log:debug("Mirrored supervisor: maybe_start for group ~tp," - " overall ~tp returned 'already running: ~tp'", [Group, Overall, Pid]), + " overall ~p returned 'already running: ~tp'", [Group, Overall, Pid]), {error, {already_started, Pid}}; Else -> rabbit_log:debug("Mirrored supervisor: maybe_start for group ~tp," @@ -330,9 +297,8 @@ handle_call({start_child, ChildSpec}, _From, {reply, Result, State}; handle_call({delete_child, Id}, _From, State = #state{delegate = Delegate, - group = Group, - tx_fun = TxFun}) -> - {reply, stop(Group, TxFun, Delegate, Id), State}; + group = Group}) -> + {reply, stop(Group, Delegate, Id), State}; handle_call({msg, F, A}, _From, State = #state{delegate = Delegate}) -> {reply, apply(?SUPERVISOR, F, [Delegate | A]), State}; @@ -371,16 +337,14 @@ handle_info({'DOWN', _Ref, process, Pid, Reason}, handle_info({'DOWN', _Ref, process, Pid, _Reason}, State = #state{delegate = Delegate, group = Group, - tx_fun = TxFun, overall = O, child_order = ChildOrder}) -> %% No guarantee pg will have received the DOWN before us. R = case lists:sort(pg:get_members(Group)) -- [Pid] of - [O | _] -> ChildSpecs = - TxFun(fun() -> update_all(O, Pid) end), + [O | _] -> ChildSpecs = update_all(O, Pid), [start(Delegate, ChildSpec) || ChildSpec <- restore_child_order(ChildSpecs, - ChildOrder)]; + ChildOrder)]; _ -> [] end, case errors(R) of @@ -402,9 +366,10 @@ code_change(_OldVsn, State, _Extra) -> tell_all_peers_to_die(Group, Reason) -> [cast(P, {die, Reason}) || P <- pg:get_members(Group) -- [self()]]. -maybe_start(Group, TxFun, Overall, Delegate, ChildSpec) -> - rabbit_log:debug("Mirrored supervisor: asked to consider starting, group: ~tp", [Group]), - try TxFun(fun() -> check_start(Group, Overall, Delegate, ChildSpec) end) of +maybe_start(Group, Overall, Delegate, ChildSpec) -> + rabbit_log:debug("Mirrored supervisor: asked to consider starting, group: ~tp", + [Group]), + try check_start(Group, Overall, Delegate, ChildSpec) of start -> rabbit_log:debug("Mirrored supervisor: check_start for group ~tp," " overall ~tp returned 'do start'", [Group, Overall]), @@ -412,62 +377,36 @@ maybe_start(Group, TxFun, Overall, Delegate, ChildSpec) -> undefined -> rabbit_log:debug("Mirrored supervisor: check_start for group ~tp," " overall ~tp returned 'undefined'", [Group, Overall]), - already_in_mnesia; - Pid -> + already_in_store; + Pid -> rabbit_log:debug("Mirrored supervisor: check_start for group ~tp," - " overall ~tp returned 'already running (~tp)'", [Group, Overall, Pid]), - {already_in_mnesia, Pid} + " overall ~tp returned 'already running (~tp)'", + [Group, Overall, Pid]), + {already_in_store, Pid} catch %% If we are torn down while in the transaction... {error, E} -> {error, E} end. check_start(Group, Overall, Delegate, ChildSpec) -> - rabbit_log:debug("Mirrored supervisor: check_start for group ~tp, id: ~tp, overall: ~tp", - [Group, id(ChildSpec), Overall]), - ReadResult = mnesia:wread({?TABLE, {Group, id(ChildSpec)}}), - rabbit_log:debug("Mirrored supervisor: check_start table ~ts read for key ~tp returned ~tp", - [?TABLE, {Group, id(ChildSpec)}, ReadResult]), - case ReadResult of - [] -> _ = write(Group, Overall, ChildSpec), - start; - [S] -> #mirrored_sup_childspec{key = {Group, Id}, - mirroring_pid = Pid} = S, - case Overall of - Pid -> - rabbit_log:debug("Mirrored supervisor: overall matched mirrored pid ~tp", [Pid]), - child(Delegate, Id); - _ -> - rabbit_log:debug("Mirrored supervisor: overall ~tp did not match mirrored pid ~tp", [Overall, Pid]), - rabbit_log:debug("Mirrored supervisor: supervisor(~tp) returned ~tp", [Pid, supervisor(Pid)]), - case supervisor(Pid) of - dead -> - _ = write(Group, Overall, ChildSpec), - start; - Delegate0 -> - child(Delegate0, Id) - end - end + Id = id(ChildSpec), + rabbit_log:debug("Mirrored supervisor: check_start for group ~tp, id: ~tp, " + "overall: ~tp", [Group, Id, Overall]), + case rabbit_db_msup:create_or_update(Group, Overall, Delegate, ChildSpec, Id) of + Delegate0 when is_pid(Delegate0) -> + child(Delegate0, Id); + Other -> + Other end. supervisor(Pid) -> with_exit_handler(fun() -> dead end, fun() -> delegate(Pid) end). -write(Group, Overall, ChildSpec) -> - S = #mirrored_sup_childspec{key = {Group, id(ChildSpec)}, - mirroring_pid = Overall, - childspec = ChildSpec}, - ok = mnesia:write(?TABLE, S, write), - ChildSpec. - -delete(Group, Id) -> - ok = mnesia:delete({?TABLE, {Group, Id}}). - start(Delegate, ChildSpec) -> apply(?SUPERVISOR, start_child, [Delegate, ChildSpec]). -stop(Group, TxFun, Delegate, Id) -> - try TxFun(fun() -> check_stop(Group, Delegate, Id) end) of +stop(Group, Delegate, Id) -> + try check_stop(Group, Delegate, Id) of deleted -> apply(?SUPERVISOR, delete_child, [Delegate, Id]); running -> {error, running} catch @@ -476,46 +415,30 @@ stop(Group, TxFun, Delegate, Id) -> check_stop(Group, Delegate, Id) -> case child(Delegate, Id) of - undefined -> delete(Group, Id), - deleted; - _ -> running + undefined -> + rabbit_db_msup:delete(Group, Id), + deleted; + _ -> + running end. id({Id, _, _, _, _, _}) -> Id. update_all(Overall, OldOverall) -> - MatchHead = #mirrored_sup_childspec{mirroring_pid = OldOverall, - key = '$1', - childspec = '$2', - _ = '_'}, - [write(Group, Overall, C) || - [{Group, _Id}, C] <- mnesia:select(?TABLE, [{MatchHead, [], ['$$']}])]. + rabbit_db_msup:update_all(Overall, OldOverall). delete_all(Group) -> - MatchHead = #mirrored_sup_childspec{key = {Group, '_'}, - childspec = '$1', - _ = '_'}, - [delete(Group, id(C)) || - C <- mnesia:select(?TABLE, [{MatchHead, [], ['$1']}])]. + rabbit_db_msup:delete_all(Group). errors(Results) -> [E || {error, E} <- Results]. %%---------------------------------------------------------------------------- -create_tables() -> create_tables([?TABLE_DEF]). - -create_tables([]) -> - ok; -create_tables([{Table, Attributes} | Ts]) -> - case mnesia:create_table(Table, Attributes) of - {atomic, ok} -> create_tables(Ts); - {aborted, {already_exists, ?TABLE}} -> create_tables(Ts); - Err -> Err - end. +create_tables() -> + rabbit_db_msup:create_tables(). table_definitions() -> - {Name, Attributes} = ?TABLE_DEF, - [{Name, [?TABLE_MATCH | Attributes]}]. + rabbit_db_msup:table_definitions(). %%---------------------------------------------------------------------------- diff --git a/deps/rabbit/src/mirrored_supervisor_sups.erl b/deps/rabbit/src/mirrored_supervisor_sups.erl index 0c18be499a7d..06650ebb27a8 100644 --- a/deps/rabbit/src/mirrored_supervisor_sups.erl +++ b/deps/rabbit/src/mirrored_supervisor_sups.erl @@ -16,8 +16,8 @@ %%---------------------------------------------------------------------------- -init({overall, _Group, _TxFun, ignore}) -> ignore; -init({overall, Group, TxFun, {ok, {Restart, ChildSpecs}}}) -> +init({overall, _Group, ignore}) -> ignore; +init({overall, Group, {ok, {Restart, ChildSpecs}}}) -> %% Important: Delegate MUST start before Mirroring so that when we %% shut down from above it shuts down last, so Mirroring does not %% see it die. @@ -26,7 +26,7 @@ init({overall, Group, TxFun, {ok, {Restart, ChildSpecs}}}) -> {ok, {{one_for_all, 0, 1}, [{delegate, {?SUPERVISOR, start_link, [?MODULE, {delegate, Restart}]}, temporary, 16#ffffffff, supervisor, [?SUPERVISOR]}, - {mirroring, {?GS_MODULE, start_internal, [Group, TxFun, ChildSpecs]}, + {mirroring, {?GS_MODULE, start_internal, [Group, ChildSpecs]}, permanent, 16#ffffffff, worker, [?MODULE]}]}}; diff --git a/deps/rabbit/src/rabbit_db_msup.erl b/deps/rabbit/src/rabbit_db_msup.erl new file mode 100644 index 000000000000..394d14f9a1d4 --- /dev/null +++ b/deps/rabbit/src/rabbit_db_msup.erl @@ -0,0 +1,176 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_msup). + +-define(TABLE, mirrored_sup_childspec). +-define(TABLE_DEF, + {?TABLE, + [{record_name, mirrored_sup_childspec}, + {type, ordered_set}, + {attributes, record_info(fields, mirrored_sup_childspec)}]}). +-define(TABLE_MATCH, {match, #mirrored_sup_childspec{ _ = '_' }}). + +-export([create_tables/0, table_definitions/0, + create_or_update/5, delete/2, + find_mirror/2, update_all/2, delete_all/1]). + +-record(mirrored_sup_childspec, {key, mirroring_pid, childspec}). + +-spec create_tables() -> Ret when + Ret :: 'ok' | {error, Reason :: term()}. + +create_tables() -> + rabbit_db:run( + #{mnesia => fun() -> create_tables_in_mnesia([?TABLE_DEF]) end + }). + +-spec table_definitions() -> [Def] when + Def :: {Name :: atom(), term()}. + +table_definitions() -> + {Name, Attributes} = ?TABLE_DEF, + [{Name, [?TABLE_MATCH | Attributes]}]. + +-spec create_or_update(Group, Overall, Delegate, ChildSpec, Id) -> Ret when + Group :: any(), + Overall :: pid(), + Delegate :: pid() | undefined, + ChildSpec :: supervisor2:child_spec(), + Id :: {any(), any()}, + Ret :: start | undefined | pid(). + +create_or_update(Group, Overall, Delegate, ChildSpec, Id) -> + rabbit_db:run( + #{mnesia => + fun() -> + create_or_update_in_mnesia(Group, Overall, Delegate, ChildSpec, Id) + end}). + +-spec delete(Group, Id) -> ok when + Group :: any(), + Id :: any(). + +delete(Group, Id) -> + rabbit_db:run( + #{mnesia => fun() -> delete_in_mnesia(Group, Id) end + }). + +delete_in_mnesia(Group, Id) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> delete_in_mnesia_tx(Group, Id) end). + +-spec find_mirror(Group, Id) -> Ret when + Group :: any(), + Id :: any(), + Ret :: {ok, pid()} | {error, not_found}. + +find_mirror(Group, Id) -> + %% If we did this inside a tx we could still have failover + %% immediately after the tx - we can't be 100% here. So we may as + %% well dirty_select. + rabbit_db:run( + #{mnesia => fun() -> find_mirror_in_mnesia(Group, Id) end + }). + +find_mirror_in_mnesia(Group, Id) -> + MatchHead = #mirrored_sup_childspec{mirroring_pid = '$1', + key = {Group, Id}, + _ = '_'}, + case mnesia:dirty_select(?TABLE, [{MatchHead, [], ['$1']}]) of + [Mirror] -> {ok, Mirror}; + _ -> {error, not_found} + end. + +-spec update_all(Overall, Overall) -> [ChildSpec] when + Overall :: pid(), + ChildSpec :: supervisor2:child_spec(). + +update_all(Overall, OldOverall) -> + rabbit_db:run( + #{mnesia => fun() -> update_all_in_mnesia(Overall, OldOverall) end + }). + +-spec delete_all(Group) -> ok when + Group :: any(). + +delete_all(Group) -> + rabbit_db:run( + #{mnesia => fun() -> delete_all_in_mnesia(Group) end + }). + +%%---------------------------------------------------------------------------- + +create_tables_in_mnesia([]) -> + ok; +create_tables_in_mnesia([{Table, Attributes} | Ts]) -> + case mnesia:create_table(Table, Attributes) of + {atomic, ok} -> create_tables_in_mnesia(Ts); + {aborted, {already_exists, ?TABLE}} -> create_tables_in_mnesia(Ts); + Err -> Err + end. + +create_or_update_in_mnesia(Group, Overall, Delegate, ChildSpec, Id) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + ReadResult = mnesia:wread({?TABLE, {Group, Id}}), + rabbit_log:debug("Mirrored supervisor: check_start table ~ts read for key ~tp returned ~tp", + [?TABLE, {Group, Id}, ReadResult]), + case ReadResult of + [] -> _ = write_in_mnesia(Group, Overall, ChildSpec, Id), + start; + [S] -> #mirrored_sup_childspec{key = {Group, Id}, + mirroring_pid = Pid} = S, + case Overall of + Pid -> + rabbit_log:debug("Mirrored supervisor: overall matched mirrored pid ~tp", [Pid]), + Delegate; + _ -> + rabbit_log:debug("Mirrored supervisor: overall ~tp did not match mirrored pid ~tp", [Overall, Pid]), + Sup = mirrored_supervisor:supervisor(Pid), + rabbit_log:debug("Mirrored supervisor: supervisor(~tp) returned ~tp", [Pid, Sup]), + case Sup of + dead -> + _ = write_in_mnesia(Group, Overall, ChildSpec, Id), + start; + Delegate0 -> + Delegate0 + end + end + end + end). + +update_all_in_mnesia(Overall, OldOverall) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + MatchHead = #mirrored_sup_childspec{mirroring_pid = OldOverall, + key = '$1', + childspec = '$2', + _ = '_'}, + [write_in_mnesia(Group, Overall, C, Id) || + [{Group, Id}, C] <- mnesia:select(?TABLE, [{MatchHead, [], ['$$']}])] + end). + +delete_all_in_mnesia(Group) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + MatchHead = #mirrored_sup_childspec{key = {Group, '$1'}, + _ = '_'}, + [delete_in_mnesia_tx(Group, Id) || + Id <- mnesia:select(?TABLE, [{MatchHead, [], ['$1']}])] + end), + ok. + +write_in_mnesia(Group, Overall, ChildSpec, Id) -> + S = #mirrored_sup_childspec{key = {Group, Id}, + mirroring_pid = Overall, + childspec = ChildSpec}, + ok = mnesia:write(?TABLE, S, write), + ChildSpec. + +delete_in_mnesia_tx(Group, Id) -> + ok = mnesia:delete({?TABLE, {Group, Id}}). diff --git a/deps/rabbit/test/mirrored_supervisor_SUITE.erl b/deps/rabbit/test/mirrored_supervisor_SUITE.erl index 9f8880062d85..8afb3a6b1aeb 100644 --- a/deps/rabbit/test/mirrored_supervisor_SUITE.erl +++ b/deps/rabbit/test/mirrored_supervisor_SUITE.erl @@ -18,232 +18,299 @@ all() -> [ - migrate, - migrate_twice, - already_there, - delete_restart, - which_children, - large_group, - childspecs_at_init, - anonymous_supervisors, - no_migration_on_shutdown, - start_idempotence, - unsupported, - ignore, - startup_failure + {group, broker_tests} ]. +groups() -> + [ + {broker_tests, [], [ + migrate, + migrate_twice, + already_there, + delete_restart, + which_children, + large_group, + childspecs_at_init, + anonymous_supervisors, + no_migration_on_shutdown, + start_idempotence, + unsupported, + ignore, + startup_failure + ]} + ]. + + init_per_suite(Config) -> - ok = application:set_env(mnesia, dir, ?config(priv_dir, Config)), - ok = application:start(mnesia), - lists:foreach( - fun ({Tab, TabDef}) -> - TabDef1 = proplists:delete(match, TabDef), - case mnesia:create_table(Tab, TabDef1) of - {atomic, ok} -> - ok; - {aborted, Reason} -> - throw({error, - {table_creation_failed, Tab, TabDef1, Reason}}) - end - end, mirrored_supervisor:table_definitions()), - Config. + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config). end_per_suite(Config) -> - ok = application:stop(mnesia), - Config. - + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(Group, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, Group}, + {rmq_nodes_count, 1} + ]), + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_group(_Group, Config) -> + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_testcase(Testcase, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [{sup_prefix, Testcase}]), + rabbit_ct_helpers:testcase_started(Config1, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_finished(Config, Testcase). %% --------------------------------------------------------------------------- %% Functional tests %% --------------------------------------------------------------------------- %% Simplest test -migrate(_Config) -> - passed = with_sups( +migrate(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, migrate1, [?config(sup_prefix, Config)]). + +migrate1(Sup) -> + with_sups( fun([A, _]) -> - {ok, _} = ?MS:start_child(a, childspec(worker)), + {ok, _} = ?MS:start_child(sup(Sup, 1), childspec(worker)), Pid1 = pid_of(worker), kill_registered(A, Pid1), Pid2 = pid_of(worker), false = (Pid1 =:= Pid2) - end, [a, b]). + end, [sup(Sup, 1), sup(Sup, 2)], Sup). %% Is migration transitive? -migrate_twice(_Config) -> - passed = with_sups( +migrate_twice(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, migrate_twice1, [?config(sup_prefix, Config)]). + +migrate_twice1(Sup) -> + with_sups( fun([A, B]) -> - {ok, _} = ?MS:start_child(a, childspec(worker)), + {ok, _} = ?MS:start_child(sup(Sup, 1), childspec(worker)), Pid1 = pid_of(worker), kill_registered(A, Pid1), - {ok, C} = start_sup(c), + {ok, C} = start_sup(sup(Sup, 3), Sup), Pid2 = pid_of(worker), kill_registered(B, Pid2), Pid3 = pid_of(worker), false = (Pid1 =:= Pid3), kill(C) - end, [a, b]). + end, [sup(Sup, 1), sup(Sup, 2)], Sup). %% Can't start the same child twice -already_there(_Config) -> - passed = with_sups( +already_there(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, already_there1, [?config(sup_prefix, Config)]). + +already_there1(Sup) -> + with_sups( fun([_, _]) -> S = childspec(worker), - {ok, Pid} = ?MS:start_child(a, S), - {error, {already_started, Pid}} = ?MS:start_child(b, S) - end, [a, b]). + {ok, Pid} = ?MS:start_child(sup(Sup, 1), S), + {error, {already_started, Pid}} = ?MS:start_child(sup(Sup, 2), S) + end, [sup(Sup, 1), sup(Sup, 2)], Sup). %% Deleting and restarting should work as per a normal supervisor -delete_restart(_Config) -> - passed = with_sups( +delete_restart(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, delete_restart1, [?config(sup_prefix, Config)]). + +delete_restart1(Sup) -> + Sup1 = sup(Sup, 1), + Sup2 = sup(Sup, 2), + with_sups( fun([_, _]) -> S = childspec(worker), - {ok, Pid1} = ?MS:start_child(a, S), - {error, running} = ?MS:delete_child(a, worker), - ok = ?MS:terminate_child(a, worker), - ok = ?MS:delete_child(a, worker), - {ok, Pid2} = ?MS:start_child(b, S), + {ok, Pid1} = ?MS:start_child(Sup1, S), + {error, running} = ?MS:delete_child(Sup1, id(worker)), + ok = ?MS:terminate_child(Sup1, id(worker)), + ok = ?MS:delete_child(Sup1, id(worker)), + {ok, Pid2} = ?MS:start_child(Sup2, S), false = (Pid1 =:= Pid2), - ok = ?MS:terminate_child(b, worker), - {ok, Pid3} = ?MS:restart_child(b, worker), + ok = ?MS:terminate_child(Sup2, id(worker)), + {ok, Pid3} = ?MS:restart_child(Sup2, id(worker)), Pid3 = pid_of(worker), false = (Pid2 =:= Pid3), %% Not the same supervisor as the worker is on - ok = ?MS:terminate_child(a, worker), - ok = ?MS:delete_child(a, worker), - {ok, Pid4} = ?MS:start_child(a, S), + ok = ?MS:terminate_child(Sup1, id(worker)), + ok = ?MS:delete_child(Sup1, id(worker)), + {ok, Pid4} = ?MS:start_child(Sup1, S), false = (Pid3 =:= Pid4) - end, [a, b]). + end, [Sup1, Sup2], Sup). -which_children(_Config) -> - passed = with_sups( +which_children(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, which_children1, [?config(sup_prefix, Config)]). + +which_children1(Sup) -> + Sup1 = sup(Sup, 1), + Sup2 = sup(Sup, 2), + with_sups( fun([A, B] = Both) -> ?MS:start_child(A, childspec(worker)), assert_wc(Both, fun ([C]) -> true = is_pid(wc_pid(C)) end), - ok = ?MS:terminate_child(a, worker), + ok = ?MS:terminate_child(Sup1, id(worker)), assert_wc(Both, fun ([C]) -> undefined = wc_pid(C) end), - {ok, _} = ?MS:restart_child(a, worker), + {ok, _} = ?MS:restart_child(Sup1, id(worker)), assert_wc(Both, fun ([C]) -> true = is_pid(wc_pid(C)) end), ?MS:start_child(B, childspec(worker2)), assert_wc(Both, fun (C) -> 2 = length(C) end) - end, [a, b]). + end, [Sup1, Sup2], Sup). assert_wc(Sups, Fun) -> [Fun(?MS:which_children(Sup)) || Sup <- Sups]. wc_pid(Child) -> - {worker, Pid, worker, [?MODULE]} = Child, + {_, Pid, worker, [?MODULE]} = Child, Pid. %% Not all the members of the group should actually do the failover -large_group(_Config) -> - passed = with_sups( +large_group(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, large_group1, [?config(sup_prefix, Config)]). + +large_group1(Sup) -> + with_sups( fun([A, _, _, _]) -> - {ok, _} = ?MS:start_child(a, childspec(worker)), + {ok, _} = ?MS:start_child(sup(Sup, 1), childspec(worker)), Pid1 = pid_of(worker), kill_registered(A, Pid1), Pid2 = pid_of(worker), false = (Pid1 =:= Pid2) - end, [a, b, c, d]). + end, [sup(Sup, 1), sup(Sup, 2), sup(Sup, 3), sup(Sup, 4)], Sup). %% Do childspecs work when returned from init? -childspecs_at_init(_Config) -> +childspecs_at_init(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, childspecs_at_init1, [?config(sup_prefix, Config)]). + +childspecs_at_init1(Sup) -> S = childspec(worker), - passed = with_sups( + with_sups( fun([A, _]) -> Pid1 = pid_of(worker), kill_registered(A, Pid1), Pid2 = pid_of(worker), false = (Pid1 =:= Pid2) - end, [{a, [S]}, {b, [S]}]). + end, [{sup(Sup, 1), [S]}, {sup(Sup, 2), [S]}], Sup). + +anonymous_supervisors(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, anonymous_supervisors1, [?config(sup_prefix, Config)]). -anonymous_supervisors(_Config) -> - passed = with_sups( +anonymous_supervisors1(Sup) -> + with_sups( fun([A, _B]) -> {ok, _} = ?MS:start_child(A, childspec(worker)), Pid1 = pid_of(worker), kill_registered(A, Pid1), Pid2 = pid_of(worker), false = (Pid1 =:= Pid2) - end, [anon, anon]). + end, [sup(Sup, 1), sup(Sup, 2)], Sup). %% When a mirrored_supervisor terminates, we should not migrate, but %% the whole supervisor group should shut down. To test this we set up %% a situation where the gen_server will only fail if it's running %% under the supervisor called 'evil'. It should not migrate to %% 'good' and survive, rather the whole group should go away. -no_migration_on_shutdown(_Config) -> - passed = with_sups( +no_migration_on_shutdown(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, no_migration_on_shutdown1, [?config(sup_prefix, Config)]). + +no_migration_on_shutdown1(Sup) -> + with_sups( fun([Evil, _]) -> - {ok, _} = ?MS:start_child(Evil, childspec(worker)), + {ok, _} = ?MS:start_child(Evil, childspec(worker3)), try - call(worker, ping, 1000, 100), + call(worker3, ping, 1000, 100), exit(worker_should_not_have_migrated) catch exit:{timeout_waiting_for_server, _, _} -> ok end - end, [evil, good]). + end, [evil, good], Sup). -start_idempotence(_Config) -> - passed = with_sups( +start_idempotence(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, start_idempotence1, [?config(sup_prefix, Config)]). + +start_idempotence1(Sup) -> + Sup1 = sup(Sup, 1), + with_sups( fun([_]) -> - CS = childspec(worker), - {ok, Pid} = ?MS:start_child(a, CS), - {error, {already_started, Pid}} = ?MS:start_child(a, CS), - ?MS:terminate_child(a, worker), - {error, already_present} = ?MS:start_child(a, CS) - end, [a]). - -unsupported(_Config) -> + CS = childspec(worker2), + {ok, Pid} = ?MS:start_child(Sup1, CS), + {error, {already_started, Pid}} = ?MS:start_child(Sup1, CS), + ?MS:terminate_child(Sup1, id(worker2)), + {error, already_present} = ?MS:start_child(Sup1, CS) + end, [Sup1], Sup). + +unsupported(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, unsupported1, [?config(sup_prefix, Config)]). + +unsupported1(Sup) -> try - ?MS:start_link({global, foo}, get_group(group), fun tx_fun/1, ?MODULE, - {one_for_one, []}), + ?MS:start_link({global, foo}, Sup, ?MODULE, {one_for_one, []}), exit(no_global) catch error:badarg -> ok end, try - {ok, _} = ?MS:start_link({local, foo}, get_group(group), - fun tx_fun/1, ?MODULE, {simple_one_for_one, []}), + {ok, _} = ?MS:start_link({local, foo}, Sup, ?MODULE, {simple_one_for_one, []}), exit(no_sofo) catch error:badarg -> - ok + passed end. %% Just test we don't blow up -ignore(_Config) -> - ?MS:start_link({local, foo}, get_group(group), fun tx_fun/1, ?MODULE, - {fake_strategy_for_ignore, []}). +ignore(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, ignore1, [?config(sup_prefix, Config)]). + +ignore1(Sup) -> + ?MS:start_link({local, foo}, Sup, ?MODULE, {fake_strategy_for_ignore, []}), + passed. -startup_failure(_Config) -> - [test_startup_failure(F) || F <- [want_error, want_exit]]. +startup_failure(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, startup_failure1, [?config(sup_prefix, Config)]). -test_startup_failure(Fail) -> +startup_failure1(Sup) -> + [test_startup_failure(F, Sup) || F <- [want_error, want_exit]], + passed. + +test_startup_failure(Fail, Group) -> process_flag(trap_exit, true), - ?MS:start_link(get_group(group), fun tx_fun/1, ?MODULE, - {one_for_one, [childspec(Fail)]}), + ?MS:start_link(Group, ?MODULE, {one_for_one, [childspec(Fail)]}), receive {'EXIT', _, shutdown} -> ok after 1000 -> exit({did_not_exit, Fail}) end, - process_flag(trap_exit, false). + process_flag(trap_exit, false), + ok. %% --------------------------------------------------------------------------- -with_sups(Fun, Sups) -> - inc_group(), - Pids = [begin {ok, Pid} = start_sup(Sup), Pid end || Sup <- Sups], +with_sups(Fun, Sups, Group) -> + Pids = [begin {ok, Pid} = start_sup(Sup, Group), Pid end || Sup <- Sups], Fun(Pids), [kill(Pid) || Pid <- Pids, is_process_alive(Pid)], timer:sleep(500), passed. -start_sup(Spec) -> - start_sup(Spec, group). - start_sup({Name, ChildSpecs}, Group) -> - {ok, Pid} = start_sup0(Name, get_group(Group), ChildSpecs), + {ok, Pid} = start_sup0(Name, Group, ChildSpecs), %% We are not a supervisor, when we kill the supervisor we do not %% want to die! unlink(Pid), @@ -253,36 +320,23 @@ start_sup(Name, Group) -> start_sup({Name, []}, Group). start_sup0(anon, Group, ChildSpecs) -> - ?MS:start_link(Group, fun tx_fun/1, ?MODULE, + ?MS:start_link(Group, ?MODULE, {one_for_one, ChildSpecs}); start_sup0(Name, Group, ChildSpecs) -> - ?MS:start_link({local, Name}, Group, fun tx_fun/1, ?MODULE, + ?MS:start_link({local, Name}, Group, ?MODULE, {one_for_one, ChildSpecs}). childspec(Id) -> - {Id,{?SERVER, start_link, [Id]}, transient, 16#ffffffff, worker, [?MODULE]}. + {id(Id), {?SERVER, start_link, [Id]}, transient, 16#ffffffff, worker, [?MODULE]}. + +id(Id) -> + {[Id], Id}. pid_of(Id) -> {received, Pid, ping} = call(Id, ping), Pid. -tx_fun(Fun) -> - case mnesia:sync_transaction(Fun) of - {atomic, Result} -> Result; - {aborted, Reason} -> throw({error, Reason}) - end. - -inc_group() -> - Count = case get(counter) of - undefined -> 0; - C -> C - end + 1, - put(counter, Count). - -get_group(Group) -> - {Group, get(counter)}. - call(Id, Msg) -> call(Id, Msg, 10*1000, 100). call(Id, Msg, MaxDelay, Decr) -> @@ -326,3 +380,6 @@ init({fake_strategy_for_ignore, _ChildSpecs}) -> init({Strategy, ChildSpecs}) -> {ok, {{Strategy, 0, 1}, ChildSpecs}}. + +sup(Prefix, Number) -> + rabbit_data_coercion:to_atom(lists:flatten(io_lib:format("~p~p", [Prefix, Number]))). From 90aaf3a87d56630be8216f474ef1ac5d65b69b44 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Fri, 13 Jan 2023 10:40:32 +0100 Subject: [PATCH 04/12] Adapt federation and shovel to changes in exchange behaviour and mirrored_supervisor --- .../src/rabbit_federation_exchange.erl | 14 +++----------- .../rabbit_federation_exchange_link_sup_sup.erl | 1 - .../src/rabbit_federation_queue_link_sup_sup.erl | 1 - .../src/rabbit_federation_upstream_exchange.erl | 4 ++-- .../src/rabbit_shovel_dyn_worker_sup_sup.erl | 2 +- .../src/rabbit_shovel_worker_sup.erl | 1 - 6 files changed, 6 insertions(+), 17 deletions(-) diff --git a/deps/rabbitmq_federation/src/rabbit_federation_exchange.erl b/deps/rabbitmq_federation/src/rabbit_federation_exchange.erl index 5485f6cda20f..0ff8db37a2a2 100644 --- a/deps/rabbitmq_federation/src/rabbit_federation_exchange.erl +++ b/deps/rabbitmq_federation/src/rabbit_federation_exchange.erl @@ -21,7 +21,7 @@ -behaviour(rabbit_exchange_decorator). -export([description/0, serialise_events/1]). --export([create/2, delete/3, policy_changed/2, +-export([create/2, delete/2, policy_changed/2, add_binding/3, remove_bindings/3, route/2, active_for/1]). %%---------------------------------------------------------------------------- @@ -31,22 +31,16 @@ description() -> serialise_events(X) -> federate(X). -create(transaction, _X) -> - ok; -create(none, X) -> +create(_Serial, X) -> maybe_start(X). -delete(transaction, _X, _Bs) -> - ok; -delete(none, X, _Bs) -> +delete(_Serial, X) -> maybe_stop(X). policy_changed(OldX, NewX) -> maybe_stop(OldX), maybe_start(NewX). -add_binding(transaction, _X, _B) -> - ok; add_binding(Serial, X = #exchange{name = XName}, B) -> case federate(X) of true -> _ = rabbit_federation_exchange_link:add_binding(Serial, XName, B), @@ -54,8 +48,6 @@ add_binding(Serial, X = #exchange{name = XName}, B) -> false -> ok end. -remove_bindings(transaction, _X, _Bs) -> - ok; remove_bindings(Serial, X = #exchange{name = XName}, Bs) -> case federate(X) of true -> _ = rabbit_federation_exchange_link:remove_bindings(Serial, XName, Bs), diff --git a/deps/rabbitmq_federation/src/rabbit_federation_exchange_link_sup_sup.erl b/deps/rabbitmq_federation/src/rabbit_federation_exchange_link_sup_sup.erl index 69f2b930c67f..dd4402e467b9 100644 --- a/deps/rabbitmq_federation/src/rabbit_federation_exchange_link_sup_sup.erl +++ b/deps/rabbitmq_federation/src/rabbit_federation_exchange_link_sup_sup.erl @@ -27,7 +27,6 @@ start_link() -> %% The scope is stopped in stop/1. _ = rabbit_federation_pg:start_scope(), mirrored_supervisor:start_link({local, ?SUPERVISOR}, ?SUPERVISOR, - fun rabbit_misc:execute_mnesia_transaction/1, ?MODULE, []). %% Note that the next supervisor down, rabbit_federation_link_sup, is common diff --git a/deps/rabbitmq_federation/src/rabbit_federation_queue_link_sup_sup.erl b/deps/rabbitmq_federation/src/rabbit_federation_queue_link_sup_sup.erl index 9ce6806e7404..65c3770917ab 100644 --- a/deps/rabbitmq_federation/src/rabbit_federation_queue_link_sup_sup.erl +++ b/deps/rabbitmq_federation/src/rabbit_federation_queue_link_sup_sup.erl @@ -28,7 +28,6 @@ start_link() -> %% The scope is stopped in stop/1. _ = rabbit_federation_pg:start_scope(), mirrored_supervisor:start_link({local, ?SUPERVISOR}, ?SUPERVISOR, - fun rabbit_misc:execute_mnesia_transaction/1, ?MODULE, []). %% Note that the next supervisor down, rabbit_federation_link_sup, is common diff --git a/deps/rabbitmq_federation/src/rabbit_federation_upstream_exchange.erl b/deps/rabbitmq_federation/src/rabbit_federation_upstream_exchange.erl index e7f5a4b18628..950019262894 100644 --- a/deps/rabbitmq_federation/src/rabbit_federation_upstream_exchange.erl +++ b/deps/rabbitmq_federation/src/rabbit_federation_upstream_exchange.erl @@ -23,7 +23,7 @@ -export([description/0, serialise_events/0, route/2]). -export([validate/1, validate_binding/2, - create/2, delete/3, policy_changed/2, + create/2, delete/2, policy_changed/2, add_binding/3, remove_bindings/3, assert_args_equivalence/2]). -export([info/1, info/2]). @@ -64,7 +64,7 @@ validate(#exchange{arguments = Args}) -> validate_binding(_X, _B) -> ok. create(_Tx, _X) -> ok. -delete(_Tx, _X, _Bs) -> ok. +delete(_Tx, _X) -> ok. policy_changed(_X1, _X2) -> ok. add_binding(_Tx, _X, _B) -> ok. remove_bindings(_Tx, _X, _Bs) -> ok. diff --git a/deps/rabbitmq_shovel/src/rabbit_shovel_dyn_worker_sup_sup.erl b/deps/rabbitmq_shovel/src/rabbit_shovel_dyn_worker_sup_sup.erl index 86279435e107..c9b82335f05d 100644 --- a/deps/rabbitmq_shovel/src/rabbit_shovel_dyn_worker_sup_sup.erl +++ b/deps/rabbitmq_shovel/src/rabbit_shovel_dyn_worker_sup_sup.erl @@ -20,7 +20,7 @@ start_link() -> Pid = case mirrored_supervisor:start_link( {local, ?SUPERVISOR}, ?SUPERVISOR, - fun rabbit_misc:execute_mnesia_transaction/1, ?MODULE, []) of + ?MODULE, []) of {ok, Pid0} -> Pid0; {error, {already_started, Pid0}} -> Pid0 end, diff --git a/deps/rabbitmq_shovel/src/rabbit_shovel_worker_sup.erl b/deps/rabbitmq_shovel/src/rabbit_shovel_worker_sup.erl index f3e44cf63ddf..35764c628da0 100644 --- a/deps/rabbitmq_shovel/src/rabbit_shovel_worker_sup.erl +++ b/deps/rabbitmq_shovel/src/rabbit_shovel_worker_sup.erl @@ -15,7 +15,6 @@ start_link(ShovelName, ShovelConfig) -> mirrored_supervisor:start_link({local, ShovelName}, ShovelName, - fun rabbit_misc:execute_mnesia_transaction/1, ?MODULE, [ShovelName, ShovelConfig]). init([Name, Config]) -> From 5cdb6e590a2b3a9c1605736eb2c65b6432c0ad15 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Fri, 13 Jan 2023 13:27:44 +0100 Subject: [PATCH 05/12] Move rabbit_exchange_type to rabbit --- deps/{rabbit_common => rabbit}/src/rabbit_exchange_type.erl | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename deps/{rabbit_common => rabbit}/src/rabbit_exchange_type.erl (100%) diff --git a/deps/rabbit_common/src/rabbit_exchange_type.erl b/deps/rabbit/src/rabbit_exchange_type.erl similarity index 100% rename from deps/rabbit_common/src/rabbit_exchange_type.erl rename to deps/rabbit/src/rabbit_exchange_type.erl From ba1670a95a71829b3d6ee4555b64ad386ec66614 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Fri, 13 Jan 2023 13:41:19 +0100 Subject: [PATCH 06/12] Move recent history exchange Mnesia-specific code to rabbit_db_* modules --- .../src/rabbit_db_rh_exchange.erl | 115 ++++++++++++++++++ .../rabbit_exchange_type_recent_history.erl | 53 ++------ 2 files changed, 124 insertions(+), 44 deletions(-) create mode 100644 deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl diff --git a/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl b/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl new file mode 100644 index 000000000000..49b04dca839b --- /dev/null +++ b/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl @@ -0,0 +1,115 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_rh_exchange). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include("rabbit_recent_history.hrl"). + +-export([ + setup_schema/0, + get/1, + insert/3, + delete/0, + delete/1 + ]). + +%% ------------------------------------------------------------------- +%% setup_schema(). +%% ------------------------------------------------------------------- + +setup_schema() -> + rabbit_db:run( + #{mnesia => fun() -> setup_schema_in_mnesia() end + }). + +setup_schema_in_mnesia() -> + _ = mnesia:create_table(?RH_TABLE, + [{attributes, record_info(fields, cached)}, + {record_name, cached}, + {type, set}]), + _ = mnesia:add_table_copy(?RH_TABLE, node(), ram_copies), + rabbit_table:wait([?RH_TABLE]), + ok. + +%% ------------------------------------------------------------------- +%% get(). +%% ------------------------------------------------------------------- + +get(XName) -> + rabbit_db:run( + #{mnesia => fun() -> get_in_mnesia(XName) end + }). + +get_in_mnesia(XName) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> get_in_mnesia_tx(XName) end). + +get_in_mnesia_tx(XName) -> + case mnesia:read(?RH_TABLE, XName) of + [] -> + []; + [#cached{key = XName, content=Cached}] -> + Cached + end. + +%% ------------------------------------------------------------------- +%% insert(). +%% ------------------------------------------------------------------- + +insert(XName, Message, Length) -> + rabbit_db:run( + #{mnesia => fun() -> insert_in_mnesia(XName, Message, Length) end + }). + +insert_in_mnesia(XName, Message, Length) -> + rabbit_misc:execute_mnesia_transaction( + fun () -> + Cached = get_in_mnesia_tx(XName), + insert_in_mnesia(XName, Cached, Message, Length) + end). + +insert_in_mnesia(Key, Cached, Message, undefined) -> + insert0_in_mnesia(Key, Cached, Message, ?KEEP_NB); +insert_in_mnesia(Key, Cached, Message, {_Type, Length}) -> + insert0_in_mnesia(Key, Cached, Message, Length). + +insert0_in_mnesia(Key, Cached, Message, Length) -> + mnesia:write(?RH_TABLE, + #cached{key = Key, + content = [Message|lists:sublist(Cached, Length-1)]}, + write). + +add_to_cache(Cached, Message, undefined) -> + add_to_cache(Cached, Message, ?KEEP_NB); +add_to_cache(Cached, Message, {_Type, Length}) -> + add_to_cache(Cached, Message, Length); +add_to_cache(Cached, Message, Length) -> + [Message|lists:sublist(Cached, Length-1)]. + +%% ------------------------------------------------------------------- +%% delete(). +%% ------------------------------------------------------------------- + +delete() -> + rabbit_db:run( + #{mnesia => fun() -> delete_in_mnesia() end + }). + +delete_in_mnesia() -> + _ = mnesia:delete_table(?RH_TABLE). + +delete(XName) -> + rabbit_db:run( + #{mnesia => fun() -> delete_in_mnesia(XName) end + }). + +delete_in_mnesia(XName) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + mnesia:delete(?RH_TABLE, XName, write) + end). diff --git a/deps/rabbitmq_recent_history_exchange/src/rabbit_exchange_type_recent_history.erl b/deps/rabbitmq_recent_history_exchange/src/rabbit_exchange_type_recent_history.erl index c0c08bed213e..685c8463db9e 100644 --- a/deps/rabbitmq_recent_history_exchange/src/rabbit_exchange_type_recent_history.erl +++ b/deps/rabbitmq_recent_history_exchange/src/rabbit_exchange_type_recent_history.erl @@ -15,7 +15,7 @@ -import(rabbit_misc, [table_lookup/2]). -export([description/0, serialise_events/0, route/2]). --export([validate/1, validate_binding/2, create/2, delete/3, add_binding/3, +-export([validate/1, validate_binding/2, create/2, delete/2, add_binding/3, remove_bindings/3, assert_args_equivalence/2, policy_changed/2]). -export([setup_schema/0, disable_plugin/0]). -export([info/1, info/2]). @@ -73,16 +73,10 @@ validate_binding(_X, _B) -> ok. create(_Tx, _X) -> ok. policy_changed(_X1, _X2) -> ok. -delete(transaction, #exchange{ name = XName }, _Bs) -> - rabbit_misc:execute_mnesia_transaction( - fun() -> - mnesia:delete(?RH_TABLE, XName, write) - end), - ok; -delete(none, _Exchange, _Bs) -> - ok. +delete(none, #exchange{ name = XName }) -> + rabbit_db_rh_exchange:delete(XName). -add_binding(transaction, #exchange{ name = XName }, +add_binding(none, #exchange{ name = XName }, #binding{ destination = #resource{kind = queue} = QName }) -> _ = case rabbit_amqqueue:lookup(QName) of {error, not_found} -> @@ -92,7 +86,7 @@ add_binding(transaction, #exchange{ name = XName }, deliver_messages([Q], Msgs) end, ok; -add_binding(transaction, #exchange{ name = XName }, +add_binding(none, #exchange{ name = XName }, #binding{ destination = #resource{kind = exchange} = DestName }) -> _ = case rabbit_exchange:lookup(DestName) of {error, not_found} -> @@ -122,17 +116,11 @@ assert_args_equivalence(X, Args) -> %%---------------------------------------------------------------------------- setup_schema() -> - _ = mnesia:create_table(?RH_TABLE, - [{attributes, record_info(fields, cached)}, - {record_name, cached}, - {type, set}]), - _ = mnesia:add_table_copy(?RH_TABLE, node(), ram_copies), - rabbit_table:wait([?RH_TABLE]), - ok. + rabbit_db_rh_exchange:setup_schema(). disable_plugin() -> rabbit_registry:unregister(exchange, <<"x-recent-history">>), - _ = mnesia:delete_table(?RH_TABLE), + rabbit_db_rh_exchange:delete(), ok. %%---------------------------------------------------------------------------- @@ -157,33 +145,10 @@ maybe_cache_msg(XName, end. cache_msg(XName, Message, Length) -> - rabbit_misc:execute_mnesia_transaction( - fun () -> - Cached = get_msgs_from_cache(XName), - store_msg(XName, Cached, Message, Length) - end). + rabbit_db_rh_exchange:insert(XName, Message, Length). get_msgs_from_cache(XName) -> - rabbit_misc:execute_mnesia_transaction( - fun () -> - case mnesia:read(?RH_TABLE, XName) of - [] -> - []; - [#cached{key = XName, content=Cached}] -> - Cached - end - end). - -store_msg(Key, Cached, Message, undefined) -> - store_msg0(Key, Cached, Message, ?KEEP_NB); -store_msg(Key, Cached, Message, {_Type, Length}) -> - store_msg0(Key, Cached, Message, Length). - -store_msg0(Key, Cached, Message, Length) -> - mnesia:write(?RH_TABLE, - #cached{key = Key, - content = [Message|lists:sublist(Cached, Length-1)]}, - write). + rabbit_db_rh_exchange:get(XName). deliver_messages(Qs, Msgs) -> lists:map( From 5b39e7e4cec8d844c2ac4ad5809065ecaa0251b2 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Fri, 13 Jan 2023 14:01:52 +0100 Subject: [PATCH 07/12] Move jms topic exchange Mnesia-specific code to rabbit_db_* modules --- .../src/rabbit_db_jms_exchange.erl | 143 ++++++++++++++++++ .../src/rabbit_jms_topic_exchange.erl | 83 ++-------- .../test/rjms_topic_selector_unit_SUITE.erl | 27 +--- 3 files changed, 162 insertions(+), 91 deletions(-) create mode 100644 deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange.erl diff --git a/deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange.erl b/deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange.erl new file mode 100644 index 000000000000..6ce28123ea83 --- /dev/null +++ b/deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange.erl @@ -0,0 +1,143 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2023 VMware, Inc. or its affiliates. All rights reserved. +%% ----------------------------------------------------------------------------- +-module(rabbit_db_jms_exchange). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include("rabbit_jms_topic_exchange.hrl"). + +-export([ + setup_schema/0, + create_or_update/3, + insert/2, + get/1, + delete/1, + delete/3 + ]). + +%% ------------------------------------------------------------------- +%% setup_schema() +%% ------------------------------------------------------------------- + +setup_schema() -> + rabbit_db:run( + #{mnesia => fun() -> setup_schema_in_mnesia() end + }). + +setup_schema_in_mnesia() -> + case mnesia:create_table( ?JMS_TOPIC_TABLE + , [ {attributes, record_info(fields, ?JMS_TOPIC_RECORD)} + , {record_name, ?JMS_TOPIC_RECORD} + , {type, set} ] + ) of + {atomic, ok} -> ok; + {aborted, {already_exists, ?JMS_TOPIC_TABLE}} -> ok + end, + ok. + +%% ------------------------------------------------------------------- +%% create_or_update(). +%% ------------------------------------------------------------------- + +create_or_update(XName, BindingKeyAndFun, ErrorFun) -> + rabbit_db:run( + #{mnesia => + fun() -> create_or_update_in_mnesia(XName, BindingKeyAndFun, ErrorFun) end + }). + +create_or_update_in_mnesia(XName, BindingKeyAndFun, ErrorFun) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + #?JMS_TOPIC_RECORD{x_selector_funs = BindingFuns} = + read_state_in_mnesia(XName, ErrorFun), + write_state_fun_in_mnesia(XName, put_item(BindingFuns, BindingKeyAndFun)) + end). + +%% ------------------------------------------------------------------- +%% insert(). +%% ------------------------------------------------------------------- + +insert(XName, BFuns) -> + rabbit_db:run( + #{mnesia => fun() -> insert_in_mnesia(XName, BFuns) end + }). + +insert_in_mnesia(XName, BFuns) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + write_state_fun_in_mnesia(XName, BFuns) + end). + +%% ------------------------------------------------------------------- +%% get(). +%% ------------------------------------------------------------------- + +get(XName) -> + rabbit_db:run( + #{mnesia => fun() -> get_in_mnesia(XName) end + }). + +get_in_mnesia(XName) -> + mnesia:async_dirty( + fun() -> + case mnesia:read(?JMS_TOPIC_TABLE, XName, read) of + [#?JMS_TOPIC_RECORD{x_selector_funs = BindingFuns}] -> + BindingFuns; + _ -> + not_found + end + end, + [] + ). + +%% ------------------------------------------------------------------- +%% delete(). +%% ------------------------------------------------------------------- + +delete(XName) -> + rabbit_db:run( + #{mnesia => fun() -> delete_in_mnesia(XName) end + }). + +delete_in_mnesia(XName) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> mnesia:delete(?JMS_TOPIC_TABLE, XName, write) end). + +delete(XName, BindingKeys, ErrorFun) -> + rabbit_db:run( + #{mnesia => + fun() -> delete_in_mnesia(XName, BindingKeys, ErrorFun) end + }). + +delete_in_mnesia(XName, BindingKeys, ErrorFun) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + #?JMS_TOPIC_RECORD{x_selector_funs = BindingFuns} = + read_state_in_mnesia(XName, ErrorFun), + write_state_fun_in_mnesia(XName, remove_items(BindingFuns, BindingKeys)) + end). + +read_state_in_mnesia(XName, ErrorFun) -> + case mnesia:read(?JMS_TOPIC_TABLE, XName, write) of + [Rec] -> Rec; + _ -> ErrorFun(XName) + end. + +write_state_fun_in_mnesia(XName, BFuns) -> + mnesia:write( ?JMS_TOPIC_TABLE + , #?JMS_TOPIC_RECORD{x_name = XName, x_selector_funs = BFuns} + , write ). + +%% ------------------------------------------------------------------- +%% dictionary handling +%% ------------------------------------------------------------------- + +% add an item to the dictionary of binding functions +put_item(Dict, {Key, Item}) -> dict:store(Key, Item, Dict). + +% remove a list of keyed items from the dictionary, by key +remove_items(Dict, []) -> Dict; +remove_items(Dict, [Key | Keys]) -> remove_items(dict:erase(Key, Dict), Keys). diff --git a/deps/rabbitmq_jms_topic_exchange/src/rabbit_jms_topic_exchange.erl b/deps/rabbitmq_jms_topic_exchange/src/rabbit_jms_topic_exchange.erl index 8032feb49c3d..66288d3f9a15 100644 --- a/deps/rabbitmq_jms_topic_exchange/src/rabbit_jms_topic_exchange.erl +++ b/deps/rabbitmq_jms_topic_exchange/src/rabbit_jms_topic_exchange.erl @@ -22,7 +22,7 @@ , route/2 , validate/1 , create/2 - , delete/3 + , delete/2 , validate_binding/2 , add_binding/3 , remove_bindings/3 @@ -55,14 +55,7 @@ % Initialise database table for all exchanges of type <<"x-jms-topic">> setup_db_schema() -> - case mnesia:create_table( ?JMS_TOPIC_TABLE - , [ {attributes, record_info(fields, ?JMS_TOPIC_RECORD)} - , {record_name, ?JMS_TOPIC_RECORD} - , {type, set} ] - ) of - {atomic, ok} -> ok; - {aborted, {already_exists, ?JMS_TOPIC_TABLE}} -> ok - end. + rabbit_db_jms_exchange:setup_schema(). %%---------------------------------------------------------------------------- %% R E F E R E N C E T Y P E I N F O R M A T I O N @@ -111,30 +104,25 @@ route( #exchange{name = XName} validate(_X) -> ok. % After exchange declaration and recovery -create(transaction, #exchange{name = XName}) -> - add_initial_record(XName); -create(_Tx, _X) -> - ok. +create(none, #exchange{name = XName}) -> + add_initial_record(XName). % Delete an exchange -delete(transaction, #exchange{name = XName}, _Bs) -> - delete_state(XName), - ok; -delete(_Tx, _X, _Bs) -> - ok. +delete(none, #exchange{name = XName}) -> + delete_state(XName). % Before add binding validate_binding(_X, _B) -> ok. % A new binding has ben added or recovered -add_binding( Tx +add_binding( none , #exchange{name = XName} , #binding{key = BindingKey, destination = Dest, args = Args} ) -> Selector = get_string_arg(Args, ?RJMS_COMPILED_SELECTOR_ARG), BindGen = generate_binding_fun(Selector), - case {Tx, BindGen} of - {transaction, {ok, BindFun}} -> + case BindGen of + {ok, BindFun} -> add_binding_fun(XName, {{BindingKey, Dest}, BindFun}); {none, error} -> parsing_error(XName, Selector, Dest); @@ -144,13 +132,11 @@ add_binding( Tx ok. % Binding removal -remove_bindings( transaction +remove_bindings( none , #exchange{name = XName} , Bindings ) -> remove_binding_funs(XName, Bindings), - ok; -remove_bindings(_Tx, _X, _Bs) -> ok. % Exchange argument equivalence @@ -234,66 +220,27 @@ selector_match(Selector, Headers) -> % get binding funs from state (using dirty_reads) get_binding_funs_x(XName) -> - mnesia:async_dirty( - fun() -> - case read_state_no_error(XName) of - not_found -> - not_found; - #?JMS_TOPIC_RECORD{x_selector_funs = BindingFuns} -> - BindingFuns - end - end, - [] - ). + rabbit_db_jms_exchange:get(XName). add_initial_record(XName) -> write_state_fun(XName, dict:new()). % add binding fun to binding fun dictionary add_binding_fun(XName, BindingKeyAndFun) -> - #?JMS_TOPIC_RECORD{x_selector_funs = BindingFuns} = read_state_for_update(XName), - write_state_fun(XName, put_item(BindingFuns, BindingKeyAndFun)). + rabbit_db_jms_exchange:create_or_update(XName, BindingKeyAndFun, fun exchange_state_corrupt_error/1). % remove binding funs from binding fun dictionary remove_binding_funs(XName, Bindings) -> BindingKeys = [ {BindingKey, DestName} || #binding{key = BindingKey, destination = DestName} <- Bindings ], - #?JMS_TOPIC_RECORD{x_selector_funs = BindingFuns} = read_state_for_update(XName), - write_state_fun(XName, remove_items(BindingFuns, BindingKeys)). - -% add an item to the dictionary of binding functions -put_item(Dict, {Key, Item}) -> dict:store(Key, Item, Dict). - -% remove a list of keyed items from the dictionary, by key -remove_items(Dict, []) -> Dict; -remove_items(Dict, [Key | Keys]) -> remove_items(dict:erase(Key, Dict), Keys). + rabbit_db_jms_exchange:delete(XName, BindingKeys, fun exchange_state_corrupt_error/1). % delete all the state saved for this exchange delete_state(XName) -> - mnesia:delete(?JMS_TOPIC_TABLE, XName, write). - -% Basic read for update -read_state_for_update(XName) -> read_state(XName, write). - -% Lockable read -read_state(XName, Lock) -> - case mnesia:read(?JMS_TOPIC_TABLE, XName, Lock) of - [Rec] -> Rec; - _ -> exchange_state_corrupt_error(XName) - end. - -read_state_no_error(XName) -> - case mnesia:read(?JMS_TOPIC_TABLE, XName, read) of - [Rec] -> Rec; - _ -> not_found - end. - - + rabbit_db_jms_exchange:delete(XName). % Basic write write_state_fun(XName, BFuns) -> - mnesia:write( ?JMS_TOPIC_TABLE - , #?JMS_TOPIC_RECORD{x_name = XName, x_selector_funs = BFuns} - , write ). + rabbit_db_jms_exchange:insert(XName, BFuns). %%---------------------------------------------------------------------------- %% E R R O R S diff --git a/deps/rabbitmq_jms_topic_exchange/test/rjms_topic_selector_unit_SUITE.erl b/deps/rabbitmq_jms_topic_exchange/test/rjms_topic_selector_unit_SUITE.erl index cb0befc113ba..303c8253fe37 100644 --- a/deps/rabbitmq_jms_topic_exchange/test/rjms_topic_selector_unit_SUITE.erl +++ b/deps/rabbitmq_jms_topic_exchange/test/rjms_topic_selector_unit_SUITE.erl @@ -20,15 +20,8 @@ -import(rabbit_jms_topic_exchange, [ description/0 , serialise_events/0 - , route/2 , validate/1 - , create/2 - , delete/3 - , validate_binding/2 - , add_binding/3 - , remove_bindings/3 - , assert_args_equivalence/2 - , policy_changed/3 ]). + , validate_binding/2 ]). all() -> @@ -42,10 +35,7 @@ groups() -> description_test, serialise_events_test, validate_test, - create_test, - delete_test, - validate_binding_test, - add_binding_test + validate_binding_test ]} ]. @@ -82,19 +72,10 @@ serialise_events_test(_Config) -> ?assertMatch(false, serialise_events()). validate_test(_Config) -> - ?assertEqual(ok, validate(any_exchange)). - -create_test(_Config) -> - ?assertEqual(ok, create(none, any_exchange)). - -delete_test(_Config) -> - ?assertEqual(ok, delete(none, any_exchange, any_bindings)). + ?assertEqual(ok, validate(dummy_exchange())). validate_binding_test(_Config) -> - ?assertEqual(ok, validate_binding(any_exchange, any_bindings)). - -add_binding_test(_Config) -> - ?assertEqual(ok, add_binding(none, dummy_exchange(), dummy_binding())). + ?assertEqual(ok, validate_binding(dummy_exchange(), dummy_binding())). dummy_exchange() -> #exchange{name = <<"XName">>, arguments = []}. From 783996f53df9bdbed39d08b7ab79079f98b4a975 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Fri, 13 Jan 2023 15:36:58 +0100 Subject: [PATCH 08/12] Move consistent hash exchange Mnesia-specific code to rabbit_db_ch_exchange module --- .../src/rabbit_db_ch_exchange.erl | 132 +++++++++ .../rabbit_exchange_type_consistent_hash.erl | 256 +++++++----------- 2 files changed, 234 insertions(+), 154 deletions(-) create mode 100644 deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange.erl diff --git a/deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange.erl b/deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange.erl new file mode 100644 index 000000000000..d5eae063d3dd --- /dev/null +++ b/deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange.erl @@ -0,0 +1,132 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2023 VMware, Inc. or its affiliates. All rights reserved. +%% +-module(rabbit_db_ch_exchange). + +-export([ + setup_schema/0, + create/1, + create_binding/4, + get/1, + delete/1, + delete_bindings/2 + ]). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include("rabbitmq_consistent_hash_exchange.hrl"). + +-define(HASH_RING_STATE_TABLE, rabbit_exchange_type_consistent_hash_ring_state). + +setup_schema() -> + rabbit_db:run( + #{mnesia => fun() -> setup_schema_in_mnesia() end + }). + +setup_schema_in_mnesia() -> + mnesia:create_table(?HASH_RING_STATE_TABLE, [{record_name, chx_hash_ring}, + {attributes, record_info(fields, chx_hash_ring)}, + {type, ordered_set}]), + mnesia:add_table_copy(?HASH_RING_STATE_TABLE, node(), ram_copies), + rabbit_table:wait([?HASH_RING_STATE_TABLE]). + +create(X) -> + rabbit_db:run( + #{mnesia => fun() -> create_in_mnesia(X) end + }). + +create_in_mnesia(X) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> create_in_mnesia_tx(X) end). + +create_in_mnesia_tx(X) -> + case mnesia:read(?HASH_RING_STATE_TABLE, X) of + [_] -> ok; + [] -> + rabbit_log:debug("Consistent hashing exchange: will initialise hashing ring schema database record"), + mnesia:write_lock_table(?HASH_RING_STATE_TABLE), + ok = mnesia:write(?HASH_RING_STATE_TABLE, #chx_hash_ring{ + exchange = X, + next_bucket_number = 0, + bucket_map = #{}}, write) + end. + +create_binding(Src, Dst, Weight, UpdateFun) -> + rabbit_db:run( + #{mnesia => fun() -> create_binding_in_mnesia(Src, Dst, Weight, UpdateFun) end + }). + +create_binding_in_mnesia(Src, Dst, Weight, UpdateFun) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + create_binding_in_mnesia_tx(Src, Dst, Weight, UpdateFun) + end). + +create_binding_in_mnesia_tx(Src, Dst, Weight, UpdateFun) -> + case mnesia:read(?HASH_RING_STATE_TABLE, Src) of + [Chx0] -> + case UpdateFun(Chx0, Dst, Weight) of + already_exists -> + already_exists; + Chx -> + ok = mnesia:write(?HASH_RING_STATE_TABLE, Chx, write), + created + end; + [] -> + create_in_mnesia_tx(Src), + create_binding_in_mnesia_tx(Src, Dst, Weight, UpdateFun) + end. + +get(XName) -> + rabbit_db:run( + #{mnesia => fun() -> get_in_mnesia(XName) end + }). + +get_in_mnesia(XName) -> + case ets:lookup(?HASH_RING_STATE_TABLE, XName) of + [] -> + undefined; + [Chx] -> + Chx + end. + +delete(XName) -> + rabbit_db:run( + #{mnesia => fun() -> delete_in_mnesia(XName) end + }). + +delete_in_mnesia(XName) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + mnesia:write_lock_table(?HASH_RING_STATE_TABLE), + mnesia:delete({?HASH_RING_STATE_TABLE, XName}) + end). + +delete_bindings(Bindings, DeleteFun) -> + rabbit_db:run( + #{mnesia => fun() -> delete_bindings_in_mnesia(Bindings, DeleteFun) end + }). + +delete_bindings_in_mnesia(Bindings, DeleteFun) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + [delete_binding_in_mnesia(Binding, DeleteFun) || Binding <- Bindings] + end). + +delete_binding_in_mnesia(#binding{source = S, destination = D, key = RK}, DeleteFun) -> + rabbit_log:debug("Consistent hashing exchange: removing binding " + "from exchange ~ts to destination ~ts with routing key '~ts'", + [rabbit_misc:rs(S), rabbit_misc:rs(D), RK]), + case mnesia:read(?HASH_RING_STATE_TABLE, S) of + [Chx0] -> + case DeleteFun(Chx0, D) of + not_found -> + ok; + Chx -> + ok = mnesia:write(?HASH_RING_STATE_TABLE, Chx, write) + end; + [] -> + {not_found, S} + end. diff --git a/deps/rabbitmq_consistent_hash_exchange/src/rabbit_exchange_type_consistent_hash.erl b/deps/rabbitmq_consistent_hash_exchange/src/rabbit_exchange_type_consistent_hash.erl index ce0b39c3ba03..88f3af8a0919 100644 --- a/deps/rabbitmq_consistent_hash_exchange/src/rabbit_exchange_type_consistent_hash.erl +++ b/deps/rabbitmq_consistent_hash_exchange/src/rabbit_exchange_type_consistent_hash.erl @@ -15,7 +15,7 @@ -export([description/0, serialise_events/0, route/2]). -export([validate/1, validate_binding/2, - create/2, delete/3, policy_changed/2, + create/2, delete/2, policy_changed/2, add_binding/3, remove_bindings/3, assert_args_equivalence/2]). -export([init/0]). -export([info/1, info/2]). @@ -41,20 +41,14 @@ %% This data model allows for efficient routing and exchange deletion %% but less efficient (linear) binding management. --define(HASH_RING_STATE_TABLE, rabbit_exchange_type_consistent_hash_ring_state). - -define(PROPERTIES, [<<"correlation_id">>, <<"message_id">>, <<"timestamp">>]). %% OTP 19.3 does not support exs1024s -define(SEED_ALGORITHM, exs1024). init() -> - mnesia:create_table(?HASH_RING_STATE_TABLE, [{record_name, chx_hash_ring}, - {attributes, record_info(fields, chx_hash_ring)}, - {type, ordered_set}]), - mnesia:add_table_copy(?HASH_RING_STATE_TABLE, node(), ram_copies), - rabbit_table:wait([?HASH_RING_STATE_TABLE]), - _ = recover(), + rabbit_db_ch_exchange:setup_schema(), + recover(), ok. info(_X) -> []. @@ -68,16 +62,15 @@ serialise_events() -> false. route(#exchange {name = Name, arguments = Args}, #delivery {message = Msg}) -> - case ets:lookup(?HASH_RING_STATE_TABLE, Name) of - [] -> + case rabbit_db_ch_exchange:get(Name) of + undefined -> []; - [#chx_hash_ring{bucket_map = BM}] -> + #chx_hash_ring{bucket_map = BM} -> case maps:size(BM) of 0 -> []; N -> K = value_to_hash(hash_on(Args), Msg), SelectedBucket = jump_consistent_hash(K, N), - case maps:get(SelectedBucket, BM, undefined) of undefined -> rabbit_log:warning("Bucket ~tp not found", [SelectedBucket]), @@ -118,171 +111,126 @@ validate_binding(_X, #binding { key = K }) -> {error, {binding_invalid, "The binding key must be an integer: ~tp", [K]}} end. -maybe_initialise_hash_ring_state(transaction, #exchange{name = Name}) -> - maybe_initialise_hash_ring_state(transaction, Name); -maybe_initialise_hash_ring_state(transaction, X = #resource{}) -> - case mnesia:read(?HASH_RING_STATE_TABLE, X) of - [_] -> ok; - [] -> - rabbit_log:debug("Consistent hashing exchange: will initialise hashing ring schema database record"), - mnesia:write_lock_table(?HASH_RING_STATE_TABLE), - ok = mnesia:write(?HASH_RING_STATE_TABLE, #chx_hash_ring{ - exchange = X, - next_bucket_number = 0, - bucket_map = #{}}, write) - end; - -maybe_initialise_hash_ring_state(_, X) -> - rabbit_misc:execute_mnesia_transaction( - fun() -> maybe_initialise_hash_ring_state(transaction, X) end). +maybe_initialise_hash_ring_state(#exchange{name = Name}) -> + maybe_initialise_hash_ring_state(Name); +maybe_initialise_hash_ring_state(X = #resource{}) -> + rabbit_db_ch_exchange:create(X). recover() -> %% topology recovery has already happened, we have to recover state for any durable %% consistent hash exchanges since plugin activation was moved later in boot process %% starting with RabbitMQ 3.8.4 case list_exchanges() of - {ok, Xs} -> - rabbit_log:debug("Consistent hashing exchange: have ~b durable exchanges to recover", [length(Xs)]), - [recover_exchange_and_bindings(X) || X <- lists:usort(Xs)]; - {aborted, Reason} -> + {error, Reason} -> rabbit_log:error( - "Consistent hashing exchange: failed to recover durable exchange ring state, reason: ~tp", - [Reason]) + "Consistent hashing exchange: failed to recover durable exchange ring state, reason: ~tp", + [Reason]); + Xs -> + rabbit_log:debug("Consistent hashing exchange: have ~b durable exchanges to recover", [length(Xs)]), + %% TODO we need to know if we're first on the cluster to reset storage. In mnesia it's a ram table + [recover_exchange_and_bindings(X) || X <- lists:usort(Xs)] end. list_exchanges() -> - case mnesia:transaction( - fun () -> - mnesia:match_object(rabbit_exchange, - #exchange{durable = true, type = 'x-consistent-hash', _ = '_'}, write) - end) of - {atomic, Xs} -> - {ok, Xs}; - {aborted, Reason} -> - {aborted, Reason} - end. + Pattern = #exchange{durable = true, type = 'x-consistent-hash', _ = '_'}, + rabbit_db_exchange:match(Pattern). recover_exchange_and_bindings(#exchange{name = XName} = X) -> - mnesia:transaction( - fun () -> - rabbit_log:debug("Consistent hashing exchange: will recover exchange ~ts", [rabbit_misc:rs(XName)]), - create(transaction, X), - rabbit_log:debug("Consistent hashing exchange: recovered exchange ~ts", [rabbit_misc:rs(XName)]), - Bindings = rabbit_binding:list_for_source(XName), - rabbit_log:debug("Consistent hashing exchange: have ~b bindings to recover for exchange ~ts", - [length(Bindings), rabbit_misc:rs(XName)]), - [add_binding(transaction, X, B) || B <- lists:usort(Bindings)], - rabbit_log:debug("Consistent hashing exchange: recovered bindings for exchange ~ts", - [rabbit_misc:rs(XName)]) - end). - -create(transaction, X) -> - maybe_initialise_hash_ring_state(transaction, X); -create(Tx, X) -> - maybe_initialise_hash_ring_state(Tx, X). - -delete(transaction, #exchange{name = Name}, _Bs) -> - mnesia:write_lock_table(?HASH_RING_STATE_TABLE), - - ok = mnesia:delete({?HASH_RING_STATE_TABLE, Name}); -delete(_Tx, _X, _Bs) -> - ok. + rabbit_log:debug("Consistent hashing exchange: will recover exchange ~ts", [rabbit_misc:rs(XName)]), + create(none, X), + rabbit_log:debug("Consistent hashing exchange: recovered exchange ~ts", [rabbit_misc:rs(XName)]), + Bindings = rabbit_binding:list_for_source(XName), + rabbit_log:debug("Consistent hashing exchange: have ~b bindings to recover for exchange ~ts", + [length(Bindings), rabbit_misc:rs(XName)]), + [add_binding(none, X, B) || B <- lists:usort(Bindings)], + rabbit_log:debug("Consistent hashing exchange: recovered bindings for exchange ~ts", + [rabbit_misc:rs(XName)]). + +create(_Serial, X) -> + maybe_initialise_hash_ring_state(X). + +delete(_Serial, #exchange{name = XName}) -> + rabbit_db_ch_exchange:delete(XName). policy_changed(_X1, _X2) -> ok. -add_binding(transaction, X, - B = #binding{source = S, destination = D, key = K}) -> +add_binding(_Serial, _X, #binding{source = S, destination = D, key = K}) -> Weight = rabbit_data_coercion:to_integer(K), + rabbit_log:debug("Consistent hashing exchange: adding binding from " + "exchange ~ts to destination ~ts with routing key '~ts'", [rabbit_misc:rs(S), rabbit_misc:rs(D), K]), + case rabbit_db_ch_exchange:create_binding(S, D, Weight, fun chx_hash_ring_update_fun/3) of + already_exists -> + rabbit_log:debug("Consistent hashing exchange: NOT adding binding from " + "exchange ~s to destination ~s with routing key '~s' " + "because this binding (possibly with a different " + "routing key) already exists", + [rabbit_misc:rs(S), rabbit_misc:rs(D), K]); + created -> + rabbit_log:debug("Consistent hashing exchange: adding binding from " + "exchange ~s to destination ~s with routing key '~s'", + [rabbit_misc:rs(S), rabbit_misc:rs(D), K]) + end. - case mnesia:read(?HASH_RING_STATE_TABLE, S) of - [State0 = #chx_hash_ring{bucket_map = BM0, - next_bucket_number = NexN0}] -> - case map_has_value(BM0, D) of - true -> - rabbit_log:debug("Consistent hashing exchange: NOT adding binding from " - "exchange ~ts to destination ~ts with routing key '~ts' " - "because this binding (possibly with a different " - "routing key) already exists", - [rabbit_misc:rs(S), rabbit_misc:rs(D), K]); - false -> - rabbit_log:debug("Consistent hashing exchange: adding binding from " - "exchange ~ts to destination ~ts with routing key '~ts'", - [rabbit_misc:rs(S), rabbit_misc:rs(D), K]), - NextN = NexN0 + Weight, - %% hi/lo bucket counters are 0-based but weight is 1-based - Range = lists:seq(NexN0, (NextN - 1)), - BM = lists:foldl(fun(Key, Acc) -> - maps:put(Key, D, Acc) - end, BM0, Range), - State = State0#chx_hash_ring{bucket_map = BM, - next_bucket_number = NextN}, - - ok = mnesia:write(?HASH_RING_STATE_TABLE, State, write), - ok - end; - [] -> - maybe_initialise_hash_ring_state(transaction, S), - add_binding(transaction, X, B) - end; -add_binding(none, _X, _B) -> - ok. - -remove_bindings(transaction, _X, Bindings) -> - [remove_binding(B) || B <- Bindings], +chx_hash_ring_update_fun(#chx_hash_ring{bucket_map = BM0, + next_bucket_number = NexN0} = Chx0, + Dst, Weight) -> + case map_has_value(BM0, Dst) of + true -> + already_exists; + false -> + NextN = NexN0 + Weight, + %% hi/lo bucket counters are 0-based but weight is 1-based + Range = lists:seq(NexN0, (NextN - 1)), + BM = lists:foldl(fun(Key, Acc) -> + maps:put(Key, Dst, Acc) + end, BM0, Range), + Chx0#chx_hash_ring{bucket_map = BM, + next_bucket_number = NextN} + end. - ok; -remove_bindings(none, X, Bindings) -> - rabbit_misc:execute_mnesia_transaction( - fun() -> remove_bindings(transaction, X, Bindings) end), +remove_bindings(_Serial, _X, Bindings) -> + Ret = rabbit_db_ch_exchange:delete_bindings(Bindings, fun ch_hash_ring_delete_fun/2), + [rabbit_log:warning("Can't remove binding: hash ring state for exchange ~s wasn't found", + [rabbit_misc:rs(X)]) || {not_found, X} <- Ret], ok. --spec remove_binding(#binding{}) -> ok. -remove_binding(#binding{source = S, destination = D, key = RK}) -> - rabbit_log:debug("Consistent hashing exchange: removing binding " - "from exchange ~ts to destination ~ts with routing key '~ts'", - [rabbit_misc:rs(S), rabbit_misc:rs(D), RK]), - - case mnesia:read(?HASH_RING_STATE_TABLE, S) of - [State0 = #chx_hash_ring{bucket_map = BM0, - next_bucket_number = NexN0}] -> - %% Buckets with lower numbers stay as is; buckets that - %% belong to this binding are removed; buckets with - %% greater numbers are updated (their numbers are adjusted downwards) - BucketsOfThisBinding = maps:filter(fun (_K, V) -> V =:= D end, BM0), - case maps:size(BucketsOfThisBinding) of - 0 -> ok; - N when N >= 1 -> - KeysOfThisBinding = lists:usort(maps:keys(BucketsOfThisBinding)), - LastBucket = lists:last(KeysOfThisBinding), - FirstBucket = hd(KeysOfThisBinding), - BucketsDownTheRing = maps:filter(fun (K, _) -> K > LastBucket end, BM0), - UnchangedBuckets = maps:filter(fun (K, _) -> K < FirstBucket end, BM0), - - %% final state with "down the ring" buckets updated - NewBucketsDownTheRing = maps:fold( - fun(K0, V, Acc) -> - maps:put(K0 - N, V, Acc) - end, #{}, BucketsDownTheRing), - BM1 = maps:merge(UnchangedBuckets, NewBucketsDownTheRing), - NextN = NexN0 - N, - State = State0#chx_hash_ring{bucket_map = BM1, - next_bucket_number = NextN}, - - ok = mnesia:write(?HASH_RING_STATE_TABLE, State, write) - end; - [] -> - rabbit_log:warning("Can't remove binding: hash ring state for exchange ~ts wasn't found", - [rabbit_misc:rs(S)]), - ok +ch_hash_ring_delete_fun(#chx_hash_ring{bucket_map = BM0, + next_bucket_number = NexN0} = Chx0, + Dst) -> + %% Buckets with lower numbers stay as is; buckets that + %% belong to this binding are removed; buckets with + %% greater numbers are updated (their numbers are adjusted downwards) + BucketsOfThisBinding = maps:filter(fun (_K, V) -> V =:= Dst end, BM0), + case maps:size(BucketsOfThisBinding) of + 0 -> + not_found; + N when N >= 1 -> + KeysOfThisBinding = lists:usort(maps:keys(BucketsOfThisBinding)), + LastBucket = lists:last(KeysOfThisBinding), + FirstBucket = hd(KeysOfThisBinding), + BucketsDownTheRing = maps:filter(fun (K, _) -> K > LastBucket end, BM0), + UnchangedBuckets = maps:filter(fun (K, _) -> K < FirstBucket end, BM0), + + %% final state with "down the ring" buckets updated + NewBucketsDownTheRing = maps:fold( + fun(K0, V, Acc) -> + maps:put(K0 - N, V, Acc) + end, #{}, BucketsDownTheRing), + BM1 = maps:merge(UnchangedBuckets, NewBucketsDownTheRing), + NextN = NexN0 - N, + Chx0#chx_hash_ring{bucket_map = BM1, + next_bucket_number = NextN} end. -spec ring_state(vhost:name(), rabbit_misc:resource_name()) -> {ok, #chx_hash_ring{}} | {error, not_found}. -ring_state(VirtualHost, Exchange) -> - Resource = rabbit_misc:r(VirtualHost, exchange, Exchange), - case mnesia:dirty_read(?HASH_RING_STATE_TABLE, Resource) of - [] -> {error, not_found}; - [Row] -> {ok, Row} +ring_state(VirtualHost, XName) -> + Exchange = rabbit_misc:r(VirtualHost, exchange, XName), + case rabbit_db_ch_exchange:get(Exchange) of + undefined -> + {error, not_found}; + Chx -> + {ok, Chx} end. assert_args_equivalence(X, Args) -> From f2443f6d107ce454ebca88ba26a5b7156892b2e6 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Fri, 13 Jan 2023 17:22:37 +0100 Subject: [PATCH 09/12] Move mnesia queries from rabbit_misc to rabbit_mnesia --- deps/rabbit/src/rabbit_channel_tracking.erl | 4 +- .../rabbit/src/rabbit_connection_tracking.erl | 6 +- deps/rabbit/src/rabbit_core_ff.erl | 2 +- deps/rabbit/src/rabbit_db_binding.erl | 14 +- deps/rabbit/src/rabbit_db_exchange.erl | 22 +-- deps/rabbit/src/rabbit_db_msup.erl | 8 +- deps/rabbit/src/rabbit_db_policy.erl | 2 +- deps/rabbit/src/rabbit_db_queue.erl | 26 ++-- deps/rabbit/src/rabbit_db_rtparams.erl | 12 +- deps/rabbit/src/rabbit_db_topic_exchange.erl | 6 +- deps/rabbit/src/rabbit_db_user.erl | 34 ++--- deps/rabbit/src/rabbit_db_vhost.erl | 10 +- .../src/rabbit_mirror_queue_coordinator.erl | 2 +- .../rabbit/src/rabbit_mirror_queue_master.erl | 2 +- deps/rabbit/src/rabbit_mirror_queue_misc.erl | 6 +- deps/rabbit/src/rabbit_mirror_queue_slave.erl | 6 +- deps/rabbit/src/rabbit_mnesia.erl | 107 +++++++++++++++ deps/rabbit/src/rabbit_networking.erl | 2 +- deps/rabbit/test/dead_lettering_SUITE.erl | 11 +- deps/rabbit/test/queue_parallel_SUITE.erl | 10 +- deps/rabbit/test/topic_permission_SUITE.erl | 2 +- .../rabbit/test/unit_access_control_SUITE.erl | 2 +- deps/rabbit/test/unit_gm_SUITE.erl | 10 +- deps/rabbit_common/src/rabbit_misc.erl | 129 +----------------- .../src/rabbit_db_ch_exchange.erl | 8 +- .../src/rabbit_federation_exchange_link.erl | 2 +- .../src/rabbit_db_jms_exchange.erl | 8 +- .../src/rabbit_db_rh_exchange.erl | 6 +- 28 files changed, 217 insertions(+), 242 deletions(-) diff --git a/deps/rabbit/src/rabbit_channel_tracking.erl b/deps/rabbit/src/rabbit_channel_tracking.erl index d9d6d0e82962..0df63fddab10 100644 --- a/deps/rabbit/src/rabbit_channel_tracking.erl +++ b/deps/rabbit/src/rabbit_channel_tracking.erl @@ -467,7 +467,7 @@ close_channels(_TrackedChannels = []) -> ok. migrate_tracking_records() -> Node = node(), - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> Table = tracked_channel_table_name_for(Node), _ = mnesia:lock({table, Table}, read), @@ -477,7 +477,7 @@ migrate_tracking_records() -> ets:insert(tracked_channel, Channel) end, Channels) end), - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> Table = tracked_channel_per_user_table_name_for(Node), _ = mnesia:lock({table, Table}, read), diff --git a/deps/rabbit/src/rabbit_connection_tracking.erl b/deps/rabbit/src/rabbit_connection_tracking.erl index 4bc1f91ed4eb..607a63474ca1 100644 --- a/deps/rabbit/src/rabbit_connection_tracking.erl +++ b/deps/rabbit/src/rabbit_connection_tracking.erl @@ -731,7 +731,7 @@ close_connection(#tracked_connection{pid = Pid}, Message) -> migrate_tracking_records() -> Node = node(), - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> Table = tracked_connection_table_name_for(Node), _ = mnesia:lock({table, Table}, read), @@ -741,7 +741,7 @@ migrate_tracking_records() -> ets:insert(tracked_connection, Connection) end, Connections) end), - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> Table = tracked_connection_per_user_table_name_for(Node), _ = mnesia:lock({table, Table}, read), @@ -752,7 +752,7 @@ migrate_tracking_records() -> ets:update_counter(tracked_connection_per_user, Username, C, {Username, 0}) end, Connections) end), - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> Table = tracked_connection_per_vhost_table_name_for(Node), _ = mnesia:lock({table, Table}, read), diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl index b94c7ff8f2db..3ccf55336a91 100644 --- a/deps/rabbit/src/rabbit_core_ff.erl +++ b/deps/rabbit/src/rabbit_core_ff.erl @@ -160,7 +160,7 @@ direct_exchange_routing_v2_enable(#{feature_name := FeatureName}) -> listener_records_in_ets_enable(#{feature_name := FeatureName}) -> try - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> _ = mnesia:lock({table, rabbit_listener}, read), Listeners = mnesia:select( diff --git a/deps/rabbit/src/rabbit_db_binding.erl b/deps/rabbit/src/rabbit_db_binding.erl index 95abc5bf48f1..90526dbc910f 100644 --- a/deps/rabbit/src/rabbit_db_binding.erl +++ b/deps/rabbit/src/rabbit_db_binding.erl @@ -254,7 +254,7 @@ recover_in_mnesia(RecoverFun) -> [RecoverFun(Route, Src, Dst, fun recover_semi_durable_route/2) || #route{binding = #binding{destination = Dst, source = Src}} = Route <- - rabbit_misc:dirty_read_all(rabbit_semi_durable_route)]. + rabbit_mnesia:dirty_read_all(rabbit_semi_durable_route)]. create_index_route_table() -> rabbit_db:run( @@ -333,7 +333,7 @@ binding_action_in_mnesia(#binding{source = SrcName, destination = DstName}, Fun, ErrFun) -> SrcTable = table_for_resource(SrcName), DstTable = table_for_resource(DstName), - rabbit_misc:execute_mnesia_tx_with_tail( + rabbit_mnesia:execute_mnesia_tx_with_tail( fun () -> case {mnesia:read({SrcTable, SrcName}), mnesia:read({DstTable, DstName})} of @@ -371,11 +371,11 @@ create_in_mnesia(Binding, ChecksFun) -> end, fun not_found_or_absent_errs_in_mnesia/1). populate_index_route_table_in_mnesia() -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> _ = mnesia:lock({table, rabbit_route}, read), _ = mnesia:lock({table, rabbit_index_route}, write), - Routes = rabbit_misc:dirty_read_all(rabbit_route), + Routes = rabbit_mnesia:dirty_read_all(rabbit_route), lists:foreach(fun(#route{binding = #binding{source = Exchange}} = Route) -> case rabbit_db_exchange:get(Exchange) of {ok, X} -> @@ -542,11 +542,11 @@ not_found_or_absent_in_mnesia(#resource{kind = queue} = Name) -> end. recover_in_mnesia() -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> _ = mnesia:lock({table, rabbit_durable_route}, read), _ = mnesia:lock({table, rabbit_semi_durable_route}, write), - Routes = rabbit_misc:dirty_read_all(rabbit_durable_route), + Routes = rabbit_mnesia:dirty_read_all(rabbit_durable_route), Fun = fun(Route) -> mnesia:dirty_write(rabbit_semi_durable_route, Route) end, @@ -556,7 +556,7 @@ recover_in_mnesia() -> recover_semi_durable_route(#route{binding = B} = Route, X) -> MaybeSerial = rabbit_exchange:serialise_events(X), - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> case mnesia:read(rabbit_semi_durable_route, B, read) of [] -> no_recover; diff --git a/deps/rabbit/src/rabbit_db_exchange.erl b/deps/rabbit/src/rabbit_db_exchange.erl index d048d28d3e67..e4cbd2ad4df8 100644 --- a/deps/rabbit/src/rabbit_db_exchange.erl +++ b/deps/rabbit/src/rabbit_db_exchange.erl @@ -132,7 +132,7 @@ get(Name) -> }). get_in_mnesia(Name) -> - rabbit_misc:dirty_read({rabbit_exchange, Name}). + rabbit_mnesia:dirty_read({rabbit_exchange, Name}). %% ------------------------------------------------------------------- %% get_many(). @@ -192,7 +192,7 @@ update(XName, Fun) -> }). update_in_mnesia(XName, Fun) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> update_in_mnesia_tx(XName, Fun) end). @@ -218,7 +218,7 @@ create_or_get(X) -> }). create_or_get_in_mnesia(#exchange{name = XName} = X) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> case mnesia:wread({rabbit_exchange, XName}) of [] -> @@ -246,7 +246,7 @@ insert(Xs) -> }). insert_in_mnesia(Xs) when is_list(Xs) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> [mnesia:write(rabbit_durable_exchange, X, write) || X <- Xs] end), @@ -271,7 +271,7 @@ peek_serial(XName) -> }). peek_serial_in_mnesia(XName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> peek_serial_in_mnesia_tx(XName, read) end). @@ -295,7 +295,7 @@ next_serial(X) -> }). next_serial_in_mnesia(X) -> - rabbit_misc:execute_mnesia_transaction(fun() -> + rabbit_mnesia:execute_mnesia_transaction(fun() -> next_serial_in_mnesia_tx(X) end). @@ -332,7 +332,7 @@ delete_in_mnesia(XName, IfUnused) -> true -> fun conditional_delete_in_mnesia/2; false -> fun unconditional_delete_in_mnesia/2 end, - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> case mnesia:wread({rabbit_exchange, XName}) of [X] -> DeletionFun(X, false); @@ -358,7 +358,7 @@ delete_serial(XName) -> }). delete_serial_in_mnesia(XName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> mnesia:delete({rabbit_exchange_serial, XName}) end). @@ -458,7 +458,7 @@ delete_in_mnesia(X = #exchange{name = XName}, OnlyDurable, RemoveBindingsForSour get_many_in_mnesia(Table, [Name]) -> ets:lookup(Table, Name); get_many_in_mnesia(Table, Names) when is_list(Names) -> %% Normally we'd call mnesia:dirty_read/1 here, but that is quite - %% expensive for reasons explained in rabbit_misc:dirty_read/1. + %% expensive for reasons explained in rabbit_mnesia:dirty_read/1. lists:append([ets:lookup(Table, Name) || Name <- Names]). conditional_delete_in_mnesia(X = #exchange{name = XName}, OnlyDurable) -> @@ -500,7 +500,7 @@ insert_ram_in_mnesia_tx(X) -> X1. recover_in_mnesia(VHost) -> - rabbit_misc:table_filter( + rabbit_mnesia:table_filter( fun (#exchange{name = XName}) -> XName#resource.virtual_host =:= VHost andalso mnesia:read({rabbit_exchange, XName}) =:= [] @@ -508,7 +508,7 @@ recover_in_mnesia(VHost) -> fun (X, true) -> X; (X, false) -> - X1 = rabbit_misc:execute_mnesia_transaction( + X1 = rabbit_mnesia:execute_mnesia_transaction( fun() -> insert_in_mnesia_tx(X) end), Serial = rabbit_exchange:serial(X1), rabbit_exchange:callback(X1, create, Serial, [X1]) diff --git a/deps/rabbit/src/rabbit_db_msup.erl b/deps/rabbit/src/rabbit_db_msup.erl index 394d14f9a1d4..0f56b609d726 100644 --- a/deps/rabbit/src/rabbit_db_msup.erl +++ b/deps/rabbit/src/rabbit_db_msup.erl @@ -61,7 +61,7 @@ delete(Group, Id) -> }). delete_in_mnesia(Group, Id) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> delete_in_mnesia_tx(Group, Id) end). -spec find_mirror(Group, Id) -> Ret when @@ -115,7 +115,7 @@ create_tables_in_mnesia([{Table, Attributes} | Ts]) -> end. create_or_update_in_mnesia(Group, Overall, Delegate, ChildSpec, Id) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> ReadResult = mnesia:wread({?TABLE, {Group, Id}}), rabbit_log:debug("Mirrored supervisor: check_start table ~ts read for key ~tp returned ~tp", @@ -145,7 +145,7 @@ create_or_update_in_mnesia(Group, Overall, Delegate, ChildSpec, Id) -> end). update_all_in_mnesia(Overall, OldOverall) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> MatchHead = #mirrored_sup_childspec{mirroring_pid = OldOverall, key = '$1', @@ -156,7 +156,7 @@ update_all_in_mnesia(Overall, OldOverall) -> end). delete_all_in_mnesia(Group) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> MatchHead = #mirrored_sup_childspec{key = {Group, '$1'}, _ = '_'}, diff --git a/deps/rabbit/src/rabbit_db_policy.erl b/deps/rabbit/src/rabbit_db_policy.erl index 50317381e1ab..25ae02ac2a3c 100644 --- a/deps/rabbit/src/rabbit_db_policy.erl +++ b/deps/rabbit/src/rabbit_db_policy.erl @@ -31,7 +31,7 @@ update(VHost, GetUpdatedExchangeFun, GetUpdatedQueueFun) -> update_in_mnesia(VHost, GetUpdatedExchangeFun, GetUpdatedQueueFun) -> Tabs = [rabbit_queue, rabbit_durable_queue, rabbit_exchange, rabbit_durable_exchange], - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> [mnesia:lock({table, T}, write) || T <- Tabs], %% [1] Exchanges0 = rabbit_db_exchange:get_all(VHost), diff --git a/deps/rabbit/src/rabbit_db_queue.erl b/deps/rabbit/src/rabbit_db_queue.erl index 811e0f0ae569..e713759a5dbb 100644 --- a/deps/rabbit/src/rabbit_db_queue.erl +++ b/deps/rabbit/src/rabbit_db_queue.erl @@ -195,7 +195,7 @@ get(Name) -> }). get_in_mnesia(Name) -> - rabbit_misc:dirty_read({rabbit_queue, Name}). + rabbit_mnesia:dirty_read({rabbit_queue, Name}). get_durable(Names) when is_list(Names) -> rabbit_db:run( @@ -207,7 +207,7 @@ get_durable(Name) -> }). get_durable_in_mnesia(Name) -> - rabbit_misc:dirty_read({rabbit_durable_queue, Name}). + rabbit_mnesia:dirty_read({rabbit_durable_queue, Name}). delete_transient(Queues) -> rabbit_db:run( @@ -215,7 +215,7 @@ delete_transient(Queues) -> }). delete_transient_in_mnesia(Queues) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> [{QName, delete_transient_in_mnesia_tx(QName)} || QName <- Queues] @@ -227,7 +227,7 @@ on_node_up(Node, Fun) -> }). on_node_up_in_mnesia(Node, Fun) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> Qs = mnesia:match_object(rabbit_queue, amqqueue:pattern_match_all(), write), @@ -241,7 +241,7 @@ on_node_down(Node, Fun) -> }). on_node_down_in_mnesia(Node, Fun) -> - Qs = rabbit_misc:execute_mnesia_transaction( + Qs = rabbit_mnesia:execute_mnesia_transaction( fun () -> qlc:e(qlc:q([amqqueue:get_name(Q) || Q <- mnesia:table(rabbit_queue), Fun(Node, Q) @@ -274,7 +274,7 @@ update(QName, Fun) -> }). update_in_mnesia(QName, Fun) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> update_in_mnesia_tx(QName, Fun) end). @@ -327,7 +327,7 @@ create_or_get(DurableQ, Q) -> create_or_get_in_mnesia(DurableQ, Q) -> QueueName = amqqueue:get_name(Q), - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> case mnesia:wread({rabbit_queue, QueueName}) of [] -> @@ -349,7 +349,7 @@ insert(DurableQ, Q) -> }). insert_in_mnesia(DurableQ, Q) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> insert_in_mnesia_tx(DurableQ, Q) end). @@ -360,7 +360,7 @@ insert(Qs) -> }). insert_many_in_mnesia(Qs) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> [ok = mnesia:write(rabbit_durable_queue, Q, write) || Q <- Qs] end). @@ -420,7 +420,7 @@ get_many_in_mnesia(Table, [Name]) -> ets:lookup(Table, Name); get_many_in_mnesia(Table, Names) when is_list(Names) -> %% Normally we'd call mnesia:dirty_read/1 here, but that is quite - %% expensive for reasons explained in rabbit_misc:dirty_read/1. + %% expensive for reasons explained in rabbit_mnesia:dirty_read/1. lists:append([ets:lookup(Table, Name) || Name <- Names]). delete_transient_in_mnesia_tx(QName) -> @@ -438,7 +438,7 @@ not_found_or_absent_queue_dirty_in_mnesia(Name) -> %% We should read from both tables inside a tx, to get a %% consistent view. But the chances of an inconsistency are small, %% and only affect the error kind. - case rabbit_misc:dirty_read({rabbit_durable_queue, Name}) of + case rabbit_mnesia:dirty_read({rabbit_durable_queue, Name}) of {error, not_found} -> not_found; {ok, Q} -> {absent, Q, nodedown} end. @@ -476,7 +476,7 @@ list_with_possible_retry_in_mnesia(Fun) -> end. delete_in_mnesia(QueueName, Reason) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> case {mnesia:wread({rabbit_queue, QueueName}), mnesia:wread({rabbit_durable_queue, QueueName})} of @@ -521,7 +521,7 @@ list_for_count_in_mnesia(VHost) -> end). update_decorators_in_mnesia(Name) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> case mnesia:wread({rabbit_queue, Name}) of [Q] -> ok = mnesia:write(rabbit_queue, rabbit_queue_decorator:set(Q), diff --git a/deps/rabbit/src/rabbit_db_rtparams.erl b/deps/rabbit/src/rabbit_db_rtparams.erl index 0a133be43fd9..4f7a9bd52efa 100644 --- a/deps/rabbit/src/rabbit_db_rtparams.erl +++ b/deps/rabbit/src/rabbit_db_rtparams.erl @@ -37,7 +37,7 @@ set(Key, Term) when is_atom(Key) -> #{mnesia => fun() -> set_in_mnesia(Key, Term) end}). set_in_mnesia(Key, Term) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> set_in_mnesia_tx(Key, Term) end). -spec set(VHostName, Comp, Name, Term) -> Ret when @@ -63,7 +63,7 @@ set(VHostName, Comp, Name, Term) #{mnesia => fun() -> set_in_mnesia(VHostName, Key, Term) end}). set_in_mnesia(VHostName, Key, Term) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( rabbit_db_vhost:with_fun_in_mnesia_tx( VHostName, fun() -> set_in_mnesia_tx(Key, Term) end)). @@ -131,7 +131,7 @@ get_or_set(Key, Default) -> #{mnesia => fun() -> get_or_set_in_mnesia(Key, Default) end}). get_or_set_in_mnesia(Key, Default) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> get_or_set_in_mnesia_tx(Key, Default) end). get_or_set_in_mnesia_tx(Key, Default) -> @@ -162,7 +162,7 @@ get_all() -> #{mnesia => fun() -> get_all_in_mnesia() end}). get_all_in_mnesia() -> - rabbit_misc:dirty_read_all(?MNESIA_TABLE). + rabbit_mnesia:dirty_read_all(?MNESIA_TABLE). -spec get_all(VHostName, Comp) -> Ret when VHostName :: vhost:name() | '_', @@ -230,14 +230,14 @@ delete(VHostName, Comp, Name) fun() -> delete_matching_in_mnesia(VHostName, Comp, Name) end}). delete_in_mnesia(Key) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> delete_in_mnesia_tx(Key) end). delete_in_mnesia_tx(Key) -> mnesia:delete(?MNESIA_TABLE, Key, write). delete_matching_in_mnesia(VHostName, Comp, Name) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> delete_matching_in_mnesia_tx(VHostName, Comp, Name) end). delete_matching_in_mnesia_tx(VHostName, Comp, Name) -> diff --git a/deps/rabbit/src/rabbit_db_topic_exchange.erl b/deps/rabbit/src/rabbit_db_topic_exchange.erl index d58e5597fe7e..89acc9d266f0 100644 --- a/deps/rabbit/src/rabbit_db_topic_exchange.erl +++ b/deps/rabbit/src/rabbit_db_topic_exchange.erl @@ -85,7 +85,7 @@ split_topic_key(Key) -> split_topic_key(Key, [], []). insert_in_mnesia(XName, RoutingKey, Destination, Args) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> FinalNode = follow_down_create(XName, split_topic_key(RoutingKey)), trie_add_binding(XName, FinalNode, Destination, Args), @@ -93,7 +93,7 @@ insert_in_mnesia(XName, RoutingKey, Destination, Args) -> end). delete_all_for_exchange_in_mnesia(XName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> trie_remove_all_nodes(XName), trie_remove_all_edges(XName), @@ -149,7 +149,7 @@ delete_in_mnesia_tx(Bs) -> ok. delete_in_mnesia(Bs) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> delete_in_mnesia_tx(Bs) end). split_topic_key(<<>>, [], []) -> diff --git a/deps/rabbit/src/rabbit_db_user.erl b/deps/rabbit/src/rabbit_db_user.erl index 68b3561f05ea..ba2adaf8131f 100644 --- a/deps/rabbit/src/rabbit_db_user.erl +++ b/deps/rabbit/src/rabbit_db_user.erl @@ -51,7 +51,7 @@ create(User) -> #{mnesia => fun() -> create_in_mnesia(Username, User) end}). create_in_mnesia(Username, User) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> create_in_mnesia_tx(Username, User) end). create_in_mnesia_tx(Username, User) -> @@ -78,7 +78,7 @@ update(Username, UpdateFun) #{mnesia => fun() -> update_in_mnesia(Username, UpdateFun) end}). update_in_mnesia(Username, UpdateFun) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> update_in_mnesia_tx(Username, UpdateFun) end). update_in_mnesia_tx(Username, UpdateFun) -> @@ -218,26 +218,26 @@ check_and_match_user_permissions(Username, VHostName) fun() -> match_user_permissions_in_mnesia(Username, VHostName) end}). match_user_permissions_in_mnesia('_' = Username, '_' = VHostName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> match_user_permissions_in_mnesia_tx(Username, VHostName) end); match_user_permissions_in_mnesia('_' = Username, VHostName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( rabbit_db_vhost:with_fun_in_mnesia_tx( VHostName, fun() -> match_user_permissions_in_mnesia_tx(Username, VHostName) end)); match_user_permissions_in_mnesia(Username, '_' = VHostName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( with_fun_in_mnesia_tx( Username, fun() -> match_user_permissions_in_mnesia_tx(Username, VHostName) end)); match_user_permissions_in_mnesia(Username, VHostName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( with_fun_in_mnesia_tx( Username, rabbit_db_vhost:with_fun_in_mnesia_tx( @@ -278,7 +278,7 @@ set_user_permissions( end}). set_user_permissions_in_mnesia(Username, VHostName, UserPermission) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( with_fun_in_mnesia_tx( Username, rabbit_db_vhost:with_fun_in_mnesia_tx( @@ -307,7 +307,7 @@ clear_user_permissions(Username, VHostName) fun() -> clear_user_permissions_in_mnesia(Username, VHostName) end}). clear_user_permissions_in_mnesia(Username, VHostName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> clear_user_permissions_in_mnesia_tx(Username, VHostName) end). clear_user_permissions_in_mnesia_tx(Username, VHostName) -> @@ -340,7 +340,7 @@ clear_matching_user_permissions(Username, VHostName) }). clear_matching_user_permissions_in_mnesia(Username, VHostName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> clear_matching_user_permissions_in_mnesia_tx( Username, VHostName) end). @@ -422,14 +422,14 @@ check_and_match_topic_permissions(Username, VHostName, ExchangeName) match_topic_permissions_in_mnesia( '_' = Username, '_' = VHostName, ExchangeName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> match_topic_permissions_in_mnesia_tx( Username, VHostName, ExchangeName) end); match_topic_permissions_in_mnesia( '_' = Username, VHostName, ExchangeName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( rabbit_db_vhost:with_fun_in_mnesia_tx( VHostName, fun() -> @@ -438,7 +438,7 @@ match_topic_permissions_in_mnesia( end)); match_topic_permissions_in_mnesia( Username, '_' = VHostName, ExchangeName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( with_fun_in_mnesia_tx( Username, fun() -> @@ -447,7 +447,7 @@ match_topic_permissions_in_mnesia( end)); match_topic_permissions_in_mnesia( Username, VHostName, ExchangeName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( with_fun_in_mnesia_tx( Username, rabbit_db_vhost:with_fun_in_mnesia_tx( @@ -495,7 +495,7 @@ set_topic_permissions( end}). set_topic_permissions_in_mnesia(Username, VHostName, TopicPermission) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( with_fun_in_mnesia_tx( Username, rabbit_db_vhost:with_fun_in_mnesia_tx( @@ -531,7 +531,7 @@ clear_topic_permissions(Username, VHostName, ExchangeName) end}). clear_topic_permissions_in_mnesia(Username, VHostName, ExchangeName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> clear_topic_permissions_in_mnesia_tx( Username, VHostName, ExchangeName) @@ -569,7 +569,7 @@ clear_matching_topic_permissions(Username, VHostName, ExchangeName) clear_matching_topic_permissions_in_mnesia( Username, VHostName, ExchangeName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> clear_matching_topic_permissions_in_mnesia_tx( Username, VHostName, ExchangeName) @@ -604,7 +604,7 @@ delete(Username) when is_binary(Username) -> #{mnesia => fun() -> delete_in_mnesia(Username) end}). delete_in_mnesia(Username) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> delete_in_mnesia_tx(Username) end). delete_in_mnesia_tx(Username) -> diff --git a/deps/rabbit/src/rabbit_db_vhost.erl b/deps/rabbit/src/rabbit_db_vhost.erl index 91b9e10e6d5c..1a01860a39c1 100644 --- a/deps/rabbit/src/rabbit_db_vhost.erl +++ b/deps/rabbit/src/rabbit_db_vhost.erl @@ -53,7 +53,7 @@ create_or_get(VHostName, Limits, Metadata) #{mnesia => fun() -> create_or_get_in_mnesia(VHostName, VHost) end}). create_or_get_in_mnesia(VHostName, VHost) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> create_or_get_in_mnesia_tx(VHostName, VHost) end). create_or_get_in_mnesia_tx(VHostName, VHost) -> @@ -99,7 +99,7 @@ do_merge_metadata(VHostName, Metadata) -> #{mnesia => fun() -> merge_metadata_in_mnesia(VHostName, Metadata) end}). merge_metadata_in_mnesia(VHostName, Metadata) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> merge_metadata_in_mnesia_tx(VHostName, Metadata) end). merge_metadata_in_mnesia_tx(VHostName, Metadata) -> @@ -135,7 +135,7 @@ set_tags(VHostName, Tags) #{mnesia => fun() -> set_tags_in_mnesia(VHostName, ConvertedTags) end}). set_tags_in_mnesia(VHostName, Tags) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> set_tags_in_mnesia_tx(VHostName, Tags) end). set_tags_in_mnesia_tx(VHostName, Tags) -> @@ -251,7 +251,7 @@ update(VHostName, UpdateFun) #{mnesia => fun() -> update_in_mnesia(VHostName, UpdateFun) end}). update_in_mnesia(VHostName, UpdateFun) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> update_in_mnesia_tx(VHostName, UpdateFun) end). update_in_mnesia_tx(VHostName, UpdateFun) @@ -312,7 +312,7 @@ delete(VHostName) when is_binary(VHostName) -> #{mnesia => fun() -> delete_in_mnesia(VHostName) end}). delete_in_mnesia(VHostName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> delete_in_mnesia_tx(VHostName) end). delete_in_mnesia_tx(VHostName) -> diff --git a/deps/rabbit/src/rabbit_mirror_queue_coordinator.erl b/deps/rabbit/src/rabbit_mirror_queue_coordinator.erl index 4a2b8bb6368f..e6b2b90b3678 100644 --- a/deps/rabbit/src/rabbit_mirror_queue_coordinator.erl +++ b/deps/rabbit/src/rabbit_mirror_queue_coordinator.erl @@ -320,7 +320,7 @@ init([Q, GM, DeathFun, DepthFun]) when ?is_amqqueue(Q) -> undefined -> {ok, GM2} = gm:start_link( QueueName, ?MODULE, [self()], - fun rabbit_misc:execute_mnesia_transaction/1), + fun rabbit_mnesia:execute_mnesia_transaction/1), receive {joined, GM2, _Members} -> ok end, diff --git a/deps/rabbit/src/rabbit_mirror_queue_master.erl b/deps/rabbit/src/rabbit_mirror_queue_master.erl index 2e30f43c2889..12ef161f9246 100644 --- a/deps/rabbit/src/rabbit_mirror_queue_master.erl +++ b/deps/rabbit/src/rabbit_mirror_queue_master.erl @@ -106,7 +106,7 @@ init_with_existing_bq(Q0, BQ, BQS) when ?is_amqqueue(Q0) -> %% need to handle migration. ok = rabbit_amqqueue:store_queue(Q3) end, - ok = rabbit_misc:execute_mnesia_transaction(Fun), + ok = rabbit_mnesia:execute_mnesia_transaction(Fun), {_MNode, SNodes} = rabbit_mirror_queue_misc:suggested_queue_nodes(Q0), %% We need synchronous add here (i.e. do not return until the %% mirror is running) so that when queue declaration is finished diff --git a/deps/rabbit/src/rabbit_mirror_queue_misc.erl b/deps/rabbit/src/rabbit_mirror_queue_misc.erl index 9978264241ee..e30664ac0b37 100644 --- a/deps/rabbit/src/rabbit_mirror_queue_misc.erl +++ b/deps/rabbit/src/rabbit_mirror_queue_misc.erl @@ -60,7 +60,7 @@ {'error', {'not_synced', [pid()]}}. remove_from_queue(QueueName, Self, DeadGMPids) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> %% Someone else could have deleted the queue before we %% get here. Or, gm group could've altered. see rabbitmq-server#914 @@ -162,7 +162,7 @@ slaves_to_start_on_failure(Q, DeadGMPids) -> on_vhost_up(VHost) -> QNames = - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> mnesia:foldl( fun @@ -353,7 +353,7 @@ stop_all_slaves(Reason, SPids, QName, GM, WaitTimeout) -> %% Normally when we remove a mirror another mirror or master will %% notice and update Mnesia. But we just removed them all, and %% have stopped listening ourselves. So manually clean up. - rabbit_misc:execute_mnesia_transaction(fun () -> + rabbit_mnesia:execute_mnesia_transaction(fun () -> [Q0] = mnesia:read({rabbit_queue, QName}), Q1 = amqqueue:set_gm_pids(Q0, []), Q2 = amqqueue:set_slave_pids(Q1, []), diff --git a/deps/rabbit/src/rabbit_mirror_queue_slave.erl b/deps/rabbit/src/rabbit_mirror_queue_slave.erl index 2b9dfc96d0d7..09440eafee99 100644 --- a/deps/rabbit/src/rabbit_mirror_queue_slave.erl +++ b/deps/rabbit/src/rabbit_mirror_queue_slave.erl @@ -106,7 +106,7 @@ handle_go(Q0) when ?is_amqqueue(Q0) -> %% process_flag(trap_exit, true), %% amqqueue_process traps exits too. {ok, GM} = gm:start_link(QName, ?MODULE, [self()], - fun rabbit_misc:execute_mnesia_transaction/1), + fun rabbit_mnesia:execute_mnesia_transaction/1), MRef = erlang:monitor(process, GM), %% We ignore the DOWN message because we are also linked and %% trapping exits, we just want to not get stuck and we will exit @@ -118,7 +118,7 @@ handle_go(Q0) when ?is_amqqueue(Q0) -> end, Self = self(), Node = node(), - case rabbit_misc:execute_mnesia_transaction( + case rabbit_mnesia:execute_mnesia_transaction( fun() -> init_it(Self, GM, Node, QName) end) of {new, QPid, GMPids} -> ok = file_handle_cache:register_callback( @@ -1088,7 +1088,7 @@ record_synchronised(Q0) when ?is_amqqueue(Q0) -> {ok, Q2} end end, - case rabbit_misc:execute_mnesia_transaction(F) of + case rabbit_mnesia:execute_mnesia_transaction(F) of ok -> ok; {ok, Q2} -> rabbit_mirror_queue_misc:maybe_drop_master_after_sync(Q2) end. diff --git a/deps/rabbit/src/rabbit_mnesia.erl b/deps/rabbit/src/rabbit_mnesia.erl index 4b9c35fc07c2..4e13163c799d 100644 --- a/deps/rabbit/src/rabbit_mnesia.erl +++ b/deps/rabbit/src/rabbit_mnesia.erl @@ -43,6 +43,16 @@ schema_info/1 ]). +%% Mnesia queries +-export([ + table_filter/3, + dirty_read_all/1, + dirty_read/1, + execute_mnesia_tx_with_tail/1, + execute_mnesia_transaction/1, + execute_mnesia_transaction/2 + ]). + %% Used internally in rpc calls -export([node_info/0, remove_node_if_mnesia_running/1]). @@ -796,6 +806,103 @@ info(Table, Items) -> All = [{name, Table} | mnesia:table_info(Table, all)], [{Item, proplists:get_value(Item, All)} || Item <- Items]. +%%-------------------------------------------------------------------- +%% Queries +%%-------------------------------------------------------------------- + +-spec table_filter + (fun ((A) -> boolean()), fun ((A, boolean()) -> 'ok'), atom()) -> [A]. +%% Apply a pre-post-commit function to all entries in a table that +%% satisfy a predicate, and return those entries. +%% +%% We ignore entries that have been modified or removed. +table_filter(Pred, PrePostCommitFun, TableName) -> + lists:foldl( + fun (E, Acc) -> + case execute_mnesia_transaction( + fun () -> mnesia:match_object(TableName, E, read) =/= [] + andalso Pred(E) end, + fun (false, _Tx) -> false; + (true, Tx) -> PrePostCommitFun(E, Tx), true + end) of + false -> Acc; + true -> [E | Acc] + end + end, [], dirty_read_all(TableName)). + +-spec dirty_read_all(atom()) -> [any()]. +dirty_read_all(TableName) -> + mnesia:dirty_select(TableName, [{'$1',[],['$1']}]). + +-spec dirty_read({atom(), any()}) -> + rabbit_types:ok_or_error2(any(), 'not_found'). +%% Normally we'd call mnesia:dirty_read/1 here, but that is quite +%% expensive due to general mnesia overheads (figuring out table types +%% and locations, etc). We get away with bypassing these because we +%% know that the tables we are looking at here +%% - are not the schema table +%% - have a local ram copy +%% - do not have any indices +dirty_read({Table, Key}) -> + case ets:lookup(Table, Key) of + [Result] -> {ok, Result}; + [] -> {error, not_found} + end. + +-spec execute_mnesia_tx_with_tail + (rabbit_misc:thunk(fun ((boolean()) -> B))) -> B | (fun ((boolean()) -> B)). +%% Like execute_mnesia_transaction/2, but TxFun is expected to return a +%% TailFun which gets called (only) immediately after the tx commit +execute_mnesia_tx_with_tail(TxFun) -> + case mnesia:is_transaction() of + true -> execute_mnesia_transaction(TxFun); + false -> TailFun = execute_mnesia_transaction(TxFun), + TailFun() + end. + +-spec execute_mnesia_transaction(rabbit_misc:thunk(A)) -> A. +execute_mnesia_transaction(TxFun) -> + %% Making this a sync_transaction allows us to use dirty_read + %% elsewhere and get a consistent result even when that read + %% executes on a different node. + case worker_pool:submit( + fun () -> + case mnesia:is_transaction() of + false -> DiskLogBefore = mnesia_dumper:get_log_writes(), + Res = mnesia:sync_transaction(TxFun), + DiskLogAfter = mnesia_dumper:get_log_writes(), + case DiskLogAfter == DiskLogBefore of + true -> file_handle_cache_stats:update( + mnesia_ram_tx), + Res; + false -> file_handle_cache_stats:update( + mnesia_disk_tx), + {sync, Res} + end; + true -> mnesia:sync_transaction(TxFun) + end + end, single) of + {sync, {atomic, Result}} -> mnesia_sync:sync(), Result; + {sync, {aborted, Reason}} -> throw({error, Reason}); + {atomic, Result} -> Result; + {aborted, Reason} -> throw({error, Reason}) + end. + +-spec execute_mnesia_transaction(rabbit_misc:thunk(A), fun ((A, boolean()) -> B)) -> B. +%% Like execute_mnesia_transaction/1 with additional Pre- and Post- +%% commit function +execute_mnesia_transaction(TxFun, PrePostCommitFun) -> + case mnesia:is_transaction() of + true -> throw(unexpected_transaction); + false -> ok + end, + PrePostCommitFun(execute_mnesia_transaction( + fun () -> + Result = TxFun(), + PrePostCommitFun(Result, true), + Result + end), false). + %%-------------------------------------------------------------------- %% Internal helpers %%-------------------------------------------------------------------- diff --git a/deps/rabbit/src/rabbit_networking.erl b/deps/rabbit/src/rabbit_networking.erl index f2095129515b..d01416b305fe 100644 --- a/deps/rabbit/src/rabbit_networking.erl +++ b/deps/rabbit/src/rabbit_networking.erl @@ -257,7 +257,7 @@ listener_of_protocol(Protocol) -> end. listener_of_protocol_mnesia(Protocol) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> MatchSpec = #listener{ node = node(), diff --git a/deps/rabbit/test/dead_lettering_SUITE.erl b/deps/rabbit/test/dead_lettering_SUITE.erl index 32bb7b484813..69d0a487d98a 100644 --- a/deps/rabbit/test/dead_lettering_SUITE.erl +++ b/deps/rabbit/test/dead_lettering_SUITE.erl @@ -1359,13 +1359,10 @@ dead_letter_extra_bcc(Config) -> ok. set_queue_options(QName, Options) -> - rabbit_misc:execute_mnesia_transaction( - fun() -> - rabbit_amqqueue:update(rabbit_misc:r(<<"/">>, queue, QName), - fun(Q) -> - amqqueue:set_options(Q, Options) - end) - end). + rabbit_amqqueue:update(rabbit_misc:r(<<"/">>, queue, QName), + fun(Q) -> + amqqueue:set_options(Q, Options) + end). metric_maxlen(Config) -> {_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), diff --git a/deps/rabbit/test/queue_parallel_SUITE.erl b/deps/rabbit/test/queue_parallel_SUITE.erl index 5ceb0795e3f9..4e47d763c6e6 100644 --- a/deps/rabbit/test/queue_parallel_SUITE.erl +++ b/deps/rabbit/test/queue_parallel_SUITE.erl @@ -838,9 +838,7 @@ set_queue_options(Config, QName, Options) -> rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, set_queue_options1, [QName, Options]). set_queue_options1(QName, Options) -> - rabbit_misc:execute_mnesia_transaction(fun() -> - rabbit_amqqueue:update(rabbit_misc:r(<<"/">>, queue, QName), - fun(Q) -> - amqqueue:set_options(Q, Options) - end) - end). + rabbit_amqqueue:update(rabbit_misc:r(<<"/">>, queue, QName), + fun(Q) -> + amqqueue:set_options(Q, Options) + end). diff --git a/deps/rabbit/test/topic_permission_SUITE.erl b/deps/rabbit/test/topic_permission_SUITE.erl index d6c323ca47ec..14474411a871 100644 --- a/deps/rabbit/test/topic_permission_SUITE.erl +++ b/deps/rabbit/test/topic_permission_SUITE.erl @@ -145,7 +145,7 @@ topic_permission_checks(Config) -> topic_permission_checks1(_Config) -> 0 = length(ets:tab2list(rabbit_topic_permission)), - rabbit_misc:execute_mnesia_transaction(fun() -> + rabbit_mnesia:execute_mnesia_transaction(fun() -> ok = mnesia:write(rabbit_vhost, vhost:new(<<"/">>, []), write), diff --git a/deps/rabbit/test/unit_access_control_SUITE.erl b/deps/rabbit/test/unit_access_control_SUITE.erl index 475768725ff8..f3ee24cd6790 100644 --- a/deps/rabbit/test/unit_access_control_SUITE.erl +++ b/deps/rabbit/test/unit_access_control_SUITE.erl @@ -422,7 +422,7 @@ topic_matching1(_Config) -> passed. exchange_op_callback(X, Fun, Args) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> rabbit_exchange:callback(X, Fun, transaction, [X] ++ Args) end), rabbit_exchange:callback(X, Fun, none, [X] ++ Args). diff --git a/deps/rabbit/test/unit_gm_SUITE.erl b/deps/rabbit/test/unit_gm_SUITE.erl index 870f93ff52d6..75024ababd90 100644 --- a/deps/rabbit/test/unit_gm_SUITE.erl +++ b/deps/rabbit/test/unit_gm_SUITE.erl @@ -70,7 +70,7 @@ member_death(_Config) -> fun (Pid, Pid2) -> {ok, Pid3} = gm:start_link( ?MODULE, ?MODULE, self(), - fun rabbit_misc:execute_mnesia_transaction/1), + fun rabbit_mnesia:execute_mnesia_transaction/1), passed = receive_joined(Pid3, [Pid, Pid2, Pid3], timeout_joining_gm_group_3), passed = receive_birth(Pid, Pid3, timeout_waiting_for_birth_3_1), @@ -120,10 +120,10 @@ down_in_members_change(_Config) -> %% Setup ok = gm:create_tables(), {ok, Pid} = gm:start_link(?MODULE, ?MODULE, self(), - fun rabbit_misc:execute_mnesia_transaction/1), + fun rabbit_mnesia:execute_mnesia_transaction/1), passed = receive_joined(Pid, [Pid], timeout_joining_gm_group_1), {ok, Pid2} = gm:start_link(?MODULE, ?MODULE, self(), - fun rabbit_misc:execute_mnesia_transaction/1), + fun rabbit_mnesia:execute_mnesia_transaction/1), passed = receive_joined(Pid2, [Pid, Pid2], timeout_joining_gm_group_2), passed = receive_birth(Pid, Pid2, timeout_waiting_for_birth_2), @@ -167,11 +167,11 @@ with_two_members(Fun) -> ok = gm:create_tables(), {ok, Pid} = gm:start_link(?MODULE, ?MODULE, self(), - fun rabbit_misc:execute_mnesia_transaction/1), + fun rabbit_mnesia:execute_mnesia_transaction/1), passed = receive_joined(Pid, [Pid], timeout_joining_gm_group_1), {ok, Pid2} = gm:start_link(?MODULE, ?MODULE, self(), - fun rabbit_misc:execute_mnesia_transaction/1), + fun rabbit_mnesia:execute_mnesia_transaction/1), passed = receive_joined(Pid2, [Pid, Pid2], timeout_joining_gm_group_2), passed = receive_birth(Pid, Pid2, timeout_waiting_for_birth_2), diff --git a/deps/rabbit_common/src/rabbit_misc.erl b/deps/rabbit_common/src/rabbit_misc.erl index b211d1efb711..3319a404fe50 100644 --- a/deps/rabbit_common/src/rabbit_misc.erl +++ b/deps/rabbit_common/src/rabbit_misc.erl @@ -23,7 +23,6 @@ -export([die/1, frame_error/2, amqp_error/4, quit/1, protocol_error/3, protocol_error/4, protocol_error/1]). -export([type_class/1, assert_args_equivalence/4, assert_field_equivalence/4]). --export([dirty_read/1]). -export([table_lookup/2, set_table_value/4, amqp_table/1, to_amqp_table/1]). -export([r/3, r/2, r_arg/4, rs/1]). -export([enable_cover/0, report_cover/0]). @@ -31,15 +30,10 @@ -export([start_cover/1]). -export([throw_on_error/2, with_exit_handler/2, is_abnormal_exit/1, filter_exit_map/2]). --export([with_user/2]). --export([execute_mnesia_transaction/1]). --export([execute_mnesia_transaction/2]). --export([execute_mnesia_tx_with_tail/1]). -export([ensure_ok/2]). -export([tcp_name/3, format_inet_error/1]). -export([upmap/2, map_in_order/2, utf8_safe/1]). --export([table_filter/3]). --export([dirty_read_all/1, dirty_foreach_key/2, dirty_dump_log/1]). +-export([dirty_dump_log/1]). -export([format/2, format_many/1, format_stderr/2]). -export([unfold/2, ceil/1, queue_fold/3]). -export([sort_field_table/1]). @@ -140,8 +134,6 @@ -spec equivalence_fail (any(), any(), rabbit_types:r(any()), atom() | binary()) -> rabbit_types:connection_exit(). --spec dirty_read({atom(), any()}) -> - rabbit_types:ok_or_error2(any(), 'not_found'). -spec table_lookup(rabbit_framing:amqp_table(), binary()) -> 'undefined' | {rabbit_framing:amqp_field_type(), any()}. -spec set_table_value @@ -172,22 +164,12 @@ -spec with_exit_handler(thunk(A), thunk(A)) -> A. -spec is_abnormal_exit(any()) -> boolean(). -spec filter_exit_map(fun ((A) -> B), [A]) -> [B]. --spec with_user(rabbit_types:username(), thunk(A)) -> A. --spec execute_mnesia_transaction(thunk(A)) -> A. --spec execute_mnesia_transaction(thunk(A), fun ((A, boolean()) -> B)) -> B. --spec execute_mnesia_tx_with_tail - (thunk(fun ((boolean()) -> B))) -> B | (fun ((boolean()) -> B)). -spec ensure_ok(ok_or_error(), atom()) -> 'ok'. -spec tcp_name(atom(), inet:ip_address(), rabbit_net:ip_port()) -> atom(). -spec format_inet_error(atom()) -> string(). -spec upmap(fun ((A) -> B), [A]) -> [B]. -spec map_in_order(fun ((A) -> B), [A]) -> [B]. --spec table_filter - (fun ((A) -> boolean()), fun ((A, boolean()) -> 'ok'), atom()) -> [A]. --spec dirty_read_all(atom()) -> [any()]. --spec dirty_foreach_key(fun ((any()) -> any()), atom()) -> - 'ok' | 'aborted'. -spec dirty_dump_log(file:filename()) -> ok_or_error(). -spec format(string(), [any()]) -> string(). -spec format_many([{string(), [any()]}]) -> string(). @@ -361,19 +343,6 @@ val(Value) -> false -> "'~tp'" end, [Value]). -%% Normally we'd call mnesia:dirty_read/1 here, but that is quite -%% expensive due to general mnesia overheads (figuring out table types -%% and locations, etc). We get away with bypassing these because we -%% know that the tables we are looking at here -%% - are not the schema table -%% - have a local ram copy -%% - do not have any indices -dirty_read({Table, Key}) -> - case ets:lookup(Table, Key) of - [Result] -> {ok, Result}; - [] -> {error, not_found} - end. - %% %% Attribute Tables %% @@ -544,67 +513,6 @@ filter_exit_map(F, L) -> fun () -> Ref end, fun () -> F(I) end) || I <- L]). - -with_user(Username, Thunk) -> - fun () -> - case mnesia:read({rabbit_user, Username}) of - [] -> - mnesia:abort({no_such_user, Username}); - [_U] -> - Thunk() - end - end. - -execute_mnesia_transaction(TxFun) -> - %% Making this a sync_transaction allows us to use dirty_read - %% elsewhere and get a consistent result even when that read - %% executes on a different node. - case worker_pool:submit( - fun () -> - case mnesia:is_transaction() of - false -> DiskLogBefore = mnesia_dumper:get_log_writes(), - Res = mnesia:sync_transaction(TxFun), - DiskLogAfter = mnesia_dumper:get_log_writes(), - case DiskLogAfter == DiskLogBefore of - true -> file_handle_cache_stats:update( - mnesia_ram_tx), - Res; - false -> file_handle_cache_stats:update( - mnesia_disk_tx), - {sync, Res} - end; - true -> mnesia:sync_transaction(TxFun) - end - end, single) of - {sync, {atomic, Result}} -> mnesia_sync:sync(), Result; - {sync, {aborted, Reason}} -> throw({error, Reason}); - {atomic, Result} -> Result; - {aborted, Reason} -> throw({error, Reason}) - end. - -%% Like execute_mnesia_transaction/1 with additional Pre- and Post- -%% commit function -execute_mnesia_transaction(TxFun, PrePostCommitFun) -> - case mnesia:is_transaction() of - true -> throw(unexpected_transaction); - false -> ok - end, - PrePostCommitFun(execute_mnesia_transaction( - fun () -> - Result = TxFun(), - PrePostCommitFun(Result, true), - Result - end), false). - -%% Like execute_mnesia_transaction/2, but TxFun is expected to return a -%% TailFun which gets called (only) immediately after the tx commit -execute_mnesia_tx_with_tail(TxFun) -> - case mnesia:is_transaction() of - true -> execute_mnesia_transaction(TxFun); - false -> TailFun = execute_mnesia_transaction(TxFun), - TailFun() - end. - ensure_ok(ok, _) -> ok; ensure_ok({error, Reason}, ErrorTag) -> throw({error, {ErrorTag, Reason}}). @@ -660,41 +568,6 @@ map_in_order(F, L) -> lists:reverse( lists:foldl(fun (E, Acc) -> [F(E) | Acc] end, [], L)). -%% Apply a pre-post-commit function to all entries in a table that -%% satisfy a predicate, and return those entries. -%% -%% We ignore entries that have been modified or removed. -table_filter(Pred, PrePostCommitFun, TableName) -> - lists:foldl( - fun (E, Acc) -> - case execute_mnesia_transaction( - fun () -> mnesia:match_object(TableName, E, read) =/= [] - andalso Pred(E) end, - fun (false, _Tx) -> false; - (true, Tx) -> PrePostCommitFun(E, Tx), true - end) of - false -> Acc; - true -> [E | Acc] - end - end, [], dirty_read_all(TableName)). - -dirty_read_all(TableName) -> - mnesia:dirty_select(TableName, [{'$1',[],['$1']}]). - -dirty_foreach_key(F, TableName) -> - dirty_foreach_key1(F, TableName, mnesia:dirty_first(TableName)). - -dirty_foreach_key1(_F, _TableName, '$end_of_table') -> - ok; -dirty_foreach_key1(F, TableName, K) -> - case catch mnesia:dirty_next(TableName, K) of - {'EXIT', _} -> - aborted; - NextKey -> - F(K), - dirty_foreach_key1(F, TableName, NextKey) - end. - dirty_dump_log(FileName) -> {ok, LH} = disk_log:open([{name, dirty_dump_log}, {mode, read_only}, diff --git a/deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange.erl b/deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange.erl index d5eae063d3dd..326ae37b191a 100644 --- a/deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange.erl +++ b/deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange.erl @@ -38,7 +38,7 @@ create(X) -> }). create_in_mnesia(X) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> create_in_mnesia_tx(X) end). create_in_mnesia_tx(X) -> @@ -59,7 +59,7 @@ create_binding(Src, Dst, Weight, UpdateFun) -> }). create_binding_in_mnesia(Src, Dst, Weight, UpdateFun) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> create_binding_in_mnesia_tx(Src, Dst, Weight, UpdateFun) end). @@ -98,7 +98,7 @@ delete(XName) -> }). delete_in_mnesia(XName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> mnesia:write_lock_table(?HASH_RING_STATE_TABLE), mnesia:delete({?HASH_RING_STATE_TABLE, XName}) @@ -110,7 +110,7 @@ delete_bindings(Bindings, DeleteFun) -> }). delete_bindings_in_mnesia(Bindings, DeleteFun) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> [delete_binding_in_mnesia(Binding, DeleteFun) || Binding <- Bindings] end). diff --git a/deps/rabbitmq_federation/src/rabbit_federation_exchange_link.erl b/deps/rabbitmq_federation/src/rabbit_federation_exchange_link.erl index 440d32d632ce..2cb9952072e2 100644 --- a/deps/rabbitmq_federation/src/rabbit_federation_exchange_link.erl +++ b/deps/rabbitmq_federation/src/rabbit_federation_exchange_link.erl @@ -439,7 +439,7 @@ go(S0 = {not_started, {Upstream, UParams, DownXName}}) -> _ -> unknown end, {Serial, Bindings} = - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> {rabbit_exchange:peek_serial(DownXName), rabbit_binding:list_for_source(DownXName)} diff --git a/deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange.erl b/deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange.erl index 6ce28123ea83..2dde2ed8613c 100644 --- a/deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange.erl +++ b/deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange.erl @@ -49,7 +49,7 @@ create_or_update(XName, BindingKeyAndFun, ErrorFun) -> }). create_or_update_in_mnesia(XName, BindingKeyAndFun, ErrorFun) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> #?JMS_TOPIC_RECORD{x_selector_funs = BindingFuns} = read_state_in_mnesia(XName, ErrorFun), @@ -66,7 +66,7 @@ insert(XName, BFuns) -> }). insert_in_mnesia(XName, BFuns) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> write_state_fun_in_mnesia(XName, BFuns) end). @@ -103,7 +103,7 @@ delete(XName) -> }). delete_in_mnesia(XName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> mnesia:delete(?JMS_TOPIC_TABLE, XName, write) end). delete(XName, BindingKeys, ErrorFun) -> @@ -113,7 +113,7 @@ delete(XName, BindingKeys, ErrorFun) -> }). delete_in_mnesia(XName, BindingKeys, ErrorFun) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> #?JMS_TOPIC_RECORD{x_selector_funs = BindingFuns} = read_state_in_mnesia(XName, ErrorFun), diff --git a/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl b/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl index 49b04dca839b..1dde6fda15ef 100644 --- a/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl +++ b/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl @@ -46,7 +46,7 @@ get(XName) -> }). get_in_mnesia(XName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> get_in_mnesia_tx(XName) end). get_in_mnesia_tx(XName) -> @@ -67,7 +67,7 @@ insert(XName, Message, Length) -> }). insert_in_mnesia(XName, Message, Length) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun () -> Cached = get_in_mnesia_tx(XName), insert_in_mnesia(XName, Cached, Message, Length) @@ -109,7 +109,7 @@ delete(XName) -> }). delete_in_mnesia(XName) -> - rabbit_misc:execute_mnesia_transaction( + rabbit_mnesia:execute_mnesia_transaction( fun() -> mnesia:delete(?RH_TABLE, XName, write) end). From c95ac0a9e676edf91bf8cc525ee1284507b32db0 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Mon, 16 Jan 2023 11:57:58 +0100 Subject: [PATCH 10/12] Move maintenance mode Mnesia-specific code to rabbit_db_maintenance module --- deps/rabbit/src/rabbit_db_maintenance.erl | 152 ++++++++++++++++++++++ deps/rabbit/src/rabbit_maintenance.erl | 71 +--------- 2 files changed, 157 insertions(+), 66 deletions(-) create mode 100644 deps/rabbit/src/rabbit_db_maintenance.erl diff --git a/deps/rabbit/src/rabbit_db_maintenance.erl b/deps/rabbit/src/rabbit_db_maintenance.erl new file mode 100644 index 000000000000..ba6e59be64db --- /dev/null +++ b/deps/rabbit/src/rabbit_db_maintenance.erl @@ -0,0 +1,152 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_maintenance). + +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([ + setup_schema/0, + set/1, + get/1, + get_consistent/1 + ]). + +-type mnesia_table() :: atom(). + +-define(TABLE, rabbit_node_maintenance_states). + +%% ------------------------------------------------------------------- +%% setup_schema(). +%% ------------------------------------------------------------------- + +-spec setup_schema() -> ok. +%% @doc Creates the internal schema used by the selected metadata store +%% +%% @private + +setup_schema() -> + rabbit_db:run( + #{mnesia => fun() -> setup_schema_in_mnesia() end + }). + +setup_schema_in_mnesia() -> + TableName = status_table_name(), + rabbit_log:info( + "Creating table ~ts for maintenance mode status", + [TableName]), + try + _ = rabbit_table:create( + TableName, + status_table_definition()) + catch throw:Reason -> + rabbit_log:error( + "Failed to create maintenance status table: ~tp", + [Reason]) + end. + +-spec status_table_name() -> mnesia_table(). +status_table_name() -> + ?TABLE. + +-spec status_table_definition() -> list(). +status_table_definition() -> + maps:to_list(#{ + record_name => node_maintenance_state, + attributes => record_info(fields, node_maintenance_state) + }). + +%% ------------------------------------------------------------------- +%% set(). +%% ------------------------------------------------------------------- + +-spec set(Status) -> ok when + Status :: rabbit_maintenance:maintenance_status(). +%% @doc Sets the maintenance status for the local node +%% +%% @private + +set(Status) -> + rabbit_db:run( + #{mnesia => fun() -> set_in_mnesia(Status) end + }). + +set_in_mnesia(Status) -> + Res = mnesia:transaction( + fun () -> + case mnesia:wread({?TABLE, node()}) of + [] -> + Row = #node_maintenance_state{ + node = node(), + status = Status + }, + mnesia:write(?TABLE, Row, write); + [Row0] -> + Row = Row0#node_maintenance_state{ + node = node(), + status = Status + }, + mnesia:write(?TABLE, Row, write) + end + end), + case Res of + {atomic, ok} -> true; + _ -> false + end. + +%% ------------------------------------------------------------------- +%% get(). +%% ------------------------------------------------------------------- + +-spec get(Node) -> Status when + Node :: node(), + Status :: rabbit_maintenance:maintenance_status(). +%% @doc Returns the status for the given node using a local query. +%% +%% @returns the status if any, or `undefined'. +%% +%% @private + +get(Node) -> + rabbit_db:run( + #{mnesia => fun() -> get_in_mnesia(Node) end + }). + +get_in_mnesia(Node) -> + case catch mnesia:dirty_read(?TABLE, Node) of + [] -> undefined; + [#node_maintenance_state{node = Node, status = Status}] -> + Status; + _ -> undefined + end. + +%% ------------------------------------------------------------------- +%% get(). +%% ------------------------------------------------------------------- + +-spec get_consistent(Node) -> Status when + Node :: node(), + Status :: rabbit_maintenance:maintenance_status(). +%% @doc Returns the status for the given node using a consistent query. +%% +%% @returns the status if any, or `undefined'. +%% +%% @private + +get_consistent(Node) -> + rabbit_db:run( + #{mnesia => fun() -> get_consistent_in_mnesia(Node) end + }). + +get_consistent_in_mnesia(Node) -> + case mnesia:transaction(fun() -> mnesia:read(?TABLE, Node) end) of + {atomic, []} -> undefined; + {atomic, [#node_maintenance_state{node = Node, status = Status}]} -> + Status; + {atomic, _} -> undefined; + {aborted, _Reason} -> undefined + end. diff --git a/deps/rabbit/src/rabbit_maintenance.erl b/deps/rabbit/src/rabbit_maintenance.erl index d3af0bfdef18..567ba3bb5af2 100644 --- a/deps/rabbit/src/rabbit_maintenance.erl +++ b/deps/rabbit/src/rabbit_maintenance.erl @@ -28,17 +28,13 @@ random_primary_replica_transfer_candidate_node/2, transfer_leadership_of_quorum_queues/1, transfer_leadership_of_classic_mirrored_queues/1, - status_table_name/0, - status_table_definition/0, boot/0 ]). --define(TABLE, rabbit_node_maintenance_states). -define(DEFAULT_STATUS, regular). -define(DRAINING_STATUS, draining). -type maintenance_status() :: ?DEFAULT_STATUS | ?DRAINING_STATUS. --type mnesia_table() :: atom(). -export_type([ maintenance_status/0 @@ -54,35 +50,12 @@ {requires, networking}]}). boot() -> - TableName = status_table_name(), - rabbit_log:info( - "Creating table ~ts for maintenance mode status", - [TableName]), - try - _ = rabbit_table:create( - TableName, - status_table_definition()) - catch throw:Reason -> - rabbit_log:error( - "Failed to create maintenance status table: ~tp", - [Reason]) - end. + rabbit_db_maintenance:setup_schema(). %% %% API %% --spec status_table_name() -> mnesia_table(). -status_table_name() -> - ?TABLE. - --spec status_table_definition() -> list(). -status_table_definition() -> - maps:to_list(#{ - record_name => node_maintenance_state, - attributes => record_info(fields, node_maintenance_state) - }). - -spec is_enabled() -> boolean(). is_enabled() -> true. @@ -138,35 +111,12 @@ revive() -> -spec mark_as_being_drained() -> boolean(). mark_as_being_drained() -> rabbit_log:debug("Marking the node as undergoing maintenance"), - set_maintenance_status_status(?DRAINING_STATUS). + rabbit_db_maintenance:set(?DRAINING_STATUS). -spec unmark_as_being_drained() -> boolean(). unmark_as_being_drained() -> rabbit_log:debug("Unmarking the node as undergoing maintenance"), - set_maintenance_status_status(?DEFAULT_STATUS). - -set_maintenance_status_status(Status) -> - Res = mnesia:transaction(fun () -> - case mnesia:wread({?TABLE, node()}) of - [] -> - Row = #node_maintenance_state{ - node = node(), - status = Status - }, - mnesia:write(?TABLE, Row, write); - [Row0] -> - Row = Row0#node_maintenance_state{ - node = node(), - status = Status - }, - mnesia:write(?TABLE, Row, write) - end - end), - case Res of - {atomic, ok} -> true; - _ -> false - end. - + rabbit_db_maintenance:set(?DEFAULT_STATUS). -spec is_being_drained_local_read(node()) -> boolean(). is_being_drained_local_read(Node) -> @@ -180,22 +130,11 @@ is_being_drained_consistent_read(Node) -> -spec status_local_read(node()) -> maintenance_status(). status_local_read(Node) -> - case catch mnesia:dirty_read(?TABLE, Node) of - [] -> ?DEFAULT_STATUS; - [#node_maintenance_state{node = Node, status = Status}] -> - Status; - _ -> ?DEFAULT_STATUS - end. + rabbit_db_maintenance:get(Node). -spec status_consistent_read(node()) -> maintenance_status(). status_consistent_read(Node) -> - case mnesia:transaction(fun() -> mnesia:read(?TABLE, Node) end) of - {atomic, []} -> ?DEFAULT_STATUS; - {atomic, [#node_maintenance_state{node = Node, status = Status}]} -> - Status; - {atomic, _} -> ?DEFAULT_STATUS; - {aborted, _Reason} -> ?DEFAULT_STATUS - end. + rabbit_db_maintenance:get_consistent(Node). -spec filter_out_drained_nodes_local_read([node()]) -> [node()]. filter_out_drained_nodes_local_read(Nodes) -> From 7a12cf840d7e0917aec19ec1c6519fe08d9849d1 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Mon, 16 Jan 2023 12:48:00 +0100 Subject: [PATCH 11/12] Adapt plugins to new exchange and exchange decorator API --- deps/rabbit/src/rabbit_exchange_decorator.erl | 2 +- .../src/rabbit_event_exchange_decorator.erl | 12 ++++++------ .../src/rabbit_exchange_type_random.erl | 10 +++++----- .../src/rabbit_sharding_exchange_decorator.erl | 11 ++++------- .../rabbit_sharding_exchange_type_modulus_hash.erl | 10 +++++----- 5 files changed, 21 insertions(+), 24 deletions(-) diff --git a/deps/rabbit/src/rabbit_exchange_decorator.erl b/deps/rabbit/src/rabbit_exchange_decorator.erl index 3cafc4ee843d..24dc93232c4b 100644 --- a/deps/rabbit/src/rabbit_exchange_decorator.erl +++ b/deps/rabbit/src/rabbit_exchange_decorator.erl @@ -109,5 +109,5 @@ maybe_recover(X = #exchange{name = Name, Old -> ok; _ -> %% TODO create a tx here for non-federation decorators _ = [M:create(none, X) || M <- New -- Old], - rabbit_exchange:update_decorators(Name) + rabbit_exchange:update_decorators(Name, Decs1) end. diff --git a/deps/rabbitmq_event_exchange/src/rabbit_event_exchange_decorator.erl b/deps/rabbitmq_event_exchange/src/rabbit_event_exchange_decorator.erl index a0d5c244250f..36876d834147 100644 --- a/deps/rabbitmq_event_exchange/src/rabbit_event_exchange_decorator.erl +++ b/deps/rabbitmq_event_exchange/src/rabbit_event_exchange_decorator.erl @@ -21,7 +21,7 @@ -behaviour(rabbit_exchange_decorator). -export([description/0, serialise_events/1]). --export([create/2, delete/3, policy_changed/2, +-export([create/2, delete/2, policy_changed/2, add_binding/3, remove_bindings/3, route/2, active_for/1]). description() -> @@ -32,13 +32,13 @@ serialise_events(_) -> false. create(_, _) -> ok. -delete(_, _, _) -> +delete(_, _) -> ok. policy_changed(_, _) -> ok. -add_binding(transaction, #exchange{name = #resource{name = ?EXCH_NAME} = Name}, +add_binding(none, #exchange{name = #resource{name = ?EXCH_NAME} = Name}, _Bs) -> case rabbit_binding:list_for_source(Name) of [_] -> @@ -47,10 +47,10 @@ add_binding(transaction, #exchange{name = #resource{name = ?EXCH_NAME} = Name}, _ -> ok end; -add_binding(_, _X, _Bs) -> +add_binding(_, _, _) -> ok. -remove_bindings(transaction, #exchange{name = #resource{name = ?EXCH_NAME} = Name}, +remove_bindings(none, #exchange{name = #resource{name = ?EXCH_NAME} = Name}, _Bs) -> case rabbit_binding:list_for_source(Name) of [] -> @@ -59,7 +59,7 @@ remove_bindings(transaction, #exchange{name = #resource{name = ?EXCH_NAME} = Nam _ -> ok end; -remove_bindings(_, _X, _Bs) -> +remove_bindings(_, _, _) -> ok. route(_, _) -> []. diff --git a/deps/rabbitmq_random_exchange/src/rabbit_exchange_type_random.erl b/deps/rabbitmq_random_exchange/src/rabbit_exchange_type_random.erl index 0970684e549a..ccfcd59b3f57 100644 --- a/deps/rabbitmq_random_exchange/src/rabbit_exchange_type_random.erl +++ b/deps/rabbitmq_random_exchange/src/rabbit_exchange_type_random.erl @@ -20,7 +20,7 @@ add_binding/3, assert_args_equivalence/2, create/2, - delete/3, + delete/2, policy_changed/2, description/0, recover/2, @@ -49,12 +49,12 @@ info(_X) -> []. info(_X, _) -> []. serialise_events() -> false. validate(_X) -> ok. -create(_Tx, _X) -> ok. +create(_Serial, _X) -> ok. recover(_X, _Bs) -> ok. -delete(_Tx, _X, _Bs) -> ok. +delete(_Serial, _X) -> ok. policy_changed(_X1, _X2) -> ok. -add_binding(_Tx, _X, _B) -> ok. -remove_bindings(_Tx, _X, _Bs) -> ok. +add_binding(_Serial, _X, _B) -> ok. +remove_bindings(_Serial, _X, _Bs) -> ok. validate_binding(_X, _B) -> ok. assert_args_equivalence(X, Args) -> rabbit_exchange:assert_args_equivalence(X, Args). diff --git a/deps/rabbitmq_sharding/src/rabbit_sharding_exchange_decorator.erl b/deps/rabbitmq_sharding/src/rabbit_sharding_exchange_decorator.erl index cc93dfacc023..2dc2986fe7ed 100644 --- a/deps/rabbitmq_sharding/src/rabbit_sharding_exchange_decorator.erl +++ b/deps/rabbitmq_sharding/src/rabbit_sharding_exchange_decorator.erl @@ -20,7 +20,7 @@ -behaviour(rabbit_exchange_decorator). -export([description/0, serialise_events/1]). --export([create/2, delete/3, policy_changed/2, +-export([create/2, delete/2, policy_changed/2, add_binding/3, remove_bindings/3, route/2, active_for/1]). -import(rabbit_sharding_util, [shard/1]). @@ -32,14 +32,12 @@ description() -> serialise_events(_X) -> false. -create(transaction, _X) -> - ok; create(none, X) -> _ = maybe_start_sharding(X), ok. -add_binding(_Tx, _X, _B) -> ok. -remove_bindings(_Tx, _X, _Bs) -> ok. +add_binding(_Serial, _X, _B) -> ok. +remove_bindings(_Serial, _X, _Bs) -> ok. route(_, _) -> []. @@ -50,8 +48,7 @@ active_for(X) -> end. %% we have to remove the policy from ?SHARDING_TABLE -delete(transaction, _X, _Bs) -> ok; -delete(none, X, _Bs) -> +delete(none, X) -> _ = maybe_stop_sharding(X), ok. diff --git a/deps/rabbitmq_sharding/src/rabbit_sharding_exchange_type_modulus_hash.erl b/deps/rabbitmq_sharding/src/rabbit_sharding_exchange_type_modulus_hash.erl index 051ad39e739b..ecf6a71096c8 100644 --- a/deps/rabbitmq_sharding/src/rabbit_sharding_exchange_type_modulus_hash.erl +++ b/deps/rabbitmq_sharding/src/rabbit_sharding_exchange_type_modulus_hash.erl @@ -13,7 +13,7 @@ -export([description/0, serialise_events/0, route/2, info/1, info/2]). -export([validate/1, validate_binding/2, - create/2, delete/3, policy_changed/2, + create/2, delete/2, policy_changed/2, add_binding/3, remove_bindings/3, assert_args_equivalence/2]). -rabbit_boot_step( @@ -47,11 +47,11 @@ info(_, _) -> []. validate(_X) -> ok. validate_binding(_X, _B) -> ok. -create(_Tx, _X) -> ok. -delete(_Tx, _X, _Bs) -> ok. +create(_Serial, _X) -> ok. +delete(_Serial, _X) -> ok. policy_changed(_X1, _X2) -> ok. -add_binding(_Tx, _X, _B) -> ok. -remove_bindings(_Tx, _X, _Bs) -> ok. +add_binding(_Serial, _X, _B) -> ok. +remove_bindings(_Serial, _X, _Bs) -> ok. assert_args_equivalence(X, Args) -> rabbit_exchange:assert_args_equivalence(X, Args). From 9cf10ed8a7adb5bf41f619426253aa0661e2158b Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Mon, 16 Jan 2023 13:35:17 +0100 Subject: [PATCH 12/12] Unit test rabbit_db_* modules, spec and API updates --- deps/rabbit/BUILD.bazel | 45 +- deps/rabbit/src/rabbit_amqqueue.erl | 183 ++-- deps/rabbit/src/rabbit_amqqueue_process.erl | 2 +- deps/rabbit/src/rabbit_basic.erl | 2 +- deps/rabbit/src/rabbit_binding.erl | 24 +- deps/rabbit/src/rabbit_channel.erl | 4 +- deps/rabbit/src/rabbit_classic_queue.erl | 2 +- deps/rabbit/src/rabbit_core_ff.erl | 3 +- deps/rabbit/src/rabbit_db_binding.erl | 672 ++++++++------ deps/rabbit/src/rabbit_db_exchange.erl | 281 +++--- deps/rabbit/src/rabbit_db_maintenance.erl | 9 +- deps/rabbit/src/rabbit_db_msup.erl | 173 ++-- deps/rabbit/src/rabbit_db_policy.erl | 12 +- deps/rabbit/src/rabbit_db_queue.erl | 824 +++++++++++++----- deps/rabbit/src/rabbit_db_topic_exchange.erl | 79 +- deps/rabbit/src/rabbit_db_user.erl | 21 + deps/rabbit/src/rabbit_db_vhost.erl | 19 + deps/rabbit/src/rabbit_dead_letter.erl | 2 +- deps/rabbit/src/rabbit_exchange.erl | 22 +- .../rabbit/src/rabbit_exchange_type_topic.erl | 2 +- deps/rabbit/src/rabbit_fifo_dlx_worker.erl | 6 +- deps/rabbit/src/rabbit_maintenance.erl | 14 +- deps/rabbit/src/rabbit_policy.erl | 8 +- deps/rabbit/src/rabbit_priority_queue.erl | 2 +- deps/rabbit/src/rabbit_quorum_queue.erl | 10 +- deps/rabbit/src/rabbit_router.erl | 37 +- deps/rabbit/src/rabbit_stream_coordinator.erl | 6 +- deps/rabbit/test/rabbit_db_binding_SUITE.erl | 331 +++++++ deps/rabbit/test/rabbit_db_exchange_SUITE.erl | 330 +++++++ .../test/rabbit_db_maintenance_SUITE.erl | 93 ++ deps/rabbit/test/rabbit_db_msup_SUITE.erl | 136 +++ deps/rabbit/test/rabbit_db_policy_SUITE.erl | 96 ++ deps/rabbit/test/rabbit_db_queue_SUITE.erl | 596 +++++++++++++ .../test/rabbit_db_topic_exchange_SUITE.erl | 158 ++++ deps/rabbit/test/topic_permission_SUITE.erl | 16 +- .../rabbit/test/unit_access_control_SUITE.erl | 6 +- .../BUILD.bazel | 3 +- .../rabbit_exchange_type_consistent_hash.erl | 4 +- .../src/rabbit_federation_exchange_link.erl | 8 +- deps/rabbitmq_jms_topic_exchange/BUILD.bazel | 5 + .../src/rabbit_jms_topic_exchange.erl | 6 +- .../src/rabbit_mqtt_processor.erl | 2 +- deps/rabbitmq_random_exchange/BUILD.bazel | 7 +- .../BUILD.bazel | 5 + .../src/rabbit_db_rh_exchange.erl | 7 - .../rabbit_exchange_type_recent_history.erl | 6 +- 46 files changed, 3360 insertions(+), 919 deletions(-) create mode 100644 deps/rabbit/test/rabbit_db_binding_SUITE.erl create mode 100644 deps/rabbit/test/rabbit_db_exchange_SUITE.erl create mode 100644 deps/rabbit/test/rabbit_db_maintenance_SUITE.erl create mode 100644 deps/rabbit/test/rabbit_db_msup_SUITE.erl create mode 100644 deps/rabbit/test/rabbit_db_policy_SUITE.erl create mode 100644 deps/rabbit/test/rabbit_db_queue_SUITE.erl create mode 100644 deps/rabbit/test/rabbit_db_topic_exchange_SUITE.erl diff --git a/deps/rabbit/BUILD.bazel b/deps/rabbit/BUILD.bazel index da8aff391708..d2dc86f326b6 100644 --- a/deps/rabbit/BUILD.bazel +++ b/deps/rabbit/BUILD.bazel @@ -513,15 +513,12 @@ rabbitmq_integration_suite( ], ) -rabbitmq_suite( +rabbitmq_integration_suite( name = "mirrored_supervisor_SUITE", size = "small", additional_srcs = [ "test/mirrored_supervisor_SUITE_gs.erl", - ], - deps = [ - "//deps/rabbit_common:erlang_app", - ], + ] ) rabbitmq_suite( @@ -1096,6 +1093,9 @@ rabbitmq_integration_suite( rabbitmq_integration_suite( name = "exchanges_SUITE", size = "small", + additional_beam = [ + ":quorum_queue_utils", + ], ) rabbitmq_integration_suite( @@ -1103,6 +1103,41 @@ rabbitmq_integration_suite( size = "small", ) +rabbitmq_integration_suite( + name = "rabbit_db_queue_SUITE", + size = "small", +) + +rabbitmq_integration_suite( + name = "rabbit_db_maintenance_SUITE", + size = "small", +) + +rabbitmq_integration_suite( + name = "rabbit_db_topic_exchange_SUITE", + size = "small", +) + +rabbitmq_integration_suite( + name = "rabbit_db_exchange_SUITE", + size = "small", +) + +rabbitmq_integration_suite( + name = "rabbit_db_binding_SUITE", + size = "small", +) + +rabbitmq_integration_suite( + name = "rabbit_db_msup_SUITE", + size = "small", +) + +rabbitmq_integration_suite( + name = "rabbit_db_policy_SUITE", + size = "small", +) + assert_suites() filegroup( diff --git a/deps/rabbit/src/rabbit_amqqueue.erl b/deps/rabbit/src/rabbit_amqqueue.erl index 98a27915fb6f..d3a3e59000a8 100644 --- a/deps/rabbit/src/rabbit_amqqueue.erl +++ b/deps/rabbit/src/rabbit_amqqueue.erl @@ -12,7 +12,7 @@ delete_immediately/1, delete_exclusive/2, delete/4, purge/1, forget_all_durable/1]). -export([pseudo_queue/2, pseudo_queue/3, immutable/1]). --export([exists/1, lookup/1, lookup/2, lookup_many/1, +-export([exists/1, lookup/1, lookup/2, lookup_many/1, lookup_durable_queue/1, not_found_or_absent_dirty/1, with/2, with/3, with_or_die/2, assert_equivalence/5, @@ -162,24 +162,28 @@ start(Qs) -> amqqueue:is_classic(Q)], ok. -mark_local_durable_queues_stopped(VHost) -> - Qs0 = find_local_durable_queues(VHost), - Qs = [amqqueue:set_state(Q, stopped) - || Q <- Qs0, amqqueue:get_type(Q) =:= rabbit_classic_queue, - amqqueue:get_state(Q) =/= stopped ], - rabbit_db_queue:insert(Qs). - -find_local_durable_queues(VHost) -> - Qs = rabbit_db_queue:get_all_durable(VHost), - lists:filter(fun(Q) -> - rabbit_queue_type:is_recoverable(Q) - end, Qs). +mark_local_durable_queues_stopped(VHostName) -> + rabbit_db_queue:update_durable( + fun(Q) -> + amqqueue:set_state(Q, stopped) + end, + fun(Q) -> + amqqueue:get_vhost(Q) =:= VHostName andalso + rabbit_queue_type:is_recoverable(Q) andalso + amqqueue:get_type(Q) =:= rabbit_classic_queue andalso + amqqueue:get_state(Q) =/= stopped + end). + +find_local_durable_queues(VHostName) -> + rabbit_db_queue:filter_all_durable(fun(Q) -> + amqqueue:get_vhost(Q) =:= VHostName andalso + rabbit_queue_type:is_recoverable(Q) + end). find_recoverable_queues() -> - Qs = rabbit_db_queue:get_all_durable(), - lists:filter(fun(Q) -> - rabbit_queue_type:is_recoverable(Q) - end, Qs). + rabbit_db_queue:filter_all_durable(fun(Q) -> + rabbit_queue_type:is_recoverable(Q) + end). -spec declare(name(), boolean(), @@ -248,13 +252,12 @@ internal_declare(Q, Recover) -> do_internal_declare(Q0, true) -> Q = amqqueue:set_state(Q0, live), - store_queue(Q), + ok = store_queue(Q), {created, Q0}; do_internal_declare(Q0, false) -> Q = rabbit_policy:set(amqqueue:set_state(Q0, live)), Queue = rabbit_queue_decorator:set(Q), - DurableQueue = amqqueue:reset_mirroring_and_decorators(Q), - rabbit_db_queue:create_or_get(DurableQueue, Queue). + rabbit_db_queue:create_or_get(Queue). -spec update (name(), fun((amqqueue:amqqueue()) -> amqqueue:amqqueue())) -> @@ -272,8 +275,7 @@ ensure_rabbit_queue_record_is_initialized(Q) -> store_queue(Q0) -> Q = rabbit_queue_decorator:set(Q0), - DurableQ = amqqueue:reset_mirroring_and_decorators(Q0), - rabbit_db_queue:insert(DurableQ, Q). + rabbit_db_queue:set(Q). -spec update_decorators(name()) -> 'ok'. @@ -316,14 +318,17 @@ is_server_named_allowed(Args) -> ([name()]) -> [amqqueue:amqqueue()]. -lookup([]) -> []; %% optimisation -lookup(Names) -> - rabbit_db_queue:get(Names). +lookup(Name) when is_record(Name, resource) -> + rabbit_db_queue:get(Name). + +lookup_durable_queue(QName) -> + rabbit_db_queue:get_durable(QName). -spec lookup_many ([name()]) -> [amqqueue:amqqueue()]. +lookup_many([]) -> []; %% optimisation lookup_many(Names) when is_list(Names) -> - lookup(Names). + rabbit_db_queue:get_many(Names). -spec lookup(binary(), binary()) -> rabbit_types:ok(amqqueue:amqqueue()) | @@ -341,7 +346,15 @@ exists(Name) -> -spec not_found_or_absent_dirty(name()) -> not_found_or_absent(). not_found_or_absent_dirty(Name) -> - rabbit_db_queue:not_found_or_absent_queue_dirty(Name). + %% We should read from both tables inside a tx, to get a + %% consistent view. But the chances of an inconsistency are small, + %% and only affect the error kind. + case rabbit_db_queue:get_durable(Name) of + {error, not_found} -> + not_found; + {ok, Q} -> + {absent, Q, nodedown} + end. -spec get_rebalance_lock(pid()) -> {true, {rebalance_queues, pid()}} | false. @@ -542,7 +555,7 @@ with(#resource{} = Name, F, E, RetriesLeft) -> fun () -> retry_wait(Q, F, E, RetriesLeft) end, fun () -> F(Q) end); {error, not_found} -> - E(rabbit_db_queue:not_found_or_absent_queue_dirty(Name)) + E(not_found_or_absent_dirty(Name)) end. -spec retry_wait(amqqueue:amqqueue(), @@ -1239,16 +1252,18 @@ list_down(VHostPath) -> false -> []; true -> Alive = sets:from_list([amqqueue:get_name(Q) || Q <- list(VHostPath)]), - Durable = rabbit_db_queue:get_all_durable(VHostPath), NodesRunning = rabbit_nodes:all_running(), - lists:filter(fun (Q) -> - N = amqqueue:get_name(Q), - Pid = amqqueue:get_pid(Q), - St = amqqueue:get_state(Q), - (St =:= stopped andalso not lists:member(node(Pid), NodesRunning)) - orelse - (not sets:is_element(N, Alive)) - end, Durable) + rabbit_db_queue:filter_all_durable( + fun (Q) -> + N = amqqueue:get_name(Q), + Pid = amqqueue:get_pid(Q), + St = amqqueue:get_state(Q), + amqqueue:get_vhost(Q) =:= VHostPath + andalso + ((St =:= stopped andalso not lists:member(node(Pid), NodesRunning)) + orelse + (not sets:is_element(N, Alive))) + end) end. count(VHost) -> @@ -1671,7 +1686,7 @@ internal_delete(QueueName, ActingUser, Reason) -> ok -> ok; Deletions -> - rabbit_binding:process_deletions(Deletions), + _ = rabbit_binding:process_deletions(Deletions), rabbit_binding:notify_deletions(Deletions, ?INTERNAL_USER), rabbit_core_metrics:queue_deleted(QueueName), ok = rabbit_event:notify(queue_deleted, @@ -1683,12 +1698,12 @@ internal_delete(QueueName, ActingUser, Reason) -> forget_all_durable(Node) -> UpdateFun = fun(Q) -> - forget_node_for_queue(Node, Q) - end, + forget_node_for_queue(Node, Q) + end, FilterFun = fun(Q) -> is_local_to_node(amqqueue:get_pid(Q), Node) end, - rabbit_db_queue:match_and_update(amqqueue:pattern_match_all(), UpdateFun, FilterFun). + rabbit_db_queue:foreach_durable(UpdateFun, FilterFun). %% Try to promote a mirror while down - it should recover as a %% leader. We try to take the oldest mirror here for best chance of @@ -1717,7 +1732,11 @@ 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 = rabbit_db_queue:insert([Q1]); + %% rabbit_db_queue:set_many/1 just stores a durable queue record, + %% that is the only one required here. + %% rabbit_db_queue:set/1 writes both durable and transient, thus + %% can't be used for this operation. + ok = rabbit_db_queue:set_many([Q1]); {true, rabbit_quorum_queue} -> ok end. @@ -1809,43 +1828,45 @@ has_synchronised_mirrors_online(Q) -> -spec on_node_up(node()) -> 'ok'. on_node_up(Node) -> - rabbit_db_queue:on_node_up(Node, fun maybe_clear_recoverable_node/2). - -maybe_clear_recoverable_node(Node, Q) -> - SPids = amqqueue:get_sync_slave_pids(Q), - RSs = amqqueue:get_recoverable_slaves(Q), - case lists:member(Node, RSs) of - true -> - %% There is a race with - %% rabbit_mirror_queue_slave:record_synchronised/1 called - %% by the incoming mirror node and this function, called - %% by the leader node. If this function is executed after - %% record_synchronised/1, the node is erroneously removed - %% from the recoverable mirror list. - %% - %% We check if the mirror node's queue PID is alive. If it is - %% the case, then this function is executed after. In this - %% situation, we don't touch the queue record, it is already - %% correct. - DoClearNode = - case [SP || SP <- SPids, node(SP) =:= Node] of - [SPid] -> not rabbit_misc:is_process_alive(SPid); - _ -> true - end, - if - DoClearNode -> RSs1 = RSs -- [Node], - store_queue( - amqqueue:set_recoverable_slaves(Q, RSs1)); - true -> ok - end; - false -> - ok + rabbit_db_queue:foreach_transient(maybe_clear_recoverable_node(Node)). + +maybe_clear_recoverable_node(Node) -> + fun(Q) -> + SPids = amqqueue:get_sync_slave_pids(Q), + RSs = amqqueue:get_recoverable_slaves(Q), + case lists:member(Node, RSs) of + true -> + %% There is a race with + %% rabbit_mirror_queue_slave:record_synchronised/1 called + %% by the incoming mirror node and this function, called + %% by the leader node. If this function is executed after + %% record_synchronised/1, the node is erroneously removed + %% from the recoverable mirror list. + %% + %% We check if the mirror node's queue PID is alive. If it is + %% the case, then this function is executed after. In this + %% situation, we don't touch the queue record, it is already + %% correct. + DoClearNode = + case [SP || SP <- SPids, node(SP) =:= Node] of + [SPid] -> not rabbit_misc:is_process_alive(SPid); + _ -> true + end, + if + DoClearNode -> RSs1 = RSs -- [Node], + store_queue( + amqqueue:set_recoverable_slaves(Q, RSs1)); + true -> ok + end; + false -> + ok + end end. -spec on_node_down(node()) -> 'ok'. on_node_down(Node) -> - {Time, Ret} = timer:tc(fun() -> rabbit_db_queue:on_node_down(Node, fun filter_transient_queues_to_delete/2) end), + {Time, Ret} = timer:tc(fun() -> rabbit_db_queue:delete_transient(filter_transient_queues_to_delete(Node)) end), case Ret of ok -> ok; {QueueNames, Deletions} -> @@ -1859,12 +1880,14 @@ on_node_down(Node) -> ok end. -filter_transient_queues_to_delete(Node, Q) -> - amqqueue:qnode(Q) == Node andalso - not rabbit_mnesia:is_process_alive(amqqueue:get_pid(Q)) - andalso (not amqqueue:is_classic(Q) orelse not amqqueue:is_durable(Q)) - andalso (not rabbit_amqqueue:is_replicated(Q) - orelse rabbit_amqqueue:is_dead_exclusive(Q)). +filter_transient_queues_to_delete(Node) -> + fun(Q) -> + amqqueue:qnode(Q) == Node andalso + not rabbit_mnesia:is_process_alive(amqqueue:get_pid(Q)) + andalso (not amqqueue:is_classic(Q) orelse not amqqueue:is_durable(Q)) + andalso (not rabbit_amqqueue:is_replicated(Q) + orelse rabbit_amqqueue:is_dead_exclusive(Q)) + end. notify_queue_binding_deletions(QueueDeletions) when is_list(QueueDeletions) -> Deletions = rabbit_binding:process_deletions( diff --git a/deps/rabbit/src/rabbit_amqqueue_process.erl b/deps/rabbit/src/rabbit_amqqueue_process.erl index 091a3001253e..d9f7922995c1 100644 --- a/deps/rabbit/src/rabbit_amqqueue_process.erl +++ b/deps/rabbit/src/rabbit_amqqueue_process.erl @@ -283,7 +283,7 @@ terminate(shutdown = R, State = #q{backing_queue = BQ, q = Q0}) -> rabbit_core_metrics:queue_deleted(qname(State)), terminate_shutdown( fun (BQS) -> - update_state(stopped, Q0), + _ = update_state(stopped, Q0), BQ:terminate(R, BQS) end, State); terminate({shutdown, missing_owner} = Reason, State) -> diff --git a/deps/rabbit/src/rabbit_basic.erl b/deps/rabbit/src/rabbit_basic.erl index 4c67af124970..305049cde608 100644 --- a/deps/rabbit/src/rabbit_basic.erl +++ b/deps/rabbit/src/rabbit_basic.erl @@ -68,7 +68,7 @@ publish(Delivery = #delivery{ end. publish(X, Delivery) -> - Qs = rabbit_amqqueue:lookup(rabbit_exchange:route(X, Delivery)), + Qs = rabbit_amqqueue:lookup_many(rabbit_exchange:route(X, Delivery)), _ = rabbit_queue_type:deliver(Qs, Delivery, stateless), ok. diff --git a/deps/rabbit/src/rabbit_binding.erl b/deps/rabbit/src/rabbit_binding.erl index 52bdce4ebe2f..b418f0a51bfd 100644 --- a/deps/rabbit/src/rabbit_binding.erl +++ b/deps/rabbit/src/rabbit_binding.erl @@ -9,7 +9,7 @@ -include_lib("rabbit_common/include/rabbit.hrl"). -include("amqqueue.hrl"). --export([recover/0, recover/2, exists/1, add/2, add/3, remove/3]). +-export([recover/0, recover/2, exists/1, add/2, add/3, remove/2, remove/3]). -export([list/1, list_for_source/1, list_for_destination/1, list_for_source_and_destination/2, list_for_source_and_destination/3, list_explicit/0]). @@ -150,24 +150,27 @@ binding_type0(false, true) -> binding_type0(_, _) -> transient. +-spec remove(rabbit_types:binding(), rabbit_types:username()) -> bind_res(). +remove(Binding, ActingUser) -> remove(Binding, fun (_Src, _Dst) -> ok end, ActingUser). + -spec remove(rabbit_types:binding(), inner_fun(), rabbit_types:username()) -> bind_res(). remove(Binding0, InnerFun, ActingUser) -> Binding = sort_args(Binding0), case rabbit_db_binding:delete(Binding, InnerFun) of - ok -> - ok; {error, _} = Err -> Err; - Deletions -> + ok -> + ok; + {ok, Deletions} -> notify_deletions(Deletions, ActingUser) end. -spec list_explicit() -> bindings(). list_explicit() -> - rabbit_db_binding:get_all_explicit(). + rabbit_db_binding:get_all(). -spec list(rabbit_types:vhost()) -> bindings(). @@ -295,6 +298,15 @@ group_bindings_fold(Fun, Acc, [B = #binding{source = SrcName} | Bs], OnlyDurable) -> group_bindings_fold(Fun, SrcName, Acc, Bs, [B], OnlyDurable). +-spec group_bindings_fold(Fun, Name, Deletions, [Binding], [Binding], OnlyDurable) + -> Ret when + Fun :: fun((Name, [Binding], Deletions, OnlyDurable) -> + Deletions), + Name :: rabbit_exchange:name(), + Deletions :: rabbit_binding:deletions(), + Binding :: rabbit_types:binding(), + OnlyDurable :: boolean(), + Ret :: Deletions. group_bindings_fold( Fun, SrcName, Acc, [B = #binding{source = SrcName} | Bs], Bindings, OnlyDurable) -> @@ -399,7 +411,7 @@ notify_bindings_deletion(Bs, ActingUser) -> || B <- Bs], ok. --spec process_deletions(deletions()) -> rabbit_misc:thunk('ok'). +-spec process_deletions(deletions()) -> deletions(). process_deletions(Deletions) -> dict:map(fun (_XName, {X, deleted, Bindings}) -> Bs = lists:flatten(Bindings), diff --git a/deps/rabbit/src/rabbit_channel.erl b/deps/rabbit/src/rabbit_channel.erl index 2d5ccdcad422..70bce8cab0f9 100644 --- a/deps/rabbit/src/rabbit_channel.erl +++ b/deps/rabbit/src/rabbit_channel.erl @@ -2154,7 +2154,7 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ex confirm = Confirm, msg_seq_no = MsgSeqNo}, RoutedToQueueNames = [QName]}, State0 = #ch{queue_states = QueueStates0}) -> %% optimisation when there is one queue - Qs0 = rabbit_amqqueue:lookup(RoutedToQueueNames), + Qs0 = rabbit_amqqueue:lookup_many(RoutedToQueueNames), Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0), QueueNames = lists:map(fun amqqueue:get_name/1, Qs), case rabbit_queue_type:deliver(Qs, Delivery, QueueStates0) of @@ -2191,7 +2191,7 @@ deliver_to_queues({Delivery = #delivery{message = Message = #basic_message{ex confirm = Confirm, msg_seq_no = MsgSeqNo}, RoutedToQueueNames}, State0 = #ch{queue_states = QueueStates0}) -> - Qs0 = rabbit_amqqueue:lookup(RoutedToQueueNames), + Qs0 = rabbit_amqqueue:lookup_many(RoutedToQueueNames), Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0), QueueNames = lists:map(fun amqqueue:get_name/1, Qs), case rabbit_queue_type:deliver(Qs, Delivery, QueueStates0) of diff --git a/deps/rabbit/src/rabbit_classic_queue.erl b/deps/rabbit/src/rabbit_classic_queue.erl index 153889ce4646..940b122a4f36 100644 --- a/deps/rabbit/src/rabbit_classic_queue.erl +++ b/deps/rabbit/src/rabbit_classic_queue.erl @@ -127,7 +127,7 @@ is_recoverable(Q) when ?is_amqqueue(Q) -> %% 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. - (rabbit_amqqueue:exists(Q) + (not rabbit_db_queue:consistent_exists(amqqueue:get_name(Q)) orelse not rabbit_mnesia:is_process_alive(amqqueue:get_pid(Q))). recover(VHost, Queues) -> diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl index 3ccf55336a91..1a816d03a585 100644 --- a/deps/rabbit/src/rabbit_core_ff.erl +++ b/deps/rabbit/src/rabbit_core_ff.erl @@ -134,6 +134,7 @@ Ret :: rabbit_feature_flags:enable_callback_ret(). direct_exchange_routing_v2_enable(#{feature_name := FeatureName}) -> TableName = rabbit_index_route, + ok = rabbit_table:wait([rabbit_route, rabbit_exchange], _Retry = true), try case rabbit_db_binding:create_index_route_table() of ok -> @@ -142,7 +143,7 @@ direct_exchange_routing_v2_enable(#{feature_name := FeatureName}) -> ?LOG_ERROR( "Feature flags: `~ts`: failed to add copy of table ~ts to " "node ~tp: ~tp", - [FeatureName, NewTable, node(), Err], + [FeatureName, TableName, node(), Err], #{domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}), Error end diff --git a/deps/rabbit/src/rabbit_db_binding.erl b/deps/rabbit/src/rabbit_db_binding.erl index 90526dbc910f..b653cc834a5d 100644 --- a/deps/rabbit/src/rabbit_db_binding.erl +++ b/deps/rabbit/src/rabbit_db_binding.erl @@ -9,8 +9,14 @@ -include_lib("rabbit_common/include/rabbit.hrl"). --export([exists/1, create/2, delete/2, get_all/1, get_all_for_source/1, - get_all_for_destination/1, get_all/3, get_all_explicit/0, +-export([exists/1, + create/2, + delete/2, + get_all/0, + get_all/1, + get_all/3, + get_all_for_source/1, + get_all_for_destination/1, fold/2]). %% Routing. These functions are in the hot code path @@ -28,6 +34,15 @@ -export([create_index_route_table/0]). +%% For testing +-export([clear/0]). + +-define(MNESIA_TABLE, rabbit_route). +-define(MNESIA_DURABLE_TABLE, rabbit_durable_route). +-define(MNESIA_SEMI_DURABLE_TABLE, rabbit_semi_durable_route). +-define(MNESIA_REVERSE_TABLE, rabbit_reverse_route). +-define(MNESIA_INDEX_TABLE, rabbit_index_route). + %% ------------------------------------------------------------------- %% exists(). %% ------------------------------------------------------------------- @@ -49,22 +64,43 @@ exists(Binding) -> exists_in_mnesia(Binding) -> binding_action_in_mnesia( Binding, fun (_Src, _Dst) -> - rabbit_misc:const(mnesia:read({rabbit_route, Binding}) /= []) + rabbit_misc:const(mnesia:read({?MNESIA_TABLE, Binding}) /= []) end, fun not_found_or_absent_errs_in_mnesia/1). +binding_action_in_mnesia(#binding{source = SrcName, + destination = DstName}, Fun, ErrFun) -> + SrcTable = table_for_resource(SrcName), + DstTable = table_for_resource(DstName), + rabbit_mnesia:execute_mnesia_tx_with_tail( + fun () -> + case {mnesia:read({SrcTable, SrcName}), + mnesia:read({DstTable, DstName})} of + {[Src], [Dst]} -> Fun(Src, Dst); + {[], [_] } -> ErrFun([SrcName]); + {[_], [] } -> ErrFun([DstName]); + {[], [] } -> ErrFun([SrcName, DstName]) + end + end). + +table_for_resource(#resource{kind = exchange}) -> rabbit_exchange; +table_for_resource(#resource{kind = queue}) -> rabbit_queue. + +not_found_or_absent_errs_in_mnesia(Names) -> + Errs = [not_found_or_absent_in_mnesia(Name) || Name <- Names], + rabbit_misc:const({error, {resources_missing, Errs}}). + %% ------------------------------------------------------------------- %% create(). %% ------------------------------------------------------------------- -spec create(Binding, ChecksFun) -> Ret when Binding :: rabbit_types:binding(), - Src :: rabbit_types:r('exchange'), - Dst :: rabbit_types:r('exchange') | rabbit_types:r('queue'), - BindingType :: durable | semi_durable | transient, - ChecksFun :: fun((Src, Dst) -> {ok, BindingType} | {error, Reason :: any()}), + Src :: rabbit_types:binding_source(), + Dst :: rabbit_types:binding_destination(), + ChecksFun :: fun((Src, Dst) -> ok | {error, Reason :: any()}), Ret :: ok | {error, Reason :: any()}. %% @doc Writes a binding if it doesn't exist already and passes the validation in -%% `ChecksFun` i.e. exclusive access +%% `ChecksFun' i.e. exclusive access %% %% @returns ok, or an error if the validation has failed. %% @@ -75,19 +111,43 @@ create(Binding, ChecksFun) -> #{mnesia => fun() -> create_in_mnesia(Binding, ChecksFun) end }). +create_in_mnesia(Binding, ChecksFun) -> + binding_action_in_mnesia( + Binding, + fun (Src, Dst) -> + lock_resource(Src, read), + lock_resource(Dst, read), + case ChecksFun(Src, Dst) of + ok -> + BindingType = rabbit_binding:binding_type(Src, Dst), + case mnesia:read({?MNESIA_TABLE, Binding}) of + [] -> + ok = sync_route(#route{binding = Binding}, BindingType, + should_index_table(Src), fun mnesia:write/3), + MaybeSerial = rabbit_exchange:serialise_events(Src), + Serial = serial_in_mnesia(MaybeSerial, Src), + fun () -> + rabbit_exchange:callback(Src, add_binding, Serial, [Src, Binding]) + end; + [_] -> fun () -> ok end + end; + {error, _} = Err -> + rabbit_misc:const(Err) + end + end, fun not_found_or_absent_errs_in_mnesia/1). + %% ------------------------------------------------------------------- %% delete(). %% ------------------------------------------------------------------- -spec delete(Binding, ChecksFun) -> Ret when Binding :: rabbit_types:binding(), - Src :: rabbit_types:r('exchange'), - Dst :: rabbit_types:r('exchange') | rabbit_types:r('queue'), - BindingType :: durable | semi_durable | transient, - ChecksFun :: fun((Src, Dst) -> {ok, BindingType} | {error, Reason :: any()}), - Ret :: ok | {error, Reason :: any()}. + Src :: rabbit_types:binding_source(), + Dst :: rabbit_types:binding_destination(), + ChecksFun :: fun((Src, Dst) -> ok | {error, Reason :: any()}), + Ret :: ok | {ok, rabbit_binding:deletions()} | {error, Reason :: any()}. %% @doc Deletes a binding record from the database if it passes the validation in -%% `ChecksFun`. It also triggers the deletion of auto-delete exchanges if needed. +%% `ChecksFun'. It also triggers the deletion of auto-delete exchanges if needed. %% %% @private @@ -96,10 +156,82 @@ delete(Binding, ChecksFun) -> #{mnesia => fun() -> delete_in_mnesia(Binding, ChecksFun) end }). +delete_in_mnesia(Binding, ChecksFun) -> + binding_action_in_mnesia( + Binding, + fun (Src, Dst) -> + lock_resource(Src, read), + lock_resource(Dst, read), + case mnesia:read(?MNESIA_TABLE, Binding, write) of + [] -> case mnesia:read(?MNESIA_DURABLE_TABLE, Binding, write) of + [] -> rabbit_misc:const(ok); + %% We still delete the binding and run + %% all post-delete functions if there is only + %% a durable route in the database + _ -> delete_in_mnesia(Src, Dst, Binding) + end; + _ -> case ChecksFun(Src, Dst) of + ok -> delete_in_mnesia(Src, Dst, Binding); + {error, _} = Err -> rabbit_misc:const(Err) + end + end + end, fun absent_errs_only_in_mnesia/1). + +-spec delete_in_mnesia(Src, Dst, Binding) -> Ret when + Src :: rabbit_types:exchange() | amqqueue:amqqueue(), + Dst :: rabbit_types:exchange() | amqqueue:amqqueue(), + Binding :: rabbit_types:binding(), + Ret :: fun(() -> rabbit_binding:deletions()). +delete_in_mnesia(Src, Dst, B) -> + ok = sync_route(#route{binding = B}, rabbit_binding:binding_type(Src, Dst), + should_index_table(Src), fun delete/3), + Deletions0 = maybe_auto_delete_exchange_in_mnesia( + B#binding.source, [B], rabbit_binding:new_deletions(), false), + fun() -> {ok, rabbit_binding:process_deletions(Deletions0)} end. + +absent_errs_only_in_mnesia(Names) -> + Errs = [E || Name <- Names, + {absent, _Q, _Reason} = E <- [not_found_or_absent_in_mnesia(Name)]], + rabbit_misc:const(case Errs of + [] -> ok; + _ -> {error, {resources_missing, Errs}} + end). + +not_found_or_absent_in_mnesia(#resource{kind = exchange} = Name) -> + {not_found, Name}; +not_found_or_absent_in_mnesia(#resource{kind = queue} = Name) -> + %% NB: we assume that the caller has already performed a lookup on + %% rabbit_queue and not found anything + case rabbit_db_queue:get_durable_in_mnesia_tx(Name) of + {error, not_found} -> {not_found, Name}; + {ok, Q} -> {absent, Q, nodedown} + end. + %% ------------------------------------------------------------------- %% get_all(). %% ------------------------------------------------------------------- +-spec get_all() -> [Binding] when + Binding :: rabbit_types:binding(). +%% @doc Returns all explicit binding records, the bindings explicitly added and not +%% automatically generated to the default exchange. +%% +%% @returns the list of binding records. +%% +%% @private + +get_all() -> + rabbit_db:run( + #{mnesia => fun() -> get_all_in_mnesia() end + }). + +get_all_in_mnesia() -> + mnesia:async_dirty( + fun () -> + AllRoutes = mnesia:dirty_match_object(?MNESIA_TABLE, #route{_ = '_'}), + [B || #route{binding = B} <- AllRoutes] + end). + -spec get_all(VHostName) -> [Binding] when VHostName :: vhost:name(), Binding :: rabbit_types:binding(). @@ -120,10 +252,38 @@ get_all_in_mnesia(VHost) -> destination = VHostResource, _ = '_'}, _ = '_'}, - [B || #route{binding = B} <- rabbit_db:list_in_mnesia(rabbit_route, Match)]. + [B || #route{binding = B} <- rabbit_db:list_in_mnesia(?MNESIA_TABLE, Match)]. + +-spec get_all(Src, Dst, Reverse) -> [Binding] when + Src :: rabbit_types:binding_source(), + Dst :: rabbit_types:binding_destination(), + Reverse :: boolean(), + Binding :: rabbit_types:binding(). +%% @doc Returns all binding records for a given source and destination +%% in the given virtual host. +%% +%% @returns the list of binding records. +%% +%% @private + +get_all(SrcName, DstName, Reverse) -> + rabbit_db:run( + #{mnesia => fun() -> get_all_in_mnesia(SrcName, DstName, Reverse) end + }). + +get_all_in_mnesia(SrcName, DstName, Reverse) -> + Route = #route{binding = #binding{source = SrcName, + destination = DstName, + _ = '_'}}, + Fun = list_for_route(Route, Reverse), + mnesia:async_dirty(Fun). + +%% ------------------------------------------------------------------- +%% get_all_for_source(). +%% ------------------------------------------------------------------- -spec get_all_for_source(Src) -> [Binding] when - Src :: rabbit_types:r('exchange'), + Src :: rabbit_types:binding_source(), Binding :: rabbit_types:binding(). %% @doc Returns all binding records for a given exchange in the given virtual host. %% @@ -143,18 +303,22 @@ get_all_for_source_in_mnesia(Resource) -> list_for_route(Route, false) -> fun() -> - [B || #route{binding = B} <- mnesia:match_object(rabbit_route, Route, read)] + [B || #route{binding = B} <- mnesia:match_object(?MNESIA_TABLE, Route, read)] end; list_for_route(Route, true) -> fun() -> [rabbit_binding:reverse_binding(B) || #reverse_route{reverse_binding = B} <- - mnesia:match_object(rabbit_reverse_route, + mnesia:match_object(?MNESIA_REVERSE_TABLE, rabbit_binding:reverse_route(Route), read)] end. +%% ------------------------------------------------------------------- +%% get_all_for_destination(). +%% ------------------------------------------------------------------- + -spec get_all_for_destination(Dst) -> [Binding] when - Dst :: rabbit_types:r('exchange') | rabbit_types:r('queue'), + Dst :: rabbit_types:binding_destination(), Binding :: rabbit_types:binding(). %% @doc Returns all binding records for a given exchange or queue destination %% in the given virtual host. @@ -174,50 +338,9 @@ get_all_for_destination_in_mnesia(Dst) -> Fun = list_for_route(Route, true), mnesia:async_dirty(Fun). --spec get_all(Src, Dst, Reverse) -> [Binding] when - Src :: rabbit_types:r('exchange'), - Dst :: rabbit_types:r('exchange') | rabbit_types:r('queue'), - Reverse :: boolean(), - Binding :: rabbit_types:binding(). -%% @doc Returns all binding records for a given source and destination -%% in the given virtual host. -%% -%% @returns the list of binding records. -%% -%% @private - -get_all(SrcName, DstName, Reverse) -> - rabbit_db:run( - #{mnesia => fun() -> get_all_in_mnesia(SrcName, DstName, Reverse) end - }). - -get_all_in_mnesia(SrcName, DstName, Reverse) -> - Route = #route{binding = #binding{source = SrcName, - destination = DstName, - _ = '_'}}, - Fun = list_for_route(Route, Reverse), - mnesia:async_dirty(Fun). - --spec get_all_explicit() -> [Binding] when - Binding :: rabbit_types:binding(). -%% @doc Returns all explicit binding records, the bindings explicitly added and not -%% automatically generated to the default exchange. -%% -%% @returns the list of binding records. -%% -%% @private - -get_all_explicit() -> - rabbit_db:run( - #{mnesia => fun() -> get_all_explicit_in_mnesia() end - }). - -get_all_explicit_in_mnesia() -> - mnesia:async_dirty( - fun () -> - AllRoutes = mnesia:dirty_match_object(rabbit_route, #route{_ = '_'}), - [B || #route{binding = B} <- AllRoutes] - end). +%% ------------------------------------------------------------------- +%% fold(). +%% ------------------------------------------------------------------- -spec fold(Fun, Acc) -> Acc when Fun :: fun((Binding :: rabbit_types:binding(), Acc) -> Acc), @@ -238,44 +361,24 @@ fold(Fun, Acc) -> fold_in_mnesia(Fun, Acc) -> ets:foldl(fun(#route{binding = Binding}, Acc0) -> Fun(Binding, Acc0) - end, Acc, rabbit_route). - -recover() -> - rabbit_db:run( - #{mnesia => fun() -> recover_in_mnesia() end - }). - -recover(RecoverFun) -> - rabbit_db:run( - #{mnesia => fun() -> recover_in_mnesia(RecoverFun) end - }). - -recover_in_mnesia(RecoverFun) -> - [RecoverFun(Route, Src, Dst, fun recover_semi_durable_route/2) || - #route{binding = #binding{destination = Dst, - source = Src}} = Route <- - rabbit_mnesia:dirty_read_all(rabbit_semi_durable_route)]. - -create_index_route_table() -> - rabbit_db:run( - #{mnesia => fun() -> create_index_route_table_in_mnesia() end - }). - -create_index_route_table_in_mnesia() -> - TableName = rabbit_index_route, - DependantTables = [rabbit_route, rabbit_exchange], - ok = rabbit_table:wait(DependantTables, _Retry = true), - [ok = rabbit_table:create_local_copy(Tab, ram_copies) || Tab <- DependantTables], - ok = rabbit_table:create( - TableName, rabbit_table:rabbit_index_route_definition()), - case rabbit_table:ensure_table_copy(TableName, node(), ram_copies) of - ok -> - ok = populate_index_route_table_in_mnesia(); - Error -> - Error - end. + end, Acc, ?MNESIA_TABLE). %% Routing - HOT CODE PATH +%% ------------------------------------------------------------------- +%% match(). +%% ------------------------------------------------------------------- + +-spec match(Src, MatchFun) -> [Dst] when + Src :: rabbit_types:binding_source(), + Dst :: rabbit_types:binding_destination(), + Binding :: rabbit_types:binding(), + MatchFun :: fun((Binding) -> boolean()). +%% @doc Matches all binding records that have `Src' as source of the binding +%% and for which `MatchFun' returns `true'. +%% +%% @returns the list of destinations +%% +%% @private match(SrcName, Match) -> rabbit_db:run( @@ -285,10 +388,28 @@ match(SrcName, Match) -> match_in_mnesia(SrcName, Match) -> MatchHead = #route{binding = #binding{source = SrcName, _ = '_'}}, - Routes = ets:select(rabbit_route, [{MatchHead, [], [['$_']]}]), + Routes = ets:select(?MNESIA_TABLE, [{MatchHead, [], [['$_']]}]), [Dest || [#route{binding = Binding = #binding{destination = Dest}}] <- Routes, Match(Binding)]. + +%% Routing - HOT CODE PATH +%% ------------------------------------------------------------------- +%% match_routing_key(). +%% ------------------------------------------------------------------- + +-spec match_routing_key(Src, RoutingKeys, UseIndex) -> [Dst] when + Src :: rabbit_types:binding_source(), + Dst :: rabbit_types:binding_destination(), + RoutingKeys :: [binary() | '_'], + UseIndex :: boolean(). +%% @doc Matches all binding records that have `Src' as source of the binding +%% and that match any routing key in `RoutingKeys'. +%% +%% @returns the list of destinations +%% +%% @private + match_routing_key(SrcName, RoutingKeys, UseIndex) -> rabbit_db:run( #{mnesia => fun() -> match_routing_key_in_mnesia(SrcName, RoutingKeys, UseIndex) end @@ -297,91 +418,96 @@ match_routing_key(SrcName, RoutingKeys, UseIndex) -> match_routing_key_in_mnesia(SrcName, RoutingKeys, UseIndex) -> case UseIndex of true -> - route_v2(rabbit_index_route, SrcName, RoutingKeys); + route_v2(?MNESIA_INDEX_TABLE, SrcName, RoutingKeys); _ -> route_in_mnesia_v1(SrcName, RoutingKeys) end. -delete_all_for_exchange_in_mnesia(X = #exchange{name = XName}, OnlyDurable, RemoveBindingsForSource) -> - Bindings = case RemoveBindingsForSource of - true -> delete_for_source_in_mnesia(XName); - false -> [] - end, - {deleted, X, Bindings, delete_for_destination_in_mnesia(XName, OnlyDurable, fun delete_routes/1)}. +%% ------------------------------------------------------------------- +%% recover(). +%% ------------------------------------------------------------------- -delete_for_destination_in_mnesia(DstName, OnlyDurable) -> - delete_for_destination_in_mnesia(DstName, OnlyDurable, fun delete_routes/1). +-spec recover() -> ok. +%% @doc Recovers all durable routes +%% +%% @private --spec delete_transient_for_destination_in_mnesia(rabbit_types:binding_destination()) -> rabbit_binding:deletions(). -delete_transient_for_destination_in_mnesia(DstName) -> - delete_for_destination_in_mnesia(DstName, false, fun delete_transient_routes/1). +recover() -> + rabbit_db:run( + #{mnesia => fun() -> recover_in_mnesia() end + }). --spec has_for_source_in_mnesia(rabbit_types:binding_source()) -> boolean(). +recover_in_mnesia() -> + rabbit_mnesia:execute_mnesia_transaction( + fun () -> + _ = mnesia:lock({table, ?MNESIA_DURABLE_TABLE}, read), + _ = mnesia:lock({table, ?MNESIA_SEMI_DURABLE_TABLE}, write), + Routes = rabbit_mnesia:dirty_read_all(?MNESIA_DURABLE_TABLE), + Fun = fun(Route) -> + mnesia:dirty_write(?MNESIA_SEMI_DURABLE_TABLE, Route) + end, + lists:foreach(Fun, Routes), + ok + end). -has_for_source_in_mnesia(SrcName) -> - Match = #route{binding = #binding{source = SrcName, _ = '_'}}, - %% we need to check for semi-durable routes (which subsumes - %% durable routes) here too in case a bunch of routes to durable - %% queues have been removed temporarily as a result of a node - %% failure - contains(rabbit_route, Match) orelse - contains(rabbit_semi_durable_route, Match). +-spec recover(RecoverFun) -> ok when + Route :: #route{}, + Src :: rabbit_types:binding_source(), + Dst :: rabbit_types:binding_destination(), + Binding :: rabbit_types:binding(), + Exchange :: rabbit_types:exchange(), + RecoverFun :: fun((Route, Src, Dst, fun((Binding, Exchange) -> ok)) -> ok). +%% @doc Recovers all semi-durable routes +%% +%% @private -%% Internal -%% -------------------------------------------------------------- -binding_action_in_mnesia(#binding{source = SrcName, - destination = DstName}, Fun, ErrFun) -> - SrcTable = table_for_resource(SrcName), - DstTable = table_for_resource(DstName), - rabbit_mnesia:execute_mnesia_tx_with_tail( - fun () -> - case {mnesia:read({SrcTable, SrcName}), - mnesia:read({DstTable, DstName})} of - {[Src], [Dst]} -> Fun(Src, Dst); - {[], [_] } -> ErrFun([SrcName]); - {[_], [] } -> ErrFun([DstName]); - {[], [] } -> ErrFun([SrcName, DstName]) - end - end). +recover(RecoverFun) -> + rabbit_db:run( + #{mnesia => fun() -> recover_in_mnesia(RecoverFun) end + }). -table_for_resource(#resource{kind = exchange}) -> rabbit_exchange; -table_for_resource(#resource{kind = queue}) -> rabbit_queue. +recover_in_mnesia(RecoverFun) -> + [RecoverFun(Route, Src, Dst, fun recover_semi_durable_route/2) || + #route{binding = #binding{destination = Dst, + source = Src}} = Route <- + rabbit_mnesia:dirty_read_all(?MNESIA_SEMI_DURABLE_TABLE)]. -create_in_mnesia(Binding, ChecksFun) -> - binding_action_in_mnesia( - Binding, - fun (Src, Dst) -> - case ChecksFun(Src, Dst) of - ok -> - BindingType = rabbit_binding:binding_type(Src, Dst), - case mnesia:read({rabbit_route, Binding}) of - [] -> - ok = sync_route(#route{binding = Binding}, BindingType, - should_index_table(Src), fun mnesia:write/3), - MaybeSerial = rabbit_exchange:serialise_events(Src), - Serial = serial_in_mnesia(MaybeSerial, Src), - fun () -> - rabbit_exchange:callback(Src, add_binding, Serial, [Src, Binding]) - end; - [_] -> fun () -> ok end - end; - {error, _} = Err -> - rabbit_misc:const(Err) - end - end, fun not_found_or_absent_errs_in_mnesia/1). +%% ------------------------------------------------------------------- +%% create_index_route_table(). +%% ------------------------------------------------------------------- + +-spec create_index_route_table() -> ok | {error, any()}. +create_index_route_table() -> + rabbit_db:run( + #{mnesia => fun() -> create_index_route_table_in_mnesia() end + }). + +create_index_route_table_in_mnesia() -> + DependantTables = [?MNESIA_TABLE, rabbit_exchange], + ok = rabbit_table:wait(DependantTables, _Retry = true), + [ok = rabbit_table:create_local_copy(Tab, ram_copies) || Tab <- DependantTables], + ok = rabbit_table:create( + ?MNESIA_INDEX_TABLE, rabbit_table:rabbit_index_route_definition()), + case rabbit_table:ensure_table_copy(?MNESIA_INDEX_TABLE, node(), ram_copies) of + ok -> + ok = populate_index_route_table_in_mnesia(); + Error -> + Error + end. populate_index_route_table_in_mnesia() -> rabbit_mnesia:execute_mnesia_transaction( fun () -> - _ = mnesia:lock({table, rabbit_route}, read), - _ = mnesia:lock({table, rabbit_index_route}, write), - Routes = rabbit_mnesia:dirty_read_all(rabbit_route), + _ = mnesia:lock({table, ?MNESIA_TABLE}, read), + _ = mnesia:lock({table, rabbit_exchange}, read), + _ = mnesia:lock({table, ?MNESIA_INDEX_TABLE}, write), + Routes = rabbit_mnesia:dirty_read_all(?MNESIA_TABLE), lists:foreach(fun(#route{binding = #binding{source = Exchange}} = Route) -> case rabbit_db_exchange:get(Exchange) of {ok, X} -> case should_index_table(X) of true -> - mnesia:dirty_write(rabbit_index_route, + mnesia:dirty_write(?MNESIA_INDEX_TABLE, rabbit_binding:index_route(Route)); false -> ok @@ -392,33 +518,52 @@ populate_index_route_table_in_mnesia() -> end, Routes) end). -delete_in_mnesia(Binding, ChecksFun) -> - binding_action_in_mnesia( - Binding, - fun (Src, Dst) -> - lock_resource(Src, read), - lock_resource(Dst, read), - case mnesia:read(rabbit_route, Binding, write) of - [] -> case mnesia:read(rabbit_durable_route, Binding, write) of - [] -> rabbit_misc:const(ok); - %% We still delete the binding and run - %% all post-delete functions if there is only - %% a durable route in the database - _ -> delete_in_mnesia(Src, Dst, Binding) - end; - _ -> case ChecksFun(Src, Dst) of - ok -> delete_in_mnesia(Src, Dst, Binding); - {error, _} = Err -> rabbit_misc:const(Err) - end - end - end, fun absent_errs_only_in_mnesia/1). +%% ------------------------------------------------------------------- +%% delete_all_for_exchange_in_mnesia(). +%% ------------------------------------------------------------------- -delete_in_mnesia(Src, Dst, B) -> - ok = sync_route(#route{binding = B}, rabbit_binding:binding_type(Src, Dst), - should_index_table(Src), fun delete/3), - Deletions0 = maybe_auto_delete_exchange_in_mnesia( - B#binding.source, [B], rabbit_binding:new_deletions(), false), - fun() -> rabbit_binding:process_deletions(Deletions0) end. +-spec delete_all_for_exchange_in_mnesia(Exchange, OnlyDurable, RemoveBindingsForSource) + -> Ret when + Exchange :: rabbit_types:exchange(), + OnlyDurable :: boolean(), + RemoveBindingsForSource :: boolean(), + Binding :: rabbit_types:binding(), + Ret :: {deleted, Exchange, [Binding], rabbit_binding:deletions()}. + +delete_all_for_exchange_in_mnesia(X = #exchange{name = XName}, OnlyDurable, RemoveBindingsForSource) -> + Bindings = case RemoveBindingsForSource of + true -> delete_for_source_in_mnesia(XName); + false -> [] + end, + {deleted, X, Bindings, delete_for_destination_in_mnesia(XName, OnlyDurable, fun delete_routes/1)}. + +delete_for_source_in_mnesia(#exchange{name = SrcName} = SrcX) -> + delete_for_source_in_mnesia(SrcName, should_index_table(SrcX)); +delete_for_source_in_mnesia(SrcName) -> + delete_for_source_in_mnesia(SrcName, undefined). + +-spec delete_for_source_in_mnesia(rabbit_types:binding_source(), + boolean() | undefined) -> [rabbit_types:binding()]. +delete_for_source_in_mnesia(SrcName, ShouldIndexTable) -> + lock_resource(SrcName), + Match = #route{binding = #binding{source = SrcName, _ = '_'}}, + delete_routes( + lists:usort( + mnesia:dirty_match_object(?MNESIA_TABLE, Match) ++ + mnesia:dirty_match_object(?MNESIA_SEMI_DURABLE_TABLE, Match)), + ShouldIndexTable). + +%% ------------------------------------------------------------------- +%% delete_for_destination_in_mnesia(). +%% ------------------------------------------------------------------- + +-spec delete_for_destination_in_mnesia(Dst, OnlyDurable) -> Deletions when + Dst :: rabbit_types:binding_destination(), + OnlyDurable :: boolean(), + Deletions :: rabbit_binding:deletions(). + +delete_for_destination_in_mnesia(DstName, OnlyDurable) -> + delete_for_destination_in_mnesia(DstName, OnlyDurable, fun delete_routes/1). delete_for_destination_in_mnesia(DstName, OnlyDurable, Fun) -> lock_resource(DstName), @@ -428,33 +573,71 @@ delete_for_destination_in_mnesia(DstName, OnlyDurable, Fun) -> false -> [rabbit_binding:reverse_route(R) || R <- mnesia:dirty_match_object( - rabbit_reverse_route, MatchRev)]; + ?MNESIA_REVERSE_TABLE, MatchRev)]; true -> lists:usort( mnesia:dirty_match_object( - rabbit_durable_route, MatchFwd) ++ + ?MNESIA_DURABLE_TABLE, MatchFwd) ++ mnesia:dirty_match_object( - rabbit_semi_durable_route, MatchFwd)) + ?MNESIA_SEMI_DURABLE_TABLE, MatchFwd)) end, Bindings = Fun(Routes), rabbit_binding:group_bindings_fold(fun maybe_auto_delete_exchange_in_mnesia/4, lists:keysort(#binding.source, Bindings), OnlyDurable). -delete_for_source_in_mnesia(#exchange{name = SrcName} = SrcX) -> - delete_for_source_in_mnesia(SrcName, should_index_table(SrcX)); -delete_for_source_in_mnesia(SrcName) -> - delete_for_source_in_mnesia(SrcName, undefined). +%% ------------------------------------------------------------------- +%% delete_transient_for_destination_in_mnesia(). +%% ------------------------------------------------------------------- --spec delete_for_source_in_mnesia(rabbit_types:binding_source(), - boolean() | undefined) -> [rabbit_types:binding()]. -delete_for_source_in_mnesia(SrcName, ShouldIndexTable) -> - lock_resource(SrcName), +-spec delete_transient_for_destination_in_mnesia(rabbit_types:binding_destination()) -> rabbit_binding:deletions(). +delete_transient_for_destination_in_mnesia(DstName) -> + delete_for_destination_in_mnesia(DstName, false, fun delete_transient_routes/1). + +delete_transient_routes(Routes) -> + lists:map(fun(#route{binding = #binding{source = Src} = Binding} = Route) -> + {ok, X} = rabbit_db_exchange:get(Src), + ok = sync_transient_route(Route, should_index_table(X), fun delete/3), + Binding + end, Routes). + +%% ------------------------------------------------------------------- +%% has_for_source_in_mnesia(). +%% ------------------------------------------------------------------- + +-spec has_for_source_in_mnesia(rabbit_types:binding_source()) -> boolean(). + +has_for_source_in_mnesia(SrcName) -> Match = #route{binding = #binding{source = SrcName, _ = '_'}}, - delete_routes( - lists:usort( - mnesia:dirty_match_object(rabbit_route, Match) ++ - mnesia:dirty_match_object(rabbit_semi_durable_route, Match)), - ShouldIndexTable). + %% we need to check for semi-durable routes (which subsumes + %% durable routes) here too in case a bunch of routes to durable + %% queues have been removed temporarily as a result of a node + %% failure + contains(?MNESIA_TABLE, Match) orelse + contains(?MNESIA_SEMI_DURABLE_TABLE, Match). + +%% ------------------------------------------------------------------- +%% clear(). +%% ------------------------------------------------------------------- + +-spec clear() -> ok. +%% @doc Deletes all bindings. +%% +%% @private +clear() -> + rabbit_db:run( + #{mnesia => fun() -> clear_in_mnesia() end}). + +clear_in_mnesia() -> + {atomic, ok} = mnesia:clear_table(?MNESIA_TABLE), + {atomic, ok} = mnesia:clear_table(?MNESIA_DURABLE_TABLE), + {atomic, ok} = mnesia:clear_table(?MNESIA_SEMI_DURABLE_TABLE), + {atomic, ok} = mnesia:clear_table(?MNESIA_REVERSE_TABLE), + {atomic, ok} = mnesia:clear_table(?MNESIA_INDEX_TABLE), + ok. + +%% -------------------------------------------------------------- +%% Internal +%% -------------------------------------------------------------- delete_routes(Routes) -> delete_routes(Routes, undefined). @@ -463,11 +646,11 @@ delete_routes(Routes, ShouldIndexTable) -> %% operations on disk tables, which require an fsync. {RamRoutes, DiskRoutes} = lists:partition(fun (R) -> mnesia:read( - rabbit_durable_route, R#route.binding, read) == [] end, + ?MNESIA_DURABLE_TABLE, R#route.binding, read) == [] end, Routes), {RamOnlyRoutes, SemiDurableRoutes} = lists:partition(fun (R) -> mnesia:read( - rabbit_semi_durable_route, R#route.binding, read) == [] end, + ?MNESIA_SEMI_DURABLE_TABLE, R#route.binding, read) == [] end, RamRoutes), %% Of course the destination might not really be durable but it's %% just as easy to try to delete it from the semi-durable table @@ -493,13 +676,6 @@ delete_routes(Routes, ShouldIndexTable) -> end, [R#route.binding || R <- Routes]. -delete_transient_routes(Routes) -> - lists:map(fun(#route{binding = #binding{source = Src} = Binding} = Route) -> - {ok, X} = rabbit_db_exchange:get(Src), - ok = sync_transient_route(Route, should_index_table(X), fun delete/3), - Binding - end, Routes). - delete(Tab, #route{binding = B}, LockKind) -> mnesia:delete(Tab, B, LockKind); delete(Tab, #reverse_route{reverse_binding = B}, LockKind) -> @@ -521,44 +697,11 @@ should_index_table(#exchange{name = #resource{name = Name}, should_index_table(_) -> false. -not_found_or_absent_errs_in_mnesia(Names) -> - Errs = [not_found_or_absent_in_mnesia(Name) || Name <- Names], - rabbit_misc:const({error, {resources_missing, Errs}}). - -absent_errs_only_in_mnesia(Names) -> - Errs = [E || Name <- Names, - {absent, _Q, _Reason} = E <- [not_found_or_absent_in_mnesia(Name)]], - rabbit_misc:const(case Errs of - [] -> ok; - _ -> {error, {resources_missing, Errs}} - end). - -not_found_or_absent_in_mnesia(#resource{kind = exchange} = Name) -> - {not_found, Name}; -not_found_or_absent_in_mnesia(#resource{kind = queue} = Name) -> - case rabbit_db_queue:not_found_or_absent_queue_in_mnesia(Name) of - not_found -> {not_found, Name}; - {absent, _Q, _Reason} = R -> R - end. - -recover_in_mnesia() -> - rabbit_mnesia:execute_mnesia_transaction( - fun () -> - _ = mnesia:lock({table, rabbit_durable_route}, read), - _ = mnesia:lock({table, rabbit_semi_durable_route}, write), - Routes = rabbit_mnesia:dirty_read_all(rabbit_durable_route), - Fun = fun(Route) -> - mnesia:dirty_write(rabbit_semi_durable_route, Route) - end, - lists:foreach(Fun, Routes), - ok - end). - recover_semi_durable_route(#route{binding = B} = Route, X) -> MaybeSerial = rabbit_exchange:serialise_events(X), rabbit_mnesia:execute_mnesia_transaction( fun () -> - case mnesia:read(rabbit_semi_durable_route, B, read) of + case mnesia:read(?MNESIA_SEMI_DURABLE_TABLE, B, read) of [] -> no_recover; _ -> ok = sync_transient_route(Route, should_index_table(X), fun mnesia:write/3), serial_in_mnesia(MaybeSerial, X) @@ -572,22 +715,22 @@ recover_semi_durable_route(#route{binding = B} = Route, X) -> serial_in_mnesia(false, _) -> none; serial_in_mnesia(true, X) -> - rabbit_db_exchange:next_serial_in_mnesia_tx(X). + rabbit_db_exchange:next_serial_in_mnesia_tx(X#exchange.name). sync_route(Route, durable, ShouldIndexTable, Fun) -> - ok = Fun(rabbit_durable_route, Route, write), + ok = Fun(?MNESIA_DURABLE_TABLE, Route, write), sync_route(Route, semi_durable, ShouldIndexTable, Fun); sync_route(Route, semi_durable, ShouldIndexTable, Fun) -> - ok = Fun(rabbit_semi_durable_route, Route, write), + ok = Fun(?MNESIA_SEMI_DURABLE_TABLE, Route, write), sync_route(Route, transient, ShouldIndexTable, Fun); sync_route(Route, transient, ShouldIndexTable, Fun) -> sync_transient_route(Route, ShouldIndexTable, Fun). sync_transient_route(Route, ShouldIndexTable, Fun) -> - ok = Fun(rabbit_route, Route, write), - ok = Fun(rabbit_reverse_route, rabbit_binding:reverse_route(Route), write), + ok = Fun(?MNESIA_TABLE, Route, write), + ok = Fun(?MNESIA_REVERSE_TABLE, rabbit_binding:reverse_route(Route), write), sync_index_route(Route, ShouldIndexTable, Fun). sync_index_route(Route, true, Fun) -> @@ -596,13 +739,13 @@ sync_index_route(Route, true, Fun) -> %% (i.e. feature flag migration) runs in parallel. case rabbit_feature_flags:is_enabled(direct_exchange_routing_v2, non_blocking) of true -> - ok = Fun(rabbit_index_route, rabbit_binding:index_route(Route), write); + ok = Fun(?MNESIA_INDEX_TABLE, rabbit_binding:index_route(Route), write); false -> ok; state_changing -> - case rabbit_table:exists(rabbit_index_route) of + case rabbit_table:exists(?MNESIA_INDEX_TABLE) of true -> - ok = Fun(rabbit_index_route, rabbit_binding:index_route(Route), write); + ok = Fun(?MNESIA_INDEX_TABLE, rabbit_binding:index_route(Route), write); false -> ok end @@ -610,6 +753,13 @@ sync_index_route(Route, true, Fun) -> sync_index_route(_, _, _) -> ok. +-spec maybe_auto_delete_exchange_in_mnesia(ExchangeName, [Binding], Deletions, OnlyDurable) + -> Ret when + ExchangeName :: rabbit_exchange:name(), + Binding :: rabbit_types:binding(), + Deletions :: rabbit_binding:deletions(), + OnlyDurable :: boolean(), + Ret :: rabbit_binding:deletions(). maybe_auto_delete_exchange_in_mnesia(XName, Bindings, Deletions, OnlyDurable) -> {Entry, Deletions1} = case rabbit_db_exchange:maybe_auto_delete_in_mnesia(XName, OnlyDurable) of @@ -626,7 +776,7 @@ maybe_auto_delete_exchange_in_mnesia(XName, Bindings, Deletions, OnlyDurable) -> lock_resource(Name) -> lock_resource(Name, write). lock_resource(Name, LockKind) -> - _ = mnesia:lock({global, Name, mnesia:table_info(rabbit_route, where_to_write)}, + _ = mnesia:lock({global, Name, mnesia:table_info(?MNESIA_TABLE, where_to_write)}, LockKind), ok. @@ -644,7 +794,7 @@ route_in_mnesia_v1(SrcName, [RoutingKey]) -> destination = '$1', key = RoutingKey, _ = '_'}}, - ets:select(rabbit_route, [{MatchHead, [], ['$1']}]); + ets:select(?MNESIA_TABLE, [{MatchHead, [], ['$1']}]); route_in_mnesia_v1(SrcName, [_|_] = RoutingKeys) -> %% Normally we'd call mnesia:dirty_select/2 here, but that is quite %% expensive for the same reasons as above, and, additionally, due to @@ -663,7 +813,7 @@ route_in_mnesia_v1(SrcName, [_|_] = RoutingKeys) -> _ = '_'}}, Conditions = [list_to_tuple(['orelse' | [{'=:=', '$2', RKey} || RKey <- RoutingKeys]])], - ets:select(rabbit_route, [{MatchHead, Conditions, ['$1']}]). + ets:select(?MNESIA_TABLE, [{MatchHead, Conditions, ['$1']}]). %% rabbit_router:match_routing_key/2 uses ets:select/2 to get destinations. %% ets:select/2 is expensive because it needs to compile the match spec every diff --git a/deps/rabbit/src/rabbit_db_exchange.erl b/deps/rabbit/src/rabbit_db_exchange.erl index e4cbd2ad4df8..093b0c6e8775 100644 --- a/deps/rabbit/src/rabbit_db_exchange.erl +++ b/deps/rabbit/src/rabbit_db_exchange.erl @@ -19,7 +19,7 @@ count/0, update/2, create_or_get/1, - insert/1, + set/1, peek_serial/1, next_serial/1, delete/2, @@ -37,7 +37,12 @@ update_in_mnesia_tx/2 ]). --type name() :: rabbit_types:r('exchange'). +%% For testing +-export([clear/0]). + +-define(MNESIA_TABLE, rabbit_exchange). +-define(MNESIA_DURABLE_TABLE, rabbit_durable_exchange). +-define(MNESIA_SERIAL_TABLE, rabbit_exchange_serial). %% ------------------------------------------------------------------- %% get_all(). @@ -56,7 +61,7 @@ get_all() -> #{mnesia => fun() -> get_all_in_mnesia() end}). get_all_in_mnesia() -> - rabbit_db:list_in_mnesia(rabbit_exchange, #exchange{_ = '_'}). + rabbit_db:list_in_mnesia(?MNESIA_TABLE, #exchange{_ = '_'}). -spec get_all(VHostName) -> [Exchange] when VHostName :: vhost:name(), @@ -74,7 +79,11 @@ get_all(VHost) -> get_all_in_mnesia(VHost) -> Match = #exchange{name = rabbit_misc:r(VHost, exchange), _ = '_'}, - rabbit_db:list_in_mnesia(rabbit_exchange, Match). + rabbit_db:list_in_mnesia(?MNESIA_TABLE, Match). + +%% ------------------------------------------------------------------- +%% get_all_durable(). +%% ------------------------------------------------------------------- -spec get_all_durable() -> [Exchange] when Exchange :: rabbit_types:exchange(). @@ -96,8 +105,8 @@ get_all_durable_in_mnesia() -> %% list(). %% ------------------------------------------------------------------- --spec list() -> [Exchange] when - Exchange :: rabbit_types:exchange(). +-spec list() -> [ExchangeName] when + ExchangeName :: rabbit_exchange:name(). %% @doc Lists the names of all exchanges. %% %% @returns a list of exchange names. @@ -110,14 +119,14 @@ list() -> }). list_in_mnesia() -> - mnesia:dirty_all_keys(rabbit_exchange). + mnesia:dirty_all_keys(?MNESIA_TABLE). %% ------------------------------------------------------------------- %% get(). %% ------------------------------------------------------------------- -spec get(ExchangeName) -> Ret when - ExchangeName :: name(), + ExchangeName :: rabbit_exchange:name(), Ret :: {ok, Exchange :: rabbit_types:exchange()} | {error, not_found}. %% @doc Returns the record of the exchange named `Name'. %% @@ -132,14 +141,14 @@ get(Name) -> }). get_in_mnesia(Name) -> - rabbit_mnesia:dirty_read({rabbit_exchange, Name}). + rabbit_mnesia:dirty_read({?MNESIA_TABLE, Name}). %% ------------------------------------------------------------------- %% get_many(). %% ------------------------------------------------------------------- -spec get_many([ExchangeName]) -> [Exchange] when - ExchangeName :: name(), + ExchangeName :: rabbit_exchange:name(), Exchange :: rabbit_types:exchange(). %% @doc Returns the records of the exchanges named `Name'. %% @@ -149,9 +158,15 @@ get_in_mnesia(Name) -> get_many(Names) when is_list(Names) -> rabbit_db:run( - #{mnesia => fun() -> get_many_in_mnesia(rabbit_exchange, Names) end + #{mnesia => fun() -> get_many_in_mnesia(?MNESIA_TABLE, Names) end }). +get_many_in_mnesia(Table, [Name]) -> ets:lookup(Table, Name); +get_many_in_mnesia(Table, Names) when is_list(Names) -> + %% Normally we'd call mnesia:dirty_read/1 here, but that is quite + %% expensive for reasons explained in rabbit_mnesia:dirty_read/1. + lists:append([ets:lookup(Table, Name) || Name <- Names]). + %% ------------------------------------------------------------------- %% count(). %% ------------------------------------------------------------------- @@ -168,16 +183,15 @@ count() -> #{mnesia => fun() -> count_in_mnesia() end}). count_in_mnesia() -> - mnesia:table_info(rabbit_exchange, size). + mnesia:table_info(?MNESIA_TABLE, size). %% ------------------------------------------------------------------- %% update(). %% ------------------------------------------------------------------- --spec update(ExchangeName, UpdateFun) -> Ret when - ExchangeName :: name(), - UpdateFun :: fun((Exchange) -> Exchange), - Ret :: Exchange :: rabbit_types:exchange() | not_found. +-spec update(ExchangeName, UpdateFun) -> ok when + ExchangeName :: rabbit_exchange:name(), + UpdateFun :: fun((Exchange) -> Exchange). %% @doc Updates an existing exchange record using the result of %% `UpdateFun'. %% @@ -194,9 +208,36 @@ update(XName, Fun) -> update_in_mnesia(XName, Fun) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> - update_in_mnesia_tx(XName, Fun) + _ = update_in_mnesia_tx(XName, Fun), + ok end). +-spec update_in_mnesia_tx(ExchangeName, UpdateFun) -> Ret when + ExchangeName :: rabbit_exchange:name(), + Exchange :: rabbit_types:exchange(), + UpdateFun :: fun((Exchange) -> Exchange), + Ret :: not_found | Exchange. + +update_in_mnesia_tx(Name, Fun) -> + Table = {?MNESIA_TABLE, Name}, + case mnesia:wread(Table) of + [X] -> X1 = Fun(X), + set_in_mnesia_tx(X1); + [] -> not_found + end. + +set_in_mnesia_tx(X = #exchange{durable = true}) -> + mnesia:write(rabbit_durable_exchange, X#exchange{decorators = undefined}, + write), + set_ram_in_mnesia_tx(X); +set_in_mnesia_tx(X = #exchange{durable = false}) -> + set_ram_in_mnesia_tx(X). + +set_ram_in_mnesia_tx(X) -> + X1 = rabbit_exchange_decorator:set(X), + ok = mnesia:write(?MNESIA_TABLE, X1, write), + X1. + %% ------------------------------------------------------------------- %% create_or_get(). %% ------------------------------------------------------------------- @@ -220,19 +261,19 @@ create_or_get(X) -> create_or_get_in_mnesia(#exchange{name = XName} = X) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> - case mnesia:wread({rabbit_exchange, XName}) of + case mnesia:wread({?MNESIA_TABLE, XName}) of [] -> - {new, insert_in_mnesia_tx(X)}; + {new, set_in_mnesia_tx(X)}; [ExistingX] -> {existing, ExistingX} end end). %% ------------------------------------------------------------------- -%% insert(). +%% set(). %% ------------------------------------------------------------------- --spec insert([Exchange]) -> ok when +-spec set([Exchange]) -> ok when Exchange :: rabbit_types:exchange(). %% @doc Writes the exchange records. %% @@ -240,12 +281,12 @@ create_or_get_in_mnesia(#exchange{name = XName} = X) -> %% %% @private -insert(Xs) -> +set(Xs) -> rabbit_db:run( - #{mnesia => fun() -> insert_in_mnesia(Xs) end + #{mnesia => fun() -> set_in_mnesia(Xs) end }). -insert_in_mnesia(Xs) when is_list(Xs) -> +set_in_mnesia(Xs) when is_list(Xs) -> rabbit_mnesia:execute_mnesia_transaction( fun () -> [mnesia:write(rabbit_durable_exchange, X, write) || X <- Xs] @@ -257,7 +298,7 @@ insert_in_mnesia(Xs) when is_list(Xs) -> %% ------------------------------------------------------------------- -spec peek_serial(ExchangeName) -> Serial when - ExchangeName :: name(), + ExchangeName :: rabbit_exchange:name(), Serial :: integer(). %% @doc Returns the next serial number without increasing it. %% @@ -276,12 +317,18 @@ peek_serial_in_mnesia(XName) -> peek_serial_in_mnesia_tx(XName, read) end). +peek_serial_in_mnesia_tx(XName, LockType) -> + case mnesia:read(?MNESIA_SERIAL_TABLE, XName, LockType) of + [#exchange_serial{next = Serial}] -> Serial; + _ -> 1 + end. + %% ------------------------------------------------------------------- %% next_serial(). %% ------------------------------------------------------------------- -spec next_serial(ExchangeName) -> Serial when - ExchangeName :: name(), + ExchangeName :: rabbit_exchange:name(), Serial :: integer(). %% @doc Returns the next serial number and increases it. %% @@ -289,29 +336,39 @@ peek_serial_in_mnesia(XName) -> %% %% @private -next_serial(X) -> +next_serial(XName) -> rabbit_db:run( - #{mnesia => fun() -> next_serial_in_mnesia(X) end + #{mnesia => fun() -> next_serial_in_mnesia(XName) end }). -next_serial_in_mnesia(X) -> +next_serial_in_mnesia(XName) -> rabbit_mnesia:execute_mnesia_transaction(fun() -> - next_serial_in_mnesia_tx(X) + next_serial_in_mnesia_tx(XName) end). +-spec next_serial_in_mnesia_tx(ExchangeName) -> Serial when + ExchangeName :: rabbit_exchange:name(), + Serial :: integer(). + +next_serial_in_mnesia_tx(XName) -> + Serial = peek_serial_in_mnesia_tx(XName, write), + ok = mnesia:write(?MNESIA_SERIAL_TABLE, + #exchange_serial{name = XName, next = Serial + 1}, write), + Serial. + %% ------------------------------------------------------------------- %% delete(). %% ------------------------------------------------------------------- -spec delete(ExchangeName, IfUnused) -> Ret when - ExchangeName :: name(), + ExchangeName :: rabbit_exchange:name(), IfUnused :: boolean(), Exchange :: rabbit_types:exchange(), Binding :: rabbit_types:binding(), Deletions :: dict:dict(), - Ret :: {error, not_found} | {deleted, Exchange, [Binding], Deletions}. -%% @doc Deletes an exchange record from the database. If `IfUnused` is set -%% to `true`, it is only deleted when there are no bindings present on the + Ret :: {error, not_found} | {error, in_use} | {deleted, Exchange, [Binding], Deletions}. +%% @doc Deletes an exchange record from the database. If `IfUnused' is set +%% to `true', it is only deleted when there are no bindings present on the %% exchange. %% %% @returns an error if the exchange does not exist or a tuple with the exchange, @@ -334,18 +391,41 @@ delete_in_mnesia(XName, IfUnused) -> end, rabbit_mnesia:execute_mnesia_transaction( fun() -> - case mnesia:wread({rabbit_exchange, XName}) of + case mnesia:wread({?MNESIA_TABLE, XName}) of [X] -> DeletionFun(X, false); [] -> {error, not_found} end end). +conditional_delete_in_mnesia(X = #exchange{name = XName}, OnlyDurable) -> + case rabbit_db_binding:has_for_source_in_mnesia(XName) of + false -> delete_in_mnesia(X, OnlyDurable, false); + true -> {error, in_use} + end. + +unconditional_delete_in_mnesia(X, OnlyDurable) -> + delete_in_mnesia(X, OnlyDurable, true). + +-spec delete_in_mnesia(Exchange, OnlyDurable, RemoveBindingsForSource) -> Ret when + Exchange :: rabbit_types:exchange(), + OnlyDurable :: boolean(), + RemoveBindingsForSource :: boolean(), + Exchange :: rabbit_types:exchange(), + Binding :: rabbit_types:binding(), + Deletions :: dict:dict(), + Ret :: {error, not_found} | {error, in_use} | {deleted, Exchange, [Binding], Deletions}. +delete_in_mnesia(X = #exchange{name = XName}, OnlyDurable, RemoveBindingsForSource) -> + ok = mnesia:delete({?MNESIA_TABLE, XName}), + mnesia:delete({?MNESIA_DURABLE_TABLE, XName}), + rabbit_db_binding:delete_all_for_exchange_in_mnesia( + X, OnlyDurable, RemoveBindingsForSource). + %% ------------------------------------------------------------------- %% delete_serial(). %% ------------------------------------------------------------------- -spec delete_serial(ExchangeName) -> ok when - ExchangeName :: name(). + ExchangeName :: rabbit_exchange:name(). %% @doc Deletes an exchange serial record from the database. %% %% @returns ok @@ -360,14 +440,15 @@ delete_serial(XName) -> delete_serial_in_mnesia(XName) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> - mnesia:delete({rabbit_exchange_serial, XName}) + mnesia:delete({?MNESIA_SERIAL_TABLE, XName}) end). %% ------------------------------------------------------------------- %% recover(). %% ------------------------------------------------------------------- --spec recover(VHostName) -> ok when +-spec recover(VHostName) -> [Exchange] when + Exchange :: rabbit_types:exchange(), VHostName :: vhost:name(). %% @doc Recovers all exchanges for a given vhost %% @@ -380,13 +461,30 @@ recover(VHost) -> #{mnesia => fun() -> recover_in_mnesia(VHost) end }). +recover_in_mnesia(VHost) -> + rabbit_mnesia:table_filter( + fun (#exchange{name = XName}) -> + XName#resource.virtual_host =:= VHost andalso + mnesia:read({?MNESIA_TABLE, XName}) =:= [] + end, + fun (X, true) -> + X; + (X, false) -> + X1 = rabbit_mnesia:execute_mnesia_transaction( + fun() -> set_ram_in_mnesia_tx(X) end), + Serial = rabbit_exchange:serial(X1), + rabbit_exchange:callback(X1, create, Serial, [X1]) + end, + ?MNESIA_DURABLE_TABLE). + %% ------------------------------------------------------------------- %% match(). %% ------------------------------------------------------------------- --spec match(Pattern) -> [Exchange] when +-spec match(Pattern) -> Ret when Pattern :: #exchange{}, - Exchange :: rabbit_types:exchange(). + Exchange :: rabbit_types:exchange(), + Ret :: [Exchange] | {error, Reason :: any()}. %% @doc Returns all exchanges that match a given pattern %% %% @returns a list of exchange records @@ -401,7 +499,7 @@ match(Pattern) -> match_in_mnesia(Pattern) -> case mnesia:transaction( fun() -> - mnesia:match_object(rabbit_exchange, Pattern, read) + mnesia:match_object(?MNESIA_TABLE, Pattern, read) end) of {atomic, Xs} -> Xs; {aborted, Err} -> {error, Err} @@ -412,7 +510,7 @@ match_in_mnesia(Pattern) -> %% ------------------------------------------------------------------- -spec exists(ExchangeName) -> Exists when - ExchangeName :: name(), + ExchangeName :: rabbit_exchange:name(), Exists :: boolean(). %% @doc Indicates if the exchange named `Name' exists. %% @@ -425,58 +523,41 @@ exists(Name) -> #{mnesia => fun() -> exists_in_mnesia(Name) end}). exists_in_mnesia(Name) -> - ets:member(rabbit_exchange, Name). - -%% Internal -%% -------------------------------------------------------------- - -peek_serial_in_mnesia_tx(XName, LockType) -> - case mnesia:read(rabbit_exchange_serial, XName, LockType) of - [#exchange_serial{next = Serial}] -> Serial; - _ -> 1 - end. - -next_serial_in_mnesia_tx(#exchange{name = XName}) -> - Serial = peek_serial_in_mnesia_tx(XName, write), - ok = mnesia:write(rabbit_exchange_serial, - #exchange_serial{name = XName, next = Serial + 1}, write), - Serial. + ets:member(?MNESIA_TABLE, Name). -update_in_mnesia_tx(Name, Fun) -> - Table = {rabbit_exchange, Name}, - case mnesia:wread(Table) of - [X] -> X1 = Fun(X), - insert_in_mnesia_tx(X1); - [] -> not_found - end. +%% ------------------------------------------------------------------- +%% clear(). +%% ------------------------------------------------------------------- -delete_in_mnesia(X = #exchange{name = XName}, OnlyDurable, RemoveBindingsForSource) -> - ok = mnesia:delete({rabbit_exchange, XName}), - mnesia:delete({rabbit_durable_exchange, XName}), - rabbit_db_binding:delete_all_for_exchange_in_mnesia(X, OnlyDurable, RemoveBindingsForSource). +-spec clear() -> ok. +%% @doc Deletes all exchanges. +%% +%% @private -get_many_in_mnesia(Table, [Name]) -> ets:lookup(Table, Name); -get_many_in_mnesia(Table, Names) when is_list(Names) -> - %% Normally we'd call mnesia:dirty_read/1 here, but that is quite - %% expensive for reasons explained in rabbit_mnesia:dirty_read/1. - lists:append([ets:lookup(Table, Name) || Name <- Names]). +clear() -> + rabbit_db:run( + #{mnesia => fun() -> clear_in_mnesia() end}). -conditional_delete_in_mnesia(X = #exchange{name = XName}, OnlyDurable) -> - case rabbit_db_binding:has_for_source_in_mnesia(XName) of - false -> delete_in_mnesia(X, OnlyDurable, false); - true -> {error, in_use} - end. +clear_in_mnesia() -> + {atomic, ok} = mnesia:clear_table(?MNESIA_TABLE), + {atomic, ok} = mnesia:clear_table(?MNESIA_DURABLE_TABLE), + {atomic, ok} = mnesia:clear_table(?MNESIA_SERIAL_TABLE), + ok. -unconditional_delete_in_mnesia(X, OnlyDurable) -> - delete_in_mnesia(X, OnlyDurable, true). +%% ------------------------------------------------------------------- +%% maybe_auto_delete_in_mnesia(). +%% ------------------------------------------------------------------- --spec maybe_auto_delete_in_mnesia - (rabbit_types:exchange(), boolean()) - -> 'not_deleted' | {'deleted', rabbit_binding:deletions()}. +-spec maybe_auto_delete_in_mnesia(ExchangeName, boolean()) -> Ret when + ExchangeName :: rabbit_exchange:name(), + Exchange :: rabbit_types:exchange(), + Deletions :: rabbit_binding:deletions(), + Ret :: {'not_deleted', 'undefined' | Exchange} | + {'deleted', Exchange, Deletions}. maybe_auto_delete_in_mnesia(XName, OnlyDurable) -> case mnesia:read({case OnlyDurable of - true -> rabbit_durable_exchange; - false -> rabbit_exchange + true -> ?MNESIA_DURABLE_TABLE; + false -> ?MNESIA_TABLE end, XName}) of [] -> {not_deleted, undefined}; [#exchange{auto_delete = false} = X] -> {not_deleted, X}; @@ -486,31 +567,3 @@ maybe_auto_delete_in_mnesia(XName, OnlyDurable) -> {deleted, X, [], Deletions} -> {deleted, X, Deletions} end end. - -insert_in_mnesia_tx(X = #exchange{durable = true}) -> - mnesia:write(rabbit_durable_exchange, X#exchange{decorators = undefined}, - write), - insert_ram_in_mnesia_tx(X); -insert_in_mnesia_tx(X = #exchange{durable = false}) -> - insert_ram_in_mnesia_tx(X). - -insert_ram_in_mnesia_tx(X) -> - X1 = rabbit_exchange_decorator:set(X), - ok = mnesia:write(rabbit_exchange, X1, write), - X1. - -recover_in_mnesia(VHost) -> - rabbit_mnesia:table_filter( - fun (#exchange{name = XName}) -> - XName#resource.virtual_host =:= VHost andalso - mnesia:read({rabbit_exchange, XName}) =:= [] - end, - fun (X, true) -> - X; - (X, false) -> - X1 = rabbit_mnesia:execute_mnesia_transaction( - fun() -> insert_in_mnesia_tx(X) end), - Serial = rabbit_exchange:serial(X1), - rabbit_exchange:callback(X1, create, Serial, [X1]) - end, - rabbit_durable_exchange). diff --git a/deps/rabbit/src/rabbit_db_maintenance.erl b/deps/rabbit/src/rabbit_db_maintenance.erl index ba6e59be64db..6036dc4149c2 100644 --- a/deps/rabbit/src/rabbit_db_maintenance.erl +++ b/deps/rabbit/src/rabbit_db_maintenance.erl @@ -64,8 +64,9 @@ status_table_definition() -> %% set(). %% ------------------------------------------------------------------- --spec set(Status) -> ok when - Status :: rabbit_maintenance:maintenance_status(). +-spec set(Status) -> Ret when + Status :: rabbit_maintenance:maintenance_status(), + Ret :: boolean(). %% @doc Sets the maintenance status for the local node %% %% @private @@ -104,7 +105,7 @@ set_in_mnesia(Status) -> -spec get(Node) -> Status when Node :: node(), - Status :: rabbit_maintenance:maintenance_status(). + Status :: undefined | rabbit_maintenance:maintenance_status(). %% @doc Returns the status for the given node using a local query. %% %% @returns the status if any, or `undefined'. @@ -130,7 +131,7 @@ get_in_mnesia(Node) -> -spec get_consistent(Node) -> Status when Node :: node(), - Status :: rabbit_maintenance:maintenance_status(). + Status :: undefined | rabbit_maintenance:maintenance_status(). %% @doc Returns the status for the given node using a consistent query. %% %% @returns the status if any, or `undefined'. diff --git a/deps/rabbit/src/rabbit_db_msup.erl b/deps/rabbit/src/rabbit_db_msup.erl index 0f56b609d726..35b8b36d04b0 100644 --- a/deps/rabbit/src/rabbit_db_msup.erl +++ b/deps/rabbit/src/rabbit_db_msup.erl @@ -7,6 +7,18 @@ -module(rabbit_db_msup). +-export([ + create_tables/0, + table_definitions/0, + create_or_update/5, + find_mirror/2, + update_all/2, + delete/2, + delete_all/1 + ]). + +-export([clear/0]). + -define(TABLE, mirrored_sup_childspec). -define(TABLE_DEF, {?TABLE, @@ -15,12 +27,12 @@ {attributes, record_info(fields, mirrored_sup_childspec)}]}). -define(TABLE_MATCH, {match, #mirrored_sup_childspec{ _ = '_' }}). --export([create_tables/0, table_definitions/0, - create_or_update/5, delete/2, - find_mirror/2, update_all/2, delete_all/1]). - -record(mirrored_sup_childspec, {key, mirroring_pid, childspec}). +%% ------------------------------------------------------------------- +%% create_tables(). +%% ------------------------------------------------------------------- + -spec create_tables() -> Ret when Ret :: 'ok' | {error, Reason :: term()}. @@ -29,6 +41,19 @@ create_tables() -> #{mnesia => fun() -> create_tables_in_mnesia([?TABLE_DEF]) end }). +create_tables_in_mnesia([]) -> + ok; +create_tables_in_mnesia([{Table, Attributes} | Ts]) -> + case mnesia:create_table(Table, Attributes) of + {atomic, ok} -> create_tables_in_mnesia(Ts); + {aborted, {already_exists, ?TABLE}} -> create_tables_in_mnesia(Ts); + Err -> Err + end. + +%% ------------------------------------------------------------------- +%% table_definitions(). +%% ------------------------------------------------------------------- + -spec table_definitions() -> [Def] when Def :: {Name :: atom(), term()}. @@ -36,6 +61,10 @@ table_definitions() -> {Name, Attributes} = ?TABLE_DEF, [{Name, [?TABLE_MATCH | Attributes]}]. +%% ------------------------------------------------------------------- +%% create_or_update(). +%% ------------------------------------------------------------------- + -spec create_or_update(Group, Overall, Delegate, ChildSpec, Id) -> Ret when Group :: any(), Overall :: pid(), @@ -51,6 +80,47 @@ create_or_update(Group, Overall, Delegate, ChildSpec, Id) -> create_or_update_in_mnesia(Group, Overall, Delegate, ChildSpec, Id) end}). +create_or_update_in_mnesia(Group, Overall, Delegate, ChildSpec, Id) -> + rabbit_mnesia:execute_mnesia_transaction( + fun() -> + ReadResult = mnesia:wread({?TABLE, {Group, Id}}), + rabbit_log:debug("Mirrored supervisor: check_start table ~ts read for key ~tp returned ~tp", + [?TABLE, {Group, Id}, ReadResult]), + case ReadResult of + [] -> _ = write_in_mnesia(Group, Overall, ChildSpec, Id), + start; + [S] -> #mirrored_sup_childspec{key = {Group, Id}, + mirroring_pid = Pid} = S, + case Overall of + Pid -> + rabbit_log:debug("Mirrored supervisor: overall matched mirrored pid ~tp", [Pid]), + Delegate; + _ -> + rabbit_log:debug("Mirrored supervisor: overall ~tp did not match mirrored pid ~tp", [Overall, Pid]), + Sup = mirrored_supervisor:supervisor(Pid), + rabbit_log:debug("Mirrored supervisor: supervisor(~tp) returned ~tp", [Pid, Sup]), + case Sup of + dead -> + _ = write_in_mnesia(Group, Overall, ChildSpec, Id), + start; + Delegate0 -> + Delegate0 + end + end + end + end). + +write_in_mnesia(Group, Overall, ChildSpec, Id) -> + S = #mirrored_sup_childspec{key = {Group, Id}, + mirroring_pid = Overall, + childspec = ChildSpec}, + ok = mnesia:write(?TABLE, S, write), + ChildSpec. + +%% ------------------------------------------------------------------- +%% delete(). +%% ------------------------------------------------------------------- + -spec delete(Group, Id) -> ok when Group :: any(), Id :: any(). @@ -62,7 +132,13 @@ delete(Group, Id) -> delete_in_mnesia(Group, Id) -> rabbit_mnesia:execute_mnesia_transaction( - fun() -> delete_in_mnesia_tx(Group, Id) end). + fun() -> + ok = mnesia:delete({?TABLE, {Group, Id}}) + end). + +%% ------------------------------------------------------------------- +%% find_mirror(). +%% ------------------------------------------------------------------- -spec find_mirror(Group, Id) -> Ret when Group :: any(), @@ -86,6 +162,10 @@ find_mirror_in_mnesia(Group, Id) -> _ -> {error, not_found} end. +%% ------------------------------------------------------------------- +%% update_all(). +%% ------------------------------------------------------------------- + -spec update_all(Overall, Overall) -> [ChildSpec] when Overall :: pid(), ChildSpec :: supervisor2:child_spec(). @@ -95,55 +175,6 @@ update_all(Overall, OldOverall) -> #{mnesia => fun() -> update_all_in_mnesia(Overall, OldOverall) end }). --spec delete_all(Group) -> ok when - Group :: any(). - -delete_all(Group) -> - rabbit_db:run( - #{mnesia => fun() -> delete_all_in_mnesia(Group) end - }). - -%%---------------------------------------------------------------------------- - -create_tables_in_mnesia([]) -> - ok; -create_tables_in_mnesia([{Table, Attributes} | Ts]) -> - case mnesia:create_table(Table, Attributes) of - {atomic, ok} -> create_tables_in_mnesia(Ts); - {aborted, {already_exists, ?TABLE}} -> create_tables_in_mnesia(Ts); - Err -> Err - end. - -create_or_update_in_mnesia(Group, Overall, Delegate, ChildSpec, Id) -> - rabbit_mnesia:execute_mnesia_transaction( - fun() -> - ReadResult = mnesia:wread({?TABLE, {Group, Id}}), - rabbit_log:debug("Mirrored supervisor: check_start table ~ts read for key ~tp returned ~tp", - [?TABLE, {Group, Id}, ReadResult]), - case ReadResult of - [] -> _ = write_in_mnesia(Group, Overall, ChildSpec, Id), - start; - [S] -> #mirrored_sup_childspec{key = {Group, Id}, - mirroring_pid = Pid} = S, - case Overall of - Pid -> - rabbit_log:debug("Mirrored supervisor: overall matched mirrored pid ~tp", [Pid]), - Delegate; - _ -> - rabbit_log:debug("Mirrored supervisor: overall ~tp did not match mirrored pid ~tp", [Overall, Pid]), - Sup = mirrored_supervisor:supervisor(Pid), - rabbit_log:debug("Mirrored supervisor: supervisor(~tp) returned ~tp", [Pid, Sup]), - case Sup of - dead -> - _ = write_in_mnesia(Group, Overall, ChildSpec, Id), - start; - Delegate0 -> - Delegate0 - end - end - end - end). - update_all_in_mnesia(Overall, OldOverall) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> @@ -155,22 +186,38 @@ update_all_in_mnesia(Overall, OldOverall) -> [{Group, Id}, C] <- mnesia:select(?TABLE, [{MatchHead, [], ['$$']}])] end). +%% ------------------------------------------------------------------- +%% delete_all(). +%% ------------------------------------------------------------------- + +-spec delete_all(Group) -> ok when + Group :: any(). + +delete_all(Group) -> + rabbit_db:run( + #{mnesia => fun() -> delete_all_in_mnesia(Group) end + }). + delete_all_in_mnesia(Group) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> MatchHead = #mirrored_sup_childspec{key = {Group, '$1'}, _ = '_'}, - [delete_in_mnesia_tx(Group, Id) || + [ok = mnesia:delete({?TABLE, {Group, Id}}) || Id <- mnesia:select(?TABLE, [{MatchHead, [], ['$1']}])] end), ok. -write_in_mnesia(Group, Overall, ChildSpec, Id) -> - S = #mirrored_sup_childspec{key = {Group, Id}, - mirroring_pid = Overall, - childspec = ChildSpec}, - ok = mnesia:write(?TABLE, S, write), - ChildSpec. +%% ------------------------------------------------------------------- +%% clear(). +%% ------------------------------------------------------------------- + +-spec clear() -> ok. -delete_in_mnesia_tx(Group, Id) -> - ok = mnesia:delete({?TABLE, {Group, Id}}). +clear() -> + rabbit_db:run( + #{mnesia => fun() -> clear_in_mnesia() end}). + +clear_in_mnesia() -> + {atomic, ok} = mnesia:clear_table(?TABLE), + ok. diff --git a/deps/rabbit/src/rabbit_db_policy.erl b/deps/rabbit/src/rabbit_db_policy.erl index 25ae02ac2a3c..ee713aa94ca7 100644 --- a/deps/rabbit/src/rabbit_db_policy.erl +++ b/deps/rabbit/src/rabbit_db_policy.erl @@ -12,12 +12,18 @@ -export([update/3]). +%% ------------------------------------------------------------------- +%% update(). +%% ------------------------------------------------------------------- + -spec update(VHostName, UpdateXFun, UpdateQFun) -> Ret when VHostName :: vhost:name(), Exchange :: rabbit_types:exchange(), Queue :: amqqueue:amqqueue(), - UpdateXFun :: fun((Exchange) -> Exchange), - UpdateQFun :: fun((Queue) -> Queue), + UpdateXFun :: fun((Exchange) -> #{exchange => Exchange, + update_function => fun((Exchange) -> Exchange)}), + UpdateQFun :: fun((Queue) -> #{queue => Queue, + update_function => fun((Queue) -> Queue)}), Ret :: {[{Exchange, Exchange}], [{Queue, Queue}]}. update(VHost, GetUpdatedExchangeFun, GetUpdatedQueueFun) -> @@ -33,7 +39,7 @@ update_in_mnesia(VHost, GetUpdatedExchangeFun, GetUpdatedQueueFun) -> rabbit_exchange, rabbit_durable_exchange], rabbit_mnesia:execute_mnesia_transaction( fun() -> - [mnesia:lock({table, T}, write) || T <- Tabs], %% [1] + _ = [mnesia:lock({table, T}, write) || T <- Tabs], %% [1] Exchanges0 = rabbit_db_exchange:get_all(VHost), Queues0 = rabbit_db_queue:get_all(VHost), Exchanges = [GetUpdatedExchangeFun(X) || X <- Exchanges0], diff --git a/deps/rabbit/src/rabbit_db_queue.erl b/deps/rabbit/src/rabbit_db_queue.erl index e713759a5dbb..407d58e6af06 100644 --- a/deps/rabbit/src/rabbit_db_queue.erl +++ b/deps/rabbit/src/rabbit_db_queue.erl @@ -13,6 +13,7 @@ -export([ get/1, + get_many/1, get_all/0, get_all/1, get_all_by_type/1, @@ -20,9 +21,9 @@ list/0, count/0, count/1, - create_or_get/2, - insert/2, - insert/1, + create_or_get/1, + set/1, + set_many/1, delete/2, update/2, update_decorators/1, @@ -31,28 +32,36 @@ -export([ get_all_durable/0, - get_all_durable/1, get_all_durable_by_type/1, - get_durable/1 + filter_all_durable/1, + update_durable/2, + get_durable/1, + get_many_durable/1, + consistent_exists/1 ]). --export([delete_transient/1]). --export([on_node_up/2, - on_node_down/2]). +%% Used by on_node_up and on_node_down +-export([foreach_transient/1, + delete_transient/1]). --export([match_and_update/3]). --export([insert_dirty/1]). +%% Used only by forget all durable +-export([foreach_durable/2, + internal_delete/3]). --export([not_found_or_absent_queue_dirty/1]). - --export([internal_delete/3]). +-export([set_dirty/1]). %% Used by other rabbit_db_* modules -export([ update_in_mnesia_tx/2, - not_found_or_absent_queue_in_mnesia/1 + get_durable_in_mnesia_tx/1 ]). +%% For testing +-export([clear/0]). + +-define(MNESIA_TABLE, rabbit_queue). +-define(MNESIA_DURABLE_TABLE, rabbit_durable_queue). + %% ------------------------------------------------------------------- %% get_all(). %% ------------------------------------------------------------------- @@ -74,7 +83,7 @@ get_all() -> get_all_in_mnesia() -> list_with_possible_retry_in_mnesia( fun() -> - rabbit_db:list_in_mnesia(rabbit_queue, amqqueue:pattern_match_all()) + rabbit_db:list_in_mnesia(?MNESIA_TABLE, amqqueue:pattern_match_all()) end). -spec get_all(VHostName) -> [Queue] when @@ -87,11 +96,18 @@ get_all_in_mnesia() -> %% %% @private -get_all(VHost) -> +get_all(VHostName) -> rabbit_db:run( - #{mnesia => fun() -> get_all_in_mnesia(VHost) end + #{mnesia => fun() -> get_all_in_mnesia(VHostName) end }). +get_all_in_mnesia(VHostName) -> + list_with_possible_retry_in_mnesia( + fun() -> + Pattern = amqqueue:pattern_match_on_name(rabbit_misc:r(VHostName, queue)), + rabbit_db:list_in_mnesia(?MNESIA_TABLE, Pattern) + end). + %% ------------------------------------------------------------------- %% get_all_durable(). %% ------------------------------------------------------------------- @@ -113,31 +129,19 @@ get_all_durable() -> get_all_durable_in_mnesia() -> list_with_possible_retry_in_mnesia( fun() -> - rabbit_db:list_in_mnesia(rabbit_durable_queue, amqqueue:pattern_match_all()) + rabbit_db:list_in_mnesia(?MNESIA_DURABLE_TABLE, amqqueue:pattern_match_all()) end). --spec get_all_durable(VHostName) -> [Queue] when - VHostName :: vhost:name(), +-spec get_all_durable_by_type(Type) -> [Queue] when + Type :: atom(), Queue :: amqqueue:amqqueue(). -%% @doc Gets all durable queues belonging to the given virtual host +%% @doc Gets all durable queues of the given type %% %% @returns a list of queue records. %% %% @private -get_all_durable(VHost) -> - rabbit_db:run( - #{mnesia => fun() -> get_all_durable_in_mnesia(VHost) end - }). - -get_all_durable_in_mnesia(VHost) -> - list_with_possible_retry_in_mnesia( - fun() -> - Pattern = amqqueue:pattern_match_on_name(rabbit_misc:r(VHost, queue)), - rabbit_db:list_in_mnesia(rabbit_durable_queue, Pattern) - end). - get_all_durable_by_type(Type) -> rabbit_db:run( #{mnesia => fun() -> get_all_durable_by_type_in_mnesia(Type) end @@ -145,7 +149,47 @@ get_all_durable_by_type(Type) -> get_all_durable_by_type_in_mnesia(Type) -> Pattern = amqqueue:pattern_match_on_type(Type), - rabbit_db:list_in_mnesia(rabbit_durable_queue, Pattern). + rabbit_db:list_in_mnesia(?MNESIA_DURABLE_TABLE, Pattern). + +%% ------------------------------------------------------------------- +%% filter_all_durable(). +%% ------------------------------------------------------------------- + +-spec filter_all_durable(FilterFun) -> [Queue] when + Queue :: amqqueue:amqqueue(), + FilterFun :: fun((Queue) -> boolean()). + +%% @doc Filters all durable queues +%% +%% @returns a list of queue records. +%% +%% @private + +filter_all_durable(FilterFun) -> + rabbit_db:run( + #{mnesia => fun() -> filter_all_durable_in_mnesia(FilterFun) end + }). + +filter_all_durable_in_mnesia(FilterFun) -> + rabbit_mnesia:execute_mnesia_transaction( + fun () -> + qlc:e(qlc:q([Q || Q <- mnesia:table(?MNESIA_DURABLE_TABLE), + FilterFun(Q) + ])) + end). + +%% ------------------------------------------------------------------- +%% list(). +%% ------------------------------------------------------------------- + +-spec list() -> [QName] when + QName :: rabbit_amqqueue:name(). + +%% @doc Returns all queue names. +%% +%% @returns the list of all queue names. +%% +%% @private list() -> rabbit_db:run( @@ -153,7 +197,20 @@ list() -> }). list_in_mnesia() -> - mnesia:dirty_all_keys(rabbit_queue). + mnesia:dirty_all_keys(?MNESIA_TABLE). + +%% ------------------------------------------------------------------- +%% count(). +%% ------------------------------------------------------------------- + +-spec count() -> Count when + Count :: integer(). + +%% @doc Counts the number of queues +%% +%% @returns the number of queues. +%% +%% @private count() -> rabbit_db:run( @@ -161,141 +218,307 @@ count() -> }). count_in_mnesia() -> - mnesia:table_info(rabbit_queue, size). + mnesia:table_info(?MNESIA_TABLE, size). + +-spec count(VHostName) -> Count when + VHostName :: vhost:name(), + Count :: integer(). -count(VHost) -> +%% @doc Counts the number of queues for the given vhost +%% +%% @returns the number of queues for the given vhost +%% +%% @private + +count(VHostName) -> try - list_for_count(VHost) + list_for_count(VHostName) catch _:Err -> rabbit_log:error("Failed to fetch number of queues in vhost ~p:~n~p", - [VHost, Err]), + [VHostName, Err]), 0 end. +list_for_count(VHostName) -> + rabbit_db:run( + #{mnesia => fun() -> list_for_count_in_mnesia(VHostName) end + }). + +list_for_count_in_mnesia(VHostName) -> + %% this is certainly suboptimal but there is no way to count + %% things using a secondary index in Mnesia. Our counter-table-per-node + %% won't work here because with master migration of mirrored queues + %% the "ownership" of queues by nodes becomes a non-trivial problem + %% that requires a proper consensus algorithm. + list_with_possible_retry_in_mnesia( + fun() -> + length(mnesia:dirty_index_read(?MNESIA_TABLE, + VHostName, + amqqueue:field_vhost())) + end). + +%% ------------------------------------------------------------------- +%% delete(). +%% ------------------------------------------------------------------- + +-spec delete(QName, Reason) -> Ret when + QName :: rabbit_amqqueue:name(), + Reason :: atom(), + Ret :: ok | Deletions :: rabbit_binding:deletions(). + delete(QueueName, Reason) -> rabbit_db:run( #{mnesia => fun() -> delete_in_mnesia(QueueName, Reason) end }). +delete_in_mnesia(QueueName, Reason) -> + rabbit_mnesia:execute_mnesia_transaction( + fun () -> + case {mnesia:wread({?MNESIA_TABLE, QueueName}), + mnesia:wread({?MNESIA_DURABLE_TABLE, QueueName})} of + {[], []} -> + ok; + _ -> + internal_delete_in_mnesia(QueueName, false, Reason) + end + end). + +%% ------------------------------------------------------------------- +%% internal_delete(). +%% ------------------------------------------------------------------- + +-spec internal_delete(QName, OnlyDurable, Reason) -> Deletions when + QName :: rabbit_amqqueue:name(), + OnlyDurable :: boolean(), + Reason :: atom(), + Deletions :: rabbit_binding:deletions(). + internal_delete(QueueName, OnlyDurable, Reason) -> %% Only used by rabbit_amqqueue:forget_node_for_queue, which is only called - %% by `rabbit_mnesia:remove_node_if_mnesia_running`. Thus, once mnesia and/or + %% by `rabbit_mnesia:remove_node_if_mnesia_running'. Thus, once mnesia and/or %% HA queues are removed it can be removed. rabbit_db:run( #{mnesia => fun() -> internal_delete_in_mnesia(QueueName, OnlyDurable, Reason) end }). -get(Names) when is_list(Names) -> +internal_delete_in_mnesia(QueueName, OnlyDurable, Reason) -> + ok = mnesia:delete({?MNESIA_TABLE, QueueName}), + case Reason of + auto_delete -> + %% efficiency improvement when a channel with many auto-delete queues + %% is being closed + case mnesia:wread({?MNESIA_DURABLE_TABLE, QueueName}) of + [] -> ok; + [_] -> ok = mnesia:delete({?MNESIA_DURABLE_TABLE, QueueName}) + end; + _ -> + mnesia:delete({?MNESIA_DURABLE_TABLE, QueueName}) + end, + %% we want to execute some things, as decided by rabbit_exchange, + %% after the transaction. + rabbit_db_binding:delete_for_destination_in_mnesia(QueueName, OnlyDurable). + +%% ------------------------------------------------------------------- +%% get_many(). +%% ------------------------------------------------------------------- + +-spec get_many([QName]) -> Ret when + QName :: rabbit_amqqueue:name(), + Ret :: [Queue :: amqqueue:amqqueue()]. +get_many(Names) when is_list(Names) -> rabbit_db:run( - #{mnesia => fun() -> get_many_in_mnesia(rabbit_queue, Names) end - }); + #{mnesia => fun() -> get_many_in_mnesia(?MNESIA_TABLE, Names) end + }). + +get_many_in_mnesia(Table, [Name]) -> + ets:lookup(Table, Name); +get_many_in_mnesia(Table, Names) when is_list(Names) -> + %% Normally we'd call mnesia:dirty_read/1 here, but that is quite + %% expensive for reasons explained in rabbit_mnesia:dirty_read/1. + lists:append([ets:lookup(Table, Name) || Name <- Names]). + +%% ------------------------------------------------------------------- +%% get(). +%% ------------------------------------------------------------------- + +-spec get(QName) -> Ret when + QName :: rabbit_amqqueue:name(), + Ret :: {ok, Queue :: amqqueue:amqqueue()} | {error, not_found}. get(Name) -> rabbit_db:run( #{mnesia => fun() -> get_in_mnesia(Name) end }). get_in_mnesia(Name) -> - rabbit_mnesia:dirty_read({rabbit_queue, Name}). + rabbit_mnesia:dirty_read({?MNESIA_TABLE, Name}). + +%% ------------------------------------------------------------------- +%% get_durable(). +%% ------------------------------------------------------------------- + +-spec get_durable(QName) -> Ret when + QName :: rabbit_amqqueue:name(), + Ret :: {ok, Queue :: amqqueue:amqqueue()} | {error, not_found}. -get_durable(Names) when is_list(Names) -> - rabbit_db:run( - #{mnesia => fun() -> get_many_in_mnesia(rabbit_durable_queue, Names) end - }); get_durable(Name) -> rabbit_db:run( #{mnesia => fun() -> get_durable_in_mnesia(Name) end }). get_durable_in_mnesia(Name) -> - rabbit_mnesia:dirty_read({rabbit_durable_queue, Name}). + rabbit_mnesia:dirty_read({?MNESIA_DURABLE_TABLE, Name}). + +%% ------------------------------------------------------------------- +%% get_many_durable(). +%% ------------------------------------------------------------------- + +-spec get_many_durable([QName]) -> Ret when + QName :: rabbit_amqqueue:name(), + Ret :: [Queue :: amqqueue:amqqueue()]. -delete_transient(Queues) -> +get_many_durable(Names) when is_list(Names) -> rabbit_db:run( - #{mnesia => fun() -> delete_transient_in_mnesia(Queues) end + #{mnesia => fun() -> get_many_in_mnesia(?MNESIA_DURABLE_TABLE, Names) end }). -delete_transient_in_mnesia(Queues) -> - rabbit_mnesia:execute_mnesia_transaction( - fun () -> - [{QName, delete_transient_in_mnesia_tx(QName)} - || QName <- Queues] - end). +%% ------------------------------------------------------------------- +%% update(). +%% ------------------------------------------------------------------- + +-spec update(QName, UpdateFun) -> Ret when + QName :: rabbit_amqqueue:name(), + Queue :: amqqueue:amqqueue(), + UpdateFun :: fun((Queue) -> Queue), + Ret :: Queue | not_found. +%% @doc Updates an existing queue record using `UpdateFun'. +%% +%% @private -on_node_up(Node, Fun) -> +update(QName, Fun) -> rabbit_db:run( - #{mnesia => fun() -> on_node_up_in_mnesia(Node, Fun) end + #{mnesia => fun() -> update_in_mnesia(QName, Fun) end }). -on_node_up_in_mnesia(Node, Fun) -> +update_in_mnesia(QName, Fun) -> rabbit_mnesia:execute_mnesia_transaction( - fun () -> - Qs = mnesia:match_object(rabbit_queue, - amqqueue:pattern_match_all(), write), - [Fun(Node, Q) || Q <- Qs], - ok + fun() -> + update_in_mnesia_tx(QName, Fun) end). -on_node_down(Node, Fun) -> - rabbit_db:run( - #{mnesia => fun() -> on_node_down_in_mnesia(Node, Fun) end - }). - -on_node_down_in_mnesia(Node, Fun) -> - Qs = rabbit_mnesia:execute_mnesia_transaction( - fun () -> - qlc:e(qlc:q([amqqueue:get_name(Q) || Q <- mnesia:table(rabbit_queue), - Fun(Node, Q) - ])) - end), - lists:unzip(lists:flatten( - [case delete_transient(Queues) of - {error, noproc} -> []; - {error, {timeout, _}} -> []; - Value -> Value - end || Queues <- partition_queues(Qs)] - )). +%% ------------------------------------------------------------------- +%% update_decorators(). +%% ------------------------------------------------------------------- -% If there are many queues and we delete them all in a single Mnesia transaction, -% this can block all other Mnesia operations for a really long time. -% In situations where a node wants to (re-)join a cluster, -% Mnesia won't be able to sync on the new node until this operation finishes. -% As a result, we want to have multiple Mnesia transactions so that other -% operations can make progress in between these queue delete transactions. -% -% 10 queues per Mnesia transaction is an arbitrary number, but it seems to work OK with 50k queues per node. -partition_queues([Q0,Q1,Q2,Q3,Q4,Q5,Q6,Q7,Q8,Q9 | T]) -> - [[Q0,Q1,Q2,Q3,Q4,Q5,Q6,Q7,Q8,Q9] | partition_queues(T)]; -partition_queues(T) -> - [T]. +-spec update_decorators(QName) -> ok when + QName :: rabbit_amqqueue:name(). +%% @doc Updates an existing queue record adding the active queue decorators. +%% +%% @private -update(QName, Fun) -> +update_decorators(QName) -> rabbit_db:run( - #{mnesia => fun() -> update_in_mnesia(QName, Fun) end + #{mnesia => fun() -> update_decorators_in_mnesia(QName) end }). -update_in_mnesia(QName, Fun) -> +update_decorators_in_mnesia(Name) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> - update_in_mnesia_tx(QName, Fun) + case mnesia:wread({?MNESIA_TABLE, Name}) of + [Q] -> ok = mnesia:write(?MNESIA_TABLE, rabbit_queue_decorator:set(Q), + write); + [] -> ok + end end). -update_decorators(Name) -> +%% ------------------------------------------------------------------- +%% update_durable(). +%% ------------------------------------------------------------------- + +-spec update_durable(UpdateFun, FilterFun) -> ok when + UpdateFun :: fun((Queue) -> any()), + FilterFun :: fun((Queue) -> boolean()). +%% @doc Applies `UpdateFun' to all durable queue records that match `FilterFun' +%% and stores them +%% +%% @private + +update_durable(UpdateFun, FilterFun) -> rabbit_db:run( - #{mnesia => fun() -> update_decorators_in_mnesia(Name) end + #{mnesia => + fun() -> update_durable_in_mnesia(UpdateFun, FilterFun) end }). -not_found_or_absent_queue_dirty(Name) -> +update_durable_in_mnesia(UpdateFun, FilterFun) -> + Pattern = amqqueue:pattern_match_all(), + {atomic, ok} = + mnesia:sync_transaction( + fun () -> + Qs = mnesia:match_object(?MNESIA_DURABLE_TABLE, Pattern, write), + _ = [mnesia:write(?MNESIA_DURABLE_TABLE, UpdateFun(Q), write) + || Q <- Qs, FilterFun(Q)], + ok + end), + ok. + +%% ------------------------------------------------------------------- +%% exists(). +%% ------------------------------------------------------------------- + +-spec exists(QName) -> Exists when + QName :: rabbit_amqqueue:name(), + Exists :: boolean(). +%% @doc Indicates if queue named `QName' exists. +%% +%% @returns true if the queue exists, false otherwise. +%% +%% @private + +exists(QName) -> rabbit_db:run( - #{mnesia => fun() -> not_found_or_absent_queue_dirty_in_mnesia(Name) end + #{mnesia => fun() -> exists_in_mnesia(QName) end }). -exists(Name) -> +exists_in_mnesia(QName) -> + ets:member(?MNESIA_TABLE, QName). + +%% ------------------------------------------------------------------- +%% exists(). +%% ------------------------------------------------------------------- + +-spec consistent_exists(QName) -> Exists when + QName :: rabbit_amqqueue:name(), + Exists :: boolean(). +%% @doc Indicates if queue named `QName' exists using a consistent read. +%% Just used by `rabbit_classic_queue:is_recoverable` for transient queues. +%% +%% @returns true if the queue exists, false otherwise. +%% +%% @private + +consistent_exists(QName) -> rabbit_db:run( - #{mnesia => fun() -> exists_in_mnesia(Name) end + #{mnesia => fun() -> consistent_exists_in_mnesia(QName) end }). -exists_in_mnesia(Name) -> - ets:member(rabbit_queue, Name). +consistent_exists_in_mnesia(QName) -> + case mnesia:read({?MNESIA_TABLE, QName}) of + [] -> false; + [_] -> true + end. + +%% ------------------------------------------------------------------- +%% get_all_by_type(). +%% ------------------------------------------------------------------- + +-spec get_all_by_type(Type) -> [Queue] when + Type :: atom(), + Queue :: amqqueue:amqqueue(). + +%% @doc Gets all queues belonging to the given type +%% +%% @returns a list of queue records. +%% +%% @private get_all_by_type(Type) -> Pattern = amqqueue:pattern_match_on_type(Type), @@ -304,101 +527,284 @@ get_all_by_type(Type) -> }). get_all_by_pattern_in_mnesia(Pattern) -> - rabbit_db:list_in_mnesia(rabbit_queue, Pattern). + rabbit_db:list_in_mnesia(?MNESIA_TABLE, Pattern). -get_all_by_type_and_node(VHost, Type, Node) -> +%% ------------------------------------------------------------------- +%% get_all_by_type_and_node(). +%% ------------------------------------------------------------------- + +-spec get_all_by_type_and_node(VHostName, Type, Node) -> [Queue] when + VHostName :: vhost:name(), + Type :: atom(), + Node :: 'none' | atom(), + Queue :: amqqueue:amqqueue(). + +%% @doc Gets all queues belonging to the given type +%% +%% @returns a list of queue records. +%% +%% @private + +get_all_by_type_and_node(VHostName, Type, Node) -> rabbit_db:run( - #{mnesia => fun() -> get_all_by_type_and_node_in_mnesia(VHost, Type, Node) end + #{mnesia => fun() -> get_all_by_type_and_node_in_mnesia(VHostName, Type, Node) end }). -get_all_by_type_and_node_in_mnesia(VHost, Type, Node) -> +get_all_by_type_and_node_in_mnesia(VHostName, Type, Node) -> mnesia:async_dirty( fun () -> - qlc:e(qlc:q([Q || Q <- mnesia:table(rabbit_durable_queue), + qlc:e(qlc:q([Q || Q <- mnesia:table(?MNESIA_DURABLE_TABLE), amqqueue:get_type(Q) =:= Type, - amqqueue:get_vhost(Q) =:= VHost, + amqqueue:get_vhost(Q) =:= VHostName, amqqueue:qnode(Q) == Node])) end). -create_or_get(DurableQ, Q) -> +%% ------------------------------------------------------------------- +%% create_or_get(). +%% ------------------------------------------------------------------- + +-spec create_or_get(Queue) -> Ret when + Queue :: amqqueue:amqqueue(), + Ret :: {created, Queue} | {existing, Queue} | {absent, Queue, nodedown}. +%% @doc Writes a queue record if it doesn't exist already or returns the existing one +%% +%% @returns the existing record if there is one in the database already, or the newly +%% created record. +%% +%% @private + +create_or_get(Q) -> rabbit_db:run( - #{mnesia => fun() -> create_or_get_in_mnesia(DurableQ, Q) end + #{mnesia => fun() -> create_or_get_in_mnesia(Q) end }). -create_or_get_in_mnesia(DurableQ, Q) -> +create_or_get_in_mnesia(Q) -> + DurableQ = amqqueue:reset_mirroring_and_decorators(Q), QueueName = amqqueue:get_name(Q), rabbit_mnesia:execute_mnesia_transaction( fun () -> - case mnesia:wread({rabbit_queue, QueueName}) of + case mnesia:wread({?MNESIA_TABLE, QueueName}) of [] -> - case not_found_or_absent_queue_in_mnesia(QueueName) of - not_found -> - insert_in_mnesia_tx(DurableQ, Q), + case get_durable_in_mnesia_tx(QueueName) of + {error, not_found} -> + set_in_mnesia_tx(DurableQ, Q), {created, Q}; - {absent, _Q, _} = R -> - R + {ok, Q} -> + {absent, Q, nodedown} end; [ExistingQ] -> {existing, ExistingQ} end end). -insert(DurableQ, Q) -> +%% ------------------------------------------------------------------- +%% set(). +%% ------------------------------------------------------------------- + +-spec set(Queue) -> ok when + Queue :: amqqueue:amqqueue(). +%% @doc Writes a queue record. If the queue is durable, it writes both instances: +%% durable and transient. For the durable one, it resets mirrors and decorators. +%% The transient one is left as it is. +%% +%% @private + +set(Q) -> rabbit_db:run( - #{mnesia => fun() -> insert_in_mnesia(DurableQ, Q) end + #{mnesia => fun() -> set_in_mnesia(Q) end }). -insert_in_mnesia(DurableQ, Q) -> +set_in_mnesia(Q) -> + DurableQ = amqqueue:reset_mirroring_and_decorators(Q), rabbit_mnesia:execute_mnesia_transaction( fun () -> - insert_in_mnesia_tx(DurableQ, Q) + set_in_mnesia_tx(DurableQ, Q) end). -insert(Qs) -> +set_in_mnesia_tx(DurableQ, Q) -> + case ?amqqueue_is_durable(Q) of + true -> + ok = mnesia:write(?MNESIA_DURABLE_TABLE, DurableQ, write); + false -> + ok + end, + ok = mnesia:write(?MNESIA_TABLE, Q, write). + +%% ------------------------------------------------------------------- +%% set_many(). +%% ------------------------------------------------------------------- + +-spec set_many([Queue]) -> ok when + Queue :: amqqueue:amqqueue(). +%% @doc Writes a list of durable queue records. +%% It is responsibility of the calling function to ensure all records are durable. +%% Once transient entities are deprecated, this is a non-issue. +%% +%% @private + +set_many(Qs) -> rabbit_db:run( - #{mnesia => fun() -> insert_many_in_mnesia(Qs) end + #{mnesia => fun() -> set_many_in_mnesia(Qs) end }). -insert_many_in_mnesia(Qs) -> +set_many_in_mnesia(Qs) -> + {atomic, ok} = + %% Just to be nested in forget_node_for_queue + mnesia:transaction( + fun() -> + [ok = mnesia:write(?MNESIA_DURABLE_TABLE, Q, write) || Q <- Qs], + ok + end), + ok. + +%% ------------------------------------------------------------------- +%% delete_transient(). +%% ------------------------------------------------------------------- + +-spec delete_transient(FilterFun) -> Ret when + Queue :: amqqueue:amqqueue(), + FilterFun :: fun((Queue) -> boolean()), + QName :: rabbit_amqqueue:name(), + Ret :: {[QName], [Deletions :: rabbit_binding:deletions()]}. +%% @doc Deletes all transient queues that match `FilterFun'. +%% +%% @private + +delete_transient(FilterFun) -> + rabbit_db:run( + #{mnesia => fun() -> delete_transient_in_mnesia(FilterFun) end + }). + +delete_transient_in_mnesia(FilterFun) -> + Qs = rabbit_mnesia:execute_mnesia_transaction( + fun () -> + qlc:e(qlc:q([amqqueue:get_name(Q) || Q <- mnesia:table(?MNESIA_TABLE), + FilterFun(Q) + ])) + end), + lists:unzip(lists:flatten( + [delete_many_transient_in_mnesia(Queues) || Queues <- partition_queues(Qs)] + )). + +-spec delete_many_transient_in_mnesia([QName]) -> Ret when + QName :: rabbit_amqqueue:name(), + Ret :: [{QName, Deletions :: rabbit_binding:deletions()}]. + +delete_many_transient_in_mnesia(Queues) -> rabbit_mnesia:execute_mnesia_transaction( - fun() -> - [ok = mnesia:write(rabbit_durable_queue, Q, write) || Q <- Qs] + fun () -> + [{QName, delete_transient_in_mnesia_tx(QName)} + || QName <- Queues] + end). + +delete_transient_in_mnesia_tx(QName) -> + ok = mnesia:delete({?MNESIA_TABLE, QName}), + rabbit_db_binding:delete_transient_for_destination_in_mnesia(QName). + +% If there are many queues and we delete them all in a single Mnesia transaction, +% this can block all other Mnesia operations for a really long time. +% In situations where a node wants to (re-)join a cluster, +% Mnesia won't be able to sync on the new node until this operation finishes. +% As a result, we want to have multiple Mnesia transactions so that other +% operations can make progress in between these queue delete transactions. +% +% 10 queues per Mnesia transaction is an arbitrary number, but it seems to work OK with 50k queues per node. +partition_queues([Q0,Q1,Q2,Q3,Q4,Q5,Q6,Q7,Q8,Q9 | T]) -> + [[Q0,Q1,Q2,Q3,Q4,Q5,Q6,Q7,Q8,Q9] | partition_queues(T)]; +partition_queues(T) -> + [T]. + +%% ------------------------------------------------------------------- +%% foreach_transient(). +%% ------------------------------------------------------------------- + +-spec foreach_transient(UpdateFun) -> ok when + Queue :: amqqueue:amqqueue(), + UpdateFun :: fun((Queue) -> any()). +%% @doc Applies `UpdateFun' to all transient queue records. +%% +%% @private + +foreach_transient(UpdateFun) -> + rabbit_db:run( + #{mnesia => fun() -> foreach_transient_in_mnesia(UpdateFun) end + }). + +foreach_transient_in_mnesia(UpdateFun) -> + Pattern = amqqueue:pattern_match_all(), + rabbit_mnesia:execute_mnesia_transaction( + fun () -> + Qs = mnesia:match_object(?MNESIA_TABLE, Pattern, write), + _ = [UpdateFun(Q) || Q <- Qs], + ok end). -match_and_update(Pattern, UpdateFun, FilterFun) -> +%% ------------------------------------------------------------------- +%% foreach_durable(). +%% ------------------------------------------------------------------- + +-spec foreach_durable(UpdateFun, FilterFun) -> ok when + UpdateFun :: fun((Queue) -> any()), + FilterFun :: fun((Queue) -> boolean()). +%% @doc Applies `UpdateFun' to all durable queue records that match `FilterFun'. +%% +%% @private + +foreach_durable(UpdateFun, FilterFun) -> rabbit_db:run( #{mnesia => - fun() -> match_and_update_in_mnesia(Pattern, UpdateFun, FilterFun) end + fun() -> foreach_durable_in_mnesia(UpdateFun, FilterFun) end }). -match_and_update_in_mnesia(Pattern, UpdateFun, FilterFun) -> +foreach_durable_in_mnesia(UpdateFun, FilterFun) -> %% Note rabbit is not running so we avoid e.g. the worker pool. Also why %% we don't invoke the return from rabbit_binding:process_deletions/1. + Pattern = amqqueue:pattern_match_all(), {atomic, ok} = mnesia:sync_transaction( fun () -> - Qs = mnesia:match_object(rabbit_durable_queue, Pattern, write), + Qs = mnesia:match_object(?MNESIA_DURABLE_TABLE, Pattern, write), _ = [UpdateFun(Q) || Q <- Qs, FilterFun(Q)], ok end), ok. -insert_dirty(Q) -> +%% ------------------------------------------------------------------- +%% set_dirty(). +%% ------------------------------------------------------------------- + +-spec set_dirty(Queue) -> ok when + Queue :: amqqueue:amqqueue(). +%% @doc Writes a transient queue record +%% +%% @private + +set_dirty(Q) -> rabbit_db:run( - #{mnesia => fun() -> insert_dirty_in_mnesia(Q) end + #{mnesia => fun() -> set_dirty_in_mnesia(Q) end }). -insert_dirty_in_mnesia(Q) -> - ok = mnesia:dirty_write(rabbit_queue, rabbit_queue_decorator:set(Q)). +set_dirty_in_mnesia(Q) -> + ok = mnesia:dirty_write(?MNESIA_TABLE, rabbit_queue_decorator:set(Q)). + +%% ------------------------------------------------------------------- +%% update_in_mnesia_tx(). +%% ------------------------------------------------------------------- + +-spec update_in_mnesia_tx(QName, UpdateFun) -> Ret when + QName :: rabbit_amqqueue:name(), + Queue :: amqqueue:amqqueue(), + UpdateFun :: fun((Queue) -> Queue), + Ret :: Queue | not_found. update_in_mnesia_tx(Name, Fun) -> - case mnesia:wread({rabbit_queue, Name}) of + case mnesia:wread({?MNESIA_TABLE, Name}) of [Q] -> Durable = amqqueue:is_durable(Q), Q1 = Fun(Q), - ok = mnesia:write(rabbit_queue, Q1, write), + ok = mnesia:write(?MNESIA_TABLE, Q1, write), case Durable of - true -> ok = mnesia:write(rabbit_durable_queue, Q1, write); + true -> ok = mnesia:write(?MNESIA_DURABLE_TABLE, Q1, write); _ -> ok end, Q1; @@ -406,54 +812,52 @@ update_in_mnesia_tx(Name, Fun) -> not_found end. -not_found_or_absent_queue_in_mnesia(Name) -> - %% NB: we assume that the caller has already performed a lookup on - %% rabbit_queue and not found anything - case mnesia:read({rabbit_durable_queue, Name}) of - [] -> not_found; - [Q] -> {absent, Q, nodedown} %% Q exists on stopped node +%% ------------------------------------------------------------------- +%% get_durable_in_mnesia_tx(). +%% ------------------------------------------------------------------- + +-spec get_durable_in_mnesia_tx(QName) -> Ret when + QName :: rabbit_amqqueue:name(), + Ret :: {ok, Queue :: amqqueue:amqqueue()} | {error, not_found}. + +get_durable_in_mnesia_tx(Name) -> + case mnesia:read({?MNESIA_DURABLE_TABLE, Name}) of + [] -> {error, not_found}; + [Q] -> {ok, Q} end. -%% Internal -%% -------------------------------------------------------------- -get_many_in_mnesia(Table, [Name]) -> - ets:lookup(Table, Name); -get_many_in_mnesia(Table, Names) when is_list(Names) -> - %% Normally we'd call mnesia:dirty_read/1 here, but that is quite - %% expensive for reasons explained in rabbit_mnesia:dirty_read/1. - lists:append([ets:lookup(Table, Name) || Name <- Names]). +%% ------------------------------------------------------------------- +%% clear(). +%% ------------------------------------------------------------------- -delete_transient_in_mnesia_tx(QName) -> - ok = mnesia:delete({rabbit_queue, QName}), - rabbit_db_binding:delete_transient_for_destination_in_mnesia(QName). +-spec clear() -> ok. +%% @doc Deletes all queues. +%% +%% @private -get_all_in_mnesia(VHost) -> - list_with_possible_retry_in_mnesia( - fun() -> - Pattern = amqqueue:pattern_match_on_name(rabbit_misc:r(VHost, queue)), - rabbit_db:list_in_mnesia(rabbit_queue, Pattern) - end). +clear() -> + rabbit_db:run( + #{mnesia => fun() -> clear_in_mnesia() end}). -not_found_or_absent_queue_dirty_in_mnesia(Name) -> - %% We should read from both tables inside a tx, to get a - %% consistent view. But the chances of an inconsistency are small, - %% and only affect the error kind. - case rabbit_mnesia:dirty_read({rabbit_durable_queue, Name}) of - {error, not_found} -> not_found; - {ok, Q} -> {absent, Q, nodedown} - end. +clear_in_mnesia() -> + {atomic, ok} = mnesia:clear_table(?MNESIA_TABLE), + {atomic, ok} = mnesia:clear_table(?MNESIA_DURABLE_TABLE), + ok. + +%% Internal +%% -------------------------------------------------------------- list_with_possible_retry_in_mnesia(Fun) -> %% amqqueue migration: - %% The `rabbit_queue` or `rabbit_durable_queue` tables + %% The `rabbit_queue' or `rabbit_durable_queue' tables %% might be migrated between the time we query the pattern - %% (with the `amqqueue` module) and the time we call - %% `mnesia:dirty_match_object()`. This would lead to an empty list + %% (with the `amqqueue' module) and the time we call + %% `mnesia:dirty_match_object()'. This would lead to an empty list %% (no object matching the now incorrect pattern), not a Mnesia %% error. %% %% So if the result is an empty list and the version of the - %% `amqqueue` record changed in between, we retry the operation. + %% `amqqueue' record changed in between, we retry the operation. %% %% However, we don't do this if inside a Mnesia transaction: we %% could end up with a live lock between this started transaction @@ -474,67 +878,3 @@ list_with_possible_retry_in_mnesia(Fun) -> Ret -> Ret end. - -delete_in_mnesia(QueueName, Reason) -> - rabbit_mnesia:execute_mnesia_transaction( - fun () -> - case {mnesia:wread({rabbit_queue, QueueName}), - mnesia:wread({rabbit_durable_queue, QueueName})} of - {[], []} -> - ok; - _ -> - internal_delete_in_mnesia(QueueName, false, Reason) - end - end). - -internal_delete_in_mnesia(QueueName, OnlyDurable, Reason) -> - ok = mnesia:delete({rabbit_queue, QueueName}), - case Reason of - auto_delete -> - case mnesia:wread({rabbit_durable_queue, QueueName}) of - [] -> ok; - [_] -> ok = mnesia:delete({rabbit_durable_queue, QueueName}) - end; - _ -> - mnesia:delete({rabbit_durable_queue, QueueName}) - end, - %% we want to execute some things, as decided by rabbit_exchange, - %% after the transaction. - rabbit_db_binding:delete_for_destination_in_mnesia(QueueName, OnlyDurable). - -list_for_count(VHost) -> - rabbit_db:run( - #{mnesia => fun() -> list_for_count_in_mnesia(VHost) end - }). - -list_for_count_in_mnesia(VHost) -> - %% this is certainly suboptimal but there is no way to count - %% things using a secondary index in Mnesia. Our counter-table-per-node - %% won't work here because with master migration of mirrored queues - %% the "ownership" of queues by nodes becomes a non-trivial problem - %% that requires a proper consensus algorithm. - list_with_possible_retry_in_mnesia( - fun() -> - length(mnesia:dirty_index_read(rabbit_queue, - VHost, - amqqueue:field_vhost())) - end). - -update_decorators_in_mnesia(Name) -> - rabbit_mnesia:execute_mnesia_transaction( - fun() -> - case mnesia:wread({rabbit_queue, Name}) of - [Q] -> ok = mnesia:write(rabbit_queue, rabbit_queue_decorator:set(Q), - write); - [] -> ok - end - end). - -insert_in_mnesia_tx(DurableQ, Q) -> - case ?amqqueue_is_durable(Q) of - true -> - ok = mnesia:write(rabbit_durable_queue, DurableQ, write); - false -> - ok - end, - ok = mnesia:write(rabbit_queue, Q, write). diff --git a/deps/rabbit/src/rabbit_db_topic_exchange.erl b/deps/rabbit/src/rabbit_db_topic_exchange.erl index 89acc9d266f0..5c0f0283ef75 100644 --- a/deps/rabbit/src/rabbit_db_topic_exchange.erl +++ b/deps/rabbit/src/rabbit_db_topic_exchange.erl @@ -9,21 +9,28 @@ -include_lib("rabbit_common/include/rabbit.hrl"). --export([insert/1, delete_all_for_exchange/1, delete/1, match/2]). +-export([set/1, delete_all_for_exchange/1, delete/1, match/2]). + +%% For testing +-export([clear/0]). + +-define(MNESIA_NODE_TABLE, rabbit_topic_trie_node). +-define(MNESIA_EDGE_TABLE, rabbit_topic_trie_edge). +-define(MNESIA_BINDING_TABLE, rabbit_topic_trie_binding). %% ------------------------------------------------------------------- -%% insert(). +%% set(). %% ------------------------------------------------------------------- --spec insert(Binding) -> ok when +-spec set(Binding) -> ok when Binding :: rabbit_types:binding(). -%% @doc Inserts a topic binding. +%% @doc Sets a topic binding. %% %% @private -insert(#binding{source = XName, key = RoutingKey, destination = Destination, args = Args}) -> +set(#binding{source = XName, key = RoutingKey, destination = Destination, args = Args}) -> rabbit_db:run( - #{mnesia => fun() -> insert_in_mnesia(XName, RoutingKey, Destination, Args) end + #{mnesia => fun() -> set_in_mnesia(XName, RoutingKey, Destination, Args) end }). %% ------------------------------------------------------------------- @@ -31,7 +38,7 @@ insert(#binding{source = XName, key = RoutingKey, destination = Destination, arg %% ------------------------------------------------------------------- -spec delete_all_for_exchange(ExchangeName) -> ok when - ExchangeName :: rabbit_types:r('exchange'). + ExchangeName :: rabbit_exchange:name(). %% @doc Deletes all topic bindings for the exchange named `ExchangeName' %% %% @private @@ -61,7 +68,7 @@ delete(Bs) when is_list(Bs) -> %% ------------------------------------------------------------------- -spec match(ExchangeName, RoutingKey) -> ok when - ExchangeName :: rabbit_types:r('exchange'), + ExchangeName :: rabbit_exchange:name(), RoutingKey :: binary(). %% @doc Finds the topic binding matching the given exchange and routing key and returns %% the destination of the binding @@ -78,13 +85,33 @@ match(XName, RoutingKey) -> end }). +%% ------------------------------------------------------------------- +%% clear(). +%% ------------------------------------------------------------------- + +-spec clear() -> ok. +%% @doc Deletes all topic bindings +%% +%% @private + +clear() -> + rabbit_db:run( + #{mnesia => fun() -> clear_in_mnesia() end + }). + +clear_in_mnesia() -> + {atomic, ok} = mnesia:clear_table(?MNESIA_NODE_TABLE), + {atomic, ok} = mnesia:clear_table(?MNESIA_EDGE_TABLE), + {atomic, ok} = mnesia:clear_table(?MNESIA_BINDING_TABLE), + ok. + %% Internal %% -------------------------------------------------------------- split_topic_key(Key) -> split_topic_key(Key, [], []). -insert_in_mnesia(XName, RoutingKey, Destination, Args) -> +set_in_mnesia(XName, RoutingKey, Destination, Args) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> FinalNode = follow_down_create(XName, split_topic_key(RoutingKey)), @@ -106,19 +133,19 @@ match_in_mnesia(XName, RoutingKey) -> mnesia:async_dirty(fun trie_match/2, [XName, Words]). trie_remove_all_nodes(X) -> - remove_all(rabbit_topic_trie_node, + remove_all(?MNESIA_NODE_TABLE, #topic_trie_node{trie_node = #trie_node{exchange_name = X, _ = '_'}, _ = '_'}). trie_remove_all_edges(X) -> - remove_all(rabbit_topic_trie_edge, + remove_all(?MNESIA_EDGE_TABLE, #topic_trie_edge{trie_edge = #trie_edge{exchange_name = X, _ = '_'}, _ = '_'}). trie_remove_all_bindings(X) -> - remove_all(rabbit_topic_trie_binding, + remove_all(?MNESIA_BINDING_TABLE, #topic_trie_binding{ trie_binding = #trie_binding{exchange_name = X, _ = '_'}, _ = '_'}). @@ -131,12 +158,12 @@ delete_in_mnesia_tx(Bs) -> %% See rabbit_binding:lock_route_tables for the rationale for %% taking table locks. _ = case Bs of - [_] -> ok; - _ -> [mnesia:lock({table, T}, write) || - T <- [rabbit_topic_trie_node, - rabbit_topic_trie_edge, - rabbit_topic_trie_binding]] - end, + [_] -> ok; + _ -> [mnesia:lock({table, T}, write) || + T <- [?MNESIA_NODE_TABLE, + ?MNESIA_EDGE_TABLE, + ?MNESIA_BINDING_TABLE]] + end, [case follow_down_get_path(X, split_topic_key(K)) of {ok, Path = [{FinalNode, _} | _]} -> trie_remove_binding(X, FinalNode, D, Args), @@ -222,7 +249,7 @@ follow_down(X, CurNode, AccFun, Acc, Words = [W | RestW]) -> remove_path_if_empty(_, [{root, none}]) -> ok; remove_path_if_empty(X, [{Node, W} | [{Parent, _} | _] = RestPath]) -> - case mnesia:read(rabbit_topic_trie_node, + case mnesia:read(?MNESIA_NODE_TABLE, #trie_node{exchange_name = X, node_id = Node}, write) of [] -> trie_remove_edge(X, Parent, Node, W), remove_path_if_empty(X, RestPath); @@ -230,7 +257,7 @@ remove_path_if_empty(X, [{Node, W} | [{Parent, _} | _] = RestPath]) -> end. trie_child(X, Node, Word) -> - case mnesia:read({rabbit_topic_trie_edge, + case mnesia:read({?MNESIA_EDGE_TABLE, #trie_edge{exchange_name = X, node_id = Node, word = Word}}) of @@ -244,10 +271,10 @@ trie_bindings(X, Node) -> node_id = Node, destination = '$1', arguments = '_'}}, - mnesia:select(rabbit_topic_trie_binding, [{MatchHead, [], ['$1']}]). + mnesia:select(?MNESIA_BINDING_TABLE, [{MatchHead, [], ['$1']}]). trie_update_node_counts(X, Node, Field, Delta) -> - E = case mnesia:read(rabbit_topic_trie_node, + E = case mnesia:read(?MNESIA_NODE_TABLE, #trie_node{exchange_name = X, node_id = Node}, write) of [] -> #topic_trie_node{trie_node = #trie_node{ @@ -259,9 +286,9 @@ trie_update_node_counts(X, Node, Field, Delta) -> end, case setelement(Field, E, element(Field, E) + Delta) of #topic_trie_node{edge_count = 0, binding_count = 0} -> - ok = mnesia:delete_object(rabbit_topic_trie_node, E, write); + ok = mnesia:delete_object(?MNESIA_NODE_TABLE, E, write); EN -> - ok = mnesia:write(rabbit_topic_trie_node, EN, write) + ok = mnesia:write(?MNESIA_NODE_TABLE, EN, write) end. trie_add_edge(X, FromNode, ToNode, W) -> @@ -273,7 +300,7 @@ trie_remove_edge(X, FromNode, ToNode, W) -> trie_edge_op(X, FromNode, ToNode, W, fun mnesia:delete_object/3). trie_edge_op(X, FromNode, ToNode, W, Op) -> - ok = Op(rabbit_topic_trie_edge, + ok = Op(?MNESIA_EDGE_TABLE, #topic_trie_edge{trie_edge = #trie_edge{exchange_name = X, node_id = FromNode, word = W}, @@ -289,7 +316,7 @@ trie_remove_binding(X, Node, D, Args) -> trie_binding_op(X, Node, D, Args, fun mnesia:delete_object/3). trie_binding_op(X, Node, D, Args, Op) -> - ok = Op(rabbit_topic_trie_binding, + ok = Op(?MNESIA_BINDING_TABLE, #topic_trie_binding{ trie_binding = #trie_binding{exchange_name = X, node_id = Node, diff --git a/deps/rabbit/src/rabbit_db_user.erl b/deps/rabbit/src/rabbit_db_user.erl index ba2adaf8131f..8b59512af87c 100644 --- a/deps/rabbit/src/rabbit_db_user.erl +++ b/deps/rabbit/src/rabbit_db_user.erl @@ -28,6 +28,8 @@ clear_matching_topic_permissions/3, delete/1]). +-export([clear/0]). + -define(MNESIA_TABLE, rabbit_user). -define(PERM_MNESIA_TABLE, rabbit_user_permission). -define(TOPIC_PERM_MNESIA_TABLE, rabbit_topic_permission). @@ -644,3 +646,22 @@ topic_permission_pattern(Username, VHostName, ExchangeName) -> virtual_host = VHostName}, exchange = ExchangeName}, permission = '_'}. + +%% ------------------------------------------------------------------- +%% clear(). +%% ------------------------------------------------------------------- + +-spec clear() -> ok. +%% @doc Deletes all users and permissions. +%% +%% @private + +clear() -> + rabbit_db:run( + #{mnesia => fun() -> clear_in_mnesia() end}). + +clear_in_mnesia() -> + {atomic, ok} = mnesia:clear_table(?MNESIA_TABLE), + {atomic, ok} = mnesia:clear_table(?PERM_MNESIA_TABLE), + {atomic, ok} = mnesia:clear_table(?TOPIC_PERM_MNESIA_TABLE), + ok. diff --git a/deps/rabbit/src/rabbit_db_vhost.erl b/deps/rabbit/src/rabbit_db_vhost.erl index 1a01860a39c1..1d35e22c3088 100644 --- a/deps/rabbit/src/rabbit_db_vhost.erl +++ b/deps/rabbit/src/rabbit_db_vhost.erl @@ -24,6 +24,8 @@ with_fun_in_mnesia_tx/2, delete/1]). +-export([clear/0]). + -define(MNESIA_TABLE, rabbit_vhost). %% ------------------------------------------------------------------- @@ -319,3 +321,20 @@ delete_in_mnesia_tx(VHostName) -> Existed = mnesia:wread({?MNESIA_TABLE, VHostName}) =/= [], mnesia:delete({?MNESIA_TABLE, VHostName}), Existed. + +%% ------------------------------------------------------------------- +%% clear(). +%% ------------------------------------------------------------------- + +-spec clear() -> ok. +%% @doc Deletes all vhosts. +%% +%% @private + +clear() -> + rabbit_db:run( + #{mnesia => fun() -> clear_in_mnesia() end}). + +clear_in_mnesia() -> + {atomic, ok} = mnesia:clear_table(?MNESIA_TABLE), + ok. diff --git a/deps/rabbit/src/rabbit_dead_letter.erl b/deps/rabbit/src/rabbit_dead_letter.erl index 22c633bb5925..7212ed32dfd1 100644 --- a/deps/rabbit/src/rabbit_dead_letter.erl +++ b/deps/rabbit/src/rabbit_dead_letter.erl @@ -29,7 +29,7 @@ publish(Msg, Reason, X, RK, SourceQName) -> {QNames, Cycles} = detect_cycles(Reason, DLMsg, rabbit_exchange:route(X, Delivery)), lists:foreach(fun log_cycle_once/1, Cycles), - Qs0 = rabbit_amqqueue:lookup(QNames), + Qs0 = rabbit_amqqueue:lookup_many(QNames), Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0), _ = rabbit_queue_type:deliver(Qs, Delivery, stateless), ok. diff --git a/deps/rabbit/src/rabbit_exchange.erl b/deps/rabbit/src/rabbit_exchange.erl index bcc04c99f45b..db7058cd68c8 100644 --- a/deps/rabbit/src/rabbit_exchange.erl +++ b/deps/rabbit/src/rabbit_exchange.erl @@ -17,13 +17,11 @@ route/2, delete/3, validate_binding/2, count/0]). -export([list_names/0]). -export([serialise_events/1]). -%% these must be run inside a mnesia tx -export([serial/1, peek_serial/1]). %%---------------------------------------------------------------------------- -export_type([name/0, type/0]). - -type name() :: rabbit_types:r('exchange'). -type type() :: atom(). -type fun_name() :: atom(). @@ -79,13 +77,12 @@ serialise_events(X = #exchange{type = Type, decorators = Decorators}) -> rabbit_exchange_decorator:select(all, Decorators)) orelse (type_to_module(Type)):serialise_events(). --spec serial(rabbit_types:exchange()) -> - fun((boolean()) -> 'none' | pos_integer()). +-spec serial(rabbit_types:exchange()) -> 'none' | pos_integer(). serial(X) -> case serialise_events(X) of false -> 'none'; - true -> rabbit_db_exchange:next_serial(X) + true -> rabbit_db_exchange:next_serial(X#exchange.name) end. -spec declare @@ -215,7 +212,7 @@ list() -> -spec count() -> non_neg_integer(). count() -> - rabbit_db_exchange:get_all(). + rabbit_db_exchange:count(). -spec list_names() -> [rabbit_exchange:name()]. @@ -254,8 +251,7 @@ update_scratch(Name, App, Fun) -> {ok, X} -> rabbit_exchange_decorator:active(X); {error, not_found} -> [] end, - rabbit_db_exchange:update(Name, update_scratch_fun(App, Fun, Decorators)), - ok. + ok = rabbit_db_exchange:update(Name, update_scratch_fun(App, Fun, Decorators)). update_scratch_fun(App, Fun, Decorators) -> fun(X = #exchange{scratches = Scratches0}) -> @@ -272,11 +268,11 @@ update_scratch_fun(App, Fun, Decorators) -> decorators = Decorators} end. --spec update_decorators(name(), [atom()] | none | undefined) -> 'ok'. - +-spec update_decorators(name(), {[Decorator], [Decorator]}) -> 'ok' when + Decorator :: atom(). update_decorators(Name, Decorators) -> Fun = fun(X) -> X#exchange{decorators = Decorators} end, - rabbit_db_exchange:update(Name, Fun). + ok = rabbit_db_exchange:update(Name, Fun). -spec immutable(rabbit_types:exchange()) -> rabbit_types:exchange(). @@ -444,9 +440,7 @@ process_deletions({error, _} = E) -> process_deletions({deleted, #exchange{name = XName} = X, Bs, Deletions}) -> rabbit_binding:process_deletions( rabbit_binding:add_deletion( - XName, {X, deleted, Bs}, Deletions)); -process_deletions(Deletions) -> - rabbit_binding:process_deletions(Deletions). + XName, {X, deleted, Bs}, Deletions)). -spec validate_binding (rabbit_types:exchange(), rabbit_types:binding()) diff --git a/deps/rabbit/src/rabbit_exchange_type_topic.erl b/deps/rabbit/src/rabbit_exchange_type_topic.erl index 8ed3f7e1b508..ee94022b5f55 100644 --- a/deps/rabbit/src/rabbit_exchange_type_topic.erl +++ b/deps/rabbit/src/rabbit_exchange_type_topic.erl @@ -49,7 +49,7 @@ delete(_Serial, #exchange{name = X}) -> policy_changed(_X1, _X2) -> ok. add_binding(_Serial, _Exchange, Binding) -> - rabbit_db_topic_exchange:insert(Binding). + rabbit_db_topic_exchange:set(Binding). remove_bindings(_Serial, _X, Bs) -> rabbit_db_topic_exchange:delete(Bs). diff --git a/deps/rabbit/src/rabbit_fifo_dlx_worker.erl b/deps/rabbit/src/rabbit_fifo_dlx_worker.erl index 3dbbfa79c5ad..90c0b0996e1f 100644 --- a/deps/rabbit/src/rabbit_fifo_dlx_worker.erl +++ b/deps/rabbit/src/rabbit_fifo_dlx_worker.erl @@ -317,7 +317,7 @@ forward(ConsumedMsg, ConsumedMsgId, ConsumedQRef, DLX, Reason, RouteToQs0 = rabbit_exchange:route(DLX, Delivery), {RouteToQs1, Cycles} = rabbit_dead_letter:detect_cycles(Reason, Msg, RouteToQs0), State1 = log_cycles(Cycles, RKeys, State0), - RouteToQs2 = rabbit_amqqueue:lookup(RouteToQs1), + RouteToQs2 = rabbit_amqqueue:lookup_many(RouteToQs1), RouteToQs = rabbit_amqqueue:prepend_extra_bcc(RouteToQs2), State2 = case RouteToQs of [] -> @@ -469,7 +469,7 @@ redeliver0(#pending{delivery = #delivery{message = BasicMsg} = Delivery0, %% queues that do not exist. Therefore, filter out non-existent target queues. RouteToQs0 = queue_names( rabbit_amqqueue:prepend_extra_bcc( - rabbit_amqqueue:lookup( + rabbit_amqqueue:lookup_many( rabbit_exchange:route(DLX, Delivery)))), case {RouteToQs0, Settled} of {[], [_|_]} -> @@ -501,7 +501,7 @@ redeliver0(#pending{delivery = #delivery{message = BasicMsg} = Delivery0, %% to be routed to is moved back to 'unsettled'. rejected = []}, State = State0#state{pendings = maps:update(OutSeq, Pend, Pendings)}, - deliver_to_queues(Delivery, rabbit_amqqueue:lookup(RouteToQs), State) + deliver_to_queues(Delivery, rabbit_amqqueue:lookup_many(RouteToQs), State) end end. diff --git a/deps/rabbit/src/rabbit_maintenance.erl b/deps/rabbit/src/rabbit_maintenance.erl index 567ba3bb5af2..aceb3ec181e4 100644 --- a/deps/rabbit/src/rabbit_maintenance.erl +++ b/deps/rabbit/src/rabbit_maintenance.erl @@ -130,11 +130,21 @@ is_being_drained_consistent_read(Node) -> -spec status_local_read(node()) -> maintenance_status(). status_local_read(Node) -> - rabbit_db_maintenance:get(Node). + case rabbit_db_maintenance:get(Node) of + undefined -> + ?DEFAULT_STATUS; + Status -> + Status + end. -spec status_consistent_read(node()) -> maintenance_status(). status_consistent_read(Node) -> - rabbit_db_maintenance:get_consistent(Node). + case rabbit_db_maintenance:get_consistent(Node) of + undefined -> + ?DEFAULT_STATUS; + Status -> + Status + end. -spec filter_out_drained_nodes_local_read([node()]) -> [node()]. filter_out_drained_nodes_local_read(Nodes) -> diff --git a/deps/rabbit/src/rabbit_policy.erl b/deps/rabbit/src/rabbit_policy.erl index c09fd5b5a664..bdb96468d903 100644 --- a/deps/rabbit/src/rabbit_policy.erl +++ b/deps/rabbit/src/rabbit_policy.erl @@ -269,7 +269,7 @@ recover0() -> operator_policy = match(Name, OpPolicies)}) || X = #exchange{name = Name} <- Xs0], Qs = rabbit_amqqueue:list_durable(), - _ = rabbit_db_exchange:insert(Xs), + _ = rabbit_db_exchange:set(Xs), Qs0 = [begin QName = amqqueue:get_name(Q0), Policy1 = match(QName, Policies), @@ -278,7 +278,9 @@ recover0() -> Q2 = amqqueue:set_operator_policy(Q1, OpPolicy1), rabbit_queue_decorator:set(Q2) end || Q0 <- Qs], - _ = rabbit_db_queue:insert(Qs0), + %% This function is just used to recover policies, thus no transient entities + %% are considered for this process as there is none to recover on boot. + _ = rabbit_db_queue:set_many(Qs0), ok. invalid_file() -> @@ -493,8 +495,6 @@ update_queue(Policy, OpPolicy, Decorators) -> amqqueue:set_decorators(Queue3, Decorators) end. -maybe_notify_of_policy_change(no_change, _PolicyDef, _ActingUser)-> - ok; maybe_notify_of_policy_change({X1 = #exchange{}, X2 = #exchange{}}, _PolicyDef, _ActingUser) -> rabbit_exchange:policy_changed(X1, X2); %% policy has been cleared diff --git a/deps/rabbit/src/rabbit_priority_queue.erl b/deps/rabbit/src/rabbit_priority_queue.erl index 3747adc719fb..382b513bec22 100644 --- a/deps/rabbit/src/rabbit_priority_queue.erl +++ b/deps/rabbit/src/rabbit_priority_queue.erl @@ -106,7 +106,7 @@ mutate_name_bin(P, NameBin) -> <>. expand_queues(QNames) -> - Qs = rabbit_db_queue:get_durable(QNames), + Qs = rabbit_db_queue:get_many_durable(QNames), lists:unzip(lists:append([expand_queue(Q) || Q <- Qs])). expand_queue(Q) -> diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 962c1decbb5b..75d7299a5b1b 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -326,7 +326,7 @@ become_leader(QName, Name) -> %% we need to ensure there is no chance of blocking as else the ra node %% may not be able to establish its leadership spawn(fun() -> - rabbit_amqqueue:update(QName, Fun), + _ = rabbit_amqqueue:update(QName, Fun), case rabbit_amqqueue:lookup(QName) of {ok, Q0} when ?is_amqqueue(Q0) -> Nodes = get_nodes(Q0), @@ -538,7 +538,7 @@ repair_amqqueue_nodes(Q0) -> TS = TS0#{nodes => RaNodes}, amqqueue:set_type_state(Q, TS) end, - rabbit_amqqueue:update(QName, Fun), + _ = rabbit_amqqueue:update(QName, Fun), repaired end. @@ -600,7 +600,7 @@ recover(_Vhost, Queues) -> %% present in the rabbit_queue table and not just in %% rabbit_durable_queue %% So many code paths are dependent on this. - ok = rabbit_db_queue:insert_dirty(Q0), + ok = rabbit_db_queue:set_dirty(Q0), Q = Q0, case Res of ok -> @@ -1099,7 +1099,7 @@ add_member(Q, Node, Timeout) when ?amqqueue_is_quorum(Q) -> end), amqqueue:set_pid(Q2, Leader) end, - rabbit_amqqueue:update(QName, Fun), + _ = rabbit_amqqueue:update(QName, Fun), ok; {timeout, _} -> _ = ra:force_delete_server(?RA_SYSTEM, ServerId), @@ -1153,7 +1153,7 @@ delete_member(Q, Node) when ?amqqueue_is_quorum(Q) -> Ts#{nodes => lists:delete(Node, Nodes)} end) end, - rabbit_amqqueue:update(QName, Fun), + _ = rabbit_amqqueue:update(QName, Fun), case ra:force_delete_server(?RA_SYSTEM, ServerId) of ok -> ok; diff --git a/deps/rabbit/src/rabbit_router.erl b/deps/rabbit/src/rabbit_router.erl index be71476b6d6c..b894cfa3a10e 100644 --- a/deps/rabbit/src/rabbit_router.erl +++ b/deps/rabbit/src/rabbit_router.erl @@ -25,42 +25,11 @@ match_result(). match_bindings(SrcName, Match) -> - MatchHead = #route{binding = #binding{source = SrcName, - _ = '_'}}, - Routes = ets:select(rabbit_route, [{MatchHead, [], [['$_']]}]), - [Dest || [#route{binding = Binding = #binding{destination = Dest}}] <- - Routes, Match(Binding)]. + rabbit_db_binding:match(SrcName, Match). -spec match_routing_key(rabbit_types:binding_source(), [routing_key(), ...] | ['_']) -> match_result(). -match_routing_key(SrcName, [RoutingKey]) -> - find_routes(#route{binding = #binding{source = SrcName, - destination = '$1', - key = RoutingKey, - _ = '_'}}, - []); -match_routing_key(SrcName, [_|_] = RoutingKeys) -> - find_routes(#route{binding = #binding{source = SrcName, - destination = '$1', - key = '$2', - _ = '_'}}, - [list_to_tuple(['orelse' | [{'=:=', '$2', RKey} || - RKey <- RoutingKeys]])]). - -%%-------------------------------------------------------------------- - -%% Normally we'd call mnesia:dirty_select/2 here, but that is quite -%% expensive for the same reasons as above, and, additionally, due to -%% mnesia 'fixing' the table with ets:safe_fixtable/2, which is wholly -%% unnecessary. According to the ets docs (and the code in erl_db.c), -%% 'select' is safe anyway ("Functions that internally traverse over a -%% table, like select and match, will give the same guarantee as -%% safe_fixtable.") and, furthermore, even the lower level iterators -%% ('first' and 'next') are safe on ordered_set tables ("Note that for -%% tables of the ordered_set type, safe_fixtable/2 is not necessary as -%% calls to first/1 and next/2 will always succeed."), which -%% rabbit_route is. -find_routes(MatchHead, Conditions) -> - ets:select(rabbit_route, [{MatchHead, Conditions, ['$1']}]). +match_routing_key(SrcName, RoutingKeys) -> + rabbit_db_binding:match_routing_key(SrcName, RoutingKeys, false). diff --git a/deps/rabbit/src/rabbit_stream_coordinator.erl b/deps/rabbit/src/rabbit_stream_coordinator.erl index 05e99547bc95..d555fb9dd838 100644 --- a/deps/rabbit/src/rabbit_stream_coordinator.erl +++ b/deps/rabbit/src/rabbit_stream_coordinator.erl @@ -153,7 +153,7 @@ restart_stream(QRes) -> {timeout, term()}. restart_stream(QRes, Options) when element(1, QRes) == resource -> - restart_stream(hd(rabbit_amqqueue:lookup([QRes])), Options); + restart_stream(hd(rabbit_amqqueue:lookup_many([QRes])), Options); restart_stream(Q, Options) when ?is_amqqueue(Q) andalso ?amqqueue_is_stream(Q) -> @@ -1089,10 +1089,10 @@ phase_update_mnesia(StreamId, Args, #{reference := QName, %% we need to re-initialise the queue record %% if the stream id is a match case rabbit_amqqueue:lookup_durable_queue(QName) of - [] -> + {error, not_found} -> %% queue not found at all, it must have been deleted ok; - [Q] -> + {ok, Q} -> case amqqueue:get_type_state(Q) of #{name := S} when S == StreamId -> rabbit_log:debug("~ts: initializing queue record for stream id ~ts", diff --git a/deps/rabbit/test/rabbit_db_binding_SUITE.erl b/deps/rabbit/test/rabbit_db_binding_SUITE.erl new file mode 100644 index 000000000000..ed5376e3ea09 --- /dev/null +++ b/deps/rabbit/test/rabbit_db_binding_SUITE.erl @@ -0,0 +1,331 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_binding_SUITE). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("common_test/include/ct.hrl"). + +-compile(export_all). + +-define(VHOST, <<"/">>). + +all() -> + [ + {group, all_tests} + ]. + +groups() -> + [ + {all_tests, [], all_tests()} + ]. + +all_tests() -> + [ + create, + exists, + delete, + auto_delete, + get_all, + get_all_by_vhost, + get_all_for_source, + get_all_for_destination, + get_all_for_source_and_destination, + get_all_for_source_and_destination_reverse, + fold, + match, + match_routing_key + ]. + +%% ------------------------------------------------------------------- +%% Test suite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(Group, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, Group}, + {rmq_nodes_count, 1} + ]), + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_group(_Group, Config) -> + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_db_exchange, clear, []), + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_db_binding, clear, []), + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% --------------------------------------------------------------------------- +%% Test Cases +%% --------------------------------------------------------------------------- + +create(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, create1, [Config]). + +create1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1 = #exchange{name = XName1, durable = true}, + Exchange2 = #exchange{name = XName2, durable = true}, + Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, + ?assertMatch({error, {resources_missing, [_, _]}}, + rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), + ?assertMatch({error, {resources_missing, [_]}}, + rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange2)), + ?assertMatch({error, too_bad}, + rabbit_db_binding:create(Binding, fun(_, _) -> {error, too_bad} end)), + ?assertMatch(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + passed. + +exists(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, exists1, [Config]). + +exists1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1 = #exchange{name = XName1, durable = true}, + Exchange2 = #exchange{name = XName2, durable = true}, + Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, + ?assertEqual(false, rabbit_db_exchange:exists(Binding)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange2)), + ?assertMatch(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + ?assertEqual(true, rabbit_db_binding:exists(Binding)), + passed. + +delete(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete1, [Config]). + +delete1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1 = #exchange{name = XName1, durable = true, auto_delete = false}, + Exchange2 = #exchange{name = XName2, durable = true, auto_delete = false}, + Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, + ?assertEqual(ok, rabbit_db_binding:delete(Binding, fun(_, _) -> ok end)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange2)), + ?assertMatch(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + Ret = rabbit_db_binding:delete(Binding, fun(_, _) -> ok end), + ?assertMatch({ok, _}, Ret), + {ok, Deletions} = Ret, + ?assertMatch({#exchange{}, not_deleted, [#binding{}], none}, + dict:fetch(XName1, Deletions)), + ?assertEqual(false, rabbit_db_binding:exists(Binding)), + passed. + +auto_delete(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete1, [Config]). + +auto_delete1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1 = #exchange{name = XName1, durable = true, auto_delete = true}, + Exchange2 = #exchange{name = XName2, durable = true, auto_delete = false}, + Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, + ?assertEqual(ok, rabbit_db_binding:delete(Binding, fun(_, _) -> ok end)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange2)), + ?assertMatch(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + Ret = rabbit_db_binding:delete(Binding, fun(_, _) -> ok end), + ?assertMatch({ok, _}, Ret), + {ok, Deletions} = Ret, + ?assertMatch({#exchange{}, deleted, [#binding{}], none}, + dict:fetch(XName1, Deletions)), + ?assertEqual(false, rabbit_db_binding:exists(Binding)), + passed. + +get_all(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_all1, [Config]). + +get_all1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1 = #exchange{name = XName1, durable = true}, + Exchange2 = #exchange{name = XName2, durable = true}, + Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, + ?assertEqual([], rabbit_db_binding:get_all()), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange2)), + ?assertMatch(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + ?assertEqual([Binding], rabbit_db_binding:get_all()), + passed. + +get_all_by_vhost(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_all_by_vhost1, [Config]). + +get_all_by_vhost1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1 = #exchange{name = XName1, durable = true}, + Exchange2 = #exchange{name = XName2, durable = true}, + Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, + ?assertEqual([], rabbit_db_binding:get_all(?VHOST)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange2)), + ?assertMatch(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + ?assertEqual([Binding], rabbit_db_binding:get_all(?VHOST)), + ?assertEqual([], rabbit_db_binding:get_all(<<"other-vhost">>)), + passed. + +get_all_for_source(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, get_all_for_source1, [Config]). + +get_all_for_source1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1 = #exchange{name = XName1, durable = true}, + Exchange2 = #exchange{name = XName2, durable = true}, + Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, + ?assertEqual([], rabbit_db_binding:get_all_for_source(XName1)), + ?assertEqual([], rabbit_db_binding:get_all_for_source(XName2)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange2)), + ?assertMatch(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + ?assertEqual([Binding], rabbit_db_binding:get_all_for_source(XName1)), + ?assertEqual([], rabbit_db_binding:get_all_for_source(XName2)), + passed. + +get_all_for_destination(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, get_all_for_destination1, [Config]). + +get_all_for_destination1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1 = #exchange{name = XName1, durable = true}, + Exchange2 = #exchange{name = XName2, durable = true}, + Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, + ?assertEqual([], rabbit_db_binding:get_all_for_destination(XName1)), + ?assertEqual([], rabbit_db_binding:get_all_for_destination(XName2)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange2)), + ?assertMatch(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + ?assertEqual([], rabbit_db_binding:get_all_for_destination(XName1)), + ?assertEqual([Binding], rabbit_db_binding:get_all_for_destination(XName2)), + passed. + +get_all_for_source_and_destination(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, get_all_for_source_and_destination1, [Config]). + +get_all_for_source_and_destination1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1 = #exchange{name = XName1, durable = true}, + Exchange2 = #exchange{name = XName2, durable = true}, + Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, + ?assertEqual([], rabbit_db_binding:get_all(XName1, XName2, false)), + ?assertEqual([], rabbit_db_binding:get_all(XName2, XName1, false)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange2)), + ?assertMatch(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + ?assertEqual([Binding], rabbit_db_binding:get_all(XName1, XName2, false)), + ?assertEqual([], rabbit_db_binding:get_all(XName1, XName1, false)), + ?assertEqual([], rabbit_db_binding:get_all(XName2, XName1, false)), + ?assertEqual([], rabbit_db_binding:get_all(XName2, XName2, false)), + passed. + +get_all_for_source_and_destination_reverse(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, get_all_for_source_and_destination_reverse1, [Config]). + +get_all_for_source_and_destination_reverse1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1 = #exchange{name = XName1, durable = true}, + Exchange2 = #exchange{name = XName2, durable = true}, + Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, + ?assertEqual([], rabbit_db_binding:get_all(XName1, XName2, true)), + ?assertEqual([], rabbit_db_binding:get_all(XName2, XName1, true)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange2)), + ?assertMatch(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + ?assertEqual([Binding], rabbit_db_binding:get_all(XName1, XName2, true)), + ?assertEqual([], rabbit_db_binding:get_all(XName1, XName1, true)), + ?assertEqual([], rabbit_db_binding:get_all(XName2, XName1, true)), + ?assertEqual([], rabbit_db_binding:get_all(XName2, XName2, true)), + passed. + +fold(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, fold1, [Config]). + +fold1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1 = #exchange{name = XName1, durable = true}, + Exchange2 = #exchange{name = XName2, durable = true}, + Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, + ?assertEqual([], rabbit_db_binding:fold(fun(B, Acc) -> [B | Acc] end, [])), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange2)), + ?assertMatch(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + ?assertEqual([Binding], rabbit_db_binding:fold(fun(B, Acc) -> [B | Acc] end, [])), + passed. + +match(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, match1, [Config]). + +match1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1 = #exchange{name = XName1, durable = true}, + Exchange2 = #exchange{name = XName2, durable = true}, + Binding = #binding{source = XName1, key = <<"">>, destination = XName2, + args = #{foo => bar}}, + ?assertEqual([], rabbit_db_binding:match(XName1, fun(#binding{args = Args}) -> + maps:get(foo, Args) =:= bar + end)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange2)), + ?assertMatch(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + ?assertEqual([XName2], + rabbit_db_binding:match(XName1, fun(#binding{args = Args}) -> + maps:get(foo, Args) =:= bar + end)), + ?assertEqual([], + rabbit_db_binding:match(XName1, fun(#binding{args = Args}) -> + maps:is_key(headers, Args) + end)), + passed. + +match_routing_key(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, match1, [Config]). + +match_routing_key1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1 = #exchange{name = XName1, durable = true}, + Exchange2 = #exchange{name = XName2, durable = true}, + Binding = #binding{source = XName1, key = <<"*.*">>, destination = XName2, + args = #{foo => bar}}, + ?assertEqual([], rabbit_db_binding:match_routing_key(XName1, [<<"a.b.c">>], false)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange2)), + ?assertMatch(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + ?assertEqual([], rabbit_db_binding:match_routing_key(XName1, [<<"a.b.c">>], false)), + ?assertEqual([XName2], rabbit_db_binding:match_routing_key(XName1, [<<"a.b">>], false)), + passed. diff --git a/deps/rabbit/test/rabbit_db_exchange_SUITE.erl b/deps/rabbit/test/rabbit_db_exchange_SUITE.erl new file mode 100644 index 000000000000..33982e8b34e0 --- /dev/null +++ b/deps/rabbit/test/rabbit_db_exchange_SUITE.erl @@ -0,0 +1,330 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_exchange_SUITE). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("common_test/include/ct.hrl"). + +-compile(export_all). + +-define(VHOST, <<"/">>). + +all() -> + [ + {group, all_tests} + ]. + +groups() -> + [ + {all_tests, [], all_tests()} + ]. + +all_tests() -> + [ + create_or_get, + get, + get_many, + get_all, + get_all_by_vhost, + get_all_durable, + list, + count, + update, + set, + peek_serial, + next_serial, + delete_serial, + delete, + delete_if_unused, + exists, + match, + recover + ]. + +%% ------------------------------------------------------------------- +%% Test suite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(Group, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, Group}, + {rmq_nodes_count, 1} + ]), + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_group(_Group, Config) -> + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_testcase(Testcase, Config) -> + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_db_exchange, clear, []), + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_db_exchange, clear, []), + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_db_binding, clear, []), + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% --------------------------------------------------------------------------- +%% Test Cases +%% --------------------------------------------------------------------------- + +create_or_get(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, create_or_get1, [Config]). + +create_or_get1(_Config) -> + XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + Exchange0 = #exchange{name = XName, durable = true}, + Exchange = rabbit_exchange_decorator:set(Exchange0), + ?assertMatch({new, Exchange}, rabbit_db_exchange:create_or_get(Exchange0)), + ?assertEqual({existing, Exchange}, rabbit_db_exchange:create_or_get(Exchange0)), + passed. + +get(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get1, [Config]). + +get1(_Config) -> + XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + Exchange0 = #exchange{name = XName, durable = true}, + Exchange = rabbit_exchange_decorator:set(Exchange0), + ?assertEqual({error, not_found}, rabbit_db_exchange:get(XName)), + ?assertEqual({new, Exchange}, rabbit_db_exchange:create_or_get(Exchange0)), + ?assertEqual({ok, Exchange}, rabbit_db_exchange:get(XName)), + passed. + +get_many(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_many1, [Config]). + +get_many1(_Config) -> + XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + Exchange0 = #exchange{name = XName, durable = true}, + Exchange = rabbit_exchange_decorator:set(Exchange0), + ?assertEqual([], rabbit_db_exchange:get_many([XName])), + ?assertEqual({new, Exchange}, rabbit_db_exchange:create_or_get(Exchange0)), + ?assertEqual([Exchange], rabbit_db_exchange:get_many([XName])), + passed. + +get_all(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_all1, [Config]). + +get_all1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1_0 = #exchange{name = XName1, durable = true}, + Exchange2_0 = #exchange{name = XName2, durable = true}, + Exchange1 = rabbit_exchange_decorator:set(Exchange1_0), + Exchange2 = rabbit_exchange_decorator:set(Exchange2_0), + All = lists:sort([Exchange1, Exchange2]), + ?assertEqual([], rabbit_db_exchange:get_all()), + create([Exchange1_0, Exchange2_0]), + ?assertEqual(All, lists:sort(rabbit_db_exchange:get_all())), + passed. + +get_all_by_vhost(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_all_by_vhost1, [Config]). + +get_all_by_vhost1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1_0 = #exchange{name = XName1, durable = true}, + Exchange2_0 = #exchange{name = XName2, durable = true}, + Exchange1 = rabbit_exchange_decorator:set(Exchange1_0), + Exchange2 = rabbit_exchange_decorator:set(Exchange2_0), + All = lists:sort([Exchange1, Exchange2]), + ?assertEqual([], rabbit_db_exchange:get_all(?VHOST)), + create([Exchange1_0, Exchange2_0]), + ?assertEqual(All, lists:sort(rabbit_db_exchange:get_all(?VHOST))), + ?assertEqual([], lists:sort(rabbit_db_exchange:get_all(<<"other-vhost">>))), + passed. + +get_all_durable(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_all_durable1, [Config]). + +get_all_durable1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1_0 = #exchange{name = XName1, durable = true}, + Exchange2_0 = #exchange{name = XName2, durable = true}, + All = lists:sort([Exchange1_0, Exchange2_0]), + ?assertEqual([], rabbit_db_exchange:get_all_durable()), + create([Exchange1_0, Exchange2_0]), + ?assertEqual(All, lists:sort(rabbit_db_exchange:get_all_durable())), + passed. + +list(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, list1, [Config]). + +list1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1_0 = #exchange{name = XName1, durable = true}, + Exchange2_0 = #exchange{name = XName2, durable = true}, + All = lists:sort([XName1, XName2]), + ?assertEqual([], rabbit_db_exchange:list()), + create([Exchange1_0, Exchange2_0]), + ?assertEqual(All, lists:sort(rabbit_db_exchange:list())), + passed. + +count(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, count1, [Config]). + +count1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1_0 = #exchange{name = XName1, durable = true}, + Exchange2_0 = #exchange{name = XName2, durable = true}, + ?assertEqual(0, rabbit_db_exchange:count()), + create([Exchange1_0, Exchange2_0]), + ?assertEqual(2, rabbit_db_exchange:count()), + passed. + +update(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, update1, [Config]). + +update1(_Config) -> + XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + Exchange = #exchange{name = XName, durable = true}, + ?assertEqual(ok, + rabbit_db_exchange:update(XName, fun(X) -> X#exchange{type = topic} end)), + create([Exchange]), + ?assertEqual(ok, + rabbit_db_exchange:update(XName, fun(X) -> X#exchange{type = topic} end)), + {ok, Exchange0} = rabbit_db_exchange:get(XName), + ?assertEqual(topic, Exchange0#exchange.type), + passed. + +set(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, set1, [Config]). + +set1(_Config) -> + XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + Exchange = #exchange{name = XName, durable = true}, + ?assertEqual(ok, rabbit_db_exchange:set([Exchange])), + ?assertEqual({error, not_found}, rabbit_db_exchange:get(XName)), + ?assertEqual([Exchange], rabbit_db_exchange:get_all_durable()), + passed. + +peek_serial(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, peek_serial1, [Config]). + +peek_serial1(_Config) -> + XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + ?assertEqual(1, rabbit_db_exchange:peek_serial(XName)), + ?assertEqual(1, rabbit_db_exchange:peek_serial(XName)), + ?assertEqual(1, rabbit_db_exchange:next_serial(XName)), + ?assertEqual(2, rabbit_db_exchange:peek_serial(XName)), + ?assertEqual(2, rabbit_db_exchange:peek_serial(XName)), + passed. + +next_serial(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, next_serial1, [Config]). + +next_serial1(_Config) -> + XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + ?assertEqual(1, rabbit_db_exchange:next_serial(XName)), + ?assertEqual(2, rabbit_db_exchange:next_serial(XName)), + ?assertEqual(3, rabbit_db_exchange:next_serial(XName)), + passed. + +delete_serial(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_serial1, [Config]). + +delete_serial1(_Config) -> + XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + ?assertEqual(1, rabbit_db_exchange:next_serial(XName)), + ?assertEqual(2, rabbit_db_exchange:next_serial(XName)), + ?assertEqual(ok, rabbit_db_exchange:delete_serial(XName)), + ?assertEqual(1, rabbit_db_exchange:peek_serial(XName)), + passed. + +delete(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete1, [Config]). + +delete1(_Config) -> + XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + Exchange0 = #exchange{name = XName, durable = true}, + ?assertMatch({error, not_found}, rabbit_db_exchange:delete(XName, false)), + create([Exchange0]), + ?assertMatch({ok, #exchange{name = XName}}, rabbit_db_exchange:get(XName)), + ?assertMatch([#exchange{name = XName}], rabbit_db_exchange:get_all_durable()), + ?assertMatch({deleted, #exchange{name = XName}, [], _}, + rabbit_db_exchange:delete(XName, false)), + ?assertEqual({error, not_found}, rabbit_db_exchange:get(XName)), + ?assertEqual([], rabbit_db_exchange:get_all_durable()), + passed. + +delete_if_unused(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_if_unused1, [Config]). + +delete_if_unused1(_Config) -> + XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Exchange1 = #exchange{name = XName1, durable = true}, + Exchange2 = #exchange{name = XName2, durable = true}, + Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, + ?assertMatch({error, not_found}, rabbit_db_exchange:delete(XName1, true)), + create([Exchange1, Exchange2]), + ?assertEqual(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), + ?assertMatch({ok, #exchange{name = XName1}}, rabbit_db_exchange:get(XName1)), + ?assertMatch([#exchange{}, #exchange{}], rabbit_db_exchange:get_all_durable()), + ?assertMatch({error, in_use}, rabbit_db_exchange:delete(XName1, true)), + ?assertMatch({ok, #exchange{name = XName1}}, rabbit_db_exchange:get(XName1)), + ?assertMatch([#exchange{}, #exchange{}], rabbit_db_exchange:get_all_durable()), + passed. + +exists(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, exists1, [Config]). + +exists1(_Config) -> + XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + Exchange = #exchange{name = XName, durable = true}, + ?assertEqual(false, rabbit_db_exchange:exists(XName)), + create([Exchange]), + ?assertEqual(true, rabbit_db_exchange:exists(XName)), + passed. + +match(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, match1, [Config]). + +match1(_Config) -> + XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + Exchange = #exchange{name = XName, durable = true, type = topic}, + Pattern = #exchange{durable = true, type = topic, _ = '_'}, + ?assertEqual([], rabbit_db_exchange:match(Pattern)), + create([Exchange]), + ?assertMatch([#exchange{name = XName}], rabbit_db_exchange:match(Pattern)), + passed. + +recover(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, recover1, [Config]). + +recover1(_Config) -> + XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + Exchange = #exchange{name = XName, durable = true}, + ?assertEqual(ok, rabbit_db_exchange:set([Exchange])), + ?assertEqual({error, not_found}, rabbit_db_exchange:get(XName)), + ?assertEqual([Exchange], rabbit_db_exchange:get_all_durable()), + ?assertMatch([Exchange], rabbit_db_exchange:recover(?VHOST)), + ?assertMatch({ok, #exchange{name = XName}}, rabbit_db_exchange:get(XName)), + ?assertEqual([Exchange], rabbit_db_exchange:get_all_durable()), + passed. + +create(Exchanges) -> + [?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange)) + || Exchange <- Exchanges]. diff --git a/deps/rabbit/test/rabbit_db_maintenance_SUITE.erl b/deps/rabbit/test/rabbit_db_maintenance_SUITE.erl new file mode 100644 index 000000000000..491cdfb9a34a --- /dev/null +++ b/deps/rabbit/test/rabbit_db_maintenance_SUITE.erl @@ -0,0 +1,93 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_maintenance_SUITE). + +-include_lib("eunit/include/eunit.hrl"). +-include_lib("common_test/include/ct.hrl"). + +-compile(export_all). + +all() -> + [ + {group, all_tests} + ]. + +groups() -> + [ + {all_tests, [], all_tests()} + ]. + +all_tests() -> + [ + setup_schema, + set_and_get, + set_and_get_consistent + ]. + +%% ------------------------------------------------------------------- +%% Test suite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(Group, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, Group}, + {rmq_nodes_count, 1} + ]), + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_group(_Group, Config) -> + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% --------------------------------------------------------------------------- +%% Test Cases +%% --------------------------------------------------------------------------- + +setup_schema(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, setup_schema1, [Config]). + +setup_schema1(_Config) -> + ?assertEqual(ok, rabbit_db_maintenance:setup_schema()), + passed. + +set_and_get(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, set_and_get1, [Config]). + +set_and_get1(_Config) -> + ?assertEqual(true, rabbit_db_maintenance:set(ready)), + ?assertEqual(ready, rabbit_db_maintenance:get(node())), + ?assertEqual(undefined, rabbit_db_maintenance:get('another-node')), + passed. + +set_and_get_consistent(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, set_and_get_consistent1, [Config]). + +set_and_get_consistent1(_Config) -> + ?assertEqual(true, rabbit_db_maintenance:set(ready)), + ?assertEqual(ready, rabbit_db_maintenance:get_consistent(node())), + ?assertEqual(undefined, rabbit_db_maintenance:get_consistent('another-node')), + passed. diff --git a/deps/rabbit/test/rabbit_db_msup_SUITE.erl b/deps/rabbit/test/rabbit_db_msup_SUITE.erl new file mode 100644 index 000000000000..20ca9f1fbfd9 --- /dev/null +++ b/deps/rabbit/test/rabbit_db_msup_SUITE.erl @@ -0,0 +1,136 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_msup_SUITE). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("common_test/include/ct.hrl"). + +-compile(export_all). + +all() -> + [ + {group, all_tests} + ]. + +groups() -> + [ + {all_tests, [], all_tests()} + ]. + +all_tests() -> + [ + create_tables, + create_or_update, + find_mirror, + delete, + delete_all, + update_all + ]. + +%% ------------------------------------------------------------------- +%% Test suite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(Group, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, Group}, + {rmq_nodes_count, 1} + ]), + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_group(_Group, Config) -> + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_db_msup, clear, []), + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% --------------------------------------------------------------------------- +%% Test Cases +%% --------------------------------------------------------------------------- + +create_tables(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, create_tables1, [Config]). + +create_tables1(_Config) -> + ?assertEqual(ok, rabbit_db_msup:create_tables()), + passed. + +create_or_update(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, create_or_update1, [Config]). + +create_or_update1(_Config) -> + Overall = spawn(fun() -> ok end), + Spec = #{id => id, start => {m, f, args}}, + ?assertEqual(start, + rabbit_db_msup:create_or_update(group, Overall, undefined, Spec, id)), + passed. + +find_mirror(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, find_mirror1, [Config]). + +find_mirror1(_Config) -> + Overall = spawn(fun() -> ok end), + Spec = #{id => id, start => {m, f, args}}, + ?assertEqual(start, rabbit_db_msup:create_or_update(group, Overall, undefined, + Spec, id)), + ?assertEqual({ok, Overall}, rabbit_db_msup:find_mirror(group, id)), + passed. + +delete(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete1, [Config]). + +delete1(_Config) -> + Overall = spawn(fun() -> ok end), + Spec = #{id => id, start => {m, f, args}}, + ?assertEqual(start, rabbit_db_msup:create_or_update(group, Overall, undefined, + Spec, id)), + ?assertEqual(ok, rabbit_db_msup:delete(group, id)), + ?assertEqual({error, not_found}, rabbit_db_msup:find_mirror(group, id)), + passed. + +delete_all(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_all1, [Config]). + +delete_all1(_Config) -> + Overall = spawn(fun() -> ok end), + Spec = #{id => id, start => {m, f, args}}, + ?assertEqual(start, rabbit_db_msup:create_or_update(group, Overall, undefined, + Spec, id)), + ?assertEqual(ok, rabbit_db_msup:delete_all(group)), + ?assertEqual({error, not_found}, rabbit_db_msup:find_mirror(group, id)), + passed. + +update_all(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, update_all1, [Config]). + +update_all1(_Config) -> + OldOverall = spawn(fun() -> ok end), + Overall = spawn(fun() -> ok end), + Spec = #{id => id, start => {m, f, args}}, + ?assertEqual(start, rabbit_db_msup:create_or_update(group, OldOverall, undefined, + Spec, id)), + ?assertEqual({ok, OldOverall}, rabbit_db_msup:find_mirror(group, id)), + ?assertEqual([Spec], rabbit_db_msup:update_all(Overall, OldOverall)), + ?assertEqual({ok, Overall}, rabbit_db_msup:find_mirror(group, id)), + passed. diff --git a/deps/rabbit/test/rabbit_db_policy_SUITE.erl b/deps/rabbit/test/rabbit_db_policy_SUITE.erl new file mode 100644 index 000000000000..2afa3acd5d9f --- /dev/null +++ b/deps/rabbit/test/rabbit_db_policy_SUITE.erl @@ -0,0 +1,96 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_policy_SUITE). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("common_test/include/ct.hrl"). + +-compile(export_all). + +-define(VHOST, <<"/">>). + +all() -> + [ + {group, all_tests} + ]. + +groups() -> + [ + {all_tests, [], all_tests()} + ]. + +all_tests() -> + [ + update + ]. + +%% ------------------------------------------------------------------- +%% Test suite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(Group, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, Group}, + {rmq_nodes_count, 1} + ]), + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_group(_Group, Config) -> + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_testcase(Testcase, Config) -> + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_db_exchange, clear, []), + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_db_exchange, clear, []), + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% --------------------------------------------------------------------------- +%% Test Cases +%% --------------------------------------------------------------------------- + +update(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, update1, [Config]). + +update1(_Config) -> + XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + Exchange = #exchange{name = XName, durable = true}, + ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange)), + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + Queue = amqqueue:new(QName, none, true, false, none, [], ?VHOST, #{}, + rabbit_classic_queue), + ?assertEqual({created, Queue}, rabbit_db_queue:create_or_get(Queue)), + ?assertMatch( + {[{_, _}], [{_, _}]}, + rabbit_db_policy:update(?VHOST, + fun(X) -> #{exchange => X, + update_function => + fun(X0) -> + X0#exchange{policy = new_policy} + end} + end, + fun(Q) -> #{queue => Q, + update_function => + fun(Q0) -> + amqqueue:set_policy(Q0, random_policy) + end} + end)), + passed. diff --git a/deps/rabbit/test/rabbit_db_queue_SUITE.erl b/deps/rabbit/test/rabbit_db_queue_SUITE.erl new file mode 100644 index 000000000000..3cafb91443d0 --- /dev/null +++ b/deps/rabbit/test/rabbit_db_queue_SUITE.erl @@ -0,0 +1,596 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_queue_SUITE). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("common_test/include/ct.hrl"). +-include("amqqueue.hrl"). + +-compile(export_all). + +-define(VHOST, <<"/">>). + +all() -> + [ + {group, all_tests}, + {group, mnesia_store} + ]. + +groups() -> + [ + {all_tests, [], all_tests()}, + {mnesia_store, [], mnesia_tests()} + ]. + +all_tests() -> + [ + create_or_get, + get, + get_many, + get_all, + get_all_by_vhost, + get_all_by_type, + get_all_by_type_and_node, + list, + count, + count_by_vhost, + set, + set_many, + delete, + update, + exists, + get_all_durable, + get_all_durable_by_type, + filter_all_durable, + get_durable, + get_many_durable, + set_dirty, + internal_delete, + update_durable + ]. + +mnesia_tests() -> + [ + foreach_durable, + foreach_transient, + delete_transient, + update_in_mnesia_tx, + get_durable_in_mnesia_tx + ]. + +%% ------------------------------------------------------------------- +%% Test suite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(Group, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, Group}, + {rmq_nodes_count, 1} + ]), + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_group(_Group, Config) -> + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_db_queue, clear, []), + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% --------------------------------------------------------------------------- +%% Test Cases +%% --------------------------------------------------------------------------- + +create_or_get(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, create_or_get1, [Config]). + +create_or_get1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + Q = new_queue(QName, rabbit_classic_queue), + ?assertEqual({created, Q}, rabbit_db_queue:create_or_get(Q)), + ?assertEqual({existing, Q}, rabbit_db_queue:create_or_get(Q)), + %% TODO absent + passed. + +get(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get1, [Config]). + +get1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + Q = new_queue(QName, rabbit_classic_queue), + ok = rabbit_db_queue:set(Q), + ?assertEqual({ok, Q}, rabbit_db_queue:get(QName)), + ?assertEqual({error, not_found}, + rabbit_db_queue:get(rabbit_misc:r(?VHOST, queue, <<"test-queue2">>))), + passed. + +get_many(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_many1, [Config]). + +get_many1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Q = new_queue(QName, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_classic_queue), + ok = rabbit_db_queue:set(Q), + ?assertEqual([Q], rabbit_db_queue:get_many([QName])), + ?assertEqual([Q], rabbit_db_queue:get_many([QName, QName2])), + ?assertEqual([], rabbit_db_queue:get_many([QName2])), + ok = rabbit_db_queue:set(Q2), + ?assertEqual([Q, Q2], rabbit_db_queue:get_many([QName, QName2])), + passed. + +get_all(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_all1, [Config]). + +get_all1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Q = new_queue(QName, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_classic_queue), + All = lists:sort([Q, Q2]), + ?assertEqual([], rabbit_db_queue:get_all()), + set_list([Q, Q2]), + ?assertEqual(All, lists:sort(rabbit_db_queue:get_all())), + passed. + +get_all_by_vhost(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_all_by_vhost1, [Config]). + +get_all_by_vhost1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Q = new_queue(QName, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_classic_queue), + All = lists:sort([Q, Q2]), + ?assertEqual([], rabbit_db_queue:get_all(?VHOST)), + ?assertEqual([], rabbit_db_queue:get_all(<<"some-vhost">>)), + set_list([Q, Q2]), + ?assertEqual(All, lists:sort(rabbit_db_queue:get_all(?VHOST))), + ?assertEqual([], rabbit_db_queue:get_all(<<"some-vhost">>)), + passed. + +get_all_by_type(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_all_by_type1, [Config]). + +get_all_by_type1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + QName3 = rabbit_misc:r(?VHOST, queue, <<"test-queue3">>), + QName4 = rabbit_misc:r(?VHOST, queue, <<"test-queue4">>), + Q = new_queue(QName, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_quorum_queue), + Q3 = new_queue(QName3, rabbit_quorum_queue), + Q4 = new_queue(QName4, rabbit_stream_queue), + Quorum = lists:sort([Q2, Q3]), + ?assertEqual([], rabbit_db_queue:get_all_by_type(rabbit_classic_queue)), + ?assertEqual([], lists:sort(rabbit_db_queue:get_all_by_type(rabbit_quorum_queue))), + ?assertEqual([], rabbit_db_queue:get_all_by_type(rabbit_stream_queue)), + set_list([Q, Q2, Q3, Q4]), + ?assertEqual([Q], rabbit_db_queue:get_all_by_type(rabbit_classic_queue)), + ?assertEqual(Quorum, lists:sort(rabbit_db_queue:get_all_by_type(rabbit_quorum_queue))), + ?assertEqual([Q4], rabbit_db_queue:get_all_by_type(rabbit_stream_queue)), + passed. + +get_all_by_type_and_node(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_all_by_type_and_node1, [Config]). + +get_all_by_type_and_node1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + QName3 = rabbit_misc:r(?VHOST, queue, <<"test-queue3">>), + QName4 = rabbit_misc:r(?VHOST, queue, <<"test-queue4">>), + Pid = spawn(fun() -> ok end), + Q = new_queue(QName, rabbit_classic_queue, Pid), + Q2 = new_queue(QName2, rabbit_quorum_queue), + Q3 = new_queue(QName3, rabbit_quorum_queue, Pid), + Q4 = new_queue(QName4, rabbit_stream_queue, Pid), + Node = node(), + ?assertEqual([], rabbit_db_queue:get_all_by_type_and_node(?VHOST, rabbit_classic_queue, Node)), + ?assertEqual([], lists:sort(rabbit_db_queue:get_all_by_type_and_node(?VHOST, rabbit_quorum_queue, Node))), + ?assertEqual([], rabbit_db_queue:get_all_by_type_and_node(?VHOST, rabbit_stream_queue, Node)), + set_list([Q, Q2, Q3, Q4]), + ?assertEqual([Q], rabbit_db_queue:get_all_by_type_and_node(?VHOST, rabbit_classic_queue, Node)), + ?assertEqual([], rabbit_db_queue:get_all_by_type_and_node(<<"other-vhost">>, rabbit_classic_queue, Node)), + ?assertEqual([Q3], lists:sort(rabbit_db_queue:get_all_by_type_and_node(?VHOST, rabbit_quorum_queue, Node))), + ?assertEqual([Q4], rabbit_db_queue:get_all_by_type_and_node(?VHOST, rabbit_stream_queue, Node)), + passed. + +list(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, list1, [Config]). + +list1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Q = new_queue(QName, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_classic_queue), + All = lists:sort([QName, QName2]), + ?assertEqual([], rabbit_db_queue:list()), + set_list([Q, Q2]), + ?assertEqual(All, lists:sort(rabbit_db_queue:list())), + passed. + +count(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, count1, [Config]). + +count1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + QName3 = rabbit_misc:r(?VHOST, queue, <<"test-queue3">>), + QName4 = rabbit_misc:r(?VHOST, queue, <<"test-queue4">>), + Q = new_queue(QName, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_quorum_queue), + Q3 = new_queue(QName3, rabbit_quorum_queue), + Q4 = new_queue(QName4, rabbit_stream_queue), + ?assertEqual(0, rabbit_db_queue:count()), + set_list([Q, Q2, Q3, Q4]), + ?assertEqual(4, rabbit_db_queue:count()), + passed. + +count_by_vhost(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, count_by_vhost1, [Config]). + +count_by_vhost1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + QName3 = rabbit_misc:r(?VHOST, queue, <<"test-queue3">>), + QName4 = rabbit_misc:r(?VHOST, queue, <<"test-queue4">>), + Q = new_queue(QName, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_quorum_queue), + Q3 = new_queue(QName3, rabbit_quorum_queue), + Q4 = new_queue(QName4, rabbit_stream_queue), + ?assertEqual(0, rabbit_db_queue:count(?VHOST)), + set_list([Q, Q2, Q3, Q4]), + ?assertEqual(4, rabbit_db_queue:count(?VHOST)), + ?assertEqual(0, rabbit_db_queue:count(<<"other-vhost">>)), + passed. + +set(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, set1, [Config]). + +set1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + Q = new_queue(QName, rabbit_classic_queue), + ?assertEqual(ok, rabbit_db_queue:set(Q)), + ?assertEqual(ok, rabbit_db_queue:set(Q)), + ?assertEqual({ok, Q}, rabbit_db_queue:get(QName)), + passed. + +set_many(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, set_many1, [Config]). + +set_many1(_Config) -> + QName1 = rabbit_misc:r(?VHOST, queue, <<"test-queue1">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + QName3 = rabbit_misc:r(?VHOST, queue, <<"test-queue3">>), + Q1 = new_queue(QName1, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_classic_queue), + Q3 = new_queue(QName3, rabbit_classic_queue), + ?assertEqual(ok, rabbit_db_queue:set_many([])), + ?assertEqual(ok, rabbit_db_queue:set_many([Q1, Q2, Q3])), + ?assertEqual({ok, Q1}, rabbit_db_queue:get_durable(QName1)), + ?assertEqual({ok, Q2}, rabbit_db_queue:get_durable(QName2)), + ?assertEqual({ok, Q3}, rabbit_db_queue:get_durable(QName3)), + passed. + +delete(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete1, [Config]). + +delete1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + Q = new_queue(QName, rabbit_classic_queue), + ?assertEqual(ok, rabbit_db_queue:set(Q)), + ?assertEqual({ok, Q}, rabbit_db_queue:get(QName)), + %% TODO Can we handle the deletions outside of rabbit_db_queue? Probably not because + %% they should be done in a single transaction, but what a horrid API to have! + Dict = rabbit_db_queue:delete(QName, normal), + ?assertEqual(0, dict:size(Dict)), + ?assertEqual(ok, rabbit_db_queue:delete(QName, normal)), + ?assertEqual({error, not_found}, rabbit_db_queue:get(QName)), + passed. + +update(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, update1, [Config]). + +update1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Q = new_queue(QName, rabbit_classic_queue), + Pid = spawn(fun() -> ok end), + Q2 = amqqueue:set_pid(Q, Pid), + ?assertEqual(ok, rabbit_db_queue:set(Q)), + ?assertEqual({ok, Q}, rabbit_db_queue:get(QName)), + ?assertEqual(Q2, rabbit_db_queue:update(QName, fun(_) -> Q2 end)), + ?assertEqual({ok, Q2}, rabbit_db_queue:get(QName)), + ?assertEqual(not_found, rabbit_db_queue:update(QName2, fun(_) -> Q2 end)), + passed. + +update_decorators(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, update_decorators1, [Config]). + +update_decorators1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + Q = new_queue(QName, rabbit_classic_queue), + ?assertEqual(ok, rabbit_db_queue:set(Q)), + ?assertEqual({ok, Q}, rabbit_db_queue:get(QName)), + ?assertEqual(undefined, amqqueue:get_decorators(Q)), + %% Not really testing we set a decorator, but at least the field is being updated + ?assertEqual(ok, rabbit_db_queue:update_decorators(QName)), + {ok, Q1} = rabbit_db_queue:get(QName), + ?assertEqual([], amqqueue:get_decorators(Q1)), + passed. + +exists(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, exists1, [Config]). + +exists1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + Q = new_queue(QName, rabbit_classic_queue), + ?assertEqual(false, rabbit_db_queue:exists(QName)), + ?assertEqual(ok, rabbit_db_queue:set(Q)), + ?assertEqual(true, rabbit_db_queue:exists(QName)), + passed. + +get_all_durable(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_all_durable1, [Config]). + +get_all_durable1(_Config) -> + QName1 = rabbit_misc:r(?VHOST, queue, <<"test-queue1">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + QName3 = rabbit_misc:r(?VHOST, queue, <<"test-queue3">>), + Q1 = new_queue(QName1, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_classic_queue), + Q3 = new_queue(QName3, rabbit_classic_queue), + All = lists:sort([Q1, Q2, Q3]), + ?assertEqual([], rabbit_db_queue:get_all_durable()), + set_list(All), + ?assertEqual(All, lists:sort(rabbit_db_queue:get_all_durable())), + passed. + +get_all_durable_by_type(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_all_durable_by_type1, [Config]). + +get_all_durable_by_type1(_Config) -> + QName1 = rabbit_misc:r(?VHOST, queue, <<"test-queue1">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + QName3 = rabbit_misc:r(?VHOST, queue, <<"test-queue3">>), + QName4 = rabbit_misc:r(?VHOST, queue, <<"test-queue4">>), + Q1 = new_queue(QName1, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_quorum_queue), + Q3 = new_queue(QName3, rabbit_stream_queue), + Q4 = new_queue(QName4, rabbit_classic_queue), + All = lists:sort([Q1, Q2, Q3]), + ok = rabbit_db_queue:set_dirty(Q4), + ?assertEqual([], rabbit_db_queue:get_all_durable_by_type(rabbit_classic_queue)), + set_list(All), + ?assertEqual([Q1], rabbit_db_queue:get_all_durable_by_type(rabbit_classic_queue)), + ?assertEqual([Q2], rabbit_db_queue:get_all_durable_by_type(rabbit_quorum_queue)), + ?assertEqual([Q3], rabbit_db_queue:get_all_durable_by_type(rabbit_stream_queue)), + passed. + +filter_all_durable(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, filter_all_durable1, [Config]). + +filter_all_durable1(_Config) -> + QName1 = rabbit_misc:r(?VHOST, queue, <<"test-queue1">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + QName3 = rabbit_misc:r(?VHOST, queue, <<"test-queue3">>), + Q1 = new_queue(QName1, rabbit_quorum_queue), + Q2 = new_queue(QName2, rabbit_classic_queue), + Q3 = new_queue(QName3, rabbit_classic_queue), + All = lists:sort([Q2, Q3]), + ?assertEqual([], rabbit_db_queue:filter_all_durable( + fun(Q) -> + amqqueue:get_type(Q) =:= rabbit_classic_queue + end)), + set_list([Q1, Q2, Q3]), + ?assertEqual(All, lists:sort(rabbit_db_queue:filter_all_durable( + fun(Q) -> + amqqueue:get_type(Q) =:= rabbit_classic_queue + end))), + passed. + +get_durable(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_durable1, [Config]). + +get_durable1(_Config) -> + QName1 = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Q1 = new_queue(QName1, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_classic_queue), + ok = rabbit_db_queue:set(Q1), + ok = rabbit_db_queue:set_dirty(Q2), + ?assertEqual({ok, Q1}, rabbit_db_queue:get_durable(QName1)), + ?assertEqual({error, not_found}, rabbit_db_queue:get_durable(QName2)), + passed. + +get_many_durable(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_many_durable1, [Config]). + +get_many_durable1(_Config) -> + QName1 = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Q1 = new_queue(QName1, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_classic_queue), + ok = rabbit_db_queue:set(Q1), + ok = rabbit_db_queue:set_dirty(Q2), + ?assertEqual([Q1], rabbit_db_queue:get_many_durable([QName1])), + ?assertEqual([], rabbit_db_queue:get_many_durable([QName2])), + passed. + +update_durable(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, update_durable1, [Config]). + +update_durable1(_Config) -> + QName1 = rabbit_misc:r(?VHOST, queue, <<"test-queue1">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Q1 = new_queue(QName1, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_classic_queue), + ?assertEqual(ok, rabbit_db_queue:set(Q1)), + ?assertEqual(ok, rabbit_db_queue:set_dirty(Q2)), + ?assertEqual(ok, rabbit_db_queue:update_durable( + fun(Q0) -> + amqqueue:set_policy(Q0, my_policy) + end, + fun(Q0) when ?is_amqqueue(Q0) -> true end)), + {ok, Q0} = rabbit_db_queue:get_durable(QName1), + ?assertMatch(my_policy, amqqueue:get_policy(Q0)), + {ok, Q00} = rabbit_db_queue:get(QName1), + ?assertMatch(undefined, amqqueue:get_policy(Q00)), + passed. + +foreach_durable(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, foreach_durable1, [Config]). + +foreach_durable1(_Config) -> + QName1 = rabbit_misc:r(?VHOST, queue, <<"test-queue1">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Q1 = new_queue(QName1, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_classic_queue), + ?assertEqual(ok, rabbit_db_queue:set(Q1)), + ?assertEqual(ok, rabbit_db_queue:set_dirty(Q2)), + ?assertEqual(ok, rabbit_db_queue:foreach_durable( + fun(Q0) -> + rabbit_db_queue:internal_delete(amqqueue:get_name(Q0), true, normal) + end, + fun(Q0) when ?is_amqqueue(Q0) -> true end)), + ?assertEqual({error, not_found}, rabbit_db_queue:get(QName1)), + ?assertEqual({error, not_found}, rabbit_db_queue:get_durable(QName1)), + ?assertMatch({ok, _}, rabbit_db_queue:get(QName2)), + passed. + +foreach_transient(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, foreach_transient1, [Config]). + +foreach_transient1(_Config) -> + QName1 = rabbit_misc:r(?VHOST, queue, <<"test-queue1">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Q1 = new_queue(QName1, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_classic_queue), + ?assertEqual(ok, rabbit_db_queue:set(Q1)), + ?assertEqual(ok, rabbit_db_queue:set_dirty(Q2)), + ?assertEqual(ok, rabbit_db_queue:foreach_transient( + fun(Q0) -> + rabbit_db_queue:internal_delete(amqqueue:get_name(Q0), true, normal) + end)), + ?assertEqual({error, not_found}, rabbit_db_queue:get(QName1)), + ?assertEqual({error, not_found}, rabbit_db_queue:get_durable(QName1)), + ?assertEqual({error, not_found}, rabbit_db_queue:get(QName2)), + passed. + +delete_transient(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_transient1, [Config]). + +delete_transient1(_Config) -> + QName1 = rabbit_misc:r(?VHOST, queue, <<"test-queue1">>), + QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Q1 = new_queue(QName1, rabbit_classic_queue), + Q2 = new_queue(QName2, rabbit_quorum_queue), + ?assertEqual(ok, rabbit_db_queue:set_dirty(Q1)), + ?assertEqual(ok, rabbit_db_queue:set_dirty(Q2)), + ?assertMatch({[QName1], _}, + rabbit_db_queue:delete_transient( + fun(Q0) when ?is_amqqueue(Q0) -> + amqqueue:get_type(Q0) == rabbit_classic_queue + end)), + ?assertEqual({error, not_found}, rabbit_db_queue:get(QName1)), + ?assertMatch({ok, _}, rabbit_db_queue:get(QName2)), + passed. + +set_dirty(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, set_dirty1, [Config]). + +set_dirty1(_Config) -> + QName1 = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + Q1 = new_queue(QName1, rabbit_classic_queue), + Q2 = amqqueue:set_decorators(Q1, []), + ok = rabbit_db_queue:set_dirty(Q1), + ?assertEqual({ok, Q2}, rabbit_db_queue:get(QName1)), + ?assertEqual({error, not_found}, rabbit_db_queue:get_durable(QName1)), + passed. + +internal_delete(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, internal_delete1, [Config]). + +internal_delete1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + Q = new_queue(QName, rabbit_classic_queue), + ?assertEqual(ok, rabbit_db_queue:set(Q)), + ?assertEqual(ok, rabbit_db_queue:foreach_durable( + fun(Q0) -> rabbit_db_queue:internal_delete(amqqueue:get_name(Q0), + false, normal) end, + fun(Q0) when ?is_amqqueue(Q0) -> true end)), + ?assertEqual({error, not_found}, rabbit_db_queue:get(QName)), + ?assertEqual({error, not_found}, rabbit_db_queue:get_durable(QName)), + passed. + +update_in_mnesia_tx(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, update_in_mnesia_tx1, [Config]). + +update_in_mnesia_tx1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + Q = new_queue(QName, rabbit_classic_queue), + Pid = spawn(fun() -> ok end), + ?assertEqual({atomic, not_found}, + mnesia:transaction(fun() -> + rabbit_db_queue:update_in_mnesia_tx( + QName, + fun(Q0) -> amqqueue:set_pid(Q0, Pid) end) + end)), + ?assertEqual(ok, rabbit_db_queue:set(Q)), + {atomic, Q1} = + mnesia:transaction(fun() -> + rabbit_db_queue:update_in_mnesia_tx( + QName, + fun(Q0) -> amqqueue:set_pid(Q0, Pid) end) + end), + ?assertEqual(Pid, amqqueue:get_pid(Q1)), + ?assertEqual({ok, Q1}, rabbit_db_queue:get(QName)), + ?assertEqual({ok, Q1}, rabbit_db_queue:get_durable(QName)), + passed. + +get_durable_in_mnesia_tx(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, get_durable_in_mnesia_tx1, [Config]). + +get_durable_in_mnesia_tx1(_Config) -> + QName = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + Q = new_queue(QName, rabbit_classic_queue), + ?assertEqual({atomic, {error, not_found}}, + mnesia:transaction(fun() -> + rabbit_db_queue:get_durable_in_mnesia_tx(QName) + end)), + ?assertEqual(ok, rabbit_db_queue:set(Q)), + ?assertEqual({atomic, {ok, Q}}, + mnesia:transaction(fun() -> + rabbit_db_queue:get_durable_in_mnesia_tx(QName) + end)), + passed. + +set_list(Qs) -> + [?assertEqual(ok, rabbit_db_queue:set(Q)) || Q <- Qs]. + +new_queue(QName, Type) -> + new_queue(QName, Type, none). + +new_queue(#resource{virtual_host = VHost} = QName, Type, Pid) -> + amqqueue:new(QName, Pid, true, false, none, [], VHost, #{}, Type). diff --git a/deps/rabbit/test/rabbit_db_topic_exchange_SUITE.erl b/deps/rabbit/test/rabbit_db_topic_exchange_SUITE.erl new file mode 100644 index 000000000000..8c2d424e2040 --- /dev/null +++ b/deps/rabbit/test/rabbit_db_topic_exchange_SUITE.erl @@ -0,0 +1,158 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_topic_exchange_SUITE). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("common_test/include/ct.hrl"). + +-compile(export_all). + +-define(VHOST, <<"/">>). + +all() -> + [ + {group, all_tests} + ]. + +groups() -> + [ + {all_tests, [], all_tests()} + ]. + +all_tests() -> + [ + set, + delete, + delete_all_for_exchange, + match + ]. + +%% ------------------------------------------------------------------- +%% Test suite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(Group, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, Group}, + {rmq_nodes_count, 1} + ]), + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_group(_Group, Config) -> + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_db_topic_exchange, clear, []), + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% --------------------------------------------------------------------------- +%% Test Cases +%% --------------------------------------------------------------------------- + +set(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, set1, [Config]). + +set1(_Config) -> + Src = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + Dst = rabbit_misc:r(?VHOST, queue, <<"test-queue">>), + RoutingKey = <<"a.b.c">>, + Binding = #binding{source = Src, key = RoutingKey, destination = Dst, args = #{}}, + ?assertEqual([], rabbit_db_topic_exchange:match(Src, RoutingKey)), + ?assertEqual(ok, rabbit_db_topic_exchange:set(Binding)), + ?assertEqual(ok, rabbit_db_topic_exchange:set(Binding)), + ?assertEqual([Dst], rabbit_db_topic_exchange:match(Src, RoutingKey)), + passed. + +delete(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete1, [Config]). + +delete1(_Config) -> + Src = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + Dst1 = rabbit_misc:r(?VHOST, queue, <<"test-queue1">>), + Dst2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Dst3= rabbit_misc:r(?VHOST, queue, <<"test-queue3">>), + Dsts = lists:sort([Dst1, Dst2, Dst3]), + RoutingKey = <<"a.b.c">>, + Binding1 = #binding{source = Src, key = RoutingKey, destination = Dst1, args = #{}}, + Binding2 = #binding{source = Src, key = RoutingKey, destination = Dst2, args = #{}}, + Binding3 = #binding{source = Src, key = RoutingKey, destination = Dst3, args = #{}}, + ?assertEqual(ok, rabbit_db_topic_exchange:delete([Binding1])), + ?assertEqual(ok, rabbit_db_topic_exchange:set(Binding1)), + ?assertEqual(ok, rabbit_db_topic_exchange:set(Binding2)), + ?assertEqual(ok, rabbit_db_topic_exchange:set(Binding3)), + ?assertEqual(Dsts, lists:sort(rabbit_db_topic_exchange:match(Src, RoutingKey))), + ?assertEqual(ok, rabbit_db_topic_exchange:delete([Binding1, Binding2])), + ?assertEqual([Dst3], rabbit_db_topic_exchange:match(Src, RoutingKey)), + passed. + +delete_all_for_exchange(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_all_for_exchange1, [Config]). + +delete_all_for_exchange1(_Config) -> + Src1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), + Src2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), + Dst1 = rabbit_misc:r(?VHOST, queue, <<"test-queue1">>), + Dst2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Dsts = lists:sort([Dst1, Dst2]), + RoutingKey = <<"a.b.c">>, + ?assertEqual(ok, rabbit_db_topic_exchange:delete_all_for_exchange(Src1)), + set(Src1, RoutingKey, Dst1), + set(Src1, RoutingKey, Dst2), + set(Src2, RoutingKey, Dst1), + ?assertEqual(Dsts, lists:sort(rabbit_db_topic_exchange:match(Src1, RoutingKey))), + ?assertEqual([Dst1], rabbit_db_topic_exchange:match(Src2, RoutingKey)), + ?assertEqual(ok, rabbit_db_topic_exchange:delete_all_for_exchange(Src1)), + ?assertEqual([], rabbit_db_topic_exchange:match(Src1, RoutingKey)), + ?assertEqual([Dst1], rabbit_db_topic_exchange:match(Src2, RoutingKey)), + passed. + +match(Config) -> + passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, match1, [Config]). + +match1(_Config) -> + Src = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + Dst1 = rabbit_misc:r(?VHOST, queue, <<"test-queue1">>), + Dst2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Dst3 = rabbit_misc:r(?VHOST, queue, <<"test-queue3">>), + Dst4 = rabbit_misc:r(?VHOST, queue, <<"test-queue4">>), + Dst5 = rabbit_misc:r(?VHOST, queue, <<"test-queue5">>), + Dst6 = rabbit_misc:r(?VHOST, queue, <<"test-queue6">>), + set(Src, <<"a.b.c">>, Dst1), + set(Src, <<"a.*.c">>, Dst2), + set(Src, <<"*.#">>, Dst3), + set(Src, <<"#">>, Dst4), + set(Src, <<"#.#">>, Dst5), + set(Src, <<"a.*">>, Dst6), + Dsts1 = lists:sort([Dst1, Dst2, Dst3, Dst4, Dst5]), + ?assertEqual(Dsts1, lists:usort(rabbit_db_topic_exchange:match(Src, <<"a.b.c">>))), + Dsts2 = lists:sort([Dst3, Dst4, Dst5, Dst6]), + ?assertEqual(Dsts2, lists:usort(rabbit_db_topic_exchange:match(Src, <<"a.b">>))), + Dsts3 = lists:sort([Dst4, Dst5]), + ?assertEqual(Dsts3, lists:usort(rabbit_db_topic_exchange:match(Src, <<"">>))), + Dsts4 = lists:sort([Dst3, Dst4, Dst5]), + ?assertEqual(Dsts4, lists:usort(rabbit_db_topic_exchange:match(Src, <<"zen.rabbit">>))), + passed. + +set(Src, RoutingKey, Dst) -> + Binding = #binding{source = Src, key = RoutingKey, destination = Dst, args = #{}}, + ok = rabbit_db_topic_exchange:set(Binding). diff --git a/deps/rabbit/test/topic_permission_SUITE.erl b/deps/rabbit/test/topic_permission_SUITE.erl index 14474411a871..b39b3e8b77ad 100644 --- a/deps/rabbit/test/topic_permission_SUITE.erl +++ b/deps/rabbit/test/topic_permission_SUITE.erl @@ -47,10 +47,8 @@ init_per_testcase(Testcase, Config) -> rabbit_ct_helpers:testcase_started(Config, Testcase). clear_tables() -> - {atomic, ok} = mnesia:clear_table(rabbit_topic_permission), - {atomic, ok} = mnesia:clear_table(rabbit_vhost), - {atomic, ok} = mnesia:clear_table(rabbit_user), - ok. + ok = rabbit_db_vhost:clear(), + ok = rabbit_db_user:clear(). end_per_testcase(Testcase, Config) -> rabbit_ct_helpers:testcase_finished(Config, Testcase). @@ -145,14 +143,8 @@ topic_permission_checks(Config) -> topic_permission_checks1(_Config) -> 0 = length(ets:tab2list(rabbit_topic_permission)), - rabbit_mnesia:execute_mnesia_transaction(fun() -> - ok = mnesia:write(rabbit_vhost, - vhost:new(<<"/">>, []), - write), - ok = mnesia:write(rabbit_vhost, - vhost:new(<<"other-vhost">>, []), - write) - end), + rabbit_db_vhost:create_or_get(<<"/">>, [], #{}), + rabbit_db_vhost:create_or_get(<<"other-vhost">>, [], #{}), rabbit_auth_backend_internal:add_user(<<"guest">>, <<"guest">>, <<"acting-user">>), rabbit_auth_backend_internal:add_user(<<"dummy">>, <<"dummy">>, <<"acting-user">>), diff --git a/deps/rabbit/test/unit_access_control_SUITE.erl b/deps/rabbit/test/unit_access_control_SUITE.erl index f3ee24cd6790..d921ace881d8 100644 --- a/deps/rabbit/test/unit_access_control_SUITE.erl +++ b/deps/rabbit/test/unit_access_control_SUITE.erl @@ -391,7 +391,7 @@ topic_matching1(_Config) -> lists:nth(11, Bindings), lists:nth(19, Bindings), lists:nth(21, Bindings), lists:nth(28, Bindings)], exchange_op_callback(X, remove_bindings, [RemovedBindings]), - RemainingBindings = ordsets:to_list( + _RemainingBindings = ordsets:to_list( ordsets:subtract(ordsets:from_list(Bindings), ordsets:from_list(RemovedBindings))), @@ -416,14 +416,12 @@ topic_matching1(_Config) -> {"args-test", ["t6", "t22", "t23", "t24", "t25", "t27"]}]), %% remove the entire exchange - exchange_op_callback(X, delete, [RemainingBindings]), + exchange_op_callback(X, delete, []), %% none should match now test_topic_expect_match(X, [{"a.b.c", []}, {"b.b.c", []}, {"", []}]), passed. exchange_op_callback(X, Fun, Args) -> - rabbit_mnesia:execute_mnesia_transaction( - fun () -> rabbit_exchange:callback(X, Fun, transaction, [X] ++ Args) end), rabbit_exchange:callback(X, Fun, none, [X] ++ Args). test_topic_expect_match(X, List) -> diff --git a/deps/rabbitmq_consistent_hash_exchange/BUILD.bazel b/deps/rabbitmq_consistent_hash_exchange/BUILD.bazel index 4401984fb8da..bba8fbc60596 100644 --- a/deps/rabbitmq_consistent_hash_exchange/BUILD.bazel +++ b/deps/rabbitmq_consistent_hash_exchange/BUILD.bazel @@ -15,6 +15,7 @@ APP_NAME = "rabbitmq_consistent_hash_exchange" APP_DESCRIPTION = "Consistent Hash Exchange Type" BUILD_DEPS = [ + "//deps/rabbit:erlang_app", "//deps/rabbitmq_cli:erlang_app", ] @@ -42,7 +43,7 @@ xref( plt( name = "base_plt", libs = ["//deps/rabbitmq_cli:elixir"], - deps = ["//deps/rabbitmq_cli:elixir"] + BUILD_DEPS + DEPS + RUNTIME_DEPS, + deps = ["//deps/rabbitmq_cli:elixir"] + BUILD_DEPS + DEPS, ) dialyze( diff --git a/deps/rabbitmq_consistent_hash_exchange/src/rabbit_exchange_type_consistent_hash.erl b/deps/rabbitmq_consistent_hash_exchange/src/rabbit_exchange_type_consistent_hash.erl index 88f3af8a0919..83c60d22bb84 100644 --- a/deps/rabbitmq_consistent_hash_exchange/src/rabbit_exchange_type_consistent_hash.erl +++ b/deps/rabbitmq_consistent_hash_exchange/src/rabbit_exchange_type_consistent_hash.erl @@ -32,7 +32,7 @@ [exchange, <<"x-consistent-hash">>]}}]}). -rabbit_boot_step( - {rabbit_exchange_type_consistent_hash_mnesia, + {rabbit_exchange_type_consistent_hash_metadata_store, [{description, "exchange type x-consistent-hash: shared state"}, {mfa, {?MODULE, init, []}}, {requires, database}, @@ -48,7 +48,7 @@ init() -> rabbit_db_ch_exchange:setup_schema(), - recover(), + _ = recover(), ok. info(_X) -> []. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_exchange_link.erl b/deps/rabbitmq_federation/src/rabbit_federation_exchange_link.erl index 2cb9952072e2..e71c6a95092b 100644 --- a/deps/rabbitmq_federation/src/rabbit_federation_exchange_link.erl +++ b/deps/rabbitmq_federation/src/rabbit_federation_exchange_link.erl @@ -438,12 +438,8 @@ go(S0 = {not_started, {Upstream, UParams, DownXName}}) -> {longstr, N} -> N; _ -> unknown end, - {Serial, Bindings} = - rabbit_mnesia:execute_mnesia_transaction( - fun () -> - {rabbit_exchange:peek_serial(DownXName), - rabbit_binding:list_for_source(DownXName)} - end), + {Serial, Bindings} = {rabbit_exchange:peek_serial(DownXName), + rabbit_binding:list_for_source(DownXName)}, true = is_integer(Serial), %% If we are very short lived, Serial can be undefined at %% this point (since the deletion of the X could have diff --git a/deps/rabbitmq_jms_topic_exchange/BUILD.bazel b/deps/rabbitmq_jms_topic_exchange/BUILD.bazel index 2a7fb0543378..20e1d48405d7 100644 --- a/deps/rabbitmq_jms_topic_exchange/BUILD.bazel +++ b/deps/rabbitmq_jms_topic_exchange/BUILD.bazel @@ -16,6 +16,10 @@ APP_DESCRIPTION = "RabbitMQ JMS topic selector exchange plugin" APP_MODULE = "rabbit_federation_app" +BUILD_DEPS = [ + "//deps/rabbit:erlang_app", +] + DEPS = [ "//deps/rabbit_common:erlang_app", ] @@ -33,6 +37,7 @@ rabbitmq_app( app_module = APP_MODULE, app_name = APP_NAME, runtime_deps = RUNTIME_DEPS, + build_deps = BUILD_DEPS, deps = DEPS, extra_apps = EXTRA_APPS, ) diff --git a/deps/rabbitmq_jms_topic_exchange/src/rabbit_jms_topic_exchange.erl b/deps/rabbitmq_jms_topic_exchange/src/rabbit_jms_topic_exchange.erl index 66288d3f9a15..d9087215e9ba 100644 --- a/deps/rabbitmq_jms_topic_exchange/src/rabbit_jms_topic_exchange.erl +++ b/deps/rabbitmq_jms_topic_exchange/src/rabbit_jms_topic_exchange.erl @@ -124,10 +124,8 @@ add_binding( none case BindGen of {ok, BindFun} -> add_binding_fun(XName, {{BindingKey, Dest}, BindFun}); - {none, error} -> - parsing_error(XName, Selector, Dest); - _ -> - ok + error -> + parsing_error(XName, Selector, Dest) end, ok. diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl index 952b26e96187..fea18533b1e9 100644 --- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl +++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl @@ -1253,7 +1253,7 @@ deliver_to_queues(Delivery, RoutedToQNames, State0 = #state{queue_states = QStates0, cfg = #cfg{proto_ver = ProtoVer}}) -> - Qs0 = rabbit_amqqueue:lookup(RoutedToQNames), + Qs0 = rabbit_amqqueue:lookup_many(RoutedToQNames), Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0), case rabbit_queue_type:deliver(Qs, Delivery, QStates0) of {ok, QStates, Actions} -> diff --git a/deps/rabbitmq_random_exchange/BUILD.bazel b/deps/rabbitmq_random_exchange/BUILD.bazel index 3bfc23269910..803791b8dbc3 100644 --- a/deps/rabbitmq_random_exchange/BUILD.bazel +++ b/deps/rabbitmq_random_exchange/BUILD.bazel @@ -11,6 +11,10 @@ APP_NAME = "rabbitmq_random_exchange" APP_DESCRIPTION = "RabbitMQ Random Exchange" +BUILD_DEPS = [ + "//deps/rabbit:erlang_app", +] + DEPS = [ "//deps/rabbit_common:erlang_app", ] @@ -22,6 +26,7 @@ RUNTIME_DEPS = [ rabbitmq_app( app_description = APP_DESCRIPTION, app_name = APP_NAME, + build_deps = BUILD_DEPS, runtime_deps = RUNTIME_DEPS, deps = DEPS, ) @@ -30,7 +35,7 @@ xref() plt( name = "base_plt", - deps = DEPS, + deps = BUILD_DEPS + DEPS, ) dialyze( diff --git a/deps/rabbitmq_recent_history_exchange/BUILD.bazel b/deps/rabbitmq_recent_history_exchange/BUILD.bazel index 39a84db6ec3e..458d84861e72 100644 --- a/deps/rabbitmq_recent_history_exchange/BUILD.bazel +++ b/deps/rabbitmq_recent_history_exchange/BUILD.bazel @@ -14,6 +14,10 @@ APP_NAME = "rabbitmq_recent_history_exchange" APP_DESCRIPTION = "RabbitMQ Recent History Exchange" +BUILD_DEPS = [ + "//deps/rabbit:erlang_app", +] + DEPS = [ "//deps/rabbit_common:erlang_app", ] @@ -26,6 +30,7 @@ rabbitmq_app( app_description = APP_DESCRIPTION, app_extra_keys = BROKER_VERSION_REQUIREMENTS_ANY, app_name = APP_NAME, + build_deps = BUILD_DEPS, runtime_deps = RUNTIME_DEPS, deps = DEPS, ) diff --git a/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl b/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl index 1dde6fda15ef..6feee9ad9b0a 100644 --- a/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl +++ b/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl @@ -84,13 +84,6 @@ insert0_in_mnesia(Key, Cached, Message, Length) -> content = [Message|lists:sublist(Cached, Length-1)]}, write). -add_to_cache(Cached, Message, undefined) -> - add_to_cache(Cached, Message, ?KEEP_NB); -add_to_cache(Cached, Message, {_Type, Length}) -> - add_to_cache(Cached, Message, Length); -add_to_cache(Cached, Message, Length) -> - [Message|lists:sublist(Cached, Length-1)]. - %% ------------------------------------------------------------------- %% delete(). %% ------------------------------------------------------------------- diff --git a/deps/rabbitmq_recent_history_exchange/src/rabbit_exchange_type_recent_history.erl b/deps/rabbitmq_recent_history_exchange/src/rabbit_exchange_type_recent_history.erl index 685c8463db9e..7b20758196ca 100644 --- a/deps/rabbitmq_recent_history_exchange/src/rabbit_exchange_type_recent_history.erl +++ b/deps/rabbitmq_recent_history_exchange/src/rabbit_exchange_type_recent_history.erl @@ -28,8 +28,8 @@ {requires, rabbit_registry}, {enables, kernel_ready}]}). --rabbit_boot_step({rabbit_exchange_type_recent_history_mnesia, - [{description, "recent history exchange type: mnesia"}, +-rabbit_boot_step({rabbit_exchange_type_recent_history_metadata_store, + [{description, "recent history exchange type: metadata store"}, {mfa, {?MODULE, setup_schema, []}}, {requires, database}, {enables, external_infrastructure}]}). @@ -96,7 +96,7 @@ add_binding(none, #exchange{ name = XName }, [begin Delivery = rabbit_basic:delivery(false, false, Msg, undefined), Qs = rabbit_exchange:route(X, Delivery), - case rabbit_amqqueue:lookup(Qs) of + case rabbit_amqqueue:lookup_many(Qs) of [] -> destination_not_found_error(Qs); QPids ->