From 0c24d97b8ff685425284a01218cb40ffb5b7b909 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Mon, 12 May 2025 11:38:10 +0200 Subject: [PATCH 1/9] Split rabbitmq_federation: rabbitmq_queue_federation, rabbitmq_exchange_federation and rabbitmq_federation_common --- .gitignore | 3 + deps/rabbitmq_exchange_federation/Makefile | 25 + .../README-hacking | 143 +++ deps/rabbitmq_exchange_federation/README.md | 23 + .../include/rabbit_exchange_federation.hrl | 8 + .../src/rabbit_exchange_federation_app.erl | 51 + .../src/rabbit_exchange_federation_sup.erl | 64 ++ .../src/rabbit_federation_exchange.erl | 97 ++ .../src/rabbit_federation_exchange_link.erl | 696 +++++++++++++ ...abbit_federation_exchange_link_sup_sup.erl | 90 ++ .../rabbit_federation_upstream_exchange.erl | 91 ++ .../test/definition_import_SUITE.erl | 104 ++ .../definition_import_SUITE_data/case1.json | 52 + .../test/exchange_SUITE.erl | 913 ++++++++++++++++++ ...change_federation_status_command_SUITE.erl | 168 ++++ .../test/rabbit_federation_test_util.erl | 246 +++++ .../restart_federation_link_command_SUITE.erl | 101 ++ .../test/unit_inbroker_SUITE.erl | 110 +++ deps/rabbitmq_federation_common/Makefile | 25 + .../include/logging.hrl | 3 + .../include/rabbit_federation.hrl | 48 + ...I.Ctl.Commands.FederationStatusCommand.erl | 117 +++ ....Commands.RestartFederationLinkCommand.erl | 84 ++ .../src/rabbit_federation_common_app.erl | 33 + .../src/rabbit_federation_db.erl | 45 + .../src/rabbit_federation_event.erl | 54 ++ .../src/rabbit_federation_link_sup.erl | 111 +++ .../src/rabbit_federation_link_util.erl | 359 +++++++ .../src/rabbit_federation_parameters.erl | 143 +++ .../src/rabbit_federation_pg.erl | 23 + .../src/rabbit_federation_status.erl | 178 ++++ .../src/rabbit_federation_sup.erl | 66 ++ .../src/rabbit_federation_upstream.erl | 166 ++++ .../src/rabbit_federation_util.erl | 102 ++ .../src/rabbit_log_federation.erl | 107 ++ .../definition_import_SUITE_data/case1.json | 52 + .../test/unit_SUITE.erl | 65 ++ .../test/unit_inbroker_SUITE.erl | 201 ++++ deps/rabbitmq_queue_federation/Makefile | 24 + deps/rabbitmq_queue_federation/README-hacking | 143 +++ deps/rabbitmq_queue_federation/README.md | 23 + .../include/rabbit_queue_federation.hrl | 8 + .../src/rabbit_federation_queue.erl | 109 +++ .../src/rabbit_federation_queue_link.erl | 327 +++++++ .../rabbit_federation_queue_link_sup_sup.erl | 98 ++ .../src/rabbit_queue_federation_app.erl | 51 + .../src/rabbit_queue_federation_sup.erl | 64 ++ .../test/definition_import_SUITE.erl | 104 ++ .../definition_import_SUITE_data/case1.json | 52 + .../test/queue_SUITE.erl | 395 ++++++++ .../queue_federation_status_command_SUITE.erl | 172 ++++ .../test/rabbit_federation_status_SUITE.erl | 108 +++ .../test/rabbit_federation_test_util.erl | 299 ++++++ .../rabbit_queue_federation_status_SUITE.erl | 107 ++ .../restart_federation_link_command_SUITE.erl | 100 ++ 55 files changed, 7151 insertions(+) create mode 100644 deps/rabbitmq_exchange_federation/Makefile create mode 100644 deps/rabbitmq_exchange_federation/README-hacking create mode 100644 deps/rabbitmq_exchange_federation/README.md create mode 100644 deps/rabbitmq_exchange_federation/include/rabbit_exchange_federation.hrl create mode 100644 deps/rabbitmq_exchange_federation/src/rabbit_exchange_federation_app.erl create mode 100644 deps/rabbitmq_exchange_federation/src/rabbit_exchange_federation_sup.erl create mode 100644 deps/rabbitmq_exchange_federation/src/rabbit_federation_exchange.erl create mode 100644 deps/rabbitmq_exchange_federation/src/rabbit_federation_exchange_link.erl create mode 100644 deps/rabbitmq_exchange_federation/src/rabbit_federation_exchange_link_sup_sup.erl create mode 100644 deps/rabbitmq_exchange_federation/src/rabbit_federation_upstream_exchange.erl create mode 100644 deps/rabbitmq_exchange_federation/test/definition_import_SUITE.erl create mode 100644 deps/rabbitmq_exchange_federation/test/definition_import_SUITE_data/case1.json create mode 100644 deps/rabbitmq_exchange_federation/test/exchange_SUITE.erl create mode 100644 deps/rabbitmq_exchange_federation/test/exchange_federation_status_command_SUITE.erl create mode 100644 deps/rabbitmq_exchange_federation/test/rabbit_federation_test_util.erl create mode 100644 deps/rabbitmq_exchange_federation/test/restart_federation_link_command_SUITE.erl create mode 100644 deps/rabbitmq_exchange_federation/test/unit_inbroker_SUITE.erl create mode 100644 deps/rabbitmq_federation_common/Makefile create mode 100644 deps/rabbitmq_federation_common/include/logging.hrl create mode 100644 deps/rabbitmq_federation_common/include/rabbit_federation.hrl create mode 100644 deps/rabbitmq_federation_common/src/Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand.erl create mode 100644 deps/rabbitmq_federation_common/src/Elixir.RabbitMQ.CLI.Ctl.Commands.RestartFederationLinkCommand.erl create mode 100644 deps/rabbitmq_federation_common/src/rabbit_federation_common_app.erl create mode 100644 deps/rabbitmq_federation_common/src/rabbit_federation_db.erl create mode 100644 deps/rabbitmq_federation_common/src/rabbit_federation_event.erl create mode 100644 deps/rabbitmq_federation_common/src/rabbit_federation_link_sup.erl create mode 100644 deps/rabbitmq_federation_common/src/rabbit_federation_link_util.erl create mode 100644 deps/rabbitmq_federation_common/src/rabbit_federation_parameters.erl create mode 100644 deps/rabbitmq_federation_common/src/rabbit_federation_pg.erl create mode 100644 deps/rabbitmq_federation_common/src/rabbit_federation_status.erl create mode 100644 deps/rabbitmq_federation_common/src/rabbit_federation_sup.erl create mode 100644 deps/rabbitmq_federation_common/src/rabbit_federation_upstream.erl create mode 100644 deps/rabbitmq_federation_common/src/rabbit_federation_util.erl create mode 100644 deps/rabbitmq_federation_common/src/rabbit_log_federation.erl create mode 100644 deps/rabbitmq_federation_common/test/definition_import_SUITE_data/case1.json create mode 100644 deps/rabbitmq_federation_common/test/unit_SUITE.erl create mode 100644 deps/rabbitmq_federation_common/test/unit_inbroker_SUITE.erl create mode 100644 deps/rabbitmq_queue_federation/Makefile create mode 100644 deps/rabbitmq_queue_federation/README-hacking create mode 100644 deps/rabbitmq_queue_federation/README.md create mode 100644 deps/rabbitmq_queue_federation/include/rabbit_queue_federation.hrl create mode 100644 deps/rabbitmq_queue_federation/src/rabbit_federation_queue.erl create mode 100644 deps/rabbitmq_queue_federation/src/rabbit_federation_queue_link.erl create mode 100644 deps/rabbitmq_queue_federation/src/rabbit_federation_queue_link_sup_sup.erl create mode 100644 deps/rabbitmq_queue_federation/src/rabbit_queue_federation_app.erl create mode 100644 deps/rabbitmq_queue_federation/src/rabbit_queue_federation_sup.erl create mode 100644 deps/rabbitmq_queue_federation/test/definition_import_SUITE.erl create mode 100644 deps/rabbitmq_queue_federation/test/definition_import_SUITE_data/case1.json create mode 100644 deps/rabbitmq_queue_federation/test/queue_SUITE.erl create mode 100644 deps/rabbitmq_queue_federation/test/queue_federation_status_command_SUITE.erl create mode 100644 deps/rabbitmq_queue_federation/test/rabbit_federation_status_SUITE.erl create mode 100644 deps/rabbitmq_queue_federation/test/rabbit_federation_test_util.erl create mode 100644 deps/rabbitmq_queue_federation/test/rabbit_queue_federation_status_SUITE.erl create mode 100644 deps/rabbitmq_queue_federation/test/restart_federation_link_command_SUITE.erl diff --git a/.gitignore b/.gitignore index eee87485f4e8..272050aff697 100644 --- a/.gitignore +++ b/.gitignore @@ -48,7 +48,9 @@ elvis !/deps/rabbitmq_ct_helpers/ !/deps/rabbitmq_ct_client_helpers/ !/deps/rabbitmq_event_exchange/ +!/deps/rabbitmq_exchange_federation/ !/deps/rabbitmq_federation/ +!/deps/rabbitmq_federation_common/ !/deps/rabbitmq_federation_management/ !/deps/rabbitmq_federation_prometheus/ !/deps/rabbitmq_jms_topic_exchange/ @@ -62,6 +64,7 @@ elvis !/deps/rabbitmq_peer_discovery_k8s/ !/deps/rabbitmq_prelaunch/ !/deps/rabbitmq_prometheus/ +!/deps/rabbitmq_queue_federation/ !/deps/rabbitmq_random_exchange/ !/deps/rabbitmq_recent_history_exchange/ !/deps/rabbitmq_sharding/ diff --git a/deps/rabbitmq_exchange_federation/Makefile b/deps/rabbitmq_exchange_federation/Makefile new file mode 100644 index 000000000000..eb0a51622e9f --- /dev/null +++ b/deps/rabbitmq_exchange_federation/Makefile @@ -0,0 +1,25 @@ +PROJECT = rabbitmq_exchange_federation +PROJECT_DESCRIPTION = RabbitMQ Exchange Federation +PROJECT_MOD = rabbit_exchange_federation_app + +define PROJECT_ENV +[ + {pgroup_name_cluster_id, false}, + {internal_exchange_check_interval, 90000} + ] +endef + +define PROJECT_APP_EXTRA_KEYS + {broker_version_requirements, []} +endef + +DEPS = rabbit_common rabbit amqp_client rabbitmq_federation_common +TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers + +PLT_APPS += rabbitmq_cli + +DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-plugin.mk +DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk + +include ../../rabbitmq-components.mk +include ../../erlang.mk diff --git a/deps/rabbitmq_exchange_federation/README-hacking b/deps/rabbitmq_exchange_federation/README-hacking new file mode 100644 index 000000000000..6432552fe33a --- /dev/null +++ b/deps/rabbitmq_exchange_federation/README-hacking @@ -0,0 +1,143 @@ +This file is intended to tell you How It All Works, concentrating on +the things you might not expect. + +The theory +========== + +The 'x-federation' exchange is defined in +rabbit_federation_exchange. This starts up a bunch of link processes +(one for each upstream) which: + + * Connect to the upstream broker + * Create a queue and bind it to the upstream exchange + * Keep bindings in sync with the downstream exchange + * Consume messages from the upstream queue and republish them to the + downstream exchange (matching confirms with acks) + +Each link process monitors the connections / channels it opens, and +dies if they do. We use a supervisor2 to ensure that we get some +backoff when restarting. + +We use process groups to identify all link processes for a certain +exchange, as well as all link processes together. + +However, there are a bunch of wrinkles: + + +Wrinkle: The exchange will be recovered when the Erlang client is not available +=============================================================================== + +Exchange recovery happens within the rabbit application - therefore at +the time that the exchange is recovered, we can't make any connections +since the amqp_client application has not yet started. Each link +therefore initially has a state 'not_started'. When it is created it +checks to see if the rabbitmq_federation application is running. If +so, it starts fully. If not, it goes into the 'not_started' +state. When rabbitmq_federation starts, it sends a 'go' message to all +links, prodding them to bring up the link. + + +Wrinkle: On reconnect we want to assert bindings atomically +=========================================================== + +If the link goes down for whatever reason, then by the time it comes +up again the bindings downstream may no longer be in sync with those +upstream. Therefore on link establishment we want to ensure that a +certain set of bindings exists. (Of course bringing up a link for the +first time is a simple case of this.) And we want to do this with AMQP +methods. But if we were to tear down all bindings and recreate them, +we would have a time period when messages would not be forwarded for +bindings that *do* still exist before and after. + +We use exchange to exchange bindings to work around this: + +We bind the upstream exchange (X) to the upstream queue (Q) via an +internal fanout exchange (IXA) like so: (routing keys R1 and R2): + + X----R1,R2--->IXA---->Q + +This has the same effect as binding the queue to the exchange directly. + +Now imagine the link has gone down, and is about to be +reestablished. In the meanwhile, routing has changed downstream so +that we now want routing keys R1 and R3. On link reconnection we can +create and bind another internal fanout exchange IXB: + + X----R1,R2--->IXA---->Q + | ^ + | | + \----R1,R3--->IXB-----/ + +and then delete the original exchange IXA: + + X Q + | ^ + | | + \----R1,R3--->IXB-----/ + +This means that messages matching R1 are always routed during the +switchover. Messages for R3 will start being routed as soon as we bind +the second exchange, and messages for R2 will be stopped in a timely +way. Of course this could lag the downstream situation somewhat, in +which case some R2 messages will get thrown away downstream since they +are unroutable. However this lag is inevitable when the link goes +down. + +This means that the downstream only needs to keep track of whether the +upstream is currently going via internal exchange A or B. This is +held in the exchange scratch space in Mnesia. + + +Wrinkle: We need to amalgamate bindings +======================================= + +Since we only bind to one exchange upstream, but the downstream +exchange can be bound to many queues, we can have duplicated bindings +downstream (same source, routing key and args but different +destination) that cannot be duplicated upstream (since the destination +is the same). The link therefore maintains a mapping of (Key, Args) to +set(Dest). Duplicated bindings do not get repeated upstream, and are +only unbound upstream when the last one goes away downstream. + +Furthermore, this works as an optimisation since this will tend to +reduce upstream binding count and churn. + + +Wrinkle: We may receive binding events out of order +=================================================== + +The rabbit_federation_exchange callbacks are invoked by channel +processes within rabbit. Therefore they can be executed concurrently, +and can arrive at the link processes in an order that does not +correspond to the wall clock. + +We need to keep the state of the link in sync with Mnesia. Therefore +not only do we need to impose an ordering on these events, we need to +impose Mnesia's ordering on them. We therefore added a function to the +callback interface, serialise_events. When this returns true, the +callback mechanism inside rabbit increments a per-exchange counter +within an Mnesia transaction, and returns the value as part of the +add_binding and remove_binding callbacks. The link process then queues +up these events, and replays them in order. The link process's state +thus always follows Mnesia (it may be delayed, but the effects happen +in the same order). + + +Other issues +============ + +Since links are implemented in terms of AMQP, link failure may cause +messages to be redelivered. If you're unlucky this could lead to +duplication. + +Message duplication can also happen with some topologies. In some +cases it may not be possible to set max_hops such that messages arrive +once at every node. + +While we correctly order bind / unbind events, we don't do the same +thing for exchange creation / deletion. (This is harder - if you +delete and recreate an exchange with the same name, is it the same +exchange? What about if its type changes?) This would only be an issue +if exchanges churn rapidly; however we could get into a state where +Mnesia sees CDCD but we see CDDC and leave a process running when we +shouldn't. diff --git a/deps/rabbitmq_exchange_federation/README.md b/deps/rabbitmq_exchange_federation/README.md new file mode 100644 index 000000000000..d96c13a02e57 --- /dev/null +++ b/deps/rabbitmq_exchange_federation/README.md @@ -0,0 +1,23 @@ +## RabbitMQ Federation + +RabbitMQ federation offers a group of features for loosely +coupled and WAN-friendly distributed RabbitMQ setups. Note that +this is not an alternative to queue mirroring. + + +## Supported RabbitMQ Versions + +This plugin ships with RabbitMQ, there is no need to +install it separately. + + +## Documentation + +See [RabbitMQ federation plugin](https://www.rabbitmq.com/federation.html) on rabbitmq.com. + + +## License and Copyright + +Released under [the same license as RabbitMQ](https://www.rabbitmq.com/mpl.html). + +2007-2015 (c) 2007-2024 Broadcom. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. diff --git a/deps/rabbitmq_exchange_federation/include/rabbit_exchange_federation.hrl b/deps/rabbitmq_exchange_federation/include/rabbit_exchange_federation.hrl new file mode 100644 index 000000000000..e8ddecc7614e --- /dev/null +++ b/deps/rabbitmq_exchange_federation/include/rabbit_exchange_federation.hrl @@ -0,0 +1,8 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-define(FEDERATION_PG_SCOPE, rabbitmq_exchange_federation_pg_scope). diff --git a/deps/rabbitmq_exchange_federation/src/rabbit_exchange_federation_app.erl b/deps/rabbitmq_exchange_federation/src/rabbit_exchange_federation_app.erl new file mode 100644 index 000000000000..dfdc0677d10b --- /dev/null +++ b/deps/rabbitmq_exchange_federation/src/rabbit_exchange_federation_app.erl @@ -0,0 +1,51 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_exchange_federation_app). + +-include_lib("rabbitmq_federation_common/include/rabbit_federation.hrl"). +-include("rabbit_exchange_federation.hrl"). + +-behaviour(application). +-export([start/2, stop/1]). + +%% Dummy supervisor - see Ulf Wiger's comment at +%% http://erlang.org/pipermail/erlang-questions/2010-April/050508.html + +%% All of our actual server processes are supervised by +%% rabbit_federation_sup, which is started by a rabbit_boot_step +%% (since it needs to start up before queue / exchange recovery, so it +%% can't be part of our application). +%% +%% However, we still need an application behaviour since we need to +%% know when our application has started since then the Erlang client +%% will have started and we can therefore start our links going. Since +%% the application behaviour needs a tree of processes to supervise, +%% this is it... +-behaviour(supervisor). +-export([init/1]). + +start(_Type, _StartArgs) -> + ets:insert(?FEDERATION_ETS, + {rabbitmq_exchange_federation, + #{link_module => rabbit_federation_exchange_link_sup_sup}}), + supervisor:start_link({local, ?MODULE}, ?MODULE, []). + +stop(_State) -> + ets:delete(?FEDERATION_ETS, rabbitmq_exchange_federation), + rabbit_federation_pg:stop_scope(?FEDERATION_PG_SCOPE), + ok. + +%%---------------------------------------------------------------------------- + +init([]) -> + Flags = #{ + strategy => one_for_one, + intensity => 3, + period => 10 + }, + {ok, {Flags, []}}. diff --git a/deps/rabbitmq_exchange_federation/src/rabbit_exchange_federation_sup.erl b/deps/rabbitmq_exchange_federation/src/rabbit_exchange_federation_sup.erl new file mode 100644 index 000000000000..886435630e99 --- /dev/null +++ b/deps/rabbitmq_exchange_federation/src/rabbit_exchange_federation_sup.erl @@ -0,0 +1,64 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_exchange_federation_sup). + +-behaviour(supervisor). + +%% Supervises everything. There is just one of these. + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include("rabbit_exchange_federation.hrl"). + +-define(SUPERVISOR, ?MODULE). + +-export([start_link/0, stop/0]). + +-export([init/1]). + +%% This supervisor needs to be part of the rabbit application since +%% a) it needs to be in place when exchange recovery takes place +%% b) it needs to go up and down with rabbit + +-rabbit_boot_step({rabbit_exchange_federation_supervisor, + [{description, "federation"}, + {mfa, {rabbit_sup, start_child, [?MODULE]}}, + {requires, [kernel_ready, rabbit_federation_supervisor]}, + {cleanup, {?MODULE, stop, []}}, + {enables, rabbit_federation_exchange}]}). + +%%---------------------------------------------------------------------------- + +start_link() -> + supervisor:start_link({local, ?SUPERVISOR}, ?MODULE, []). + +stop() -> + ok = supervisor:terminate_child(rabbit_sup, ?MODULE), + ok = supervisor:delete_child(rabbit_sup, ?MODULE). + +%%---------------------------------------------------------------------------- + +init([]) -> + XLinkSupSup = #{ + id => x_links, + start => {rabbit_federation_exchange_link_sup_sup, start_link, []}, + restart => transient, + shutdown => ?SUPERVISOR_WAIT, + type => supervisor, + modules =>[rabbit_federation_exchange_link_sup_sup] + }, + %% with default reconnect-delay of 5 second, this supports up to + %% 100 links constantly failing and being restarted a minute + %% (or 200 links if reconnect-delay is 10 seconds, 600 with 30 seconds, + %% etc: N * (60/reconnect-delay) <= 1200) + Flags = #{ + strategy => one_for_one, + intensity => 1200, + period => 60 + }, + Specs = [XLinkSupSup], + {ok, {Flags, Specs}}. diff --git a/deps/rabbitmq_exchange_federation/src/rabbit_federation_exchange.erl b/deps/rabbitmq_exchange_federation/src/rabbit_federation_exchange.erl new file mode 100644 index 000000000000..cc41a22b6edf --- /dev/null +++ b/deps/rabbitmq_exchange_federation/src/rabbit_federation_exchange.erl @@ -0,0 +1,97 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +%% TODO rename this +-module(rabbit_federation_exchange). + +-include_lib("amqp_client/include/amqp_client.hrl"). +-include("rabbit_exchange_federation.hrl"). + +-rabbit_boot_step({?MODULE, + [{description, "federation exchange decorator"}, + {mfa, {rabbit_registry, register, + [exchange_decorator, <<"federation">>, ?MODULE]}}, + {cleanup, {rabbit_registry, unregister, + [exchange_decorator, <<"federation">>]}}, + {requires, [rabbit_registry, recovery]}]}). + +-behaviour(rabbit_exchange_decorator). + +-export([description/0, serialise_events/1]). +-export([create/2, delete/2, policy_changed/2, + add_binding/3, remove_bindings/3, route/2, active_for/1]). + +%%---------------------------------------------------------------------------- + +description() -> + [{description, <<"Federation exchange decorator">>}]. + +serialise_events(X) -> federate(X). + +create(_Serial, X) -> + maybe_start(X). + +delete(_Serial, X) -> + maybe_stop(X). + +policy_changed(OldX, NewX) -> + maybe_stop(OldX), + maybe_start(NewX). + +add_binding(Serial, X = #exchange{name = XName}, B) -> + case federate(X) of + true -> _ = rabbit_federation_exchange_link:add_binding(Serial, XName, B), + ok; + false -> ok + end. + +remove_bindings(Serial, X = #exchange{name = XName}, Bs) -> + case federate(X) of + true -> _ = rabbit_federation_exchange_link:remove_bindings(Serial, XName, Bs), + ok; + false -> ok + end. + +route(_, _) -> []. + +active_for(X) -> + case federate(X) of + true -> noroute; + false -> none + end. + +%%---------------------------------------------------------------------------- + +%% Don't federate default exchange, we can't bind to it +federate(#exchange{name = #resource{name = <<"">>}}) -> + false; + +%% Don't federate any of our intermediate exchanges. Note that we use +%% internal=true since older brokers may not declare +%% x-federation-upstream on us. Also other internal exchanges should +%% probably not be federated. +federate(#exchange{internal = true}) -> + false; + +federate(X) -> + rabbit_federation_upstream:federate(X). + +maybe_start(X = #exchange{name = XName})-> + case federate(X) of + true -> ok = rabbit_federation_db:prune_scratch( + XName, rabbit_federation_upstream:for(X)), + ok = rabbit_federation_exchange_link_sup_sup:start_child(X), + ok; + false -> ok + end. + +maybe_stop(X = #exchange{name = XName}) -> + case federate(X) of + true -> ok = rabbit_federation_exchange_link_sup_sup:stop_child(X), + rabbit_federation_status:remove_exchange_or_queue(XName); + false -> ok + end. diff --git a/deps/rabbitmq_exchange_federation/src/rabbit_federation_exchange_link.erl b/deps/rabbitmq_exchange_federation/src/rabbit_federation_exchange_link.erl new file mode 100644 index 000000000000..81d8a493335f --- /dev/null +++ b/deps/rabbitmq_exchange_federation/src/rabbit_federation_exchange_link.erl @@ -0,0 +1,696 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_exchange_link). + +-include_lib("amqp_client/include/amqp_client.hrl"). +-include_lib("rabbitmq_federation_common/include/rabbit_federation.hrl"). +-include("rabbit_exchange_federation.hrl"). + +-behaviour(gen_server2). + +-export([go/0, add_binding/3, remove_bindings/3]). +-export([list_routing_keys/1]). %% For testing + +-export([start_link/1]). + +-export([init/1, handle_call/3, handle_cast/2, handle_info/2, + terminate/2, code_change/3]). + +-import(rabbit_misc, [pget/2]). +-import(rabbit_federation_util, [name/1, vhost/1, pgname/1]). + +-record(state, {upstream, + upstream_params, + upstream_name, + connection, + channel, + cmd_channel, + consumer_tag, + queue, + internal_exchange, + waiting_cmds = gb_trees:empty(), + next_serial, + bindings = #{}, + downstream_connection, + downstream_channel, + downstream_exchange, + unacked, + internal_exchange_timer, + internal_exchange_interval}). + +%%---------------------------------------------------------------------------- + +%% We start off in a state where we do not connect, since we can first +%% start during exchange recovery, when rabbit is not fully started +%% and the Erlang client is not running. This then gets invoked when +%% the federation app is started. +go() -> + _ = rabbit_federation_pg:start_scope(?FEDERATION_PG_SCOPE), + cast(go). + +add_binding(S, XN, B) -> cast(XN, {enqueue, S, {add_binding, B}}). +remove_bindings(S, XN, Bs) -> cast(XN, {enqueue, S, {remove_bindings, Bs}}). + +list_routing_keys(XN) -> call(XN, list_routing_keys). + +%%---------------------------------------------------------------------------- + +start_link(Args) -> + gen_server2:start_link(?MODULE, Args, [{timeout, infinity}]). + +init({Upstream, XName}) -> + %% If we are starting up due to a policy change then it's possible + %% for the exchange to have been deleted before we got here, in which + %% case it's possible that delete callback would also have been called + %% before we got here. So check if we still exist. + case rabbit_exchange:lookup(XName) of + {ok, X} -> + DeobfuscatedUpstream = rabbit_federation_util:deobfuscate_upstream(Upstream), + DeobfuscatedUParams = rabbit_federation_upstream:to_params(DeobfuscatedUpstream, X), + UParams = rabbit_federation_util:obfuscate_upstream_params(DeobfuscatedUParams), + rabbit_federation_status:report(Upstream, UParams, XName, starting), + join(rabbit_federation_exchanges), + join({rabbit_federation_exchange, XName}), + gen_server2:cast(self(), maybe_go), + {ok, {not_started, {Upstream, UParams, XName}}}; + {error, not_found} -> + rabbit_federation_link_util:log_warning(XName, "not found, stopping link", []), + {stop, gone} + end. + +handle_call(list_routing_keys, _From, State = #state{bindings = Bindings}) -> + {reply, lists:sort([K || {K, _} <- maps:keys(Bindings)]), State}; + +handle_call(Msg, _From, State) -> + {stop, {unexpected_call, Msg}, State}. + +handle_cast(maybe_go, State = {not_started, _Args}) -> + go(State); + +handle_cast(go, S0 = {not_started, _Args}) -> + go(S0); + +%% There's a small race - I think we can realise federation is up +%% before 'go' gets invoked. Ignore. +handle_cast(go, State) -> + {noreply, State}; + +handle_cast({enqueue, _, _}, State = {not_started, _}) -> + {noreply, State}; + +handle_cast({enqueue, Serial, Cmd}, + State = #state{waiting_cmds = Waiting, + downstream_exchange = XName}) -> + Waiting1 = gb_trees:insert(Serial, Cmd, Waiting), + try + {noreply, play_back_commands(State#state{waiting_cmds = Waiting1})} + catch exit:{{shutdown, {server_initiated_close, 404, Text}}, _} -> + rabbit_federation_link_util:log_warning( + XName, "detected upstream changes, restarting link: ~tp", [Text]), + {stop, {shutdown, restart}, State} + end; + +handle_cast(Msg, State) -> + {stop, {unexpected_cast, Msg}, State}. + +handle_info(#'basic.consume_ok'{}, State) -> + {noreply, State}; + +handle_info(#'basic.ack'{} = Ack, State = #state{channel = Ch, + unacked = Unacked}) -> + Unacked1 = rabbit_federation_link_util:ack(Ack, Ch, Unacked), + {noreply, State#state{unacked = Unacked1}}; + +handle_info(#'basic.nack'{} = Nack, State = #state{channel = Ch, + unacked = Unacked}) -> + Unacked1 = rabbit_federation_link_util:nack(Nack, Ch, Unacked), + {noreply, State#state{unacked = Unacked1}}; + +handle_info({#'basic.deliver'{routing_key = Key, + redelivered = Redelivered} = DeliverMethod, Msg}, + State = #state{ + upstream = Upstream = #upstream{max_hops = MaxH}, + upstream_params = UParams = #upstream_params{x_or_q = UpstreamX}, + upstream_name = UName, + downstream_exchange = #resource{name = XNameBin, virtual_host = DVhost}, + downstream_channel = DCh, + channel = Ch, + unacked = Unacked}) -> + UVhost = vhost(UpstreamX), + PublishMethod = #'basic.publish'{exchange = XNameBin, + routing_key = Key}, + HeadersFun = fun (H) -> update_routing_headers(UParams, UName, UVhost, Redelivered, H) end, + %% We need to check should_forward/2 here in case the upstream + %% does not have federation and thus is using a fanout exchange. + ForwardFun = fun (H) -> + DName = rabbit_nodes:cluster_name(), + rabbit_federation_util:should_forward(H, MaxH, DName, DVhost) + end, + Unacked1 = rabbit_federation_link_util:forward( + Upstream, DeliverMethod, Ch, DCh, PublishMethod, + HeadersFun, ForwardFun, Msg, Unacked), + {noreply, State#state{unacked = Unacked1}}; + +handle_info(#'basic.cancel'{}, State = #state{upstream = Upstream, + upstream_params = UParams, + downstream_exchange = XName}) -> + rabbit_federation_link_util:connection_error( + local, basic_cancel, Upstream, UParams, XName, State); + +handle_info({'DOWN', _Ref, process, Pid, Reason}, + State = #state{downstream_channel = DCh, + channel = Ch, + cmd_channel = CmdCh, + upstream = Upstream, + upstream_params = UParams, + downstream_exchange = XName}) -> + handle_down(Pid, Reason, Ch, CmdCh, DCh, + {Upstream, UParams, XName}, State); + +handle_info(check_internal_exchange, State = #state{internal_exchange = IntXNameBin, + internal_exchange_interval = Interval}) -> + case check_internal_exchange(IntXNameBin, State) of + upstream_not_found -> + rabbit_log_federation:warning("Federation link could not find upstream exchange '~ts' and will restart", + [IntXNameBin]), + {stop, {shutdown, restart}, State}; + _ -> + TRef = erlang:send_after(Interval, self(), check_internal_exchange), + {noreply, State#state{internal_exchange_timer = TRef}} + end; + +handle_info(Msg, State) -> + {stop, {unexpected_info, Msg}, State}. + +terminate(_Reason, {not_started, _}) -> + ok; +terminate(Reason, #state{downstream_connection = DConn, + connection = Conn, + upstream = Upstream, + upstream_params = UParams, + downstream_exchange = XName, + internal_exchange_timer = TRef, + internal_exchange = IntExchange, + queue = Queue}) when Reason =:= shutdown; + Reason =:= {shutdown, restart}; + Reason =:= gone -> + _ = timer:cancel(TRef), + rabbit_federation_link_util:ensure_connection_closed(DConn), + + rabbit_log:debug("Exchange federation: link is shutting down, resource cleanup mode: ~tp", [Upstream#upstream.resource_cleanup_mode]), + case Upstream#upstream.resource_cleanup_mode of + never -> ok; + _ -> + %% This is a normal shutdown and we are allowed to clean up the internally used queue and exchange + rabbit_log:debug("Federated exchange '~ts' link will delete its internal queue '~ts'", [Upstream#upstream.exchange_name, Queue]), + delete_upstream_queue(Conn, Queue), + rabbit_log:debug("Federated exchange '~ts' link will delete its upstream exchange", [Upstream#upstream.exchange_name]), + delete_upstream_exchange(Conn, IntExchange) + end, + + rabbit_federation_link_util:ensure_connection_closed(Conn), + rabbit_federation_link_util:log_terminate(Reason, Upstream, UParams, XName), + ok; +%% unexpected shutdown +terminate(Reason, #state{downstream_connection = DConn, + connection = Conn, + upstream = Upstream, + upstream_params = UParams, + downstream_exchange = XName, + internal_exchange_timer = TRef}) -> + _ = timer:cancel(TRef), + + rabbit_federation_link_util:ensure_connection_closed(DConn), + + %% unlike in the clean shutdown case above, we keep the queue + %% and exchange around + + rabbit_federation_link_util:ensure_connection_closed(Conn), + rabbit_federation_link_util:log_terminate(Reason, Upstream, UParams, XName), + ok. + +code_change(_OldVsn, State, _Extra) -> + {ok, State}. + +%%---------------------------------------------------------------------------- + +call(XName, Msg) -> [gen_server2:call(Pid, Msg, infinity) || Pid <- x(XName)]. +cast(Msg) -> [gen_server2:cast(Pid, Msg) || Pid <- all()]. +cast(XName, Msg) -> [gen_server2:cast(Pid, Msg) || Pid <- x(XName)]. + +join(Name) -> + ok = pg:join(?FEDERATION_PG_SCOPE, pgname(Name), self()). + +all() -> + pg:get_members(?FEDERATION_PG_SCOPE, pgname(rabbit_federation_exchanges)). + +x(XName) -> + pg:get_members(?FEDERATION_PG_SCOPE, pgname({rabbit_federation_exchange, XName})). + +%%---------------------------------------------------------------------------- + +handle_command({add_binding, Binding}, State) -> + add_binding(Binding, State); + +handle_command({remove_bindings, Bindings}, State) -> + lists:foldl(fun remove_binding/2, State, Bindings). + +play_back_commands(State = #state{waiting_cmds = Waiting, + next_serial = Next}) -> + case gb_trees:is_empty(Waiting) of + false -> case gb_trees:take_smallest(Waiting) of + {Next, Cmd, Waiting1} -> + %% The next one. Just execute it. + play_back_commands( + handle_command(Cmd, State#state{ + waiting_cmds = Waiting1, + next_serial = Next + 1})); + {Serial, _Cmd, Waiting1} when Serial < Next -> + %% This command came from before we executed + %% binding:list_for_source. Ignore it. + play_back_commands(State#state{ + waiting_cmds = Waiting1}); + _ -> + %% Some future command. Don't do anything. + State + end; + true -> State + end. + +add_binding(B, State) -> + binding_op(fun record_binding/2, bind_cmd(bind, B, State), B, State). + +remove_binding(B, State) -> + binding_op(fun forget_binding/2, bind_cmd(unbind, B, State), B, State). + +record_binding(B = #binding{destination = Dest}, + State = #state{bindings = Bs}) -> + {DoIt, Set} = case maps:find(key(B), Bs) of + error -> {true, sets:from_list([Dest])}; + {ok, Dests} -> {false, sets:add_element( + Dest, Dests)} + end, + {DoIt, State#state{bindings = maps:put(key(B), Set, Bs)}}. + +forget_binding(B = #binding{destination = Dest}, + State = #state{bindings = Bs}) -> + Dests = sets:del_element(Dest, maps:get(key(B), Bs)), + {DoIt, Bs1} = case sets:size(Dests) of + 0 -> {true, maps:remove(key(B), Bs)}; + _ -> {false, maps:put(key(B), Dests, Bs)} + end, + {DoIt, State#state{bindings = Bs1}}. + +binding_op(UpdateFun, Cmd, B = #binding{args = Args}, + State = #state{cmd_channel = Ch}) -> + {DoIt, State1} = + case rabbit_misc:table_lookup(Args, ?BINDING_HEADER) of + undefined -> UpdateFun(B, State); + {array, _} -> {Cmd =/= ignore, State} + end, + case DoIt of + true -> amqp_channel:call(Ch, Cmd); + false -> ok + end, + State1. + +bind_cmd(Type, #binding{key = Key, args = Args}, + State = #state{internal_exchange = IntXNameBin, + upstream_params = UpstreamParams, + upstream = Upstream}) -> + #upstream_params{x_or_q = X} = UpstreamParams, + #upstream{bind_nowait = Nowait} = Upstream, + case update_binding(Args, State) of + ignore -> ignore; + NewArgs -> bind_cmd0(Type, name(X), IntXNameBin, Key, NewArgs, Nowait) + end. + +bind_cmd0(bind, Source, Destination, RoutingKey, Arguments, Nowait) -> + #'exchange.bind'{source = Source, + destination = Destination, + routing_key = RoutingKey, + arguments = Arguments, + nowait = Nowait}; + +bind_cmd0(unbind, Source, Destination, RoutingKey, Arguments, Nowait) -> + #'exchange.unbind'{source = Source, + destination = Destination, + routing_key = RoutingKey, + arguments = Arguments, + nowait = Nowait}. + +%% This function adds information about the current node to the +%% binding arguments, or returns 'ignore' if it determines the binding +%% should propagate no further. The interesting part is the latter. +%% +%% We want bindings to propagate in the same way as messages +%% w.r.t. max_hops - if we determine that a message can get from node +%% A to B (assuming bindings are in place) then it follows that a +%% binding at B should propagate back to A, and no further. There is +%% no point in propagating bindings past the point where messages +%% would propagate, and we will lose messages if bindings don't +%% propagate as far. +%% +%% Note that we still want to have limits on how far messages can +%% propagate: limiting our bindings is not enough, since other +%% bindings from other nodes can overlap. +%% +%% So in short we want bindings to obey max_hops. However, they can't +%% just obey the max_hops of the current link, since they are +%% travelling in the opposite direction to messages! Consider the +%% following federation: +%% +%% A -----------> B -----------> C +%% max_hops=1 max_hops=2 +%% +%% where the arrows indicate message flow. A binding created at C +%% should propagate to B, then to A, and no further. Therefore every +%% time we traverse a link, we keep a count of the number of hops that +%% a message could have made so far to reach this point, and still be +%% able to propagate. When this number ("hops" below) reaches 0 we +%% propagate no further. +%% +%% hops(link(N)) is given by: +%% +%% min(hops(link(N-1))-1, max_hops(link(N))) +%% +%% where link(N) is the link that bindings propagate over after N +%% steps (e.g. link(1) is CB above, link(2) is BA). +%% +%% In other words, we count down to 0 from the link with the most +%% restrictive max_hops we have yet passed through. + +update_binding(Args, #state{downstream_exchange = X, + upstream = Upstream, + upstream_params = #upstream_params{x_or_q = UpstreamX}, + upstream_name = UName}) -> + #upstream{max_hops = MaxHops} = Upstream, + UVhost = vhost(UpstreamX), + Hops = case rabbit_misc:table_lookup(Args, ?BINDING_HEADER) of + undefined -> MaxHops; + {array, All} -> [{table, Prev} | _] = All, + PrevHops = get_hops(Prev), + case rabbit_federation_util:already_seen( + UName, UVhost, All) of + true -> 0; + false -> lists:min([PrevHops - 1, MaxHops]) + end + end, + case Hops of + 0 -> ignore; + _ -> Cluster = rabbit_nodes:cluster_name(), + ABSuffix = rabbit_federation_db:get_active_suffix( + X, Upstream, <<"A">>), + DVhost = vhost(X), + DName = name(X), + Down = <>, + Info = [{<<"cluster-name">>, longstr, Cluster}, + {<<"vhost">>, longstr, DVhost}, + {<<"exchange">>, longstr, Down}, + {<<"hops">>, short, Hops}], + rabbit_basic:prepend_table_header(?BINDING_HEADER, Info, Args) + end. + + + +key(#binding{key = Key, args = Args}) -> {Key, Args}. + +go(S0 = {not_started, {Upstream, UParams, DownXName}}) -> + Unacked = rabbit_federation_link_util:unacked_new(), + log_link_startup_attempt(Upstream, DownXName), + rabbit_federation_link_util:start_conn_ch( + fun (Conn, Ch, DConn, DCh) -> + {ok, CmdCh} = + case Upstream#upstream.channel_use_mode of + single -> reuse_command_channel(Ch, Upstream, DownXName); + multiple -> open_command_channel(Conn, Upstream, UParams, DownXName, S0); + _ -> open_command_channel(Conn, Upstream, UParams, DownXName, S0) + end, + erlang:monitor(process, CmdCh), + Props = pget(server_properties, + amqp_connection:info(Conn, [server_properties])), + UName = case rabbit_misc:table_lookup( + Props, <<"cluster_name">>) of + {longstr, N} -> N; + _ -> unknown + 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 + %% overtaken the creation of this process). However, this + %% is not a big deal - 'undefined' just becomes the next + %% serial we will process. Since it compares larger than + %% any number we never process any commands. And we will + %% soon get told to stop anyway. + {ok, Interval} = application:get_env(rabbitmq_exchange_federation, + internal_exchange_check_interval), + State = ensure_upstream_bindings( + consume_from_upstream_queue( + #state{upstream = Upstream, + upstream_params = UParams, + upstream_name = UName, + connection = Conn, + channel = Ch, + cmd_channel = CmdCh, + next_serial = Serial, + downstream_connection = DConn, + downstream_channel = DCh, + downstream_exchange = DownXName, + unacked = Unacked, + internal_exchange_interval = Interval}), + Bindings), + rabbit_log_federation:info("Federation link for ~ts (upstream: ~ts) will perform internal exchange checks " + "every ~b seconds", [rabbit_misc:rs(DownXName), UName, round(Interval / 1000)]), + TRef = erlang:send_after(Interval, self(), check_internal_exchange), + {noreply, State#state{internal_exchange_timer = TRef}} + end, Upstream, UParams, DownXName, S0). + +log_link_startup_attempt(#upstream{name = Name, channel_use_mode = ChMode}, DownXName) -> + rabbit_log_federation:debug("Will try to start a federation link for ~ts, upstream: '~ts', channel use mode: ~ts", + [rabbit_misc:rs(DownXName), Name, ChMode]). + +%% If channel use mode is 'single', reuse the message transfer channel. +%% Otherwise open a separate one. +reuse_command_channel(MainCh, #upstream{name = UName}, DownXName) -> + rabbit_log_federation:debug("Will use a single channel for both schema operations and message transfer on links to upstream '~ts' for downstream federated ~ts", + [UName, rabbit_misc:rs(DownXName)]), + {ok, MainCh}. + +open_command_channel(Conn, Upstream = #upstream{name = UName}, UParams, DownXName, S0) -> + rabbit_log_federation:debug("Will open a command channel to upstream '~ts' for downstream federated ~ts", + [UName, rabbit_misc:rs(DownXName)]), + case amqp_connection:open_channel(Conn) of + {ok, CCh} -> + erlang:monitor(process, CCh), + {ok, CCh}; + E -> + rabbit_federation_link_util:ensure_connection_closed(Conn), + _ = rabbit_federation_link_util:connection_error(command_channel, E, + Upstream, UParams, DownXName, S0), + E + end. + +consume_from_upstream_queue( + State = #state{upstream = Upstream, + upstream_params = UParams, + channel = Ch, + downstream_exchange = DownXName}) -> + #upstream{prefetch_count = Prefetch, + expires = Expiry, + message_ttl = TTL, + queue_type = QueueType} = Upstream, + #upstream_params{x_or_q = X, + params = Params} = UParams, + Q = upstream_queue_name(name(X), vhost(Params), DownXName), + Args = [A || {_K, _T, V} = A + <- [{<<"x-expires">>, long, Expiry}, + {<<"x-message-ttl">>, long, TTL}, + {<<"x-internal-purpose">>, longstr, <<"federation">>}, + {<<"x-queue-type">>, longstr, atom_to_binary(QueueType)} + ], + V =/= none], + amqp_channel:call(Ch, #'queue.declare'{queue = Q, + durable = true, + arguments = Args}), + NoAck = Upstream#upstream.ack_mode =:= 'no-ack', + case NoAck of + false -> amqp_channel:call(Ch, #'basic.qos'{prefetch_count = Prefetch}); + true -> ok + end, + #'basic.consume_ok'{consumer_tag = CTag} = + amqp_channel:subscribe(Ch, #'basic.consume'{queue = Q, + no_ack = NoAck}, self()), + State#state{consumer_tag = CTag, + queue = Q}. + +ensure_upstream_bindings(State = #state{upstream = Upstream, + connection = Conn, + channel = Ch, + downstream_exchange = DownXName, + queue = Q}, Bindings) -> + OldSuffix = rabbit_federation_db:get_active_suffix( + DownXName, Upstream, <<"A">>), + Suffix = case OldSuffix of + <<"A">> -> <<"B">>; + <<"B">> -> <<"A">> + end, + IntXNameBin = upstream_exchange_name(Q, Suffix), + ensure_upstream_exchange(State), + ensure_internal_exchange(IntXNameBin, State), + amqp_channel:call(Ch, #'queue.bind'{exchange = IntXNameBin, queue = Q}), + State1 = State#state{internal_exchange = IntXNameBin}, + rabbit_federation_db:set_active_suffix(DownXName, Upstream, Suffix), + State2 = lists:foldl(fun add_binding/2, State1, Bindings), + OldIntXNameBin = upstream_exchange_name(Q, OldSuffix), + delete_upstream_exchange(Conn, OldIntXNameBin), + State2. + +ensure_upstream_exchange(#state{upstream_params = UParams, + connection = Conn, + channel = Ch}) -> + #upstream_params{x_or_q = X} = UParams, + #exchange{type = Type, + durable = Durable, + auto_delete = AutoDelete, + internal = Internal, + arguments = Arguments} = X, + Decl = #'exchange.declare'{exchange = name(X), + type = list_to_binary(atom_to_list(Type)), + durable = Durable, + auto_delete = AutoDelete, + internal = Internal, + arguments = Arguments}, + rabbit_federation_link_util:disposable_channel_call( + Conn, Decl#'exchange.declare'{passive = true}, + fun(?NOT_FOUND, _Text) -> + amqp_channel:call(Ch, Decl) + end). + +ensure_internal_exchange(IntXNameBin, + #state{upstream = #upstream{max_hops = MaxHops, name = UName}, + upstream_params = UParams, + connection = Conn, + channel = Ch, + downstream_exchange = #resource{virtual_host = DVhost}}) -> + rabbit_log_federation:debug("Exchange federation will set up exchange '~ts' in upstream '~ts'", + [IntXNameBin, UName]), + #upstream_params{params = Params} = rabbit_federation_util:deobfuscate_upstream_params(UParams), + rabbit_log_federation:debug("Will delete upstream exchange '~ts'", [IntXNameBin]), + delete_upstream_exchange(Conn, IntXNameBin), + rabbit_log_federation:debug("Will declare an internal upstream exchange '~ts'", [IntXNameBin]), + Base = #'exchange.declare'{exchange = IntXNameBin, + durable = true, + internal = true, + auto_delete = true}, + Purpose = [{<<"x-internal-purpose">>, longstr, <<"federation">>}], + XFUArgs = [{?MAX_HOPS_ARG, long, MaxHops}, + {?DOWNSTREAM_NAME_ARG, longstr, cycle_detection_node_identifier()}, + {?DOWNSTREAM_VHOST_ARG, longstr, DVhost} + | Purpose], + XFU = Base#'exchange.declare'{type = <<"x-federation-upstream">>, + arguments = XFUArgs}, + Fan = Base#'exchange.declare'{type = <<"fanout">>, + arguments = Purpose}, + rabbit_federation_link_util:disposable_connection_call( + Params, XFU, fun(?COMMAND_INVALID, _Text) -> + amqp_channel:call(Ch, Fan) + end). + +check_internal_exchange(IntXNameBin, + #state{upstream = #upstream{max_hops = MaxHops, name = UName}, + upstream_params = UParams, + downstream_exchange = XName = #resource{virtual_host = DVhost}}) -> + #upstream_params{params = Params} = + rabbit_federation_util:deobfuscate_upstream_params(UParams), + rabbit_log_federation:debug("Exchange federation will check on exchange '~ts' in upstream '~ts'", + [IntXNameBin, UName]), + Base = #'exchange.declare'{exchange = IntXNameBin, + passive = true, + durable = true, + internal = true, + auto_delete = true}, + Purpose = [{<<"x-internal-purpose">>, longstr, <<"federation">>}], + XFUArgs = [{?MAX_HOPS_ARG, long, MaxHops}, + {?DOWNSTREAM_NAME_ARG, longstr, cycle_detection_node_identifier()}, + {?DOWNSTREAM_VHOST_ARG, longstr, DVhost} + | Purpose], + XFU = Base#'exchange.declare'{type = <<"x-federation-upstream">>, + arguments = XFUArgs}, + rabbit_federation_link_util:disposable_connection_call( + Params, XFU, fun(404, Text) -> + rabbit_federation_link_util:log_warning( + XName, "detected internal upstream exchange changes," + " restarting link: ~tp", [Text]), + upstream_not_found; + (Code, Text) -> + rabbit_federation_link_util:log_warning( + XName, "internal upstream exchange check failed: ~tp ~tp", + [Code, Text]), + error + end). + +upstream_queue_name(XNameBin, VHost, #resource{name = DownXNameBin, + virtual_host = DownVHost}) -> + Node = rabbit_nodes:cluster_name(), + DownPart = case DownVHost of + VHost -> case DownXNameBin of + XNameBin -> <<"">>; + _ -> <<":", DownXNameBin/binary>> + end; + _ -> <<":", DownVHost/binary, + ":", DownXNameBin/binary>> + end, + <<"federation: ", XNameBin/binary, " -> ", Node/binary, DownPart/binary>>. + +cycle_detection_node_identifier() -> + rabbit_nodes:cluster_name(). + +upstream_exchange_name(UpstreamQName, Suffix) -> + <>. + +delete_upstream_exchange(Conn, XNameBin) -> + rabbit_federation_link_util:disposable_channel_call( + Conn, #'exchange.delete'{exchange = XNameBin}). + +delete_upstream_queue(Conn, Queue) -> + rabbit_federation_link_util:disposable_channel_call( + Conn, #'queue.delete'{queue = Queue}). + +update_routing_headers(#upstream_params{table = Table}, UpstreamName, UVhost, Redelivered, Headers) -> + NewValue = Table ++ + [{<<"redelivered">>, bool, Redelivered}] ++ + header_for_upstream_name(UpstreamName) ++ + header_for_upstream_vhost(UVhost), + rabbit_basic:prepend_table_header(?ROUTING_HEADER, NewValue, Headers). + +header_for_upstream_name(unknown) -> []; +header_for_upstream_name(Name) -> [{<<"cluster-name">>, longstr, Name}]. + +header_for_upstream_vhost(unknown) -> []; +header_for_upstream_vhost(Name) -> [{<<"vhost">>, longstr, Name}]. + +get_hops(Table) -> + case rabbit_misc:table_lookup(Table, <<"hops">>) of + %% see rabbit_binary_generator + {short, N} -> N; + {long, N} -> N; + {byte, N} -> N; + {signedint, N} -> N; + {unsignedbyte, N} -> N; + {unsignedshort, N} -> N; + {unsignedint, N} -> N; + {_, N} when is_integer(N) andalso N >= 0 -> N + end. + +handle_down(DCh, Reason, _Ch, _CmdCh, DCh, Args, State) -> + rabbit_federation_link_util:handle_downstream_down(Reason, Args, State); +handle_down(ChPid, Reason, Ch, CmdCh, _DCh, Args, State) + when ChPid =:= Ch; ChPid =:= CmdCh -> + rabbit_federation_link_util:handle_upstream_down(Reason, Args, State). diff --git a/deps/rabbitmq_exchange_federation/src/rabbit_federation_exchange_link_sup_sup.erl b/deps/rabbitmq_exchange_federation/src/rabbit_federation_exchange_link_sup_sup.erl new file mode 100644 index 000000000000..4371fb0f0b7c --- /dev/null +++ b/deps/rabbitmq_exchange_federation/src/rabbit_federation_exchange_link_sup_sup.erl @@ -0,0 +1,90 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_exchange_link_sup_sup). + +-behaviour(mirrored_supervisor). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include("rabbit_exchange_federation.hrl"). +-define(SUPERVISOR, ?MODULE). + +%% Supervises the upstream links for all exchanges (but not queues). We need +%% different handling here since exchanges want a mirrored sup. + +-export([start_link/0, start_child/1, adjust/1, stop_child/1]). +-export([init/1]). +-export([id_to_khepri_path/1]). + +%%---------------------------------------------------------------------------- + +start_link() -> + _ = pg:start_link(), + %% This scope is used by concurrently starting exchange and queue links, + %% and other places, so we have to start it very early outside of the supervision tree. + %% The scope is stopped in stop/1. + _ = rabbit_federation_pg:start_scope(?FEDERATION_PG_SCOPE), + mirrored_supervisor:start_link({local, ?SUPERVISOR}, ?SUPERVISOR, + ?MODULE, []). + +%% Note that the next supervisor down, rabbit_federation_link_sup, is common +%% between exchanges and queues. +start_child(X) -> + case mirrored_supervisor:start_child( + ?SUPERVISOR, + {id(X), {rabbit_federation_link_sup, start_link, + [rabbit_federation_exchange_link, X]}, + transient, ?SUPERVISOR_WAIT, supervisor, + [rabbit_federation_link_sup]}) of + {ok, _Pid} -> ok; + {error, {already_started, _Pid}} -> + #exchange{name = ExchangeName} = X, + rabbit_log_federation:debug("Federation link for exchange ~tp was already started", + [rabbit_misc:rs(ExchangeName)]), + ok; + %% A link returned {stop, gone}, the link_sup shut down, that's OK. + {error, {shutdown, _}} -> ok + end. + +adjust({clear_upstream, VHost, UpstreamName}) -> + _ = [rabbit_federation_link_sup:adjust(Pid, rabbit_federation_exchange_link, X, + {clear_upstream, UpstreamName}) || + {#exchange{name = Name} = X, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR), + Name#resource.virtual_host == VHost], + ok; +adjust(Reason) -> + _ = [rabbit_federation_link_sup:adjust(Pid, rabbit_federation_exchange_link, + X, Reason) || + {X, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR)], + ok. + +stop_child(X) -> + case mirrored_supervisor:terminate_child(?SUPERVISOR, id(X)) of + ok -> ok; + {error, Err} -> + #exchange{name = ExchangeName} = X, + rabbit_log_federation:warning( + "Attempt to stop a federation link for exchange ~tp failed: ~tp", + [rabbit_misc:rs(ExchangeName), Err]), + ok + end, + ok = mirrored_supervisor:delete_child(?SUPERVISOR, id(X)). + +%%---------------------------------------------------------------------------- + +init([]) -> + {ok, {{one_for_one, 1200, 60}, []}}. + +%% See comment in rabbit_federation_queue_link_sup_sup:id/1 +id(X = #exchange{policy = Policy}) -> + X1 = rabbit_exchange:immutable(X), + X2 = X1#exchange{policy = Policy}, + X2. + +id_to_khepri_path( + #exchange{name = #resource{virtual_host = VHost, name = Name}}) -> + [exchange, VHost, Name]. diff --git a/deps/rabbitmq_exchange_federation/src/rabbit_federation_upstream_exchange.erl b/deps/rabbitmq_exchange_federation/src/rabbit_federation_upstream_exchange.erl new file mode 100644 index 000000000000..23e4de27ce22 --- /dev/null +++ b/deps/rabbitmq_exchange_federation/src/rabbit_federation_upstream_exchange.erl @@ -0,0 +1,91 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_upstream_exchange). + +-rabbit_boot_step({?MODULE, + [{description, "federation upstream exchange type"}, + {mfa, {rabbit_registry, register, + [exchange, <<"x-federation-upstream">>, ?MODULE]}}, + {requires, rabbit_registry}, + {cleanup, {rabbit_registry, unregister, + [exchange, <<"x-federation-upstream">>]}}, + {enables, recovery}]}). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("rabbitmq_federation_common/include/rabbit_federation.hrl"). +-include("rabbit_exchange_federation.hrl"). + +-behaviour(rabbit_exchange_type). + +-export([description/0, serialise_events/0, route/3]). +-export([validate/1, validate_binding/2, + create/2, delete/2, policy_changed/2, + add_binding/3, remove_bindings/3, assert_args_equivalence/2]). +-export([info/1, info/2]). + +%%---------------------------------------------------------------------------- + +info(_X) -> []. +info(_X, _) -> []. + +description() -> + [{description, <<"Federation upstream helper exchange">>}, + {internal_purpose, federation}]. + +serialise_events() -> false. + +route(X = #exchange{arguments = Args}, Msg, _Opts) -> + %% This arg was introduced in the same release as this exchange type; + %% it must be set + {long, MaxHops} = rabbit_misc:table_lookup(Args, ?MAX_HOPS_ARG), + %% Will be missing for pre-3.3.0 versions + DName = case rabbit_misc:table_lookup(Args, ?DOWNSTREAM_NAME_ARG) of + {longstr, Val0} -> Val0; + _ -> unknown + end, + %% Will be missing for pre-3.8.9 versions + DVhost = case rabbit_misc:table_lookup(Args, ?DOWNSTREAM_VHOST_ARG) of + {longstr, Val1} -> Val1; + _ -> unknown + end, + case should_forward(Msg, MaxHops, DName, DVhost) of + true -> rabbit_exchange_type_fanout:route(X, Msg); + false -> [] + end. + + +should_forward(Msg, MaxHops, DName, DVhost) -> + case mc:x_header(?ROUTING_HEADER, Msg) of + {list, A} -> + length(A) < MaxHops andalso + not already_seen(DName, DVhost, A); + _ -> + true + end. + +already_seen(DName, DVhost, List) -> + lists:any(fun (Map) -> + {utf8, DName} =:= mc_util:amqp_map_get(<<"cluster-name">>, Map, undefined) andalso + {utf8, DVhost} =:= mc_util:amqp_map_get(<<"vhost">>, Map, undefined) + end, List). + + +validate(#exchange{arguments = Args}) -> + rabbit_federation_util:validate_arg(?MAX_HOPS_ARG, long, Args). + +validate_binding(_X, _B) -> ok. +create(_Serial, _X) -> ok. +delete(_Serial, _X) -> ok. +policy_changed(_X1, _X2) -> ok. +add_binding(_Serial, _X, _B) -> ok. +remove_bindings(_Serial, _X, _Bs) -> ok. + +assert_args_equivalence(X = #exchange{name = Name, + arguments = Args}, ReqArgs) -> + rabbit_misc:assert_args_equivalence(Args, ReqArgs, Name, [?MAX_HOPS_ARG]), + rabbit_exchange:assert_args_equivalence(X, Args). diff --git a/deps/rabbitmq_exchange_federation/test/definition_import_SUITE.erl b/deps/rabbitmq_exchange_federation/test/definition_import_SUITE.erl new file mode 100644 index 000000000000..d656d187f1e1 --- /dev/null +++ b/deps/rabbitmq_exchange_federation/test/definition_import_SUITE.erl @@ -0,0 +1,104 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(definition_import_SUITE). + +-include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-compile(export_all). + +all() -> + [ + {group, roundtrip} + ]. + +groups() -> + [ + {roundtrip, [], [ + export_import_round_trip + ]} + ]. + +%% ------------------------------------------------------------------- +%% Test suite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + inets:start(), + Config. +end_per_suite(Config) -> + Config. + +init_per_group(Group, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, Group} + ]), + rabbit_ct_helpers:run_setup_steps(Config1, rabbit_ct_broker_helpers:setup_steps()). + +end_per_group(_, Config) -> + rabbit_ct_helpers:run_teardown_steps(Config, 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). + +%% +%% Tests +%% + +export_import_round_trip(Config) -> + case rabbit_ct_helpers:is_mixed_versions() of + false -> + import_file_case(Config, "case1"), + Defs = export(Config), + import_raw(Config, rabbit_json:encode(Defs)); + _ -> + %% skip the test in mixed version mode + {skip, "Should not run in mixed version environments"} + end. + +%% +%% Implementation +%% + +import_file_case(Config, CaseName) -> + CasePath = filename:join([ + ?config(data_dir, Config), + CaseName ++ ".json" + ]), + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, run_import_case, [CasePath]), + ok. + + +import_raw(Config, Body) -> + case rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_definitions, import_raw, [Body]) of + ok -> ok; + {error, E} -> + ct:pal("Import of JSON definitions ~tp failed: ~tp~n", [Body, E]), + ct:fail({expected_failure, Body, E}) + end. + +export(Config) -> + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, run_export, []). + +run_export() -> + rabbit_definitions:all_definitions(). + +run_import_case(Path) -> + {ok, Body} = file:read_file(Path), + ct:pal("Successfully loaded a definition to import from ~tp~n", [Path]), + case rabbit_definitions:import_raw(Body) of + ok -> ok; + {error, E} -> + ct:pal("Import case ~tp failed: ~tp~n", [Path, E]), + ct:fail({expected_failure, Path, E}) + end. diff --git a/deps/rabbitmq_exchange_federation/test/definition_import_SUITE_data/case1.json b/deps/rabbitmq_exchange_federation/test/definition_import_SUITE_data/case1.json new file mode 100644 index 000000000000..e549e4fd6c1d --- /dev/null +++ b/deps/rabbitmq_exchange_federation/test/definition_import_SUITE_data/case1.json @@ -0,0 +1,52 @@ +{ + "permissions": [ + { + "configure": ".*", + "read": ".*", + "user": "guest", + "vhost": "/", + "write": ".*" + } + ], + "bindings": [], + "queues": [], + "parameters": [ + { + "component": "federation-upstream-set", + "name": "location-1", + "value": [ + { + "upstream":"up-1" + }, + { + "upstream":"up-2" + } + ], + "vhost":"/"}], + "policies": [], + "rabbitmq_version": "3.13.0+376.g1bc0d89.dirty", + "users": [ + { + "hashing_algorithm": "rabbit_password_hashing_sha256", + "limits": {}, + "name": "guest", + "password_hash": "jTcCKuOmGJeeRQ/K1LG5sdZLcdnEnqv8wcrP2n68R7nMuqy2", + "tags": ["administrator"] + } + ], + "rabbit_version": "3.13.0+376.g1bc0d89.dirty", + "exchanges": [], + "topic_permissions": [], + "vhosts": [ + { + "limits": [], + "metadata": + { + "description": "Default virtual host", + "tags": [] + }, + "name":"/" + } + ], + "global_parameters": [] +} diff --git a/deps/rabbitmq_exchange_federation/test/exchange_SUITE.erl b/deps/rabbitmq_exchange_federation/test/exchange_SUITE.erl new file mode 100644 index 000000000000..52b3e6bf2b19 --- /dev/null +++ b/deps/rabbitmq_exchange_federation/test/exchange_SUITE.erl @@ -0,0 +1,913 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(exchange_SUITE). + +-include_lib("eunit/include/eunit.hrl"). +-include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). + +-include("rabbit_exchange_federation.hrl"). + +-compile(export_all). + +-import(rabbit_federation_test_util, + [expect/3, expect/4, expect_empty/2]). + +all() -> + [ + {group, essential}, + {group, cluster_size_3}, + {group, rolling_upgrade} + ]. + +groups() -> + [ + {essential, [], essential()}, + {cluster_size_3, [], [max_hops]}, + {rolling_upgrade, [], [child_id_format]}, + {cycle_protection, [], [ + %% TBD: port from v3.10.x in an Erlang 25-compatible way + ]}, + {channel_use_mod_single, [], [ + %% TBD: port from v3.10.x in an Erlang 25-compatible way + ]} + ]. + +essential() -> + [ + single_upstream, + single_upstream_quorum, + multiple_upstreams, + multiple_upstreams_pattern, + single_upstream_multiple_uris, + multiple_downstreams, + e2e_binding, + unbind_on_delete, + unbind_on_client_unbind, + exchange_federation_link_status, + lookup_exchange_status + ]. + +suite() -> + [{timetrap, {minutes, 3}}]. + +%% ------------------------------------------------------------------- +%% 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). + +%% Some of the "regular" tests but in the single channel mode. +init_per_group(essential, Config) -> + SetupFederation = [ + fun(Config1) -> + rabbit_federation_test_util:setup_federation_with_upstream_params(Config1, [ + {<<"channel-use-mode">>, <<"single">>} + ]) + end + ], + Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"), + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, Suffix}, + {rmq_nodes_count, 1} + ]), + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps() ++ + SetupFederation); +init_per_group(cluster_size_3 = Group, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodes_count, 3} + ]), + init_per_group1(Group, Config1); +init_per_group(rolling_upgrade = Group, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodes_count, 5}, + {rmq_nodes_clustered, false} + ]), + init_per_group1(Group, Config1); +init_per_group(Group, Config) -> + init_per_group1(Group, Config). + + +init_per_group1(_Group, Config) -> + Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"), + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, Suffix}, + {rmq_nodes_clustered, false} + ]), + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_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 +%% + +single_upstream(Config) -> + FedX = <<"single_upstream.federated">>, + UpX = <<"single_upstream.upstream.x">>, + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, + {<<"exchange">>, UpX} + ]), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed.x">>, <<"^single_upstream.federated">>, <<"exchanges">>, + [ + {<<"federation-upstream">>, <<"localhost">>} + ]), + + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + + Xs = [ + exchange_declare_method(FedX) + ], + declare_exchanges(Ch, Xs), + + RK = <<"key">>, + Q = declare_and_bind_queue(Ch, FedX, RK), + await_binding(Config, 0, UpX, RK), + publish_expect(Ch, UpX, RK, Q, <<"single_upstream payload">>), + + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + assert_federation_internal_queue_type(Config, Server, rabbit_classic_queue), + + rabbit_ct_client_helpers:close_channel(Ch), + clean_up_federation_related_bits(Config). + +single_upstream_quorum(Config) -> + FedX = <<"single_upstream_quorum.federated">>, + UpX = <<"single_upstream_quorum.upstream.x">>, + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, + {<<"exchange">>, UpX}, + {<<"queue-type">>, <<"quorum">>} + ]), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed.x">>, <<"^single_upstream_quorum.federated">>, <<"exchanges">>, + [ + {<<"federation-upstream">>, <<"localhost">>} + ]), + + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + + Xs = [ + exchange_declare_method(FedX) + ], + declare_exchanges(Ch, Xs), + + RK = <<"key">>, + Q = declare_and_bind_queue(Ch, FedX, RK), + await_binding(Config, 0, UpX, RK), + publish_expect(Ch, UpX, RK, Q, <<"single_upstream_quorum payload">>), + + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + assert_federation_internal_queue_type(Config, Server, rabbit_quorum_queue), + + rabbit_ct_client_helpers:close_channel(Ch), + clean_up_federation_related_bits(Config). + +multiple_upstreams(Config) -> + FedX = <<"multiple_upstreams.federated">>, + UpX1 = <<"upstream.x.1">>, + UpX2 = <<"upstream.x.2">>, + set_up_upstreams(Config), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed.x">>, <<"^multiple_upstreams.federated">>, <<"exchanges">>, + [ + {<<"federation-upstream-set">>, <<"all">>} + ]), + + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + Xs = [ + exchange_declare_method(FedX) + ], + declare_exchanges(Ch, Xs), + + RK = <<"multiple_upstreams.key">>, + Q = declare_and_bind_queue(Ch, FedX, RK), + await_binding(Config, 0, UpX1, RK), + await_binding(Config, 0, UpX2, RK), + publish_expect(Ch, UpX1, RK, Q, <<"multiple_upstreams payload">>), + publish_expect(Ch, UpX2, RK, Q, <<"multiple_upstreams payload">>), + + rabbit_ct_client_helpers:close_channel(Ch), + clean_up_federation_related_bits(Config). + + +multiple_upstreams_pattern(Config) -> + FedX = <<"multiple_upstreams_pattern.federated">>, + UpX1 = <<"upstream.x.1">>, + UpX2 = <<"upstream.x.2">>, + set_up_upstreams(Config), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed.x">>, <<"^multiple_upstreams_pattern.federated">>, <<"exchanges">>, + [ + {<<"federation-upstream-pattern">>, <<"^localhost">>} + ]), + + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + Xs = [ + exchange_declare_method(FedX) + ], + declare_exchanges(Ch, Xs), + + RK = <<"multiple_upstreams_pattern.key">>, + Q = declare_and_bind_queue(Ch, FedX, RK), + await_binding(Config, 0, UpX1, RK), + await_binding(Config, 0, UpX2, RK), + publish_expect(Ch, UpX1, RK, Q, <<"multiple_upstreams_pattern payload">>), + publish_expect(Ch, UpX2, RK, Q, <<"multiple_upstreams_pattern payload">>), + + rabbit_ct_client_helpers:close_channel(Ch), + clean_up_federation_related_bits(Config). + + +single_upstream_multiple_uris(Config) -> + FedX = <<"single_upstream_multiple_uris.federated">>, + UpX = <<"single_upstream_multiple_uris.upstream.x">>, + URIs = [ + rabbit_ct_broker_helpers:node_uri(Config, 0), + rabbit_ct_broker_helpers:node_uri(Config, 0, [use_ipaddr]) + ], + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost">>, + [ + {<<"uri">>, URIs}, + {<<"exchange">>, UpX} + ]), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed.x">>, <<"^single_upstream_multiple_uris.federated">>, <<"exchanges">>, + [ + {<<"federation-upstream">>, <<"localhost">>} + ]), + + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + + Xs = [ + exchange_declare_method(FedX) + ], + declare_exchanges(Ch, Xs), + + RK = <<"key">>, + Q = declare_and_bind_queue(Ch, FedX, RK), + await_binding(Config, 0, UpX, RK), + publish_expect(Ch, UpX, RK, Q, <<"single_upstream_multiple_uris payload">>), + + rabbit_ct_client_helpers:close_channel(Ch), + clean_up_federation_related_bits(Config). + +multiple_downstreams(Config) -> + FedX = <<"multiple_downstreams.federated">>, + UpX = <<"multiple_downstreams.upstream.x">>, + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, + {<<"exchange">>, UpX} + ]), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed.x">>, <<"^multiple_downstreams.federated">>, <<"exchanges">>, + [ + {<<"federation-upstream">>, <<"localhost">>} + ]), + + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + + Xs = [ + exchange_declare_method(FedX) + ], + declare_exchanges(Ch, Xs), + + RK = <<"key">>, + Q1 = declare_and_bind_queue(Ch, FedX, RK), + _ = declare_and_bind_queue(Ch, FedX, RK), + await_binding(Config, 0, UpX, RK), + publish(Ch, UpX, RK, <<"multiple_downstreams payload 1">>), + publish(Ch, UpX, RK, <<"multiple_downstreams payload 2">>), + expect(Ch, Q1, [<<"multiple_downstreams payload 1">>]), + expect(Ch, Q1, [<<"multiple_downstreams payload 2">>]), + + rabbit_ct_client_helpers:close_channel(Ch), + clean_up_federation_related_bits(Config). + +e2e_binding(Config) -> + FedX = <<"e2e_binding.federated">>, + E2EX = <<"e2e_binding.e2e">>, + UpX = <<"e2e_binding.upstream.x">>, + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, + {<<"exchange">>, UpX} + ]), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed.x">>, <<"^e2e_binding.federated">>, <<"exchanges">>, + [ + {<<"federation-upstream">>, <<"localhost">>} + ]), + + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + + Xs = [ + exchange_declare_method(FedX, <<"fanout">>), + exchange_declare_method(E2EX, <<"fanout">>) + ], + declare_exchanges(Ch, Xs), + Key = <<"key">>, + %% federated exchange routes to the E2E fanout + bind_exchange(Ch, E2EX, FedX, Key), + + RK = <<"key">>, + Q = declare_and_bind_queue(Ch, E2EX, RK), + await_binding(Config, 0, UpX, RK), + publish_expect(Ch, UpX, RK, Q, <<"e2e_binding payload">>), + + rabbit_ct_client_helpers:close_channel(Ch), + clean_up_federation_related_bits(Config). + +unbind_on_delete(Config) -> + FedX = <<"unbind_on_delete.federated">>, + UpX = <<"unbind_on_delete.upstream.x">>, + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, + {<<"exchange">>, UpX} + ]), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed.x">>, <<"^unbind_on_delete.federated">>, <<"exchanges">>, + [ + {<<"federation-upstream">>, <<"localhost">>} + ]), + + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + + Xs = [ + exchange_declare_method(FedX) + ], + declare_exchanges(Ch, Xs), + + RK = <<"key">>, + Q1 = declare_and_bind_queue(Ch, FedX, RK), + Q2 = declare_and_bind_queue(Ch, FedX, RK), + await_binding(Config, 0, UpX, RK), + delete_queue(Ch, Q2), + publish_expect(Ch, UpX, RK, Q1, <<"unbind_on_delete payload">>), + + rabbit_ct_client_helpers:close_channel(Ch), + clean_up_federation_related_bits(Config). + +unbind_on_client_unbind(Config) -> + FedX = <<"unbind_on_client_unbind.federated">>, + UpX = <<"unbind_on_client_unbind.upstream.x">>, + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, + {<<"exchange">>, UpX} + ]), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed.x">>, <<"^unbind_on_client_unbind.federated">>, <<"exchanges">>, + [ + {<<"federation-upstream">>, <<"localhost">>} + ]), + + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + + Xs = [ + exchange_declare_method(FedX) + ], + declare_exchanges(Ch, Xs), + + RK = <<"key">>, + Q1 = declare_and_bind_queue(Ch, FedX, RK), + Q2 = declare_and_bind_queue(Ch, FedX, RK), + await_binding(Config, 0, UpX, RK), + unbind_queue(Ch, Q2, UpX, RK), + publish_expect(Ch, UpX, RK, Q1, <<"unbind_on_delete payload">>), + + rabbit_ct_client_helpers:close_channel(Ch), + clean_up_federation_related_bits(Config). + +max_hops(Config) -> + case rabbit_ct_helpers:is_mixed_versions() of + false -> + [NodeA, NodeB, NodeC] = rabbit_ct_broker_helpers:get_node_configs( + Config, nodename), + await_credentials_obfuscation_seeding_on_two_nodes(Config), + + UpX = <<"ring">>, + + %% form of ring of upstreams, + %% A upstream points at B + rabbit_ct_broker_helpers:set_parameter( + Config, NodeA, <<"federation-upstream">>, <<"upstream">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, NodeB)}, + {<<"exchange">>, UpX}, + {<<"max-hops">>, 2} + ]), + %% B upstream points at C + rabbit_ct_broker_helpers:set_parameter( + Config, NodeB, <<"federation-upstream">>, <<"upstream">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, NodeC)}, + {<<"exchange">>, UpX}, + {<<"max-hops">>, 2} + ]), + %% C upstream points at A + rabbit_ct_broker_helpers:set_parameter( + Config, NodeC, <<"federation-upstream">>, <<"upstream">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, NodeA)}, + {<<"exchange">>, UpX}, + {<<"max-hops">>, 2} + ]), + + %% policy on A + [begin + rabbit_ct_broker_helpers:set_policy( + Config, Node, + <<"fed.x">>, <<"^ring">>, <<"exchanges">>, + [ + {<<"federation-upstream">>, <<"upstream">>} + ]) + end || Node <- [NodeA, NodeB, NodeC]], + + NodeACh = rabbit_ct_client_helpers:open_channel(Config, NodeA), + NodeBCh = rabbit_ct_client_helpers:open_channel(Config, NodeB), + NodeCCh = rabbit_ct_client_helpers:open_channel(Config, NodeC), + + FedX = <<"ring">>, + X = exchange_declare_method(FedX), + declare_exchange(NodeACh, X), + declare_exchange(NodeBCh, X), + declare_exchange(NodeCCh, X), + + Q1 = declare_and_bind_queue(NodeACh, <<"ring">>, <<"key">>), + Q2 = declare_and_bind_queue(NodeBCh, <<"ring">>, <<"key">>), + Q3 = declare_and_bind_queue(NodeCCh, <<"ring">>, <<"key">>), + + await_binding(Config, NodeA, <<"ring">>, <<"key">>, 3), + await_binding(Config, NodeB, <<"ring">>, <<"key">>, 3), + await_binding(Config, NodeC, <<"ring">>, <<"key">>, 3), + + publish(NodeACh, <<"ring">>, <<"key">>, <<"HELLO flopsy">>), + publish(NodeBCh, <<"ring">>, <<"key">>, <<"HELLO mopsy">>), + publish(NodeCCh, <<"ring">>, <<"key">>, <<"HELLO cottontail">>), + + Msgs = [<<"HELLO flopsy">>, <<"HELLO mopsy">>, <<"HELLO cottontail">>], + expect(NodeACh, Q1, Msgs), + expect(NodeBCh, Q2, Msgs), + expect(NodeCCh, Q3, Msgs), + expect_empty(NodeACh, Q1), + expect_empty(NodeBCh, Q2), + expect_empty(NodeCCh, Q3), + + clean_up_federation_related_bits(Config); + true -> + %% skip the test in mixed version mode + {skip, "Should not run in mixed version environments"} + end. + +exchange_federation_link_status(Config) -> + FedX = <<"single_upstream.federated">>, + UpX = <<"single_upstream.upstream.x">>, + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, + {<<"exchange">>, UpX} + ]), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed.x">>, <<"^single_upstream.federated">>, <<"exchanges">>, + [ + {<<"federation-upstream">>, <<"localhost">>} + ]), + + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + + Xs = [ + exchange_declare_method(FedX) + ], + declare_exchanges(Ch, Xs), + + RK = <<"key">>, + _ = declare_and_bind_queue(Ch, FedX, RK), + await_binding(Config, 0, UpX, RK), + + [Link] = rabbit_ct_broker_helpers:rpc(Config, 0, + rabbit_federation_status, status, + []), + true = is_binary(proplists:get_value(id, Link)), + + clean_up_federation_related_bits(Config). + +lookup_exchange_status(Config) -> + FedX = <<"single_upstream.federated">>, + UpX = <<"single_upstream.upstream.x">>, + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, + {<<"exchange">>, UpX} + ]), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed.x">>, <<"^single_upstream.federated">>, <<"exchanges">>, + [ + {<<"federation-upstream">>, <<"localhost">>} + ]), + + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + + Xs = [ + exchange_declare_method(FedX) + ], + declare_exchanges(Ch, Xs), + + RK = <<"key">>, + _ = declare_and_bind_queue(Ch, FedX, RK), + await_binding(Config, 0, UpX, RK), + + [Link] = rabbit_ct_broker_helpers:rpc(Config, 0, + rabbit_federation_status, status, []), + Id = proplists:get_value(id, Link), + Props = rabbit_ct_broker_helpers:rpc(Config, 0, + rabbit_federation_status, lookup, [Id]), + lists:all(fun(K) -> lists:keymember(K, 1, Props) end, + [key, uri, status, timestamp, id, supervisor, upstream]), + + clean_up_federation_related_bits(Config). + +child_id_format(Config) -> + [UpstreamNode, + OldNodeA, + NewNodeB, + OldNodeC, + NewNodeD] = rabbit_ct_broker_helpers:get_node_configs( + Config, nodename), + + %% Create a cluster with the nodes running the old version of RabbitMQ in + %% mixed-version testing. + %% + %% Note: we build this on the assumption that `rabbit_ct_broker_helpers' + %% starts nodes this way: + %% Node 1: the primary copy of RabbitMQ the test is started from + %% Node 2: the secondary umbrella (if any) + %% Node 3: the primary copy + %% Node 4: the secondary umbrella + %% ... + %% + %% Therefore, `UpstreamNode' will use the primary copy, `OldNodeA' the + %% secondary umbrella, `NewNodeB' the primary copy, and so on. + Config1 = rabbit_ct_broker_helpers:cluster_nodes( + Config, [OldNodeA, OldNodeC]), + + %% Prepare the whole federated exchange on that old cluster. + UpstreamName = <<"fed_on_upgrade">>, + rabbit_ct_broker_helpers:set_parameter( + Config1, OldNodeA, <<"federation-upstream">>, UpstreamName, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config1, UpstreamNode)} + ]), + + rabbit_ct_broker_helpers:set_policy( + Config1, OldNodeA, + <<"fed_on_upgrade_policy">>, <<"^fed_">>, <<"all">>, + [ + {<<"federation-upstream-pattern">>, UpstreamName} + ]), + + XName = <<"fed_ex_on_upgrade_cluster">>, + X = exchange_declare_method(XName, <<"direct">>), + {Conn1, Ch1} = rabbit_ct_client_helpers:open_connection_and_channel( + Config1, OldNodeA), + ?assertEqual({'exchange.declare_ok'}, declare_exchange(Ch1, X)), + rabbit_ct_client_helpers:close_channel(Ch1), + rabbit_ct_client_helpers:close_connection(Conn1), + + %% Verify the format of the child ID. In the main branch, the format was + %% temporarily a size-2 tuple with a list as the first element. This was + %% not kept later and the original ID format is used in old and new nodes. + [{Id, _, _, _}] = rabbit_ct_broker_helpers:rpc( + Config1, OldNodeA, + mirrored_supervisor, which_children, + [rabbit_federation_exchange_link_sup_sup]), + case Id of + %% This is the format we expect everywhere. + #exchange{name = #resource{name = XName}} -> + %% Verify that the supervisors exist on all nodes. + lists:foreach( + fun(Node) -> + ?assertMatch( + [{#exchange{name = #resource{name = XName}}, + _, _, _}], + rabbit_ct_broker_helpers:rpc( + Config1, Node, + mirrored_supervisor, which_children, + [rabbit_federation_exchange_link_sup_sup])) + end, [OldNodeA, OldNodeC]), + + %% Simulate a rolling upgrade by: + %% 1. adding new nodes to the old cluster + %% 2. stopping the old nodes + %% + %% After that, the supervisors run on the new code. + Config2 = rabbit_ct_broker_helpers:cluster_nodes( + Config1, OldNodeA, [NewNodeB, NewNodeD]), + ok = rabbit_ct_broker_helpers:stop_broker(Config2, OldNodeA), + ok = rabbit_ct_broker_helpers:reset_node(Config1, OldNodeA), + ok = rabbit_ct_broker_helpers:stop_broker(Config2, OldNodeC), + ok = rabbit_ct_broker_helpers:reset_node(Config2, OldNodeC), + + %% Verify that the supervisors still use the same IDs. + lists:foreach( + fun(Node) -> + ?assertMatch( + [{#exchange{name = #resource{name = XName}}, + _, _, _}], + rabbit_ct_broker_helpers:rpc( + Config2, Node, + mirrored_supervisor, which_children, + [rabbit_federation_exchange_link_sup_sup])) + end, [NewNodeB, NewNodeD]), + + %% Delete the exchange: it should work because the ID format is the + %% one expected. + %% + %% During the transient period where the ID format was changed, + %% this would crash with a badmatch because the running + %% supervisor's ID would not match the content of the database. + {Conn2, Ch2} = rabbit_ct_client_helpers:open_connection_and_channel( + Config2, NewNodeB), + ?assertEqual({'exchange.delete_ok'}, delete_exchange(Ch2, XName)), + rabbit_ct_client_helpers:close_channel(Ch2), + rabbit_ct_client_helpers:close_connection(Conn2); + + %% This is the transient format we are not interested in as it only + %% lived in a development branch. + {List, #exchange{name = #resource{name = XName}}} + when is_list(List) -> + {skip, "Testcase skipped with the transiently changed ID format"} + end. + +%% +%% Test helpers +%% + +clean_up_federation_related_bits(Config) -> + delete_all_queues_on(Config, 0), + delete_all_exchanges_on(Config, 0), + delete_all_policies_on(Config, 0), + delete_all_runtime_parameters_on(Config, 0). + +set_up_upstream(Config) -> + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, + {<<"exchange">>, <<"upstream">>} + ]). + +set_up_upstreams(Config) -> + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost1">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, + {<<"exchange">>, <<"upstream.x.1">>} + ]), + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost2">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, + {<<"exchange">>, <<"upstream.x.2">>} + ]). + +set_up_upstreams_including_unavailable(Config) -> + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"unavailable-node">>, + [ + {<<"uri">>, <<"amqp://unavailable-node">>}, + {<<"reconnect-delay">>, 600000} + ]), + + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost">>, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)} + ]). + +declare_exchanges(Ch, Frames) -> + [declare_exchange(Ch, F) || F <- Frames]. +delete_exchanges(Ch, Frames) -> + [delete_exchange(Ch, X) || #'exchange.declare'{exchange = X} <- Frames]. + +declare_exchange(Ch, X) -> + #'exchange.declare_ok'{} = amqp_channel:call(Ch, X). + +declare_queue(Ch) -> + #'queue.declare_ok'{queue = Q} = + amqp_channel:call(Ch, #'queue.declare'{exclusive = true}), + Q. + +declare_queue(Ch, Q) -> + amqp_channel:call(Ch, Q). + +bind_queue(Ch, Q, X, Key) -> + amqp_channel:call(Ch, #'queue.bind'{queue = Q, + exchange = X, + routing_key = Key}). + +unbind_queue(Ch, Q, X, Key) -> + amqp_channel:call(Ch, #'queue.unbind'{queue = Q, + exchange = X, + routing_key = Key}). + +bind_exchange(Ch, D, S, Key) -> + amqp_channel:call(Ch, #'exchange.bind'{destination = D, + source = S, + routing_key = Key}). + +declare_and_bind_queue(Ch, X, Key) -> + Q = declare_queue(Ch), + bind_queue(Ch, Q, X, Key), + Q. + + +delete_exchange(Ch, XName) -> + amqp_channel:call(Ch, #'exchange.delete'{exchange = XName}). + +delete_queue(Ch, QName) -> + amqp_channel:call(Ch, #'queue.delete'{queue = QName}). + +exchange_declare_method(Name) -> + exchange_declare_method(Name, <<"topic">>). + +exchange_declare_method(Name, Type) -> + #'exchange.declare'{exchange = Name, + type = Type, + durable = true}. + +delete_all_queues_on(Config, Node) -> + [rabbit_ct_broker_helpers:rpc( + Config, Node, rabbit_amqqueue, delete, [Q, false, false, + <<"acting-user">>]) || + Q <- all_queues_on(Config, Node)]. + +delete_all_exchanges_on(Config, Node) -> + [rabbit_ct_broker_helpers:rpc( + Config, Node, rabbit_exchange, delete, [X, false, + <<"acting-user">>]) || + #exchange{name = X} <- all_exchanges_on(Config, Node)]. + +delete_all_policies_on(Config, Node) -> + [rabbit_ct_broker_helpers:rpc( + Config, Node, rabbit_policy, delete, [V, Name, <<"acting-user">>]) || + #{name := Name, vhost := V} <- all_policies_on(Config, Node)]. + +delete_all_runtime_parameters_on(Config, Node) -> + [rabbit_ct_broker_helpers:rpc( + Config, Node, rabbit_runtime_parameters, clear, [V, Component, Name, <<"acting-user">>]) || + #{component := Component, name := Name, vhost := V} <- all_runtime_parameters_on(Config, Node)]. + + +all_queues_on(Config, Node) -> + Ret = rabbit_ct_broker_helpers:rpc(Config, Node, + rabbit_amqqueue, list, [<<"/">>]), + case Ret of + {badrpc, _} -> []; + Qs -> Qs + end. + +all_exchanges_on(Config, Node) -> + Ret = rabbit_ct_broker_helpers:rpc(Config, Node, + rabbit_exchange, list, [<<"/">>]), + case Ret of + {badrpc, _} -> []; + Xs -> Xs + end. + +all_policies_on(Config, Node) -> + Ret = rabbit_ct_broker_helpers:rpc(Config, Node, + rabbit_policy, list, [<<"/">>]), + case Ret of + {badrpc, _} -> []; + Xs -> [maps:from_list(PList) || PList <- Xs] + end. + +all_runtime_parameters_on(Config, Node) -> + Ret = rabbit_ct_broker_helpers:rpc(Config, Node, + rabbit_runtime_parameters, list, [<<"/">>]), + case Ret of + {badrpc, _} -> []; + Xs -> [maps:from_list(PList) || PList <- Xs] + end. + +await_binding(Config, Node, X, Key) -> + await_binding(Config, Node, X, Key, 1). + +await_binding(Config, Node, X, Key, ExpectedBindingCount) when is_integer(ExpectedBindingCount) -> + await_binding(Config, Node, <<"/">>, X, Key, ExpectedBindingCount). + +await_binding(Config, Node, Vhost, X, Key, ExpectedBindingCount) when is_integer(ExpectedBindingCount) -> + Attempts = 100, + await_binding(Config, Node, Vhost, X, Key, ExpectedBindingCount, Attempts). + +await_binding(_Config, _Node, _Vhost, _X, _Key, ExpectedBindingCount, 0) -> + {error, rabbit_misc:format("expected ~b bindings but they did not materialize in time", [ExpectedBindingCount])}; +await_binding(Config, Node, Vhost, X, Key, ExpectedBindingCount, AttemptsLeft) when is_integer(ExpectedBindingCount) -> + case bound_keys_from(Config, Node, Vhost, X, Key) of + Bs when length(Bs) < ExpectedBindingCount -> + timer:sleep(1000), + await_binding(Config, Node, Vhost, X, Key, ExpectedBindingCount, AttemptsLeft - 1); + Bs when length(Bs) =:= ExpectedBindingCount -> + ok; + Bs -> + {error, rabbit_misc:format("expected ~b bindings, got ~b", [ExpectedBindingCount, length(Bs)])} + end. + +await_bindings(Config, Node, X, Keys) -> + [await_binding(Config, Node, X, Key) || Key <- Keys]. + +await_binding_absent(Config, Node, X, Key) -> + case bound_keys_from(Config, Node, <<"/">>, X, Key) of + [] -> ok; + _ -> timer:sleep(100), + await_binding_absent(Config, Node, X, Key) + end. + +bound_keys_from(Config, Node, Vhost, X, Key) -> + Res = rabbit_misc:r(Vhost, exchange, X), + List = rabbit_ct_broker_helpers:rpc(Config, Node, + rabbit_binding, list_for_source, [Res]), + [K || #binding{key = K} <- List, K =:= Key]. + +publish_expect(Ch, X, Key, Q, Payload) -> + publish(Ch, X, Key, Payload), + expect(Ch, Q, [Payload]). + +publish(Ch, X, Key, Payload) when is_binary(Payload) -> + publish(Ch, X, Key, #amqp_msg{payload = Payload}); + +publish(Ch, X, Key, Msg = #amqp_msg{}) -> + amqp_channel:call(Ch, #'basic.publish'{exchange = X, + routing_key = Key}, Msg). + +await_credentials_obfuscation_seeding_on_two_nodes(Config) -> + %% give credentials_obfuscation a moment to start and be seeded + rabbit_ct_helpers:await_condition(fun() -> + rabbit_ct_broker_helpers:rpc(Config, 0, credentials_obfuscation, enabled, []) and + rabbit_ct_broker_helpers:rpc(Config, 1, credentials_obfuscation, enabled, []) + end), + + timer:sleep(1000). + +assert_federation_internal_queue_type(Config, Server, Expected) -> + Qs = all_queues_on(Config, Server), + FedQs = lists:filter( + fun(Q) -> + lists:member( + {<<"x-internal-purpose">>, longstr, <<"federation">>}, amqqueue:get_arguments(Q)) + end, + Qs), + FedQTypes = lists:map(fun(Q) -> amqqueue:get_type(Q) end, FedQs), + ?assertEqual([Expected], lists:uniq(FedQTypes)). diff --git a/deps/rabbitmq_exchange_federation/test/exchange_federation_status_command_SUITE.erl b/deps/rabbitmq_exchange_federation/test/exchange_federation_status_command_SUITE.erl new file mode 100644 index 000000000000..50b97f9199bc --- /dev/null +++ b/deps/rabbitmq_exchange_federation/test/exchange_federation_status_command_SUITE.erl @@ -0,0 +1,168 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(exchange_federation_status_command_SUITE). + +-include_lib("amqp_client/include/amqp_client.hrl"). + +-compile(export_all). + +-define(CMD, 'Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand'). + +all() -> + [ + {group, not_federated}, + {group, federated}, + {group, federated_down} + ]. + +groups() -> + [ + {not_federated, [], [ + run_not_federated, + output_not_federated + ]}, + {federated, [], [ + run_federated, + output_federated + ]}, + {federated_down, [], [ + run_down_federated + ]} + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, ?MODULE} + ]), + Config2 = rabbit_ct_helpers:run_setup_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()), + Config2. + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_group(federated, Config) -> + rabbit_federation_test_util:setup_federation(Config), + Config; +init_per_group(federated_down, Config) -> + rabbit_federation_test_util:setup_down_federation(Config), + Config; +init_per_group(_, Config) -> + Config. + +end_per_group(_, Config) -> + Config. + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% ------------------------------------------------------------------- +%% Testcases. +%% ------------------------------------------------------------------- +run_not_federated(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + {stream, []} = ?CMD:run([], Opts#{only_down => false}). + +output_not_federated(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + {stream, []} = ?CMD:output({stream, []}, Opts). + +run_federated(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + %% All + rabbit_federation_test_util:with_ch( + Config, + fun(_) -> + timer:sleep(3000), + {stream, [Props]} = ?CMD:run([], Opts#{only_down => false}), + <<"upstream">> = proplists:get_value(upstream_exchange, Props), + <<"fed1.downstream">> = proplists:get_value(exchange, Props), + exchange = proplists:get_value(type, Props), + running = proplists:get_value(status, Props) + end, + [rabbit_federation_test_util:x(<<"fed1.downstream">>)]), + %% Down + rabbit_federation_test_util:with_ch( + Config, + fun(_) -> + {stream, []} = ?CMD:run([], Opts#{only_down => true}) + end, + [rabbit_federation_test_util:x(<<"fed1.downstream">>)]). + +run_down_federated(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + %% All + rabbit_federation_test_util:with_ch( + Config, + fun(_) -> + rabbit_ct_helpers:await_condition( + fun() -> + {stream, ManyProps} = ?CMD:run([], Opts#{only_down => false}), + Links = [{proplists:get_value(upstream, Props), + proplists:get_value(status, Props)} + || Props <- ManyProps], + [{<<"broken-bunny">>, error}, {<<"localhost">>, running}] + == lists:sort(Links) + end, 15000) + end, + [rabbit_federation_test_util:x(<<"fed1.downstream">>)]), + %% Down + rabbit_federation_test_util:with_ch( + Config, + fun(_) -> + rabbit_ct_helpers:await_condition( + fun() -> + {stream, Props} = ?CMD:run([], Opts#{only_down => true}), + (length(Props) == 1) + andalso (<<"broken-bunny">> == proplists:get_value(upstream, hd(Props))) + andalso (error == proplists:get_value(status, hd(Props))) + end, 15000) + end, + [rabbit_federation_test_util:x(<<"fed1.downstream">>)]). + +output_federated(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + Input = {stream,[[{queue, <<"fed1.downstream">>}, + {consumer_tag, <<"fed.tag">>}, + {upstream_queue, <<"upstream">>}, + {type, queue}, + {vhost, <<"/">>}, + {upstream, <<"localhost">>}, + {status, running}, + {local_connection, <<"">>}, + {uri, <<"amqp://localhost:21000">>}, + {timestamp, {{2016,11,21},{8,51,19}}}]]}, + {stream, [#{queue := <<"fed1.downstream">>, + upstream_queue := <<"upstream">>, + type := queue, + vhost := <<"/">>, + upstream := <<"localhost">>, + status := running, + local_connection := <<"">>, + uri := <<"amqp://localhost:21000">>, + last_changed := <<"2016-11-21 08:51:19">>, + exchange := <<>>, + upstream_exchange := <<>>, + error := <<>>}]} + = ?CMD:output(Input, Opts). diff --git a/deps/rabbitmq_exchange_federation/test/rabbit_federation_test_util.erl b/deps/rabbitmq_exchange_federation/test/rabbit_federation_test_util.erl new file mode 100644 index 000000000000..60a99370001b --- /dev/null +++ b/deps/rabbitmq_exchange_federation/test/rabbit_federation_test_util.erl @@ -0,0 +1,246 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_test_util). + +-include("rabbit_exchange_federation.hrl"). +-include_lib("rabbitmq_federation_common/include/rabbit_federation.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). + +-compile(export_all). + +-import(rabbit_misc, [pget/2]). + +setup_federation(Config) -> + setup_federation_with_upstream_params(Config, []). + +setup_federation_with_upstream_params(Config, ExtraParams) -> + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream">>, <<"localhost">>, [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, + {<<"consumer-tag">>, <<"fed.tag">>} + ] ++ ExtraParams), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream">>, <<"local5673">>, [ + {<<"uri">>, <<"amqp://localhost:1">>} + ] ++ ExtraParams), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream-set">>, <<"upstream">>, [ + [ + {<<"upstream">>, <<"localhost">>}, + {<<"exchange">>, <<"upstream">>}, + {<<"queue">>, <<"upstream">>} + ] + ]), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream-set">>, <<"upstream2">>, [ + [ + {<<"upstream">>, <<"localhost">>}, + {<<"exchange">>, <<"upstream2">>}, + {<<"queue">>, <<"upstream2">>} + ] + ]), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream-set">>, <<"localhost">>, [ + [{<<"upstream">>, <<"localhost">>}] + ]), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream-set">>, <<"upstream12">>, [ + [ + {<<"upstream">>, <<"localhost">>}, + {<<"exchange">>, <<"upstream">>}, + {<<"queue">>, <<"upstream">>} + ], [ + {<<"upstream">>, <<"localhost">>}, + {<<"exchange">>, <<"upstream2">>}, + {<<"queue">>, <<"upstream2">>} + ] + ]), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream-set">>, <<"one">>, [ + [ + {<<"upstream">>, <<"localhost">>}, + {<<"exchange">>, <<"one">>}, + {<<"queue">>, <<"one">>} + ] + ]), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream-set">>, <<"two">>, [ + [ + {<<"upstream">>, <<"localhost">>}, + {<<"exchange">>, <<"two">>}, + {<<"queue">>, <<"two">>} + ] + ]), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream-set">>, <<"upstream5673">>, [ + [ + {<<"upstream">>, <<"local5673">>}, + {<<"exchange">>, <<"upstream">>} + ] + ]), + + rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_policy, set, + [<<"/">>, <<"fed">>, <<"^fed1\.">>, [{<<"federation-upstream-set">>, <<"upstream">>}], + 0, <<"all">>, <<"acting-user">>]), + + rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_policy, set, + [<<"/">>, <<"fed2">>, <<"^fed2\.">>, [{<<"federation-upstream-set">>, <<"upstream2">>}], + 0, <<"all">>, <<"acting-user">>]), + + rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_policy, set, + [<<"/">>, <<"fed12">>, <<"^fed3\.">>, [{<<"federation-upstream-set">>, <<"upstream12">>}], + 2, <<"all">>, <<"acting-user">>]), + + rabbit_ct_broker_helpers:set_policy(Config, 0, + <<"one">>, <<"^two$">>, <<"all">>, [ + {<<"federation-upstream-set">>, <<"one">>}]), + + rabbit_ct_broker_helpers:set_policy(Config, 0, + <<"two">>, <<"^one$">>, <<"all">>, [ + {<<"federation-upstream-set">>, <<"two">>}]), + + rabbit_ct_broker_helpers:set_policy(Config, 0, + <<"hare">>, <<"^hare\.">>, <<"all">>, [ + {<<"federation-upstream-set">>, <<"upstream5673">>}]), + + rabbit_ct_broker_helpers:set_policy(Config, 0, + <<"all">>, <<"^all\.">>, <<"all">>, [ + {<<"federation-upstream-set">>, <<"all">>}]), + + rabbit_ct_broker_helpers:set_policy(Config, 0, + <<"new">>, <<"^new\.">>, <<"all">>, [ + {<<"federation-upstream-set">>, <<"new-set">>}]), + Config. + +setup_down_federation(Config) -> + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"broken-bunny">>, + [{<<"uri">>, <<"amqp://broken-bunny">>}, + {<<"reconnect-delay">>, 600000}]), + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost">>, + [{<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}]), + rabbit_ct_broker_helpers:set_parameter( + Config, 0, + <<"federation-upstream-set">>, <<"upstream">>, + [[{<<"upstream">>, <<"localhost">>}, + {<<"exchange">>, <<"upstream">>}, + {<<"queue">>, <<"upstream">>}], + [{<<"upstream">>, <<"broken-bunny">>}, + {<<"exchange">>, <<"upstream">>}, + {<<"queue">>, <<"upstream">>}]]), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed">>, <<"^fed1\.">>, <<"all">>, [{<<"federation-upstream-set">>, <<"upstream">>}]), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed">>, <<"^fed1\.">>, <<"all">>, [{<<"federation-upstream-set">>, <<"upstream">>}]), + Config. + +expect(Ch, Q, Fun) when is_function(Fun) -> + amqp_channel:subscribe(Ch, #'basic.consume'{queue = Q, + no_ack = true}, self()), + CTag = receive + #'basic.consume_ok'{consumer_tag = CT} -> CT + end, + Fun(), + amqp_channel:call(Ch, #'basic.cancel'{consumer_tag = CTag}); + +expect(Ch, Q, Payloads) -> + expect(Ch, Q, fun() -> expect(Payloads) end). + +expect(Ch, Q, Payloads, Timeout) -> + expect(Ch, Q, fun() -> expect(Payloads, Timeout) end). + +expect([]) -> + ok; +expect(Payloads) -> + expect(Payloads, 60000). + +expect([], _Timeout) -> + ok; +expect(Payloads, Timeout) -> + receive + {#'basic.deliver'{delivery_tag = DTag}, #amqp_msg{payload = Payload}} -> + case lists:member(Payload, Payloads) of + true -> + ct:pal("Consumed a message: ~tp ~tp left: ~tp", [Payload, DTag, length(Payloads) - 1]), + expect(Payloads -- [Payload], Timeout); + false -> ?assert(false, rabbit_misc:format("received an unexpected payload ~tp", [Payload])) + end + after Timeout -> + ct:fail("Did not receive expected payloads ~tp in time", [Payloads]) + end. + +expect_empty(Ch, Q) -> + ?assertMatch(#'basic.get_empty'{}, + amqp_channel:call(Ch, #'basic.get'{ queue = Q })). + +%%---------------------------------------------------------------------------- +xr(Name) -> rabbit_misc:r(<<"/">>, exchange, Name). + +with_ch(Config, Fun, Methods) -> + Ch = rabbit_ct_client_helpers:open_channel(Config), + declare_all(Config, Ch, Methods), + %% Clean up queues even after test failure. + try + Fun(Ch) + after + delete_all(Ch, Methods), + rabbit_ct_client_helpers:close_channel(Ch) + end, + ok. + +declare_all(Config, Ch, Methods) -> [maybe_declare(Config, Ch, Op) || Op <- Methods]. +delete_all(Ch, Methods) -> + [delete_queue(Ch, Q) || #'queue.declare'{queue = Q} <- Methods]. + +maybe_declare(Config, Ch, #'queue.declare'{} = Method) -> + OneOffCh = rabbit_ct_client_helpers:open_channel(Config), + try + amqp_channel:call(OneOffCh, Method#'queue.declare'{passive = true}) + catch exit:{{shutdown, {server_initiated_close, ?NOT_FOUND, _Message}}, _} -> + amqp_channel:call(Ch, Method) + after + catch rabbit_ct_client_helpers:close_channel(OneOffCh) + end; +maybe_declare(_Config, Ch, #'exchange.declare'{} = Method) -> + amqp_channel:call(Ch, Method). + +delete_queue(Ch, Q) -> + amqp_channel:call(Ch, #'queue.delete'{queue = Q}). + +q(Name) -> + q(Name, []). + +q(Name, undefined) -> + q(Name, []); +q(Name, Args) -> + #'queue.declare'{queue = Name, + durable = true, + arguments = Args}. + +x(Name) -> + x(Name, <<"topic">>). + +x(Name, Type) -> + #'exchange.declare'{exchange = Name, + type = Type, + durable = true}. diff --git a/deps/rabbitmq_exchange_federation/test/restart_federation_link_command_SUITE.erl b/deps/rabbitmq_exchange_federation/test/restart_federation_link_command_SUITE.erl new file mode 100644 index 000000000000..2043c0d17410 --- /dev/null +++ b/deps/rabbitmq_exchange_federation/test/restart_federation_link_command_SUITE.erl @@ -0,0 +1,101 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(restart_federation_link_command_SUITE). + +-include_lib("amqp_client/include/amqp_client.hrl"). +-include("rabbit_exchange_federation.hrl"). + +-compile(export_all). + +-define(CMD, 'Elixir.RabbitMQ.CLI.Ctl.Commands.RestartFederationLinkCommand'). + +all() -> + [ + {group, federated_down} + ]. + +groups() -> + [ + {federated_down, [], [ + run, + run_not_found, + output + ]} + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, ?MODULE} + ]), + Config2 = rabbit_ct_helpers:run_setup_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()), + Config2. + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_group(federated_down, Config) -> + rabbit_federation_test_util:setup_down_federation(Config), + Config; +init_per_group(_, Config) -> + Config. + +end_per_group(_, Config) -> + Config. + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% ------------------------------------------------------------------- +%% Testcases. +%% ------------------------------------------------------------------- +run_not_federated(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + {stream, []} = ?CMD:run([], Opts#{'only-down' => false}). + +output_not_federated(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + {stream, []} = ?CMD:output({stream, []}, Opts). + +run(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + rabbit_federation_test_util:with_ch( + Config, + fun(_) -> + timer:sleep(3000), + [Link | _] = rabbit_ct_broker_helpers:rpc(Config, 0, + rabbit_federation_status, status, + []), + Id = proplists:get_value(id, Link), + ok = ?CMD:run([Id], Opts) + end, + [rabbit_federation_test_util:x(<<"fed1.downstream">>)]). + +run_not_found(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + {error, _ErrorMsg} = ?CMD:run([<<"MakingItUp">>], Opts). + +output(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + ok = ?CMD:output(ok, Opts). diff --git a/deps/rabbitmq_exchange_federation/test/unit_inbroker_SUITE.erl b/deps/rabbitmq_exchange_federation/test/unit_inbroker_SUITE.erl new file mode 100644 index 000000000000..b5da5393e78a --- /dev/null +++ b/deps/rabbitmq_exchange_federation/test/unit_inbroker_SUITE.erl @@ -0,0 +1,110 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(unit_inbroker_SUITE). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-compile(export_all). + +-define(US_NAME, <<"upstream">>). +-define(DS_NAME, <<"fed.downstream">>). + +all() -> + [ + {group, non_parallel_tests} + ]. + +groups() -> + [ + {non_parallel_tests, [], [ + serialisation + ]} + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, ?MODULE} + ]), + rabbit_ct_helpers:run_setup_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_group(_, Config) -> + Config. + +end_per_group(_, Config) -> + Config. + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% ------------------------------------------------------------------- +%% Testcases. +%% ------------------------------------------------------------------- + +%% Test that we apply binding changes in the correct order even when +%% they arrive out of order. +serialisation(Config) -> + ok = rabbit_ct_broker_helpers:rpc(Config, 0, + ?MODULE, serialisation1, []). + +serialisation1() -> + with_exchanges( + fun(X) -> + [B1, B2, B3] = [b(K) || K <- [<<"1">>, <<"2">>, <<"3">>]], + remove_bindings(4, X, [B1, B3]), + add_binding(5, X, B1), + add_binding(1, X, B1), + add_binding(2, X, B2), + add_binding(3, X, B3), + %% List of lists because one for each link + Keys = rabbit_federation_exchange_link:list_routing_keys( + X#exchange.name), + [[<<"1">>, <<"2">>]] =:= Keys + end). + +with_exchanges(Fun) -> + {ok, _} = rabbit_exchange:declare( + r(?US_NAME), fanout, false, false, false, [], + <<"acting-user">>), + {ok, X} = rabbit_exchange:declare( + r(?DS_NAME), fanout, false, false, false, [], + <<"acting-user">>), + Fun(X), + %% Delete downstream first or it will recreate the upstream + rabbit_exchange:delete(r(?DS_NAME), false, <<"acting-user">>), + rabbit_exchange:delete(r(?US_NAME), false, <<"acting-user">>), + ok. + +add_binding(Ser, X, B) -> + rabbit_federation_exchange:add_binding(transaction, X, B), + rabbit_federation_exchange:add_binding(Ser, X, B). + +remove_bindings(Ser, X, Bs) -> + rabbit_federation_exchange:remove_bindings(transaction, X, Bs), + rabbit_federation_exchange:remove_bindings(Ser, X, Bs). + +r(Name) -> rabbit_misc:r(<<"/">>, exchange, Name). + +b(Key) -> + #binding{source = ?DS_NAME, destination = <<"whatever">>, + key = Key, args = []}. diff --git a/deps/rabbitmq_federation_common/Makefile b/deps/rabbitmq_federation_common/Makefile new file mode 100644 index 000000000000..beab43bb81ff --- /dev/null +++ b/deps/rabbitmq_federation_common/Makefile @@ -0,0 +1,25 @@ +PROJECT = rabbitmq_federation_common +PROJECT_DESCRIPTION = RabbitMQ Federation Common +PROJECT_MOD = rabbit_federation_common_app + +define PROJECT_ENV +[ + {pgroup_name_cluster_id, false}, + {internal_exchange_check_interval, 90000} + ] +endef + +define PROJECT_APP_EXTRA_KEYS + {broker_version_requirements, []} +endef + +DEPS = rabbit_common rabbit amqp_client +TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers + +PLT_APPS += rabbitmq_cli + +DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-plugin.mk +DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk + +include ../../rabbitmq-components.mk +include ../../erlang.mk diff --git a/deps/rabbitmq_federation_common/include/logging.hrl b/deps/rabbitmq_federation_common/include/logging.hrl new file mode 100644 index 000000000000..20ad0459af58 --- /dev/null +++ b/deps/rabbitmq_federation_common/include/logging.hrl @@ -0,0 +1,3 @@ +-include_lib("rabbit_common/include/logging.hrl"). + +-define(RMQLOG_DOMAIN_FEDERATION, ?DEFINE_RMQLOG_DOMAIN(queue_federation)). diff --git a/deps/rabbitmq_federation_common/include/rabbit_federation.hrl b/deps/rabbitmq_federation_common/include/rabbit_federation.hrl new file mode 100644 index 000000000000..96361e516f8d --- /dev/null +++ b/deps/rabbitmq_federation_common/include/rabbit_federation.hrl @@ -0,0 +1,48 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-record(upstream, {uris, + exchange_name, + queue_name, + consumer_tag, + prefetch_count, + max_hops, + reconnect_delay, + expires, + message_ttl, + trust_user_id, + ack_mode, + queue_type, + name, + bind_nowait, + resource_cleanup_mode, + channel_use_mode + }). + +-record(upstream_params, + {uri, + params, + x_or_q, + %% The next two can be derived from the above three, but we don't + %% want to do that every time we forward a message. + safe_uri, + table}). + +%% Name of the message header used to collect the hop (forwarding) path +%% metadata as the message is forwarded by exchange federation. +-define(ROUTING_HEADER, <<"x-received-from">>). +-define(BINDING_HEADER, <<"x-bound-from">>). +-define(MAX_HOPS_ARG, <<"x-max-hops">>). +%% Identifies a cluster, used by exchange federation cycle detection +-define(DOWNSTREAM_NAME_ARG, <<"x-downstream-name">>). +%% Identifies a virtual host, used by exchange federation cycle detection +-define(DOWNSTREAM_VHOST_ARG, <<"x-downstream-vhost">>). +-define(DEF_PREFETCH, 1000). + +-define(FEDERATION_GUIDE_URL, <<"https://rabbitmq.com/docs/federation/">>). + +-define(FEDERATION_ETS, rabbit_federation_common). diff --git a/deps/rabbitmq_federation_common/src/Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand.erl b/deps/rabbitmq_federation_common/src/Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand.erl new file mode 100644 index 000000000000..aa4794aace7c --- /dev/null +++ b/deps/rabbitmq_federation_common/src/Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand.erl @@ -0,0 +1,117 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module('Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand'). + +-include("rabbit_federation.hrl"). + +-behaviour('Elixir.RabbitMQ.CLI.CommandBehaviour'). + +-export([ + usage/0, + usage_additional/0, + usage_doc_guides/0, + flags/0, + validate/2, + merge_defaults/2, + banner/2, + run/2, + switches/0, + aliases/0, + output/2, + scopes/0, + formatter/0, + help_section/0, + description/0 + ]). + + +%%---------------------------------------------------------------------------- +%% Callbacks +%%---------------------------------------------------------------------------- +usage() -> + <<"federation_status [--only-down]">>. + +usage_additional() -> + [ + {<<"--only-down">>, <<"only display links that failed or are not currently connected">>} + ]. + +usage_doc_guides() -> + [?FEDERATION_GUIDE_URL]. + +help_section() -> + {plugin, federation}. + +description() -> + <<"Displays federation link status">>. + +flags() -> + []. + +validate(_,_) -> + ok. + +formatter() -> + 'Elixir.RabbitMQ.CLI.Formatters.Erlang'. + +merge_defaults(A, Opts) -> + {A, maps:merge(#{only_down => false}, Opts)}. + +banner(_, #{node := Node, only_down := true}) -> + erlang:iolist_to_binary([<<"Listing federation links which are down on node ">>, + atom_to_binary(Node, utf8), <<"...">>]); +banner(_, #{node := Node, only_down := false}) -> + erlang:iolist_to_binary([<<"Listing federation links on node ">>, + atom_to_binary(Node, utf8), <<"...">>]). + +run(_Args, #{node := Node, only_down := OnlyDown}) -> + case rabbit_misc:rpc_call(Node, rabbit_federation_status, status, []) of + {badrpc, _} = Error -> + Error; + Status -> + {stream, filter(Status, OnlyDown)} + end. + +switches() -> + [{only_down, boolean}]. + +aliases() -> + []. + +output({stream, FederationStatus}, _) -> + Formatted = [begin + Timestamp = proplists:get_value(timestamp, St), + Map0 = maps:remove(timestamp, maps:from_list(St)), + Map1 = maps:merge(#{queue => <<>>, + exchange => <<>>, + upstream_queue => <<>>, + upstream_exchange => <<>>, + local_connection => <<>>, + error => <<>>}, Map0), + Map1#{last_changed => fmt_ts(Timestamp)} + end || St <- FederationStatus], + {stream, Formatted}; +output(E, _Opts) -> + 'Elixir.RabbitMQ.CLI.DefaultOutput':output(E). + +scopes() -> + ['ctl', 'diagnostics']. + +%%---------------------------------------------------------------------------- +%% Formatting +%%---------------------------------------------------------------------------- +fmt_ts({{YY, MM, DD}, {Hour, Min, Sec}}) -> + erlang:list_to_binary( + io_lib:format("~4..0w-~2..0w-~2..0w ~2..0w:~2..0w:~2..0w", + [YY, MM, DD, Hour, Min, Sec])). + +filter(Status, _OnlyDown = false) -> + Status; +filter(Status, _OnlyDown = true) -> + [St || St <- Status, + not lists:member(proplists:get_value(status, St), [running, starting])]. diff --git a/deps/rabbitmq_federation_common/src/Elixir.RabbitMQ.CLI.Ctl.Commands.RestartFederationLinkCommand.erl b/deps/rabbitmq_federation_common/src/Elixir.RabbitMQ.CLI.Ctl.Commands.RestartFederationLinkCommand.erl new file mode 100644 index 000000000000..b26804ee5012 --- /dev/null +++ b/deps/rabbitmq_federation_common/src/Elixir.RabbitMQ.CLI.Ctl.Commands.RestartFederationLinkCommand.erl @@ -0,0 +1,84 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module('Elixir.RabbitMQ.CLI.Ctl.Commands.RestartFederationLinkCommand'). + +-include("rabbit_federation.hrl"). + +-behaviour('Elixir.RabbitMQ.CLI.CommandBehaviour'). + +-export([ + usage/0, + usage_additional/0, + usage_doc_guides/0, + flags/0, + validate/2, + merge_defaults/2, + banner/2, + run/2, + aliases/0, + output/2, + help_section/0, + description/0 + ]). + + +%%---------------------------------------------------------------------------- +%% Callbacks +%%---------------------------------------------------------------------------- +usage() -> + <<"restart_federation_link ">>. + +usage_additional() -> + [ + {<<"">>, <<"ID of the link to restart">>} + ]. + +usage_doc_guides() -> + [?FEDERATION_GUIDE_URL]. + +help_section() -> + {plugin, federation}. + +description() -> + <<"Restarts a running federation link">>. + +flags() -> + []. + +validate([], _Opts) -> + {validation_failure, not_enough_args}; +validate([_, _ | _], _Opts) -> + {validation_failure, too_many_args}; +validate([_], _) -> + ok. + +merge_defaults(A, O) -> + {A, O}. + +banner([Link], #{node := Node}) -> + erlang:iolist_to_binary([<<"Restarting federation link ">>, Link, << " on node ">>, + atom_to_binary(Node, utf8)]). + +run([Id], #{node := Node}) -> + case rabbit_misc:rpc_call(Node, rabbit_federation_status, lookup, [Id]) of + {badrpc, _} = Error -> + Error; + not_found -> + {error, <<"Link with the given ID was not found">>}; + Obj -> + Upstream = proplists:get_value(upstream, Obj), + Supervisor = proplists:get_value(supervisor, Obj), + rabbit_misc:rpc_call(Node, rabbit_federation_link_sup, restart, + [Supervisor, Upstream]) + end. + +aliases() -> + []. + +output(Output, _Opts) -> + 'Elixir.RabbitMQ.CLI.DefaultOutput':output(Output). diff --git a/deps/rabbitmq_federation_common/src/rabbit_federation_common_app.erl b/deps/rabbitmq_federation_common/src/rabbit_federation_common_app.erl new file mode 100644 index 000000000000..88700f8fd1e2 --- /dev/null +++ b/deps/rabbitmq_federation_common/src/rabbit_federation_common_app.erl @@ -0,0 +1,33 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_common_app). + +-include("rabbit_federation.hrl"). + +-behaviour(application). +-export([start/2, stop/1]). + +-behaviour(supervisor). +-export([init/1]). + +start(_Type, _StartArgs) -> + ?FEDERATION_ETS = ets:new(?FEDERATION_ETS, [set, public, named_table]), + supervisor:start_link({local, ?MODULE}, ?MODULE, []). + +stop(_State) -> + ok. + +%%---------------------------------------------------------------------------- + +init([]) -> + Flags = #{ + strategy => one_for_one, + intensity => 3, + period => 10 + }, + {ok, {Flags, []}}. diff --git a/deps/rabbitmq_federation_common/src/rabbit_federation_db.erl b/deps/rabbitmq_federation_common/src/rabbit_federation_db.erl new file mode 100644 index 000000000000..a02cea4ba1d3 --- /dev/null +++ b/deps/rabbitmq_federation_common/src/rabbit_federation_db.erl @@ -0,0 +1,45 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_db). + +-include("rabbit_federation.hrl"). +-define(DICT, orddict). + +-export([get_active_suffix/3, set_active_suffix/3, prune_scratch/2]). + +%%---------------------------------------------------------------------------- + +get_active_suffix(XName, Upstream, Default) -> + case rabbit_exchange:lookup_scratch(XName, federation) of + {ok, Dict} -> + case ?DICT:find(key(Upstream), Dict) of + {ok, Suffix} -> Suffix; + error -> Default + end; + {error, not_found} -> + Default + end. + +set_active_suffix(XName, Upstream, Suffix) -> + ok = rabbit_exchange:update_scratch( + XName, federation, + fun(D) -> ?DICT:store(key(Upstream), Suffix, ensure(D)) end). + +prune_scratch(XName, Upstreams) -> + ok = rabbit_exchange:update_scratch( + XName, federation, + fun(D) -> Keys = [key(U) || U <- Upstreams], + ?DICT:filter( + fun(K, _V) -> lists:member(K, Keys) end, ensure(D)) + end). + +key(#upstream{name = UpstreamName, exchange_name = XNameBin}) -> + {UpstreamName, XNameBin}. + +ensure(undefined) -> ?DICT:new(); +ensure(D) -> D. diff --git a/deps/rabbitmq_federation_common/src/rabbit_federation_event.erl b/deps/rabbitmq_federation_common/src/rabbit_federation_event.erl new file mode 100644 index 000000000000..aae9b3f2ed99 --- /dev/null +++ b/deps/rabbitmq_federation_common/src/rabbit_federation_event.erl @@ -0,0 +1,54 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_event). +-behaviour(gen_event). + +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([add_handler/0, remove_handler/0]). + +-export([init/1, handle_call/2, handle_event/2, handle_info/2, + terminate/2, code_change/3]). + +-import(rabbit_misc, [pget/2]). + +%%---------------------------------------------------------------------------- + +add_handler() -> + gen_event:add_handler(rabbit_event, ?MODULE, []). + +remove_handler() -> + gen_event:delete_handler(rabbit_event, ?MODULE, []). + +init([]) -> + {ok, []}. + +handle_call(_Request, State) -> + {ok, not_understood, State}. + +handle_event(#event{type = parameter_set, + props = Props0}, State) -> + Props = rabbit_data_coercion:to_list(Props0), + case {pget(component, Props), pget(name, Props)} of + {global, cluster_name} -> + rabbit_federation_parameters:adjust(everything); + _ -> + ok + end, + {ok, State}; +handle_event(_Event, State) -> + {ok, State}. + +handle_info(_Info, State) -> + {ok, State}. + +terminate(_Arg, _State) -> + ok. + +code_change(_OldVsn, State, _Extra) -> + {ok, State}. diff --git a/deps/rabbitmq_federation_common/src/rabbit_federation_link_sup.erl b/deps/rabbitmq_federation_common/src/rabbit_federation_link_sup.erl new file mode 100644 index 000000000000..7c76aafbd994 --- /dev/null +++ b/deps/rabbitmq_federation_common/src/rabbit_federation_link_sup.erl @@ -0,0 +1,111 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_link_sup). + +-behaviour(supervisor2). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("rabbit/include/amqqueue.hrl"). +-include("rabbit_federation.hrl"). + +%% Supervises the upstream links for an exchange or queue. + +-export([start_link/2, adjust/4, restart/2]). +-export([init/1]). + +start_link(LinkMod, Q) -> + supervisor2:start_link(?MODULE, [LinkMod, Q]). + +adjust(Sup, LinkMod, XorQ, everything) -> + _ = [stop(Sup, Upstream, XorQ) || + {Upstream, _, _, _} <- supervisor2:which_children(Sup)], + [{ok, _Pid} = supervisor2:start_child(Sup, Spec) + || Spec <- specs(LinkMod, XorQ)]; + +adjust(Sup, LinkMod, XorQ, {upstream, UpstreamName}) -> + OldUpstreams0 = children(Sup, UpstreamName), + NewUpstreams0 = rabbit_federation_upstream:for(XorQ, UpstreamName), + %% If any haven't changed, don't restart them. The broker will + %% avoid telling us about connections that have not changed + %% syntactically, but even if one has, this XorQ may not have that + %% connection in an upstream, so we still need to check here. + {OldUpstreams, NewUpstreams} = + lists:foldl( + fun (OldU, {OldUs, NewUs}) -> + case lists:member(OldU, NewUs) of + true -> {OldUs -- [OldU], NewUs -- [OldU]}; + false -> {OldUs, NewUs} + end + end, {OldUpstreams0, NewUpstreams0}, OldUpstreams0), + _ = [stop(Sup, OldUpstream, XorQ) || OldUpstream <- OldUpstreams], + [start(Sup, LinkMod, NewUpstream, XorQ) || NewUpstream <- NewUpstreams]; + +adjust(Sup, _LinkMod, XorQ, {clear_upstream, UpstreamName}) -> + ok = rabbit_federation_db:prune_scratch( + name(XorQ), rabbit_federation_upstream:for(XorQ)), + [stop(Sup, Upstream, XorQ) || Upstream <- children(Sup, UpstreamName)]; + +adjust(Sup, LinkMod, X = #exchange{name = XName}, {upstream_set, _Set}) -> + _ = adjust(Sup, LinkMod, X, everything), + case rabbit_federation_upstream:federate(X) of + false -> ok; + true -> ok = rabbit_federation_db:prune_scratch( + XName, rabbit_federation_upstream:for(X)) + end; +adjust(Sup, LinkMod, Q, {upstream_set, _}) when ?is_amqqueue(Q) -> + adjust(Sup, LinkMod, Q, everything); +adjust(Sup, LinkMod, XorQ, {clear_upstream_set, _}) -> + adjust(Sup, LinkMod, XorQ, everything). + +restart(Sup, Upstream) -> + ok = supervisor2:terminate_child(Sup, Upstream), + {ok, _Pid} = supervisor2:restart_child(Sup, Upstream), + ok. + +start(Sup, LinkMod, Upstream, XorQ) -> + {ok, _Pid} = supervisor2:start_child(Sup, spec(LinkMod, rabbit_federation_util:obfuscate_upstream(Upstream), XorQ)), + ok. + +stop(Sup, Upstream, XorQ) -> + ok = supervisor2:terminate_child(Sup, Upstream), + ok = supervisor2:delete_child(Sup, Upstream), + %% While the link will report its own removal, that only works if + %% the link was actually up. If the link was broken and failing to + %% come up, the possibility exists that there *is* no link + %% process, but we still have a report in the status table. So + %% remove it here too. + %% TODO how do we figure out the module without adding a dependency? + rabbit_federation_status:remove(Upstream, name(XorQ)). + +children(Sup, UpstreamName) -> + rabbit_federation_util:find_upstreams( + UpstreamName, [U || {U, _, _, _} <- supervisor2:which_children(Sup)]). + +%%---------------------------------------------------------------------------- + +init([LinkMod, XorQ]) -> + %% 1, ?MAX_WAIT so that we always give up after one fast retry and get + %% into the reconnect delay. + {ok, {{one_for_one, 1, ?MAX_WAIT}, specs(LinkMod, XorQ)}}. + +specs(LinkMod, XorQ) -> + [spec(LinkMod, rabbit_federation_util:obfuscate_upstream(Upstream), XorQ) + || Upstream <- rabbit_federation_upstream:for(XorQ)]. + +spec(LinkMod, U = #upstream{reconnect_delay = Delay}, #exchange{name = XName}) -> + {U, {LinkMod, start_link, [{U, XName}]}, + {permanent, Delay}, ?WORKER_WAIT, worker, + [LinkMod]}; + +spec(LinkMod, Upstream = #upstream{reconnect_delay = Delay}, Q) when ?is_amqqueue(Q) -> + {Upstream, {LinkMod, start_link, [{Upstream, Q}]}, + {permanent, Delay}, ?WORKER_WAIT, worker, + [LinkMod]}. + +name(#exchange{name = XName}) -> XName; +name(Q) when ?is_amqqueue(Q) -> amqqueue:get_name(Q). diff --git a/deps/rabbitmq_federation_common/src/rabbit_federation_link_util.erl b/deps/rabbitmq_federation_common/src/rabbit_federation_link_util.erl new file mode 100644 index 000000000000..16c87d2cc9c7 --- /dev/null +++ b/deps/rabbitmq_federation_common/src/rabbit_federation_link_util.erl @@ -0,0 +1,359 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_link_util). + +-include_lib("rabbit/include/amqqueue.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). +-include("rabbit_federation.hrl"). + +%% real +-export([start_conn_ch/5, disposable_channel_call/2, disposable_channel_call/3, + disposable_connection_call/3, ensure_connection_closed/1, + log_terminate/4, unacked_new/0, ack/3, nack/3, forward/9, + handle_downstream_down/3, handle_upstream_down/3, + get_connection_name/2, log_debug/3, log_info/3, log_warning/3, + log_error/3]). + +%% temp +-export([connection_error/6]). + +-import(rabbit_misc, [pget/2]). + +-define(MAX_CONNECTION_CLOSE_TIMEOUT, 10000). + +%%---------------------------------------------------------------------------- + +start_conn_ch(Fun, OUpstream, OUParams, + XorQName = #resource{virtual_host = DownVHost}, State) -> + + Upstream = rabbit_federation_util:deobfuscate_upstream(OUpstream), + UParams = rabbit_federation_util:deobfuscate_upstream_params(OUParams), + + ConnName = get_connection_name(Upstream, UParams), + case open_monitor(#amqp_params_direct{virtual_host = DownVHost}, ConnName) of + {ok, DConn, DCh} -> + case Upstream#upstream.ack_mode of + 'on-confirm' -> + #'confirm.select_ok'{} = + amqp_channel:call(DCh, #'confirm.select'{}), + amqp_channel:register_confirm_handler(DCh, self()); + _ -> + ok + end, + case open_monitor(UParams#upstream_params.params, ConnName) of + {ok, Conn, Ch} -> + %% Don't trap exits until we have established + %% connections so that if we try to delete + %% federation upstreams while waiting for a + %% connection to be established then we don't + %% block + process_flag(trap_exit, true), + try + R = Fun(Conn, Ch, DConn, DCh), + log_info( + XorQName, "connected to ~ts", + [rabbit_federation_upstream:params_to_string( + UParams)]), + Name = pget(name, amqp_connection:info(DConn, [name])), + rabbit_federation_status:report( + OUpstream, OUParams, XorQName, {running, Name}), + R + catch exit:E -> + %% terminate/2 will not get this, as we + %% have not put them in our state yet + ensure_connection_closed(DConn), + ensure_connection_closed(Conn), + connection_error(remote_start, E, + OUpstream, OUParams, XorQName, State) + end; + E -> + ensure_connection_closed(DConn), + connection_error(remote_start, E, + OUpstream, OUParams, XorQName, State) + end; + E -> + connection_error(local_start, E, + OUpstream, OUParams, XorQName, State) + end. + +get_connection_name(#upstream{name = UpstreamName}, + #upstream_params{x_or_q = Resource}) when is_record(Resource, exchange) orelse ?is_amqqueue(Resource) -> + connection_name(UpstreamName, rabbit_policy:name(Resource)); + +get_connection_name(_, _) -> + connection_name(undefined, undefined). + +connection_name(Upstream, Policy) when is_binary(Upstream), is_binary(Policy) -> + <<<<"Federation link (upstream: ">>/binary, Upstream/binary, <<", policy: ">>/binary, Policy/binary, <<")">>/binary>>; +connection_name(_, _) -> + <<"Federation link">>. + +open_monitor(Params, Name) -> + case open(Params, Name) of + {ok, Conn, Ch} -> erlang:monitor(process, Ch), + {ok, Conn, Ch}; + E -> E + end. + +open(Params, Name) -> + try + amqp_connection:start(Params, Name) + of + {ok, Conn} -> + try + amqp_connection:open_channel(Conn) + of + {ok, Ch} -> {ok, Conn, Ch}; + E -> ensure_connection_closed(Conn), + E + catch + _:E -> + ensure_connection_closed(Conn), + E + end; + E -> E + catch + _:E -> E + end. + +ensure_channel_closed(Ch) -> catch amqp_channel:close(Ch). + +ensure_connection_closed(Conn) -> + catch amqp_connection:close(Conn, ?MAX_CONNECTION_CLOSE_TIMEOUT). + +connection_error(remote_start, {{shutdown, {server_initiated_close, Code, Message}}, _} = E, + Upstream, UParams, XorQName, State) -> + rabbit_federation_status:report( + Upstream, UParams, XorQName, clean_reason(E)), + log_warning(XorQName, + "did not connect to ~ts. Server has closed the connection due to an error, code: ~tp, " + "message: ~ts", + [rabbit_federation_upstream:params_to_string(UParams), + Code, Message]), + {stop, {shutdown, restart}, State}; + +connection_error(remote_start, E, Upstream, UParams, XorQName, State) -> + rabbit_federation_status:report( + Upstream, UParams, XorQName, clean_reason(E)), + log_warning(XorQName, "did not connect to ~ts. Reason: ~tp", + [rabbit_federation_upstream:params_to_string(UParams), + E]), + {stop, {shutdown, restart}, State}; + +connection_error(remote, E, Upstream, UParams, XorQName, State) -> + rabbit_federation_status:report( + Upstream, UParams, XorQName, clean_reason(E)), + log_info(XorQName, "disconnected from ~ts~n~tp", + [rabbit_federation_upstream:params_to_string(UParams), E]), + {stop, {shutdown, restart}, State}; + +connection_error(command_channel, E, Upstream, UParams, XorQName, State) -> + rabbit_federation_status:report( + Upstream, UParams, XorQName, clean_reason(E)), + log_info(XorQName, "failed to open a command channel for upstream ~ts~n~tp", + [rabbit_federation_upstream:params_to_string(UParams), E]), + {stop, {shutdown, restart}, State}; + +connection_error(local, basic_cancel, Upstream, UParams, XorQName, State) -> + rabbit_federation_status:report( + Upstream, UParams, XorQName, {error, basic_cancel}), + log_info(XorQName, "received a 'basic.cancel'", []), + {stop, {shutdown, restart}, State}; + +connection_error(local_start, E, Upstream, UParams, XorQName, State) -> + rabbit_federation_status:report( + Upstream, UParams, XorQName, clean_reason(E)), + log_warning(XorQName, "did not connect locally~n~tp", [E]), + {stop, {shutdown, restart}, State}. + +%% If we terminate due to a gen_server call exploding (almost +%% certainly due to an amqp_channel:call() exploding) then we do not +%% want to report the gen_server call in our status. +clean_reason({E = {shutdown, _}, _}) -> E; +clean_reason(E) -> E. + +%% local / disconnected never gets invoked, see handle_info({'DOWN', ... + +%%---------------------------------------------------------------------------- + +unacked_new() -> gb_trees:empty(). + +ack(#'basic.ack'{delivery_tag = Seq, + multiple = Multiple}, Ch, Unack) -> + amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = gb_trees:get(Seq, Unack), + multiple = Multiple}), + remove_delivery_tags(Seq, Multiple, Unack). + + +%% Note: at time of writing the broker will never send requeue=false. And it's +%% hard to imagine why it would. But we may as well handle it. +nack(#'basic.nack'{delivery_tag = Seq, + multiple = Multiple, + requeue = Requeue}, Ch, Unack) -> + amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = gb_trees:get(Seq, Unack), + multiple = Multiple, + requeue = Requeue}), + remove_delivery_tags(Seq, Multiple, Unack). + +remove_delivery_tags(Seq, false, Unacked) -> + gb_trees:delete(Seq, Unacked); +remove_delivery_tags(Seq, true, Unacked) -> + case gb_trees:is_empty(Unacked) of + true -> Unacked; + false -> {Smallest, _Val, Unacked1} = gb_trees:take_smallest(Unacked), + case Smallest > Seq of + true -> Unacked; + false -> remove_delivery_tags(Seq, true, Unacked1) + end + end. + +forward(#upstream{ack_mode = AckMode, + trust_user_id = Trust}, + #'basic.deliver'{delivery_tag = DT}, + Ch, DCh, PublishMethod, HeadersFun, ForwardFun, Msg, Unacked) -> + Headers = extract_headers(Msg), + case ForwardFun(Headers) of + true -> Msg1 = maybe_clear_user_id( + Trust, update_headers(HeadersFun(Headers), Msg)), + Seq = case AckMode of + 'on-confirm' -> amqp_channel:next_publish_seqno(DCh); + _ -> ignore + end, + amqp_channel:cast(DCh, PublishMethod, Msg1), + case AckMode of + 'on-confirm' -> + gb_trees:insert(Seq, DT, Unacked); + 'on-publish' -> + amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DT}), + Unacked; + 'no-ack' -> + Unacked + end; + false -> amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DT}), + %% Drop it, but acknowledge it! + Unacked + end. + +maybe_clear_user_id(false, Msg = #amqp_msg{props = Props}) -> + Msg#amqp_msg{props = Props#'P_basic'{user_id = undefined}}; +maybe_clear_user_id(true, Msg) -> + Msg. + +extract_headers(#amqp_msg{props = #'P_basic'{headers = Headers}}) -> + Headers. + +update_headers(Headers, Msg = #amqp_msg{props = Props}) -> + Msg#amqp_msg{props = Props#'P_basic'{headers = Headers}}. + +%%---------------------------------------------------------------------------- + +%% If the downstream channel shuts down cleanly, we can just ignore it +%% - we're the same node, we're presumably about to go down too. +handle_downstream_down(shutdown, _Args, State) -> + {noreply, State}; + +handle_downstream_down(Reason, _Args, State) -> + {stop, {downstream_channel_down, Reason}, State}. + +%% If the upstream channel goes down for an intelligible reason, just +%% log it and die quietly. +handle_upstream_down(shutdown, {Upstream, UParams, XName}, State) -> + connection_error( + remote, {upstream_channel_down, shutdown}, Upstream, UParams, XName, State); +handle_upstream_down({shutdown, Reason}, {Upstream, UParams, XName}, State) -> + connection_error( + remote, {upstream_channel_down, Reason}, Upstream, UParams, XName, State); + +handle_upstream_down(Reason, _Args, State) -> + {stop, {upstream_channel_down, Reason}, State}. + +%%---------------------------------------------------------------------------- + +log_terminate(gone, _Upstream, _UParams, _XorQName) -> + %% the link cannot start, this has been logged already + ok; +log_terminate({shutdown, restart}, _Upstream, _UParams, _XorQName) -> + %% We've already logged this before munging the reason + ok; +log_terminate(shutdown, Upstream, UParams, XorQName) -> + %% The supervisor is shutting us down; we are probably restarting + %% the link because configuration has changed. So try to shut down + %% nicely so that we do not cause unacked messages to be + %% redelivered. + log_info(XorQName, "disconnecting from ~ts", + [rabbit_federation_upstream:params_to_string(UParams)]), + rabbit_federation_status:remove(Upstream, XorQName); + +log_terminate(Reason, Upstream, UParams, XorQName) -> + %% Unexpected death. sasl will log it, but we should update + %% rabbit_federation_status. + rabbit_federation_status:report( + Upstream, UParams, XorQName, clean_reason(Reason)). + +log_debug(XorQName, Fmt, Args) -> log(debug, XorQName, Fmt, Args). +log_info(XorQName, Fmt, Args) -> log(info, XorQName, Fmt, Args). +log_warning(XorQName, Fmt, Args) -> log(warning, XorQName, Fmt, Args). +log_error(XorQName, Fmt, Args) -> log(error, XorQName, Fmt, Args). + +log(Level, XorQName, Fmt0, Args0) -> + Fmt = "Federation ~ts " ++ Fmt0, + Args = [rabbit_misc:rs(XorQName) | Args0], + case Level of + debug -> rabbit_log_federation:debug(Fmt, Args); + info -> rabbit_log_federation:info(Fmt, Args); + warning -> rabbit_log_federation:warning(Fmt, Args); + error -> rabbit_log_federation:error(Fmt, Args) + end. + +%%---------------------------------------------------------------------------- + +disposable_channel_call(Conn, Method) -> + disposable_channel_call(Conn, Method, fun(_, _) -> ok end). + +disposable_channel_call(Conn, Method, ErrFun) -> + try + {ok, Ch} = amqp_connection:open_channel(Conn), + try + amqp_channel:call(Ch, Method) + catch exit:{{shutdown, {server_initiated_close, Code, Message}}, _} -> + ErrFun(Code, Message) + after + ensure_channel_closed(Ch) + end + catch + Exception:Reason -> + rabbit_log_federation:error("Federation link could not create a disposable (one-off) channel due to an error ~tp: ~tp", [Exception, Reason]) + end. + +disposable_connection_call(Params, Method, ErrFun) -> + try + rabbit_log_federation:debug("Disposable connection parameters: ~tp", [Params]), + case open(Params, <<"Disposable exchange federation link connection">>) of + {ok, Conn, Ch} -> + try + amqp_channel:call(Ch, Method) + catch exit:{{shutdown, {connection_closing, {server_initiated_close, Code, Message}}}, _} -> + ErrFun(Code, Message); + exit:{{shutdown, {server_initiated_close, Code, Message}}, _} -> + ErrFun(Code, Message) + after + ensure_connection_closed(Conn) + end; + {error, {auth_failure, Message}} -> + rabbit_log_federation:error("Federation link could not open a disposable (one-off) connection " + "due to an authentication failure: ~ts", [Message]); + Error -> + rabbit_log_federation:error("Federation link could not open a disposable (one-off) connection, " + "reason: ~tp", [Error]), + Error + end + catch + Exception:Reason -> + rabbit_log_federation:error("Federation link could not create a disposable (one-off) connection " + "due to an error ~tp: ~tp", [Exception, Reason]) + end. diff --git a/deps/rabbitmq_federation_common/src/rabbit_federation_parameters.erl b/deps/rabbitmq_federation_common/src/rabbit_federation_parameters.erl new file mode 100644 index 000000000000..b364a6849ac5 --- /dev/null +++ b/deps/rabbitmq_federation_common/src/rabbit_federation_parameters.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) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_parameters). +-behaviour(rabbit_runtime_parameter). +-behaviour(rabbit_policy_validator). + +-include("rabbit_federation.hrl"). + +-export([validate/5, notify/5, notify_clear/4]). +-export([register/0, unregister/0, validate_policy/1, adjust/1]). + +-define(RUNTIME_PARAMETERS, + [{runtime_parameter, <<"federation">>}, + {runtime_parameter, <<"federation-upstream">>}, + {runtime_parameter, <<"federation-upstream-set">>}, + {policy_validator, <<"federation-upstream">>}, + {policy_validator, <<"federation-upstream-pattern">>}, + {policy_validator, <<"federation-upstream-set">>}]). + +-rabbit_boot_step({?MODULE, + [{description, "federation parameters"}, + {mfa, {rabbit_federation_parameters, register, []}}, + {requires, rabbit_registry}, + {cleanup, {rabbit_federation_parameters, unregister, []}}, + {enables, recovery}]}). + +register() -> + [rabbit_registry:register(Class, Name, ?MODULE) || + {Class, Name} <- ?RUNTIME_PARAMETERS], + ok. + +unregister() -> + [rabbit_registry:unregister(Class, Name) || + {Class, Name} <- ?RUNTIME_PARAMETERS], + ok. + +validate(_VHost, <<"federation-upstream-set">>, Name, Term0, _User) -> + Term = [rabbit_data_coercion:to_proplist(Upstream) || Upstream <- Term0], + [rabbit_parameter_validation:proplist( + Name, + [{<<"upstream">>, fun rabbit_parameter_validation:binary/2, mandatory} | + shared_validation()], Upstream) + || Upstream <- Term]; + +validate(_VHost, <<"federation-upstream">>, Name, Term0, _User) -> + Term = rabbit_data_coercion:to_proplist(Term0), + rabbit_parameter_validation:proplist( + Name, [{<<"uri">>, fun validate_uri/2, mandatory} | + shared_validation()], Term); + +validate(_VHost, _Component, Name, _Term, _User) -> + {error, "name not recognised: ~tp", [Name]}. + +notify(_VHost, <<"federation-upstream-set">>, Name, _Term, _Username) -> + adjust({upstream_set, Name}); + +notify(_VHost, <<"federation-upstream">>, Name, _Term, _Username) -> + adjust({upstream, Name}). + +notify_clear(_VHost, <<"federation-upstream-set">>, Name, _Username) -> + adjust({clear_upstream_set, Name}); + +notify_clear(VHost, <<"federation-upstream">>, Name, _Username) -> + adjust({clear_upstream, VHost, Name}). + +adjust(Thing) -> + Plugins = ets:tab2list(?FEDERATION_ETS), + _ = [Module:adjust(Thing) || {_Name, #{link_module := Module}} <- Plugins], + ok. + +%%---------------------------------------------------------------------------- + +shared_validation() -> + [{<<"exchange">>, fun rabbit_parameter_validation:binary/2, optional}, + {<<"queue">>, fun rabbit_parameter_validation:binary/2, optional}, + {<<"consumer-tag">>, fun rabbit_parameter_validation:binary/2, optional}, + {<<"prefetch-count">>, fun rabbit_parameter_validation:number/2, optional}, + {<<"reconnect-delay">>,fun rabbit_parameter_validation:number/2, optional}, + {<<"max-hops">>, fun rabbit_parameter_validation:number/2, optional}, + {<<"expires">>, fun rabbit_parameter_validation:number/2, optional}, + {<<"message-ttl">>, fun rabbit_parameter_validation:number/2, optional}, + {<<"trust-user-id">>, fun rabbit_parameter_validation:boolean/2, optional}, + {<<"ack-mode">>, rabbit_parameter_validation:enum( + ['no-ack', 'on-publish', 'on-confirm']), optional}, + {<<"resource-cleanup-mode">>, rabbit_parameter_validation:enum( + ['default', 'never']), optional}, + {<<"queue-type">>, rabbit_parameter_validation:enum( + ['classic', 'quorum']), optional}, + {<<"bind-nowait">>, fun rabbit_parameter_validation:boolean/2, optional}, + {<<"channel-use-mode">>, rabbit_parameter_validation:enum( + ['multiple', 'single']), optional}]. + +validate_uri(Name, Term) when is_binary(Term) -> + case rabbit_parameter_validation:binary(Name, Term) of + ok -> case amqp_uri:parse(binary_to_list(Term)) of + {ok, _} -> ok; + {error, E} -> {error, "\"~ts\" not a valid URI: ~tp", [Term, E]} + end; + E -> E + end; +validate_uri(Name, Term) -> + case rabbit_parameter_validation:list(Name, Term) of + ok -> case [V || U <- Term, + V <- [validate_uri(Name, U)], + element(1, V) =:= error] of + [] -> ok; + [E | _] -> E + end; + E -> E + end. + +%%---------------------------------------------------------------------------- + +validate_policy([{<<"federation-upstream-set">>, Value}]) + when is_binary(Value) -> + ok; +validate_policy([{<<"federation-upstream-set">>, Value}]) -> + {error, "~tp is not a valid federation upstream set name", [Value]}; + +validate_policy([{<<"federation-upstream-pattern">>, Value}]) + when is_binary(Value) -> + case re:compile(Value) of + {ok, _} -> ok; + {error, Reason} -> {error, "could not compile pattern ~ts to a regular expression. " + "Error: ~tp", [Value, Reason]} + end; +validate_policy([{<<"federation-upstream-pattern">>, Value}]) -> + {error, "~tp is not a valid federation upstream pattern name", [Value]}; + +validate_policy([{<<"federation-upstream">>, Value}]) + when is_binary(Value) -> + ok; +validate_policy([{<<"federation-upstream">>, Value}]) -> + {error, "~tp is not a valid federation upstream name", [Value]}; + +validate_policy(L) when length(L) >= 2 -> + {error, "cannot specify federation-upstream, federation-upstream-set " + "or federation-upstream-pattern together", []}. diff --git a/deps/rabbitmq_federation_common/src/rabbit_federation_pg.erl b/deps/rabbitmq_federation_common/src/rabbit_federation_pg.erl new file mode 100644 index 000000000000..2f3ee5f24464 --- /dev/null +++ b/deps/rabbitmq_federation_common/src/rabbit_federation_pg.erl @@ -0,0 +1,23 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_pg). + +-export([start_scope/1, stop_scope/1]). + +start_scope(Scope) -> + rabbit_log_federation:debug("Starting pg scope ~ts", [Scope]), + _ = pg:start_link(Scope). + +stop_scope(Scope) -> + case whereis(Scope) of + Pid when is_pid(Pid) -> + rabbit_log_federation:debug("Stopping pg scope ~ts", [Scope]), + exit(Pid, normal); + _ -> + ok + end. diff --git a/deps/rabbitmq_federation_common/src/rabbit_federation_status.erl b/deps/rabbitmq_federation_common/src/rabbit_federation_status.erl new file mode 100644 index 000000000000..a880394eb496 --- /dev/null +++ b/deps/rabbitmq_federation_common/src/rabbit_federation_status.erl @@ -0,0 +1,178 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_status). +-behaviour(gen_server). + +-include_lib("amqp_client/include/amqp_client.hrl"). +-include("rabbit_federation.hrl"). + +-export([start_link/0]). + +-export([report/4, remove_exchange_or_queue/1, remove/2, status/0, status/1, lookup/1]). + +-export([init/1, handle_call/3, handle_cast/2, handle_info/2, + terminate/2, code_change/3]). + +-import(rabbit_federation_util, [name/1]). + +-define(SERVER, ?MODULE). +-define(ETS_NAME, ?MODULE). + +-record(state, {}). +-record(entry, {key, uri, status, timestamp, id, supervisor, upstream}). + +start_link() -> + gen_server:start_link({local, ?SERVER}, ?MODULE, [], []). + +report(Upstream, UParams, XorQName, Status) -> + [Supervisor | _] = get('$ancestors'), + gen_server:cast(?SERVER, {report, Supervisor, Upstream, UParams, XorQName, + Status, calendar:local_time()}). + +remove_exchange_or_queue(XorQName) -> + gen_server:call(?SERVER, {remove_exchange_or_queue, XorQName}, infinity). + +remove(Upstream, XorQName) -> + gen_server:call(?SERVER, {remove, Upstream, XorQName}, infinity). + +status() -> + status(infinity). + +status(Timeout) -> + gen_server:call(?SERVER, status, Timeout). + +lookup(Id) -> + gen_server:call(?SERVER, {lookup, Id}, infinity). + +init([]) -> + ?ETS_NAME = ets:new(?ETS_NAME, + [named_table, {keypos, #entry.key}, private]), + {ok, #state{}}. + +handle_call({remove_exchange_or_queue, XorQName}, _From, State) -> + [link_gone(Entry) + || Entry <- ets:match_object(?ETS_NAME, match_entry(xorqkey(XorQName)))], + {reply, ok, State}; + +handle_call({remove, Upstream, XorQName}, _From, State) -> + case ets:match_object(?ETS_NAME, match_entry(key(XorQName, Upstream))) of + [Entry] -> link_gone(Entry); + [] -> ok + end, + {reply, ok, State}; + +handle_call({lookup, Id}, _From, State) -> + Link = case ets:match_object(?ETS_NAME, match_id(Id)) of + [Entry] -> + [{key, Entry#entry.key}, + {uri, Entry#entry.uri}, + {status, Entry#entry.status}, + {timestamp, Entry#entry.timestamp}, + {id, Entry#entry.id}, + {supervisor, Entry#entry.supervisor}, + {upstream, Entry#entry.upstream}]; + [] -> not_found + end, + {reply, Link, State}; + +handle_call(status, _From, State) -> + Entries = ets:tab2list(?ETS_NAME), + {reply, [format(Entry) || Entry <- Entries], State}. + +handle_cast({report, Supervisor, Upstream, #upstream_params{safe_uri = URI}, + XorQName, Status, Timestamp}, State) -> + Key = key(XorQName, Upstream), + Entry = #entry{key = Key, + status = Status, + uri = URI, + timestamp = Timestamp, + supervisor = Supervisor, + upstream = Upstream, + id = unique_id(Key)}, + true = ets:insert(?ETS_NAME, Entry), + rabbit_event:notify(federation_link_status, format(Entry)), + {noreply, State}. + +handle_info(_Info, State) -> + {noreply, State}. + +terminate(_Reason, _State) -> + ok. + +code_change(_OldVsn, State, _Extra) -> + {ok, State}. + +format(#entry{status = Status, + uri = URI, + timestamp = Timestamp} = Entry) -> + identity(Entry) ++ split_status(Status) ++ [{uri, URI}, + {timestamp, Timestamp}]. + +identity(#entry{key = {#resource{virtual_host = VHost, + kind = Type, + name = XorQNameBin}, + UpstreamName, UXorQNameBin}, + id = Id, + upstream = #upstream{consumer_tag = ConsumerTag}}) -> + case Type of + exchange -> [{exchange, XorQNameBin}, + {upstream_exchange, UXorQNameBin}]; + queue -> [{queue, XorQNameBin}, + {upstream_queue, UXorQNameBin}, + {consumer_tag, ConsumerTag}] + end ++ [{type, Type}, + {vhost, VHost}, + {upstream, UpstreamName}, + {id, Id}]. + +unique_id(Key = {#resource{}, UpName, ResName}) when is_binary(UpName), is_binary(ResName) -> + PHash = erlang:phash2(Key, 1 bsl 32), + << << case N >= 10 of + true -> N - 10 + $a; + false -> N + $0 end >> + || <> <= <> >>. + +split_status({running, ConnName}) -> [{status, running}, + {local_connection, ConnName}]; +split_status({Status, Error}) -> [{status, Status}, + {error, Error}]; +split_status(Status) when is_atom(Status) -> [{status, Status}]. + +link_gone(Entry) -> + rabbit_event:notify(federation_link_removed, identity(Entry)), + true = ets:delete_object(?ETS_NAME, Entry). + +%% We don't want to key off the entire upstream, bits of it may change +key(XName = #resource{kind = exchange}, #upstream{name = UpstreamName, + exchange_name = UXNameBin}) -> + {XName, UpstreamName, UXNameBin}; + +key(QName = #resource{kind = queue}, #upstream{name = UpstreamName, + queue_name = UQNameBin}) -> + {QName, UpstreamName, UQNameBin}. + +xorqkey(XorQName) -> + {XorQName, '_', '_'}. + +match_entry(Key) -> + #entry{key = Key, + uri = '_', + status = '_', + timestamp = '_', + id = '_', + supervisor = '_', + upstream = '_'}. + +match_id(Id) -> + #entry{key = '_', + uri = '_', + status = '_', + timestamp = '_', + id = Id, + supervisor = '_', + upstream = '_'}. diff --git a/deps/rabbitmq_federation_common/src/rabbit_federation_sup.erl b/deps/rabbitmq_federation_common/src/rabbit_federation_sup.erl new file mode 100644 index 000000000000..bbe0f71badab --- /dev/null +++ b/deps/rabbitmq_federation_common/src/rabbit_federation_sup.erl @@ -0,0 +1,66 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_sup). + +-behaviour(supervisor). + +%% Supervises everything. There is just one of these. + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include("rabbit_federation.hrl"). + +-define(SUPERVISOR, ?MODULE). + +-export([start_link/0, stop/0]). + +-export([init/1]). + +%% This supervisor needs to be part of the rabbit application since +%% a) it needs to be in place when exchange recovery takes place +%% b) it needs to go up and down with rabbit + +-rabbit_boot_step({rabbit_federation_supervisor, + [{description, "federation"}, + {mfa, {rabbit_sup, start_child, [?MODULE]}}, + {requires, kernel_ready}, + {cleanup, {?MODULE, stop, []}}]}). + +%%---------------------------------------------------------------------------- + +start_link() -> + R = supervisor:start_link({local, ?SUPERVISOR}, ?MODULE, []), + rabbit_federation_event:add_handler(), + R. + +stop() -> + rabbit_federation_event:remove_handler(), + ok = supervisor:terminate_child(rabbit_sup, ?MODULE), + ok = supervisor:delete_child(rabbit_sup, ?MODULE). + +%%---------------------------------------------------------------------------- + +init([]) -> + Status = #{ + id => status, + start => {rabbit_federation_status, start_link, []}, + restart => transient, + shutdown => ?WORKER_WAIT, + type => worker, + modules => [rabbit_federation_status] + }, + %% with default reconnect-delay of 5 second, this supports up to + %% 100 links constantly failing and being restarted a minute + %% (or 200 links if reconnect-delay is 10 seconds, 600 with 30 seconds, + %% etc: N * (60/reconnect-delay) <= 1200) + Flags = #{ + strategy => one_for_one, + intensity => 1200, + period => 60 + }, + Specs = [Status], + {ok, {Flags, Specs}}. diff --git a/deps/rabbitmq_federation_common/src/rabbit_federation_upstream.erl b/deps/rabbitmq_federation_common/src/rabbit_federation_upstream.erl new file mode 100644 index 000000000000..1f6b62deda5c --- /dev/null +++ b/deps/rabbitmq_federation_common/src/rabbit_federation_upstream.erl @@ -0,0 +1,166 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_upstream). + +-include("rabbit_federation.hrl"). +-include_lib("rabbit/include/amqqueue.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). + +-export([federate/1, for/1, for/2, params_to_string/1, to_params/2]). +%% For testing +-export([from_set/2, from_pattern/2, remove_credentials/1]). + +-import(rabbit_misc, [pget/2, pget/3]). +-import(rabbit_federation_util, [name/1, vhost/1, r/1]). +-import(rabbit_data_coercion, [to_atom/1]). + +%%---------------------------------------------------------------------------- + +federate(XorQ) -> + rabbit_policy:get(<<"federation-upstream">>, XorQ) =/= undefined orelse + rabbit_policy:get(<<"federation-upstream-set">>, XorQ) =/= undefined orelse + rabbit_policy:get(<<"federation-upstream-pattern">>, XorQ) =/= undefined. + +for(XorQ) -> + case federate(XorQ) of + false -> []; + true -> from_set_contents(upstreams(XorQ), XorQ) + end. + +for(XorQ, UpstreamName) -> + case federate(XorQ) of + false -> []; + true -> rabbit_federation_util:find_upstreams( + UpstreamName, from_set_contents(upstreams(XorQ), XorQ)) + end. + +upstreams(XorQ) -> + UName = rabbit_policy:get(<<"federation-upstream">>, XorQ), + USetName = rabbit_policy:get(<<"federation-upstream-set">>, XorQ), + UPatternValue = rabbit_policy:get(<<"federation-upstream-pattern">>, XorQ), + %% Cannot define 2 at a time, see rabbit_federation_parameters:validate_policy/1 + case {UName, USetName, UPatternValue} of + {undefined, undefined, undefined} -> []; + {undefined, undefined, _} -> find_contents(UPatternValue, vhost(XorQ)); + {undefined, _, undefined} -> set_contents(USetName, vhost(XorQ)); + {_, undefined, undefined} -> [[{<<"upstream">>, UName}]] + end. + +params_table(SafeURI, XorQ) -> + Key = case XorQ of + #exchange{} -> <<"exchange">>; + Q when ?is_amqqueue(Q) -> <<"queue">> + end, + [{<<"uri">>, longstr, SafeURI}, + {Key, longstr, name(XorQ)}]. + +params_to_string(#upstream_params{safe_uri = SafeURI, + x_or_q = XorQ}) -> + print("~ts on ~ts", [rabbit_misc:rs(r(XorQ)), SafeURI]). + +remove_credentials(URI) -> + list_to_binary(amqp_uri:remove_credentials(binary_to_list(URI))). + +to_params(Upstream = #upstream{uris = URIs}, XorQ) -> + URI = lists:nth(rand:uniform(length(URIs)), URIs), + {ok, Params} = amqp_uri:parse(binary_to_list(URI), vhost(XorQ)), + XorQ1 = with_name(Upstream, vhost(Params), XorQ), + SafeURI = remove_credentials(URI), + #upstream_params{params = Params, + uri = URI, + x_or_q = XorQ1, + safe_uri = SafeURI, + table = params_table(SafeURI, XorQ)}. + +print(Fmt, Args) -> iolist_to_binary(io_lib:format(Fmt, Args)). + +from_set(SetName, XorQ) -> + from_set_contents(set_contents(SetName, vhost(XorQ)), XorQ). + +from_pattern(SetName, XorQ) -> + from_set_contents(find_contents(SetName, vhost(XorQ)), XorQ). + +set_contents(<<"all">>, VHost) -> + Upstreams0 = rabbit_runtime_parameters:list( + VHost, <<"federation-upstream">>), + Upstreams = [rabbit_data_coercion:to_list(U) || U <- Upstreams0], + [[{<<"upstream">>, pget(name, U)}] || U <- Upstreams]; + +set_contents(SetName, VHost) -> + case rabbit_runtime_parameters:value( + VHost, <<"federation-upstream-set">>, SetName) of + not_found -> []; + Set -> Set + end. + +find_contents(RegExp, VHost) -> + Upstreams0 = rabbit_runtime_parameters:list( + VHost, <<"federation-upstream">>), + Upstreams = [rabbit_data_coercion:to_list(U) || U <- Upstreams0, + re:run(pget(name, U), RegExp) =/= nomatch], + [[{<<"upstream">>, pget(name, U)}] || U <- Upstreams]. + +from_set_contents(Set, XorQ) -> + Results = [from_set_element(P, XorQ) || P <- Set], + [R || R <- Results, R =/= not_found]. + +from_set_element(UpstreamSetElem0, XorQ) -> + UpstreamSetElem = rabbit_data_coercion:to_proplist(UpstreamSetElem0), + Name = bget(upstream, UpstreamSetElem, []), + case rabbit_runtime_parameters:value( + vhost(XorQ), <<"federation-upstream">>, Name) of + not_found -> not_found; + Upstream -> from_upstream_or_set( + UpstreamSetElem, Name, Upstream, XorQ) + end. + +from_upstream_or_set(US, Name, U, XorQ) -> + URIParam = bget(uri, US, U), + URIs = case URIParam of + B when is_binary(B) -> [B]; + L when is_list(L) -> L + end, + #upstream{uris = URIs, + exchange_name = bget(exchange, US, U, name(XorQ)), + queue_name = bget(queue, US, U, name(XorQ)), + consumer_tag = bget('consumer-tag', US, U, <<"federation-link-", Name/binary>>), + prefetch_count = bget('prefetch-count', US, U, ?DEF_PREFETCH), + reconnect_delay = bget('reconnect-delay', US, U, 5), + max_hops = bget('max-hops', US, U, 1), + expires = bget(expires, US, U, none), + message_ttl = bget('message-ttl', US, U, none), + trust_user_id = bget('trust-user-id', US, U, false), + ack_mode = to_atom(bget('ack-mode', US, U, <<"on-confirm">>)), + queue_type = to_atom(bget('queue-type', US, U, <<"classic">>)), + name = Name, + bind_nowait = bget('bind-nowait', US, U, false), + resource_cleanup_mode = to_atom(bget('resource-cleanup-mode', US, U, <<"default">>)), + channel_use_mode = to_atom(bget('channel-use-mode', US, U, multiple)) + }. + +%%---------------------------------------------------------------------------- + +bget(K, L1, L2) -> bget(K, L1, L2, undefined). + +bget(K0, L1, L2, D) -> + K = a2b(K0), + %% coerce maps to proplists + PL1 = rabbit_data_coercion:to_list(L1), + PL2 = rabbit_data_coercion:to_list(L2), + case pget(K, PL1, undefined) of + undefined -> pget(K, PL2, D); + Result -> Result + end. + +a2b(A) -> list_to_binary(atom_to_list(A)). + +with_name(#upstream{exchange_name = XNameBin}, VHostBin, X = #exchange{}) -> + X#exchange{name = rabbit_misc:r(VHostBin, exchange, XNameBin)}; + +with_name(#upstream{queue_name = QNameBin}, VHostBin, Q) when ?is_amqqueue(Q) -> + amqqueue:set_name(Q, rabbit_misc:r(VHostBin, queue, QNameBin)). diff --git a/deps/rabbitmq_federation_common/src/rabbit_federation_util.erl b/deps/rabbitmq_federation_common/src/rabbit_federation_util.erl new file mode 100644 index 000000000000..64c22c7b679d --- /dev/null +++ b/deps/rabbitmq_federation_common/src/rabbit_federation_util.erl @@ -0,0 +1,102 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_util). + +-include_lib("rabbit/include/amqqueue.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). +-include("rabbit_federation.hrl"). + +-export([should_forward/4, find_upstreams/2, already_seen/3]). +-export([validate_arg/3, fail/2, name/1, vhost/1, r/1, pgname/1]). +-export([obfuscate_upstream/1, deobfuscate_upstream/1, obfuscate_upstream_params/1, deobfuscate_upstream_params/1]). + +-import(rabbit_misc, [pget_or_die/2, pget/3]). + +%%---------------------------------------------------------------------------- + +should_forward(undefined, _MaxHops, _DName, _DVhost) -> + true; +should_forward(Headers, MaxHops, DName, DVhost) -> + case rabbit_misc:table_lookup(Headers, ?ROUTING_HEADER) of + {array, A} -> length(A) < MaxHops andalso not already_seen(DName, DVhost, A); + _ -> true + end. + +%% Used to detect message and binding forwarding cycles. +already_seen(UpstreamID, UpstreamVhost, Array) -> + lists:any(fun ({table, T}) -> + {longstr, UpstreamID} =:= rabbit_misc:table_lookup(T, <<"cluster-name">>) andalso + {longstr, UpstreamVhost} =:= rabbit_misc:table_lookup(T, <<"vhost">>); + (_) -> + false + end, Array). + +find_upstreams(Name, Upstreams) -> + [U || U = #upstream{name = Name2} <- Upstreams, + Name =:= Name2]. + +validate_arg(Name, Type, Args) -> + case rabbit_misc:table_lookup(Args, Name) of + {Type, _} -> ok; + undefined -> fail("Argument ~ts missing", [Name]); + _ -> fail("Argument ~ts must be of type ~ts", [Name, Type]) + end. + +-spec fail(io:format(), [term()]) -> no_return(). + +fail(Fmt, Args) -> rabbit_misc:protocol_error(precondition_failed, Fmt, Args). + +name( #resource{name = XorQName}) -> XorQName; +name(#exchange{name = #resource{name = XName}}) -> XName; +name(Q) when ?is_amqqueue(Q) -> #resource{name = QName} = amqqueue:get_name(Q), QName. + +vhost( #resource{virtual_host = VHost}) -> VHost; +vhost(#exchange{name = #resource{virtual_host = VHost}}) -> VHost; +vhost(Q) when ?is_amqqueue(Q) -> #resource{virtual_host = VHost} = amqqueue:get_name(Q), VHost; +vhost(#amqp_params_direct{virtual_host = VHost}) -> VHost; +vhost(#amqp_params_network{virtual_host = VHost}) -> VHost. + +r(#exchange{name = XName}) -> XName; +r(Q) when ?is_amqqueue(Q) -> amqqueue:get_name(Q). + +pgname(Name) -> + case application:get_env(rabbitmq_federation, pgroup_name_cluster_id) of + {ok, false} -> Name; + {ok, true} -> {rabbit_nodes:cluster_name(), Name}; + %% default value is 'false', so do the same thing + {ok, undefined} -> Name; + _ -> Name + end. + +obfuscate_upstream(#upstream{uris = Uris} = Upstream) -> + Upstream#upstream{uris = [credentials_obfuscation:encrypt(Uri) || Uri <- Uris]}. + +obfuscate_upstream_params(#upstream_params{uri = Uri, params = #amqp_params_network{password = Password} = Params} = UParams) -> + UParams#upstream_params{ + uri = credentials_obfuscation:encrypt(Uri), + params = Params#amqp_params_network{password = credentials_obfuscation:encrypt(rabbit_data_coercion:to_binary(Password))} + }; +obfuscate_upstream_params(#upstream_params{uri = Uri, params = #amqp_params_direct{password = Password} = Params} = UParams) -> + UParams#upstream_params{ + uri = credentials_obfuscation:encrypt(Uri), + params = Params#amqp_params_direct{password = credentials_obfuscation:encrypt(rabbit_data_coercion:to_binary(Password))} + }. + +deobfuscate_upstream(#upstream{uris = EncryptedUris} = Upstream) -> + Upstream#upstream{uris = [credentials_obfuscation:decrypt(EncryptedUri) || EncryptedUri <- EncryptedUris]}. + +deobfuscate_upstream_params(#upstream_params{uri = EncryptedUri, params = #amqp_params_network{password = EncryptedPassword} = Params} = UParams) -> + UParams#upstream_params{ + uri = credentials_obfuscation:decrypt(EncryptedUri), + params = Params#amqp_params_network{password = credentials_obfuscation:decrypt(EncryptedPassword)} + }; +deobfuscate_upstream_params(#upstream_params{uri = EncryptedUri, params = #amqp_params_direct{password = EncryptedPassword} = Params} = UParams) -> + UParams#upstream_params{ + uri = credentials_obfuscation:decrypt(EncryptedUri), + params = Params#amqp_params_direct{password = credentials_obfuscation:decrypt(EncryptedPassword)} + }. diff --git a/deps/rabbitmq_federation_common/src/rabbit_log_federation.erl b/deps/rabbitmq_federation_common/src/rabbit_log_federation.erl new file mode 100644 index 000000000000..3b7c80d412f4 --- /dev/null +++ b/deps/rabbitmq_federation_common/src/rabbit_log_federation.erl @@ -0,0 +1,107 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +%% @doc Compatibility module for the old Lager-based logging API. +-module(rabbit_log_federation). + +-export([debug/1, debug/2, debug/3, + info/1, info/2, info/3, + notice/1, notice/2, notice/3, + warning/1, warning/2, warning/3, + error/1, error/2, error/3, + critical/1, critical/2, critical/3, + alert/1, alert/2, alert/3, + emergency/1, emergency/2, emergency/3, + none/1, none/2, none/3]). + +-include("logging.hrl"). + +-compile({no_auto_import, [error/2, error/3]}). + +%%---------------------------------------------------------------------------- + +-spec debug(string()) -> 'ok'. +-spec debug(string(), [any()]) -> 'ok'. +-spec debug(pid() | [tuple()], string(), [any()]) -> 'ok'. +-spec info(string()) -> 'ok'. +-spec info(string(), [any()]) -> 'ok'. +-spec info(pid() | [tuple()], string(), [any()]) -> 'ok'. +-spec notice(string()) -> 'ok'. +-spec notice(string(), [any()]) -> 'ok'. +-spec notice(pid() | [tuple()], string(), [any()]) -> 'ok'. +-spec warning(string()) -> 'ok'. +-spec warning(string(), [any()]) -> 'ok'. +-spec warning(pid() | [tuple()], string(), [any()]) -> 'ok'. +-spec error(string()) -> 'ok'. +-spec error(string(), [any()]) -> 'ok'. +-spec error(pid() | [tuple()], string(), [any()]) -> 'ok'. +-spec critical(string()) -> 'ok'. +-spec critical(string(), [any()]) -> 'ok'. +-spec critical(pid() | [tuple()], string(), [any()]) -> 'ok'. +-spec alert(string()) -> 'ok'. +-spec alert(string(), [any()]) -> 'ok'. +-spec alert(pid() | [tuple()], string(), [any()]) -> 'ok'. +-spec emergency(string()) -> 'ok'. +-spec emergency(string(), [any()]) -> 'ok'. +-spec emergency(pid() | [tuple()], string(), [any()]) -> 'ok'. +-spec none(string()) -> 'ok'. +-spec none(string(), [any()]) -> 'ok'. +-spec none(pid() | [tuple()], string(), [any()]) -> 'ok'. + +%%---------------------------------------------------------------------------- + +debug(Format) -> debug(Format, []). +debug(Format, Args) -> debug(self(), Format, Args). +debug(Pid, Format, Args) -> + logger:debug(Format, Args, #{pid => Pid, + domain => ?RMQLOG_DOMAIN_FEDERATION}). + +info(Format) -> info(Format, []). +info(Format, Args) -> info(self(), Format, Args). +info(Pid, Format, Args) -> + logger:info(Format, Args, #{pid => Pid, + domain => ?RMQLOG_DOMAIN_FEDERATION}). + +notice(Format) -> notice(Format, []). +notice(Format, Args) -> notice(self(), Format, Args). +notice(Pid, Format, Args) -> + logger:notice(Format, Args, #{pid => Pid, + domain => ?RMQLOG_DOMAIN_FEDERATION}). + +warning(Format) -> warning(Format, []). +warning(Format, Args) -> warning(self(), Format, Args). +warning(Pid, Format, Args) -> + logger:warning(Format, Args, #{pid => Pid, + domain => ?RMQLOG_DOMAIN_FEDERATION}). + +error(Format) -> error(Format, []). +error(Format, Args) -> error(self(), Format, Args). +error(Pid, Format, Args) -> + logger:error(Format, Args, #{pid => Pid, + domain => ?RMQLOG_DOMAIN_FEDERATION}). + +critical(Format) -> critical(Format, []). +critical(Format, Args) -> critical(self(), Format, Args). +critical(Pid, Format, Args) -> + logger:critical(Format, Args, #{pid => Pid, + domain => ?RMQLOG_DOMAIN_FEDERATION}). + +alert(Format) -> alert(Format, []). +alert(Format, Args) -> alert(self(), Format, Args). +alert(Pid, Format, Args) -> + logger:alert(Format, Args, #{pid => Pid, + domain => ?RMQLOG_DOMAIN_FEDERATION}). + +emergency(Format) -> emergency(Format, []). +emergency(Format, Args) -> emergency(self(), Format, Args). +emergency(Pid, Format, Args) -> + logger:emergency(Format, Args, #{pid => Pid, + domain => ?RMQLOG_DOMAIN_FEDERATION}). + +none(_Format) -> ok. +none(_Format, _Args) -> ok. +none(_Pid, _Format, _Args) -> ok. diff --git a/deps/rabbitmq_federation_common/test/definition_import_SUITE_data/case1.json b/deps/rabbitmq_federation_common/test/definition_import_SUITE_data/case1.json new file mode 100644 index 000000000000..e549e4fd6c1d --- /dev/null +++ b/deps/rabbitmq_federation_common/test/definition_import_SUITE_data/case1.json @@ -0,0 +1,52 @@ +{ + "permissions": [ + { + "configure": ".*", + "read": ".*", + "user": "guest", + "vhost": "/", + "write": ".*" + } + ], + "bindings": [], + "queues": [], + "parameters": [ + { + "component": "federation-upstream-set", + "name": "location-1", + "value": [ + { + "upstream":"up-1" + }, + { + "upstream":"up-2" + } + ], + "vhost":"/"}], + "policies": [], + "rabbitmq_version": "3.13.0+376.g1bc0d89.dirty", + "users": [ + { + "hashing_algorithm": "rabbit_password_hashing_sha256", + "limits": {}, + "name": "guest", + "password_hash": "jTcCKuOmGJeeRQ/K1LG5sdZLcdnEnqv8wcrP2n68R7nMuqy2", + "tags": ["administrator"] + } + ], + "rabbit_version": "3.13.0+376.g1bc0d89.dirty", + "exchanges": [], + "topic_permissions": [], + "vhosts": [ + { + "limits": [], + "metadata": + { + "description": "Default virtual host", + "tags": [] + }, + "name":"/" + } + ], + "global_parameters": [] +} diff --git a/deps/rabbitmq_federation_common/test/unit_SUITE.erl b/deps/rabbitmq_federation_common/test/unit_SUITE.erl new file mode 100644 index 000000000000..f26c10e82b5a --- /dev/null +++ b/deps/rabbitmq_federation_common/test/unit_SUITE.erl @@ -0,0 +1,65 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(unit_SUITE). +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). +-include("rabbit_federation.hrl"). + +-compile(export_all). + +all() -> [ + obfuscate_upstream, + obfuscate_upstream_params_network, + obfuscate_upstream_params_network_with_char_list_password_value, + obfuscate_upstream_params_direct +]. + +init_per_suite(Config) -> + application:ensure_all_started(credentials_obfuscation), + Config. + +end_per_suite(Config) -> + Config. + +obfuscate_upstream(_Config) -> + Upstream = #upstream{uris = [<<"amqp://guest:password@localhost">>]}, + ObfuscatedUpstream = rabbit_federation_util:obfuscate_upstream(Upstream), + ?assertEqual(Upstream, rabbit_federation_util:deobfuscate_upstream(ObfuscatedUpstream)), + ok. + +obfuscate_upstream_params_network(_Config) -> + UpstreamParams = #upstream_params{ + uri = <<"amqp://guest:password@localhost">>, + params = #amqp_params_network{password = <<"password">>} + }, + ObfuscatedUpstreamParams = rabbit_federation_util:obfuscate_upstream_params(UpstreamParams), + ?assertEqual(UpstreamParams, rabbit_federation_util:deobfuscate_upstream_params(ObfuscatedUpstreamParams)), + ok. + +obfuscate_upstream_params_network_with_char_list_password_value(_Config) -> + Input = #upstream_params{ + uri = <<"amqp://guest:password@localhost">>, + params = #amqp_params_network{password = "password"} + }, + Output = #upstream_params{ + uri = <<"amqp://guest:password@localhost">>, + params = #amqp_params_network{password = <<"password">>} + }, + ObfuscatedUpstreamParams = rabbit_federation_util:obfuscate_upstream_params(Input), + ?assertEqual(Output, rabbit_federation_util:deobfuscate_upstream_params(ObfuscatedUpstreamParams)), + ok. + + obfuscate_upstream_params_direct(_Config) -> + UpstreamParams = #upstream_params{ + uri = <<"amqp://guest:password@localhost">>, + params = #amqp_params_direct{password = <<"password">>} + }, + ObfuscatedUpstreamParams = rabbit_federation_util:obfuscate_upstream_params(UpstreamParams), + ?assertEqual(UpstreamParams, rabbit_federation_util:deobfuscate_upstream_params(ObfuscatedUpstreamParams)), + ok. diff --git a/deps/rabbitmq_federation_common/test/unit_inbroker_SUITE.erl b/deps/rabbitmq_federation_common/test/unit_inbroker_SUITE.erl new file mode 100644 index 000000000000..eb40a1d16762 --- /dev/null +++ b/deps/rabbitmq_federation_common/test/unit_inbroker_SUITE.erl @@ -0,0 +1,201 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(unit_inbroker_SUITE). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-include("rabbit_federation.hrl"). + +-compile(export_all). + +-define(US_NAME, <<"upstream">>). +-define(DS_NAME, <<"fed.downstream">>). + +all() -> + [ + {group, non_parallel_tests} + ]. + +groups() -> + [ + {non_parallel_tests, [], [ + scratch_space, + remove_credentials, + get_connection_name, + upstream_validation, + upstream_set_validation + ]} + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, ?MODULE} + ]), + rabbit_ct_helpers:run_setup_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_group(_, Config) -> + Config. + +end_per_group(_, Config) -> + Config. + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% ------------------------------------------------------------------- +%% Testcases. +%% ------------------------------------------------------------------- + +scratch_space(Config) -> + ok = rabbit_ct_broker_helpers:rpc(Config, 0, + ?MODULE, scratch_space1, []). + +scratch_space1() -> + A = <<"A">>, + B = <<"B">>, + DB = rabbit_federation_db, + with_exchanges( + fun(#exchange{name = N}) -> + DB:set_active_suffix(N, upstream(x), A), + DB:set_active_suffix(N, upstream(y), A), + DB:prune_scratch(N, [upstream(y), upstream(z)]), + DB:set_active_suffix(N, upstream(y), B), + DB:set_active_suffix(N, upstream(z), A), + none = DB:get_active_suffix(N, upstream(x), none), + B = DB:get_active_suffix(N, upstream(y), none), + A = DB:get_active_suffix(N, upstream(z), none) + end). + +remove_credentials(Config) -> + Test0 = fun (In, Exp) -> + Act = rabbit_ct_broker_helpers:rpc(Config, 0, + rabbit_federation_upstream, remove_credentials, [In]), + Exp = Act + end, + Cat = fun (Bs) -> + list_to_binary(lists:append([binary_to_list(B) || B <- Bs])) + end, + Test = fun (Scheme, Rest) -> + Exp = Cat([Scheme, Rest]), + Test0(Exp, Exp), + Test0(Cat([Scheme, <<"user@">>, Rest]), Exp), + Test0(Cat([Scheme, <<"user:pass@">>, Rest]), Exp) + end, + Test(<<"amqp://">>, <<"">>), + Test(<<"amqp://">>, <<"localhost">>), + Test(<<"amqp://">>, <<"localhost/">>), + Test(<<"amqp://">>, <<"localhost/foo">>), + Test(<<"amqp://">>, <<"localhost:5672">>), + Test(<<"amqp://">>, <<"localhost:5672/foo">>), + Test(<<"amqps://">>, <<"localhost:5672/%2f">>), + ok. + +get_connection_name(Config) -> + Amqqueue = rabbit_ct_broker_helpers:rpc( + Config, 0, + amqqueue, new, [rabbit_misc:r(<<"/">>, queue, <<"queue">>), + self(), + false, + false, + none, + [], + undefined, + #{}, + classic]), + AmqqueueWithPolicy = amqqueue:set_policy(Amqqueue, [{name, <<"my.federation.policy">>}]), + AmqqueueWithEmptyPolicy = amqqueue:set_policy(Amqqueue, []), + + + <<"Federation link (upstream: my.upstream, policy: my.federation.policy)">> = rabbit_federation_link_util:get_connection_name( + #upstream{name = <<"my.upstream">>}, + #upstream_params{x_or_q = AmqqueueWithPolicy} + ), + <<"Federation link (upstream: my.upstream, policy: my.federation.policy)">> = rabbit_federation_link_util:get_connection_name( + #upstream{name = <<"my.upstream">>}, + #upstream_params{x_or_q = #exchange{policy = [{name, <<"my.federation.policy">>}]}} + ), + <<"Federation link">> = rabbit_federation_link_util:get_connection_name( + #upstream{}, + #upstream_params{x_or_q = AmqqueueWithEmptyPolicy} + ), + <<"Federation link">> = rabbit_federation_link_util:get_connection_name( + #upstream{}, + #upstream_params{x_or_q = #exchange{policy = []}} + ), + <<"Federation link">> = rabbit_federation_link_util:get_connection_name( + whatever, + whatever + ), + ok. + +upstream_set_validation(_Config) -> + ?assertEqual(rabbit_federation_parameters:validate(<<"/">>, <<"federation-upstream-set">>, + <<"a-name">>, + [[{<<"upstream">>, <<"devtest1">>}], + [{<<"upstream">>, <<"devtest2">>}]], + <<"acting-user">>), + [[ok], [ok]]), + ?assertEqual(rabbit_federation_parameters:validate(<<"/">>, <<"federation-upstream-set">>, + <<"a-name">>, + [#{<<"upstream">> => <<"devtest3">>}, + #{<<"upstream">> => <<"devtest4">>}], + <<"acting-user">>), + [[ok], [ok]]), + ok. + +upstream_validation(_Config) -> + ?assertEqual(rabbit_federation_parameters:validate(<<"/">>, <<"federation-upstream">>, + <<"a-name">>, + [{<<"uri">>, <<"amqp://127.0.0.1/%2f">>}], + <<"acting-user">>), + [ok]), + ?assertEqual(rabbit_federation_parameters:validate(<<"/">>, <<"federation-upstream">>, + <<"a-name">>, + #{<<"uri">> => <<"amqp://127.0.0.1/%2f">>}, + <<"acting-user">>), + [ok]), + ok. + +with_exchanges(Fun) -> + {ok, _} = rabbit_exchange:declare( + r(?US_NAME), fanout, false, false, false, [], + <<"acting-user">>), + {ok, X} = rabbit_exchange:declare( + r(?DS_NAME), fanout, false, false, false, [], + <<"acting-user">>), + Fun(X), + %% Delete downstream first or it will recreate the upstream + rabbit_exchange:delete(r(?DS_NAME), false, <<"acting-user">>), + rabbit_exchange:delete(r(?US_NAME), false, <<"acting-user">>), + ok. + +r(Name) -> rabbit_misc:r(<<"/">>, exchange, Name). + +b(Key) -> + #binding{source = ?DS_NAME, destination = <<"whatever">>, + key = Key, args = []}. + +upstream(UpstreamName) -> + #upstream{name = atom_to_list(UpstreamName), + exchange_name = <<"upstream">>}. diff --git a/deps/rabbitmq_queue_federation/Makefile b/deps/rabbitmq_queue_federation/Makefile new file mode 100644 index 000000000000..1e30fe9d7c9e --- /dev/null +++ b/deps/rabbitmq_queue_federation/Makefile @@ -0,0 +1,24 @@ +PROJECT = rabbitmq_queue_federation +PROJECT_DESCRIPTION = RabbitMQ Queue Federation +PROJECT_MOD = rabbit_queue_federation_app + +define PROJECT_ENV +[ + {pgroup_name_cluster_id, false} + ] +endef + +define PROJECT_APP_EXTRA_KEYS + {broker_version_requirements, []} +endef + +DEPS = rabbit_common rabbit amqp_client rabbitmq_federation_common +TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers + +PLT_APPS += rabbitmq_cli + +DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-plugin.mk +DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk + +include ../../rabbitmq-components.mk +include ../../erlang.mk diff --git a/deps/rabbitmq_queue_federation/README-hacking b/deps/rabbitmq_queue_federation/README-hacking new file mode 100644 index 000000000000..6432552fe33a --- /dev/null +++ b/deps/rabbitmq_queue_federation/README-hacking @@ -0,0 +1,143 @@ +This file is intended to tell you How It All Works, concentrating on +the things you might not expect. + +The theory +========== + +The 'x-federation' exchange is defined in +rabbit_federation_exchange. This starts up a bunch of link processes +(one for each upstream) which: + + * Connect to the upstream broker + * Create a queue and bind it to the upstream exchange + * Keep bindings in sync with the downstream exchange + * Consume messages from the upstream queue and republish them to the + downstream exchange (matching confirms with acks) + +Each link process monitors the connections / channels it opens, and +dies if they do. We use a supervisor2 to ensure that we get some +backoff when restarting. + +We use process groups to identify all link processes for a certain +exchange, as well as all link processes together. + +However, there are a bunch of wrinkles: + + +Wrinkle: The exchange will be recovered when the Erlang client is not available +=============================================================================== + +Exchange recovery happens within the rabbit application - therefore at +the time that the exchange is recovered, we can't make any connections +since the amqp_client application has not yet started. Each link +therefore initially has a state 'not_started'. When it is created it +checks to see if the rabbitmq_federation application is running. If +so, it starts fully. If not, it goes into the 'not_started' +state. When rabbitmq_federation starts, it sends a 'go' message to all +links, prodding them to bring up the link. + + +Wrinkle: On reconnect we want to assert bindings atomically +=========================================================== + +If the link goes down for whatever reason, then by the time it comes +up again the bindings downstream may no longer be in sync with those +upstream. Therefore on link establishment we want to ensure that a +certain set of bindings exists. (Of course bringing up a link for the +first time is a simple case of this.) And we want to do this with AMQP +methods. But if we were to tear down all bindings and recreate them, +we would have a time period when messages would not be forwarded for +bindings that *do* still exist before and after. + +We use exchange to exchange bindings to work around this: + +We bind the upstream exchange (X) to the upstream queue (Q) via an +internal fanout exchange (IXA) like so: (routing keys R1 and R2): + + X----R1,R2--->IXA---->Q + +This has the same effect as binding the queue to the exchange directly. + +Now imagine the link has gone down, and is about to be +reestablished. In the meanwhile, routing has changed downstream so +that we now want routing keys R1 and R3. On link reconnection we can +create and bind another internal fanout exchange IXB: + + X----R1,R2--->IXA---->Q + | ^ + | | + \----R1,R3--->IXB-----/ + +and then delete the original exchange IXA: + + X Q + | ^ + | | + \----R1,R3--->IXB-----/ + +This means that messages matching R1 are always routed during the +switchover. Messages for R3 will start being routed as soon as we bind +the second exchange, and messages for R2 will be stopped in a timely +way. Of course this could lag the downstream situation somewhat, in +which case some R2 messages will get thrown away downstream since they +are unroutable. However this lag is inevitable when the link goes +down. + +This means that the downstream only needs to keep track of whether the +upstream is currently going via internal exchange A or B. This is +held in the exchange scratch space in Mnesia. + + +Wrinkle: We need to amalgamate bindings +======================================= + +Since we only bind to one exchange upstream, but the downstream +exchange can be bound to many queues, we can have duplicated bindings +downstream (same source, routing key and args but different +destination) that cannot be duplicated upstream (since the destination +is the same). The link therefore maintains a mapping of (Key, Args) to +set(Dest). Duplicated bindings do not get repeated upstream, and are +only unbound upstream when the last one goes away downstream. + +Furthermore, this works as an optimisation since this will tend to +reduce upstream binding count and churn. + + +Wrinkle: We may receive binding events out of order +=================================================== + +The rabbit_federation_exchange callbacks are invoked by channel +processes within rabbit. Therefore they can be executed concurrently, +and can arrive at the link processes in an order that does not +correspond to the wall clock. + +We need to keep the state of the link in sync with Mnesia. Therefore +not only do we need to impose an ordering on these events, we need to +impose Mnesia's ordering on them. We therefore added a function to the +callback interface, serialise_events. When this returns true, the +callback mechanism inside rabbit increments a per-exchange counter +within an Mnesia transaction, and returns the value as part of the +add_binding and remove_binding callbacks. The link process then queues +up these events, and replays them in order. The link process's state +thus always follows Mnesia (it may be delayed, but the effects happen +in the same order). + + +Other issues +============ + +Since links are implemented in terms of AMQP, link failure may cause +messages to be redelivered. If you're unlucky this could lead to +duplication. + +Message duplication can also happen with some topologies. In some +cases it may not be possible to set max_hops such that messages arrive +once at every node. + +While we correctly order bind / unbind events, we don't do the same +thing for exchange creation / deletion. (This is harder - if you +delete and recreate an exchange with the same name, is it the same +exchange? What about if its type changes?) This would only be an issue +if exchanges churn rapidly; however we could get into a state where +Mnesia sees CDCD but we see CDDC and leave a process running when we +shouldn't. diff --git a/deps/rabbitmq_queue_federation/README.md b/deps/rabbitmq_queue_federation/README.md new file mode 100644 index 000000000000..d96c13a02e57 --- /dev/null +++ b/deps/rabbitmq_queue_federation/README.md @@ -0,0 +1,23 @@ +## RabbitMQ Federation + +RabbitMQ federation offers a group of features for loosely +coupled and WAN-friendly distributed RabbitMQ setups. Note that +this is not an alternative to queue mirroring. + + +## Supported RabbitMQ Versions + +This plugin ships with RabbitMQ, there is no need to +install it separately. + + +## Documentation + +See [RabbitMQ federation plugin](https://www.rabbitmq.com/federation.html) on rabbitmq.com. + + +## License and Copyright + +Released under [the same license as RabbitMQ](https://www.rabbitmq.com/mpl.html). + +2007-2015 (c) 2007-2024 Broadcom. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. diff --git a/deps/rabbitmq_queue_federation/include/rabbit_queue_federation.hrl b/deps/rabbitmq_queue_federation/include/rabbit_queue_federation.hrl new file mode 100644 index 000000000000..9b9ae71aa9ee --- /dev/null +++ b/deps/rabbitmq_queue_federation/include/rabbit_queue_federation.hrl @@ -0,0 +1,8 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-define(FEDERATION_PG_SCOPE, rabbitmq_queue_federation_pg_scope). diff --git a/deps/rabbitmq_queue_federation/src/rabbit_federation_queue.erl b/deps/rabbitmq_queue_federation/src/rabbit_federation_queue.erl new file mode 100644 index 000000000000..b4923f5b283c --- /dev/null +++ b/deps/rabbitmq_queue_federation/src/rabbit_federation_queue.erl @@ -0,0 +1,109 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_queue). + +-rabbit_boot_step({?MODULE, + [{description, "federation queue decorator"}, + {mfa, {rabbit_queue_decorator, register, + [<<"federation">>, ?MODULE]}}, + {requires, rabbit_registry}, + {cleanup, {rabbit_queue_decorator, unregister, + [<<"federation">>]}}, + {enables, recovery}]}). + +-include_lib("rabbit/include/amqqueue.hrl"). +-include("rabbit_queue_federation.hrl"). + +-behaviour(rabbit_queue_decorator). + +-export([startup/1, shutdown/1, policy_changed/2, active_for/1, + consumer_state_changed/3]). +-export([policy_changed_local/2]). + +%%---------------------------------------------------------------------------- + +startup(Q) -> + case active_for(Q) of + true -> rabbit_federation_queue_link_sup_sup:start_child(Q); + false -> ok + end, + ok. + +shutdown(Q) when ?is_amqqueue(Q) -> + QName = amqqueue:get_name(Q), + case active_for(Q) of + true -> rabbit_federation_queue_link_sup_sup:stop_child(Q), + rabbit_federation_status:remove_exchange_or_queue(QName); + false -> ok + end, + ok. + +policy_changed(Q1, Q2) when ?is_amqqueue(Q1) -> + QName = amqqueue:get_name(Q1), + case rabbit_amqqueue:lookup(QName) of + {ok, Q0} when ?is_amqqueue(Q0) -> + rpc:call(amqqueue:qnode(Q0), rabbit_federation_queue, + policy_changed_local, [Q1, Q2]); + {error, not_found} -> + ok + end. + +policy_changed_local(Q1, Q2) -> + shutdown(Q1), + startup(Q2). + +active_for(Q) -> + Args = amqqueue:get_arguments(Q), + case rabbit_misc:table_lookup(Args, <<"x-internal-purpose">>) of + {longstr, _} -> false; %% [0] + _ -> rabbit_federation_upstream:federate(Q) + end. +%% [0] Currently the only "internal purpose" is federation, but I +%% suspect if we introduce another one it will also be for something +%% that doesn't want to be federated. + +%% We need to reconsider whether we need to run or pause every time +%% the consumer state changes in the queue. But why can the state +%% change? +%% +%% consumer blocked | We may have no more active consumers, and thus need to +%% | pause +%% | +%% consumer unblocked | We don't care +%% | +%% queue empty | The queue has become empty therefore we need to run to +%% | get more messages +%% | +%% basic consume | We don't care +%% | +%% basic cancel | We may have no more active consumers, and thus need to +%% | pause +%% | +%% refresh | We asked for it (we have started a new link after +%% | failover and need something to prod us into action +%% | (or not)). +%% +%% In the cases where we don't care it's not prohibitively expensive +%% for us to be here anyway, so never mind. +%% +%% Note that there is no "queue became non-empty" state change - that's +%% because of the queue invariant. If the queue transitions from empty to +%% non-empty then it must have no active consumers - in which case it stays +%% the same from our POV. + +consumer_state_changed(Q, MaxActivePriority, IsEmpty) -> + QName = amqqueue:get_name(Q), + _ = case IsEmpty andalso active_unfederated(MaxActivePriority) of + true -> rabbit_federation_queue_link:run(QName); + false -> rabbit_federation_queue_link:pause(QName) + end, + ok. + +active_unfederated(empty) -> false; +active_unfederated(P) when P >= 0 -> true; +active_unfederated(_P) -> false. diff --git a/deps/rabbitmq_queue_federation/src/rabbit_federation_queue_link.erl b/deps/rabbitmq_queue_federation/src/rabbit_federation_queue_link.erl new file mode 100644 index 000000000000..fda313f63db6 --- /dev/null +++ b/deps/rabbitmq_queue_federation/src/rabbit_federation_queue_link.erl @@ -0,0 +1,327 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_queue_link). + +-include_lib("rabbit/include/amqqueue.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). +-include_lib("rabbitmq_federation_common/include/rabbit_federation.hrl"). +-include("rabbit_queue_federation.hrl"). + +-behaviour(gen_server2). + +-export([start_link/1, go/0, run/1, pause/1]). + +-export([init/1, handle_call/3, handle_cast/2, handle_info/2, + terminate/2, code_change/3]). + +-import(rabbit_misc, [pget/2]). +-import(rabbit_federation_util, [name/1, pgname/1]). + +-record(not_started, {queue, run, upstream, upstream_params}). +-record(state, {queue, run, conn, ch, dconn, dch, upstream, upstream_params, + unacked}). + +start_link(Args) -> + gen_server2:start_link(?MODULE, Args, [{timeout, infinity}]). + +run(QName) -> cast(QName, run). +pause(QName) -> cast(QName, pause). +go() -> + _ = rabbit_federation_pg:start_scope(?FEDERATION_PG_SCOPE), + cast(go). + +%%---------------------------------------------------------------------------- +%%call(QName, Msg) -> [gen_server2:call(Pid, Msg, infinity) || Pid <- q(QName)]. +cast(Msg) -> [gen_server2:cast(Pid, Msg) || Pid <- all()]. +cast(QName, Msg) -> [gen_server2:cast(Pid, Msg) || Pid <- q(QName)]. + +join(Name) -> + ok = pg:join(?FEDERATION_PG_SCOPE, pgname(Name), self()). + +all() -> + pg:get_members(?FEDERATION_PG_SCOPE, pgname(rabbit_federation_queues)). + +q(QName) -> + pg:get_members(?FEDERATION_PG_SCOPE, pgname({rabbit_federation_queue, QName})). + +%%---------------------------------------------------------------------------- + +init({Upstream, Queue}) when ?is_amqqueue(Queue) -> + QName = amqqueue:get_name(Queue), + case rabbit_amqqueue:lookup(QName) of + {ok, Q} -> + DeobfuscatedUpstream = rabbit_federation_util:deobfuscate_upstream(Upstream), + DeobfuscatedUParams = rabbit_federation_upstream:to_params(DeobfuscatedUpstream, Queue), + UParams = rabbit_federation_util:obfuscate_upstream_params(DeobfuscatedUParams), + rabbit_federation_status:report(Upstream, UParams, QName, starting), + join(rabbit_federation_queues), + join({rabbit_federation_queue, QName}), + gen_server2:cast(self(), maybe_go), + rabbit_amqqueue:notify_decorators(Q), + {ok, #not_started{queue = Queue, + run = false, + upstream = Upstream, + upstream_params = UParams}}; + {error, not_found} -> + rabbit_federation_link_util:log_warning(QName, "not found, stopping link", []), + {stop, gone} + end. + +handle_call(Msg, _From, State) -> + {stop, {unexpected_call, Msg}, {unexpected_call, Msg}, State}. + +handle_cast(maybe_go, State) -> + go(State); + +handle_cast(go, State = #not_started{}) -> + go(State); + +handle_cast(go, State) -> + {noreply, State}; + +handle_cast(run, State = #state{upstream = Upstream, + upstream_params = UParams, + ch = Ch, + run = false}) -> + consume(Ch, Upstream, UParams#upstream_params.x_or_q), + {noreply, State#state{run = true}}; + +handle_cast(run, State = #not_started{}) -> + {noreply, State#not_started{run = true}}; + +handle_cast(run, State) -> + %% Already started + {noreply, State}; + +handle_cast(pause, State = #state{run = false}) -> + %% Already paused + {noreply, State}; + +handle_cast(pause, State = #not_started{}) -> + {noreply, State#not_started{run = false}}; + +handle_cast(pause, State = #state{ch = Ch, upstream = Upstream}) -> + cancel(Ch, Upstream), + {noreply, State#state{run = false}}; + +handle_cast(Msg, State) -> + {stop, {unexpected_cast, Msg}, State}. + +handle_info(#'basic.consume_ok'{}, State) -> + {noreply, State}; + +handle_info(#'basic.ack'{} = Ack, State = #state{ch = Ch, + unacked = Unacked}) -> + Unacked1 = rabbit_federation_link_util:ack(Ack, Ch, Unacked), + {noreply, State#state{unacked = Unacked1}}; + +handle_info(#'basic.nack'{} = Nack, State = #state{ch = Ch, + unacked = Unacked}) -> + Unacked1 = rabbit_federation_link_util:nack(Nack, Ch, Unacked), + {noreply, State#state{unacked = Unacked1}}; + +handle_info({#'basic.deliver'{redelivered = Redelivered, + exchange = X, + routing_key = K} = DeliverMethod, Msg}, + State = #state{queue = Q, + upstream = Upstream, + upstream_params = UParams, + ch = Ch, + dch = DCh, + unacked = Unacked}) when ?is_amqqueue(Q) -> + QName = amqqueue:get_name(Q), + PublishMethod = #'basic.publish'{exchange = <<"">>, + routing_key = QName#resource.name}, + HeadersFun = fun (H) -> update_headers(UParams, Redelivered, X, K, H) end, + ForwardFun = fun (_H) -> true end, + Unacked1 = rabbit_federation_link_util:forward( + Upstream, DeliverMethod, Ch, DCh, PublishMethod, + HeadersFun, ForwardFun, Msg, Unacked), + %% TODO actually we could reject when 'stopped' + {noreply, State#state{unacked = Unacked1}}; + +handle_info(#'basic.cancel'{}, + State = #state{queue = Q, + upstream = Upstream, + upstream_params = UParams}) when ?is_amqqueue(Q) -> + QName = amqqueue:get_name(Q), + rabbit_federation_link_util:connection_error( + local, basic_cancel, Upstream, UParams, QName, State); + +handle_info({'DOWN', _Ref, process, Pid, Reason}, + State = #state{dch = DCh, + ch = Ch, + upstream = Upstream, + upstream_params = UParams, + queue = Q}) when ?is_amqqueue(Q) -> + QName = amqqueue:get_name(Q), + handle_down(Pid, Reason, Ch, DCh, {Upstream, UParams, QName}, State); + +handle_info(Msg, State) -> + {stop, {unexpected_info, Msg}, State}. + +terminate(Reason, #not_started{upstream = Upstream, + upstream_params = UParams, + queue = Q}) when ?is_amqqueue(Q) -> + QName = amqqueue:get_name(Q), + rabbit_federation_link_util:log_terminate(Reason, Upstream, UParams, QName), + _ = pg:leave(?FEDERATION_PG_SCOPE, pgname({rabbit_federation_queue, QName}), self()), + ok; + +terminate(Reason, #state{dconn = DConn, + conn = Conn, + upstream = Upstream, + upstream_params = UParams, + queue = Q}) when ?is_amqqueue(Q) -> + QName = amqqueue:get_name(Q), + rabbit_federation_link_util:ensure_connection_closed(DConn), + rabbit_federation_link_util:ensure_connection_closed(Conn), + rabbit_federation_link_util:log_terminate(Reason, Upstream, UParams, QName), + _ = pg:leave(?FEDERATION_PG_SCOPE, pgname({rabbit_federation_queue, QName}), self()), + ok. + +code_change(_OldVsn, State, _Extra) -> + {ok, State}. + +%%---------------------------------------------------------------------------- + +go(S0 = #not_started{run = Run, + upstream = Upstream = #upstream{ + prefetch_count = Prefetch}, + upstream_params = UParams, + queue = Queue}) when ?is_amqqueue(Queue) -> + QName = amqqueue:get_name(Queue), + #upstream_params{x_or_q = UQueue} = UParams, + Durable = amqqueue:is_durable(UQueue), + AutoDelete = amqqueue:is_auto_delete(UQueue), + Args = amqqueue:get_arguments(UQueue), + Unacked = rabbit_federation_link_util:unacked_new(), + rabbit_federation_link_util:start_conn_ch( + fun (Conn, Ch, DConn, DCh) -> + check_upstream_suitable(Conn), + Declare = #'queue.declare'{queue = name(UQueue), + durable = Durable, + auto_delete = AutoDelete, + arguments = Args}, + rabbit_federation_link_util:disposable_channel_call( + Conn, Declare#'queue.declare'{passive = true}, + fun(?NOT_FOUND, _Text) -> + amqp_channel:call(Ch, Declare) + end), + case Upstream#upstream.ack_mode of + 'no-ack' -> ok; + _ -> amqp_channel:call( + Ch, #'basic.qos'{prefetch_count = Prefetch}) + end, + amqp_selective_consumer:register_default_consumer(Ch, self()), + case Run of + true -> consume(Ch, Upstream, UQueue); + false -> ok + end, + {noreply, #state{queue = Queue, + run = Run, + conn = Conn, + ch = Ch, + dconn = DConn, + dch = DCh, + upstream = Upstream, + upstream_params = UParams, + unacked = Unacked}} + end, Upstream, UParams, QName, S0). + +check_upstream_suitable(Conn) -> + Props = pget(server_properties, + amqp_connection:info(Conn, [server_properties])), + {table, Caps} = rabbit_misc:table_lookup(Props, <<"capabilities">>), + case rabbit_misc:table_lookup(Caps, <<"consumer_priorities">>) of + {bool, true} -> ok; + _ -> exit({error, upstream_lacks_consumer_priorities}) + end. + +update_headers(UParams, Redelivered, X, K, undefined) -> + update_headers(UParams, Redelivered, X, K, []); + +update_headers(#upstream_params{table = Table}, Redelivered, X, K, Headers) -> + {Headers1, Count} = + case rabbit_misc:table_lookup(Headers, ?ROUTING_HEADER) of + undefined -> + %% We only want to record the original exchange and + %% routing key the first time a message gets + %% forwarded; after that it's known that they were + %% <<>> and QueueName respectively. + {init_x_original_source_headers(Headers, X, K), 0}; + {array, Been} -> + update_visit_count(Table, Been, Headers); + %% this means the header comes from the client + %% which re-published the message, most likely unintentionally. + %% We can't assume much about the value, so we simply ignore it. + _Other -> + {init_x_original_source_headers(Headers, X, K), 0} + end, + rabbit_basic:prepend_table_header( + ?ROUTING_HEADER, Table ++ [{<<"redelivered">>, bool, Redelivered}, + {<<"visit-count">>, long, Count + 1}], + swap_cc_header(Headers1)). + +init_x_original_source_headers(Headers, X, K) -> + rabbit_misc:set_table_value( + rabbit_misc:set_table_value( + Headers, <<"x-original-exchange">>, longstr, X), + <<"x-original-routing-key">>, longstr, K). + +update_visit_count(Table, Been, Headers) -> + {Found, Been1} = lists:partition( + fun(I) -> visit_match(I, Table) end, + Been), + C = case Found of + [] -> 0; + [{table, T}] -> case rabbit_misc:table_lookup( + T, <<"visit-count">>) of + {_, I} when is_number(I) -> I; + _ -> 0 + end + end, + {rabbit_misc:set_table_value( + Headers, ?ROUTING_HEADER, array, Been1), C}. + +swap_cc_header(Table) -> + [{case K of + <<"CC">> -> <<"x-original-cc">>; + _ -> K + end, T, V} || {K, T, V} <- Table]. + +visit_match({table, T}, Info) -> + lists:all(fun (K) -> + rabbit_misc:table_lookup(T, K) =:= + rabbit_misc:table_lookup(Info, K) + end, [<<"uri">>, <<"virtual_host">>, <<"queue">>]); +visit_match(_ ,_) -> + false. + +consumer_tag(#upstream{consumer_tag = ConsumerTag}) -> + ConsumerTag. + +consume(Ch, Upstream, UQueue) -> + ConsumerTag = consumer_tag(Upstream), + NoAck = Upstream#upstream.ack_mode =:= 'no-ack', + amqp_channel:cast( + Ch, #'basic.consume'{queue = name(UQueue), + no_ack = NoAck, + nowait = true, + consumer_tag = ConsumerTag, + arguments = [{<<"x-priority">>, long, -1}]}). + +cancel(Ch, Upstream) -> + ConsumerTag = consumer_tag(Upstream), + amqp_channel:cast(Ch, #'basic.cancel'{nowait = true, + consumer_tag = ConsumerTag}). + +handle_down(DCh, Reason, _Ch, DCh, Args, State) -> + rabbit_federation_link_util:handle_downstream_down(Reason, Args, State); +handle_down(Ch, Reason, Ch, _DCh, Args, State) -> + rabbit_federation_link_util:handle_upstream_down(Reason, Args, State). diff --git a/deps/rabbitmq_queue_federation/src/rabbit_federation_queue_link_sup_sup.erl b/deps/rabbitmq_queue_federation/src/rabbit_federation_queue_link_sup_sup.erl new file mode 100644 index 000000000000..945c5d35cc85 --- /dev/null +++ b/deps/rabbitmq_queue_federation/src/rabbit_federation_queue_link_sup_sup.erl @@ -0,0 +1,98 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_queue_link_sup_sup). + +-behaviour(mirrored_supervisor). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("rabbit/include/amqqueue.hrl"). +-include("rabbit_queue_federation.hrl"). +-define(SUPERVISOR, ?MODULE). + +%% Supervises the upstream links for all queues (but not exchanges). We need +%% different handling here since queues do not want a mirrored sup. + +-export([start_link/0, start_child/1, adjust/1, stop_child/1]). +-export([init/1]). +-export([id_to_khepri_path/1]). + +%%---------------------------------------------------------------------------- + +start_link() -> + _ = pg:start_link(), + %% This scope is used by concurrently starting exchange and queue links, + %% and other places, so we have to start it very early outside of the supervision tree. + %% The scope is stopped in stop/1. + _ = rabbit_federation_pg:start_scope(?FEDERATION_PG_SCOPE), + mirrored_supervisor:start_link({local, ?SUPERVISOR}, ?SUPERVISOR, + ?MODULE, []). + +%% Note that the next supervisor down, rabbit_federation_link_sup, is common +%% between exchanges and queues. +start_child(Q) -> + case mirrored_supervisor:start_child( + ?SUPERVISOR, + {id(Q), {rabbit_federation_link_sup, start_link, [rabbit_federation_queue_link, Q]}, + transient, ?SUPERVISOR_WAIT, supervisor, + [rabbit_federation_link_sup]}) of + {ok, _Pid} -> ok; + {error, {already_started, _Pid}} -> + QueueName = amqqueue:get_name(Q), + rabbit_log_federation:warning("Federation link for queue ~tp was already started", + [rabbit_misc:rs(QueueName)]), + ok; + %% A link returned {stop, gone}, the link_sup shut down, that's OK. + {error, {shutdown, _}} -> ok + end. + + +adjust({clear_upstream, VHost, UpstreamName}) -> + _ = [rabbit_federation_link_sup:adjust(Pid, rabbit_federation_queue_link, Q, {clear_upstream, UpstreamName}) || + {Q, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR), + ?amqqueue_vhost_equals(Q, VHost)], + ok; +adjust(Reason) -> + _ = [rabbit_federation_link_sup:adjust(Pid, rabbit_federation_queue_link, Q, Reason) || + {Q, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR)], + ok. + +stop_child(Q) -> + case mirrored_supervisor:terminate_child(?SUPERVISOR, id(Q)) of + ok -> ok; + {error, Err} -> + QueueName = amqqueue:get_name(Q), + rabbit_log_federation:warning( + "Attempt to stop a federation link for queue ~tp failed: ~tp", + [rabbit_misc:rs(QueueName), Err]), + ok + end, + _ = mirrored_supervisor:delete_child(?SUPERVISOR, id(Q)). + +%%---------------------------------------------------------------------------- + +init([]) -> + {ok, {{one_for_one, 1200, 60}, []}}. + +%% Clean out all mutable aspects of the queue except policy. We need +%% to keep the entire queue around rather than just take its name +%% since we will want to know its policy to determine how to federate +%% it, and its immutable properties in case we want to redeclare it +%% upstream. We don't just take its name and look it up again since +%% that would introduce race conditions when policies change +%% frequently. Note that since we take down all the links and start +%% again when policies change, the policy will always be correct, so +%% we don't clear it out here and can trust it. +id(Q) when ?is_amqqueue(Q) -> + Policy = amqqueue:get_policy(Q), + Q1 = amqqueue:set_immutable(Q), + Q2 = amqqueue:set_policy(Q1, Policy), + Q2. + +id_to_khepri_path(Id) when ?is_amqqueue(Id) -> + #resource{virtual_host = VHost, name = Name} = amqqueue:get_name(Id), + [queue, VHost, Name]. diff --git a/deps/rabbitmq_queue_federation/src/rabbit_queue_federation_app.erl b/deps/rabbitmq_queue_federation/src/rabbit_queue_federation_app.erl new file mode 100644 index 000000000000..541a59d4db0d --- /dev/null +++ b/deps/rabbitmq_queue_federation/src/rabbit_queue_federation_app.erl @@ -0,0 +1,51 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_queue_federation_app). + +-include_lib("rabbitmq_federation_common/include/rabbit_federation.hrl"). +-include("rabbit_queue_federation.hrl"). + +-behaviour(application). +-export([start/2, stop/1]). + +%% Dummy supervisor - see Ulf Wiger's comment at +%% http://erlang.org/pipermail/erlang-questions/2010-April/050508.html + +%% All of our actual server processes are supervised by +%% rabbit_federation_sup, which is started by a rabbit_boot_step +%% (since it needs to start up before queue / exchange recovery, so it +%% can't be part of our application). +%% +%% However, we still need an application behaviour since we need to +%% know when our application has started since then the Erlang client +%% will have started and we can therefore start our links going. Since +%% the application behaviour needs a tree of processes to supervise, +%% this is it... +-behaviour(supervisor). +-export([init/1]). + +start(_Type, _StartArgs) -> + ets:insert(?FEDERATION_ETS, + {rabbitmq_queue_federation, + #{link_module => rabbit_federation_queue_link_sup_sup}}), + supervisor:start_link({local, ?MODULE}, ?MODULE, []). + +stop(_State) -> + ets:delete(?FEDERATION_ETS, rabbitmq_queue_federation), + rabbit_federation_pg:stop_scope(?FEDERATION_PG_SCOPE), + ok. + +%%---------------------------------------------------------------------------- + +init([]) -> + Flags = #{ + strategy => one_for_one, + intensity => 3, + period => 10 + }, + {ok, {Flags, []}}. diff --git a/deps/rabbitmq_queue_federation/src/rabbit_queue_federation_sup.erl b/deps/rabbitmq_queue_federation/src/rabbit_queue_federation_sup.erl new file mode 100644 index 000000000000..0a37547c5bc6 --- /dev/null +++ b/deps/rabbitmq_queue_federation/src/rabbit_queue_federation_sup.erl @@ -0,0 +1,64 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_queue_federation_sup). + +-behaviour(supervisor). + +%% Supervises everything. There is just one of these. + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include("rabbit_queue_federation.hrl"). + +-define(SUPERVISOR, ?MODULE). + +-export([start_link/0, stop/0]). + +-export([init/1]). + +%% This supervisor needs to be part of the rabbit application since +%% a) it needs to be in place when exchange recovery takes place +%% b) it needs to go up and down with rabbit + +-rabbit_boot_step({rabbit_queue_federation_supervisor, + [{description, "federation"}, + {mfa, {rabbit_sup, start_child, [?MODULE]}}, + {requires, [kernel_ready, rabbit_federation_supervisor]}, + {cleanup, {?MODULE, stop, []}}, + {enables, rabbit_federation_queue}]}). + +%%---------------------------------------------------------------------------- + +start_link() -> + supervisor:start_link({local, ?SUPERVISOR}, ?MODULE, []). + +stop() -> + ok = supervisor:terminate_child(rabbit_sup, ?MODULE), + ok = supervisor:delete_child(rabbit_sup, ?MODULE). + +%%---------------------------------------------------------------------------- + +init([]) -> + QLinkSupSup = #{ + id => q_links, + start => {rabbit_federation_queue_link_sup_sup, start_link, []}, + restart => transient, + shutdown => ?SUPERVISOR_WAIT, + type => supervisor, + modules => [rabbit_federation_queue_link_sup_sup] + }, + %% with default reconnect-delay of 5 second, this supports up to + %% 100 links constantly failing and being restarted a minute + %% (or 200 links if reconnect-delay is 10 seconds, 600 with 30 seconds, + %% etc: N * (60/reconnect-delay) <= 1200) + Flags = #{ + strategy => one_for_one, + intensity => 1200, + period => 60 + }, + Specs = [QLinkSupSup], + {ok, {Flags, Specs}}. diff --git a/deps/rabbitmq_queue_federation/test/definition_import_SUITE.erl b/deps/rabbitmq_queue_federation/test/definition_import_SUITE.erl new file mode 100644 index 000000000000..d656d187f1e1 --- /dev/null +++ b/deps/rabbitmq_queue_federation/test/definition_import_SUITE.erl @@ -0,0 +1,104 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(definition_import_SUITE). + +-include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-compile(export_all). + +all() -> + [ + {group, roundtrip} + ]. + +groups() -> + [ + {roundtrip, [], [ + export_import_round_trip + ]} + ]. + +%% ------------------------------------------------------------------- +%% Test suite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + inets:start(), + Config. +end_per_suite(Config) -> + Config. + +init_per_group(Group, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, Group} + ]), + rabbit_ct_helpers:run_setup_steps(Config1, rabbit_ct_broker_helpers:setup_steps()). + +end_per_group(_, Config) -> + rabbit_ct_helpers:run_teardown_steps(Config, 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). + +%% +%% Tests +%% + +export_import_round_trip(Config) -> + case rabbit_ct_helpers:is_mixed_versions() of + false -> + import_file_case(Config, "case1"), + Defs = export(Config), + import_raw(Config, rabbit_json:encode(Defs)); + _ -> + %% skip the test in mixed version mode + {skip, "Should not run in mixed version environments"} + end. + +%% +%% Implementation +%% + +import_file_case(Config, CaseName) -> + CasePath = filename:join([ + ?config(data_dir, Config), + CaseName ++ ".json" + ]), + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, run_import_case, [CasePath]), + ok. + + +import_raw(Config, Body) -> + case rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_definitions, import_raw, [Body]) of + ok -> ok; + {error, E} -> + ct:pal("Import of JSON definitions ~tp failed: ~tp~n", [Body, E]), + ct:fail({expected_failure, Body, E}) + end. + +export(Config) -> + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, run_export, []). + +run_export() -> + rabbit_definitions:all_definitions(). + +run_import_case(Path) -> + {ok, Body} = file:read_file(Path), + ct:pal("Successfully loaded a definition to import from ~tp~n", [Path]), + case rabbit_definitions:import_raw(Body) of + ok -> ok; + {error, E} -> + ct:pal("Import case ~tp failed: ~tp~n", [Path, E]), + ct:fail({expected_failure, Path, E}) + end. diff --git a/deps/rabbitmq_queue_federation/test/definition_import_SUITE_data/case1.json b/deps/rabbitmq_queue_federation/test/definition_import_SUITE_data/case1.json new file mode 100644 index 000000000000..e549e4fd6c1d --- /dev/null +++ b/deps/rabbitmq_queue_federation/test/definition_import_SUITE_data/case1.json @@ -0,0 +1,52 @@ +{ + "permissions": [ + { + "configure": ".*", + "read": ".*", + "user": "guest", + "vhost": "/", + "write": ".*" + } + ], + "bindings": [], + "queues": [], + "parameters": [ + { + "component": "federation-upstream-set", + "name": "location-1", + "value": [ + { + "upstream":"up-1" + }, + { + "upstream":"up-2" + } + ], + "vhost":"/"}], + "policies": [], + "rabbitmq_version": "3.13.0+376.g1bc0d89.dirty", + "users": [ + { + "hashing_algorithm": "rabbit_password_hashing_sha256", + "limits": {}, + "name": "guest", + "password_hash": "jTcCKuOmGJeeRQ/K1LG5sdZLcdnEnqv8wcrP2n68R7nMuqy2", + "tags": ["administrator"] + } + ], + "rabbit_version": "3.13.0+376.g1bc0d89.dirty", + "exchanges": [], + "topic_permissions": [], + "vhosts": [ + { + "limits": [], + "metadata": + { + "description": "Default virtual host", + "tags": [] + }, + "name":"/" + } + ], + "global_parameters": [] +} diff --git a/deps/rabbitmq_queue_federation/test/queue_SUITE.erl b/deps/rabbitmq_queue_federation/test/queue_SUITE.erl new file mode 100644 index 000000000000..c8f3280ca038 --- /dev/null +++ b/deps/rabbitmq_queue_federation/test/queue_SUITE.erl @@ -0,0 +1,395 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(queue_SUITE). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). + +-compile(nowarn_export_all). +-compile(export_all). + +-import(rabbit_federation_test_util, + [wait_for_federation/2, expect/3, expect/4, + set_upstream/4, set_upstream/5, clear_upstream/3, set_upstream_set/4, clear_upstream_set/3, + set_policy/5, clear_policy/3, + set_policy_pattern/5, set_policy_upstream/5, q/2, with_ch/3, + maybe_declare_queue/3, delete_queue/2]). + +-define(INITIAL_WAIT, 6000). +-define(EXPECT_FEDERATION_TIMEOUT, 30000). + +all() -> + [ + {group, classic_queue}, + {group, quorum_queue}, + {group, mixed} + ]. + +groups() -> + [ + {classic_queue, [], all_tests()}, + {quorum_queue, [], all_tests()}, + {mixed, [], all_tests()} + ]. + +all_tests() -> + [ + {without_disambiguate, [], [ + {cluster_size_1, [], [ + simple, + multiple_upstreams_pattern, + multiple_downstreams, + message_flow, + dynamic_reconfiguration, + federate_unfederate, + dynamic_plugin_stop_start + ]} + ]}, + {with_disambiguate, [], [ + {cluster_size_2, [], [restart_upstream]} + ]} + ]. + +%% ------------------------------------------------------------------- +%% 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(classic_queue, Config) -> + rabbit_ct_helpers:set_config( + Config, + [ + {source_queue_type, classic}, + {source_queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]}, + {target_queue_type, classic}, + {target_queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]} + ]); +init_per_group(quorum_queue, Config) -> + rabbit_ct_helpers:set_config( + Config, + [ + {source_queue_type, quorum}, + {source_queue_args, [{<<"x-queue-type">>, longstr, <<"quorum">>}]}, + {target_queue_type, quorum}, + {target_queue_args, [{<<"x-queue-type">>, longstr, <<"quorum">>}]} + ]); +init_per_group(mixed, Config) -> + rabbit_ct_helpers:set_config( + Config, + [ + {source_queue_type, classic}, + {source_queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]}, + {target_queue_type, quorum}, + {target_queue_args, [{<<"x-queue-type">>, longstr, <<"quorum">>}]} + ]); +init_per_group(without_disambiguate, Config) -> + rabbit_ct_helpers:set_config(Config, + {disambiguate_step, []}); +init_per_group(with_disambiguate, Config) -> + rabbit_ct_helpers:set_config(Config, + {disambiguate_step, [fun rabbit_federation_test_util:disambiguate/1]}); +init_per_group(cluster_size_1 = Group, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodes_count, 1} + ]), + init_per_group1(Group, Config1); +init_per_group(cluster_size_2 = Group, Config) -> + case rabbit_ct_helpers:is_mixed_versions() of + true -> + {skip, "not mixed versions compatible"}; + _ -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodes_count, 2} + ]), + init_per_group1(Group, Config1) + end. + +init_per_group1(Group, Config) -> + SetupFederation = case Group of + cluster_size_1 -> [fun rabbit_federation_test_util:setup_federation/1]; + cluster_size_2 -> [] + end, + Disambiguate = ?config(disambiguate_step, Config), + Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"), + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, Suffix}, + {rmq_nodes_clustered, false} + ]), + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps() ++ + SetupFederation ++ Disambiguate). + +end_per_group(without_disambiguate, Config) -> + Config; +end_per_group(with_disambiguate, Config) -> + Config; +end_per_group(classic_queue, Config) -> + Config; +end_per_group(quorum_queue, Config) -> + Config; +end_per_group(mixed, Config) -> + Config; +end_per_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). + +%% ------------------------------------------------------------------- +%% Testcases. +%% ------------------------------------------------------------------- + +simple(Config) -> + with_ch(Config, + fun (Ch) -> + expect_federation(Ch, <<"upstream">>, <<"fed1.downstream">>) + end, upstream_downstream(Config)). + +multiple_upstreams_pattern(Config) -> + set_upstream(Config, 0, <<"local453x">>, + rabbit_ct_broker_helpers:node_uri(Config, 0), [ + {<<"exchange">>, <<"upstream">>}, + {<<"queue">>, <<"upstream">>}]), + + set_upstream(Config, 0, <<"zzzzzZZzz">>, + rabbit_ct_broker_helpers:node_uri(Config, 0), [ + {<<"exchange">>, <<"upstream-zzz">>}, + {<<"queue">>, <<"upstream-zzz">>}]), + + set_upstream(Config, 0, <<"local3214x">>, + rabbit_ct_broker_helpers:node_uri(Config, 0), [ + {<<"exchange">>, <<"upstream2">>}, + {<<"queue">>, <<"upstream2">>}]), + + set_policy_pattern(Config, 0, <<"pattern">>, <<"^pattern\.">>, <<"local\\d+x">>), + + SourceArgs = ?config(source_queue_args, Config), + TargetArgs = ?config(target_queue_args, Config), + with_ch(Config, + fun (Ch) -> + expect_federation(Ch, <<"upstream">>, <<"pattern.downstream">>, ?EXPECT_FEDERATION_TIMEOUT), + expect_federation(Ch, <<"upstream2">>, <<"pattern.downstream">>, ?EXPECT_FEDERATION_TIMEOUT) + end, [q(<<"upstream">>, SourceArgs), + q(<<"upstream2">>, SourceArgs), + q(<<"pattern.downstream">>, TargetArgs)]), + + clear_upstream(Config, 0, <<"local453x">>), + clear_upstream(Config, 0, <<"local3214x">>), + clear_policy(Config, 0, <<"pattern">>). + +multiple_downstreams(Config) -> + Args = ?config(target_queue_args, Config), + with_ch(Config, + fun (Ch) -> + timer:sleep(?INITIAL_WAIT), + expect_federation(Ch, <<"upstream">>, <<"fed1.downstream">>, ?EXPECT_FEDERATION_TIMEOUT), + expect_federation(Ch, <<"upstream2">>, <<"fed2.downstream">>, ?EXPECT_FEDERATION_TIMEOUT) + end, upstream_downstream(Config) ++ [q(<<"fed2.downstream">>, Args)]). + +message_flow(Config) -> + %% TODO: specifc source / target here + Args = ?config(source_queue_args, Config), + with_ch(Config, + fun (Ch) -> + timer:sleep(?INITIAL_WAIT), + publish_expect(Ch, <<>>, <<"one">>, <<"one">>, <<"first one">>, ?EXPECT_FEDERATION_TIMEOUT), + publish_expect(Ch, <<>>, <<"two">>, <<"two">>, <<"first two">>, ?EXPECT_FEDERATION_TIMEOUT), + Seq = lists:seq(1, 50), + [publish(Ch, <<>>, <<"one">>, <<"bulk">>) || _ <- Seq], + [publish(Ch, <<>>, <<"two">>, <<"bulk">>) || _ <- Seq], + expect(Ch, <<"one">>, repeat(100, <<"bulk">>)), + expect_empty(Ch, <<"one">>), + expect_empty(Ch, <<"two">>), + [publish(Ch, <<>>, <<"one">>, <<"bulk">>) || _ <- Seq], + [publish(Ch, <<>>, <<"two">>, <<"bulk">>) || _ <- Seq], + expect(Ch, <<"two">>, repeat(100, <<"bulk">>)), + expect_empty(Ch, <<"one">>), + expect_empty(Ch, <<"two">>), + %% We clear the federation configuration to avoid a race condition + %% when deleting the queues in quorum mode. The federation link + %% would restart and lead to a state where nothing happened for + %% minutes. + clear_upstream_set(Config, 0, <<"one">>), + clear_upstream_set(Config, 0, <<"two">>) + end, [q(<<"one">>, Args), + q(<<"two">>, Args)]). + +dynamic_reconfiguration(Config) -> + with_ch(Config, + fun (Ch) -> + timer:sleep(?INITIAL_WAIT), + expect_federation(Ch, <<"upstream">>, <<"fed1.downstream">>, ?EXPECT_FEDERATION_TIMEOUT), + + %% Test that clearing connections works + clear_upstream(Config, 0, <<"localhost">>), + expect_no_federation(Ch, <<"upstream">>, <<"fed1.downstream">>), + + %% Test that reading them and changing them works + set_upstream(Config, 0, + <<"localhost">>, rabbit_ct_broker_helpers:node_uri(Config, 0)), + %% Do it twice so we at least hit the no-restart optimisation + URI = rabbit_ct_broker_helpers:node_uri(Config, 0, [use_ipaddr]), + set_upstream(Config, 0, <<"localhost">>, URI), + set_upstream(Config, 0, <<"localhost">>, URI), + expect_federation(Ch, <<"upstream">>, <<"fed1.downstream">>) + end, upstream_downstream(Config)). + +federate_unfederate(Config) -> + Args = ?config(target_queue_args, Config), + with_ch(Config, + fun (Ch) -> + timer:sleep(?INITIAL_WAIT), + expect_federation(Ch, <<"upstream">>, <<"fed1.downstream">>, ?EXPECT_FEDERATION_TIMEOUT), + expect_federation(Ch, <<"upstream2">>, <<"fed2.downstream">>, ?EXPECT_FEDERATION_TIMEOUT), + + %% clear the policy + rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"fed">>), + + expect_no_federation(Ch, <<"upstream">>, <<"fed1.downstream">>), + expect_no_federation(Ch, <<"upstream2">>, <<"fed2.downstream">>), + + rabbit_ct_broker_helpers:set_policy(Config, 0, + <<"fed">>, <<"^fed1\.">>, <<"all">>, [ + {<<"federation-upstream-set">>, <<"upstream">>}]) + end, upstream_downstream(Config) ++ [q(<<"fed2.downstream">>, Args)]). + +dynamic_plugin_stop_start(Config) -> + DownQ2 = <<"fed2.downstream">>, + Args = ?config(target_queue_args, Config), + with_ch(Config, + fun (Ch) -> + timer:sleep(?INITIAL_WAIT), + UpQ1 = <<"upstream">>, + UpQ2 = <<"upstream2">>, + DownQ1 = <<"fed1.downstream">>, + expect_federation(Ch, UpQ1, DownQ1, ?EXPECT_FEDERATION_TIMEOUT), + expect_federation(Ch, UpQ2, DownQ2, ?EXPECT_FEDERATION_TIMEOUT), + + %% Disable the plugin, the link disappears + ct:pal("Stopping rabbitmq_federation"), + ok = rabbit_ct_broker_helpers:disable_plugin(Config, 0, "rabbitmq_queue_federation"), + + expect_no_federation(Ch, UpQ1, DownQ1), + expect_no_federation(Ch, UpQ2, DownQ2), + + maybe_declare_queue(Config, Ch, q(DownQ1, Args)), + maybe_declare_queue(Config, Ch, q(DownQ2, Args)), + ct:pal("Re-starting rabbitmq_federation"), + ok = rabbit_ct_broker_helpers:enable_plugin(Config, 0, "rabbitmq_queue_federation"), + timer:sleep(?INITIAL_WAIT), + + %% Declare a queue then re-enable the plugin, the links appear + rabbit_ct_helpers:await_condition( + fun() -> + Status = rabbit_ct_broker_helpers:rpc(Config, 0, + rabbit_federation_status, status, []), + L = [ + Entry || Entry <- Status, + proplists:get_value(queue, Entry) =:= DownQ1 orelse + proplists:get_value(queue, Entry) =:= DownQ2, + proplists:get_value(upstream_queue, Entry) =:= UpQ1 orelse + proplists:get_value(upstream_queue, Entry) =:= UpQ2, + proplists:get_value(status, Entry) =:= running + ], + length(L) =:= 2 + end, 90000), + expect_federation(Ch, UpQ1, DownQ1, 120000) + end, upstream_downstream(Config) ++ [q(DownQ2, Args)]). + +restart_upstream(Config) -> + [Rabbit, Hare] = rabbit_ct_broker_helpers:get_node_configs(Config, + nodename), + set_policy_upstream(Config, Rabbit, <<"^test$">>, + rabbit_ct_broker_helpers:node_uri(Config, Hare), []), + + Downstream = rabbit_ct_client_helpers:open_channel(Config, Rabbit), + Upstream = rabbit_ct_client_helpers:open_channel(Config, Hare), + + SourceArgs = ?config(source_queue_args, Config), + TargetArgs = ?config(target_queue_args, Config), + maybe_declare_queue(Config, Upstream, q(<<"test">>, SourceArgs)), + maybe_declare_queue(Config, Downstream, q(<<"test">>, TargetArgs)), + Seq = lists:seq(1, 100), + [publish(Upstream, <<>>, <<"test">>, <<"bulk">>) || _ <- Seq], + expect(Upstream, <<"test">>, repeat(25, <<"bulk">>)), + expect(Downstream, <<"test">>, repeat(25, <<"bulk">>)), + + rabbit_ct_client_helpers:close_channels_and_connection(Config, Hare), + ok = rabbit_ct_broker_helpers:restart_node(Config, Hare), + Upstream2 = rabbit_ct_client_helpers:open_channel(Config, Hare), + + expect(Upstream2, <<"test">>, repeat(25, <<"bulk">>)), + expect(Downstream, <<"test">>, repeat(25, <<"bulk">>)), + expect_empty(Upstream2, <<"test">>), + expect_empty(Downstream, <<"test">>), + + ok. + +%upstream_has_no_federation(Config) -> +% %% TODO +% ok. + +%%---------------------------------------------------------------------------- +repeat(Count, Item) -> [Item || _ <- lists:seq(1, Count)]. + +%%---------------------------------------------------------------------------- + +publish(Ch, X, Key, Payload) when is_binary(Payload) -> + publish(Ch, X, Key, #amqp_msg{payload = Payload}); + +publish(Ch, X, Key, Msg = #amqp_msg{}) -> + amqp_channel:call(Ch, #'basic.publish'{exchange = X, + routing_key = Key}, Msg). + +publish_expect(Ch, X, Key, Q, Payload) -> + publish(Ch, X, Key, Payload), + expect(Ch, Q, [Payload]). + +publish_expect(Ch, X, Key, Q, Payload, Timeout) -> + publish(Ch, X, Key, Payload), + expect(Ch, Q, [Payload], Timeout). + +%% Doubled due to our strange basic.get behaviour. +expect_empty(Ch, Q) -> + rabbit_federation_test_util:expect_empty(Ch, Q), + rabbit_federation_test_util:expect_empty(Ch, Q). + +expect_federation(Ch, UpstreamQ, DownstreamQ) -> + Base = <<"HELLO">>, + Payload = <>, + publish_expect(Ch, <<>>, UpstreamQ, DownstreamQ, Payload). + +expect_federation(Ch, UpstreamQ, DownstreamQ, Timeout) -> + Base = <<"HELLO">>, + Payload = <>, + publish_expect(Ch, <<>>, UpstreamQ, DownstreamQ, Payload, Timeout). + +expect_no_federation(Ch, UpstreamQ, DownstreamQ) -> + publish(Ch, <<>>, UpstreamQ, <<"HELLO">>), + expect_empty(Ch, DownstreamQ), + expect(Ch, UpstreamQ, [<<"HELLO">>]). + +upstream_downstream() -> + upstream_downstream([]). + +upstream_downstream(Config) -> + SourceArgs = ?config(source_queue_args, Config), + TargetArgs = ?config(target_queue_args, Config), + [q(<<"upstream">>, SourceArgs), q(<<"fed1.downstream">>, TargetArgs)]. diff --git a/deps/rabbitmq_queue_federation/test/queue_federation_status_command_SUITE.erl b/deps/rabbitmq_queue_federation/test/queue_federation_status_command_SUITE.erl new file mode 100644 index 000000000000..84ed176d103a --- /dev/null +++ b/deps/rabbitmq_queue_federation/test/queue_federation_status_command_SUITE.erl @@ -0,0 +1,172 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(queue_federation_status_command_SUITE). + +-include_lib("amqp_client/include/amqp_client.hrl"). + +-compile(export_all). + +-define(CMD, 'Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand'). + +all() -> + [ + {group, not_federated}, + {group, federated}, + {group, federated_down} + ]. + +groups() -> + [ + {not_federated, [], [ + run_not_federated, + output_not_federated + ]}, + {federated, [], [ + run_federated, + output_federated + ]}, + {federated_down, [], [ + run_down_federated + ]} + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, ?MODULE} + ]), + Config2 = rabbit_ct_helpers:run_setup_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()), + Config2. + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_group(federated, Config) -> + rabbit_federation_test_util:setup_federation(Config), + Config; +init_per_group(federated_down, Config) -> + rabbit_federation_test_util:setup_down_federation(Config), + Config; +init_per_group(_, Config) -> + Config. + +end_per_group(_, Config) -> + Config. + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% ------------------------------------------------------------------- +%% Testcases. +%% ------------------------------------------------------------------- +run_not_federated(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + {stream, []} = ?CMD:run([], Opts#{only_down => false}). + +output_not_federated(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + {stream, []} = ?CMD:output({stream, []}, Opts). + +run_federated(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + %% All + rabbit_federation_test_util:with_ch( + Config, + fun(_) -> + timer:sleep(3000), + {stream, [Props]} = ?CMD:run([], Opts#{only_down => false}), + <<"upstream">> = proplists:get_value(upstream_queue, Props), + <<"fed1.downstream">> = proplists:get_value(queue, Props), + <<"fed.tag">> = proplists:get_value(consumer_tag, Props), + running = proplists:get_value(status, Props) + end, + [rabbit_federation_test_util:q(<<"upstream">>), + rabbit_federation_test_util:q(<<"fed1.downstream">>)]), + %% Down + rabbit_federation_test_util:with_ch( + Config, + fun(_) -> + {stream, []} = ?CMD:run([], Opts#{only_down => true}) + end, + [rabbit_federation_test_util:q(<<"upstream">>), + rabbit_federation_test_util:q(<<"fed1.downstream">>)]). + +run_down_federated(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + %% All + rabbit_federation_test_util:with_ch( + Config, + fun(_) -> + rabbit_ct_helpers:await_condition( + fun() -> + {stream, ManyProps} = ?CMD:run([], Opts#{only_down => false}), + Links = [{proplists:get_value(upstream, Props), + proplists:get_value(status, Props)} + || Props <- ManyProps], + [{<<"broken-bunny">>, error}, {<<"localhost">>, running}] + == lists:sort(Links) + end, 15000) + end, + [rabbit_federation_test_util:q(<<"upstream">>), + rabbit_federation_test_util:q(<<"fed1.downstream">>)]), + %% Down + rabbit_federation_test_util:with_ch( + Config, + fun(_) -> + rabbit_ct_helpers:await_condition( + fun() -> + {stream, Props} = ?CMD:run([], Opts#{only_down => true}), + (length(Props) == 1) + andalso (<<"broken-bunny">> == proplists:get_value(upstream, hd(Props))) + andalso (error == proplists:get_value(status, hd(Props))) + end, 15000) + end, + [rabbit_federation_test_util:q(<<"upstream">>), + rabbit_federation_test_util:q(<<"fed1.downstream">>)]). + +output_federated(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + Input = {stream,[[{queue, <<"fed1.downstream">>}, + {consumer_tag, <<"fed.tag">>}, + {upstream_queue, <<"upstream">>}, + {type, queue}, + {vhost, <<"/">>}, + {upstream, <<"localhost">>}, + {status, running}, + {local_connection, <<"">>}, + {uri, <<"amqp://localhost:21000">>}, + {timestamp, {{2016,11,21},{8,51,19}}}]]}, + {stream, [#{queue := <<"fed1.downstream">>, + upstream_queue := <<"upstream">>, + type := queue, + vhost := <<"/">>, + upstream := <<"localhost">>, + status := running, + local_connection := <<"">>, + uri := <<"amqp://localhost:21000">>, + last_changed := <<"2016-11-21 08:51:19">>, + exchange := <<>>, + upstream_exchange := <<>>, + error := <<>>}]} + = ?CMD:output(Input, Opts). diff --git a/deps/rabbitmq_queue_federation/test/rabbit_federation_status_SUITE.erl b/deps/rabbitmq_queue_federation/test/rabbit_federation_status_SUITE.erl new file mode 100644 index 000000000000..1af890cb91e0 --- /dev/null +++ b/deps/rabbitmq_queue_federation/test/rabbit_federation_status_SUITE.erl @@ -0,0 +1,108 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_status_SUITE). + +-include_lib("amqp_client/include/amqp_client.hrl"). + +-include("rabbit_queue_federation.hrl"). + +-compile(export_all). + +-import(rabbit_federation_test_util, + [expect/3, expect_empty/2, + set_upstream/4, clear_upstream/3, set_upstream_set/4, + set_policy/5, clear_policy/3, + with_ch/3]). + +all() -> + [ + {group, non_parallel_tests} + ]. + +groups() -> + [ + {non_parallel_tests, [], [ + queue_status, + lookup_queue_status, + lookup_bad_status + ]} + ]. + +suite() -> + [{timetrap, {minutes, 5}}]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, ?MODULE} + ]), + rabbit_ct_helpers:run_setup_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps() ++ + [fun rabbit_federation_test_util:setup_federation/1]). +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_group(_, Config) -> + Config. + +end_per_group(_, Config) -> + Config. + +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 +%% ------------------------------------------------------------------- + +queue_status(Config) -> + with_ch( + Config, + fun (_Ch) -> + timer:sleep(3000), + [Link] = rabbit_ct_broker_helpers:rpc(Config, 0, + rabbit_federation_status, status, + []), + true = is_binary(proplists:get_value(id, Link)) + end, queue_SUITE:upstream_downstream()). + +lookup_queue_status(Config) -> + with_ch( + Config, + fun (_Ch) -> + timer:sleep(3000), + [Link] = rabbit_ct_broker_helpers:rpc(Config, 0, + rabbit_federation_status, status, + []), + Id = proplists:get_value(id, Link), + Props = rabbit_ct_broker_helpers:rpc(Config, 0, + rabbit_federation_status, lookup, + [Id]), + lists:all(fun(K) -> lists:keymember(K, 1, Props) end, + [key, uri, status, timestamp, id, supervisor, upstream]) + end, queue_SUITE:upstream_downstream()). + +lookup_bad_status(Config) -> + with_ch( + Config, + fun (_Ch) -> + timer:sleep(3000), + not_found = rabbit_ct_broker_helpers:rpc( + Config, 0, + rabbit_federation_status, lookup, + [<<"justmadeitup">>]) + end, queue_SUITE:upstream_downstream()). diff --git a/deps/rabbitmq_queue_federation/test/rabbit_federation_test_util.erl b/deps/rabbitmq_queue_federation/test/rabbit_federation_test_util.erl new file mode 100644 index 000000000000..fecfda3ed0de --- /dev/null +++ b/deps/rabbitmq_queue_federation/test/rabbit_federation_test_util.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) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_federation_test_util). + +-include_lib("rabbitmq_federation_common/include/rabbit_federation.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). + +-compile(export_all). + +-import(rabbit_misc, [pget/2]). + +setup_federation(Config) -> + setup_federation_with_upstream_params(Config, []). + +setup_federation_with_upstream_params(Config, ExtraParams) -> + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream">>, <<"localhost">>, [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, + {<<"consumer-tag">>, <<"fed.tag">>} + ] ++ ExtraParams), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream">>, <<"local5673">>, [ + {<<"uri">>, <<"amqp://localhost:1">>} + ] ++ ExtraParams), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream-set">>, <<"upstream">>, [ + [ + {<<"upstream">>, <<"localhost">>}, + {<<"exchange">>, <<"upstream">>}, + {<<"queue">>, <<"upstream">>} + ] + ]), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream-set">>, <<"upstream2">>, [ + [ + {<<"upstream">>, <<"localhost">>}, + {<<"exchange">>, <<"upstream2">>}, + {<<"queue">>, <<"upstream2">>} + ] + ]), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream-set">>, <<"localhost">>, [ + [{<<"upstream">>, <<"localhost">>}] + ]), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream-set">>, <<"upstream12">>, [ + [ + {<<"upstream">>, <<"localhost">>}, + {<<"exchange">>, <<"upstream">>}, + {<<"queue">>, <<"upstream">>} + ], [ + {<<"upstream">>, <<"localhost">>}, + {<<"exchange">>, <<"upstream2">>}, + {<<"queue">>, <<"upstream2">>} + ] + ]), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream-set">>, <<"one">>, [ + [ + {<<"upstream">>, <<"localhost">>}, + {<<"exchange">>, <<"one">>}, + {<<"queue">>, <<"one">>} + ] + ]), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream-set">>, <<"two">>, [ + [ + {<<"upstream">>, <<"localhost">>}, + {<<"exchange">>, <<"two">>}, + {<<"queue">>, <<"two">>} + ] + ]), + + rabbit_ct_broker_helpers:set_parameter(Config, 0, + <<"federation-upstream-set">>, <<"upstream5673">>, [ + [ + {<<"upstream">>, <<"local5673">>}, + {<<"exchange">>, <<"upstream">>} + ] + ]), + + rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_policy, set, + [<<"/">>, <<"fed">>, <<"^fed1\.">>, [{<<"federation-upstream-set">>, <<"upstream">>}], + 0, <<"all">>, <<"acting-user">>]), + + rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_policy, set, + [<<"/">>, <<"fed2">>, <<"^fed2\.">>, [{<<"federation-upstream-set">>, <<"upstream2">>}], + 0, <<"all">>, <<"acting-user">>]), + + rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_policy, set, + [<<"/">>, <<"fed12">>, <<"^fed3\.">>, [{<<"federation-upstream-set">>, <<"upstream12">>}], + 2, <<"all">>, <<"acting-user">>]), + + rabbit_ct_broker_helpers:set_policy(Config, 0, + <<"one">>, <<"^two$">>, <<"all">>, [ + {<<"federation-upstream-set">>, <<"one">>}]), + + rabbit_ct_broker_helpers:set_policy(Config, 0, + <<"two">>, <<"^one$">>, <<"all">>, [ + {<<"federation-upstream-set">>, <<"two">>}]), + + rabbit_ct_broker_helpers:set_policy(Config, 0, + <<"hare">>, <<"^hare\.">>, <<"all">>, [ + {<<"federation-upstream-set">>, <<"upstream5673">>}]), + + rabbit_ct_broker_helpers:set_policy(Config, 0, + <<"all">>, <<"^all\.">>, <<"all">>, [ + {<<"federation-upstream-set">>, <<"all">>}]), + + rabbit_ct_broker_helpers:set_policy(Config, 0, + <<"new">>, <<"^new\.">>, <<"all">>, [ + {<<"federation-upstream-set">>, <<"new-set">>}]), + Config. + +setup_down_federation(Config) -> + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"broken-bunny">>, + [{<<"uri">>, <<"amqp://broken-bunny">>}, + {<<"reconnect-delay">>, 600000}]), + rabbit_ct_broker_helpers:set_parameter( + Config, 0, <<"federation-upstream">>, <<"localhost">>, + [{<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}]), + rabbit_ct_broker_helpers:set_parameter( + Config, 0, + <<"federation-upstream-set">>, <<"upstream">>, + [[{<<"upstream">>, <<"localhost">>}, + {<<"exchange">>, <<"upstream">>}, + {<<"queue">>, <<"upstream">>}], + [{<<"upstream">>, <<"broken-bunny">>}, + {<<"exchange">>, <<"upstream">>}, + {<<"queue">>, <<"upstream">>}]]), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed">>, <<"^fed1\.">>, <<"all">>, [{<<"federation-upstream-set">>, <<"upstream">>}]), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"fed">>, <<"^fed1\.">>, <<"all">>, [{<<"federation-upstream-set">>, <<"upstream">>}]), + Config. + +expect(Ch, Q, Fun) when is_function(Fun) -> + amqp_channel:subscribe(Ch, #'basic.consume'{queue = Q, + no_ack = true}, self()), + CTag = receive + #'basic.consume_ok'{consumer_tag = CT} -> CT + end, + Fun(), + amqp_channel:call(Ch, #'basic.cancel'{consumer_tag = CTag}); + +expect(Ch, Q, Payloads) -> + expect(Ch, Q, fun() -> expect(Payloads) end). + +expect(Ch, Q, Payloads, Timeout) -> + expect(Ch, Q, fun() -> expect(Payloads, Timeout) end). + +expect([]) -> + ok; +expect(Payloads) -> + expect(Payloads, 60000). + +expect([], _Timeout) -> + ok; +expect(Payloads, Timeout) -> + receive + {#'basic.deliver'{delivery_tag = DTag}, #amqp_msg{payload = Payload}} -> + case lists:member(Payload, Payloads) of + true -> + ct:pal("Consumed a message: ~tp ~tp left: ~tp", [Payload, DTag, length(Payloads) - 1]), + expect(Payloads -- [Payload], Timeout); + false -> ?assert(false, rabbit_misc:format("received an unexpected payload ~tp", [Payload])) + end + after Timeout -> + ct:fail("Did not receive expected payloads ~tp in time", [Payloads]) + end. + +expect_empty(Ch, Q) -> + ?assertMatch(#'basic.get_empty'{}, + amqp_channel:call(Ch, #'basic.get'{ queue = Q })). + +set_upstream(Config, Node, Name, URI) -> + set_upstream(Config, Node, Name, URI, []). + +set_upstream(Config, Node, Name, URI, Extra) -> + rabbit_ct_broker_helpers:set_parameter(Config, Node, + <<"federation-upstream">>, Name, [{<<"uri">>, URI} | Extra]). + +set_upstream_in_vhost(Config, Node, VirtualHost, Name, URI) -> + set_upstream_in_vhost(Config, Node, VirtualHost, Name, URI, []). + +set_upstream_in_vhost(Config, Node, VirtualHost, Name, URI, Extra) -> + rabbit_ct_broker_helpers:set_parameter(Config, Node, VirtualHost, + <<"federation-upstream">>, Name, [{<<"uri">>, URI} | Extra]). + +clear_upstream(Config, Node, Name) -> + rabbit_ct_broker_helpers:clear_parameter(Config, Node, + <<"federation-upstream">>, Name). + +set_upstream_set(Config, Node, Name, Set) -> + rabbit_ct_broker_helpers:set_parameter(Config, Node, + <<"federation-upstream-set">>, Name, + [[{<<"upstream">>, UStream} | Extra] || {UStream, Extra} <- Set]). + +clear_upstream_set(Config, Node, Name) -> + rabbit_ct_broker_helpers:clear_parameter(Config, Node, + <<"federation-upstream-set">>, Name). + +set_policy(Config, Node, Name, Pattern, UpstreamSet) -> + rabbit_ct_broker_helpers:set_policy(Config, Node, + Name, Pattern, <<"all">>, + [{<<"federation-upstream-set">>, UpstreamSet}]). + +set_policy_pattern(Config, Node, Name, Pattern, Regex) -> + rabbit_ct_broker_helpers:set_policy(Config, Node, + Name, Pattern, <<"all">>, + [{<<"federation-upstream-pattern">>, Regex}]). + +clear_policy(Config, Node, Name) -> + rabbit_ct_broker_helpers:clear_policy(Config, Node, Name). + +set_policy_upstream(Config, Node, Pattern, URI, Extra) -> + set_policy_upstreams(Config, Node, Pattern, [{URI, Extra}]). + +set_policy_upstreams(Config, Node, Pattern, URIExtras) -> + put(upstream_num, 1), + [set_upstream(Config, Node, gen_upstream_name(), URI, Extra) + || {URI, Extra} <- URIExtras], + set_policy(Config, Node, Pattern, Pattern, <<"all">>). + +gen_upstream_name() -> + list_to_binary("upstream-" ++ integer_to_list(next_upstream_num())). + +next_upstream_num() -> + R = get(upstream_num) + 1, + put(upstream_num, R), + R. + +%% Make sure that even though multiple nodes are in a single +%% distributed system, we still keep all our process groups separate. +disambiguate(Config) -> + rabbit_ct_broker_helpers:rpc_all(Config, + application, set_env, + [rabbitmq_federation, pgroup_name_cluster_id, true]), + Config. + +%%---------------------------------------------------------------------------- + +with_ch(Config, Fun, Methods) -> + Ch = rabbit_ct_client_helpers:open_channel(Config), + declare_all(Config, Ch, Methods), + %% Clean up queues even after test failure. + try + Fun(Ch) + after + delete_all(Ch, Methods), + rabbit_ct_client_helpers:close_channel(Ch) + end, + ok. + +declare_all(Config, Ch, Methods) -> [maybe_declare_queue(Config, Ch, Op) || Op <- Methods]. +delete_all(Ch, Methods) -> + [delete_queue(Ch, Q) || #'queue.declare'{queue = Q} <- Methods]. + +maybe_declare_queue(Config, Ch, Method) -> + OneOffCh = rabbit_ct_client_helpers:open_channel(Config), + try + amqp_channel:call(OneOffCh, Method#'queue.declare'{passive = true}) + catch exit:{{shutdown, {server_initiated_close, ?NOT_FOUND, _Message}}, _} -> + amqp_channel:call(Ch, Method) + after + catch rabbit_ct_client_helpers:close_channel(OneOffCh) + end. + +delete_queue(Ch, Q) -> + amqp_channel:call(Ch, #'queue.delete'{queue = Q}). + +q(Name) -> + q(Name, []). + +q(Name, undefined) -> + q(Name, []); +q(Name, Args) -> + #'queue.declare'{queue = Name, + durable = true, + arguments = Args}. diff --git a/deps/rabbitmq_queue_federation/test/rabbit_queue_federation_status_SUITE.erl b/deps/rabbitmq_queue_federation/test/rabbit_queue_federation_status_SUITE.erl new file mode 100644 index 000000000000..42142dd79800 --- /dev/null +++ b/deps/rabbitmq_queue_federation/test/rabbit_queue_federation_status_SUITE.erl @@ -0,0 +1,107 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(rabbit_queue_federation_status_SUITE). + +-include_lib("amqp_client/include/amqp_client.hrl"). + +-include("rabbit_queue_federation.hrl"). + +-compile(export_all). + +-import(rabbit_federation_test_util, + [expect/3, expect_empty/2, + set_upstream/4, clear_upstream/3, set_upstream_set/4, + set_policy/5, clear_policy/3, + with_ch/3]). + +all() -> + [ + {group, non_parallel_tests} + ]. + +groups() -> + [ + {non_parallel_tests, [], [ + queue_status, + lookup_queue_status, + lookup_bad_status + ]} + ]. + +suite() -> + [{timetrap, {minutes, 5}}]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, ?MODULE} + ]), + rabbit_ct_helpers:run_setup_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps() ++ + [fun rabbit_federation_test_util:setup_federation/1]). +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_group(_, Config) -> + Config. + +end_per_group(_, Config) -> + Config. + +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 +%% ------------------------------------------------------------------- + +queue_status(Config) -> + with_ch( + Config, + fun (_Ch) -> + timer:sleep(3000), + [Link] = rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_federation_status, status, + []), + true = is_binary(proplists:get_value(id, Link)) + end, queue_SUITE:upstream_downstream()). + +lookup_queue_status(Config) -> + with_ch( + Config, + fun (_Ch) -> + timer:sleep(3000), + [Link] = rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_federation_status, status, + []), + Id = proplists:get_value(id, Link), + Props = rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_federation_status, lookup, + [Id]), + lists:all(fun(K) -> lists:keymember(K, 1, Props) end, + [key, uri, status, timestamp, id, supervisor, upstream]) + end, queue_SUITE:upstream_downstream()). + +lookup_bad_status(Config) -> + with_ch( + Config, + fun (_Ch) -> + timer:sleep(3000), + not_found = rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_federation_status, lookup, + [<<"justmadeitup">>]) + end, queue_SUITE:upstream_downstream()). diff --git a/deps/rabbitmq_queue_federation/test/restart_federation_link_command_SUITE.erl b/deps/rabbitmq_queue_federation/test/restart_federation_link_command_SUITE.erl new file mode 100644 index 000000000000..74565771648e --- /dev/null +++ b/deps/rabbitmq_queue_federation/test/restart_federation_link_command_SUITE.erl @@ -0,0 +1,100 @@ +%% 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-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +%% + +-module(restart_federation_link_command_SUITE). + +-include_lib("amqp_client/include/amqp_client.hrl"). + +-compile(export_all). + +-define(CMD, 'Elixir.RabbitMQ.CLI.Ctl.Commands.RestartFederationLinkCommand'). + +all() -> + [ + {group, federated_down} + ]. + +groups() -> + [ + {federated_down, [], [ + run, + run_not_found, + output + ]} + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, ?MODULE} + ]), + Config2 = rabbit_ct_helpers:run_setup_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()), + Config2. + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). + +init_per_group(federated_down, Config) -> + rabbit_federation_test_util:setup_down_federation(Config), + Config; +init_per_group(_, Config) -> + Config. + +end_per_group(_, Config) -> + Config. + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% ------------------------------------------------------------------- +%% Testcases. +%% ------------------------------------------------------------------- +run_not_federated(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + {stream, []} = ?CMD:run([], Opts#{'only-down' => false}). + +output_not_federated(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + {stream, []} = ?CMD:output({stream, []}, Opts). + +run(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + rabbit_federation_test_util:with_ch( + Config, + fun(_) -> + timer:sleep(3000), + [Link | _] = rabbit_ct_broker_helpers:rpc(Config, 0, + rabbit_federation_status, status, []), + Id = proplists:get_value(id, Link), + ok = ?CMD:run([Id], Opts) + end, + [rabbit_federation_test_util:q(<<"upstream">>), + rabbit_federation_test_util:q(<<"fed1.downstream">>)]). + +run_not_found(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + {error, _ErrorMsg} = ?CMD:run([<<"MakingItUp">>], Opts). + +output(Config) -> + [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Opts = #{node => A}, + ok = ?CMD:output(ok, Opts). From 84fab75dc052e4fab2b0a5070764050ebaef5976 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Mon, 19 May 2025 12:29:47 +0200 Subject: [PATCH 2/9] Make rabbitmq_federation a no-op plugin --- deps/rabbitmq_federation/CODE_OF_CONDUCT.md | 1 - deps/rabbitmq_federation/CONTRIBUTING.md | 1 - deps/rabbitmq_federation/LICENSE | 3 - deps/rabbitmq_federation/LICENSE-MPL-RabbitMQ | 373 ------- deps/rabbitmq_federation/Makefile | 22 +- deps/rabbitmq_federation/README-hacking | 143 --- deps/rabbitmq_federation/README.md | 27 +- deps/rabbitmq_federation/include/logging.hrl | 3 - .../include/rabbit_federation.hrl | 48 - ...I.Ctl.Commands.FederationStatusCommand.erl | 117 --- ....Commands.RestartFederationLinkCommand.erl | 84 -- .../src/rabbit_federation_app.erl | 44 - .../src/rabbit_federation_db.erl | 45 - .../src/rabbit_federation_event.erl | 54 - .../src/rabbit_federation_exchange.erl | 96 -- .../src/rabbit_federation_exchange_link.erl | 695 ------------- ...abbit_federation_exchange_link_sup_sup.erl | 86 -- .../src/rabbit_federation_link_sup.erl | 109 --- .../src/rabbit_federation_link_util.erl | 359 ------- .../src/rabbit_federation_parameters.erl | 141 --- .../src/rabbit_federation_pg.erl | 25 - .../src/rabbit_federation_queue.erl | 109 --- .../src/rabbit_federation_queue_link.erl | 326 ------- .../rabbit_federation_queue_link_sup_sup.erl | 97 -- .../src/rabbit_federation_status.erl | 178 ---- .../src/rabbit_federation_sup.erl | 83 -- .../src/rabbit_federation_upstream.erl | 166 ---- .../rabbit_federation_upstream_exchange.erl | 90 -- .../src/rabbit_federation_util.erl | 102 -- .../src/rabbit_log_federation.erl | 107 -- .../src/rabbitmq_federation_noop.erl | 1 + .../test/definition_import_SUITE.erl | 146 --- .../definition_import_SUITE_data/case1.json | 52 - .../test/exchange_SUITE.erl | 920 ------------------ .../test/federation_status_command_SUITE.erl | 172 ---- deps/rabbitmq_federation/test/queue_SUITE.erl | 397 -------- .../test/rabbit_federation_status_SUITE.erl | 105 -- .../test/rabbit_federation_test_util.erl | 382 -------- .../restart_federation_link_command_SUITE.erl | 100 -- deps/rabbitmq_federation/test/unit_SUITE.erl | 65 -- .../test/unit_inbroker_SUITE.erl | 231 ----- 41 files changed, 10 insertions(+), 6295 deletions(-) delete mode 120000 deps/rabbitmq_federation/CODE_OF_CONDUCT.md delete mode 120000 deps/rabbitmq_federation/CONTRIBUTING.md delete mode 100644 deps/rabbitmq_federation/LICENSE delete mode 100644 deps/rabbitmq_federation/LICENSE-MPL-RabbitMQ delete mode 100644 deps/rabbitmq_federation/README-hacking delete mode 100644 deps/rabbitmq_federation/include/logging.hrl delete mode 100644 deps/rabbitmq_federation/include/rabbit_federation.hrl delete mode 100644 deps/rabbitmq_federation/src/Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand.erl delete mode 100644 deps/rabbitmq_federation/src/Elixir.RabbitMQ.CLI.Ctl.Commands.RestartFederationLinkCommand.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_app.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_db.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_event.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_exchange.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_exchange_link.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_exchange_link_sup_sup.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_link_sup.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_link_util.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_parameters.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_pg.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_queue.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_queue_link.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_queue_link_sup_sup.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_status.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_sup.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_upstream.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_upstream_exchange.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_federation_util.erl delete mode 100644 deps/rabbitmq_federation/src/rabbit_log_federation.erl create mode 100644 deps/rabbitmq_federation/src/rabbitmq_federation_noop.erl delete mode 100644 deps/rabbitmq_federation/test/definition_import_SUITE.erl delete mode 100644 deps/rabbitmq_federation/test/definition_import_SUITE_data/case1.json delete mode 100644 deps/rabbitmq_federation/test/exchange_SUITE.erl delete mode 100644 deps/rabbitmq_federation/test/federation_status_command_SUITE.erl delete mode 100644 deps/rabbitmq_federation/test/queue_SUITE.erl delete mode 100644 deps/rabbitmq_federation/test/rabbit_federation_status_SUITE.erl delete mode 100644 deps/rabbitmq_federation/test/rabbit_federation_test_util.erl delete mode 100644 deps/rabbitmq_federation/test/restart_federation_link_command_SUITE.erl delete mode 100644 deps/rabbitmq_federation/test/unit_SUITE.erl delete mode 100644 deps/rabbitmq_federation/test/unit_inbroker_SUITE.erl diff --git a/deps/rabbitmq_federation/CODE_OF_CONDUCT.md b/deps/rabbitmq_federation/CODE_OF_CONDUCT.md deleted file mode 120000 index a3613c99f0b0..000000000000 --- a/deps/rabbitmq_federation/CODE_OF_CONDUCT.md +++ /dev/null @@ -1 +0,0 @@ -../../CODE_OF_CONDUCT.md \ No newline at end of file diff --git a/deps/rabbitmq_federation/CONTRIBUTING.md b/deps/rabbitmq_federation/CONTRIBUTING.md deleted file mode 120000 index f939e75f21a8..000000000000 --- a/deps/rabbitmq_federation/CONTRIBUTING.md +++ /dev/null @@ -1 +0,0 @@ -../../CONTRIBUTING.md \ No newline at end of file diff --git a/deps/rabbitmq_federation/LICENSE b/deps/rabbitmq_federation/LICENSE deleted file mode 100644 index e75136bfb5f8..000000000000 --- a/deps/rabbitmq_federation/LICENSE +++ /dev/null @@ -1,3 +0,0 @@ -This package is licensed under the MPL 2.0. For the MPL 2.0, please see LICENSE-MPL-RabbitMQ. - -If you have any questions regarding licensing, please contact us at rabbitmq-core@groups.vmware.com. diff --git a/deps/rabbitmq_federation/LICENSE-MPL-RabbitMQ b/deps/rabbitmq_federation/LICENSE-MPL-RabbitMQ deleted file mode 100644 index 14e2f777f6c3..000000000000 --- a/deps/rabbitmq_federation/LICENSE-MPL-RabbitMQ +++ /dev/null @@ -1,373 +0,0 @@ -Mozilla Public License Version 2.0 -================================== - -1. Definitions --------------- - -1.1. "Contributor" - means each individual or legal entity that creates, contributes to - the creation of, or owns Covered Software. - -1.2. "Contributor Version" - means the combination of the Contributions of others (if any) used - by a Contributor and that particular Contributor's Contribution. - -1.3. "Contribution" - means Covered Software of a particular Contributor. - -1.4. "Covered Software" - means Source Code Form to which the initial Contributor has attached - the notice in Exhibit A, the Executable Form of such Source Code - Form, and Modifications of such Source Code Form, in each case - including portions thereof. - -1.5. "Incompatible With Secondary Licenses" - means - - (a) that the initial Contributor has attached the notice described - in Exhibit B to the Covered Software; or - - (b) that the Covered Software was made available under the terms of - version 1.1 or earlier of the License, but not also under the - terms of a Secondary License. - -1.6. "Executable Form" - means any form of the work other than Source Code Form. - -1.7. "Larger Work" - means a work that combines Covered Software with other material, in - a separate file or files, that is not Covered Software. - -1.8. "License" - means this document. - -1.9. "Licensable" - means having the right to grant, to the maximum extent possible, - whether at the time of the initial grant or subsequently, any and - all of the rights conveyed by this License. - -1.10. "Modifications" - means any of the following: - - (a) any file in Source Code Form that results from an addition to, - deletion from, or modification of the contents of Covered - Software; or - - (b) any new file in Source Code Form that contains any Covered - Software. - -1.11. "Patent Claims" of a Contributor - means any patent claim(s), including without limitation, method, - process, and apparatus claims, in any patent Licensable by such - Contributor that would be infringed, but for the grant of the - License, by the making, using, selling, offering for sale, having - made, import, or transfer of either its Contributions or its - Contributor Version. - -1.12. "Secondary License" - means either the GNU General Public License, Version 2.0, the GNU - Lesser General Public License, Version 2.1, the GNU Affero General - Public License, Version 3.0, or any later versions of those - licenses. - -1.13. "Source Code Form" - means the form of the work preferred for making modifications. - -1.14. "You" (or "Your") - means an individual or a legal entity exercising rights under this - License. For legal entities, "You" includes any entity that - controls, is controlled by, or is under common control with You. For - purposes of this definition, "control" means (a) the power, direct - or indirect, to cause the direction or management of such entity, - whether by contract or otherwise, or (b) ownership of more than - fifty percent (50%) of the outstanding shares or beneficial - ownership of such entity. - -2. License Grants and Conditions --------------------------------- - -2.1. Grants - -Each Contributor hereby grants You a world-wide, royalty-free, -non-exclusive license: - -(a) under intellectual property rights (other than patent or trademark) - Licensable by such Contributor to use, reproduce, make available, - modify, display, perform, distribute, and otherwise exploit its - Contributions, either on an unmodified basis, with Modifications, or - as part of a Larger Work; and - -(b) under Patent Claims of such Contributor to make, use, sell, offer - for sale, have made, import, and otherwise transfer either its - Contributions or its Contributor Version. - -2.2. Effective Date - -The licenses granted in Section 2.1 with respect to any Contribution -become effective for each Contribution on the date the Contributor first -distributes such Contribution. - -2.3. Limitations on Grant Scope - -The licenses granted in this Section 2 are the only rights granted under -this License. No additional rights or licenses will be implied from the -distribution or licensing of Covered Software under this License. -Notwithstanding Section 2.1(b) above, no patent license is granted by a -Contributor: - -(a) for any code that a Contributor has removed from Covered Software; - or - -(b) for infringements caused by: (i) Your and any other third party's - modifications of Covered Software, or (ii) the combination of its - Contributions with other software (except as part of its Contributor - Version); or - -(c) under Patent Claims infringed by Covered Software in the absence of - its Contributions. - -This License does not grant any rights in the trademarks, service marks, -or logos of any Contributor (except as may be necessary to comply with -the notice requirements in Section 3.4). - -2.4. Subsequent Licenses - -No Contributor makes additional grants as a result of Your choice to -distribute the Covered Software under a subsequent version of this -License (see Section 10.2) or under the terms of a Secondary License (if -permitted under the terms of Section 3.3). - -2.5. Representation - -Each Contributor represents that the Contributor believes its -Contributions are its original creation(s) or it has sufficient rights -to grant the rights to its Contributions conveyed by this License. - -2.6. Fair Use - -This License is not intended to limit any rights You have under -applicable copyright doctrines of fair use, fair dealing, or other -equivalents. - -2.7. Conditions - -Sections 3.1, 3.2, 3.3, and 3.4 are conditions of the licenses granted -in Section 2.1. - -3. Responsibilities -------------------- - -3.1. Distribution of Source Form - -All distribution of Covered Software in Source Code Form, including any -Modifications that You create or to which You contribute, must be under -the terms of this License. You must inform recipients that the Source -Code Form of the Covered Software is governed by the terms of this -License, and how they can obtain a copy of this License. You may not -attempt to alter or restrict the recipients' rights in the Source Code -Form. - -3.2. Distribution of Executable Form - -If You distribute Covered Software in Executable Form then: - -(a) such Covered Software must also be made available in Source Code - Form, as described in Section 3.1, and You must inform recipients of - the Executable Form how they can obtain a copy of such Source Code - Form by reasonable means in a timely manner, at a charge no more - than the cost of distribution to the recipient; and - -(b) You may distribute such Executable Form under the terms of this - License, or sublicense it under different terms, provided that the - license for the Executable Form does not attempt to limit or alter - the recipients' rights in the Source Code Form under this License. - -3.3. Distribution of a Larger Work - -You may create and distribute a Larger Work under terms of Your choice, -provided that You also comply with the requirements of this License for -the Covered Software. If the Larger Work is a combination of Covered -Software with a work governed by one or more Secondary Licenses, and the -Covered Software is not Incompatible With Secondary Licenses, this -License permits You to additionally distribute such Covered Software -under the terms of such Secondary License(s), so that the recipient of -the Larger Work may, at their option, further distribute the Covered -Software under the terms of either this License or such Secondary -License(s). - -3.4. Notices - -You may not remove or alter the substance of any license notices -(including copyright notices, patent notices, disclaimers of warranty, -or limitations of liability) contained within the Source Code Form of -the Covered Software, except that You may alter any license notices to -the extent required to remedy known factual inaccuracies. - -3.5. Application of Additional Terms - -You may choose to offer, and to charge a fee for, warranty, support, -indemnity or liability obligations to one or more recipients of Covered -Software. However, You may do so only on Your own behalf, and not on -behalf of any Contributor. You must make it absolutely clear that any -such warranty, support, indemnity, or liability obligation is offered by -You alone, and You hereby agree to indemnify every Contributor for any -liability incurred by such Contributor as a result of warranty, support, -indemnity or liability terms You offer. You may include additional -disclaimers of warranty and limitations of liability specific to any -jurisdiction. - -4. Inability to Comply Due to Statute or Regulation ---------------------------------------------------- - -If it is impossible for You to comply with any of the terms of this -License with respect to some or all of the Covered Software due to -statute, judicial order, or regulation then You must: (a) comply with -the terms of this License to the maximum extent possible; and (b) -describe the limitations and the code they affect. Such description must -be placed in a text file included with all distributions of the Covered -Software under this License. Except to the extent prohibited by statute -or regulation, such description must be sufficiently detailed for a -recipient of ordinary skill to be able to understand it. - -5. Termination --------------- - -5.1. The rights granted under this License will terminate automatically -if You fail to comply with any of its terms. However, if You become -compliant, then the rights granted under this License from a particular -Contributor are reinstated (a) provisionally, unless and until such -Contributor explicitly and finally terminates Your grants, and (b) on an -ongoing basis, if such Contributor fails to notify You of the -non-compliance by some reasonable means prior to 60 days after You have -come back into compliance. Moreover, Your grants from a particular -Contributor are reinstated on an ongoing basis if such Contributor -notifies You of the non-compliance by some reasonable means, this is the -first time You have received notice of non-compliance with this License -from such Contributor, and You become compliant prior to 30 days after -Your receipt of the notice. - -5.2. If You initiate litigation against any entity by asserting a patent -infringement claim (excluding declaratory judgment actions, -counter-claims, and cross-claims) alleging that a Contributor Version -directly or indirectly infringes any patent, then the rights granted to -You by any and all Contributors for the Covered Software under Section -2.1 of this License shall terminate. - -5.3. In the event of termination under Sections 5.1 or 5.2 above, all -end user license agreements (excluding distributors and resellers) which -have been validly granted by You or Your distributors under this License -prior to termination shall survive termination. - -************************************************************************ -* * -* 6. Disclaimer of Warranty * -* ------------------------- * -* * -* Covered Software is provided under this License on an "as is" * -* basis, without warranty of any kind, either expressed, implied, or * -* statutory, including, without limitation, warranties that the * -* Covered Software is free of defects, merchantable, fit for a * -* particular purpose or non-infringing. The entire risk as to the * -* quality and performance of the Covered Software is with You. * -* Should any Covered Software prove defective in any respect, You * -* (not any Contributor) assume the cost of any necessary servicing, * -* repair, or correction. This disclaimer of warranty constitutes an * -* essential part of this License. No use of any Covered Software is * -* authorized under this License except under this disclaimer. * -* * -************************************************************************ - -************************************************************************ -* * -* 7. Limitation of Liability * -* -------------------------- * -* * -* Under no circumstances and under no legal theory, whether tort * -* (including negligence), contract, or otherwise, shall any * -* Contributor, or anyone who distributes Covered Software as * -* permitted above, be liable to You for any direct, indirect, * -* special, incidental, or consequential damages of any character * -* including, without limitation, damages for lost profits, loss of * -* goodwill, work stoppage, computer failure or malfunction, or any * -* and all other commercial damages or losses, even if such party * -* shall have been informed of the possibility of such damages. This * -* limitation of liability shall not apply to liability for death or * -* personal injury resulting from such party's negligence to the * -* extent applicable law prohibits such limitation. Some * -* jurisdictions do not allow the exclusion or limitation of * -* incidental or consequential damages, so this exclusion and * -* limitation may not apply to You. * -* * -************************************************************************ - -8. Litigation -------------- - -Any litigation relating to this License may be brought only in the -courts of a jurisdiction where the defendant maintains its principal -place of business and such litigation shall be governed by laws of that -jurisdiction, without reference to its conflict-of-law provisions. -Nothing in this Section shall prevent a party's ability to bring -cross-claims or counter-claims. - -9. Miscellaneous ----------------- - -This License represents the complete agreement concerning the subject -matter hereof. If any provision of this License is held to be -unenforceable, such provision shall be reformed only to the extent -necessary to make it enforceable. Any law or regulation which provides -that the language of a contract shall be construed against the drafter -shall not be used to construe this License against a Contributor. - -10. Versions of the License ---------------------------- - -10.1. New Versions - -Mozilla Foundation is the license steward. Except as provided in Section -10.3, no one other than the license steward has the right to modify or -publish new versions of this License. Each version will be given a -distinguishing version number. - -10.2. Effect of New Versions - -You may distribute the Covered Software under the terms of the version -of the License under which You originally received the Covered Software, -or under the terms of any subsequent version published by the license -steward. - -10.3. Modified Versions - -If you create software not governed by this License, and you want to -create a new license for such software, you may create and use a -modified version of this License if you rename the license and remove -any references to the name of the license steward (except to note that -such modified license differs from this License). - -10.4. Distributing Source Code Form that is Incompatible With Secondary -Licenses - -If You choose to distribute Source Code Form that is Incompatible With -Secondary Licenses under the terms of this version of the License, the -notice described in Exhibit B of this License must be attached. - -Exhibit A - Source Code Form License Notice -------------------------------------------- - - 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 http://mozilla.org/MPL/2.0/. - -If it is not possible or desirable to put the notice in a particular -file, then You may include the notice in a location (such as a LICENSE -file in a relevant directory) where a recipient would be likely to look -for such a notice. - -You may add additional accurate notices of copyright ownership. - -Exhibit B - "Incompatible With Secondary Licenses" Notice ---------------------------------------------------------- - - This Source Code Form is "Incompatible With Secondary Licenses", as - defined by the Mozilla Public License, v. 2.0. diff --git a/deps/rabbitmq_federation/Makefile b/deps/rabbitmq_federation/Makefile index 13d055c45d52..bf4504bafa57 100644 --- a/deps/rabbitmq_federation/Makefile +++ b/deps/rabbitmq_federation/Makefile @@ -1,25 +1,11 @@ PROJECT = rabbitmq_federation -PROJECT_DESCRIPTION = RabbitMQ Federation -PROJECT_MOD = rabbit_federation_app +PROJECT_DESCRIPTION = Deprecated no-op RabbitMQ Federation -define PROJECT_ENV -[ - {pgroup_name_cluster_id, false}, - {internal_exchange_check_interval, 90000} - ] -endef +LOCAL_DEPS = rabbit -define PROJECT_APP_EXTRA_KEYS - {broker_version_requirements, []} -endef +DEP_PLUGINS = rabbit_common/mk/rabbitmq-build.mk -DEPS = rabbit_common rabbit amqp_client -TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers - -PLT_APPS += rabbitmq_cli - -DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-plugin.mk -DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk +.DEFAULT_GOAL = all include ../../rabbitmq-components.mk include ../../erlang.mk diff --git a/deps/rabbitmq_federation/README-hacking b/deps/rabbitmq_federation/README-hacking deleted file mode 100644 index 6432552fe33a..000000000000 --- a/deps/rabbitmq_federation/README-hacking +++ /dev/null @@ -1,143 +0,0 @@ -This file is intended to tell you How It All Works, concentrating on -the things you might not expect. - -The theory -========== - -The 'x-federation' exchange is defined in -rabbit_federation_exchange. This starts up a bunch of link processes -(one for each upstream) which: - - * Connect to the upstream broker - * Create a queue and bind it to the upstream exchange - * Keep bindings in sync with the downstream exchange - * Consume messages from the upstream queue and republish them to the - downstream exchange (matching confirms with acks) - -Each link process monitors the connections / channels it opens, and -dies if they do. We use a supervisor2 to ensure that we get some -backoff when restarting. - -We use process groups to identify all link processes for a certain -exchange, as well as all link processes together. - -However, there are a bunch of wrinkles: - - -Wrinkle: The exchange will be recovered when the Erlang client is not available -=============================================================================== - -Exchange recovery happens within the rabbit application - therefore at -the time that the exchange is recovered, we can't make any connections -since the amqp_client application has not yet started. Each link -therefore initially has a state 'not_started'. When it is created it -checks to see if the rabbitmq_federation application is running. If -so, it starts fully. If not, it goes into the 'not_started' -state. When rabbitmq_federation starts, it sends a 'go' message to all -links, prodding them to bring up the link. - - -Wrinkle: On reconnect we want to assert bindings atomically -=========================================================== - -If the link goes down for whatever reason, then by the time it comes -up again the bindings downstream may no longer be in sync with those -upstream. Therefore on link establishment we want to ensure that a -certain set of bindings exists. (Of course bringing up a link for the -first time is a simple case of this.) And we want to do this with AMQP -methods. But if we were to tear down all bindings and recreate them, -we would have a time period when messages would not be forwarded for -bindings that *do* still exist before and after. - -We use exchange to exchange bindings to work around this: - -We bind the upstream exchange (X) to the upstream queue (Q) via an -internal fanout exchange (IXA) like so: (routing keys R1 and R2): - - X----R1,R2--->IXA---->Q - -This has the same effect as binding the queue to the exchange directly. - -Now imagine the link has gone down, and is about to be -reestablished. In the meanwhile, routing has changed downstream so -that we now want routing keys R1 and R3. On link reconnection we can -create and bind another internal fanout exchange IXB: - - X----R1,R2--->IXA---->Q - | ^ - | | - \----R1,R3--->IXB-----/ - -and then delete the original exchange IXA: - - X Q - | ^ - | | - \----R1,R3--->IXB-----/ - -This means that messages matching R1 are always routed during the -switchover. Messages for R3 will start being routed as soon as we bind -the second exchange, and messages for R2 will be stopped in a timely -way. Of course this could lag the downstream situation somewhat, in -which case some R2 messages will get thrown away downstream since they -are unroutable. However this lag is inevitable when the link goes -down. - -This means that the downstream only needs to keep track of whether the -upstream is currently going via internal exchange A or B. This is -held in the exchange scratch space in Mnesia. - - -Wrinkle: We need to amalgamate bindings -======================================= - -Since we only bind to one exchange upstream, but the downstream -exchange can be bound to many queues, we can have duplicated bindings -downstream (same source, routing key and args but different -destination) that cannot be duplicated upstream (since the destination -is the same). The link therefore maintains a mapping of (Key, Args) to -set(Dest). Duplicated bindings do not get repeated upstream, and are -only unbound upstream when the last one goes away downstream. - -Furthermore, this works as an optimisation since this will tend to -reduce upstream binding count and churn. - - -Wrinkle: We may receive binding events out of order -=================================================== - -The rabbit_federation_exchange callbacks are invoked by channel -processes within rabbit. Therefore they can be executed concurrently, -and can arrive at the link processes in an order that does not -correspond to the wall clock. - -We need to keep the state of the link in sync with Mnesia. Therefore -not only do we need to impose an ordering on these events, we need to -impose Mnesia's ordering on them. We therefore added a function to the -callback interface, serialise_events. When this returns true, the -callback mechanism inside rabbit increments a per-exchange counter -within an Mnesia transaction, and returns the value as part of the -add_binding and remove_binding callbacks. The link process then queues -up these events, and replays them in order. The link process's state -thus always follows Mnesia (it may be delayed, but the effects happen -in the same order). - - -Other issues -============ - -Since links are implemented in terms of AMQP, link failure may cause -messages to be redelivered. If you're unlucky this could lead to -duplication. - -Message duplication can also happen with some topologies. In some -cases it may not be possible to set max_hops such that messages arrive -once at every node. - -While we correctly order bind / unbind events, we don't do the same -thing for exchange creation / deletion. (This is harder - if you -delete and recreate an exchange with the same name, is it the same -exchange? What about if its type changes?) This would only be an issue -if exchanges churn rapidly; however we could get into a state where -Mnesia sees CDCD but we see CDDC and leave a process running when we -shouldn't. diff --git a/deps/rabbitmq_federation/README.md b/deps/rabbitmq_federation/README.md index d96c13a02e57..86a5e4bc1fbd 100644 --- a/deps/rabbitmq_federation/README.md +++ b/deps/rabbitmq_federation/README.md @@ -1,23 +1,6 @@ -## RabbitMQ Federation +This no-op plugin exists only such that deployment tools can continue to enable and disable this plugin without erroring: -RabbitMQ federation offers a group of features for loosely -coupled and WAN-friendly distributed RabbitMQ setups. Note that -this is not an alternative to queue mirroring. - - -## Supported RabbitMQ Versions - -This plugin ships with RabbitMQ, there is no need to -install it separately. - - -## Documentation - -See [RabbitMQ federation plugin](https://www.rabbitmq.com/federation.html) on rabbitmq.com. - - -## License and Copyright - -Released under [the same license as RabbitMQ](https://www.rabbitmq.com/mpl.html). - -2007-2015 (c) 2007-2024 Broadcom. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. +``` +rabbitmq-plugins enable rabbitmq_federation +rabbitmq-plugins disable rabbitmq_federation +``` diff --git a/deps/rabbitmq_federation/include/logging.hrl b/deps/rabbitmq_federation/include/logging.hrl deleted file mode 100644 index 019713e11b45..000000000000 --- a/deps/rabbitmq_federation/include/logging.hrl +++ /dev/null @@ -1,3 +0,0 @@ --include_lib("rabbit_common/include/logging.hrl"). - --define(RMQLOG_DOMAIN_FEDERATION, ?DEFINE_RMQLOG_DOMAIN(federation)). diff --git a/deps/rabbitmq_federation/include/rabbit_federation.hrl b/deps/rabbitmq_federation/include/rabbit_federation.hrl deleted file mode 100644 index e5be82ef4969..000000000000 --- a/deps/rabbitmq_federation/include/rabbit_federation.hrl +++ /dev/null @@ -1,48 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --record(upstream, {uris, - exchange_name, - queue_name, - consumer_tag, - prefetch_count, - max_hops, - reconnect_delay, - expires, - message_ttl, - trust_user_id, - ack_mode, - queue_type, - name, - bind_nowait, - resource_cleanup_mode, - channel_use_mode - }). - --record(upstream_params, - {uri, - params, - x_or_q, - %% The next two can be derived from the above three, but we don't - %% want to do that every time we forward a message. - safe_uri, - table}). - -%% Name of the message header used to collect the hop (forwarding) path -%% metadata as the message is forwarded by exchange federation. --define(ROUTING_HEADER, <<"x-received-from">>). --define(BINDING_HEADER, <<"x-bound-from">>). --define(MAX_HOPS_ARG, <<"x-max-hops">>). -%% Identifies a cluster, used by exchange federation cycle detection --define(DOWNSTREAM_NAME_ARG, <<"x-downstream-name">>). -%% Identifies a virtual host, used by exchange federation cycle detection --define(DOWNSTREAM_VHOST_ARG, <<"x-downstream-vhost">>). --define(DEF_PREFETCH, 1000). - --define(FEDERATION_GUIDE_URL, <<"https://rabbitmq.com/docs/federation/">>). - --define(FEDERATION_PG_SCOPE, rabbitmq_federation_pg_scope). diff --git a/deps/rabbitmq_federation/src/Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand.erl b/deps/rabbitmq_federation/src/Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand.erl deleted file mode 100644 index aa4794aace7c..000000000000 --- a/deps/rabbitmq_federation/src/Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand.erl +++ /dev/null @@ -1,117 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module('Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand'). - --include("rabbit_federation.hrl"). - --behaviour('Elixir.RabbitMQ.CLI.CommandBehaviour'). - --export([ - usage/0, - usage_additional/0, - usage_doc_guides/0, - flags/0, - validate/2, - merge_defaults/2, - banner/2, - run/2, - switches/0, - aliases/0, - output/2, - scopes/0, - formatter/0, - help_section/0, - description/0 - ]). - - -%%---------------------------------------------------------------------------- -%% Callbacks -%%---------------------------------------------------------------------------- -usage() -> - <<"federation_status [--only-down]">>. - -usage_additional() -> - [ - {<<"--only-down">>, <<"only display links that failed or are not currently connected">>} - ]. - -usage_doc_guides() -> - [?FEDERATION_GUIDE_URL]. - -help_section() -> - {plugin, federation}. - -description() -> - <<"Displays federation link status">>. - -flags() -> - []. - -validate(_,_) -> - ok. - -formatter() -> - 'Elixir.RabbitMQ.CLI.Formatters.Erlang'. - -merge_defaults(A, Opts) -> - {A, maps:merge(#{only_down => false}, Opts)}. - -banner(_, #{node := Node, only_down := true}) -> - erlang:iolist_to_binary([<<"Listing federation links which are down on node ">>, - atom_to_binary(Node, utf8), <<"...">>]); -banner(_, #{node := Node, only_down := false}) -> - erlang:iolist_to_binary([<<"Listing federation links on node ">>, - atom_to_binary(Node, utf8), <<"...">>]). - -run(_Args, #{node := Node, only_down := OnlyDown}) -> - case rabbit_misc:rpc_call(Node, rabbit_federation_status, status, []) of - {badrpc, _} = Error -> - Error; - Status -> - {stream, filter(Status, OnlyDown)} - end. - -switches() -> - [{only_down, boolean}]. - -aliases() -> - []. - -output({stream, FederationStatus}, _) -> - Formatted = [begin - Timestamp = proplists:get_value(timestamp, St), - Map0 = maps:remove(timestamp, maps:from_list(St)), - Map1 = maps:merge(#{queue => <<>>, - exchange => <<>>, - upstream_queue => <<>>, - upstream_exchange => <<>>, - local_connection => <<>>, - error => <<>>}, Map0), - Map1#{last_changed => fmt_ts(Timestamp)} - end || St <- FederationStatus], - {stream, Formatted}; -output(E, _Opts) -> - 'Elixir.RabbitMQ.CLI.DefaultOutput':output(E). - -scopes() -> - ['ctl', 'diagnostics']. - -%%---------------------------------------------------------------------------- -%% Formatting -%%---------------------------------------------------------------------------- -fmt_ts({{YY, MM, DD}, {Hour, Min, Sec}}) -> - erlang:list_to_binary( - io_lib:format("~4..0w-~2..0w-~2..0w ~2..0w:~2..0w:~2..0w", - [YY, MM, DD, Hour, Min, Sec])). - -filter(Status, _OnlyDown = false) -> - Status; -filter(Status, _OnlyDown = true) -> - [St || St <- Status, - not lists:member(proplists:get_value(status, St), [running, starting])]. diff --git a/deps/rabbitmq_federation/src/Elixir.RabbitMQ.CLI.Ctl.Commands.RestartFederationLinkCommand.erl b/deps/rabbitmq_federation/src/Elixir.RabbitMQ.CLI.Ctl.Commands.RestartFederationLinkCommand.erl deleted file mode 100644 index b26804ee5012..000000000000 --- a/deps/rabbitmq_federation/src/Elixir.RabbitMQ.CLI.Ctl.Commands.RestartFederationLinkCommand.erl +++ /dev/null @@ -1,84 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module('Elixir.RabbitMQ.CLI.Ctl.Commands.RestartFederationLinkCommand'). - --include("rabbit_federation.hrl"). - --behaviour('Elixir.RabbitMQ.CLI.CommandBehaviour'). - --export([ - usage/0, - usage_additional/0, - usage_doc_guides/0, - flags/0, - validate/2, - merge_defaults/2, - banner/2, - run/2, - aliases/0, - output/2, - help_section/0, - description/0 - ]). - - -%%---------------------------------------------------------------------------- -%% Callbacks -%%---------------------------------------------------------------------------- -usage() -> - <<"restart_federation_link ">>. - -usage_additional() -> - [ - {<<"">>, <<"ID of the link to restart">>} - ]. - -usage_doc_guides() -> - [?FEDERATION_GUIDE_URL]. - -help_section() -> - {plugin, federation}. - -description() -> - <<"Restarts a running federation link">>. - -flags() -> - []. - -validate([], _Opts) -> - {validation_failure, not_enough_args}; -validate([_, _ | _], _Opts) -> - {validation_failure, too_many_args}; -validate([_], _) -> - ok. - -merge_defaults(A, O) -> - {A, O}. - -banner([Link], #{node := Node}) -> - erlang:iolist_to_binary([<<"Restarting federation link ">>, Link, << " on node ">>, - atom_to_binary(Node, utf8)]). - -run([Id], #{node := Node}) -> - case rabbit_misc:rpc_call(Node, rabbit_federation_status, lookup, [Id]) of - {badrpc, _} = Error -> - Error; - not_found -> - {error, <<"Link with the given ID was not found">>}; - Obj -> - Upstream = proplists:get_value(upstream, Obj), - Supervisor = proplists:get_value(supervisor, Obj), - rabbit_misc:rpc_call(Node, rabbit_federation_link_sup, restart, - [Supervisor, Upstream]) - end. - -aliases() -> - []. - -output(Output, _Opts) -> - 'Elixir.RabbitMQ.CLI.DefaultOutput':output(Output). diff --git a/deps/rabbitmq_federation/src/rabbit_federation_app.erl b/deps/rabbitmq_federation/src/rabbit_federation_app.erl deleted file mode 100644 index e3115066ce1b..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_app.erl +++ /dev/null @@ -1,44 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_app). - --behaviour(application). --export([start/2, stop/1]). - -%% Dummy supervisor - see Ulf Wiger's comment at -%% http://erlang.org/pipermail/erlang-questions/2010-April/050508.html - -%% All of our actual server processes are supervised by -%% rabbit_federation_sup, which is started by a rabbit_boot_step -%% (since it needs to start up before queue / exchange recovery, so it -%% can't be part of our application). -%% -%% However, we still need an application behaviour since we need to -%% know when our application has started since then the Erlang client -%% will have started and we can therefore start our links going. Since -%% the application behaviour needs a tree of processes to supervise, -%% this is it... --behaviour(supervisor). --export([init/1]). - -start(_Type, _StartArgs) -> - supervisor:start_link({local, ?MODULE}, ?MODULE, []). - -stop(_State) -> - rabbit_federation_pg:stop_scope(), - ok. - -%%---------------------------------------------------------------------------- - -init([]) -> - Flags = #{ - strategy => one_for_one, - intensity => 3, - period => 10 - }, - {ok, {Flags, []}}. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_db.erl b/deps/rabbitmq_federation/src/rabbit_federation_db.erl deleted file mode 100644 index a02cea4ba1d3..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_db.erl +++ /dev/null @@ -1,45 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_db). - --include("rabbit_federation.hrl"). --define(DICT, orddict). - --export([get_active_suffix/3, set_active_suffix/3, prune_scratch/2]). - -%%---------------------------------------------------------------------------- - -get_active_suffix(XName, Upstream, Default) -> - case rabbit_exchange:lookup_scratch(XName, federation) of - {ok, Dict} -> - case ?DICT:find(key(Upstream), Dict) of - {ok, Suffix} -> Suffix; - error -> Default - end; - {error, not_found} -> - Default - end. - -set_active_suffix(XName, Upstream, Suffix) -> - ok = rabbit_exchange:update_scratch( - XName, federation, - fun(D) -> ?DICT:store(key(Upstream), Suffix, ensure(D)) end). - -prune_scratch(XName, Upstreams) -> - ok = rabbit_exchange:update_scratch( - XName, federation, - fun(D) -> Keys = [key(U) || U <- Upstreams], - ?DICT:filter( - fun(K, _V) -> lists:member(K, Keys) end, ensure(D)) - end). - -key(#upstream{name = UpstreamName, exchange_name = XNameBin}) -> - {UpstreamName, XNameBin}. - -ensure(undefined) -> ?DICT:new(); -ensure(D) -> D. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_event.erl b/deps/rabbitmq_federation/src/rabbit_federation_event.erl deleted file mode 100644 index aae9b3f2ed99..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_event.erl +++ /dev/null @@ -1,54 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_event). --behaviour(gen_event). - --include_lib("rabbit_common/include/rabbit.hrl"). - --export([add_handler/0, remove_handler/0]). - --export([init/1, handle_call/2, handle_event/2, handle_info/2, - terminate/2, code_change/3]). - --import(rabbit_misc, [pget/2]). - -%%---------------------------------------------------------------------------- - -add_handler() -> - gen_event:add_handler(rabbit_event, ?MODULE, []). - -remove_handler() -> - gen_event:delete_handler(rabbit_event, ?MODULE, []). - -init([]) -> - {ok, []}. - -handle_call(_Request, State) -> - {ok, not_understood, State}. - -handle_event(#event{type = parameter_set, - props = Props0}, State) -> - Props = rabbit_data_coercion:to_list(Props0), - case {pget(component, Props), pget(name, Props)} of - {global, cluster_name} -> - rabbit_federation_parameters:adjust(everything); - _ -> - ok - end, - {ok, State}; -handle_event(_Event, State) -> - {ok, State}. - -handle_info(_Info, State) -> - {ok, State}. - -terminate(_Arg, _State) -> - ok. - -code_change(_OldVsn, State, _Extra) -> - {ok, State}. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_exchange.erl b/deps/rabbitmq_federation/src/rabbit_federation_exchange.erl deleted file mode 100644 index 52931042ae69..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_exchange.erl +++ /dev/null @@ -1,96 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - -%% TODO rename this --module(rabbit_federation_exchange). - --include_lib("amqp_client/include/amqp_client.hrl"). - --rabbit_boot_step({?MODULE, - [{description, "federation exchange decorator"}, - {mfa, {rabbit_registry, register, - [exchange_decorator, <<"federation">>, ?MODULE]}}, - {cleanup, {rabbit_registry, unregister, - [exchange_decorator, <<"federation">>]}}, - {requires, [rabbit_registry, recovery]}]}). - --behaviour(rabbit_exchange_decorator). - --export([description/0, serialise_events/1]). --export([create/2, delete/2, policy_changed/2, - add_binding/3, remove_bindings/3, route/2, active_for/1]). - -%%---------------------------------------------------------------------------- - -description() -> - [{description, <<"Federation exchange decorator">>}]. - -serialise_events(X) -> federate(X). - -create(_Serial, X) -> - maybe_start(X). - -delete(_Serial, X) -> - maybe_stop(X). - -policy_changed(OldX, NewX) -> - maybe_stop(OldX), - maybe_start(NewX). - -add_binding(Serial, X = #exchange{name = XName}, B) -> - case federate(X) of - true -> _ = rabbit_federation_exchange_link:add_binding(Serial, XName, B), - ok; - false -> ok - end. - -remove_bindings(Serial, X = #exchange{name = XName}, Bs) -> - case federate(X) of - true -> _ = rabbit_federation_exchange_link:remove_bindings(Serial, XName, Bs), - ok; - false -> ok - end. - -route(_, _) -> []. - -active_for(X) -> - case federate(X) of - true -> noroute; - false -> none - end. - -%%---------------------------------------------------------------------------- - -%% Don't federate default exchange, we can't bind to it -federate(#exchange{name = #resource{name = <<"">>}}) -> - false; - -%% Don't federate any of our intermediate exchanges. Note that we use -%% internal=true since older brokers may not declare -%% x-federation-upstream on us. Also other internal exchanges should -%% probably not be federated. -federate(#exchange{internal = true}) -> - false; - -federate(X) -> - rabbit_federation_upstream:federate(X). - -maybe_start(X = #exchange{name = XName})-> - case federate(X) of - true -> ok = rabbit_federation_db:prune_scratch( - XName, rabbit_federation_upstream:for(X)), - ok = rabbit_federation_exchange_link_sup_sup:start_child(X), - ok; - false -> ok - end. - -maybe_stop(X = #exchange{name = XName}) -> - case federate(X) of - true -> ok = rabbit_federation_exchange_link_sup_sup:stop_child(X), - rabbit_federation_status:remove_exchange_or_queue(XName); - false -> ok - end. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_exchange_link.erl b/deps/rabbitmq_federation/src/rabbit_federation_exchange_link.erl deleted file mode 100644 index 3509a7b2fd89..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_exchange_link.erl +++ /dev/null @@ -1,695 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_exchange_link). - --include_lib("amqp_client/include/amqp_client.hrl"). --include("rabbit_federation.hrl"). - --behaviour(gen_server2). - --export([go/0, add_binding/3, remove_bindings/3]). --export([list_routing_keys/1]). %% For testing - --export([start_link/1]). - --export([init/1, handle_call/3, handle_cast/2, handle_info/2, - terminate/2, code_change/3]). - --import(rabbit_misc, [pget/2]). --import(rabbit_federation_util, [name/1, vhost/1, pgname/1]). - --record(state, {upstream, - upstream_params, - upstream_name, - connection, - channel, - cmd_channel, - consumer_tag, - queue, - internal_exchange, - waiting_cmds = gb_trees:empty(), - next_serial, - bindings = #{}, - downstream_connection, - downstream_channel, - downstream_exchange, - unacked, - internal_exchange_timer, - internal_exchange_interval}). - -%%---------------------------------------------------------------------------- - -%% We start off in a state where we do not connect, since we can first -%% start during exchange recovery, when rabbit is not fully started -%% and the Erlang client is not running. This then gets invoked when -%% the federation app is started. -go() -> - _ = rabbit_federation_pg:start_scope(), - cast(go). - -add_binding(S, XN, B) -> cast(XN, {enqueue, S, {add_binding, B}}). -remove_bindings(S, XN, Bs) -> cast(XN, {enqueue, S, {remove_bindings, Bs}}). - -list_routing_keys(XN) -> call(XN, list_routing_keys). - -%%---------------------------------------------------------------------------- - -start_link(Args) -> - gen_server2:start_link(?MODULE, Args, [{timeout, infinity}]). - -init({Upstream, XName}) -> - %% If we are starting up due to a policy change then it's possible - %% for the exchange to have been deleted before we got here, in which - %% case it's possible that delete callback would also have been called - %% before we got here. So check if we still exist. - case rabbit_exchange:lookup(XName) of - {ok, X} -> - DeobfuscatedUpstream = rabbit_federation_util:deobfuscate_upstream(Upstream), - DeobfuscatedUParams = rabbit_federation_upstream:to_params(DeobfuscatedUpstream, X), - UParams = rabbit_federation_util:obfuscate_upstream_params(DeobfuscatedUParams), - rabbit_federation_status:report(Upstream, UParams, XName, starting), - join(rabbit_federation_exchanges), - join({rabbit_federation_exchange, XName}), - gen_server2:cast(self(), maybe_go), - {ok, {not_started, {Upstream, UParams, XName}}}; - {error, not_found} -> - rabbit_federation_link_util:log_warning(XName, "not found, stopping link", []), - {stop, gone} - end. - -handle_call(list_routing_keys, _From, State = #state{bindings = Bindings}) -> - {reply, lists:sort([K || {K, _} <- maps:keys(Bindings)]), State}; - -handle_call(Msg, _From, State) -> - {stop, {unexpected_call, Msg}, State}. - -handle_cast(maybe_go, State = {not_started, _Args}) -> - go(State); - -handle_cast(go, S0 = {not_started, _Args}) -> - go(S0); - -%% There's a small race - I think we can realise federation is up -%% before 'go' gets invoked. Ignore. -handle_cast(go, State) -> - {noreply, State}; - -handle_cast({enqueue, _, _}, State = {not_started, _}) -> - {noreply, State}; - -handle_cast({enqueue, Serial, Cmd}, - State = #state{waiting_cmds = Waiting, - downstream_exchange = XName}) -> - Waiting1 = gb_trees:insert(Serial, Cmd, Waiting), - try - {noreply, play_back_commands(State#state{waiting_cmds = Waiting1})} - catch exit:{{shutdown, {server_initiated_close, 404, Text}}, _} -> - rabbit_federation_link_util:log_warning( - XName, "detected upstream changes, restarting link: ~tp", [Text]), - {stop, {shutdown, restart}, State} - end; - -handle_cast(Msg, State) -> - {stop, {unexpected_cast, Msg}, State}. - -handle_info(#'basic.consume_ok'{}, State) -> - {noreply, State}; - -handle_info(#'basic.ack'{} = Ack, State = #state{channel = Ch, - unacked = Unacked}) -> - Unacked1 = rabbit_federation_link_util:ack(Ack, Ch, Unacked), - {noreply, State#state{unacked = Unacked1}}; - -handle_info(#'basic.nack'{} = Nack, State = #state{channel = Ch, - unacked = Unacked}) -> - Unacked1 = rabbit_federation_link_util:nack(Nack, Ch, Unacked), - {noreply, State#state{unacked = Unacked1}}; - -handle_info({#'basic.deliver'{routing_key = Key, - redelivered = Redelivered} = DeliverMethod, Msg}, - State = #state{ - upstream = Upstream = #upstream{max_hops = MaxH}, - upstream_params = UParams = #upstream_params{x_or_q = UpstreamX}, - upstream_name = UName, - downstream_exchange = #resource{name = XNameBin, virtual_host = DVhost}, - downstream_channel = DCh, - channel = Ch, - unacked = Unacked}) -> - UVhost = vhost(UpstreamX), - PublishMethod = #'basic.publish'{exchange = XNameBin, - routing_key = Key}, - HeadersFun = fun (H) -> update_routing_headers(UParams, UName, UVhost, Redelivered, H) end, - %% We need to check should_forward/2 here in case the upstream - %% does not have federation and thus is using a fanout exchange. - ForwardFun = fun (H) -> - DName = rabbit_nodes:cluster_name(), - rabbit_federation_util:should_forward(H, MaxH, DName, DVhost) - end, - Unacked1 = rabbit_federation_link_util:forward( - Upstream, DeliverMethod, Ch, DCh, PublishMethod, - HeadersFun, ForwardFun, Msg, Unacked), - {noreply, State#state{unacked = Unacked1}}; - -handle_info(#'basic.cancel'{}, State = #state{upstream = Upstream, - upstream_params = UParams, - downstream_exchange = XName}) -> - rabbit_federation_link_util:connection_error( - local, basic_cancel, Upstream, UParams, XName, State); - -handle_info({'DOWN', _Ref, process, Pid, Reason}, - State = #state{downstream_channel = DCh, - channel = Ch, - cmd_channel = CmdCh, - upstream = Upstream, - upstream_params = UParams, - downstream_exchange = XName}) -> - handle_down(Pid, Reason, Ch, CmdCh, DCh, - {Upstream, UParams, XName}, State); - -handle_info(check_internal_exchange, State = #state{internal_exchange = IntXNameBin, - internal_exchange_interval = Interval}) -> - case check_internal_exchange(IntXNameBin, State) of - upstream_not_found -> - rabbit_log_federation:warning("Federation link could not find upstream exchange '~ts' and will restart", - [IntXNameBin]), - {stop, {shutdown, restart}, State}; - _ -> - TRef = erlang:send_after(Interval, self(), check_internal_exchange), - {noreply, State#state{internal_exchange_timer = TRef}} - end; - -handle_info(Msg, State) -> - {stop, {unexpected_info, Msg}, State}. - -terminate(_Reason, {not_started, _}) -> - ok; -terminate(Reason, #state{downstream_connection = DConn, - connection = Conn, - upstream = Upstream, - upstream_params = UParams, - downstream_exchange = XName, - internal_exchange_timer = TRef, - internal_exchange = IntExchange, - queue = Queue}) when Reason =:= shutdown; - Reason =:= {shutdown, restart}; - Reason =:= gone -> - _ = timer:cancel(TRef), - rabbit_federation_link_util:ensure_connection_closed(DConn), - - rabbit_log:debug("Exchange federation: link is shutting down, resource cleanup mode: ~tp", [Upstream#upstream.resource_cleanup_mode]), - case Upstream#upstream.resource_cleanup_mode of - never -> ok; - _ -> - %% This is a normal shutdown and we are allowed to clean up the internally used queue and exchange - rabbit_log:debug("Federated exchange '~ts' link will delete its internal queue '~ts'", [Upstream#upstream.exchange_name, Queue]), - delete_upstream_queue(Conn, Queue), - rabbit_log:debug("Federated exchange '~ts' link will delete its upstream exchange", [Upstream#upstream.exchange_name]), - delete_upstream_exchange(Conn, IntExchange) - end, - - rabbit_federation_link_util:ensure_connection_closed(Conn), - rabbit_federation_link_util:log_terminate(Reason, Upstream, UParams, XName), - ok; -%% unexpected shutdown -terminate(Reason, #state{downstream_connection = DConn, - connection = Conn, - upstream = Upstream, - upstream_params = UParams, - downstream_exchange = XName, - internal_exchange_timer = TRef}) -> - _ = timer:cancel(TRef), - - rabbit_federation_link_util:ensure_connection_closed(DConn), - - %% unlike in the clean shutdown case above, we keep the queue - %% and exchange around - - rabbit_federation_link_util:ensure_connection_closed(Conn), - rabbit_federation_link_util:log_terminate(Reason, Upstream, UParams, XName), - ok. - -code_change(_OldVsn, State, _Extra) -> - {ok, State}. - -%%---------------------------------------------------------------------------- - -call(XName, Msg) -> [gen_server2:call(Pid, Msg, infinity) || Pid <- x(XName)]. -cast(Msg) -> [gen_server2:cast(Pid, Msg) || Pid <- all()]. -cast(XName, Msg) -> [gen_server2:cast(Pid, Msg) || Pid <- x(XName)]. - -join(Name) -> - ok = pg:join(?FEDERATION_PG_SCOPE, pgname(Name), self()). - -all() -> - pg:get_members(?FEDERATION_PG_SCOPE, pgname(rabbit_federation_exchanges)). - -x(XName) -> - pg:get_members(?FEDERATION_PG_SCOPE, pgname({rabbit_federation_exchange, XName})). - -%%---------------------------------------------------------------------------- - -handle_command({add_binding, Binding}, State) -> - add_binding(Binding, State); - -handle_command({remove_bindings, Bindings}, State) -> - lists:foldl(fun remove_binding/2, State, Bindings). - -play_back_commands(State = #state{waiting_cmds = Waiting, - next_serial = Next}) -> - case gb_trees:is_empty(Waiting) of - false -> case gb_trees:take_smallest(Waiting) of - {Next, Cmd, Waiting1} -> - %% The next one. Just execute it. - play_back_commands( - handle_command(Cmd, State#state{ - waiting_cmds = Waiting1, - next_serial = Next + 1})); - {Serial, _Cmd, Waiting1} when Serial < Next -> - %% This command came from before we executed - %% binding:list_for_source. Ignore it. - play_back_commands(State#state{ - waiting_cmds = Waiting1}); - _ -> - %% Some future command. Don't do anything. - State - end; - true -> State - end. - -add_binding(B, State) -> - binding_op(fun record_binding/2, bind_cmd(bind, B, State), B, State). - -remove_binding(B, State) -> - binding_op(fun forget_binding/2, bind_cmd(unbind, B, State), B, State). - -record_binding(B = #binding{destination = Dest}, - State = #state{bindings = Bs}) -> - {DoIt, Set} = case maps:find(key(B), Bs) of - error -> {true, sets:from_list([Dest])}; - {ok, Dests} -> {false, sets:add_element( - Dest, Dests)} - end, - {DoIt, State#state{bindings = maps:put(key(B), Set, Bs)}}. - -forget_binding(B = #binding{destination = Dest}, - State = #state{bindings = Bs}) -> - Dests = sets:del_element(Dest, maps:get(key(B), Bs)), - {DoIt, Bs1} = case sets:size(Dests) of - 0 -> {true, maps:remove(key(B), Bs)}; - _ -> {false, maps:put(key(B), Dests, Bs)} - end, - {DoIt, State#state{bindings = Bs1}}. - -binding_op(UpdateFun, Cmd, B = #binding{args = Args}, - State = #state{cmd_channel = Ch}) -> - {DoIt, State1} = - case rabbit_misc:table_lookup(Args, ?BINDING_HEADER) of - undefined -> UpdateFun(B, State); - {array, _} -> {Cmd =/= ignore, State} - end, - case DoIt of - true -> amqp_channel:call(Ch, Cmd); - false -> ok - end, - State1. - -bind_cmd(Type, #binding{key = Key, args = Args}, - State = #state{internal_exchange = IntXNameBin, - upstream_params = UpstreamParams, - upstream = Upstream}) -> - #upstream_params{x_or_q = X} = UpstreamParams, - #upstream{bind_nowait = Nowait} = Upstream, - case update_binding(Args, State) of - ignore -> ignore; - NewArgs -> bind_cmd0(Type, name(X), IntXNameBin, Key, NewArgs, Nowait) - end. - -bind_cmd0(bind, Source, Destination, RoutingKey, Arguments, Nowait) -> - #'exchange.bind'{source = Source, - destination = Destination, - routing_key = RoutingKey, - arguments = Arguments, - nowait = Nowait}; - -bind_cmd0(unbind, Source, Destination, RoutingKey, Arguments, Nowait) -> - #'exchange.unbind'{source = Source, - destination = Destination, - routing_key = RoutingKey, - arguments = Arguments, - nowait = Nowait}. - -%% This function adds information about the current node to the -%% binding arguments, or returns 'ignore' if it determines the binding -%% should propagate no further. The interesting part is the latter. -%% -%% We want bindings to propagate in the same way as messages -%% w.r.t. max_hops - if we determine that a message can get from node -%% A to B (assuming bindings are in place) then it follows that a -%% binding at B should propagate back to A, and no further. There is -%% no point in propagating bindings past the point where messages -%% would propagate, and we will lose messages if bindings don't -%% propagate as far. -%% -%% Note that we still want to have limits on how far messages can -%% propagate: limiting our bindings is not enough, since other -%% bindings from other nodes can overlap. -%% -%% So in short we want bindings to obey max_hops. However, they can't -%% just obey the max_hops of the current link, since they are -%% travelling in the opposite direction to messages! Consider the -%% following federation: -%% -%% A -----------> B -----------> C -%% max_hops=1 max_hops=2 -%% -%% where the arrows indicate message flow. A binding created at C -%% should propagate to B, then to A, and no further. Therefore every -%% time we traverse a link, we keep a count of the number of hops that -%% a message could have made so far to reach this point, and still be -%% able to propagate. When this number ("hops" below) reaches 0 we -%% propagate no further. -%% -%% hops(link(N)) is given by: -%% -%% min(hops(link(N-1))-1, max_hops(link(N))) -%% -%% where link(N) is the link that bindings propagate over after N -%% steps (e.g. link(1) is CB above, link(2) is BA). -%% -%% In other words, we count down to 0 from the link with the most -%% restrictive max_hops we have yet passed through. - -update_binding(Args, #state{downstream_exchange = X, - upstream = Upstream, - upstream_params = #upstream_params{x_or_q = UpstreamX}, - upstream_name = UName}) -> - #upstream{max_hops = MaxHops} = Upstream, - UVhost = vhost(UpstreamX), - Hops = case rabbit_misc:table_lookup(Args, ?BINDING_HEADER) of - undefined -> MaxHops; - {array, All} -> [{table, Prev} | _] = All, - PrevHops = get_hops(Prev), - case rabbit_federation_util:already_seen( - UName, UVhost, All) of - true -> 0; - false -> lists:min([PrevHops - 1, MaxHops]) - end - end, - case Hops of - 0 -> ignore; - _ -> Cluster = rabbit_nodes:cluster_name(), - ABSuffix = rabbit_federation_db:get_active_suffix( - X, Upstream, <<"A">>), - DVhost = vhost(X), - DName = name(X), - Down = <>, - Info = [{<<"cluster-name">>, longstr, Cluster}, - {<<"vhost">>, longstr, DVhost}, - {<<"exchange">>, longstr, Down}, - {<<"hops">>, short, Hops}], - rabbit_basic:prepend_table_header(?BINDING_HEADER, Info, Args) - end. - - - -key(#binding{key = Key, args = Args}) -> {Key, Args}. - -go(S0 = {not_started, {Upstream, UParams, DownXName}}) -> - Unacked = rabbit_federation_link_util:unacked_new(), - log_link_startup_attempt(Upstream, DownXName), - rabbit_federation_link_util:start_conn_ch( - fun (Conn, Ch, DConn, DCh) -> - {ok, CmdCh} = - case Upstream#upstream.channel_use_mode of - single -> reuse_command_channel(Ch, Upstream, DownXName); - multiple -> open_command_channel(Conn, Upstream, UParams, DownXName, S0); - _ -> open_command_channel(Conn, Upstream, UParams, DownXName, S0) - end, - erlang:monitor(process, CmdCh), - Props = pget(server_properties, - amqp_connection:info(Conn, [server_properties])), - UName = case rabbit_misc:table_lookup( - Props, <<"cluster_name">>) of - {longstr, N} -> N; - _ -> unknown - 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 - %% overtaken the creation of this process). However, this - %% is not a big deal - 'undefined' just becomes the next - %% serial we will process. Since it compares larger than - %% any number we never process any commands. And we will - %% soon get told to stop anyway. - {ok, Interval} = application:get_env(rabbitmq_federation, - internal_exchange_check_interval), - State = ensure_upstream_bindings( - consume_from_upstream_queue( - #state{upstream = Upstream, - upstream_params = UParams, - upstream_name = UName, - connection = Conn, - channel = Ch, - cmd_channel = CmdCh, - next_serial = Serial, - downstream_connection = DConn, - downstream_channel = DCh, - downstream_exchange = DownXName, - unacked = Unacked, - internal_exchange_interval = Interval}), - Bindings), - rabbit_log_federation:info("Federation link for ~ts (upstream: ~ts) will perform internal exchange checks " - "every ~b seconds", [rabbit_misc:rs(DownXName), UName, round(Interval / 1000)]), - TRef = erlang:send_after(Interval, self(), check_internal_exchange), - {noreply, State#state{internal_exchange_timer = TRef}} - end, Upstream, UParams, DownXName, S0). - -log_link_startup_attempt(#upstream{name = Name, channel_use_mode = ChMode}, DownXName) -> - rabbit_log_federation:debug("Will try to start a federation link for ~ts, upstream: '~ts', channel use mode: ~ts", - [rabbit_misc:rs(DownXName), Name, ChMode]). - -%% If channel use mode is 'single', reuse the message transfer channel. -%% Otherwise open a separate one. -reuse_command_channel(MainCh, #upstream{name = UName}, DownXName) -> - rabbit_log_federation:debug("Will use a single channel for both schema operations and message transfer on links to upstream '~ts' for downstream federated ~ts", - [UName, rabbit_misc:rs(DownXName)]), - {ok, MainCh}. - -open_command_channel(Conn, Upstream = #upstream{name = UName}, UParams, DownXName, S0) -> - rabbit_log_federation:debug("Will open a command channel to upstream '~ts' for downstream federated ~ts", - [UName, rabbit_misc:rs(DownXName)]), - case amqp_connection:open_channel(Conn) of - {ok, CCh} -> - erlang:monitor(process, CCh), - {ok, CCh}; - E -> - rabbit_federation_link_util:ensure_connection_closed(Conn), - _ = rabbit_federation_link_util:connection_error(command_channel, E, - Upstream, UParams, DownXName, S0), - E - end. - -consume_from_upstream_queue( - State = #state{upstream = Upstream, - upstream_params = UParams, - channel = Ch, - downstream_exchange = DownXName}) -> - #upstream{prefetch_count = Prefetch, - expires = Expiry, - message_ttl = TTL, - queue_type = QueueType} = Upstream, - #upstream_params{x_or_q = X, - params = Params} = UParams, - Q = upstream_queue_name(name(X), vhost(Params), DownXName), - Args = [A || {_K, _T, V} = A - <- [{<<"x-expires">>, long, Expiry}, - {<<"x-message-ttl">>, long, TTL}, - {<<"x-internal-purpose">>, longstr, <<"federation">>}, - {<<"x-queue-type">>, longstr, atom_to_binary(QueueType)} - ], - V =/= none], - amqp_channel:call(Ch, #'queue.declare'{queue = Q, - durable = true, - arguments = Args}), - NoAck = Upstream#upstream.ack_mode =:= 'no-ack', - case NoAck of - false -> amqp_channel:call(Ch, #'basic.qos'{prefetch_count = Prefetch}); - true -> ok - end, - #'basic.consume_ok'{consumer_tag = CTag} = - amqp_channel:subscribe(Ch, #'basic.consume'{queue = Q, - no_ack = NoAck}, self()), - State#state{consumer_tag = CTag, - queue = Q}. - -ensure_upstream_bindings(State = #state{upstream = Upstream, - connection = Conn, - channel = Ch, - downstream_exchange = DownXName, - queue = Q}, Bindings) -> - OldSuffix = rabbit_federation_db:get_active_suffix( - DownXName, Upstream, <<"A">>), - Suffix = case OldSuffix of - <<"A">> -> <<"B">>; - <<"B">> -> <<"A">> - end, - IntXNameBin = upstream_exchange_name(Q, Suffix), - ensure_upstream_exchange(State), - ensure_internal_exchange(IntXNameBin, State), - amqp_channel:call(Ch, #'queue.bind'{exchange = IntXNameBin, queue = Q}), - State1 = State#state{internal_exchange = IntXNameBin}, - rabbit_federation_db:set_active_suffix(DownXName, Upstream, Suffix), - State2 = lists:foldl(fun add_binding/2, State1, Bindings), - OldIntXNameBin = upstream_exchange_name(Q, OldSuffix), - delete_upstream_exchange(Conn, OldIntXNameBin), - State2. - -ensure_upstream_exchange(#state{upstream_params = UParams, - connection = Conn, - channel = Ch}) -> - #upstream_params{x_or_q = X} = UParams, - #exchange{type = Type, - durable = Durable, - auto_delete = AutoDelete, - internal = Internal, - arguments = Arguments} = X, - Decl = #'exchange.declare'{exchange = name(X), - type = list_to_binary(atom_to_list(Type)), - durable = Durable, - auto_delete = AutoDelete, - internal = Internal, - arguments = Arguments}, - rabbit_federation_link_util:disposable_channel_call( - Conn, Decl#'exchange.declare'{passive = true}, - fun(?NOT_FOUND, _Text) -> - amqp_channel:call(Ch, Decl) - end). - -ensure_internal_exchange(IntXNameBin, - #state{upstream = #upstream{max_hops = MaxHops, name = UName}, - upstream_params = UParams, - connection = Conn, - channel = Ch, - downstream_exchange = #resource{virtual_host = DVhost}}) -> - rabbit_log_federation:debug("Exchange federation will set up exchange '~ts' in upstream '~ts'", - [IntXNameBin, UName]), - #upstream_params{params = Params} = rabbit_federation_util:deobfuscate_upstream_params(UParams), - rabbit_log_federation:debug("Will delete upstream exchange '~ts'", [IntXNameBin]), - delete_upstream_exchange(Conn, IntXNameBin), - rabbit_log_federation:debug("Will declare an internal upstream exchange '~ts'", [IntXNameBin]), - Base = #'exchange.declare'{exchange = IntXNameBin, - durable = true, - internal = true, - auto_delete = true}, - Purpose = [{<<"x-internal-purpose">>, longstr, <<"federation">>}], - XFUArgs = [{?MAX_HOPS_ARG, long, MaxHops}, - {?DOWNSTREAM_NAME_ARG, longstr, cycle_detection_node_identifier()}, - {?DOWNSTREAM_VHOST_ARG, longstr, DVhost} - | Purpose], - XFU = Base#'exchange.declare'{type = <<"x-federation-upstream">>, - arguments = XFUArgs}, - Fan = Base#'exchange.declare'{type = <<"fanout">>, - arguments = Purpose}, - rabbit_federation_link_util:disposable_connection_call( - Params, XFU, fun(?COMMAND_INVALID, _Text) -> - amqp_channel:call(Ch, Fan) - end). - -check_internal_exchange(IntXNameBin, - #state{upstream = #upstream{max_hops = MaxHops, name = UName}, - upstream_params = UParams, - downstream_exchange = XName = #resource{virtual_host = DVhost}}) -> - #upstream_params{params = Params} = - rabbit_federation_util:deobfuscate_upstream_params(UParams), - rabbit_log_federation:debug("Exchange federation will check on exchange '~ts' in upstream '~ts'", - [IntXNameBin, UName]), - Base = #'exchange.declare'{exchange = IntXNameBin, - passive = true, - durable = true, - internal = true, - auto_delete = true}, - Purpose = [{<<"x-internal-purpose">>, longstr, <<"federation">>}], - XFUArgs = [{?MAX_HOPS_ARG, long, MaxHops}, - {?DOWNSTREAM_NAME_ARG, longstr, cycle_detection_node_identifier()}, - {?DOWNSTREAM_VHOST_ARG, longstr, DVhost} - | Purpose], - XFU = Base#'exchange.declare'{type = <<"x-federation-upstream">>, - arguments = XFUArgs}, - rabbit_federation_link_util:disposable_connection_call( - Params, XFU, fun(404, Text) -> - rabbit_federation_link_util:log_warning( - XName, "detected internal upstream exchange changes," - " restarting link: ~tp", [Text]), - upstream_not_found; - (Code, Text) -> - rabbit_federation_link_util:log_warning( - XName, "internal upstream exchange check failed: ~tp ~tp", - [Code, Text]), - error - end). - -upstream_queue_name(XNameBin, VHost, #resource{name = DownXNameBin, - virtual_host = DownVHost}) -> - Node = rabbit_nodes:cluster_name(), - DownPart = case DownVHost of - VHost -> case DownXNameBin of - XNameBin -> <<"">>; - _ -> <<":", DownXNameBin/binary>> - end; - _ -> <<":", DownVHost/binary, - ":", DownXNameBin/binary>> - end, - <<"federation: ", XNameBin/binary, " -> ", Node/binary, DownPart/binary>>. - -cycle_detection_node_identifier() -> - rabbit_nodes:cluster_name(). - -upstream_exchange_name(UpstreamQName, Suffix) -> - <>. - -delete_upstream_exchange(Conn, XNameBin) -> - rabbit_federation_link_util:disposable_channel_call( - Conn, #'exchange.delete'{exchange = XNameBin}). - -delete_upstream_queue(Conn, Queue) -> - rabbit_federation_link_util:disposable_channel_call( - Conn, #'queue.delete'{queue = Queue}). - -update_routing_headers(#upstream_params{table = Table}, UpstreamName, UVhost, Redelivered, Headers) -> - NewValue = Table ++ - [{<<"redelivered">>, bool, Redelivered}] ++ - header_for_upstream_name(UpstreamName) ++ - header_for_upstream_vhost(UVhost), - rabbit_basic:prepend_table_header(?ROUTING_HEADER, NewValue, Headers). - -header_for_upstream_name(unknown) -> []; -header_for_upstream_name(Name) -> [{<<"cluster-name">>, longstr, Name}]. - -header_for_upstream_vhost(unknown) -> []; -header_for_upstream_vhost(Name) -> [{<<"vhost">>, longstr, Name}]. - -get_hops(Table) -> - case rabbit_misc:table_lookup(Table, <<"hops">>) of - %% see rabbit_binary_generator - {short, N} -> N; - {long, N} -> N; - {byte, N} -> N; - {signedint, N} -> N; - {unsignedbyte, N} -> N; - {unsignedshort, N} -> N; - {unsignedint, N} -> N; - {_, N} when is_integer(N) andalso N >= 0 -> N - end. - -handle_down(DCh, Reason, _Ch, _CmdCh, DCh, Args, State) -> - rabbit_federation_link_util:handle_downstream_down(Reason, Args, State); -handle_down(ChPid, Reason, Ch, CmdCh, _DCh, Args, State) - when ChPid =:= Ch; ChPid =:= CmdCh -> - rabbit_federation_link_util:handle_upstream_down(Reason, Args, State). 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 deleted file mode 100644 index e1a962afb5b2..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_exchange_link_sup_sup.erl +++ /dev/null @@ -1,86 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_exchange_link_sup_sup). - --behaviour(mirrored_supervisor). - --include_lib("rabbit_common/include/rabbit.hrl"). --define(SUPERVISOR, ?MODULE). - -%% Supervises the upstream links for all exchanges (but not queues). We need -%% different handling here since exchanges want a mirrored sup. - --export([start_link/0, start_child/1, adjust/1, stop_child/1]). --export([init/1]). --export([id_to_khepri_path/1]). - -%%---------------------------------------------------------------------------- - -start_link() -> - _ = pg:start_link(), - %% This scope is used by concurrently starting exchange and queue links, - %% and other places, so we have to start it very early outside of the supervision tree. - %% The scope is stopped in stop/1. - _ = rabbit_federation_pg:start_scope(), - mirrored_supervisor:start_link({local, ?SUPERVISOR}, ?SUPERVISOR, - ?MODULE, []). - -%% Note that the next supervisor down, rabbit_federation_link_sup, is common -%% between exchanges and queues. -start_child(X) -> - case mirrored_supervisor:start_child( - ?SUPERVISOR, - {id(X), {rabbit_federation_link_sup, start_link, [X]}, - transient, ?SUPERVISOR_WAIT, supervisor, - [rabbit_federation_link_sup]}) of - {ok, _Pid} -> ok; - {error, {already_started, _Pid}} -> - #exchange{name = ExchangeName} = X, - rabbit_log_federation:debug("Federation link for exchange ~tp was already started", - [rabbit_misc:rs(ExchangeName)]), - ok; - %% A link returned {stop, gone}, the link_sup shut down, that's OK. - {error, {shutdown, _}} -> ok - end. - -adjust({clear_upstream, VHost, UpstreamName}) -> - _ = [rabbit_federation_link_sup:adjust(Pid, X, {clear_upstream, UpstreamName}) || - {#exchange{name = Name} = X, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR), - Name#resource.virtual_host == VHost], - ok; -adjust(Reason) -> - _ = [rabbit_federation_link_sup:adjust(Pid, X, Reason) || - {X, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR)], - ok. - -stop_child(X) -> - case mirrored_supervisor:terminate_child(?SUPERVISOR, id(X)) of - ok -> ok; - {error, Err} -> - #exchange{name = ExchangeName} = X, - rabbit_log_federation:warning( - "Attempt to stop a federation link for exchange ~tp failed: ~tp", - [rabbit_misc:rs(ExchangeName), Err]), - ok - end, - ok = mirrored_supervisor:delete_child(?SUPERVISOR, id(X)). - -%%---------------------------------------------------------------------------- - -init([]) -> - {ok, {{one_for_one, 1200, 60}, []}}. - -%% See comment in rabbit_federation_queue_link_sup_sup:id/1 -id(X = #exchange{policy = Policy}) -> - X1 = rabbit_exchange:immutable(X), - X2 = X1#exchange{policy = Policy}, - X2. - -id_to_khepri_path( - #exchange{name = #resource{virtual_host = VHost, name = Name}}) -> - [exchange, VHost, Name]. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_link_sup.erl b/deps/rabbitmq_federation/src/rabbit_federation_link_sup.erl deleted file mode 100644 index e52c0c889cf0..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_link_sup.erl +++ /dev/null @@ -1,109 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_link_sup). - --behaviour(supervisor2). - --include_lib("rabbit_common/include/rabbit.hrl"). --include_lib("rabbit/include/amqqueue.hrl"). --include("rabbit_federation.hrl"). - -%% Supervises the upstream links for an exchange or queue. - --export([start_link/1, adjust/3, restart/2]). --export([init/1]). - -start_link(XorQ) -> - supervisor2:start_link(?MODULE, XorQ). - -adjust(Sup, XorQ, everything) -> - _ = [stop(Sup, Upstream, XorQ) || - {Upstream, _, _, _} <- supervisor2:which_children(Sup)], - [{ok, _Pid} = supervisor2:start_child(Sup, Spec) || Spec <- specs(XorQ)]; - -adjust(Sup, XorQ, {upstream, UpstreamName}) -> - OldUpstreams0 = children(Sup, UpstreamName), - NewUpstreams0 = rabbit_federation_upstream:for(XorQ, UpstreamName), - %% If any haven't changed, don't restart them. The broker will - %% avoid telling us about connections that have not changed - %% syntactically, but even if one has, this XorQ may not have that - %% connection in an upstream, so we still need to check here. - {OldUpstreams, NewUpstreams} = - lists:foldl( - fun (OldU, {OldUs, NewUs}) -> - case lists:member(OldU, NewUs) of - true -> {OldUs -- [OldU], NewUs -- [OldU]}; - false -> {OldUs, NewUs} - end - end, {OldUpstreams0, NewUpstreams0}, OldUpstreams0), - _ = [stop(Sup, OldUpstream, XorQ) || OldUpstream <- OldUpstreams], - [start(Sup, NewUpstream, XorQ) || NewUpstream <- NewUpstreams]; - -adjust(Sup, XorQ, {clear_upstream, UpstreamName}) -> - ok = rabbit_federation_db:prune_scratch( - name(XorQ), rabbit_federation_upstream:for(XorQ)), - [stop(Sup, Upstream, XorQ) || Upstream <- children(Sup, UpstreamName)]; - -adjust(Sup, X = #exchange{name = XName}, {upstream_set, _Set}) -> - _ = adjust(Sup, X, everything), - case rabbit_federation_upstream:federate(X) of - false -> ok; - true -> ok = rabbit_federation_db:prune_scratch( - XName, rabbit_federation_upstream:for(X)) - end; -adjust(Sup, Q, {upstream_set, _}) when ?is_amqqueue(Q) -> - adjust(Sup, Q, everything); -adjust(Sup, XorQ, {clear_upstream_set, _}) -> - adjust(Sup, XorQ, everything). - -restart(Sup, Upstream) -> - ok = supervisor2:terminate_child(Sup, Upstream), - {ok, _Pid} = supervisor2:restart_child(Sup, Upstream), - ok. - -start(Sup, Upstream, XorQ) -> - {ok, _Pid} = supervisor2:start_child(Sup, spec(rabbit_federation_util:obfuscate_upstream(Upstream), XorQ)), - ok. - -stop(Sup, Upstream, XorQ) -> - ok = supervisor2:terminate_child(Sup, Upstream), - ok = supervisor2:delete_child(Sup, Upstream), - %% While the link will report its own removal, that only works if - %% the link was actually up. If the link was broken and failing to - %% come up, the possibility exists that there *is* no link - %% process, but we still have a report in the status table. So - %% remove it here too. - rabbit_federation_status:remove(Upstream, name(XorQ)). - -children(Sup, UpstreamName) -> - rabbit_federation_util:find_upstreams( - UpstreamName, [U || {U, _, _, _} <- supervisor2:which_children(Sup)]). - -%%---------------------------------------------------------------------------- - -init(XorQ) -> - %% 1, ?MAX_WAIT so that we always give up after one fast retry and get - %% into the reconnect delay. - {ok, {{one_for_one, 1, ?MAX_WAIT}, specs(XorQ)}}. - -specs(XorQ) -> - [spec(rabbit_federation_util:obfuscate_upstream(Upstream), XorQ) - || Upstream <- rabbit_federation_upstream:for(XorQ)]. - -spec(U = #upstream{reconnect_delay = Delay}, #exchange{name = XName}) -> - {U, {rabbit_federation_exchange_link, start_link, [{U, XName}]}, - {permanent, Delay}, ?WORKER_WAIT, worker, - [rabbit_federation_exchange_link]}; - -spec(Upstream = #upstream{reconnect_delay = Delay}, Q) when ?is_amqqueue(Q) -> - {Upstream, {rabbit_federation_queue_link, start_link, [{Upstream, Q}]}, - {permanent, Delay}, ?WORKER_WAIT, worker, - [rabbit_federation_queue_link]}. - -name(#exchange{name = XName}) -> XName; -name(Q) when ?is_amqqueue(Q) -> amqqueue:get_name(Q). diff --git a/deps/rabbitmq_federation/src/rabbit_federation_link_util.erl b/deps/rabbitmq_federation/src/rabbit_federation_link_util.erl deleted file mode 100644 index bbedc954babf..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_link_util.erl +++ /dev/null @@ -1,359 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_link_util). - --include_lib("rabbit/include/amqqueue.hrl"). --include_lib("amqp_client/include/amqp_client.hrl"). --include("rabbit_federation.hrl"). - -%% real --export([start_conn_ch/5, disposable_channel_call/2, disposable_channel_call/3, - disposable_connection_call/3, ensure_connection_closed/1, - log_terminate/4, unacked_new/0, ack/3, nack/3, forward/9, - handle_downstream_down/3, handle_upstream_down/3, - get_connection_name/2, log_debug/3, log_info/3, log_warning/3, - log_error/3]). - -%% temp --export([connection_error/6]). - --import(rabbit_misc, [pget/2]). - --define(MAX_CONNECTION_CLOSE_TIMEOUT, 10000). - -%%---------------------------------------------------------------------------- - -start_conn_ch(Fun, OUpstream, OUParams, - XorQName = #resource{virtual_host = DownVHost}, State) -> - - Upstream = rabbit_federation_util:deobfuscate_upstream(OUpstream), - UParams = rabbit_federation_util:deobfuscate_upstream_params(OUParams), - - ConnName = get_connection_name(Upstream, UParams), - case open_monitor(#amqp_params_direct{virtual_host = DownVHost}, ConnName) of - {ok, DConn, DCh} -> - case Upstream#upstream.ack_mode of - 'on-confirm' -> - #'confirm.select_ok'{} = - amqp_channel:call(DCh, #'confirm.select'{}), - amqp_channel:register_confirm_handler(DCh, self()); - _ -> - ok - end, - case open_monitor(UParams#upstream_params.params, ConnName) of - {ok, Conn, Ch} -> - %% Don't trap exits until we have established - %% connections so that if we try to delete - %% federation upstreams while waiting for a - %% connection to be established then we don't - %% block - process_flag(trap_exit, true), - try - R = Fun(Conn, Ch, DConn, DCh), - log_info( - XorQName, "connected to ~ts", - [rabbit_federation_upstream:params_to_string( - UParams)]), - Name = pget(name, amqp_connection:info(DConn, [name])), - rabbit_federation_status:report( - OUpstream, OUParams, XorQName, {running, Name}), - R - catch exit:E -> - %% terminate/2 will not get this, as we - %% have not put them in our state yet - ensure_connection_closed(DConn), - ensure_connection_closed(Conn), - connection_error(remote_start, E, - OUpstream, OUParams, XorQName, State) - end; - E -> - ensure_connection_closed(DConn), - connection_error(remote_start, E, - OUpstream, OUParams, XorQName, State) - end; - E -> - connection_error(local_start, E, - OUpstream, OUParams, XorQName, State) - end. - -get_connection_name(#upstream{name = UpstreamName}, - #upstream_params{x_or_q = Resource}) when is_record(Resource, exchange) orelse ?is_amqqueue(Resource) -> - connection_name(UpstreamName, rabbit_policy:name(Resource)); - -get_connection_name(_, _) -> - connection_name(undefined, undefined). - -connection_name(Upstream, Policy) when is_binary(Upstream), is_binary(Policy) -> - <<<<"Federation link (upstream: ">>/binary, Upstream/binary, <<", policy: ">>/binary, Policy/binary, <<")">>/binary>>; -connection_name(_, _) -> - <<"Federation link">>. - -open_monitor(Params, Name) -> - case open(Params, Name) of - {ok, Conn, Ch} -> erlang:monitor(process, Ch), - {ok, Conn, Ch}; - E -> E - end. - -open(Params, Name) -> - try - amqp_connection:start(Params, Name) - of - {ok, Conn} -> - try - amqp_connection:open_channel(Conn) - of - {ok, Ch} -> {ok, Conn, Ch}; - E -> ensure_connection_closed(Conn), - E - catch - _:E -> - ensure_connection_closed(Conn), - E - end; - E -> E - catch - _:E -> E - end. - -ensure_channel_closed(Ch) -> catch amqp_channel:close(Ch). - -ensure_connection_closed(Conn) -> - catch amqp_connection:close(Conn, ?MAX_CONNECTION_CLOSE_TIMEOUT). - -connection_error(remote_start, {{shutdown, {server_initiated_close, Code, Message}}, _} = E, - Upstream, UParams, XorQName, State) -> - rabbit_federation_status:report( - Upstream, UParams, XorQName, clean_reason(E)), - log_warning(XorQName, - "did not connect to ~ts. Server has closed the connection due to an error, code: ~tp, " - "message: ~ts", - [rabbit_federation_upstream:params_to_string(UParams), - Code, Message]), - {stop, {shutdown, restart}, State}; - -connection_error(remote_start, E, Upstream, UParams, XorQName, State) -> - rabbit_federation_status:report( - Upstream, UParams, XorQName, clean_reason(E)), - log_warning(XorQName, "did not connect to ~ts. Reason: ~tp", - [rabbit_federation_upstream:params_to_string(UParams), - E]), - {stop, {shutdown, restart}, State}; - -connection_error(remote, E, Upstream, UParams, XorQName, State) -> - rabbit_federation_status:report( - Upstream, UParams, XorQName, clean_reason(E)), - log_info(XorQName, "disconnected from ~ts~n~tp", - [rabbit_federation_upstream:params_to_string(UParams), E]), - {stop, {shutdown, restart}, State}; - -connection_error(command_channel, E, Upstream, UParams, XorQName, State) -> - rabbit_federation_status:report( - Upstream, UParams, XorQName, clean_reason(E)), - log_info(XorQName, "failed to open a command channel for upstream ~ts~n~tp", - [rabbit_federation_upstream:params_to_string(UParams), E]), - {stop, {shutdown, restart}, State}; - -connection_error(local, basic_cancel, Upstream, UParams, XorQName, State) -> - rabbit_federation_status:report( - Upstream, UParams, XorQName, {error, basic_cancel}), - log_info(XorQName, "received a 'basic.cancel'", []), - {stop, {shutdown, restart}, State}; - -connection_error(local_start, E, Upstream, UParams, XorQName, State) -> - rabbit_federation_status:report( - Upstream, UParams, XorQName, clean_reason(E)), - log_warning(XorQName, "did not connect locally~n~tp", [E]), - {stop, {shutdown, restart}, State}. - -%% If we terminate due to a gen_server call exploding (almost -%% certainly due to an amqp_channel:call() exploding) then we do not -%% want to report the gen_server call in our status. -clean_reason({E = {shutdown, _}, _}) -> E; -clean_reason(E) -> E. - -%% local / disconnected never gets invoked, see handle_info({'DOWN', ... - -%%---------------------------------------------------------------------------- - -unacked_new() -> gb_trees:empty(). - -ack(#'basic.ack'{delivery_tag = Seq, - multiple = Multiple}, Ch, Unack) -> - amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = gb_trees:get(Seq, Unack), - multiple = Multiple}), - remove_delivery_tags(Seq, Multiple, Unack). - - -%% Note: at time of writing the broker will never send requeue=false. And it's -%% hard to imagine why it would. But we may as well handle it. -nack(#'basic.nack'{delivery_tag = Seq, - multiple = Multiple, - requeue = Requeue}, Ch, Unack) -> - amqp_channel:cast(Ch, #'basic.nack'{delivery_tag = gb_trees:get(Seq, Unack), - multiple = Multiple, - requeue = Requeue}), - remove_delivery_tags(Seq, Multiple, Unack). - -remove_delivery_tags(Seq, false, Unacked) -> - gb_trees:delete(Seq, Unacked); -remove_delivery_tags(Seq, true, Unacked) -> - case gb_trees:is_empty(Unacked) of - true -> Unacked; - false -> {Smallest, _Val, Unacked1} = gb_trees:take_smallest(Unacked), - case Smallest > Seq of - true -> Unacked; - false -> remove_delivery_tags(Seq, true, Unacked1) - end - end. - -forward(#upstream{ack_mode = AckMode, - trust_user_id = Trust}, - #'basic.deliver'{delivery_tag = DT}, - Ch, DCh, PublishMethod, HeadersFun, ForwardFun, Msg, Unacked) -> - Headers = extract_headers(Msg), - case ForwardFun(Headers) of - true -> Msg1 = maybe_clear_user_id( - Trust, update_headers(HeadersFun(Headers), Msg)), - Seq = case AckMode of - 'on-confirm' -> amqp_channel:next_publish_seqno(DCh); - _ -> ignore - end, - amqp_channel:cast(DCh, PublishMethod, Msg1), - case AckMode of - 'on-confirm' -> - gb_trees:insert(Seq, DT, Unacked); - 'on-publish' -> - amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DT}), - Unacked; - 'no-ack' -> - Unacked - end; - false -> amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DT}), - %% Drop it, but acknowledge it! - Unacked - end. - -maybe_clear_user_id(false, Msg = #amqp_msg{props = Props}) -> - Msg#amqp_msg{props = Props#'P_basic'{user_id = undefined}}; -maybe_clear_user_id(true, Msg) -> - Msg. - -extract_headers(#amqp_msg{props = #'P_basic'{headers = Headers}}) -> - Headers. - -update_headers(Headers, Msg = #amqp_msg{props = Props}) -> - Msg#amqp_msg{props = Props#'P_basic'{headers = Headers}}. - -%%---------------------------------------------------------------------------- - -%% If the downstream channel shuts down cleanly, we can just ignore it -%% - we're the same node, we're presumably about to go down too. -handle_downstream_down(shutdown, _Args, State) -> - {noreply, State}; - -handle_downstream_down(Reason, _Args, State) -> - {stop, {downstream_channel_down, Reason}, State}. - -%% If the upstream channel goes down for an intelligible reason, just -%% log it and die quietly. -handle_upstream_down(shutdown, {Upstream, UParams, XName}, State) -> - rabbit_federation_link_util:connection_error( - remote, {upstream_channel_down, shutdown}, Upstream, UParams, XName, State); -handle_upstream_down({shutdown, Reason}, {Upstream, UParams, XName}, State) -> - rabbit_federation_link_util:connection_error( - remote, {upstream_channel_down, Reason}, Upstream, UParams, XName, State); - -handle_upstream_down(Reason, _Args, State) -> - {stop, {upstream_channel_down, Reason}, State}. - -%%---------------------------------------------------------------------------- - -log_terminate(gone, _Upstream, _UParams, _XorQName) -> - %% the link cannot start, this has been logged already - ok; -log_terminate({shutdown, restart}, _Upstream, _UParams, _XorQName) -> - %% We've already logged this before munging the reason - ok; -log_terminate(shutdown, Upstream, UParams, XorQName) -> - %% The supervisor is shutting us down; we are probably restarting - %% the link because configuration has changed. So try to shut down - %% nicely so that we do not cause unacked messages to be - %% redelivered. - log_info(XorQName, "disconnecting from ~ts", - [rabbit_federation_upstream:params_to_string(UParams)]), - rabbit_federation_status:remove(Upstream, XorQName); - -log_terminate(Reason, Upstream, UParams, XorQName) -> - %% Unexpected death. sasl will log it, but we should update - %% rabbit_federation_status. - rabbit_federation_status:report( - Upstream, UParams, XorQName, clean_reason(Reason)). - -log_debug(XorQName, Fmt, Args) -> log(debug, XorQName, Fmt, Args). -log_info(XorQName, Fmt, Args) -> log(info, XorQName, Fmt, Args). -log_warning(XorQName, Fmt, Args) -> log(warning, XorQName, Fmt, Args). -log_error(XorQName, Fmt, Args) -> log(error, XorQName, Fmt, Args). - -log(Level, XorQName, Fmt0, Args0) -> - Fmt = "Federation ~ts " ++ Fmt0, - Args = [rabbit_misc:rs(XorQName) | Args0], - case Level of - debug -> rabbit_log_federation:debug(Fmt, Args); - info -> rabbit_log_federation:info(Fmt, Args); - warning -> rabbit_log_federation:warning(Fmt, Args); - error -> rabbit_log_federation:error(Fmt, Args) - end. - -%%---------------------------------------------------------------------------- - -disposable_channel_call(Conn, Method) -> - disposable_channel_call(Conn, Method, fun(_, _) -> ok end). - -disposable_channel_call(Conn, Method, ErrFun) -> - try - {ok, Ch} = amqp_connection:open_channel(Conn), - try - amqp_channel:call(Ch, Method) - catch exit:{{shutdown, {server_initiated_close, Code, Message}}, _} -> - ErrFun(Code, Message) - after - ensure_channel_closed(Ch) - end - catch - Exception:Reason -> - rabbit_log_federation:error("Federation link could not create a disposable (one-off) channel due to an error ~tp: ~tp", [Exception, Reason]) - end. - -disposable_connection_call(Params, Method, ErrFun) -> - try - rabbit_log_federation:debug("Disposable connection parameters: ~tp", [Params]), - case open(Params, <<"Disposable exchange federation link connection">>) of - {ok, Conn, Ch} -> - try - amqp_channel:call(Ch, Method) - catch exit:{{shutdown, {connection_closing, {server_initiated_close, Code, Message}}}, _} -> - ErrFun(Code, Message); - exit:{{shutdown, {server_initiated_close, Code, Message}}, _} -> - ErrFun(Code, Message) - after - ensure_connection_closed(Conn) - end; - {error, {auth_failure, Message}} -> - rabbit_log_federation:error("Federation link could not open a disposable (one-off) connection " - "due to an authentication failure: ~ts", [Message]); - Error -> - rabbit_log_federation:error("Federation link could not open a disposable (one-off) connection, " - "reason: ~tp", [Error]), - Error - end - catch - Exception:Reason -> - rabbit_log_federation:error("Federation link could not create a disposable (one-off) connection " - "due to an error ~tp: ~tp", [Exception, Reason]) - end. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_parameters.erl b/deps/rabbitmq_federation/src/rabbit_federation_parameters.erl deleted file mode 100644 index 02c498d69dd5..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_parameters.erl +++ /dev/null @@ -1,141 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_parameters). --behaviour(rabbit_runtime_parameter). --behaviour(rabbit_policy_validator). - --export([validate/5, notify/5, notify_clear/4]). --export([register/0, unregister/0, validate_policy/1, adjust/1]). - --define(RUNTIME_PARAMETERS, - [{runtime_parameter, <<"federation">>}, - {runtime_parameter, <<"federation-upstream">>}, - {runtime_parameter, <<"federation-upstream-set">>}, - {policy_validator, <<"federation-upstream">>}, - {policy_validator, <<"federation-upstream-pattern">>}, - {policy_validator, <<"federation-upstream-set">>}]). - --rabbit_boot_step({?MODULE, - [{description, "federation parameters"}, - {mfa, {rabbit_federation_parameters, register, []}}, - {requires, rabbit_registry}, - {cleanup, {rabbit_federation_parameters, unregister, []}}, - {enables, recovery}]}). - -register() -> - [rabbit_registry:register(Class, Name, ?MODULE) || - {Class, Name} <- ?RUNTIME_PARAMETERS], - ok. - -unregister() -> - [rabbit_registry:unregister(Class, Name) || - {Class, Name} <- ?RUNTIME_PARAMETERS], - ok. - -validate(_VHost, <<"federation-upstream-set">>, Name, Term0, _User) -> - Term = [rabbit_data_coercion:to_proplist(Upstream) || Upstream <- Term0], - [rabbit_parameter_validation:proplist( - Name, - [{<<"upstream">>, fun rabbit_parameter_validation:binary/2, mandatory} | - shared_validation()], Upstream) - || Upstream <- Term]; - -validate(_VHost, <<"federation-upstream">>, Name, Term0, _User) -> - Term = rabbit_data_coercion:to_proplist(Term0), - rabbit_parameter_validation:proplist( - Name, [{<<"uri">>, fun validate_uri/2, mandatory} | - shared_validation()], Term); - -validate(_VHost, _Component, Name, _Term, _User) -> - {error, "name not recognised: ~tp", [Name]}. - -notify(_VHost, <<"federation-upstream-set">>, Name, _Term, _Username) -> - adjust({upstream_set, Name}); - -notify(_VHost, <<"federation-upstream">>, Name, _Term, _Username) -> - adjust({upstream, Name}). - -notify_clear(_VHost, <<"federation-upstream-set">>, Name, _Username) -> - adjust({clear_upstream_set, Name}); - -notify_clear(VHost, <<"federation-upstream">>, Name, _Username) -> - rabbit_federation_exchange_link_sup_sup:adjust({clear_upstream, VHost, Name}), - rabbit_federation_queue_link_sup_sup:adjust({clear_upstream, VHost, Name}). - -adjust(Thing) -> - rabbit_federation_exchange_link_sup_sup:adjust(Thing), - rabbit_federation_queue_link_sup_sup:adjust(Thing). - -%%---------------------------------------------------------------------------- - -shared_validation() -> - [{<<"exchange">>, fun rabbit_parameter_validation:binary/2, optional}, - {<<"queue">>, fun rabbit_parameter_validation:binary/2, optional}, - {<<"consumer-tag">>, fun rabbit_parameter_validation:binary/2, optional}, - {<<"prefetch-count">>, fun rabbit_parameter_validation:number/2, optional}, - {<<"reconnect-delay">>,fun rabbit_parameter_validation:number/2, optional}, - {<<"max-hops">>, fun rabbit_parameter_validation:number/2, optional}, - {<<"expires">>, fun rabbit_parameter_validation:number/2, optional}, - {<<"message-ttl">>, fun rabbit_parameter_validation:number/2, optional}, - {<<"trust-user-id">>, fun rabbit_parameter_validation:boolean/2, optional}, - {<<"ack-mode">>, rabbit_parameter_validation:enum( - ['no-ack', 'on-publish', 'on-confirm']), optional}, - {<<"resource-cleanup-mode">>, rabbit_parameter_validation:enum( - ['default', 'never']), optional}, - {<<"queue-type">>, rabbit_parameter_validation:enum( - ['classic', 'quorum']), optional}, - {<<"bind-nowait">>, fun rabbit_parameter_validation:boolean/2, optional}, - {<<"channel-use-mode">>, rabbit_parameter_validation:enum( - ['multiple', 'single']), optional}]. - -validate_uri(Name, Term) when is_binary(Term) -> - case rabbit_parameter_validation:binary(Name, Term) of - ok -> case amqp_uri:parse(binary_to_list(Term)) of - {ok, _} -> ok; - {error, E} -> {error, "\"~ts\" not a valid URI: ~tp", [Term, E]} - end; - E -> E - end; -validate_uri(Name, Term) -> - case rabbit_parameter_validation:list(Name, Term) of - ok -> case [V || U <- Term, - V <- [validate_uri(Name, U)], - element(1, V) =:= error] of - [] -> ok; - [E | _] -> E - end; - E -> E - end. - -%%---------------------------------------------------------------------------- - -validate_policy([{<<"federation-upstream-set">>, Value}]) - when is_binary(Value) -> - ok; -validate_policy([{<<"federation-upstream-set">>, Value}]) -> - {error, "~tp is not a valid federation upstream set name", [Value]}; - -validate_policy([{<<"federation-upstream-pattern">>, Value}]) - when is_binary(Value) -> - case re:compile(Value) of - {ok, _} -> ok; - {error, Reason} -> {error, "could not compile pattern ~ts to a regular expression. " - "Error: ~tp", [Value, Reason]} - end; -validate_policy([{<<"federation-upstream-pattern">>, Value}]) -> - {error, "~tp is not a valid federation upstream pattern name", [Value]}; - -validate_policy([{<<"federation-upstream">>, Value}]) - when is_binary(Value) -> - ok; -validate_policy([{<<"federation-upstream">>, Value}]) -> - {error, "~tp is not a valid federation upstream name", [Value]}; - -validate_policy(L) when length(L) >= 2 -> - {error, "cannot specify federation-upstream, federation-upstream-set " - "or federation-upstream-pattern together", []}. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_pg.erl b/deps/rabbitmq_federation/src/rabbit_federation_pg.erl deleted file mode 100644 index fceb7b54217f..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_pg.erl +++ /dev/null @@ -1,25 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_pg). - --include("rabbit_federation.hrl"). - --export([start_scope/0, stop_scope/0]). - -start_scope() -> - rabbit_log_federation:debug("Starting pg scope ~ts", [?FEDERATION_PG_SCOPE]), - _ = pg:start_link(?FEDERATION_PG_SCOPE). - -stop_scope() -> - case whereis(?FEDERATION_PG_SCOPE) of - Pid when is_pid(Pid) -> - rabbit_log_federation:debug("Stopping pg scope ~ts", [?FEDERATION_PG_SCOPE]), - exit(Pid, normal); - _ -> - ok - end. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_queue.erl b/deps/rabbitmq_federation/src/rabbit_federation_queue.erl deleted file mode 100644 index 422d8fc39734..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_queue.erl +++ /dev/null @@ -1,109 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_queue). - --rabbit_boot_step({?MODULE, - [{description, "federation queue decorator"}, - {mfa, {rabbit_queue_decorator, register, - [<<"federation">>, ?MODULE]}}, - {requires, rabbit_registry}, - {cleanup, {rabbit_queue_decorator, unregister, - [<<"federation">>]}}, - {enables, recovery}]}). - --include_lib("rabbit/include/amqqueue.hrl"). --include("rabbit_federation.hrl"). - --behaviour(rabbit_queue_decorator). - --export([startup/1, shutdown/1, policy_changed/2, active_for/1, - consumer_state_changed/3]). --export([policy_changed_local/2]). - -%%---------------------------------------------------------------------------- - -startup(Q) -> - case active_for(Q) of - true -> rabbit_federation_queue_link_sup_sup:start_child(Q); - false -> ok - end, - ok. - -shutdown(Q) when ?is_amqqueue(Q) -> - QName = amqqueue:get_name(Q), - case active_for(Q) of - true -> rabbit_federation_queue_link_sup_sup:stop_child(Q), - rabbit_federation_status:remove_exchange_or_queue(QName); - false -> ok - end, - ok. - -policy_changed(Q1, Q2) when ?is_amqqueue(Q1) -> - QName = amqqueue:get_name(Q1), - case rabbit_amqqueue:lookup(QName) of - {ok, Q0} when ?is_amqqueue(Q0) -> - rpc:call(amqqueue:qnode(Q0), rabbit_federation_queue, - policy_changed_local, [Q1, Q2]); - {error, not_found} -> - ok - end. - -policy_changed_local(Q1, Q2) -> - shutdown(Q1), - startup(Q2). - -active_for(Q) -> - Args = amqqueue:get_arguments(Q), - case rabbit_misc:table_lookup(Args, <<"x-internal-purpose">>) of - {longstr, _} -> false; %% [0] - _ -> rabbit_federation_upstream:federate(Q) - end. -%% [0] Currently the only "internal purpose" is federation, but I -%% suspect if we introduce another one it will also be for something -%% that doesn't want to be federated. - -%% We need to reconsider whether we need to run or pause every time -%% the consumer state changes in the queue. But why can the state -%% change? -%% -%% consumer blocked | We may have no more active consumers, and thus need to -%% | pause -%% | -%% consumer unblocked | We don't care -%% | -%% queue empty | The queue has become empty therefore we need to run to -%% | get more messages -%% | -%% basic consume | We don't care -%% | -%% basic cancel | We may have no more active consumers, and thus need to -%% | pause -%% | -%% refresh | We asked for it (we have started a new link after -%% | failover and need something to prod us into action -%% | (or not)). -%% -%% In the cases where we don't care it's not prohibitively expensive -%% for us to be here anyway, so never mind. -%% -%% Note that there is no "queue became non-empty" state change - that's -%% because of the queue invariant. If the queue transitions from empty to -%% non-empty then it must have no active consumers - in which case it stays -%% the same from our POV. - -consumer_state_changed(Q, MaxActivePriority, IsEmpty) -> - QName = amqqueue:get_name(Q), - _ = case IsEmpty andalso active_unfederated(MaxActivePriority) of - true -> rabbit_federation_queue_link:run(QName); - false -> rabbit_federation_queue_link:pause(QName) - end, - ok. - -active_unfederated(empty) -> false; -active_unfederated(P) when P >= 0 -> true; -active_unfederated(_P) -> false. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_queue_link.erl b/deps/rabbitmq_federation/src/rabbit_federation_queue_link.erl deleted file mode 100644 index 9bfa8faf91c4..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_queue_link.erl +++ /dev/null @@ -1,326 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_queue_link). - --include_lib("rabbit/include/amqqueue.hrl"). --include_lib("amqp_client/include/amqp_client.hrl"). --include("rabbit_federation.hrl"). - --behaviour(gen_server2). - --export([start_link/1, go/0, run/1, pause/1]). - --export([init/1, handle_call/3, handle_cast/2, handle_info/2, - terminate/2, code_change/3]). - --import(rabbit_misc, [pget/2]). --import(rabbit_federation_util, [name/1, pgname/1]). - --record(not_started, {queue, run, upstream, upstream_params}). --record(state, {queue, run, conn, ch, dconn, dch, upstream, upstream_params, - unacked}). - -start_link(Args) -> - gen_server2:start_link(?MODULE, Args, [{timeout, infinity}]). - -run(QName) -> cast(QName, run). -pause(QName) -> cast(QName, pause). -go() -> - _ = rabbit_federation_pg:start_scope(), - cast(go). - -%%---------------------------------------------------------------------------- -%%call(QName, Msg) -> [gen_server2:call(Pid, Msg, infinity) || Pid <- q(QName)]. -cast(Msg) -> [gen_server2:cast(Pid, Msg) || Pid <- all()]. -cast(QName, Msg) -> [gen_server2:cast(Pid, Msg) || Pid <- q(QName)]. - -join(Name) -> - ok = pg:join(?FEDERATION_PG_SCOPE, pgname(Name), self()). - -all() -> - pg:get_members(?FEDERATION_PG_SCOPE, pgname(rabbit_federation_queues)). - -q(QName) -> - pg:get_members(?FEDERATION_PG_SCOPE, pgname({rabbit_federation_queue, QName})). - -%%---------------------------------------------------------------------------- - -init({Upstream, Queue}) when ?is_amqqueue(Queue) -> - QName = amqqueue:get_name(Queue), - case rabbit_amqqueue:lookup(QName) of - {ok, Q} -> - DeobfuscatedUpstream = rabbit_federation_util:deobfuscate_upstream(Upstream), - DeobfuscatedUParams = rabbit_federation_upstream:to_params(DeobfuscatedUpstream, Queue), - UParams = rabbit_federation_util:obfuscate_upstream_params(DeobfuscatedUParams), - rabbit_federation_status:report(Upstream, UParams, QName, starting), - join(rabbit_federation_queues), - join({rabbit_federation_queue, QName}), - gen_server2:cast(self(), maybe_go), - rabbit_amqqueue:notify_decorators(Q), - {ok, #not_started{queue = Queue, - run = false, - upstream = Upstream, - upstream_params = UParams}}; - {error, not_found} -> - rabbit_federation_link_util:log_warning(QName, "not found, stopping link", []), - {stop, gone} - end. - -handle_call(Msg, _From, State) -> - {stop, {unexpected_call, Msg}, {unexpected_call, Msg}, State}. - -handle_cast(maybe_go, State) -> - go(State); - -handle_cast(go, State = #not_started{}) -> - go(State); - -handle_cast(go, State) -> - {noreply, State}; - -handle_cast(run, State = #state{upstream = Upstream, - upstream_params = UParams, - ch = Ch, - run = false}) -> - consume(Ch, Upstream, UParams#upstream_params.x_or_q), - {noreply, State#state{run = true}}; - -handle_cast(run, State = #not_started{}) -> - {noreply, State#not_started{run = true}}; - -handle_cast(run, State) -> - %% Already started - {noreply, State}; - -handle_cast(pause, State = #state{run = false}) -> - %% Already paused - {noreply, State}; - -handle_cast(pause, State = #not_started{}) -> - {noreply, State#not_started{run = false}}; - -handle_cast(pause, State = #state{ch = Ch, upstream = Upstream}) -> - cancel(Ch, Upstream), - {noreply, State#state{run = false}}; - -handle_cast(Msg, State) -> - {stop, {unexpected_cast, Msg}, State}. - -handle_info(#'basic.consume_ok'{}, State) -> - {noreply, State}; - -handle_info(#'basic.ack'{} = Ack, State = #state{ch = Ch, - unacked = Unacked}) -> - Unacked1 = rabbit_federation_link_util:ack(Ack, Ch, Unacked), - {noreply, State#state{unacked = Unacked1}}; - -handle_info(#'basic.nack'{} = Nack, State = #state{ch = Ch, - unacked = Unacked}) -> - Unacked1 = rabbit_federation_link_util:nack(Nack, Ch, Unacked), - {noreply, State#state{unacked = Unacked1}}; - -handle_info({#'basic.deliver'{redelivered = Redelivered, - exchange = X, - routing_key = K} = DeliverMethod, Msg}, - State = #state{queue = Q, - upstream = Upstream, - upstream_params = UParams, - ch = Ch, - dch = DCh, - unacked = Unacked}) when ?is_amqqueue(Q) -> - QName = amqqueue:get_name(Q), - PublishMethod = #'basic.publish'{exchange = <<"">>, - routing_key = QName#resource.name}, - HeadersFun = fun (H) -> update_headers(UParams, Redelivered, X, K, H) end, - ForwardFun = fun (_H) -> true end, - Unacked1 = rabbit_federation_link_util:forward( - Upstream, DeliverMethod, Ch, DCh, PublishMethod, - HeadersFun, ForwardFun, Msg, Unacked), - %% TODO actually we could reject when 'stopped' - {noreply, State#state{unacked = Unacked1}}; - -handle_info(#'basic.cancel'{}, - State = #state{queue = Q, - upstream = Upstream, - upstream_params = UParams}) when ?is_amqqueue(Q) -> - QName = amqqueue:get_name(Q), - rabbit_federation_link_util:connection_error( - local, basic_cancel, Upstream, UParams, QName, State); - -handle_info({'DOWN', _Ref, process, Pid, Reason}, - State = #state{dch = DCh, - ch = Ch, - upstream = Upstream, - upstream_params = UParams, - queue = Q}) when ?is_amqqueue(Q) -> - QName = amqqueue:get_name(Q), - handle_down(Pid, Reason, Ch, DCh, {Upstream, UParams, QName}, State); - -handle_info(Msg, State) -> - {stop, {unexpected_info, Msg}, State}. - -terminate(Reason, #not_started{upstream = Upstream, - upstream_params = UParams, - queue = Q}) when ?is_amqqueue(Q) -> - QName = amqqueue:get_name(Q), - rabbit_federation_link_util:log_terminate(Reason, Upstream, UParams, QName), - _ = pg:leave(?FEDERATION_PG_SCOPE, pgname({rabbit_federation_queue, QName}), self()), - ok; - -terminate(Reason, #state{dconn = DConn, - conn = Conn, - upstream = Upstream, - upstream_params = UParams, - queue = Q}) when ?is_amqqueue(Q) -> - QName = amqqueue:get_name(Q), - rabbit_federation_link_util:ensure_connection_closed(DConn), - rabbit_federation_link_util:ensure_connection_closed(Conn), - rabbit_federation_link_util:log_terminate(Reason, Upstream, UParams, QName), - _ = pg:leave(?FEDERATION_PG_SCOPE, pgname({rabbit_federation_queue, QName}), self()), - ok. - -code_change(_OldVsn, State, _Extra) -> - {ok, State}. - -%%---------------------------------------------------------------------------- - -go(S0 = #not_started{run = Run, - upstream = Upstream = #upstream{ - prefetch_count = Prefetch}, - upstream_params = UParams, - queue = Queue}) when ?is_amqqueue(Queue) -> - QName = amqqueue:get_name(Queue), - #upstream_params{x_or_q = UQueue} = UParams, - Durable = amqqueue:is_durable(UQueue), - AutoDelete = amqqueue:is_auto_delete(UQueue), - Args = amqqueue:get_arguments(UQueue), - Unacked = rabbit_federation_link_util:unacked_new(), - rabbit_federation_link_util:start_conn_ch( - fun (Conn, Ch, DConn, DCh) -> - check_upstream_suitable(Conn), - Declare = #'queue.declare'{queue = name(UQueue), - durable = Durable, - auto_delete = AutoDelete, - arguments = Args}, - rabbit_federation_link_util:disposable_channel_call( - Conn, Declare#'queue.declare'{passive = true}, - fun(?NOT_FOUND, _Text) -> - amqp_channel:call(Ch, Declare) - end), - case Upstream#upstream.ack_mode of - 'no-ack' -> ok; - _ -> amqp_channel:call( - Ch, #'basic.qos'{prefetch_count = Prefetch}) - end, - amqp_selective_consumer:register_default_consumer(Ch, self()), - case Run of - true -> consume(Ch, Upstream, UQueue); - false -> ok - end, - {noreply, #state{queue = Queue, - run = Run, - conn = Conn, - ch = Ch, - dconn = DConn, - dch = DCh, - upstream = Upstream, - upstream_params = UParams, - unacked = Unacked}} - end, Upstream, UParams, QName, S0). - -check_upstream_suitable(Conn) -> - Props = pget(server_properties, - amqp_connection:info(Conn, [server_properties])), - {table, Caps} = rabbit_misc:table_lookup(Props, <<"capabilities">>), - case rabbit_misc:table_lookup(Caps, <<"consumer_priorities">>) of - {bool, true} -> ok; - _ -> exit({error, upstream_lacks_consumer_priorities}) - end. - -update_headers(UParams, Redelivered, X, K, undefined) -> - update_headers(UParams, Redelivered, X, K, []); - -update_headers(#upstream_params{table = Table}, Redelivered, X, K, Headers) -> - {Headers1, Count} = - case rabbit_misc:table_lookup(Headers, ?ROUTING_HEADER) of - undefined -> - %% We only want to record the original exchange and - %% routing key the first time a message gets - %% forwarded; after that it's known that they were - %% <<>> and QueueName respectively. - {init_x_original_source_headers(Headers, X, K), 0}; - {array, Been} -> - update_visit_count(Table, Been, Headers); - %% this means the header comes from the client - %% which re-published the message, most likely unintentionally. - %% We can't assume much about the value, so we simply ignore it. - _Other -> - {init_x_original_source_headers(Headers, X, K), 0} - end, - rabbit_basic:prepend_table_header( - ?ROUTING_HEADER, Table ++ [{<<"redelivered">>, bool, Redelivered}, - {<<"visit-count">>, long, Count + 1}], - swap_cc_header(Headers1)). - -init_x_original_source_headers(Headers, X, K) -> - rabbit_misc:set_table_value( - rabbit_misc:set_table_value( - Headers, <<"x-original-exchange">>, longstr, X), - <<"x-original-routing-key">>, longstr, K). - -update_visit_count(Table, Been, Headers) -> - {Found, Been1} = lists:partition( - fun(I) -> visit_match(I, Table) end, - Been), - C = case Found of - [] -> 0; - [{table, T}] -> case rabbit_misc:table_lookup( - T, <<"visit-count">>) of - {_, I} when is_number(I) -> I; - _ -> 0 - end - end, - {rabbit_misc:set_table_value( - Headers, ?ROUTING_HEADER, array, Been1), C}. - -swap_cc_header(Table) -> - [{case K of - <<"CC">> -> <<"x-original-cc">>; - _ -> K - end, T, V} || {K, T, V} <- Table]. - -visit_match({table, T}, Info) -> - lists:all(fun (K) -> - rabbit_misc:table_lookup(T, K) =:= - rabbit_misc:table_lookup(Info, K) - end, [<<"uri">>, <<"virtual_host">>, <<"queue">>]); -visit_match(_ ,_) -> - false. - -consumer_tag(#upstream{consumer_tag = ConsumerTag}) -> - ConsumerTag. - -consume(Ch, Upstream, UQueue) -> - ConsumerTag = consumer_tag(Upstream), - NoAck = Upstream#upstream.ack_mode =:= 'no-ack', - amqp_channel:cast( - Ch, #'basic.consume'{queue = name(UQueue), - no_ack = NoAck, - nowait = true, - consumer_tag = ConsumerTag, - arguments = [{<<"x-priority">>, long, -1}]}). - -cancel(Ch, Upstream) -> - ConsumerTag = consumer_tag(Upstream), - amqp_channel:cast(Ch, #'basic.cancel'{nowait = true, - consumer_tag = ConsumerTag}). - -handle_down(DCh, Reason, _Ch, DCh, Args, State) -> - rabbit_federation_link_util:handle_downstream_down(Reason, Args, State); -handle_down(Ch, Reason, Ch, _DCh, Args, State) -> - rabbit_federation_link_util:handle_upstream_down(Reason, Args, State). 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 deleted file mode 100644 index 108e4cb0f93b..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_queue_link_sup_sup.erl +++ /dev/null @@ -1,97 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_queue_link_sup_sup). - --behaviour(mirrored_supervisor). - --include_lib("rabbit_common/include/rabbit.hrl"). --include_lib("rabbit/include/amqqueue.hrl"). --define(SUPERVISOR, ?MODULE). - -%% Supervises the upstream links for all queues (but not exchanges). We need -%% different handling here since queues do not want a mirrored sup. - --export([start_link/0, start_child/1, adjust/1, stop_child/1]). --export([init/1]). --export([id_to_khepri_path/1]). - -%%---------------------------------------------------------------------------- - -start_link() -> - _ = pg:start_link(), - %% This scope is used by concurrently starting exchange and queue links, - %% and other places, so we have to start it very early outside of the supervision tree. - %% The scope is stopped in stop/1. - _ = rabbit_federation_pg:start_scope(), - mirrored_supervisor:start_link({local, ?SUPERVISOR}, ?SUPERVISOR, - ?MODULE, []). - -%% Note that the next supervisor down, rabbit_federation_link_sup, is common -%% between exchanges and queues. -start_child(Q) -> - case mirrored_supervisor:start_child( - ?SUPERVISOR, - {id(Q), {rabbit_federation_link_sup, start_link, [Q]}, - transient, ?SUPERVISOR_WAIT, supervisor, - [rabbit_federation_link_sup]}) of - {ok, _Pid} -> ok; - {error, {already_started, _Pid}} -> - QueueName = amqqueue:get_name(Q), - rabbit_log_federation:warning("Federation link for queue ~tp was already started", - [rabbit_misc:rs(QueueName)]), - ok; - %% A link returned {stop, gone}, the link_sup shut down, that's OK. - {error, {shutdown, _}} -> ok - end. - - -adjust({clear_upstream, VHost, UpstreamName}) -> - _ = [rabbit_federation_link_sup:adjust(Pid, Q, {clear_upstream, UpstreamName}) || - {Q, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR), - ?amqqueue_vhost_equals(Q, VHost)], - ok; -adjust(Reason) -> - _ = [rabbit_federation_link_sup:adjust(Pid, Q, Reason) || - {Q, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR)], - ok. - -stop_child(Q) -> - case mirrored_supervisor:terminate_child(?SUPERVISOR, id(Q)) of - ok -> ok; - {error, Err} -> - QueueName = amqqueue:get_name(Q), - rabbit_log_federation:warning( - "Attempt to stop a federation link for queue ~tp failed: ~tp", - [rabbit_misc:rs(QueueName), Err]), - ok - end, - _ = mirrored_supervisor:delete_child(?SUPERVISOR, id(Q)). - -%%---------------------------------------------------------------------------- - -init([]) -> - {ok, {{one_for_one, 1200, 60}, []}}. - -%% Clean out all mutable aspects of the queue except policy. We need -%% to keep the entire queue around rather than just take its name -%% since we will want to know its policy to determine how to federate -%% it, and its immutable properties in case we want to redeclare it -%% upstream. We don't just take its name and look it up again since -%% that would introduce race conditions when policies change -%% frequently. Note that since we take down all the links and start -%% again when policies change, the policy will always be correct, so -%% we don't clear it out here and can trust it. -id(Q) when ?is_amqqueue(Q) -> - Policy = amqqueue:get_policy(Q), - Q1 = amqqueue:set_immutable(Q), - Q2 = amqqueue:set_policy(Q1, Policy), - Q2. - -id_to_khepri_path(Id) when ?is_amqqueue(Id) -> - #resource{virtual_host = VHost, name = Name} = amqqueue:get_name(Id), - [queue, VHost, Name]. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_status.erl b/deps/rabbitmq_federation/src/rabbit_federation_status.erl deleted file mode 100644 index a880394eb496..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_status.erl +++ /dev/null @@ -1,178 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_status). --behaviour(gen_server). - --include_lib("amqp_client/include/amqp_client.hrl"). --include("rabbit_federation.hrl"). - --export([start_link/0]). - --export([report/4, remove_exchange_or_queue/1, remove/2, status/0, status/1, lookup/1]). - --export([init/1, handle_call/3, handle_cast/2, handle_info/2, - terminate/2, code_change/3]). - --import(rabbit_federation_util, [name/1]). - --define(SERVER, ?MODULE). --define(ETS_NAME, ?MODULE). - --record(state, {}). --record(entry, {key, uri, status, timestamp, id, supervisor, upstream}). - -start_link() -> - gen_server:start_link({local, ?SERVER}, ?MODULE, [], []). - -report(Upstream, UParams, XorQName, Status) -> - [Supervisor | _] = get('$ancestors'), - gen_server:cast(?SERVER, {report, Supervisor, Upstream, UParams, XorQName, - Status, calendar:local_time()}). - -remove_exchange_or_queue(XorQName) -> - gen_server:call(?SERVER, {remove_exchange_or_queue, XorQName}, infinity). - -remove(Upstream, XorQName) -> - gen_server:call(?SERVER, {remove, Upstream, XorQName}, infinity). - -status() -> - status(infinity). - -status(Timeout) -> - gen_server:call(?SERVER, status, Timeout). - -lookup(Id) -> - gen_server:call(?SERVER, {lookup, Id}, infinity). - -init([]) -> - ?ETS_NAME = ets:new(?ETS_NAME, - [named_table, {keypos, #entry.key}, private]), - {ok, #state{}}. - -handle_call({remove_exchange_or_queue, XorQName}, _From, State) -> - [link_gone(Entry) - || Entry <- ets:match_object(?ETS_NAME, match_entry(xorqkey(XorQName)))], - {reply, ok, State}; - -handle_call({remove, Upstream, XorQName}, _From, State) -> - case ets:match_object(?ETS_NAME, match_entry(key(XorQName, Upstream))) of - [Entry] -> link_gone(Entry); - [] -> ok - end, - {reply, ok, State}; - -handle_call({lookup, Id}, _From, State) -> - Link = case ets:match_object(?ETS_NAME, match_id(Id)) of - [Entry] -> - [{key, Entry#entry.key}, - {uri, Entry#entry.uri}, - {status, Entry#entry.status}, - {timestamp, Entry#entry.timestamp}, - {id, Entry#entry.id}, - {supervisor, Entry#entry.supervisor}, - {upstream, Entry#entry.upstream}]; - [] -> not_found - end, - {reply, Link, State}; - -handle_call(status, _From, State) -> - Entries = ets:tab2list(?ETS_NAME), - {reply, [format(Entry) || Entry <- Entries], State}. - -handle_cast({report, Supervisor, Upstream, #upstream_params{safe_uri = URI}, - XorQName, Status, Timestamp}, State) -> - Key = key(XorQName, Upstream), - Entry = #entry{key = Key, - status = Status, - uri = URI, - timestamp = Timestamp, - supervisor = Supervisor, - upstream = Upstream, - id = unique_id(Key)}, - true = ets:insert(?ETS_NAME, Entry), - rabbit_event:notify(federation_link_status, format(Entry)), - {noreply, State}. - -handle_info(_Info, State) -> - {noreply, State}. - -terminate(_Reason, _State) -> - ok. - -code_change(_OldVsn, State, _Extra) -> - {ok, State}. - -format(#entry{status = Status, - uri = URI, - timestamp = Timestamp} = Entry) -> - identity(Entry) ++ split_status(Status) ++ [{uri, URI}, - {timestamp, Timestamp}]. - -identity(#entry{key = {#resource{virtual_host = VHost, - kind = Type, - name = XorQNameBin}, - UpstreamName, UXorQNameBin}, - id = Id, - upstream = #upstream{consumer_tag = ConsumerTag}}) -> - case Type of - exchange -> [{exchange, XorQNameBin}, - {upstream_exchange, UXorQNameBin}]; - queue -> [{queue, XorQNameBin}, - {upstream_queue, UXorQNameBin}, - {consumer_tag, ConsumerTag}] - end ++ [{type, Type}, - {vhost, VHost}, - {upstream, UpstreamName}, - {id, Id}]. - -unique_id(Key = {#resource{}, UpName, ResName}) when is_binary(UpName), is_binary(ResName) -> - PHash = erlang:phash2(Key, 1 bsl 32), - << << case N >= 10 of - true -> N - 10 + $a; - false -> N + $0 end >> - || <> <= <> >>. - -split_status({running, ConnName}) -> [{status, running}, - {local_connection, ConnName}]; -split_status({Status, Error}) -> [{status, Status}, - {error, Error}]; -split_status(Status) when is_atom(Status) -> [{status, Status}]. - -link_gone(Entry) -> - rabbit_event:notify(federation_link_removed, identity(Entry)), - true = ets:delete_object(?ETS_NAME, Entry). - -%% We don't want to key off the entire upstream, bits of it may change -key(XName = #resource{kind = exchange}, #upstream{name = UpstreamName, - exchange_name = UXNameBin}) -> - {XName, UpstreamName, UXNameBin}; - -key(QName = #resource{kind = queue}, #upstream{name = UpstreamName, - queue_name = UQNameBin}) -> - {QName, UpstreamName, UQNameBin}. - -xorqkey(XorQName) -> - {XorQName, '_', '_'}. - -match_entry(Key) -> - #entry{key = Key, - uri = '_', - status = '_', - timestamp = '_', - id = '_', - supervisor = '_', - upstream = '_'}. - -match_id(Id) -> - #entry{key = '_', - uri = '_', - status = '_', - timestamp = '_', - id = Id, - supervisor = '_', - upstream = '_'}. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_sup.erl b/deps/rabbitmq_federation/src/rabbit_federation_sup.erl deleted file mode 100644 index 5956d6a7c87e..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_sup.erl +++ /dev/null @@ -1,83 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_sup). - --behaviour(supervisor). - -%% Supervises everything. There is just one of these. - --include_lib("rabbit_common/include/rabbit.hrl"). - --define(SUPERVISOR, rabbit_federation_sup). - --export([start_link/0, stop/0]). - --export([init/1]). - -%% This supervisor needs to be part of the rabbit application since -%% a) it needs to be in place when exchange recovery takes place -%% b) it needs to go up and down with rabbit - --rabbit_boot_step({rabbit_federation_supervisor, - [{description, "federation"}, - {mfa, {rabbit_sup, start_child, [?MODULE]}}, - {requires, kernel_ready}, - {cleanup, {?MODULE, stop, []}}, - {enables, rabbit_federation_exchange}, - {enables, rabbit_federation_queue}]}). - -%%---------------------------------------------------------------------------- - -start_link() -> - R = supervisor:start_link({local, ?SUPERVISOR}, ?MODULE, []), - rabbit_federation_event:add_handler(), - R. - -stop() -> - rabbit_federation_event:remove_handler(), - ok = supervisor:terminate_child(rabbit_sup, ?MODULE), - ok = supervisor:delete_child(rabbit_sup, ?MODULE). - -%%---------------------------------------------------------------------------- - -init([]) -> - Status = #{ - id => status, - start => {rabbit_federation_status, start_link, []}, - restart => transient, - shutdown => ?WORKER_WAIT, - type => worker, - modules => [rabbit_federation_status] - }, - XLinkSupSup = #{ - id => x_links, - start => {rabbit_federation_exchange_link_sup_sup, start_link, []}, - restart => transient, - shutdown => ?SUPERVISOR_WAIT, - type => supervisor, - modules =>[rabbit_federation_exchange_link_sup_sup] - }, - QLinkSupSup = #{ - id => q_links, - start => {rabbit_federation_queue_link_sup_sup, start_link, []}, - restart => transient, - shutdown => ?SUPERVISOR_WAIT, - type => supervisor, - modules => [rabbit_federation_queue_link_sup_sup] - }, - %% with default reconnect-delay of 5 second, this supports up to - %% 100 links constantly failing and being restarted a minute - %% (or 200 links if reconnect-delay is 10 seconds, 600 with 30 seconds, - %% etc: N * (60/reconnect-delay) <= 1200) - Flags = #{ - strategy => one_for_one, - intensity => 1200, - period => 60 - }, - Specs = [Status, XLinkSupSup, QLinkSupSup], - {ok, {Flags, Specs}}. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_upstream.erl b/deps/rabbitmq_federation/src/rabbit_federation_upstream.erl deleted file mode 100644 index 1f6b62deda5c..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_upstream.erl +++ /dev/null @@ -1,166 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_upstream). - --include("rabbit_federation.hrl"). --include_lib("rabbit/include/amqqueue.hrl"). --include_lib("amqp_client/include/amqp_client.hrl"). - --export([federate/1, for/1, for/2, params_to_string/1, to_params/2]). -%% For testing --export([from_set/2, from_pattern/2, remove_credentials/1]). - --import(rabbit_misc, [pget/2, pget/3]). --import(rabbit_federation_util, [name/1, vhost/1, r/1]). --import(rabbit_data_coercion, [to_atom/1]). - -%%---------------------------------------------------------------------------- - -federate(XorQ) -> - rabbit_policy:get(<<"federation-upstream">>, XorQ) =/= undefined orelse - rabbit_policy:get(<<"federation-upstream-set">>, XorQ) =/= undefined orelse - rabbit_policy:get(<<"federation-upstream-pattern">>, XorQ) =/= undefined. - -for(XorQ) -> - case federate(XorQ) of - false -> []; - true -> from_set_contents(upstreams(XorQ), XorQ) - end. - -for(XorQ, UpstreamName) -> - case federate(XorQ) of - false -> []; - true -> rabbit_federation_util:find_upstreams( - UpstreamName, from_set_contents(upstreams(XorQ), XorQ)) - end. - -upstreams(XorQ) -> - UName = rabbit_policy:get(<<"federation-upstream">>, XorQ), - USetName = rabbit_policy:get(<<"federation-upstream-set">>, XorQ), - UPatternValue = rabbit_policy:get(<<"federation-upstream-pattern">>, XorQ), - %% Cannot define 2 at a time, see rabbit_federation_parameters:validate_policy/1 - case {UName, USetName, UPatternValue} of - {undefined, undefined, undefined} -> []; - {undefined, undefined, _} -> find_contents(UPatternValue, vhost(XorQ)); - {undefined, _, undefined} -> set_contents(USetName, vhost(XorQ)); - {_, undefined, undefined} -> [[{<<"upstream">>, UName}]] - end. - -params_table(SafeURI, XorQ) -> - Key = case XorQ of - #exchange{} -> <<"exchange">>; - Q when ?is_amqqueue(Q) -> <<"queue">> - end, - [{<<"uri">>, longstr, SafeURI}, - {Key, longstr, name(XorQ)}]. - -params_to_string(#upstream_params{safe_uri = SafeURI, - x_or_q = XorQ}) -> - print("~ts on ~ts", [rabbit_misc:rs(r(XorQ)), SafeURI]). - -remove_credentials(URI) -> - list_to_binary(amqp_uri:remove_credentials(binary_to_list(URI))). - -to_params(Upstream = #upstream{uris = URIs}, XorQ) -> - URI = lists:nth(rand:uniform(length(URIs)), URIs), - {ok, Params} = amqp_uri:parse(binary_to_list(URI), vhost(XorQ)), - XorQ1 = with_name(Upstream, vhost(Params), XorQ), - SafeURI = remove_credentials(URI), - #upstream_params{params = Params, - uri = URI, - x_or_q = XorQ1, - safe_uri = SafeURI, - table = params_table(SafeURI, XorQ)}. - -print(Fmt, Args) -> iolist_to_binary(io_lib:format(Fmt, Args)). - -from_set(SetName, XorQ) -> - from_set_contents(set_contents(SetName, vhost(XorQ)), XorQ). - -from_pattern(SetName, XorQ) -> - from_set_contents(find_contents(SetName, vhost(XorQ)), XorQ). - -set_contents(<<"all">>, VHost) -> - Upstreams0 = rabbit_runtime_parameters:list( - VHost, <<"federation-upstream">>), - Upstreams = [rabbit_data_coercion:to_list(U) || U <- Upstreams0], - [[{<<"upstream">>, pget(name, U)}] || U <- Upstreams]; - -set_contents(SetName, VHost) -> - case rabbit_runtime_parameters:value( - VHost, <<"federation-upstream-set">>, SetName) of - not_found -> []; - Set -> Set - end. - -find_contents(RegExp, VHost) -> - Upstreams0 = rabbit_runtime_parameters:list( - VHost, <<"federation-upstream">>), - Upstreams = [rabbit_data_coercion:to_list(U) || U <- Upstreams0, - re:run(pget(name, U), RegExp) =/= nomatch], - [[{<<"upstream">>, pget(name, U)}] || U <- Upstreams]. - -from_set_contents(Set, XorQ) -> - Results = [from_set_element(P, XorQ) || P <- Set], - [R || R <- Results, R =/= not_found]. - -from_set_element(UpstreamSetElem0, XorQ) -> - UpstreamSetElem = rabbit_data_coercion:to_proplist(UpstreamSetElem0), - Name = bget(upstream, UpstreamSetElem, []), - case rabbit_runtime_parameters:value( - vhost(XorQ), <<"federation-upstream">>, Name) of - not_found -> not_found; - Upstream -> from_upstream_or_set( - UpstreamSetElem, Name, Upstream, XorQ) - end. - -from_upstream_or_set(US, Name, U, XorQ) -> - URIParam = bget(uri, US, U), - URIs = case URIParam of - B when is_binary(B) -> [B]; - L when is_list(L) -> L - end, - #upstream{uris = URIs, - exchange_name = bget(exchange, US, U, name(XorQ)), - queue_name = bget(queue, US, U, name(XorQ)), - consumer_tag = bget('consumer-tag', US, U, <<"federation-link-", Name/binary>>), - prefetch_count = bget('prefetch-count', US, U, ?DEF_PREFETCH), - reconnect_delay = bget('reconnect-delay', US, U, 5), - max_hops = bget('max-hops', US, U, 1), - expires = bget(expires, US, U, none), - message_ttl = bget('message-ttl', US, U, none), - trust_user_id = bget('trust-user-id', US, U, false), - ack_mode = to_atom(bget('ack-mode', US, U, <<"on-confirm">>)), - queue_type = to_atom(bget('queue-type', US, U, <<"classic">>)), - name = Name, - bind_nowait = bget('bind-nowait', US, U, false), - resource_cleanup_mode = to_atom(bget('resource-cleanup-mode', US, U, <<"default">>)), - channel_use_mode = to_atom(bget('channel-use-mode', US, U, multiple)) - }. - -%%---------------------------------------------------------------------------- - -bget(K, L1, L2) -> bget(K, L1, L2, undefined). - -bget(K0, L1, L2, D) -> - K = a2b(K0), - %% coerce maps to proplists - PL1 = rabbit_data_coercion:to_list(L1), - PL2 = rabbit_data_coercion:to_list(L2), - case pget(K, PL1, undefined) of - undefined -> pget(K, PL2, D); - Result -> Result - end. - -a2b(A) -> list_to_binary(atom_to_list(A)). - -with_name(#upstream{exchange_name = XNameBin}, VHostBin, X = #exchange{}) -> - X#exchange{name = rabbit_misc:r(VHostBin, exchange, XNameBin)}; - -with_name(#upstream{queue_name = QNameBin}, VHostBin, Q) when ?is_amqqueue(Q) -> - amqqueue:set_name(Q, rabbit_misc:r(VHostBin, queue, QNameBin)). diff --git a/deps/rabbitmq_federation/src/rabbit_federation_upstream_exchange.erl b/deps/rabbitmq_federation/src/rabbit_federation_upstream_exchange.erl deleted file mode 100644 index b53e4ccfad45..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_upstream_exchange.erl +++ /dev/null @@ -1,90 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_upstream_exchange). - --rabbit_boot_step({?MODULE, - [{description, "federation upstream exchange type"}, - {mfa, {rabbit_registry, register, - [exchange, <<"x-federation-upstream">>, ?MODULE]}}, - {requires, rabbit_registry}, - {cleanup, {rabbit_registry, unregister, - [exchange, <<"x-federation-upstream">>]}}, - {enables, recovery}]}). - --include_lib("rabbit_common/include/rabbit.hrl"). --include("rabbit_federation.hrl"). - --behaviour(rabbit_exchange_type). - --export([description/0, serialise_events/0, route/3]). --export([validate/1, validate_binding/2, - create/2, delete/2, policy_changed/2, - add_binding/3, remove_bindings/3, assert_args_equivalence/2]). --export([info/1, info/2]). - -%%---------------------------------------------------------------------------- - -info(_X) -> []. -info(_X, _) -> []. - -description() -> - [{description, <<"Federation upstream helper exchange">>}, - {internal_purpose, federation}]. - -serialise_events() -> false. - -route(X = #exchange{arguments = Args}, Msg, _Opts) -> - %% This arg was introduced in the same release as this exchange type; - %% it must be set - {long, MaxHops} = rabbit_misc:table_lookup(Args, ?MAX_HOPS_ARG), - %% Will be missing for pre-3.3.0 versions - DName = case rabbit_misc:table_lookup(Args, ?DOWNSTREAM_NAME_ARG) of - {longstr, Val0} -> Val0; - _ -> unknown - end, - %% Will be missing for pre-3.8.9 versions - DVhost = case rabbit_misc:table_lookup(Args, ?DOWNSTREAM_VHOST_ARG) of - {longstr, Val1} -> Val1; - _ -> unknown - end, - case should_forward(Msg, MaxHops, DName, DVhost) of - true -> rabbit_exchange_type_fanout:route(X, Msg); - false -> [] - end. - - -should_forward(Msg, MaxHops, DName, DVhost) -> - case mc:x_header(?ROUTING_HEADER, Msg) of - {list, A} -> - length(A) < MaxHops andalso - not already_seen(DName, DVhost, A); - _ -> - true - end. - -already_seen(DName, DVhost, List) -> - lists:any(fun (Map) -> - {utf8, DName} =:= mc_util:amqp_map_get(<<"cluster-name">>, Map, undefined) andalso - {utf8, DVhost} =:= mc_util:amqp_map_get(<<"vhost">>, Map, undefined) - end, List). - - -validate(#exchange{arguments = Args}) -> - rabbit_federation_util:validate_arg(?MAX_HOPS_ARG, long, Args). - -validate_binding(_X, _B) -> ok. -create(_Serial, _X) -> ok. -delete(_Serial, _X) -> ok. -policy_changed(_X1, _X2) -> ok. -add_binding(_Serial, _X, _B) -> ok. -remove_bindings(_Serial, _X, _Bs) -> ok. - -assert_args_equivalence(X = #exchange{name = Name, - arguments = Args}, ReqArgs) -> - rabbit_misc:assert_args_equivalence(Args, ReqArgs, Name, [?MAX_HOPS_ARG]), - rabbit_exchange:assert_args_equivalence(X, Args). diff --git a/deps/rabbitmq_federation/src/rabbit_federation_util.erl b/deps/rabbitmq_federation/src/rabbit_federation_util.erl deleted file mode 100644 index 64c22c7b679d..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_federation_util.erl +++ /dev/null @@ -1,102 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_util). - --include_lib("rabbit/include/amqqueue.hrl"). --include_lib("amqp_client/include/amqp_client.hrl"). --include("rabbit_federation.hrl"). - --export([should_forward/4, find_upstreams/2, already_seen/3]). --export([validate_arg/3, fail/2, name/1, vhost/1, r/1, pgname/1]). --export([obfuscate_upstream/1, deobfuscate_upstream/1, obfuscate_upstream_params/1, deobfuscate_upstream_params/1]). - --import(rabbit_misc, [pget_or_die/2, pget/3]). - -%%---------------------------------------------------------------------------- - -should_forward(undefined, _MaxHops, _DName, _DVhost) -> - true; -should_forward(Headers, MaxHops, DName, DVhost) -> - case rabbit_misc:table_lookup(Headers, ?ROUTING_HEADER) of - {array, A} -> length(A) < MaxHops andalso not already_seen(DName, DVhost, A); - _ -> true - end. - -%% Used to detect message and binding forwarding cycles. -already_seen(UpstreamID, UpstreamVhost, Array) -> - lists:any(fun ({table, T}) -> - {longstr, UpstreamID} =:= rabbit_misc:table_lookup(T, <<"cluster-name">>) andalso - {longstr, UpstreamVhost} =:= rabbit_misc:table_lookup(T, <<"vhost">>); - (_) -> - false - end, Array). - -find_upstreams(Name, Upstreams) -> - [U || U = #upstream{name = Name2} <- Upstreams, - Name =:= Name2]. - -validate_arg(Name, Type, Args) -> - case rabbit_misc:table_lookup(Args, Name) of - {Type, _} -> ok; - undefined -> fail("Argument ~ts missing", [Name]); - _ -> fail("Argument ~ts must be of type ~ts", [Name, Type]) - end. - --spec fail(io:format(), [term()]) -> no_return(). - -fail(Fmt, Args) -> rabbit_misc:protocol_error(precondition_failed, Fmt, Args). - -name( #resource{name = XorQName}) -> XorQName; -name(#exchange{name = #resource{name = XName}}) -> XName; -name(Q) when ?is_amqqueue(Q) -> #resource{name = QName} = amqqueue:get_name(Q), QName. - -vhost( #resource{virtual_host = VHost}) -> VHost; -vhost(#exchange{name = #resource{virtual_host = VHost}}) -> VHost; -vhost(Q) when ?is_amqqueue(Q) -> #resource{virtual_host = VHost} = amqqueue:get_name(Q), VHost; -vhost(#amqp_params_direct{virtual_host = VHost}) -> VHost; -vhost(#amqp_params_network{virtual_host = VHost}) -> VHost. - -r(#exchange{name = XName}) -> XName; -r(Q) when ?is_amqqueue(Q) -> amqqueue:get_name(Q). - -pgname(Name) -> - case application:get_env(rabbitmq_federation, pgroup_name_cluster_id) of - {ok, false} -> Name; - {ok, true} -> {rabbit_nodes:cluster_name(), Name}; - %% default value is 'false', so do the same thing - {ok, undefined} -> Name; - _ -> Name - end. - -obfuscate_upstream(#upstream{uris = Uris} = Upstream) -> - Upstream#upstream{uris = [credentials_obfuscation:encrypt(Uri) || Uri <- Uris]}. - -obfuscate_upstream_params(#upstream_params{uri = Uri, params = #amqp_params_network{password = Password} = Params} = UParams) -> - UParams#upstream_params{ - uri = credentials_obfuscation:encrypt(Uri), - params = Params#amqp_params_network{password = credentials_obfuscation:encrypt(rabbit_data_coercion:to_binary(Password))} - }; -obfuscate_upstream_params(#upstream_params{uri = Uri, params = #amqp_params_direct{password = Password} = Params} = UParams) -> - UParams#upstream_params{ - uri = credentials_obfuscation:encrypt(Uri), - params = Params#amqp_params_direct{password = credentials_obfuscation:encrypt(rabbit_data_coercion:to_binary(Password))} - }. - -deobfuscate_upstream(#upstream{uris = EncryptedUris} = Upstream) -> - Upstream#upstream{uris = [credentials_obfuscation:decrypt(EncryptedUri) || EncryptedUri <- EncryptedUris]}. - -deobfuscate_upstream_params(#upstream_params{uri = EncryptedUri, params = #amqp_params_network{password = EncryptedPassword} = Params} = UParams) -> - UParams#upstream_params{ - uri = credentials_obfuscation:decrypt(EncryptedUri), - params = Params#amqp_params_network{password = credentials_obfuscation:decrypt(EncryptedPassword)} - }; -deobfuscate_upstream_params(#upstream_params{uri = EncryptedUri, params = #amqp_params_direct{password = EncryptedPassword} = Params} = UParams) -> - UParams#upstream_params{ - uri = credentials_obfuscation:decrypt(EncryptedUri), - params = Params#amqp_params_direct{password = credentials_obfuscation:decrypt(EncryptedPassword)} - }. diff --git a/deps/rabbitmq_federation/src/rabbit_log_federation.erl b/deps/rabbitmq_federation/src/rabbit_log_federation.erl deleted file mode 100644 index 3b7c80d412f4..000000000000 --- a/deps/rabbitmq_federation/src/rabbit_log_federation.erl +++ /dev/null @@ -1,107 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - -%% @doc Compatibility module for the old Lager-based logging API. --module(rabbit_log_federation). - --export([debug/1, debug/2, debug/3, - info/1, info/2, info/3, - notice/1, notice/2, notice/3, - warning/1, warning/2, warning/3, - error/1, error/2, error/3, - critical/1, critical/2, critical/3, - alert/1, alert/2, alert/3, - emergency/1, emergency/2, emergency/3, - none/1, none/2, none/3]). - --include("logging.hrl"). - --compile({no_auto_import, [error/2, error/3]}). - -%%---------------------------------------------------------------------------- - --spec debug(string()) -> 'ok'. --spec debug(string(), [any()]) -> 'ok'. --spec debug(pid() | [tuple()], string(), [any()]) -> 'ok'. --spec info(string()) -> 'ok'. --spec info(string(), [any()]) -> 'ok'. --spec info(pid() | [tuple()], string(), [any()]) -> 'ok'. --spec notice(string()) -> 'ok'. --spec notice(string(), [any()]) -> 'ok'. --spec notice(pid() | [tuple()], string(), [any()]) -> 'ok'. --spec warning(string()) -> 'ok'. --spec warning(string(), [any()]) -> 'ok'. --spec warning(pid() | [tuple()], string(), [any()]) -> 'ok'. --spec error(string()) -> 'ok'. --spec error(string(), [any()]) -> 'ok'. --spec error(pid() | [tuple()], string(), [any()]) -> 'ok'. --spec critical(string()) -> 'ok'. --spec critical(string(), [any()]) -> 'ok'. --spec critical(pid() | [tuple()], string(), [any()]) -> 'ok'. --spec alert(string()) -> 'ok'. --spec alert(string(), [any()]) -> 'ok'. --spec alert(pid() | [tuple()], string(), [any()]) -> 'ok'. --spec emergency(string()) -> 'ok'. --spec emergency(string(), [any()]) -> 'ok'. --spec emergency(pid() | [tuple()], string(), [any()]) -> 'ok'. --spec none(string()) -> 'ok'. --spec none(string(), [any()]) -> 'ok'. --spec none(pid() | [tuple()], string(), [any()]) -> 'ok'. - -%%---------------------------------------------------------------------------- - -debug(Format) -> debug(Format, []). -debug(Format, Args) -> debug(self(), Format, Args). -debug(Pid, Format, Args) -> - logger:debug(Format, Args, #{pid => Pid, - domain => ?RMQLOG_DOMAIN_FEDERATION}). - -info(Format) -> info(Format, []). -info(Format, Args) -> info(self(), Format, Args). -info(Pid, Format, Args) -> - logger:info(Format, Args, #{pid => Pid, - domain => ?RMQLOG_DOMAIN_FEDERATION}). - -notice(Format) -> notice(Format, []). -notice(Format, Args) -> notice(self(), Format, Args). -notice(Pid, Format, Args) -> - logger:notice(Format, Args, #{pid => Pid, - domain => ?RMQLOG_DOMAIN_FEDERATION}). - -warning(Format) -> warning(Format, []). -warning(Format, Args) -> warning(self(), Format, Args). -warning(Pid, Format, Args) -> - logger:warning(Format, Args, #{pid => Pid, - domain => ?RMQLOG_DOMAIN_FEDERATION}). - -error(Format) -> error(Format, []). -error(Format, Args) -> error(self(), Format, Args). -error(Pid, Format, Args) -> - logger:error(Format, Args, #{pid => Pid, - domain => ?RMQLOG_DOMAIN_FEDERATION}). - -critical(Format) -> critical(Format, []). -critical(Format, Args) -> critical(self(), Format, Args). -critical(Pid, Format, Args) -> - logger:critical(Format, Args, #{pid => Pid, - domain => ?RMQLOG_DOMAIN_FEDERATION}). - -alert(Format) -> alert(Format, []). -alert(Format, Args) -> alert(self(), Format, Args). -alert(Pid, Format, Args) -> - logger:alert(Format, Args, #{pid => Pid, - domain => ?RMQLOG_DOMAIN_FEDERATION}). - -emergency(Format) -> emergency(Format, []). -emergency(Format, Args) -> emergency(self(), Format, Args). -emergency(Pid, Format, Args) -> - logger:emergency(Format, Args, #{pid => Pid, - domain => ?RMQLOG_DOMAIN_FEDERATION}). - -none(_Format) -> ok. -none(_Format, _Args) -> ok. -none(_Pid, _Format, _Args) -> ok. diff --git a/deps/rabbitmq_federation/src/rabbitmq_federation_noop.erl b/deps/rabbitmq_federation/src/rabbitmq_federation_noop.erl new file mode 100644 index 000000000000..708d7b84fe9b --- /dev/null +++ b/deps/rabbitmq_federation/src/rabbitmq_federation_noop.erl @@ -0,0 +1 @@ +-module(rabbitmq_federation_noop). diff --git a/deps/rabbitmq_federation/test/definition_import_SUITE.erl b/deps/rabbitmq_federation/test/definition_import_SUITE.erl deleted file mode 100644 index d423849090ed..000000000000 --- a/deps/rabbitmq_federation/test/definition_import_SUITE.erl +++ /dev/null @@ -1,146 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(definition_import_SUITE). - --include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). --include_lib("common_test/include/ct.hrl"). --include_lib("eunit/include/eunit.hrl"). - --compile(export_all). - -all() -> - [ - {group, roundtrip} - ]. - -groups() -> - [ - {roundtrip, [], [ - export_import_round_trip - ]} - ]. - -%% ------------------------------------------------------------------- -%% Test suite setup/teardown. -%% ------------------------------------------------------------------- - -init_per_suite(Config) -> - rabbit_ct_helpers:log_environment(), - inets:start(), - Config. -end_per_suite(Config) -> - Config. - -init_per_group(Group, Config) -> - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodename_suffix, Group} - ]), - rabbit_ct_helpers:run_setup_steps(Config1, rabbit_ct_broker_helpers:setup_steps()). - -end_per_group(_, Config) -> - rabbit_ct_helpers:run_teardown_steps(Config, 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). - -%% -%% Tests -%% - -export_import_round_trip(Config) -> - case rabbit_ct_helpers:is_mixed_versions() of - false -> - import_file_case(Config, "case1"), - Defs = export(Config), - import_raw(Config, rabbit_json:encode(Defs)); - _ -> - %% skip the test in mixed version mode - {skip, "Should not run in mixed version environments"} - end. - -%% -%% Implementation -%% - -import_file_case(Config, CaseName) -> - CasePath = filename:join([ - ?config(data_dir, Config), - CaseName ++ ".json" - ]), - rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, run_import_case, [CasePath]), - ok. - - -import_raw(Config, Body) -> - case rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_definitions, import_raw, [Body]) of - ok -> ok; - {error, E} -> - ct:pal("Import of JSON definitions ~tp failed: ~tp~n", [Body, E]), - ct:fail({expected_failure, Body, E}) - end. - -export(Config) -> - rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, run_export, []). - -run_export() -> - rabbit_definitions:all_definitions(). - -run_directory_import_case(Path, Expected) -> - ct:pal("Will load definitions from files under ~tp~n", [Path]), - Result = rabbit_definitions:maybe_load_definitions_from(true, Path), - case Expected of - ok -> - ok = Result; - error -> - ?assertMatch({error, {failed_to_import_definitions, _, _}}, Result) - end. - -run_import_case(Path) -> - {ok, Body} = file:read_file(Path), - ct:pal("Successfully loaded a definition to import from ~tp~n", [Path]), - case rabbit_definitions:import_raw(Body) of - ok -> ok; - {error, E} -> - ct:pal("Import case ~tp failed: ~tp~n", [Path, E]), - ct:fail({expected_failure, Path, E}) - end. - -run_invalid_import_case(Path) -> - {ok, Body} = file:read_file(Path), - ct:pal("Successfully loaded a definition file at ~tp~n", [Path]), - case rabbit_definitions:import_raw(Body) of - ok -> - ct:pal("Expected import case ~tp to fail~n", [Path]), - ct:fail({expected_failure, Path}); - {error, _E} -> ok - end. - -run_invalid_import_case_if_unchanged(Path) -> - Mod = rabbit_definitions_import_local_filesystem, - ct:pal("Successfully loaded a definition to import from ~tp~n", [Path]), - case rabbit_definitions:maybe_load_definitions_from_local_filesystem_if_unchanged(Mod, false, Path) of - ok -> - ct:pal("Expected import case ~tp to fail~n", [Path]), - ct:fail({expected_failure, Path}); - {error, _E} -> ok - end. - -queue_lookup(Config, VHost, Name) -> - rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_amqqueue, lookup, [rabbit_misc:r(VHost, queue, Name)]). - -vhost_lookup(Config, VHost) -> - rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_vhost, lookup, [VHost]). - -user_lookup(Config, User) -> - rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_auth_backend_internal, lookup_user, [User]). - -delete_vhost(Config, VHost) -> - rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_vhost, delete, [VHost, <<"CT tests">>]). diff --git a/deps/rabbitmq_federation/test/definition_import_SUITE_data/case1.json b/deps/rabbitmq_federation/test/definition_import_SUITE_data/case1.json deleted file mode 100644 index e549e4fd6c1d..000000000000 --- a/deps/rabbitmq_federation/test/definition_import_SUITE_data/case1.json +++ /dev/null @@ -1,52 +0,0 @@ -{ - "permissions": [ - { - "configure": ".*", - "read": ".*", - "user": "guest", - "vhost": "/", - "write": ".*" - } - ], - "bindings": [], - "queues": [], - "parameters": [ - { - "component": "federation-upstream-set", - "name": "location-1", - "value": [ - { - "upstream":"up-1" - }, - { - "upstream":"up-2" - } - ], - "vhost":"/"}], - "policies": [], - "rabbitmq_version": "3.13.0+376.g1bc0d89.dirty", - "users": [ - { - "hashing_algorithm": "rabbit_password_hashing_sha256", - "limits": {}, - "name": "guest", - "password_hash": "jTcCKuOmGJeeRQ/K1LG5sdZLcdnEnqv8wcrP2n68R7nMuqy2", - "tags": ["administrator"] - } - ], - "rabbit_version": "3.13.0+376.g1bc0d89.dirty", - "exchanges": [], - "topic_permissions": [], - "vhosts": [ - { - "limits": [], - "metadata": - { - "description": "Default virtual host", - "tags": [] - }, - "name":"/" - } - ], - "global_parameters": [] -} diff --git a/deps/rabbitmq_federation/test/exchange_SUITE.erl b/deps/rabbitmq_federation/test/exchange_SUITE.erl deleted file mode 100644 index 58d617b5def1..000000000000 --- a/deps/rabbitmq_federation/test/exchange_SUITE.erl +++ /dev/null @@ -1,920 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(exchange_SUITE). - --include_lib("eunit/include/eunit.hrl"). --include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). --include_lib("amqp_client/include/amqp_client.hrl"). - --include("rabbit_federation.hrl"). - --compile(export_all). - --import(rabbit_federation_test_util, - [expect/3, expect/4, expect_empty/2, - set_upstream/4, set_upstream/5, set_upstream_in_vhost/5, set_upstream_in_vhost/6, - clear_upstream/3, set_upstream_set/4, - set_policy/5, set_policy_pattern/5, clear_policy/3, - set_policy_upstream/5, set_policy_upstreams/4, - all_federation_links/2, federation_links_in_vhost/3, status_fields/2]). - --import(rabbit_ct_broker_helpers, - [set_policy_in_vhost/7]). - -all() -> - [ - {group, essential}, - {group, cluster_size_3}, - {group, rolling_upgrade} - ]. - -groups() -> - [ - {essential, [], essential()}, - {cluster_size_3, [], [max_hops]}, - {rolling_upgrade, [], [child_id_format]}, - {cycle_protection, [], [ - %% TBD: port from v3.10.x in an Erlang 25-compatible way - ]}, - {channel_use_mod_single, [], [ - %% TBD: port from v3.10.x in an Erlang 25-compatible way - ]} - ]. - -essential() -> - [ - single_upstream, - single_upstream_quorum, - multiple_upstreams, - multiple_upstreams_pattern, - single_upstream_multiple_uris, - multiple_downstreams, - e2e_binding, - unbind_on_delete, - unbind_on_client_unbind, - exchange_federation_link_status, - lookup_exchange_status - ]. - -suite() -> - [{timetrap, {minutes, 3}}]. - -%% ------------------------------------------------------------------- -%% 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). - -%% Some of the "regular" tests but in the single channel mode. -init_per_group(essential, Config) -> - SetupFederation = [ - fun(Config1) -> - rabbit_federation_test_util:setup_federation_with_upstream_params(Config1, [ - {<<"channel-use-mode">>, <<"single">>} - ]) - end - ], - Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"), - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodename_suffix, Suffix}, - {rmq_nodes_count, 1} - ]), - rabbit_ct_helpers:run_steps(Config1, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps() ++ - SetupFederation); -init_per_group(cluster_size_3 = Group, Config) -> - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodes_count, 3} - ]), - init_per_group1(Group, Config1); -init_per_group(rolling_upgrade = Group, Config) -> - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodes_count, 5}, - {rmq_nodes_clustered, false} - ]), - init_per_group1(Group, Config1); -init_per_group(Group, Config) -> - init_per_group1(Group, Config). - - -init_per_group1(_Group, Config) -> - Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"), - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodename_suffix, Suffix}, - {rmq_nodes_clustered, false} - ]), - rabbit_ct_helpers:run_steps(Config1, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()). - -end_per_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 -%% - -single_upstream(Config) -> - FedX = <<"single_upstream.federated">>, - UpX = <<"single_upstream.upstream.x">>, - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"localhost">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, - {<<"exchange">>, UpX} - ]), - rabbit_ct_broker_helpers:set_policy( - Config, 0, - <<"fed.x">>, <<"^single_upstream.federated">>, <<"exchanges">>, - [ - {<<"federation-upstream">>, <<"localhost">>} - ]), - - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), - - Xs = [ - exchange_declare_method(FedX) - ], - declare_exchanges(Ch, Xs), - - RK = <<"key">>, - Q = declare_and_bind_queue(Ch, FedX, RK), - await_binding(Config, 0, UpX, RK), - publish_expect(Ch, UpX, RK, Q, <<"single_upstream payload">>), - - Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), - assert_federation_internal_queue_type(Config, Server, rabbit_classic_queue), - - rabbit_ct_client_helpers:close_channel(Ch), - clean_up_federation_related_bits(Config). - -single_upstream_quorum(Config) -> - FedX = <<"single_upstream_quorum.federated">>, - UpX = <<"single_upstream_quorum.upstream.x">>, - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"localhost">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, - {<<"exchange">>, UpX}, - {<<"queue-type">>, <<"quorum">>} - ]), - rabbit_ct_broker_helpers:set_policy( - Config, 0, - <<"fed.x">>, <<"^single_upstream_quorum.federated">>, <<"exchanges">>, - [ - {<<"federation-upstream">>, <<"localhost">>} - ]), - - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), - - Xs = [ - exchange_declare_method(FedX) - ], - declare_exchanges(Ch, Xs), - - RK = <<"key">>, - Q = declare_and_bind_queue(Ch, FedX, RK), - await_binding(Config, 0, UpX, RK), - publish_expect(Ch, UpX, RK, Q, <<"single_upstream_quorum payload">>), - - Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), - assert_federation_internal_queue_type(Config, Server, rabbit_quorum_queue), - - rabbit_ct_client_helpers:close_channel(Ch), - clean_up_federation_related_bits(Config). - -multiple_upstreams(Config) -> - FedX = <<"multiple_upstreams.federated">>, - UpX1 = <<"upstream.x.1">>, - UpX2 = <<"upstream.x.2">>, - set_up_upstreams(Config), - rabbit_ct_broker_helpers:set_policy( - Config, 0, - <<"fed.x">>, <<"^multiple_upstreams.federated">>, <<"exchanges">>, - [ - {<<"federation-upstream-set">>, <<"all">>} - ]), - - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), - Xs = [ - exchange_declare_method(FedX) - ], - declare_exchanges(Ch, Xs), - - RK = <<"multiple_upstreams.key">>, - Q = declare_and_bind_queue(Ch, FedX, RK), - await_binding(Config, 0, UpX1, RK), - await_binding(Config, 0, UpX2, RK), - publish_expect(Ch, UpX1, RK, Q, <<"multiple_upstreams payload">>), - publish_expect(Ch, UpX2, RK, Q, <<"multiple_upstreams payload">>), - - rabbit_ct_client_helpers:close_channel(Ch), - clean_up_federation_related_bits(Config). - - -multiple_upstreams_pattern(Config) -> - FedX = <<"multiple_upstreams_pattern.federated">>, - UpX1 = <<"upstream.x.1">>, - UpX2 = <<"upstream.x.2">>, - set_up_upstreams(Config), - rabbit_ct_broker_helpers:set_policy( - Config, 0, - <<"fed.x">>, <<"^multiple_upstreams_pattern.federated">>, <<"exchanges">>, - [ - {<<"federation-upstream-pattern">>, <<"^localhost">>} - ]), - - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), - Xs = [ - exchange_declare_method(FedX) - ], - declare_exchanges(Ch, Xs), - - RK = <<"multiple_upstreams_pattern.key">>, - Q = declare_and_bind_queue(Ch, FedX, RK), - await_binding(Config, 0, UpX1, RK), - await_binding(Config, 0, UpX2, RK), - publish_expect(Ch, UpX1, RK, Q, <<"multiple_upstreams_pattern payload">>), - publish_expect(Ch, UpX2, RK, Q, <<"multiple_upstreams_pattern payload">>), - - rabbit_ct_client_helpers:close_channel(Ch), - clean_up_federation_related_bits(Config). - - -single_upstream_multiple_uris(Config) -> - FedX = <<"single_upstream_multiple_uris.federated">>, - UpX = <<"single_upstream_multiple_uris.upstream.x">>, - URIs = [ - rabbit_ct_broker_helpers:node_uri(Config, 0), - rabbit_ct_broker_helpers:node_uri(Config, 0, [use_ipaddr]) - ], - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"localhost">>, - [ - {<<"uri">>, URIs}, - {<<"exchange">>, UpX} - ]), - rabbit_ct_broker_helpers:set_policy( - Config, 0, - <<"fed.x">>, <<"^single_upstream_multiple_uris.federated">>, <<"exchanges">>, - [ - {<<"federation-upstream">>, <<"localhost">>} - ]), - - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), - - Xs = [ - exchange_declare_method(FedX) - ], - declare_exchanges(Ch, Xs), - - RK = <<"key">>, - Q = declare_and_bind_queue(Ch, FedX, RK), - await_binding(Config, 0, UpX, RK), - publish_expect(Ch, UpX, RK, Q, <<"single_upstream_multiple_uris payload">>), - - rabbit_ct_client_helpers:close_channel(Ch), - clean_up_federation_related_bits(Config). - -multiple_downstreams(Config) -> - FedX = <<"multiple_downstreams.federated">>, - UpX = <<"multiple_downstreams.upstream.x">>, - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"localhost">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, - {<<"exchange">>, UpX} - ]), - rabbit_ct_broker_helpers:set_policy( - Config, 0, - <<"fed.x">>, <<"^multiple_downstreams.federated">>, <<"exchanges">>, - [ - {<<"federation-upstream">>, <<"localhost">>} - ]), - - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), - - Xs = [ - exchange_declare_method(FedX) - ], - declare_exchanges(Ch, Xs), - - RK = <<"key">>, - Q1 = declare_and_bind_queue(Ch, FedX, RK), - _ = declare_and_bind_queue(Ch, FedX, RK), - await_binding(Config, 0, UpX, RK), - publish(Ch, UpX, RK, <<"multiple_downstreams payload 1">>), - publish(Ch, UpX, RK, <<"multiple_downstreams payload 2">>), - expect(Ch, Q1, [<<"multiple_downstreams payload 1">>]), - expect(Ch, Q1, [<<"multiple_downstreams payload 2">>]), - - rabbit_ct_client_helpers:close_channel(Ch), - clean_up_federation_related_bits(Config). - -e2e_binding(Config) -> - FedX = <<"e2e_binding.federated">>, - E2EX = <<"e2e_binding.e2e">>, - UpX = <<"e2e_binding.upstream.x">>, - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"localhost">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, - {<<"exchange">>, UpX} - ]), - rabbit_ct_broker_helpers:set_policy( - Config, 0, - <<"fed.x">>, <<"^e2e_binding.federated">>, <<"exchanges">>, - [ - {<<"federation-upstream">>, <<"localhost">>} - ]), - - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), - - Xs = [ - exchange_declare_method(FedX, <<"fanout">>), - exchange_declare_method(E2EX, <<"fanout">>) - ], - declare_exchanges(Ch, Xs), - Key = <<"key">>, - %% federated exchange routes to the E2E fanout - bind_exchange(Ch, E2EX, FedX, Key), - - RK = <<"key">>, - Q = declare_and_bind_queue(Ch, E2EX, RK), - await_binding(Config, 0, UpX, RK), - publish_expect(Ch, UpX, RK, Q, <<"e2e_binding payload">>), - - rabbit_ct_client_helpers:close_channel(Ch), - clean_up_federation_related_bits(Config). - -unbind_on_delete(Config) -> - FedX = <<"unbind_on_delete.federated">>, - UpX = <<"unbind_on_delete.upstream.x">>, - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"localhost">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, - {<<"exchange">>, UpX} - ]), - rabbit_ct_broker_helpers:set_policy( - Config, 0, - <<"fed.x">>, <<"^unbind_on_delete.federated">>, <<"exchanges">>, - [ - {<<"federation-upstream">>, <<"localhost">>} - ]), - - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), - - Xs = [ - exchange_declare_method(FedX) - ], - declare_exchanges(Ch, Xs), - - RK = <<"key">>, - Q1 = declare_and_bind_queue(Ch, FedX, RK), - Q2 = declare_and_bind_queue(Ch, FedX, RK), - await_binding(Config, 0, UpX, RK), - delete_queue(Ch, Q2), - publish_expect(Ch, UpX, RK, Q1, <<"unbind_on_delete payload">>), - - rabbit_ct_client_helpers:close_channel(Ch), - clean_up_federation_related_bits(Config). - -unbind_on_client_unbind(Config) -> - FedX = <<"unbind_on_client_unbind.federated">>, - UpX = <<"unbind_on_client_unbind.upstream.x">>, - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"localhost">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, - {<<"exchange">>, UpX} - ]), - rabbit_ct_broker_helpers:set_policy( - Config, 0, - <<"fed.x">>, <<"^unbind_on_client_unbind.federated">>, <<"exchanges">>, - [ - {<<"federation-upstream">>, <<"localhost">>} - ]), - - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), - - Xs = [ - exchange_declare_method(FedX) - ], - declare_exchanges(Ch, Xs), - - RK = <<"key">>, - Q1 = declare_and_bind_queue(Ch, FedX, RK), - Q2 = declare_and_bind_queue(Ch, FedX, RK), - await_binding(Config, 0, UpX, RK), - unbind_queue(Ch, Q2, UpX, RK), - publish_expect(Ch, UpX, RK, Q1, <<"unbind_on_delete payload">>), - - rabbit_ct_client_helpers:close_channel(Ch), - clean_up_federation_related_bits(Config). - -max_hops(Config) -> - case rabbit_ct_helpers:is_mixed_versions() of - false -> - [NodeA, NodeB, NodeC] = rabbit_ct_broker_helpers:get_node_configs( - Config, nodename), - await_credentials_obfuscation_seeding_on_two_nodes(Config), - - UpX = <<"ring">>, - - %% form of ring of upstreams, - %% A upstream points at B - rabbit_ct_broker_helpers:set_parameter( - Config, NodeA, <<"federation-upstream">>, <<"upstream">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, NodeB)}, - {<<"exchange">>, UpX}, - {<<"max-hops">>, 2} - ]), - %% B upstream points at C - rabbit_ct_broker_helpers:set_parameter( - Config, NodeB, <<"federation-upstream">>, <<"upstream">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, NodeC)}, - {<<"exchange">>, UpX}, - {<<"max-hops">>, 2} - ]), - %% C upstream points at A - rabbit_ct_broker_helpers:set_parameter( - Config, NodeC, <<"federation-upstream">>, <<"upstream">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, NodeA)}, - {<<"exchange">>, UpX}, - {<<"max-hops">>, 2} - ]), - - %% policy on A - [begin - rabbit_ct_broker_helpers:set_policy( - Config, Node, - <<"fed.x">>, <<"^ring">>, <<"exchanges">>, - [ - {<<"federation-upstream">>, <<"upstream">>} - ]) - end || Node <- [NodeA, NodeB, NodeC]], - - NodeACh = rabbit_ct_client_helpers:open_channel(Config, NodeA), - NodeBCh = rabbit_ct_client_helpers:open_channel(Config, NodeB), - NodeCCh = rabbit_ct_client_helpers:open_channel(Config, NodeC), - - FedX = <<"ring">>, - X = exchange_declare_method(FedX), - declare_exchange(NodeACh, X), - declare_exchange(NodeBCh, X), - declare_exchange(NodeCCh, X), - - Q1 = declare_and_bind_queue(NodeACh, <<"ring">>, <<"key">>), - Q2 = declare_and_bind_queue(NodeBCh, <<"ring">>, <<"key">>), - Q3 = declare_and_bind_queue(NodeCCh, <<"ring">>, <<"key">>), - - await_binding(Config, NodeA, <<"ring">>, <<"key">>, 3), - await_binding(Config, NodeB, <<"ring">>, <<"key">>, 3), - await_binding(Config, NodeC, <<"ring">>, <<"key">>, 3), - - publish(NodeACh, <<"ring">>, <<"key">>, <<"HELLO flopsy">>), - publish(NodeBCh, <<"ring">>, <<"key">>, <<"HELLO mopsy">>), - publish(NodeCCh, <<"ring">>, <<"key">>, <<"HELLO cottontail">>), - - Msgs = [<<"HELLO flopsy">>, <<"HELLO mopsy">>, <<"HELLO cottontail">>], - expect(NodeACh, Q1, Msgs), - expect(NodeBCh, Q2, Msgs), - expect(NodeCCh, Q3, Msgs), - expect_empty(NodeACh, Q1), - expect_empty(NodeBCh, Q2), - expect_empty(NodeCCh, Q3), - - clean_up_federation_related_bits(Config); - true -> - %% skip the test in mixed version mode - {skip, "Should not run in mixed version environments"} - end. - -exchange_federation_link_status(Config) -> - FedX = <<"single_upstream.federated">>, - UpX = <<"single_upstream.upstream.x">>, - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"localhost">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, - {<<"exchange">>, UpX} - ]), - rabbit_ct_broker_helpers:set_policy( - Config, 0, - <<"fed.x">>, <<"^single_upstream.federated">>, <<"exchanges">>, - [ - {<<"federation-upstream">>, <<"localhost">>} - ]), - - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), - - Xs = [ - exchange_declare_method(FedX) - ], - declare_exchanges(Ch, Xs), - - RK = <<"key">>, - _ = declare_and_bind_queue(Ch, FedX, RK), - await_binding(Config, 0, UpX, RK), - - [Link] = rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_federation_status, status, []), - true = is_binary(proplists:get_value(id, Link)), - - clean_up_federation_related_bits(Config). - -lookup_exchange_status(Config) -> - FedX = <<"single_upstream.federated">>, - UpX = <<"single_upstream.upstream.x">>, - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"localhost">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, - {<<"exchange">>, UpX} - ]), - rabbit_ct_broker_helpers:set_policy( - Config, 0, - <<"fed.x">>, <<"^single_upstream.federated">>, <<"exchanges">>, - [ - {<<"federation-upstream">>, <<"localhost">>} - ]), - - Ch = rabbit_ct_client_helpers:open_channel(Config, 0), - - Xs = [ - exchange_declare_method(FedX) - ], - declare_exchanges(Ch, Xs), - - RK = <<"key">>, - _ = declare_and_bind_queue(Ch, FedX, RK), - await_binding(Config, 0, UpX, RK), - - [Link] = rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_federation_status, status, []), - Id = proplists:get_value(id, Link), - Props = rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_federation_status, lookup, [Id]), - lists:all(fun(K) -> lists:keymember(K, 1, Props) end, - [key, uri, status, timestamp, id, supervisor, upstream]), - - clean_up_federation_related_bits(Config). - -child_id_format(Config) -> - [UpstreamNode, - OldNodeA, - NewNodeB, - OldNodeC, - NewNodeD] = rabbit_ct_broker_helpers:get_node_configs( - Config, nodename), - - %% Create a cluster with the nodes running the old version of RabbitMQ in - %% mixed-version testing. - %% - %% Note: we build this on the assumption that `rabbit_ct_broker_helpers' - %% starts nodes this way: - %% Node 1: the primary copy of RabbitMQ the test is started from - %% Node 2: the secondary umbrella (if any) - %% Node 3: the primary copy - %% Node 4: the secondary umbrella - %% ... - %% - %% Therefore, `UpstreamNode' will use the primary copy, `OldNodeA' the - %% secondary umbrella, `NewNodeB' the primary copy, and so on. - Config1 = rabbit_ct_broker_helpers:cluster_nodes( - Config, [OldNodeA, OldNodeC]), - - %% Prepare the whole federated exchange on that old cluster. - UpstreamName = <<"fed_on_upgrade">>, - rabbit_ct_broker_helpers:set_parameter( - Config1, OldNodeA, <<"federation-upstream">>, UpstreamName, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config1, UpstreamNode)} - ]), - - rabbit_ct_broker_helpers:set_policy( - Config1, OldNodeA, - <<"fed_on_upgrade_policy">>, <<"^fed_">>, <<"all">>, - [ - {<<"federation-upstream-pattern">>, UpstreamName} - ]), - - XName = <<"fed_ex_on_upgrade_cluster">>, - X = exchange_declare_method(XName, <<"direct">>), - {Conn1, Ch1} = rabbit_ct_client_helpers:open_connection_and_channel( - Config1, OldNodeA), - ?assertEqual({'exchange.declare_ok'}, declare_exchange(Ch1, X)), - rabbit_ct_client_helpers:close_channel(Ch1), - rabbit_ct_client_helpers:close_connection(Conn1), - - %% Verify the format of the child ID. In the main branch, the format was - %% temporarily a size-2 tuple with a list as the first element. This was - %% not kept later and the original ID format is used in old and new nodes. - [{Id, _, _, _}] = rabbit_ct_broker_helpers:rpc( - Config1, OldNodeA, - mirrored_supervisor, which_children, - [rabbit_federation_exchange_link_sup_sup]), - case Id of - %% This is the format we expect everywhere. - #exchange{name = #resource{name = XName}} -> - %% Verify that the supervisors exist on all nodes. - lists:foreach( - fun(Node) -> - ?assertMatch( - [{#exchange{name = #resource{name = XName}}, - _, _, _}], - rabbit_ct_broker_helpers:rpc( - Config1, Node, - mirrored_supervisor, which_children, - [rabbit_federation_exchange_link_sup_sup])) - end, [OldNodeA, OldNodeC]), - - %% Simulate a rolling upgrade by: - %% 1. adding new nodes to the old cluster - %% 2. stopping the old nodes - %% - %% After that, the supervisors run on the new code. - Config2 = rabbit_ct_broker_helpers:cluster_nodes( - Config1, OldNodeA, [NewNodeB, NewNodeD]), - ok = rabbit_ct_broker_helpers:stop_broker(Config2, OldNodeA), - ok = rabbit_ct_broker_helpers:reset_node(Config1, OldNodeA), - ok = rabbit_ct_broker_helpers:stop_broker(Config2, OldNodeC), - ok = rabbit_ct_broker_helpers:reset_node(Config2, OldNodeC), - - %% Verify that the supervisors still use the same IDs. - lists:foreach( - fun(Node) -> - ?assertMatch( - [{#exchange{name = #resource{name = XName}}, - _, _, _}], - rabbit_ct_broker_helpers:rpc( - Config2, Node, - mirrored_supervisor, which_children, - [rabbit_federation_exchange_link_sup_sup])) - end, [NewNodeB, NewNodeD]), - - %% Delete the exchange: it should work because the ID format is the - %% one expected. - %% - %% During the transient period where the ID format was changed, - %% this would crash with a badmatch because the running - %% supervisor's ID would not match the content of the database. - {Conn2, Ch2} = rabbit_ct_client_helpers:open_connection_and_channel( - Config2, NewNodeB), - ?assertEqual({'exchange.delete_ok'}, delete_exchange(Ch2, XName)), - rabbit_ct_client_helpers:close_channel(Ch2), - rabbit_ct_client_helpers:close_connection(Conn2); - - %% This is the transient format we are not interested in as it only - %% lived in a development branch. - {List, #exchange{name = #resource{name = XName}}} - when is_list(List) -> - {skip, "Testcase skipped with the transiently changed ID format"} - end. - -%% -%% Test helpers -%% - -clean_up_federation_related_bits(Config) -> - delete_all_queues_on(Config, 0), - delete_all_exchanges_on(Config, 0), - delete_all_policies_on(Config, 0), - delete_all_runtime_parameters_on(Config, 0). - -set_up_upstream(Config) -> - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"localhost">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, - {<<"exchange">>, <<"upstream">>} - ]). - -set_up_upstreams(Config) -> - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"localhost1">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, - {<<"exchange">>, <<"upstream.x.1">>} - ]), - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"localhost2">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, - {<<"exchange">>, <<"upstream.x.2">>} - ]). - -set_up_upstreams_including_unavailable(Config) -> - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"unavailable-node">>, - [ - {<<"uri">>, <<"amqp://unavailable-node">>}, - {<<"reconnect-delay">>, 600000} - ]), - - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"localhost">>, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)} - ]). - -declare_exchanges(Ch, Frames) -> - [declare_exchange(Ch, F) || F <- Frames]. -delete_exchanges(Ch, Frames) -> - [delete_exchange(Ch, X) || #'exchange.declare'{exchange = X} <- Frames]. - -declare_exchange(Ch, X) -> - amqp_channel:call(Ch, X). - -declare_queue(Ch) -> - #'queue.declare_ok'{queue = Q} = - amqp_channel:call(Ch, #'queue.declare'{exclusive = true}), - Q. - -declare_queue(Ch, Q) -> - amqp_channel:call(Ch, Q). - -bind_queue(Ch, Q, X, Key) -> - amqp_channel:call(Ch, #'queue.bind'{queue = Q, - exchange = X, - routing_key = Key}). - -unbind_queue(Ch, Q, X, Key) -> - amqp_channel:call(Ch, #'queue.unbind'{queue = Q, - exchange = X, - routing_key = Key}). - -bind_exchange(Ch, D, S, Key) -> - amqp_channel:call(Ch, #'exchange.bind'{destination = D, - source = S, - routing_key = Key}). - -declare_and_bind_queue(Ch, X, Key) -> - Q = declare_queue(Ch), - bind_queue(Ch, Q, X, Key), - Q. - - -delete_exchange(Ch, XName) -> - amqp_channel:call(Ch, #'exchange.delete'{exchange = XName}). - -delete_queue(Ch, QName) -> - amqp_channel:call(Ch, #'queue.delete'{queue = QName}). - -exchange_declare_method(Name) -> - exchange_declare_method(Name, <<"topic">>). - -exchange_declare_method(Name, Type) -> - #'exchange.declare'{exchange = Name, - type = Type, - durable = true}. - -delete_all_queues_on(Config, Node) -> - [rabbit_ct_broker_helpers:rpc( - Config, Node, rabbit_amqqueue, delete, [Q, false, false, - <<"acting-user">>]) || - Q <- all_queues_on(Config, Node)]. - -delete_all_exchanges_on(Config, Node) -> - [rabbit_ct_broker_helpers:rpc( - Config, Node, rabbit_exchange, delete, [X, false, - <<"acting-user">>]) || - #exchange{name = X} <- all_exchanges_on(Config, Node)]. - -delete_all_policies_on(Config, Node) -> - [rabbit_ct_broker_helpers:rpc( - Config, Node, rabbit_policy, delete, [V, Name, <<"acting-user">>]) || - #{name := Name, vhost := V} <- all_policies_on(Config, Node)]. - -delete_all_runtime_parameters_on(Config, Node) -> - [rabbit_ct_broker_helpers:rpc( - Config, Node, rabbit_runtime_parameters, clear, [V, Component, Name, <<"acting-user">>]) || - #{component := Component, name := Name, vhost := V} <- all_runtime_parameters_on(Config, Node)]. - - -all_queues_on(Config, Node) -> - Ret = rabbit_ct_broker_helpers:rpc(Config, Node, - rabbit_amqqueue, list, [<<"/">>]), - case Ret of - {badrpc, _} -> []; - Qs -> Qs - end. - -all_exchanges_on(Config, Node) -> - Ret = rabbit_ct_broker_helpers:rpc(Config, Node, - rabbit_exchange, list, [<<"/">>]), - case Ret of - {badrpc, _} -> []; - Xs -> Xs - end. - -all_policies_on(Config, Node) -> - Ret = rabbit_ct_broker_helpers:rpc(Config, Node, - rabbit_policy, list, [<<"/">>]), - case Ret of - {badrpc, _} -> []; - Xs -> [maps:from_list(PList) || PList <- Xs] - end. - -all_runtime_parameters_on(Config, Node) -> - Ret = rabbit_ct_broker_helpers:rpc(Config, Node, - rabbit_runtime_parameters, list, [<<"/">>]), - case Ret of - {badrpc, _} -> []; - Xs -> [maps:from_list(PList) || PList <- Xs] - end. - -await_binding(Config, Node, X, Key) -> - await_binding(Config, Node, X, Key, 1). - -await_binding(Config, Node, X, Key, ExpectedBindingCount) when is_integer(ExpectedBindingCount) -> - await_binding(Config, Node, <<"/">>, X, Key, ExpectedBindingCount). - -await_binding(Config, Node, Vhost, X, Key, ExpectedBindingCount) when is_integer(ExpectedBindingCount) -> - Attempts = 100, - await_binding(Config, Node, Vhost, X, Key, ExpectedBindingCount, Attempts). - -await_binding(_Config, _Node, _Vhost, _X, _Key, ExpectedBindingCount, 0) -> - {error, rabbit_misc:format("expected ~b bindings but they did not materialize in time", [ExpectedBindingCount])}; -await_binding(Config, Node, Vhost, X, Key, ExpectedBindingCount, AttemptsLeft) when is_integer(ExpectedBindingCount) -> - case bound_keys_from(Config, Node, Vhost, X, Key) of - Bs when length(Bs) < ExpectedBindingCount -> - timer:sleep(1000), - await_binding(Config, Node, Vhost, X, Key, ExpectedBindingCount, AttemptsLeft - 1); - Bs when length(Bs) =:= ExpectedBindingCount -> - ok; - Bs -> - {error, rabbit_misc:format("expected ~b bindings, got ~b", [ExpectedBindingCount, length(Bs)])} - end. - -await_bindings(Config, Node, X, Keys) -> - [await_binding(Config, Node, X, Key) || Key <- Keys]. - -await_binding_absent(Config, Node, X, Key) -> - case bound_keys_from(Config, Node, <<"/">>, X, Key) of - [] -> ok; - _ -> timer:sleep(100), - await_binding_absent(Config, Node, X, Key) - end. - -bound_keys_from(Config, Node, Vhost, X, Key) -> - Res = rabbit_misc:r(Vhost, exchange, X), - List = rabbit_ct_broker_helpers:rpc(Config, Node, - rabbit_binding, list_for_source, [Res]), - [K || #binding{key = K} <- List, K =:= Key]. - -publish_expect(Ch, X, Key, Q, Payload) -> - publish(Ch, X, Key, Payload), - expect(Ch, Q, [Payload]). - -publish(Ch, X, Key, Payload) when is_binary(Payload) -> - publish(Ch, X, Key, #amqp_msg{payload = Payload}); - -publish(Ch, X, Key, Msg = #amqp_msg{}) -> - amqp_channel:call(Ch, #'basic.publish'{exchange = X, - routing_key = Key}, Msg). - -await_credentials_obfuscation_seeding_on_two_nodes(Config) -> - %% give credentials_obfuscation a moment to start and be seeded - rabbit_ct_helpers:await_condition(fun() -> - rabbit_ct_broker_helpers:rpc(Config, 0, credentials_obfuscation, enabled, []) and - rabbit_ct_broker_helpers:rpc(Config, 1, credentials_obfuscation, enabled, []) - end), - - timer:sleep(1000). - -assert_federation_internal_queue_type(Config, Server, Expected) -> - Qs = all_queues_on(Config, Server), - FedQs = lists:filter( - fun(Q) -> - lists:member( - {<<"x-internal-purpose">>, longstr, <<"federation">>}, amqqueue:get_arguments(Q)) - end, - Qs), - FedQTypes = lists:map(fun(Q) -> amqqueue:get_type(Q) end, FedQs), - ?assertEqual([Expected], lists:uniq(FedQTypes)). diff --git a/deps/rabbitmq_federation/test/federation_status_command_SUITE.erl b/deps/rabbitmq_federation/test/federation_status_command_SUITE.erl deleted file mode 100644 index 2ca0dd8c2342..000000000000 --- a/deps/rabbitmq_federation/test/federation_status_command_SUITE.erl +++ /dev/null @@ -1,172 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(federation_status_command_SUITE). - --include_lib("amqp_client/include/amqp_client.hrl"). - --compile(export_all). - --define(CMD, 'Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand'). - -all() -> - [ - {group, not_federated}, - {group, federated}, - {group, federated_down} - ]. - -groups() -> - [ - {not_federated, [], [ - run_not_federated, - output_not_federated - ]}, - {federated, [], [ - run_federated, - output_federated - ]}, - {federated_down, [], [ - run_down_federated - ]} - ]. - -%% ------------------------------------------------------------------- -%% Testsuite setup/teardown. -%% ------------------------------------------------------------------- - -init_per_suite(Config) -> - rabbit_ct_helpers:log_environment(), - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodename_suffix, ?MODULE} - ]), - Config2 = rabbit_ct_helpers:run_setup_steps(Config1, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()), - Config2. - -end_per_suite(Config) -> - rabbit_ct_helpers:run_teardown_steps(Config, - rabbit_ct_client_helpers:teardown_steps() ++ - rabbit_ct_broker_helpers:teardown_steps()). - -init_per_group(federated, Config) -> - rabbit_federation_test_util:setup_federation(Config), - Config; -init_per_group(federated_down, Config) -> - rabbit_federation_test_util:setup_down_federation(Config), - Config; -init_per_group(_, Config) -> - Config. - -end_per_group(_, Config) -> - Config. - -init_per_testcase(Testcase, Config) -> - rabbit_ct_helpers:testcase_started(Config, Testcase). - -end_per_testcase(Testcase, Config) -> - rabbit_ct_helpers:testcase_finished(Config, Testcase). - -%% ------------------------------------------------------------------- -%% Testcases. -%% ------------------------------------------------------------------- -run_not_federated(Config) -> - [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Opts = #{node => A}, - {stream, []} = ?CMD:run([], Opts#{only_down => false}). - -output_not_federated(Config) -> - [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Opts = #{node => A}, - {stream, []} = ?CMD:output({stream, []}, Opts). - -run_federated(Config) -> - [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Opts = #{node => A}, - %% All - rabbit_federation_test_util:with_ch( - Config, - fun(_) -> - timer:sleep(3000), - {stream, [Props]} = ?CMD:run([], Opts#{only_down => false}), - <<"upstream">> = proplists:get_value(upstream_queue, Props), - <<"fed1.downstream">> = proplists:get_value(queue, Props), - <<"fed.tag">> = proplists:get_value(consumer_tag, Props), - running = proplists:get_value(status, Props) - end, - [rabbit_federation_test_util:q(<<"upstream">>), - rabbit_federation_test_util:q(<<"fed1.downstream">>)]), - %% Down - rabbit_federation_test_util:with_ch( - Config, - fun(_) -> - {stream, []} = ?CMD:run([], Opts#{only_down => true}) - end, - [rabbit_federation_test_util:q(<<"upstream">>), - rabbit_federation_test_util:q(<<"fed1.downstream">>)]). - -run_down_federated(Config) -> - [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Opts = #{node => A}, - %% All - rabbit_federation_test_util:with_ch( - Config, - fun(_) -> - rabbit_ct_helpers:await_condition( - fun() -> - {stream, ManyProps} = ?CMD:run([], Opts#{only_down => false}), - Links = [{proplists:get_value(upstream, Props), - proplists:get_value(status, Props)} - || Props <- ManyProps], - [{<<"broken-bunny">>, error}, {<<"localhost">>, running}] - == lists:sort(Links) - end, 15000) - end, - [rabbit_federation_test_util:q(<<"upstream">>), - rabbit_federation_test_util:q(<<"fed1.downstream">>)]), - %% Down - rabbit_federation_test_util:with_ch( - Config, - fun(_) -> - rabbit_ct_helpers:await_condition( - fun() -> - {stream, Props} = ?CMD:run([], Opts#{only_down => true}), - (length(Props) == 1) - andalso (<<"broken-bunny">> == proplists:get_value(upstream, hd(Props))) - andalso (error == proplists:get_value(status, hd(Props))) - end, 15000) - end, - [rabbit_federation_test_util:q(<<"upstream">>), - rabbit_federation_test_util:q(<<"fed1.downstream">>)]). - -output_federated(Config) -> - [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Opts = #{node => A}, - Input = {stream,[[{queue, <<"fed1.downstream">>}, - {consumer_tag, <<"fed.tag">>}, - {upstream_queue, <<"upstream">>}, - {type, queue}, - {vhost, <<"/">>}, - {upstream, <<"localhost">>}, - {status, running}, - {local_connection, <<"">>}, - {uri, <<"amqp://localhost:21000">>}, - {timestamp, {{2016,11,21},{8,51,19}}}]]}, - {stream, [#{queue := <<"fed1.downstream">>, - upstream_queue := <<"upstream">>, - type := queue, - vhost := <<"/">>, - upstream := <<"localhost">>, - status := running, - local_connection := <<"">>, - uri := <<"amqp://localhost:21000">>, - last_changed := <<"2016-11-21 08:51:19">>, - exchange := <<>>, - upstream_exchange := <<>>, - error := <<>>}]} - = ?CMD:output(Input, Opts). diff --git a/deps/rabbitmq_federation/test/queue_SUITE.erl b/deps/rabbitmq_federation/test/queue_SUITE.erl deleted file mode 100644 index bcc7192b34ae..000000000000 --- a/deps/rabbitmq_federation/test/queue_SUITE.erl +++ /dev/null @@ -1,397 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(queue_SUITE). - --include_lib("common_test/include/ct.hrl"). --include_lib("amqp_client/include/amqp_client.hrl"). - --compile(nowarn_export_all). --compile(export_all). - --import(rabbit_federation_test_util, - [wait_for_federation/2, expect/3, expect/4, - set_upstream/4, set_upstream/5, clear_upstream/3, set_upstream_set/4, clear_upstream_set/3, - set_policy/5, clear_policy/3, - set_policy_pattern/5, set_policy_upstream/5, q/2, with_ch/3, - maybe_declare_queue/3, delete_queue/2, - federation_links_in_vhost/3]). - --define(INITIAL_WAIT, 6000). --define(EXPECT_FEDERATION_TIMEOUT, 30000). - -all() -> - [ - {group, classic_queue}, - {group, quorum_queue}, - {group, mixed} - ]. - -groups() -> - [ - {classic_queue, [], all_tests()}, - {quorum_queue, [], all_tests()}, - {mixed, [], all_tests()} - ]. - -all_tests() -> - [ - {without_disambiguate, [], [ - {cluster_size_1, [], [ - simple, - multiple_upstreams_pattern, - multiple_downstreams, - message_flow, - dynamic_reconfiguration, - federate_unfederate, - dynamic_plugin_stop_start - ]} - ]}, - {with_disambiguate, [], [ - {cluster_size_2, [], [restart_upstream]} - ]} - ]. - -%% ------------------------------------------------------------------- -%% 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(classic_queue, Config) -> - rabbit_ct_helpers:set_config( - Config, - [ - {source_queue_type, classic}, - {source_queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]}, - {target_queue_type, classic}, - {target_queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]} - ]); -init_per_group(quorum_queue, Config) -> - rabbit_ct_helpers:set_config( - Config, - [ - {source_queue_type, quorum}, - {source_queue_args, [{<<"x-queue-type">>, longstr, <<"quorum">>}]}, - {target_queue_type, quorum}, - {target_queue_args, [{<<"x-queue-type">>, longstr, <<"quorum">>}]} - ]); -init_per_group(mixed, Config) -> - rabbit_ct_helpers:set_config( - Config, - [ - {source_queue_type, classic}, - {source_queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]}, - {target_queue_type, quorum}, - {target_queue_args, [{<<"x-queue-type">>, longstr, <<"quorum">>}]} - ]); -init_per_group(without_disambiguate, Config) -> - rabbit_ct_helpers:set_config(Config, - {disambiguate_step, []}); -init_per_group(with_disambiguate, Config) -> - rabbit_ct_helpers:set_config(Config, - {disambiguate_step, [fun rabbit_federation_test_util:disambiguate/1]}); -init_per_group(cluster_size_1 = Group, Config) -> - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodes_count, 1} - ]), - init_per_group1(Group, Config1); -init_per_group(cluster_size_2 = Group, Config) -> - case rabbit_ct_helpers:is_mixed_versions() of - true -> - {skip, "not mixed versions compatible"}; - _ -> - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodes_count, 2} - ]), - init_per_group1(Group, Config1) - end. - -init_per_group1(Group, Config) -> - SetupFederation = case Group of - cluster_size_1 -> [fun rabbit_federation_test_util:setup_federation/1]; - cluster_size_2 -> [] - end, - Disambiguate = ?config(disambiguate_step, Config), - Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"), - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodename_suffix, Suffix}, - {rmq_nodes_clustered, false} - ]), - rabbit_ct_helpers:run_steps(Config1, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps() ++ - SetupFederation ++ Disambiguate). - -end_per_group(without_disambiguate, Config) -> - Config; -end_per_group(with_disambiguate, Config) -> - Config; -end_per_group(classic_queue, Config) -> - Config; -end_per_group(quorum_queue, Config) -> - Config; -end_per_group(mixed, Config) -> - Config; -end_per_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). - -%% ------------------------------------------------------------------- -%% Testcases. -%% ------------------------------------------------------------------- - -simple(Config) -> - with_ch(Config, - fun (Ch) -> - expect_federation(Ch, <<"upstream">>, <<"fed1.downstream">>) - end, upstream_downstream(Config)). - -multiple_upstreams_pattern(Config) -> - set_upstream(Config, 0, <<"local453x">>, - rabbit_ct_broker_helpers:node_uri(Config, 0), [ - {<<"exchange">>, <<"upstream">>}, - {<<"queue">>, <<"upstream">>}]), - - set_upstream(Config, 0, <<"zzzzzZZzz">>, - rabbit_ct_broker_helpers:node_uri(Config, 0), [ - {<<"exchange">>, <<"upstream-zzz">>}, - {<<"queue">>, <<"upstream-zzz">>}]), - - set_upstream(Config, 0, <<"local3214x">>, - rabbit_ct_broker_helpers:node_uri(Config, 0), [ - {<<"exchange">>, <<"upstream2">>}, - {<<"queue">>, <<"upstream2">>}]), - - set_policy_pattern(Config, 0, <<"pattern">>, <<"^pattern\.">>, <<"local\\d+x">>), - - SourceArgs = ?config(source_queue_args, Config), - TargetArgs = ?config(target_queue_args, Config), - with_ch(Config, - fun (Ch) -> - expect_federation(Ch, <<"upstream">>, <<"pattern.downstream">>, ?EXPECT_FEDERATION_TIMEOUT), - expect_federation(Ch, <<"upstream2">>, <<"pattern.downstream">>, ?EXPECT_FEDERATION_TIMEOUT) - end, [q(<<"upstream">>, SourceArgs), - q(<<"upstream2">>, SourceArgs), - q(<<"pattern.downstream">>, TargetArgs)]), - - clear_upstream(Config, 0, <<"local453x">>), - clear_upstream(Config, 0, <<"local3214x">>), - clear_policy(Config, 0, <<"pattern">>). - -multiple_downstreams(Config) -> - Args = ?config(target_queue_args, Config), - with_ch(Config, - fun (Ch) -> - timer:sleep(?INITIAL_WAIT), - expect_federation(Ch, <<"upstream">>, <<"fed1.downstream">>, ?EXPECT_FEDERATION_TIMEOUT), - expect_federation(Ch, <<"upstream2">>, <<"fed2.downstream">>, ?EXPECT_FEDERATION_TIMEOUT) - end, upstream_downstream(Config) ++ [q(<<"fed2.downstream">>, Args)]). - -message_flow(Config) -> - %% TODO: specifc source / target here - Args = ?config(source_queue_args, Config), - with_ch(Config, - fun (Ch) -> - timer:sleep(?INITIAL_WAIT), - publish_expect(Ch, <<>>, <<"one">>, <<"one">>, <<"first one">>, ?EXPECT_FEDERATION_TIMEOUT), - publish_expect(Ch, <<>>, <<"two">>, <<"two">>, <<"first two">>, ?EXPECT_FEDERATION_TIMEOUT), - Seq = lists:seq(1, 50), - [publish(Ch, <<>>, <<"one">>, <<"bulk">>) || _ <- Seq], - [publish(Ch, <<>>, <<"two">>, <<"bulk">>) || _ <- Seq], - expect(Ch, <<"one">>, repeat(100, <<"bulk">>)), - expect_empty(Ch, <<"one">>), - expect_empty(Ch, <<"two">>), - [publish(Ch, <<>>, <<"one">>, <<"bulk">>) || _ <- Seq], - [publish(Ch, <<>>, <<"two">>, <<"bulk">>) || _ <- Seq], - expect(Ch, <<"two">>, repeat(100, <<"bulk">>)), - expect_empty(Ch, <<"one">>), - expect_empty(Ch, <<"two">>), - %% We clear the federation configuration to avoid a race condition - %% when deleting the queues in quorum mode. The federation link - %% would restart and lead to a state where nothing happened for - %% minutes. - clear_upstream_set(Config, 0, <<"one">>), - clear_upstream_set(Config, 0, <<"two">>) - end, [q(<<"one">>, Args), - q(<<"two">>, Args)]). - -dynamic_reconfiguration(Config) -> - with_ch(Config, - fun (Ch) -> - timer:sleep(?INITIAL_WAIT), - expect_federation(Ch, <<"upstream">>, <<"fed1.downstream">>, ?EXPECT_FEDERATION_TIMEOUT), - - %% Test that clearing connections works - clear_upstream(Config, 0, <<"localhost">>), - expect_no_federation(Ch, <<"upstream">>, <<"fed1.downstream">>), - - %% Test that reading them and changing them works - set_upstream(Config, 0, - <<"localhost">>, rabbit_ct_broker_helpers:node_uri(Config, 0)), - %% Do it twice so we at least hit the no-restart optimisation - URI = rabbit_ct_broker_helpers:node_uri(Config, 0, [use_ipaddr]), - set_upstream(Config, 0, <<"localhost">>, URI), - set_upstream(Config, 0, <<"localhost">>, URI), - expect_federation(Ch, <<"upstream">>, <<"fed1.downstream">>) - end, upstream_downstream(Config)). - -federate_unfederate(Config) -> - Args = ?config(target_queue_args, Config), - with_ch(Config, - fun (Ch) -> - timer:sleep(?INITIAL_WAIT), - expect_federation(Ch, <<"upstream">>, <<"fed1.downstream">>, ?EXPECT_FEDERATION_TIMEOUT), - expect_federation(Ch, <<"upstream2">>, <<"fed2.downstream">>, ?EXPECT_FEDERATION_TIMEOUT), - - %% clear the policy - rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"fed">>), - - expect_no_federation(Ch, <<"upstream">>, <<"fed1.downstream">>), - expect_no_federation(Ch, <<"upstream2">>, <<"fed2.downstream">>), - - rabbit_ct_broker_helpers:set_policy(Config, 0, - <<"fed">>, <<"^fed1\.">>, <<"all">>, [ - {<<"federation-upstream-set">>, <<"upstream">>}]) - end, upstream_downstream(Config) ++ [q(<<"fed2.downstream">>, Args)]). - -dynamic_plugin_stop_start(Config) -> - DownQ2 = <<"fed2.downstream">>, - Args = ?config(target_queue_args, Config), - with_ch(Config, - fun (Ch) -> - timer:sleep(?INITIAL_WAIT), - UpQ1 = <<"upstream">>, - UpQ2 = <<"upstream2">>, - DownQ1 = <<"fed1.downstream">>, - expect_federation(Ch, UpQ1, DownQ1, ?EXPECT_FEDERATION_TIMEOUT), - expect_federation(Ch, UpQ2, DownQ2, ?EXPECT_FEDERATION_TIMEOUT), - - %% Disable the plugin, the link disappears - ct:pal("Stopping rabbitmq_federation"), - ok = rabbit_ct_broker_helpers:disable_plugin(Config, 0, "rabbitmq_federation"), - - expect_no_federation(Ch, UpQ1, DownQ1), - expect_no_federation(Ch, UpQ2, DownQ2), - - maybe_declare_queue(Config, Ch, q(DownQ1, Args)), - maybe_declare_queue(Config, Ch, q(DownQ2, Args)), - ct:pal("Re-starting rabbitmq_federation"), - ok = rabbit_ct_broker_helpers:enable_plugin(Config, 0, "rabbitmq_federation"), - timer:sleep(?INITIAL_WAIT), - - %% Declare a queue then re-enable the plugin, the links appear - wait_for_federation( - 90, - fun() -> - Status = rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_federation_status, status, []), - L = [ - Entry || Entry <- Status, - proplists:get_value(queue, Entry) =:= DownQ1 orelse - proplists:get_value(queue, Entry) =:= DownQ2, - proplists:get_value(upstream_queue, Entry) =:= UpQ1 orelse - proplists:get_value(upstream_queue, Entry) =:= UpQ2, - proplists:get_value(status, Entry) =:= running - ], - length(L) =:= 2 - end), - expect_federation(Ch, UpQ1, DownQ1, 120000) - end, upstream_downstream(Config) ++ [q(DownQ2, Args)]). - -restart_upstream(Config) -> - [Rabbit, Hare] = rabbit_ct_broker_helpers:get_node_configs(Config, - nodename), - set_policy_upstream(Config, Rabbit, <<"^test$">>, - rabbit_ct_broker_helpers:node_uri(Config, Hare), []), - - Downstream = rabbit_ct_client_helpers:open_channel(Config, Rabbit), - Upstream = rabbit_ct_client_helpers:open_channel(Config, Hare), - - SourceArgs = ?config(source_queue_args, Config), - TargetArgs = ?config(target_queue_args, Config), - maybe_declare_queue(Config, Upstream, q(<<"test">>, SourceArgs)), - maybe_declare_queue(Config, Downstream, q(<<"test">>, TargetArgs)), - Seq = lists:seq(1, 100), - [publish(Upstream, <<>>, <<"test">>, <<"bulk">>) || _ <- Seq], - expect(Upstream, <<"test">>, repeat(25, <<"bulk">>)), - expect(Downstream, <<"test">>, repeat(25, <<"bulk">>)), - - rabbit_ct_client_helpers:close_channels_and_connection(Config, Hare), - ok = rabbit_ct_broker_helpers:restart_node(Config, Hare), - Upstream2 = rabbit_ct_client_helpers:open_channel(Config, Hare), - - expect(Upstream2, <<"test">>, repeat(25, <<"bulk">>)), - expect(Downstream, <<"test">>, repeat(25, <<"bulk">>)), - expect_empty(Upstream2, <<"test">>), - expect_empty(Downstream, <<"test">>), - - ok. - -%upstream_has_no_federation(Config) -> -% %% TODO -% ok. - -%%---------------------------------------------------------------------------- -repeat(Count, Item) -> [Item || _ <- lists:seq(1, Count)]. - -%%---------------------------------------------------------------------------- - -publish(Ch, X, Key, Payload) when is_binary(Payload) -> - publish(Ch, X, Key, #amqp_msg{payload = Payload}); - -publish(Ch, X, Key, Msg = #amqp_msg{}) -> - amqp_channel:call(Ch, #'basic.publish'{exchange = X, - routing_key = Key}, Msg). - -publish_expect(Ch, X, Key, Q, Payload) -> - publish(Ch, X, Key, Payload), - expect(Ch, Q, [Payload]). - -publish_expect(Ch, X, Key, Q, Payload, Timeout) -> - publish(Ch, X, Key, Payload), - expect(Ch, Q, [Payload], Timeout). - -%% Doubled due to our strange basic.get behaviour. -expect_empty(Ch, Q) -> - rabbit_federation_test_util:expect_empty(Ch, Q), - rabbit_federation_test_util:expect_empty(Ch, Q). - -expect_federation(Ch, UpstreamQ, DownstreamQ) -> - Base = <<"HELLO">>, - Payload = <>, - publish_expect(Ch, <<>>, UpstreamQ, DownstreamQ, Payload). - -expect_federation(Ch, UpstreamQ, DownstreamQ, Timeout) -> - Base = <<"HELLO">>, - Payload = <>, - publish_expect(Ch, <<>>, UpstreamQ, DownstreamQ, Payload, Timeout). - -expect_no_federation(Ch, UpstreamQ, DownstreamQ) -> - publish(Ch, <<>>, UpstreamQ, <<"HELLO">>), - expect_empty(Ch, DownstreamQ), - expect(Ch, UpstreamQ, [<<"HELLO">>]). - -upstream_downstream() -> - upstream_downstream([]). - -upstream_downstream(Config) -> - SourceArgs = ?config(source_queue_args, Config), - TargetArgs = ?config(target_queue_args, Config), - [q(<<"upstream">>, SourceArgs), q(<<"fed1.downstream">>, TargetArgs)]. diff --git a/deps/rabbitmq_federation/test/rabbit_federation_status_SUITE.erl b/deps/rabbitmq_federation/test/rabbit_federation_status_SUITE.erl deleted file mode 100644 index 39f13f685f58..000000000000 --- a/deps/rabbitmq_federation/test/rabbit_federation_status_SUITE.erl +++ /dev/null @@ -1,105 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_status_SUITE). - --include_lib("amqp_client/include/amqp_client.hrl"). - --include("rabbit_federation.hrl"). - --compile(export_all). - --import(rabbit_federation_test_util, - [expect/3, expect_empty/2, - set_upstream/4, clear_upstream/3, set_upstream_set/4, - set_policy/5, clear_policy/3, - set_policy_upstream/5, set_policy_upstreams/4, - no_plugins/1, with_ch/3]). - -all() -> - [ - {group, non_parallel_tests} - ]. - -groups() -> - [ - {non_parallel_tests, [], [ - queue_status, - lookup_queue_status, - lookup_bad_status - ]} - ]. - -suite() -> - [{timetrap, {minutes, 5}}]. - -%% ------------------------------------------------------------------- -%% Testsuite setup/teardown. -%% ------------------------------------------------------------------- -init_per_suite(Config) -> - rabbit_ct_helpers:log_environment(), - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodename_suffix, ?MODULE} - ]), - rabbit_ct_helpers:run_setup_steps(Config1, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps() ++ - [fun rabbit_federation_test_util:setup_federation/1]). -end_per_suite(Config) -> - rabbit_ct_helpers:run_teardown_steps(Config, - rabbit_ct_client_helpers:teardown_steps() ++ - rabbit_ct_broker_helpers:teardown_steps()). - -init_per_group(_, Config) -> - Config. - -end_per_group(_, Config) -> - Config. - -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 -%% ------------------------------------------------------------------- - -queue_status(Config) -> - with_ch( - Config, - fun (_Ch) -> - timer:sleep(3000), - [Link] = rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_federation_status, status, []), - true = is_binary(proplists:get_value(id, Link)) - end, queue_SUITE:upstream_downstream()). - -lookup_queue_status(Config) -> - with_ch( - Config, - fun (_Ch) -> - timer:sleep(3000), - [Link] = rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_federation_status, status, []), - Id = proplists:get_value(id, Link), - Props = rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_federation_status, lookup, [Id]), - lists:all(fun(K) -> lists:keymember(K, 1, Props) end, - [key, uri, status, timestamp, id, supervisor, upstream]) - end, queue_SUITE:upstream_downstream()). - -lookup_bad_status(Config) -> - with_ch( - Config, - fun (_Ch) -> - timer:sleep(3000), - not_found = rabbit_ct_broker_helpers:rpc( - Config, 0, - rabbit_federation_status, lookup, [<<"justmadeitup">>]) - end, queue_SUITE:upstream_downstream()). diff --git a/deps/rabbitmq_federation/test/rabbit_federation_test_util.erl b/deps/rabbitmq_federation/test/rabbit_federation_test_util.erl deleted file mode 100644 index 8a49b9087645..000000000000 --- a/deps/rabbitmq_federation/test/rabbit_federation_test_util.erl +++ /dev/null @@ -1,382 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(rabbit_federation_test_util). - --include("rabbit_federation.hrl"). --include_lib("eunit/include/eunit.hrl"). --include_lib("amqp_client/include/amqp_client.hrl"). - --compile(export_all). - --deprecated({wait_for_federation,2,"Use rabbit_ct_helpers:await_condition or ?awaitMatch instead"}). - --import(rabbit_misc, [pget/2]). - -setup_federation(Config) -> - setup_federation_with_upstream_params(Config, []). - -setup_federation_with_upstream_params(Config, ExtraParams) -> - rabbit_ct_broker_helpers:set_parameter(Config, 0, - <<"federation-upstream">>, <<"localhost">>, [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}, - {<<"consumer-tag">>, <<"fed.tag">>} - ] ++ ExtraParams), - - rabbit_ct_broker_helpers:set_parameter(Config, 0, - <<"federation-upstream">>, <<"local5673">>, [ - {<<"uri">>, <<"amqp://localhost:1">>} - ] ++ ExtraParams), - - rabbit_ct_broker_helpers:set_parameter(Config, 0, - <<"federation-upstream-set">>, <<"upstream">>, [ - [ - {<<"upstream">>, <<"localhost">>}, - {<<"exchange">>, <<"upstream">>}, - {<<"queue">>, <<"upstream">>} - ] - ]), - - rabbit_ct_broker_helpers:set_parameter(Config, 0, - <<"federation-upstream-set">>, <<"upstream2">>, [ - [ - {<<"upstream">>, <<"localhost">>}, - {<<"exchange">>, <<"upstream2">>}, - {<<"queue">>, <<"upstream2">>} - ] - ]), - - rabbit_ct_broker_helpers:set_parameter(Config, 0, - <<"federation-upstream-set">>, <<"localhost">>, [ - [{<<"upstream">>, <<"localhost">>}] - ]), - - rabbit_ct_broker_helpers:set_parameter(Config, 0, - <<"federation-upstream-set">>, <<"upstream12">>, [ - [ - {<<"upstream">>, <<"localhost">>}, - {<<"exchange">>, <<"upstream">>}, - {<<"queue">>, <<"upstream">>} - ], [ - {<<"upstream">>, <<"localhost">>}, - {<<"exchange">>, <<"upstream2">>}, - {<<"queue">>, <<"upstream2">>} - ] - ]), - - rabbit_ct_broker_helpers:set_parameter(Config, 0, - <<"federation-upstream-set">>, <<"one">>, [ - [ - {<<"upstream">>, <<"localhost">>}, - {<<"exchange">>, <<"one">>}, - {<<"queue">>, <<"one">>} - ] - ]), - - rabbit_ct_broker_helpers:set_parameter(Config, 0, - <<"federation-upstream-set">>, <<"two">>, [ - [ - {<<"upstream">>, <<"localhost">>}, - {<<"exchange">>, <<"two">>}, - {<<"queue">>, <<"two">>} - ] - ]), - - rabbit_ct_broker_helpers:set_parameter(Config, 0, - <<"federation-upstream-set">>, <<"upstream5673">>, [ - [ - {<<"upstream">>, <<"local5673">>}, - {<<"exchange">>, <<"upstream">>} - ] - ]), - - rabbit_ct_broker_helpers:rpc( - Config, 0, rabbit_policy, set, - [<<"/">>, <<"fed">>, <<"^fed1\.">>, [{<<"federation-upstream-set">>, <<"upstream">>}], - 0, <<"all">>, <<"acting-user">>]), - - rabbit_ct_broker_helpers:rpc( - Config, 0, rabbit_policy, set, - [<<"/">>, <<"fed2">>, <<"^fed2\.">>, [{<<"federation-upstream-set">>, <<"upstream2">>}], - 0, <<"all">>, <<"acting-user">>]), - - rabbit_ct_broker_helpers:rpc( - Config, 0, rabbit_policy, set, - [<<"/">>, <<"fed12">>, <<"^fed3\.">>, [{<<"federation-upstream-set">>, <<"upstream12">>}], - 2, <<"all">>, <<"acting-user">>]), - - rabbit_ct_broker_helpers:set_policy(Config, 0, - <<"one">>, <<"^two$">>, <<"all">>, [ - {<<"federation-upstream-set">>, <<"one">>}]), - - rabbit_ct_broker_helpers:set_policy(Config, 0, - <<"two">>, <<"^one$">>, <<"all">>, [ - {<<"federation-upstream-set">>, <<"two">>}]), - - rabbit_ct_broker_helpers:set_policy(Config, 0, - <<"hare">>, <<"^hare\.">>, <<"all">>, [ - {<<"federation-upstream-set">>, <<"upstream5673">>}]), - - rabbit_ct_broker_helpers:set_policy(Config, 0, - <<"all">>, <<"^all\.">>, <<"all">>, [ - {<<"federation-upstream-set">>, <<"all">>}]), - - rabbit_ct_broker_helpers:set_policy(Config, 0, - <<"new">>, <<"^new\.">>, <<"all">>, [ - {<<"federation-upstream-set">>, <<"new-set">>}]), - Config. - -setup_down_federation(Config) -> - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"broken-bunny">>, - [{<<"uri">>, <<"amqp://broken-bunny">>}, - {<<"reconnect-delay">>, 600000}]), - rabbit_ct_broker_helpers:set_parameter( - Config, 0, <<"federation-upstream">>, <<"localhost">>, - [{<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config, 0)}]), - rabbit_ct_broker_helpers:set_parameter( - Config, 0, - <<"federation-upstream-set">>, <<"upstream">>, - [[{<<"upstream">>, <<"localhost">>}, - {<<"exchange">>, <<"upstream">>}, - {<<"queue">>, <<"upstream">>}], - [{<<"upstream">>, <<"broken-bunny">>}, - {<<"exchange">>, <<"upstream">>}, - {<<"queue">>, <<"upstream">>}]]), - rabbit_ct_broker_helpers:set_policy( - Config, 0, - <<"fed">>, <<"^fed1\.">>, <<"all">>, [{<<"federation-upstream-set">>, <<"upstream">>}]), - rabbit_ct_broker_helpers:set_policy( - Config, 0, - <<"fed">>, <<"^fed1\.">>, <<"all">>, [{<<"federation-upstream-set">>, <<"upstream">>}]), - Config. - -wait_for_federation(Retries, Fun) -> - case Fun() of - true -> - ok; - false when Retries > 0 -> - timer:sleep(1000), - wait_for_federation(Retries - 1, Fun); - false -> - throw({timeout_while_waiting_for_federation, Fun}) - end. - -expect(Ch, Q, Fun) when is_function(Fun) -> - amqp_channel:subscribe(Ch, #'basic.consume'{queue = Q, - no_ack = true}, self()), - CTag = receive - #'basic.consume_ok'{consumer_tag = CT} -> CT - end, - Fun(), - amqp_channel:call(Ch, #'basic.cancel'{consumer_tag = CTag}); - -expect(Ch, Q, Payloads) -> - expect(Ch, Q, fun() -> expect(Payloads) end). - -expect(Ch, Q, Payloads, Timeout) -> - expect(Ch, Q, fun() -> expect(Payloads, Timeout) end). - -expect([]) -> - ok; -expect(Payloads) -> - expect(Payloads, 60000). - -expect([], _Timeout) -> - ok; -expect(Payloads, Timeout) -> - receive - {#'basic.deliver'{delivery_tag = DTag}, #amqp_msg{payload = Payload}} -> - case lists:member(Payload, Payloads) of - true -> - ct:pal("Consumed a message: ~tp ~tp left: ~tp", [Payload, DTag, length(Payloads) - 1]), - expect(Payloads -- [Payload], Timeout); - false -> ?assert(false, rabbit_misc:format("received an unexpected payload ~tp", [Payload])) - end - after Timeout -> - ct:fail("Did not receive expected payloads ~tp in time", [Payloads]) - end. - -expect_empty(Ch, Q) -> - ?assertMatch(#'basic.get_empty'{}, - amqp_channel:call(Ch, #'basic.get'{ queue = Q })). - -set_upstream(Config, Node, Name, URI) -> - set_upstream(Config, Node, Name, URI, []). - -set_upstream(Config, Node, Name, URI, Extra) -> - rabbit_ct_broker_helpers:set_parameter(Config, Node, - <<"federation-upstream">>, Name, [{<<"uri">>, URI} | Extra]). - -set_upstream_in_vhost(Config, Node, VirtualHost, Name, URI) -> - set_upstream_in_vhost(Config, Node, VirtualHost, Name, URI, []). - -set_upstream_in_vhost(Config, Node, VirtualHost, Name, URI, Extra) -> - rabbit_ct_broker_helpers:set_parameter(Config, Node, VirtualHost, - <<"federation-upstream">>, Name, [{<<"uri">>, URI} | Extra]). - -clear_upstream(Config, Node, Name) -> - rabbit_ct_broker_helpers:clear_parameter(Config, Node, - <<"federation-upstream">>, Name). - -set_upstream_set(Config, Node, Name, Set) -> - rabbit_ct_broker_helpers:set_parameter(Config, Node, - <<"federation-upstream-set">>, Name, - [[{<<"upstream">>, UStream} | Extra] || {UStream, Extra} <- Set]). - -clear_upstream_set(Config, Node, Name) -> - rabbit_ct_broker_helpers:clear_parameter(Config, Node, - <<"federation-upstream-set">>, Name). - -set_policy(Config, Node, Name, Pattern, UpstreamSet) -> - rabbit_ct_broker_helpers:set_policy(Config, Node, - Name, Pattern, <<"all">>, - [{<<"federation-upstream-set">>, UpstreamSet}]). - -set_policy_pattern(Config, Node, Name, Pattern, Regex) -> - rabbit_ct_broker_helpers:set_policy(Config, Node, - Name, Pattern, <<"all">>, - [{<<"federation-upstream-pattern">>, Regex}]). - -clear_policy(Config, Node, Name) -> - rabbit_ct_broker_helpers:clear_policy(Config, Node, Name). - -set_policy_upstream(Config, Node, Pattern, URI, Extra) -> - set_policy_upstreams(Config, Node, Pattern, [{URI, Extra}]). - -set_policy_upstreams(Config, Node, Pattern, URIExtras) -> - put(upstream_num, 1), - [set_upstream(Config, Node, gen_upstream_name(), URI, Extra) - || {URI, Extra} <- URIExtras], - set_policy(Config, Node, Pattern, Pattern, <<"all">>). - -gen_upstream_name() -> - list_to_binary("upstream-" ++ integer_to_list(next_upstream_num())). - -next_upstream_num() -> - R = get(upstream_num) + 1, - put(upstream_num, R), - R. - -%% Make sure that even though multiple nodes are in a single -%% distributed system, we still keep all our process groups separate. -disambiguate(Config) -> - rabbit_ct_broker_helpers:rpc_all(Config, - application, set_env, - [rabbitmq_federation, pgroup_name_cluster_id, true]), - Config. - -no_plugins(Cfg) -> - [{K, case K of - plugins -> none; - _ -> V - end} || {K, V} <- Cfg]. - -%%---------------------------------------------------------------------------- - -all_federation_links(Config, Node) -> - rabbit_ct_broker_helpers:rpc(Config, Node, rabbit_federation_status, status, []). - -federation_links_in_vhost(Config, Node, VirtualHost) -> - Links = rabbit_ct_broker_helpers:rpc(Config, Node, rabbit_federation_status, status, []), - lists:filter( - fun(Link) -> - VirtualHost =:= proplists:get_value(vhost, Link) - end, Links). - -status_fields(Prop, Statuses) -> - lists:usort( - lists:map( - fun(Link) -> proplists:get_value(Prop, Link) end, - Statuses)). - -assert_status(Config, Node, XorQs, Names) -> - rabbit_ct_broker_helpers:rpc(Config, Node, - ?MODULE, assert_status1, [XorQs, Names]). - -assert_status1(XorQs, Names) -> - [begin - ct:pal("links(XorQ) for ~tp: ~tp", [XorQ, links(XorQ)]), - ct:pal("rabbit_federation_status:status(): ~tp", [rabbit_federation_status:status()]) - end || XorQ <- XorQs], - Links = lists:append([links(XorQ) || XorQ <- XorQs]), - Remaining = lists:foldl(fun (Link, Status) -> - assert_link_status(Link, Status, Names) - end, rabbit_federation_status:status(), Links), - ?assertEqual([], Remaining), - ok. - -assert_link_status({DXorQNameBin, UpstreamName, UXorQNameBin}, Status, - {TypeName, UpstreamTypeName}) -> - {This, Rest} = lists:partition( - fun(St) -> - pget(upstream, St) =:= UpstreamName andalso - pget(TypeName, St) =:= DXorQNameBin andalso - pget(UpstreamTypeName, St) =:= UXorQNameBin - end, Status), - ?assertMatch([_], This), - Rest. - -links(#'exchange.declare'{exchange = Name}) -> - case rabbit_exchange:lookup(xr(Name)) of - {ok, X} -> - case rabbit_policy:get(<<"federation-upstream-set">>, X) of - undefined -> - case rabbit_policy:get(<<"federation-upstream-pattern">>, X) of - undefined -> []; - Regex -> - [{Name, U#upstream.name, U#upstream.exchange_name} || - U <- rabbit_federation_upstream:from_pattern(Regex, X)] - end; - Set -> - [{Name, U#upstream.name, U#upstream.exchange_name} || - U <- rabbit_federation_upstream:from_set(Set, X)] - end; - {error, not_found} -> - [] - end. - -xr(Name) -> rabbit_misc:r(<<"/">>, exchange, Name). - -with_ch(Config, Fun, Methods) -> - Ch = rabbit_ct_client_helpers:open_channel(Config), - declare_all(Config, Ch, Methods), - %% Clean up queues even after test failure. - try - Fun(Ch) - after - delete_all(Ch, Methods), - rabbit_ct_client_helpers:close_channel(Ch) - end, - ok. - -declare_all(Config, Ch, Methods) -> [maybe_declare_queue(Config, Ch, Op) || Op <- Methods]. -delete_all(Ch, Methods) -> - [delete_queue(Ch, Q) || #'queue.declare'{queue = Q} <- Methods]. - -maybe_declare_queue(Config, Ch, Method) -> - OneOffCh = rabbit_ct_client_helpers:open_channel(Config), - try - amqp_channel:call(OneOffCh, Method#'queue.declare'{passive = true}) - catch exit:{{shutdown, {server_initiated_close, ?NOT_FOUND, _Message}}, _} -> - amqp_channel:call(Ch, Method) - after - catch rabbit_ct_client_helpers:close_channel(OneOffCh) - end. - -delete_queue(Ch, Q) -> - amqp_channel:call(Ch, #'queue.delete'{queue = Q}). - -q(Name) -> - q(Name, []). - -q(Name, undefined) -> - q(Name, []); -q(Name, Args) -> - #'queue.declare'{queue = Name, - durable = true, - arguments = Args}. diff --git a/deps/rabbitmq_federation/test/restart_federation_link_command_SUITE.erl b/deps/rabbitmq_federation/test/restart_federation_link_command_SUITE.erl deleted file mode 100644 index 74565771648e..000000000000 --- a/deps/rabbitmq_federation/test/restart_federation_link_command_SUITE.erl +++ /dev/null @@ -1,100 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(restart_federation_link_command_SUITE). - --include_lib("amqp_client/include/amqp_client.hrl"). - --compile(export_all). - --define(CMD, 'Elixir.RabbitMQ.CLI.Ctl.Commands.RestartFederationLinkCommand'). - -all() -> - [ - {group, federated_down} - ]. - -groups() -> - [ - {federated_down, [], [ - run, - run_not_found, - output - ]} - ]. - -%% ------------------------------------------------------------------- -%% Testsuite setup/teardown. -%% ------------------------------------------------------------------- - -init_per_suite(Config) -> - rabbit_ct_helpers:log_environment(), - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodename_suffix, ?MODULE} - ]), - Config2 = rabbit_ct_helpers:run_setup_steps(Config1, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()), - Config2. - -end_per_suite(Config) -> - rabbit_ct_helpers:run_teardown_steps(Config, - rabbit_ct_client_helpers:teardown_steps() ++ - rabbit_ct_broker_helpers:teardown_steps()). - -init_per_group(federated_down, Config) -> - rabbit_federation_test_util:setup_down_federation(Config), - Config; -init_per_group(_, Config) -> - Config. - -end_per_group(_, Config) -> - Config. - -init_per_testcase(Testcase, Config) -> - rabbit_ct_helpers:testcase_started(Config, Testcase). - -end_per_testcase(Testcase, Config) -> - rabbit_ct_helpers:testcase_finished(Config, Testcase). - -%% ------------------------------------------------------------------- -%% Testcases. -%% ------------------------------------------------------------------- -run_not_federated(Config) -> - [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Opts = #{node => A}, - {stream, []} = ?CMD:run([], Opts#{'only-down' => false}). - -output_not_federated(Config) -> - [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Opts = #{node => A}, - {stream, []} = ?CMD:output({stream, []}, Opts). - -run(Config) -> - [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Opts = #{node => A}, - rabbit_federation_test_util:with_ch( - Config, - fun(_) -> - timer:sleep(3000), - [Link | _] = rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_federation_status, status, []), - Id = proplists:get_value(id, Link), - ok = ?CMD:run([Id], Opts) - end, - [rabbit_federation_test_util:q(<<"upstream">>), - rabbit_federation_test_util:q(<<"fed1.downstream">>)]). - -run_not_found(Config) -> - [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Opts = #{node => A}, - {error, _ErrorMsg} = ?CMD:run([<<"MakingItUp">>], Opts). - -output(Config) -> - [A] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Opts = #{node => A}, - ok = ?CMD:output(ok, Opts). diff --git a/deps/rabbitmq_federation/test/unit_SUITE.erl b/deps/rabbitmq_federation/test/unit_SUITE.erl deleted file mode 100644 index f26c10e82b5a..000000000000 --- a/deps/rabbitmq_federation/test/unit_SUITE.erl +++ /dev/null @@ -1,65 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(unit_SUITE). --include_lib("common_test/include/ct.hrl"). --include_lib("eunit/include/eunit.hrl"). --include_lib("amqp_client/include/amqp_client.hrl"). --include("rabbit_federation.hrl"). - --compile(export_all). - -all() -> [ - obfuscate_upstream, - obfuscate_upstream_params_network, - obfuscate_upstream_params_network_with_char_list_password_value, - obfuscate_upstream_params_direct -]. - -init_per_suite(Config) -> - application:ensure_all_started(credentials_obfuscation), - Config. - -end_per_suite(Config) -> - Config. - -obfuscate_upstream(_Config) -> - Upstream = #upstream{uris = [<<"amqp://guest:password@localhost">>]}, - ObfuscatedUpstream = rabbit_federation_util:obfuscate_upstream(Upstream), - ?assertEqual(Upstream, rabbit_federation_util:deobfuscate_upstream(ObfuscatedUpstream)), - ok. - -obfuscate_upstream_params_network(_Config) -> - UpstreamParams = #upstream_params{ - uri = <<"amqp://guest:password@localhost">>, - params = #amqp_params_network{password = <<"password">>} - }, - ObfuscatedUpstreamParams = rabbit_federation_util:obfuscate_upstream_params(UpstreamParams), - ?assertEqual(UpstreamParams, rabbit_federation_util:deobfuscate_upstream_params(ObfuscatedUpstreamParams)), - ok. - -obfuscate_upstream_params_network_with_char_list_password_value(_Config) -> - Input = #upstream_params{ - uri = <<"amqp://guest:password@localhost">>, - params = #amqp_params_network{password = "password"} - }, - Output = #upstream_params{ - uri = <<"amqp://guest:password@localhost">>, - params = #amqp_params_network{password = <<"password">>} - }, - ObfuscatedUpstreamParams = rabbit_federation_util:obfuscate_upstream_params(Input), - ?assertEqual(Output, rabbit_federation_util:deobfuscate_upstream_params(ObfuscatedUpstreamParams)), - ok. - - obfuscate_upstream_params_direct(_Config) -> - UpstreamParams = #upstream_params{ - uri = <<"amqp://guest:password@localhost">>, - params = #amqp_params_direct{password = <<"password">>} - }, - ObfuscatedUpstreamParams = rabbit_federation_util:obfuscate_upstream_params(UpstreamParams), - ?assertEqual(UpstreamParams, rabbit_federation_util:deobfuscate_upstream_params(ObfuscatedUpstreamParams)), - ok. diff --git a/deps/rabbitmq_federation/test/unit_inbroker_SUITE.erl b/deps/rabbitmq_federation/test/unit_inbroker_SUITE.erl deleted file mode 100644 index 83b44579c74f..000000000000 --- a/deps/rabbitmq_federation/test/unit_inbroker_SUITE.erl +++ /dev/null @@ -1,231 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. -%% - --module(unit_inbroker_SUITE). - --include_lib("rabbit_common/include/rabbit.hrl"). --include_lib("eunit/include/eunit.hrl"). - --include("rabbit_federation.hrl"). - --compile(export_all). - --define(US_NAME, <<"upstream">>). --define(DS_NAME, <<"fed.downstream">>). - -all() -> - [ - {group, non_parallel_tests} - ]. - -groups() -> - [ - {non_parallel_tests, [], [ - serialisation, - scratch_space, - remove_credentials, - get_connection_name, - upstream_validation, - upstream_set_validation - ]} - ]. - -%% ------------------------------------------------------------------- -%% Testsuite setup/teardown. -%% ------------------------------------------------------------------- - -init_per_suite(Config) -> - rabbit_ct_helpers:log_environment(), - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodename_suffix, ?MODULE} - ]), - rabbit_ct_helpers:run_setup_steps(Config1, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()). - -end_per_suite(Config) -> - rabbit_ct_helpers:run_teardown_steps(Config, - rabbit_ct_client_helpers:teardown_steps() ++ - rabbit_ct_broker_helpers:teardown_steps()). - -init_per_group(_, Config) -> - Config. - -end_per_group(_, Config) -> - Config. - -init_per_testcase(Testcase, Config) -> - rabbit_ct_helpers:testcase_started(Config, Testcase). - -end_per_testcase(Testcase, Config) -> - rabbit_ct_helpers:testcase_finished(Config, Testcase). - -%% ------------------------------------------------------------------- -%% Testcases. -%% ------------------------------------------------------------------- - -%% Test that we apply binding changes in the correct order even when -%% they arrive out of order. -serialisation(Config) -> - ok = rabbit_ct_broker_helpers:rpc(Config, 0, - ?MODULE, serialisation1, []). - -serialisation1() -> - with_exchanges( - fun(X) -> - [B1, B2, B3] = [b(K) || K <- [<<"1">>, <<"2">>, <<"3">>]], - remove_bindings(4, X, [B1, B3]), - add_binding(5, X, B1), - add_binding(1, X, B1), - add_binding(2, X, B2), - add_binding(3, X, B3), - %% List of lists because one for each link - Keys = rabbit_federation_exchange_link:list_routing_keys( - X#exchange.name), - [[<<"1">>, <<"2">>]] =:= Keys - end). - -scratch_space(Config) -> - ok = rabbit_ct_broker_helpers:rpc(Config, 0, - ?MODULE, scratch_space1, []). - -scratch_space1() -> - A = <<"A">>, - B = <<"B">>, - DB = rabbit_federation_db, - with_exchanges( - fun(#exchange{name = N}) -> - DB:set_active_suffix(N, upstream(x), A), - DB:set_active_suffix(N, upstream(y), A), - DB:prune_scratch(N, [upstream(y), upstream(z)]), - DB:set_active_suffix(N, upstream(y), B), - DB:set_active_suffix(N, upstream(z), A), - none = DB:get_active_suffix(N, upstream(x), none), - B = DB:get_active_suffix(N, upstream(y), none), - A = DB:get_active_suffix(N, upstream(z), none) - end). - -remove_credentials(Config) -> - Test0 = fun (In, Exp) -> - Act = rabbit_ct_broker_helpers:rpc(Config, 0, - rabbit_federation_upstream, remove_credentials, [In]), - Exp = Act - end, - Cat = fun (Bs) -> - list_to_binary(lists:append([binary_to_list(B) || B <- Bs])) - end, - Test = fun (Scheme, Rest) -> - Exp = Cat([Scheme, Rest]), - Test0(Exp, Exp), - Test0(Cat([Scheme, <<"user@">>, Rest]), Exp), - Test0(Cat([Scheme, <<"user:pass@">>, Rest]), Exp) - end, - Test(<<"amqp://">>, <<"">>), - Test(<<"amqp://">>, <<"localhost">>), - Test(<<"amqp://">>, <<"localhost/">>), - Test(<<"amqp://">>, <<"localhost/foo">>), - Test(<<"amqp://">>, <<"localhost:5672">>), - Test(<<"amqp://">>, <<"localhost:5672/foo">>), - Test(<<"amqps://">>, <<"localhost:5672/%2f">>), - ok. - -get_connection_name(Config) -> - Amqqueue = rabbit_ct_broker_helpers:rpc( - Config, 0, - amqqueue, new, [rabbit_misc:r(<<"/">>, queue, <<"queue">>), - self(), - false, - false, - none, - [], - undefined, - #{}, - classic]), - AmqqueueWithPolicy = amqqueue:set_policy(Amqqueue, [{name, <<"my.federation.policy">>}]), - AmqqueueWithEmptyPolicy = amqqueue:set_policy(Amqqueue, []), - - - <<"Federation link (upstream: my.upstream, policy: my.federation.policy)">> = rabbit_federation_link_util:get_connection_name( - #upstream{name = <<"my.upstream">>}, - #upstream_params{x_or_q = AmqqueueWithPolicy} - ), - <<"Federation link (upstream: my.upstream, policy: my.federation.policy)">> = rabbit_federation_link_util:get_connection_name( - #upstream{name = <<"my.upstream">>}, - #upstream_params{x_or_q = #exchange{policy = [{name, <<"my.federation.policy">>}]}} - ), - <<"Federation link">> = rabbit_federation_link_util:get_connection_name( - #upstream{}, - #upstream_params{x_or_q = AmqqueueWithEmptyPolicy} - ), - <<"Federation link">> = rabbit_federation_link_util:get_connection_name( - #upstream{}, - #upstream_params{x_or_q = #exchange{policy = []}} - ), - <<"Federation link">> = rabbit_federation_link_util:get_connection_name( - whatever, - whatever - ), - ok. - -upstream_set_validation(_Config) -> - ?assertEqual(rabbit_federation_parameters:validate(<<"/">>, <<"federation-upstream-set">>, - <<"a-name">>, - [[{<<"upstream">>, <<"devtest1">>}], - [{<<"upstream">>, <<"devtest2">>}]], - <<"acting-user">>), - [[ok], [ok]]), - ?assertEqual(rabbit_federation_parameters:validate(<<"/">>, <<"federation-upstream-set">>, - <<"a-name">>, - [#{<<"upstream">> => <<"devtest3">>}, - #{<<"upstream">> => <<"devtest4">>}], - <<"acting-user">>), - [[ok], [ok]]), - ok. - -upstream_validation(_Config) -> - ?assertEqual(rabbit_federation_parameters:validate(<<"/">>, <<"federation-upstream">>, - <<"a-name">>, - [{<<"uri">>, <<"amqp://127.0.0.1/%2f">>}], - <<"acting-user">>), - [ok]), - ?assertEqual(rabbit_federation_parameters:validate(<<"/">>, <<"federation-upstream">>, - <<"a-name">>, - #{<<"uri">> => <<"amqp://127.0.0.1/%2f">>}, - <<"acting-user">>), - [ok]), - ok. - -with_exchanges(Fun) -> - {ok, _} = rabbit_exchange:declare( - r(?US_NAME), fanout, false, false, false, [], - <<"acting-user">>), - {ok, X} = rabbit_exchange:declare( - r(?DS_NAME), fanout, false, false, false, [], - <<"acting-user">>), - Fun(X), - %% Delete downstream first or it will recreate the upstream - rabbit_exchange:delete(r(?DS_NAME), false, <<"acting-user">>), - rabbit_exchange:delete(r(?US_NAME), false, <<"acting-user">>), - ok. - -add_binding(Ser, X, B) -> - rabbit_federation_exchange:add_binding(transaction, X, B), - rabbit_federation_exchange:add_binding(Ser, X, B). - -remove_bindings(Ser, X, Bs) -> - rabbit_federation_exchange:remove_bindings(transaction, X, Bs), - rabbit_federation_exchange:remove_bindings(Ser, X, Bs). - -r(Name) -> rabbit_misc:r(<<"/">>, exchange, Name). - -b(Key) -> - #binding{source = ?DS_NAME, destination = <<"whatever">>, - key = Key, args = []}. - -upstream(UpstreamName) -> - #upstream{name = atom_to_list(UpstreamName), - exchange_name = <<"upstream">>}. From e7e83bb02e8f50daae68861e4e7b3250c3005677 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Tue, 20 May 2025 16:26:52 +0200 Subject: [PATCH 3/9] Update federation management deps --- deps/rabbitmq_federation_management/Makefile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/deps/rabbitmq_federation_management/Makefile b/deps/rabbitmq_federation_management/Makefile index 71ecae0fa504..960be5f55366 100644 --- a/deps/rabbitmq_federation_management/Makefile +++ b/deps/rabbitmq_federation_management/Makefile @@ -5,8 +5,8 @@ define PROJECT_APP_EXTRA_KEYS {broker_version_requirements, []} endef -DEPS = amqp_client rabbit_common rabbit rabbitmq_management rabbitmq_federation -TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers +DEPS = amqp_client rabbit_common rabbit rabbitmq_management rabbitmq_federation_common +TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_queue_federation rabbitmq_exchange_federation DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-plugin.mk DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk From 9f785d5af658b980e9b260b800d467e9e229ae3c Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Tue, 20 May 2025 16:58:40 +0200 Subject: [PATCH 4/9] Update Github workflows with new federation plugins --- .github/workflows/test-make-tests.yaml | 4 +++- .github/workflows/test-make-type-check.yaml | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/.github/workflows/test-make-tests.yaml b/.github/workflows/test-make-tests.yaml index a4ffd93c453c..cdffd87189d7 100644 --- a/.github/workflows/test-make-tests.yaml +++ b/.github/workflows/test-make-tests.yaml @@ -86,7 +86,8 @@ jobs: - rabbitmq_cli - rabbitmq_consistent_hash_exchange - rabbitmq_event_exchange - - rabbitmq_federation + - rabbitmq_exchange_federation + - rabbitmq_federation_common - rabbitmq_federation_management - rabbitmq_federation_prometheus - rabbitmq_jms_topic_exchange @@ -98,6 +99,7 @@ jobs: - rabbitmq_peer_discovery_k8s - rabbitmq_prelaunch - rabbitmq_prometheus + - rabbitmq_queue_federation - rabbitmq_recent_history_exchange - rabbitmq_sharding - rabbitmq_shovel diff --git a/.github/workflows/test-make-type-check.yaml b/.github/workflows/test-make-type-check.yaml index bf977874aff9..d1459bceeb26 100644 --- a/.github/workflows/test-make-type-check.yaml +++ b/.github/workflows/test-make-type-check.yaml @@ -34,7 +34,8 @@ jobs: - rabbitmq_aws - rabbitmq_consistent_hash_exchange - rabbitmq_event_exchange - - rabbitmq_federation + - rabbitmq_exchange_federation + - rabbitmq_federation_common - rabbitmq_federation_management - rabbitmq_federation_prometheus - rabbitmq_jms_topic_exchange @@ -50,6 +51,7 @@ jobs: - rabbitmq_peer_discovery_k8s - rabbitmq_prelaunch - rabbitmq_prometheus + - rabbitmq_queue_federation - rabbitmq_recent_history_exchange - rabbitmq_sharding - rabbitmq_shovel From 889c5a74ecc91c2e076f863e22bfdd9baf335ada Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Tue, 20 May 2025 18:15:30 +0200 Subject: [PATCH 5/9] Update rabbitmq_federation_prometheus dependencies --- deps/rabbitmq_federation_prometheus/Makefile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/deps/rabbitmq_federation_prometheus/Makefile b/deps/rabbitmq_federation_prometheus/Makefile index 81e2b259b7b4..c7955f4b5e5b 100644 --- a/deps/rabbitmq_federation_prometheus/Makefile +++ b/deps/rabbitmq_federation_prometheus/Makefile @@ -6,8 +6,8 @@ define PROJECT_APP_EXTRA_KEYS {broker_version_requirements, []} endef -DEPS = rabbit_common rabbit rabbitmq_federation rabbitmq_prometheus -TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers +DEPS = rabbit_common rabbit rabbitmq_federation_common rabbitmq_prometheus +TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_queue_federation rabbitmq_exchange_federation DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-plugin.mk DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk From d55ebb3d9a8d7c76d60f7fa5a93281147b7c0488 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Tue, 20 May 2025 18:18:47 +0200 Subject: [PATCH 6/9] Update rabbitmq_mqtt federation test dependency --- deps/rabbitmq_mqtt/Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/deps/rabbitmq_mqtt/Makefile b/deps/rabbitmq_mqtt/Makefile index fde095031a52..27fc5846ccd9 100644 --- a/deps/rabbitmq_mqtt/Makefile +++ b/deps/rabbitmq_mqtt/Makefile @@ -43,7 +43,7 @@ export BUILD_WITHOUT_QUIC LOCAL_DEPS = ssl DEPS = ranch rabbit amqp10_common -TEST_DEPS = cowlib emqtt ct_helper rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_management amqp_client rabbitmq_consistent_hash_exchange rabbitmq_amqp_client rabbitmq_stomp rabbitmq_stream rabbitmq_federation +TEST_DEPS = cowlib emqtt ct_helper rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_management amqp_client rabbitmq_consistent_hash_exchange rabbitmq_amqp_client rabbitmq_stomp rabbitmq_stream rabbitmq_exchange_federation PLT_APPS += rabbitmq_cli elixir From 628dce03a26c624fc432bb0f11c234e0e6c4f614 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Wed, 21 May 2025 12:50:59 +0200 Subject: [PATCH 7/9] Update rabbitmq_cli federation test dependency --- deps/rabbitmq_cli/Makefile | 2 +- .../test/ctl/list_parameters_command_test.exs | 2 +- .../plugins/disable_plugins_command_test.exs | 42 ++++++------- .../plugins/enable_plugins_command_test.exs | 54 ++++++++--------- .../test/plugins/is_enabled_command_test.exs | 2 +- .../plugins/list_plugins_command_test.exs | 60 +++++++++---------- .../test/plugins/plugins_formatter_test.exs | 16 ++--- .../test/plugins/set_plugins_command_test.exs | 34 +++++------ deps/rabbitmq_cli/test/test_helper.exs | 4 +- 9 files changed, 108 insertions(+), 108 deletions(-) diff --git a/deps/rabbitmq_cli/Makefile b/deps/rabbitmq_cli/Makefile index ac74acc6880d..ee55b9a13007 100644 --- a/deps/rabbitmq_cli/Makefile +++ b/deps/rabbitmq_cli/Makefile @@ -122,7 +122,7 @@ $(LINKED_ESCRIPTS): $(ESCRIPT_FILE) tests:: escript test-deps $(verbose) $(MAKE) -C ../../ install-cli $(verbose) $(MAKE) -C ../../ start-background-broker \ - PLUGINS="rabbitmq_federation rabbitmq_stomp rabbitmq_stream_management amqp_client" \ + PLUGINS="rabbitmq_federation_common rabbitmq_stomp rabbitmq_stream_management amqp_client" \ $(if $(filter khepri,$(RABBITMQ_METADATA_STORE)),,RABBITMQ_FEATURE_FLAGS="-khepri_db") $(gen_verbose) $(MIX_TEST) \ $(if $(RABBITMQ_METADATA_STORE),--exclude $(filter-out $(RABBITMQ_METADATA_STORE),khepri mnesia),) \ diff --git a/deps/rabbitmq_cli/test/ctl/list_parameters_command_test.exs b/deps/rabbitmq_cli/test/ctl/list_parameters_command_test.exs index 022b41e814be..eb21bc055fce 100644 --- a/deps/rabbitmq_cli/test/ctl/list_parameters_command_test.exs +++ b/deps/rabbitmq_cli/test/ctl/list_parameters_command_test.exs @@ -37,7 +37,7 @@ defmodule ListParametersCommandTest do rabbitmq_home: rabbitmq_home } - set_enabled_plugins([:rabbitmq_stomp, :rabbitmq_federation], :online, node, opts) + set_enabled_plugins([:rabbitmq_stomp, :rabbitmq_federation_common], :online, node, opts) add_vhost(@vhost) diff --git a/deps/rabbitmq_cli/test/plugins/disable_plugins_command_test.exs b/deps/rabbitmq_cli/test/plugins/disable_plugins_command_test.exs index f8b5ef5a644b..19938f83dda1 100644 --- a/deps/rabbitmq_cli/test/plugins/disable_plugins_command_test.exs +++ b/deps/rabbitmq_cli/test/plugins/disable_plugins_command_test.exs @@ -52,7 +52,7 @@ defmodule DisablePluginsCommandTest do setup context do set_enabled_plugins( - [:rabbitmq_stomp, :rabbitmq_federation], + [:rabbitmq_stomp, :rabbitmq_federation_common], :online, get_rabbit_hostname(), context[:opts] @@ -105,14 +105,14 @@ defmodule DisablePluginsCommandTest do @command.run(["rabbitmq_stomp"], Map.merge(context[:opts], %{node: :nonode})) assert [ - [:rabbitmq_federation], - %{mode: :offline, disabled: [:rabbitmq_stomp], set: [:rabbitmq_federation]} + [:rabbitmq_federation_common], + %{mode: :offline, disabled: [:rabbitmq_stomp], set: [:rabbitmq_federation_common]} ] == Enum.to_list(test_stream) - assert {:ok, [[:rabbitmq_federation]]} == :file.consult(context[:opts][:enabled_plugins_file]) + assert {:ok, [[:rabbitmq_federation_common]]} == :file.consult(context[:opts][:enabled_plugins_file]) - assert [:amqp_client, :rabbitmq_federation, :rabbitmq_stomp] == + assert [:amqp_client, :rabbitmq_federation_common, :rabbitmq_stomp] == Enum.sort(:rabbit_misc.rpc_call(context[:opts][:node], :rabbit_plugins, :active, [])) end @@ -125,13 +125,13 @@ defmodule DisablePluginsCommandTest do ) assert [ - [:rabbitmq_federation], - %{mode: :offline, disabled: [:rabbitmq_stomp], set: [:rabbitmq_federation]} + [:rabbitmq_federation_common], + %{mode: :offline, disabled: [:rabbitmq_stomp], set: [:rabbitmq_federation_common]} ] == Enum.to_list(test_stream) - assert {:ok, [[:rabbitmq_federation]]} == :file.consult(context[:opts][:enabled_plugins_file]) + assert {:ok, [[:rabbitmq_federation_common]]} == :file.consult(context[:opts][:enabled_plugins_file]) - assert [:amqp_client, :rabbitmq_federation, :rabbitmq_stomp] == + assert [:amqp_client, :rabbitmq_federation_common, :rabbitmq_stomp] == Enum.sort(:rabbit_misc.rpc_call(context[:opts][:node], :rabbit_plugins, :active, [])) end @@ -139,13 +139,13 @@ defmodule DisablePluginsCommandTest do context do assert {:stream, test_stream0} = @command.run( - ["rabbitmq_federation"], + ["rabbitmq_federation_common"], Map.merge(context[:opts], %{offline: true, online: false}) ) assert [ [:rabbitmq_stomp], - %{mode: :offline, disabled: [:rabbitmq_federation], set: [:rabbitmq_stomp]} + %{mode: :offline, disabled: [:rabbitmq_federation_common], set: [:rabbitmq_stomp]} ] == Enum.to_list(test_stream0) assert {:ok, [[:rabbitmq_stomp]]} == :file.consult(context[:opts][:enabled_plugins_file]) @@ -166,31 +166,31 @@ defmodule DisablePluginsCommandTest do assert {:stream, test_stream0} = @command.run(["rabbitmq_stomp"], context[:opts]) assert [ - [:rabbitmq_federation], + [:rabbitmq_federation_common], %{ mode: :online, started: [], stopped: [:rabbitmq_stomp], disabled: [:rabbitmq_stomp], - set: [:rabbitmq_federation] + set: [:rabbitmq_federation_common] } ] == Enum.to_list(test_stream0) - assert {:ok, [[:rabbitmq_federation]]} == :file.consult(context[:opts][:enabled_plugins_file]) + assert {:ok, [[:rabbitmq_federation_common]]} == :file.consult(context[:opts][:enabled_plugins_file]) - assert [:amqp_client, :rabbitmq_federation] == + assert [:amqp_client, :rabbitmq_federation_common] == Enum.sort(:rabbit_misc.rpc_call(context[:opts][:node], :rabbit_plugins, :active, [])) - assert {:stream, test_stream1} = @command.run(["rabbitmq_federation"], context[:opts]) + assert {:stream, test_stream1} = @command.run(["rabbitmq_federation_common"], context[:opts]) assert [ [], %{ mode: :online, started: [], - stopped: [:rabbitmq_federation], - disabled: [:rabbitmq_federation], + stopped: [:rabbitmq_federation_common], + disabled: [:rabbitmq_federation_common], set: [] } ] == @@ -205,7 +205,7 @@ defmodule DisablePluginsCommandTest do test "can disable multiple plugins at once", context do assert {:stream, test_stream} = - @command.run(["rabbitmq_stomp", "rabbitmq_federation"], context[:opts]) + @command.run(["rabbitmq_stomp", "rabbitmq_federation_common"], context[:opts]) [[], m0] = Enum.to_list(test_stream) @@ -214,7 +214,7 @@ defmodule DisablePluginsCommandTest do |> Map.update!(:stopped, &Enum.sort/1) |> Map.update!(:disabled, &Enum.sort/1) - expected_list = Enum.sort([:rabbitmq_federation, :rabbitmq_stomp]) + expected_list = Enum.sort([:rabbitmq_federation_common, :rabbitmq_stomp]) assert [ [], @@ -243,7 +243,7 @@ defmodule DisablePluginsCommandTest do |> Map.update!(:stopped, &Enum.sort/1) |> Map.update!(:disabled, &Enum.sort/1) - expected_list = Enum.sort([:rabbitmq_federation, :rabbitmq_stomp]) + expected_list = Enum.sort([:rabbitmq_federation_common, :rabbitmq_stomp]) assert [ [], diff --git a/deps/rabbitmq_cli/test/plugins/enable_plugins_command_test.exs b/deps/rabbitmq_cli/test/plugins/enable_plugins_command_test.exs index 424a9ade1aad..6740cd89e889 100644 --- a/deps/rabbitmq_cli/test/plugins/enable_plugins_command_test.exs +++ b/deps/rabbitmq_cli/test/plugins/enable_plugins_command_test.exs @@ -66,7 +66,7 @@ defmodule EnablePluginsCommandTest do def reset_enabled_plugins_to_preconfigured_defaults(context) do set_enabled_plugins( - [:rabbitmq_stomp, :rabbitmq_federation], + [:rabbitmq_stomp, :rabbitmq_federation_common], :online, get_rabbit_hostname(), context[:opts] @@ -120,7 +120,7 @@ defmodule EnablePluginsCommandTest do check_plugins_enabled([:rabbitmq_stomp], context) - assert [:amqp_client, :rabbitmq_federation, :rabbitmq_stomp] == + assert [:amqp_client, :rabbitmq_federation_common, :rabbitmq_stomp] == currently_active_plugins(context) end @@ -144,7 +144,7 @@ defmodule EnablePluginsCommandTest do check_plugins_enabled([:rabbitmq_stomp], context) assert_equal_sets( - [:amqp_client, :rabbitmq_federation, :rabbitmq_stomp], + [:amqp_client, :rabbitmq_federation_common, :rabbitmq_stomp], currently_active_plugins(context) ) end @@ -169,21 +169,21 @@ defmodule EnablePluginsCommandTest do assert {:stream, test_stream1} = @command.run( - ["rabbitmq_federation"], + ["rabbitmq_federation_common"], Map.merge(context[:opts], %{offline: true, online: false}) ) assert [ - [:rabbitmq_federation, :rabbitmq_stomp], + [:rabbitmq_federation_common, :rabbitmq_stomp], %{ mode: :offline, - enabled: [:rabbitmq_federation], - set: [:rabbitmq_federation, :rabbitmq_stomp] + enabled: [:rabbitmq_federation_common], + set: [:rabbitmq_federation_common, :rabbitmq_stomp] } ] == Enum.to_list(test_stream1) - check_plugins_enabled([:rabbitmq_stomp, :rabbitmq_federation], context) + check_plugins_enabled([:rabbitmq_stomp, :rabbitmq_federation_common], context) end test "run: updates plugin list and starts newly enabled plugins", context do @@ -207,24 +207,24 @@ defmodule EnablePluginsCommandTest do check_plugins_enabled([:rabbitmq_stomp], context) assert_equal_sets([:amqp_client, :rabbitmq_stomp], currently_active_plugins(context)) - {:stream, test_stream1} = @command.run(["rabbitmq_federation"], context[:opts]) + {:stream, test_stream1} = @command.run(["rabbitmq_federation_common"], context[:opts]) assert [ - [:rabbitmq_federation, :rabbitmq_stomp], + [:rabbitmq_federation_common, :rabbitmq_stomp], %{ mode: :online, - started: [:rabbitmq_federation], + started: [:rabbitmq_federation_common], stopped: [], - enabled: [:rabbitmq_federation], - set: [:rabbitmq_federation, :rabbitmq_stomp] + enabled: [:rabbitmq_federation_common], + set: [:rabbitmq_federation_common, :rabbitmq_stomp] } ] == Enum.to_list(test_stream1) - check_plugins_enabled([:rabbitmq_stomp, :rabbitmq_federation], context) + check_plugins_enabled([:rabbitmq_stomp, :rabbitmq_federation_common], context) assert_equal_sets( - [:amqp_client, :rabbitmq_federation, :rabbitmq_stomp], + [:amqp_client, :rabbitmq_federation_common, :rabbitmq_stomp], currently_active_plugins(context) ) @@ -236,24 +236,24 @@ defmodule EnablePluginsCommandTest do set_enabled_plugins([], :online, context[:opts][:node], context[:opts]) assert {:stream, test_stream} = - @command.run(["rabbitmq_stomp", "rabbitmq_federation"], context[:opts]) + @command.run(["rabbitmq_stomp", "rabbitmq_federation_common"], context[:opts]) assert [ - [:rabbitmq_federation, :rabbitmq_stomp], + [:rabbitmq_federation_common, :rabbitmq_stomp], %{ mode: :online, - started: [:rabbitmq_federation, :rabbitmq_stomp], + started: [:rabbitmq_federation_common, :rabbitmq_stomp], stopped: [], - enabled: [:rabbitmq_federation, :rabbitmq_stomp], - set: [:rabbitmq_federation, :rabbitmq_stomp] + enabled: [:rabbitmq_federation_common, :rabbitmq_stomp], + set: [:rabbitmq_federation_common, :rabbitmq_stomp] } ] == Enum.to_list(test_stream) - check_plugins_enabled([:rabbitmq_stomp, :rabbitmq_federation], context) + check_plugins_enabled([:rabbitmq_stomp, :rabbitmq_federation_common], context) assert_equal_sets( - [:amqp_client, :rabbitmq_federation, :rabbitmq_stomp], + [:amqp_client, :rabbitmq_federation_common, :rabbitmq_stomp], currently_active_plugins(context) ) @@ -262,18 +262,18 @@ defmodule EnablePluginsCommandTest do test "run: does not enable an already implicitly enabled plugin", context do # Clears enabled plugins file and stop all plugins - set_enabled_plugins([:rabbitmq_federation], :online, context[:opts][:node], context[:opts]) + set_enabled_plugins([:rabbitmq_federation_common], :online, context[:opts][:node], context[:opts]) assert {:stream, test_stream} = @command.run(["amqp_client"], context[:opts]) assert [ - [:rabbitmq_federation], - %{mode: :online, started: [], stopped: [], enabled: [], set: [:rabbitmq_federation]} + [:rabbitmq_federation_common], + %{mode: :online, started: [], stopped: [], enabled: [], set: [:rabbitmq_federation_common]} ] == Enum.to_list(test_stream) - check_plugins_enabled([:rabbitmq_federation], context) + check_plugins_enabled([:rabbitmq_federation_common], context) - assert [:amqp_client, :rabbitmq_federation] == + assert [:amqp_client, :rabbitmq_federation_common] == currently_active_plugins(context) reset_enabled_plugins_to_preconfigured_defaults(context) diff --git a/deps/rabbitmq_cli/test/plugins/is_enabled_command_test.exs b/deps/rabbitmq_cli/test/plugins/is_enabled_command_test.exs index 3fdc83350735..39d9b4d5c8df 100644 --- a/deps/rabbitmq_cli/test/plugins/is_enabled_command_test.exs +++ b/deps/rabbitmq_cli/test/plugins/is_enabled_command_test.exs @@ -99,7 +99,7 @@ defmodule PluginIsEnabledCommandTest do assert match?( {:ok, _}, - assert(@command.run(["rabbitmq_stomp", "rabbitmq_federation"], opts)) + assert(@command.run(["rabbitmq_stomp", "rabbitmq_federation_common"], opts)) ) end diff --git a/deps/rabbitmq_cli/test/plugins/list_plugins_command_test.exs b/deps/rabbitmq_cli/test/plugins/list_plugins_command_test.exs index 4bd6fb764c80..8ec2936804a2 100644 --- a/deps/rabbitmq_cli/test/plugins/list_plugins_command_test.exs +++ b/deps/rabbitmq_cli/test/plugins/list_plugins_command_test.exs @@ -13,7 +13,7 @@ defmodule ListPluginsCommandTest do def reset_enabled_plugins_to_preconfigured_defaults(context) do set_enabled_plugins( - [:rabbitmq_stomp, :rabbitmq_federation], + [:rabbitmq_stomp, :rabbitmq_federation_common], :online, get_rabbit_hostname(), context[:opts] @@ -124,7 +124,7 @@ defmodule ListPluginsCommandTest do end) expected_plugins = [ - %{name: :rabbitmq_federation, enabled: :enabled, running: false}, + %{name: :rabbitmq_federation_common, enabled: :enabled, running: false}, %{name: :rabbitmq_stomp, enabled: :enabled, running: false} ] @@ -146,7 +146,7 @@ defmodule ListPluginsCommandTest do end) expected_plugins = [ - %{name: :rabbitmq_federation, enabled: :enabled, running: true}, + %{name: :rabbitmq_federation_common, enabled: :enabled, running: true}, %{name: :rabbitmq_stomp, enabled: :enabled, running: false} ] @@ -163,7 +163,7 @@ defmodule ListPluginsCommandTest do expected_plugins = [ %{ - name: :rabbitmq_federation, + name: :rabbitmq_federation_common, enabled: :enabled, running: true, dependencies: [:amqp_client] @@ -186,7 +186,7 @@ defmodule ListPluginsCommandTest do test "run: reports plugin names in minimal mode", context do reset_enabled_plugins_to_preconfigured_defaults(context) - expected_plugins = [%{name: :rabbitmq_federation}, %{name: :rabbitmq_stomp}] + expected_plugins = [%{name: :rabbitmq_federation_common}, %{name: :rabbitmq_stomp}] %{status: :running, plugins: actual_plugins} = @command.run([".*"], Map.merge(context[:opts], %{minimal: true})) @@ -196,11 +196,11 @@ defmodule ListPluginsCommandTest do test "run: by default lists all plugins", context do reset_enabled_plugins_to_preconfigured_defaults(context) - set_enabled_plugins([:rabbitmq_federation], :online, context[:opts][:node], context[:opts]) + set_enabled_plugins([:rabbitmq_federation_common], :online, context[:opts][:node], context[:opts]) on_exit(fn -> set_enabled_plugins( - [:rabbitmq_stomp, :rabbitmq_federation], + [:rabbitmq_stomp, :rabbitmq_federation_common], :online, context[:opts][:node], context[:opts] @@ -208,7 +208,7 @@ defmodule ListPluginsCommandTest do end) expected_plugins = [ - %{name: :rabbitmq_federation, enabled: :enabled, running: true}, + %{name: :rabbitmq_federation_common, enabled: :enabled, running: true}, %{name: :rabbitmq_stomp, enabled: :not_enabled, running: false} ] @@ -221,18 +221,18 @@ defmodule ListPluginsCommandTest do test "run: with --enabled flag, lists only explicitly enabled plugins", context do reset_enabled_plugins_to_preconfigured_defaults(context) - set_enabled_plugins([:rabbitmq_federation], :online, context[:opts][:node], context[:opts]) + set_enabled_plugins([:rabbitmq_federation_common], :online, context[:opts][:node], context[:opts]) on_exit(fn -> set_enabled_plugins( - [:rabbitmq_stomp, :rabbitmq_federation], + [:rabbitmq_stomp, :rabbitmq_federation_common], :online, context[:opts][:node], context[:opts] ) end) - expected_plugins = [%{name: :rabbitmq_federation, enabled: :enabled, running: true}] + expected_plugins = [%{name: :rabbitmq_federation_common, enabled: :enabled, running: true}] %{ status: :running, @@ -245,18 +245,18 @@ defmodule ListPluginsCommandTest do test "run: with --implicitly_enabled flag lists explicitly and implicitly enabled plugins", context do reset_enabled_plugins_to_preconfigured_defaults(context) - set_enabled_plugins([:rabbitmq_federation], :online, context[:opts][:node], context[:opts]) + set_enabled_plugins([:rabbitmq_federation_common], :online, context[:opts][:node], context[:opts]) on_exit(fn -> set_enabled_plugins( - [:rabbitmq_stomp, :rabbitmq_federation], + [:rabbitmq_stomp, :rabbitmq_federation_common], :online, context[:opts][:node], context[:opts] ) end) - expected_plugins = [%{name: :rabbitmq_federation, enabled: :enabled, running: true}] + expected_plugins = [%{name: :rabbitmq_federation_common, enabled: :enabled, running: true}] %{ status: :running, @@ -268,11 +268,11 @@ defmodule ListPluginsCommandTest do test "run: filters plugins by name with pattern provided", context do reset_enabled_plugins_to_preconfigured_defaults(context) - set_enabled_plugins([:rabbitmq_federation], :online, context[:opts][:node], context[:opts]) + set_enabled_plugins([:rabbitmq_federation_common], :online, context[:opts][:node], context[:opts]) on_exit(fn -> set_enabled_plugins( - [:rabbitmq_stomp, :rabbitmq_federation], + [:rabbitmq_stomp, :rabbitmq_federation_common], :online, context[:opts][:node], context[:opts] @@ -282,7 +282,7 @@ defmodule ListPluginsCommandTest do %{status: :running, plugins: actual_plugins} = @command.run(["fede"], Map.merge(context[:opts], %{minimal: true})) - assert_plugin_states(actual_plugins, [%{name: :rabbitmq_federation}]) + assert_plugin_states(actual_plugins, [%{name: :rabbitmq_federation_common}]) %{status: :running, plugins: actual_plugins2} = @command.run(["stomp$"], Map.merge(context[:opts], %{minimal: true})) @@ -295,7 +295,7 @@ defmodule ListPluginsCommandTest do opts = get_opts_with_non_existing_plugins_directory(context) expected_plugins = [ - %{name: :rabbitmq_federation}, + %{name: :rabbitmq_federation_common}, %{name: :rabbitmq_stomp} ] @@ -310,7 +310,7 @@ defmodule ListPluginsCommandTest do reset_enabled_plugins_to_preconfigured_defaults(context) opts = get_opts_with_existing_plugins_directory(context) - expected_plugins = [%{name: :rabbitmq_federation}, %{name: :rabbitmq_stomp}] + expected_plugins = [%{name: :rabbitmq_federation_common}, %{name: :rabbitmq_stomp}] %{status: :running, plugins: actual_plugins} = @command.run([".*"], Map.merge(opts, %{minimal: true})) @@ -329,7 +329,7 @@ defmodule ListPluginsCommandTest do expected_plugins = [ %{name: :mock_rabbitmq_plugins_01}, %{name: :mock_rabbitmq_plugins_02}, - %{name: :rabbitmq_federation}, + %{name: :rabbitmq_federation_common}, %{name: :rabbitmq_stomp} ] @@ -365,7 +365,7 @@ defmodule ListPluginsCommandTest do running: false, version: ~c"0.2.0" }, - %{name: :rabbitmq_federation, enabled: :enabled, running: true}, + %{name: :rabbitmq_federation_common, enabled: :enabled, running: true}, %{name: :rabbitmq_stomp, enabled: :enabled, running: true} ] @@ -385,7 +385,7 @@ defmodule ListPluginsCommandTest do switch_plugins_directories(context[:opts][:plugins_dir], opts[:plugins_dir]) set_enabled_plugins( - [:mock_rabbitmq_plugins_02, :rabbitmq_federation, :rabbitmq_stomp], + [:mock_rabbitmq_plugins_02, :rabbitmq_federation_common, :rabbitmq_stomp], :online, get_rabbit_hostname(), opts @@ -405,7 +405,7 @@ defmodule ListPluginsCommandTest do version: ~c"0.1.0", running_version: ~c"0.1.0" }, - %{name: :rabbitmq_federation, enabled: :enabled, running: true}, + %{name: :rabbitmq_federation_common, enabled: :enabled, running: true}, %{name: :rabbitmq_stomp, enabled: :enabled, running: true} ] @@ -435,7 +435,7 @@ defmodule ListPluginsCommandTest do version: ~c"0.2.0", running_version: ~c"0.1.0" }, - %{name: :rabbitmq_federation, enabled: :enabled, running: true}, + %{name: :rabbitmq_federation_common, enabled: :enabled, running: true}, %{name: :rabbitmq_stomp, enabled: :enabled, running: true} ] @@ -459,7 +459,7 @@ defmodule ListPluginsCommandTest do reset_enabled_plugins_to_preconfigured_defaults(context) set_enabled_plugins( - [:rabbitmq_federation, missing_plugin], + [:rabbitmq_federation_common, missing_plugin], :online, context[:opts][:node], context[:opts] @@ -474,7 +474,7 @@ defmodule ListPluginsCommandTest do context = Map.replace(context, :opts, opts) set_enabled_plugins( - [:rabbitmq_stomp, :rabbitmq_federation], + [:rabbitmq_stomp, :rabbitmq_federation_common], :online, context[:opts][:node], context[:opts] @@ -482,7 +482,7 @@ defmodule ListPluginsCommandTest do end) expected_plugins = [ - %{name: :rabbitmq_federation, enabled: :enabled, running: true}, + %{name: :rabbitmq_federation_common, enabled: :enabled, running: true}, %{name: :rabbitmq_stomp, enabled: :not_enabled, running: false} ] @@ -520,7 +520,7 @@ defmodule ListPluginsCommandTest do reset_enabled_plugins_to_preconfigured_defaults(context) set_enabled_plugins( - [:rabbitmq_federation, missing_plugin], + [:rabbitmq_federation_common, missing_plugin], :online, context[:opts][:node], context[:opts] @@ -535,7 +535,7 @@ defmodule ListPluginsCommandTest do context = Map.replace(context, :opts, opts) set_enabled_plugins( - [:rabbitmq_stomp, :rabbitmq_federation], + [:rabbitmq_stomp, :rabbitmq_federation_common], :online, context[:opts][:node], context[:opts] @@ -543,7 +543,7 @@ defmodule ListPluginsCommandTest do end) expected_plugins = [ - %{name: :rabbitmq_federation, enabled: :enabled, running: true}, + %{name: :rabbitmq_federation_common, enabled: :enabled, running: true}, %{name: :rabbitmq_stomp, enabled: :not_enabled, running: false} ] diff --git a/deps/rabbitmq_cli/test/plugins/plugins_formatter_test.exs b/deps/rabbitmq_cli/test/plugins/plugins_formatter_test.exs index eda17b519905..4d6d476d93e9 100644 --- a/deps/rabbitmq_cli/test/plugins/plugins_formatter_test.exs +++ b/deps/rabbitmq_cli/test/plugins/plugins_formatter_test.exs @@ -37,7 +37,7 @@ defmodule PluginsFormatterTest do running_version: ~c"0.1.0" }, %{ - name: :rabbitmq_federation, + name: :rabbitmq_federation_common, enabled: :enabled, running: true, version: ~c"3.7.0", @@ -60,7 +60,7 @@ defmodule PluginsFormatterTest do "amqp_client", "mock_rabbitmq_plugins_01", "mock_rabbitmq_plugins_02", - "rabbitmq_federation", + "rabbitmq_federation_common", "rabbitmq_stomp" ] end @@ -93,7 +93,7 @@ defmodule PluginsFormatterTest do running_version: ~c"0.1.0" }, %{ - name: :rabbitmq_federation, + name: :rabbitmq_federation_common, enabled: :enabled, running: true, version: ~c"3.7.0", @@ -116,11 +116,11 @@ defmodule PluginsFormatterTest do " Configured: E = explicitly enabled; e = implicitly enabled", " | Status: * = running on rabbit@localhost", " |/", - "[e*] amqp_client 3.7.0", - "[ ] mock_rabbitmq_plugins_01 0.2.0", - "[E*] mock_rabbitmq_plugins_02 0.1.0 (pending upgrade to 0.2.0)", - "[E*] rabbitmq_federation 3.7.0", - "[E*] rabbitmq_stomp 3.7.0" + "[e*] amqp_client 3.7.0", + "[ ] mock_rabbitmq_plugins_01 0.2.0", + "[E*] mock_rabbitmq_plugins_02 0.1.0 (pending upgrade to 0.2.0)", + "[E*] rabbitmq_federation_common 3.7.0", + "[E*] rabbitmq_stomp 3.7.0" ] end end diff --git a/deps/rabbitmq_cli/test/plugins/set_plugins_command_test.exs b/deps/rabbitmq_cli/test/plugins/set_plugins_command_test.exs index e25af5c1f584..dea77614728a 100644 --- a/deps/rabbitmq_cli/test/plugins/set_plugins_command_test.exs +++ b/deps/rabbitmq_cli/test/plugins/set_plugins_command_test.exs @@ -41,7 +41,7 @@ defmodule SetPluginsCommandTest do setup context do set_enabled_plugins( - [:rabbitmq_stomp, :rabbitmq_federation], + [:rabbitmq_stomp, :rabbitmq_federation_common], :online, get_rabbit_hostname(), context[:opts] @@ -92,7 +92,7 @@ defmodule SetPluginsCommandTest do assert {:ok, [[:rabbitmq_stomp]]} = :file.consult(context[:opts][:enabled_plugins_file]) - assert [:amqp_client, :rabbitmq_federation, :rabbitmq_stomp] = + assert [:amqp_client, :rabbitmq_federation_common, :rabbitmq_stomp] = Enum.sort(:rabbit_misc.rpc_call(context[:opts][:node], :rabbit_plugins, :active, [])) end @@ -108,7 +108,7 @@ defmodule SetPluginsCommandTest do assert {:ok, [[:rabbitmq_stomp]]} = :file.consult(context[:opts][:enabled_plugins_file]) - assert [:amqp_client, :rabbitmq_federation, :rabbitmq_stomp] = + assert [:amqp_client, :rabbitmq_federation_common, :rabbitmq_stomp] = Enum.sort(:rabbit_misc.rpc_call(context[:opts][:node], :rabbit_plugins, :active, [])) end @@ -120,7 +120,7 @@ defmodule SetPluginsCommandTest do %{ mode: :online, started: [], - stopped: [:rabbitmq_federation], + stopped: [:rabbitmq_federation_common], set: [:rabbitmq_stomp] } ] = Enum.to_list(test_stream0) @@ -130,21 +130,21 @@ defmodule SetPluginsCommandTest do assert [:amqp_client, :rabbitmq_stomp] = Enum.sort(:rabbit_misc.rpc_call(context[:opts][:node], :rabbit_plugins, :active, [])) - assert {:stream, test_stream1} = @command.run(["rabbitmq_federation"], context[:opts]) + assert {:stream, test_stream1} = @command.run(["rabbitmq_federation_common"], context[:opts]) assert [ - [:rabbitmq_federation], + [:rabbitmq_federation_common], %{ mode: :online, - started: [:rabbitmq_federation], + started: [:rabbitmq_federation_common], stopped: [:rabbitmq_stomp], - set: [:rabbitmq_federation] + set: [:rabbitmq_federation_common] } ] = Enum.to_list(test_stream1) - assert {:ok, [[:rabbitmq_federation]]} = :file.consult(context[:opts][:enabled_plugins_file]) + assert {:ok, [[:rabbitmq_federation_common]]} = :file.consult(context[:opts][:enabled_plugins_file]) - assert [:amqp_client, :rabbitmq_federation] = + assert [:amqp_client, :rabbitmq_federation_common] = Enum.sort(:rabbit_misc.rpc_call(context[:opts][:node], :rabbit_plugins, :active, [])) end @@ -156,7 +156,7 @@ defmodule SetPluginsCommandTest do %{ mode: :online, started: [], - stopped: [:rabbitmq_federation, :rabbitmq_stomp], + stopped: [:rabbitmq_federation_common, :rabbitmq_stomp], set: [] } ] = Enum.to_list(test_stream) @@ -171,22 +171,22 @@ defmodule SetPluginsCommandTest do set_enabled_plugins([], :online, get_rabbit_hostname(), context[:opts]) assert {:stream, test_stream} = - @command.run(["rabbitmq_federation", "rabbitmq_stomp"], context[:opts]) + @command.run(["rabbitmq_federation_common", "rabbitmq_stomp"], context[:opts]) assert [ - [:rabbitmq_federation, :rabbitmq_stomp], + [:rabbitmq_federation_common, :rabbitmq_stomp], %{ mode: :online, - started: [:rabbitmq_federation, :rabbitmq_stomp], + started: [:rabbitmq_federation_common, :rabbitmq_stomp], stopped: [], - set: [:rabbitmq_federation, :rabbitmq_stomp] + set: [:rabbitmq_federation_common, :rabbitmq_stomp] } ] = Enum.to_list(test_stream) - assert {:ok, [[:rabbitmq_federation, :rabbitmq_stomp]]} = + assert {:ok, [[:rabbitmq_federation_common, :rabbitmq_stomp]]} = :file.consult(context[:opts][:enabled_plugins_file]) - assert [:amqp_client, :rabbitmq_federation, :rabbitmq_stomp] = + assert [:amqp_client, :rabbitmq_federation_common, :rabbitmq_stomp] = Enum.sort(:rabbit_misc.rpc_call(context[:opts][:node], :rabbit_plugins, :active, [])) end diff --git a/deps/rabbitmq_cli/test/test_helper.exs b/deps/rabbitmq_cli/test/test_helper.exs index d7f218715530..6ab840acb504 100644 --- a/deps/rabbitmq_cli/test/test_helper.exs +++ b/deps/rabbitmq_cli/test/test_helper.exs @@ -814,13 +814,13 @@ defmodule TestHelper do plugins = currently_active_plugins(%{opts: %{node: node}}) - case Enum.member?(plugins, :rabbitmq_federation) do + case Enum.member?(plugins, :rabbitmq_federation_common) do true -> :ok false -> set_enabled_plugins( - plugins ++ [:rabbitmq_federation], + plugins ++ [:rabbitmq_federation_common], :online, get_rabbit_hostname(), opts From e54b8d78886c2bd50e55baabf81f0101351287a6 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Wed, 21 May 2025 15:00:20 +0200 Subject: [PATCH 8/9] Skip exchange federation testscase in mixed-version --- .../test/exchange_SUITE.erl | 226 +++++++++--------- 1 file changed, 116 insertions(+), 110 deletions(-) diff --git a/deps/rabbitmq_exchange_federation/test/exchange_SUITE.erl b/deps/rabbitmq_exchange_federation/test/exchange_SUITE.erl index 52b3e6bf2b19..2b65fb731082 100644 --- a/deps/rabbitmq_exchange_federation/test/exchange_SUITE.erl +++ b/deps/rabbitmq_exchange_federation/test/exchange_SUITE.erl @@ -579,116 +579,122 @@ lookup_exchange_status(Config) -> clean_up_federation_related_bits(Config). child_id_format(Config) -> - [UpstreamNode, - OldNodeA, - NewNodeB, - OldNodeC, - NewNodeD] = rabbit_ct_broker_helpers:get_node_configs( - Config, nodename), - - %% Create a cluster with the nodes running the old version of RabbitMQ in - %% mixed-version testing. - %% - %% Note: we build this on the assumption that `rabbit_ct_broker_helpers' - %% starts nodes this way: - %% Node 1: the primary copy of RabbitMQ the test is started from - %% Node 2: the secondary umbrella (if any) - %% Node 3: the primary copy - %% Node 4: the secondary umbrella - %% ... - %% - %% Therefore, `UpstreamNode' will use the primary copy, `OldNodeA' the - %% secondary umbrella, `NewNodeB' the primary copy, and so on. - Config1 = rabbit_ct_broker_helpers:cluster_nodes( - Config, [OldNodeA, OldNodeC]), - - %% Prepare the whole federated exchange on that old cluster. - UpstreamName = <<"fed_on_upgrade">>, - rabbit_ct_broker_helpers:set_parameter( - Config1, OldNodeA, <<"federation-upstream">>, UpstreamName, - [ - {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config1, UpstreamNode)} - ]), - - rabbit_ct_broker_helpers:set_policy( - Config1, OldNodeA, - <<"fed_on_upgrade_policy">>, <<"^fed_">>, <<"all">>, - [ - {<<"federation-upstream-pattern">>, UpstreamName} - ]), - - XName = <<"fed_ex_on_upgrade_cluster">>, - X = exchange_declare_method(XName, <<"direct">>), - {Conn1, Ch1} = rabbit_ct_client_helpers:open_connection_and_channel( - Config1, OldNodeA), - ?assertEqual({'exchange.declare_ok'}, declare_exchange(Ch1, X)), - rabbit_ct_client_helpers:close_channel(Ch1), - rabbit_ct_client_helpers:close_connection(Conn1), - - %% Verify the format of the child ID. In the main branch, the format was - %% temporarily a size-2 tuple with a list as the first element. This was - %% not kept later and the original ID format is used in old and new nodes. - [{Id, _, _, _}] = rabbit_ct_broker_helpers:rpc( - Config1, OldNodeA, - mirrored_supervisor, which_children, - [rabbit_federation_exchange_link_sup_sup]), - case Id of - %% This is the format we expect everywhere. - #exchange{name = #resource{name = XName}} -> - %% Verify that the supervisors exist on all nodes. - lists:foreach( - fun(Node) -> - ?assertMatch( - [{#exchange{name = #resource{name = XName}}, - _, _, _}], - rabbit_ct_broker_helpers:rpc( - Config1, Node, - mirrored_supervisor, which_children, - [rabbit_federation_exchange_link_sup_sup])) - end, [OldNodeA, OldNodeC]), - - %% Simulate a rolling upgrade by: - %% 1. adding new nodes to the old cluster - %% 2. stopping the old nodes - %% - %% After that, the supervisors run on the new code. - Config2 = rabbit_ct_broker_helpers:cluster_nodes( - Config1, OldNodeA, [NewNodeB, NewNodeD]), - ok = rabbit_ct_broker_helpers:stop_broker(Config2, OldNodeA), - ok = rabbit_ct_broker_helpers:reset_node(Config1, OldNodeA), - ok = rabbit_ct_broker_helpers:stop_broker(Config2, OldNodeC), - ok = rabbit_ct_broker_helpers:reset_node(Config2, OldNodeC), - - %% Verify that the supervisors still use the same IDs. - lists:foreach( - fun(Node) -> - ?assertMatch( - [{#exchange{name = #resource{name = XName}}, - _, _, _}], - rabbit_ct_broker_helpers:rpc( - Config2, Node, - mirrored_supervisor, which_children, - [rabbit_federation_exchange_link_sup_sup])) - end, [NewNodeB, NewNodeD]), - - %% Delete the exchange: it should work because the ID format is the - %% one expected. - %% - %% During the transient period where the ID format was changed, - %% this would crash with a badmatch because the running - %% supervisor's ID would not match the content of the database. - {Conn2, Ch2} = rabbit_ct_client_helpers:open_connection_and_channel( - Config2, NewNodeB), - ?assertEqual({'exchange.delete_ok'}, delete_exchange(Ch2, XName)), - rabbit_ct_client_helpers:close_channel(Ch2), - rabbit_ct_client_helpers:close_connection(Conn2); - - %% This is the transient format we are not interested in as it only - %% lived in a development branch. - {List, #exchange{name = #resource{name = XName}}} - when is_list(List) -> - {skip, "Testcase skipped with the transiently changed ID format"} - end. + case rabbit_ct_helpers:is_mixed_versions() of + false -> + [UpstreamNode, + OldNodeA, + NewNodeB, + OldNodeC, + NewNodeD] = rabbit_ct_broker_helpers:get_node_configs( + Config, nodename), + + %% Create a cluster with the nodes running the old version of RabbitMQ in + %% mixed-version testing. + %% + %% Note: we build this on the assumption that `rabbit_ct_broker_helpers' + %% starts nodes this way: + %% Node 1: the primary copy of RabbitMQ the test is started from + %% Node 2: the secondary umbrella (if any) + %% Node 3: the primary copy + %% Node 4: the secondary umbrella + %% ... + %% + %% Therefore, `UpstreamNode' will use the primary copy, `OldNodeA' the + %% secondary umbrella, `NewNodeB' the primary copy, and so on. + Config1 = rabbit_ct_broker_helpers:cluster_nodes( + Config, [OldNodeA, OldNodeC]), + + %% Prepare the whole federated exchange on that old cluster. + UpstreamName = <<"fed_on_upgrade">>, + rabbit_ct_broker_helpers:set_parameter( + Config1, OldNodeA, <<"federation-upstream">>, UpstreamName, + [ + {<<"uri">>, rabbit_ct_broker_helpers:node_uri(Config1, UpstreamNode)} + ]), + + rabbit_ct_broker_helpers:set_policy( + Config1, OldNodeA, + <<"fed_on_upgrade_policy">>, <<"^fed_">>, <<"all">>, + [ + {<<"federation-upstream-pattern">>, UpstreamName} + ]), + + XName = <<"fed_ex_on_upgrade_cluster">>, + X = exchange_declare_method(XName, <<"direct">>), + {Conn1, Ch1} = rabbit_ct_client_helpers:open_connection_and_channel( + Config1, OldNodeA), + ?assertEqual({'exchange.declare_ok'}, declare_exchange(Ch1, X)), + rabbit_ct_client_helpers:close_channel(Ch1), + rabbit_ct_client_helpers:close_connection(Conn1), + + %% Verify the format of the child ID. In the main branch, the format was + %% temporarily a size-2 tuple with a list as the first element. This was + %% not kept later and the original ID format is used in old and new nodes. + [{Id, _, _, _}] = rabbit_ct_broker_helpers:rpc( + Config1, OldNodeA, + mirrored_supervisor, which_children, + [rabbit_federation_exchange_link_sup_sup]), + case Id of + %% This is the format we expect everywhere. + #exchange{name = #resource{name = XName}} -> + %% Verify that the supervisors exist on all nodes. + lists:foreach( + fun(Node) -> + ?assertMatch( + [{#exchange{name = #resource{name = XName}}, + _, _, _}], + rabbit_ct_broker_helpers:rpc( + Config1, Node, + mirrored_supervisor, which_children, + [rabbit_federation_exchange_link_sup_sup])) + end, [OldNodeA, OldNodeC]), + + %% Simulate a rolling upgrade by: + %% 1. adding new nodes to the old cluster + %% 2. stopping the old nodes + %% + %% After that, the supervisors run on the new code. + Config2 = rabbit_ct_broker_helpers:cluster_nodes( + Config1, OldNodeA, [NewNodeB, NewNodeD]), + ok = rabbit_ct_broker_helpers:stop_broker(Config2, OldNodeA), + ok = rabbit_ct_broker_helpers:reset_node(Config1, OldNodeA), + ok = rabbit_ct_broker_helpers:stop_broker(Config2, OldNodeC), + ok = rabbit_ct_broker_helpers:reset_node(Config2, OldNodeC), + + %% Verify that the supervisors still use the same IDs. + lists:foreach( + fun(Node) -> + ?assertMatch( + [{#exchange{name = #resource{name = XName}}, + _, _, _}], + rabbit_ct_broker_helpers:rpc( + Config2, Node, + mirrored_supervisor, which_children, + [rabbit_federation_exchange_link_sup_sup])) + end, [NewNodeB, NewNodeD]), + + %% Delete the exchange: it should work because the ID format is the + %% one expected. + %% + %% During the transient period where the ID format was changed, + %% this would crash with a badmatch because the running + %% supervisor's ID would not match the content of the database. + {Conn2, Ch2} = rabbit_ct_client_helpers:open_connection_and_channel( + Config2, NewNodeB), + ?assertEqual({'exchange.delete_ok'}, delete_exchange(Ch2, XName)), + rabbit_ct_client_helpers:close_channel(Ch2), + rabbit_ct_client_helpers:close_connection(Conn2); + + %% This is the transient format we are not interested in as it only + %% lived in a development branch. + {List, #exchange{name = #resource{name = XName}}} + when is_list(List) -> + {skip, "Testcase skipped with the transiently changed ID format"} + end; + true -> + %% skip the test in mixed version mode + {skip, "Should not run in mixed version environments"} + end. %% %% Test helpers From 99fd7419b8f8687cf16b66ab98ff2a4229185cb1 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Wed, 21 May 2025 16:45:08 +0200 Subject: [PATCH 9/9] Update Selenium tests to use new federation plugins --- selenium/test/basic-auth/enabled_plugins | 2 +- selenium/test/multi-oauth/enabled_plugins | 2 +- selenium/test/oauth/enabled_plugins | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/selenium/test/basic-auth/enabled_plugins b/selenium/test/basic-auth/enabled_plugins index 0ec08b648cb9..c2e7ec0875a5 100644 --- a/selenium/test/basic-auth/enabled_plugins +++ b/selenium/test/basic-auth/enabled_plugins @@ -1,2 +1,2 @@ [rabbitmq_management,rabbitmq_stream,rabbitmq_stream_common,rabbitmq_stream_management, -rabbitmq_top,rabbitmq_tracing,rabbitmq_federation_management,rabbitmq_shovel_management]. +rabbitmq_top,rabbitmq_tracing,rabbitmq_queue_federation,rabbitmq_federation_management,rabbitmq_shovel_management]. diff --git a/selenium/test/multi-oauth/enabled_plugins b/selenium/test/multi-oauth/enabled_plugins index ea686b9f2b51..5fb84b6d150d 100644 --- a/selenium/test/multi-oauth/enabled_plugins +++ b/selenium/test/multi-oauth/enabled_plugins @@ -3,7 +3,7 @@ rabbitmq_auth_backend_http,rabbitmq_auth_backend_ldap, rabbitmq_auth_backend_oauth2,rabbitmq_auth_mechanism_ssl,rabbitmq_aws, rabbitmq_consistent_hash_exchange,rabbitmq_event_exchange, - rabbitmq_federation,rabbitmq_federation_management, + rabbitmq_federation_common,rabbitmq_queue_federation, rabbitmq_federation_management, rabbitmq_jms_topic_exchange,rabbitmq_management,rabbitmq_management_agent, rabbitmq_mqtt,rabbitmq_peer_discovery_aws,rabbitmq_peer_discovery_common, rabbitmq_peer_discovery_consul,rabbitmq_peer_discovery_etcd, diff --git a/selenium/test/oauth/enabled_plugins b/selenium/test/oauth/enabled_plugins index 8dbd7d6cbf63..2592cc7b89f4 100644 --- a/selenium/test/oauth/enabled_plugins +++ b/selenium/test/oauth/enabled_plugins @@ -3,7 +3,7 @@ rabbitmq_auth_backend_http,rabbitmq_auth_backend_ldap, rabbitmq_auth_backend_oauth2,rabbitmq_auth_mechanism_ssl,rabbitmq_aws, rabbitmq_consistent_hash_exchange,rabbitmq_event_exchange, - rabbitmq_federation,rabbitmq_federation_management, + rabbitmq_federation_common,rabbitmq_queue_federation,rabbitmq_federation_management, rabbitmq_federation_prometheus,rabbitmq_jms_topic_exchange, rabbitmq_management,rabbitmq_management_agent,rabbitmq_mqtt, rabbitmq_peer_discovery_aws,rabbitmq_peer_discovery_common,