Split rabbitmq_federation: rabbitmq_queue_federation, rabbitmq_exchange_federation and rabbitmq_federation_common
This commit is contained in:
parent
4c34155886
commit
74dfa06c29
|
@ -48,7 +48,9 @@ elvis
|
||||||
!/deps/rabbitmq_ct_helpers/
|
!/deps/rabbitmq_ct_helpers/
|
||||||
!/deps/rabbitmq_ct_client_helpers/
|
!/deps/rabbitmq_ct_client_helpers/
|
||||||
!/deps/rabbitmq_event_exchange/
|
!/deps/rabbitmq_event_exchange/
|
||||||
|
!/deps/rabbitmq_exchange_federation/
|
||||||
!/deps/rabbitmq_federation/
|
!/deps/rabbitmq_federation/
|
||||||
|
!/deps/rabbitmq_federation_common/
|
||||||
!/deps/rabbitmq_federation_management/
|
!/deps/rabbitmq_federation_management/
|
||||||
!/deps/rabbitmq_federation_prometheus/
|
!/deps/rabbitmq_federation_prometheus/
|
||||||
!/deps/rabbitmq_jms_topic_exchange/
|
!/deps/rabbitmq_jms_topic_exchange/
|
||||||
|
@ -62,6 +64,7 @@ elvis
|
||||||
!/deps/rabbitmq_peer_discovery_k8s/
|
!/deps/rabbitmq_peer_discovery_k8s/
|
||||||
!/deps/rabbitmq_prelaunch/
|
!/deps/rabbitmq_prelaunch/
|
||||||
!/deps/rabbitmq_prometheus/
|
!/deps/rabbitmq_prometheus/
|
||||||
|
!/deps/rabbitmq_queue_federation/
|
||||||
!/deps/rabbitmq_random_exchange/
|
!/deps/rabbitmq_random_exchange/
|
||||||
!/deps/rabbitmq_recent_history_exchange/
|
!/deps/rabbitmq_recent_history_exchange/
|
||||||
!/deps/rabbitmq_sharding/
|
!/deps/rabbitmq_sharding/
|
||||||
|
|
|
@ -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
|
|
@ -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.
|
|
@ -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.
|
|
@ -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).
|
|
@ -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, []}}.
|
|
@ -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}}.
|
|
@ -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.
|
|
@ -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).
|
90
deps/rabbitmq_exchange_federation/src/rabbit_federation_exchange_link_sup_sup.erl
vendored
Normal file
90
deps/rabbitmq_exchange_federation/src/rabbit_federation_exchange_link_sup_sup.erl
vendored
Normal 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].
|
91
deps/rabbitmq_exchange_federation/src/rabbit_federation_upstream_exchange.erl
vendored
Normal file
91
deps/rabbitmq_exchange_federation/src/rabbit_federation_upstream_exchange.erl
vendored
Normal 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).
|
|
@ -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.
|
52
deps/rabbitmq_exchange_federation/test/definition_import_SUITE_data/case1.json
vendored
Normal file
52
deps/rabbitmq_exchange_federation/test/definition_import_SUITE_data/case1.json
vendored
Normal 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": []
|
||||||
|
}
|
|
@ -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)).
|
168
deps/rabbitmq_exchange_federation/test/exchange_federation_status_command_SUITE.erl
vendored
Normal file
168
deps/rabbitmq_exchange_federation/test/exchange_federation_status_command_SUITE.erl
vendored
Normal 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).
|
|
@ -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}.
|
101
deps/rabbitmq_exchange_federation/test/restart_federation_link_command_SUITE.erl
vendored
Normal file
101
deps/rabbitmq_exchange_federation/test/restart_federation_link_command_SUITE.erl
vendored
Normal 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).
|
|
@ -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 = []}.
|
|
@ -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
|
|
@ -0,0 +1,3 @@
|
||||||
|
-include_lib("rabbit_common/include/logging.hrl").
|
||||||
|
|
||||||
|
-define(RMQLOG_DOMAIN_FEDERATION, ?DEFINE_RMQLOG_DOMAIN(queue_federation)).
|
|
@ -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).
|
117
deps/rabbitmq_federation_common/src/Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand.erl
vendored
Normal file
117
deps/rabbitmq_federation_common/src/Elixir.RabbitMQ.CLI.Ctl.Commands.FederationStatusCommand.erl
vendored
Normal 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])].
|
|
@ -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).
|
|
@ -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, []}}.
|
|
@ -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.
|
|
@ -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}.
|
|
@ -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).
|
|
@ -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.
|
|
@ -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", []}.
|
|
@ -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.
|
|
@ -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 = '_'}.
|
|
@ -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}}.
|
|
@ -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)).
|
|
@ -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)}
|
||||||
|
}.
|
|
@ -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.
|
52
deps/rabbitmq_federation_common/test/definition_import_SUITE_data/case1.json
vendored
Normal file
52
deps/rabbitmq_federation_common/test/definition_import_SUITE_data/case1.json
vendored
Normal 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": []
|
||||||
|
}
|
|
@ -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.
|
|
@ -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">>}.
|
|
@ -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
|
|
@ -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.
|
|
@ -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.
|
|
@ -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).
|
|
@ -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.
|
|
@ -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).
|
|
@ -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].
|
|
@ -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, []}}.
|
|
@ -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}}.
|
|
@ -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.
|
|
@ -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": []
|
||||||
|
}
|
|
@ -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)].
|
172
deps/rabbitmq_queue_federation/test/queue_federation_status_command_SUITE.erl
vendored
Normal file
172
deps/rabbitmq_queue_federation/test/queue_federation_status_command_SUITE.erl
vendored
Normal 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).
|
|
@ -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()).
|
|
@ -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}.
|
107
deps/rabbitmq_queue_federation/test/rabbit_queue_federation_status_SUITE.erl
vendored
Normal file
107
deps/rabbitmq_queue_federation/test/rabbit_queue_federation_status_SUITE.erl
vendored
Normal 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()).
|
100
deps/rabbitmq_queue_federation/test/restart_federation_link_command_SUITE.erl
vendored
Normal file
100
deps/rabbitmq_queue_federation/test/restart_federation_link_command_SUITE.erl
vendored
Normal 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).
|
Loading…
Reference in New Issue