Skip to content

Commit 3a65695

Browse files
committed
Support exchange federation with MQTT 5.0 subscribers
## What? This commit fixes #13040. Prior to this commit, exchange federation crashed if the MQTT topic exchange (`amq.topic` by default) got federated and MQTT 5.0 clients subscribed on the downstream. That's because the federation plugin sends bindings from downstream to upstream via AMQP 0.9.1. However, binding arguments containing Erlang record `mqtt_subscription_opts` (henceforth binding args v1) cannot be encoded in AMQP 0.9.1. ## Why? Federating the MQTT topic exchange could be useful for warm standby use cases. ## How? This commit makes binding arguments a valid AMQP 0.9.1 table (henceforth binding args v2). Binding args v2 can only be used if all nodes support it. Hence binding args v2 comes with feature flag `rabbitmq_4.1.0`. Note that the AMQP over WebSocket [PR](#13071) already introduces this same feature flag. Although the feature flag subsystem supports plugins to define their own feature flags, and the MQTT plugin defined its own feature flags in the past, reusing feature flag `rabbitmq_4.1.0` is simpler. This commit also avoids database migrations for both Mnesia and Khepri if feature flag `rabbitmq_4.1.0` gets enabled. Instead, it's simpler to migrate binding args v1 to binding args v2 at MQTT connection establishment time if the feature flag is enabled. (If the feature flag is disabled at connection etablishment time, but gets enabled during the connection lifetime, the connection keeps using bindings args v1.) This commit adds two new suites: 1. `federation_SUITE` which tests that federating the MQTT topic exchange works, and 2. `feature_flag_SUITE` which tests the binding args migration from v1 to v2.
1 parent a51d8a5 commit 3a65695

File tree

5 files changed

+325
-31
lines changed

5 files changed

+325
-31
lines changed

deps/rabbit/src/rabbit_core_ff.erl

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -205,3 +205,10 @@
205205
stability => stable,
206206
depends_on => [message_containers]
207207
}}).
208+
209+
-rabbit_feature_flag(
210+
{'rabbitmq_4.1.0',
211+
#{desc => "Allows rolling upgrades to 4.1.x",
212+
stability => stable,
213+
depends_on => ['rabbitmq_4.0.0']
214+
}}).

deps/rabbitmq_mqtt/Makefile

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ export BUILD_WITHOUT_QUIC
4343

4444
LOCAL_DEPS = ssl
4545
DEPS = ranch rabbit amqp10_common
46-
TEST_DEPS = emqtt ct_helper rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_management amqp_client rabbitmq_consistent_hash_exchange rabbitmq_amqp_client rabbitmq_stomp rabbitmq_stream
46+
TEST_DEPS = emqtt ct_helper rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_management amqp_client rabbitmq_consistent_hash_exchange rabbitmq_amqp_client rabbitmq_stomp rabbitmq_stream rabbitmq_federation
4747

4848
PLT_APPS += rabbitmqctl elixir
4949

@@ -94,7 +94,7 @@ define ct_master.erl
9494
halt(0)
9595
endef
9696

97-
PARALLEL_CT_SET_1_A = auth retainer
97+
PARALLEL_CT_SET_1_A = auth retainer federation feature_flag
9898
PARALLEL_CT_SET_1_B = cluster command config config_schema mc_mqtt packet_prop \
9999
processor protocol_interop proxy_protocol rabbit_mqtt_confirms reader util
100100
PARALLEL_CT_SET_1_C = java v5

deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl

Lines changed: 103 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -88,8 +88,13 @@
8888
send_fun :: send_fun(),
8989
%% Maximum MQTT packet size in bytes for packets sent from server to client.
9090
max_packet_size_outbound :: max_packet_size(),
91-
topic_alias_maximum_outbound :: non_neg_integer()
92-
}).
91+
topic_alias_maximum_outbound :: non_neg_integer(),
92+
%% https://github.com/rabbitmq/rabbitmq-server/issues/13040
93+
%% The database stores the MQTT subscription options in the binding arguments for:
94+
%% * v1 as Erlang record #mqtt_subscription_opts{}
95+
%% * v2 as AMQP 0.9.1 table
96+
binding_args_v2 :: boolean()
97+
}).
9398

