Split rabbitmq_federation: rabbitmq_queue_federation, rabbitmq_exchange_federation and rabbitmq_federation_common

This commit is contained in:
Diana Parra Corbacho 2025-05-12 11:38:10 +02:00
parent 4c34155886
commit 74dfa06c29
55 changed files with 7151 additions and 0 deletions

3
.gitignore vendored
View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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 = <<DVhost/binary,":", DName/binary, " ", ABSuffix/binary>>,
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) ->
<<UpstreamQName/binary, " ", Suffix/binary>>.
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).

View File

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

View File

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

View File

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

View File

@ -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": []
}

View File

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

View File

@ -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, <<"<rmq-ct-federation_status_command_SUITE-1-21000@localhost.1.563.0>">>},
{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 := <<"<rmq-ct-federation_status_command_SUITE-1-21000@localhost.1.563.0>">>,
uri := <<"amqp://localhost:21000">>,
last_changed := <<"2016-11-21 08:51:19">>,
exchange := <<>>,
upstream_exchange := <<>>,
error := <<>>}]}
= ?CMD:output(Input, Opts).

View File

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

View File

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

View File

@ -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 = []}.

View File

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

View File

@ -0,0 +1,3 @@
-include_lib("rabbit_common/include/logging.hrl").
-define(RMQLOG_DOMAIN_FEDERATION, ?DEFINE_RMQLOG_DOMAIN(queue_federation)).

View File

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

View File

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

View File

@ -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 <link_id>">>.
usage_additional() ->
[
{<<"<link_id>">>, <<"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).

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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 >>
|| <<N:4>> <= <<PHash:32>> >>.
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 = '_'}.

View File

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

View File

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

View File

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

View File

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

View File

@ -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": []
}

View File

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

View File

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

24
deps/rabbitmq_queue_federation/Makefile vendored Normal file
View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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": []
}

View File

@ -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 = <<Base/binary, "-to-", UpstreamQ/binary>>,
publish_expect(Ch, <<>>, UpstreamQ, DownstreamQ, Payload).
expect_federation(Ch, UpstreamQ, DownstreamQ, Timeout) ->
Base = <<"HELLO">>,
Payload = <<Base/binary, "-to-", UpstreamQ/binary>>,
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)].

View File

@ -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, <<"<rmq-ct-federation_status_command_SUITE-1-21000@localhost.1.563.0>">>},
{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 := <<"<rmq-ct-federation_status_command_SUITE-1-21000@localhost.1.563.0>">>,
uri := <<"amqp://localhost:21000">>,
last_changed := <<"2016-11-21 08:51:19">>,
exchange := <<>>,
upstream_exchange := <<>>,
error := <<>>}]}
= ?CMD:output(Input, Opts).

View File

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

View File

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

View File

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

View File

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