9499
-record(state,
95100
{cfg :: #cfg{},
@@ -207,6 +212,9 @@ process_connect(
207212
{TraceState, ConnName} = init_trace(VHost, ConnName0),
208213
ok = rabbit_mqtt_keepalive:start(KeepaliveSecs, Socket),
209214
Exchange = rabbit_misc:r(VHost, exchange, persistent_term:get(?PERSISTENT_TERM_EXCHANGE)),
215+
%% To simplify logic, we decide at connection establishment time to stick
216+
%% with either binding args v1 or v2 for the lifetime of the connection.
217+
BindingArgsV2 = rabbit_feature_flags:is_enabled('rabbitmq_4.1.0'),
210218
S = #state{
211219
cfg = #cfg{socket = Socket,
212220
proto_ver = proto_integer_to_atom(ProtoVer),
@@ -229,7 +237,8 @@ process_connect(
229237
user_prop = maps:get('User-Property', ConnectProps, []),
230238
will_msg = WillMsg,
231239
max_packet_size_outbound = MaxPacketSize,
232-
topic_alias_maximum_outbound = TopicAliasMaxOutbound},
240+
topic_alias_maximum_outbound = TopicAliasMaxOutbound,
241+
binding_args_v2 = BindingArgsV2},
233242
auth_state = #auth_state{
234243
user = User,
235244
authz_ctx = AuthzCtx}},
@@ -432,7 +441,8 @@ process_request(?SUBSCRIBE,
432441
packet_id = SubscribePktId,
433442
subscriptions = Subscriptions},
434443
payload = undefined},
435-
#state{cfg = #cfg{proto_ver = ProtoVer}} = State0) ->
444+
State0 = #state{cfg = #cfg{proto_ver = ProtoVer,
445+
binding_args_v2 = BindingArgsV2}}) ->
436446
?LOG_DEBUG("Received a SUBSCRIBE with subscription(s) ~p", [Subscriptions]),
437447
{ResultRev, RetainedRev, State1} =
438448
lists:foldl(
@@ -460,7 +470,7 @@ process_request(?SUBSCRIBE,
460470
maybe
461471
{ok, Q} ?= ensure_queue(QoS, S0),
462472
QName = amqqueue:get_name(Q),
463-
BindingArgs = binding_args_for_proto_ver(ProtoVer, TopicFilter, Opts),
473+
BindingArgs = binding_args_for_proto_ver(ProtoVer, TopicFilter, Opts, BindingArgsV2),
464474
ok ?= add_subscription(TopicFilter, BindingArgs, QName, S0),
465475
ok ?= maybe_delete_old_subscription(TopicFilter, Opts, S0),
466476
Subs = maps:put(TopicFilter, Opts, S0#state.subscriptions),
@@ -508,10 +518,11 @@ process_request(?UNSUBSCRIBE,
508518
{ReasonCodes, State} =
509519
lists:foldl(
510520
fun(TopicFilter, {L, #state{subscriptions = Subs0,
511-
cfg = #cfg{proto_ver = ProtoVer}} = S0}) ->
521+
cfg = #cfg{proto_ver = ProtoVer,
522+
binding_args_v2 = BindingArgsV2}} = S0}) ->
512523
case maps:take(TopicFilter, Subs0) of
513524
{Opts, Subs} ->
514-
BindingArgs = binding_args_for_proto_ver(ProtoVer, TopicFilter, Opts),
525+
BindingArgs = binding_args_for_proto_ver(ProtoVer, TopicFilter, Opts, BindingArgsV2),
515526
case delete_subscription(
516527
TopicFilter, BindingArgs, Opts#mqtt_subscription_opts.qos, S0) of
517528
ok ->
@@ -872,52 +883,76 @@ init_subscriptions(_SessionPresent = _SubscriptionsPresent = true,
872883
init_subscriptions(_, State) ->
873884
{ok, State}.
874885

886+
%% We suppress a warning because rabbit_misc:table_lookup/2 declares the correct spec and
887+
%% we must handle binding args v1 where binding arguments are not a valid AMQP 0.9.1 table.
888+
-dialyzer({no_match, init_subscriptions0/2}).
889+
875890
-spec init_subscriptions0(qos(), state()) ->
876891
{ok, subscriptions()} | {error, reason_code()}.
877-
init_subscriptions0(QoS, State0 = #state{cfg = #cfg{proto_ver = ProtoVer,
878-
exchange = Exchange}}) ->
892+
init_subscriptions0(QoS, State = #state{cfg = #cfg{proto_ver = ProtoVer,
893+
exchange = Exchange,
894+
binding_args_v2 = BindingArgsV2}}) ->
879895
Bindings =
880896
rabbit_binding:list_for_source_and_destination(
881897
Exchange,
882-
queue_name(QoS, State0),
898+
queue_name(QoS, State),
883899
%% Querying table rabbit_route is catastrophic for CPU usage.
884900
%% Querying table rabbit_reverse_route is acceptable because
885901
%% the source exchange is always the same in the MQTT plugin whereas
886902
%% the destination queue is different for each MQTT client and
887903
%% rabbit_reverse_route is sorted by destination queue.
888904
_Reverse = true),
889905
try
890-
Subs = lists:foldl(
906+
Subs = lists:map(
891907
fun(#binding{key = Key,
892-
args = Args = []},
893-
Acc) ->
908+
args = Args = []}) ->
894909
Opts = #mqtt_subscription_opts{qos = QoS},
895910
TopicFilter = amqp_to_mqtt(Key),
896911
case ProtoVer of
897912
?MQTT_PROTO_V5 ->
898913
%% session upgrade
899-
NewBindingArgs = binding_args_for_proto_ver(ProtoVer, TopicFilter, Opts),
900-
ok = recreate_subscription(TopicFilter, Args, NewBindingArgs, QoS, State0);
914+
NewBindingArgs = binding_args_for_proto_ver(ProtoVer, TopicFilter, Opts, BindingArgsV2),
915+
ok = recreate_subscription(TopicFilter, Args, NewBindingArgs, QoS, State);
901916
_ ->
902917
ok
903918
end,
904-
maps:put(TopicFilter, Opts, Acc);
919+
{TopicFilter, Opts};
905920
(#binding{key = Key,
906-
args = Args},
907-
Acc) ->
908-
Opts0 = #mqtt_subscription_opts{} = lists:keyfind(mqtt_subscription_opts, 1, Args),
921+
args = Args}) ->
909922
TopicFilter = amqp_to_mqtt(Key),
910923
Opts = case ProtoVer of
911924
?MQTT_PROTO_V5 ->
912-
Opts0;
925+
case rabbit_misc:table_lookup(Args, <<"x-mqtt-subscription-opts">>) of
926+
{table, Table} ->
927+
%% binding args v2
928+
subscription_opts_from_table(Table);
929+
undefined ->
930+
%% binding args v1
931+
Opts0 = #mqtt_subscription_opts{} = lists:keyfind(
932+
mqtt_subscription_opts, 1, Args),
933+
case BindingArgsV2 of
934+
true ->
935+
%% Migrate v1 to v2.
936+
%% Note that this migration must be in place even for some versions
937+
%% (jump upgrade) after feature flag 'rabbitmq_4.1.0' has become
938+
%% required since enabling the feature flag doesn't migrate binding
939+
%% args for existing connections.
940+
NewArgs = binding_args_for_proto_ver(
941+
ProtoVer, TopicFilter, Opts0, BindingArgsV2),
942+
ok = recreate_subscription(TopicFilter, Args, NewArgs, QoS, State);
943+
false ->
944+
ok
945+
end,
946+
Opts0
947+
end;
913948
_ ->
914949
%% session downgrade
915-
ok = recreate_subscription(TopicFilter, Args, [], QoS, State0),
950+
ok = recreate_subscription(TopicFilter, Args, [], QoS, State),
916951
#mqtt_subscription_opts{qos = QoS}
917952
end,
918-
maps:put(TopicFilter, Opts, Acc)
919-
end, #{}, Bindings),
920-
{ok, Subs}
953+
{TopicFilter, Opts}
954+
end, Bindings),
955+
{ok, maps:from_list(Subs)}
921956
catch throw:{error, Reason} ->
922957
Rc = case Reason of
923958
access_refused -> ?RC_NOT_AUTHORIZED;
@@ -1482,14 +1517,52 @@ consume(Q, QoS, #state{
14821517
Err
14831518
end.
14841519

1485-
binding_args_for_proto_ver(?MQTT_PROTO_V3, _, _) ->
1520+
binding_args_for_proto_ver(?MQTT_PROTO_V3, _, _, _) ->
14861521
[];
1487-
binding_args_for_proto_ver(?MQTT_PROTO_V4, _, _) ->
1522+
binding_args_for_proto_ver(?MQTT_PROTO_V4, _, _, _) ->
14881523
[];
1489-
binding_args_for_proto_ver(?MQTT_PROTO_V5, TopicFilter, SubOpts) ->
1524+
binding_args_for_proto_ver(?MQTT_PROTO_V5, TopicFilter, SubOpts0, V2) ->
1525+
SubOpts = case V2 of
1526+
true ->
1527+
Table = subscription_opts_to_table(SubOpts0),
1528+
{<<"x-mqtt-subscription-opts">>, table, Table};
1529+
false ->
1530+
SubOpts0
1531+
end,
14901532
BindingKey = mqtt_to_amqp(TopicFilter),
14911533
[SubOpts, {<<"x-binding-key">>, longstr, BindingKey}].
14921534

1535+
subscription_opts_to_table(#mqtt_subscription_opts{
1536+
qos = Qos,
1537+
no_local = NoLocal,
1538+
retain_as_published = RetainAsPublished,
1539+
retain_handling = RetainHandling,
1540+
id = Id}) ->
1541+
Table0 = [{<<"qos">>, unsignedbyte, Qos},
1542+
{<<"no-local">>, bool, NoLocal},
1543+
{<<"retain-as-published">>, bool, RetainAsPublished},
1544+
{<<"retain-handling">>, unsignedbyte, RetainHandling}],
1545+
Table = case Id of
1546+
undefined ->
1547+
Table0;
1548+
_ ->
1549+
[{<<"id">>, unsignedint, Id} | Table0]
1550+
end,
1551+
rabbit_misc:sort_field_table(Table).
1552+
1553+
subscription_opts_from_table(Table) ->
1554+
#{<<"qos">> := Qos,
1555+
<<"no-local">> := NoLocal,
1556+
<<"retain-as-published">> := RetainAsPublished,
1557+
<<"retain-handling">> := RetainHandling
1558+
} = Map = rabbit_misc:amqp_table(Table),
1559+
#mqtt_subscription_opts{
1560+
qos = Qos,
1561+
no_local = NoLocal,
1562+
retain_as_published = RetainAsPublished,
1563+
retain_handling = RetainHandling,
1564+
id = maps:get(<<"id">>, Map, undefined)}.
1565+
14931566
add_subscription(TopicFilter, BindingArgs, Qos, State)
14941567
when is_integer(Qos) ->
14951568
add_subscription(TopicFilter, BindingArgs, queue_name(Qos, State), State);
@@ -1506,12 +1579,13 @@ delete_subscription(TopicFilter, BindingArgs, Qos, State) ->
15061579
%% Subscription will be identical to that in the previous Subscription, although its
15071580
%% Subscription Options could be different." [v5 3.8.4]
15081581
maybe_delete_old_subscription(TopicFilter, Opts, State = #state{subscriptions = Subs,
1509-
cfg = #cfg{proto_ver = ProtoVer}}) ->
1582+
cfg = #cfg{proto_ver = ProtoVer,
1583+
binding_args_v2 = BindingArgsV2}}) ->
15101584
case Subs of
15111585
#{TopicFilter := OldOpts}
15121586
when OldOpts =/= Opts ->
15131587
delete_subscription(TopicFilter,
1514-
binding_args_for_proto_ver(ProtoVer, TopicFilter, OldOpts),
1588+
binding_args_for_proto_ver(ProtoVer, TopicFilter, OldOpts, BindingArgsV2),
15151589
OldOpts#mqtt_subscription_opts.qos,
15161590
State);
15171591
_ ->
Lines changed: 111 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,111 @@
1+
%% This Source Code Form is subject to the terms of the Mozilla Public
2+
%% License, v. 2.0. If a copy of the MPL was not distributed with this
3+
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4+
%%
5+
%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
6+
7+
%% This suite should be deleted when feature flag 'rabbitmq_4.1.0' becomes required.
8+
-module(feature_flag_SUITE).
9+
-compile([export_all,
10+
nowarn_export_all]).
11+
12+
-include_lib("eunit/include/eunit.hrl").
13+
14+
-import(util,
15+
[connect/2,
16+
connect/3,
17+
non_clean_sess_opts/0
18+
]).
19+
20+
-define(RC_SESSION_TAKEN_OVER, 16#8E).
21+
22+
all() ->
23+
[migrate_binding_args].
24+
25+
init_per_suite(Config) ->
26+
rabbit_ct_helpers:log_environment(),
27+
Config1 = rabbit_ct_helpers:set_config(
28+
Config,
29+
[{mqtt_version, v5},
30+
{rmq_nodename_suffix, ?MODULE}]),
31+
Config2 = rabbit_ct_helpers:merge_app_env(
32+
Config1,
33+
{rabbit, [{forced_feature_flags_on_init, []}]}),
34+
rabbit_ct_helpers:run_setup_steps(
35+
Config2,
36+
rabbit_ct_broker_helpers:setup_steps() ++
37+
rabbit_ct_client_helpers:setup_steps()).
38+
39+
end_per_suite(Config) ->
40+
rabbit_ct_helpers:run_teardown_steps(
41+
Config,
42+
rabbit_ct_client_helpers:teardown_steps() ++
43+
rabbit_ct_broker_helpers:teardown_steps()).
44+
45+
init_per_testcase(Testcase, Config) ->
46+
rabbit_ct_helpers:testcase_started(Config, Testcase).
47+
48+
end_per_testcase(Testcase, Config) ->
49+
rabbit_ct_helpers:testcase_finished(Config, Testcase).
50+
51+
migrate_binding_args(Config) ->
52+
%% Feature flag rabbitmq_4.1.0 enables binding arguments v2.
53+
FeatureFlag = 'rabbitmq_4.1.0',
54+
?assertNot(rabbit_ct_broker_helpers:is_feature_flag_enabled(Config, FeatureFlag)),
55+
56+
Sub1a = connect(<<"sub 1">>, Config, non_clean_sess_opts()),
57+
{ok, _, [0]} = emqtt:subscribe(Sub1a, <<"x/+">>, qos0),
58+
ok = emqtt:disconnect(Sub1a),
59+
60+
Sub2a = connect(<<"sub 2">>, Config,non_clean_sess_opts()),
61+
{ok, _, [0, 1]} = emqtt:subscribe(
62+
Sub2a,
63+
#{'Subscription-Identifier' => 9},
64+
[{<<"x/y">>, [{nl, false}, {rap, false}, {qos, qos0}]},
65+
{<<"z">>, [{nl, true}, {rap, true}, {qos, qos1}]}]),
66+
67+
Pub = connect(<<"pub">>, Config),
68+
{ok, _} = emqtt:publish(Pub, <<"x/y">>, <<"m1">>, [{retain, true}, {qos, 1}]),
69+
receive {publish, #{client_pid := Sub2a,
70+
qos := 0,
71+
topic := <<"x/y">>,
72+
payload := <<"m1">>,
73+
retain := false}} -> ok
74+
after 10_000 -> ct:fail({missing_publish, ?LINE})
75+
end,
76+
77+
?assertEqual(ok, rabbit_ct_broker_helpers:enable_feature_flag(Config, FeatureFlag)),
78+
79+
%% Connecting causes binding args to be migrated from v1 to v2.
80+
Sub1b = connect(<<"sub 1">>, Config, [{clean_start, false}]),
81+
receive {publish, #{client_pid := Sub1b,
82+
qos := 0,
83+
topic := <<"x/y">>,
84+
payload := <<"m1">>}} -> ok
85+
after 10_000 -> ct:fail({missing_publish, ?LINE})
86+
end,
87+
88+
unlink(Sub2a),
89+
%% Connecting causes binding args to be migrated from v1 to v2.
90+
Sub2b = connect(<<"sub 2">>, Config, [{clean_start, false}]),
91+
receive {disconnected, ?RC_SESSION_TAKEN_OVER, #{}} -> ok
92+
after 10_000 -> ct:fail({missing_disconnected, ?LINE})
93+
end,
94+
95+
{ok, _} = emqtt:publish(Sub2b, <<"z">>, <<"m2">>, qos1),
96+
%% We should not receive m2 since it's a local publish.
97+
{ok, _} = emqtt:publish(Pub, <<"z">>, <<"m3">>, [{retain, true}, {qos, qos1}]),
98+
receive {publish, Publish} ->
99+
?assertMatch(#{client_pid := Sub2b,
100+
qos := 1,
101+
topic := <<"z">>,
102+
payload := <<"m3">>,
103+
properties := #{'Subscription-Identifier' := 9},
104+
retain := true},
105+
Publish)
106+
after 10_000 -> ct:fail({missing_publish, ?LINE})
107+
end,
108+
109+
ok = emqtt:disconnect(Sub1b),
110+
ok = emqtt:disconnect(Sub2b),
111+
ok = emqtt:disconnect(Pub).

0 commit comments

Comments
 (0)