diff --git a/deps/rabbit/apps/rabbitmq_prelaunch/src/rabbit_prelaunch_conf.erl b/deps/rabbit/apps/rabbitmq_prelaunch/src/rabbit_prelaunch_conf.erl
index b6156ff5c27c..f67a0c1bfd94 100644
--- a/deps/rabbit/apps/rabbitmq_prelaunch/src/rabbit_prelaunch_conf.erl
+++ b/deps/rabbit/apps/rabbitmq_prelaunch/src/rabbit_prelaunch_conf.erl
@@ -122,8 +122,13 @@ set_default_config() ->
{schedule_ms_limit, 0},
{heap_word_limit, 0},
{busy_port, false},
- {busy_dist_port, true}]}
- | OsirisConfig
+ {busy_dist_port, true}]},
+ {mnesia,
+ [
+ {dump_log_write_threshold, 5000},
+ {dump_log_time_threshold, 90000}
+ ]}
+ | OsirisConfig
],
apply_erlang_term_based_config(Config).
diff --git a/deps/rabbit/docs/rabbitmq.conf.example b/deps/rabbit/docs/rabbitmq.conf.example
index c3358bcd6b5d..00dec1473ce7 100644
--- a/deps/rabbit/docs/rabbitmq.conf.example
+++ b/deps/rabbit/docs/rabbitmq.conf.example
@@ -755,7 +755,7 @@
# stomp.tcp_listen_options.nodelay = true
#
# stomp.tcp_listen_options.exit_on_close = true
-# stomp.tcp_listen_options.send_timeout = 120
+# stomp.tcp_listen_options.send_timeout = 120000
## Proxy protocol support
##
@@ -838,7 +838,7 @@
# mqtt.tcp_listen_options.nodelay = true
#
# mqtt.tcp_listen_options.exit_on_close = true
-# mqtt.tcp_listen_options.send_timeout = 120
+# mqtt.tcp_listen_options.send_timeout = 120000
## TLS listener settings
## ## See https://rabbitmq.com/mqtt.html and https://rabbitmq.com/ssl.html for details.
diff --git a/deps/rabbit/src/rabbit.erl b/deps/rabbit/src/rabbit.erl
index 9d5bb05bda6b..bbb7b8ed5001 100644
--- a/deps/rabbit/src/rabbit.erl
+++ b/deps/rabbit/src/rabbit.erl
@@ -727,7 +727,8 @@ status() ->
true ->
[{virtual_host_count, rabbit_vhost:count()},
{connection_count,
- length(rabbit_networking:connections_local())},
+ length(rabbit_networking:connections_local()) +
+ length(rabbit_networking:local_non_amqp_connections())},
{queue_count, total_queue_count()}];
false ->
[]
@@ -1163,12 +1164,12 @@ config_locations() ->
% This event is necessary for the stats timer to be initialized with
% the correct values once the management agent has started
force_event_refresh(Ref) ->
- % direct connections, e.g. MQTT, STOMP
+ % direct connections, e.g. STOMP
ok = rabbit_direct:force_event_refresh(Ref),
% AMQP connections
ok = rabbit_networking:force_connection_event_refresh(Ref),
- % "external" connections, which are not handled by the "AMQP core",
- % e.g. connections to the stream plugin
+ % non-AMQP connections, which are not handled by the "AMQP core",
+ % e.g. connections to the stream and MQTT plugins
ok = rabbit_networking:force_non_amqp_connection_event_refresh(Ref),
ok = rabbit_channel:force_event_refresh(Ref),
ok = rabbit_amqqueue:force_event_refresh(Ref).
diff --git a/deps/rabbit/src/rabbit_access_control.erl b/deps/rabbit/src/rabbit_access_control.erl
index c47a5dd82391..c7aa589eb235 100644
--- a/deps/rabbit/src/rabbit_access_control.erl
+++ b/deps/rabbit/src/rabbit_access_control.erl
@@ -18,7 +18,7 @@
-export_type([permission_atom/0]).
--type permission_atom() :: 'configure' | 'read' | 'write'.
+-type permission_atom() :: 'configure' | 'write' | 'read'.
%%----------------------------------------------------------------------------
@@ -194,8 +194,8 @@ check_resource_access(User = #user{username = Username,
check_access(
fun() -> Module:check_resource_access(
auth_user(User, Impl), Resource, Permission, Context) end,
- Module, "access to ~ts refused for user '~ts'",
- [rabbit_misc:rs(Resource), Username]);
+ Module, "~s access to ~s refused for user '~s'",
+ [Permission, rabbit_misc:rs(Resource), Username]);
(_, Else) -> Else
end, ok, Modules).
@@ -207,8 +207,8 @@ check_topic_access(User = #user{username = Username,
check_access(
fun() -> Module:check_topic_access(
auth_user(User, Impl), Resource, Permission, Context) end,
- Module, "access to topic '~ts' in exchange ~ts refused for user '~ts'",
- [maps:get(routing_key, Context), rabbit_misc:rs(Resource), Username]);
+ Module, "~s access to topic '~s' in exchange ~s refused for user '~s'",
+ [Permission, maps:get(routing_key, Context), rabbit_misc:rs(Resource), Username]);
(_, Else) -> Else
end, ok, Modules).
diff --git a/deps/rabbit/src/rabbit_alarm.erl b/deps/rabbit/src/rabbit_alarm.erl
index 4e908200772b..750b3f222d2b 100644
--- a/deps/rabbit/src/rabbit_alarm.erl
+++ b/deps/rabbit/src/rabbit_alarm.erl
@@ -29,6 +29,9 @@
-export([remote_conserve_resources/3]). %% Internal use only
+-export_type([resource_alarm_source/0,
+ resource_alert/0]).
+
-define(SERVER, ?MODULE).
-define(FILE_DESCRIPTOR_RESOURCE, <<"file descriptors">>).
@@ -46,6 +49,9 @@
-type resource_alarm_source() :: 'disk' | 'memory'.
-type resource_alarm() :: {resource_limit, resource_alarm_source(), node()}.
-type alarm() :: local_alarm() | resource_alarm().
+-type resource_alert() :: {WasAlarmSetForNode :: boolean(),
+ IsThereAnyAlarmsWithSameSourceInTheCluster :: boolean(),
+ NodeForWhichAlarmWasSetOrCleared :: node()}.
%%----------------------------------------------------------------------------
diff --git a/deps/rabbit/src/rabbit_amqqueue.erl b/deps/rabbit/src/rabbit_amqqueue.erl
index 880992a03845..9c14c2a29c72 100644
--- a/deps/rabbit/src/rabbit_amqqueue.erl
+++ b/deps/rabbit/src/rabbit_amqqueue.erl
@@ -1763,7 +1763,7 @@ basic_get(Q, NoAck, LimiterPid, CTag, QStates) ->
non_neg_integer(), rabbit_types:ctag(), boolean(),
rabbit_framing:amqp_table(), any(), rabbit_types:username(),
rabbit_queue_type:state()) ->
- {ok, rabbit_queue_type:state(), rabbit_queue_type:actions()} |
+ {ok, rabbit_queue_type:state()} |
{error, term()} |
{protocol_error, Type :: atom(), Reason :: string(), Args :: term()}.
basic_consume(Q, NoAck, ChPid, LimiterPid,
diff --git a/deps/rabbit/src/rabbit_amqqueue_process.erl b/deps/rabbit/src/rabbit_amqqueue_process.erl
index 08cacb762fdd..c4835a5238a0 100644
--- a/deps/rabbit/src/rabbit_amqqueue_process.erl
+++ b/deps/rabbit/src/rabbit_amqqueue_process.erl
@@ -121,6 +121,7 @@
-define(CREATION_EVENT_KEYS,
[name,
+ type,
durable,
auto_delete,
arguments,
@@ -129,7 +130,7 @@
user_who_performed_action
]).
--define(INFO_KEYS, [pid | ?CREATION_EVENT_KEYS ++ ?STATISTICS_KEYS -- [name]]).
+-define(INFO_KEYS, [pid | ?CREATION_EVENT_KEYS ++ ?STATISTICS_KEYS -- [name, type]]).
%%----------------------------------------------------------------------------
diff --git a/deps/rabbit/src/rabbit_binding.erl b/deps/rabbit/src/rabbit_binding.erl
index f998903a51ff..b1b9ba963129 100644
--- a/deps/rabbit/src/rabbit_binding.erl
+++ b/deps/rabbit/src/rabbit_binding.erl
@@ -11,8 +11,8 @@
-export([recover/0, recover/2, exists/1, add/2, add/3, remove/1, remove/2, remove/3, remove/4]).
-export([list/1, list_for_source/1, list_for_destination/1,
- list_for_source_and_destination/2, list_explicit/0,
- list_between/2, has_any_between/2]).
+ list_for_source_and_destination/2, list_for_source_and_destination/3,
+ list_explicit/0, list_between/2, has_any_between/2]).
-export([new_deletions/0, combine_deletions/2, add_deletion/3,
process_deletions/2, binding_action/3]).
-export([info_keys/0, info/1, info/2, info_all/1, info_all/2, info_all/4]).
@@ -253,26 +253,18 @@ list(VHostPath) ->
list_for_source(?DEFAULT_EXCHANGE(VHostPath)) ->
implicit_bindings(VHostPath);
list_for_source(SrcName) ->
- mnesia:async_dirty(
- fun() ->
- Route = #route{binding = #binding{source = SrcName, _ = '_'}},
- [B || #route{binding = B}
- <- mnesia:match_object(rabbit_route, Route, read)]
- end).
+ Route = #route{binding = #binding{source = SrcName, _ = '_'}},
+ Fun = list_for_route(Route, false),
+ mnesia:async_dirty(Fun).
-spec list_for_destination
(rabbit_types:binding_destination()) -> bindings().
list_for_destination(DstName = #resource{}) ->
- ExplicitBindings = mnesia:async_dirty(
- fun() ->
- Route = #route{binding = #binding{destination = DstName,
- _ = '_'}},
- [reverse_binding(B) ||
- #reverse_route{reverse_binding = B} <-
- mnesia:match_object(rabbit_reverse_route,
- reverse_route(Route), read)]
- end),
+ Route = #route{binding = #binding{destination = DstName,
+ _ = '_'}},
+ Fun = list_for_route(Route, true),
+ ExplicitBindings = mnesia:async_dirty(Fun),
implicit_for_destination(DstName) ++ ExplicitBindings.
-spec list_between(
@@ -316,27 +308,40 @@ implicit_for_destination(DstQueue = #resource{kind = queue,
implicit_for_destination(_) ->
[].
--spec list_for_source_and_destination
- (rabbit_types:binding_source(), rabbit_types:binding_destination()) ->
- bindings().
+-spec list_for_source_and_destination(rabbit_types:binding_source(), rabbit_types:binding_destination()) ->
+ bindings().
+list_for_source_and_destination(SrcName, DstName) ->
+ list_for_source_and_destination(SrcName, DstName, false).
+-spec list_for_source_and_destination(rabbit_types:binding_source(), rabbit_types:binding_destination(), boolean()) ->
+ bindings().
list_for_source_and_destination(?DEFAULT_EXCHANGE(VHostPath),
#resource{kind = queue,
virtual_host = VHostPath,
- name = QName} = DstQueue) ->
+ name = QName} = DstQueue,
+ _Reverse) ->
[#binding{source = ?DEFAULT_EXCHANGE(VHostPath),
destination = DstQueue,
key = QName,
args = []}];
-list_for_source_and_destination(SrcName, DstName) ->
- mnesia:async_dirty(
- fun() ->
- Route = #route{binding = #binding{source = SrcName,
- destination = DstName,
- _ = '_'}},
- [B || #route{binding = B} <- mnesia:match_object(rabbit_route,
- Route, read)]
- end).
+list_for_source_and_destination(SrcName, DstName, Reverse) ->
+ Route = #route{binding = #binding{source = SrcName,
+ destination = DstName,
+ _ = '_'}},
+ Fun = list_for_route(Route, Reverse),
+ mnesia:async_dirty(Fun).
+
+list_for_route(Route, false) ->
+ fun() ->
+ [B || #route{binding = B} <- mnesia:match_object(rabbit_route, Route, read)]
+ end;
+list_for_route(Route, true) ->
+ fun() ->
+ [reverse_binding(B) ||
+ #reverse_route{reverse_binding = B} <-
+ mnesia:match_object(rabbit_reverse_route,
+ reverse_route(Route), read)]
+ end.
-spec info_keys() -> rabbit_types:info_keys().
diff --git a/deps/rabbit/src/rabbit_channel.erl b/deps/rabbit/src/rabbit_channel.erl
index 2366437102f9..2d5ccdcad422 100644
--- a/deps/rabbit/src/rabbit_channel.erl
+++ b/deps/rabbit/src/rabbit_channel.erl
@@ -104,9 +104,7 @@
%% same as capabilities in the reader
capabilities,
- %% tracing exchange resource if tracing is enabled,
- %% 'none' otherwise
- trace_state,
+ trace_state :: rabbit_trace:state(),
consumer_prefetch,
%% Message content size limit
max_message_size,
@@ -743,27 +741,6 @@ handle_cast({mandatory_received, _MsgSeqNo}, State) ->
%% NB: don't call noreply/1 since we don't want to send confirms.
noreply_coalesce(State);
-handle_cast({reject_publish, _MsgSeqNo, QPid} = Evt, State) ->
- %% For backwards compatibility
- QRef = rabbit_queue_type:find_name_from_pid(QPid, State#ch.queue_states),
- case QRef of
- undefined ->
- %% ignore if no queue could be found for the given pid
- noreply(State);
- _ ->
- handle_cast({queue_event, QRef, Evt}, State)
- end;
-
-handle_cast({confirm, _MsgSeqNo, QPid} = Evt, State) ->
- %% For backwards compatibility
- QRef = rabbit_queue_type:find_name_from_pid(QPid, State#ch.queue_states),
- case QRef of
- undefined ->
- %% ignore if no queue could be found for the given pid
- noreply(State);
- _ ->
- handle_cast({queue_event, QRef, Evt}, State)
- end;
handle_cast({queue_event, QRef, Evt},
#ch{queue_states = QueueStates0} = State0) ->
case rabbit_queue_type:handle_event(QRef, Evt, QueueStates0) of
@@ -771,26 +748,22 @@ handle_cast({queue_event, QRef, Evt},
State1 = State0#ch{queue_states = QState1},
State = handle_queue_actions(Actions, State1),
noreply_coalesce(State);
- eol ->
- State1 = handle_consuming_queue_down_or_eol(QRef, State0),
+ {eol, Actions} ->
+ State1 = handle_queue_actions(Actions, State0),
+ State2 = handle_consuming_queue_down_or_eol(QRef, State1),
{ConfirmMXs, UC1} =
- rabbit_confirms:remove_queue(QRef, State1#ch.unconfirmed),
+ rabbit_confirms:remove_queue(QRef, State2#ch.unconfirmed),
%% Deleted queue is a special case.
%% Do not nack the "rejected" messages.
- State2 = record_confirms(ConfirmMXs,
- State1#ch{unconfirmed = UC1}),
+ State3 = record_confirms(ConfirmMXs,
+ State2#ch{unconfirmed = UC1}),
_ = erase_queue_stats(QRef),
noreply_coalesce(
- State2#ch{queue_states = rabbit_queue_type:remove(QRef, QueueStates0)});
+ State3#ch{queue_states = rabbit_queue_type:remove(QRef, QueueStates0)});
{protocol_error, Type, Reason, ReasonArgs} ->
rabbit_misc:protocol_error(Type, Reason, ReasonArgs)
end.
-handle_info({ra_event, {Name, _} = From, Evt}, State) ->
- %% For backwards compatibility
- QRef = find_queue_name_from_quorum_name(Name, State#ch.queue_states),
- handle_cast({queue_event, QRef, {From, Evt}}, State);
-
handle_info({bump_credit, Msg}, State) ->
%% A rabbit_amqqueue_process is granting credit to our channel. If
%% our channel was being blocked by this process, and no other
@@ -811,11 +784,11 @@ handle_info(emit_stats, State) ->
%% stats timer.
{noreply, send_confirms_and_nacks(State1), hibernate};
-handle_info({'DOWN', _MRef, process, QPid, Reason},
+handle_info({{'DOWN', QName}, _MRef, process, QPid, Reason},
#ch{queue_states = QStates0,
queue_monitors = _QMons} = State0) ->
credit_flow:peer_down(QPid),
- case rabbit_queue_type:handle_down(QPid, Reason, QStates0) of
+ case rabbit_queue_type:handle_down(QPid, QName, Reason, QStates0) of
{ok, QState1, Actions} ->
State1 = State0#ch{queue_states = QState1},
State = handle_queue_actions(Actions, State1),
@@ -1091,7 +1064,7 @@ build_topic_variable_map(AuthzContext, VHost, Username) ->
%% Use tuple representation of amqp_params to avoid a dependency on amqp_client.
%% Extracts variable map only from amqp_params_direct, not amqp_params_network.
-%% amqp_params_direct records are usually used by plugins (e.g. MQTT, STOMP)
+%% amqp_params_direct records are usually used by plugins (e.g. STOMP)
extract_variable_map_from_amqp_params({amqp_params, {amqp_params_direct, _, _, _, _,
{amqp_adapter_info, _,_,_,_,_,_,AdditionalInfo}, _}}) ->
proplists:get_value(variable_map, AdditionalInfo, #{});
@@ -1813,7 +1786,7 @@ basic_consume(QueueName, NoAck, ConsumerPrefetch, ActualConsumerTag,
Username, QueueStates0),
Q}
end) of
- {{ok, QueueStates, Actions}, Q} when ?is_amqqueue(Q) ->
+ {{ok, QueueStates}, Q} when ?is_amqqueue(Q) ->
rabbit_global_counters:consumer_created(amqp091),
CM1 = maps:put(
ActualConsumerTag,
@@ -1822,10 +1795,9 @@ basic_consume(QueueName, NoAck, ConsumerPrefetch, ActualConsumerTag,
State1 = State#ch{consumer_mapping = CM1,
queue_states = QueueStates},
- State2 = handle_queue_actions(Actions, State1),
{ok, case NoWait of
- true -> consumer_monitor(ActualConsumerTag, State2);
- false -> State2
+ true -> consumer_monitor(ActualConsumerTag, State1);
+ false -> State1
end};
{{error, exclusive_consume_unavailable} = E, _Q} ->
E;
@@ -2873,38 +2845,29 @@ handle_queue_actions(Actions, #ch{} = State0) ->
confirm(MsgSeqNos, QRef, S0);
({rejected, _QRef, MsgSeqNos}, S0) ->
{U, Rej} =
- lists:foldr(
- fun(SeqNo, {U1, Acc}) ->
- case rabbit_confirms:reject(SeqNo, U1) of
- {ok, MX, U2} ->
- {U2, [MX | Acc]};
- {error, not_found} ->
- {U1, Acc}
- end
- end, {S0#ch.unconfirmed, []}, MsgSeqNos),
+ lists:foldr(
+ fun(SeqNo, {U1, Acc}) ->
+ case rabbit_confirms:reject(SeqNo, U1) of
+ {ok, MX, U2} ->
+ {U2, [MX | Acc]};
+ {error, not_found} ->
+ {U1, Acc}
+ end
+ end, {S0#ch.unconfirmed, []}, MsgSeqNos),
S = S0#ch{unconfirmed = U},
record_rejects(Rej, S);
({deliver, CTag, AckRequired, Msgs}, S0) ->
handle_deliver(CTag, AckRequired, Msgs, S0);
({queue_down, QRef}, S0) ->
- handle_consuming_queue_down_or_eol(QRef, S0)
-
+ handle_consuming_queue_down_or_eol(QRef, S0);
+ ({block, QName}, S0) ->
+ credit_flow:block(QName),
+ S0;
+ ({unblock, QName}, S0) ->
+ credit_flow:unblock(QName),
+ S0
end, State0, Actions).
-find_queue_name_from_quorum_name(Name, QStates) ->
- Fun = fun(K, _V, undefined) ->
- {ok, Q} = rabbit_amqqueue:lookup(K),
- case amqqueue:get_pid(Q) of
- {Name, _} ->
- amqqueue:get_name(Q);
- _ ->
- undefined
- end;
- (_, _, Acc) ->
- Acc
- end,
- rabbit_queue_type:fold_state(Fun, undefined, QStates).
-
maybe_increase_global_publishers(#ch{publishing_mode = true} = State0) ->
State0;
maybe_increase_global_publishers(State0) ->
diff --git a/deps/rabbit/src/rabbit_classic_queue.erl b/deps/rabbit/src/rabbit_classic_queue.erl
index e349b5e57254..a5b92bda94dc 100644
--- a/deps/rabbit/src/rabbit_classic_queue.erl
+++ b/deps/rabbit/src/rabbit_classic_queue.erl
@@ -4,14 +4,18 @@
-include("amqqueue.hrl").
-include_lib("rabbit_common/include/rabbit.hrl").
+%% TODO possible to use sets / maps instead of lists?
+%% Check performance with QoS 1 and 1 million target queues.
-record(msg_status, {pending :: [pid()],
confirmed = [] :: [pid()]}).
-define(STATE, ?MODULE).
--record(?STATE, {pid :: undefined | pid(), %% the current master pid
- qref :: term(), %% TODO
- unconfirmed = #{} ::
- #{non_neg_integer() => #msg_status{}}}).
+-record(?STATE, {
+ %% the current master pid
+ pid :: undefined | pid(),
+ unconfirmed = #{} :: #{non_neg_integer() => #msg_status{}},
+ monitored = #{} :: #{pid() => ok}
+ }).
-opaque state() :: #?STATE{}.
@@ -33,15 +37,16 @@
update/2,
consume/3,
cancel/5,
- handle_event/2,
+ handle_event/3,
deliver/2,
- settle/4,
- credit/4,
- dequeue/4,
+ settle/5,
+ credit/5,
+ dequeue/5,
info/2,
state_info/1,
capabilities/0,
- notify_decorators/1
+ notify_decorators/1,
+ is_stateful/0
]).
-export([delete_crashed/1,
@@ -156,9 +161,7 @@ stat(Q) ->
-spec init(amqqueue:amqqueue()) -> {ok, state()}.
init(Q) when ?amqqueue_is_classic(Q) ->
- QName = amqqueue:get_name(Q),
- {ok, #?STATE{pid = amqqueue:get_pid(Q),
- qref = QName}}.
+ {ok, #?STATE{pid = amqqueue:get_pid(Q)}}.
-spec close(state()) -> ok.
close(_State) ->
@@ -174,7 +177,7 @@ update(Q, #?STATE{pid = Pid} = State) when ?amqqueue_is_classic(Q) ->
State#?STATE{pid = NewPid}
end.
-consume(Q, Spec, State) when ?amqqueue_is_classic(Q) ->
+consume(Q, Spec, State0) when ?amqqueue_is_classic(Q) ->
QPid = amqqueue:get_pid(Q),
QRef = amqqueue:get_name(Q),
#{no_ack := NoAck,
@@ -194,9 +197,9 @@ consume(Q, Spec, State) when ?amqqueue_is_classic(Q) ->
ExclusiveConsume, Args, OkMsg, ActingUser},
infinity]}) of
ok ->
- %% ask the host process to monitor this pid
%% TODO: track pids as they change
- {ok, State#?STATE{pid = QPid}, [{monitor, QPid, QRef}]};
+ State = ensure_monitor(QPid, QRef, State0),
+ {ok, State#?STATE{pid = QPid}};
Err ->
Err
end.
@@ -211,63 +214,62 @@ cancel(Q, ConsumerTag, OkMsg, ActingUser, State) ->
Err -> Err
end.
--spec settle(rabbit_queue_type:settle_op(), rabbit_types:ctag(),
- [non_neg_integer()], state()) ->
+-spec settle(rabbit_amqqueue:name(), rarabbit_queue_type:settle_op(),
+ rabbit_types:ctag(), [non_neg_integer()], state()) ->
{state(), rabbit_queue_type:actions()}.
-settle(complete, _CTag, MsgIds, State) ->
+settle(_QName, complete, _CTag, MsgIds, State) ->
Pid = State#?STATE.pid,
delegate:invoke_no_result(Pid,
{gen_server2, cast, [{ack, MsgIds, self()}]}),
{State, []};
-settle(Op, _CTag, MsgIds, State) ->
+settle(_QName, Op, _CTag, MsgIds, State) ->
ChPid = self(),
ok = delegate:invoke_no_result(State#?STATE.pid,
{gen_server2, cast,
[{reject, Op == requeue, MsgIds, ChPid}]}),
{State, []}.
-credit(CTag, Credit, Drain, State) ->
+credit(_QName, CTag, Credit, Drain, State) ->
ChPid = self(),
delegate:invoke_no_result(State#?STATE.pid,
{gen_server2, cast,
[{credit, ChPid, CTag, Credit, Drain}]}),
{State, []}.
-handle_event({confirm, MsgSeqNos, Pid}, #?STATE{qref = QRef,
- unconfirmed = U0} = State) ->
+handle_event(QName, {confirm, MsgSeqNos, Pid}, #?STATE{unconfirmed = U0} = State) ->
%% confirms should never result in rejections
{Unconfirmed, ConfirmedSeqNos, []} =
settle_seq_nos(MsgSeqNos, Pid, U0, confirm),
- Actions = [{settled, QRef, ConfirmedSeqNos}],
+ Actions = [{settled, QName, ConfirmedSeqNos}],
%% handle confirm event from queues
%% in this case the classic queue should track each individual publish and
%% the processes involved and only emit a settle action once they have all
%% been received (or DOWN has been received).
%% Hence this part of the confirm logic is queue specific.
{ok, State#?STATE{unconfirmed = Unconfirmed}, Actions};
-handle_event({deliver, _, _, _} = Delivery, #?STATE{} = State) ->
+handle_event(_QName, {deliver, _, _, _} = Delivery, #?STATE{} = State) ->
{ok, State, [Delivery]};
-handle_event({reject_publish, SeqNo, _QPid},
- #?STATE{qref = QRef,
- unconfirmed = U0} = State) ->
+handle_event(QName, {reject_publish, SeqNo, _QPid},
+ #?STATE{unconfirmed = U0} = State) ->
%% It does not matter which queue rejected the message,
%% if any queue did, it should not be confirmed.
{U, Rejected} = reject_seq_no(SeqNo, U0),
- Actions = [{rejected, QRef, Rejected}],
+ Actions = [{rejected, QName, Rejected}],
{ok, State#?STATE{unconfirmed = U}, Actions};
-handle_event({down, Pid, Info}, #?STATE{qref = QRef,
- pid = MasterPid,
- unconfirmed = U0} = State0) ->
+handle_event(QName, {down, Pid, Info}, #?STATE{monitored = Monitored,
+ pid = MasterPid,
+ unconfirmed = U0} = State0) ->
+ State = State0#?STATE{monitored = maps:remove(Pid, Monitored)},
Actions0 = case Pid =:= MasterPid of
true ->
- [{queue_down, QRef}];
+ [{queue_down, QName}];
false ->
[]
end,
case rabbit_misc:is_abnormal_exit(Info) of
false when Info =:= normal andalso Pid == MasterPid ->
%% queue was deleted and masterpid is down
- eol;
+ {eol, []};
false ->
%% this assumes the mirror isn't part of the active set
MsgSeqNos = maps:keys(
@@ -277,9 +279,9 @@ handle_event({down, Pid, Info}, #?STATE{qref = QRef,
{Unconfirmed, Settled, Rejected} =
settle_seq_nos(MsgSeqNos, Pid, U0, down),
Actions = settlement_action(
- settled, QRef, Settled,
- settlement_action(rejected, QRef, Rejected, Actions0)),
- {ok, State0#?STATE{unconfirmed = Unconfirmed}, Actions};
+ settled, QName, Settled,
+ settlement_action(rejected, QName, Rejected, Actions0)),
+ {ok, State#?STATE{unconfirmed = Unconfirmed}, Actions};
true ->
%% any abnormal exit should be considered a full reject of the
%% oustanding message ids - If the message didn't get to all
@@ -294,12 +296,12 @@ handle_event({down, Pid, Info}, #?STATE{qref = QRef,
end
end, [], U0),
U = maps:without(MsgIds, U0),
- {ok, State0#?STATE{unconfirmed = U},
- [{rejected, QRef, MsgIds} | Actions0]}
+ {ok, State#?STATE{unconfirmed = U},
+ [{rejected, QName, MsgIds} | Actions0]}
end;
-handle_event({send_drained, _} = Action, State) ->
+handle_event(_QName, {send_drained, _} = Action, State) ->
{ok, State, [Action]};
-handle_event({send_credit_reply, _} = Action, State) ->
+handle_event(_QName, {send_credit_reply, _} = Action, State) ->
{ok, State, [Action]}.
settlement_action(_Type, _QRef, [], Acc) ->
@@ -319,7 +321,7 @@ deliver(Qs0, #delivery{flow = Flow,
Msg = Msg0#basic_message{id = rabbit_guid:gen()},
Delivery = Delivery0#delivery{message = Msg},
- {MPids, SPids, Qs, Actions} = qpids(Qs0, Confirm, MsgNo),
+ {MPids, SPids, Qs} = qpids(Qs0, Confirm, MsgNo),
case Flow of
%% Here we are tracking messages sent by the rabbit_channel
%% process. We are accessing the rabbit_channel process
@@ -334,14 +336,14 @@ deliver(Qs0, #delivery{flow = Flow,
SMsg = {deliver, Delivery, true},
delegate:invoke_no_result(MPids, {gen_server2, cast, [MMsg]}),
delegate:invoke_no_result(SPids, {gen_server2, cast, [SMsg]}),
- {Qs, Actions}.
+ {Qs, []}.
--spec dequeue(NoAck :: boolean(), LimiterPid :: pid(),
- rabbit_types:ctag(), state()) ->
+-spec dequeue(rabbit_amqqueue:name(), NoAck :: boolean(),
+ LimiterPid :: pid(), rabbit_types:ctag(), state()) ->
{ok, Count :: non_neg_integer(), rabbit_amqqueue:qmsg(), state()} |
{empty, state()}.
-dequeue(NoAck, LimiterPid, _CTag, State) ->
+dequeue(_QName, NoAck, LimiterPid, _CTag, State) ->
QPid = State#?STATE.pid,
case delegate:invoke(QPid, {gen_server2, call,
[{basic_get, self(), NoAck, LimiterPid}, infinity]}) of
@@ -382,14 +384,16 @@ purge(Q) when ?is_amqqueue(Q) ->
qpids(Qs, Confirm, MsgNo) ->
lists:foldl(
- fun ({Q, S0}, {MPidAcc, SPidAcc, Qs0, Actions0}) ->
+ fun ({Q, S0}, {MPidAcc, SPidAcc, Qs0}) ->
QPid = amqqueue:get_pid(Q),
SPids = amqqueue:get_slave_pids(Q),
QRef = amqqueue:get_name(Q),
- Actions = [{monitor, QPid, QRef}
- | [{monitor, P, QRef} || P <- SPids]] ++ Actions0,
+ S1 = ensure_monitor(QPid, QRef, S0),
+ S2 = lists:foldl(fun(SPid, Acc) ->
+ ensure_monitor(SPid, QRef, Acc)
+ end, S1, SPids),
%% confirm record only if necessary
- S = case S0 of
+ S = case S2 of
#?STATE{unconfirmed = U0} ->
Rec = [QPid | SPids],
U = case Confirm of
@@ -398,14 +402,14 @@ qpids(Qs, Confirm, MsgNo) ->
true ->
U0#{MsgNo => #msg_status{pending = Rec}}
end,
- S0#?STATE{pid = QPid,
+ S2#?STATE{pid = QPid,
unconfirmed = U};
stateless ->
- S0
+ S2
end,
{[QPid | MPidAcc], SPidAcc ++ SPids,
- [{Q, S} | Qs0], Actions}
- end, {[], [], [], []}, Qs).
+ [{Q, S} | Qs0]}
+ end, {[], [], []}, Qs).
%% internal-ish
-spec wait_for_promoted_or_stopped(amqqueue:amqqueue()) ->
@@ -480,6 +484,8 @@ notify_decorators(Q) when ?is_amqqueue(Q) ->
QPid = amqqueue:get_pid(Q),
delegate:invoke_no_result(QPid, {gen_server2, cast, [notify_decorators]}).
+is_stateful() -> true.
+
reject_seq_no(SeqNo, U0) ->
reject_seq_no(SeqNo, U0, []).
@@ -522,59 +528,33 @@ update_msg_status(confirm, Pid, #msg_status{pending = P,
update_msg_status(down, Pid, #msg_status{pending = P} = S) ->
S#msg_status{pending = lists:delete(Pid, P)}.
+ensure_monitor(_, _, State = stateless) ->
+ State;
+ensure_monitor(Pid, _, State = #?STATE{monitored = Monitored})
+ when is_map_key(Pid, Monitored) ->
+ State;
+ensure_monitor(Pid, QName, State = #?STATE{monitored = Monitored}) ->
+ _ = erlang:monitor(process, Pid, [{tag, {'DOWN', QName}}]),
+ State#?STATE{monitored = Monitored#{Pid => ok}}.
+
%% part of channel <-> queue api
confirm_to_sender(Pid, QName, MsgSeqNos) ->
- %% the stream queue included the queue type refactoring and thus requires
- %% a different message format
- case rabbit_queue_type:is_supported() of
- true ->
- gen_server:cast(Pid,
- {queue_event, QName,
- {confirm, MsgSeqNos, self()}});
- false ->
- gen_server2:cast(Pid, {confirm, MsgSeqNos, self()})
- end.
+ Msg = {confirm, MsgSeqNos, self()},
+ gen_server:cast(Pid, {queue_event, QName, Msg}).
send_rejection(Pid, QName, MsgSeqNo) ->
- case rabbit_queue_type:is_supported() of
- true ->
- gen_server:cast(Pid, {queue_event, QName,
- {reject_publish, MsgSeqNo, self()}});
- false ->
- gen_server2:cast(Pid, {reject_publish, MsgSeqNo, self()})
- end.
+ Msg = {reject_publish, MsgSeqNo, self()},
+ gen_server:cast(Pid, {queue_event, QName, Msg}).
deliver_to_consumer(Pid, QName, CTag, AckRequired, Message) ->
- case has_classic_queue_type_delivery_support() of
- true ->
- Deliver = {deliver, CTag, AckRequired, [Message]},
- Evt = {queue_event, QName, Deliver},
- gen_server:cast(Pid, Evt);
- false ->
- Deliver = {deliver, CTag, AckRequired, Message},
- gen_server2:cast(Pid, Deliver)
- end.
+ Deliver = {deliver, CTag, AckRequired, [Message]},
+ Evt = {queue_event, QName, Deliver},
+ gen_server:cast(Pid, Evt).
send_drained(Pid, QName, CTagCredits) ->
- case has_classic_queue_type_delivery_support() of
- true ->
- gen_server:cast(Pid, {queue_event, QName,
- {send_drained, CTagCredits}});
- false ->
- gen_server2:cast(Pid, {send_drained, CTagCredits})
- end.
+ gen_server:cast(Pid, {queue_event, QName,
+ {send_drained, CTagCredits}}).
send_credit_reply(Pid, QName, Len) when is_integer(Len) ->
- case rabbit_queue_type:is_supported() of
- true ->
- gen_server:cast(Pid, {queue_event, QName,
- {send_credit_reply, Len}});
- false ->
- gen_server2:cast(Pid, {send_credit_reply, Len})
- end.
-
-has_classic_queue_type_delivery_support() ->
- %% some queue_events were missed in the initial queue_type implementation
- %% this feature flag enables those and completes the initial queue type
- %% API for classic queues
- rabbit_feature_flags:is_enabled(classic_queue_type_delivery_support).
+ gen_server:cast(Pid, {queue_event, QName,
+ {send_credit_reply, Len}}).
diff --git a/deps/rabbit/src/rabbit_confirms.erl b/deps/rabbit/src/rabbit_confirms.erl
index 2fe032d1f1a6..5479b358c3ad 100644
--- a/deps/rabbit/src/rabbit_confirms.erl
+++ b/deps/rabbit/src/rabbit_confirms.erl
@@ -44,7 +44,7 @@ insert(SeqNo, QNames, #resource{kind = exchange} = XName,
unconfirmed = U0} = State)
when is_integer(SeqNo)
andalso is_list(QNames)
- andalso is_map_key(SeqNo, U0) == false ->
+ andalso not is_map_key(SeqNo, U0) ->
U = U0#{SeqNo => {XName, maps:from_list([{Q, ok} || Q <- QNames])}},
S = case S0 of
undefined -> SeqNo;
diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl
index 1ec4d5f06531..2be2bd23b354 100644
--- a/deps/rabbit/src/rabbit_core_ff.erl
+++ b/deps/rabbit/src/rabbit_core_ff.erl
@@ -34,7 +34,8 @@
{stream_queue,
#{desc => "Support queues of type `stream`",
doc_url => "https://www.rabbitmq.com/stream.html",
- stability => stable,
+ %%TODO remove compatibility code
+ stability => required,
depends_on => [quorum_queue]
}}).
@@ -111,14 +112,15 @@
{classic_queue_type_delivery_support,
#{desc => "Bug fix for classic queue deliveries using mixed versions",
doc_url => "https://github.com/rabbitmq/rabbitmq-server/issues/5931",
- stability => stable,
+ %%TODO remove compatibility code
+ stability => required,
depends_on => [stream_queue]
}}).
-rabbit_feature_flag(
{restart_streams,
#{desc => "Support for restarting streams with optional preferred next leader argument. "
- "Used to implement stream leader rebalancing",
+ "Used to implement stream leader rebalancing",
stability => stable,
depends_on => [stream_queue]
}}).
diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl
index bec5a2bacefe..6403ff90f6bc 100644
--- a/deps/rabbit/src/rabbit_fifo_client.erl
+++ b/deps/rabbit/src/rabbit_fifo_client.erl
@@ -12,22 +12,20 @@
-module(rabbit_fifo_client).
-export([
+ init/1,
init/2,
- init/3,
- init/5,
checkout/5,
cancel_checkout/2,
- enqueue/2,
enqueue/3,
- dequeue/3,
+ enqueue/4,
+ dequeue/4,
settle/3,
return/3,
discard/3,
credit/4,
- handle_ra_event/3,
+ handle_ra_event/4,
untracked_enqueue/2,
purge/1,
- cluster_name/1,
update_machine_state/2,
pending_size/1,
stat/1,
@@ -44,22 +42,18 @@
-type seq() :: non_neg_integer().
-type action() :: {send_credit_reply, Available :: non_neg_integer()} |
{send_drained, CTagCredit ::
- {rabbit_fifo:consumer_tag(), non_neg_integer()}}.
+ {rabbit_fifo:consumer_tag(), non_neg_integer()}} |
+ rabbit_queue_type:action().
-type actions() :: [action()].
--type cluster_name() :: rabbit_types:r(queue).
-
-record(consumer, {last_msg_id :: seq() | -1,
ack = false :: boolean(),
delivery_count = 0 :: non_neg_integer()}).
--record(cfg, {cluster_name :: cluster_name(),
- servers = [] :: [ra:server_id()],
+-record(cfg, {servers = [] :: [ra:server_id()],
soft_limit = ?SOFT_LIMIT :: non_neg_integer(),
- block_handler = fun() -> ok end :: fun(() -> term()),
- unblock_handler = fun() -> ok end :: fun(() -> ok),
- timeout :: non_neg_integer(),
- version = 0 :: non_neg_integer()}).
+ timeout :: non_neg_integer()
+ }).
-record(state, {cfg :: #cfg{},
leader :: undefined | ra:server_id(),
@@ -87,55 +81,39 @@
%% @doc Create the initial state for a new rabbit_fifo sessions. A state is needed
%% to interact with a rabbit_fifo queue using @module.
-%% @param ClusterName the id of the cluster to interact with
%% @param Servers The known servers of the queue. If the current leader is known
%% ensure the leader node is at the head of the list.
--spec init(cluster_name(), [ra:server_id()]) -> state().
-init(ClusterName, Servers) ->
- init(ClusterName, Servers, ?SOFT_LIMIT).
+-spec init([ra:server_id()]) -> state().
+init(Servers) ->
+ init(Servers, ?SOFT_LIMIT).
%% @doc Create the initial state for a new rabbit_fifo sessions. A state is needed
%% to interact with a rabbit_fifo queue using @module.
-%% @param ClusterName the id of the cluster to interact with
%% @param Servers The known servers of the queue. If the current leader is known
%% ensure the leader node is at the head of the list.
%% @param MaxPending size defining the max number of pending commands.
--spec init(cluster_name(), [ra:server_id()], non_neg_integer()) -> state().
-init(ClusterName = #resource{}, Servers, SoftLimit) ->
- Timeout = application:get_env(kernel, net_ticktime, 60) + 5,
- #state{cfg = #cfg{cluster_name = ClusterName,
- servers = Servers,
- soft_limit = SoftLimit,
- timeout = Timeout * 1000}}.
-
--spec init(cluster_name(), [ra:server_id()], non_neg_integer(), fun(() -> ok),
- fun(() -> ok)) -> state().
-init(ClusterName = #resource{}, Servers, SoftLimit, BlockFun, UnblockFun) ->
- %% net ticktime is in seconds
+-spec init([ra:server_id()], non_neg_integer()) -> state().
+init(Servers, SoftLimit) ->
Timeout = application:get_env(kernel, net_ticktime, 60) + 5,
- #state{cfg = #cfg{cluster_name = ClusterName,
- servers = Servers,
- block_handler = BlockFun,
- unblock_handler = UnblockFun,
+ #state{cfg = #cfg{servers = Servers,
soft_limit = SoftLimit,
timeout = Timeout * 1000}}.
-
%% @doc Enqueues a message.
+%% @param QueueName Name of the queue.
%% @param Correlation an arbitrary erlang term used to correlate this
%% command when it has been applied.
%% @param Msg an arbitrary erlang term representing the message.
%% @param State the current {@module} state.
%% @returns
-%% `{ok | slow, State}' if the command was successfully sent. If the return
-%% tag is `slow' it means the limit is approaching and it is time to slow down
-%% the sending rate.
+%% `{ok, State, Actions}' if the command was successfully sent
%% {@module} assigns a sequence number to every raft command it issues. The
%% SequenceNumber can be correlated to the applied sequence numbers returned
%% by the {@link handle_ra_event/2. handle_ra_event/2} function.
--spec enqueue(Correlation :: term(), Msg :: term(), State :: state()) ->
- {ok | slow | reject_publish, state()}.
-enqueue(Correlation, Msg,
+-spec enqueue(rabbit_amqqueue:name(), Correlation :: term(),
+ Msg :: term(), State :: state()) ->
+ {ok, state(), actions()} | {reject_publish, state()}.
+enqueue(QName, Correlation, Msg,
#state{queue_status = undefined,
next_enqueue_seq = 1,
cfg = #cfg{servers = Servers,
@@ -145,7 +123,7 @@ enqueue(Correlation, Msg,
case rpc:call(Node, ra_machine, version, [{machine, rabbit_fifo, #{}}]) of
0 ->
%% the leader is running the old version
- enqueue(Correlation, Msg, State0#state{queue_status = go});
+ enqueue(QName, Correlation, Msg, State0#state{queue_status = go});
N when is_integer(N) ->
%% were running the new version on the leader do sync initialisation
%% of enqueuer session
@@ -155,8 +133,8 @@ enqueue(Correlation, Msg,
{reject_publish, State0#state{leader = Leader,
queue_status = reject_publish}};
{ok, ok, Leader} ->
- enqueue(Correlation, Msg, State0#state{leader = Leader,
- queue_status = go});
+ enqueue(QName, Correlation, Msg, State0#state{leader = Leader,
+ queue_status = go});
{error, {no_more_servers_to_try, _Errs}} ->
%% if we are not able to process the register command
%% it is safe to reject the message as we never attempted
@@ -172,18 +150,17 @@ enqueue(Correlation, Msg,
{badrpc, nodedown} ->
{reject_publish, State0}
end;
-enqueue(_Correlation, _Msg,
+enqueue(_QName, _Correlation, _Msg,
#state{queue_status = reject_publish,
cfg = #cfg{}} = State) ->
{reject_publish, State};
-enqueue(Correlation, Msg,
+enqueue(QName, Correlation, Msg,
#state{slow = Slow,
pending = Pending,
queue_status = go,
next_seq = Seq,
next_enqueue_seq = EnqueueSeq,
- cfg = #cfg{soft_limit = SftLmt,
- block_handler = BlockFun}} = State0) ->
+ cfg = #cfg{soft_limit = SftLmt}} = State0) ->
Server = pick_server(State0),
% by default there is no correlation id
Cmd = rabbit_fifo:make_enqueue(self(), EnqueueSeq, Msg),
@@ -198,46 +175,44 @@ enqueue(Correlation, Msg,
slow = Tag == slow},
case Tag of
slow when not Slow ->
- BlockFun(),
- {slow, set_timer(State)};
+ {ok, set_timer(QName, State), [{block, cluster_name(State)}]};
_ ->
- {ok, State}
+ {ok, State, []}
end.
%% @doc Enqueues a message.
+%% @param QueueName Name of the queue.
%% @param Msg an arbitrary erlang term representing the message.
%% @param State the current {@module} state.
%% @returns
-%% `{ok | slow, State}' if the command was successfully sent. If the return
-%% tag is `slow' it means the limit is approaching and it is time to slow down
-%% the sending rate.
+%% `{ok, State, Actions}' if the command was successfully sent.
%% {@module} assigns a sequence number to every raft command it issues. The
%% SequenceNumber can be correlated to the applied sequence numbers returned
%% by the {@link handle_ra_event/2. handle_ra_event/2} function.
%%
--spec enqueue(Msg :: term(), State :: state()) ->
- {ok | slow | reject_publish, state()}.
-enqueue(Msg, State) ->
- enqueue(undefined, Msg, State).
+-spec enqueue(rabbit_amqqueue:name(), Msg :: term(), State :: state()) ->
+ {ok, state(), actions()} | {reject_publish, state()}.
+enqueue(QName, Msg, State) ->
+ enqueue(QName, undefined, Msg, State).
%% @doc Dequeue a message from the queue.
%%
%% This is a synchronous call. I.e. the call will block until the command
%% has been accepted by the ra process or it times out.
%%
+%% @param QueueName Name of the queue.
%% @param ConsumerTag a unique tag to identify this particular consumer.
%% @param Settlement either `settled' or `unsettled'. When `settled' no
%% further settlement needs to be done.
%% @param State The {@module} state.
%%
%% @returns `{ok, IdMsg, State}' or `{error | timeout, term()}'
--spec dequeue(rabbit_fifo:consumer_tag(),
+-spec dequeue(rabbit_amqqueue:name(), rabbit_fifo:consumer_tag(),
Settlement :: settled | unsettled, state()) ->
{ok, non_neg_integer(), term(), non_neg_integer()}
| {empty, state()} | {error | timeout, term()}.
-dequeue(ConsumerTag, Settlement,
- #state{cfg = #cfg{timeout = Timeout,
- cluster_name = QName}} = State0) ->
+dequeue(QueueName, ConsumerTag, Settlement,
+ #state{cfg = #cfg{timeout = Timeout}} = State0) ->
Node = pick_server(State0),
ConsumerId = consumer_id(ConsumerTag),
case ra:process_command(Node,
@@ -255,7 +230,7 @@ dequeue(ConsumerTag, Settlement,
IsDelivered = Count > 0,
Msg = add_delivery_count_header(Msg0, Count),
{ok, MsgsReady,
- {QName, qref(Leader), MsgId, IsDelivered, Msg},
+ {QueueName, qref(Leader), MsgId, IsDelivered, Msg},
State0#state{leader = Leader}};
{ok, {error, _} = Err, _Leader} ->
Err;
@@ -274,10 +249,6 @@ add_delivery_count_header(Msg, _Count) ->
%% @param ConsumerTag the tag uniquely identifying the consumer.
%% @param MsgIds the message ids received with the {@link rabbit_fifo:delivery/0.}
%% @param State the {@module} state
-%% @returns
-%% `{ok | slow, State}' if the command was successfully sent. If the return
-%% tag is `slow' it means the limit is approaching and it is time to slow down
-%% the sending rate.
%%
-spec settle(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) ->
{state(), list()}.
@@ -305,9 +276,7 @@ settle(ConsumerTag, [_|_] = MsgIds,
%% from {@link rabbit_fifo:delivery/0.}
%% @param State the {@module} state
%% @returns
-%% `{State, list()}' if the command was successfully sent. If the return
-%% tag is `slow' it means the limit is approaching and it is time to slow down
-%% the sending rate.
+%% `{State, list()}' if the command was successfully sent.
%%
-spec return(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) ->
{state(), list()}.
@@ -334,10 +303,6 @@ return(ConsumerTag, [_|_] = MsgIds,
%% @param MsgIds the message ids to discard
%% from {@link rabbit_fifo:delivery/0.}
%% @param State the {@module} state
-%% @returns
-%% `{ok | slow, State}' if the command was successfully sent. If the return
-%% tag is `slow' it means the limit is approaching and it is time to slow down
-%% the sending rate.
-spec discard(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) ->
{state(), list()}.
discard(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) ->
@@ -501,11 +466,6 @@ stat(Leader, Timeout) ->
{timeout, _} = Error -> Error
end.
-%% @doc returns the cluster name
--spec cluster_name(state()) -> cluster_name().
-cluster_name(#state{cfg = #cfg{cluster_name = ClusterName}}) ->
- ClusterName.
-
update_machine_state(Server, Conf) ->
case ra:process_command(Server, rabbit_fifo:make_update_config(Conf), ?COMMAND_TIMEOUT) of
{ok, ok, _} ->
@@ -536,6 +496,7 @@ update_machine_state(Server, Conf) ->
%% end
%% '''
%%
+%% @param QName Name of the queue.
%% @param From the {@link ra:server_id().} of the sending process.
%% @param Event the body of the `ra_event'.
%% @param State the current {@module} state.
@@ -557,13 +518,12 @@ update_machine_state(Server, Conf) ->
%%
`MsgId' is a consumer scoped monotonically incrementing id that can be
%% used to {@link settle/3.} (roughly: AMQP 0.9.1 ack) message once finished
%% with them.
--spec handle_ra_event(ra:server_id(), ra_server_proc:ra_event_body(), state()) ->
+-spec handle_ra_event(rabbit_amqqueue:name(), ra:server_id(),
+ ra_server_proc:ra_event_body(), state()) ->
{internal, Correlators :: [term()], actions(), state()} |
- {rabbit_fifo:client_msg(), state()} | eol.
-handle_ra_event(From, {applied, Seqs},
- #state{cfg = #cfg{cluster_name = QRef,
- soft_limit = SftLmt,
- unblock_handler = UnblockFun}} = State0) ->
+ {rabbit_fifo:client_msg(), state()} | {eol, actions()}.
+handle_ra_event(QName, From, {applied, Seqs},
+ #state{cfg = #cfg{soft_limit = SftLmt}} = State0) ->
{Corrs, Actions0, State1} = lists:foldl(fun seq_applied/2,
{[], [], State0#state{leader = From}},
@@ -572,7 +532,12 @@ handle_ra_event(From, {applied, Seqs},
[] ->
lists:reverse(Actions0);
_ ->
- [{settled, QRef, Corrs}
+ %%TODO consider using lists:foldr/3 above because
+ %% Corrs is returned in the wrong order here.
+ %% The wrong order does not matter much because the channel sorts the
+ %% sequence numbers before confirming to the client. But rabbit_fifo_client
+ %% is sequence numer agnostic: it handles any correlation terms.
+ [{settled, QName, Corrs}
| lists:reverse(Actions0)]
end,
case maps:size(State1#state.pending) < SftLmt of
@@ -600,18 +565,17 @@ handle_ra_event(From, {applied, Seqs},
send_command(Node, undefined, C,
normal, S0)
end, State2, Commands),
- UnblockFun(),
- {ok, State, Actions};
+ {ok, State, [{unblock, cluster_name(State)} | Actions]};
_ ->
{ok, State1, Actions}
end;
-handle_ra_event(From, {machine, {delivery, _ConsumerTag, _} = Del}, State0) ->
- handle_delivery(From, Del, State0);
-handle_ra_event(_, {machine, {queue_status, Status}},
+handle_ra_event(QName, From, {machine, {delivery, _ConsumerTag, _} = Del}, State0) ->
+ handle_delivery(QName, From, Del, State0);
+handle_ra_event(_QName, _, {machine, {queue_status, Status}},
#state{} = State) ->
%% just set the queue status
{ok, State#state{queue_status = Status}, []};
-handle_ra_event(Leader, {machine, leader_change},
+handle_ra_event(_QName, Leader, {machine, leader_change},
#state{leader = OldLeader} = State0) ->
%% we need to update leader
%% and resend any pending commands
@@ -619,29 +583,29 @@ handle_ra_event(Leader, {machine, leader_change},
[?MODULE, OldLeader, Leader]),
State = resend_all_pending(State0#state{leader = Leader}),
{ok, State, []};
-handle_ra_event(_From, {rejected, {not_leader, Leader, _Seq}},
+handle_ra_event(_QName, _From, {rejected, {not_leader, Leader, _Seq}},
#state{leader = Leader} = State) ->
{ok, State, []};
-handle_ra_event(_From, {rejected, {not_leader, Leader, _Seq}},
+handle_ra_event(_QName, _From, {rejected, {not_leader, Leader, _Seq}},
#state{leader = OldLeader} = State0) ->
rabbit_log:debug("~ts: Detected QQ leader change (rejection) from ~w to ~w",
[?MODULE, OldLeader, Leader]),
State = resend_all_pending(State0#state{leader = Leader}),
{ok, cancel_timer(State), []};
-handle_ra_event(_From, {rejected, {not_leader, _UndefinedMaybe, _Seq}}, State0) ->
+handle_ra_event(_QName, _From, {rejected, {not_leader, _UndefinedMaybe, _Seq}}, State0) ->
% TODO: how should these be handled? re-sent on timer or try random
{ok, State0, []};
-handle_ra_event(_, timeout, #state{cfg = #cfg{servers = Servers}} = State0) ->
+handle_ra_event(QName, _, timeout, #state{cfg = #cfg{servers = Servers}} = State0) ->
case find_leader(Servers) of
undefined ->
%% still no leader, set the timer again
- {ok, set_timer(State0), []};
+ {ok, set_timer(QName, State0), []};
Leader ->
State = resend_all_pending(State0#state{leader = Leader}),
{ok, State, []}
end;
-handle_ra_event(_Leader, {machine, eol}, _State0) ->
- eol.
+handle_ra_event(_QName, _Leader, {machine, eol}, State) ->
+ {eol, [{unblock, cluster_name(State)}]}.
%% @doc Attempts to enqueue a message using cast semantics. This provides no
%% guarantees or retries if the message fails to achieve consensus or if the
@@ -737,9 +701,8 @@ maybe_auto_ack(false, {deliver, Tag, _Ack, Msgs} = Deliver, State0) ->
{State, Actions} = settle(Tag, MsgIds, State0),
{ok, State, [Deliver] ++ Actions}.
-handle_delivery(Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs},
- #state{cfg = #cfg{cluster_name = QName},
- consumer_deliveries = CDels0} = State0)
+handle_delivery(QName, Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs},
+ #state{consumer_deliveries = CDels0} = State0)
when is_map_key(Tag, CDels0) ->
QRef = qref(Leader),
{LastId, _} = lists:last(IdMsgs),
@@ -782,7 +745,7 @@ handle_delivery(Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs},
[] ->
{ok, State0, []};
IdMsgs2 ->
- handle_delivery(Leader, {delivery, Tag, IdMsgs2}, State0)
+ handle_delivery(QName, Leader, {delivery, Tag, IdMsgs2}, State0)
end;
C when FstId =:= 0 ->
% the very first delivery
@@ -793,7 +756,7 @@ handle_delivery(Leader, {delivery, Tag, [{FstId, _} | _] = IdMsgs},
C#consumer{last_msg_id = LastId},
CDels0)})
end;
-handle_delivery(_Leader, {delivery, Tag, [_ | _] = IdMsgs},
+handle_delivery(_QName, _Leader, {delivery, Tag, [_ | _] = IdMsgs},
#state{consumer_deliveries = CDels0} = State0)
when not is_map_key(Tag, CDels0) ->
%% Note:
@@ -903,9 +866,8 @@ add_command(Cid, return, MsgIds, Acc) ->
add_command(Cid, discard, MsgIds, Acc) ->
[rabbit_fifo:make_discard(Cid, MsgIds) | Acc].
-set_timer(#state{leader = Leader0,
- cfg = #cfg{servers = [Server | _],
- cluster_name = QName}} = State) ->
+set_timer(QName, #state{leader = Leader0,
+ cfg = #cfg{servers = [Server | _]}} = State) ->
Leader = case Leader0 of
undefined -> Server;
_ ->
@@ -950,3 +912,8 @@ find_leader([Server | Servers]) ->
qref({Ref, _}) -> Ref;
qref(Ref) -> Ref.
+
+-spec cluster_name(state()) ->
+ atom().
+cluster_name(#state{cfg = #cfg{servers = [{Name, _Node} | _]}}) ->
+ Name.
diff --git a/deps/rabbit/src/rabbit_fifo_dlx_worker.erl b/deps/rabbit/src/rabbit_fifo_dlx_worker.erl
index 8322de87bb80..3dbbfa79c5ad 100644
--- a/deps/rabbit/src/rabbit_fifo_dlx_worker.erl
+++ b/deps/rabbit/src/rabbit_fifo_dlx_worker.erl
@@ -151,8 +151,9 @@ handle_cast({queue_event, QRef, Evt},
State1 = State0#state{queue_type_state = QTypeState1},
State = handle_queue_actions(Actions, State1),
{noreply, State};
- eol ->
- remove_queue(QRef, State0);
+ {eol, Actions} ->
+ State = handle_queue_actions(Actions, State0),
+ remove_queue(QRef, State);
{protocol_error, _Type, _Reason, _Args} ->
{noreply, State0}
end;
@@ -177,10 +178,10 @@ handle_info({'DOWN', Ref, process, _, _},
rabbit_log:debug("~ts terminating itself because leader of ~ts is down...",
[?MODULE, rabbit_misc:rs(QRef)]),
supervisor:terminate_child(rabbit_fifo_dlx_sup, self());
-handle_info({'DOWN', _MRef, process, QPid, Reason},
+handle_info({{'DOWN', QName}, _MRef, process, QPid, Reason},
#state{queue_type_state = QTypeState0} = State0) ->
%% received from target classic queue
- case rabbit_queue_type:handle_down(QPid, Reason, QTypeState0) of
+ case rabbit_queue_type:handle_down(QPid, QName, Reason, QTypeState0) of
{ok, QTypeState, Actions} ->
State = State0#state{queue_type_state = QTypeState},
{noreply, handle_queue_actions(Actions, State)};
@@ -247,6 +248,10 @@ handle_queue_actions(Actions, State0) ->
handle_rejected(QRef, MsgSeqs, S0);
({queue_down, _QRef}, S0) ->
%% target classic queue is down, but not deleted
+ S0;
+ ({block, _QName}, S0) ->
+ S0;
+ ({unblock, _QName}, S0) ->
S0
end, State0, Actions).
diff --git a/deps/rabbit/src/rabbit_limiter.erl b/deps/rabbit/src/rabbit_limiter.erl
index dfba86b3127d..eb4df6268554 100644
--- a/deps/rabbit/src/rabbit_limiter.erl
+++ b/deps/rabbit/src/rabbit_limiter.erl
@@ -133,7 +133,7 @@
-type lstate() :: #lstate{pid :: pid(),
prefetch_limited :: boolean()}.
--type qstate() :: #qstate{pid :: pid(),
+-type qstate() :: #qstate{pid :: pid() | none,
state :: 'dormant' | 'active' | 'suspended'}.
-type credit_mode() :: 'manual' | 'drain' | 'auto'.
diff --git a/deps/rabbit/src/rabbit_memory_monitor.erl b/deps/rabbit/src/rabbit_memory_monitor.erl
index 73f86a79f920..4e71c35a1c5d 100644
--- a/deps/rabbit/src/rabbit_memory_monitor.erl
+++ b/deps/rabbit/src/rabbit_memory_monitor.erl
@@ -75,6 +75,9 @@ report_ram_duration(Pid, QueueDuration) ->
stop() ->
gen_server2:cast(?SERVER, stop).
+-spec conserve_resources(pid(),
+ rabbit_alarm:resource_alarm_source(),
+ rabbit_alarm:resource_alert()) -> ok.
%% Paging should be enabled/disabled only in response to disk resource alarms
%% for the current node.
conserve_resources(Pid, disk, {_, Conserve, Node}) when node(Pid) =:= Node ->
diff --git a/deps/rabbit/src/rabbit_mirror_queue_sync.erl b/deps/rabbit/src/rabbit_mirror_queue_sync.erl
index f99d8a2fe0da..bf422e2fcf09 100644
--- a/deps/rabbit/src/rabbit_mirror_queue_sync.erl
+++ b/deps/rabbit/src/rabbit_mirror_queue_sync.erl
@@ -314,6 +314,9 @@ broadcast(SPids, Msg) ->
SPid ! Msg
end || SPid <- SPids].
+-spec conserve_resources(pid(),
+ rabbit_alarm:resource_alarm_source(),
+ rabbit_alarm:resource_alert()) -> ok.
conserve_resources(Pid, Source, {_, Conserve, _}) ->
Pid ! {conserve_resources, Source, Conserve},
ok.
diff --git a/deps/rabbit/src/rabbit_networking.erl b/deps/rabbit/src/rabbit_networking.erl
index 9b9ee8213edf..f2095129515b 100644
--- a/deps/rabbit/src/rabbit_networking.erl
+++ b/deps/rabbit/src/rabbit_networking.erl
@@ -38,7 +38,7 @@
%% Used by TCP-based transports, e.g. STOMP adapter
-export([tcp_listener_addresses/1,
- tcp_listener_spec/9, tcp_listener_spec/10,
+ tcp_listener_spec/9, tcp_listener_spec/10, tcp_listener_spec/11,
ensure_ssl/0, fix_ssl_options/1, poodle_check/1]).
-export([tcp_listener_started/4, tcp_listener_stopped/4]).
@@ -206,13 +206,24 @@ tcp_listener_spec(NamePrefix, Address, SocketOpts, Transport, ProtoSup, ProtoOpt
any(), protocol(), non_neg_integer(), non_neg_integer(), label()) ->
supervisor:child_spec().
-tcp_listener_spec(NamePrefix, {IPAddress, Port, Family}, SocketOpts,
+tcp_listener_spec(NamePrefix, Address, SocketOpts,
Transport, ProtoSup, ProtoOpts, Protocol, NumAcceptors,
ConcurrentConnsSupsCount, Label) ->
- Args = [IPAddress, Port, Transport, [Family | SocketOpts], ProtoSup, ProtoOpts,
+ tcp_listener_spec(NamePrefix, Address, SocketOpts, Transport, ProtoSup, ProtoOpts,
+ Protocol, NumAcceptors, ConcurrentConnsSupsCount, supervisor, Label).
+
+-spec tcp_listener_spec
+ (name_prefix(), address(), [gen_tcp:listen_option()], module(), module(),
+ any(), protocol(), non_neg_integer(), non_neg_integer(), supervisor:worker(), label()) ->
+ supervisor:child_spec().
+
+tcp_listener_spec(NamePrefix, {IPAddress, Port, Family}, SocketOpts,
+ Transport, ProtoHandler, ProtoOpts, Protocol, NumAcceptors,
+ ConcurrentConnsSupsCount, ConnectionType, Label) ->
+ Args = [IPAddress, Port, Transport, [Family | SocketOpts], ProtoHandler, ProtoOpts,
{?MODULE, tcp_listener_started, [Protocol, SocketOpts]},
{?MODULE, tcp_listener_stopped, [Protocol, SocketOpts]},
- NumAcceptors, ConcurrentConnsSupsCount, Label],
+ NumAcceptors, ConcurrentConnsSupsCount, ConnectionType, Label],
{rabbit_misc:tcp_name(NamePrefix, IPAddress, Port),
{tcp_listener_sup, start_link, Args},
transient, infinity, supervisor, [tcp_listener_sup]}.
diff --git a/deps/rabbit/src/rabbit_queue_consumers.erl b/deps/rabbit/src/rabbit_queue_consumers.erl
index d98c26a4d93d..4bcb897f9881 100644
--- a/deps/rabbit/src/rabbit_queue_consumers.erl
+++ b/deps/rabbit/src/rabbit_queue_consumers.erl
@@ -120,7 +120,7 @@ count() -> lists:sum([Count || #cr{consumer_count = Count} <- all_ch_record()]).
unacknowledged_message_count() ->
lists:sum([?QUEUE:len(C#cr.acktags) || C <- all_ch_record()]).
--spec add(rabbit_amqqueue:name(), ch(), rabbit_types:ctag(), boolean(), pid(), boolean(),
+-spec add(rabbit_amqqueue:name(), ch(), rabbit_types:ctag(), boolean(), pid() | none, boolean(),
non_neg_integer(), rabbit_framing:amqp_table(), boolean(),
rabbit_types:username(), state())
-> state().
diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl
index a29f8cec6855..51ee4d74dbcc 100644
--- a/deps/rabbit/src/rabbit_queue_type.erl
+++ b/deps/rabbit/src/rabbit_queue_type.erl
@@ -15,7 +15,6 @@
discover/1,
feature_flag_name/1,
default/0,
- is_supported/0,
is_enabled/1,
is_compatible/4,
declare/2,
@@ -28,13 +27,14 @@
remove/2,
info/2,
state_info/1,
+ format_status/1,
info_down/2,
info_down/3,
%% stateful client API
new/2,
consume/3,
cancel/5,
- handle_down/3,
+ handle_down/4,
handle_event/3,
module/2,
deliver/3,
@@ -42,7 +42,6 @@
credit/5,
dequeue/5,
fold_state/3,
- find_name_from_pid/2,
is_policy_applicable/2,
is_server_named_allowed/1,
arguments/1,
@@ -50,8 +49,7 @@
notify_decorators/1
]).
--type queue_name() :: rabbit_types:r(queue).
--type queue_ref() :: queue_name() | atom().
+-type queue_name() :: rabbit_amqqueue:name().
-type queue_state() :: term().
-type msg_tag() :: term().
-type arguments() :: queue_arguments | consumer_arguments.
@@ -66,17 +64,13 @@
%% TODO resolve all registered queue types from registry
-define(QUEUE_TYPES, [rabbit_classic_queue, rabbit_quorum_queue, rabbit_stream_queue]).
--define(QREF(QueueReference),
- (is_tuple(QueueReference) andalso element(1, QueueReference) == resource)
- orelse is_atom(QueueReference)).
-%% anything that the host process needs to do on behalf of the queue type
-%% session, like knowing when to notify on monitor down
+%% anything that the host process needs to do on behalf of the queue type session
-type action() ::
- {monitor, Pid :: pid(), queue_ref()} |
%% indicate to the queue type module that a message has been delivered
%% fully to the queue
{settled, Success :: boolean(), [msg_tag()]} |
- {deliver, rabbit_types:ctag(), boolean(), [rabbit_amqqueue:qmsg()]}.
+ {deliver, rabbit_types:ctag(), boolean(), [rabbit_amqqueue:qmsg()]} |
+ {block | unblock, QueueName :: term()}.
-type actions() :: [action()].
@@ -85,7 +79,6 @@
term().
-record(ctx, {module :: module(),
- name :: queue_name(),
%% "publisher confirm queue accounting"
%% queue type implementation should emit a:
%% {settle, Success :: boolean(), msg_tag()}
@@ -97,15 +90,14 @@
state :: queue_state()}).
--record(?STATE, {ctxs = #{} :: #{queue_ref() => #ctx{}},
- monitor_registry = #{} :: #{pid() => queue_ref()}
+-record(?STATE, {ctxs = #{} :: #{queue_name() => #ctx{}}
}).
-opaque state() :: #?STATE{}.
-type consume_spec() :: #{no_ack := boolean(),
channel_pid := pid(),
- limiter_pid => pid(),
+ limiter_pid => pid() | none,
limiter_active => boolean(),
prefetch_count => non_neg_integer(),
consumer_tag := rabbit_types:ctag(),
@@ -116,9 +108,6 @@
-% copied from rabbit_amqqueue
--type absent_reason() :: 'nodedown' | 'crashed' | stopped | timeout.
-
-type settle_op() :: 'complete' | 'requeue' | 'discard'.
-export_type([state/0,
@@ -136,7 +125,7 @@
-callback declare(amqqueue:amqqueue(), node()) ->
{'new' | 'existing' | 'owner_died', amqqueue:amqqueue()} |
- {'absent', amqqueue:amqqueue(), absent_reason()} |
+ {'absent', amqqueue:amqqueue(), rabbit_amqqueue:absent_reason()} |
{'protocol_error', Type :: atom(), Reason :: string(), Args :: term()} |
{'error', Err :: term() }.
@@ -148,20 +137,21 @@
rabbit_types:error(in_use | not_empty) |
{protocol_error, Type :: atom(), Reason :: string(), Args :: term()}.
--callback recover(rabbit_types:vhost(), [amqqueue:amqqueue()]) ->
- {Recovered :: [amqqueue:amqqueue()],
- Failed :: [amqqueue:amqqueue()]}.
-
%% checks if the queue should be recovered
-callback is_recoverable(amqqueue:amqqueue()) ->
boolean().
+-callback recover(rabbit_types:vhost(), [amqqueue:amqqueue()]) ->
+ {Recovered :: [amqqueue:amqqueue()],
+ Failed :: [amqqueue:amqqueue()]}.
+
-callback purge(amqqueue:amqqueue()) ->
{ok, non_neg_integer()} | {error, term()}.
-callback policy_changed(amqqueue:amqqueue()) -> ok.
-%% stateful
+-callback is_stateful() -> boolean().
+
%% intitialise and return a queue type specific session context
-callback init(amqqueue:amqqueue()) -> {ok, queue_state()} | {error, Reason :: term()}.
@@ -184,24 +174,26 @@
%% any async events returned from the queue system should be processed through
%% this
--callback handle_event(Event :: event(),
+-callback handle_event(queue_name(),
+ Event :: event(),
queue_state()) ->
- {ok, queue_state(), actions()} | {error, term()} | eol |
+ {ok, queue_state(), actions()} | {error, term()} | {eol, actions()} |
{protocol_error, Type :: atom(), Reason :: string(), Args :: term()}.
-callback deliver([{amqqueue:amqqueue(), queue_state()}],
Delivery :: term()) ->
{[{amqqueue:amqqueue(), queue_state()}], actions()}.
--callback settle(settle_op(), rabbit_types:ctag(), [non_neg_integer()], queue_state()) ->
+-callback settle(queue_name(), settle_op(), rabbit_types:ctag(),
+ [non_neg_integer()], queue_state()) ->
{queue_state(), actions()} |
{'protocol_error', Type :: atom(), Reason :: string(), Args :: term()}.
--callback credit(rabbit_types:ctag(),
+-callback credit(queue_name(), rabbit_types:ctag(),
non_neg_integer(), Drain :: boolean(), queue_state()) ->
{queue_state(), actions()}.
--callback dequeue(NoAck :: boolean(), LimiterPid :: pid(),
+-callback dequeue(queue_name(), NoAck :: boolean(), LimiterPid :: pid(),
rabbit_types:ctag(), queue_state()) ->
{ok, Count :: non_neg_integer(), rabbit_amqqueue:qmsg(), queue_state()} |
{empty, queue_state()} |
@@ -245,12 +237,6 @@ feature_flag_name(_) ->
default() ->
rabbit_classic_queue.
-%% is the queue type API supported in the cluster
-is_supported() ->
- %% the stream_queue feature enables
- %% the queue_type internal message API
- rabbit_feature_flags:is_enabled(stream_queue).
-
%% is a specific queue type implementation enabled
-spec is_enabled(module()) -> boolean().
is_enabled(Type) ->
@@ -263,7 +249,7 @@ is_compatible(Type, Durable, Exclusive, AutoDelete) ->
-spec declare(amqqueue:amqqueue(), node()) ->
{'new' | 'existing' | 'owner_died', amqqueue:amqqueue()} |
- {'absent', amqqueue:amqqueue(), absent_reason()} |
+ {'absent', amqqueue:amqqueue(), rabbit_amqqueue:absent_reason()} |
{protocol_error, Type :: atom(), Reason :: string(), Args :: term()} |
{'error', Err :: term() }.
declare(Q0, Node) ->
@@ -297,7 +283,7 @@ stat(Q) ->
Mod = amqqueue:get_type(Q),
Mod:stat(Q).
--spec remove(queue_ref(), state()) -> state().
+-spec remove(queue_name(), state()) -> state().
remove(QRef, #?STATE{ctxs = Ctxs0} = State) ->
case maps:take(QRef, Ctxs0) of
error ->
@@ -319,17 +305,16 @@ info(Q, Items) ->
fold_state(Fun, Acc, #?STATE{ctxs = Ctxs}) ->
maps:fold(Fun, Acc, Ctxs).
-%% slight hack to help provide backwards compatibility in the channel
-%% better than scanning the entire queue state
-find_name_from_pid(Pid, #?STATE{monitor_registry = Mons}) ->
- maps:get(Pid, Mons, undefined).
-
state_info(#ctx{state = S,
module = Mod}) ->
Mod:state_info(S);
state_info(_) ->
#{}.
+-spec format_status(state()) -> map().
+format_status(#?STATE{ctxs = Ctxs}) ->
+ #{num_queue_clients => maps:size(Ctxs)}.
+
down_keys() -> ?DOWN_KEYS.
info_down(Q, DownReason) ->
@@ -386,12 +371,11 @@ init() ->
-spec close(state()) -> ok.
close(#?STATE{ctxs = Contexts}) ->
- _ = maps:map(
- fun (_, #ctx{module = Mod,
- state = S}) ->
- ok = Mod:close(S)
- end, Contexts),
- ok.
+ maps:foreach(
+ fun (_, #ctx{module = Mod,
+ state = S}) ->
+ ok = Mod:close(S)
+ end, Contexts).
-spec new(amqqueue:amqqueue(), state()) -> state().
new(Q, State) when ?is_amqqueue(Q) ->
@@ -399,13 +383,13 @@ new(Q, State) when ?is_amqqueue(Q) ->
set_ctx(Q, Ctx, State).
-spec consume(amqqueue:amqqueue(), consume_spec(), state()) ->
- {ok, state(), actions()} | {error, term()}.
+ {ok, state()} | {error, term()}.
consume(Q, Spec, State) ->
#ctx{state = CtxState0} = Ctx = get_ctx(Q, State),
Mod = amqqueue:get_type(Q),
case Mod:consume(Q, Spec, CtxState0) of
- {ok, CtxState, Actions} ->
- return_ok(set_ctx(Q, Ctx#ctx{state = CtxState}, State), Actions);
+ {ok, CtxState} ->
+ {ok, set_ctx(Q, Ctx#ctx{state = CtxState}, State)};
Err ->
Err
end.
@@ -437,43 +421,36 @@ is_recoverable(Q) ->
{Recovered :: [amqqueue:amqqueue()],
Failed :: [amqqueue:amqqueue()]}.
recover(VHost, Qs) ->
- ByType0 = lists:map(fun(T) -> {T, []} end, ?QUEUE_TYPES),
- ByType1 = maps:from_list(ByType0),
+ ByType0 = maps:from_keys(?QUEUE_TYPES, []),
ByType = lists:foldl(
fun (Q, Acc) ->
T = amqqueue:get_type(Q),
maps:update_with(T, fun (X) ->
[Q | X]
- end, Acc)
- end, ByType1, Qs),
- maps:fold(fun (Mod, Queues, {R0, F0}) ->
- {Taken, {R, F}} = timer:tc(Mod, recover, [VHost, Queues]),
- rabbit_log:info("Recovering ~b queues of type ~ts took ~bms",
- [length(Queues), Mod, Taken div 1000]),
- {R0 ++ R, F0 ++ F}
- end, {[], []}, ByType).
-
--spec handle_down(pid(), term(), state()) ->
- {ok, state(), actions()} | {eol, state(), queue_ref()} | {error, term()}.
-handle_down(Pid, Info, #?STATE{monitor_registry = Reg0} = State0) ->
- %% lookup queue ref in monitor registry
- case maps:take(Pid, Reg0) of
- {QRef, Reg} ->
- case handle_event(QRef, {down, Pid, Info}, State0) of
- {ok, State, Actions} ->
- {ok, State#?STATE{monitor_registry = Reg}, Actions};
- eol ->
- {eol, State0#?STATE{monitor_registry = Reg}, QRef};
- Err ->
- Err
- end;
- error ->
- {ok, State0, []}
+ end, [Q], Acc)
+ end, ByType0, Qs),
+ maps:fold(fun (Mod, Queues, {R0, F0}) ->
+ {Taken, {R, F}} = timer:tc(Mod, recover, [VHost, Queues]),
+ rabbit_log:info("Recovering ~b queues of type ~ts took ~bms",
+ [length(Queues), Mod, Taken div 1000]),
+ {R0 ++ R, F0 ++ F}
+ end, {[], []}, ByType).
+
+-spec handle_down(pid(), queue_name(), term(), state()) ->
+ {ok, state(), actions()} | {eol, state(), queue_name()} | {error, term()}.
+handle_down(Pid, QName, Info, State0) ->
+ case handle_event(QName, {down, Pid, Info}, State0) of
+ {ok, State, Actions} ->
+ {ok, State, Actions};
+ {eol, []} ->
+ {eol, State0, QName};
+ Err ->
+ Err
end.
%% messages sent from queues
--spec handle_event(queue_ref(), term(), state()) ->
- {ok, state(), actions()} | eol | {error, term()} |
+-spec handle_event(queue_name(), term(), state()) ->
+ {ok, state(), actions()} | {eol, actions()} | {error, term()} |
{protocol_error, Type :: atom(), Reason :: string(), Args :: term()}.
handle_event(QRef, Evt, Ctxs) ->
%% events can arrive after a queue state has been cleared up
@@ -481,17 +458,17 @@ handle_event(QRef, Evt, Ctxs) ->
case get_ctx(QRef, Ctxs, undefined) of
#ctx{module = Mod,
state = State0} = Ctx ->
- case Mod:handle_event(Evt, State0) of
+ case Mod:handle_event(QRef, Evt, State0) of
{ok, State, Actions} ->
- return_ok(set_ctx(QRef, Ctx#ctx{state = State}, Ctxs), Actions);
- Err ->
- Err
+ {ok, set_ctx(QRef, Ctx#ctx{state = State}, Ctxs), Actions};
+ Other ->
+ Other
end;
undefined ->
{ok, Ctxs, []}
end.
--spec module(queue_ref(), state()) ->
+-spec module(queue_name(), state()) ->
{ok, module()} | {error, not_found}.
module(QRef, State) ->
%% events can arrive after a queue state has been cleared up
@@ -515,22 +492,34 @@ deliver(Qs, Delivery, State) ->
end.
deliver0(Qs, Delivery, stateless) ->
- _ = lists:map(fun(Q) ->
- Mod = amqqueue:get_type(Q),
- _ = Mod:deliver([{Q, stateless}], Delivery)
- end, Qs),
+ ByType = lists:foldl(fun(Q, Acc) ->
+ Mod = amqqueue:get_type(Q),
+ maps:update_with(
+ Mod, fun(A) ->
+ [{Q, stateless} | A]
+ end, [{Q, stateless}], Acc)
+ end, #{}, Qs),
+ maps:foreach(fun(Mod, QSs) ->
+ _ = Mod:deliver(QSs, Delivery)
+ end, ByType),
{ok, stateless, []};
deliver0(Qs, Delivery, #?STATE{} = State0) ->
%% TODO: optimise single queue case?
%% sort by queue type - then dispatch each group
ByType = lists:foldl(
fun (Q, Acc) ->
- T = amqqueue:get_type(Q),
- Ctx = get_ctx(Q, State0),
+ Mod = amqqueue:get_type(Q),
+ QState = case Mod:is_stateful() of
+ true ->
+ #ctx{state = S} = get_ctx(Q, State0),
+ S;
+ false ->
+ stateless
+ end,
maps:update_with(
- T, fun (A) ->
- [{Q, Ctx#ctx.state} | A]
- end, [{Q, Ctx#ctx.state}], Acc)
+ Mod, fun (A) ->
+ [{Q, QState} | A]
+ end, [{Q, QState}], Acc)
end, #{}, Qs),
%%% dispatch each group to queue type interface?
{Xs, Actions} = maps:fold(fun(Mod, QSs, {X0, A0}) ->
@@ -542,15 +531,13 @@ deliver0(Qs, Delivery, #?STATE{} = State0) ->
Ctx = get_ctx_with(Q, Acc, S),
set_ctx(qref(Q), Ctx#ctx{state = S}, Acc)
end, State0, Xs),
- return_ok(State, Actions).
-
+ {ok, State, Actions}.
--spec settle(queue_ref(), settle_op(), rabbit_types:ctag(),
+-spec settle(queue_name(), settle_op(), rabbit_types:ctag(),
[non_neg_integer()], state()) ->
{ok, state(), actions()} |
{'protocol_error', Type :: atom(), Reason :: string(), Args :: term()}.
-settle(QRef, Op, CTag, MsgIds, Ctxs)
- when ?QREF(QRef) ->
+settle(#resource{kind = queue} = QRef, Op, CTag, MsgIds, Ctxs) ->
case get_ctx(QRef, Ctxs, undefined) of
undefined ->
%% if we receive a settlement and there is no queue state it means
@@ -558,7 +545,7 @@ settle(QRef, Op, CTag, MsgIds, Ctxs)
{ok, Ctxs, []};
#ctx{state = State0,
module = Mod} = Ctx ->
- case Mod:settle(Op, CTag, MsgIds, State0) of
+ case Mod:settle(QRef, Op, CTag, MsgIds, State0) of
{State, Actions} ->
{ok, set_ctx(QRef, Ctx#ctx{state = State}, Ctxs), Actions};
Err ->
@@ -566,13 +553,14 @@ settle(QRef, Op, CTag, MsgIds, Ctxs)
end
end.
--spec credit(amqqueue:amqqueue() | queue_ref(),
+-spec credit(amqqueue:amqqueue() | queue_name(),
rabbit_types:ctag(), non_neg_integer(),
boolean(), state()) -> {ok, state(), actions()}.
credit(Q, CTag, Credit, Drain, Ctxs) ->
#ctx{state = State0,
module = Mod} = Ctx = get_ctx(Q, Ctxs),
- {State, Actions} = Mod:credit(CTag, Credit, Drain, State0),
+ QName = amqqueue:get_name(Q),
+ {State, Actions} = Mod:credit(QName, CTag, Credit, Drain, State0),
{ok, set_ctx(Q, Ctx#ctx{state = State}, Ctxs), Actions}.
-spec dequeue(amqqueue:amqqueue(), boolean(),
@@ -583,7 +571,8 @@ credit(Q, CTag, Credit, Drain, Ctxs) ->
dequeue(Q, NoAck, LimiterPid, CTag, Ctxs) ->
#ctx{state = State0} = Ctx = get_ctx(Q, Ctxs),
Mod = amqqueue:get_type(Q),
- case Mod:dequeue(NoAck, LimiterPid, CTag, State0) of
+ QName = amqqueue:get_name(Q),
+ case Mod:dequeue(QName, NoAck, LimiterPid, CTag, State0) of
{ok, Num, Msg, State} ->
{ok, Num, Msg, set_ctx(Q, Ctx#ctx{state = State}, Ctxs)};
{empty, State} ->
@@ -609,24 +598,20 @@ get_ctx_with(Q, #?STATE{ctxs = Contexts}, InitState)
_ when InitState == undefined ->
%% not found and no initial state passed - initialize new state
Mod = amqqueue:get_type(Q),
- Name = amqqueue:get_name(Q),
case Mod:init(Q) of
{error, Reason} ->
exit({Reason, Ref});
{ok, QState} ->
#ctx{module = Mod,
- name = Name,
state = QState}
end;
_ ->
%% not found - initialize with supplied initial state
Mod = amqqueue:get_type(Q),
- Name = amqqueue:get_name(Q),
#ctx{module = Mod,
- name = Name,
state = InitState}
end;
-get_ctx_with(QRef, Contexts, undefined) when ?QREF(QRef) ->
+get_ctx_with(#resource{kind = queue} = QRef, Contexts, undefined) ->
case get_ctx(QRef, Contexts, undefined) of
undefined ->
exit({queue_context_not_found, QRef});
@@ -639,10 +624,6 @@ get_ctx(QRef, #?STATE{ctxs = Contexts}, Default) ->
%% if we use a QRef it should always be initialised
maps:get(Ref, Contexts, Default).
-set_ctx(Q, Ctx, #?STATE{ctxs = Contexts} = State)
- when ?is_amqqueue(Q) ->
- Ref = qref(Q),
- State#?STATE{ctxs = maps:put(Ref, Ctx, Contexts)};
set_ctx(QRef, Ctx, #?STATE{ctxs = Contexts} = State) ->
Ref = qref(QRef),
State#?STATE{ctxs = maps:put(Ref, Ctx, Contexts)}.
@@ -651,27 +632,3 @@ qref(#resource{kind = queue} = QName) ->
QName;
qref(Q) when ?is_amqqueue(Q) ->
amqqueue:get_name(Q).
-
-return_ok(State0, []) ->
- {ok, State0, []};
-return_ok(State0, Actions0) ->
- {State, Actions} =
- lists:foldl(
- fun({monitor, Pid, QRef},
- {#?STATE{monitor_registry = M0} = S0, A0}) ->
- case M0 of
- #{Pid := QRef} ->
- %% already monitored by the qref
- {S0, A0};
- #{Pid := _} ->
- %% TODO: allow multiple Qrefs to monitor the same pid
- exit(return_ok_duplicate_monitored_pid);
- _ ->
- _ = erlang:monitor(process, Pid),
- M = M0#{Pid => QRef},
- {S0#?STATE{monitor_registry = M}, A0}
- end;
- (Act, {S, A0}) ->
- {S, [Act | A0]}
- end, {State0, []}, Actions0),
- {ok, State, lists:reverse(Actions)}.
diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl
index 88709c465156..7ede155cf5d0 100644
--- a/deps/rabbit/src/rabbit_quorum_queue.erl
+++ b/deps/rabbit/src/rabbit_quorum_queue.erl
@@ -12,7 +12,7 @@
-export([init/1,
close/1,
update/2,
- handle_event/2]).
+ handle_event/3]).
-export([is_recoverable/1,
recover/2,
stop/1,
@@ -22,12 +22,11 @@
delete/4,
delete_immediately/2]).
-export([state_info/1, info/2, stat/1, infos/1]).
--export([settle/4, dequeue/4, consume/3, cancel/5]).
--export([credit/4]).
+-export([settle/5, dequeue/5, consume/3, cancel/5]).
+-export([credit/5]).
-export([purge/1]).
--export([stateless_deliver/2, deliver/3, deliver/2]).
+-export([stateless_deliver/2, deliver/2]).
-export([dead_letter_publish/5]).
--export([queue_name/1]).
-export([cluster_state/1, status/2]).
-export([update_consumer_handler/8, update_consumer/9]).
-export([cancel_consumer_handler/2, cancel_consumer/3]).
@@ -64,7 +63,8 @@
-export([is_enabled/0,
is_compatible/3,
- declare/2]).
+ declare/2,
+ is_stateful/0]).
-import(rabbit_queue_type_util, [args_policy_lookup/3,
qname_to_internal_name/1]).
@@ -132,13 +132,10 @@ init(Q) when ?is_amqqueue(Q) ->
%% know what to do if the queue has `disappeared`. Let it crash.
{Name, _LeaderNode} = Leader = amqqueue:get_pid(Q),
Nodes = get_nodes(Q),
- QName = amqqueue:get_name(Q),
%% Ensure the leader is listed first
Servers0 = [{Name, N} || N <- Nodes],
Servers = [Leader | lists:delete(Leader, Servers0)],
- {ok, rabbit_fifo_client:init(QName, Servers, SoftLimit,
- fun() -> credit_flow:block(Name) end,
- fun() -> credit_flow:unblock(Name), ok end)}.
+ {ok, rabbit_fifo_client:init(Servers, SoftLimit)}.
-spec close(rabbit_fifo_client:state()) -> ok.
close(_State) ->
@@ -150,13 +147,14 @@ update(Q, State) when ?amqqueue_is_quorum(Q) ->
%% QQ state maintains it's own updates
State.
--spec handle_event({amqqueue:ra_server_id(), any()},
+-spec handle_event(rabbit_amqquue:name(),
+ {amqqueue:ra_server_id(), any()},
rabbit_fifo_client:state()) ->
{ok, rabbit_fifo_client:state(), rabbit_queue_type:actions()} |
- eol |
+ {eol, rabbit_queue_type:actions()} |
{protocol_error, Type :: atom(), Reason :: string(), Args :: term()}.
-handle_event({From, Evt}, QState) ->
- rabbit_fifo_client:handle_ra_event(From, Evt, QState).
+handle_event(QName, {From, Evt}, QState) ->
+ rabbit_fifo_client:handle_ra_event(QName, From, Evt, QState).
-spec declare(amqqueue:amqqueue(), node()) ->
{new | existing, amqqueue:amqqueue()} |
@@ -721,22 +719,22 @@ delete_immediately(Resource, {_Name, _} = QPid) ->
rabbit_core_metrics:queue_deleted(Resource),
ok.
-settle(complete, CTag, MsgIds, QState) ->
+settle(_QName, complete, CTag, MsgIds, QState) ->
rabbit_fifo_client:settle(quorum_ctag(CTag), MsgIds, QState);
-settle(requeue, CTag, MsgIds, QState) ->
+settle(_QName, requeue, CTag, MsgIds, QState) ->
rabbit_fifo_client:return(quorum_ctag(CTag), MsgIds, QState);
-settle(discard, CTag, MsgIds, QState) ->
+settle(_QName, discard, CTag, MsgIds, QState) ->
rabbit_fifo_client:discard(quorum_ctag(CTag), MsgIds, QState).
-credit(CTag, Credit, Drain, QState) ->
+credit(_QName, CTag, Credit, Drain, QState) ->
rabbit_fifo_client:credit(quorum_ctag(CTag), Credit, Drain, QState).
--spec dequeue(NoAck :: boolean(), pid(),
+-spec dequeue(rabbit_amqqueue:name(), NoAck :: boolean(), pid(),
rabbit_types:ctag(), rabbit_fifo_client:state()) ->
{empty, rabbit_fifo_client:state()} |
{ok, QLen :: non_neg_integer(), qmsg(), rabbit_fifo_client:state()} |
{error, term()}.
-dequeue(NoAck, _LimiterPid, CTag0, QState0) ->
+dequeue(QName, NoAck, _LimiterPid, CTag0, QState0) ->
CTag = quorum_ctag(CTag0),
Settlement = case NoAck of
true ->
@@ -744,7 +742,7 @@ dequeue(NoAck, _LimiterPid, CTag0, QState0) ->
false ->
unsettled
end,
- rabbit_fifo_client:dequeue(CTag, Settlement, QState0).
+ rabbit_fifo_client:dequeue(QName, CTag, Settlement, QState0).
-spec consume(amqqueue:amqqueue(),
rabbit_queue_type:consume_spec(),
@@ -816,7 +814,7 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) ->
emit_consumer_created(ChPid, ConsumerTag, ExclusiveConsume,
AckRequired, QName, Prefetch,
Args, none, ActingUser),
- {ok, QState, []};
+ {ok, QState};
{error, Error} ->
Error;
{timeout, _} ->
@@ -831,7 +829,7 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) ->
emit_consumer_created(ChPid, ConsumerTag, ExclusiveConsume,
AckRequired, QName, Prefetch,
Args, none, ActingUser),
- {ok, QState, []}
+ {ok, QState}
end.
cancel(_Q, ConsumerTag, OkMsg, _ActingUser, State) ->
@@ -863,19 +861,20 @@ stateless_deliver(ServerId, Delivery) ->
ok = rabbit_fifo_client:untracked_enqueue([ServerId],
Delivery#delivery.message).
--spec deliver(Confirm :: boolean(), rabbit_types:delivery(),
- rabbit_fifo_client:state()) ->
- {ok | slow, rabbit_fifo_client:state()} |
+-spec deliver(rabbit_amqqueue:name(), Confirm :: boolean(),
+ rabbit_types:delivery(), rabbit_fifo_client:state()) ->
+ {ok, rabbit_fifo_client:state(), rabbit_queue_type:actions()} |
{reject_publish, rabbit_fifo_client:state()}.
-deliver(false, Delivery, QState0) ->
- case rabbit_fifo_client:enqueue(Delivery#delivery.message, QState0) of
- {ok, _} = Res -> Res;
- {slow, _} = Res -> Res;
+deliver(QName, false, Delivery, QState0) ->
+ case rabbit_fifo_client:enqueue(QName, Delivery#delivery.message, QState0) of
+ {ok, _State, _Actions} = Res ->
+ Res;
{reject_publish, State} ->
- {ok, State}
+ {ok, State, []}
end;
-deliver(true, Delivery, QState0) ->
- rabbit_fifo_client:enqueue(Delivery#delivery.msg_seq_no,
+deliver(QName, true, Delivery, QState0) ->
+ rabbit_fifo_client:enqueue(QName,
+ Delivery#delivery.msg_seq_no,
Delivery#delivery.message, QState0).
deliver(QSs, #delivery{message = #basic_message{content = Content0} = Msg,
@@ -890,13 +889,13 @@ deliver(QSs, #delivery{message = #basic_message{content = Content0} = Msg,
[QRef], Delivery#delivery.message),
{Qs, Actions};
({Q, S0}, {Qs, Actions}) ->
- case deliver(Confirm, Delivery, S0) of
+ QName = amqqueue:get_name(Q),
+ case deliver(QName, Confirm, Delivery, S0) of
{reject_publish, S} ->
Seq = Delivery#delivery.msg_seq_no,
- QName = rabbit_fifo_client:cluster_name(S),
{[{Q, S} | Qs], [{rejected, QName, [Seq]} | Actions]};
- {_, S} ->
- {[{Q, S} | Qs], Actions}
+ {ok, S, As} ->
+ {[{Q, S} | Qs], As ++ Actions}
end
end, {[], []}, QSs).
@@ -1325,18 +1324,8 @@ dlh(undefined, _, Strategy, _, QName) ->
"because dead-letter-exchange is not configured.",
[rabbit_misc:rs(QName), Strategy]),
undefined;
-dlh(Exchange, RoutingKey, <<"at-least-once">>, reject_publish, QName) ->
- %% Feature flag stream_queue includes the rabbit_queue_type refactor
- %% which is required by rabbit_fifo_dlx_worker.
- case rabbit_queue_type:is_supported() of
- true ->
- at_least_once;
- false ->
- rabbit_log:warning("Falling back to dead-letter-strategy at-most-once for ~ts "
- "because feature flag stream_queue is disabled.",
- [rabbit_misc:rs(QName)]),
- dlh_at_most_once(Exchange, RoutingKey, QName)
- end;
+dlh(_, _, <<"at-least-once">>, reject_publish, _) ->
+ at_least_once;
dlh(Exchange, RoutingKey, <<"at-least-once">>, drop_head, QName) ->
rabbit_log:warning("Falling back to dead-letter-strategy at-most-once for ~ts "
"because configured dead-letter-strategy at-least-once is incompatible with "
@@ -1592,9 +1581,6 @@ quorum_ctag(Other) ->
maybe_send_reply(_ChPid, undefined) -> ok;
maybe_send_reply(ChPid, Msg) -> ok = rabbit_channel:send_command(ChPid, Msg).
-queue_name(RaFifoState) ->
- rabbit_fifo_client:cluster_name(RaFifoState).
-
get_default_quorum_initial_group_size(Arguments) ->
case rabbit_misc:table_lookup(Arguments, <<"x-quorum-initial-group-size">>) of
undefined ->
@@ -1712,8 +1698,8 @@ ets_lookup_element(Tbl, Key, Pos, Default) ->
Default
end.
-erpc_call(Node, M, F, A, Timeout)
- when is_integer(Timeout) andalso Node == node() ->
+erpc_call(Node, M, F, A, _Timeout)
+ when Node =:= node() ->
%% Only timeout 'infinity' optimises the local call in OTP 23-25 avoiding a new process being spawned:
%% https://github.com/erlang/otp/blob/47f121af8ee55a0dbe2a8c9ab85031ba052bad6b/lib/kernel/src/erpc.erl#L121
try erpc:call(Node, M, F, A, infinity) of
@@ -1737,4 +1723,4 @@ erpc_call(Node, M, F, A, Timeout) ->
{error, noconnection}
end.
-
+is_stateful() -> true.
diff --git a/deps/rabbit/src/rabbit_reader.erl b/deps/rabbit/src/rabbit_reader.erl
index 6503748cb83a..c605f9039161 100644
--- a/deps/rabbit/src/rabbit_reader.erl
+++ b/deps/rabbit/src/rabbit_reader.erl
@@ -143,12 +143,6 @@
%%--------------------------------------------------------------------------
--type resource_alert() :: {WasAlarmSetForNode :: boolean(),
- IsThereAnyAlarmsWithSameSourceInTheCluster :: boolean(),
- NodeForWhichAlarmWasSetOrCleared :: node()}.
-
-%%--------------------------------------------------------------------------
-
-spec start_link(pid(), any()) -> rabbit_types:ok(pid()).
start_link(HelperSup, Ref) ->
@@ -210,7 +204,9 @@ info(Pid, Items) ->
force_event_refresh(Pid, Ref) ->
gen_server:cast(Pid, {force_event_refresh, Ref}).
--spec conserve_resources(pid(), atom(), resource_alert()) -> 'ok'.
+-spec conserve_resources(pid(),
+ rabbit_alarm:resource_alarm_source(),
+ rabbit_alarm:resource_alert()) -> 'ok'.
conserve_resources(Pid, Source, {_, Conserve, _}) ->
Pid ! {conserve_resources, Source, Conserve},
@@ -1505,13 +1501,18 @@ i(SockStat, S) when SockStat =:= recv_oct;
fun ([{_, I}]) -> I end, S);
i(ssl, #v1{sock = Sock, proxy_socket = ProxySock}) ->
rabbit_net:proxy_ssl_info(Sock, ProxySock) /= nossl;
-i(ssl_protocol, S) -> ssl_info(fun ({P, _}) -> P end, S);
-i(ssl_key_exchange, S) -> ssl_info(fun ({_, {K, _, _}}) -> K end, S);
-i(ssl_cipher, S) -> ssl_info(fun ({_, {_, C, _}}) -> C end, S);
-i(ssl_hash, S) -> ssl_info(fun ({_, {_, _, H}}) -> H end, S);
-i(peer_cert_issuer, S) -> cert_info(fun rabbit_ssl:peer_cert_issuer/1, S);
-i(peer_cert_subject, S) -> cert_info(fun rabbit_ssl:peer_cert_subject/1, S);
-i(peer_cert_validity, S) -> cert_info(fun rabbit_ssl:peer_cert_validity/1, S);
+i(SSL, #v1{sock = Sock, proxy_socket = ProxySock})
+ when SSL =:= ssl;
+ SSL =:= ssl_protocol;
+ SSL =:= ssl_key_exchange;
+ SSL =:= ssl_cipher;
+ SSL =:= ssl_hash ->
+ rabbit_ssl:info(SSL, {Sock, ProxySock});
+i(Cert, #v1{sock = Sock})
+ when Cert =:= peer_cert_issuer;
+ Cert =:= peer_cert_subject;
+ Cert =:= peer_cert_validity ->
+ rabbit_ssl:cert_info(Cert, Sock);
i(channels, #v1{channel_count = ChannelCount}) -> ChannelCount;
i(state, #v1{connection_state = ConnectionState,
throttle = #throttle{blocked_by = Reasons,
@@ -1574,25 +1575,6 @@ socket_info(Get, Select, #v1{sock = Sock}) ->
{error, _} -> 0
end.
-ssl_info(F, #v1{sock = Sock, proxy_socket = ProxySock}) ->
- case rabbit_net:proxy_ssl_info(Sock, ProxySock) of
- nossl -> '';
- {error, _} -> '';
- {ok, Items} ->
- P = proplists:get_value(protocol, Items),
- #{cipher := C,
- key_exchange := K,
- mac := H} = proplists:get_value(selected_cipher_suite, Items),
- F({P, {K, C, H}})
- end.
-
-cert_info(F, #v1{sock = Sock}) ->
- case rabbit_net:peercert(Sock) of
- nossl -> '';
- {error, _} -> '';
- {ok, Cert} -> list_to_binary(F(Cert))
- end.
-
maybe_emit_stats(State) ->
rabbit_event:if_enabled(State, #v1.stats_timer,
fun() -> emit_stats(State) end).
diff --git a/deps/rabbit/src/rabbit_ssl.erl b/deps/rabbit/src/rabbit_ssl.erl
index 7c6873e6d5ce..abd94f4ba733 100644
--- a/deps/rabbit/src/rabbit_ssl.erl
+++ b/deps/rabbit/src/rabbit_ssl.erl
@@ -14,6 +14,7 @@
-export([cipher_suites_erlang/2, cipher_suites_erlang/1,
cipher_suites_openssl/2, cipher_suites_openssl/1,
cipher_suites/1]).
+-export([info/2, cert_info/2]).
%%--------------------------------------------------------------------------
@@ -181,3 +182,34 @@ otp_san_type(uri) -> uniformResourceIdentifier;
otp_san_type(other_name) -> otherName;
otp_san_type(Other) -> Other.
+info(ssl_protocol, Socks) -> info0(fun ({P, _}) -> P end, Socks);
+info(ssl_key_exchange, Socks) -> info0(fun ({_, {K, _, _}}) -> K end, Socks);
+info(ssl_cipher, Socks) -> info0(fun ({_, {_, C, _}}) -> C end, Socks);
+info(ssl_hash, Socks) -> info0(fun ({_, {_, _, H}}) -> H end, Socks);
+info(ssl, {Sock, ProxySock}) -> rabbit_net:proxy_ssl_info(Sock, ProxySock) /= nossl.
+
+info0(F, {Sock, ProxySock}) ->
+ case rabbit_net:proxy_ssl_info(Sock, ProxySock) of
+ nossl -> '';
+ {error, _} -> '';
+ {ok, Items} ->
+ P = proplists:get_value(protocol, Items),
+ #{cipher := C,
+ key_exchange := K,
+ mac := H} = proplists:get_value(selected_cipher_suite, Items),
+ F({P, {K, C, H}})
+ end.
+
+cert_info(peer_cert_issuer, Sock) ->
+ cert_info0(fun peer_cert_issuer/1, Sock);
+cert_info(peer_cert_subject, Sock) ->
+ cert_info0(fun peer_cert_subject/1, Sock);
+cert_info(peer_cert_validity, Sock) ->
+ cert_info0(fun peer_cert_validity/1, Sock).
+
+cert_info0(F, Sock) ->
+ case rabbit_net:peercert(Sock) of
+ nossl -> '';
+ {error, _} -> '';
+ {ok, Cert} -> list_to_binary(F(Cert))
+ end.
diff --git a/deps/rabbit/src/rabbit_stream_queue.erl b/deps/rabbit/src/rabbit_stream_queue.erl
index 3b9698def214..2672c0e3d4f5 100644
--- a/deps/rabbit/src/rabbit_stream_queue.erl
+++ b/deps/rabbit/src/rabbit_stream_queue.erl
@@ -19,11 +19,11 @@
is_recoverable/1,
consume/3,
cancel/5,
- handle_event/2,
+ handle_event/3,
deliver/2,
- settle/4,
- credit/4,
- dequeue/4,
+ settle/5,
+ credit/5,
+ dequeue/5,
info/2,
queue_length/1,
get_replicas/1,
@@ -34,7 +34,8 @@
state_info/1,
stat/1,
capabilities/0,
- notify_decorators/1]).
+ notify_decorators/1,
+ is_stateful/0]).
-export([list_with_minimum_quorum/0]).
@@ -67,8 +68,7 @@
-type msg_id() :: non_neg_integer().
-type msg() :: term(). %% TODO: refine
--record(stream, {name :: rabbit_types:r('queue'),
- credit :: integer(),
+-record(stream, {credit :: integer(),
max :: non_neg_integer(),
start_offset = 0 :: non_neg_integer(),
listening_offset = 0 :: non_neg_integer(),
@@ -246,7 +246,7 @@ consume(Q, Spec, QState0) when ?amqqueue_is_stream(Q) ->
%% really it should be sent by the stream queue process like classic queues
%% do
maybe_send_reply(ChPid, OkMsg),
- begin_stream(QState0, Q, ConsumerTag, OffsetSpec, ConsumerPrefetchCount)
+ begin_stream(QState0, ConsumerTag, OffsetSpec, ConsumerPrefetchCount)
end;
Err ->
Err
@@ -297,7 +297,7 @@ get_local_pid(#stream_client{stream_id = StreamId,
end.
begin_stream(#stream_client{name = QName, readers = Readers0} = State0,
- Q, Tag, Offset, Max) ->
+ Tag, Offset, Max) ->
{LocalPid, State} = get_local_pid(State0),
case LocalPid of
undefined ->
@@ -315,15 +315,13 @@ begin_stream(#stream_client{name = QName, readers = Readers0} = State0,
{timestamp, _} -> NextOffset;
_ -> Offset
end,
- Str0 = #stream{name = amqqueue:get_name(Q),
- credit = Max,
+ Str0 = #stream{credit = Max,
start_offset = StartOffset,
listening_offset = NextOffset,
log = Seg0,
max = Max},
- Actions = [],
{ok, State#stream_client{local_pid = LocalPid,
- readers = Readers0#{Tag => Str0}}, Actions}
+ readers = Readers0#{Tag => Str0}}}
end.
cancel(_Q, ConsumerTag, OkMsg, ActingUser, #stream_client{readers = Readers0,
@@ -343,13 +341,13 @@ cancel(_Q, ConsumerTag, OkMsg, ActingUser, #stream_client{readers = Readers0,
{ok, State}
end.
-credit(CTag, Credit, Drain, #stream_client{readers = Readers0,
+credit(QName, CTag, Credit, Drain, #stream_client{readers = Readers0,
name = Name,
local_pid = LocalPid} = State) ->
{Readers1, Msgs} = case Readers0 of
#{CTag := #stream{credit = Credit0} = Str0} ->
Str1 = Str0#stream{credit = Credit0 + Credit},
- {Str, Msgs0} = stream_entries(Name, LocalPid, Str1),
+ {Str, Msgs0} = stream_entries(QName, Name, LocalPid, Str1),
{Readers0#{CTag => Str}, Msgs0};
_ ->
{Readers0, []}
@@ -378,18 +376,18 @@ deliver(QSs, #delivery{confirm = Confirm} = Delivery) ->
%% [QRef], Delivery#delivery.message),
{Qs, Actions};
({Q, S0}, {Qs, Actions}) ->
- S = deliver(Confirm, Delivery, S0),
- {[{Q, S} | Qs], Actions}
+ {S, As} = deliver(Confirm, Delivery, S0),
+ {[{Q, S} | Qs], As ++ Actions}
end, {[], []}, QSs).
deliver(_Confirm, #delivery{message = Msg, msg_seq_no = MsgId},
- #stream_client{name = Name,
- leader = LeaderPid,
- writer_id = WriterId,
- next_seq = Seq,
- correlation = Correlation0,
- soft_limit = SftLmt,
- slow = Slow0} = State) ->
+ #stream_client{name = Name,
+ leader = LeaderPid,
+ writer_id = WriterId,
+ next_seq = Seq,
+ correlation = Correlation0,
+ soft_limit = SftLmt,
+ slow = Slow0} = State) ->
ok = osiris:write(LeaderPid, WriterId, Seq, msg_to_iodata(Msg)),
Correlation = case MsgId of
undefined ->
@@ -397,23 +395,22 @@ deliver(_Confirm, #delivery{message = Msg, msg_seq_no = MsgId},
_ when is_number(MsgId) ->
Correlation0#{Seq => {MsgId, Msg}}
end,
- Slow = case maps:size(Correlation) >= SftLmt of
- true when not Slow0 ->
- credit_flow:block(Name),
- true;
- Bool ->
- Bool
- end,
- State#stream_client{next_seq = Seq + 1,
- correlation = Correlation,
- slow = Slow}.
-
--spec dequeue(_, _, _, client()) -> no_return().
-dequeue(_, _, _, #stream_client{name = Name}) ->
+ {Slow, Actions} = case maps:size(Correlation) >= SftLmt of
+ true when not Slow0 ->
+ {true, [{block, Name}]};
+ Bool ->
+ {Bool, []}
+ end,
+ {State#stream_client{next_seq = Seq + 1,
+ correlation = Correlation,
+ slow = Slow}, Actions}.
+
+-spec dequeue(_, _, _, _, client()) -> no_return().
+dequeue(_, _, _, _, #stream_client{name = Name}) ->
{protocol_error, not_implemented, "basic.get not supported by stream queues ~ts",
[rabbit_misc:rs(Name)]}.
-handle_event({osiris_written, From, _WriterId, Corrs},
+handle_event(_QName, {osiris_written, From, _WriterId, Corrs},
State = #stream_client{correlation = Correlation0,
soft_limit = SftLmt,
slow = Slow0,
@@ -424,16 +421,15 @@ handle_event({osiris_written, From, _WriterId, Corrs},
end, [], maps:with(Corrs, Correlation0))),
Correlation = maps:without(Corrs, Correlation0),
- Slow = case maps:size(Correlation) < SftLmt of
- true when Slow0 ->
- credit_flow:unblock(Name),
- false;
- _ ->
- Slow0
- end,
+ {Slow, Actions} = case maps:size(Correlation) < SftLmt of
+ true when Slow0 ->
+ {false, [{unblock, Name}]};
+ _ ->
+ {Slow0, []}
+ end,
{ok, State#stream_client{correlation = Correlation,
- slow = Slow}, [{settled, From, MsgIds}]};
-handle_event({osiris_offset, _From, _Offs},
+ slow = Slow}, [{settled, From, MsgIds} | Actions]};
+handle_event(QName, {osiris_offset, _From, _Offs},
State = #stream_client{local_pid = LocalPid,
readers = Readers0,
name = Name}) ->
@@ -441,19 +437,19 @@ handle_event({osiris_offset, _From, _Offs},
%% current committed
{Readers, TagMsgs} = maps:fold(
fun (Tag, Str0, {Acc, TM}) ->
- {Str, Msgs} = stream_entries(Name, LocalPid, Str0),
+ {Str, Msgs} = stream_entries(QName, Name, LocalPid, Str0),
{Acc#{Tag => Str}, [{Tag, LocalPid, Msgs} | TM]}
end, {#{}, []}, Readers0),
Ack = true,
Deliveries = [{deliver, Tag, Ack, OffsetMsg}
|| {Tag, _LeaderPid, OffsetMsg} <- TagMsgs],
{ok, State#stream_client{readers = Readers}, Deliveries};
-handle_event({stream_leader_change, Pid}, State) ->
+handle_event(_QName, {stream_leader_change, Pid}, State) ->
{ok, update_leader_pid(Pid, State), []};
-handle_event({stream_local_member_change, Pid}, #stream_client{local_pid = P} = State)
+handle_event(_QName, {stream_local_member_change, Pid}, #stream_client{local_pid = P} = State)
when P == Pid ->
{ok, State, []};
-handle_event({stream_local_member_change, Pid}, State = #stream_client{name = QName,
+handle_event(_QName, {stream_local_member_change, Pid}, State = #stream_client{name = QName,
readers = Readers0}) ->
rabbit_log:debug("Local member change event for ~tp", [QName]),
Readers1 = maps:fold(fun(T, #stream{log = Log0} = S0, Acc) ->
@@ -471,8 +467,8 @@ handle_event({stream_local_member_change, Pid}, State = #stream_client{name = QN
end, #{}, Readers0),
{ok, State#stream_client{local_pid = Pid, readers = Readers1}, []};
-handle_event(eol, _State) ->
- eol.
+handle_event(_QName, eol, #stream_client{name = Name}) ->
+ {eol, [{unblock, Name}]}.
is_recoverable(Q) ->
Node = node(),
@@ -487,20 +483,20 @@ recover(_VHost, Queues) ->
{[Q | R0], F0}
end, {[], []}, Queues).
-settle(complete, CTag, MsgIds, #stream_client{readers = Readers0,
+settle(QName, complete, CTag, MsgIds, #stream_client{readers = Readers0,
local_pid = LocalPid,
name = Name} = State) ->
Credit = length(MsgIds),
{Readers, Msgs} = case Readers0 of
#{CTag := #stream{credit = Credit0} = Str0} ->
Str1 = Str0#stream{credit = Credit0 + Credit},
- {Str, Msgs0} = stream_entries(Name, LocalPid, Str1),
+ {Str, Msgs0} = stream_entries(QName, Name, LocalPid, Str1),
{Readers0#{CTag => Str}, Msgs0};
_ ->
{Readers0, []}
end,
{State#stream_client{readers = Readers}, [{deliver, CTag, true, Msgs}]};
-settle(_, _, _, #stream_client{name = Name}) ->
+settle(_, _, _, _, #stream_client{name = Name}) ->
{protocol_error, not_implemented,
"basic.nack and basic.reject not supported by stream queues ~ts",
[rabbit_misc:rs(Name)]}.
@@ -774,10 +770,9 @@ init(Q) when ?is_amqqueue(Q) ->
end.
close(#stream_client{readers = Readers}) ->
- _ = maps:map(fun (_, #stream{log = Log}) ->
+ maps:foreach(fun (_, #stream{log = Log}) ->
osiris_log:close(Log)
- end, Readers),
- ok.
+ end, Readers).
update(Q, State)
when ?is_amqqueue(Q) ->
@@ -943,12 +938,11 @@ check_queue_exists_in_local_node(Q) ->
maybe_send_reply(_ChPid, undefined) -> ok;
maybe_send_reply(ChPid, Msg) -> ok = rabbit_channel:send_command(ChPid, Msg).
-stream_entries(Name, LocalPid, Str) ->
- stream_entries(Name, LocalPid, Str, []).
+stream_entries(QName, Name, LocalPid, Str) ->
+ stream_entries(QName, Name, LocalPid, Str, []).
-stream_entries(Name, LocalPid,
- #stream{name = QName,
- credit = Credit,
+stream_entries(QName, Name, LocalPid,
+ #stream{credit = Credit,
start_offset = StartOffs,
listening_offset = LOffs,
log = Seg0} = Str0, MsgIn)
@@ -987,10 +981,10 @@ stream_entries(Name, LocalPid,
false ->
%% if there are fewer Msgs than Entries0 it means there were non-events
%% in the log and we should recurse and try again
- stream_entries(Name, LocalPid, Str, MsgIn ++ Msgs)
+ stream_entries(QName, Name, LocalPid, Str, MsgIn ++ Msgs)
end
end;
-stream_entries(_Name, _LocalPid, Str, Msgs) ->
+stream_entries(_QName, _Name, _LocalPid, Str, Msgs) ->
{Str, Msgs}.
binary_to_msg(#resource{virtual_host = VHost,
@@ -1096,3 +1090,5 @@ list_with_minimum_quorum() ->
RunningMembers = maps:filter(fun(_, {State, _}) -> State =/= undefined end, Members),
map_size(RunningMembers) =< map_size(Members) div 2 + 1
end, rabbit_amqqueue:list_local_stream_queues()).
+
+is_stateful() -> true.
diff --git a/deps/rabbit/src/rabbit_trace.erl b/deps/rabbit/src/rabbit_trace.erl
index 02f78a26fe12..6c556301373b 100644
--- a/deps/rabbit/src/rabbit_trace.erl
+++ b/deps/rabbit/src/rabbit_trace.erl
@@ -7,23 +7,28 @@
-module(rabbit_trace).
--export([init/1, enabled/1, tap_in/6, tap_out/5, start/1, stop/1]).
+-export([init/1, enabled/1, tap_in/5, tap_in/6,
+ tap_out/4, tap_out/5, start/1, stop/1]).
-include_lib("rabbit_common/include/rabbit.hrl").
-include_lib("rabbit_common/include/rabbit_framing.hrl").
-define(TRACE_VHOSTS, trace_vhosts).
-define(XNAME, <<"amq.rabbitmq.trace">>).
+%% "The channel number is 0 for all frames which are global to the connection" [AMQP 0-9-1 spec]
+-define(CONNECTION_GLOBAL_CHANNEL_NUM, 0).
%%----------------------------------------------------------------------------
--type state() :: rabbit_types:exchange() | 'none'.
+-opaque state() :: rabbit_types:exchange() | 'none'.
+-export_type([state/0]).
%%----------------------------------------------------------------------------
-spec init(rabbit_types:vhost()) -> state().
-init(VHost) ->
+init(VHost)
+ when is_binary(VHost) ->
case enabled(VHost) of
false -> none;
true -> {ok, X} = rabbit_exchange:lookup(
@@ -31,11 +36,21 @@ init(VHost) ->
X
end.
--spec enabled(rabbit_types:vhost()) -> boolean().
+-spec enabled(rabbit_types:vhost() | state()) -> boolean().
-enabled(VHost) ->
+enabled(VHost)
+ when is_binary(VHost) ->
{ok, VHosts} = application:get_env(rabbit, ?TRACE_VHOSTS),
- lists:member(VHost, VHosts).
+ lists:member(VHost, VHosts);
+enabled(none) ->
+ false;
+enabled(#exchange{}) ->
+ true.
+
+-spec tap_in(rabbit_types:basic_message(), [rabbit_amqqueue:name()],
+ binary(), rabbit_types:username(), state()) -> 'ok'.
+tap_in(Msg, QNames, ConnName, Username, State) ->
+ tap_in(Msg, QNames, ConnName, ?CONNECTION_GLOBAL_CHANNEL_NUM, Username, State).
-spec tap_in(rabbit_types:basic_message(), [rabbit_amqqueue:name()],
binary(), rabbit_channel:channel_number(),
@@ -53,6 +68,11 @@ tap_in(Msg = #basic_message{exchange_name = #resource{name = XName,
{<<"routed_queues">>, array,
[{longstr, QName#resource.name} || QName <- QNames]}]).
+-spec tap_out(rabbit_amqqueue:qmsg(), binary(),
+ rabbit_types:username(), state()) -> 'ok'.
+tap_out(Msg, ConnName, Username, State) ->
+ tap_out(Msg, ConnName, ?CONNECTION_GLOBAL_CHANNEL_NUM, Username, State).
+
-spec tap_out(rabbit_amqqueue:qmsg(), binary(),
rabbit_channel:channel_number(),
rabbit_types:username(), state()) -> 'ok'.
@@ -73,7 +93,8 @@ tap_out({#resource{name = QName, virtual_host = VHost},
-spec start(rabbit_types:vhost()) -> 'ok'.
-start(VHost) ->
+start(VHost)
+ when is_binary(VHost) ->
case lists:member(VHost, vhosts_with_tracing_enabled()) of
true ->
rabbit_log:info("Tracing is already enabled for vhost '~ts'", [VHost]),
@@ -87,7 +108,8 @@ start(VHost) ->
-spec stop(rabbit_types:vhost()) -> 'ok'.
-stop(VHost) ->
+stop(VHost)
+ when is_binary(VHost) ->
case lists:member(VHost, vhosts_with_tracing_enabled()) of
true ->
rabbit_log:info("Disabling tracing for vhost '~ts'", [VHost]),
@@ -101,11 +123,14 @@ update_config(Fun) ->
VHosts0 = vhosts_with_tracing_enabled(),
VHosts = Fun(VHosts0),
application:set_env(rabbit, ?TRACE_VHOSTS, VHosts),
- rabbit_log:debug("Will now refresh channel state after virtual host tracing changes"),
+ NonAmqpPids = rabbit_networking:local_non_amqp_connections(),
+ rabbit_log:debug("Will now refresh state of channels and of ~b non AMQP 0.9.1 "
+ "connections after virtual host tracing changes",
+ [length(NonAmqpPids)]),
+ lists:foreach(fun(Pid) -> gen_server:cast(Pid, refresh_config) end, NonAmqpPids),
{Time, _} = timer:tc(fun rabbit_channel:refresh_config_local/0),
- rabbit_log:debug("Refreshed channel state in ~fs", [Time/1000000]),
-
+ rabbit_log:debug("Refreshed channel state in ~fs", [Time/1_000_000]),
ok.
vhosts_with_tracing_enabled() ->
diff --git a/deps/rabbit/src/tcp_listener_sup.erl b/deps/rabbit/src/tcp_listener_sup.erl
index 57955dcb75fc..7ed78c923236 100644
--- a/deps/rabbit/src/tcp_listener_sup.erl
+++ b/deps/rabbit/src/tcp_listener_sup.erl
@@ -16,24 +16,24 @@
-behaviour(supervisor).
--export([start_link/11]).
+-export([start_link/12]).
-export([init/1]).
-type mfargs() :: {atom(), atom(), [any()]}.
-spec start_link
(inet:ip_address(), inet:port_number(), module(), [gen_tcp:listen_option()],
- module(), any(), mfargs(), mfargs(), integer(), integer(), string()) ->
+ module(), any(), mfargs(), mfargs(), integer(), integer(), supervisor:worker(), string()) ->
rabbit_types:ok_pid_or_error().
start_link(IPAddress, Port, Transport, SocketOpts, ProtoSup, ProtoOpts, OnStartup, OnShutdown,
- ConcurrentAcceptorCount, ConcurrentConnsSups, Label) ->
+ ConcurrentAcceptorCount, ConcurrentConnsSups, ConnectionType, Label) ->
supervisor:start_link(
?MODULE, {IPAddress, Port, Transport, SocketOpts, ProtoSup, ProtoOpts, OnStartup, OnShutdown,
- ConcurrentAcceptorCount, ConcurrentConnsSups, Label}).
+ ConcurrentAcceptorCount, ConcurrentConnsSups, ConnectionType, Label}).
init({IPAddress, Port, Transport, SocketOpts, ProtoSup, ProtoOpts, OnStartup, OnShutdown,
- ConcurrentAcceptorCount, ConcurrentConnsSups, Label}) ->
+ ConcurrentAcceptorCount, ConcurrentConnsSups, ConnectionType, Label}) ->
{ok, AckTimeout} = application:get_env(rabbit, ssl_handshake_timeout),
MaxConnections = max_conn(rabbit_misc:get_env(rabbit, connection_max, infinity),
ConcurrentConnsSups),
@@ -41,7 +41,7 @@ init({IPAddress, Port, Transport, SocketOpts, ProtoSup, ProtoOpts, OnStartup, On
num_acceptors => ConcurrentAcceptorCount,
max_connections => MaxConnections,
handshake_timeout => AckTimeout,
- connection_type => supervisor,
+ connection_type => ConnectionType,
socket_opts => [{ip, IPAddress},
{port, Port} |
SocketOpts],
diff --git a/deps/rabbit/test/backing_queue_SUITE.erl b/deps/rabbit/test/backing_queue_SUITE.erl
index a02eb959c278..d0eb1216bb73 100644
--- a/deps/rabbit/test/backing_queue_SUITE.erl
+++ b/deps/rabbit/test/backing_queue_SUITE.erl
@@ -1616,12 +1616,7 @@ wait_for_confirms(Unconfirmed) ->
true -> ok;
false ->
receive
- {'$gen_cast',
- {queue_event, _QName, {confirm, Confirmed, _}}} ->
- wait_for_confirms(
- sets:subtract(
- Unconfirmed, sets:from_list(Confirmed)));
- {'$gen_cast', {confirm, Confirmed, _}} ->
+ {'$gen_cast', {queue_event, _QName, {confirm, Confirmed, _}}} ->
wait_for_confirms(
sets:subtract(
Unconfirmed, sets:from_list(Confirmed)))
diff --git a/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl b/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl
index 6593c2d6d9e2..081461a41bf6 100644
--- a/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl
+++ b/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl
@@ -1,3 +1,9 @@
+%% 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) 2016-2023 VMware, Inc. or its affiliates. All rights reserved.
+
-module(direct_exchange_routing_v2_SUITE).
%% Test suite for the feature flag direct_exchange_routing_v2
diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl
index 485ab9926d12..15f15102a68f 100644
--- a/deps/rabbit/test/quorum_queue_SUITE.erl
+++ b/deps/rabbit/test/quorum_queue_SUITE.erl
@@ -171,8 +171,6 @@ init_per_suite(Config0) ->
rabbit_ct_helpers:log_environment(),
Config1 = rabbit_ct_helpers:merge_app_env(
Config0, {rabbit, [{quorum_tick_interval, 1000}]}),
- rabbit_ct_helpers:merge_app_env(
- Config1, {aten, [{poll_interval, 1000}]}),
rabbit_ct_helpers:run_setup_steps(Config1, []).
end_per_suite(Config) ->
diff --git a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl
index d9d8142eb0ca..5fc5a2400a00 100644
--- a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl
+++ b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl
@@ -25,6 +25,8 @@
-import(rabbit_ct_helpers, [eventually/1,
eventually/3,
consistently/1]).
+-import(rabbit_ct_broker_helpers, [rpc/5,
+ rpc/6]).
-import(quorum_queue_SUITE, [publish/2,
consume/3]).
@@ -94,9 +96,8 @@ init_per_group(Group, Config, NodesCount) ->
Config2 = rabbit_ct_helpers:run_steps(Config1,
[fun merge_app_env/1 ] ++
rabbit_ct_broker_helpers:setup_steps()),
- ok = rabbit_ct_broker_helpers:rpc(
- Config2, 0, application, set_env,
- [rabbit, channel_tick_interval, 100]),
+ ok = rpc(Config2, 0, application, set_env,
+ [rabbit, channel_tick_interval, 100]),
case rabbit_ct_broker_helpers:enable_feature_flag(Config2, stream_queue) of
ok -> Config2;
Skip -> Skip
@@ -147,6 +148,10 @@ end_per_testcase(Testcase, Config) ->
delete_queue(Ch, ?config(target_queue_5, Config)),
delete_queue(Ch, ?config(target_queue_6, Config)),
#'exchange.delete_ok'{} = amqp_channel:call(Ch, #'exchange.delete'{exchange = ?config(dead_letter_exchange, Config)}),
+
+ DlxWorkers = rabbit_ct_broker_helpers:rpc_all(Config, supervisor, which_children, [rabbit_fifo_dlx_sup]),
+ ?assert(lists:all(fun(L) -> L =:= [] end, DlxWorkers)),
+
Config1 = rabbit_ct_helpers:run_steps(
Config,
rabbit_ct_client_helpers:teardown_steps()),
@@ -499,7 +504,7 @@ drop_head_falls_back_to_at_most_once(Config) ->
consistently(
?_assertMatch(
[_, {active, 0}, _, _],
- rabbit_ct_broker_helpers:rpc(Config, Server, supervisor, count_children, [rabbit_fifo_dlx_sup]))).
+ rpc(Config, Server, supervisor, count_children, [rabbit_fifo_dlx_sup]))).
%% Test that dynamically switching dead-letter-strategy works.
switch_strategy(Config) ->
@@ -973,12 +978,12 @@ single_dlx_worker(Config) ->
assert_active_dlx_workers(0, Config, Follower0),
ok = rabbit_ct_broker_helpers:start_node(Config, Server1),
consistently(
- ?_assertMatch(
- [_, {active, 0}, _, _],
- rabbit_ct_broker_helpers:rpc(Config, Server1, supervisor, count_children, [rabbit_fifo_dlx_sup], 1000))),
+ ?_assertEqual(
+ 0,
+ length(rpc(Config, Server1, supervisor, which_children, [rabbit_fifo_dlx_sup], 1000)))),
- Pid = rabbit_ct_broker_helpers:rpc(Config, Leader0, erlang, whereis, [RaName]),
- true = rabbit_ct_broker_helpers:rpc(Config, Leader0, erlang, exit, [Pid, kill]),
+ Pid = rpc(Config, Leader0, erlang, whereis, [RaName]),
+ true = rpc(Config, Leader0, erlang, exit, [Pid, kill]),
{ok, _, {_, Leader1}} = ?awaitMatch({ok, _, _},
ra:members({RaName, Follower0}),
1000),
@@ -989,9 +994,7 @@ single_dlx_worker(Config) ->
assert_active_dlx_workers(1, Config, Leader1).
assert_active_dlx_workers(N, Config, Server) ->
- ?assertMatch(
- [_, {active, N}, _, _],
- rabbit_ct_broker_helpers:rpc(Config, Server, supervisor, count_children, [rabbit_fifo_dlx_sup], 1000)).
+ ?assertEqual(N, length(rpc(Config, Server, supervisor, which_children, [rabbit_fifo_dlx_sup], 2000))).
declare_queue(Channel, Queue, Args) ->
#'queue.declare_ok'{} = amqp_channel:call(Channel, #'queue.declare'{
@@ -1012,7 +1015,7 @@ delete_queue(Channel, Queue) ->
#'queue.delete_ok'{message_count = 0} = amqp_channel:call(Channel, #'queue.delete'{queue = Queue}).
get_global_counters(Config) ->
- rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_global_counters, overview, []).
+ rpc(Config, 0, rabbit_global_counters, overview, []).
%% Returns the delta of Metric between testcase start and now.
counted(Metric, Config) ->
diff --git a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl
index 74881791f4eb..39f4d52b01c0 100644
--- a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl
+++ b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl
@@ -89,7 +89,7 @@ basics(Config) ->
UId = ?config(uid, Config),
ConsumerTag = UId,
ok = start_cluster(ClusterName, [ServerId]),
- FState0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
+ FState0 = rabbit_fifo_client:init([ServerId]),
{ok, FState1} = rabbit_fifo_client:checkout(ConsumerTag, 1, simple_prefetch,
#{}, FState0),
@@ -97,19 +97,19 @@ basics(Config) ->
% create segment the segment will trigger a snapshot
timer:sleep(1000),
- {ok, FState2} = rabbit_fifo_client:enqueue(one, FState1),
+ {ok, FState2, []} = rabbit_fifo_client:enqueue(ClusterName, one, FState1),
DeliverFun = fun DeliverFun(S0, F) ->
receive
{ra_event, From, Evt} ->
ct:pal("ra_event ~p", [Evt]),
- case rabbit_fifo_client:handle_ra_event(From, Evt, S0) of
+ case rabbit_fifo_client:handle_ra_event(ClusterName, From, Evt, S0) of
{ok, S1,
[{deliver, C, true,
[{_Qname, _QRef, MsgId, _SomBool, _Msg}]}]} ->
{S, _A} = rabbit_fifo_client:F(C, [MsgId], S1),
%% settle applied event
- process_ra_event(S, ?RA_EVENT_TIMEOUT);
+ process_ra_event(ClusterName, S, ?RA_EVENT_TIMEOUT);
{ok, S, _} ->
DeliverFun(S, F)
end
@@ -129,13 +129,13 @@ basics(Config) ->
receive
{ra_event, From, Evt} ->
ct:pal("ra_event ~p", [Evt]),
- {ok, F6, _} = rabbit_fifo_client:handle_ra_event(From, Evt, FState5),
+ {ok, F6, _} = rabbit_fifo_client:handle_ra_event(ClusterName, From, Evt, FState5),
F6
after 5000 ->
exit(leader_change_timeout)
end,
- {ok, FState6} = rabbit_fifo_client:enqueue(two, FState5b),
+ {ok, FState6, []} = rabbit_fifo_client:enqueue(ClusterName, two, FState5b),
_FState8 = DeliverFun(FState6, return),
rabbit_quorum_queue:stop_server(ServerId),
@@ -146,11 +146,11 @@ return(Config) ->
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
- F00 = rabbit_fifo_client:init(ClusterName, [ServerId]),
- {ok, F0} = rabbit_fifo_client:enqueue(1, msg1, F00),
- {ok, F1} = rabbit_fifo_client:enqueue(2, msg2, F0),
- {_, _, F2} = process_ra_events(receive_ra_events(2, 0), F1),
- {ok, _, {_, _, MsgId, _, _}, F} = rabbit_fifo_client:dequeue(<<"tag">>, unsettled, F2),
+ F00 = rabbit_fifo_client:init([ServerId]),
+ {ok, F0, []} = rabbit_fifo_client:enqueue(ClusterName, 1, msg1, F00),
+ {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, 2, msg2, F0),
+ {_, _, F2} = process_ra_events(receive_ra_events(2, 0), ClusterName, F1),
+ {ok, _, {_, _, MsgId, _, _}, F} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, unsettled, F2),
_F2 = rabbit_fifo_client:return(<<"tag">>, [MsgId], F),
rabbit_quorum_queue:stop_server(ServerId),
@@ -160,11 +160,11 @@ rabbit_fifo_returns_correlation(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
- {ok, F1} = rabbit_fifo_client:enqueue(corr1, msg1, F0),
+ F0 = rabbit_fifo_client:init([ServerId]),
+ {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, corr1, msg1, F0),
receive
{ra_event, Frm, E} ->
- case rabbit_fifo_client:handle_ra_event(Frm, E, F1) of
+ case rabbit_fifo_client:handle_ra_event(ClusterName, Frm, E, F1) of
{ok, _F2, [{settled, _, _}]} ->
ok;
Del ->
@@ -180,13 +180,13 @@ duplicate_delivery(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
+ F0 = rabbit_fifo_client:init([ServerId]),
{ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F0),
- {ok, F2} = rabbit_fifo_client:enqueue(corr1, msg1, F1),
+ {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, corr1, msg1, F1),
Fun = fun Loop(S0) ->
receive
{ra_event, Frm, E} = Evt ->
- case rabbit_fifo_client:handle_ra_event(Frm, E, S0) of
+ case rabbit_fifo_client:handle_ra_event(ClusterName, Frm, E, S0) of
{ok, S1, [{settled, _, _}]} ->
Loop(S1);
{ok, S1, _} ->
@@ -196,7 +196,7 @@ duplicate_delivery(Config) ->
%% repeat or crash
receive
{ra_event, F, E1} ->
- case rabbit_fifo_client:handle_ra_event(
+ case rabbit_fifo_client:handle_ra_event(ClusterName,
F, E1, S1) of
{ok, S2, _} ->
S2
@@ -215,11 +215,11 @@ usage(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
+ F0 = rabbit_fifo_client:init([ServerId]),
{ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F0),
- {ok, F2} = rabbit_fifo_client:enqueue(corr1, msg1, F1),
- {ok, F3} = rabbit_fifo_client:enqueue(corr2, msg2, F2),
- {_, _, _} = process_ra_events(receive_ra_events(2, 2), F3),
+ {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, corr1, msg1, F1),
+ {ok, F3, []} = rabbit_fifo_client:enqueue(ClusterName, corr2, msg2, F2),
+ {_, _, _} = process_ra_events(receive_ra_events(2, 2), ClusterName, F3),
% force tick and usage stats emission
ServerId ! tick_timeout,
timer:sleep(50),
@@ -235,17 +235,17 @@ resends_lost_command(Config) ->
ok = meck:new(ra, [passthrough]),
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
- {ok, F1} = rabbit_fifo_client:enqueue(msg1, F0),
+ F0 = rabbit_fifo_client:init([ServerId]),
+ {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F0),
% lose the enqueue
meck:expect(ra, pipeline_command, fun (_, _, _) -> ok end),
- {ok, F2} = rabbit_fifo_client:enqueue(msg2, F1),
+ {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, msg2, F1),
meck:unload(ra),
- {ok, F3} = rabbit_fifo_client:enqueue(msg3, F2),
- {_, _, F4} = process_ra_events(receive_ra_events(2, 0), F3),
- {ok, _, {_, _, _, _, msg1}, F5} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F4),
- {ok, _, {_, _, _, _, msg2}, F6} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F5),
- {ok, _, {_, _, _, _, msg3}, _F7} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F6),
+ {ok, F3, []} = rabbit_fifo_client:enqueue(ClusterName, msg3, F2),
+ {_, _, F4} = process_ra_events(receive_ra_events(2, 0), ClusterName, F3),
+ {ok, _, {_, _, _, _, msg1}, F5} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F4),
+ {ok, _, {_, _, _, _, msg2}, F6} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F5),
+ {ok, _, {_, _, _, _, msg3}, _F7} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F6),
rabbit_quorum_queue:stop_server(ServerId),
ok.
@@ -254,10 +254,10 @@ two_quick_enqueues(Config) ->
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
- F1 = element(2, rabbit_fifo_client:enqueue(msg1, F0)),
- {ok, F2} = rabbit_fifo_client:enqueue(msg2, F1),
- _ = process_ra_events(receive_ra_events(2, 0), F2),
+ F0 = rabbit_fifo_client:init([ServerId]),
+ F1 = element(2, rabbit_fifo_client:enqueue(ClusterName, msg1, F0)),
+ {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, msg2, F1),
+ _ = process_ra_events(receive_ra_events(2, 0), ClusterName, F2),
rabbit_quorum_queue:stop_server(ServerId),
ok.
@@ -266,12 +266,12 @@ detects_lost_delivery(Config) ->
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
- F000 = rabbit_fifo_client:init(ClusterName, [ServerId]),
- {ok, F00} = rabbit_fifo_client:enqueue(msg1, F000),
- {_, _, F0} = process_ra_events(receive_ra_events(1, 0), F00),
+ F000 = rabbit_fifo_client:init([ServerId]),
+ {ok, F00, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F000),
+ {_, _, F0} = process_ra_events(receive_ra_events(1, 0), ClusterName, F00),
{ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F0),
- {ok, F2} = rabbit_fifo_client:enqueue(msg2, F1),
- {ok, F3} = rabbit_fifo_client:enqueue(msg3, F2),
+ {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, msg2, F1),
+ {ok, F3, []} = rabbit_fifo_client:enqueue(ClusterName, msg3, F2),
% lose first delivery
receive
{ra_event, _, {machine, {delivery, _, [{_, {_, msg1}}]}}} ->
@@ -281,7 +281,7 @@ detects_lost_delivery(Config) ->
end,
% assert three deliveries were received
- {[_, _, _], _, _} = process_ra_events(receive_ra_events(2, 2), F3),
+ {[_, _, _], _, _} = process_ra_events(receive_ra_events(2, 2), ClusterName, F3),
rabbit_quorum_queue:stop_server(ServerId),
ok.
@@ -290,14 +290,14 @@ returns_after_down(Config) ->
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
- {ok, F1} = rabbit_fifo_client:enqueue(msg1, F0),
- {_, _, F2} = process_ra_events(receive_ra_events(1, 0), F1),
+ F0 = rabbit_fifo_client:init([ServerId]),
+ {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F0),
+ {_, _, F2} = process_ra_events(receive_ra_events(1, 0), ClusterName, F1),
% start a consumer in a separate processes
% that exits after checkout
Self = self(),
_Pid = spawn(fun () ->
- F = rabbit_fifo_client:init(ClusterName, [ServerId]),
+ F = rabbit_fifo_client:init([ServerId]),
{ok, _} = rabbit_fifo_client:checkout(<<"tag">>, 10,
simple_prefetch,
#{}, F),
@@ -306,7 +306,7 @@ returns_after_down(Config) ->
receive checkout_done -> ok after 1000 -> exit(checkout_done_timeout) end,
timer:sleep(1000),
% message should be available for dequeue
- {ok, _, {_, _, _, _, msg1}, _} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F2),
+ {ok, _, {_, _, _, _, msg1}, _} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F2),
rabbit_quorum_queue:stop_server(ServerId),
ok.
@@ -315,10 +315,10 @@ resends_after_lost_applied(Config) ->
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
- {ok, F1} = rabbit_fifo_client:enqueue(msg1, F0),
- {_, _, F2} = process_ra_events(receive_ra_events(1, 0), F1),
- {ok, F3} = rabbit_fifo_client:enqueue(msg2, F2),
+ F0 = rabbit_fifo_client:init([ServerId]),
+ {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F0),
+ {_, _, F2} = process_ra_events(receive_ra_events(1, 0), ClusterName, F1),
+ {ok, F3, []} = rabbit_fifo_client:enqueue(ClusterName, msg2, F2),
% lose an applied event
receive
{ra_event, _, {applied, _}} ->
@@ -327,11 +327,11 @@ resends_after_lost_applied(Config) ->
exit(await_ra_event_timeout)
end,
% send another message
- {ok, F4} = rabbit_fifo_client:enqueue(msg3, F3),
- {_, _, F5} = process_ra_events(receive_ra_events(1, 0), F4),
- {ok, _, {_, _, _, _, msg1}, F6} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F5),
- {ok, _, {_, _, _, _, msg2}, F7} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F6),
- {ok, _, {_, _, _, _, msg3}, _F8} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F7),
+ {ok, F4, []} = rabbit_fifo_client:enqueue(ClusterName, msg3, F3),
+ {_, _, F5} = process_ra_events(receive_ra_events(1, 0), ClusterName, F4),
+ {ok, _, {_, _, _, _, msg1}, F6} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F5),
+ {ok, _, {_, _, _, _, msg2}, F7} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F6),
+ {ok, _, {_, _, _, _, msg3}, _F8} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F7),
rabbit_quorum_queue:stop_server(ServerId),
ok.
@@ -349,13 +349,13 @@ handles_reject_notification(Config) ->
{auto, 10, simple_prefetch},
#{})),
% reverse order - should try the first node in the list first
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId2, ServerId1]),
- {ok, F1} = rabbit_fifo_client:enqueue(one, F0),
+ F0 = rabbit_fifo_client:init([ServerId2, ServerId1]),
+ {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, one, F0),
timer:sleep(500),
% the applied notification
- _F2 = process_ra_events(receive_ra_events(1, 0), F1),
+ _F2 = process_ra_events(receive_ra_events(1, 0), ClusterName, F1),
rabbit_quorum_queue:stop_server(ServerId1),
rabbit_quorum_queue:stop_server(ServerId2),
ok.
@@ -378,12 +378,12 @@ discard(Config) ->
ok = ra:trigger_election(ServerId),
_ = ra:members(ServerId),
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
+ F0 = rabbit_fifo_client:init([ServerId]),
{ok, F1} = rabbit_fifo_client:checkout(<<"tag">>, 10,
simple_prefetch, #{}, F0),
- {ok, F2} = rabbit_fifo_client:enqueue(msg1, F1),
- F3 = discard_next_delivery(F2, 5000),
- {empty, _F4} = rabbit_fifo_client:dequeue(<<"tag1">>, settled, F3),
+ {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F1),
+ F3 = discard_next_delivery(ClusterName, F2, 5000),
+ {empty, _F4} = rabbit_fifo_client:dequeue(ClusterName, <<"tag1">>, settled, F3),
receive
{dead_letter, Reason, Letters} ->
[msg1] = Letters,
@@ -400,23 +400,23 @@ cancel_checkout(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4),
- {ok, F1} = rabbit_fifo_client:enqueue(m1, F0),
+ F0 = rabbit_fifo_client:init([ServerId], 4),
+ {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, m1, F0),
{ok, F2} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F1),
- {_, _, F3} = process_ra_events(receive_ra_events(1, 1), F2, [], [], fun (_, S) -> S end),
+ {_, _, F3} = process_ra_events(receive_ra_events(1, 1), ClusterName, F2, [], [], fun (_, S) -> S end),
{ok, F4} = rabbit_fifo_client:cancel_checkout(<<"tag">>, F3),
{F5, _} = rabbit_fifo_client:return(<<"tag">>, [0], F4),
- {ok, _, {_, _, _, _, m1}, F5} = rabbit_fifo_client:dequeue(<<"d1">>, settled, F5),
+ {ok, _, {_, _, _, _, m1}, F5} = rabbit_fifo_client:dequeue(ClusterName, <<"d1">>, settled, F5),
ok.
lost_delivery(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4),
- {ok, F1} = rabbit_fifo_client:enqueue(m1, F0),
+ F0 = rabbit_fifo_client:init([ServerId], 4),
+ {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, m1, F0),
{_, _, F2} = process_ra_events(
- receive_ra_events(1, 0), F1, [], [], fun (_, S) -> S end),
+ receive_ra_events(1, 0), ClusterName, F1, [], [], fun (_, S) -> S end),
{ok, F3} = rabbit_fifo_client:checkout(<<"tag">>, 10, simple_prefetch, #{}, F2),
%% drop a delivery, simulating e.g. a full distribution buffer
receive
@@ -427,11 +427,11 @@ lost_delivery(Config) ->
exit(await_ra_event_timeout)
end,
% send another message
- {ok, F4} = rabbit_fifo_client:enqueue(m2, F3),
+ {ok, F4, []} = rabbit_fifo_client:enqueue(ClusterName, m2, F3),
%% this hsould trigger the fifo client to fetch any missing messages
%% from the server
{_, _, _F5} = process_ra_events(
- receive_ra_events(1, 1), F4, [], [],
+ receive_ra_events(1, 1), ClusterName, F4, [], [],
fun ({deliver, _, _, Dels}, S) ->
[{_, _, _, _, M1},
{_, _, _, _, M2}] = Dels,
@@ -445,39 +445,39 @@ credit(Config) ->
ClusterName = ?config(cluster_name, Config),
ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4),
- {ok, F1} = rabbit_fifo_client:enqueue(m1, F0),
- {ok, F2} = rabbit_fifo_client:enqueue(m2, F1),
- {_, _, F3} = process_ra_events(receive_ra_events(2, 0), F2),
+ F0 = rabbit_fifo_client:init([ServerId], 4),
+ {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, m1, F0),
+ {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, m2, F1),
+ {_, _, F3} = process_ra_events(receive_ra_events(2, 0), ClusterName, F2),
%% checkout with 0 prefetch
{ok, F4} = rabbit_fifo_client:checkout(<<"tag">>, 0, credited, #{}, F3),
%% assert no deliveries
- {_, _, F5} = process_ra_events(receive_ra_events(), F4, [], [],
+ {_, _, F5} = process_ra_events(receive_ra_events(), ClusterName, F4, [], [],
fun
(D, _) -> error({unexpected_delivery, D})
end),
%% provide some credit
{F6, []} = rabbit_fifo_client:credit(<<"tag">>, 1, false, F5),
{[{_, _, _, _, m1}], [{send_credit_reply, _}], F7} =
- process_ra_events(receive_ra_events(1, 1), F6),
+ process_ra_events(receive_ra_events(1, 1), ClusterName, F6),
%% credit and drain
{F8, []} = rabbit_fifo_client:credit(<<"tag">>, 4, true, F7),
{[{_, _, _, _, m2}], [{send_credit_reply, _}, {send_drained, _}], F9} =
- process_ra_events(receive_ra_events(2, 1), F8),
+ process_ra_events(receive_ra_events(2, 1), ClusterName, F8),
flush(),
%% enqueue another message - at this point the consumer credit should be
%% all used up due to the drain
- {ok, F10} = rabbit_fifo_client:enqueue(m3, F9),
+ {ok, F10, []} = rabbit_fifo_client:enqueue(ClusterName, m3, F9),
%% assert no deliveries
- {_, _, F11} = process_ra_events(receive_ra_events(), F10, [], [],
+ {_, _, F11} = process_ra_events(receive_ra_events(), ClusterName, F10, [], [],
fun
(D, _) -> error({unexpected_delivery, D})
end),
%% credit again and receive the last message
{F12, []} = rabbit_fifo_client:credit(<<"tag">>, 10, false, F11),
- {[{_, _, _, _, m3}], _, _} = process_ra_events(receive_ra_events(1, 1), F12),
+ {[{_, _, _, _, m3}], _, _} = process_ra_events(receive_ra_events(1, 1), ClusterName, F12),
ok.
untracked_enqueue(Config) ->
@@ -487,23 +487,24 @@ untracked_enqueue(Config) ->
ok = rabbit_fifo_client:untracked_enqueue([ServerId], msg1),
timer:sleep(100),
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId]),
- {ok, _, {_, _, _, _, msg1}, _F5} = rabbit_fifo_client:dequeue(<<"tag">>, settled, F0),
+ F0 = rabbit_fifo_client:init([ServerId]),
+ {ok, _, {_, _, _, _, msg1}, _F5} = rabbit_fifo_client:dequeue(ClusterName, <<"tag">>, settled, F0),
rabbit_quorum_queue:stop_server(ServerId),
ok.
flow(Config) ->
ClusterName = ?config(cluster_name, Config),
- ServerId = ?config(node_id, Config),
+ {Name, _Node} = ServerId = ?config(node_id, Config),
ok = start_cluster(ClusterName, [ServerId]),
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 3),
- {ok, F1} = rabbit_fifo_client:enqueue(m1, F0),
- {ok, F2} = rabbit_fifo_client:enqueue(m2, F1),
- {ok, F3} = rabbit_fifo_client:enqueue(m3, F2),
- {slow, F4} = rabbit_fifo_client:enqueue(m4, F3),
- {_, _, F5} = process_ra_events(receive_ra_events(4, 0), F4),
- {ok, _} = rabbit_fifo_client:enqueue(m5, F5),
+ F0 = rabbit_fifo_client:init([ServerId], 3),
+ {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, m1, F0),
+ {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, m2, F1),
+ {ok, F3, []} = rabbit_fifo_client:enqueue(ClusterName, m3, F2),
+ {ok, F4, [{block, Name}]} = rabbit_fifo_client:enqueue(ClusterName, m4, F3),
+ {_, Actions, F5} = process_ra_events(receive_ra_events(4, 0), ClusterName, F4),
+ true = lists:member({unblock, Name}, Actions),
+ {ok, _, []} = rabbit_fifo_client:enqueue(ClusterName, m5, F5),
rabbit_quorum_queue:stop_server(ServerId),
ok.
@@ -514,10 +515,10 @@ test_queries(Config) ->
ok = start_cluster(ClusterName, [ServerId]),
Self = self(),
P = spawn(fun () ->
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4),
- {ok, F1} = rabbit_fifo_client:enqueue(m1, F0),
- {ok, F2} = rabbit_fifo_client:enqueue(m2, F1),
- process_ra_events(receive_ra_events(2, 0), F2),
+ F0 = rabbit_fifo_client:init([ServerId], 4),
+ {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, m1, F0),
+ {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, m2, F1),
+ process_ra_events(receive_ra_events(2, 0), ClusterName, F2),
Self ! ready,
receive stop -> ok end
end),
@@ -526,7 +527,7 @@ test_queries(Config) ->
after 5000 ->
exit(ready_timeout)
end,
- F0 = rabbit_fifo_client:init(ClusterName, [ServerId], 4),
+ F0 = rabbit_fifo_client:init([ServerId], 4),
{ok, _} = rabbit_fifo_client:checkout(<<"tag">>, 1, simple_prefetch, #{}, F0),
{ok, {_, Ready}, _} = ra:local_query(ServerId,
fun rabbit_fifo:query_messages_ready/1),
@@ -550,16 +551,16 @@ dequeue(Config) ->
UId = ?config(uid, Config),
Tag = UId,
ok = start_cluster(ClusterName, [ServerId]),
- F1 = rabbit_fifo_client:init(ClusterName, [ServerId]),
- {empty, F1b} = rabbit_fifo_client:dequeue(Tag, settled, F1),
- {ok, F2_} = rabbit_fifo_client:enqueue(msg1, F1b),
- {_, _, F2} = process_ra_events(receive_ra_events(1, 0), F2_),
-
- % {ok, {{0, {_, msg1}}, _}, F3} = rabbit_fifo_client:dequeue(Tag, settled, F2),
- {ok, _, {_, _, 0, _, msg1}, F3} = rabbit_fifo_client:dequeue(Tag, settled, F2),
- {ok, F4_} = rabbit_fifo_client:enqueue(msg2, F3),
- {_, _, F4} = process_ra_events(receive_ra_events(1, 0), F4_),
- {ok, _, {_, _, MsgId, _, msg2}, F5} = rabbit_fifo_client:dequeue(Tag, unsettled, F4),
+ F1 = rabbit_fifo_client:init([ServerId]),
+ {empty, F1b} = rabbit_fifo_client:dequeue(ClusterName, Tag, settled, F1),
+ {ok, F2_, []} = rabbit_fifo_client:enqueue(ClusterName, msg1, F1b),
+ {_, _, F2} = process_ra_events(receive_ra_events(1, 0), ClusterName, F2_),
+
+ % {ok, {{0, {_, msg1}}, _}, F3} = rabbit_fifo_client:dequeue(ClusterName, Tag, settled, F2),
+ {ok, _, {_, _, 0, _, msg1}, F3} = rabbit_fifo_client:dequeue(ClusterName, Tag, settled, F2),
+ {ok, F4_, []} = rabbit_fifo_client:enqueue(ClusterName, msg2, F3),
+ {_, _, F4} = process_ra_events(receive_ra_events(1, 0), ClusterName, F4_),
+ {ok, _, {_, _, MsgId, _, msg2}, F5} = rabbit_fifo_client:dequeue(ClusterName, Tag, unsettled, F4),
{_F6, _A} = rabbit_fifo_client:settle(Tag, [MsgId], F5),
rabbit_quorum_queue:stop_server(ServerId),
ok.
@@ -572,12 +573,12 @@ conf(ClusterName, UId, ServerId, _, Peers) ->
initial_members => Peers,
machine => {module, rabbit_fifo, #{}}}.
-process_ra_event(State, Wait) ->
+process_ra_event(ClusterName, State, Wait) ->
receive
{ra_event, From, Evt} ->
ct:pal("Ra_event ~p", [Evt]),
{ok, S, _Actions} =
- rabbit_fifo_client:handle_ra_event(From, Evt, State),
+ rabbit_fifo_client:handle_ra_event(ClusterName, From, Evt, State),
S
after Wait ->
flush(),
@@ -615,19 +616,19 @@ receive_ra_events(Acc) ->
Acc
end.
-process_ra_events(Events, State) ->
+process_ra_events(Events, ClusterName, State) ->
DeliveryFun = fun ({deliver, _, Tag, Msgs}, S) ->
MsgIds = [element(1, M) || M <- Msgs],
{S0, _} = rabbit_fifo_client:settle(Tag, MsgIds, S),
S0
end,
- process_ra_events(Events, State, [], [], DeliveryFun).
+ process_ra_events(Events, ClusterName, State, [], [], DeliveryFun).
-process_ra_events([], State0, Acc, Actions0, _DeliveryFun) ->
+process_ra_events([], _ClusterName, State0, Acc, Actions0, _DeliveryFun) ->
{Acc, Actions0, State0};
-process_ra_events([{ra_event, From, Evt} | Events], State0, Acc,
+process_ra_events([{ra_event, From, Evt} | Events], ClusterName, State0, Acc,
Actions0, DeliveryFun) ->
- case rabbit_fifo_client:handle_ra_event(From, Evt, State0) of
+ case rabbit_fifo_client:handle_ra_event(ClusterName, From, Evt, State0) of
{ok, State1, Actions1} ->
{Msgs, Actions, State} =
lists:foldl(
@@ -636,15 +637,15 @@ process_ra_events([{ra_event, From, Evt} | Events], State0, Acc,
(Ac, {M, A, S}) ->
{M, A ++ [Ac], S}
end, {Acc, [], State1}, Actions1),
- process_ra_events(Events, State, Msgs, Actions0 ++ Actions, DeliveryFun);
+ process_ra_events(Events, ClusterName, State, Msgs, Actions0 ++ Actions, DeliveryFun);
eol ->
eol
end.
-discard_next_delivery(State0, Wait) ->
+discard_next_delivery(ClusterName, State0, Wait) ->
receive
{ra_event, _, {machine, {delivery, _, _}}} = Evt ->
- element(3, process_ra_events([Evt], State0, [], [],
+ element(3, process_ra_events([Evt], ClusterName, State0, [], [],
fun ({deliver, Tag, _, Msgs}, S) ->
MsgIds = [element(3, M) || M <- Msgs],
{S0, _} = rabbit_fifo_client:discard(Tag, MsgIds, S),
@@ -654,30 +655,6 @@ discard_next_delivery(State0, Wait) ->
State0
end.
-return_next_delivery(State0, Wait) ->
- receive
- {ra_event, _, {machine, {delivery, _, _}}} = Evt ->
- element(3, process_ra_events([Evt], State0, [], [],
- fun ({deliver, Tag, _, Msgs}, S) ->
- MsgIds = [element(3, M) || M <- Msgs],
- {S0, _} = rabbit_fifo_client:return(Tag, MsgIds, S),
- S0
- end))
- after Wait ->
- State0
- end.
-
-validate_process_down(Name, 0) ->
- exit({process_not_down, Name});
-validate_process_down(Name, Num) ->
- case whereis(Name) of
- undefined ->
- ok;
- _ ->
- timer:sleep(100),
- validate_process_down(Name, Num-1)
- end.
-
start_cluster(ClusterName, ServerIds, RaFifoConfig) ->
{ok, Started, _} = ra:start_cluster(?RA_SYSTEM,
ClusterName#resource.name,
diff --git a/deps/rabbit_common/include/rabbit.hrl b/deps/rabbit_common/include/rabbit.hrl
index e4976267a2b2..444e807fbf35 100644
--- a/deps/rabbit_common/include/rabbit.hrl
+++ b/deps/rabbit_common/include/rabbit.hrl
@@ -226,11 +226,11 @@
-define(SUPERVISOR_WAIT,
rabbit_misc:get_env(rabbit, supervisor_shutdown_timeout, infinity)).
-define(WORKER_WAIT,
- rabbit_misc:get_env(rabbit, worker_shutdown_timeout, 300000)).
+ rabbit_misc:get_env(rabbit, worker_shutdown_timeout, 300_000)).
-define(MSG_STORE_WORKER_WAIT,
- rabbit_misc:get_env(rabbit, msg_store_shutdown_timeout, 600000)).
+ rabbit_misc:get_env(rabbit, msg_store_shutdown_timeout, 600_000)).
-define(CLASSIC_QUEUE_WORKER_WAIT,
- rabbit_misc:get_env(rabbit, classic_queue_shutdown_timeout, 600000)).
+ rabbit_misc:get_env(rabbit, classic_queue_shutdown_timeout, 600_000)).
-define(HIBERNATE_AFTER_MIN, 1000).
-define(DESIRED_HIBERNATE, 10000).
diff --git a/deps/rabbit_common/src/credit_flow.erl b/deps/rabbit_common/src/credit_flow.erl
index 33c345315b74..b77b5116267b 100644
--- a/deps/rabbit_common/src/credit_flow.erl
+++ b/deps/rabbit_common/src/credit_flow.erl
@@ -88,9 +88,8 @@
end).
%% If current process was blocked by credit flow in the last
-%% STATE_CHANGE_INTERVAL milliseconds, state/0 will report it as "in
-%% flow".
--define(STATE_CHANGE_INTERVAL, 1000000).
+%% STATE_CHANGE_INTERVAL microseconds, state/0 will report it as "in flow".
+-define(STATE_CHANGE_INTERVAL, 1_000_000).
-ifdef(CREDIT_FLOW_TRACING).
-define(TRACE_BLOCKED(SELF, FROM), rabbit_event:notify(credit_flow_blocked,
diff --git a/deps/rabbit_common/src/rabbit_auth_mechanism.erl b/deps/rabbit_common/src/rabbit_auth_mechanism.erl
index 6090c4177146..d950769b5046 100644
--- a/deps/rabbit_common/src/rabbit_auth_mechanism.erl
+++ b/deps/rabbit_common/src/rabbit_auth_mechanism.erl
@@ -5,6 +5,19 @@
%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
%%
+%% This module defines the interface for SASL mechanisms.
+%%
+%% The central idea behind SASL is to decouple application protocols
+%% from authentication mechanisms.
+%%
+%% Protocols AMQP 0.9.1, AMQP 1.0, Stream, and MQTT 5.0 support SASL.
+%%
+%% Therefore, their respective protocol readers call a rabbit_auth_mechanism_* module.
+%% The SASL mechanism knows how to extract credentials (e.g. username and password, or just
+%% a username from a client certificate) and asks rabbit_access_control to
+%% authenticate the client. rabbit_access_control in turn delegates authentication
+%% (and authorization) to a rabbit_auth_backend_* module.
+
-module(rabbit_auth_mechanism).
-behaviour(rabbit_registry_class).
diff --git a/deps/rabbit_common/src/rabbit_event.erl b/deps/rabbit_common/src/rabbit_event.erl
index 4bf9b0f52a5c..a6760b238858 100644
--- a/deps/rabbit_common/src/rabbit_event.erl
+++ b/deps/rabbit_common/src/rabbit_event.erl
@@ -25,7 +25,7 @@
%%----------------------------------------------------------------------------
--export_type([event_type/0, event_props/0, event_timestamp/0, event/0]).
+-export_type([event_type/0, event_props/0, event_timestamp/0, event/0, state/0]).
-type event_type() :: atom().
-type event_props() :: term().
@@ -41,6 +41,7 @@
-type timer_fun() :: fun (() -> 'ok').
-type container() :: tuple().
-type pos() :: non_neg_integer().
+-opaque state() :: #state{}.
-spec start_link() -> rabbit_types:ok_pid_or_error().
-spec init_stats_timer(container(), pos()) -> container().
diff --git a/deps/rabbit_common/src/rabbit_misc.erl b/deps/rabbit_common/src/rabbit_misc.erl
index d19f60a2d19a..a79e273d17fb 100644
--- a/deps/rabbit_common/src/rabbit_misc.erl
+++ b/deps/rabbit_common/src/rabbit_misc.erl
@@ -84,6 +84,7 @@
-export([raw_read_file/1]).
-export([find_child/2]).
-export([is_regular_file/1]).
+-export([pipeline/3]).
%% Horrible macro to use in guards
-define(IS_BENIGN_EXIT(R),
@@ -1471,7 +1472,6 @@ whereis_name(Name) ->
%% End copypasta from gen_server2.erl
%% -------------------------------------------------------------------------
-
%% This will execute a Powershell command without an intervening cmd.exe
%% process. Output lines can't exceed 512 bytes.
%%
@@ -1552,3 +1552,45 @@ find_powershell() ->
PwshExe ->
PwshExe
end.
+
+%% -------------------------------------------------------------------------
+%% Begin copy from
+%% https://github.com/emqx/emqx/blob/cffdcb42843d48bf99d8bd13695bc73149c98a23/apps/emqx/src/emqx_misc.erl#L141-L157
+
+%%--------------------------------------------------------------------
+%% Copyright (c) 2017-2022 EMQ Technologies Co., Ltd. All Rights Reserved.
+%%
+%% Licensed under the Apache License, Version 2.0 (the "License");
+%% you may not use this file except in compliance with the License.
+%% You may obtain a copy of the License at
+%%
+%% http://www.apache.org/licenses/LICENSE-2.0
+%%
+%% Unless required by applicable law or agreed to in writing, software
+%% distributed under the License is distributed on an "AS IS" BASIS,
+%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+%% See the License for the specific language governing permissions and
+%% limitations under the License.
+%%--------------------------------------------------------------------
+
+pipeline([], Input, State) ->
+ {ok, Input, State};
+pipeline([Fun | More], Input, State) ->
+ case apply_fun(Fun, Input, State) of
+ ok -> pipeline(More, Input, State);
+ {ok, NState} -> pipeline(More, Input, NState);
+ {ok, Output, NState} -> pipeline(More, Output, NState);
+ {error, Reason} -> {error, Reason, State};
+ {error, Reason, NState} -> {error, Reason, NState}
+ end.
+
+-compile({inline, [apply_fun/3]}).
+apply_fun(Fun, Input, State) ->
+ case erlang:fun_info(Fun, arity) of
+ {arity, 1} -> Fun(Input);
+ {arity, 2} -> Fun(Input, State)
+ end.
+
+%% End copy from
+%% https://github.com/emqx/emqx/blob/cffdcb42843d48bf99d8bd13695bc73149c98a23/apps/emqx/src/emqx_misc.erl#L141-L157
+%% -------------------------------------------------------------------------
\ No newline at end of file
diff --git a/deps/rabbit_common/src/rabbit_writer.erl b/deps/rabbit_common/src/rabbit_writer.erl
index be306fac39e1..a284d203b1b5 100644
--- a/deps/rabbit_common/src/rabbit_writer.erl
+++ b/deps/rabbit_common/src/rabbit_writer.erl
@@ -58,13 +58,13 @@
%% data pending delivery (between socket
%% flushes)
pending,
- %% defines how ofter gc will be executed
+ %% defines how often gc will be executed
writer_gc_threshold
}).
-define(HIBERNATE_AFTER, 5000).
%% 1GB
--define(DEFAULT_GC_THRESHOLD, 1000000000).
+-define(DEFAULT_GC_THRESHOLD, 1_000_000_000).
%%---------------------------------------------------------------------------
diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl
index a780d00df9d0..e0f0c91db816 100644
--- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl
+++ b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl
@@ -81,6 +81,9 @@ system_terminate(Reason, _Parent, _Deb, _State) ->
system_code_change(Misc, _Module, _OldVsn, _Extra) ->
{ok, Misc}.
+-spec conserve_resources(pid(),
+ rabbit_alarm:resource_alarm_source(),
+ rabbit_alarm:resource_alert()) -> ok.
conserve_resources(Pid, Source, {_, Conserve, _}) ->
Pid ! {conserve_resources, Source, Conserve},
ok.
@@ -785,16 +788,17 @@ info_internal(SockStat, S) when SockStat =:= recv_oct;
socket_info(fun (Sock) -> rabbit_net:getstat(Sock, [SockStat]) end,
fun ([{_, I}]) -> I end, S);
info_internal(ssl, #v1{sock = Sock}) -> rabbit_net:is_ssl(Sock);
-info_internal(ssl_protocol, S) -> ssl_info(fun ({P, _}) -> P end, S);
-info_internal(ssl_key_exchange, S) -> ssl_info(fun ({_, {K, _, _}}) -> K end, S);
-info_internal(ssl_cipher, S) -> ssl_info(fun ({_, {_, C, _}}) -> C end, S);
-info_internal(ssl_hash, S) -> ssl_info(fun ({_, {_, _, H}}) -> H end, S);
-info_internal(peer_cert_issuer, S) ->
- cert_info(fun rabbit_ssl:peer_cert_issuer/1, S);
-info_internal(peer_cert_subject, S) ->
- cert_info(fun rabbit_ssl:peer_cert_subject/1, S);
-info_internal(peer_cert_validity, S) ->
- cert_info(fun rabbit_ssl:peer_cert_validity/1, S).
+info_internal(SSL, #v1{sock = Sock, proxy_socket = ProxySock})
+ when SSL =:= ssl_protocol;
+ SSL =:= ssl_key_exchange;
+ SSL =:= ssl_cipher;
+ SSL =:= ssl_hash ->
+ rabbit_ssl:info(SSL, {Sock, ProxySock});
+info_internal(Cert, #v1{sock = Sock})
+ when Cert =:= peer_cert_issuer;
+ Cert =:= peer_cert_subject;
+ Cert =:= peer_cert_validity ->
+ rabbit_ssl:cert_info(Cert, Sock).
%% From rabbit_reader
socket_info(Get, Select, #v1{sock = Sock}) ->
@@ -802,22 +806,3 @@ socket_info(Get, Select, #v1{sock = Sock}) ->
{ok, T} -> Select(T);
{error, _} -> ''
end.
-
-ssl_info(F, #v1{sock = Sock, proxy_socket = ProxySock}) ->
- case rabbit_net:proxy_ssl_info(Sock, ProxySock) of
- nossl -> '';
- {error, _} -> '';
- {ok, Items} ->
- P = proplists:get_value(protocol, Items),
- #{cipher := C,
- key_exchange := K,
- mac := H} = proplists:get_value(selected_cipher_suite, Items),
- F({P, {K, C, H}})
- end.
-
-cert_info(F, #v1{sock = Sock}) ->
- case rabbit_net:peercert(Sock) of
- nossl -> '';
- {error, _} -> '';
- {ok, Cert} -> list_to_binary(F(Cert))
- end.
diff --git a/deps/rabbitmq_ct_helpers/BUILD.bazel b/deps/rabbitmq_ct_helpers/BUILD.bazel
index 5c46b301f0b1..f323291a06a9 100644
--- a/deps/rabbitmq_ct_helpers/BUILD.bazel
+++ b/deps/rabbitmq_ct_helpers/BUILD.bazel
@@ -10,5 +10,6 @@ erlang_app(
deps = [
"//deps/rabbit_common:erlang_app",
"@proper//:erlang_app",
+ "@meck//:erlang_app",
],
)
diff --git a/deps/rabbitmq_ct_helpers/Makefile b/deps/rabbitmq_ct_helpers/Makefile
index d81e011ae6fb..5dc6e160bd7b 100644
--- a/deps/rabbitmq_ct_helpers/Makefile
+++ b/deps/rabbitmq_ct_helpers/Makefile
@@ -1,7 +1,7 @@
PROJECT = rabbitmq_ct_helpers
PROJECT_DESCRIPTION = Common Test helpers for RabbitMQ
-DEPS = rabbit_common proper inet_tcp_proxy
+DEPS = rabbit_common proper inet_tcp_proxy meck
TEST_DEPS = rabbit
dep_rabbit_common = git-subfolder https://github.com/rabbitmq/rabbitmq-server main deps/rabbit_common
diff --git a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl
index da3b2d9ad24c..fcb6302cc90b 100644
--- a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl
+++ b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl
@@ -25,6 +25,8 @@
rewrite_node_config_file/2,
cluster_nodes/1, cluster_nodes/2,
+ setup_meck/1,
+
get_node_configs/1, get_node_configs/2,
get_node_config/2, get_node_config/3, set_node_config/3,
nodename_to_index/2,
@@ -36,7 +38,7 @@
add_code_path_to_node/2,
add_code_path_to_all_nodes/2,
- rpc/5, rpc/6,
+ rpc/4, rpc/5, rpc/6,
rpc_all/4, rpc_all/5,
start_node/2,
@@ -187,6 +189,7 @@
tcp_port_mqtt,
tcp_port_mqtt_tls,
tcp_port_web_mqtt,
+ tcp_port_web_mqtt_tls,
tcp_port_stomp,
tcp_port_stomp_tls,
tcp_port_web_stomp,
@@ -511,6 +514,9 @@ update_tcp_ports_in_rmq_config(NodeConfig, [tcp_port_mqtt_tls = Key | Rest]) ->
NodeConfig1 = rabbit_ct_helpers:merge_app_env(NodeConfig,
{rabbitmq_mqtt, [{ssl_listeners, [?config(Key, NodeConfig)]}]}),
update_tcp_ports_in_rmq_config(NodeConfig1, Rest);
+update_tcp_ports_in_rmq_config(NodeConfig, [tcp_port_web_mqtt_tls | Rest]) ->
+ %% Skip this one, because we need more than just a port to configure
+ update_tcp_ports_in_rmq_config(NodeConfig, Rest);
update_tcp_ports_in_rmq_config(NodeConfig, [tcp_port_web_mqtt = Key | Rest]) ->
NodeConfig1 = rabbit_ct_helpers:merge_app_env(NodeConfig,
{rabbitmq_web_mqtt, [{tcp_config, [{port, ?config(Key, NodeConfig)}]}]}),
@@ -633,6 +639,7 @@ do_start_rabbitmq_node(Config, NodeConfig, I) ->
InitialNodename = ?config(initial_nodename, NodeConfig),
DistPort = ?config(tcp_port_erlang_dist, NodeConfig),
ConfigFile = ?config(erlang_node_config_filename, NodeConfig),
+ AdditionalErlArgs = rabbit_ct_helpers:get_config(Config, additional_erl_args, []),
%% Use inet_proxy_dist to handle distribution. This is used by the
%% partitions testsuite.
DistMod = rabbit_ct_helpers:get_config(Config, erlang_dist_module),
@@ -729,7 +736,7 @@ do_start_rabbitmq_node(Config, NodeConfig, I) ->
{"RABBITMQ_DIST_PORT=~b", [DistPort]},
{"RABBITMQ_CONFIG_FILE=~ts", [ConfigFile]},
{"RABBITMQ_SERVER_START_ARGS=~ts", [StartArgs1]},
- "RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS=+S 2 +sbwt very_short +A 24",
+ {"RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS=+S 2 +sbwt very_short +A 24 ~ts", [AdditionalErlArgs]},
"RABBITMQ_LOG=debug",
"RMQCTL_WAIT_TIMEOUT=180",
{"TEST_TMPDIR=~ts", [PrivDir]}
@@ -1454,11 +1461,10 @@ set_vhost_limit(Config, Node, VHost, Limit0, Value) ->
max_queues -> <<"max-queues">>;
Other -> rabbit_data_coercion:to_binary(Other)
end,
- Definition = rabbit_json:encode(#{Limit => Value}),
+ Limits = [{Limit, Value}],
rpc(Config, Node,
- rabbit_vhost_limit,
- set,
- [VHost, Definition, <<"ct-tests">>]).
+ rabbit_vhost_limit, set,
+ [VHost, Limits, <<"ct-tests">>]).
set_user_limits(Config, Username, Limits) ->
set_user_limits(Config, 0, Username, Limits).
@@ -1552,6 +1558,9 @@ add_code_path_to_all_nodes(Config, Module) ->
|| Nodename <- Nodenames],
ok.
+rpc(Config, Module, Function, Args) ->
+ rpc(Config, 0, Module, Function, Args).
+
rpc(Config, Node, Module, Function, Args)
when is_atom(Node) andalso Node =/= undefined ->
rpc(Config, Node, Module, Function, Args, infinity);
@@ -1946,3 +1955,9 @@ if_cover(F) ->
_ ->
F()
end.
+
+setup_meck(Config) ->
+ {Mod, Bin, File} = code:get_object_code(meck),
+ [true | _] = rpc_all(Config, code, add_path, [filename:dirname(File)]),
+ [{module, Mod} | _] = rpc_all(Config, code, load_binary, [Mod, File, Bin]),
+ ok.
diff --git a/deps/rabbitmq_management/priv/www/js/formatters.js b/deps/rabbitmq_management/priv/www/js/formatters.js
index 9ef81a8ac7d1..132afb6afc69 100644
--- a/deps/rabbitmq_management/priv/www/js/formatters.js
+++ b/deps/rabbitmq_management/priv/www/js/formatters.js
@@ -275,6 +275,7 @@ function fmt_rate_num(num) {
}
function fmt_num_thousands(num) {
+ if (num == undefined) return UNKNOWN_REPR;
var conv_num = parseFloat(num); // to avoid errors, if someone calls fmt_num_thousands(someNumber.toFixed(0))
return fmt_num_thousands_unfixed(conv_num.toFixed(0));
}
@@ -288,11 +289,9 @@ function fmt_num_thousands_unfixed(num) {
}
function fmt_percent(num) {
- if (num === '') {
- return 'N/A';
- } else {
- return Math.round(num * 100) + '%';
- }
+ if (num == undefined) return UNKNOWN_REPR;
+ if (num === '') return UNKNOWN_REPR;
+ return Math.round(num * 100) + '%';
}
function pick_rate(fmt, obj, name, mode) {
diff --git a/deps/rabbitmq_management/priv/www/js/global.js b/deps/rabbitmq_management/priv/www/js/global.js
index 2b81dd8c9ded..023b5b3c57d8 100644
--- a/deps/rabbitmq_management/priv/www/js/global.js
+++ b/deps/rabbitmq_management/priv/www/js/global.js
@@ -112,7 +112,7 @@ var ALL_COLUMNS =
['channels', 'Channels', true],
['channel_max', 'Channel max', false],
['frame_max', 'Frame max', false],
- ['auth_mechanism', 'Auth mechanism', false],
+ ['auth_mechanism', 'SASL auth mechanism', false],
['client', 'Client', false]],
'Network': [['from_client', 'From client', true],
['to_client', 'To client', true],
diff --git a/deps/rabbitmq_management/priv/www/js/tmpl/connection.ejs b/deps/rabbitmq_management/priv/www/js/tmpl/connection.ejs
index f107a3981c1a..f834b02fb5e0 100644
--- a/deps/rabbitmq_management/priv/www/js/tmpl/connection.ejs
+++ b/deps/rabbitmq_management/priv/www/js/tmpl/connection.ejs
@@ -44,7 +44,7 @@
<% if (connection.auth_mechanism) { %>
- | Authentication |
+ SASL auth mechanism |
<%= connection.auth_mechanism %> |
<% } %>
diff --git a/deps/rabbitmq_management/priv/www/js/tmpl/connections.ejs b/deps/rabbitmq_management/priv/www/js/tmpl/connections.ejs
index a75fa7f56011..464894d20876 100644
--- a/deps/rabbitmq_management/priv/www/js/tmpl/connections.ejs
+++ b/deps/rabbitmq_management/priv/www/js/tmpl/connections.ejs
@@ -52,7 +52,7 @@
<%= fmt_sort('Frame max', 'frame_max') %> |
<% } %>
<% if (show_column('connections', 'auth_mechanism')) { %>
- <%= fmt_sort('Auth mechanism', 'auth_mechanism') %> |
+ <%= fmt_sort('SASL auth mechanism', 'auth_mechanism') %> |
<% } %>
<% if (show_column('connections', 'client')) { %>
<%= fmt_sort('Client', 'properties') %> |
diff --git a/deps/rabbitmq_management/src/rabbit_mgmt_wm_connection.erl b/deps/rabbitmq_management/src/rabbit_mgmt_wm_connection.erl
index 5e14eb710465..11d8fc02018a 100644
--- a/deps/rabbitmq_management/src/rabbit_mgmt_wm_connection.erl
+++ b/deps/rabbitmq_management/src/rabbit_mgmt_wm_connection.erl
@@ -90,11 +90,14 @@ force_close_connection(ReqData, Conn, Pid) ->
undefined -> "Closed via management plugin";
V -> binary_to_list(V)
end,
- case proplists:get_value(type, Conn) of
- direct -> amqp_direct_connection:server_close(Pid, 320, Reason);
- network -> rabbit_networking:close_connection(Pid, Reason);
- _ ->
- % best effort, this will work for connections to the stream plugin
- gen_server:call(Pid, {shutdown, Reason}, infinity)
- end,
- ok.
+ case proplists:get_value(type, Conn) of
+ direct ->
+ amqp_direct_connection:server_close(Pid, 320, Reason);
+ network ->
+ rabbit_networking:close_connection(Pid, Reason);
+ _ ->
+ %% Best effort will work for following plugins:
+ %% rabbitmq_stream, rabbitmq_mqtt, rabbitmq_web_mqtt
+ _ = Pid ! {shutdown, Reason},
+ ok
+ end.
diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel
index a337e7b04bbf..be73e3e9a862 100644
--- a/deps/rabbitmq_mqtt/BUILD.bazel
+++ b/deps/rabbitmq_mqtt/BUILD.bazel
@@ -2,8 +2,8 @@ load("@rules_erlang//:xref2.bzl", "xref")
load("@rules_erlang//:dialyze.bzl", "dialyze", "plt")
load(
"//:rabbitmq.bzl",
- "BROKER_VERSION_REQUIREMENTS_ANY",
"RABBITMQ_DIALYZER_OPTS",
+ "BROKER_VERSION_REQUIREMENTS_ANY",
"assert_suites",
"broker_for_integration_suites",
"rabbitmq_app",
@@ -36,9 +36,13 @@ APP_ENV = """[
{num_tcp_acceptors, 10},
{num_ssl_acceptors, 10},
{tcp_listen_options, [{backlog, 128},
- {nodelay, true}]},
+ {nodelay, true},
+ {send_timeout, 15000},
+ {send_timeout_close, true}
+ ]},
{proxy_protocol, false},
- {sparkplug, false}
+ {sparkplug, false},
+ {mailbox_soft_limit, 1000}
]"""
BUILD_DEPS = [
@@ -48,14 +52,11 @@ BUILD_DEPS = [
DEPS = [
"//deps/amqp_client:erlang_app",
"//deps/rabbit_common:erlang_app",
+ "//deps/rabbit:erlang_app",
"@ra//:erlang_app",
"@ranch//:erlang_app",
]
-RUNTIME_DEPS = [
- "//deps/rabbit:erlang_app",
-]
-
rabbitmq_app(
app_description = APP_DESCRIPTION,
app_env = APP_ENV,
@@ -63,7 +64,6 @@ rabbitmq_app(
app_module = APP_MODULE,
app_name = APP_NAME,
build_deps = BUILD_DEPS,
- runtime_deps = RUNTIME_DEPS,
deps = DEPS,
)
@@ -74,16 +74,20 @@ xref(
plt(
name = "base_plt",
libs = ["//deps/rabbitmq_cli:elixir"],
- deps = ["//deps/rabbitmq_cli:elixir"] + BUILD_DEPS + DEPS + RUNTIME_DEPS,
+ deps = ["//deps/rabbitmq_cli:elixir"] + BUILD_DEPS + DEPS,
)
dialyze(
dialyzer_opts = RABBITMQ_DIALYZER_OPTS,
plt = ":base_plt",
- warnings_as_errors = False,
)
-broker_for_integration_suites()
+broker_for_integration_suites(
+ extra_plugins = [
+ "//deps/rabbitmq_management:erlang_app",
+ "//deps/rabbitmq_web_mqtt:erlang_app",
+ ],
+)
rabbitmq_test_helper(
name = "rabbit_auth_backend_mqtt_mock",
@@ -95,12 +99,34 @@ rabbitmq_test_helper(
],
)
+rabbitmq_test_helper(
+ name = "util",
+ srcs = [
+ "test/util.erl",
+ ],
+ deps = [
+ "//deps/rabbit_common:erlang_app",
+ ],
+ hdrs = [
+ "include/rabbit_mqtt.hrl"
+ ],
+)
+
+rabbitmq_test_helper(
+ name = "event_recorder",
+ srcs = [
+ "test/event_recorder.erl",
+ ],
+ deps = [
+ "//deps/rabbit_common:erlang_app",
+ ],
+)
+
rabbitmq_integration_suite(
name = "auth_SUITE",
additional_beam = [
":rabbit_auth_backend_mqtt_mock",
],
- flaky = True,
runtime_deps = [
"@emqtt//:erlang_app",
],
@@ -109,10 +135,12 @@ rabbitmq_integration_suite(
rabbitmq_integration_suite(
name = "cluster_SUITE",
size = "large",
- flaky = True,
runtime_deps = [
"@emqtt//:erlang_app",
],
+ additional_beam = [
+ ":util",
+ ],
)
rabbitmq_integration_suite(
@@ -120,15 +148,22 @@ rabbitmq_integration_suite(
runtime_deps = [
"@emqtt//:erlang_app",
],
+ additional_beam = [
+ ":util",
+ ],
)
rabbitmq_integration_suite(
name = "config_schema_SUITE",
)
+
+rabbitmq_integration_suite(
+ name = "config_SUITE",
+)
+
rabbitmq_integration_suite(
name = "java_SUITE",
- flaky = True,
)
rabbitmq_suite(
@@ -154,6 +189,10 @@ rabbitmq_integration_suite(
runtime_deps = [
"@emqtt//:erlang_app",
],
+ additional_beam = [
+ ":event_recorder",
+ ":util",
+ ],
)
rabbitmq_integration_suite(
@@ -161,6 +200,9 @@ rabbitmq_integration_suite(
runtime_deps = [
"@emqtt//:erlang_app",
],
+ additional_beam = [
+ ":util",
+ ],
)
rabbitmq_suite(
@@ -171,6 +213,31 @@ rabbitmq_suite(
],
)
+rabbitmq_integration_suite(
+ name = "ff_SUITE",
+ runtime_deps = [
+ "@emqtt//:erlang_app",
+ ],
+ additional_beam = [
+ ":util",
+ ],
+)
+
+rabbitmq_integration_suite(
+ name = "shared_SUITE",
+ size = "large",
+ timeout = "eternal",
+ runtime_deps = [
+ "@emqtt//:erlang_app",
+ "//deps/rabbitmq_management_agent:erlang_app",
+ "@gun//:erlang_app",
+ ],
+ additional_beam = [
+ ":event_recorder",
+ ":util",
+ ],
+)
+
assert_suites()
alias(
diff --git a/deps/rabbitmq_mqtt/Makefile b/deps/rabbitmq_mqtt/Makefile
index d07c6ba04b61..774101913185 100644
--- a/deps/rabbitmq_mqtt/Makefile
+++ b/deps/rabbitmq_mqtt/Makefile
@@ -21,9 +21,13 @@ define PROJECT_ENV
{num_tcp_acceptors, 10},
{num_ssl_acceptors, 10},
{tcp_listen_options, [{backlog, 128},
- {nodelay, true}]},
+ {nodelay, true},
+ {send_timeout, 15000},
+ {send_timeout_close, true}
+ ]},
{proxy_protocol, false},
- {sparkplug, false}
+ {sparkplug, false},
+ {mailbox_soft_limit, 1000}
]
endef
@@ -36,7 +40,7 @@ BUILD_WITHOUT_QUIC=1
export BUILD_WITHOUT_QUIC
DEPS = ranch rabbit_common rabbit amqp_client ra
-TEST_DEPS = emqtt ct_helper rabbitmq_ct_helpers rabbitmq_ct_client_helpers
+TEST_DEPS = emqtt ct_helper rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_management rabbitmq_web_mqtt
dep_ct_helper = git https://github.com/extend/ct_helper.git master
dep_emqtt = git https://github.com/emqx/emqtt.git 1.7.0-rc.2
diff --git a/deps/rabbitmq_mqtt/include/mqtt_machine.hrl b/deps/rabbitmq_mqtt/include/mqtt_machine.hrl
index b4c9eb2cdd59..61fc8f759359 100644
--- a/deps/rabbitmq_mqtt/include/mqtt_machine.hrl
+++ b/deps/rabbitmq_mqtt/include/mqtt_machine.hrl
@@ -5,9 +5,12 @@
%% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved.
%%
--record(machine_state, {client_ids = #{},
- pids = #{},
- %% add acouple of fields for future extensibility
- reserved_1,
- reserved_2}).
+-record(machine_state, {
+ %% client ID to connection PID
+ client_ids = #{},
+ %% connection PID to list of client IDs
+ pids = #{},
+ %% add acouple of fields for future extensibility
+ reserved_1,
+ reserved_2}).
diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl
index e75786db15be..c845240e55d3 100644
--- a/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl
+++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt.hrl
@@ -5,88 +5,75 @@
%% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved.
%%
--define(CLIENT_ID_MAXLEN, 23).
-
-%% reader state
--record(state, { socket,
- conn_name,
- await_recv,
- deferred_recv,
- received_connect_frame,
- connection_state,
- keepalive,
- keepalive_sup,
- conserve,
- parse_state,
- proc_state,
- connection,
- stats_timer }).
-
-%% processor state
--record(proc_state, { socket,
- subscriptions,
- consumer_tags,
- unacked_pubs,
- awaiting_ack,
- awaiting_seqno,
- message_id,
- client_id,
- clean_sess,
- will_msg,
- channels,
- connection,
- exchange,
- adapter_info,
- ssl_login_name,
- %% Retained messages handler. See rabbit_mqtt_retainer_sup
- %% and rabbit_mqtt_retainer.
- retainer_pid,
- auth_state,
- send_fun,
- peer_addr,
- mqtt2amqp_fun,
- amqp2mqtt_fun,
- register_state }).
-
--record(auth_state, {username,
- user,
- vhost}).
+-define(APP_NAME, rabbitmq_mqtt).
+-define(PG_SCOPE, pg_scope_rabbitmq_mqtt_clientid).
+-define(QUEUE_TYPE_QOS_0, rabbit_mqtt_qos0_queue).
+-define(PERSISTENT_TERM_MAILBOX_SOFT_LIMIT, mqtt_mailbox_soft_limit).
+-define(MQTT_GUIDE_URL, <<"https://rabbitmq.com/mqtt.html">>).
+-define(MQTT_PROTO_V3, mqtt310).
+-define(MQTT_PROTO_V4, mqtt311).
-%% does not include vhost: it is used in
-%% the table name
--record(retained_message, {topic,
- mqtt_msg}).
+-define(ITEMS,
+ [pid,
+ protocol,
+ host,
+ port,
+ peer_host,
+ peer_port,
+ ssl,
+ ssl_protocol,
+ ssl_key_exchange,
+ ssl_cipher,
+ ssl_hash,
+ vhost,
+ user
+ ]).
-define(INFO_ITEMS,
- [host,
- port,
- peer_host,
- peer_port,
- protocol,
- channels,
- channel_max,
- frame_max,
- client_properties,
- ssl,
- ssl_protocol,
- ssl_key_exchange,
- ssl_cipher,
- ssl_hash,
- conn_name,
- connection_state,
- connection,
- consumer_tags,
- unacked_pubs,
- awaiting_ack,
- awaiting_seqno,
- message_id,
- client_id,
- clean_sess,
- will_msg,
- exchange,
- ssl_login_name,
- retainer_pid,
- user,
- vhost]).
+ ?ITEMS ++
+ [
+ client_id,
+ conn_name,
+ connection_state,
+ ssl_login_name,
+ recv_cnt,
+ recv_oct,
+ send_cnt,
+ send_oct,
+ send_pend,
+ clean_sess,
+ will_msg,
+ retainer_pid,
+ exchange,
+ prefetch,
+ messages_unconfirmed,
+ messages_unacknowledged
+ ]).
--define(MQTT_GUIDE_URL, <<"https://rabbitmq.com/mqtt.html">>).
+-define(CREATION_EVENT_KEYS,
+ ?ITEMS ++
+ [name,
+ client_properties,
+ peer_cert_issuer,
+ peer_cert_subject,
+ peer_cert_validity,
+ auth_mechanism,
+ timeout,
+ frame_max,
+ channel_max,
+ connected_at,
+ node,
+ user_who_performed_action
+ ]).
+
+-define(SIMPLE_METRICS,
+ [pid,
+ recv_oct,
+ send_oct,
+ reductions]).
+-define(OTHER_METRICS,
+ [recv_cnt,
+ send_cnt,
+ send_pend,
+ garbage_collection,
+ state]).
diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl
deleted file mode 100644
index 82917560cbeb..000000000000
--- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_frame.hrl
+++ /dev/null
@@ -1,90 +0,0 @@
-%% This Source Code Form is subject to the terms of the Mozilla Public
-%% License, v. 2.0. If a copy of the MPL was not distributed with this
-%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
-%%
-%% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved.
-%%
-
--define(PROTOCOL_NAMES, [{3, "MQIsdp"}, {4, "MQTT"}]).
-
-%% frame types
-
--define(CONNECT, 1).
--define(CONNACK, 2).
--define(PUBLISH, 3).
--define(PUBACK, 4).
--define(PUBREC, 5).
--define(PUBREL, 6).
--define(PUBCOMP, 7).
--define(SUBSCRIBE, 8).
--define(SUBACK, 9).
--define(UNSUBSCRIBE, 10).
--define(UNSUBACK, 11).
--define(PINGREQ, 12).
--define(PINGRESP, 13).
--define(DISCONNECT, 14).
-
-%% connect return codes
-
--define(CONNACK_ACCEPT, 0).
--define(CONNACK_PROTO_VER, 1). %% unacceptable protocol version
--define(CONNACK_INVALID_ID, 2). %% identifier rejected
--define(CONNACK_SERVER, 3). %% server unavailable
--define(CONNACK_CREDENTIALS, 4). %% bad user name or password
--define(CONNACK_AUTH, 5). %% not authorized
-
-%% qos levels
-
--define(QOS_0, 0).
--define(QOS_1, 1).
--define(QOS_2, 2).
-
-%% TODO
--type message_id() :: any().
-
--record(mqtt_frame, {fixed,
- variable,
- payload}).
-
--record(mqtt_frame_fixed, {type = 0,
- dup = 0,
- qos = 0,
- retain = 0}).
-
--record(mqtt_frame_connect, {proto_ver,
- will_retain,
- will_qos,
- will_flag,
- clean_sess,
- keep_alive,
- client_id,
- will_topic,
- will_msg,
- username,
- password}).
-
--record(mqtt_frame_connack, {session_present,
- return_code}).
-
--record(mqtt_frame_publish, {topic_name,
- message_id}).
-
--record(mqtt_frame_subscribe,{message_id,
- topic_table}).
-
--record(mqtt_frame_suback, {message_id,
- qos_table = []}).
-
--record(mqtt_topic, {name,
- qos}).
-
--record(mqtt_frame_other, {other}).
-
--record(mqtt_msg, {retain :: boolean(),
- qos :: ?QOS_0 | ?QOS_1 | ?QOS_2,
- topic :: string(),
- dup :: boolean(),
- message_id :: message_id(),
- payload :: binary()}).
-
--type mqtt_msg() :: #mqtt_msg{}.
diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl
new file mode 100644
index 000000000000..d222611fda5e
--- /dev/null
+++ b/deps/rabbitmq_mqtt/include/rabbit_mqtt_packet.hrl
@@ -0,0 +1,114 @@
+%% 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) 2020-2023 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-type option(T) :: undefined | T.
+
+-define(PROTOCOL_NAMES,
+ [{3, <<"MQIsdp">>},
+ {4, <<"MQTT">>}]).
+
+%% packet types
+
+-define(CONNECT, 1).
+-define(CONNACK, 2).
+-define(PUBLISH, 3).
+-define(PUBACK, 4).
+-define(PUBREC, 5).
+-define(PUBREL, 6).
+-define(PUBCOMP, 7).
+-define(SUBSCRIBE, 8).
+-define(SUBACK, 9).
+-define(UNSUBSCRIBE, 10).
+-define(UNSUBACK, 11).
+-define(PINGREQ, 12).
+-define(PINGRESP, 13).
+-define(DISCONNECT, 14).
+
+-type packet_type() :: ?CONNECT .. ?DISCONNECT.
+
+%% connect return codes
+
+%% Connection accepted.
+-define(CONNACK_ACCEPT, 0).
+%% The Server does not support the level of the MQTT protocol requested by the Client.
+-define(CONNACK_UNACCEPTABLE_PROTO_VER, 1).
+%% The Client identifier is correct UTF-8 but not allowed by the Server.
+-define(CONNACK_ID_REJECTED, 2).
+%% The Network Connection has been made but the MQTT service is unavailable.
+-define(CONNACK_SERVER_UNAVAILABLE, 3).
+%% The data in the user name or password is malformed.
+-define(CONNACK_BAD_CREDENTIALS, 4).
+%% The Client is not authorized to connect.
+-define(CONNACK_NOT_AUTHORIZED, 5).
+
+%% qos levels
+
+-define(QOS_0, 0).
+-define(QOS_1, 1).
+-define(QOS_2, 2).
+-define(SUBACK_FAILURE, 16#80).
+
+-type qos() :: ?QOS_0 | ?QOS_1 | ?QOS_2.
+
+%% Packet identifier is a non zero two byte integer.
+-type packet_id() :: 1..16#ffff.
+
+-record(mqtt_packet_fixed, {type = 0,
+ dup = 0,
+ qos = 0,
+ retain = 0
+ }).
+
+-record(mqtt_packet, {fixed :: #mqtt_packet_fixed{},
+ variable :: option(tuple()),
+ payload :: option(iodata())
+ }).
+
+-type mqtt_packet() :: #mqtt_packet{}.
+
+-record(mqtt_packet_connect, {proto_ver,
+ will_retain,
+ will_qos,
+ will_flag,
+ clean_sess,
+ keep_alive,
+ client_id,
+ will_topic,
+ will_msg,
+ username,
+ password}).
+
+-record(mqtt_packet_connack, {session_present,
+ return_code}).
+
+-record(mqtt_packet_publish, {topic_name :: undefined | binary(),
+ packet_id :: packet_id()}).
+
+-record(mqtt_topic, {name :: binary(),
+ qos}).
+
+-record(mqtt_packet_subscribe,{packet_id :: packet_id(),
+ topic_table :: nonempty_list(#mqtt_topic{})
+ }).
+
+-record(mqtt_packet_suback, {packet_id :: packet_id(),
+ qos_table = []}).
+
+-record(mqtt_msg, {retain :: boolean(),
+ qos :: qos(),
+ topic :: binary(),
+ dup :: option(boolean()),
+ packet_id :: option(packet_id()),
+ payload :: binary()}).
+
+-type mqtt_msg() :: #mqtt_msg{}.
+
+%% does not include vhost because vhost is used in the (D)ETS table name
+-record(retained_message, {topic :: binary(),
+ mqtt_msg :: mqtt_msg()}).
+
+-type retained_message() :: #retained_message{}.
diff --git a/deps/rabbitmq_mqtt/include/rabbit_mqtt_retained_msg_store.hrl b/deps/rabbitmq_mqtt/include/rabbit_mqtt_retained_msg_store.hrl
deleted file mode 100644
index 278336b3b73c..000000000000
--- a/deps/rabbitmq_mqtt/include/rabbit_mqtt_retained_msg_store.hrl
+++ /dev/null
@@ -1,6 +0,0 @@
-%% This Source Code Form is subject to the terms of the Mozilla Public
-%% License, v. 2.0. If a copy of the MPL was not distributed with this
-%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
-%%
-%% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved.
-%%
diff --git a/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.DecommissionMqttNodeCommand.erl b/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.DecommissionMqttNodeCommand.erl
index cbe5ad8bb541..fa8e09341c6b 100644
--- a/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.DecommissionMqttNodeCommand.erl
+++ b/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.DecommissionMqttNodeCommand.erl
@@ -58,7 +58,6 @@ run([Node], #{node := NodeName,
" from the cluster", [Node]))};
Result ->
%% 'ok' or 'timeout'
- %% TODO: Ra will timeout if the node is not a cluster member - should this be fixed??
Result
end.
diff --git a/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl b/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl
index 1aca58265f10..07265b56e109 100644
--- a/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl
+++ b/deps/rabbitmq_mqtt/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand.erl
@@ -30,14 +30,15 @@ scopes() -> [ctl, diagnostics].
switches() -> [{verbose, boolean}].
aliases() -> [{'V', verbose}].
-description() -> <<"Lists MQTT connections on the target node">>.
+description() -> <<"Lists all MQTT connections">>.
help_section() ->
{plugin, mqtt}.
validate(Args, _) ->
+ InfoItems = lists:map(fun atom_to_list/1, ?INFO_ITEMS),
case 'Elixir.RabbitMQ.CLI.Ctl.InfoKeys':validate_info_keys(Args,
- ?INFO_ITEMS) of
+ InfoItems) of
{ok, _} -> ok;
Error -> Error
end.
@@ -68,9 +69,7 @@ run(Args, #{node := NodeName,
false -> 'Elixir.RabbitMQ.CLI.Ctl.InfoKeys':prepare_info_keys(Args)
end,
- %% a node uses the Raft-based collector to list connections, which knows about all connections in the cluster
- %% so no need to reach out to all the nodes
- Nodes = [NodeName],
+ Nodes = 'Elixir.RabbitMQ.CLI.Core.Helpers':nodes_in_cluster(NodeName),
'Elixir.RabbitMQ.CLI.Ctl.RpcStream':receive_list_items(
NodeName,
diff --git a/deps/rabbitmq_mqtt/src/mqtt_machine.erl b/deps/rabbitmq_mqtt/src/mqtt_machine.erl
index 4449d989cecc..20e69a7a2a8f 100644
--- a/deps/rabbitmq_mqtt/src/mqtt_machine.erl
+++ b/deps/rabbitmq_mqtt/src/mqtt_machine.erl
@@ -14,7 +14,8 @@
init/1,
apply/3,
state_enter/2,
- notify_connection/2]).
+ notify_connection/2,
+ overview/1]).
-type state() :: #machine_state{}.
@@ -171,6 +172,8 @@ apply(_Meta, Unknown, State) ->
logger:error("MQTT Raft state machine v1 received unknown command ~tp", [Unknown]),
{State, {error, {unknown_command, Unknown}}, []}.
+-spec state_enter(ra_server:ra_state() | eol, state()) ->
+ ra_machine:effects().
state_enter(leader, State) ->
%% re-request monitors for all known pids, this would clean up
%% records for all connections are no longer around, e.g. right after node restart
@@ -178,9 +181,16 @@ state_enter(leader, State) ->
state_enter(_, _) ->
[].
+-spec overview(state()) -> map().
+overview(#machine_state{client_ids = ClientIds,
+ pids = Pids}) ->
+ #{num_client_ids => maps:size(ClientIds),
+ num_pids => maps:size(Pids)}.
+
%% ==========================
%% Avoids blocking the Raft leader.
+-spec notify_connection(pid(), duplicate_id | decommission_node) -> pid().
notify_connection(Pid, Reason) ->
spawn(fun() -> gen_server2:cast(Pid, Reason) end).
diff --git a/deps/rabbitmq_mqtt/src/mqtt_machine_v0.erl b/deps/rabbitmq_mqtt/src/mqtt_machine_v0.erl
index c76c268c177d..4b32ac88dd98 100644
--- a/deps/rabbitmq_mqtt/src/mqtt_machine_v0.erl
+++ b/deps/rabbitmq_mqtt/src/mqtt_machine_v0.erl
@@ -113,6 +113,8 @@ apply(_Meta, Unknown, State) ->
logger:error("MQTT Raft state machine received an unknown command ~tp", [Unknown]),
{State, {error, {unknown_command, Unknown}}, []}.
+-spec state_enter(ra_server:ra_state(), state()) ->
+ ra_machine:effects().
state_enter(leader, State) ->
%% re-request monitors for all known pids, this would clean up
%% records for all connections are no longer around, e.g. right after node restart
@@ -123,6 +125,7 @@ state_enter(_, _) ->
%% ==========================
%% Avoids blocking the Raft leader.
+-spec notify_connection(pid(), duplicate_id | decommission_node) -> pid().
notify_connection(Pid, Reason) ->
spawn(fun() -> gen_server2:cast(Pid, Reason) end).
diff --git a/deps/rabbitmq_mqtt/src/mqtt_node.erl b/deps/rabbitmq_mqtt/src/mqtt_node.erl
index 4aaf5862f0f9..a6442fa85b90 100644
--- a/deps/rabbitmq_mqtt/src/mqtt_node.erl
+++ b/deps/rabbitmq_mqtt/src/mqtt_node.erl
@@ -6,12 +6,13 @@
%%
-module(mqtt_node).
--export([start/0, node_id/0, server_id/0, all_node_ids/0, leave/1, trigger_election/0]).
+-export([start/0, node_id/0, server_id/0, all_node_ids/0, leave/1, trigger_election/0,
+ delete/1]).
-define(ID_NAME, mqtt_node).
--define(START_TIMEOUT, 100000).
+-define(START_TIMEOUT, 100_000).
-define(RETRY_INTERVAL, 5000).
--define(RA_OPERATION_TIMEOUT, 60000).
+-define(RA_OPERATION_TIMEOUT, 60_000).
-define(RA_SYSTEM, coordination).
node_id() ->
@@ -33,7 +34,7 @@ start() ->
start(300, Repetitions).
start(_Delay, AttemptsLeft) when AttemptsLeft =< 0 ->
- start_server(),
+ ok = start_server(),
trigger_election();
start(Delay, AttemptsLeft) ->
NodeId = server_id(),
@@ -59,16 +60,15 @@ start(Delay, AttemptsLeft) ->
%% This is required when we start a node for the first time.
%% Using default timeout because it supposed to reply fast.
rabbit_log:info("MQTT: discovered ~tp cluster peers that support client ID tracking", [length(Peers)]),
- start_server(),
- join_peers(NodeId, Peers),
+ ok = start_server(),
+ _ = join_peers(NodeId, Peers),
ra:trigger_election(NodeId, ?RA_OPERATION_TIMEOUT)
end;
_ ->
- join_peers(NodeId, Nodes),
- ra:restart_server(?RA_SYSTEM, NodeId),
+ _ = join_peers(NodeId, Nodes),
+ ok = ra:restart_server(?RA_SYSTEM, NodeId),
ra:trigger_election(NodeId, ?RA_OPERATION_TIMEOUT)
- end,
- ok.
+ end.
compatible_peer_servers() ->
all_node_ids() -- [(node_id())].
@@ -130,3 +130,35 @@ can_participate_in_clientid_tracking(Node) ->
{badrpc, _} -> false;
_ -> true
end.
+
+-spec delete(Args) -> Ret when
+ Args :: rabbit_feature_flags:enable_callback_args(),
+ Ret :: rabbit_feature_flags:enable_callback_ret().
+delete(_) ->
+ RaNodes = all_node_ids(),
+ Nodes = lists:map(fun({_, N}) -> N end, RaNodes),
+ LockId = {?ID_NAME, node_id()},
+ rabbit_log:info("Trying to acquire lock ~p on nodes ~p ...", [LockId, Nodes]),
+ true = global:set_lock(LockId, Nodes),
+ rabbit_log:info("Acquired lock ~p", [LockId]),
+ try whereis(?ID_NAME) of
+ undefined ->
+ rabbit_log:info("Local Ra process ~s does not exist", [?ID_NAME]),
+ ok;
+ _ ->
+ rabbit_log:info("Deleting Ra cluster ~s ...", [?ID_NAME]),
+ try ra:delete_cluster(RaNodes, ?RA_OPERATION_TIMEOUT) of
+ {ok, _Leader} ->
+ rabbit_log:info("Successfully deleted Ra cluster ~s", [?ID_NAME]),
+ ok;
+ {error, _} = Err ->
+ rabbit_log:info("Failed to delete Ra cluster ~s: ~p", [?ID_NAME, Err]),
+ Err
+ catch exit:{{shutdown, delete}, _Stacktrace} ->
+ rabbit_log:info("Ra cluster ~s already being deleted", [?ID_NAME]),
+ ok
+ end
+ after
+ true = global:del_lock(LockId, Nodes),
+ rabbit_log:info("Released lock ~p", [LockId])
+ end.
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl
index dee18859f43e..5fb18612553d 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt.erl
@@ -8,14 +8,28 @@
-module(rabbit_mqtt).
-behaviour(application).
+
+-include("rabbit_mqtt.hrl").
+-include_lib("stdlib/include/assert.hrl").
+
-export([start/2, stop/1]).
-export([emit_connection_info_all/4,
- close_local_client_connections/1]).
+ emit_connection_info_local/3,
+ close_local_client_connections/1,
+ %% Exported for tests, but could also be used for debugging.
+ local_connection_pids/0]).
start(normal, []) ->
+ init_global_counters(),
+ persist_static_configuration(),
{ok, Listeners} = application:get_env(tcp_listeners),
{ok, SslListeners} = application:get_env(ssl_listeners),
- ok = mqtt_node:start(),
+ case rabbit_mqtt_ff:track_client_id_in_ra() of
+ true ->
+ ok = mqtt_node:start();
+ false ->
+ ok
+ end,
Result = rabbit_mqtt_sup:start_link({Listeners, SslListeners}, []),
EMPid = case rabbit_event:start_link() of
{ok, Pid} -> Pid;
@@ -27,19 +41,70 @@ start(normal, []) ->
stop(_) ->
rabbit_mqtt_sup:stop_listeners().
-emit_connection_info_all(_Nodes, Items, Ref, AggregatorPid) ->
+-spec emit_connection_info_all([node()], rabbit_types:info_keys(), reference(), pid()) -> term().
+emit_connection_info_all(Nodes, Items, Ref, AggregatorPid) ->
+ case rabbit_mqtt_ff:track_client_id_in_ra() of
+ true ->
+ %% Ra tracks connections cluster-wide.
+ AllPids = rabbit_mqtt_collector:list_pids(),
+ emit_connection_info(Items, Ref, AggregatorPid, AllPids),
+ %% Our node already emitted infos for all connections. Therefore, for the
+ %% remaining nodes, we send back 'finished' so that the CLI does not time out.
+ [AggregatorPid ! {Ref, finished} || _ <- lists:seq(1, length(Nodes) - 1)];
+ false ->
+ Pids = [spawn_link(Node, ?MODULE, emit_connection_info_local,
+ [Items, Ref, AggregatorPid])
+ || Node <- Nodes],
+ rabbit_control_misc:await_emitters_termination(Pids)
+ end.
+
+-spec emit_connection_info_local(rabbit_types:info_keys(), reference(), pid()) -> ok.
+emit_connection_info_local(Items, Ref, AggregatorPid) ->
+ LocalPids = local_connection_pids(),
+ emit_connection_info(Items, Ref, AggregatorPid, LocalPids).
+
+emit_connection_info(Items, Ref, AggregatorPid, Pids) ->
rabbit_control_misc:emitting_map_with_exit_handler(
- AggregatorPid,
- Ref,
+ AggregatorPid, Ref,
fun(Pid) ->
rabbit_mqtt_reader:info(Pid, Items)
- end,
- rabbit_mqtt_collector:list_pids()
- ).
+ end, Pids).
-spec close_local_client_connections(string() | binary()) -> {'ok', non_neg_integer()}.
close_local_client_connections(Reason) ->
- AllPids = rabbit_mqtt_collector:list_pids(),
- LocalPids = lists:filter(fun(Pid) -> node(Pid) =:= node() end, AllPids),
- [rabbit_mqtt_reader:close_connection(Pid, Reason) || Pid <- LocalPids],
- {ok, length(LocalPids)}.
+ Pids = local_connection_pids(),
+ lists:foreach(fun(Pid) ->
+ rabbit_mqtt_reader:close_connection(Pid, Reason)
+ end, Pids),
+ {ok, length(Pids)}.
+
+-spec local_connection_pids() -> [pid()].
+local_connection_pids() ->
+ case rabbit_mqtt_ff:track_client_id_in_ra() of
+ true ->
+ AllPids = rabbit_mqtt_collector:list_pids(),
+ lists:filter(fun(Pid) -> node(Pid) =:= node() end, AllPids);
+ false ->
+ PgScope = persistent_term:get(?PG_SCOPE),
+ lists:flatmap(fun(Group) ->
+ pg:get_local_members(PgScope, Group)
+ end, pg:which_groups(PgScope))
+ end.
+
+init_global_counters() ->
+ init_global_counters(?MQTT_PROTO_V3),
+ init_global_counters(?MQTT_PROTO_V4).
+
+init_global_counters(ProtoVer) ->
+ Proto = {protocol, ProtoVer},
+ rabbit_global_counters:init([Proto]),
+ rabbit_global_counters:init([Proto, {queue_type, ?QUEUE_TYPE_QOS_0}]),
+ rabbit_global_counters:init([Proto, {queue_type, rabbit_classic_queue}]),
+ rabbit_global_counters:init([Proto, {queue_type, rabbit_quorum_queue}]).
+
+persist_static_configuration() ->
+ rabbit_mqtt_util:init_sparkplug(),
+
+ {ok, MailboxSoftLimit} = application:get_env(?APP_NAME, mailbox_soft_limit),
+ ?assert(is_number(MailboxSoftLimit)),
+ ok = persistent_term:put(?PERSISTENT_TERM_MAILBOX_SOFT_LIMIT, MailboxSoftLimit).
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_collector.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_collector.erl
index 28c0506ef70b..5b5050d64efd 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_collector.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_collector.erl
@@ -36,6 +36,7 @@ register(ServerId, ClientId, Pid) ->
erlang:send_after(5000, self(), {ra_event, undefined, register_timeout}),
{ok, Corr}.
+-spec unregister(binary(), pid()) -> ok.
unregister(ClientId, Pid) ->
{ClusterName, _} = mqtt_node:server_id(),
case ra_leaderboard:lookup_leader(ClusterName) of
@@ -49,6 +50,7 @@ unregister(ClientId, Pid) ->
list_pids() ->
list(fun(#machine_state{pids = Pids}) -> maps:keys(Pids) end).
+-spec list() -> term().
list() ->
list(fun(#machine_state{client_ids = Ids}) -> maps:to_list(Ids) end).
@@ -76,6 +78,7 @@ list(QF) ->
end
end.
+-spec leave(binary()) -> ok | timeout | nodedown.
leave(NodeBin) ->
Node = binary_to_atom(NodeBin, utf8),
ServerId = mqtt_node:server_id(),
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl
new file mode 100644
index 000000000000..a31c94bd16df
--- /dev/null
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl
@@ -0,0 +1,98 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2020-2023 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_mqtt_confirms).
+
+-include("rabbit_mqtt_packet.hrl").
+-compile({no_auto_import, [size/1]}).
+
+-export([init/0,
+ insert/3,
+ confirm/3,
+ reject/2,
+ remove_queue/2,
+ size/1,
+ contains/2]).
+
+%% As done in OTP's sets module:
+%% Empty list is cheaper to serialize than atom.
+-define(VALUE, []).
+
+-type queue_name() :: rabbit_amqqueue:name().
+-opaque state() :: #{packet_id() => #{queue_name() => ?VALUE}}.
+-export_type([state/0]).
+
+-spec init() -> state().
+init() ->
+ maps:new().
+
+-spec size(state()) -> non_neg_integer().
+size(State) ->
+ maps:size(State).
+
+-spec contains(packet_id(), state()) -> boolean().
+contains(PktId, State) ->
+ maps:is_key(PktId, State).
+
+-spec insert(packet_id(), [queue_name()], state()) -> state().
+insert(PktId, QNames, State)
+ when is_integer(PktId) andalso
+ PktId > 0 andalso
+ not is_map_key(PktId, State) ->
+ QMap = maps:from_keys(QNames, ?VALUE),
+ maps:put(PktId, QMap, State).
+
+-spec confirm([packet_id()], queue_name(), state()) ->
+ {[packet_id()], state()}.
+confirm(PktIds, QName, State0) ->
+ {L0, State} = lists:foldl(fun(PktId, Acc) ->
+ confirm_one(PktId, QName, Acc)
+ end, {[], State0}, PktIds),
+ L = lists:reverse(L0),
+ {L, State}.
+
+-spec reject(packet_id(), state()) ->
+ {ok, state()} | {error, not_found}.
+reject(PktId, State0)
+ when is_integer(PktId) ->
+ case maps:take(PktId, State0) of
+ {_, State} ->
+ {ok, State};
+ error ->
+ {error, not_found}
+ end.
+
+%% idempotent
+-spec remove_queue(queue_name(), state()) ->
+ {[packet_id()], state()}.
+remove_queue(QName, State) ->
+ PktIds = maps:fold(
+ fun(PktId, QMap, PktIds)
+ when is_map_key(QName, QMap) ->
+ [PktId | PktIds];
+ (_, _, PktIds) ->
+ PktIds
+ end, [], State),
+ confirm(lists:sort(PktIds), QName, State).
+
+%% INTERNAL
+
+confirm_one(PktId, QName, {PktIds, State0})
+ when is_integer(PktId) ->
+ case maps:take(PktId, State0) of
+ {QMap0, State1}
+ when is_map_key(QName, QMap0)
+ andalso map_size(QMap0) =:= 1 ->
+ %% last queue confirm
+ {[PktId| PktIds], State1};
+ {QMap0, State1} ->
+ QMap = maps:remove(QName, QMap0),
+ State = maps:put(PktId, QMap, State1),
+ {PktIds, State};
+ error ->
+ {PktIds, State0}
+ end.
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_connection_info.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_connection_info.erl
deleted file mode 100644
index 2beb99b43916..000000000000
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_connection_info.erl
+++ /dev/null
@@ -1,25 +0,0 @@
-%% This Source Code Form is subject to the terms of the Mozilla Public
-%% License, v. 2.0. If a copy of the MPL was not distributed with this
-%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
-%%
-%% Copyright (c) 2017-2023 VMware, Inc. or its affiliates. All rights reserved.
-%%
--module(rabbit_mqtt_connection_info).
-
-%% Module to add the MQTT client ID to authentication properties
-
-%% API
--export([additional_authn_params/4]).
-
-additional_authn_params(_Creds, _VHost, _Pid, Infos) ->
- case proplists:get_value(variable_map, Infos, undefined) of
- VariableMap when is_map(VariableMap) ->
- case maps:get(<<"client_id">>, VariableMap, []) of
- ClientId when is_binary(ClientId)->
- [{client_id, ClientId}];
- [] ->
- []
- end;
- _ ->
- []
- end.
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_connection_sup.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_connection_sup.erl
deleted file mode 100644
index 3d4644ee4efc..000000000000
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_connection_sup.erl
+++ /dev/null
@@ -1,64 +0,0 @@
-%% This Source Code Form is subject to the terms of the Mozilla Public
-%% License, v. 2.0. If a copy of the MPL was not distributed with this
-%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
-%%
-%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
-%%
-
--module(rabbit_mqtt_connection_sup).
-
--behaviour(supervisor).
--behaviour(ranch_protocol).
-
--include_lib("rabbit_common/include/rabbit.hrl").
-
--export([start_link/3, start_keepalive_link/0]).
-
--export([init/1]).
-
-%%----------------------------------------------------------------------------
-
-start_link(Ref, _Transport, []) ->
- {ok, SupPid} = supervisor:start_link(?MODULE, []),
- {ok, KeepaliveSup} = supervisor:start_child(
- SupPid,
- #{
- id => rabbit_mqtt_keepalive_sup,
- start => {rabbit_mqtt_connection_sup, start_keepalive_link, []},
- restart => transient,
- significant => true,
- shutdown => infinity,
- type => supervisor,
- modules => [rabbit_keepalive_sup]
- }
- ),
- {ok, ReaderPid} = supervisor:start_child(
- SupPid,
- #{
- id => rabbit_mqtt_reader,
- start => {rabbit_mqtt_reader, start_link, [KeepaliveSup, Ref]},
- restart => transient,
- significant => true,
- shutdown => ?WORKER_WAIT,
- type => worker,
- modules => [rabbit_mqtt_reader]
- }
- ),
- {ok, SupPid, ReaderPid}.
-
-start_keepalive_link() ->
- supervisor:start_link(?MODULE, []).
-
-%%----------------------------------------------------------------------------
-
-init([]) ->
- {ok,
- {
- #{
- strategy => one_for_all,
- intensity => 0,
- period => 1,
- auto_shutdown => any_significant
- },
- []
- }}.
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl
new file mode 100644
index 000000000000..2432cc2ac5d0
--- /dev/null
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_ff.erl
@@ -0,0 +1,29 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2018-2023 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_mqtt_ff).
+
+-include("rabbit_mqtt.hrl").
+
+-export([track_client_id_in_ra/0]).
+
+-rabbit_feature_flag(
+ {?QUEUE_TYPE_QOS_0,
+ #{desc => "Support pseudo queue type for MQTT QoS 0 subscribers omitting a queue process",
+ stability => stable
+ }}).
+
+-rabbit_feature_flag(
+ {delete_ra_cluster_mqtt_node,
+ #{desc => "Delete Ra cluster 'mqtt_node' since MQTT client IDs are tracked locally",
+ stability => stable,
+ callbacks => #{enable => {mqtt_node, delete}}
+ }}).
+
+-spec track_client_id_in_ra() -> boolean().
+track_client_id_in_ra() ->
+ not rabbit_feature_flags:is_enabled(delete_ra_cluster_mqtt_node).
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_internal_event_handler.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_internal_event_handler.erl
index 6209bb32188e..711c065accf2 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_internal_event_handler.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_internal_event_handler.erl
@@ -9,37 +9,30 @@
-behaviour(gen_event).
--export([init/1, handle_event/2, handle_call/2, handle_info/2, terminate/2, code_change/3]).
+-export([init/1, handle_event/2, handle_call/2]).
-import(rabbit_misc, [pget/2]).
+-define(STATE, []).
+
init([]) ->
- {ok, []}.
-
-handle_event({event, vhost_created, Info, _, _}, State) ->
- Name = pget(name, Info),
- rabbit_mqtt_retainer_sup:child_for_vhost(Name),
- {ok, State};
-handle_event({event, vhost_deleted, Info, _, _}, State) ->
- Name = pget(name, Info),
- rabbit_mqtt_retainer_sup:delete_child(Name),
- {ok, State};
-handle_event({event, maintenance_connections_closed, _Info, _, _}, State) ->
+ {ok, ?STATE}.
+
+handle_event({event, vhost_created, Info, _, _}, ?STATE) ->
+ Name = pget(name, Info),
+ rabbit_mqtt_retainer_sup:child_for_vhost(Name),
+ {ok, ?STATE};
+handle_event({event, vhost_deleted, Info, _, _}, ?STATE) ->
+ Name = pget(name, Info),
+ rabbit_mqtt_retainer_sup:delete_child(Name),
+ {ok, ?STATE};
+handle_event({event, maintenance_connections_closed, _Info, _, _}, ?STATE) ->
%% we should close our connections
{ok, NConnections} = rabbit_mqtt:close_local_client_connections("node is being put into maintenance mode"),
rabbit_log:warning("Closed ~b local MQTT client connections", [NConnections]),
- {ok, State};
-handle_event(_Event, State) ->
- {ok, State}.
-
-handle_call(_Request, State) ->
- {ok, State}.
-
-handle_info(_Info, State) ->
- {ok, State}.
-
-terminate(_Reason, _State) ->
- ok.
+ {ok, ?STATE};
+handle_event(_Event, ?STATE) ->
+ {ok, ?STATE}.
-code_change(_OldVsn, State, _Extra) ->
- {ok, State}.
+handle_call(_Request, ?STATE) ->
+ {ok, ok, ?STATE}.
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl
new file mode 100644
index 000000000000..6b7b94b54c23
--- /dev/null
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_keepalive.erl
@@ -0,0 +1,114 @@
+-module(rabbit_mqtt_keepalive).
+
+-export([init/0,
+ start/2,
+ handle/2,
+ start_timer/1,
+ cancel_timer/1,
+ interval_secs/1]).
+
+-export_type([state/0]).
+
+-record(state, {
+ %% Keep Alive value as sent in the CONNECT packet.
+ interval_secs :: pos_integer(),
+ timer :: reference(),
+ socket :: inet:socket(),
+ recv_oct :: non_neg_integer(),
+ received :: boolean()}).
+
+-opaque(state() :: disabled | #state{}).
+
+-spec init() -> state().
+init() ->
+ disabled.
+
+-spec start(IntervalSeconds :: non_neg_integer(), inet:socket()) -> ok.
+start(0, _Sock) ->
+ ok;
+start(Seconds, Sock)
+ when is_integer(Seconds) andalso Seconds > 0 ->
+ self() ! {keepalive, {init, Seconds, Sock}},
+ ok.
+
+-spec handle(Request :: term(), state()) ->
+ {ok, state()} | {error, Reason :: term()}.
+handle({init, IntervalSecs, Sock}, _State) ->
+ case rabbit_net:getstat(Sock, [recv_oct]) of
+ {ok, [{recv_oct, RecvOct}]} ->
+ {ok, #state{interval_secs = IntervalSecs,
+ timer = start_timer0(IntervalSecs),
+ socket = Sock,
+ recv_oct = RecvOct,
+ received = true}};
+ {error, _} = Err ->
+ Err
+ end;
+handle(check, State = #state{socket = Sock,
+ recv_oct = SameRecvOct,
+ received = ReceivedPreviously}) ->
+ case rabbit_net:getstat(Sock, [recv_oct]) of
+ {ok, [{recv_oct, SameRecvOct}]}
+ when ReceivedPreviously ->
+ %% Did not receive from socket for the 1st time.
+ {ok, start_timer(State#state{received = false})};
+ {ok, [{recv_oct, SameRecvOct}]} ->
+ %% Did not receive from socket for 2nd time.
+ {error, timeout};
+ {ok, [{recv_oct, NewRecvOct}]} ->
+ %% Received from socket.
+ {ok, start_timer(State#state{recv_oct = NewRecvOct,
+ received = true})};
+ {error, _} = Err ->
+ Err
+ end.
+
+-spec start_timer(state()) -> state().
+start_timer(#state{interval_secs = Seconds} = State) ->
+ State#state{timer = start_timer0(Seconds)};
+start_timer(disabled) ->
+ disabled.
+
+-spec start_timer0(pos_integer()) -> reference().
+start_timer0(KeepAliveSeconds) ->
+ erlang:send_after(timer_ms(KeepAliveSeconds), self(), {keepalive, check}).
+
+-spec cancel_timer(state()) -> state().
+cancel_timer(#state{timer = Ref} = State)
+ when is_reference(Ref) ->
+ ok = erlang:cancel_timer(Ref, [{async, true},
+ {info, false}]),
+ State;
+cancel_timer(disabled) ->
+ disabled.
+
+%% "If the Keep Alive value is non-zero and the Server does not receive a Control
+%% Packet from the Client within one and a half times the Keep Alive time period,
+%% it MUST disconnect the Network Connection to the Client as if the network had
+%% failed" [MQTT-3.1.2-24]
+%%
+%% We check every (1.5 / 2 = 0.75) * KeepaliveInterval whether we received
+%% any data from the client. If there was no activity for two consecutive times,
+%% we close the connection.
+%% We choose 0.75 (instead of a larger or smaller factor) to have the right balance
+%% between not checking too often (since it could become expensive when there are
+%% millions of clients) and not checking too rarely (to detect dead clients promptly).
+%%
+%% See https://github.com/emqx/emqx/issues/460
+%% PING
+%% | DOWN
+%% | |<-------Delay Time--------->
+%% t0---->|----------|----------|----------|---->tn
+%% | | |
+%% Ok Retry Timeout
+-spec timer_ms(pos_integer()) ->
+ pos_integer().
+timer_ms(KeepaliveSeconds) ->
+ round(0.75 * timer:seconds(KeepaliveSeconds)).
+
+-spec interval_secs(state()) ->
+ non_neg_integer().
+interval_secs(#state{interval_secs = Seconds}) ->
+ Seconds;
+interval_secs(disabled) ->
+ 0.
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl
similarity index 53%
rename from deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl
rename to deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl
index b79d0bc45857..ededed8c5b01 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_frame.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_packet.erl
@@ -5,27 +5,35 @@
%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
%%
--module(rabbit_mqtt_frame).
+-module(rabbit_mqtt_packet).
--export([parse/2, initial_state/0]).
--export([serialise/1]).
+-include("rabbit_mqtt_packet.hrl").
+-include("rabbit_mqtt.hrl").
--include("rabbit_mqtt_frame.hrl").
+-export([parse/2, initial_state/0, serialise/2]).
+-export_type([state/0]).
+
+-opaque state() :: none | fun().
-define(RESERVED, 0).
-define(MAX_LEN, 16#fffffff).
-define(HIGHBIT, 2#10000000).
-define(LOWBITS, 2#01111111).
+-spec initial_state() -> state().
initial_state() -> none.
+-spec parse(binary(), state()) ->
+ {more, state()} |
+ {ok, mqtt_packet(), binary()} |
+ {error, any()}.
parse(<<>>, none) ->
{more, fun(Bin) -> parse(Bin, none) end};
parse(<>, none) ->
- parse_remaining_len(Rest, #mqtt_frame_fixed{ type = MessageType,
- dup = bool(Dup),
- qos = QoS,
- retain = bool(Retain) });
+ parse_remaining_len(Rest, #mqtt_packet_fixed{ type = MessageType,
+ dup = bool(Dup),
+ qos = QoS,
+ retain = bool(Retain) });
parse(Bin, Cont) -> Cont(Bin).
parse_remaining_len(<<>>, Fixed) ->
@@ -35,19 +43,20 @@ parse_remaining_len(Rest, Fixed) ->
parse_remaining_len(_Bin, _Fixed, _Multiplier, Length)
when Length > ?MAX_LEN ->
- {error, invalid_mqtt_frame_len};
+ {error, invalid_mqtt_packet_len};
parse_remaining_len(<<>>, Fixed, Multiplier, Length) ->
{more, fun(Bin) -> parse_remaining_len(Bin, Fixed, Multiplier, Length) end};
parse_remaining_len(<<1:1, Len:7, Rest/binary>>, Fixed, Multiplier, Value) ->
parse_remaining_len(Rest, Fixed, Multiplier * ?HIGHBIT, Value + Len * Multiplier);
parse_remaining_len(<<0:1, Len:7, Rest/binary>>, Fixed, Multiplier, Value) ->
- parse_frame(Rest, Fixed, Value + Len * Multiplier).
+ parse_packet(Rest, Fixed, Value + Len * Multiplier).
-parse_frame(Bin, #mqtt_frame_fixed{ type = Type,
- qos = Qos } = Fixed, Length) ->
+parse_packet(Bin, #mqtt_packet_fixed{ type = Type,
+ qos = Qos } = Fixed, Length)
+ when Length =< ?MAX_LEN ->
case {Type, Bin} of
- {?CONNECT, <>} ->
- {ProtoName, Rest1} = parse_utf(FrameBin),
+ {?CONNECT, <>} ->
+ {ProtoName, Rest1} = parse_utf(PacketBin),
<> = Rest1,
<
wrap(Fixed,
- #mqtt_frame_connect{
+ #mqtt_packet_connect{
proto_ver = ProtoVersion,
will_retain = bool(WillRetain),
will_qos = WillQos,
@@ -81,36 +90,37 @@ parse_frame(Bin, #mqtt_frame_fixed{ type = Type,
false ->
{error, protocol_header_corrupt}
end;
- {?PUBLISH, <>} ->
- {TopicName, Rest1} = parse_utf(FrameBin),
- {MessageId, Payload} = case Qos of
+ {?PUBLISH, <>} ->
+ {TopicName, Rest1} = parse_utf(PacketBin),
+ {PacketId, Payload} = case Qos of
0 -> {undefined, Rest1};
_ -> <> = Rest1,
{M, R}
end,
- wrap(Fixed, #mqtt_frame_publish { topic_name = TopicName,
- message_id = MessageId },
+ wrap(Fixed, #mqtt_packet_publish { topic_name = TopicName,
+ packet_id = PacketId },
Payload, Rest);
- {?PUBACK, <>} ->
- <> = FrameBin,
- wrap(Fixed, #mqtt_frame_publish { message_id = MessageId }, Rest);
- {Subs, <>}
+ {?PUBACK, <>} ->
+ <> = PacketBin,
+ wrap(Fixed, #mqtt_packet_publish { packet_id = PacketId }, Rest);
+ {Subs, <>}
when Subs =:= ?SUBSCRIBE orelse Subs =:= ?UNSUBSCRIBE ->
1 = Qos,
- <> = FrameBin,
+ <> = PacketBin,
Topics = parse_topics(Subs, Rest1, []),
- wrap(Fixed, #mqtt_frame_subscribe { message_id = MessageId,
- topic_table = Topics }, Rest);
+ wrap(Fixed, #mqtt_packet_subscribe { packet_id = PacketId,
+ topic_table = Topics }, Rest);
{Minimal, Rest}
when Minimal =:= ?DISCONNECT orelse Minimal =:= ?PINGREQ ->
Length = 0,
wrap(Fixed, Rest);
- {_, TooShortBin} ->
+ {_, TooShortBin}
+ when byte_size(TooShortBin) < Length ->
{more, fun(BinMore) ->
- parse_frame(<>,
- Fixed, Length)
+ parse_packet(<>,
+ Fixed, Length)
end}
- end.
+ end.
parse_topics(_, <<>>, Topics) ->
Topics;
@@ -122,11 +132,11 @@ parse_topics(?UNSUBSCRIBE = Sub, Bin, Topics) ->
parse_topics(Sub, Rest, [#mqtt_topic { name = Name } | Topics]).
wrap(Fixed, Variable, Payload, Rest) ->
- {ok, #mqtt_frame { variable = Variable, fixed = Fixed, payload = Payload }, Rest}.
+ {ok, #mqtt_packet { variable = Variable, fixed = Fixed, payload = Payload }, Rest}.
wrap(Fixed, Variable, Rest) ->
- {ok, #mqtt_frame { variable = Variable, fixed = Fixed }, Rest}.
+ {ok, #mqtt_packet { variable = Variable, fixed = Fixed }, Rest}.
wrap(Fixed, Rest) ->
- {ok, #mqtt_frame { fixed = Fixed }, Rest}.
+ {ok, #mqtt_packet { fixed = Fixed }, Rest}.
parse_utf(Bin, 0) ->
{undefined, Bin};
@@ -134,7 +144,7 @@ parse_utf(Bin, _) ->
parse_utf(Bin).
parse_utf(<>) ->
- {binary_to_list(Str), Rest}.
+ {Str, Rest}.
parse_msg(Bin, 0) ->
{undefined, Bin};
@@ -146,63 +156,74 @@ bool(1) -> true.
%% serialisation
-serialise(#mqtt_frame{ fixed = Fixed,
+-spec serialise(#mqtt_packet{}, ?MQTT_PROTO_V3 | ?MQTT_PROTO_V4) ->
+ iodata().
+serialise(#mqtt_packet{fixed = Fixed,
variable = Variable,
- payload = Payload }) ->
- serialise_variable(Fixed, Variable, serialise_payload(Payload)).
-
-serialise_payload(undefined) -> <<>>;
-serialise_payload(B) when is_binary(B) -> B.
-
-serialise_variable(#mqtt_frame_fixed { type = ?CONNACK } = Fixed,
- #mqtt_frame_connack { session_present = SessionPresent,
- return_code = ReturnCode },
- <<>> = PayloadBin) ->
+ payload = Payload}, Vsn) ->
+ serialise_variable(Fixed, Variable, serialise_payload(Payload), Vsn).
+
+serialise_payload(undefined) ->
+ <<>>;
+serialise_payload(P)
+ when is_binary(P) orelse is_list(P) ->
+ P.
+
+serialise_variable(#mqtt_packet_fixed { type = ?CONNACK } = Fixed,
+ #mqtt_packet_connack { session_present = SessionPresent,
+ return_code = ReturnCode },
+ <<>> = PayloadBin, _Vsn) ->
VariableBin = <>,
serialise_fixed(Fixed, VariableBin, PayloadBin);
-serialise_variable(#mqtt_frame_fixed { type = SubAck } = Fixed,
- #mqtt_frame_suback { message_id = MessageId,
- qos_table = Qos },
- <<>> = _PayloadBin)
+serialise_variable(#mqtt_packet_fixed { type = SubAck } = Fixed,
+ #mqtt_packet_suback { packet_id = PacketId,
+ qos_table = Qos },
+ <<>> = _PayloadBin, Vsn)
when SubAck =:= ?SUBACK orelse SubAck =:= ?UNSUBACK ->
- VariableBin = <>,
- QosBin = << <> || Q <- Qos >>,
+ VariableBin = <>,
+ QosBin = case Vsn of
+ ?MQTT_PROTO_V3 ->
+ << <> || Q <- Qos >>;
+ ?MQTT_PROTO_V4 ->
+ %% Allow error code (0x80) in the MQTT SUBACK message.
+ << <> || Q <- Qos >>
+ end,
serialise_fixed(Fixed, VariableBin, QosBin);
-serialise_variable(#mqtt_frame_fixed { type = ?PUBLISH,
- qos = Qos } = Fixed,
- #mqtt_frame_publish { topic_name = TopicName,
- message_id = MessageId },
- PayloadBin) ->
+serialise_variable(#mqtt_packet_fixed { type = ?PUBLISH,
+ qos = Qos } = Fixed,
+ #mqtt_packet_publish { topic_name = TopicName,
+ packet_id = PacketId },
+ Payload, _Vsn) ->
TopicBin = serialise_utf(TopicName),
- MessageIdBin = case Qos of
+ PacketIdBin = case Qos of
0 -> <<>>;
- 1 -> <>
+ 1 -> <>
end,
- serialise_fixed(Fixed, <>, PayloadBin);
+ serialise_fixed(Fixed, <>, Payload);
-serialise_variable(#mqtt_frame_fixed { type = ?PUBACK } = Fixed,
- #mqtt_frame_publish { message_id = MessageId },
- PayloadBin) ->
- MessageIdBin = <>,
- serialise_fixed(Fixed, MessageIdBin, PayloadBin);
+serialise_variable(#mqtt_packet_fixed { type = ?PUBACK } = Fixed,
+ #mqtt_packet_publish { packet_id = PacketId },
+ PayloadBin, _Vsn) ->
+ PacketIdBin = <>,
+ serialise_fixed(Fixed, PacketIdBin, PayloadBin);
-serialise_variable(#mqtt_frame_fixed {} = Fixed,
+serialise_variable(#mqtt_packet_fixed {} = Fixed,
undefined,
- <<>> = _PayloadBin) ->
+ <<>> = _PayloadBin, _Vsn) ->
serialise_fixed(Fixed, <<>>, <<>>).
-serialise_fixed(#mqtt_frame_fixed{ type = Type,
- dup = Dup,
- qos = Qos,
- retain = Retain }, VariableBin, PayloadBin)
+serialise_fixed(#mqtt_packet_fixed{ type = Type,
+ dup = Dup,
+ qos = Qos,
+ retain = Retain }, VariableBin, Payload)
when is_integer(Type) andalso ?CONNECT =< Type andalso Type =< ?DISCONNECT ->
- Len = size(VariableBin) + size(PayloadBin),
+ Len = size(VariableBin) + iolist_size(Payload),
true = (Len =< ?MAX_LEN),
LenBin = serialise_len(Len),
- <>.
+ [<>, Payload].
serialise_utf(String) ->
StringBin = unicode:characters_to_binary(String),
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl
index a78fb6c3c75d..58bccaa2f5fd 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl
@@ -5,652 +5,873 @@
%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
%%
+%% This module contains code that is common to MQTT and Web MQTT connections.
-module(rabbit_mqtt_processor).
--export([info/2, initial_state/2, initial_state/5,
- process_frame/2, amqp_pub/2, amqp_callback/2, send_will/1,
- close_connection/1, handle_pre_hibernate/0,
- handle_ra_event/2]).
+-export([info/2, initial_state/2, initial_state/4,
+ process_packet/2, serialise/2,
+ terminate/4, handle_pre_hibernate/0,
+ handle_ra_event/2, handle_down/2, handle_queue_event/2,
+ proto_version_tuple/1, throttle/3, format_status/1,
+ update_trace/2]).
%% for testing purposes
--export([get_vhost_username/1, get_vhost/3, get_vhost_from_user_mapping/2,
- add_client_id_to_adapter_info/2, maybe_quorum/3]).
+-export([get_vhost_username/1, get_vhost/3, get_vhost_from_user_mapping/2]).
--include_lib("amqp_client/include/amqp_client.hrl").
--include("rabbit_mqtt_frame.hrl").
--include("rabbit_mqtt.hrl").
-
--define(APP, rabbitmq_mqtt).
--define(FRAME_TYPE(Frame, Type),
- Frame = #mqtt_frame{ fixed = #mqtt_frame_fixed{ type = Type }}).
--define(MAX_TOPIC_PERMISSION_CACHE_SIZE, 12).
-
-initial_state(Socket, SSLLoginName) ->
- RealSocket = rabbit_net:unwrap_socket(Socket),
- {ok, {PeerAddr, _PeerPort}} = rabbit_net:peername(RealSocket),
- initial_state(RealSocket, SSLLoginName,
- adapter_info(Socket, 'MQTT'),
- fun serialise_and_send_to_client/2, PeerAddr).
-
-initial_state(Socket, SSLLoginName,
- AdapterInfo0 = #amqp_adapter_info{additional_info = Extra},
- SendFun, PeerAddr) ->
- {ok, {mqtt2amqp_fun, M2A}, {amqp2mqtt_fun, A2M}} =
- rabbit_mqtt_util:get_topic_translation_funs(),
- %% MQTT connections use exactly one channel. The frame max is not
- %% applicable and there is no way to know what client is used.
- AdapterInfo = AdapterInfo0#amqp_adapter_info{additional_info = [
- {channels, 1},
- {channel_max, 1},
- {frame_max, 0},
- {client_properties,
- [{<<"product">>, longstr, <<"MQTT client">>}]} | Extra]},
- #proc_state{ unacked_pubs = gb_trees:empty(),
- awaiting_ack = gb_trees:empty(),
- message_id = 1,
- subscriptions = #{},
- consumer_tags = {undefined, undefined},
- channels = {undefined, undefined},
- exchange = rabbit_mqtt_util:env(exchange),
- socket = Socket,
- adapter_info = AdapterInfo,
- ssl_login_name = SSLLoginName,
- send_fun = SendFun,
- peer_addr = PeerAddr,
- mqtt2amqp_fun = M2A,
- amqp2mqtt_fun = A2M}.
-
-process_frame(#mqtt_frame{ fixed = #mqtt_frame_fixed{ type = Type }},
- PState = #proc_state{ connection = undefined } )
- when Type =/= ?CONNECT ->
- {error, connect_expected, PState};
-process_frame(Frame = #mqtt_frame{ fixed = #mqtt_frame_fixed{ type = Type }},
- PState) ->
- try process_request(Type, Frame, PState) of
- {ok, PState1} -> {ok, PState1, PState1#proc_state.connection};
- Ret -> Ret
- catch
- _:{{shutdown, {server_initiated_close, 403, _}}, _} ->
- %% NB: MQTT spec says we should ack normally, ie pretend
- %% there was no auth error, but here we are closing the
- %% connection with an error. This is what happens anyway
- %% if there is an authorization failure at the AMQP 0-9-1
- %% client level. And error was already logged by AMQP
- %% channel, so no need for custom logging.
- {error, access_refused, PState}
- end.
+-export_type([state/0]).
-add_client_id_to_adapter_info(ClientId, #amqp_adapter_info{additional_info = AdditionalInfo0} = AdapterInfo) ->
- AdditionalInfo1 = [{variable_map, #{<<"client_id">> => ClientId}}
- | AdditionalInfo0],
- ClientProperties = proplists:get_value(client_properties, AdditionalInfo1, [])
- ++ [{client_id, longstr, ClientId}],
- AdditionalInfo2 = case lists:keysearch(client_properties, 1, AdditionalInfo1) of
- {value, _} ->
- lists:keyreplace(client_properties,
- 1,
- AdditionalInfo1,
- {client_properties, ClientProperties});
- false ->
- [{client_properties, ClientProperties} | AdditionalInfo1]
- end,
- AdapterInfo#amqp_adapter_info{additional_info = AdditionalInfo2}.
-
-process_connect(#mqtt_frame{ variable = #mqtt_frame_connect{
- username = Username,
- password = Password,
- proto_ver = ProtoVersion,
- clean_sess = CleanSess,
- client_id = ClientId0,
- keep_alive = Keepalive} = Var},
- PState0 = #proc_state{ ssl_login_name = SSLLoginName,
- send_fun = SendFun,
- adapter_info = AdapterInfo,
- peer_addr = Addr}) ->
- ClientId = case ClientId0 of
- [] -> rabbit_mqtt_util:gen_client_id();
- [_|_] -> ClientId0
- end,
- rabbit_log_connection:debug("Received a CONNECT, client ID: ~tp (expanded to ~tp), username: ~tp, "
- "clean session: ~tp, protocol version: ~tp, keepalive: ~tp",
- [ClientId0, ClientId, Username, CleanSess, ProtoVersion, Keepalive]),
- AdapterInfo1 = add_client_id_to_adapter_info(rabbit_data_coercion:to_binary(ClientId), AdapterInfo),
- PState1 = PState0#proc_state{adapter_info = AdapterInfo1},
- Ip = list_to_binary(inet:ntoa(Addr)),
- {Return, PState5} =
- case {lists:member(ProtoVersion, proplists:get_keys(?PROTOCOL_NAMES)),
- ClientId0 =:= [] andalso CleanSess =:= false} of
- {false, _} ->
- {?CONNACK_PROTO_VER, PState1};
- {_, true} ->
- {?CONNACK_INVALID_ID, PState1};
- _ ->
- case creds(Username, Password, SSLLoginName) of
- nocreds ->
- rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt),
- rabbit_log_connection:error("MQTT login failed: no credentials provided"),
- {?CONNACK_CREDENTIALS, PState1};
- {invalid_creds, {undefined, Pass}} when is_list(Pass) ->
- rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt),
- rabbit_log_connection:error("MQTT login failed: no username is provided"),
- {?CONNACK_CREDENTIALS, PState1};
- {invalid_creds, {User, undefined}} when is_list(User) ->
- rabbit_core_metrics:auth_attempt_failed(Ip, User, mqtt),
- rabbit_log_connection:error("MQTT login failed for user '~tp': no password provided", [User]),
- {?CONNACK_CREDENTIALS, PState1};
- {UserBin, PassBin} ->
- case process_login(UserBin, PassBin, ProtoVersion, PState1) of
- connack_dup_auth ->
- maybe_clean_sess(PState1);
- {?CONNACK_ACCEPT, Conn, VHost, AState} ->
- case rabbit_mqtt_collector:register(ClientId, self()) of
- {ok, Corr} ->
- RetainerPid = rabbit_mqtt_retainer_sup:child_for_vhost(VHost),
- link(Conn),
- {ok, Ch} = amqp_connection:open_channel(Conn),
- link(Ch),
- amqp_channel:enable_delivery_flow_control(Ch),
- Prefetch = rabbit_mqtt_util:env(prefetch),
- #'basic.qos_ok'{} = amqp_channel:call(Ch,
- #'basic.qos'{prefetch_count = Prefetch}),
- rabbit_mqtt_reader:start_keepalive(self(), Keepalive),
- PState3 = PState1#proc_state{
- will_msg = make_will_msg(Var),
- clean_sess = CleanSess,
- channels = {Ch, undefined},
- connection = Conn,
- client_id = ClientId,
- retainer_pid = RetainerPid,
- auth_state = AState,
- register_state = {pending, Corr}},
- maybe_clean_sess(PState3);
- %% e.g. this node was removed from the MQTT cluster members
- {error, _} = Err ->
- rabbit_log_connection:error("MQTT cannot accept a connection: "
- "client ID tracker is unavailable: ~tp", [Err]),
- %% ignore all exceptions, we are shutting down
- catch amqp_connection:close(Conn),
- {?CONNACK_SERVER, PState1};
- {timeout, _} ->
- rabbit_log_connection:error("MQTT cannot accept a connection: "
- "client ID registration timed out"),
- %% ignore all exceptions, we are shutting down
- catch amqp_connection:close(Conn),
- {?CONNACK_SERVER, PState1}
- end;
- ConnAck -> {ConnAck, PState1}
- end
- end
- end,
- {ReturnCode, SessionPresent} = case Return of
- {?CONNACK_ACCEPT, Bool} -> {?CONNACK_ACCEPT, Bool};
- Other -> {Other, false}
- end,
- SendFun(#mqtt_frame{fixed = #mqtt_frame_fixed{type = ?CONNACK},
- variable = #mqtt_frame_connack{
- session_present = SessionPresent,
- return_code = ReturnCode}},
- PState5),
- case ReturnCode of
- ?CONNACK_ACCEPT -> {ok, PState5};
- ?CONNACK_CREDENTIALS -> {error, unauthenticated, PState5};
- ?CONNACK_AUTH -> {error, unauthorized, PState5};
- ?CONNACK_SERVER -> {error, unavailable, PState5};
- ?CONNACK_INVALID_ID -> {error, invalid_client_id, PState5};
- ?CONNACK_PROTO_VER -> {error, unsupported_protocol_version, PState5}
- end.
+-import(rabbit_mqtt_util, [mqtt_to_amqp/1,
+ amqp_to_mqtt/1]).
-process_request(?CONNECT, Frame, PState = #proc_state{socket = Socket}) ->
+-include_lib("kernel/include/logger.hrl").
+-include_lib("rabbit_common/include/rabbit.hrl").
+-include_lib("rabbit_common/include/rabbit_framing.hrl").
+-include_lib("rabbit/include/amqqueue.hrl").
+-include("rabbit_mqtt.hrl").
+-include("rabbit_mqtt_packet.hrl").
+
+-define(MAX_PERMISSION_CACHE_SIZE, 12).
+-define(CONSUMER_TAG, <<"mqtt">>).
+
+-record(auth_state,
+ {username :: rabbit_types:username(),
+ user :: #user{},
+ vhost :: rabbit_types:vhost(),
+ authz_ctx :: #{binary() := binary()}
+ }).
+
+-record(cfg,
+ {socket :: rabbit_net:socket(),
+ proto_ver :: option(mqtt310 | mqtt311),
+ clean_sess :: option(boolean()),
+ will_msg :: option(mqtt_msg()),
+ exchange :: option(rabbit_exchange:name()),
+ %% Set if client has at least one subscription with QoS 1.
+ queue_qos1 :: option(rabbit_amqqueue:name()),
+ %% Did the client ever sent us a PUBLISH packet?
+ published = false :: boolean(),
+ ssl_login_name :: none | binary(),
+ retainer_pid :: option(pid()),
+ delivery_flow :: flow | noflow,
+ trace_state :: option(rabbit_trace:state()),
+ prefetch :: non_neg_integer(),
+ client_id :: option(binary()),
+ conn_name :: option(binary()),
+ peer_addr :: inet:ip_address(),
+ host :: inet:ip_address(),
+ port :: inet:port_number(),
+ peer_host :: inet:ip_address(),
+ peer_port :: inet:port_number(),
+ connected_at :: pos_integer(),
+ send_fun :: fun((Packet :: tuple(), state()) -> term())
+ }).
+
+-record(state,
+ {cfg :: #cfg{},
+ queue_states = rabbit_queue_type:init() :: rabbit_queue_type:state(),
+ %% Packet IDs published to queues but not yet confirmed.
+ unacked_client_pubs = rabbit_mqtt_confirms:init() :: rabbit_mqtt_confirms:state(),
+ %% Packet IDs published to MQTT subscribers but not yet acknowledged.
+ unacked_server_pubs = #{} :: #{packet_id() => QueueMsgId :: non_neg_integer()},
+ %% Packet ID of next PUBLISH packet (with QoS > 0) sent from server to client.
+ %% (Not to be confused with packet IDs sent from client to server which can be the
+ %% same IDs because client and server assign IDs independently of each other.)
+ packet_id = 1 :: packet_id(),
+ subscriptions = #{} :: #{Topic :: binary() => QoS :: ?QOS_0..?QOS_1},
+ auth_state :: option(#auth_state{}),
+ register_state :: option(registered | {pending, reference()}),
+ %% quorum queues and streams whose soft limit has been exceeded
+ queues_soft_limit_exceeded = sets:new([{version, 2}]) :: sets:set(),
+ qos0_messages_dropped = 0 :: non_neg_integer()
+ }).
+
+-opaque state() :: #state{}.
+
+-spec initial_state(Socket :: any(), ConnectionName :: binary()) ->
+ state().
+initial_state(Socket, ConnectionName) ->
+ {ok, {PeerAddr, _PeerPort}} = rabbit_net:peername(Socket),
+ initial_state(Socket,
+ ConnectionName,
+ fun serialise_and_send_to_client/2,
+ PeerAddr).
+
+-spec initial_state(Socket :: any(),
+ ConnectionName :: binary(),
+ SendFun :: fun((mqtt_packet(), state()) -> any()),
+ PeerAddr :: inet:ip_address()) ->
+ state().
+initial_state(Socket, ConnectionName, SendFun, PeerAddr) ->
+ Flow = case rabbit_misc:get_env(rabbit, mirroring_flow_control, true) of
+ true -> flow;
+ false -> noflow
+ end,
+ {ok, {PeerHost, PeerPort, Host, Port}} = rabbit_net:socket_ends(Socket, inbound),
+ #state{cfg = #cfg{socket = Socket,
+ conn_name = ConnectionName,
+ ssl_login_name = ssl_login_name(Socket),
+ send_fun = SendFun,
+ prefetch = rabbit_mqtt_util:env(prefetch),
+ delivery_flow = Flow,
+ connected_at = os:system_time(milli_seconds),
+ peer_addr = PeerAddr,
+ peer_host = PeerHost,
+ peer_port = PeerPort,
+ host = Host,
+ port = Port}}.
+
+-spec process_packet(mqtt_packet(), state()) ->
+ {ok, state()} |
+ {stop, disconnect, state()} |
+ {error, Reason :: term(), state()}.
+process_packet(#mqtt_packet{fixed = #mqtt_packet_fixed{type = Type}},
+ State = #state{auth_state = undefined})
+ when Type =/= ?CONNECT ->
+ {error, connect_expected, State};
+process_packet(Packet = #mqtt_packet{fixed = #mqtt_packet_fixed{type = Type}}, State) ->
+ process_request(Type, Packet, State).
+
+-spec process_request(packet_type(), mqtt_packet(), state()) ->
+ {ok, state()} |
+ {stop, disconnect, state()} |
+ {error, Reason :: term(), state()}.
+process_request(?CONNECT, Packet, State = #state{cfg = #cfg{socket = Socket}}) ->
%% Check whether peer closed the connection.
%% For example, this can happen when connection was blocked because of resource
%% alarm and client therefore disconnected due to client side CONNACK timeout.
case rabbit_net:socket_ends(Socket, inbound) of
{error, Reason} ->
- {error, {socket_ends, Reason}, PState};
+ {error, {socket_ends, Reason}, State};
_ ->
- process_connect(Frame, PState)
+ process_connect(Packet, State)
end;
process_request(?PUBACK,
- #mqtt_frame{
- variable = #mqtt_frame_publish{ message_id = MessageId }},
- #proc_state{ channels = {Channel, _},
- awaiting_ack = Awaiting } = PState) ->
- %% tag can be missing because of bogus clients and QoS downgrades
- case gb_trees:is_defined(MessageId, Awaiting) of
- false ->
- {ok, PState};
- true ->
- Tag = gb_trees:get(MessageId, Awaiting),
- amqp_channel:cast(Channel, #'basic.ack'{ delivery_tag = Tag }),
- {ok, PState#proc_state{ awaiting_ack = gb_trees:delete(MessageId, Awaiting) }}
+ #mqtt_packet{variable = #mqtt_packet_publish{packet_id = PacketId}},
+ #state{unacked_server_pubs = U0,
+ queue_states = QStates0,
+ cfg = #cfg{queue_qos1 = QName}} = State) ->
+ case maps:take(PacketId, U0) of
+ {QMsgId, U} ->
+ case rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0) of
+ {ok, QStates, Actions} ->
+ message_acknowledged(QName, State),
+ {ok, handle_queue_actions(Actions, State#state{unacked_server_pubs = U,
+ queue_states = QStates})};
+ {protocol_error, _ErrorType, _Reason, _ReasonArgs} = Err ->
+ {error, Err, State}
+ end;
+ error ->
+ {ok, State}
end;
process_request(?PUBLISH,
- Frame = #mqtt_frame{
- fixed = Fixed = #mqtt_frame_fixed{ qos = ?QOS_2 }},
- PState) ->
+ Packet = #mqtt_packet{
+ fixed = Fixed = #mqtt_packet_fixed{qos = ?QOS_2}},
+ State) ->
% Downgrade QOS_2 to QOS_1
process_request(?PUBLISH,
- Frame#mqtt_frame{
- fixed = Fixed#mqtt_frame_fixed{ qos = ?QOS_1 }},
- PState);
+ Packet#mqtt_packet{
+ fixed = Fixed#mqtt_packet_fixed{qos = ?QOS_1}},
+ State);
process_request(?PUBLISH,
- #mqtt_frame{
- fixed = #mqtt_frame_fixed{ qos = Qos,
- retain = Retain,
- dup = Dup },
- variable = #mqtt_frame_publish{ topic_name = Topic,
- message_id = MessageId },
- payload = Payload },
- PState = #proc_state{retainer_pid = RPid,
- amqp2mqtt_fun = Amqp2MqttFun}) ->
- check_publish(Topic, fun() ->
- Msg = #mqtt_msg{retain = Retain,
- qos = Qos,
- topic = Topic,
- dup = Dup,
- message_id = MessageId,
- payload = Payload},
- Result = amqp_pub(Msg, PState),
- case Retain of
- false -> ok;
- true -> hand_off_to_retainer(RPid, Amqp2MqttFun, Topic, Msg)
- end,
- {ok, Result}
- end, PState);
-
-process_request(?SUBSCRIBE,
- #mqtt_frame{
- variable = #mqtt_frame_subscribe{
- message_id = SubscribeMsgId,
- topic_table = Topics},
- payload = undefined},
- #proc_state{channels = {Channel, _},
- exchange = Exchange,
- retainer_pid = RPid,
- send_fun = SendFun,
- message_id = StateMsgId,
- mqtt2amqp_fun = Mqtt2AmqpFun} = PState0) ->
- rabbit_log_connection:debug("Received a SUBSCRIBE for topic(s) ~tp", [Topics]),
-
- {QosResponse, PState1} =
- lists:foldl(fun (#mqtt_topic{name = TopicName,
- qos = Qos}, {QosList, PState}) ->
- SupportedQos = supported_subs_qos(Qos),
- {Queue, #proc_state{subscriptions = Subs} = PState1} =
- ensure_queue(SupportedQos, PState),
- RoutingKey = Mqtt2AmqpFun(TopicName),
- Binding = #'queue.bind'{
- queue = Queue,
- exchange = Exchange,
- routing_key = RoutingKey},
- #'queue.bind_ok'{} = amqp_channel:call(Channel, Binding),
- SupportedQosList = case maps:find(TopicName, Subs) of
- {ok, L} -> [SupportedQos|L];
- error -> [SupportedQos]
- end,
- {[SupportedQos | QosList],
- PState1 #proc_state{
- subscriptions =
- maps:put(TopicName, SupportedQosList, Subs)}}
- end, {[], PState0}, Topics),
- SendFun(#mqtt_frame{fixed = #mqtt_frame_fixed{type = ?SUBACK},
- variable = #mqtt_frame_suback{
- message_id = SubscribeMsgId,
- qos_table = QosResponse}}, PState1),
- %% we may need to send up to length(Topics) messages.
- %% if QoS is > 0 then we need to generate a message id,
- %% and increment the counter.
- StartMsgId = safe_max_id(SubscribeMsgId, StateMsgId),
- N = lists:foldl(fun (Topic, Acc) ->
- case maybe_send_retained_message(RPid, Topic, Acc, PState1) of
- {true, X} -> Acc + X;
- false -> Acc
+ #mqtt_packet{
+ fixed = #mqtt_packet_fixed{qos = Qos,
+ retain = Retain,
+ dup = Dup },
+ variable = #mqtt_packet_publish{topic_name = Topic,
+ packet_id = PacketId },
+ payload = Payload},
+ State0 = #state{unacked_client_pubs = U,
+ cfg = #cfg{retainer_pid = RPid,
+ proto_ver = ProtoVer}}) ->
+ rabbit_global_counters:messages_received(ProtoVer, 1),
+ State = maybe_increment_publisher(State0),
+ Publish = fun() ->
+ Msg = #mqtt_msg{retain = Retain,
+ qos = Qos,
+ topic = Topic,
+ dup = Dup,
+ packet_id = PacketId,
+ payload = Payload},
+ case publish_to_queues(Msg, State) of
+ {ok, _} = Ok ->
+ case Retain of
+ false ->
+ ok;
+ true ->
+ hand_off_to_retainer(RPid, Topic, Msg)
+ end,
+ Ok;
+ Error ->
+ Error
end
- end, StartMsgId, Topics),
- {ok, PState1#proc_state{message_id = N}};
+ end,
+ case Qos of
+ N when N > ?QOS_0 ->
+ rabbit_global_counters:messages_received_confirm(ProtoVer, 1),
+ case rabbit_mqtt_confirms:contains(PacketId, U) of
+ false ->
+ publish_to_queues_with_checks(Topic, Publish, State);
+ true ->
+ %% Client re-sent this PUBLISH packet.
+ %% We already sent this message to target queues awaiting confirmations.
+ %% Hence, we ignore this re-send.
+ {ok, State}
+ end;
+ _ ->
+ publish_to_queues_with_checks(Topic, Publish, State)
+ end;
-process_request(?UNSUBSCRIBE,
- #mqtt_frame{
- variable = #mqtt_frame_subscribe{ message_id = MessageId,
- topic_table = Topics },
- payload = undefined }, #proc_state{ channels = {Channel, _},
- exchange = Exchange,
- client_id = ClientId,
- subscriptions = Subs0,
- send_fun = SendFun,
- mqtt2amqp_fun = Mqtt2AmqpFun } = PState) ->
- rabbit_log_connection:debug("Received an UNSUBSCRIBE for topic(s) ~tp", [Topics]),
- Queues = rabbit_mqtt_util:subcription_queue_name(ClientId),
- Subs1 =
+process_request(?SUBSCRIBE,
+ #mqtt_packet{
+ variable = #mqtt_packet_subscribe{
+ packet_id = SubscribePktId,
+ topic_table = Topics},
+ payload = undefined},
+ #state{cfg = #cfg{send_fun = SendFun,
+ retainer_pid = RPid}} = State0) ->
+ ?LOG_DEBUG("Received a SUBSCRIBE for topic(s) ~p", [Topics]),
+ {QosResponse, State1} =
lists:foldl(
- fun (#mqtt_topic{ name = TopicName }, Subs) ->
- QosSubs = case maps:find(TopicName, Subs) of
- {ok, Val} when is_list(Val) -> lists:usort(Val);
- error -> []
- end,
- RoutingKey = Mqtt2AmqpFun(TopicName),
- lists:foreach(
- fun (QosSub) ->
- Queue = element(QosSub + 1, Queues),
- Binding = #'queue.unbind'{
- queue = Queue,
- exchange = Exchange,
- routing_key = RoutingKey},
- #'queue.unbind_ok'{} = amqp_channel:call(Channel, Binding)
- end, QosSubs),
- maps:remove(TopicName, Subs)
- end, Subs0, Topics),
- SendFun(#mqtt_frame{ fixed = #mqtt_frame_fixed { type = ?UNSUBACK },
- variable = #mqtt_frame_suback{ message_id = MessageId }},
- PState),
- {ok, PState #proc_state{ subscriptions = Subs1 }};
-
-process_request(?PINGREQ, #mqtt_frame{}, #proc_state{ send_fun = SendFun } = PState) ->
- rabbit_log_connection:debug("Received a PINGREQ"),
- SendFun(#mqtt_frame{ fixed = #mqtt_frame_fixed{ type = ?PINGRESP }},
- PState),
- rabbit_log_connection:debug("Sent a PINGRESP"),
- {ok, PState};
-
-process_request(?DISCONNECT, #mqtt_frame{}, PState) ->
- rabbit_log_connection:debug("Received a DISCONNECT"),
- {stop, PState}.
-
-hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, #mqtt_msg{payload = <<"">>}) ->
- Topic1 = Amqp2MqttFun(Topic0),
- rabbit_mqtt_retainer:clear(RetainerPid, Topic1),
- ok;
-hand_off_to_retainer(RetainerPid, Amqp2MqttFun, Topic0, Msg) ->
- Topic1 = Amqp2MqttFun(Topic0),
- rabbit_mqtt_retainer:retain(RetainerPid, Topic1, Msg),
- ok.
-
-maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos}, MsgId,
- #proc_state{ send_fun = SendFun,
- amqp2mqtt_fun = Amqp2MqttFun } = PState) ->
- Topic1 = Amqp2MqttFun(Topic0),
- case rabbit_mqtt_retainer:fetch(RPid, Topic1) of
- undefined -> false;
- Msg ->
- %% calculate effective QoS as the lower value of SUBSCRIBE frame QoS
- %% and retained message QoS. The spec isn't super clear on this, we
- %% do what Mosquitto does, per user feedback.
- Qos = erlang:min(SubscribeQos, Msg#mqtt_msg.qos),
- Id = case Qos of
- ?QOS_0 -> undefined;
- ?QOS_1 -> MsgId
- end,
- SendFun(#mqtt_frame{fixed = #mqtt_frame_fixed{
- type = ?PUBLISH,
- qos = Qos,
- dup = false,
- retain = Msg#mqtt_msg.retain
- }, variable = #mqtt_frame_publish{
- message_id = Id,
- topic_name = Topic1
- },
- payload = Msg#mqtt_msg.payload}, PState),
- case Qos of
- ?QOS_0 -> false;
- ?QOS_1 -> {true, 1}
- end
- end.
-
--spec amqp_callback(#'basic.ack'{} | {#'basic.deliver'{}, #amqp_msg{}, {pid(), pid(), pid()}}, #proc_state{}) -> {'ok', #proc_state{}} | {'error', term(), term()}.
-amqp_callback({#'basic.deliver'{ consumer_tag = ConsumerTag,
- delivery_tag = DeliveryTag,
- routing_key = RoutingKey },
- #amqp_msg{ props = #'P_basic'{ headers = Headers },
- payload = Payload },
- DeliveryCtx} = Delivery,
- #proc_state{ channels = {Channel, _},
- awaiting_ack = Awaiting,
- message_id = MsgId,
- send_fun = SendFun,
- amqp2mqtt_fun = Amqp2MqttFun } = PState) ->
- notify_received(DeliveryCtx),
- case {delivery_dup(Delivery), delivery_qos(ConsumerTag, Headers, PState)} of
- {true, {?QOS_0, ?QOS_1}} ->
- amqp_channel:cast(
- Channel, #'basic.ack'{ delivery_tag = DeliveryTag }),
- {ok, PState};
- {true, {?QOS_0, ?QOS_0}} ->
- {ok, PState};
- {Dup, {DeliveryQos, _SubQos} = Qos} ->
- TopicName = Amqp2MqttFun(RoutingKey),
- SendFun(
- #mqtt_frame{ fixed = #mqtt_frame_fixed{
- type = ?PUBLISH,
- qos = DeliveryQos,
- dup = Dup },
- variable = #mqtt_frame_publish{
- message_id =
- case DeliveryQos of
- ?QOS_0 -> undefined;
- ?QOS_1 -> MsgId
- end,
- topic_name = TopicName },
- payload = Payload}, PState),
- case Qos of
- {?QOS_0, ?QOS_0} ->
- {ok, PState};
- {?QOS_1, ?QOS_1} ->
- Awaiting1 = gb_trees:insert(MsgId, DeliveryTag, Awaiting),
- PState1 = PState#proc_state{ awaiting_ack = Awaiting1 },
- PState2 = next_msg_id(PState1),
- {ok, PState2};
- {?QOS_0, ?QOS_1} ->
- amqp_channel:cast(
- Channel, #'basic.ack'{ delivery_tag = DeliveryTag }),
- {ok, PState}
+ fun(_Topic, {[?SUBACK_FAILURE | _] = L, S}) ->
+ %% Once a subscription failed, mark all following subscriptions
+ %% as failed instead of creating bindings because we are going
+ %% to close the client connection anyway.
+ {[?SUBACK_FAILURE | L], S};
+ (#mqtt_topic{name = TopicName,
+ qos = TopicQos},
+ {L, S0}) ->
+ QoS = supported_sub_qos(TopicQos),
+ case maybe_replace_old_sub(TopicName, QoS, S0) of
+ {ok, S1} ->
+ case ensure_queue(QoS, S1) of
+ {ok, Q} ->
+ QName = amqqueue:get_name(Q),
+ case bind(QName, TopicName, S1) of
+ {ok, _Output, S2 = #state{subscriptions = Subs}} ->
+ S3 = S2#state{subscriptions = maps:put(TopicName, QoS, Subs)},
+ maybe_increment_consumer(S2, S3),
+ case self_consumes(Q) of
+ false ->
+ case consume(Q, QoS, S3) of
+ {ok, S4} ->
+ {[QoS | L], S4};
+ {error, _Reason} ->
+ {[?SUBACK_FAILURE | L], S3}
+ end;
+ true ->
+ {[QoS | L], S3}
+ end;
+ {error, Reason, S2} ->
+ ?LOG_ERROR("Failed to bind ~s with topic ~s: ~p",
+ [rabbit_misc:rs(QName), TopicName, Reason]),
+ {[?SUBACK_FAILURE | L], S2}
+ end;
+ {error, _} ->
+ {[?SUBACK_FAILURE | L], S1}
+ end;
+ {error, _Reason, S1} ->
+ {[?SUBACK_FAILURE | L], S1}
end
- end;
+ end, {[], State0}, Topics),
-amqp_callback(#'basic.ack'{ multiple = true, delivery_tag = Tag } = Ack,
- PState = #proc_state{ unacked_pubs = UnackedPubs,
- send_fun = SendFun }) ->
- case gb_trees:size(UnackedPubs) > 0 andalso
- gb_trees:take_smallest(UnackedPubs) of
- {TagSmall, MsgId, UnackedPubs1} when TagSmall =< Tag ->
- SendFun(
- #mqtt_frame{ fixed = #mqtt_frame_fixed{ type = ?PUBACK },
- variable = #mqtt_frame_publish{ message_id = MsgId }},
- PState),
- amqp_callback(Ack, PState #proc_state{ unacked_pubs = UnackedPubs1 });
+ SendFun(
+ #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?SUBACK},
+ variable = #mqtt_packet_suback{
+ packet_id = SubscribePktId,
+ qos_table = QosResponse}},
+ State1),
+
+ case QosResponse of
+ [?SUBACK_FAILURE | _] ->
+ {error, subscribe_error, State1};
_ ->
- {ok, PState}
+ State = lists:foldl(fun(Topic, S) ->
+ maybe_send_retained_message(RPid, Topic, S)
+ end, State1, Topics),
+ {ok, State}
end;
-amqp_callback(#'basic.ack'{ multiple = false, delivery_tag = Tag },
- PState = #proc_state{ unacked_pubs = UnackedPubs,
- send_fun = SendFun }) ->
+process_request(?UNSUBSCRIBE,
+ #mqtt_packet{variable = #mqtt_packet_subscribe{packet_id = PacketId,
+ topic_table = Topics},
+ payload = undefined},
+ State0 = #state{cfg = #cfg{send_fun = SendFun}}) ->
+ ?LOG_DEBUG("Received an UNSUBSCRIBE for topic(s) ~p", [Topics]),
+ State = lists:foldl(
+ fun(#mqtt_topic{name = TopicName}, #state{subscriptions = Subs0} = S0) ->
+ case maps:take(TopicName, Subs0) of
+ {QoS, Subs} ->
+ QName = queue_name(QoS, S0),
+ case unbind(QName, TopicName, S0) of
+ {ok, _, S1} ->
+ S = S1#state{subscriptions = Subs},
+ maybe_decrement_consumer(S1, S),
+ S;
+ {error, Reason, S} ->
+ ?LOG_ERROR("Failed to unbind ~s with topic ~s: ~p",
+ [rabbit_misc:rs(QName), TopicName, Reason]),
+ S
+ end;
+ error ->
+ S0
+ end
+ end, State0, Topics),
+ SendFun(
+ #mqtt_packet{fixed = #mqtt_packet_fixed {type = ?UNSUBACK},
+ variable = #mqtt_packet_suback{packet_id = PacketId}},
+ State),
+
+ {ok, State};
+
+process_request(?PINGREQ, #mqtt_packet{},
+ State = #state{cfg = #cfg{send_fun = SendFun,
+ client_id = ClientId}}) ->
+ ?LOG_DEBUG("Received a PINGREQ, client ID: ~s", [ClientId]),
SendFun(
- #mqtt_frame{ fixed = #mqtt_frame_fixed{ type = ?PUBACK },
- variable = #mqtt_frame_publish{
- message_id = gb_trees:get(
- Tag, UnackedPubs) }}, PState),
- {ok, PState #proc_state{ unacked_pubs = gb_trees:delete(Tag, UnackedPubs) }}.
-
-delivery_dup({#'basic.deliver'{ redelivered = Redelivered },
- #amqp_msg{ props = #'P_basic'{ headers = Headers }},
- _DeliveryCtx}) ->
- case rabbit_mqtt_util:table_lookup(Headers, <<"x-mqtt-dup">>) of
- undefined -> Redelivered;
- {bool, Dup} -> Redelivered orelse Dup
+ #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?PINGRESP}},
+ State),
+ ?LOG_DEBUG("Sent a PINGRESP, client ID: ~s", [ClientId]),
+ {ok, State};
+
+process_request(?DISCONNECT, #mqtt_packet{}, State) ->
+ ?LOG_DEBUG("Received a DISCONNECT"),
+ {stop, disconnect, State}.
+
+process_connect(#mqtt_packet{
+ variable = #mqtt_packet_connect{
+ username = Username,
+ proto_ver = ProtoVersion,
+ clean_sess = CleanSess,
+ client_id = ClientId,
+ keep_alive = Keepalive} = PacketConnect},
+ State0 = #state{cfg = #cfg{send_fun = SendFun}}) ->
+ ?LOG_DEBUG("Received a CONNECT, client ID: ~s, username: ~s, "
+ "clean session: ~s, protocol version: ~p, keepalive: ~p",
+ [ClientId, Username, CleanSess, ProtoVersion, Keepalive]),
+ {ReturnCode, SessPresent, State} =
+ case rabbit_misc:pipeline([fun check_protocol_version/1,
+ fun check_client_id/1,
+ fun check_credentials/2,
+ fun login/2,
+ fun register_client/2,
+ fun start_keepalive/2,
+ fun notify_connection_created/1,
+ fun init_trace/2,
+ fun handle_clean_sess_qos0/2,
+ fun handle_clean_sess_qos1/2,
+ fun cache_subscriptions/2
+ ],
+ PacketConnect, State0) of
+ {ok, SessPresent0, State1} ->
+ {?CONNACK_ACCEPT, SessPresent0, State1};
+ {error, ConnectionRefusedReturnCode, State1} ->
+ {ConnectionRefusedReturnCode, false, State1}
+ end,
+ Response = #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?CONNACK},
+ variable = #mqtt_packet_connack{
+ session_present = SessPresent,
+ return_code = ReturnCode}},
+ SendFun(Response, State),
+ return_connack(ReturnCode, State).
+
+check_protocol_version(#mqtt_packet_connect{proto_ver = ProtoVersion}) ->
+ case lists:member(ProtoVersion, proplists:get_keys(?PROTOCOL_NAMES)) of
+ true ->
+ ok;
+ false ->
+ {error, ?CONNACK_UNACCEPTABLE_PROTO_VER}
end.
-ensure_valid_mqtt_message_id(Id) when Id >= 16#ffff ->
- 1;
-ensure_valid_mqtt_message_id(Id) ->
- Id.
-
-safe_max_id(Id0, Id1) ->
- ensure_valid_mqtt_message_id(erlang:max(Id0, Id1)).
-
-next_msg_id(PState = #proc_state{ message_id = MsgId0 }) ->
- MsgId1 = ensure_valid_mqtt_message_id(MsgId0 + 1),
- PState#proc_state{ message_id = MsgId1 }.
-
-%% decide at which qos level to deliver based on subscription
-%% and the message publish qos level. non-MQTT publishes are
-%% assumed to be qos 1, regardless of delivery_mode.
-delivery_qos(Tag, _Headers, #proc_state{ consumer_tags = {Tag, _} }) ->
- {?QOS_0, ?QOS_0};
-delivery_qos(Tag, Headers, #proc_state{ consumer_tags = {_, Tag} }) ->
- case rabbit_mqtt_util:table_lookup(Headers, <<"x-mqtt-publish-qos">>) of
- {byte, Qos} -> {lists:min([Qos, ?QOS_1]), ?QOS_1};
- undefined -> {?QOS_1, ?QOS_1}
+check_client_id(#mqtt_packet_connect{clean_sess = false,
+ client_id = <<>>}) ->
+ {error, ?CONNACK_ID_REJECTED};
+check_client_id(_) ->
+ ok.
+
+check_credentials(Packet = #mqtt_packet_connect{username = Username,
+ password = Password},
+ State = #state{cfg = #cfg{ssl_login_name = SslLoginName,
+ peer_addr = PeerAddr}}) ->
+ Ip = list_to_binary(inet:ntoa(PeerAddr)),
+ case creds(Username, Password, SslLoginName) of
+ nocreds ->
+ rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt),
+ ?LOG_ERROR("MQTT login failed: no credentials provided"),
+ {error, ?CONNACK_BAD_CREDENTIALS};
+ {invalid_creds, {undefined, Pass}} when is_binary(Pass) ->
+ rabbit_core_metrics:auth_attempt_failed(Ip, <<>>, mqtt),
+ ?LOG_ERROR("MQTT login failed: no username is provided"),
+ {error, ?CONNACK_BAD_CREDENTIALS};
+ {invalid_creds, {User, undefined}} when is_binary(User) ->
+ rabbit_core_metrics:auth_attempt_failed(Ip, User, mqtt),
+ ?LOG_ERROR("MQTT login failed for user '~p': no password provided", [User]),
+ {error, ?CONNACK_BAD_CREDENTIALS};
+ {UserBin, PassBin} ->
+ {ok, {UserBin, PassBin, Packet}, State}
end.
-maybe_clean_sess(PState = #proc_state { clean_sess = false,
- connection = Conn,
- auth_state = #auth_state{vhost = VHost},
- client_id = ClientId }) ->
- SessionPresent = session_present(VHost, ClientId),
- case SessionPresent of
- false ->
- %% ensure_queue/2 not only ensures that queue is created, but also starts consuming from it.
- %% Let's avoid creating that queue until explicitly asked by a client.
- %% Then publish-only clients, that connect with clean_sess=true due to some misconfiguration,
- %% will consume less resources.
- {{?CONNACK_ACCEPT, SessionPresent}, PState};
+login({UserBin, PassBin,
+ Packet = #mqtt_packet_connect{client_id = ClientId0,
+ clean_sess = CleanSess}},
+ State0 = #state{cfg = Cfg0}) ->
+ ClientId = ensure_client_id(ClientId0),
+ case process_login(UserBin, PassBin, ClientId, State0) of
+ {ok, State} ->
+ Cfg = Cfg0#cfg{client_id = ClientId,
+ clean_sess = CleanSess},
+ {ok, Packet, State#state{cfg = Cfg}};
+ {error, _ConnectionRefusedReturnCode, _State} = Err ->
+ Err
+ end.
+
+-spec ensure_client_id(binary()) -> binary().
+ensure_client_id(<<>>) ->
+ rabbit_data_coercion:to_binary(
+ rabbit_misc:base64url(
+ rabbit_guid:gen_secure()));
+ensure_client_id(ClientId)
+ when is_binary(ClientId) ->
+ ClientId.
+
+register_client(Packet = #mqtt_packet_connect{proto_ver = ProtoVersion},
+ State = #state{auth_state = #auth_state{vhost = VHost},
+ cfg = Cfg0 = #cfg{client_id = ClientId}}) ->
+ NewProcState =
+ fun(RegisterState) ->
+ rabbit_mqtt_util:register_clientid(VHost, ClientId),
+ RetainerPid = rabbit_mqtt_retainer_sup:child_for_vhost(VHost),
+ ExchangeBin = rabbit_mqtt_util:env(exchange),
+ ExchangeName = rabbit_misc:r(VHost, exchange, ExchangeBin),
+ Cfg = Cfg0#cfg{exchange = ExchangeName,
+ will_msg = make_will_msg(Packet),
+ retainer_pid = RetainerPid,
+ proto_ver = proto_integer_to_atom(ProtoVersion)},
+ State#state{cfg = Cfg,
+ register_state = RegisterState}
+ end,
+ case rabbit_mqtt_ff:track_client_id_in_ra() of
true ->
- try ensure_queue(?QOS_1, PState) of
- {_Queue, PState1} -> {{?CONNACK_ACCEPT, SessionPresent}, PState1}
- catch
- exit:({{shutdown, {server_initiated_close, 403, _}}, _}) ->
- %% Connection is not yet propagated to #proc_state{}, let's close it here
- catch amqp_connection:close(Conn),
- rabbit_log_connection:error("MQTT cannot recover a session, user is missing permissions"),
- {?CONNACK_SERVER, PState};
- C:E:S ->
- %% Connection is not yet propagated to
- %% #proc_state{}, let's close it here.
- %% This is an exceptional situation anyway, but
- %% doing this will prevent second crash from
- %% amqp client being logged.
- catch amqp_connection:close(Conn),
- erlang:raise(C, E, S)
+ case rabbit_mqtt_collector:register(ClientId, self()) of
+ {ok, Corr} ->
+ {ok, NewProcState({pending, Corr})};
+ {error, _} = Err ->
+ %% e.g. this node was removed from the MQTT cluster members
+ ?LOG_ERROR("MQTT cannot accept a connection: client ID tracker is unavailable: ~p",
+ [Err]),
+ {error, ?CONNACK_SERVER_UNAVAILABLE}
+ end;
+ false ->
+ {ok, NewProcState(undefined)}
+ end.
+
+notify_connection_created(#mqtt_packet_connect{}) ->
+ rabbit_networking:register_non_amqp_connection(self()),
+ self() ! connection_created,
+ ok.
+
+init_trace(#mqtt_packet_connect{}, State = #state{cfg = #cfg{conn_name = ConnName}}) ->
+ {ok, update_trace(ConnName, State)}.
+
+-spec update_trace(binary(), state()) -> state().
+update_trace(ConnName0, State = #state{cfg = Cfg0,
+ auth_state = #auth_state{vhost = VHost}}) ->
+ ConnName = case rabbit_trace:enabled(VHost) of
+ true ->
+ ConnName0;
+ false ->
+ %% We won't need conn_name. Use less memmory by setting to undefined.
+ undefined
+ end,
+ Cfg = Cfg0#cfg{conn_name = ConnName,
+ trace_state = rabbit_trace:init(VHost)},
+ State#state{cfg = Cfg}.
+
+return_connack(?CONNACK_ACCEPT, S) ->
+ {ok, S};
+return_connack(?CONNACK_BAD_CREDENTIALS, S) ->
+ {error, unauthenticated, S};
+return_connack(?CONNACK_NOT_AUTHORIZED, S) ->
+ {error, unauthorized, S};
+return_connack(?CONNACK_SERVER_UNAVAILABLE, S) ->
+ {error, unavailable, S};
+return_connack(?CONNACK_ID_REJECTED, S) ->
+ {error, invalid_client_id, S};
+return_connack(?CONNACK_UNACCEPTABLE_PROTO_VER, S) ->
+ {error, unsupported_protocol_version, S}.
+
+-spec self_consumes(amqqueue:amqqueue()) -> boolean().
+self_consumes(Queue) ->
+ case amqqueue:get_type(Queue) of
+ ?QUEUE_TYPE_QOS_0 ->
+ false;
+ _ ->
+ lists:any(fun(Consumer) ->
+ element(1, Consumer) =:= self()
+ end, rabbit_amqqueue:consumers(Queue))
+ end.
+
+start_keepalive(#mqtt_packet_connect{keep_alive = Seconds},
+ #state{cfg = #cfg{socket = Socket}}) ->
+ ok = rabbit_mqtt_keepalive:start(Seconds, Socket).
+
+handle_clean_sess_qos0(#mqtt_packet_connect{}, State) ->
+ handle_clean_sess(false, ?QOS_0, State).
+
+handle_clean_sess_qos1(QoS0SessPresent, State) ->
+ handle_clean_sess(QoS0SessPresent, ?QOS_1, State).
+
+handle_clean_sess(_, QoS,
+ State = #state{cfg = #cfg{clean_sess = true},
+ auth_state = #auth_state{user = User,
+ username = Username,
+ authz_ctx = AuthzCtx}}) ->
+ %% "If the Server accepts a connection with CleanSession set to 1, the Server
+ %% MUST set Session Present to 0 in the CONNACK packet [MQTT-3.2.2-1].
+ SessPresent = false,
+ case get_queue(QoS, State) of
+ {error, _} ->
+ {ok, SessPresent, State};
+ {ok, Q0} ->
+ QName = amqqueue:get_name(Q0),
+ %% configure access to queue required for queue.delete
+ case check_resource_access(User, QName, configure, AuthzCtx) of
+ ok ->
+ delete_queue(QName, Username),
+ {ok, SessPresent, State};
+ {error, access_refused} ->
+ {error, ?CONNACK_NOT_AUTHORIZED}
end
end;
-maybe_clean_sess(PState = #proc_state { clean_sess = true,
- connection = Conn,
- auth_state = #auth_state{vhost = VHost},
- client_id = ClientId }) ->
- {_, Queue} = rabbit_mqtt_util:subcription_queue_name(ClientId),
- {ok, Channel} = amqp_connection:open_channel(Conn),
- case session_present(VHost, ClientId) of
- false ->
- {{?CONNACK_ACCEPT, false}, PState};
- true ->
- try amqp_channel:call(Channel, #'queue.delete'{ queue = Queue }) of
- #'queue.delete_ok'{} -> {{?CONNACK_ACCEPT, false}, PState}
- catch
- exit:({{shutdown, {server_initiated_close, 403, _}}, _}) ->
- %% Connection is not yet propagated to #proc_state{}, let's close it here
- catch amqp_connection:close(Conn),
- rabbit_log_connection:error("MQTT cannot start a clean session: "
- "`configure` permission missing for queue `~tp`", [Queue]),
- {?CONNACK_SERVER, PState}
- after
- catch amqp_channel:close(Channel)
+handle_clean_sess(SessPresent, QoS,
+ State0 = #state{cfg = #cfg{clean_sess = false,
+ proto_ver = ProtoVer}}) ->
+ case get_queue(QoS, State0) of
+ {error, _} ->
+ %% Queue will be created later when client subscribes.
+ {ok, SessPresent, State0};
+ {ok, Q} ->
+ case consume(Q, QoS, State0) of
+ {ok, State} ->
+ case SessPresent of
+ false ->
+ rabbit_global_counters:consumer_created(ProtoVer);
+ true ->
+ %% We already incremented the consumer counter for QoS 0.
+ ok
+ end,
+ {ok, _SessionPresent = true, State};
+ {error, access_refused} ->
+ {error, ?CONNACK_NOT_AUTHORIZED};
+ {error, _Reason} ->
+ %% Let's use most generic error return code.
+ {error, ?CONNACK_SERVER_UNAVAILABLE}
end
end.
-session_present(VHost, ClientId) ->
- {_, QueueQ1} = rabbit_mqtt_util:subcription_queue_name(ClientId),
- QueueName = rabbit_misc:r(VHost, queue, QueueQ1),
- rabbit_amqqueue:exists(QueueName).
+-spec get_queue(qos(), state()) ->
+ {ok, amqqueue:amqqueue()} |
+ {error, not_found | {resource_locked, amqqueue:amqqueue()}}.
+get_queue(QoS, State) ->
+ QName = queue_name(QoS, State),
+ case rabbit_amqqueue:lookup(QName) of
+ {ok, Q} = Ok ->
+ try rabbit_amqqueue:check_exclusive_access(Q, self()) of
+ ok ->
+ Ok
+ catch
+ exit:#amqp_error{name = resource_locked} ->
+ %% This can happen when same client ID re-connects
+ %% while its old connection is not yet closed.
+ {error, {resource_locked, Q}}
+ end;
+ {error, not_found} = Err ->
+ Err
+ end.
+
+queue_name(?QOS_1, #state{cfg = #cfg{queue_qos1 = #resource{kind = queue} = Name}}) ->
+ Name;
+queue_name(QoS, #state{auth_state = #auth_state{vhost = VHost},
+ cfg = #cfg{client_id = ClientId}}) ->
+ QNameBin = rabbit_mqtt_util:queue_name_bin(ClientId, QoS),
+ rabbit_misc:r(VHost, queue, QNameBin).
+
+%% Query subscriptions from the database and hold them in process state
+%% to avoid future mnesia:match_object/3 queries.
+cache_subscriptions(_SessionPresent = _SubscriptionsPresent = true, State) ->
+ SubsQos0 = topic_names(?QOS_0, State),
+ SubsQos1 = topic_names(?QOS_1, State),
+ Subs = maps:merge(maps:from_keys(SubsQos0, ?QOS_0),
+ maps:from_keys(SubsQos1, ?QOS_1)),
+ {ok, State#state{subscriptions = Subs}};
+cache_subscriptions(_, _) ->
+ ok.
+
+topic_names(QoS, State = #state{cfg = #cfg{exchange = Exchange}}) ->
+ Bindings =
+ rabbit_binding:list_for_source_and_destination(
+ Exchange,
+ queue_name(QoS, State),
+ %% Querying table rabbit_route is catastrophic for CPU usage.
+ %% Querying table rabbit_reverse_route is acceptable because
+ %% the source exchange is always the same in the MQTT plugin whereas
+ %% the destination queue is different for each MQTT client and
+ %% rabbit_reverse_route is sorted by destination queue.
+ _Reverse = true),
+ lists:map(fun(B) -> amqp_to_mqtt(B#binding.key) end, Bindings).
+
+%% "If a Server receives a SUBSCRIBE Packet containing a Topic Filter that is identical
+%% to an existing Subscription’s Topic Filter then it MUST completely replace that
+%% existing Subscription with a new Subscription. The Topic Filter in the new Subscription
+%% will be identical to that in the previous Subscription, although its maximum QoS value
+%% could be different." [MQTT-3.8.4-3].
+maybe_replace_old_sub(TopicName, QoS, State = #state{subscriptions = Subs}) ->
+ case Subs of
+ #{TopicName := OldQoS}
+ when OldQoS =/= QoS ->
+ replace_old_sub(OldQoS, TopicName, State);
+ _ ->
+ {ok, State}
+ end.
+
+replace_old_sub(QoS, TopicName, State0)->
+ QName = queue_name(QoS, State0),
+ case unbind(QName, TopicName, State0) of
+ {ok, _Output, State} ->
+ {ok, State};
+ {error, Reason, _State} = Err ->
+ ?LOG_ERROR("Failed to unbind ~s with topic '~s': ~p",
+ [rabbit_misc:rs(QName), TopicName, Reason]),
+ Err
+ end.
-make_will_msg(#mqtt_frame_connect{ will_flag = false }) ->
+-spec hand_off_to_retainer(pid(), binary(), mqtt_msg()) -> ok.
+hand_off_to_retainer(RetainerPid, Topic0, #mqtt_msg{payload = <<"">>}) ->
+ Topic1 = amqp_to_mqtt(Topic0),
+ rabbit_mqtt_retainer:clear(RetainerPid, Topic1),
+ ok;
+hand_off_to_retainer(RetainerPid, Topic0, Msg) ->
+ Topic1 = amqp_to_mqtt(Topic0),
+ rabbit_mqtt_retainer:retain(RetainerPid, Topic1, Msg),
+ ok.
+
+maybe_send_retained_message(RPid, #mqtt_topic{name = Topic0, qos = SubscribeQos},
+ State0 = #state{packet_id = PacketId0,
+ cfg = #cfg{send_fun = SendFun}}) ->
+ Topic1 = amqp_to_mqtt(Topic0),
+ case rabbit_mqtt_retainer:fetch(RPid, Topic1) of
+ undefined ->
+ State0;
+ Msg ->
+ Qos = effective_qos(Msg#mqtt_msg.qos, SubscribeQos),
+ {PacketId, State} = case Qos of
+ ?QOS_0 ->
+ {undefined, State0};
+ ?QOS_1 ->
+ {PacketId0, State0#state{packet_id = increment_packet_id(PacketId0)}}
+ end,
+ SendFun(
+ #mqtt_packet{fixed = #mqtt_packet_fixed{
+ type = ?PUBLISH,
+ qos = Qos,
+ dup = false,
+ retain = Msg#mqtt_msg.retain
+ }, variable = #mqtt_packet_publish{
+ packet_id = PacketId,
+ topic_name = Topic1
+ },
+ payload = Msg#mqtt_msg.payload},
+ State),
+ State
+ end.
+
+make_will_msg(#mqtt_packet_connect{will_flag = false}) ->
undefined;
-make_will_msg(#mqtt_frame_connect{ will_retain = Retain,
+make_will_msg(#mqtt_packet_connect{will_retain = Retain,
will_qos = Qos,
will_topic = Topic,
- will_msg = Msg }) ->
- #mqtt_msg{ retain = Retain,
- qos = Qos,
- topic = Topic,
- dup = false,
- payload = Msg }.
-
-process_login(_UserBin, _PassBin, _ProtoVersion,
- #proc_state{channels = {Channel, _},
- peer_addr = Addr,
- auth_state = #auth_state{username = Username,
- vhost = VHost}}) when is_pid(Channel) ->
- UsernameStr = rabbit_data_coercion:to_list(Username),
- VHostStr = rabbit_data_coercion:to_list(VHost),
+ will_msg = Msg}) ->
+ #mqtt_msg{retain = Retain,
+ qos = Qos,
+ topic = Topic,
+ dup = false,
+ payload = Msg}.
+
+process_login(_UserBin, _PassBin, ClientId,
+ #state{cfg = #cfg{peer_addr = Addr},
+ auth_state = #auth_state{username = Username,
+ user = User,
+ vhost = VHost
+ }} = State)
+ when Username =/= undefined, User =/= undefined, VHost =/= underfined ->
rabbit_core_metrics:auth_attempt_failed(list_to_binary(inet:ntoa(Addr)), Username, mqtt),
- rabbit_log_connection:warning("MQTT detected duplicate connect/login attempt for user ~tp, vhost ~tp",
- [UsernameStr, VHostStr]),
- connack_dup_auth;
-process_login(UserBin, PassBin, ProtoVersion,
- #proc_state{channels = {undefined, undefined},
- socket = Sock,
- adapter_info = AdapterInfo,
- ssl_login_name = SslLoginName,
- peer_addr = Addr}) ->
- {ok, {_, _, _, ToPort}} = rabbit_net:socket_ends(Sock, inbound),
- {VHostPickedUsing, {VHost, UsernameBin}} = get_vhost(UserBin, SslLoginName, ToPort),
- rabbit_log_connection:debug(
- "MQTT vhost picked using ~ts",
- [human_readable_vhost_lookup_strategy(VHostPickedUsing)]),
- RemoteAddress = list_to_binary(inet:ntoa(Addr)),
+ ?LOG_ERROR(
+ "MQTT detected duplicate connect attempt for client ID '~ts', user '~ts', vhost '~ts'",
+ [ClientId, Username, VHost]),
+ {error, ?CONNACK_ID_REJECTED, State};
+process_login(UserBin, PassBin, ClientId,
+ #state{auth_state = undefined,
+ cfg = #cfg{socket = Sock,
+ ssl_login_name = SslLoginName,
+ peer_addr = Addr
+ }} = State0) ->
+ {ok, {_PeerHost, _PeerPort, _Host, Port}} = rabbit_net:socket_ends(Sock, inbound),
+ {VHostPickedUsing, {VHost, UsernameBin}} = get_vhost(UserBin, SslLoginName, Port),
+ ?LOG_DEBUG("MQTT vhost picked using ~s",
+ [human_readable_vhost_lookup_strategy(VHostPickedUsing)]),
+ RemoteIpAddressBin = list_to_binary(inet:ntoa(Addr)),
+ Input = #{vhost => VHost,
+ username_bin => UsernameBin,
+ pass_bin => PassBin,
+ client_id => ClientId},
+ case rabbit_misc:pipeline(
+ [fun check_vhost_exists/1,
+ fun check_vhost_connection_limit/1,
+ fun check_vhost_alive/1,
+ fun check_user_login/2,
+ fun check_user_connection_limit/1,
+ fun check_vhost_access/2,
+ fun check_user_loopback/2
+ ],
+ Input, State0) of
+ {ok, _Output, State} ->
+ rabbit_core_metrics:auth_attempt_succeeded(RemoteIpAddressBin, UsernameBin, mqtt),
+ {ok, State};
+ {error, _ConnectionRefusedReturnCode, _State} = Err ->
+ rabbit_core_metrics:auth_attempt_failed(RemoteIpAddressBin, UsernameBin, mqtt),
+ Err
+ end.
+
+check_vhost_exists(#{vhost := VHost,
+ username_bin := UsernameBin}) ->
case rabbit_vhost:exists(VHost) of
true ->
- case amqp_connection:start(#amqp_params_direct{
- username = UsernameBin,
- password = PassBin,
- virtual_host = VHost,
- adapter_info = set_proto_version(AdapterInfo, ProtoVersion)}) of
- {ok, Connection} ->
- case rabbit_access_control:check_user_loopback(UsernameBin, Addr) of
- ok ->
- rabbit_core_metrics:auth_attempt_succeeded(RemoteAddress, UsernameBin,
- mqtt),
- [{internal_user, InternalUser}] = amqp_connection:info(
- Connection, [internal_user]),
- {?CONNACK_ACCEPT, Connection, VHost,
- #auth_state{user = InternalUser,
- username = UsernameBin,
- vhost = VHost}};
- not_allowed ->
- rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin,
- mqtt),
- amqp_connection:close(Connection),
- rabbit_log_connection:warning(
- "MQTT login failed for user ~ts: "
- "this user's access is restricted to localhost",
- [binary_to_list(UsernameBin)]),
- ?CONNACK_AUTH
- end;
- {error, {auth_failure, Explanation}} ->
- rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, mqtt),
- rabbit_log_connection:error("MQTT login failed for user '~ts', authentication failed: ~ts",
- [binary_to_list(UserBin), Explanation]),
- ?CONNACK_CREDENTIALS;
- {error, access_refused} ->
- rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, mqtt),
- rabbit_log_connection:warning("MQTT login failed for user '~ts': "
- "virtual host access not allowed",
- [binary_to_list(UserBin)]),
- ?CONNACK_AUTH;
- {error, not_allowed} ->
- rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, mqtt),
- %% when vhost allowed for TLS connection
- rabbit_log_connection:warning("MQTT login failed for user '~ts': "
- "virtual host access not allowed",
- [binary_to_list(UserBin)]),
- ?CONNACK_AUTH
- end;
+ ok;
false ->
- rabbit_core_metrics:auth_attempt_failed(RemoteAddress, UsernameBin, mqtt),
- rabbit_log_connection:error("MQTT login failed for user '~ts': virtual host '~ts' does not exist",
- [UserBin, VHost]),
- ?CONNACK_CREDENTIALS
+ ?LOG_ERROR("MQTT login failed for user '~s': virtual host '~s' does not exist",
+ [UsernameBin, VHost]),
+ {error, ?CONNACK_BAD_CREDENTIALS}
+ end.
+
+check_vhost_connection_limit(#{vhost := VHost,
+ client_id := ClientId,
+ username_bin := Username}) ->
+ case rabbit_vhost_limit:is_over_connection_limit(VHost) of
+ false ->
+ ok;
+ {true, Limit} ->
+ ?LOG_ERROR(
+ "Failed to create MQTT connection because vhost connection limit is reached; "
+ "vhost: '~s'; connection limit: ~p; user: '~s'; client ID '~s'",
+ [VHost, Limit, Username, ClientId]),
+ {error, ?CONNACK_NOT_AUTHORIZED}
+ end.
+
+check_vhost_alive(#{vhost := VHost,
+ client_id := ClientId,
+ username_bin := UsernameBin}) ->
+ case rabbit_vhost_sup_sup:is_vhost_alive(VHost) of
+ true ->
+ ok;
+ false ->
+ ?LOG_ERROR(
+ "Failed to create MQTT connection because vhost is down; "
+ "vhost: ~s; user: ~s; client ID: ~s",
+ [VHost, UsernameBin, ClientId]),
+ {error, ?CONNACK_NOT_AUTHORIZED}
+ end.
+
+check_user_login(#{vhost := VHost,
+ username_bin := UsernameBin,
+ pass_bin := PassBin,
+ client_id := ClientId
+ } = In, State) ->
+ AuthProps = case PassBin of
+ none ->
+ %% SSL user name provided.
+ %% Authenticating using username only.
+ [];
+ _ ->
+ [{password, PassBin},
+ {vhost, VHost},
+ {client_id, ClientId}]
+ end,
+ case rabbit_access_control:check_user_login(
+ UsernameBin, AuthProps) of
+ {ok, User = #user{username = Username}} ->
+ notify_auth_result(user_authentication_success, Username, State),
+ {ok, maps:put(user, User, In), State};
+ {refused, Username, Msg, Args} ->
+ ?LOG_ERROR(
+ "Error on MQTT connection ~p~n"
+ "access refused for user '~s' in vhost '~s' "
+ ++ Msg,
+ [self(), Username, VHost] ++ Args),
+ notify_auth_result(user_authentication_failure, Username, State),
+ {error, ?CONNACK_BAD_CREDENTIALS}
+ end.
+
+notify_auth_result(AuthResult, Username, #state{cfg = #cfg{conn_name = ConnName}}) ->
+ rabbit_event:notify(
+ AuthResult,
+ [{name, Username},
+ {connection_name, ConnName},
+ {connection_type, network}]).
+
+check_user_connection_limit(#{user := #user{username = Username},
+ client_id := ClientId}) ->
+ case rabbit_auth_backend_internal:is_over_connection_limit(Username) of
+ false ->
+ ok;
+ {true, Limit} ->
+ ?LOG_ERROR(
+ "Failed to create MQTT connection because user connection limit is reached; "
+ "user: '~s'; connection limit: ~p; client ID '~s'",
+ [Username, Limit, ClientId]),
+ {error, ?CONNACK_NOT_AUTHORIZED}
+ end.
+
+
+check_vhost_access(#{vhost := VHost,
+ client_id := ClientId,
+ user := User = #user{username = Username}
+ } = In,
+ #state{cfg = #cfg{peer_addr = PeerAddr}} = State) ->
+ AuthzCtx = #{<<"client_id">> => ClientId},
+ try rabbit_access_control:check_vhost_access(
+ User,
+ VHost,
+ {ip, PeerAddr},
+ AuthzCtx) of
+ ok ->
+ {ok, maps:put(authz_ctx, AuthzCtx, In), State}
+ catch exit:#amqp_error{name = not_allowed} ->
+ ?LOG_ERROR(
+ "Error on MQTT connection ~p~n"
+ "access refused for user '~s'",
+ [self(), Username]),
+ {error, ?CONNACK_NOT_AUTHORIZED}
+ end.
+
+check_user_loopback(#{vhost := VHost,
+ username_bin := UsernameBin,
+ user := User,
+ authz_ctx := AuthzCtx
+ },
+ #state{cfg = #cfg{peer_addr = PeerAddr}} = State) ->
+ case rabbit_access_control:check_user_loopback(UsernameBin, PeerAddr) of
+ ok ->
+ AuthState = #auth_state{user = User,
+ username = UsernameBin,
+ vhost = VHost,
+ authz_ctx = AuthzCtx},
+ {ok, State#state{auth_state = AuthState}};
+ not_allowed ->
+ ?LOG_WARNING(
+ "MQTT login failed: user '~s' can only connect via localhost",
+ [UsernameBin]),
+ {error, ?CONNACK_NOT_AUTHORIZED}
end.
get_vhost(UserBin, none, Port) ->
@@ -666,8 +887,8 @@ get_vhost_no_ssl(UserBin, Port) ->
{vhost_in_username_or_default, get_vhost_username(UserBin)};
false ->
PortVirtualHostMapping = rabbit_runtime_parameters:value_global(
- mqtt_port_to_vhost_mapping
- ),
+ mqtt_port_to_vhost_mapping
+ ),
case get_vhost_from_port_mapping(Port, PortVirtualHostMapping) of
undefined ->
{default_vhost, {rabbit_mqtt_util:env(vhost), UserBin}};
@@ -678,13 +899,13 @@ get_vhost_no_ssl(UserBin, Port) ->
get_vhost_ssl(UserBin, SslLoginName, Port) ->
UserVirtualHostMapping = rabbit_runtime_parameters:value_global(
- mqtt_default_vhosts
- ),
+ mqtt_default_vhosts
+ ),
case get_vhost_from_user_mapping(SslLoginName, UserVirtualHostMapping) of
undefined ->
PortVirtualHostMapping = rabbit_runtime_parameters:value_global(
- mqtt_port_to_vhost_mapping
- ),
+ mqtt_port_to_vhost_mapping
+ ),
case get_vhost_from_port_mapping(Port, PortVirtualHostMapping) of
undefined ->
{vhost_in_username_or_default, get_vhost_username(UserBin)};
@@ -696,7 +917,7 @@ get_vhost_ssl(UserBin, SslLoginName, Port) ->
end.
vhost_in_username(UserBin) ->
- case application:get_env(?APP, ignore_colons_in_username) of
+ case application:get_env(?APP_NAME, ignore_colons_in_username) of
{ok, true} -> false;
_ ->
%% split at the last colon, disallowing colons in username
@@ -708,7 +929,7 @@ vhost_in_username(UserBin) ->
get_vhost_username(UserBin) ->
Default = {rabbit_mqtt_util:env(vhost), UserBin},
- case application:get_env(?APP, ignore_colons_in_username) of
+ case application:get_env(?APP_NAME, ignore_colons_in_username) of
{ok, true} -> Default;
_ ->
%% split at the last colon, disallowing colons in username
@@ -734,11 +955,11 @@ get_vhost_from_port_mapping(_Port, not_found) ->
get_vhost_from_port_mapping(Port, Mapping) ->
M = rabbit_data_coercion:to_proplist(Mapping),
Res = case rabbit_misc:pget(rabbit_data_coercion:to_binary(Port), M) of
- undefined ->
- undefined;
- VHost ->
- VHost
- end,
+ undefined ->
+ undefined;
+ VHost ->
+ VHost
+ end,
Res.
human_readable_vhost_lookup_strategy(vhost_in_username_or_default) ->
@@ -748,32 +969,24 @@ human_readable_vhost_lookup_strategy(port_to_vhost_mapping) ->
human_readable_vhost_lookup_strategy(cert_to_vhost_mapping) ->
"client certificate to vhost mapping";
human_readable_vhost_lookup_strategy(default_vhost) ->
- "plugin configuration or default";
-human_readable_vhost_lookup_strategy(Val) ->
- atom_to_list(Val).
+ "plugin configuration or default".
creds(User, Pass, SSLLoginName) ->
DefaultUser = rabbit_mqtt_util:env(default_user),
DefaultPass = rabbit_mqtt_util:env(default_pass),
- {ok, Anon} = application:get_env(?APP, allow_anonymous),
- {ok, TLSAuth} = application:get_env(?APP, ssl_cert_login),
+ {ok, Anon} = application:get_env(?APP_NAME, allow_anonymous),
+ {ok, TLSAuth} = application:get_env(?APP_NAME, ssl_cert_login),
HaveDefaultCreds = Anon =:= true andalso
- is_binary(DefaultUser) andalso
- is_binary(DefaultPass),
-
- CredentialsProvided = User =/= undefined orelse
- Pass =/= undefined,
+ is_binary(DefaultUser) andalso
+ is_binary(DefaultPass),
- CorrectCredentials = is_list(User) andalso
- is_list(Pass),
-
- SSLLoginProvided = TLSAuth =:= true andalso
- SSLLoginName =/= none,
+ CredentialsProvided = User =/= undefined orelse Pass =/= undefined,
+ CorrectCredentials = is_binary(User) andalso is_binary(Pass),
+ SSLLoginProvided = TLSAuth =:= true andalso SSLLoginName =/= none,
case {CredentialsProvided, CorrectCredentials, SSLLoginProvided, HaveDefaultCreds} of
%% Username and password take priority
- {true, true, _, _} -> {list_to_binary(User),
- list_to_binary(Pass)};
+ {true, true, _, _} -> {User, Pass};
%% Either username or password is provided
{true, false, _, _} -> {invalid_creds, {User, Pass}};
%% rabbitmq_mqtt.ssl_cert_login is true. SSL user name provided.
@@ -784,341 +997,1008 @@ creds(User, Pass, SSLLoginName) ->
_ -> nocreds
end.
-supported_subs_qos(?QOS_0) -> ?QOS_0;
-supported_subs_qos(?QOS_1) -> ?QOS_1;
-supported_subs_qos(?QOS_2) -> ?QOS_1.
+supported_sub_qos(?QOS_0) -> ?QOS_0;
+supported_sub_qos(?QOS_1) -> ?QOS_1;
+supported_sub_qos(?QOS_2) -> ?QOS_1.
delivery_mode(?QOS_0) -> 1;
delivery_mode(?QOS_1) -> 2;
delivery_mode(?QOS_2) -> 2.
-maybe_quorum(Qos1Args, CleanSession, Queue) ->
- case {rabbit_mqtt_util:env(durable_queue_type), CleanSession} of
- %% it is possible to Quorum queues only if Clean Session == False
- %% else always use Classic queues
- %% Clean Session == True sets auto-delete to True and quorum queues
- %% does not support auto-delete flag
- {quorum, false} -> lists:append(Qos1Args,
- [{<<"x-queue-type">>, longstr, <<"quorum">>}]);
-
- {quorum, true} ->
- rabbit_log:debug("Can't use quorum queue for ~ts. " ++
- "The clean session is true. Classic queue will be used", [Queue]),
- Qos1Args;
- _ -> Qos1Args
+ensure_queue(QoS, State = #state{auth_state = #auth_state{user = #user{username = Username}}}) ->
+ case get_queue(QoS, State) of
+ {ok, Q} ->
+ {ok, Q};
+ {error, {resource_locked, Q}} ->
+ QName = amqqueue:get_name(Q),
+ ?LOG_DEBUG("MQTT deleting exclusive ~s owned by ~p",
+ [rabbit_misc:rs(QName), ?amqqueue_v2_field_exclusive_owner(Q)]),
+ delete_queue(QName, Username),
+ create_queue(QoS, State);
+ {error, not_found} ->
+ create_queue(QoS, State)
end.
-%% different qos subscriptions are received in different queues
-%% with appropriate durability and timeout arguments
-%% this will lead to duplicate messages for overlapping subscriptions
-%% with different qos values - todo: prevent duplicates
-ensure_queue(Qos, #proc_state{ channels = {Channel, _},
- client_id = ClientId,
- clean_sess = CleanSess,
- consumer_tags = {TagQ0, TagQ1} = Tags} = PState) ->
- {QueueQ0, QueueQ1} = rabbit_mqtt_util:subcription_queue_name(ClientId),
- Qos1Args = case {rabbit_mqtt_util:env(subscription_ttl), CleanSess} of
- {undefined, _} ->
- [];
- {Ms, false} when is_integer(Ms) ->
- [{<<"x-expires">>, long, Ms}];
- _ ->
- []
- end,
- QueueSetup =
- case {TagQ0, TagQ1, Qos} of
- {undefined, _, ?QOS_0} ->
- {QueueQ0,
- #'queue.declare'{ queue = QueueQ0,
- durable = false,
- auto_delete = true },
- #'basic.consume'{ queue = QueueQ0,
- no_ack = true }};
- {_, undefined, ?QOS_1} ->
- {QueueQ1,
- #'queue.declare'{ queue = QueueQ1,
- durable = true,
- %% Clean session means a transient connection,
- %% translating into auto-delete.
- %%
- %% see rabbitmq/rabbitmq-mqtt#37
- auto_delete = CleanSess,
- arguments = maybe_quorum(Qos1Args, CleanSess, QueueQ1)},
- #'basic.consume'{ queue = QueueQ1,
- no_ack = false }};
- {_, _, ?QOS_0} ->
- {exists, QueueQ0};
- {_, _, ?QOS_1} ->
- {exists, QueueQ1}
- end,
- case QueueSetup of
- {Queue, Declare, Consume} ->
- #'queue.declare_ok'{} = amqp_channel:call(Channel, Declare),
- #'basic.consume_ok'{ consumer_tag = Tag } =
- amqp_channel:call(Channel, Consume),
- {Queue, PState #proc_state{ consumer_tags = setelement(Qos+1, Tags, Tag) }};
- {exists, Q} ->
- {Q, PState}
+create_queue(
+ QoS, #state{cfg = #cfg{
+ client_id = ClientId,
+ clean_sess = CleanSess},
+ auth_state = #auth_state{
+ vhost = VHost,
+ user = User = #user{username = Username},
+ authz_ctx = AuthzCtx}
+ }) ->
+ QNameBin = rabbit_mqtt_util:queue_name_bin(ClientId, QoS),
+ QName = rabbit_misc:r(VHost, queue, QNameBin),
+ %% configure access to queue required for queue.declare
+ case check_resource_access(User, QName, configure, AuthzCtx) of
+ ok ->
+ case rabbit_vhost_limit:is_over_queue_limit(VHost) of
+ false ->
+ rabbit_core_metrics:queue_declared(QName),
+ QArgs = queue_args(QoS, CleanSess),
+ Q0 = amqqueue:new(QName,
+ self(),
+ _Durable = true,
+ _AutoDelete = false,
+ queue_owner(CleanSess),
+ QArgs,
+ VHost,
+ #{user => Username},
+ queue_type(QoS, CleanSess, QArgs)
+ ),
+ case rabbit_queue_type:declare(Q0, node()) of
+ {new, Q} when ?is_amqqueue(Q) ->
+ rabbit_core_metrics:queue_created(QName),
+ {ok, Q};
+ Other ->
+ ?LOG_ERROR("Failed to declare ~s: ~p",
+ [rabbit_misc:rs(QName), Other]),
+ {error, queue_declare}
+ end;
+ {true, Limit} ->
+ ?LOG_ERROR("cannot declare ~s because "
+ "queue limit ~p in vhost '~s' is reached",
+ [rabbit_misc:rs(QName), Limit, VHost]),
+ {error, access_refused}
+ end;
+ {error, access_refused} = E ->
+ E
end.
-send_will(PState = #proc_state{will_msg = undefined}) ->
- PState;
+-spec queue_owner(CleanSession :: boolean()) ->
+ pid() | none.
+queue_owner(true) ->
+ %% Exclusive queues are auto-deleted after node restart while auto-delete queues are not.
+ %% Therefore make durable queue exclusive.
+ self();
+queue_owner(false) ->
+ none.
+
+queue_args(QoS, false) ->
+ Args = case rabbit_mqtt_util:env(subscription_ttl) of
+ Ms when is_integer(Ms) ->
+ [{<<"x-expires">>, long, Ms}];
+ _ ->
+ []
+ end,
+ case {QoS, rabbit_mqtt_util:env(durable_queue_type)} of
+ {?QOS_1, quorum} ->
+ [{<<"x-queue-type">>, longstr, <<"quorum">>} | Args];
+ _ ->
+ Args
+ end;
+queue_args(_, _) ->
+ [].
-send_will(PState = #proc_state{will_msg = WillMsg = #mqtt_msg{retain = Retain,
- topic = Topic},
- retainer_pid = RPid,
- channels = {ChQos0, ChQos1},
- amqp2mqtt_fun = Amqp2MqttFun}) ->
- case check_topic_access(Topic, write, PState) of
+queue_type(?QOS_0, true, QArgs) ->
+ case rabbit_feature_flags:is_enabled(?QUEUE_TYPE_QOS_0) of
+ true ->
+ ?QUEUE_TYPE_QOS_0;
+ false ->
+ rabbit_amqqueue:get_queue_type(QArgs)
+ end;
+queue_type(_, _, QArgs) ->
+ rabbit_amqqueue:get_queue_type(QArgs).
+
+consume(Q, QoS, #state{
+ queue_states = QStates0,
+ cfg = #cfg{prefetch = Prefetch},
+ auth_state = #auth_state{
+ authz_ctx = AuthzCtx,
+ user = User = #user{username = Username}}
+ } = State0) ->
+ QName = amqqueue:get_name(Q),
+ %% read access to queue required for basic.consume
+ case check_resource_access(User, QName, read, AuthzCtx) of
ok ->
- amqp_pub(WillMsg, PState),
- case Retain of
- false -> ok;
- true ->
- hand_off_to_retainer(RPid, Amqp2MqttFun, Topic, WillMsg)
+ case amqqueue:get_type(Q) of
+ ?QUEUE_TYPE_QOS_0 ->
+ %% Messages get delivered directly to our process without
+ %% explicitly calling rabbit_queue_type:consume/3.
+ {ok, State0};
+ _ ->
+ Spec = #{no_ack => QoS =:= ?QOS_0,
+ channel_pid => self(),
+ limiter_pid => none,
+ limiter_active => false,
+ prefetch_count => Prefetch,
+ consumer_tag => ?CONSUMER_TAG,
+ exclusive_consume => false,
+ args => [],
+ ok_msg => undefined,
+ acting_user => Username},
+ rabbit_amqqueue:with(
+ QName,
+ fun(Q1) ->
+ case rabbit_queue_type:consume(Q1, Spec, QStates0) of
+ {ok, QStates} ->
+ State1 = State0#state{queue_states = QStates},
+ State = maybe_set_queue_qos1(QoS, State1),
+ {ok, State};
+ {error, Reason} = Err ->
+ ?LOG_ERROR("Failed to consume from ~s: ~p",
+ [rabbit_misc:rs(QName), Reason]),
+ Err
+ end
+ end)
end;
- Error ->
- rabbit_log:warning(
- "Could not send last will: ~tp",
- [Error])
- end,
- case ChQos1 of
- undefined -> ok;
- _ -> amqp_channel:close(ChQos1)
- end,
- case ChQos0 of
- undefined -> ok;
- _ -> amqp_channel:close(ChQos0)
- end,
- PState #proc_state{ channels = {undefined, undefined} }.
+ {error, access_refused} = Err ->
+ Err
+ end.
-%% TODO amqp_pub/2 is publishing messages asynchronously, using
-%% amqp_channel:cast_flow/3
-%%
-%% It does access check using check_publish/3 before submitting, but
-%% this is superfluous, as actual publishing will do the same
-%% check. While check results cached, it's still some unnecessary
-%% work.
-%%
-%% And the only reason to keep it that way is that it prevents useless
-%% crash messages flooding logs, as there is no code to handle async
-%% channel crash gracefully.
-%%
-%% It'd be better to rework the whole thing, removing performance
-%% penalty and some 50 lines of duplicate code. Maybe unlinking from
-%% channel, and adding it as a child of connection supervisor instead.
-%% But exact details are not yet clear.
-amqp_pub(undefined, PState) ->
- PState;
-
-%% set up a qos1 publishing channel if necessary
-%% this channel will only be used for publishing, not consuming
-amqp_pub(Msg = #mqtt_msg{ qos = ?QOS_1 },
- PState = #proc_state{ channels = {ChQos0, undefined},
- awaiting_seqno = undefined,
- connection = Conn }) ->
- {ok, Channel} = amqp_connection:open_channel(Conn),
- #'confirm.select_ok'{} = amqp_channel:call(Channel, #'confirm.select'{}),
- amqp_channel:register_confirm_handler(Channel, self()),
- amqp_pub(Msg, PState #proc_state{ channels = {ChQos0, Channel},
- awaiting_seqno = 1 });
-
-amqp_pub(#mqtt_msg{ qos = Qos,
- topic = Topic,
- dup = Dup,
- message_id = MessageId,
- payload = Payload },
- PState = #proc_state{ channels = {ChQos0, ChQos1},
- exchange = Exchange,
- unacked_pubs = UnackedPubs,
- awaiting_seqno = SeqNo,
- mqtt2amqp_fun = Mqtt2AmqpFun }) ->
- RoutingKey = Mqtt2AmqpFun(Topic),
- Method = #'basic.publish'{ exchange = Exchange,
- routing_key = RoutingKey },
+%% To save memory, we only store the queue_qos1 value in process state if there is a QoS 1 subscription.
+%% We store it in the process state such that we don't have to build the binary on every PUBACK we receive.
+maybe_set_queue_qos1(?QOS_1, State = #state{cfg = Cfg = #cfg{queue_qos1 = undefined}}) ->
+ State#state{cfg = Cfg#cfg{queue_qos1 = queue_name(?QOS_1, State)}};
+maybe_set_queue_qos1(_, State) ->
+ State.
+
+bind(QueueName, TopicName, State) ->
+ binding_action_with_checks({QueueName, TopicName, fun rabbit_binding:add/2}, State).
+unbind(QueueName, TopicName, State) ->
+ binding_action_with_checks({QueueName, TopicName, fun rabbit_binding:remove/2}, State).
+
+binding_action_with_checks(Input, State) ->
+ %% Same permission checks required for both binding and unbinding
+ %% queue to / from topic exchange.
+ rabbit_misc:pipeline(
+ [fun check_queue_write_access/2,
+ fun check_exchange_read_access/2,
+ fun check_topic_access/2,
+ fun binding_action/2],
+ Input, State).
+
+check_queue_write_access(
+ {QueueName, _, _},
+ #state{auth_state = #auth_state{
+ user = User,
+ authz_ctx = AuthzCtx}}) ->
+ %% write access to queue required for queue.(un)bind
+ check_resource_access(User, QueueName, write, AuthzCtx).
+
+check_exchange_read_access(
+ _, #state{cfg = #cfg{exchange = ExchangeName},
+ auth_state = #auth_state{
+ user = User,
+ authz_ctx = AuthzCtx}}) ->
+ %% read access to exchange required for queue.(un)bind
+ check_resource_access(User, ExchangeName, read, AuthzCtx).
+
+check_topic_access({_, TopicName, _}, State) ->
+ check_topic_access(TopicName, read, State).
+
+binding_action(
+ {QueueName, TopicName, BindingFun},
+ #state{cfg = #cfg{exchange = ExchangeName},
+ auth_state = #auth_state{user = #user{username = Username}}}) ->
+ RoutingKey = mqtt_to_amqp(TopicName),
+ Binding = #binding{source = ExchangeName,
+ destination = QueueName,
+ key = RoutingKey},
+ BindingFun(Binding, Username).
+
+publish_to_queues(
+ #mqtt_msg{qos = Qos,
+ topic = Topic,
+ dup = Dup,
+ packet_id = PacketId,
+ payload = Payload},
+ #state{cfg = #cfg{exchange = ExchangeName,
+ delivery_flow = Flow,
+ conn_name = ConnName,
+ trace_state = TraceState},
+ auth_state = #auth_state{username = Username}
+ } = State) ->
+ RoutingKey = mqtt_to_amqp(Topic),
+ Confirm = Qos > ?QOS_0,
Headers = [{<<"x-mqtt-publish-qos">>, byte, Qos},
{<<"x-mqtt-dup">>, bool, Dup}],
- Msg = #amqp_msg{ props = #'P_basic'{ headers = Headers,
- delivery_mode = delivery_mode(Qos)},
- payload = Payload },
- {UnackedPubs1, Ch, SeqNo1} =
- case Qos =:= ?QOS_1 andalso MessageId =/= undefined of
- true -> {gb_trees:enter(SeqNo, MessageId, UnackedPubs), ChQos1,
- SeqNo + 1};
- false -> {UnackedPubs, ChQos0, SeqNo}
- end,
- amqp_channel:cast_flow(Ch, Method, Msg),
- PState #proc_state{ unacked_pubs = UnackedPubs1,
- awaiting_seqno = SeqNo1 }.
-
-adapter_info(Sock, ProtoName) ->
- amqp_connection:socket_adapter_info(Sock, {ProtoName, "N/A"}).
-
-set_proto_version(AdapterInfo = #amqp_adapter_info{protocol = {Proto, _}}, Vsn) ->
- AdapterInfo#amqp_adapter_info{protocol = {Proto,
- human_readable_mqtt_version(Vsn)}}.
-
-human_readable_mqtt_version(3) ->
- "3.1.0";
-human_readable_mqtt_version(4) ->
- "3.1.1";
-human_readable_mqtt_version(_) ->
- "N/A".
-
-serialise_and_send_to_client(Frame, #proc_state{ socket = Sock }) ->
- try rabbit_net:port_command(Sock, rabbit_mqtt_frame:serialise(Frame)) of
- Res ->
- Res
- catch _:Error ->
- rabbit_log_connection:error("MQTT: a socket write failed, the socket might already be closed"),
- rabbit_log_connection:debug("Failed to write to socket ~tp, error: ~tp, frame: ~tp",
- [Sock, Error, Frame])
+ Props = #'P_basic'{
+ headers = Headers,
+ delivery_mode = delivery_mode(Qos)},
+ {ClassId, _MethodId} = rabbit_framing_amqp_0_9_1:method_id('basic.publish'),
+ Content = #content{
+ class_id = ClassId,
+ properties = Props,
+ properties_bin = none,
+ protocol = none,
+ payload_fragments_rev = [Payload]
+ },
+ BasicMessage = #basic_message{
+ exchange_name = ExchangeName,
+ routing_keys = [RoutingKey],
+ content = Content,
+ id = <<>>, %% GUID set in rabbit_classic_queue
+ is_persistent = Confirm
+ },
+ Delivery = #delivery{
+ mandatory = false,
+ confirm = Confirm,
+ sender = self(),
+ message = BasicMessage,
+ msg_seq_no = PacketId,
+ flow = Flow
+ },
+ case rabbit_exchange:lookup(ExchangeName) of
+ {ok, Exchange} ->
+ QNames = rabbit_exchange:route(Exchange, Delivery),
+ rabbit_trace:tap_in(BasicMessage, QNames, ConnName, Username, TraceState),
+ deliver_to_queues(Delivery, QNames, State);
+ {error, not_found} ->
+ ?LOG_ERROR("~s not found", [rabbit_misc:rs(ExchangeName)]),
+ {error, exchange_not_found, State}
end.
-close_connection(PState = #proc_state{ connection = undefined }) ->
- PState;
-close_connection(PState = #proc_state{ connection = Connection,
- client_id = ClientId }) ->
- % todo: maybe clean session
- case ClientId of
- undefined -> ok;
- _ ->
- case rabbit_mqtt_collector:unregister(ClientId, self()) of
- ok -> ok;
- %% ignore as we are shutting down
- {timeout, _} -> ok
- end
- end,
- %% ignore noproc or other exceptions, we are shutting down
- catch amqp_connection:close(Connection),
- PState #proc_state{ channels = {undefined, undefined},
- connection = undefined }.
+deliver_to_queues(Delivery,
+ RoutedToQNames,
+ State0 = #state{queue_states = QStates0,
+ cfg = #cfg{proto_ver = ProtoVer}}) ->
+ Qs0 = rabbit_amqqueue:lookup(RoutedToQNames),
+ Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0),
+ case rabbit_queue_type:deliver(Qs, Delivery, QStates0) of
+ {ok, QStates, Actions} ->
+ rabbit_global_counters:messages_routed(ProtoVer, length(Qs)),
+ State = process_routing_confirm(Delivery, Qs,
+ State0#state{queue_states = QStates}),
+ %% Actions must be processed after registering confirms as actions may
+ %% contain rejections of publishes.
+ {ok, handle_queue_actions(Actions, State)};
+ {error, Reason} ->
+ ?LOG_ERROR("Failed to deliver message with packet_id=~p to queues: ~p",
+ [Delivery#delivery.msg_seq_no, Reason]),
+ {error, Reason, State0}
+ end.
+process_routing_confirm(#delivery{confirm = false},
+ [], State = #state{cfg = #cfg{proto_ver = ProtoVer}}) ->
+ rabbit_global_counters:messages_unroutable_dropped(ProtoVer, 1),
+ State;
+process_routing_confirm(#delivery{confirm = true,
+ msg_seq_no = undefined},
+ [], State = #state{cfg = #cfg{proto_ver = ProtoVer}}) ->
+ %% unroutable will message with QoS > 0
+ rabbit_global_counters:messages_unroutable_dropped(ProtoVer, 1),
+ State;
+process_routing_confirm(#delivery{confirm = true,
+ msg_seq_no = PktId},
+ [], State = #state{cfg = #cfg{proto_ver = ProtoVer}}) ->
+ rabbit_global_counters:messages_unroutable_returned(ProtoVer, 1),
+ %% MQTT 5 spec:
+ %% If the Server knows that there are no matching subscribers, it MAY use
+ %% Reason Code 0x10 (No matching subscribers) instead of 0x00 (Success).
+ send_puback(PktId, State),
+ State;
+process_routing_confirm(#delivery{confirm = false}, _, State) ->
+ State;
+process_routing_confirm(#delivery{confirm = true,
+ msg_seq_no = undefined}, [_|_], State) ->
+ %% routable will message with QoS > 0
+ State;
+process_routing_confirm(#delivery{confirm = true,
+ msg_seq_no = PktId},
+ Qs, State = #state{unacked_client_pubs = U0}) ->
+ QNames = lists:map(fun amqqueue:get_name/1, Qs),
+ U = rabbit_mqtt_confirms:insert(PktId, QNames, U0),
+ State#state{unacked_client_pubs = U}.
+
+send_puback(PktIds0, State)
+ when is_list(PktIds0) ->
+ %% Classic queues confirm messages unordered.
+ %% Let's sort them here assuming most MQTT clients send with an increasing packet identifier.
+ PktIds = lists:usort(PktIds0),
+ lists:foreach(fun(Id) ->
+ send_puback(Id, State)
+ end, PktIds);
+send_puback(PktId, State = #state{cfg = #cfg{send_fun = SendFun,
+ proto_ver = ProtoVer}}) ->
+ rabbit_global_counters:messages_confirmed(ProtoVer, 1),
+ SendFun(
+ #mqtt_packet{fixed = #mqtt_packet_fixed{type = ?PUBACK},
+ variable = #mqtt_packet_publish{packet_id = PktId}},
+ State).
+
+serialise_and_send_to_client(Packet, #state{cfg = #cfg{proto_ver = ProtoVer,
+ socket = Sock}}) ->
+ Data = rabbit_mqtt_packet:serialise(Packet, ProtoVer),
+ try rabbit_net:port_command(Sock, Data)
+ catch error:Error ->
+ ?LOG_ERROR("MQTT: a socket write failed: ~p", [Error]),
+ ?LOG_DEBUG("MQTT failed to write to socket ~p, error: ~p, "
+ "fixed packet header: ~p, variable packet header: ~p",
+ [Sock, Error, Packet#mqtt_packet.fixed, Packet#mqtt_packet.variable])
+ end.
+
+-spec serialise(#mqtt_packet{}, state()) ->
+ iodata().
+serialise(Packet, #state{cfg = #cfg{proto_ver = ProtoVer}}) ->
+ rabbit_mqtt_packet:serialise(Packet, ProtoVer).
+
+-spec terminate(boolean(), binary(), atom(), state()) ->
+ ok.
+terminate(SendWill, ConnName, ProtoFamily, State) ->
+ maybe_send_will(SendWill, ConnName, State),
+ Infos = [{name, ConnName},
+ {node, node()},
+ {pid, self()},
+ {disconnected_at, os:system_time(milli_seconds)}
+ ] ++ additional_connection_closed_info(ProtoFamily, State),
+ rabbit_core_metrics:connection_closed(self()),
+ rabbit_event:notify(connection_closed, Infos),
+ rabbit_networking:unregister_non_amqp_connection(self()),
+ maybe_unregister_client(State),
+ maybe_decrement_consumer(State),
+ maybe_decrement_publisher(State),
+ maybe_delete_mqtt_qos0_queue(State).
+
+maybe_send_will(
+ true, ConnStr,
+ #state{cfg = #cfg{retainer_pid = RPid,
+ will_msg = WillMsg = #mqtt_msg{retain = Retain,
+ topic = Topic}}
+ } = State) ->
+ ?LOG_DEBUG("sending MQTT will message to topic ~s on connection ~s",
+ [Topic, ConnStr]),
+ case check_topic_access(Topic, write, State) of
+ ok ->
+ _ = publish_to_queues(WillMsg, State),
+ case Retain of
+ false ->
+ ok;
+ true ->
+ hand_off_to_retainer(RPid, Topic, WillMsg)
+ end;
+ {error, access_refused = Reason} ->
+ ?LOG_ERROR("failed to send will message: ~p", [Reason])
+ end;
+maybe_send_will(_, _, _) ->
+ ok.
+
+additional_connection_closed_info(
+ ProtoFamily,
+ State = #state{auth_state = #auth_state{vhost = VHost,
+ username = Username}}) ->
+ [{protocol, {ProtoFamily, proto_version_tuple(State)}},
+ {vhost, VHost},
+ {user, Username}];
+additional_connection_closed_info(_, _) ->
+ [].
+
+maybe_unregister_client(#state{cfg = #cfg{client_id = ClientId}})
+ when ClientId =/= undefined ->
+ case rabbit_mqtt_ff:track_client_id_in_ra() of
+ true ->
+ %% ignore any errors as we are shutting down
+ rabbit_mqtt_collector:unregister(ClientId, self());
+ false ->
+ ok
+ end;
+maybe_unregister_client(_) ->
+ ok.
+
+maybe_delete_mqtt_qos0_queue(
+ State = #state{cfg = #cfg{clean_sess = true},
+ auth_state = #auth_state{username = Username}}) ->
+ case get_queue(?QOS_0, State) of
+ {ok, Q} ->
+ %% double check we delete the right queue
+ case {amqqueue:get_type(Q), amqqueue:get_pid(Q)} of
+ {?QUEUE_TYPE_QOS_0, Pid}
+ when Pid =:= self() ->
+ rabbit_queue_type:delete(Q, false, false, Username);
+ _ ->
+ ok
+ end;
+ _ ->
+ ok
+ end;
+maybe_delete_mqtt_qos0_queue(_) ->
+ ok.
+
+delete_queue(QName, Username) ->
+ rabbit_amqqueue:with(
+ QName,
+ fun (Q) ->
+ rabbit_queue_type:delete(Q, false, false, Username)
+ end,
+ fun (not_found) ->
+ ok;
+ ({absent, Q, crashed}) ->
+ rabbit_classic_queue:delete_crashed(Q, Username);
+ ({absent, Q, stopped}) ->
+ rabbit_classic_queue:delete_crashed(Q, Username);
+ ({absent, _Q, _Reason}) ->
+ ok
+ end).
+
+-spec handle_pre_hibernate() -> ok.
handle_pre_hibernate() ->
+ erase(permission_cache),
erase(topic_permission_cache),
ok.
+-spec handle_ra_event(register_timeout
+| {applied, [{reference(), ok}]}
+| {not_leader, term(), reference()}, state()) -> state().
handle_ra_event({applied, [{Corr, ok}]},
- PState = #proc_state{register_state = {pending, Corr}}) ->
+ State = #state{register_state = {pending, Corr}}) ->
%% success case - command was applied transition into registered state
- PState#proc_state{register_state = registered};
+ State#state{register_state = registered};
handle_ra_event({not_leader, Leader, Corr},
- PState = #proc_state{register_state = {pending, Corr},
- client_id = ClientId}) ->
- %% retry command against actual leader
- {ok, NewCorr} = rabbit_mqtt_collector:register(Leader, ClientId, self()),
- PState#proc_state{register_state = {pending, NewCorr}};
+ State = #state{register_state = {pending, Corr},
+ cfg = #cfg{client_id = ClientId}}) ->
+ case rabbit_mqtt_ff:track_client_id_in_ra() of
+ true ->
+ %% retry command against actual leader
+ {ok, NewCorr} = rabbit_mqtt_collector:register(Leader, ClientId, self()),
+ State#state{register_state = {pending, NewCorr}};
+ false ->
+ State
+ end;
handle_ra_event(register_timeout,
- PState = #proc_state{register_state = {pending, _Corr},
- client_id = ClientId}) ->
- {ok, NewCorr} = rabbit_mqtt_collector:register(ClientId, self()),
- PState#proc_state{register_state = {pending, NewCorr}};
-handle_ra_event(register_timeout, PState) ->
- PState;
-handle_ra_event(Evt, PState) ->
- %% log these?
- rabbit_log:debug("unhandled ra_event: ~w ", [Evt]),
- PState.
-
-check_publish(TopicName, Fn, PState) ->
- case check_topic_access(TopicName, write, PState) of
- ok -> Fn();
- _ -> {error, unauthorized, PState}
- end.
-
-check_topic_access(TopicName, Access,
- #proc_state{
- auth_state = #auth_state{user = User = #user{username = Username},
- vhost = VHost},
- exchange = Exchange,
- client_id = ClientId,
- mqtt2amqp_fun = Mqtt2AmqpFun }) ->
- Cache =
- case get(topic_permission_cache) of
- undefined -> [];
- Other -> Other
- end,
-
- Key = {TopicName, Username, ClientId, VHost, Exchange, Access},
+ State = #state{register_state = {pending, _Corr},
+ cfg = #cfg{client_id = ClientId}}) ->
+ case rabbit_mqtt_ff:track_client_id_in_ra() of
+ true ->
+ {ok, NewCorr} = rabbit_mqtt_collector:register(ClientId, self()),
+ State#state{register_state = {pending, NewCorr}};
+ false ->
+ State
+ end;
+handle_ra_event(register_timeout, State) ->
+ State;
+handle_ra_event(Evt, State) ->
+ ?LOG_DEBUG("unhandled ra_event: ~w ", [Evt]),
+ State.
+
+-spec handle_down(term(), state()) ->
+ {ok, state()} | {error, Reason :: any()}.
+handle_down({{'DOWN', QName}, _MRef, process, QPid, Reason},
+ State0 = #state{queue_states = QStates0,
+ unacked_client_pubs = U0}) ->
+ credit_flow:peer_down(QPid),
+ case rabbit_queue_type:handle_down(QPid, QName, Reason, QStates0) of
+ {ok, QStates1, Actions} ->
+ State1 = State0#state{queue_states = QStates1},
+ try handle_queue_actions(Actions, State1) of
+ State ->
+ {ok, State}
+ catch throw:consuming_queue_down ->
+ {error, consuming_queue_down}
+ end;
+ {eol, QStates1, QRef} ->
+ {ConfirmPktIds, U} = rabbit_mqtt_confirms:remove_queue(QRef, U0),
+ QStates = rabbit_queue_type:remove(QRef, QStates1),
+ State = State0#state{queue_states = QStates,
+ unacked_client_pubs = U},
+ send_puback(ConfirmPktIds, State),
+ {ok, State}
+ end.
+
+-spec handle_queue_event(
+ {queue_event, rabbit_amqqueue:name() | ?QUEUE_TYPE_QOS_0, term()}, state()) ->
+ {ok, state()} | {error, Reason :: any(), state()}.
+handle_queue_event({queue_event, ?QUEUE_TYPE_QOS_0, Msg},
+ State0 = #state{qos0_messages_dropped = N}) ->
+ State = case drop_qos0_message(State0) of
+ false ->
+ deliver_one_to_client(Msg, false, State0);
+ true ->
+ State0#state{qos0_messages_dropped = N + 1}
+ end,
+ {ok, State};
+handle_queue_event({queue_event, QName, Evt},
+ State0 = #state{queue_states = QStates0,
+ unacked_client_pubs = U0}) ->
+ case rabbit_queue_type:handle_event(QName, Evt, QStates0) of
+ {ok, QStates, Actions} ->
+ State1 = State0#state{queue_states = QStates},
+ State = handle_queue_actions(Actions, State1),
+ {ok, State};
+ {eol, Actions} ->
+ State1 = handle_queue_actions(Actions, State0),
+ {ConfirmPktIds, U} = rabbit_mqtt_confirms:remove_queue(QName, U0),
+ QStates = rabbit_queue_type:remove(QName, QStates0),
+ State = State1#state{queue_states = QStates,
+ unacked_client_pubs = U},
+ send_puback(ConfirmPktIds, State),
+ {ok, State};
+ {protocol_error, _Type, _Reason, _ReasonArgs} = Error ->
+ {error, Error, State0}
+ end.
+
+handle_queue_actions(Actions, #state{} = State0) ->
+ lists:foldl(
+ fun ({deliver, ?CONSUMER_TAG, Ack, Msgs}, S) ->
+ deliver_to_client(Msgs, Ack, S);
+ ({settled, QName, PktIds}, S = #state{unacked_client_pubs = U0}) ->
+ {ConfirmPktIds, U} = rabbit_mqtt_confirms:confirm(PktIds, QName, U0),
+ send_puback(ConfirmPktIds, S),
+ S#state{unacked_client_pubs = U};
+ ({rejected, _QName, PktIds}, S = #state{unacked_client_pubs = U0}) ->
+ %% Negative acks are supported in MQTT v5 only.
+ %% Therefore, in MQTT v3 and v4 we ignore rejected messages.
+ U = lists:foldl(
+ fun(PktId, Acc0) ->
+ case rabbit_mqtt_confirms:reject(PktId, Acc0) of
+ {ok, Acc} -> Acc;
+ {error, not_found} -> Acc0
+ end
+ end, U0, PktIds),
+ S#state{unacked_client_pubs = U};
+ ({block, QName}, S = #state{queues_soft_limit_exceeded = QSLE}) ->
+ S#state{queues_soft_limit_exceeded = sets:add_element(QName, QSLE)};
+ ({unblock, QName}, S = #state{queues_soft_limit_exceeded = QSLE}) ->
+ S#state{queues_soft_limit_exceeded = sets:del_element(QName, QSLE)};
+ ({queue_down, QName}, S) ->
+ handle_queue_down(QName, S)
+ end, State0, Actions).
+
+handle_queue_down(QName, State0 = #state{cfg = #cfg{client_id = ClientId}}) ->
+ %% Classic queue is down.
+ case rabbit_amqqueue:lookup(QName) of
+ {ok, Q} ->
+ case rabbit_mqtt_util:qos_from_queue_name(QName, ClientId) of
+ no_consuming_queue ->
+ State0;
+ QoS ->
+ %% Consuming classic queue is down.
+ %% Let's try to re-consume: HA failover for classic mirrored queues.
+ case consume(Q, QoS, State0) of
+ {ok, State} ->
+ State;
+ {error, _Reason} ->
+ ?LOG_INFO("Terminating MQTT connection because consuming ~s is down.",
+ [rabbit_misc:rs(QName)]),
+ throw(consuming_queue_down)
+ end
+ end;
+ {error, not_found} ->
+ State0
+ end.
+
+deliver_to_client(Msgs, Ack, State) ->
+ lists:foldl(fun(Msg, S) ->
+ deliver_one_to_client(Msg, Ack, S)
+ end, State, Msgs).
+
+deliver_one_to_client(Msg = {QNameOrType, QPid, QMsgId, _Redelivered,
+ #basic_message{content = #content{properties = #'P_basic'{headers = Headers}}}},
+ AckRequired, State0) ->
+ PublisherQoS = case rabbit_mqtt_util:table_lookup(Headers, <<"x-mqtt-publish-qos">>) of
+ {byte, QoS0} ->
+ QoS0;
+ undefined ->
+ %% non-MQTT publishes are assumed to be QoS 1 regardless of delivery_mode
+ ?QOS_1
+ end,
+ SubscriberQoS = case AckRequired of
+ true ->
+ ?QOS_1;
+ false ->
+ ?QOS_0
+ end,
+ QoS = effective_qos(PublisherQoS, SubscriberQoS),
+ State1 = maybe_publish_to_client(Msg, QoS, State0),
+ State = maybe_auto_ack(AckRequired, QoS, QNameOrType, QMsgId, State1),
+ ok = maybe_notify_sent(QNameOrType, QPid, State),
+ State.
+
+-spec effective_qos(qos(), qos()) -> qos().
+effective_qos(PublisherQoS, SubscriberQoS) ->
+ %% "The QoS of Application Messages sent in response to a Subscription MUST be the minimum
+ %% of the QoS of the originally published message and the Maximum QoS granted by the Server
+ %% [MQTT-3.8.4-8]."
+ erlang:min(PublisherQoS, SubscriberQoS).
+
+maybe_publish_to_client({_, _, _, _Redelivered = true, _}, ?QOS_0, State) ->
+ %% Do not redeliver to MQTT subscriber who gets message at most once.
+ State;
+maybe_publish_to_client(
+ {QNameOrType, _QPid, QMsgId, Redelivered,
+ #basic_message{
+ routing_keys = [RoutingKey | _CcRoutes],
+ content = #content{payload_fragments_rev = FragmentsRev}}} = Msg,
+ QoS, State0 = #state{cfg = #cfg{send_fun = SendFun}}) ->
+ {PacketId, State} = msg_id_to_packet_id(QMsgId, QoS, State0),
+ Packet =
+ #mqtt_packet{
+ fixed = #mqtt_packet_fixed{
+ type = ?PUBLISH,
+ qos = QoS,
+ %% "The value of the DUP flag from an incoming PUBLISH packet is not
+ %% propagated when the PUBLISH Packet is sent to subscribers by the Server.
+ %% The DUP flag in the outgoing PUBLISH packet is set independently to the
+ %% incoming PUBLISH packet, its value MUST be determined solely by whether
+ %% the outgoing PUBLISH packet is a retransmission [MQTT-3.3.1-3]."
+ %% Therefore, we do not consider header value <<"x-mqtt-dup">> here.
+ dup = Redelivered},
+ variable = #mqtt_packet_publish{
+ packet_id = PacketId,
+ topic_name = amqp_to_mqtt(RoutingKey)},
+ payload = lists:reverse(FragmentsRev)},
+ SendFun(Packet, State),
+ trace_tap_out(Msg, State),
+ message_delivered(QNameOrType, Redelivered, QoS, State),
+ State.
+
+msg_id_to_packet_id(_, ?QOS_0, State) ->
+ %% "A PUBLISH packet MUST NOT contain a Packet Identifier if its QoS value is set to 0 [MQTT-2.2.1-2]."
+ {undefined, State};
+msg_id_to_packet_id(QMsgId, ?QOS_1, #state{packet_id = PktId,
+ unacked_server_pubs = U} = State) ->
+ {PktId, State#state{packet_id = increment_packet_id(PktId),
+ unacked_server_pubs = maps:put(PktId, QMsgId, U)}}.
+
+-spec increment_packet_id(packet_id()) -> packet_id().
+increment_packet_id(Id)
+ when Id >= 16#ffff ->
+ 1;
+increment_packet_id(Id) ->
+ Id + 1.
+
+maybe_auto_ack(_AckRequired = true, ?QOS_0, QName, QMsgId,
+ State = #state{queue_states = QStates0}) ->
+ {ok, QStates, Actions} = rabbit_queue_type:settle(QName, complete, ?CONSUMER_TAG, [QMsgId], QStates0),
+ handle_queue_actions(Actions, State#state{queue_states = QStates});
+maybe_auto_ack(_, _, _, _, State) ->
+ State.
+
+maybe_notify_sent(?QUEUE_TYPE_QOS_0, _, _) ->
+ ok;
+maybe_notify_sent(QName, QPid, #state{queue_states = QStates}) ->
+ case rabbit_queue_type:module(QName, QStates) of
+ {ok, rabbit_classic_queue} ->
+ rabbit_amqqueue:notify_sent(QPid, self());
+ _ ->
+ ok
+ end.
+
+trace_tap_out(Msg = {#resource{}, _, _, _, _},
+ #state{auth_state = #auth_state{username = Username},
+ cfg = #cfg{conn_name = ConnName,
+ trace_state = TraceState}}) ->
+ rabbit_trace:tap_out(Msg, ConnName, Username, TraceState);
+trace_tap_out(Msg0 = {?QUEUE_TYPE_QOS_0, _, _, _, _},
+ State = #state{cfg = #cfg{trace_state = TraceState}}) ->
+ case rabbit_trace:enabled(TraceState) of
+ false ->
+ ok;
+ true ->
+ %% Pay penalty of creating queue name only if tracing is enabled.
+ QName = queue_name(?QOS_0, State),
+ Msg = setelement(1, Msg0, QName),
+ trace_tap_out(Msg, State)
+ end.
+
+publish_to_queues_with_checks(
+ TopicName, PublishFun,
+ #state{cfg = #cfg{exchange = Exchange},
+ auth_state = #auth_state{user = User,
+ authz_ctx = AuthzCtx}
+ } = State) ->
+ case check_resource_access(User, Exchange, write, AuthzCtx) of
+ ok ->
+ case check_topic_access(TopicName, write, State) of
+ ok ->
+ PublishFun();
+ {error, access_refused} ->
+ {error, unauthorized, State}
+ end;
+ {error, access_refused} ->
+ {error, unauthorized, State}
+ end.
+
+check_resource_access(User, Resource, Perm, Context) ->
+ V = {Resource, Context, Perm},
+ Cache = case get(permission_cache) of
+ undefined -> [];
+ Other -> Other
+ end,
+ case lists:member(V, Cache) of
+ true ->
+ ok;
+ false ->
+ try rabbit_access_control:check_resource_access(User, Resource, Perm, Context) of
+ ok ->
+ CacheTail = lists:sublist(Cache, ?MAX_PERMISSION_CACHE_SIZE-1),
+ put(permission_cache, [V | CacheTail]),
+ ok
+ catch
+ exit:#amqp_error{name = access_refused,
+ explanation = Msg} ->
+ ?LOG_ERROR("MQTT resource access refused: ~s", [Msg]),
+ {error, access_refused}
+ end
+ end.
+
+check_topic_access(
+ TopicName, Access,
+ #state{auth_state = #auth_state{user = User = #user{username = Username},
+ vhost = VHost,
+ authz_ctx = AuthzCtx},
+ cfg = #cfg{client_id = ClientId,
+ exchange = #resource{name = ExchangeBin}}}) ->
+ Cache = case get(topic_permission_cache) of
+ undefined -> [];
+ Other -> Other
+ end,
+ Key = {TopicName, Username, ClientId, VHost, ExchangeBin, Access},
case lists:member(Key, Cache) of
true ->
ok;
false ->
Resource = #resource{virtual_host = VHost,
kind = topic,
- name = Exchange},
-
- RoutingKey = Mqtt2AmqpFun(TopicName),
+ name = ExchangeBin},
+ RoutingKey = mqtt_to_amqp(TopicName),
Context = #{routing_key => RoutingKey,
- variable_map => #{
- <<"username">> => Username,
- <<"vhost">> => VHost,
- <<"client_id">> => rabbit_data_coercion:to_binary(ClientId)
- }
- },
-
+ variable_map => AuthzCtx},
try rabbit_access_control:check_topic_access(User, Resource, Access, Context) of
ok ->
- CacheTail = lists:sublist(Cache, ?MAX_TOPIC_PERMISSION_CACHE_SIZE - 1),
+ CacheTail = lists:sublist(Cache, ?MAX_PERMISSION_CACHE_SIZE - 1),
put(topic_permission_cache, [Key | CacheTail]),
- ok;
- R ->
- R
+ ok
catch
- _:{amqp_error, access_refused, Msg, _} ->
- rabbit_log:error("operation resulted in an error (access_refused): ~tp", [Msg]),
- {error, access_refused};
- _:Error ->
- rabbit_log:error("~tp", [Error]),
+ exit:#amqp_error{name = access_refused,
+ explanation = Msg} ->
+ ?LOG_ERROR("MQTT topic access refused: ~s", [Msg]),
{error, access_refused}
end
end.
-info(consumer_tags, #proc_state{consumer_tags = Val}) -> Val;
-info(unacked_pubs, #proc_state{unacked_pubs = Val}) -> Val;
-info(awaiting_ack, #proc_state{awaiting_ack = Val}) -> Val;
-info(awaiting_seqno, #proc_state{awaiting_seqno = Val}) -> Val;
-info(message_id, #proc_state{message_id = Val}) -> Val;
-info(client_id, #proc_state{client_id = Val}) ->
- rabbit_data_coercion:to_binary(Val);
-info(clean_sess, #proc_state{clean_sess = Val}) -> Val;
-info(will_msg, #proc_state{will_msg = Val}) -> Val;
-info(channels, #proc_state{channels = Val}) -> Val;
-info(exchange, #proc_state{exchange = Val}) -> Val;
-info(adapter_info, #proc_state{adapter_info = Val}) -> Val;
-info(ssl_login_name, #proc_state{ssl_login_name = Val}) -> Val;
-info(retainer_pid, #proc_state{retainer_pid = Val}) -> Val;
-info(user, #proc_state{auth_state = #auth_state{username = Val}}) -> Val;
-info(vhost, #proc_state{auth_state = #auth_state{vhost = Val}}) -> Val;
-info(host, #proc_state{adapter_info = #amqp_adapter_info{host = Val}}) -> Val;
-info(port, #proc_state{adapter_info = #amqp_adapter_info{port = Val}}) -> Val;
-info(peer_host, #proc_state{adapter_info = #amqp_adapter_info{peer_host = Val}}) -> Val;
-info(peer_port, #proc_state{adapter_info = #amqp_adapter_info{peer_port = Val}}) -> Val;
-info(protocol, #proc_state{adapter_info = #amqp_adapter_info{protocol = Val}}) ->
- case Val of
- {Proto, Version} -> {Proto, rabbit_data_coercion:to_binary(Version)};
- Other -> Other
- end;
-info(channels, PState) -> additional_info(channels, PState);
-info(channel_max, PState) -> additional_info(channel_max, PState);
-info(frame_max, PState) -> additional_info(frame_max, PState);
-info(client_properties, PState) -> additional_info(client_properties, PState);
-info(ssl, PState) -> additional_info(ssl, PState);
-info(ssl_protocol, PState) -> additional_info(ssl_protocol, PState);
-info(ssl_key_exchange, PState) -> additional_info(ssl_key_exchange, PState);
-info(ssl_cipher, PState) -> additional_info(ssl_cipher, PState);
-info(ssl_hash, PState) -> additional_info(ssl_hash, PState);
+-spec drop_qos0_message(state()) ->
+ boolean().
+drop_qos0_message(State) ->
+ mailbox_soft_limit_exceeded() andalso
+ is_socket_busy(State#state.cfg#cfg.socket).
+
+-spec mailbox_soft_limit_exceeded() ->
+ boolean().
+mailbox_soft_limit_exceeded() ->
+ case persistent_term:get(?PERSISTENT_TERM_MAILBOX_SOFT_LIMIT) of
+ Limit when Limit > 0 ->
+ case erlang:process_info(self(), message_queue_len) of
+ {message_queue_len, Len} when Len > Limit ->
+ true;
+ _ ->
+ false
+ end;
+ _ ->
+ false
+ end.
+
+is_socket_busy(Socket) ->
+ case rabbit_net:getstat(Socket, [send_pend]) of
+ {ok, [{send_pend, NumBytes}]}
+ when is_number(NumBytes) andalso NumBytes > 0 ->
+ true;
+ _ ->
+ false
+ end.
+
+-spec throttle(boolean(), boolean(), state()) -> boolean().
+throttle(Conserve, Connected, #state{queues_soft_limit_exceeded = QSLE,
+ cfg = #cfg{published = Published}}) ->
+ Conserve andalso (Published orelse not Connected) orelse
+ not sets:is_empty(QSLE) orelse
+ credit_flow:blocked().
+
+-spec info(rabbit_types:info_key(), state()) -> any().
+info(host, #state{cfg = #cfg{host = Val}}) -> Val;
+info(port, #state{cfg = #cfg{port = Val}}) -> Val;
+info(peer_host, #state{cfg = #cfg{peer_host = Val}}) -> Val;
+info(peer_port, #state{cfg = #cfg{peer_port = Val}}) -> Val;
+info(connected_at, #state{cfg = #cfg{connected_at = Val}}) -> Val;
+info(ssl_login_name, #state{cfg = #cfg{ssl_login_name = Val}}) -> Val;
+info(vhost, #state{auth_state = #auth_state{vhost = Val}}) -> Val;
+info(user_who_performed_action, S) ->
+ info(user, S);
+info(user, #state{auth_state = #auth_state{username = Val}}) -> Val;
+info(clean_sess, #state{cfg = #cfg{clean_sess = Val}}) -> Val;
+info(will_msg, #state{cfg = #cfg{will_msg = Val}}) -> Val;
+info(retainer_pid, #state{cfg = #cfg{retainer_pid = Val}}) -> Val;
+info(exchange, #state{cfg = #cfg{exchange = #resource{name = Val}}}) -> Val;
+info(prefetch, #state{cfg = #cfg{prefetch = Val}}) -> Val;
+info(messages_unconfirmed, #state{unacked_client_pubs = Val}) ->
+ rabbit_mqtt_confirms:size(Val);
+info(messages_unacknowledged, #state{unacked_server_pubs = Val}) ->
+ maps:size(Val);
+info(node, _) -> node();
+info(client_id, #state{cfg = #cfg{client_id = Val}}) -> Val;
+%% for rabbitmq_management/priv/www/js/tmpl/connection.ejs
+info(client_properties, #state{cfg = #cfg{client_id = Val}}) ->
+ [{client_id, longstr, Val}];
+info(channel_max, _) -> 0;
+%% Maximum packet size supported only in MQTT 5.0.
+info(frame_max, _) -> 0;
+%% SASL supported only in MQTT 5.0.
+info(auth_mechanism, _) -> none;
info(Other, _) -> throw({bad_argument, Other}).
+-spec ssl_login_name(rabbit_net:socket()) ->
+ none | binary().
+ssl_login_name(Sock) ->
+ case rabbit_net:peercert(Sock) of
+ {ok, C} -> case rabbit_ssl:peer_cert_auth_name(C) of
+ unsafe -> none;
+ not_found -> none;
+ Name -> Name
+ end;
+ {error, no_peercert} -> none;
+ nossl -> none
+ end.
-additional_info(Key,
- #proc_state{adapter_info =
- #amqp_adapter_info{additional_info = AddInfo}}) ->
- proplists:get_value(Key, AddInfo).
+proto_integer_to_atom(3) ->
+ ?MQTT_PROTO_V3;
+proto_integer_to_atom(4) ->
+ ?MQTT_PROTO_V4.
+
+-spec proto_version_tuple(state()) -> tuple().
+proto_version_tuple(#state{cfg = #cfg{proto_ver = ?MQTT_PROTO_V3}}) ->
+ {3, 1, 0};
+proto_version_tuple(#state{cfg = #cfg{proto_ver = ?MQTT_PROTO_V4}}) ->
+ {3, 1, 1}.
+
+maybe_increment_publisher(State = #state{cfg = Cfg = #cfg{published = false,
+ proto_ver = ProtoVer}}) ->
+ rabbit_global_counters:publisher_created(ProtoVer),
+ State#state{cfg = Cfg#cfg{published = true}};
+maybe_increment_publisher(State) ->
+ State.
+
+maybe_decrement_publisher(#state{cfg = #cfg{published = true,
+ proto_ver = ProtoVer}}) ->
+ rabbit_global_counters:publisher_deleted(ProtoVer);
+maybe_decrement_publisher(_) ->
+ ok.
-notify_received(undefined) ->
- %% no notification for quorum queues and streams
- ok;
-notify_received(DeliveryCtx) ->
- %% notification for flow control
- amqp_channel:notify_received(DeliveryCtx).
+%% Multiple subscriptions from the same connection count as one consumer.
+maybe_increment_consumer(#state{subscriptions = OldSubs},
+ #state{subscriptions = NewSubs,
+ cfg = #cfg{proto_ver = ProtoVer}})
+ when map_size(OldSubs) =:= 0 andalso
+ map_size(NewSubs) > 0 ->
+ rabbit_global_counters:consumer_created(ProtoVer);
+maybe_increment_consumer(_, _) ->
+ ok.
+
+maybe_decrement_consumer(#state{subscriptions = Subs,
+ cfg = #cfg{proto_ver = ProtoVer}})
+ when map_size(Subs) > 0 ->
+ rabbit_global_counters:consumer_deleted(ProtoVer);
+maybe_decrement_consumer(_) ->
+ ok.
+
+maybe_decrement_consumer(#state{subscriptions = OldSubs},
+ #state{subscriptions = NewSubs,
+ cfg = #cfg{proto_ver = ProtoVer}})
+ when map_size(OldSubs) > 0 andalso
+ map_size(NewSubs) =:= 0 ->
+ rabbit_global_counters:consumer_deleted(ProtoVer);
+maybe_decrement_consumer(_, _) ->
+ ok.
+
+message_acknowledged(QName, #state{queue_states = QStates,
+ cfg = #cfg{proto_ver = ProtoVer}}) ->
+ case rabbit_queue_type:module(QName, QStates) of
+ {ok, QType} ->
+ rabbit_global_counters:messages_acknowledged(ProtoVer, QType, 1);
+ _ ->
+ ok
+ end.
+
+message_delivered(?QUEUE_TYPE_QOS_0, false, ?QOS_0,
+ #state{cfg = #cfg{proto_ver = ProtoVer}}) ->
+ rabbit_global_counters:messages_delivered(ProtoVer, ?QUEUE_TYPE_QOS_0, 1),
+ %% Technically, the message is not acked to a queue at all.
+ %% However, from a user perspective it is still auto acked because:
+ %% "In automatic acknowledgement mode, a message is considered to be successfully
+ %% delivered immediately after it is sent."
+ rabbit_global_counters:messages_delivered_consume_auto_ack(ProtoVer, ?QUEUE_TYPE_QOS_0, 1);
+message_delivered(QName, Redelivered, QoS,
+ #state{queue_states = QStates,
+ cfg = #cfg{proto_ver = ProtoVer}}) ->
+ case rabbit_queue_type:module(QName, QStates) of
+ {ok, QType} ->
+ rabbit_global_counters:messages_delivered(ProtoVer, QType, 1),
+ message_delivered_ack(QoS, ProtoVer, QType),
+ message_redelivered(Redelivered, ProtoVer, QType);
+ _ ->
+ ok
+ end.
+
+message_delivered_ack(?QOS_0, ProtoVer, QType) ->
+ rabbit_global_counters:messages_delivered_consume_auto_ack(ProtoVer, QType, 1);
+message_delivered_ack(?QOS_1, ProtoVer, QType) ->
+ rabbit_global_counters:messages_delivered_consume_manual_ack(ProtoVer, QType, 1).
+
+message_redelivered(true, ProtoVer, QType) ->
+ rabbit_global_counters:messages_redelivered(ProtoVer, QType, 1);
+message_redelivered(_, _, _) ->
+ ok.
+
+-spec format_status(state()) -> map().
+format_status(
+ #state{queue_states = QState,
+ unacked_client_pubs = UnackClientPubs,
+ unacked_server_pubs = UnackSerPubs,
+ packet_id = PackID,
+ subscriptions = Subscriptions,
+ auth_state = AuthState,
+ register_state = RegisterState,
+ queues_soft_limit_exceeded = QSLE,
+ qos0_messages_dropped = Qos0MsgsDropped,
+ cfg = #cfg{
+ socket = Socket,
+ proto_ver = ProtoVersion,
+ clean_sess = CleanSess,
+ will_msg = WillMsg,
+ exchange = Exchange,
+ queue_qos1 = _,
+ published = Published,
+ ssl_login_name = SSLLoginName,
+ retainer_pid = RetainerPid,
+ delivery_flow = DeliveryFlow,
+ trace_state = TraceState,
+ prefetch = Prefetch,
+ client_id = ClientID,
+ conn_name = ConnName,
+ peer_addr = PeerAddr,
+ host = Host,
+ port = Port,
+ peer_host = PeerHost,
+ peer_port = PeerPort,
+ connected_at = ConnectedAt,
+ send_fun = _
+ }}) ->
+ Cfg = #{socket => Socket,
+ proto_ver => ProtoVersion,
+ clean_sess => CleanSess,
+ will_msg_defined => WillMsg =/= undefined,
+ exchange => Exchange,
+ published => Published,
+ ssl_login_name => SSLLoginName,
+ retainer_pid => RetainerPid,
+
+ delivery_flow => DeliveryFlow,
+ trace_state => TraceState,
+ prefetch => Prefetch,
+ client_id => ClientID,
+ conn_name => ConnName,
+ peer_addr => PeerAddr,
+ host => Host,
+ port => Port,
+ peer_host => PeerHost,
+ peer_port => PeerPort,
+ connected_at => ConnectedAt},
+ #{cfg => Cfg,
+ queue_states => rabbit_queue_type:format_status(QState),
+ unacked_client_pubs => UnackClientPubs,
+ unacked_server_pubs => UnackSerPubs,
+ packet_id => PackID,
+ subscriptions => Subscriptions,
+ auth_state => AuthState,
+ register_state => RegisterState,
+ queues_soft_limit_exceeded => QSLE,
+ qos0_messages_dropped => Qos0MsgsDropped}.
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl
new file mode 100644
index 000000000000..deb0eff622de
--- /dev/null
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl
@@ -0,0 +1,274 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2018-2023 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+%% This module is a pseudo queue type.
+%% Messages are consumed directly by the declaring MQTT connection process.
+%% In a certain sense, the declaring MQTT connection process' mailbox
+%% represents a superset of this queue.
+%%
+%% This queue type allows only stateless clients such that clients (i.e.
+%% MQTT publisher connection processes or AMQP 0.9.1 channels) can deliver
+%% messages to millions of these queues without requiring a lot of memory.
+%%
+%% All messages are delivered at most once.
+
+-module(rabbit_mqtt_qos0_queue).
+-behaviour(rabbit_queue_type).
+
+-include_lib("rabbit_common/include/rabbit.hrl").
+-include_lib("rabbit/include/amqqueue.hrl").
+
+%% Stateless rabbit_queue_type callbacks.
+-export([
+ is_stateful/0,
+ declare/2,
+ delete/4,
+ deliver/2,
+ is_enabled/0,
+ is_compatible/3,
+ is_recoverable/1,
+ recover/2,
+ purge/1,
+ policy_changed/1,
+ info/2,
+ stat/1,
+ capabilities/0,
+ notify_decorators/1
+ ]).
+
+%% Stateful rabbit_queue_type callbacks are unsupported by this queue type.
+-define(STATEFUL_CALLBACKS,
+ [
+ init/1,
+ close/1,
+ update/2,
+ consume/3,
+ cancel/5,
+ handle_event/3,
+ settle/5,
+ credit/5,
+ dequeue/5,
+ state_info/1
+ ]).
+-export(?STATEFUL_CALLBACKS).
+-dialyzer({nowarn_function, ?STATEFUL_CALLBACKS}).
+-define(UNSUPPORTED(Args), erlang:error(unsupported, Args)).
+
+-define(INFO_KEYS, [type, name, durable, auto_delete, arguments,
+ pid, owner_pid, state, messages]).
+
+-spec is_stateful() ->
+ boolean().
+is_stateful() ->
+ false.
+
+-spec declare(amqqueue:amqqueue(), node()) ->
+ {'new' | 'existing' | 'owner_died', amqqueue:amqqueue()} |
+ {'absent', amqqueue:amqqueue(), rabbit_amqqueue:absent_reason()}.
+declare(Q0, _Node) ->
+ %% The queue gets persisted such that routing to this
+ %% queue (via the topic exchange) works as usual.
+ case rabbit_amqqueue:internal_declare(Q0, false) of
+ {created, Q} ->
+ Opts = amqqueue:get_options(Q),
+ ActingUser = maps:get(user, Opts, ?UNKNOWN_USER),
+ rabbit_event:notify(queue_created,
+ [{name, amqqueue:get_name(Q0)},
+ {durable, true},
+ {auto_delete, false},
+ {exclusive, true},
+ {type, amqqueue:get_type(Q0)},
+ {arguments, amqqueue:get_arguments(Q0)},
+ {user_who_performed_action, ActingUser}]),
+ {new, Q};
+ Other ->
+ Other
+ end.
+
+-spec delete(amqqueue:amqqueue(),
+ boolean(),
+ boolean(),
+ rabbit_types:username()) ->
+ rabbit_types:ok(non_neg_integer()).
+delete(Q, _IfUnused, _IfEmpty, ActingUser) ->
+ QName = amqqueue:get_name(Q),
+ log_delete(QName, amqqueue:get_exclusive_owner(Q)),
+ ok = rabbit_amqqueue:internal_delete(QName, ActingUser),
+ {ok, 0}.
+
+-spec deliver([{amqqueue:amqqueue(), stateless}], Delivery :: term()) ->
+ {[], rabbit_queue_type:actions()}.
+deliver(Qs, #delivery{message = BasicMessage,
+ confirm = Confirm,
+ msg_seq_no = SeqNo}) ->
+ Msg = {queue_event, ?MODULE,
+ {?MODULE, _QPid = none, _QMsgId = none, _Redelivered = false, BasicMessage}},
+ {Pids, Actions} =
+ case Confirm of
+ false ->
+ Pids0 = lists:map(fun({Q, stateless}) -> amqqueue:get_pid(Q) end, Qs),
+ {Pids0, []};
+ true ->
+ %% We confirm the message directly here in the queue client.
+ %% Alternatively, we could have the target MQTT connection process confirm the message.
+ %% However, given that this message might be lost anyway between target MQTT connection
+ %% process and MQTT subscriber, and we know that the MQTT subscriber wants to receive
+ %% this message at most once, we confirm here directly.
+ %% Benefits:
+ %% 1. We do not block sending the confirmation back to the publishing client just because a single
+ %% (at-most-once) target queue out of potentially many (e.g. million) queues might be unavailable.
+ %% 2. Memory usage in this (publishing) process is kept lower because the target queue name can be
+ %% directly removed from rabbit_mqtt_confirms and rabbit_confirms.
+ %% 3. Reduced network traffic across RabbitMQ nodes.
+ %% 4. Lower latency of sending publisher confirmation back to the publishing client.
+ SeqNos = [SeqNo],
+ lists:mapfoldl(fun({Q, stateless}, Actions) ->
+ {amqqueue:get_pid(Q),
+ [{settled, amqqueue:get_name(Q), SeqNos} | Actions]}
+ end, [], Qs)
+ end,
+ delegate:invoke_no_result(Pids, {gen_server, cast, [Msg]}),
+ {[], Actions}.
+
+-spec is_enabled() ->
+ boolean().
+is_enabled() ->
+ true.
+
+-spec is_compatible(boolean(), boolean(), boolean()) ->
+ boolean().
+is_compatible(_Durable = true, _Exclusive = true, _AutoDelete = false) ->
+ true;
+is_compatible(_, _, _) ->
+ false.
+
+-spec is_recoverable(amqqueue:amqqueue()) ->
+ boolean().
+is_recoverable(Q) ->
+ Pid = amqqueue:get_pid(Q),
+ OwnerPid = amqqueue:get_exclusive_owner(Q),
+ node() =:= node(Pid) andalso
+ Pid =:= OwnerPid andalso
+ not is_process_alive(Pid).
+
+%% We (mis)use the recover callback to clean up our exclusive queues
+%% which otherwise do not get cleaned up after a node crash.
+-spec recover(rabbit_types:vhost(), [amqqueue:amqqueue()]) ->
+ {Recovered :: [amqqueue:amqqueue()], Failed :: [amqqueue:amqqueue()]}.
+recover(_VHost, Queues) ->
+ lists:foreach(
+ fun(Q) ->
+ %% sanity check
+ true = is_recoverable(Q),
+ QName = amqqueue:get_name(Q),
+ log_delete(QName, amqqueue:get_exclusive_owner(Q)),
+ rabbit_amqqueue:internal_delete(QName, ?INTERNAL_USER)
+ end, Queues),
+ %% We mark the queue recovery as failed because these queues are not really
+ %% recovered, but deleted.
+ {[], Queues}.
+
+log_delete(QName, ConPid) ->
+ rabbit_log_queue:debug("Deleting ~s of type ~s because its declaring connection ~tp was closed",
+ [rabbit_misc:rs(QName), ?MODULE, ConPid]).
+
+-spec purge(amqqueue:amqqueue()) ->
+ {ok, non_neg_integer()}.
+purge(_Q) ->
+ {ok, 0}.
+
+-spec policy_changed(amqqueue:amqqueue()) ->
+ ok.
+policy_changed(_Q) ->
+ ok.
+
+-spec notify_decorators(amqqueue:amqqueue()) ->
+ ok.
+notify_decorators(_) ->
+ ok.
+
+-spec stat(amqqueue:amqqueue()) ->
+ {'ok', non_neg_integer(), non_neg_integer()}.
+stat(_Q) ->
+ {ok, 0, 0}.
+
+-spec capabilities() ->
+ #{atom() := term()}.
+capabilities() ->
+ #{}.
+
+-spec info(amqqueue:amqqueue(), all_keys | rabbit_types:info_keys()) ->
+ rabbit_types:infos().
+info(Q, all_keys)
+ when ?is_amqqueue(Q) ->
+ info(Q, ?INFO_KEYS);
+info(Q, Items)
+ when ?is_amqqueue(Q) ->
+ [{Item, i(Item, Q)} || Item <- Items].
+
+i(type, _) ->
+ 'MQTT QoS 0';
+i(name, Q) ->
+ amqqueue:get_name(Q);
+i(durable, Q) ->
+ amqqueue:is_durable(Q);
+i(auto_delete, Q) ->
+ amqqueue:is_auto_delete(Q);
+i(arguments, Q) ->
+ amqqueue:get_arguments(Q);
+i(pid, Q) ->
+ amqqueue:get_pid(Q);
+i(owner_pid, Q) ->
+ amqqueue:get_exclusive_owner(Q);
+i(state, Q) ->
+ Pid = amqqueue:get_pid(Q),
+ case erpc:call(node(Pid), erlang, is_process_alive, [Pid]) of
+ true ->
+ running;
+ false ->
+ down
+ end;
+i(messages, Q) ->
+ Pid = amqqueue:get_pid(Q),
+ case erpc:call(node(Pid), erlang, process_info, [Pid, message_queue_len]) of
+ {message_queue_len, N} ->
+ N;
+ _ ->
+ 0
+ end;
+i(_, _) ->
+ ''.
+
+init(A1) ->
+ ?UNSUPPORTED([A1]).
+
+close(A1) ->
+ ?UNSUPPORTED([A1]).
+
+update(A1,A2) ->
+ ?UNSUPPORTED([A1,A2]).
+
+consume(A1,A2,A3) ->
+ ?UNSUPPORTED([A1,A2,A3]).
+
+cancel(A1,A2,A3,A4,A5) ->
+ ?UNSUPPORTED([A1,A2,A3,A4,A5]).
+
+handle_event(A1,A2,A3) ->
+ ?UNSUPPORTED([A1,A2,A3]).
+
+settle(A1,A2,A3,A4,A5) ->
+ ?UNSUPPORTED([A1,A2,A3,A4,A5]).
+
+credit(A1,A2,A3,A4,A5) ->
+ ?UNSUPPORTED([A1,A2,A3,A4,A5]).
+
+dequeue(A1,A2,A3,A4,A5) ->
+ ?UNSUPPORTED([A1,A2,A3,A4,A5]).
+
+state_info(A1) ->
+ ?UNSUPPORTED([A1]).
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl
index 108294df56b3..39499892306d 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl
@@ -7,92 +7,107 @@
-module(rabbit_mqtt_reader).
--behaviour(gen_server2).
+-behaviour(gen_server).
+-behaviour(ranch_protocol).
--export([start_link/2]).
+-include_lib("kernel/include/logger.hrl").
+-include_lib("rabbit_common/include/logging.hrl").
+
+-export([start_link/3]).
-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
- code_change/3, terminate/2, handle_pre_hibernate/1]).
+ code_change/3, terminate/2, format_status/1]).
--export([conserve_resources/3, start_keepalive/2,
+-export([conserve_resources/3,
close_connection/2]).
--export([ssl_login_name/1]).
-export([info/2]).
--include_lib("amqp_client/include/amqp_client.hrl").
-include("rabbit_mqtt.hrl").
--define(SIMPLE_METRICS, [pid, recv_oct, send_oct, reductions]).
--define(OTHER_METRICS, [recv_cnt, send_cnt, send_pend, garbage_collection, state]).
+-type option(T) :: undefined | T.
-%%----------------------------------------------------------------------------
+-define(HIBERNATE_AFTER, 1000).
+-define(PROTO_FAMILY, 'MQTT').
+
+-record(state,
+ {socket :: rabbit_net:socket(),
+ proxy_socket :: option({rabbit_proxy_socket, any(), any()}),
+ await_recv :: boolean(),
+ deferred_recv :: option(binary()),
+ parse_state :: rabbit_mqtt_packet:state(),
+ proc_state :: rabbit_mqtt_processor:state(),
+ connection_state :: running | blocked,
+ conserve :: boolean(),
+ stats_timer :: option(rabbit_event:state()),
+ keepalive = rabbit_mqtt_keepalive:init() :: rabbit_mqtt_keepalive:state(),
+ conn_name :: binary(),
+ received_connect_packet :: boolean()
+ }).
-start_link(KeepaliveSup, Ref) ->
- Pid = proc_lib:spawn_link(?MODULE, init,
- [[KeepaliveSup, Ref]]),
+-type(state() :: #state{}).
+
+%%----------------------------------------------------------------------------
+start_link(Ref, _Transport, []) ->
+ Pid = proc_lib:spawn_link(?MODULE, init, [Ref]),
{ok, Pid}.
+-spec conserve_resources(pid(),
+ rabbit_alarm:resource_alarm_source(),
+ rabbit_alarm:resource_alert()) -> ok.
conserve_resources(Pid, _, {_, Conserve, _}) ->
Pid ! {conserve_resources, Conserve},
ok.
-info(Pid, InfoItems) ->
- case InfoItems -- ?INFO_ITEMS of
- [] -> gen_server2:call(Pid, {info, InfoItems});
- UnknownItems -> throw({bad_argument, UnknownItems})
- end.
+-spec info(pid(), rabbit_types:info_keys()) ->
+ rabbit_types:infos().
+info(Pid, Items) ->
+ gen_server:call(Pid, {info, Items}).
+-spec close_connection(pid(), Reason :: any()) -> ok.
close_connection(Pid, Reason) ->
gen_server:cast(Pid, {close_connection, Reason}).
%%----------------------------------------------------------------------------
-init([KeepaliveSup, Ref]) ->
+init(Ref) ->
process_flag(trap_exit, true),
+ logger:set_process_metadata(#{domain => ?RMQLOG_DOMAIN_CONN ++ [mqtt]}),
{ok, Sock} = rabbit_networking:handshake(Ref,
- application:get_env(rabbitmq_mqtt, proxy_protocol, false)),
+ application:get_env(?APP_NAME, proxy_protocol, false)),
RealSocket = rabbit_net:unwrap_socket(Sock),
case rabbit_net:connection_string(Sock, inbound) of
{ok, ConnStr} ->
- rabbit_log_connection:debug("MQTT accepting TCP connection ~tp (~ts)", [self(), ConnStr]),
- rabbit_alarm:register(
- self(), {?MODULE, conserve_resources, []}),
- LoginTimeout = application:get_env(rabbitmq_mqtt, login_timeout, 10_000),
+ ConnName = rabbit_data_coercion:to_binary(ConnStr),
+ ?LOG_DEBUG("MQTT accepting TCP connection ~tp (~ts)", [self(), ConnName]),
+ _ = rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}),
+ LoginTimeout = application:get_env(?APP_NAME, login_timeout, 10_000),
erlang:send_after(LoginTimeout, self(), login_timeout),
- ProcessorState = rabbit_mqtt_processor:initial_state(Sock,ssl_login_name(RealSocket)),
- gen_server2:enter_loop(?MODULE, [],
- rabbit_event:init_stats_timer(
- control_throttle(
- #state{socket = RealSocket,
- conn_name = ConnStr,
- await_recv = false,
- connection_state = running,
- received_connect_frame = false,
- keepalive = {none, none},
- keepalive_sup = KeepaliveSup,
- conserve = false,
- parse_state = rabbit_mqtt_frame:initial_state(),
- proc_state = ProcessorState }), #state.stats_timer),
- {backoff, 1000, 1000, 10000});
- {network_error, Reason} ->
- rabbit_net:fast_close(RealSocket),
- terminate({shutdown, Reason}, undefined);
- {error, enotconn} ->
+ ProcessorState = rabbit_mqtt_processor:initial_state(RealSocket, ConnName),
+ State0 = #state{socket = RealSocket,
+ proxy_socket = rabbit_net:maybe_get_proxy_socket(Sock),
+ conn_name = ConnName,
+ await_recv = false,
+ connection_state = running,
+ received_connect_packet = false,
+ conserve = false,
+ parse_state = rabbit_mqtt_packet:initial_state(),
+ proc_state = ProcessorState},
+ State1 = control_throttle(State0),
+ State = rabbit_event:init_stats_timer(State1, #state.stats_timer),
+ gen_server:enter_loop(?MODULE, [], State);
+ {error, Reason = enotconn} ->
+ ?LOG_INFO("MQTT could not get connection string: ~s", [Reason]),
rabbit_net:fast_close(RealSocket),
- terminate(shutdown, undefined);
+ ignore;
{error, Reason} ->
+ ?LOG_ERROR("MQTT could not get connection string: ~p", [Reason]),
rabbit_net:fast_close(RealSocket),
- terminate({network_error, Reason}, undefined)
+ {stop, Reason}
end.
handle_call({info, InfoItems}, _From, State) ->
- Infos = lists:map(
- fun(InfoItem) ->
- {InfoItem, info_internal(InfoItem, State)}
- end,
- InfoItems),
- {reply, Infos, State};
+ {reply, infos(InfoItems, State), State, ?HIBERNATE_AFTER};
handle_call(Msg, From, State) ->
{stop, {mqtt_unexpected_call, Msg, From}, State}.
@@ -100,53 +115,64 @@ handle_call(Msg, From, State) ->
handle_cast(duplicate_id,
State = #state{ proc_state = PState,
conn_name = ConnName }) ->
- rabbit_log_connection:warning("MQTT disconnecting client ~tp with duplicate id '~ts'",
+ ?LOG_WARNING("MQTT disconnecting client ~tp with duplicate id '~ts'",
[ConnName, rabbit_mqtt_processor:info(client_id, PState)]),
{stop, {shutdown, duplicate_id}, State};
handle_cast(decommission_node,
State = #state{ proc_state = PState,
conn_name = ConnName }) ->
- rabbit_log_connection:warning("MQTT disconnecting client ~tp with client ID '~ts' as its node is about"
- " to be decommissioned",
+ ?LOG_WARNING("MQTT disconnecting client ~tp with client ID '~ts' as its node is about"
+ " to be decommissioned",
[ConnName, rabbit_mqtt_processor:info(client_id, PState)]),
{stop, {shutdown, decommission_node}, State};
handle_cast({close_connection, Reason},
State = #state{conn_name = ConnName, proc_state = PState}) ->
- rabbit_log_connection:warning("MQTT disconnecting client ~tp with client ID '~ts', reason: ~ts",
- [ConnName, rabbit_mqtt_processor:info(client_id, PState), Reason]),
+ ?LOG_WARNING("MQTT disconnecting client ~tp with client ID '~ts', reason: ~ts",
+ [ConnName, rabbit_mqtt_processor:info(client_id, PState), Reason]),
{stop, {shutdown, server_initiated_close}, State};
+handle_cast(QueueEvent = {queue_event, _, _},
+ State = #state{proc_state = PState0}) ->
+ case rabbit_mqtt_processor:handle_queue_event(QueueEvent, PState0) of
+ {ok, PState} ->
+ maybe_process_deferred_recv(control_throttle(pstate(State, PState)));
+ {error, Reason, PState} ->
+ {stop, Reason, pstate(State, PState)}
+ end;
+
+handle_cast({force_event_refresh, Ref}, State0) ->
+ Infos = infos(?CREATION_EVENT_KEYS, State0),
+ rabbit_event:notify(connection_created, Infos, Ref),
+ State = rabbit_event:init_stats_timer(State0, #state.stats_timer),
+ {noreply, State, ?HIBERNATE_AFTER};
+
+handle_cast(refresh_config, State = #state{proc_state = PState0,
+ conn_name = ConnName}) ->
+ PState = rabbit_mqtt_processor:update_trace(ConnName, PState0),
+ {noreply, pstate(State, PState), ?HIBERNATE_AFTER};
+
handle_cast(Msg, State) ->
{stop, {mqtt_unexpected_cast, Msg}, State}.
-handle_info({#'basic.deliver'{}, #amqp_msg{}} = Delivery,
- State) ->
- %% receiving a message from a quorum queue
- %% no delivery context
- handle_info(erlang:insert_element(3, Delivery, undefined), State);
-handle_info({#'basic.deliver'{}, #amqp_msg{}, _DeliveryCtx} = Delivery,
- State = #state{ proc_state = ProcState }) ->
- callback_reply(State, rabbit_mqtt_processor:amqp_callback(Delivery,
- ProcState));
-
-handle_info(#'basic.ack'{} = Ack, State = #state{ proc_state = ProcState }) ->
- callback_reply(State, rabbit_mqtt_processor:amqp_callback(Ack, ProcState));
+handle_info(connection_created, State) ->
+ Infos = infos(?CREATION_EVENT_KEYS, State),
+ rabbit_core_metrics:connection_created(self(), Infos),
+ rabbit_event:notify(connection_created, Infos),
+ {noreply, State, ?HIBERNATE_AFTER};
-handle_info(#'basic.consume_ok'{}, State) ->
+handle_info(timeout, State) ->
+ rabbit_mqtt_processor:handle_pre_hibernate(),
{noreply, State, hibernate};
-handle_info(#'basic.cancel'{}, State) ->
- {stop, {shutdown, subscription_cancelled}, State};
-
handle_info({'EXIT', _Conn, Reason}, State) ->
{stop, {connection_died, Reason}, State};
handle_info({Tag, Sock, Data},
State = #state{ socket = Sock, connection_state = blocked })
- when Tag =:= tcp; Tag =:= ssl ->
- {noreply, State#state{ deferred_recv = Data }, hibernate};
+ when Tag =:= tcp; Tag =:= ssl ->
+ {noreply, State#state{ deferred_recv = Data }, ?HIBERNATE_AFTER};
handle_info({Tag, Sock, Data},
State = #state{ socket = Sock, connection_state = running })
@@ -163,7 +189,7 @@ handle_info({Tag, Sock, Reason}, State = #state{socket = Sock})
network_error(Reason, State);
handle_info({inet_reply, Sock, ok}, State = #state{socket = Sock}) ->
- {noreply, State, hibernate};
+ {noreply, State, ?HIBERNATE_AFTER};
handle_info({inet_reply, Sock, {error, Reason}}, State = #state{socket = Sock}) ->
network_error(Reason, State);
@@ -176,240 +202,206 @@ handle_info({bump_credit, Msg}, State) ->
credit_flow:handle_bump_msg(Msg),
maybe_process_deferred_recv(control_throttle(State));
-handle_info({start_keepalives, Keepalive},
- State = #state { keepalive_sup = KeepaliveSup, socket = Sock }) ->
- %% Only the client has the responsibility for sending keepalives
- SendFun = fun() -> ok end,
- Parent = self(),
- ReceiveFun = fun() -> Parent ! keepalive_timeout end,
- Heartbeater = rabbit_heartbeat:start(
- KeepaliveSup, Sock, 0, SendFun, Keepalive, ReceiveFun),
- {noreply, State #state { keepalive = Heartbeater }};
-
-handle_info(keepalive_timeout, State = #state {conn_name = ConnStr,
- proc_state = PState}) ->
- rabbit_log_connection:error("closing MQTT connection ~tp (keepalive timeout)", [ConnStr]),
- send_will_and_terminate(PState, {shutdown, keepalive_timeout}, State);
-
-handle_info(login_timeout, State = #state{received_connect_frame = true}) ->
- {noreply, State};
-handle_info(login_timeout, State = #state{conn_name = ConnStr}) ->
- %% The connection is also closed if the CONNECT frame happens to
+handle_info({keepalive, Req}, State = #state{keepalive = KState0,
+ conn_name = ConnName}) ->
+ case rabbit_mqtt_keepalive:handle(Req, KState0) of
+ {ok, KState} ->
+ {noreply, State#state{keepalive = KState}, ?HIBERNATE_AFTER};
+ {error, timeout} ->
+ ?LOG_ERROR("closing MQTT connection ~p (keepalive timeout)", [ConnName]),
+ {stop, {shutdown, keepalive_timeout}, State};
+ {error, Reason} ->
+ {stop, Reason, State}
+ end;
+
+handle_info(login_timeout, State = #state{received_connect_packet = true}) ->
+ {noreply, State, ?HIBERNATE_AFTER};
+handle_info(login_timeout, State = #state{conn_name = ConnName}) ->
+ %% The connection is also closed if the CONNECT packet happens to
%% be already in the `deferred_recv' buffer. This can happen while
%% the connection is blocked because of a resource alarm. However
%% we don't know what is in the buffer, it can be arbitrary bytes,
%% and we don't want to skip closing the connection in that case.
- rabbit_log_connection:error("closing MQTT connection ~tp (login timeout)", [ConnStr]),
+ ?LOG_ERROR("closing MQTT connection ~tp (login timeout)", [ConnName]),
{stop, {shutdown, login_timeout}, State};
handle_info(emit_stats, State) ->
- {noreply, emit_stats(State), hibernate};
+ {noreply, emit_stats(State), ?HIBERNATE_AFTER};
handle_info({ra_event, _From, Evt},
- #state{proc_state = PState} = State) ->
+ #state{proc_state = PState0} = State) ->
%% handle applied event to ensure registration command actually got applied
%% handle not_leader notification in case we send the command to a non-leader
- PState1 = rabbit_mqtt_processor:handle_ra_event(Evt, PState),
- {noreply, State#state{proc_state = PState1}, hibernate};
+ PState = rabbit_mqtt_processor:handle_ra_event(Evt, PState0),
+ {noreply, pstate(State, PState), ?HIBERNATE_AFTER};
+
+handle_info({{'DOWN', _QName}, _MRef, process, _Pid, _Reason} = Evt,
+ #state{proc_state = PState0} = State) ->
+ case rabbit_mqtt_processor:handle_down(Evt, PState0) of
+ {ok, PState} ->
+ maybe_process_deferred_recv(control_throttle(pstate(State, PState)));
+ {error, Reason} ->
+ {stop, {shutdown, Reason, State}}
+ end;
+
+handle_info({'DOWN', _MRef, process, QPid, _Reason}, State) ->
+ rabbit_amqqueue_common:notify_sent_queue_down(QPid),
+ {noreply, State, ?HIBERNATE_AFTER};
+
+handle_info({shutdown, Explanation} = Reason, State = #state{conn_name = ConnName}) ->
+ %% rabbitmq_management plugin requests to close connection.
+ ?LOG_INFO("MQTT closing connection ~tp: ~p", [ConnName, Explanation]),
+ {stop, Reason, State};
handle_info(Msg, State) ->
{stop, {mqtt_unexpected_msg, Msg}, State}.
-terminate(Reason, State) ->
- maybe_emit_stats(State),
- do_terminate(Reason, State).
-
-handle_pre_hibernate(State) ->
- rabbit_mqtt_processor:handle_pre_hibernate(),
- {hibernate, State}.
-
-do_terminate({network_error, {ssl_upgrade_error, closed}, ConnStr}, _State) ->
- rabbit_log_connection:error("MQTT detected TLS upgrade error on ~ts: connection closed",
- [ConnStr]);
-
-do_terminate({network_error,
- {ssl_upgrade_error,
- {tls_alert, "handshake failure"}}, ConnStr}, _State) ->
- log_tls_alert(handshake_failure, ConnStr);
-do_terminate({network_error,
- {ssl_upgrade_error,
- {tls_alert, "unknown ca"}}, ConnStr}, _State) ->
- log_tls_alert(unknown_ca, ConnStr);
-do_terminate({network_error,
- {ssl_upgrade_error,
- {tls_alert, {Err, _}}}, ConnStr}, _State) ->
- log_tls_alert(Err, ConnStr);
-do_terminate({network_error,
- {ssl_upgrade_error,
- {tls_alert, Alert}}, ConnStr}, _State) ->
- log_tls_alert(Alert, ConnStr);
-do_terminate({network_error, {ssl_upgrade_error, Reason}, ConnStr}, _State) ->
- rabbit_log_connection:error("MQTT detected TLS upgrade error on ~ts: ~tp",
- [ConnStr, Reason]);
-
-do_terminate({network_error, Reason, ConnStr}, _State) ->
- rabbit_log_connection:error("MQTT detected network error on ~ts: ~tp",
- [ConnStr, Reason]);
-
-do_terminate({network_error, Reason}, _State) ->
- rabbit_log_connection:error("MQTT detected network error: ~tp", [Reason]);
-
-do_terminate(normal, #state{proc_state = ProcState,
- conn_name = ConnName}) ->
- rabbit_mqtt_processor:close_connection(ProcState),
- rabbit_log_connection:info("closing MQTT connection ~tp (~ts)", [self(), ConnName]),
+terminate(Reason, State = #state{}) ->
+ terminate(Reason, {true, State});
+terminate(Reason, {SendWill, State = #state{conn_name = ConnName,
+ keepalive = KState0,
+ proc_state = PState}}) ->
+ KState = rabbit_mqtt_keepalive:cancel_timer(KState0),
+ maybe_emit_stats(State#state{keepalive = KState}),
+ _ = rabbit_mqtt_processor:terminate(SendWill, ConnName, ?PROTO_FAMILY, PState),
+ log_terminate(Reason, State).
+
+log_terminate({network_error, {ssl_upgrade_error, closed}, ConnName}, _State) ->
+ ?LOG_ERROR("MQTT detected TLS upgrade error on ~s: connection closed", [ConnName]);
+
+log_terminate({network_error,
+ {ssl_upgrade_error,
+ {tls_alert, "handshake failure"}}, ConnName}, _State) ->
+ log_tls_alert(handshake_failure, ConnName);
+log_terminate({network_error,
+ {ssl_upgrade_error,
+ {tls_alert, "unknown ca"}}, ConnName}, _State) ->
+ log_tls_alert(unknown_ca, ConnName);
+log_terminate({network_error,
+ {ssl_upgrade_error,
+ {tls_alert, {Err, _}}}, ConnName}, _State) ->
+ log_tls_alert(Err, ConnName);
+log_terminate({network_error,
+ {ssl_upgrade_error,
+ {tls_alert, Alert}}, ConnName}, _State) ->
+ log_tls_alert(Alert, ConnName);
+log_terminate({network_error, {ssl_upgrade_error, Reason}, ConnName}, _State) ->
+ ?LOG_ERROR("MQTT detected TLS upgrade error on ~s: ~p", [ConnName, Reason]);
+
+log_terminate({network_error, Reason, ConnName}, _State) ->
+ ?LOG_ERROR("MQTT detected network error on ~s: ~p", [ConnName, Reason]);
+
+log_terminate({network_error, Reason}, _State) ->
+ ?LOG_ERROR("MQTT detected network error: ~p", [Reason]);
+
+log_terminate(normal, #state{conn_name = ConnName}) ->
+ ?LOG_INFO("closing MQTT connection ~p (~s)", [self(), ConnName]),
ok;
-do_terminate(_Reason, #state{proc_state = ProcState}) ->
- rabbit_mqtt_processor:close_connection(ProcState),
+log_terminate(_Reason, _State) ->
ok.
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
-ssl_login_name(Sock) ->
- case rabbit_net:peercert(Sock) of
- {ok, C} -> case rabbit_ssl:peer_cert_auth_name(C) of
- unsafe -> none;
- not_found -> none;
- Name -> Name
- end;
- {error, no_peercert} -> none;
- nossl -> none
- end.
-
%%----------------------------------------------------------------------------
-log_tls_alert(handshake_failure, ConnStr) ->
- rabbit_log_connection:error("MQTT detected TLS upgrade error on ~ts: handshake failure",
- [ConnStr]);
-log_tls_alert(unknown_ca, ConnStr) ->
- rabbit_log_connection:error("MQTT detected TLS certificate verification error on ~ts: alert 'unknown CA'",
- [ConnStr]);
-log_tls_alert(Alert, ConnStr) ->
- rabbit_log_connection:error("MQTT detected TLS upgrade error on ~ts: alert ~ts",
- [ConnStr, Alert]).
-
-log_new_connection(#state{conn_name = ConnStr, proc_state = PState}) ->
- rabbit_log_connection:info("accepting MQTT connection ~tp (~ts, client id: ~ts)",
- [self(), ConnStr, rabbit_mqtt_processor:info(client_id, PState)]).
-
-process_received_bytes(<<>>, State = #state{proc_state = ProcState,
- received_connect_frame = false}) ->
- MqttConn = ProcState#proc_state.connection,
- case MqttConn of
- undefined -> ok;
- _ -> log_new_connection(State)
- end,
- {noreply, ensure_stats_timer(State#state{ received_connect_frame = true }), hibernate};
+log_tls_alert(handshake_failure, ConnName) ->
+ ?LOG_ERROR("MQTT detected TLS upgrade error on ~ts: handshake failure", [ConnName]);
+log_tls_alert(unknown_ca, ConnName) ->
+ ?LOG_ERROR("MQTT detected TLS certificate verification error on ~ts: alert 'unknown CA'",
+ [ConnName]);
+log_tls_alert(Alert, ConnName) ->
+ ?LOG_ERROR("MQTT detected TLS upgrade error on ~ts: alert ~ts", [ConnName, Alert]).
+
+process_received_bytes(<<>>, State = #state{received_connect_packet = false,
+ proc_state = PState,
+ conn_name = ConnName}) ->
+ ?LOG_INFO("Accepted MQTT connection ~p (~s, client ID: ~s)",
+ [self(), ConnName, rabbit_mqtt_processor:info(client_id, PState)]),
+ {noreply, ensure_stats_timer(State#state{received_connect_packet = true}), ?HIBERNATE_AFTER};
process_received_bytes(<<>>, State) ->
- {noreply, ensure_stats_timer(State), hibernate};
+ {noreply, ensure_stats_timer(State), ?HIBERNATE_AFTER};
process_received_bytes(Bytes,
State = #state{ parse_state = ParseState,
proc_state = ProcState,
- conn_name = ConnStr }) ->
+ conn_name = ConnName }) ->
case parse(Bytes, ParseState) of
{more, ParseState1} ->
{noreply,
ensure_stats_timer( State #state{ parse_state = ParseState1 }),
- hibernate};
- {ok, Frame, Rest} ->
- case rabbit_mqtt_processor:process_frame(Frame, ProcState) of
- {ok, ProcState1, ConnPid} ->
- PS = rabbit_mqtt_frame:initial_state(),
+ ?HIBERNATE_AFTER};
+ {ok, Packet, Rest} ->
+ case rabbit_mqtt_processor:process_packet(Packet, ProcState) of
+ {ok, ProcState1} ->
process_received_bytes(
Rest,
- State #state{ parse_state = PS,
- proc_state = ProcState1,
- connection = ConnPid });
+ State #state{parse_state = rabbit_mqtt_packet:initial_state(),
+ proc_state = ProcState1});
%% PUBLISH and more
{error, unauthorized = Reason, ProcState1} ->
- rabbit_log_connection:error("MQTT connection ~ts is closing due to an authorization failure", [ConnStr]),
+ ?LOG_ERROR("MQTT connection ~ts is closing due to an authorization failure", [ConnName]),
{stop, {shutdown, Reason}, pstate(State, ProcState1)};
- %% CONNECT frames only
+ %% CONNECT packets only
{error, unauthenticated = Reason, ProcState1} ->
- rabbit_log_connection:error("MQTT connection ~ts is closing due to an authentication failure", [ConnStr]),
+ ?LOG_ERROR("MQTT connection ~ts is closing due to an authentication failure", [ConnName]),
{stop, {shutdown, Reason}, pstate(State, ProcState1)};
- %% CONNECT frames only
+ %% CONNECT packets only
{error, invalid_client_id = Reason, ProcState1} ->
- rabbit_log_connection:error("MQTT cannot accept connection ~ts: client uses an invalid ID", [ConnStr]),
+ ?LOG_ERROR("MQTT cannot accept connection ~ts: client uses an invalid ID", [ConnName]),
{stop, {shutdown, Reason}, pstate(State, ProcState1)};
- %% CONNECT frames only
+ %% CONNECT packets only
{error, unsupported_protocol_version = Reason, ProcState1} ->
- rabbit_log_connection:error("MQTT cannot accept connection ~ts: incompatible protocol version", [ConnStr]),
+ ?LOG_ERROR("MQTT cannot accept connection ~ts: incompatible protocol version", [ConnName]),
{stop, {shutdown, Reason}, pstate(State, ProcState1)};
{error, unavailable = Reason, ProcState1} ->
- rabbit_log_connection:error("MQTT cannot accept connection ~ts due to an internal error or unavailable component",
- [ConnStr]),
+ ?LOG_ERROR("MQTT cannot accept connection ~ts due to an internal error or unavailable component",
+ [ConnName]),
{stop, {shutdown, Reason}, pstate(State, ProcState1)};
{error, Reason, ProcState1} ->
- rabbit_log_connection:error("MQTT protocol error on connection ~ts: ~tp",
- [ConnStr, Reason]),
+ ?LOG_ERROR("MQTT protocol error on connection ~ts: ~tp", [ConnName, Reason]),
{stop, {shutdown, Reason}, pstate(State, ProcState1)};
- {error, Error} ->
- rabbit_log_connection:error("MQTT detected a framing error on connection ~ts: ~tp",
- [ConnStr, Error]),
- {stop, {shutdown, Error}, State};
- {stop, ProcState1} ->
- {stop, normal, pstate(State, ProcState1)}
+ {stop, disconnect, ProcState1} ->
+ {stop, normal, {_SendWill = false, pstate(State, ProcState1)}}
end;
- {error, {cannot_parse, Error, Stacktrace}} ->
- rabbit_log_connection:error("MQTT cannot parse a frame on connection '~ts', unparseable payload: ~tp, error: {~tp, ~tp} ",
- [ConnStr, Bytes, Error, Stacktrace]),
- {stop, {shutdown, Error}, State};
+ {error, {cannot_parse, Reason, Stacktrace}} ->
+ ?LOG_ERROR("MQTT cannot parse a packet on connection '~ts', reason: ~tp, "
+ "stacktrace: ~tp, payload (first 100 bytes): ~tp",
+ [ConnName, Reason, Stacktrace, rabbit_mqtt_util:truncate_binary(Bytes, 100)]),
+ {stop, {shutdown, Reason}, State};
{error, Error} ->
- rabbit_log_connection:error("MQTT detected a framing error on connection ~ts: ~tp",
- [ConnStr, Error]),
+ ?LOG_ERROR("MQTT detected a framing error on connection ~ts: ~tp", [ConnName, Error]),
{stop, {shutdown, Error}, State}
end.
-callback_reply(State, {ok, ProcState}) ->
- {noreply, pstate(State, ProcState), hibernate};
-callback_reply(State, {error, Reason, ProcState}) ->
- {stop, Reason, pstate(State, ProcState)}.
-
-start_keepalive(_, 0 ) -> ok;
-start_keepalive(Pid, Keepalive) -> Pid ! {start_keepalives, Keepalive}.
-
-pstate(State = #state {}, PState = #proc_state{}) ->
+-spec pstate(state(), rabbit_mqtt_processor:state()) -> state().
+pstate(State = #state {}, PState) ->
State #state{ proc_state = PState }.
%%----------------------------------------------------------------------------
parse(Bytes, ParseState) ->
try
- rabbit_mqtt_frame:parse(Bytes, ParseState)
+ rabbit_mqtt_packet:parse(Bytes, ParseState)
catch
_:Reason:Stacktrace ->
{error, {cannot_parse, Reason, Stacktrace}}
end.
-send_will_and_terminate(PState, State) ->
- send_will_and_terminate(PState, {shutdown, conn_closed}, State).
-
-send_will_and_terminate(PState, Reason, State = #state{conn_name = ConnStr}) ->
- rabbit_mqtt_processor:send_will(PState),
- rabbit_log_connection:debug("MQTT: about to send will message (if any) on connection ~tp", [ConnStr]),
- % todo: flush channel after publish
- {stop, Reason, State}.
-
network_error(closed,
- State = #state{conn_name = ConnStr,
- proc_state = PState}) ->
- MqttConn = PState#proc_state.connection,
- Fmt = "MQTT connection ~tp will terminate because peer closed TCP connection",
- Args = [ConnStr],
- case MqttConn of
- undefined -> rabbit_log_connection:debug(Fmt, Args);
- _ -> rabbit_log_connection:info(Fmt, Args)
+ State = #state{conn_name = ConnName,
+ received_connect_packet = Connected}) ->
+ Fmt = "MQTT connection ~p will terminate because peer closed TCP connection",
+ Args = [ConnName],
+ case Connected of
+ true -> ?LOG_INFO(Fmt, Args);
+ false -> ?LOG_DEBUG(Fmt, Args)
end,
- send_will_and_terminate(PState, State);
+ {stop, {shutdown, conn_closed}, State};
network_error(Reason,
- State = #state{conn_name = ConnStr,
- proc_state = PState}) ->
- rabbit_log_connection:info("MQTT detected network error for ~tp: ~tp",
- [ConnStr, Reason]),
- send_will_and_terminate(PState, State).
+ State = #state{conn_name = ConnName}) ->
+ ?LOG_INFO("MQTT detected network error for ~p: ~p", [ConnName, Reason]),
+ {stop, {shutdown, conn_closed}, State}.
run_socket(State = #state{ connection_state = blocked }) ->
State;
@@ -418,78 +410,144 @@ run_socket(State = #state{ deferred_recv = Data }) when Data =/= undefined ->
run_socket(State = #state{ await_recv = true }) ->
State;
run_socket(State = #state{ socket = Sock }) ->
- rabbit_net:setopts(Sock, [{active, once}]),
+ ok = rabbit_net:setopts(Sock, [{active, once}]),
State#state{ await_recv = true }.
-control_throttle(State = #state{ connection_state = Flow,
- conserve = Conserve }) ->
- case {Flow, Conserve orelse credit_flow:blocked()} of
- {running, true} -> ok = rabbit_heartbeat:pause_monitor(
- State#state.keepalive),
- State #state{ connection_state = blocked };
- {blocked, false} -> ok = rabbit_heartbeat:resume_monitor(
- State#state.keepalive),
- run_socket(State #state{
- connection_state = running });
- {_, _} -> run_socket(State)
+control_throttle(State = #state{connection_state = ConnState,
+ conserve = Conserve,
+ received_connect_packet = Connected,
+ proc_state = PState,
+ keepalive = KState
+ }) ->
+ Throttle = rabbit_mqtt_processor:throttle(Conserve, Connected, PState),
+ case {ConnState, Throttle} of
+ {running, true} ->
+ State#state{connection_state = blocked,
+ keepalive = rabbit_mqtt_keepalive:cancel_timer(KState)};
+ {blocked, false} ->
+ run_socket(State#state{connection_state = running,
+ keepalive = rabbit_mqtt_keepalive:start_timer(KState)});
+ {_, _} ->
+ run_socket(State)
end.
maybe_process_deferred_recv(State = #state{ deferred_recv = undefined }) ->
- {noreply, State, hibernate};
+ {noreply, State, ?HIBERNATE_AFTER};
maybe_process_deferred_recv(State = #state{ deferred_recv = Data, socket = Sock }) ->
handle_info({tcp, Sock, Data},
State#state{ deferred_recv = undefined }).
-maybe_emit_stats(undefined) ->
+maybe_emit_stats(#state{stats_timer = undefined}) ->
ok;
maybe_emit_stats(State) ->
rabbit_event:if_enabled(State, #state.stats_timer,
fun() -> emit_stats(State) end).
-emit_stats(State=#state{connection = C}) when C == none; C == undefined ->
+emit_stats(State=#state{received_connect_packet = false}) ->
%% Avoid emitting stats on terminate when the connection has not yet been
%% established, as this causes orphan entries on the stats database
State1 = rabbit_event:reset_stats_timer(State, #state.stats_timer),
ensure_stats_timer(State1);
emit_stats(State) ->
[{_, Pid},
- {_, Recv_oct},
- {_, Send_oct},
+ {_, RecvOct},
+ {_, SendOct},
{_, Reductions}] = infos(?SIMPLE_METRICS, State),
Infos = infos(?OTHER_METRICS, State),
rabbit_core_metrics:connection_stats(Pid, Infos),
- rabbit_core_metrics:connection_stats(Pid, Recv_oct, Send_oct, Reductions),
+ rabbit_core_metrics:connection_stats(Pid, RecvOct, SendOct, Reductions),
State1 = rabbit_event:reset_stats_timer(State, #state.stats_timer),
ensure_stats_timer(State1).
ensure_stats_timer(State = #state{}) ->
rabbit_event:ensure_stats_timer(State, #state.stats_timer, emit_stats).
-infos(Items, State) -> [{Item, info_internal(Item, State)} || Item <- Items].
+infos(Items, State) ->
+ [{Item, i(Item, State)} || Item <- Items].
-info_internal(pid, State) -> info_internal(connection, State);
-info_internal(SockStat, #state{socket = Sock}) when SockStat =:= recv_oct;
- SockStat =:= recv_cnt;
- SockStat =:= send_oct;
- SockStat =:= send_cnt;
- SockStat =:= send_pend ->
+i(SockStat, #state{socket = Sock})
+ when SockStat =:= recv_oct;
+ SockStat =:= recv_cnt;
+ SockStat =:= send_oct;
+ SockStat =:= send_cnt;
+ SockStat =:= send_pend ->
case rabbit_net:getstat(Sock, [SockStat]) of
- {ok, [{_, N}]} when is_number(N) -> N;
- _ -> 0
+ {ok, [{_, N}]} when is_number(N) ->
+ N;
+ _ ->
+ 0
end;
-info_internal(state, State) -> info_internal(connection_state, State);
-info_internal(garbage_collection, _State) ->
+i(state, S) ->
+ i(connection_state, S);
+i(garbage_collection, _) ->
rabbit_misc:get_gc_info(self());
-info_internal(reductions, _State) ->
+i(reductions, _) ->
{reductions, Reductions} = erlang:process_info(self(), reductions),
Reductions;
-info_internal(conn_name, #state{conn_name = Val}) ->
- rabbit_data_coercion:to_binary(Val);
-info_internal(connection_state, #state{received_connect_frame = false}) ->
- starting;
-info_internal(connection_state, #state{connection_state = Val}) ->
+i(name, S) ->
+ i(conn_name, S);
+i(conn_name, #state{conn_name = Val}) ->
Val;
-info_internal(connection, #state{connection = Val}) ->
+i(connection_state, #state{received_connect_packet = false}) ->
+ starting;
+i(connection_state, #state{connection_state = Val}) ->
Val;
-info_internal(Key, #state{proc_state = ProcState}) ->
+i(pid, _) ->
+ self();
+i(SSL, #state{socket = Sock, proxy_socket = ProxySock})
+ when SSL =:= ssl;
+ SSL =:= ssl_protocol;
+ SSL =:= ssl_key_exchange;
+ SSL =:= ssl_cipher;
+ SSL =:= ssl_hash ->
+ rabbit_ssl:info(SSL, {Sock, ProxySock});
+i(Cert, #state{socket = Sock})
+ when Cert =:= peer_cert_issuer;
+ Cert =:= peer_cert_subject;
+ Cert =:= peer_cert_validity ->
+ rabbit_ssl:cert_info(Cert, Sock);
+i(timeout, #state{keepalive = KState}) ->
+ rabbit_mqtt_keepalive:interval_secs(KState);
+i(protocol, #state{proc_state = ProcState}) ->
+ {?PROTO_FAMILY, rabbit_mqtt_processor:proto_version_tuple(ProcState)};
+i(Key, #state{proc_state = ProcState}) ->
rabbit_mqtt_processor:info(Key, ProcState).
+
+-spec format_status(Status) -> Status when
+ Status :: #{state => term(),
+ message => term(),
+ reason => term(),
+ log => [sys:system_event()]}.
+format_status(Status) ->
+ maps:map(
+ fun(state, State) ->
+ format_state(State);
+ (_, Value) ->
+ Value
+ end, Status).
+
+-spec format_state(state()) -> map().
+format_state(#state{socket = Socket,
+ proxy_socket = ProxySock,
+ await_recv = AwaitRecv,
+ deferred_recv = DeferredRecv,
+ parse_state = _,
+ proc_state = PState,
+ connection_state = ConnectionState,
+ conserve = Conserve,
+ stats_timer = StatsTimer,
+ keepalive = Keepalive,
+ conn_name = ConnName,
+ received_connect_packet = ReceivedConnectPacket
+ }) ->
+ #{socket => Socket,
+ proxy_socket => ProxySock,
+ await_recv => AwaitRecv,
+ deferred_recv => DeferredRecv =/= undefined,
+ proc_state => rabbit_mqtt_processor:format_status(PState),
+ connection_state => ConnectionState,
+ conserve => Conserve,
+ stats_timer => StatsTimer,
+ keepalive => Keepalive,
+ conn_name => ConnName,
+ received_connect_packet => ReceivedConnectPacket}.
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl
index 70b18881f031..4a5eea9b4286 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store.erl
@@ -7,17 +7,33 @@
-module(rabbit_mqtt_retained_msg_store).
--export([behaviour_info/1, table_name_for/1]).
-
-behaviour_info(callbacks) ->
- [{new, 2},
- {recover, 2},
- {insert, 3},
- {lookup, 2},
- {delete, 2},
- {terminate, 1}];
-behaviour_info(_Other) ->
- undefined.
-
-table_name_for(VHost) ->
- rabbit_mqtt_util:vhost_name_to_table_name(VHost).
+-include("rabbit_mqtt_packet.hrl").
+
+-callback new(Directory :: file:name_all(), rabbit_types:vhost()) ->
+ State :: any().
+
+-callback recover(Directory :: file:name_all(), rabbit_types:vhost()) ->
+ {ok, State :: any()} | {error, Reason :: term()}.
+
+-callback insert(Topic :: binary(), mqtt_msg(), State :: any()) ->
+ ok.
+
+-callback lookup(Topic :: binary(), State :: any()) ->
+ retained_message() | not_found.
+
+-callback delete(Topic :: binary(), State :: any()) ->
+ ok.
+
+-callback terminate(State :: any()) ->
+ ok.
+
+%% TODO Support retained messages in RabbitMQ cluster, for
+%% 1. support PUBLISH with retain on a different node than SUBSCRIBE
+%% 2. replicate retained message for data safety
+%%
+%% Possible solution for 1.
+%% * retained message store backend does RPCs to peer nodes to lookup and delete
+%%
+%% Possible solutions for 2.
+%% * rabbitmq_mqtt_retained_msg_store_khepri
+%% * rabbitmq_mqtt_retained_msg_store_ra (implementing our own ra machine)
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl
index 4cfc11279b05..8dbd06a44021 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_dets.erl
@@ -8,7 +8,7 @@
-module(rabbit_mqtt_retained_msg_store_dets).
-behaviour(rabbit_mqtt_retained_msg_store).
--include("rabbit_mqtt.hrl").
+-include("rabbit_mqtt_packet.hrl").
-export([new/2, recover/2, insert/3, lookup/2, delete/2, terminate/1]).
@@ -17,38 +17,49 @@
table
}).
+-type store_state() :: #store_state{}.
+-spec new(file:name_all(), rabbit_types:vhost()) -> store_state().
new(Dir, VHost) ->
Tid = open_table(Dir, VHost),
#store_state{table = Tid}.
+-spec recover(file:name_all(), rabbit_types:vhost()) ->
+ {error, uninitialized} | {ok, store_state()}.
recover(Dir, VHost) ->
case open_table(Dir, VHost) of
{error, _} -> {error, uninitialized};
{ok, Tid} -> {ok, #store_state{table = Tid}}
end.
+-spec insert(binary(), mqtt_msg(), store_state()) -> ok.
insert(Topic, Msg, #store_state{table = T}) ->
ok = dets:insert(T, #retained_message{topic = Topic, mqtt_msg = Msg}).
+-spec lookup(binary(), store_state()) -> retained_message() | not_found.
lookup(Topic, #store_state{table = T}) ->
case dets:lookup(T, Topic) of
[] -> not_found;
[Entry] -> Entry
end.
+-spec delete(binary(), store_state()) -> ok.
delete(Topic, #store_state{table = T}) ->
ok = dets:delete(T, Topic).
+-spec terminate(store_state()) -> ok.
terminate(#store_state{table = T}) ->
ok = dets:close(T).
open_table(Dir, VHost) ->
- dets:open_file(rabbit_mqtt_retained_msg_store:table_name_for(VHost),
- table_options(rabbit_mqtt_util:path_for(Dir, VHost, ".dets"))).
+ dets:open_file(rabbit_mqtt_util:vhost_name_to_table_name(VHost),
+ table_options(rabbit_mqtt_util:path_for(Dir, VHost, ".dets"))).
table_options(Path) ->
- [{type, set}, {keypos, #retained_message.topic},
- {file, Path}, {ram_file, true}, {repair, true},
- {auto_save, rabbit_misc:get_env(rabbit_mqtt,
- retained_message_store_dets_sync_interval, 2000)}].
+ [{type, set},
+ {keypos, #retained_message.topic},
+ {file, Path},
+ {ram_file, true},
+ {repair, true},
+ {auto_save, rabbit_misc:get_env(rabbit_mqtt, retained_message_store_dets_sync_interval, 2000)}
+ ].
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl
index c25c9f20984c..3a0a7384dbb5 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_ets.erl
@@ -8,7 +8,7 @@
-module(rabbit_mqtt_retained_msg_store_ets).
-behaviour(rabbit_mqtt_retained_msg_store).
--include("rabbit_mqtt.hrl").
+-include("rabbit_mqtt_packet.hrl").
-export([new/2, recover/2, insert/3, lookup/2, delete/2, terminate/1]).
@@ -19,36 +19,43 @@
filename
}).
+-type store_state() :: #store_state{}.
+-spec new(file:name_all(), rabbit_types:vhost()) -> store_state().
new(Dir, VHost) ->
Path = rabbit_mqtt_util:path_for(Dir, VHost),
- TableName = rabbit_mqtt_retained_msg_store:table_name_for(VHost),
- file:delete(Path),
+ TableName = rabbit_mqtt_util:vhost_name_to_table_name(VHost),
+ _ = file:delete(Path),
Tid = ets:new(TableName, [set, public, {keypos, #retained_message.topic}]),
#store_state{table = Tid, filename = Path}.
+-spec recover(file:name_all(), rabbit_types:vhost()) ->
+ {error, uninitialized} | {ok, store_state()}.
recover(Dir, VHost) ->
Path = rabbit_mqtt_util:path_for(Dir, VHost),
case ets:file2tab(Path) of
- {ok, Tid} -> file:delete(Path),
+ {ok, Tid} -> _ = file:delete(Path),
{ok, #store_state{table = Tid, filename = Path}};
{error, _} -> {error, uninitialized}
end.
+-spec insert(binary(), mqtt_msg(), store_state()) -> ok.
insert(Topic, Msg, #store_state{table = T}) ->
true = ets:insert(T, #retained_message{topic = Topic, mqtt_msg = Msg}),
ok.
+-spec lookup(binary(), store_state()) -> retained_message() | not_found.
lookup(Topic, #store_state{table = T}) ->
case ets:lookup(T, Topic) of
[] -> not_found;
[Entry] -> Entry
end.
+-spec delete(binary(), store_state()) -> ok.
delete(Topic, #store_state{table = T}) ->
true = ets:delete(T, Topic),
ok.
+-spec terminate(store_state()) -> ok.
terminate(#store_state{table = T, filename = Path}) ->
- ok = ets:tab2file(T, Path,
- [{extended_info, [object_count]}]).
+ ok = ets:tab2file(T, Path, [{extended_info, [object_count]}]).
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_noop.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_noop.erl
index 79cb74d7d9c9..1e62ed5abee7 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_noop.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retained_msg_store_noop.erl
@@ -8,7 +8,6 @@
-module(rabbit_mqtt_retained_msg_store_noop).
-behaviour(rabbit_mqtt_retained_msg_store).
--include("rabbit_mqtt.hrl").
-export([new/2, recover/2, insert/3, lookup/2, delete/2, terminate/1]).
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl
index 32de0bc6bc59..1c632eb5976d 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer.erl
@@ -7,42 +7,40 @@
-module(rabbit_mqtt_retainer).
--behaviour(gen_server2).
-include("rabbit_mqtt.hrl").
--include("rabbit_mqtt_frame.hrl").
+-include("rabbit_mqtt_packet.hrl").
+
+-behaviour(gen_server).
-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
- terminate/2, code_change/3, start_link/2]).
+ terminate/2, start_link/2]).
-export([retain/3, fetch/2, clear/2, store_module/0]).
--define(SERVER, ?MODULE).
--define(TIMEOUT, 30000).
+-define(TIMEOUT, 30_000).
-record(retainer_state, {store_mod,
store}).
--spec retain(pid(), string(), mqtt_msg()) ->
- {noreply, NewState :: term()} |
- {noreply, NewState :: term(), timeout() | hibernate} |
- {stop, Reason :: term(), NewState :: term()}.
-
%%----------------------------------------------------------------------------
start_link(RetainStoreMod, VHost) ->
- gen_server2:start_link(?MODULE, [RetainStoreMod, VHost], []).
+ gen_server:start_link(?MODULE, [RetainStoreMod, VHost], []).
+-spec retain(pid(), binary(), mqtt_msg()) -> ok.
retain(Pid, Topic, Msg = #mqtt_msg{retain = true}) ->
- gen_server2:cast(Pid, {retain, Topic, Msg});
-
+ gen_server:cast(Pid, {retain, Topic, Msg});
retain(_Pid, _Topic, Msg = #mqtt_msg{retain = false}) ->
throw({error, {retain_is_false, Msg}}).
+-spec fetch(pid(), binary()) ->
+ undefined | mqtt_msg().
fetch(Pid, Topic) ->
- gen_server2:call(Pid, {fetch, Topic}, ?TIMEOUT).
+ gen_server:call(Pid, {fetch, Topic}, ?TIMEOUT).
+-spec clear(pid(), binary()) -> ok.
clear(Pid, Topic) ->
- gen_server2:cast(Pid, {clear, Topic}).
+ gen_server:cast(Pid, {clear, Topic}).
%%----------------------------------------------------------------------------
@@ -56,8 +54,9 @@ init([StoreMod, VHost]) ->
end,
{ok, State}.
+-spec store_module() -> undefined | module().
store_module() ->
- case application:get_env(rabbitmq_mqtt, retained_message_store) of
+ case application:get_env(?APP_NAME, retained_message_store) of
{ok, Mod} -> Mod;
undefined -> undefined
end.
@@ -91,8 +90,4 @@ store_dir() ->
rabbit:data_dir().
terminate(_Reason, #retainer_state{store = Store, store_mod = Mod}) ->
- Mod:terminate(Store),
- ok.
-
-code_change(_OldVsn, State, _Extra) ->
- {ok, State}.
+ Mod:terminate(Store).
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer_sup.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer_sup.erl
index 178bb09b9b7a..eadaab7ca272 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer_sup.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_retainer_sup.erl
@@ -11,14 +11,13 @@
-export([start_link/1, init/1, start_child/2,start_child/1, child_for_vhost/1,
delete_child/1]).
--define(ENCODING, utf8).
-
-spec start_child(binary()) -> supervisor:startchild_ret().
-spec start_child(term(), binary()) -> supervisor:startchild_ret().
start_link(SupName) ->
supervisor:start_link(SupName, ?MODULE, []).
+-spec child_for_vhost(rabbit_types:vhost()) -> pid().
child_for_vhost(VHost) when is_binary(VHost) ->
case rabbit_mqtt_retainer_sup:start_child(VHost) of
{ok, Pid} -> Pid;
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl
index cbdff3b8e504..a04ef73cb91f 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_sup.erl
@@ -9,6 +9,7 @@
-behaviour(supervisor).
-include_lib("rabbit_common/include/rabbit.hrl").
+-include("rabbit_mqtt.hrl").
-export([start_link/2, init/1, stop_listeners/0]).
@@ -19,45 +20,60 @@ start_link(Listeners, []) ->
supervisor:start_link({local, ?MODULE}, ?MODULE, [Listeners]).
init([{Listeners, SslListeners0}]) ->
- NumTcpAcceptors = application:get_env(rabbitmq_mqtt, num_tcp_acceptors, 10),
- ConcurrentConnsSups = application:get_env(rabbitmq_mqtt, num_conns_sups, 1),
- {ok, SocketOpts} = application:get_env(rabbitmq_mqtt, tcp_listen_options),
+ NumTcpAcceptors = application:get_env(?APP_NAME, num_tcp_acceptors, 10),
+ ConcurrentConnsSups = application:get_env(?APP_NAME, num_conns_sups, 1),
+ {ok, SocketOpts} = application:get_env(?APP_NAME, tcp_listen_options),
{SslOpts, NumSslAcceptors, SslListeners}
= case SslListeners0 of
[] -> {none, 0, []};
_ -> {rabbit_networking:ensure_ssl(),
- application:get_env(rabbitmq_mqtt, num_ssl_acceptors, 10),
+ application:get_env(?APP_NAME, num_ssl_acceptors, 10),
case rabbit_networking:poodle_check('MQTT') of
ok -> SslListeners0;
danger -> []
end}
end,
+ %% Use separate process group scope per RabbitMQ node. This achieves a local-only
+ %% process group which requires less memory with millions of connections.
+ PgScope = list_to_atom(io_lib:format("~s_~s", [?PG_SCOPE, node()])),
+ persistent_term:put(?PG_SCOPE, PgScope),
{ok,
- {#{strategy => one_for_all, intensity => 10, period => 10}, [
- #{
- id => rabbit_mqtt_retainer_sup,
- start =>
- {rabbit_mqtt_retainer_sup, start_link, [{local, rabbit_mqtt_retainer_sup}]},
- restart => transient,
- shutdown => ?SUPERVISOR_WAIT,
- type => supervisor,
- modules => [rabbit_mqtt_retainer_sup]
- }
- | listener_specs(
- fun tcp_listener_spec/1,
- [SocketOpts, NumTcpAcceptors, ConcurrentConnsSups],
- Listeners
- ) ++
- listener_specs(
- fun ssl_listener_spec/1,
- [SocketOpts, SslOpts, NumSslAcceptors, ConcurrentConnsSups],
- SslListeners
- )
- ]}}.
+ {#{strategy => one_for_all,
+ intensity => 10,
+ period => 10},
+ [
+ #{id => PgScope,
+ start => {pg, start_link, [PgScope]},
+ restart => transient,
+ shutdown => ?WORKER_WAIT,
+ type => worker,
+ modules => [pg]
+ },
+ #{
+ id => rabbit_mqtt_retainer_sup,
+ start => {rabbit_mqtt_retainer_sup, start_link,
+ [{local, rabbit_mqtt_retainer_sup}]},
+ restart => transient,
+ shutdown => ?SUPERVISOR_WAIT,
+ type => supervisor,
+ modules => [rabbit_mqtt_retainer_sup]
+ }
+ | listener_specs(
+ fun tcp_listener_spec/1,
+ [SocketOpts, NumTcpAcceptors, ConcurrentConnsSups],
+ Listeners
+ ) ++
+ listener_specs(
+ fun ssl_listener_spec/1,
+ [SocketOpts, SslOpts, NumSslAcceptors, ConcurrentConnsSups],
+ SslListeners
+ )
+ ]}}.
+-spec stop_listeners() -> ok.
stop_listeners() ->
- rabbit_networking:stop_ranch_listener_of_protocol(?TCP_PROTOCOL),
- rabbit_networking:stop_ranch_listener_of_protocol(?TLS_PROTOCOL),
+ _ = rabbit_networking:stop_ranch_listener_of_protocol(?TCP_PROTOCOL),
+ _ = rabbit_networking:stop_ranch_listener_of_protocol(?TLS_PROTOCOL),
ok.
%%
@@ -73,34 +89,35 @@ listener_specs(Fun, Args, Listeners) ->
tcp_listener_spec([Address, SocketOpts, NumAcceptors, ConcurrentConnsSups]) ->
rabbit_networking:tcp_listener_spec(
- rabbit_mqtt_listener_sup,
- Address,
- SocketOpts,
- transport(?TCP_PROTOCOL),
- rabbit_mqtt_connection_sup,
- [],
- mqtt,
- NumAcceptors,
- ConcurrentConnsSups,
- "MQTT TCP listener"
- ).
+ rabbit_mqtt_listener_sup,
+ Address,
+ SocketOpts,
+ transport(?TCP_PROTOCOL),
+ rabbit_mqtt_reader,
+ [],
+ mqtt,
+ NumAcceptors,
+ ConcurrentConnsSups,
+ worker,
+ "MQTT TCP listener"
+ ).
ssl_listener_spec([Address, SocketOpts, SslOpts, NumAcceptors, ConcurrentConnsSups]) ->
rabbit_networking:tcp_listener_spec(
- rabbit_mqtt_listener_sup,
- Address,
- SocketOpts ++ SslOpts,
- transport(?TLS_PROTOCOL),
- rabbit_mqtt_connection_sup,
- [],
- 'mqtt/ssl',
- NumAcceptors,
- ConcurrentConnsSups,
- "MQTT TLS listener"
- ).
+ rabbit_mqtt_listener_sup,
+ Address,
+ SocketOpts ++ SslOpts,
+ transport(?TLS_PROTOCOL),
+ rabbit_mqtt_reader,
+ [],
+ 'mqtt/ssl',
+ NumAcceptors,
+ ConcurrentConnsSups,
+ worker,
+ "MQTT TLS listener"
+ ).
-transport(Protocol) ->
- case Protocol of
- ?TCP_PROTOCOL -> ranch_tcp;
- ?TLS_PROTOCOL -> ranch_ssl
- end.
+transport(?TCP_PROTOCOL) ->
+ ranch_tcp;
+transport(?TLS_PROTOCOL) ->
+ ranch_ssl.
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl
index c711250b030b..d098d3ff931a 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_util.erl
@@ -7,32 +7,107 @@
-module(rabbit_mqtt_util).
+-include_lib("rabbit_common/include/resource.hrl").
-include("rabbit_mqtt.hrl").
+-include("rabbit_mqtt_packet.hrl").
--export([subcription_queue_name/1,
- gen_client_id/0,
+-export([queue_name_bin/2,
+ qos_from_queue_name/2,
env/1,
table_lookup/2,
path_for/2,
path_for/3,
vhost_name_to_table_name/1,
- get_topic_translation_funs/0
+ register_clientid/2,
+ remove_duplicate_clientid_connections/2,
+ init_sparkplug/0,
+ mqtt_to_amqp/1,
+ amqp_to_mqtt/1,
+ truncate_binary/2
]).
-define(MAX_TOPIC_TRANSLATION_CACHE_SIZE, 12).
+-define(SPARKPLUG_MP_MQTT_TO_AMQP, sparkplug_mp_mqtt_to_amqp).
+-define(SPARKPLUG_MP_AMQP_TO_MQTT, sparkplug_mp_amqp_to_mqtt).
+
+-spec queue_name_bin(binary(), qos()) ->
+ binary().
+queue_name_bin(ClientId, QoS) ->
+ Prefix = queue_name_prefix(ClientId),
+ queue_name0(Prefix, QoS).
+
+queue_name0(Prefix, ?QOS_0) ->
+ <>;
+queue_name0(Prefix, ?QOS_1) ->
+ <>.
+
+-spec qos_from_queue_name(rabbit_amqqueue:name(), binary()) ->
+ qos() | no_consuming_queue.
+qos_from_queue_name(#resource{name = Name}, ClientId) ->
+ Prefix = queue_name_prefix(ClientId),
+ PrefixSize = erlang:byte_size(Prefix),
+ case Name of
+ <> ->
+ ?QOS_0;
+ <> ->
+ ?QOS_1;
+ _ ->
+ no_consuming_queue
+ end.
-subcription_queue_name(ClientId) ->
- Base = "mqtt-subscription-" ++ ClientId ++ "qos",
- {list_to_binary(Base ++ "0"), list_to_binary(Base ++ "1")}.
+queue_name_prefix(ClientId) ->
+ <<"mqtt-subscription-", ClientId/binary, "qos">>.
-cached(CacheName, Fun, Arg) ->
- Cache =
- case get(CacheName) of
- undefined ->
- [];
- Other ->
- Other
+-spec init_sparkplug() -> ok.
+init_sparkplug() ->
+ case env(sparkplug) of
+ true ->
+ {ok, M2A_SpRe} = re:compile("^sp[AB]v\\d+\\.\\d+/"),
+ {ok, A2M_SpRe} = re:compile("^sp[AB]v\\d+___\\d+\\."),
+ ok = persistent_term:put(?SPARKPLUG_MP_MQTT_TO_AMQP, M2A_SpRe),
+ ok = persistent_term:put(?SPARKPLUG_MP_AMQP_TO_MQTT, A2M_SpRe);
+ _ ->
+ ok
+ end.
+
+-spec mqtt_to_amqp(binary()) -> binary().
+mqtt_to_amqp(Topic) ->
+ T = case persistent_term:get(?SPARKPLUG_MP_MQTT_TO_AMQP, no_sparkplug) of
+ no_sparkplug ->
+ Topic;
+ M2A_SpRe ->
+ case re:run(Topic, M2A_SpRe) of
+ nomatch ->
+ Topic;
+ {match, _} ->
+ string:replace(Topic, ".", "___", leading)
+ end
end,
+ cached(mta_cache, fun to_amqp/1, T).
+
+-spec amqp_to_mqtt(binary()) -> binary().
+amqp_to_mqtt(Topic) ->
+ T = cached(atm_cache, fun to_mqtt/1, Topic),
+ case persistent_term:get(?SPARKPLUG_MP_AMQP_TO_MQTT, no_sparkplug) of
+ no_sparkplug ->
+ T;
+ A2M_SpRe ->
+ case re:run(Topic, A2M_SpRe) of
+ nomatch ->
+ T;
+ {match, _} ->
+ T1 = string:replace(T, "___", ".", leading),
+ erlang:iolist_to_binary(T1)
+ end
+ end.
+
+cached(CacheName, Fun, Arg) ->
+ Cache = case get(CacheName) of
+ undefined ->
+ [];
+ Other ->
+ Other
+ end,
case lists:keyfind(Arg, 1, Cache) of
{_, V} ->
V;
@@ -43,6 +118,11 @@ cached(CacheName, Fun, Arg) ->
V
end.
+%% amqp mqtt descr
+%% * + match one topic level
+%% # # match multiple topic levels
+%% . / topic level separator
+
to_amqp(T0) ->
T1 = string:replace(T0, "/", ".", all),
T2 = string:replace(T1, "+", "*", all),
@@ -53,69 +133,21 @@ to_mqtt(T0) ->
T2 = string:replace(T1, ".", "/", all),
erlang:iolist_to_binary(T2).
-%% amqp mqtt descr
-%% * + match one topic level
-%% # # match multiple topic levels
-%% . / topic level separator
-get_topic_translation_funs() ->
- SparkplugB = env(sparkplug),
- ToAmqpFun = fun(Topic) ->
- cached(mta_cache, fun to_amqp/1, Topic)
- end,
- ToMqttFun = fun(Topic) ->
- cached(atm_cache, fun to_mqtt/1, Topic)
- end,
- {M2AFun, A2MFun} = case SparkplugB of
- true ->
- {ok, M2A_SpRe} = re:compile("^sp[AB]v\\d+\\.\\d+/"),
- {ok, A2M_SpRe} = re:compile("^sp[AB]v\\d+___\\d+\\."),
- M2A = fun(T0) ->
- case re:run(T0, M2A_SpRe) of
- nomatch ->
- ToAmqpFun(T0);
- {match, _} ->
- T1 = string:replace(T0, ".", "___", leading),
- ToAmqpFun(T1)
- end
- end,
- A2M = fun(T0) ->
- case re:run(T0, A2M_SpRe) of
- nomatch ->
- ToMqttFun(T0);
- {match, _} ->
- T1 = ToMqttFun(T0),
- T2 = string:replace(T1, "___", ".", leading),
- erlang:iolist_to_binary(T2)
- end
- end,
- {M2A, A2M};
- _ ->
- M2A = fun(T) ->
- ToAmqpFun(T)
- end,
- A2M = fun(T) ->
- ToMqttFun(T)
- end,
- {M2A, A2M}
- end,
- {ok, {mqtt2amqp_fun, M2AFun}, {amqp2mqtt_fun, A2MFun}}.
-
-gen_client_id() ->
- lists:nthtail(1, rabbit_guid:string(rabbit_guid:gen_secure(), [])).
-
+-spec env(atom()) -> any().
env(Key) ->
- case application:get_env(rabbitmq_mqtt, Key) of
+ case application:get_env(?APP_NAME, Key) of
{ok, Val} -> coerce_env_value(Key, Val);
undefined -> undefined
end.
-%% TODO: move to rabbit_common
coerce_env_value(default_pass, Val) -> rabbit_data_coercion:to_binary(Val);
coerce_env_value(default_user, Val) -> rabbit_data_coercion:to_binary(Val);
coerce_env_value(exchange, Val) -> rabbit_data_coercion:to_binary(Val);
coerce_env_value(vhost, Val) -> rabbit_data_coercion:to_binary(Val);
coerce_env_value(_, Val) -> Val.
+-spec table_lookup(rabbit_framing:amqp_table() | undefined, binary()) ->
+ tuple() | undefined.
table_lookup(undefined, _Key) ->
undefined;
table_lookup(Table, Key) ->
@@ -127,13 +159,53 @@ vhost_name_to_dir_name(VHost, Suffix) ->
<> = erlang:md5(VHost),
"mqtt_retained_" ++ rabbit_misc:format("~36.16.0b", [Num]) ++ Suffix.
+-spec path_for(file:name_all(), rabbit_types:vhost()) -> file:filename_all().
path_for(Dir, VHost) ->
filename:join(Dir, vhost_name_to_dir_name(VHost)).
+-spec path_for(file:name_all(), rabbit_types:vhost(), string()) -> file:filename_all().
path_for(Dir, VHost, Suffix) ->
filename:join(Dir, vhost_name_to_dir_name(VHost, Suffix)).
-
+-spec vhost_name_to_table_name(rabbit_types:vhost()) ->
+ atom().
vhost_name_to_table_name(VHost) ->
- <> = erlang:md5(VHost),
- list_to_atom("rabbit_mqtt_retained_" ++ rabbit_misc:format("~36.16.0b", [Num])).
+ <> = erlang:md5(VHost),
+ list_to_atom("rabbit_mqtt_retained_" ++ rabbit_misc:format("~36.16.0b", [Num])).
+
+-spec register_clientid(rabbit_types:vhost(), binary()) -> ok.
+register_clientid(Vhost, ClientId)
+ when is_binary(Vhost), is_binary(ClientId) ->
+ PgGroup = {Vhost, ClientId},
+ ok = pg:join(persistent_term:get(?PG_SCOPE), PgGroup, self()),
+ case rabbit_mqtt_ff:track_client_id_in_ra() of
+ true ->
+ %% Ra node takes care of removing duplicate client ID connections.
+ ok;
+ false ->
+ ok = erpc:multicast([node() | nodes()],
+ ?MODULE,
+ remove_duplicate_clientid_connections,
+ [PgGroup, self()])
+ end.
+
+-spec remove_duplicate_clientid_connections({rabbit_types:vhost(), binary()}, pid()) -> ok.
+remove_duplicate_clientid_connections(PgGroup, PidToKeep) ->
+ try persistent_term:get(?PG_SCOPE) of
+ PgScope ->
+ Pids = pg:get_local_members(PgScope, PgGroup),
+ lists:foreach(fun(Pid) ->
+ gen_server:cast(Pid, duplicate_id)
+ end, Pids -- [PidToKeep])
+ catch _:badarg ->
+ %% MQTT supervision tree on this node not fully started
+ ok
+ end.
+
+-spec truncate_binary(binary(), non_neg_integer()) -> binary().
+truncate_binary(Bin, Size)
+ when is_binary(Bin) andalso byte_size(Bin) =< Size ->
+ Bin;
+truncate_binary(Bin, Size)
+ when is_binary(Bin) ->
+ binary:part(Bin, 0, Size).
diff --git a/deps/rabbitmq_mqtt/test/auth_SUITE.erl b/deps/rabbitmq_mqtt/test/auth_SUITE.erl
index b5cf4d3a3635..63ac362842e2 100644
--- a/deps/rabbitmq_mqtt/test/auth_SUITE.erl
+++ b/deps/rabbitmq_mqtt/test/auth_SUITE.erl
@@ -10,8 +10,13 @@
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
--define(CONNECT_TIMEOUT, 10000).
--define(WAIT_LOG_NO_CRASHES, {["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end}).
+
+%% defined in MQTT v4 and v5 (not in v3)
+-define(SUBACK_FAILURE, 16#80).
+
+-define(FAIL_IF_CRASH_LOG, {["Generic server.*terminating"],
+ fun () -> ct:fail(crash_detected) end}).
+-import(rabbit_ct_broker_helpers, [rpc/5]).
all() ->
[{group, anonymous_no_ssl_user},
@@ -19,7 +24,8 @@ all() ->
{group, no_ssl_user},
{group, ssl_user},
{group, client_id_propagation},
- {group, authz_handling}].
+ {group, authz},
+ {group, limit}].
groups() ->
[{anonymous_ssl_user, [],
@@ -63,15 +69,24 @@ groups() ->
{client_id_propagation, [],
[client_id_propagation]
},
- {authz_handling, [],
+ {authz, [],
[no_queue_bind_permission,
+ no_queue_unbind_permission,
no_queue_consume_permission,
no_queue_consume_permission_on_connect,
no_queue_delete_permission,
no_queue_declare_permission,
+ no_publish_permission,
no_topic_read_permission,
- no_topic_write_permission]
- }
+ no_topic_write_permission,
+ loopback_user_connects_from_remote_host
+ ]
+ },
+ {limit, [],
+ [vhost_connection_limit,
+ vhost_queue_limit,
+ user_connection_limit
+ ]}
].
init_per_suite(Config) ->
@@ -81,7 +96,7 @@ init_per_suite(Config) ->
end_per_suite(Config) ->
Config.
-init_per_group(authz_handling, Config0) ->
+init_per_group(authz, Config0) ->
User = <<"mqtt-user">>,
Password = <<"mqtt-password">>,
VHost = <<"mqtt-vhost">>,
@@ -96,7 +111,7 @@ init_per_group(authz_handling, Config0) ->
rabbit_ct_client_helpers:setup_steps()),
rabbit_ct_broker_helpers:add_user(Config1, User, Password),
rabbit_ct_broker_helpers:add_vhost(Config1, VHost),
- [Log|_] = rabbit_ct_broker_helpers:rpc(Config1, 0, rabbit, log_locations, []),
+ [Log|_] = rpc(Config1, 0, rabbit, log_locations, []),
[{mqtt_user, User}, {mqtt_vhost, VHost}, {mqtt_password, Password}, {log_location, Log}|Config1];
init_per_group(Group, Config) ->
Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"),
@@ -107,7 +122,11 @@ init_per_group(Group, Config) ->
MqttConfig = mqtt_config(Group),
AuthConfig = auth_config(Group),
rabbit_ct_helpers:run_setup_steps(Config1,
- [ fun(Conf) -> merge_app_env(MqttConfig, Conf) end ] ++
+ [ fun(Conf) -> case MqttConfig of
+ undefined -> Conf;
+ _ -> merge_app_env(MqttConfig, Conf)
+ end
+ end] ++
[ fun(Conf) -> case AuthConfig of
undefined -> Conf;
_ -> merge_app_env(AuthConfig, Conf)
@@ -138,7 +157,9 @@ mqtt_config(no_ssl_user) ->
{allow_anonymous, false}]};
mqtt_config(client_id_propagation) ->
{rabbitmq_mqtt, [{ssl_cert_login, true},
- {allow_anonymous, true}]}.
+ {allow_anonymous, true}]};
+mqtt_config(_) ->
+ undefined.
auth_config(client_id_propagation) ->
{rabbit, [
@@ -233,10 +254,7 @@ set_cert_user_on_default_vhost(Config) ->
CertFile = filename:join([CertsDir, "client", "cert.pem"]),
{ok, CertBin} = file:read_file(CertFile),
[{'Certificate', Cert, not_encrypted}] = public_key:pem_decode(CertBin),
- UserBin = rabbit_ct_broker_helpers:rpc(Config, 0,
- rabbit_ssl,
- peer_cert_auth_name,
- [Cert]),
+ UserBin = rpc(Config, 0, rabbit_ssl, peer_cert_auth_name, [Cert]),
User = binary_to_list(UserBin),
ok = rabbit_ct_broker_helpers:add_user(Config, 0, User, ""),
ok = rabbit_ct_broker_helpers:set_full_permissions(Config, User, <<"/">>),
@@ -314,17 +332,19 @@ end_per_testcase(ssl_user_port_vhost_mapping_takes_precedence_over_cert_vhost_ma
ok = rabbit_ct_broker_helpers:clear_global_parameter(Config, mqtt_port_to_vhost_mapping),
rabbit_ct_helpers:testcase_finished(Config, ssl_user_port_vhost_mapping_takes_precedence_over_cert_vhost_mapping);
end_per_testcase(Testcase, Config) when Testcase == no_queue_bind_permission;
+ Testcase == no_queue_unbind_permission;
Testcase == no_queue_consume_permission;
Testcase == no_queue_consume_permission_on_connect;
Testcase == no_queue_delete_permission;
Testcase == no_queue_declare_permission;
+ Testcase == no_publish_permission;
Testcase == no_topic_read_permission;
- Testcase == no_topic_write_permission ->
+ Testcase == no_topic_write_permission;
+ Testcase == loopback_user_connects_from_remote_host ->
%% So let's wait before logs are surely flushed
Marker = "MQTT_AUTH_SUITE_MARKER",
- rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_log, error, [Marker]),
- wait_log(Config, erlang:system_time(microsecond) + 1_000_000,
- [{[Marker], fun () -> stop end}]),
+ rpc(Config, 0, rabbit_log, error, [Marker]),
+ wait_log(Config, [{[Marker], fun () -> stop end}]),
%% Preserve file contents in case some investigation is needed, before truncating.
file:copy(?config(log_location, Config), iolist_to_binary([?config(log_location, Config), ".", atom_to_binary(Testcase)])),
@@ -426,10 +446,13 @@ ssl_user_port_vhost_mapping_takes_precedence_over_cert_vhost_mapping(Config) ->
expect_successful_connection(fun connect_ssl/1, Config).
connect_anonymous(Config) ->
+ connect_anonymous(Config, <<"simpleClient">>).
+
+connect_anonymous(Config, ClientId) ->
P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
emqtt:start_link([{host, "localhost"},
{port, P},
- {clientid, <<"simpleClient">>},
+ {clientid, ClientId},
{proto_ver, v4}]).
connect_ssl(Config) ->
@@ -451,44 +474,44 @@ client_id_propagation(Config) ->
%% setup creates the ETS table required for the mqtt auth mock
%% it blocks indefinitely so we need to spawn
Self = self(),
- _ = spawn(fun () ->
- rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_auth_backend_mqtt_mock,
- setup,
- [Self])
- end),
+ _ = spawn(
+ fun () ->
+ rpc(Config, 0, rabbit_auth_backend_mqtt_mock, setup, [Self])
+ end),
%% the setup process will notify us
receive
- ok -> ok
- after 3000 -> ok
+ ok -> ok
+ after
+ 3000 -> ct:fail("timeout waiting for rabbit_auth_backend_mqtt_mock:setup/1")
end,
ClientId = <<"client-id-propagation">>,
- {ok, C} = connect_user(<<"client-id-propagation">>, <<"client-id-propagation">>,
+ {ok, C} = connect_user(<<"fake-user">>, <<"fake-password">>,
Config, ClientId),
{ok, _} = emqtt:connect(C),
{ok, _, _} = emqtt:subscribe(C, <<"TopicA">>),
- [{authentication, AuthProps}] = rabbit_ct_broker_helpers:rpc(Config, 0,
- rabbit_auth_backend_mqtt_mock,
- get,
- [authentication]),
+ [{authentication, AuthProps}] = rpc(Config, 0,
+ rabbit_auth_backend_mqtt_mock,
+ get,
+ [authentication]),
?assertEqual(ClientId, proplists:get_value(client_id, AuthProps)),
- [{vhost_access, AuthzData}] = rabbit_ct_broker_helpers:rpc(Config, 0,
- rabbit_auth_backend_mqtt_mock,
- get,
- [vhost_access]),
+ [{vhost_access, AuthzData}] = rpc(Config, 0,
+ rabbit_auth_backend_mqtt_mock,
+ get,
+ [vhost_access]),
?assertEqual(ClientId, maps:get(<<"client_id">>, AuthzData)),
- [{resource_access, AuthzContext}] = rabbit_ct_broker_helpers:rpc(Config, 0,
- rabbit_auth_backend_mqtt_mock,
- get,
- [resource_access]),
+ [{resource_access, AuthzContext}] = rpc(Config, 0,
+ rabbit_auth_backend_mqtt_mock,
+ get,
+ [resource_access]),
?assertEqual(true, maps:size(AuthzContext) > 0),
?assertEqual(ClientId, maps:get(<<"client_id">>, AuthzContext)),
- [{topic_access, TopicContext}] = rabbit_ct_broker_helpers:rpc(Config, 0,
- rabbit_auth_backend_mqtt_mock,
- get,
- [topic_access]),
+ [{topic_access, TopicContext}] = rpc(Config, 0,
+ rabbit_auth_backend_mqtt_mock,
+ get,
+ [topic_access]),
VariableMap = maps:get(variable_map, TopicContext),
?assertEqual(ClientId, maps:get(<<"client_id">>, VariableMap)),
@@ -498,127 +521,201 @@ client_id_propagation(Config) ->
%% table in https://www.rabbitmq.com/access-control.html#authorisation
%% and which MQTT plugin tries to perform.
%%
-%% Silly MQTT doesn't allow us to see any error codes in the protocol,
-%% so the only non-intrusive way to check for `access_refused`
+%% MQTT v4 has a single SUBACK error code but does not allow to differentiate
+%% between different kind of errors why a subscription request failed.
+%% The only non-intrusive way to check for `access_refused`
%% codepath is by checking logs. Every testcase from this group
%% truncates log file beforehand, so it'd be easier to analyze. There
-%% is additional wait in the corresponding end_per_testcase that
-%% ensures that logs were for the current testcase were completely
+%% is an additional wait in the corresponding end_per_testcase that
+%% ensures that logs for the current testcase were completely
%% flushed, and won't contaminate following tests from this group.
-%%
-%% Then each test-case asserts that logs contain following things:
-%% 1) Handling of access_refused error handler in MQTT reader:
-%% https://github.com/rabbitmq/rabbitmq-server/blob/69dc53fb8938c7f135bf0002b0904cf28c25c571/deps/rabbitmq_mqtt/src/rabbit_mqtt_reader.erl#L332
-%% 2) Mention of which AMQP operation caused that error (that one is
-%% kinda superflous, it just makes sure that every AMQP operation
-%% in MQTT plugin was tested)
no_queue_bind_permission(Config) ->
- test_subscribe_permissions_combination(<<".*">>, <<"">>, <<".*">>, Config,
- ["operation queue.bind caused a channel exception access_refused"]).
+ ExpectedLogs =
+ ["MQTT resource access refused: write access to queue "
+ "'mqtt-subscription-mqtt-userqos0' in vhost 'mqtt-vhost' "
+ "refused for user 'mqtt-user'",
+ "Failed to bind queue 'mqtt-subscription-mqtt-userqos0' "
+ "in vhost 'mqtt-vhost' with topic test/topic: access_refused"
+ ],
+ test_subscribe_permissions_combination(<<".*">>, <<"">>, <<".*">>, Config, ExpectedLogs).
+
+no_queue_unbind_permission(Config) ->
+ User = ?config(mqtt_user, Config),
+ Vhost = ?config(mqtt_vhost, Config),
+ rabbit_ct_broker_helpers:set_permissions(Config, User, Vhost, <<".*">>, <<".*">>, <<".*">>),
+ P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
+ Opts = [{host, "localhost"},
+ {port, P},
+ {proto_ver, v4},
+ {clientid, User},
+ {username, User},
+ {password, ?config(mqtt_password, Config)}],
+ {ok, C1} = emqtt:start_link([{clean_start, false} | Opts]),
+ {ok, _} = emqtt:connect(C1),
+ Topic = <<"my/topic">>,
+ ?assertMatch({ok, _Properties, [1]},
+ emqtt:subscribe(C1, Topic, qos1)),
+ ok = emqtt:disconnect(C1),
+
+ %% Revoke read access to amq.topic exchange.
+ rabbit_ct_broker_helpers:set_permissions(Config, User, Vhost, <<".*">>, <<".*">>, <<"^(?!amq\.topic$)">>),
+ {ok, C2} = emqtt:start_link([{clean_start, false} | Opts]),
+ {ok, _} = emqtt:connect(C2),
+ process_flag(trap_exit, true),
+ %% We subscribe with the same client ID to the same topic again, but this time with QoS 0.
+ %% Therefore we trigger the qos1 queue to be unbound (and the qos0 queue to be bound).
+ %% However, unbinding requires read access to the exchange, which we don't have anymore.
+ ?assertMatch({ok, _Properties, [?SUBACK_FAILURE]},
+ emqtt:subscribe(C2, Topic, qos0)),
+ ok = assert_connection_closed(C2),
+ ExpectedLogs =
+ ["MQTT resource access refused: read access to exchange 'amq.topic' in vhost 'mqtt-vhost' refused for user 'mqtt-user'",
+ "Failed to unbind queue 'mqtt-subscription-mqtt-userqos1' in vhost 'mqtt-vhost' with topic 'my/topic': access_refused",
+ "MQTT protocol error on connection.*: subscribe_error"
+ ],
+ wait_log(Config, [?FAIL_IF_CRASH_LOG, {ExpectedLogs, fun () -> stop end}]),
+
+ %% Clean up the qos1 queue by connecting with clean session.
+ rabbit_ct_broker_helpers:set_permissions(Config, User, Vhost, <<".*">>, <<".*">>, <<".*">>),
+ {ok, C3} = emqtt:start_link([{clean_start, true} | Opts]),
+ {ok, _} = emqtt:connect(C3),
+ ok = emqtt:disconnect(C3).
no_queue_consume_permission(Config) ->
- test_subscribe_permissions_combination(<<".*">>, <<".*">>, <<"^amq\\.topic">>, Config,
- ["operation basic.consume caused a channel exception access_refused"]).
+ ExpectedLogs =
+ ["MQTT resource access refused: read access to queue "
+ "'mqtt-subscription-mqtt-userqos0' in vhost 'mqtt-vhost' "
+ "refused for user 'mqtt-user'"],
+ test_subscribe_permissions_combination(<<".*">>, <<".*">>, <<"^amq\\.topic">>, Config, ExpectedLogs).
no_queue_delete_permission(Config) ->
set_permissions(".*", ".*", ".*", Config),
- C1 = open_mqtt_connection(Config, [{clientid, <<"no_queue_delete_permission">>}, {clean_start, false}]),
+ ClientId = <<"no_queue_delete_permission">>,
+ {ok, C1} = connect_user(
+ ?config(mqtt_user, Config),
+ ?config(mqtt_password, Config),
+ Config,
+ ClientId,
+ [{clean_start, false}]),
+ {ok, _} = emqtt:connect(C1),
{ok, _, _} = emqtt:subscribe(C1, {<<"test/topic">>, qos1}),
ok = emqtt:disconnect(C1),
- set_permissions(<<>>, ".*", ".*", Config),
- %% And now we have a durable queue that user doesn't have permission to delete.
+ set_permissions(<<>>, ".*", ".*", Config),
+ %% Now we have a durable queue that user doesn't have permission to delete.
%% Attempt to establish clean session should fail.
{ok, C2} = connect_user(
?config(mqtt_user, Config),
?config(mqtt_password, Config),
Config,
- ?config(mqtt_user, Config),
- [{clientid, <<"no_queue_delete_permission">>},
- {clean_start, true}]),
+ ClientId,
+ [{clean_start, true}]),
unlink(C2),
?assertMatch({error, _},
emqtt:connect(C2)),
-
- wait_log(Config, erlang:system_time(microsecond) + 1_000_000,
- [{["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end}
- ,{["operation queue.delete caused a channel exception access_refused",
- "MQTT cannot start a clean session: `configure` permission missing for queue"],
- fun () -> stop end}
- ]),
+ wait_log(
+ Config,
+ [?FAIL_IF_CRASH_LOG
+ ,{[io_lib:format("MQTT resource access refused: configure access to queue "
+ "'mqtt-subscription-~sqos1' in vhost 'mqtt-vhost' refused for user 'mqtt-user'",
+ [ClientId]),
+ "MQTT connection .* is closing due to an authorization failure"],
+ fun() -> stop end}
+ ]),
ok.
no_queue_consume_permission_on_connect(Config) ->
set_permissions(".*", ".*", ".*", Config),
- C1 = open_mqtt_connection(Config, [{clientid, <<"no_queue_consume_permission_on_connect">>}, {clean_start, false}]),
+ ClientId = <<"no_queue_consume_permission_on_connect">>,
+ {ok, C1} = connect_user(
+ ?config(mqtt_user, Config),
+ ?config(mqtt_password, Config),
+ Config,
+ ClientId,
+ [{clean_start, false}]),
+ {ok, _} = emqtt:connect(C1),
{ok, _, _} = emqtt:subscribe(C1, {<<"test/topic">>, qos1}),
ok = emqtt:disconnect(C1),
- set_permissions(".*", ".*", "^amq\\.topic", Config),
+ set_permissions(".*", ".*", "^amq\\.topic", Config),
{ok, C2} = connect_user(
?config(mqtt_user, Config),
?config(mqtt_password, Config),
Config,
- ?config(mqtt_user, Config),
- [{clientid, <<"no_queue_consume_permission_on_connect">>},
- {clean_start, false}]),
+ ClientId,
+ [{clean_start, false}]),
unlink(C2),
?assertMatch({error, _},
emqtt:connect(C2)),
-
- wait_log(Config, erlang:system_time(microsecond) + 1000000,
- [{["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end}
- ,{["operation basic.consume caused a channel exception access_refused",
- "MQTT cannot recover a session, user is missing permissions"],
- fun () -> stop end}
- ]),
+ wait_log(
+ Config,
+ [?FAIL_IF_CRASH_LOG
+ ,{[io_lib:format("MQTT resource access refused: read access to queue "
+ "'mqtt-subscription-~sqos1' in vhost 'mqtt-vhost' refused for user 'mqtt-user'",
+ [ClientId]),
+ "MQTT connection .* is closing due to an authorization failure"],
+ fun () -> stop end}
+ ]),
ok.
no_queue_declare_permission(Config) ->
- rabbit_ct_broker_helpers:set_permissions(Config, ?config(mqtt_user, Config), ?config(mqtt_vhost, Config), <<"">>, <<".*">>, <<".*">>),
- P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
- {ok, C} = emqtt:start_link([{host, "localhost"},
- {port, P},
- {clientid, <<"no_queue_declare_permission">>},
- {proto_ver, v4},
- {username, ?config(mqtt_user, Config)},
- {password, ?config(mqtt_password, Config)},
- {clean_start, false}
- ]),
+ set_permissions("", ".*", ".*", Config),
+ ClientId = <<"no_queue_declare_permission">>,
+ {ok, C} = connect_user(
+ ?config(mqtt_user, Config),
+ ?config(mqtt_password, Config),
+ Config,
+ ClientId,
+ [{clean_start, true}]),
{ok, _} = emqtt:connect(C),
process_flag(trap_exit, true),
- try emqtt:subscribe(C, <<"test/topic">>) of
- _ -> exit(this_should_not_succeed)
- catch
- exit:{{shutdown, tcp_closed} , _} -> ok
- end,
- process_flag(trap_exit, false),
+ {ok, _, [?SUBACK_FAILURE]} = emqtt:subscribe(C, <<"test/topic">>, qos0),
+ ok = assert_connection_closed(C),
+ wait_log(
+ Config,
+ [?FAIL_IF_CRASH_LOG
+ ,{[io_lib:format("MQTT resource access refused: configure access to queue "
+ "'mqtt-subscription-~sqos0' in vhost 'mqtt-vhost' refused for user 'mqtt-user'",
+ [ClientId]),
+ "MQTT protocol error on connection .*: subscribe_error"],
+ fun () -> stop end}
+ ]),
+ ok.
- wait_log(Config, erlang:system_time(microsecond) + 1_000_000,
- [{["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end}
- ,{["MQTT protocol error on connection.*access_refused",
- "operation queue.declare caused a channel exception access_refused"],
- fun () -> stop end}
+no_publish_permission(Config) ->
+ set_permissions(".*", "", ".*", Config),
+ C = open_mqtt_connection(Config),
+ process_flag(trap_exit, true),
+ ok = emqtt:publish(C, <<"some/topic">>, <<"payload">>),
+ assert_connection_closed(C),
+ wait_log(Config,
+ [?FAIL_IF_CRASH_LOG
+ ,{["MQTT resource access refused: write access to exchange "
+ "'amq.topic' in vhost 'mqtt-vhost' refused for user 'mqtt-user'",
+ "MQTT connection .* is closing due to an authorization failure"],
+ fun () -> stop end}
]),
ok.
no_topic_read_permission(Config) ->
set_permissions(".*", ".*", ".*", Config),
set_topic_permissions("^allow-write\\..*", "^allow-read\\..*", Config),
-
C = open_mqtt_connection(Config),
- %% Just to be sure that our permission setup is indeed working
- {ok, _, _} = emqtt:subscribe(C, <<"allow-read/some/topic">>),
+ %% Check topic permission setup is working.
+ {ok, _, [0]} = emqtt:subscribe(C, <<"allow-read/some/topic">>),
- expect_sync_error(fun () ->
- emqtt:subscribe(C, <<"test/topic">>)
- end),
- wait_log(Config, erlang:system_time(microsecond) + 1_000_000,
- [?WAIT_LOG_NO_CRASHES
- ,{["MQTT protocol error on connection.*access_refused",
- "operation queue.bind caused a channel exception access_refused: access to topic 'test.topic' in exchange 'amq.topic' in vhost 'mqtt-vhost' refused for user 'mqtt-user'"],
+ process_flag(trap_exit, true),
+ {ok, _, [?SUBACK_FAILURE]} = emqtt:subscribe(C, <<"test/topic">>),
+ ok = assert_connection_closed(C),
+ wait_log(Config,
+ [?FAIL_IF_CRASH_LOG,
+ {["MQTT topic access refused: read access to topic 'test.topic' in exchange "
+ "'amq.topic' in vhost 'mqtt-vhost' refused for user 'mqtt-user'",
+ "Failed to bind queue 'mqtt-subscription-mqtt-userqos0' "
+ "in vhost 'mqtt-vhost' with topic test/topic: access_refused"
+ ],
fun () -> stop end}
]),
ok.
@@ -628,37 +725,50 @@ no_topic_write_permission(Config) ->
set_topic_permissions("^allow-write\\..*", "^allow-read\\..*", Config),
C = open_mqtt_connection(Config),
- %% Just to be sure that our permission setup is indeed working
+ %% Check topic permission setup is working.
{ok, _} = emqtt:publish(C, <<"allow-write/some/topic">>, <<"payload">>, qos1),
- unlink(C),
+ process_flag(trap_exit, true),
?assertMatch({error, _},
emqtt:publish(C, <<"some/other/topic">>, <<"payload">>, qos1)),
-
- wait_log(Config, erlang:system_time(microsecond) + 1_000_000,
- [?WAIT_LOG_NO_CRASHES
- ,{["access to topic 'some.other.topic' in exchange 'amq.topic' in vhost 'mqtt-vhost' refused for user 'mqtt-user'",
- "MQTT connection.*is closing due to an authorization failure"],
- fun () -> stop end}
+ wait_log(Config,
+ [?FAIL_IF_CRASH_LOG
+ ,{["MQTT topic access refused: write access to topic 'some.other.topic' in "
+ "exchange 'amq.topic' in vhost 'mqtt-vhost' refused for user 'mqtt-user'",
+ "MQTT connection .* is closing due to an authorization failure"],
+ fun () -> stop end}
]),
ok.
-expect_sync_error(Fun) ->
+loopback_user_connects_from_remote_host(Config) ->
+ set_permissions(".*", ".*", ".*", Config),
+ {ok, C} = connect_anonymous(Config),
+
+ %% CT node runs on the same host as the RabbitMQ node.
+ %% Instead of changing the IP address of CT node to a non-loopback IP address,
+ %% we mock rabbit_access_control:check_user_loopback/2.
+ rabbit_ct_broker_helpers:setup_meck(Config),
+ Mod = rabbit_access_control,
+ ok = rpc(Config, 0, meck, new, [Mod, [no_link, passthrough]]),
+ ok = rpc(Config, 0, meck, expect, [Mod, check_user_loopback, 2, not_allowed]),
+
process_flag(trap_exit, true),
- try Fun() of
- _ -> exit(this_should_not_succeed)
- catch
- exit:{{shutdown, tcp_closed} , _} -> ok
- after
- process_flag(trap_exit, false)
- end.
+ ?assertMatch({error, _}, emqtt:connect(C)),
+ wait_log(Config,
+ [?FAIL_IF_CRASH_LOG,
+ {["MQTT login failed: user 'mqtt-user' can only connect via localhost",
+ "MQTT connection .* is closing due to an authorization failure"],
+ fun () -> stop end}
+ ]),
-set_topic_permissions(WritePat, ReadPat, Config) ->
- rabbit_ct_broker_helpers:rpc(Config, 0,
- rabbit_auth_backend_internal, set_topic_permissions,
- [?config(mqtt_user, Config), ?config(mqtt_vhost, Config),
- <<"amq.topic">>, WritePat, ReadPat, <<"acting-user">>]).
+ true = rpc(Config, 0, meck, validate, [Mod]),
+ ok = rpc(Config, 0, meck, unload, [Mod]).
+set_topic_permissions(WritePat, ReadPat, Config) ->
+ rpc(Config, 0,
+ rabbit_auth_backend_internal, set_topic_permissions,
+ [?config(mqtt_user, Config), ?config(mqtt_vhost, Config),
+ <<"amq.topic">>, WritePat, ReadPat, <<"acting-user">>]).
set_permissions(PermConf, PermWrite, PermRead, Config) ->
rabbit_ct_broker_helpers:set_permissions(Config, ?config(mqtt_user, Config), ?config(mqtt_vhost, Config),
@@ -674,26 +784,36 @@ open_mqtt_connection(Config, Opts) ->
C.
test_subscribe_permissions_combination(PermConf, PermWrite, PermRead, Config, ExtraLogChecks) ->
- rabbit_ct_broker_helpers:set_permissions(Config, ?config(mqtt_user, Config), ?config(mqtt_vhost, Config), PermConf, PermWrite, PermRead),
-
- {ok, C} = connect_user(?config(mqtt_user, Config), ?config(mqtt_password, Config), Config),
- {ok, _} = emqtt:connect(C),
-
+ rabbit_ct_broker_helpers:set_permissions(Config,
+ ?config(mqtt_user, Config),
+ ?config(mqtt_vhost, Config),
+ PermConf, PermWrite, PermRead),
+ P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
+ User = ?config(mqtt_user, Config),
+ Opts = [{host, "localhost"},
+ {port, P},
+ {clientid, User},
+ {username, User},
+ {password, ?config(mqtt_password, Config)}],
+ {ok, C1} = emqtt:start_link([{proto_ver, v4} | Opts]),
+ {ok, _} = emqtt:connect(C1),
process_flag(trap_exit, true),
- try emqtt:subscribe(C, <<"test/topic">>) of
- _ -> exit(this_should_not_succeed)
- catch
- exit:{{shutdown, tcp_closed} , _} -> ok
- end,
+ %% In v4, we expect to receive a failure return code for our subscription in the SUBACK packet.
+ ?assertMatch({ok, _Properties, [?SUBACK_FAILURE]},
+ emqtt:subscribe(C1, <<"test/topic">>)),
+ ok = assert_connection_closed(C1),
+ wait_log(Config,
+ [?FAIL_IF_CRASH_LOG
+ ,{["MQTT protocol error on connection.*: subscribe_error"|ExtraLogChecks], fun () -> stop end}
+ ]),
- process_flag(trap_exit, false),
+ {ok, C2} = emqtt:start_link([{proto_ver, v3} | Opts]),
+ {ok, _} = emqtt:connect(C2),
- wait_log(Config, erlang:system_time(microsecond) + 1_000_000,
- [{["Generic server.*terminating"], fun () -> exit(there_should_be_no_crashes) end}
- ,{["MQTT protocol error on connection.*access_refused"|ExtraLogChecks],
- fun () -> stop end}
- ]),
- ok.
+ %% In v3, there is no failure return code in the SUBACK packet.
+ ?assertMatch({ok, _Properties, [0]},
+ emqtt:subscribe(C2, <<"test/topic">>)),
+ ok = assert_connection_closed(C2).
connect_user(User, Pass, Config) ->
connect_user(User, Pass, Config, User, []).
@@ -716,12 +836,11 @@ connect_user(User, Pass, Config, ClientID0, Opts) ->
[{host, "localhost"}, {port, P}, {proto_ver, v4}]).
expect_successful_connection(ConnectFun, Config) ->
- rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_core_metrics, reset_auth_attempt_metrics, []),
+ rpc(Config, 0, rabbit_core_metrics, reset_auth_attempt_metrics, []),
{ok, C} = ConnectFun(Config),
{ok, _} = emqtt:connect(C),
ok = emqtt:disconnect(C),
- [Attempt] =
- rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_core_metrics, get_auth_attempts, []),
+ [Attempt] = rpc(Config, 0, rabbit_core_metrics, get_auth_attempts, []),
?assertEqual(false, proplists:is_defined(remote_address, Attempt)),
?assertEqual(false, proplists:is_defined(username, Attempt)),
?assertEqual(proplists:get_value(protocol, Attempt), <<"mqtt">>),
@@ -730,12 +849,11 @@ expect_successful_connection(ConnectFun, Config) ->
?assertEqual(proplists:get_value(auth_attempts_succeeded, Attempt), 1).
expect_authentication_failure(ConnectFun, Config) ->
- rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_core_metrics, reset_auth_attempt_metrics, []),
+ rpc(Config, 0, rabbit_core_metrics, reset_auth_attempt_metrics, []),
{ok, C} = ConnectFun(Config),
unlink(C),
?assertMatch({error, _}, emqtt:connect(C)),
- [Attempt] =
- rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_core_metrics, get_auth_attempts, []),
+ [Attempt] = rpc(Config, 0, rabbit_core_metrics, get_auth_attempts, []),
?assertEqual(false, proplists:is_defined(remote_address, Attempt), <<>>),
?assertEqual(false, proplists:is_defined(username, Attempt)),
?assertEqual(proplists:get_value(protocol, Attempt), <<"mqtt">>),
@@ -744,23 +862,63 @@ expect_authentication_failure(ConnectFun, Config) ->
?assertEqual(proplists:get_value(auth_attempts_succeeded, Attempt), 0),
ok.
-wait_log(Config, Deadline, Clauses) ->
+vhost_connection_limit(Config) ->
+ ok = rabbit_ct_broker_helpers:set_vhost_limit(Config, 0, <<"/">>, max_connections, 2),
+ {ok, C1} = connect_anonymous(Config, <<"client1">>),
+ {ok, _} = emqtt:connect(C1),
+ {ok, C2} = connect_anonymous(Config, <<"client2">>),
+ {ok, _} = emqtt:connect(C2),
+ {ok, C3} = connect_anonymous(Config, <<"client3">>),
+ unlink(C3),
+ ?assertMatch({error, {unauthorized_client, _}}, emqtt:connect(C3)),
+ ok = emqtt:disconnect(C1),
+ ok = emqtt:disconnect(C2).
+
+vhost_queue_limit(Config) ->
+ ok = rabbit_ct_broker_helpers:set_vhost_limit(Config, 0, <<"/">>, max_queues, 1),
+ {ok, C} = connect_anonymous(Config),
+ {ok, _} = emqtt:connect(C),
+ process_flag(trap_exit, true),
+ %% qos0 queue can be created, qos1 queue fails to be created.
+ %% (RabbitMQ creates subscriptions in the reverse order of the SUBSCRIBE packet.)
+ ?assertMatch({ok, _Properties, [?SUBACK_FAILURE, ?SUBACK_FAILURE, 0]},
+ emqtt:subscribe(C, [{<<"topic1">>, qos1},
+ {<<"topic2">>, qos1},
+ {<<"topic3">>, qos0}])),
+ ok = assert_connection_closed(C).
+
+user_connection_limit(Config) ->
+ DefaultUser = <<"guest">>,
+ ok = rabbit_ct_broker_helpers:set_user_limits(Config, DefaultUser, #{max_connections => 1}),
+ {ok, C1} = connect_anonymous(Config, <<"client1">>),
+ {ok, _} = emqtt:connect(C1),
+ {ok, C2} = connect_anonymous(Config, <<"client2">>),
+ unlink(C2),
+ ?assertMatch({error, {unauthorized_client, _}}, emqtt:connect(C2)),
+ ok = emqtt:disconnect(C1),
+ ok = rabbit_ct_broker_helpers:clear_user_limits(Config, DefaultUser, max_connections).
+
+wait_log(Config, Clauses) ->
+ wait_log(Config, Clauses, erlang:monotonic_time(millisecond) + 1000).
+
+wait_log(Config, Clauses, Deadline) ->
{ok, Content} = file:read_file(?config(log_location, Config)),
- case erlang:system_time(microsecond) of
- T when T > Deadline ->
- lists:foreach(fun
- ({REs, _}) ->
- Matches = [ io_lib:format("~tp - ~ts~n", [RE, re:run(Content, RE, [{capture, none}])]) || RE <- REs ],
- ct:pal("Wait log clause status: ~ts", [Matches])
- end, Clauses),
- exit(no_log_lines_detected);
- _ -> ok
- end,
- case wait_log_check_clauses(Content, Clauses) of
- stop -> ok;
- continue ->
- timer:sleep(50),
- wait_log(Config, Deadline, Clauses)
+ case erlang:monotonic_time(millisecond) of
+ T when T =< Deadline ->
+ case wait_log_check_clauses(Content, Clauses) of
+ stop -> ok;
+ continue ->
+ timer:sleep(50),
+ wait_log(Config, Clauses, Deadline)
+ end;
+ _ ->
+ lists:foreach(
+ fun
+ ({REs, _}) ->
+ Matches = [ io_lib:format("~p - ~s~n", [RE, re:run(Content, RE, [{capture, none}])]) || RE <- REs ],
+ ct:pal("Wait log clause status: ~s", [Matches])
+ end, Clauses),
+ ct:fail(expected_logs_not_found)
end,
ok.
@@ -774,7 +932,16 @@ wait_log_check_clauses(Content, [{REs, Fun}|Rest]) ->
end.
multiple_re_match(Content, REs) ->
- lists:all(fun (RE) ->
- match == re:run(Content, RE, [{capture, none}])
- end,
- REs).
+ lists:all(
+ fun (RE) ->
+ match == re:run(Content, RE, [{capture, none}])
+ end, REs).
+
+assert_connection_closed(ClientPid) ->
+ receive
+ {'EXIT', ClientPid, {shutdown, tcp_closed}} ->
+ ok
+ after
+ 2000 ->
+ ct:fail("timed out waiting for exit message")
+ end.
diff --git a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl
index 510557285b29..0ac2434ea211 100644
--- a/deps/rabbitmq_mqtt/test/cluster_SUITE.erl
+++ b/deps/rabbitmq_mqtt/test/cluster_SUITE.erl
@@ -9,48 +9,52 @@
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
+-import(util, [expect_publishes/3,
+ connect/3,
+ connect/4,
+ await_exit/1]).
-import(rabbit_ct_broker_helpers,
[setup_steps/0,
teardown_steps/0,
get_node_config/3,
rabbitmqctl/3,
- rpc/5,
- stop_node/2,
- drain_node/2,
- revive_node/2]).
+ rpc/4,
+ stop_node/2
+ ]).
+
+-define(OPTS, [{connect_timeout, 1},
+ {ack_timeout, 1}]).
all() ->
[
- {group, cluster_size_3},
{group, cluster_size_5}
].
groups() ->
[
- {cluster_size_3, [], [
- maintenance
- ]},
- {cluster_size_5, [], [
- connection_id_tracking,
- connection_id_tracking_on_nodedown,
- connection_id_tracking_with_decommissioned_node
- ]}
+ {cluster_size_5, [],
+ [
+ connection_id_tracking,
+ connection_id_tracking_on_nodedown,
+ connection_id_tracking_with_decommissioned_node
+ ]}
].
suite() ->
- [{timetrap, {minutes, 5}}].
+ [{timetrap, {minutes, 3}}].
%% -------------------------------------------------------------------
%% Testsuite setup/teardown.
%% -------------------------------------------------------------------
merge_app_env(Config) ->
- rabbit_ct_helpers:merge_app_env(Config,
- {rabbit, [
- {collect_statistics, basic},
- {collect_statistics_interval, 100}
- ]}).
+ rabbit_ct_helpers:merge_app_env(
+ Config,
+ {rabbit, [
+ {collect_statistics, basic},
+ {collect_statistics_interval, 100}
+ ]}).
init_per_suite(Config) ->
rabbit_ct_helpers:log_environment(),
@@ -59,20 +63,9 @@ init_per_suite(Config) ->
end_per_suite(Config) ->
rabbit_ct_helpers:run_teardown_steps(Config).
-init_per_group(cluster_size_3, Config) ->
- case rabbit_ct_helpers:is_mixed_versions() of
- true ->
- {skip, "maintenance mode wrongly closes cluster-wide MQTT connections "
- " in RMQ < 3.11.2 and < 3.10.10"};
- false ->
- set_cluster_size(3, Config)
- end;
init_per_group(cluster_size_5, Config) ->
- set_cluster_size(5, Config).
-
-set_cluster_size(NodesCount, Config) ->
rabbit_ct_helpers:set_config(
- Config, [{rmq_nodes_count, NodesCount}]).
+ Config, [{rmq_nodes_count, 5}]).
end_per_group(_, Config) ->
Config.
@@ -101,26 +94,6 @@ end_per_testcase(Testcase, Config) ->
%% Test cases
%% -------------------------------------------------------------------
-maintenance(Config) ->
- {ok, MRef0, C0} = connect_to_node(Config, 0, <<"client-0">>),
- {ok, MRef1a, C1a} = connect_to_node(Config, 1, <<"client-1a">>),
- {ok, MRef1b, C1b} = connect_to_node(Config, 1, <<"client-1b">>),
- timer:sleep(500),
-
- ok = drain_node(Config, 2),
- ok = revive_node(Config, 2),
- timer:sleep(500),
- [?assert(erlang:is_process_alive(C)) || C <- [C0, C1a, C1b]],
-
- ok = drain_node(Config, 1),
- [await_disconnection(Ref) || Ref <- [MRef1a, MRef1b]],
- ok = revive_node(Config, 1),
- ?assert(erlang:is_process_alive(C0)),
-
- ok = drain_node(Config, 0),
- await_disconnection(MRef0),
- ok = revive_node(Config, 0).
-
%% Note about running this testsuite in a mixed-versions cluster:
%% All even-numbered nodes will use the same code base when using a
%% secondary Umbrella. Odd-numbered nodes might use an incompatible code
@@ -135,102 +108,71 @@ maintenance(Config) ->
%% nodes, the minimum to use Ra in proper conditions.
connection_id_tracking(Config) ->
- ID = <<"duplicate-id">>,
- {ok, MRef1, C1} = connect_to_node(Config, 0, ID),
+ Id = <<"duplicate-id">>,
+ C1 = connect(Id, Config, 0, ?OPTS),
{ok, _, _} = emqtt:subscribe(C1, <<"TopicA">>, qos0),
ok = emqtt:publish(C1, <<"TopicA">>, <<"Payload">>),
- expect_publishes(<<"TopicA">>, [<<"Payload">>]),
+ ok = expect_publishes(C1, <<"TopicA">>, [<<"Payload">>]),
%% there's one connection
- assert_connection_count(Config, 10, 2, 1),
+ assert_connection_count(Config, 4, 2, 1),
%% connect to the same node (A or 0)
- {ok, MRef2, _C2} = connect_to_node(Config, 0, ID),
-
- %% C1 is disconnected
- await_disconnection(MRef1),
+ process_flag(trap_exit, true),
+ C2 = connect(Id, Config, 0, ?OPTS),
+ await_exit(C1),
+ assert_connection_count(Config, 4, 2, 1),
%% connect to a different node (C or 2)
- {ok, _, C3} = connect_to_node(Config, 2, ID),
- assert_connection_count(Config, 10, 2, 1),
-
- %% C2 is disconnected
- await_disconnection(MRef2),
+ C3 = connect(Id, Config, 2, ?OPTS),
+ await_exit(C2),
+ assert_connection_count(Config, 4, 2, 1),
ok = emqtt:disconnect(C3).
connection_id_tracking_on_nodedown(Config) ->
Server = get_node_config(Config, 0, nodename),
- {ok, MRef, C} = connect_to_node(Config, 0, <<"simpleClient">>),
+ C = connect(<<"simpleClient">>, Config, ?OPTS),
{ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0),
ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>),
- expect_publishes(<<"TopicA">>, [<<"Payload">>]),
- assert_connection_count(Config, 10, 2, 1),
+ ok = expect_publishes(C, <<"TopicA">>, [<<"Payload">>]),
+ assert_connection_count(Config, 4, 2, 1),
+ process_flag(trap_exit, true),
ok = stop_node(Config, Server),
- await_disconnection(MRef),
- assert_connection_count(Config, 10, 2, 0),
+ await_exit(C),
+ assert_connection_count(Config, 4, 2, 0),
ok.
connection_id_tracking_with_decommissioned_node(Config) ->
- Server = get_node_config(Config, 0, nodename),
- {ok, MRef, C} = connect_to_node(Config, 0, <<"simpleClient">>),
- {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0),
- ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>),
- expect_publishes(<<"TopicA">>, [<<"Payload">>]),
-
- assert_connection_count(Config, 10, 2, 1),
- {ok, _} = rabbitmqctl(Config, 0, ["decommission_mqtt_node", Server]),
- await_disconnection(MRef),
- assert_connection_count(Config, 10, 2, 0),
- ok.
+ case rpc(Config, rabbit_mqtt_ff, track_client_id_in_ra, []) of
+ false ->
+ {skip, "This test requires client ID tracking in Ra"};
+ true ->
+ Server = get_node_config(Config, 0, nodename),
+ C = connect(<<"simpleClient">>, Config, ?OPTS),
+ {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0),
+ ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>),
+ ok = expect_publishes(C, <<"TopicA">>, [<<"Payload">>]),
+
+ assert_connection_count(Config, 4, 2, 1),
+ process_flag(trap_exit, true),
+ {ok, _} = rabbitmqctl(Config, 0, ["decommission_mqtt_node", Server]),
+ await_exit(C),
+ assert_connection_count(Config, 5, 2, 0),
+ ok
+ end.
%%
%% Helpers
%%
-assert_connection_count(_Config, 0, _, _) ->
- ct:fail("failed to complete rabbit_mqtt_collector:list/0");
+assert_connection_count(_Config, 0, _, NumElements) ->
+ ct:fail("failed to match connection count ~b", [NumElements]);
assert_connection_count(Config, Retries, NodeId, NumElements) ->
- List = rpc(Config, NodeId, rabbit_mqtt_collector, list, []),
- case length(List) == NumElements of
- true ->
+ case util:all_connection_pids(Config) of
+ Pids
+ when length(Pids) =:= NumElements ->
ok;
- false ->
- timer:sleep(200),
+ _ ->
+ timer:sleep(500),
assert_connection_count(Config, Retries-1, NodeId, NumElements)
end.
-
-
-
-connect_to_node(Config, Node, ClientID) ->
- Port = get_node_config(Config, Node, tcp_port_mqtt),
- {ok, C} = connect(Port, ClientID),
- MRef = erlang:monitor(process, C),
- {ok, MRef, C}.
-
-connect(Port, ClientID) ->
- {ok, C} = emqtt:start_link([{host, "localhost"},
- {port, Port},
- {clientid, ClientID},
- {proto_ver, v4},
- {connect_timeout, 1},
- {ack_timeout, 1}]),
- {ok, _Properties} = emqtt:connect(C),
- unlink(C),
- {ok, C}.
-
-await_disconnection(Ref) ->
- receive
- {'DOWN', Ref, _, _, _} -> ok
- after
- 20_000 -> exit(missing_down_message)
- end.
-
-expect_publishes(_Topic, []) -> ok;
-expect_publishes(Topic, [Payload|Rest]) ->
- receive
- {publish, #{topic := Topic,
- payload := Payload}} ->
- expect_publishes(Topic, Rest)
- after 5000 ->
- throw({publish_not_delivered, Payload})
- end.
diff --git a/deps/rabbitmq_mqtt/test/command_SUITE.erl b/deps/rabbitmq_mqtt/test/command_SUITE.erl
index 0b3301a95243..76421287d312 100644
--- a/deps/rabbitmq_mqtt/test/command_SUITE.erl
+++ b/deps/rabbitmq_mqtt/test/command_SUITE.erl
@@ -6,13 +6,12 @@
-module(command_SUITE).
--compile([export_all]).
+-compile([export_all, nowarn_export_all]).
--include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
-include_lib("amqp_client/include/amqp_client.hrl").
-include("rabbit_mqtt.hrl").
-
+-import(util, [connect/3]).
-define(COMMAND, 'Elixir.RabbitMQ.CLI.Ctl.Commands.ListMqttConnectionsCommand').
@@ -29,6 +28,11 @@ groups() ->
]}
].
+suite() ->
+ [
+ {timetrap, {minutes, 3}}
+ ].
+
init_per_suite(Config) ->
rabbit_ct_helpers:log_environment(),
Config1 = rabbit_ct_helpers:set_config(Config, [
@@ -71,35 +75,21 @@ merge_defaults(_Config) ->
run(Config) ->
-
Node = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
- Opts = #{node => Node, timeout => 10000, verbose => false},
+ Opts = #{node => Node, timeout => 10_000, verbose => false},
%% No connections
[] = 'Elixir.Enum':to_list(?COMMAND:run([], Opts)),
- P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
- {ok, C1} = emqtt:start_link([{host, "localhost"},
- {port, P},
- {clientid, <<"simpleClient">>},
- {proto_ver, v4},
- {ack_timeout, 1}]),
- {ok, _} = emqtt:connect(C1),
- ct:sleep(100),
+ C1 = connect(<<"simpleClient">>, Config, [{ack_timeout, 1}]),
+
+ timer:sleep(100),
[[{client_id, <<"simpleClient">>}]] =
'Elixir.Enum':to_list(?COMMAND:run([<<"client_id">>], Opts)),
-
- {ok, C2} = emqtt:start_link([{host, "localhost"},
- {port, P},
- {clientid, <<"simpleClient1">>},
- {proto_ver, v4},
- {username, <<"guest">>},
- {password, <<"guest">>},
- {ack_timeout, 1}]),
- {ok, _} = emqtt:connect(C2),
- ct:sleep(200),
+ C2 = connect(<<"simpleClient1">>, Config, [{ack_timeout, 1}]),
+ timer:sleep(200),
[[{client_id, <<"simpleClient">>}, {user, <<"guest">>}],
[{client_id, <<"simpleClient1">>}, {user, <<"guest">>}]] =
@@ -110,40 +100,29 @@ run(Config) ->
Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
start_amqp_connection(network, Node, Port),
- %% There are still just two connections
+ %% There are still just two MQTT connections
[[{client_id, <<"simpleClient">>}],
[{client_id, <<"simpleClient1">>}]] =
lists:sort('Elixir.Enum':to_list(?COMMAND:run([<<"client_id">>], Opts))),
start_amqp_connection(direct, Node, Port),
- ct:sleep(200),
+ timer:sleep(200),
- %% Still two MQTT connections, one direct AMQP 0-9-1 connection
- [[{client_id, <<"simpleClient">>}],
- [{client_id, <<"simpleClient1">>}]] =
- lists:sort('Elixir.Enum':to_list(?COMMAND:run([<<"client_id">>], Opts))),
+ %% Still two MQTT connections
+ ?assertEqual(
+ [[{client_id, <<"simpleClient">>}],
+ [{client_id, <<"simpleClient1">>}]],
+ lists:sort('Elixir.Enum':to_list(?COMMAND:run([<<"client_id">>], Opts)))),
%% Verbose returns all keys
- Infos = lists:map(fun(El) -> atom_to_binary(El, utf8) end, ?INFO_ITEMS),
- AllKeys1 = 'Elixir.Enum':to_list(?COMMAND:run(Infos, Opts)),
- AllKeys2 = 'Elixir.Enum':to_list(?COMMAND:run([], Opts#{verbose => true})),
-
- %% There are two connections
- [FirstPL, _] = AllKeys1,
- [SecondPL, _] = AllKeys2,
-
- First = maps:from_list(lists:usort(FirstPL)),
- Second = maps:from_list(lists:usort(SecondPL)),
+ AllKeys = lists:map(fun(I) -> atom_to_binary(I) end, ?INFO_ITEMS),
+ [AllInfos1Con1, _AllInfos1Con2] = 'Elixir.Enum':to_list(?COMMAND:run(AllKeys, Opts)),
+ [AllInfos2Con1, _AllInfos2Con2] = 'Elixir.Enum':to_list(?COMMAND:run([], Opts#{verbose => true})),
%% Keys are INFO_ITEMS
- KeysCount = length(?INFO_ITEMS),
- ?assert(KeysCount =:= maps:size(First)),
- ?assert(KeysCount =:= maps:size(Second)),
-
- Keys = maps:keys(First),
-
- [] = Keys -- ?INFO_ITEMS,
- [] = ?INFO_ITEMS -- Keys,
+ InfoItemsSorted = lists:sort(?INFO_ITEMS),
+ ?assertEqual(InfoItemsSorted, lists:sort(proplists:get_keys(AllInfos1Con1))),
+ ?assertEqual(InfoItemsSorted, lists:sort(proplists:get_keys(AllInfos2Con1))),
ok = emqtt:disconnect(C1),
ok = emqtt:disconnect(C2).
diff --git a/deps/rabbitmq_mqtt/test/config_SUITE.erl b/deps/rabbitmq_mqtt/test/config_SUITE.erl
new file mode 100644
index 000000000000..b2b04b52e2c6
--- /dev/null
+++ b/deps/rabbitmq_mqtt/test/config_SUITE.erl
@@ -0,0 +1,94 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
+
+-module(config_SUITE).
+-compile([export_all,
+ nowarn_export_all]).
+
+-include_lib("eunit/include/eunit.hrl").
+
+-import(rabbit_ct_broker_helpers, [rpc/5]).
+
+all() ->
+ [
+ {group, mnesia}
+ ].
+
+groups() ->
+ [
+ {mnesia, [shuffle],
+ [
+ rabbitmq_default,
+ environment_set,
+ flag_set
+ ]}
+ ].
+
+suite() ->
+ [{timetrap, {minutes, 2}}].
+
+%% -------------------------------------------------------------------
+%% 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_testcase(rabbitmq_default = Test, Config) ->
+ init_per_testcase0(Test, Config);
+init_per_testcase(environment_set = Test, Config0) ->
+ Config = rabbit_ct_helpers:merge_app_env(
+ Config0, {mnesia, [{dump_log_write_threshold, 25000},
+ {dump_log_time_threshold, 60000}]}),
+ init_per_testcase0(Test, Config);
+init_per_testcase(flag_set = Test, Config0) ->
+ Config = [{additional_erl_args, "-mnesia dump_log_write_threshold 15000"} | Config0],
+ init_per_testcase0(Test, Config).
+
+init_per_testcase0(Testcase, Config0) ->
+ Config1 = rabbit_ct_helpers:set_config(Config0, {rmq_nodename_suffix, Testcase}),
+ Config = rabbit_ct_helpers:run_steps(
+ Config1,
+ rabbit_ct_broker_helpers:setup_steps() ++
+ rabbit_ct_client_helpers:setup_steps()),
+ rabbit_ct_helpers:testcase_started(Config, Testcase).
+
+end_per_testcase(Testcase, Config0) ->
+ Config = rabbit_ct_helpers:testcase_finished(Config0, Testcase),
+ rabbit_ct_helpers:run_teardown_steps(
+ Config,
+ rabbit_ct_client_helpers:teardown_steps() ++
+ rabbit_ct_broker_helpers:teardown_steps()).
+
+%% -------------------------------------------------------------------
+%% Testsuite cases
+%% -------------------------------------------------------------------
+
+%% The MQTT plugin expects Mnesia dump_log_write_threshold to be increased
+%% from 1000 (Mnesia default) to 5000 (RabbitMQ default).
+rabbitmq_default(Config) ->
+ ?assertEqual(5_000,
+ rpc(Config, 0, mnesia, system_info, [dump_log_write_threshold])),
+ ?assertEqual(90_000,
+ rpc(Config, 0, mnesia, system_info, [dump_log_time_threshold])).
+
+%% User configured setting in advanced.config should be respected.
+environment_set(Config) ->
+ ?assertEqual(25_000,
+ rpc(Config, 0, mnesia, system_info, [dump_log_write_threshold])),
+ ?assertEqual(60_000,
+ rpc(Config, 0, mnesia, system_info, [dump_log_time_threshold])).
+
+%% User configured setting in RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS should be respected.
+flag_set(Config) ->
+ ?assertEqual(15_000,
+ rpc(Config, 0, mnesia, system_info, [dump_log_write_threshold])),
+ ?assertEqual(90_000,
+ rpc(Config, 0, mnesia, system_info, [dump_log_time_threshold])).
diff --git a/deps/rabbitmq_mqtt/test/event_recorder.erl b/deps/rabbitmq_mqtt/test/event_recorder.erl
new file mode 100644
index 000000000000..cd495f9427a5
--- /dev/null
+++ b/deps/rabbitmq_mqtt/test/event_recorder.erl
@@ -0,0 +1,27 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(event_recorder).
+-behaviour(gen_event).
+-export([init/1, handle_event/2, handle_call/2]).
+-define(INIT_STATE, []).
+
+-include_lib("rabbit_common/include/rabbit.hrl").
+
+init(_) ->
+ {ok, ?INIT_STATE}.
+
+handle_event(#event{type = T}, State)
+ when T =:= node_stats orelse
+ T =:= node_node_stats orelse
+ T =:= node_node_deleted ->
+ {ok, State};
+handle_event(Event, State) ->
+ {ok, [Event | State]}.
+
+handle_call(take_state, State) ->
+ {ok, lists:reverse(State), ?INIT_STATE}.
diff --git a/deps/rabbitmq_mqtt/test/ff_SUITE.erl b/deps/rabbitmq_mqtt/test/ff_SUITE.erl
new file mode 100644
index 000000000000..a7c528c64076
--- /dev/null
+++ b/deps/rabbitmq_mqtt/test/ff_SUITE.erl
@@ -0,0 +1,132 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2016-2023 VMware, Inc. or its affiliates. All rights reserved.
+
+-module(ff_SUITE).
+
+-compile([export_all, nowarn_export_all]).
+
+-include_lib("common_test/include/ct.hrl").
+-include_lib("eunit/include/eunit.hrl").
+
+-import(rabbit_ct_broker_helpers, [rpc/5]).
+-import(rabbit_ct_helpers, [eventually/1]).
+-import(util, [expect_publishes/3,
+ get_global_counters/4,
+ connect/2,
+ connect/4]).
+
+-define(PROTO_VER, v4).
+
+all() ->
+ [
+ {group, cluster_size_3}
+ ].
+
+groups() ->
+ [
+ {cluster_size_3, [], [delete_ra_cluster_mqtt_node,
+ rabbit_mqtt_qos0_queue]}
+ ].
+
+suite() ->
+ [
+ {timetrap, {minutes, 2}}
+ ].
+
+init_per_suite(Config) ->
+ rabbit_ct_helpers:log_environment(),
+ rabbit_ct_helpers:run_setup_steps(Config, []).
+
+end_per_suite(Config) ->
+ rabbit_ct_helpers:run_teardown_steps(Config).
+
+init_per_group(Group = cluster_size_3, Config0) ->
+ Config1 = rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 3},
+ {rmq_nodename_suffix, Group}]),
+ Config = rabbit_ct_helpers:merge_app_env(
+ Config1, {rabbit, [{forced_feature_flags_on_init, []}]}),
+ rabbit_ct_helpers:run_steps(Config,
+ rabbit_ct_broker_helpers:setup_steps() ++
+ rabbit_ct_client_helpers:setup_steps()).
+
+end_per_group(_Group, Config) ->
+ rabbit_ct_helpers:run_steps(Config,
+ rabbit_ct_client_helpers:teardown_steps() ++
+ rabbit_ct_broker_helpers:teardown_steps()).
+
+init_per_testcase(TestCase, Config) ->
+ case rabbit_ct_broker_helpers:is_feature_flag_supported(Config, TestCase) of
+ true ->
+ Config;
+ false ->
+ {skip, io_lib:format("feature flag ~s is unsupported",
+ [TestCase])}
+ end.
+
+end_per_testcase(_TestCase, Config) ->
+ Config.
+
+delete_ra_cluster_mqtt_node(Config) ->
+ FeatureFlag = ?FUNCTION_NAME,
+ C = connect(<<"my-client">>, Config, 1, []),
+ timer:sleep(500),
+ %% old client ID tracking works
+ ?assertEqual(1, length(util:all_connection_pids(Config))),
+ %% Ra processes are alive
+ ?assert(lists:all(fun erlang:is_pid/1,
+ rabbit_ct_broker_helpers:rpc_all(Config, erlang, whereis, [mqtt_node]))),
+
+ ?assertEqual(ok,
+ rabbit_ct_broker_helpers:enable_feature_flag(Config, FeatureFlag)),
+
+ %% Ra processes should be gone
+ rabbit_ct_helpers:eventually(
+ ?_assert(lists:all(fun(Pid) -> Pid =:= undefined end,
+ rabbit_ct_broker_helpers:rpc_all(Config, erlang, whereis, [mqtt_node])))),
+ %% new client ID tracking works
+ ?assertEqual(1, length(util:all_connection_pids(Config))),
+ ?assert(erlang:is_process_alive(C)),
+ ok = emqtt:disconnect(C).
+
+rabbit_mqtt_qos0_queue(Config) ->
+ FeatureFlag = ?FUNCTION_NAME,
+ Msg = Topic = ClientId = atom_to_binary(?FUNCTION_NAME),
+
+ C1 = connect(ClientId, Config),
+ {ok, _, [0]} = emqtt:subscribe(C1, Topic, qos0),
+ ok = emqtt:publish(C1, Topic, Msg, qos0),
+ ok = expect_publishes(C1, Topic, [Msg]),
+ ?assertEqual(1,
+ length(rpc(Config, 0, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]))),
+
+ ?assertEqual(ok,
+ rabbit_ct_broker_helpers:enable_feature_flag(Config, FeatureFlag)),
+
+ %% Queue type does not chanage for existing connection.
+ ?assertEqual(1,
+ length(rpc(Config, 0, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]))),
+ ok = emqtt:publish(C1, Topic, Msg, qos0),
+ ok = expect_publishes(C1, Topic, [Msg]),
+ ?assertMatch(#{messages_delivered_total := 2,
+ messages_delivered_consume_auto_ack_total := 2},
+ get_global_counters(Config, ?PROTO_VER, 0, [{queue_type, rabbit_classic_queue}])),
+
+ %% Reconnecting with the same client ID will terminate the old connection.
+ true = unlink(C1),
+ C2 = connect(ClientId, Config),
+ {ok, _, [0]} = emqtt:subscribe(C2, Topic, qos0),
+ %% This time, we get the new queue type.
+ eventually(
+ ?_assertEqual(0,
+ length(rpc(Config, 0, rabbit_amqqueue, list_by_type, [rabbit_classic_queue])))),
+ ?assertEqual(1,
+ length(rpc(Config, 0, rabbit_amqqueue, list_by_type, [FeatureFlag]))),
+ ok = emqtt:publish(C2, Topic, Msg, qos0),
+ ok = expect_publishes(C2, Topic, [Msg]),
+ ?assertMatch(#{messages_delivered_total := 1,
+ messages_delivered_consume_auto_ack_total := 1},
+ get_global_counters(Config, ?PROTO_VER, 0, [{queue_type, FeatureFlag}])),
+ ok = emqtt:disconnect(C2).
diff --git a/deps/rabbitmq_mqtt/test/java_SUITE.erl b/deps/rabbitmq_mqtt/test/java_SUITE.erl
index e0038a796c22..5155f1ba778d 100644
--- a/deps/rabbitmq_mqtt/test/java_SUITE.erl
+++ b/deps/rabbitmq_mqtt/test/java_SUITE.erl
@@ -6,12 +6,11 @@
%%
-module(java_SUITE).
--compile([export_all]).
+-compile([export_all, nowarn_export_all]).
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
--define(BASE_CONF_RABBIT, {rabbit, [{ssl_options, [{fail_if_no_peer_cert, false}]}]}).
-define(BASE_CONF_MQTT,
{rabbitmq_mqtt, [
{ssl_cert_login, true},
diff --git a/deps/rabbitmq_mqtt/test/java_SUITE_data/mvnw b/deps/rabbitmq_mqtt/test/java_SUITE_data/mvnw
index 8b9da3b8b600..88203bbfeccf 100755
--- a/deps/rabbitmq_mqtt/test/java_SUITE_data/mvnw
+++ b/deps/rabbitmq_mqtt/test/java_SUITE_data/mvnw
@@ -114,7 +114,6 @@ if $mingw ; then
M2_HOME="`(cd "$M2_HOME"; pwd)`"
[ -n "$JAVA_HOME" ] &&
JAVA_HOME="`(cd "$JAVA_HOME"; pwd)`"
- # TODO classpath?
fi
if [ -z "$JAVA_HOME" ]; then
diff --git a/deps/rabbitmq_mqtt/test/java_SUITE_data/pom.xml b/deps/rabbitmq_mqtt/test/java_SUITE_data/pom.xml
index b27b58c172e4..ce790d213095 100644
--- a/deps/rabbitmq_mqtt/test/java_SUITE_data/pom.xml
+++ b/deps/rabbitmq_mqtt/test/java_SUITE_data/pom.xml
@@ -22,13 +22,13 @@
com.rabbitmq
amqp-client
- 5.7.3
+ 5.16.0
test
org.junit.jupiter
junit-jupiter
- 5.5.2
+ 5.9.1
test
diff --git a/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java b/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java
index b5d9848e8b95..42dfd61af17c 100644
--- a/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java
+++ b/deps/rabbitmq_mqtt/test/java_SUITE_data/src/test/java/com/rabbitmq/mqtt/test/MqttTest.java
@@ -740,7 +740,7 @@ public Socket createSocket() {
}
@Test public void publishMultiple() throws MqttException, InterruptedException {
- int pubCount = 50;
+ int pubCount = 1000;
for (int subQos=0; subQos <= 2; subQos++){
for (int pubQos=0; pubQos <= 2; pubQos++){
// avoid reusing the client in this test as a shared
@@ -962,6 +962,26 @@ public void handleDelivery(String consumerTag, Envelope envelope, AMQP.BasicProp
client.disconnect();
}
+ // "A Server MAY allow a Client to supply a ClientId that has a length of zero bytes, however if it does so
+ // the Server MUST treat this as a special case and assign a unique ClientId to that Client." [MQTT-3.1.3-6]
+ // RabbitMQ allows a Client to supply a ClientId that has a length of zero bytes.
+ @Test public void emptyClientId(TestInfo info) throws MqttException, InterruptedException {
+ String emptyClientId = "";
+ MqttConnectOptions client_opts = new TestMqttConnectOptions();
+ MqttClient client = newConnectedClient(emptyClientId, client_opts);
+ MqttClient client2 = newConnectedClient(emptyClientId, client_opts);
+ client.setCallback(this);
+ client2.setCallback(this);
+ client.subscribe("/test-topic/#");
+ client2.subscribe("/test-topic/#");
+
+ publish(client, "/test-topic/1", 0, "my-message".getBytes());
+ waitAtMost(() -> receivedMessagesSize() == 2);
+
+ disconnect(client);
+ disconnect(client2);
+ }
+
private void publish(MqttClient client, String topicName, int qos, byte[] payload) throws MqttException {
publish(client, topicName, qos, payload, false);
}
diff --git a/deps/rabbitmq_mqtt/test/mqtt_machine_SUITE.erl b/deps/rabbitmq_mqtt/test/mqtt_machine_SUITE.erl
index 7ce08cbc2cdf..274877cdc841 100644
--- a/deps/rabbitmq_mqtt/test/mqtt_machine_SUITE.erl
+++ b/deps/rabbitmq_mqtt/test/mqtt_machine_SUITE.erl
@@ -1,9 +1,6 @@
-module(mqtt_machine_SUITE).
--compile(export_all).
-
--export([
- ]).
+-compile([export_all, nowarn_export_all]).
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
diff --git a/deps/rabbitmq_mqtt/test/processor_SUITE.erl b/deps/rabbitmq_mqtt/test/processor_SUITE.erl
index 310c2a8cdb66..15ae0dd5374c 100644
--- a/deps/rabbitmq_mqtt/test/processor_SUITE.erl
+++ b/deps/rabbitmq_mqtt/test/processor_SUITE.erl
@@ -6,7 +6,7 @@
-module(processor_SUITE).
--compile([export_all]).
+-compile([export_all, nowarn_export_all]).
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
@@ -23,9 +23,7 @@ groups() ->
ignores_colons_in_username_if_option_set,
interprets_colons_in_username_if_option_not_set,
get_vhosts_from_global_runtime_parameter,
- get_vhost,
- add_client_id_to_adapter_info,
- quorum_configuration
+ get_vhost
]}
].
@@ -186,17 +184,6 @@ get_vhost(_Config) ->
clear_vhost_global_parameters(),
ok.
-add_client_id_to_adapter_info(_Config) ->
- TestFun = fun(AdapterInfo) ->
- Info0 = rabbit_mqtt_processor:add_client_id_to_adapter_info(<<"my-client-id">>, AdapterInfo),
- AdditionalInfo0 = Info0#amqp_adapter_info.additional_info,
- ?assertEqual(#{<<"client_id">> => <<"my-client-id">>}, proplists:get_value(variable_map, AdditionalInfo0)),
- ClientProperties = proplists:get_value(client_properties, AdditionalInfo0),
- ?assertEqual([{client_id,longstr,<<"my-client-id">>}], ClientProperties)
- end,
- lists:foreach(TestFun, [#amqp_adapter_info{}, #amqp_adapter_info{additional_info = [{client_properties, []}]}]),
- ok.
-
set_global_parameter(Key, Term) ->
InsertParameterFun = fun () ->
mnesia:write(rabbit_runtime_parameters, #runtime_parameters{key = Key, value = Term}, write)
@@ -210,22 +197,3 @@ clear_vhost_global_parameters() ->
ok = mnesia:delete(rabbit_runtime_parameters, mqtt_port_to_vhost_mapping, write)
end,
{atomic, ok} = mnesia:transaction(DeleteParameterFun).
-
-quorum_configuration(_Config) ->
- MyArgs = [],
-%% default setting with CleanSession = true of false
- QMustBeClassic = rabbit_mqtt_processor:maybe_quorum(MyArgs, true, <<"">>),
- ?assertEqual(QMustBeClassic, []),
-%% default setting with CleanSession = true of false
- QMustBeClassicEvenFalse = rabbit_mqtt_processor:maybe_quorum(MyArgs, false, <<"">>),
- ?assertEqual(QMustBeClassicEvenFalse, []),
- application:set_env(rabbitmq_mqtt, durable_queue_type, quorum),
-%% quorum setting with CleanSession == false must me quorum
- QMustBeQuorum = rabbit_mqtt_processor:maybe_quorum(MyArgs, false, <<"">>),
- ?assertEqual(QMustBeQuorum, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
-
- %% quorum setting with CleanSession == true must me classic since
- %% quorum does not support auto-delete
- QEvenQuorumMustBeClassic = rabbit_mqtt_processor:maybe_quorum(MyArgs, true, <<"">>),
- ?assertEqual(QEvenQuorumMustBeClassic, []),
- ok.
diff --git a/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl b/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl
index 06ffb5930e9c..551b14c86588 100644
--- a/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl
+++ b/deps/rabbitmq_mqtt/test/proxy_protocol_SUITE.erl
@@ -5,7 +5,7 @@
%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
%%
-module(proxy_protocol_SUITE).
--compile([export_all]).
+-compile([export_all, nowarn_export_all]).
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
@@ -64,10 +64,10 @@ proxy_protocol(Config) ->
{ok, Socket} = gen_tcp:connect({127,0,0,1}, Port,
[binary, {active, false}, {packet, raw}]),
ok = inet:send(Socket, "PROXY TCP4 192.168.1.1 192.168.1.2 80 81\r\n"),
- ok = inet:send(Socket, mqtt_3_1_1_connect_frame()),
+ ok = inet:send(Socket, mqtt_3_1_1_connect_packet()),
{ok, _Packet} = gen_tcp:recv(Socket, 0, ?TIMEOUT),
- ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0,
- ?MODULE, connection_name, []),
+ timer:sleep(10),
+ ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, connection_name, []),
match = re:run(ConnectionName, <<"^192.168.1.1:80 -> 192.168.1.2:81$">>, [{capture, none}]),
gen_tcp:close(Socket),
ok.
@@ -79,24 +79,23 @@ proxy_protocol_tls(Config) ->
[binary, {active, false}, {packet, raw}]),
ok = inet:send(Socket, "PROXY TCP4 192.168.1.1 192.168.1.2 80 81\r\n"),
{ok, SslSocket} = ssl:connect(Socket, [], ?TIMEOUT),
- ok = ssl:send(SslSocket, mqtt_3_1_1_connect_frame()),
+ ok = ssl:send(SslSocket, mqtt_3_1_1_connect_packet()),
{ok, _Packet} = ssl:recv(SslSocket, 0, ?TIMEOUT),
- ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0,
- ?MODULE, connection_name, []),
+ timer:sleep(10),
+ ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, connection_name, []),
match = re:run(ConnectionName, <<"^192.168.1.1:80 -> 192.168.1.2:81$">>, [{capture, none}]),
gen_tcp:close(Socket),
ok.
connection_name() ->
- Connections = ets:tab2list(connection_created),
- {_Key, Values} = lists:nth(1, Connections),
+ [{_Key, Values}] = ets:tab2list(connection_created),
{_, Name} = lists:keyfind(name, 1, Values),
Name.
merge_app_env(MqttConfig, Config) ->
rabbit_ct_helpers:merge_app_env(Config, MqttConfig).
-mqtt_3_1_1_connect_frame() ->
+mqtt_3_1_1_connect_packet() ->
<<16,
24,
0,
diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl
index a409b7974d5f..4c65e20e4790 100644
--- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl
+++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl
@@ -11,23 +11,35 @@
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
+-import(rabbit_ct_broker_helpers, [rpc/4]).
+-import(rabbit_ct_helpers, [eventually/3]).
+-import(util, [all_connection_pids/1,
+ publish_qos1_timeout/4,
+ expect_publishes/3,
+ connect/2,
+ connect/3,
+ await_exit/1]).
+
all() ->
[
- {group, non_parallel_tests}
+ {group, tests}
].
groups() ->
[
- {non_parallel_tests, [], [
- block,
- block_connack_timeout,
- handle_invalid_frames,
- login_timeout,
- stats,
- quorum_session_false,
- quorum_session_true,
- classic_session_true,
- classic_session_false
+ {tests, [],
+ [
+ block_connack_timeout,
+ handle_invalid_packets,
+ login_timeout,
+ stats,
+ quorum_clean_session_false,
+ quorum_clean_session_true,
+ classic_clean_session_true,
+ classic_clean_session_false,
+ non_clean_sess_empty_client_id,
+ event_authentication_failure,
+ rabbit_mqtt_qos0_queue_overflow
]}
].
@@ -79,50 +91,18 @@ end_per_testcase(Testcase, Config) ->
%% Testsuite cases
%% -------------------------------------------------------------------
-block(Config) ->
- P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
- {ok, C} = emqtt:start_link([{host, "localhost"},
- {port, P},
- {clientid, <<"simpleClient">>},
- {proto_ver, v4}]),
- {ok, _Properties} = emqtt:connect(C),
-
- %% Only here to ensure the connection is really up
- {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>),
- ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>),
- expect_publishes(<<"TopicA">>, [<<"Payload">>]),
- {ok, _, _} = emqtt:unsubscribe(C, <<"TopicA">>),
-
- {ok, _, _} = emqtt:subscribe(C, <<"Topic1">>),
- {ok, _} = publish_qos1(C, <<"Topic1">>, <<"Not blocked yet">>),
-
- ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.00000001]),
- % %% Let it block
- timer:sleep(100),
-
- %% Blocked, but still will publish
- puback_timeout = publish_qos1(C, <<"Topic1">>, <<"Now blocked">>),
- puback_timeout = publish_qos1(C, <<"Topic1">>, <<"Still blocked">>),
-
- %% Unblock
- rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]),
- expect_publishes(<<"Topic1">>, [<<"Not blocked yet">>,
- <<"Now blocked">>,
- <<"Still blocked">>]),
- ok = emqtt:disconnect(C).
-
block_connack_timeout(Config) ->
P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
Ports0 = rpc(Config, erlang, ports, []),
- ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.00000001]),
+ ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0]),
%% Let connection block.
timer:sleep(100),
- %% We can still connect via TCP, but CONNECT frame will not be processed on the server.
+ %% We can still connect via TCP, but CONNECT packet will not be processed on the server.
{ok, Client} = emqtt:start_link([{host, "localhost"},
{port, P},
- {clientid, <<"simpleClient">>},
+ {clientid, atom_to_binary(?FUNCTION_NAME)},
{proto_ver, v4},
{connect_timeout, 1}]),
unlink(Client),
@@ -141,7 +121,7 @@ block_connack_timeout(Config) ->
{connected, MqttReader} = rpc(Config, erlang, port_info, [NewPort, connected]),
MqttReaderMRef = monitor(process, MqttReader),
- %% Unblock connection. CONNECT frame will be processed on the server.
+ %% Unblock connection. CONNECT packet will be processed on the server.
rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]),
receive
@@ -153,18 +133,20 @@ block_connack_timeout(Config) ->
ct:fail("missing peername_not_known from server")
end,
%% Ensure that our client is not registered.
- [] = rpc(Config, rabbit_mqtt_collector, list, []),
+ ?assertEqual([], all_connection_pids(Config)),
ok.
-handle_invalid_frames(Config) ->
+handle_invalid_packets(Config) ->
N = rpc(Config, ets, info, [connection_metrics, size]),
P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
{ok, C} = gen_tcp:connect("localhost", P, []),
Bin = <<"GET / HTTP/1.1\r\nHost: www.rabbitmq.com\r\nUser-Agent: curl/7.43.0\r\nAccept: */*">>,
gen_tcp:send(C, Bin),
gen_tcp:close(C),
+ %% Wait for stats being emitted (every 100ms)
+ timer:sleep(300),
%% No new stats entries should be inserted as connection never got to initialize
- N = rpc(Config, ets, info, [connection_metrics, size]).
+ ?assertEqual(N, rpc(Config, ets, info, [connection_metrics, size])).
login_timeout(Config) ->
rpc(Config, application, set_env, [rabbitmq_mqtt, login_timeout, 400]),
@@ -178,23 +160,12 @@ login_timeout(Config) ->
end.
stats(Config) ->
- P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
- %% CMN = rpc(Config, ets, info, [connection_metrics, size]),
- %% CCMN = rpc(Config, ets, info, [connection_coarse_metrics, size]),
- {ok, C} = emqtt:start_link([{host, "localhost"},
- {port, P},
- {clientid, <<"simpleClient">>},
- {proto_ver, v4}]),
- {ok, _Properties} = emqtt:connect(C),
- %% Ensure that there are some stats
- {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0),
- ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>),
- expect_publishes(<<"TopicA">>, [<<"Payload">>]),
- {ok, _, _} = emqtt:unsubscribe(C, <<"TopicA">>),
- timer:sleep(1000), %% Wait for stats to be emitted, which it does every 100ms
+ C = connect(?FUNCTION_NAME, Config),
+ %% Wait for stats being emitted (every 100ms)
+ timer:sleep(300),
%% Retrieve the connection Pid
- [{_, Reader}] = rpc(Config, rabbit_mqtt_collector, list, []),
- [{_, Pid}] = rpc(Config, rabbit_mqtt_reader, info, [Reader, [connection]]),
+ [Pid] = all_connection_pids(Config),
+ [{pid, Pid}] = rpc(Config, rabbit_mqtt_reader, info, [Pid, [pid]]),
%% Verify the content of the metrics, garbage_collection must be present
[{Pid, Props}] = rpc(Config, ets, lookup, [connection_metrics, Pid]),
true = proplists:is_defined(garbage_collection, Props),
@@ -203,10 +174,10 @@ stats(Config) ->
[connection_coarse_metrics, Pid]),
ok = emqtt:disconnect(C).
-get_durable_queue_type(Server, Q0) ->
- QNameRes = rabbit_misc:r(<<"/">>, queue, Q0),
- {ok, Q1} = rpc:call(Server, rabbit_amqqueue, lookup, [QNameRes]),
- amqqueue:get_type(Q1).
+get_durable_queue_type(Server, QNameBin) ->
+ QName = rabbit_misc:r(<<"/">>, queue, QNameBin),
+ {ok, Q} = rpc:call(Server, rabbit_amqqueue, lookup, [QName]),
+ amqqueue:get_type(Q).
set_env(QueueType) ->
application:set_env(rabbitmq_mqtt, durable_queue_type, QueueType).
@@ -214,80 +185,142 @@ set_env(QueueType) ->
get_env() ->
rabbit_mqtt_util:env(durable_queue_type).
-
-validate_durable_queue_type(Config, ClientName, CleanSession, Expected) ->
- P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
+validate_durable_queue_type(Config, ClientName, CleanSession, ExpectedQueueType) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
- {ok, C} = emqtt:start_link([{host, "localhost"},
- {port, P},
- {clean_start, CleanSession},
- {clientid, ClientName},
- {proto_ver, v4}]),
- {ok, _Properties} = emqtt:connect(C),
+ C = connect(ClientName, Config, [{clean_start, CleanSession}]),
{ok, _, _} = emqtt:subscribe(C, <<"TopicB">>, qos1),
ok = emqtt:publish(C, <<"TopicB">>, <<"Payload">>),
- expect_publishes(<<"TopicB">>, [<<"Payload">>]),
+ ok = expect_publishes(C, <<"TopicB">>, [<<"Payload">>]),
{ok, _, _} = emqtt:unsubscribe(C, <<"TopicB">>),
Prefix = <<"mqtt-subscription-">>,
Suffix = <<"qos1">>,
- Q= <>,
- ?assertEqual(Expected,get_durable_queue_type(Server,Q)),
- timer:sleep(500),
+ QNameBin = <>,
+ ?assertEqual(ExpectedQueueType, get_durable_queue_type(Server, QNameBin)),
ok = emqtt:disconnect(C).
-%% quorum queue test when enable
-quorum_session_false(Config) ->
- %% test if the quorum queue is enable after the setting
+quorum_clean_session_false(Config) ->
Default = rpc(Config, reader_SUITE, get_env, []),
rpc(Config, reader_SUITE, set_env, [quorum]),
- validate_durable_queue_type(Config, <<"qCleanSessionFalse">>, false, rabbit_quorum_queue),
+ validate_durable_queue_type(Config, <<"quorumCleanSessionFalse">>, false, rabbit_quorum_queue),
rpc(Config, reader_SUITE, set_env, [Default]).
-quorum_session_true(Config) ->
- %% in case clean session == true must be classic since quorum
- %% doesn't support auto-delete
+quorum_clean_session_true(Config) ->
Default = rpc(Config, reader_SUITE, get_env, []),
rpc(Config, reader_SUITE, set_env, [quorum]),
- validate_durable_queue_type(Config, <<"qCleanSessionTrue">>, true, rabbit_classic_queue),
+ %% Since we use a clean session and quorum queues cannot be auto-delete or exclusive,
+ %% we expect a classic queue.
+ validate_durable_queue_type(Config, <<"quorumCleanSessionTrue">>, true, rabbit_classic_queue),
rpc(Config, reader_SUITE, set_env, [Default]).
-classic_session_true(Config) ->
- %% with default configuration the queue is classic
- validate_durable_queue_type(Config, <<"cCleanSessionTrue">>, true, rabbit_classic_queue).
-
-classic_session_false(Config) ->
- %% with default configuration the queue is classic
- validate_durable_queue_type(Config, <<"cCleanSessionFalse">>, false, rabbit_classic_queue).
+classic_clean_session_true(Config) ->
+ validate_durable_queue_type(Config, <<"classicCleanSessionTrue">>, true, rabbit_classic_queue).
+
+classic_clean_session_false(Config) ->
+ validate_durable_queue_type(Config, <<"classicCleanSessionFalse">>, false, rabbit_classic_queue).
+
+%% "If the Client supplies a zero-byte ClientId with CleanSession set to 0,
+%% the Server MUST respond to the CONNECT Packet with a CONNACK return code 0x02
+%% (Identifier rejected) and then close the Network Connection" [MQTT-3.1.3-8].
+non_clean_sess_empty_client_id(Config) ->
+ {ok, C} = emqtt:start_link(
+ [{clientid, <<>>},
+ {clean_start, false},
+ {proto_ver, v4},
+ {host, "localhost"},
+ {port, rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt)}
+ ]),
+ process_flag(trap_exit, true),
+ ?assertMatch({error, {client_identifier_not_valid, _}},
+ emqtt:connect(C)),
+ ok = await_exit(C).
+
+event_authentication_failure(Config) ->
+ {ok, C} = emqtt:start_link(
+ [{username, <<"Trudy">>},
+ {password, <<"fake-password">>},
+ {host, "localhost"},
+ {port, rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt)},
+ {clientid, atom_to_binary(?FUNCTION_NAME)},
+ {proto_ver, v4}]),
+ true = unlink(C),
+
+ ok = rabbit_ct_broker_helpers:add_code_path_to_all_nodes(Config, event_recorder),
+ Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
+ ok = gen_event:add_handler({rabbit_event, Server}, event_recorder, []),
+
+ ?assertMatch({error, _}, emqtt:connect(C)),
+
+ [E, _ConnectionClosedEvent] = util:get_events(Server),
+ util:assert_event_type(user_authentication_failure, E),
+ util:assert_event_prop([{name, <<"Trudy">>},
+ {connection_type, network}],
+ E),
+
+ ok = gen_event:delete_handler({rabbit_event, Server}, event_recorder, []).
+
+%% Test that queue type rabbit_mqtt_qos0_queue drops QoS 0 messages when its
+%% max length is reached.
+rabbit_mqtt_qos0_queue_overflow(Config) ->
+ Topic = atom_to_binary(?FUNCTION_NAME),
+ Msg = binary:copy(<<"x">>, 1000),
+ NumMsgs = 10_000,
+
+ %% Provoke TCP back-pressure from client to server by using very small buffers.
+ Opts = [{tcp_opts, [{recbuf, 512},
+ {buffer, 512}]}],
+ Sub = connect(<<"subscriber">>, Config, Opts),
+ {ok, _, [0]} = emqtt:subscribe(Sub, Topic, qos0),
+ [ServerConnectionPid] = all_connection_pids(Config),
+
+ %% Suspend the receiving client such that it stops reading from its socket
+ %% causing TCP back-pressure to the server being applied.
+ true = erlang:suspend_process(Sub),
+
+ %% Let's overflow the receiving server MQTT connection process
+ %% (i.e. the rabbit_mqtt_qos0_queue) by sending many large messages.
+ Pub = connect(<<"publisher">>, Config),
+ lists:foreach(fun(_) ->
+ ok = emqtt:publish(Pub, Topic, Msg, qos0)
+ end, lists:seq(1, NumMsgs)),
+
+ %% Give the server some time to process (either send or drop) the messages.
+ timer:sleep(2000),
+
+ %% Let's resume the receiving client to receive any remaining messages that did
+ %% not get dropped.
+ true = erlang:resume_process(Sub),
+ NumReceived = num_received(Topic, Msg, 0),
+
+ {status, _, _, [_, _, _, _, Misc]} = sys:get_status(ServerConnectionPid),
+ [State] = [S || {data, [{"State", S}]} <- Misc],
+ #{proc_state := #{qos0_messages_dropped := NumDropped}} = State,
+ ct:pal("NumReceived=~b~nNumDropped=~b", [NumReceived, NumDropped]),
+
+ %% We expect that
+ %% 1. all sent messages were either received or dropped
+ ?assertEqual(NumMsgs, NumReceived + NumDropped),
+ case rabbit_ct_helpers:is_mixed_versions(Config) of
+ false ->
+ %% 2. at least one message was dropped (otherwise our whole test case did not
+ %% test what it was supposed to test: that messages are dropped due to the
+ %% server being overflowed with messages while the client receives too slowly)
+ ?assert(NumDropped >= 1);
+ true ->
+ %% Feature flag rabbit_mqtt_qos0_queue is disabled.
+ ?assertEqual(0, NumDropped)
+ end,
+ %% 3. we received at least 1000 messages because everything below the default
+ %% of mailbox_soft_limit=1000 should not be dropped
+ ?assert(NumReceived >= 1000),
+ ok = emqtt:disconnect(Sub),
+ ok = emqtt:disconnect(Pub).
-expect_publishes(_Topic, []) -> ok;
-expect_publishes(Topic, [Payload|Rest]) ->
+num_received(Topic, Payload, N) ->
receive
{publish, #{topic := Topic,
payload := Payload}} ->
- expect_publishes(Topic, Rest)
- after 5000 ->
- throw({publish_not_delivered, Payload})
- end.
-
-rpc(Config, M, F, A) ->
- rabbit_ct_broker_helpers:rpc(Config, 0, M, F, A).
-
-publish_qos1(Client, Topic, Payload) ->
- Mref = erlang:monitor(process, Client),
- ok = emqtt:publish_async(Client, Topic, #{}, Payload, [{qos, 1}], infinity,
- {fun ?MODULE:sync_publish_result/3, [self(), Mref]}),
- receive
- {Mref, Reply} ->
- erlang:demonitor(Mref, [flush]),
- Reply;
- {'DOWN', Mref, process, Client, Reason} ->
- ct:fail("client is down: ~tp", [Reason])
- after
- 1000 ->
- erlang:demonitor(Mref, [flush]),
- puback_timeout
+ num_received(Topic, Payload, N + 1)
+ after 1000 ->
+ N
end.
-
-sync_publish_result(Caller, Mref, Result) ->
- erlang:send(Caller, {Mref, Result}).
diff --git a/deps/rabbitmq_mqtt/test/retainer_SUITE.erl b/deps/rabbitmq_mqtt/test/retainer_SUITE.erl
index 6001f8cfe269..3a2585e8fe5b 100644
--- a/deps/rabbitmq_mqtt/test/retainer_SUITE.erl
+++ b/deps/rabbitmq_mqtt/test/retainer_SUITE.erl
@@ -5,27 +5,36 @@
%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
%%
-module(retainer_SUITE).
--compile([export_all]).
+-compile([export_all, nowarn_export_all]).
-include_lib("common_test/include/ct.hrl").
+-import(util, [expect_publishes/3,
+ connect/3]).
all() ->
[
- {group, non_parallel_tests}
+ {group, dets},
+ {group, ets},
+ {group, noop}
].
groups() ->
[
- {non_parallel_tests, [], [
- coerce_configuration_data,
- should_translate_amqp2mqtt_on_publish,
- should_translate_amqp2mqtt_on_retention,
- should_translate_amqp2mqtt_on_retention_search
- ]}
+ {dets, [], tests()},
+ {ets, [], tests()},
+ {noop, [], [does_not_retain]}
+ ].
+
+tests() ->
+ [
+ coerce_configuration_data,
+ should_translate_amqp2mqtt_on_publish,
+ should_translate_amqp2mqtt_on_retention,
+ should_translate_amqp2mqtt_on_retention_search
].
suite() ->
- [{timetrap, {seconds, 600}}].
+ [{timetrap, {minutes, 2}}].
%% -------------------------------------------------------------------
%% Testsuite setup/teardown.
@@ -33,36 +42,38 @@ suite() ->
init_per_suite(Config) ->
rabbit_ct_helpers:log_environment(),
- Config1 = rabbit_ct_helpers:set_config(Config, [
- {rmq_nodename_suffix, ?MODULE},
- {rmq_extra_tcp_ports, [tcp_port_mqtt_extra,
- tcp_port_mqtt_tls_extra]}
- ]),
- % see https://github.com/rabbitmq/rabbitmq-mqtt/issues/86
- RabbitConfig = {rabbit, [
- {default_user, "guest"},
- {default_pass, "guest"},
- {default_vhost, "/"},
- {default_permissions, [".*", ".*", ".*"]}
- ]},
- rabbit_ct_helpers:run_setup_steps(Config1,
- [ fun(Conf) -> merge_app_env(RabbitConfig, Conf) end ] ++
- rabbit_ct_broker_helpers:setup_steps() ++
- rabbit_ct_client_helpers:setup_steps()).
-
-merge_app_env(MqttConfig, Config) ->
- rabbit_ct_helpers:merge_app_env(Config, MqttConfig).
+ Config.
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.
+init_per_group(Group, Config0) ->
+ Config = rabbit_ct_helpers:set_config(
+ Config0,
+ [
+ {rmq_nodename_suffix, Group},
+ {rmq_extra_tcp_ports, [tcp_port_mqtt_extra,
+ tcp_port_mqtt_tls_extra]}
+ ]),
+ Mod = list_to_atom("rabbit_mqtt_retained_msg_store_" ++ atom_to_list(Group)),
+ Env = [{rabbitmq_mqtt, [{retained_message_store, Mod}]},
+ {rabbit, [
+ {default_user, "guest"},
+ {default_pass, "guest"},
+ {default_vhost, "/"},
+ {default_permissions, [".*", ".*", ".*"]}
+ ]}],
+ rabbit_ct_helpers:run_setup_steps(
+ Config,
+ [fun(Conf) -> rabbit_ct_helpers:merge_app_env(Conf, Env) end] ++
+ rabbit_ct_broker_helpers:setup_steps() ++
+ rabbit_ct_client_helpers:setup_steps()).
+
end_per_group(_, Config) ->
- Config.
+ rabbit_ct_helpers:run_teardown_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).
@@ -76,12 +87,11 @@ end_per_testcase(Testcase, Config) ->
%% -------------------------------------------------------------------
coerce_configuration_data(Config) ->
- P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
- C = connect(P),
+ C = connect(<<"simpleClientRetainer">>, Config, [{ack_timeout, 1}]),
{ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0),
ok = emqtt:publish(C, <<"TopicA">>, <<"Payload">>),
- expect_publishes(<<"TopicA">>, [<<"Payload">>]),
+ ok = expect_publishes(C, <<"TopicA">>, [<<"Payload">>]),
ok = emqtt:disconnect(C).
@@ -91,12 +101,11 @@ coerce_configuration_data(Config) ->
%% sent messages for the translated topic (TopicA/Device/Field)
%% -------------------------------------------------------------------
should_translate_amqp2mqtt_on_publish(Config) ->
- P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
- C = connect(P),
+ C = connect(<<"simpleClientRetainer">>, Config, [{ack_timeout, 1}]),
%% there's an active consumer
{ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device.Field">>, qos1),
ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]),
- expect_publishes(<<"TopicA/Device/Field">>, [<<"Payload">>]),
+ ok = expect_publishes(C, <<"TopicA/Device/Field">>, [<<"Payload">>]),
ok = emqtt:disconnect(C).
%% -------------------------------------------------------------------
@@ -105,12 +114,11 @@ should_translate_amqp2mqtt_on_publish(Config) ->
%% sent the retained message for the translated topic (TopicA/Device/Field)
%% -------------------------------------------------------------------
should_translate_amqp2mqtt_on_retention(Config) ->
- P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
- C = connect(P),
+ C = connect(<<"simpleClientRetainer">>, Config, [{ack_timeout, 1}]),
%% publish with retain = true before a consumer comes around
ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]),
{ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device.Field">>, qos1),
- expect_publishes(<<"TopicA/Device/Field">>, [<<"Payload">>]),
+ ok = expect_publishes(C, <<"TopicA/Device/Field">>, [<<"Payload">>]),
ok = emqtt:disconnect(C).
%% -------------------------------------------------------------------
@@ -119,29 +127,20 @@ should_translate_amqp2mqtt_on_retention(Config) ->
%% sent retained message for the translated topic (TopicA/Device/Field)
%% -------------------------------------------------------------------
should_translate_amqp2mqtt_on_retention_search(Config) ->
- P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
- C = connect(P),
+ C = connect(<<"simpleClientRetainer">>, Config, [{ack_timeout, 1}]),
ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]),
{ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device/Field">>, qos1),
- expect_publishes(<<"TopicA/Device/Field">>, [<<"Payload">>]),
+ ok = expect_publishes(C, <<"TopicA/Device/Field">>, [<<"Payload">>]),
ok = emqtt:disconnect(C).
-connect(Port) ->
- {ok, C} = emqtt:start_link(
- [{host, "localhost"},
- {port, Port},
- {clientid, <<"simpleClientRetainer">>},
- {proto_ver, v4},
- {ack_timeout, 1}]),
- {ok, _Properties} = emqtt:connect(C),
- C.
-
-expect_publishes(_Topic, []) -> ok;
-expect_publishes(Topic, [Payload|Rest]) ->
+does_not_retain(Config) ->
+ C = connect(<<"simpleClientRetainer">>, Config, [{ack_timeout, 1}]),
+ ok = emqtt:publish(C, <<"TopicA/Device.Field">>, #{}, <<"Payload">>, [{retain, true}]),
+ {ok, _, _} = emqtt:subscribe(C, <<"TopicA/Device.Field">>, qos1),
receive
- {publish, #{topic := Topic,
- payload := Payload}} ->
- expect_publishes(Topic, Rest)
- after 1500 ->
- throw({publish_not_delivered, Payload})
- end.
+ Unexpected ->
+ ct:fail("Unexpected message: ~p", [Unexpected])
+ after 1000 ->
+ ok
+ end,
+ ok = emqtt:disconnect(C).
diff --git a/deps/rabbitmq_mqtt/test/shared_SUITE.erl b/deps/rabbitmq_mqtt/test/shared_SUITE.erl
new file mode 100644
index 000000000000..71183d5f9cca
--- /dev/null
+++ b/deps/rabbitmq_mqtt/test/shared_SUITE.erl
@@ -0,0 +1,1394 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
+
+%% Test suite shared between rabbitmq_mqtt and rabbitmq_web_mqtt.
+-module(shared_SUITE).
+-compile([export_all,
+ nowarn_export_all]).
+
+-include_lib("common_test/include/ct.hrl").
+-include_lib("eunit/include/eunit.hrl").
+-include_lib("amqp_client/include/amqp_client.hrl").
+-include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl").
+-include_lib("rabbitmq_ct_helpers/include/rabbit_mgmt_test.hrl").
+
+-import(rabbit_ct_broker_helpers,
+ [rabbitmqctl_list/3,
+ rpc/4,
+ rpc/5,
+ rpc_all/4,
+ get_node_config/3,
+ drain_node/2,
+ revive_node/2
+ ]).
+-import(rabbit_ct_helpers,
+ [eventually/3,
+ eventually/1]).
+-import(util,
+ [all_connection_pids/1,
+ get_global_counters/2, get_global_counters/3, get_global_counters/4,
+ expect_publishes/3,
+ connect/2, connect/3, connect/4,
+ get_events/1, assert_event_type/2, assert_event_prop/2,
+ await_exit/1, await_exit/2,
+ publish_qos1_timeout/4]).
+-import(rabbit_mgmt_test_util,
+ [http_get/2,
+ http_delete/3]).
+
+all() ->
+ [
+ {group, mqtt}
+ ,{group, web_mqtt}
+ ].
+
+groups() ->
+ [
+ {mqtt, [], subgroups()}
+ ,{web_mqtt, [], subgroups()}
+ ].
+
+subgroups() ->
+ [
+ {cluster_size_1, [],
+ [
+ {global_counters, [],
+ [
+ global_counters_v3,
+ global_counters_v4
+ ]},
+ {tests, [],
+ [
+ block_only_publisher
+ ,many_qos1_messages
+ ,subscription_ttl
+ ,management_plugin_connection
+ ,management_plugin_enable
+ ,disconnect
+ ,pubsub_shared_connection
+ ,pubsub_separate_connections
+ ,will_with_disconnect
+ ,will_without_disconnect
+ ,quorum_queue_rejects
+ ,events
+ ,internal_event_handler
+ ,non_clean_sess_reconnect_qos1
+ ,non_clean_sess_reconnect_qos0
+ ,non_clean_sess_reconnect_qos0_and_qos1
+ ,subscribe_same_topic_same_qos
+ ,subscribe_same_topic_different_qos
+ ,subscribe_multiple
+ ,large_message_mqtt_to_mqtt
+ ,large_message_amqp_to_mqtt
+ ,keepalive
+ ,keepalive_turned_off
+ ,duplicate_client_id
+ ,block
+ ,amqp_to_mqtt_qos0
+ ,clean_session_disconnect_client
+ ,clean_session_kill_node
+ ,rabbit_status_connection_count
+ ,trace
+ ]}
+ ]},
+ {cluster_size_3, [],
+ [
+ queue_down_qos1,
+ consuming_classic_mirrored_queue_down,
+ consuming_classic_queue_down,
+ flow_classic_mirrored_queue,
+ flow_quorum_queue,
+ flow_stream,
+ rabbit_mqtt_qos0_queue,
+ cli_list_queues,
+ maintenance,
+ delete_create_queue,
+ publish_to_all_queue_types_qos0,
+ publish_to_all_queue_types_qos1
+ ]}
+ ].
+
+suite() ->
+ [{timetrap, {minutes, 5}}].
+
+%% -------------------------------------------------------------------
+%% 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(mqtt, Config) ->
+ rabbit_ct_helpers:set_config(Config, {websocket, false});
+init_per_group(web_mqtt, Config) ->
+ rabbit_ct_helpers:set_config(Config, {websocket, true});
+
+init_per_group(cluster_size_1, Config) ->
+ rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 1}]);
+init_per_group(cluster_size_3 = Group, Config) ->
+ init_per_group0(Group,
+ rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 3}]));
+init_per_group(Group, Config)
+ when Group =:= global_counters orelse
+ Group =:= tests ->
+ init_per_group0(Group, Config).
+
+init_per_group0(Group, Config0) ->
+ Suffix = lists:flatten(io_lib:format("~s_websocket_~w", [Group, ?config(websocket, Config0)])),
+ Config1 = rabbit_ct_helpers:set_config(
+ Config0,
+ [{rmq_nodename_suffix, Suffix},
+ {rmq_extra_tcp_ports, [tcp_port_mqtt_extra,
+ tcp_port_mqtt_tls_extra]}]),
+ Config = rabbit_ct_helpers:merge_app_env(
+ Config1,
+ {rabbit, [{classic_queue_default_version, 2}]}),
+ rabbit_ct_helpers:run_steps(
+ Config,
+ rabbit_ct_broker_helpers:setup_steps() ++
+ rabbit_ct_client_helpers:setup_steps()).
+
+end_per_group(G, Config)
+ when G =:= mqtt;
+ G =:= web_mqtt;
+ G =:= cluster_size_1 ->
+ Config;
+end_per_group(_, Config) ->
+ rabbit_ct_helpers:run_teardown_steps(
+ Config,
+ rabbit_ct_client_helpers:teardown_steps() ++
+ rabbit_ct_broker_helpers:teardown_steps()).
+
+init_per_testcase(Testcase = maintenance, Config) ->
+ case rabbit_ct_helpers:is_mixed_versions() of
+ true ->
+ {skip, "maintenance mode wrongly closes cluster-wide MQTT connections "
+ "in RMQ < 3.11.2 and < 3.10.10"};
+ false ->
+ init_per_testcase0(Testcase, Config)
+ end;
+init_per_testcase(T, Config)
+ when T =:= management_plugin_connection;
+ T =:= management_plugin_enable ->
+ ok = inets:start(),
+ init_per_testcase0(T, Config);
+init_per_testcase(Testcase, Config) ->
+ init_per_testcase0(Testcase, Config).
+
+init_per_testcase0(Testcase, Config) ->
+ Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
+ [ok = rabbit_ct_broker_helpers:enable_plugin(Config, N, rabbitmq_web_mqtt) || N <- Nodes],
+ rabbit_ct_helpers:testcase_started(Config, Testcase).
+
+end_per_testcase(T, Config)
+ when T =:= management_plugin_connection;
+ T =:= management_plugin_enable ->
+ ok = inets:stop(),
+ end_per_testcase0(T, Config);
+end_per_testcase(Testcase, Config) ->
+ end_per_testcase0(Testcase, Config).
+
+end_per_testcase0(Testcase, Config) ->
+ rabbit_ct_client_helpers:close_channels_and_connection(Config, 0),
+ rabbit_ct_helpers:testcase_finished(Config, Testcase).
+
+%% -------------------------------------------------------------------
+%% Testsuite cases
+%% -------------------------------------------------------------------
+
+disconnect(Config) ->
+ C = connect(?FUNCTION_NAME, Config),
+ eventually(?_assertEqual(1, length(all_connection_pids(Config)))),
+ process_flag(trap_exit, true),
+ ok = emqtt:disconnect(C),
+ await_exit(C, normal),
+ eventually(?_assertEqual([], all_connection_pids(Config))),
+ ok.
+
+pubsub_shared_connection(Config) ->
+ C = connect(?FUNCTION_NAME, Config),
+
+ Topic = <<"/topic/test-mqtt">>,
+ {ok, _, [1]} = emqtt:subscribe(C, Topic, qos1),
+
+ Payload = <<"a\x00a">>,
+ ?assertMatch({ok, #{packet_id := _,
+ reason_code := 0,
+ reason_code_name := success
+ }},
+ emqtt:publish(C, Topic, Payload, [{qos, 1}])),
+ ok = expect_publishes(C, Topic, [Payload]),
+ ok = emqtt:disconnect(C).
+
+pubsub_separate_connections(Config) ->
+ Pub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config),
+ Sub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_subscriber">>, Config),
+
+ Topic = <<"/topic/test-mqtt">>,
+ {ok, _, [1]} = emqtt:subscribe(Sub, Topic, qos1),
+
+ Payload = <<"a\x00a">>,
+ ?assertMatch({ok, #{packet_id := _,
+ reason_code := 0,
+ reason_code_name := success
+ }},
+ emqtt:publish(Pub, Topic, Payload, [{qos, 1}])),
+ ok = expect_publishes(Sub, Topic, [Payload]),
+ ok = emqtt:disconnect(Pub),
+ ok = emqtt:disconnect(Sub).
+
+will_with_disconnect(Config) ->
+ LastWillTopic = <<"/topic/last-will">>,
+ LastWillMsg = <<"last will message">>,
+ PubOpts = [{will_topic, LastWillTopic},
+ {will_payload, LastWillMsg},
+ {will_qos, 1}],
+ Pub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config, PubOpts),
+ Sub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_subscriber">>, Config),
+ {ok, _, [1]} = emqtt:subscribe(Sub, LastWillTopic, qos1),
+
+ %% Client sends DISCONNECT packet. Therefore, will message should not be sent.
+ ok = emqtt:disconnect(Pub),
+ ?assertEqual({publish_not_received, LastWillMsg},
+ expect_publishes(Sub, LastWillTopic, [LastWillMsg])),
+
+ ok = emqtt:disconnect(Sub).
+
+will_without_disconnect(Config) ->
+ LastWillTopic = <<"/topic/last-will">>,
+ LastWillMsg = <<"last will message">>,
+ PubOpts = [{will_topic, LastWillTopic},
+ {will_payload, LastWillMsg},
+ {will_qos, 1}],
+ Pub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config, PubOpts),
+ timer:sleep(100),
+ [ServerPublisherPid] = all_connection_pids(Config),
+ Sub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_subscriber">>, Config),
+ {ok, _, [1]} = emqtt:subscribe(Sub, LastWillTopic, qos1),
+
+ %% Client does not send DISCONNECT packet. Therefore, will message should be sent.
+ unlink(Pub),
+ erlang:exit(ServerPublisherPid, test_will),
+ ?assertEqual(ok, expect_publishes(Sub, LastWillTopic, [LastWillMsg])),
+
+ ok = emqtt:disconnect(Sub).
+
+quorum_queue_rejects(Config) ->
+ {_Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config),
+ Name = atom_to_binary(?FUNCTION_NAME),
+
+ ok = rabbit_ct_broker_helpers:set_policy(
+ Config, 0, <<"qq-policy">>, Name, <<"queues">>, [{<<"max-length">>, 1},
+ {<<"overflow">>, <<"reject-publish">>}]),
+ declare_queue(Ch, Name, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
+ bind(Ch, Name, Name),
+
+ C = connect(Name, Config, [{retry_interval, 1}]),
+ {ok, _} = emqtt:publish(C, Name, <<"m1">>, qos1),
+ {ok, _} = emqtt:publish(C, Name, <<"m2">>, qos1),
+ %% We expect m3 to be rejected and dropped.
+ ?assertEqual(puback_timeout, util:publish_qos1_timeout(C, Name, <<"m3">>, 700)),
+
+ ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"m1">>}},
+ amqp_channel:call(Ch, #'basic.get'{queue = Name, no_ack = true})),
+ ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"m2">>}},
+ amqp_channel:call(Ch, #'basic.get'{queue = Name, no_ack = true})),
+ %% m3 is re-sent by emqtt.
+ ?awaitMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"m3">>}},
+ amqp_channel:call(Ch, #'basic.get'{queue = Name, no_ack = true}),
+ 2000, 200),
+
+ ok = emqtt:disconnect(C),
+ delete_queue(Ch, Name),
+ ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"qq-policy">>).
+
+publish_to_all_queue_types_qos0(Config) ->
+ publish_to_all_queue_types(Config, qos0).
+
+publish_to_all_queue_types_qos1(Config) ->
+ publish_to_all_queue_types(Config, qos1).
+
+publish_to_all_queue_types(Config, QoS) ->
+ Ch = rabbit_ct_client_helpers:open_channel(Config),
+
+ CQ = <<"classic-queue">>,
+ CMQ = <<"classic-mirrored-queue">>,
+ QQ = <<"quorum-queue">>,
+ SQ = <<"stream-queue">>,
+ Topic = <<"mytopic">>,
+
+ declare_queue(Ch, CQ, []),
+ bind(Ch, CQ, Topic),
+
+ ok = rabbit_ct_broker_helpers:set_ha_policy(Config, 0, CMQ, <<"all">>),
+ declare_queue(Ch, CMQ, []),
+ bind(Ch, CMQ, Topic),
+
+ declare_queue(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
+ bind(Ch, QQ, Topic),
+
+ declare_queue(Ch, SQ, [{<<"x-queue-type">>, longstr, <<"stream">>}]),
+ bind(Ch, SQ, Topic),
+
+ NumMsgs = 2000,
+ C = connect(?FUNCTION_NAME, Config, [{retry_interval, 2}]),
+ lists:foreach(fun(N) ->
+ case emqtt:publish(C, Topic, integer_to_binary(N), QoS) of
+ ok ->
+ ok;
+ {ok, _} ->
+ ok;
+ Other ->
+ ct:fail("Failed to publish: ~p", [Other])
+ end
+ end, lists:seq(1, NumMsgs)),
+
+ eventually(?_assert(
+ begin
+ L = rabbitmqctl_list(Config, 0, ["list_queues", "messages", "--no-table-headers"]),
+ length(L) =:= 4 andalso
+ lists:all(fun([Bin]) ->
+ N = binary_to_integer(Bin),
+ case QoS of
+ qos0 ->
+ N =:= NumMsgs;
+ qos1 ->
+ %% Allow for some duplicates when client resends
+ %% a message that gets acked at roughly the same time.
+ N >= NumMsgs andalso
+ N < NumMsgs * 2
+ end
+ end, L)
+ end), 2000, 10),
+
+ delete_queue(Ch, [CQ, CMQ, QQ, SQ]),
+ ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, CMQ),
+ ok = emqtt:disconnect(C),
+ ?awaitMatch([],
+ all_connection_pids(Config), 10_000, 1000).
+
+flow_classic_mirrored_queue(Config) ->
+ QueueName = <<"flow">>,
+ ok = rabbit_ct_broker_helpers:set_ha_policy(Config, 0, QueueName, <<"all">>),
+ flow(Config, {rabbit, credit_flow_default_credit, {2, 1}}, <<"classic">>),
+ ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, QueueName).
+
+flow_quorum_queue(Config) ->
+ flow(Config, {rabbit, quorum_commands_soft_limit, 1}, <<"quorum">>).
+
+flow_stream(Config) ->
+ flow(Config, {rabbit, stream_messages_soft_limit, 1}, <<"stream">>).
+
+flow(Config, {App, Par, Val}, QueueType)
+ when is_binary(QueueType) ->
+ {ok, DefaultVal} = rpc(Config, application, get_env, [App, Par]),
+ Result = rpc_all(Config, application, set_env, [App, Par, Val]),
+ ?assert(lists:all(fun(R) -> R =:= ok end, Result)),
+
+ Ch = rabbit_ct_client_helpers:open_channel(Config),
+ QueueName = Topic = atom_to_binary(?FUNCTION_NAME),
+ declare_queue(Ch, QueueName, [{<<"x-queue-type">>, longstr, QueueType}]),
+ bind(Ch, QueueName, Topic),
+
+ NumMsgs = 1000,
+ C = connect(?FUNCTION_NAME, Config, [{retry_interval, 600},
+ {max_inflight, NumMsgs}]),
+ TestPid = self(),
+ lists:foreach(
+ fun(N) ->
+ %% Publish async all messages at once to trigger flow control
+ ok = emqtt:publish_async(C, Topic, integer_to_binary(N), qos1,
+ {fun(N0, {ok, #{reason_code_name := success}}) ->
+ TestPid ! {self(), N0}
+ end, [N]})
+ end, lists:seq(1, NumMsgs)),
+ ok = await_confirms_ordered(C, 1, NumMsgs),
+ eventually(?_assertEqual(
+ [[integer_to_binary(NumMsgs)]],
+ rabbitmqctl_list(Config, 0, ["list_queues", "messages", "--no-table-headers"])
+ ), 1000, 10),
+
+ delete_queue(Ch, QueueName),
+ ok = emqtt:disconnect(C),
+ ?awaitMatch([],
+ all_connection_pids(Config), 10_000, 1000),
+ Result = rpc_all(Config, application, set_env, [App, Par, DefaultVal]),
+ ok.
+
+events(Config) ->
+ ok = rabbit_ct_broker_helpers:add_code_path_to_all_nodes(Config, event_recorder),
+ Server = get_node_config(Config, 0, nodename),
+ ok = gen_event:add_handler({rabbit_event, Server}, event_recorder, []),
+
+ ClientId = atom_to_binary(?FUNCTION_NAME),
+ C = connect(ClientId, Config),
+
+ [E0, E1] = get_events(Server),
+ assert_event_type(user_authentication_success, E0),
+ assert_event_prop([{name, <<"guest">>},
+ {connection_type, network}],
+ E0),
+ assert_event_type(connection_created, E1),
+ [ConnectionPid] = all_connection_pids(Config),
+ Proto = case ?config(websocket, Config) of
+ true -> 'Web MQTT';
+ false -> 'MQTT'
+ end,
+ ExpectedConnectionProps = [{protocol, {Proto, {3,1,1}}},
+ {node, Server},
+ {vhost, <<"/">>},
+ {user, <<"guest">>},
+ {pid, ConnectionPid}],
+ assert_event_prop(ExpectedConnectionProps, E1),
+
+ {ok, _, _} = emqtt:subscribe(C, <<"TopicA">>, qos0),
+
+ QueueNameBin = <<"mqtt-subscription-", ClientId/binary, "qos0">>,
+ QueueName = {resource, <<"/">>, queue, QueueNameBin},
+ [E2, E3 | E4] = get_events(Server),
+ QueueType = case rabbit_ct_helpers:is_mixed_versions(Config) of
+ false ->
+ ?assertEqual([], E4),
+ rabbit_mqtt_qos0_queue;
+ true ->
+ %% Feature flag rabbit_mqtt_qos0_queue is disabled.
+ [ConsumerCreated] = E4,
+ assert_event_type(consumer_created, ConsumerCreated),
+ assert_event_prop([{queue, QueueName},
+ {ack_required, false},
+ {exclusive, false},
+ {arguments, []}],
+ ConsumerCreated),
+ classic
+ end,
+ assert_event_type(queue_created, E2),
+ assert_event_prop([{name, QueueName},
+ {durable, true},
+ {auto_delete, false},
+ {exclusive, true},
+ {type, QueueType},
+ {arguments, []}],
+ E2),
+ assert_event_type(binding_created, E3),
+ assert_event_prop([{source_name, <<"amq.topic">>},
+ {source_kind, exchange},
+ {destination_name, QueueNameBin},
+ {destination_kind, queue},
+ {routing_key, <<"TopicA">>},
+ {arguments, []}],
+ E3),
+
+ {ok, _, _} = emqtt:unsubscribe(C, <<"TopicA">>),
+
+ [E5] = get_events(Server),
+ assert_event_type(binding_deleted, E5),
+
+ ok = emqtt:disconnect(C),
+
+ [E6, E7 | E8] = get_events(Server),
+ assert_event_type(connection_closed, E6),
+ assert_event_prop(ExpectedConnectionProps, E6),
+ case rabbit_ct_helpers:is_mixed_versions(Config) of
+ false ->
+ assert_event_type(queue_deleted, E7),
+ assert_event_prop({name, QueueName}, E7);
+ true ->
+ assert_event_type(consumer_deleted, E7),
+ assert_event_prop({queue, QueueName}, E7),
+ [QueueDeleted] = E8,
+ assert_event_type(queue_deleted, QueueDeleted),
+ assert_event_prop({name, QueueName}, QueueDeleted)
+ end,
+
+ ok = gen_event:delete_handler({rabbit_event, Server}, event_recorder, []).
+
+internal_event_handler(Config) ->
+ Server = get_node_config(Config, 0, nodename),
+ ok = gen_event:call({rabbit_event, Server}, rabbit_mqtt_internal_event_handler, ignored_request, 1000).
+
+global_counters_v3(Config) ->
+ global_counters(Config, v3).
+
+global_counters_v4(Config) ->
+ global_counters(Config, v4).
+
+global_counters(Config, ProtoVer) ->
+ C = connect(?FUNCTION_NAME, Config, [{proto_ver, ProtoVer}]),
+
+ Topic0 = <<"test-topic0">>,
+ Topic1 = <<"test-topic1">>,
+ Topic2 = <<"test-topic2">>,
+ {ok, _, [0]} = emqtt:subscribe(C, Topic0, qos0),
+ {ok, _, [1]} = emqtt:subscribe(C, Topic1, qos1),
+ {ok, _, [1]} = emqtt:subscribe(C, Topic2, qos1),
+
+ ok = emqtt:publish(C, Topic0, <<"testm0">>, qos0),
+ ok = emqtt:publish(C, Topic1, <<"testm1">>, qos0),
+ {ok, _} = emqtt:publish(C, Topic2, <<"testm2">>, qos1),
+ ok = emqtt:publish(C, <<"no/queue/bound">>, <<"msg-dropped">>, qos0),
+ {ok, _} = emqtt:publish(C, <<"no/queue/bound">>, <<"msg-returned">>, qos1),
+
+ ok = expect_publishes(C, Topic0, [<<"testm0">>]),
+ ok = expect_publishes(C, Topic1, [<<"testm1">>]),
+ ok = expect_publishes(C, Topic2, [<<"testm2">>]),
+
+ ?assertEqual(#{publishers => 1,
+ consumers => 1,
+ messages_confirmed_total => 2,
+ messages_received_confirm_total => 2,
+ messages_received_total => 5,
+ messages_routed_total => 3,
+ messages_unroutable_dropped_total => 1,
+ messages_unroutable_returned_total => 1},
+ get_global_counters(Config, ProtoVer)),
+
+ case rabbit_ct_helpers:is_mixed_versions(Config) of
+ false ->
+ ?assertEqual(#{messages_delivered_total => 2,
+ messages_acknowledged_total => 1,
+ messages_delivered_consume_auto_ack_total => 1,
+ messages_delivered_consume_manual_ack_total => 1,
+ messages_delivered_get_auto_ack_total => 0,
+ messages_delivered_get_manual_ack_total => 0,
+ messages_get_empty_total => 0,
+ messages_redelivered_total => 0},
+ get_global_counters(Config, ProtoVer, 0, [{queue_type, rabbit_classic_queue}])),
+ ?assertEqual(#{messages_delivered_total => 1,
+ messages_acknowledged_total => 0,
+ messages_delivered_consume_auto_ack_total => 1,
+ messages_delivered_consume_manual_ack_total => 0,
+ messages_delivered_get_auto_ack_total => 0,
+ messages_delivered_get_manual_ack_total => 0,
+ messages_get_empty_total => 0,
+ messages_redelivered_total => 0},
+ get_global_counters(Config, ProtoVer, 0, [{queue_type, rabbit_mqtt_qos0_queue}]));
+ true ->
+ %% Feature flag rabbit_mqtt_qos0_queue is disabled.
+ ?assertEqual(#{messages_delivered_total => 3,
+ messages_acknowledged_total => 1,
+ messages_delivered_consume_auto_ack_total => 2,
+ messages_delivered_consume_manual_ack_total => 1,
+ messages_delivered_get_auto_ack_total => 0,
+ messages_delivered_get_manual_ack_total => 0,
+ messages_get_empty_total => 0,
+ messages_redelivered_total => 0},
+ get_global_counters(Config, ProtoVer, 0, [{queue_type, rabbit_classic_queue}]))
+ end,
+
+ {ok, _, _} = emqtt:unsubscribe(C, Topic1),
+ ?assertEqual(1, maps:get(consumers, get_global_counters(Config, ProtoVer))),
+
+ ok = emqtt:disconnect(C),
+ ?assertEqual(#{publishers => 0,
+ consumers => 0,
+ messages_confirmed_total => 2,
+ messages_received_confirm_total => 2,
+ messages_received_total => 5,
+ messages_routed_total => 3,
+ messages_unroutable_dropped_total => 1,
+ messages_unroutable_returned_total => 1},
+ get_global_counters(Config, ProtoVer)).
+
+queue_down_qos1(Config) ->
+ {Conn1, Ch1} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 1),
+ CQ = Topic = atom_to_binary(?FUNCTION_NAME),
+ declare_queue(Ch1, CQ, []),
+ bind(Ch1, CQ, Topic),
+ ok = rabbit_ct_client_helpers:close_connection_and_channel(Conn1, Ch1),
+ ok = rabbit_ct_broker_helpers:stop_node(Config, 1),
+
+ C = connect(?FUNCTION_NAME, Config, [{retry_interval, 2}]),
+
+ %% classic queue is down, therefore message is rejected
+ ?assertEqual(puback_timeout, util:publish_qos1_timeout(C, Topic, <<"msg">>, 500)),
+
+ ok = rabbit_ct_broker_helpers:start_node(Config, 1),
+ %% classic queue is up, therefore message should arrive
+ eventually(?_assertEqual([[<<"1">>]],
+ rabbitmqctl_list(Config, 1, ["list_queues", "messages", "--no-table-headers"])),
+ 500, 20),
+
+ Ch0 = rabbit_ct_client_helpers:open_channel(Config, 0),
+ delete_queue(Ch0, CQ),
+ ok = emqtt:disconnect(C).
+
+%% Even though classic mirrored queues are deprecated, we know that some users have set up
+%% a policy to mirror MQTT queues. So, we need to support that use case in RabbitMQ 3.x
+%% and failover consumption when the classic mirrored queue leader fails.
+consuming_classic_mirrored_queue_down(Config) ->
+ [Server1, Server2, _Server3] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
+ ClientId = Topic = PolicyName = atom_to_binary(?FUNCTION_NAME),
+
+ ok = rabbit_ct_broker_helpers:set_policy(
+ Config, Server1, PolicyName, <<".*">>, <<"queues">>,
+ [{<<"ha-mode">>, <<"all">>},
+ {<<"queue-master-locator">>, <<"client-local">>}]),
+
+ %% Declare queue leader on Server1.
+ C1 = connect(ClientId, Config, Server1, [{clean_start, false}]),
+ {ok, _, _} = emqtt:subscribe(C1, Topic, qos1),
+ ok = emqtt:disconnect(C1),
+
+ %% Consume from Server2.
+ C2 = connect(ClientId, Config, Server2, [{clean_start, false}]),
+
+ %% Sanity check that consumption works.
+ {ok, _} = emqtt:publish(C2, Topic, <<"m1">>, qos1),
+ ok = expect_publishes(C2, Topic, [<<"m1">>]),
+
+ %% Let's stop the queue leader node.
+ ok = rabbit_ct_broker_helpers:stop_node(Config, Server1),
+
+ %% Consumption should continue to work.
+ {ok, _} = emqtt:publish(C2, Topic, <<"m2">>, qos1),
+ ok = expect_publishes(C2, Topic, [<<"m2">>]),
+
+ %% Cleanup
+ ok = emqtt:disconnect(C2),
+ ok = rabbit_ct_broker_helpers:start_node(Config, Server1),
+ ?assertMatch([_Q],
+ rpc(Config, Server1, rabbit_amqqueue, list, [])),
+ C3 = connect(ClientId, Config, Server2, [{clean_start, true}]),
+ ok = emqtt:disconnect(C3),
+ ?assertEqual([],
+ rpc(Config, Server1, rabbit_amqqueue, list, [])),
+ ok = rabbit_ct_broker_helpers:clear_policy(Config, Server1, PolicyName).
+
+%% Consuming classic queue on a different node goes down.
+consuming_classic_queue_down(Config) ->
+ [Server1, _Server2, Server3] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
+ ClientId = Topic = atom_to_binary(?FUNCTION_NAME),
+
+ %% Declare classic queue on Server1.
+ C1 = connect(ClientId, Config, [{clean_start, false}]),
+ {ok, _, _} = emqtt:subscribe(C1, Topic, qos1),
+ ok = emqtt:disconnect(C1),
+
+ %% Consume from Server3.
+ C2 = connect(ClientId, Config, Server3, [{clean_start, false}]),
+
+ ProtoVer = v4,
+ ?assertMatch(#{consumers := 1},
+ get_global_counters(Config, ProtoVer, Server3)),
+
+ %% Let's stop the queue leader node.
+ process_flag(trap_exit, true),
+ ok = rabbit_ct_broker_helpers:stop_node(Config, Server1),
+
+ %% When the dedicated MQTT connection (non-mirrored classic) queue goes down, it is reasonable
+ %% that the server closes the MQTT connection because the MQTT client cannot consume anymore.
+ eventually(?_assertMatch(#{consumers := 0},
+ get_global_counters(Config, ProtoVer, Server3)),
+ 1000, 5),
+ await_exit(C2),
+
+ %% Cleanup
+ ok = rabbit_ct_broker_helpers:start_node(Config, Server1),
+ C3 = connect(ClientId, Config, Server3, [{clean_start, true}]),
+ ok = emqtt:disconnect(C3),
+ ?assertEqual([],
+ rpc(Config, Server1, rabbit_amqqueue, list, [])),
+ ok.
+
+delete_create_queue(Config) ->
+ Ch = rabbit_ct_client_helpers:open_channel(Config),
+ CQ1 = <<"classic-queue-1-delete-create">>,
+ CQ2 = <<"classic-queue-2-delete-create">>,
+ QQ = <<"quorum-queue-delete-create">>,
+ Topic = atom_to_binary(?FUNCTION_NAME),
+
+ DeclareQueues = fun() ->
+ declare_queue(Ch, CQ1, []),
+ bind(Ch, CQ1, Topic),
+ declare_queue(Ch, CQ2, []),
+ bind(Ch, CQ2, Topic),
+ declare_queue(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
+ bind(Ch, QQ, Topic)
+ end,
+ DeclareQueues(),
+
+ %% some large retry_interval to avoid re-sending
+ C = connect(?FUNCTION_NAME, Config, [{retry_interval, 300}]),
+ NumMsgs = 50,
+ TestPid = self(),
+ spawn(
+ fun() ->
+ lists:foreach(
+ fun(N) ->
+ ok = emqtt:publish_async(C, Topic, integer_to_binary(N), qos1,
+ {fun(N0, {ok, #{reason_code_name := success}}) ->
+ TestPid ! {self(), N0}
+ end, [N]})
+ end, lists:seq(1, NumMsgs))
+ end),
+
+ %% Delete queues while sending to them.
+ %% We want to test the path where a queue is deleted while confirms are outstanding.
+ timer:sleep(2),
+ delete_queue(Ch, [CQ1, QQ]),
+ %% Give queues some time to be fully deleted
+ timer:sleep(2000),
+
+ %% We expect confirms for all messages.
+ %% Confirm here does not mean that messages made it ever to the deleted queues.
+ %% It is valid for confirms to sporadically arrive out of order: This happens when the classic
+ %% queue is being deleted while the remaining messages are routed and confirmed to the 2nd and 3rd queues
+ %% before the monitor to the classic queue fires.
+ ok = await_confirms_unordered(C, NumMsgs),
+
+ %% Recreate the same queues.
+ DeclareQueues(),
+
+ %% Sending a message to each of them should work.
+ {ok, _} = emqtt:publish(C, Topic, <<"m">>, qos1),
+ eventually(?_assertEqual(lists:sort([[CQ1, <<"1">>],
+ %% This queue should have all messages because we did not delete it.
+ [CQ2, integer_to_binary(NumMsgs + 1)],
+ [QQ, <<"1">>]]),
+ lists:sort(rabbitmqctl_list(Config, 0, ["list_queues", "name", "messages", "--no-table-headers"]))),
+ 1000, 10),
+
+ delete_queue(Ch, [CQ1, CQ2, QQ]),
+ ok = emqtt:disconnect(C).
+
+subscription_ttl(Config) ->
+ TTL = 1000,
+ App = rabbitmq_mqtt,
+ Par = ClientId = ?FUNCTION_NAME,
+ {ok, DefaultVal} = rpc(Config, application, get_env, [App, Par]),
+ ok = rpc(Config, application, set_env, [App, Par, TTL]),
+
+ C = connect(ClientId, Config, [{clean_start, false}]),
+ {ok, _, [0, 1]} = emqtt:subscribe(C, [{<<"topic0">>, qos0},
+ {<<"topic1">>, qos1}]),
+ ok = emqtt:disconnect(C),
+
+ ?assertEqual(2, rpc(Config, rabbit_amqqueue, count, [])),
+ timer:sleep(TTL + 100),
+ ?assertEqual(0, rpc(Config, rabbit_amqqueue, count, [])),
+
+ ok = rpc(Config, application, set_env, [App, Par, DefaultVal]).
+
+non_clean_sess_reconnect_qos1(Config) ->
+ non_clean_sess_reconnect(Config, qos1).
+
+non_clean_sess_reconnect_qos0(Config) ->
+ non_clean_sess_reconnect(Config, qos0).
+
+non_clean_sess_reconnect(Config, SubscriptionQoS) ->
+ Pub = connect(<<"publisher">>, Config),
+ Topic = ClientId = atom_to_binary(?FUNCTION_NAME),
+ ProtoVer = v4,
+
+ C1 = connect(ClientId, Config, [{clean_start, false}]),
+ {ok, _, _} = emqtt:subscribe(C1, Topic, SubscriptionQoS),
+ ?assertMatch(#{consumers := 1},
+ get_global_counters(Config, ProtoVer)),
+
+ ok = emqtt:disconnect(C1),
+ ?assertMatch(#{consumers := 0},
+ get_global_counters(Config, ProtoVer)),
+
+ timer:sleep(20),
+ ok = emqtt:publish(Pub, Topic, <<"msg-3-qos0">>, qos0),
+ {ok, _} = emqtt:publish(Pub, Topic, <<"msg-4-qos1">>, qos1),
+
+ C2 = connect(ClientId, Config, [{clean_start, false}]),
+ ?assertMatch(#{consumers := 1},
+ get_global_counters(Config, ProtoVer)),
+
+ ok = emqtt:publish(Pub, Topic, <<"msg-5-qos0">>, qos0),
+ {ok, _} = emqtt:publish(Pub, Topic, <<"msg-6-qos1">>, qos1),
+
+ %% shouldn't receive message after unsubscribe
+ {ok, _, _} = emqtt:unsubscribe(C2, Topic),
+ ?assertMatch(#{consumers := 0},
+ get_global_counters(Config, ProtoVer)),
+ {ok, _} = emqtt:publish(Pub, Topic, <<"msg-7-qos0">>, qos1),
+
+ %% "After the disconnection of a Session that had CleanSession set to 0, the Server MUST store
+ %% further QoS 1 and QoS 2 messages that match any subscriptions that the client had at the
+ %% time of disconnection as part of the Session state [MQTT-3.1.2-5].
+ %% It MAY also store QoS 0 messages that meet the same criteria."
+ %% Starting with RabbitMQ v3.12 we store QoS 0 messages as well.
+ ok = expect_publishes(C2, Topic, [<<"msg-3-qos0">>, <<"msg-4-qos1">>,
+ <<"msg-5-qos0">>, <<"msg-6-qos1">>]),
+ {publish_not_received, <<"msg-7-qos0">>} = expect_publishes(C2, Topic, [<<"msg-7-qos0">>]),
+
+ ok = emqtt:disconnect(Pub),
+ ok = emqtt:disconnect(C2),
+ %% connect with clean sess true to clean up
+ C3 = connect(ClientId, Config, [{clean_start, true}]),
+ ok = emqtt:disconnect(C3).
+
+non_clean_sess_reconnect_qos0_and_qos1(Config) ->
+ Pub = connect(<<"publisher">>, Config),
+ ProtoVer = v4,
+ Topic0 = <<"t/0">>,
+ Topic1 = <<"t/1">>,
+ ClientId = ?FUNCTION_NAME,
+
+ C1 = connect(ClientId, Config, [{clean_start, false}]),
+ {ok, _, [1, 0]} = emqtt:subscribe(C1, [{Topic1, qos1}, {Topic0, qos0}]),
+ ?assertMatch(#{consumers := 1},
+ get_global_counters(Config, ProtoVer)),
+
+ ok = emqtt:disconnect(C1),
+ ?assertMatch(#{consumers := 0},
+ get_global_counters(Config, ProtoVer)),
+
+ {ok, _} = emqtt:publish(Pub, Topic0, <<"msg-0">>, qos1),
+ {ok, _} = emqtt:publish(Pub, Topic1, <<"msg-1">>, qos1),
+
+ C2 = connect(ClientId, Config, [{clean_start, false}]),
+ ?assertMatch(#{consumers := 1},
+ get_global_counters(Config, ProtoVer)),
+
+ ok = expect_publishes(C2, Topic0, [<<"msg-0">>]),
+ ok = expect_publishes(C2, Topic1, [<<"msg-1">>]),
+
+ ok = emqtt:disconnect(Pub),
+ ok = emqtt:disconnect(C2),
+ C3 = connect(ClientId, Config, [{clean_start, true}]),
+ ok = emqtt:disconnect(C3).
+
+subscribe_same_topic_same_qos(Config) ->
+ C = connect(?FUNCTION_NAME, Config),
+ Topic = <<"a/b">>,
+
+ {ok, _} = emqtt:publish(C, Topic, <<"retained">>, [{retain, true},
+ {qos, 1}]),
+ %% Subscribe with QoS 0
+ {ok, _, [0]} = emqtt:subscribe(C, Topic, qos0),
+ {ok, _} = emqtt:publish(C, Topic, <<"msg1">>, qos1),
+ %% Subscribe to same topic with same QoS
+ {ok, _, [0]} = emqtt:subscribe(C, Topic, qos0),
+ {ok, _} = emqtt:publish(C, Topic, <<"msg2">>, qos1),
+
+ %% "Any existing retained messages matching the Topic Filter MUST be re-sent" [MQTT-3.8.4-3]
+ ok = expect_publishes(C, Topic, [<<"retained">>, <<"msg1">>,
+ <<"retained">>, <<"msg2">>
+ ]),
+ ok = emqtt:disconnect(C).
+
+subscribe_same_topic_different_qos(Config) ->
+ C = connect(?FUNCTION_NAME, Config, [{clean_start, false}]),
+ Topic = <<"b/c">>,
+
+ {ok, _} = emqtt:publish(C, Topic, <<"retained">>, [{retain, true},
+ {qos, 1}]),
+ %% Subscribe with QoS 0
+ {ok, _, [0]} = emqtt:subscribe(C, Topic, qos0),
+ {ok, _} = emqtt:publish(C, Topic, <<"msg1">>, qos1),
+ %% Subscribe to same topic with QoS 1
+ {ok, _, [1]} = emqtt:subscribe(C, Topic, qos1),
+ {ok, _} = emqtt:publish(C, Topic, <<"msg2">>, qos1),
+ %% Subscribe to same topic with QoS 0 again
+ {ok, _, [0]} = emqtt:subscribe(C, Topic, qos0),
+ {ok, _} = emqtt:publish(C, Topic, <<"msg3">>, qos1),
+
+ %% "Any existing retained messages matching the Topic Filter MUST be re-sent" [MQTT-3.8.4-3]
+ ok = expect_publishes(C, Topic, [<<"retained">>, <<"msg1">>,
+ <<"retained">>, <<"msg2">>,
+ <<"retained">>, <<"msg3">>]),
+
+ %% There should be exactly one consumer for each queue: qos0 and qos1
+ Consumers = rpc(Config, rabbit_amqqueue, consumers_all, [<<"/">>]),
+ ?assertEqual(2, length(Consumers)),
+
+ ok = emqtt:disconnect(C),
+ C1 = connect(?FUNCTION_NAME, Config, [{clean_start, true}]),
+ ok = emqtt:disconnect(C1).
+
+subscribe_multiple(Config) ->
+ C = connect(?FUNCTION_NAME, Config),
+ %% Subscribe to multiple topics at once
+ ?assertMatch({ok, _, [0, 1]},
+ emqtt:subscribe(C, [{<<"topic0">>, qos0},
+ {<<"topic1">>, qos1}])),
+ ok = emqtt:disconnect(C).
+
+large_message_mqtt_to_mqtt(Config) ->
+ Topic = ClientId = atom_to_binary(?FUNCTION_NAME),
+ C = connect(ClientId, Config),
+ {ok, _, [1]} = emqtt:subscribe(C, {Topic, qos1}),
+
+ Payload0 = binary:copy(<<"x">>, 8_000_000),
+ Payload = <>,
+ {ok, _} = emqtt:publish(C, Topic, Payload, qos1),
+ ok = expect_publishes(C, Topic, [Payload]),
+ ok = emqtt:disconnect(C).
+
+large_message_amqp_to_mqtt(Config) ->
+ Topic = ClientId = atom_to_binary(?FUNCTION_NAME),
+ C = connect(ClientId, Config),
+ {ok, _, [1]} = emqtt:subscribe(C, {Topic, qos1}),
+
+ Ch = rabbit_ct_client_helpers:open_channel(Config),
+ Payload0 = binary:copy(<<"x">>, 8_000_000),
+ Payload = <>,
+ amqp_channel:call(Ch,
+ #'basic.publish'{exchange = <<"amq.topic">>,
+ routing_key = Topic},
+ #amqp_msg{payload = Payload}),
+ ok = expect_publishes(C, Topic, [Payload]),
+ ok = emqtt:disconnect(C).
+
+amqp_to_mqtt_qos0(Config) ->
+ Topic = ClientId = Payload = atom_to_binary(?FUNCTION_NAME),
+ C = connect(ClientId, Config),
+ {ok, _, [0]} = emqtt:subscribe(C, {Topic, qos0}),
+
+ Ch = rabbit_ct_client_helpers:open_channel(Config),
+ amqp_channel:call(Ch,
+ #'basic.publish'{exchange = <<"amq.topic">>,
+ routing_key = Topic},
+ #amqp_msg{payload = Payload}),
+ ok = expect_publishes(C, Topic, [Payload]),
+ ok = emqtt:disconnect(C).
+
+%% Packet identifier is a non zero two byte integer.
+%% Test that the server wraps around the packet identifier.
+many_qos1_messages(Config) ->
+ Topic = ClientId = atom_to_binary(?FUNCTION_NAME),
+ C = connect(ClientId, Config, 0, [{retry_interval, 600}]),
+ {ok, _, [1]} = emqtt:subscribe(C, {Topic, qos1}),
+ NumMsgs = 16#ffff + 100,
+ Payloads = lists:map(fun integer_to_binary/1, lists:seq(1, NumMsgs)),
+ lists:foreach(fun(P) ->
+ {ok, _} = emqtt:publish(C, Topic, P, qos1)
+ end, Payloads),
+ expect_publishes(C, Topic, Payloads),
+ ok = emqtt:disconnect(C).
+
+%% This test is mostly interesting in mixed version mode where feature flag
+%% rabbit_mqtt_qos0_queue is disabled and therefore a classic queue gets created.
+rabbit_mqtt_qos0_queue(Config) ->
+ Topic = atom_to_binary(?FUNCTION_NAME),
+
+ %% Place MQTT subscriber process on new node in mixed version.
+ Sub = connect(<<"subscriber">>, Config),
+ {ok, _, [0]} = emqtt:subscribe(Sub, Topic, qos0),
+
+ %% Place MQTT publisher process on old node in mixed version.
+ Pub = connect(<<"publisher">>, Config, 1, []),
+
+ Msg = <<"msg">>,
+ ok = emqtt:publish(Pub, Topic, Msg, qos0),
+ ok = expect_publishes(Sub, Topic, [Msg]),
+
+ ok = emqtt:disconnect(Sub),
+ ok = emqtt:disconnect(Pub).
+
+%% Test that MQTT connection can be listed and closed via the rabbitmq_management plugin.
+management_plugin_connection(Config) ->
+ KeepaliveSecs = 99,
+ ClientId = atom_to_binary(?FUNCTION_NAME),
+ Node = atom_to_binary(get_node_config(Config, 0, nodename)),
+ C = connect(ClientId, Config, [{keepalive, KeepaliveSecs}]),
+
+ eventually(?_assertEqual(1, length(http_get(Config, "/connections"))), 1000, 10),
+ [#{client_properties := #{client_id := ClientId},
+ timeout := KeepaliveSecs,
+ node := Node,
+ name := ConnectionName}] = http_get(Config, "/connections"),
+
+ process_flag(trap_exit, true),
+ http_delete(Config,
+ "/connections/" ++ binary_to_list(uri_string:quote((ConnectionName))),
+ ?NO_CONTENT),
+ await_exit(C),
+ ?assertEqual([], http_get(Config, "/connections")),
+ eventually(?_assertEqual([], all_connection_pids(Config)), 500, 3).
+
+management_plugin_enable(Config) ->
+ ?assertEqual(0, length(http_get(Config, "/connections"))),
+ ok = rabbit_ct_broker_helpers:disable_plugin(Config, 0, rabbitmq_management),
+ ok = rabbit_ct_broker_helpers:disable_plugin(Config, 0, rabbitmq_management_agent),
+
+ %% If the (web) MQTT connection is established **before** the management plugin is enabled,
+ %% the management plugin should still list the (web) MQTT connection.
+ C = connect(?FUNCTION_NAME, Config),
+ ok = rabbit_ct_broker_helpers:enable_plugin(Config, 0, rabbitmq_management_agent),
+ ok = rabbit_ct_broker_helpers:enable_plugin(Config, 0, rabbitmq_management),
+ eventually(?_assertEqual(1, length(http_get(Config, "/connections"))), 1000, 10),
+
+ ok = emqtt:disconnect(C).
+
+%% Test that queues of type rabbit_mqtt_qos0_queue can be listed via rabbitmqctl.
+cli_list_queues(Config) ->
+ C = connect(?FUNCTION_NAME, Config),
+ {ok, _, _} = emqtt:subscribe(C, <<"a/b/c">>, qos0),
+
+ Qs = rabbit_ct_broker_helpers:rabbitmqctl_list(
+ Config, 1,
+ ["list_queues", "--no-table-headers",
+ "type", "name", "state", "durable", "auto_delete",
+ "arguments", "pid", "owner_pid", "messages", "exclusive_consumer_tag"
+ ]),
+ ExpectedQueueType = case rabbit_ct_helpers:is_mixed_versions(Config) of
+ false ->
+ <<"MQTT QoS 0">>;
+ true ->
+ <<"classic">>
+ end,
+ ?assertMatch([[ExpectedQueueType, <<"mqtt-subscription-cli_list_queuesqos0">>,
+ <<"running">>, <<"true">>, <<"false">>, <<"[]">>, _, _, <<"0">>, <<"">>]],
+ Qs),
+
+ ?assertEqual([],
+ rabbit_ct_broker_helpers:rabbitmqctl_list(
+ Config, 1, ["list_queues", "--local", "--no-table-headers"])
+ ),
+
+ ok = emqtt:disconnect(C).
+
+maintenance(Config) ->
+ C0 = connect(<<"client-0">>, Config, 0, []),
+ C1a = connect(<<"client-1a">>, Config, 1, []),
+ C1b = connect(<<"client-1b">>, Config, 1, []),
+
+ timer:sleep(500),
+
+ ok = drain_node(Config, 2),
+ ok = revive_node(Config, 2),
+ timer:sleep(500),
+ [?assert(erlang:is_process_alive(C)) || C <- [C0, C1a, C1b]],
+
+ process_flag(trap_exit, true),
+ ok = drain_node(Config, 1),
+ [await_exit(Pid) || Pid <- [C1a, C1b]],
+ ok = revive_node(Config, 1),
+ ?assert(erlang:is_process_alive(C0)),
+
+ ok = drain_node(Config, 0),
+ await_exit(C0),
+ ok = revive_node(Config, 0).
+
+keepalive(Config) ->
+ KeepaliveSecs = 1,
+ KeepaliveMs = timer:seconds(KeepaliveSecs),
+ ProtoVer = v4,
+ WillTopic = <<"will/topic">>,
+ WillPayload = <<"will-payload">>,
+ C1 = connect(?FUNCTION_NAME, Config, [{keepalive, KeepaliveSecs},
+ {proto_ver, ProtoVer},
+ {will_topic, WillTopic},
+ {will_payload, WillPayload},
+ {will_retain, true},
+ {will_qos, 0}]),
+ ok = emqtt:publish(C1, <<"ignored">>, <<"msg">>),
+
+ %% Connection should stay up when client sends PING requests.
+ timer:sleep(KeepaliveMs),
+ ?assertMatch(#{publishers := 1},
+ util:get_global_counters(Config, ProtoVer)),
+
+ %% Mock the server socket to not have received any bytes.
+ rabbit_ct_broker_helpers:setup_meck(Config),
+ Mod = rabbit_net,
+ ok = rpc(Config, meck, new, [Mod, [no_link, passthrough]]),
+ ok = rpc(Config, meck, expect, [Mod, getstat, 2, {ok, [{recv_oct, 999}]} ]),
+ process_flag(trap_exit, true),
+
+ %% We expect the server to respect the keepalive closing the connection.
+ eventually(?_assertMatch(#{publishers := 0},
+ util:get_global_counters(Config, ProtoVer)),
+ KeepaliveMs, 3 * KeepaliveSecs),
+ await_exit(C1),
+
+ true = rpc(Config, meck, validate, [Mod]),
+ ok = rpc(Config, meck, unload, [Mod]),
+
+ C2 = connect(<<"client2">>, Config),
+ {ok, _, [0]} = emqtt:subscribe(C2, WillTopic),
+ receive {publish, #{client_pid := C2,
+ dup := false,
+ qos := 0,
+ retain := true,
+ topic := WillTopic,
+ payload := WillPayload}} -> ok
+ after 3000 -> ct:fail("missing will")
+ end,
+ ok = emqtt:disconnect(C2).
+
+keepalive_turned_off(Config) ->
+ %% "A Keep Alive value of zero (0) has the effect of turning off the keep alive mechanism."
+ KeepaliveSecs = 0,
+ C = connect(?FUNCTION_NAME, Config, [{keepalive, KeepaliveSecs}]),
+ ok = emqtt:publish(C, <<"TopicB">>, <<"Payload">>),
+
+ %% Mock the server socket to not have received any bytes.
+ rabbit_ct_broker_helpers:setup_meck(Config),
+ Mod = rabbit_net,
+ ok = rpc(Config, meck, new, [Mod, [no_link, passthrough]]),
+ ok = rpc(Config, meck, expect, [Mod, getstat, 2, {ok, [{recv_oct, 999}]} ]),
+
+ rabbit_ct_helpers:consistently(?_assert(erlang:is_process_alive(C))),
+
+ true = rpc(Config, meck, validate, [Mod]),
+ ok = rpc(Config, meck, unload, [Mod]),
+ ok = emqtt:disconnect(C).
+
+duplicate_client_id(Config) ->
+ DuplicateClientId = ?FUNCTION_NAME,
+ C1 = connect(DuplicateClientId, Config),
+ eventually(?_assertEqual(1, length(all_connection_pids(Config)))),
+
+ process_flag(trap_exit, true),
+ C2 = connect(DuplicateClientId, Config),
+ await_exit(C1),
+ timer:sleep(200),
+ ?assertEqual(1, length(all_connection_pids(Config))),
+
+ ok = emqtt:disconnect(C2).
+
+block(Config) ->
+ Topic = ClientId = atom_to_binary(?FUNCTION_NAME),
+ C = connect(ClientId, Config),
+
+ {ok, _, _} = emqtt:subscribe(C, Topic),
+ {ok, _} = emqtt:publish(C, Topic, <<"Not blocked yet">>, [{qos, 1}]),
+
+ ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0]),
+ %% Let it block
+ timer:sleep(100),
+
+ %% Blocked, but still will publish when unblocked
+ puback_timeout = publish_qos1_timeout(C, Topic, <<"Now blocked">>, 1000),
+ puback_timeout = publish_qos1_timeout(C, Topic, <<"Still blocked">>, 1000),
+
+ %% Unblock
+ rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]),
+ ok = expect_publishes(C, Topic, [<<"Not blocked yet">>,
+ <<"Now blocked">>,
+ <<"Still blocked">>]),
+ ok = emqtt:disconnect(C).
+
+block_only_publisher(Config) ->
+ Topic = atom_to_binary(?FUNCTION_NAME),
+
+ Opts = [{ack_timeout, 1}],
+ Con = connect(<<"background-connection">>, Config, Opts),
+ Sub = connect(<<"subscriber-connection">>, Config, Opts),
+ Pub = connect(<<"publisher-connection">>, Config, Opts),
+ PubSub = connect(<<"publisher-and-subscriber-connection">>, Config, Opts),
+
+ {ok, _, [1]} = emqtt:subscribe(Sub, Topic, qos1),
+ {ok, _, [1]} = emqtt:subscribe(PubSub, Topic, qos1),
+ {ok, _} = emqtt:publish(Pub, Topic, <<"from Pub">>, [{qos, 1}]),
+ {ok, _} = emqtt:publish(PubSub, Topic, <<"from PubSub">>, [{qos, 1}]),
+ ok = expect_publishes(Sub, Topic, [<<"from Pub">>, <<"from PubSub">>]),
+ ok = expect_publishes(PubSub, Topic, [<<"from Pub">>, <<"from PubSub">>]),
+
+ ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0]),
+ %% Let it block
+ timer:sleep(100),
+
+ %% We expect that the publishing connections are blocked.
+ [?assertEqual({error, ack_timeout}, emqtt:ping(Pid)) || Pid <- [Pub, PubSub]],
+ %% We expect that the non-publishing connections are not blocked.
+ [?assertEqual(pong, emqtt:ping(Pid)) || Pid <- [Con, Sub]],
+
+ %% While the memory alarm is on, let's turn a non-publishing connection
+ %% into a publishing connection.
+ {ok, _} = emqtt:publish(Con, Topic, <<"from Con 1">>, [{qos, 1}]),
+ %% The very first message still goes through.
+ ok = expect_publishes(Sub, Topic, [<<"from Con 1">>]),
+ %% But now the new publisher should be blocked as well.
+ ?assertEqual({error, ack_timeout}, emqtt:ping(Con)),
+ ?assertEqual(puback_timeout, publish_qos1_timeout(Con, Topic, <<"from Con 2">>, 500)),
+ ?assertEqual(pong, emqtt:ping(Sub)),
+
+ rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]),
+ %% Let it unblock
+ timer:sleep(100),
+
+ %% All connections are unblocked.
+ [?assertEqual(pong, emqtt:ping(Pid)) || Pid <- [Con, Sub, Pub, PubSub]],
+ %% The publishing connections should be able to publish again.
+ {ok, _} = emqtt:publish(Con, Topic, <<"from Con 3">>, [{qos, 1}]),
+ ok = expect_publishes(Sub, Topic, [<<"from Con 2">>, <<"from Con 3">>]),
+ ok = expect_publishes(PubSub, Topic, [<<"from Con 1">>, <<"from Con 2">>, <<"from Con 3">>]),
+
+ [ok = emqtt:disconnect(Pid) || Pid <- [Con, Sub, Pub, PubSub]].
+
+clean_session_disconnect_client(Config) ->
+ C = connect(?FUNCTION_NAME, Config),
+ {ok, _, _} = emqtt:subscribe(C, <<"topic0">>, qos0),
+ {ok, _, _} = emqtt:subscribe(C, <<"topic1">>, qos1),
+ QsQos0 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_mqtt_qos0_queue]),
+ QsClassic = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]),
+ case rabbit_ct_helpers:is_mixed_versions(Config) of
+ false ->
+ ?assertEqual(1, length(QsQos0)),
+ ?assertEqual(1, length(QsClassic));
+ true ->
+ ?assertEqual(0, length(QsQos0)),
+ ?assertEqual(2, length(QsClassic))
+ end,
+
+ ok = emqtt:disconnect(C),
+ %% After terminating a clean session, we expect any session state to be cleaned up on the server.
+ timer:sleep(200), %% Give some time to clean up exclusive classic queue.
+ L = rpc(Config, rabbit_amqqueue, list, []),
+ ?assertEqual(0, length(L)).
+
+clean_session_kill_node(Config) ->
+ C = connect(?FUNCTION_NAME, Config),
+ {ok, _, _} = emqtt:subscribe(C, <<"topic0">>, qos0),
+ {ok, _, _} = emqtt:subscribe(C, <<"topic1">>, qos1),
+ QsQos0 = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_mqtt_qos0_queue]),
+ QsClassic = rpc(Config, rabbit_amqqueue, list_by_type, [rabbit_classic_queue]),
+ case rabbit_ct_helpers:is_mixed_versions(Config) of
+ false ->
+ ?assertEqual(1, length(QsQos0)),
+ ?assertEqual(1, length(QsClassic));
+ true ->
+ ?assertEqual(0, length(QsQos0)),
+ ?assertEqual(2, length(QsClassic))
+ end,
+ ?assertEqual(2, rpc(Config, ets, info, [rabbit_durable_queue, size])),
+
+ unlink(C),
+ ok = rabbit_ct_broker_helpers:kill_node(Config, 0),
+ ok = rabbit_ct_broker_helpers:start_node(Config, 0),
+
+ %% After terminating a clean session by a node crash, we expect any session
+ %% state to be cleaned up on the server once the server comes back up.
+ ?assertEqual(0, rpc(Config, ets, info, [rabbit_durable_queue, size])).
+
+rabbit_status_connection_count(Config) ->
+ _ = rabbit_ct_client_helpers:open_connection(Config, 0),
+ C = connect(?FUNCTION_NAME, Config),
+
+ {ok, String} = rabbit_ct_broker_helpers:rabbitmqctl(Config, 0, ["status"]),
+ ?assertNotEqual(nomatch, string:find(String, "Connection count: 2")),
+
+ ok = emqtt:disconnect(C).
+
+trace(Config) ->
+ Server = atom_to_binary(get_node_config(Config, 0, nodename)),
+ Topic = Payload = TraceQ = atom_to_binary(?FUNCTION_NAME),
+ Ch = rabbit_ct_client_helpers:open_channel(Config),
+ declare_queue(Ch, TraceQ, []),
+ #'queue.bind_ok'{} = amqp_channel:call(
+ Ch, #'queue.bind'{queue = TraceQ,
+ exchange = <<"amq.rabbitmq.trace">>,
+ routing_key = <<"#">>}),
+
+ %% We expect traced messages for connections created before and connections
+ %% created after tracing is enabled.
+ Pub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_publisher">>, Config),
+ {ok, _} = rabbit_ct_broker_helpers:rabbitmqctl(Config, 0, ["trace_on"]),
+ Sub = connect(<<(atom_to_binary(?FUNCTION_NAME))/binary, "_subscriber">>, Config),
+
+ {ok, _, [0]} = emqtt:subscribe(Sub, Topic, qos0),
+ {ok, _} = emqtt:publish(Pub, Topic, Payload, qos1),
+ ok = expect_publishes(Sub, Topic, [Payload]),
+
+ {#'basic.get_ok'{routing_key = <<"publish.amq.topic">>},
+ #amqp_msg{props = #'P_basic'{headers = PublishHeaders},
+ payload = Payload}} =
+ amqp_channel:call(Ch, #'basic.get'{queue = TraceQ, no_ack = false}),
+ ?assertMatch(#{<<"exchange_name">> := <<"amq.topic">>,
+ <<"routing_keys">> := [Topic],
+ <<"connection">> := <<"127.0.0.1:", _/binary>>,
+ <<"node">> := Server,
+ <<"vhost">> := <<"/">>,
+ <<"channel">> := 0,
+ <<"user">> := <<"guest">>,
+ <<"properties">> := #{<<"delivery_mode">> := 2,
+ <<"headers">> := #{<<"x-mqtt-publish-qos">> := 1,
+ <<"x-mqtt-dup">> := false}},
+ <<"routed_queues">> := [<<"mqtt-subscription-trace_subscriberqos0">>]},
+ rabbit_misc:amqp_table(PublishHeaders)),
+
+ {#'basic.get_ok'{routing_key = <<"deliver.mqtt-subscription-trace_subscriberqos0">>},
+ #amqp_msg{props = #'P_basic'{headers = DeliverHeaders},
+ payload = Payload}} =
+ amqp_channel:call(Ch, #'basic.get'{queue = TraceQ, no_ack = false}),
+ ?assertMatch(#{<<"exchange_name">> := <<"amq.topic">>,
+ <<"routing_keys">> := [Topic],
+ <<"connection">> := <<"127.0.0.1:", _/binary>>,
+ <<"node">> := Server,
+ <<"vhost">> := <<"/">>,
+ <<"channel">> := 0,
+ <<"user">> := <<"guest">>,
+ <<"properties">> := #{<<"delivery_mode">> := 2,
+ <<"headers">> := #{<<"x-mqtt-publish-qos">> := 1,
+ <<"x-mqtt-dup">> := false}},
+ <<"redelivered">> := 0},
+ rabbit_misc:amqp_table(DeliverHeaders)),
+
+ {ok, _} = rabbit_ct_broker_helpers:rabbitmqctl(Config, 0, ["trace_off"]),
+ {ok, _} = emqtt:publish(Pub, Topic, Payload, qos1),
+ ok = expect_publishes(Sub, Topic, [Payload]),
+ ?assertMatch(#'basic.get_empty'{},
+ amqp_channel:call(Ch, #'basic.get'{queue = TraceQ, no_ack = false})),
+
+ delete_queue(Ch, TraceQ),
+ [ok = emqtt:disconnect(C) || C <- [Pub, Sub]].
+
+%% -------------------------------------------------------------------
+%% Internal helpers
+%% -------------------------------------------------------------------
+
+await_confirms_ordered(_, To, To) ->
+ ok;
+await_confirms_ordered(From, N, To) ->
+ Expected = {From, N},
+ receive
+ Expected ->
+ await_confirms_ordered(From, N + 1, To);
+ Got ->
+ ct:fail("Received unexpected message. Expected: ~p Got: ~p", [Expected, Got])
+ after 10_000 ->
+ ct:fail("Did not receive expected message: ~p", [Expected])
+ end.
+
+await_confirms_unordered(_, 0) ->
+ ok;
+await_confirms_unordered(From, Left) ->
+ receive
+ {From, _N} ->
+ await_confirms_unordered(From, Left - 1);
+ Other ->
+ ct:fail("Received unexpected message: ~p", [Other])
+ after 10_000 ->
+ ct:fail("~b confirms are missing", [Left])
+ end.
+
+declare_queue(Ch, QueueName, Args)
+ when is_pid(Ch), is_binary(QueueName), is_list(Args) ->
+ #'queue.declare_ok'{} = amqp_channel:call(
+ Ch, #'queue.declare'{
+ queue = QueueName,
+ durable = true,
+ arguments = Args}).
+
+delete_queue(Ch, QueueNames)
+ when is_pid(Ch), is_list(QueueNames) ->
+ lists:foreach(
+ fun(Q) ->
+ delete_queue(Ch, Q)
+ end, QueueNames);
+delete_queue(Ch, QueueName)
+ when is_pid(Ch), is_binary(QueueName) ->
+ #'queue.delete_ok'{} = amqp_channel:call(
+ Ch, #'queue.delete'{
+ queue = QueueName}).
+
+bind(Ch, QueueName, Topic)
+ when is_pid(Ch), is_binary(QueueName), is_binary(Topic) ->
+ #'queue.bind_ok'{} = amqp_channel:call(
+ Ch, #'queue.bind'{queue = QueueName,
+ exchange = <<"amq.topic">>,
+ routing_key = Topic}).
diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl
new file mode 100644
index 000000000000..44c684a6c5d6
--- /dev/null
+++ b/deps/rabbitmq_mqtt/test/util.erl
@@ -0,0 +1,140 @@
+-module(util).
+
+-include("rabbit_mqtt.hrl").
+-include_lib("rabbit_common/include/rabbit.hrl").
+-include_lib("eunit/include/eunit.hrl").
+
+-export([all_connection_pids/1,
+ publish_qos1_timeout/4,
+ sync_publish_result/3,
+ get_global_counters/2,
+ get_global_counters/3,
+ get_global_counters/4,
+ expect_publishes/3,
+ connect/2,
+ connect/3,
+ connect/4,
+ get_events/1,
+ assert_event_type/2,
+ assert_event_prop/2,
+ await_exit/1,
+ await_exit/2
+ ]).
+
+all_connection_pids(Config) ->
+ Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
+ Result = erpc:multicall(Nodes, rabbit_mqtt, local_connection_pids, [], 5000),
+ lists:foldl(fun({ok, Pids}, Acc) ->
+ Pids ++ Acc;
+ (_, Acc) ->
+ Acc
+ end, [], Result).
+
+publish_qos1_timeout(Client, Topic, Payload, Timeout) ->
+ Mref = erlang:monitor(process, Client),
+ ok = emqtt:publish_async(Client, Topic, #{}, Payload, [{qos, 1}], infinity,
+ {fun ?MODULE:sync_publish_result/3, [self(), Mref]}),
+ receive
+ {Mref, Reply} ->
+ erlang:demonitor(Mref, [flush]),
+ Reply;
+ {'DOWN', Mref, process, Client, Reason} ->
+ ct:fail("client is down: ~tp", [Reason])
+ after
+ Timeout ->
+ erlang:demonitor(Mref, [flush]),
+ puback_timeout
+ end.
+
+sync_publish_result(Caller, Mref, Result) ->
+ erlang:send(Caller, {Mref, Result}).
+
+expect_publishes(_, _, []) ->
+ ok;
+expect_publishes(Client, Topic, [Payload|Rest])
+ when is_pid(Client) ->
+ receive
+ {publish, #{client_pid := Client,
+ topic := Topic,
+ payload := Payload}} ->
+ expect_publishes(Client, Topic, Rest);
+ {publish, #{client_pid := Client,
+ topic := Topic,
+ payload := Other}} ->
+ ct:fail("Received unexpected PUBLISH payload. Expected: ~p Got: ~p",
+ [Payload, Other])
+ after 3000 ->
+ {publish_not_received, Payload}
+ end.
+
+get_global_counters(Config, ProtoVer) ->
+ get_global_counters(Config, ProtoVer, 0, []).
+
+get_global_counters(Config, ProtoVer, Node) ->
+ get_global_counters(Config, ProtoVer, Node, []).
+
+get_global_counters(Config, v3, Node, QType) ->
+ get_global_counters(Config, ?MQTT_PROTO_V3, Node, QType);
+get_global_counters(Config, v4, Node, QType) ->
+ get_global_counters(Config, ?MQTT_PROTO_V4, Node, QType);
+get_global_counters(Config, Proto, Node, QType) ->
+ maps:get([{protocol, Proto}] ++ QType,
+ rabbit_ct_broker_helpers:rpc(Config, Node, rabbit_global_counters, overview, [])).
+
+get_events(Node) ->
+ timer:sleep(300), %% events are sent and processed asynchronously
+ Result = gen_event:call({rabbit_event, Node}, event_recorder, take_state),
+ ?assert(is_list(Result)),
+ Result.
+
+assert_event_type(ExpectedType, #event{type = ActualType}) ->
+ ?assertEqual(ExpectedType, ActualType).
+
+assert_event_prop(ExpectedProp = {Key, _Value}, #event{props = Props}) ->
+ ?assertEqual(ExpectedProp, lists:keyfind(Key, 1, Props));
+assert_event_prop(ExpectedProps, Event)
+ when is_list(ExpectedProps) ->
+ lists:foreach(fun(P) ->
+ assert_event_prop(P, Event)
+ end, ExpectedProps).
+
+await_exit(Pid) ->
+ receive
+ {'EXIT', Pid, _} -> ok
+ after
+ 20_000 -> ct:fail({missing_exit, Pid})
+ end.
+
+await_exit(Pid, Reason) ->
+ receive
+ {'EXIT', Pid, Reason} -> ok
+ after
+ 20_000 -> ct:fail({missing_exit, Pid})
+ end.
+
+connect(ClientId, Config) ->
+ connect(ClientId, Config, []).
+
+connect(ClientId, Config, AdditionalOpts) ->
+ connect(ClientId, Config, 0, AdditionalOpts).
+
+connect(ClientId, Config, Node, AdditionalOpts) ->
+ {Port, WsOpts, Connect} =
+ case rabbit_ct_helpers:get_config(Config, websocket, false) of
+ false ->
+ {rabbit_ct_broker_helpers:get_node_config(Config, Node, tcp_port_mqtt),
+ [],
+ fun emqtt:connect/1};
+ true ->
+ {rabbit_ct_broker_helpers:get_node_config(Config, Node, tcp_port_web_mqtt),
+ [{ws_path, "/ws"}],
+ fun emqtt:ws_connect/1}
+ end,
+ Options = [{host, "localhost"},
+ {port, Port},
+ {proto_ver, v4},
+ {clientid, rabbit_data_coercion:to_binary(ClientId)}
+ ] ++ WsOpts ++ AdditionalOpts,
+ {ok, C} = emqtt:start_link(Options),
+ {ok, _Properties} = Connect(C),
+ C.
diff --git a/deps/rabbitmq_mqtt/test/util_SUITE.erl b/deps/rabbitmq_mqtt/test/util_SUITE.erl
index c15eb1dee407..3d058500abce 100644
--- a/deps/rabbitmq_mqtt/test/util_SUITE.erl
+++ b/deps/rabbitmq_mqtt/test/util_SUITE.erl
@@ -5,26 +5,26 @@
%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
-module(util_SUITE).
--compile([export_all]).
+-compile([export_all, nowarn_export_all]).
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
all() ->
[
- {group, util_tests}
+ {group, tests}
].
groups() ->
[
- {util_tests, [parallel], [
- coerce_exchange,
- coerce_vhost,
- coerce_default_user,
- coerce_default_pass,
- mqtt_amqp_topic_translation
- ]
- }
+ {tests, [parallel], [
+ coerce_exchange,
+ coerce_vhost,
+ coerce_default_user,
+ coerce_default_pass,
+ mqtt_amqp_topic_translation
+ ]
+ }
].
suite() ->
@@ -55,26 +55,24 @@ coerce_default_pass(_) ->
mqtt_amqp_topic_translation(_) ->
ok = application:set_env(rabbitmq_mqtt, sparkplug, true),
- {ok, {mqtt2amqp_fun, Mqtt2AmqpFun}, {amqp2mqtt_fun, Amqp2MqttFun}} =
- rabbit_mqtt_util:get_topic_translation_funs(),
+ ok = rabbit_mqtt_util:init_sparkplug(),
- T0 = "/foo/bar/+/baz",
+ T0 = <<"/foo/bar/+/baz">>,
T0_As_Amqp = <<".foo.bar.*.baz">>,
T0_As_Mqtt = <<"/foo/bar/+/baz">>,
- ?assertEqual(T0_As_Amqp, Mqtt2AmqpFun(T0)),
- ?assertEqual(T0_As_Mqtt, Amqp2MqttFun(T0_As_Amqp)),
+ ?assertEqual(T0_As_Amqp, rabbit_mqtt_util:mqtt_to_amqp(T0)),
+ ?assertEqual(T0_As_Mqtt, rabbit_mqtt_util:amqp_to_mqtt(T0_As_Amqp)),
- T1 = "spAv1.0/foo/bar/+/baz",
+ T1 = <<"spAv1.0/foo/bar/+/baz">>,
T1_As_Amqp = <<"spAv1___0.foo.bar.*.baz">>,
T1_As_Mqtt = <<"spAv1.0/foo/bar/+/baz">>,
- ?assertEqual(T1_As_Amqp, Mqtt2AmqpFun(T1)),
- ?assertEqual(T1_As_Mqtt, Amqp2MqttFun(T1_As_Amqp)),
+ ?assertEqual(T1_As_Amqp, rabbit_mqtt_util:mqtt_to_amqp(T1)),
+ ?assertEqual(T1_As_Mqtt, rabbit_mqtt_util:amqp_to_mqtt(T1_As_Amqp)),
- T2 = "spBv2.90/foo/bar/+/baz",
+ T2 = <<"spBv2.90/foo/bar/+/baz">>,
T2_As_Amqp = <<"spBv2___90.foo.bar.*.baz">>,
T2_As_Mqtt = <<"spBv2.90/foo/bar/+/baz">>,
- ?assertEqual(T2_As_Amqp, Mqtt2AmqpFun(T2)),
- ?assertEqual(T2_As_Mqtt, Amqp2MqttFun(T2_As_Amqp)),
+ ?assertEqual(T2_As_Amqp, rabbit_mqtt_util:mqtt_to_amqp(T2)),
+ ?assertEqual(T2_As_Mqtt, rabbit_mqtt_util:amqp_to_mqtt(T2_As_Amqp)),
- ok = application:unset_env(rabbitmq_mqtt, sparkplug),
- ok.
+ ok = application:unset_env(rabbitmq_mqtt, sparkplug).
diff --git a/deps/rabbitmq_stomp/src/rabbit_stomp_reader.erl b/deps/rabbitmq_stomp/src/rabbit_stomp_reader.erl
index 5a57366fcfc8..9b85444769c1 100644
--- a/deps/rabbitmq_stomp/src/rabbit_stomp_reader.erl
+++ b/deps/rabbitmq_stomp/src/rabbit_stomp_reader.erl
@@ -57,12 +57,13 @@ init([SupHelperPid, Ref, Configuration]) ->
case rabbit_net:connection_string(Sock, inbound) of
{ok, ConnStr} ->
+ ConnName = rabbit_data_coercion:to_binary(ConnStr),
ProcInitArgs = processor_args(Configuration, Sock),
ProcState = rabbit_stomp_processor:initial_state(Configuration,
ProcInitArgs),
rabbit_log_connection:info("accepting STOMP connection ~tp (~ts)",
- [self(), ConnStr]),
+ [self(), ConnName]),
ParseState = rabbit_stomp_frame:initial_state(),
_ = register_resource_alarm(),
@@ -74,7 +75,7 @@ init([SupHelperPid, Ref, Configuration]) ->
rabbit_event:init_stats_timer(
run_socket(control_throttle(
#reader_state{socket = RealSocket,
- conn_name = ConnStr,
+ conn_name = ConnName,
parse_state = ParseState,
processor_state = ProcState,
heartbeat_sup = SupHelperPid,
@@ -252,6 +253,9 @@ process_received_bytes(Bytes,
{stop, normal, State}
end.
+-spec conserve_resources(pid(),
+ rabbit_alarm:resource_alarm_source(),
+ rabbit_alarm:resource_alert()) -> ok.
conserve_resources(Pid, _Source, {_, Conserve, _}) ->
Pid ! {conserve_resources, Conserve},
ok.
@@ -301,34 +305,34 @@ code_change(_OldVsn, State, _Extra) ->
{ok, State}.
-log_reason({network_error, {ssl_upgrade_error, closed}, ConnStr}, _State) ->
+log_reason({network_error, {ssl_upgrade_error, closed}, ConnName}, _State) ->
rabbit_log_connection:error("STOMP detected TLS upgrade error on ~ts: connection closed",
- [ConnStr]);
+ [ConnName]);
log_reason({network_error,
{ssl_upgrade_error,
- {tls_alert, "handshake failure"}}, ConnStr}, _State) ->
- log_tls_alert(handshake_failure, ConnStr);
+ {tls_alert, "handshake failure"}}, ConnName}, _State) ->
+ log_tls_alert(handshake_failure, ConnName);
log_reason({network_error,
{ssl_upgrade_error,
- {tls_alert, "unknown ca"}}, ConnStr}, _State) ->
- log_tls_alert(unknown_ca, ConnStr);
+ {tls_alert, "unknown ca"}}, ConnName}, _State) ->
+ log_tls_alert(unknown_ca, ConnName);
log_reason({network_error,
{ssl_upgrade_error,
- {tls_alert, {Err, _}}}, ConnStr}, _State) ->
- log_tls_alert(Err, ConnStr);
+ {tls_alert, {Err, _}}}, ConnName}, _State) ->
+ log_tls_alert(Err, ConnName);
log_reason({network_error,
{ssl_upgrade_error,
- {tls_alert, Alert}}, ConnStr}, _State) ->
- log_tls_alert(Alert, ConnStr);
-log_reason({network_error, {ssl_upgrade_error, Reason}, ConnStr}, _State) ->
+ {tls_alert, Alert}}, ConnName}, _State) ->
+ log_tls_alert(Alert, ConnName);
+log_reason({network_error, {ssl_upgrade_error, Reason}, ConnName}, _State) ->
rabbit_log_connection:error("STOMP detected TLS upgrade error on ~ts: ~tp",
- [ConnStr, Reason]);
+ [ConnName, Reason]);
-log_reason({network_error, Reason, ConnStr}, _State) ->
+log_reason({network_error, Reason, ConnName}, _State) ->
rabbit_log_connection:error("STOMP detected network error on ~ts: ~tp",
- [ConnStr, Reason]);
+ [ConnName, Reason]);
log_reason({network_error, Reason}, _State) ->
rabbit_log_connection:error("STOMP detected network error: ~tp", [Reason]);
@@ -355,15 +359,15 @@ log_reason(Reason, #reader_state{processor_state = ProcState}) ->
rabbit_log_connection:warning("STOMP connection ~ts terminated"
" with reason ~tp, closing it", [AdapterName, Reason]).
-log_tls_alert(handshake_failure, ConnStr) ->
+log_tls_alert(handshake_failure, ConnName) ->
rabbit_log_connection:error("STOMP detected TLS upgrade error on ~ts: handshake failure",
- [ConnStr]);
-log_tls_alert(unknown_ca, ConnStr) ->
+ [ConnName]);
+log_tls_alert(unknown_ca, ConnName) ->
rabbit_log_connection:error("STOMP detected TLS certificate verification error on ~ts: alert 'unknown CA'",
- [ConnStr]);
-log_tls_alert(Alert, ConnStr) ->
+ [ConnName]);
+log_tls_alert(Alert, ConnName) ->
rabbit_log_connection:error("STOMP detected TLS upgrade error on ~ts: alert ~ts",
- [ConnStr, Alert]).
+ [ConnName, Alert]).
%%----------------------------------------------------------------------------
diff --git a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl
index 06d8ce3b9794..39ab2f0d12d6 100644
--- a/deps/rabbitmq_stream/src/rabbit_stream_reader.erl
+++ b/deps/rabbitmq_stream/src/rabbit_stream_reader.erl
@@ -82,7 +82,7 @@
heartbeater :: any(),
client_properties = #{} :: #{binary() => binary()},
monitors = #{} :: #{reference() => stream()},
- stats_timer :: undefined | reference(),
+ stats_timer :: undefined | rabbit_event:state(),
resource_alarm :: boolean(),
send_file_oct ::
atomics:atomics_ref(), % number of bytes sent with send_file (for metrics)
@@ -119,7 +119,6 @@
ssl_key_exchange,
ssl_cipher,
ssl_hash,
- protocol,
user,
vhost,
protocol,
@@ -551,11 +550,11 @@ invalid_transition(Transport, Socket, From, To) ->
close_immediately(Transport, Socket),
stop.
-resource_alarm(ConnectionPid, disk,
- {_WasAlarmSetForNode,
- IsThereAnyAlarmsForSameResourceInTheCluster, _Node}) ->
- ConnectionPid
- ! {resource_alarm, IsThereAnyAlarmsForSameResourceInTheCluster},
+-spec resource_alarm(pid(),
+ rabbit_alarm:resource_alarm_source(),
+ rabbit_alarm:resource_alert()) -> ok.
+resource_alarm(ConnectionPid, disk, {_, Conserve, _}) ->
+ ConnectionPid ! {resource_alarm, Conserve},
ok;
resource_alarm(_ConnectionPid, _Resource, _Alert) ->
ok.
@@ -925,6 +924,14 @@ open(info, emit_stats,
StatemData) ->
Connection1 = emit_stats(Connection, State),
{keep_state, StatemData#statem_data{connection = Connection1}};
+open(info, {shutdown, Explanation} = Reason,
+ #statem_data{connection = Connection}) ->
+ %% rabbitmq_management or rabbitmq_stream_management plugin
+ %% requests to close connection.
+ rabbit_log_connection:info("Forcing stream connection ~tp closing: ~tp",
+ [self(), Explanation]),
+ _ = demonitor_all_streams(Connection),
+ {stop, Reason};
open(info, Unknown, _StatemData) ->
rabbit_log_connection:warning("Received unknown message ~tp in state ~ts",
[Unknown, ?FUNCTION_NAME]),
@@ -944,13 +951,6 @@ open({call, From}, {publishers_info, Items},
#statem_data{connection = Connection}) ->
{keep_state_and_data,
{reply, From, publishers_infos(Items, Connection)}};
-open({call, From}, {shutdown, Explanation},
- #statem_data{connection = Connection}) ->
- % likely closing call from the management plugin
- rabbit_log_connection:info("Forcing stream connection ~tp closing: ~tp",
- [self(), Explanation]),
- _ = demonitor_all_streams(Connection),
- {stop_and_reply, normal, {reply, From, ok}};
open(cast,
{queue_event, _, {osiris_written, _, undefined, CorrelationList}},
#statem_data{transport = Transport,
@@ -1140,7 +1140,10 @@ open(cast, {force_event_refresh, Ref},
rabbit_event:init_stats_timer(Connection,
#stream_connection.stats_timer),
Connection2 = ensure_stats_timer(Connection1),
- {keep_state, StatemData#statem_data{connection = Connection2}}.
+ {keep_state, StatemData#statem_data{connection = Connection2}};
+open(cast, refresh_config, _StatemData) ->
+ %% tracing not supported
+ keep_state_and_data.
close_sent(enter, _OldState,
#statem_data{config =
@@ -3565,23 +3568,18 @@ i(host, #stream_connection{host = Host}, _) ->
Host;
i(peer_host, #stream_connection{peer_host = PeerHost}, _) ->
PeerHost;
-i(ssl, #stream_connection{socket = Socket, proxy_socket = ProxySock},
- _) ->
- rabbit_net:proxy_ssl_info(Socket, ProxySock) /= nossl;
-i(peer_cert_subject, S, _) ->
- cert_info(fun rabbit_ssl:peer_cert_subject/1, S);
-i(peer_cert_issuer, S, _) ->
- cert_info(fun rabbit_ssl:peer_cert_issuer/1, S);
-i(peer_cert_validity, S, _) ->
- cert_info(fun rabbit_ssl:peer_cert_validity/1, S);
-i(ssl_protocol, S, _) ->
- ssl_info(fun({P, _}) -> P end, S);
-i(ssl_key_exchange, S, _) ->
- ssl_info(fun({_, {K, _, _}}) -> K end, S);
-i(ssl_cipher, S, _) ->
- ssl_info(fun({_, {_, C, _}}) -> C end, S);
-i(ssl_hash, S, _) ->
- ssl_info(fun({_, {_, _, H}}) -> H end, S);
+i(SSL, #stream_connection{socket = Sock, proxy_socket = ProxySock}, _)
+ when SSL =:= ssl;
+ SSL =:= ssl_protocol;
+ SSL =:= ssl_key_exchange;
+ SSL =:= ssl_cipher;
+ SSL =:= ssl_hash ->
+ rabbit_ssl:info(SSL, {Sock, ProxySock});
+i(Cert, #stream_connection{socket = Sock},_)
+ when Cert =:= peer_cert_issuer;
+ Cert =:= peer_cert_subject;
+ Cert =:= peer_cert_validity ->
+ rabbit_ssl:cert_info(Cert, Sock);
i(channels, _, _) ->
0;
i(protocol, _, _) ->
@@ -3623,32 +3621,6 @@ i(_Unknown, _, _) ->
send(Transport, Socket, Data) when is_atom(Transport) ->
Transport:send(Socket, Data).
-cert_info(F, #stream_connection{socket = Sock}) ->
- case rabbit_net:peercert(Sock) of
- nossl ->
- '';
- {error, _} ->
- '';
- {ok, Cert} ->
- list_to_binary(F(Cert))
- end.
-
-ssl_info(F,
- #stream_connection{socket = Sock, proxy_socket = ProxySock}) ->
- case rabbit_net:proxy_ssl_info(Sock, ProxySock) of
- nossl ->
- '';
- {error, _} ->
- '';
- {ok, Items} ->
- P = proplists:get_value(protocol, Items),
- #{cipher := C,
- key_exchange := K,
- mac := H} =
- proplists:get_value(selected_cipher_suite, Items),
- F({P, {K, C, H}})
- end.
-
get_chunk_selector(Properties) ->
binary_to_atom(maps:get(<<"chunk_selector">>, Properties,
<<"user_data">>)).
diff --git a/deps/rabbitmq_stream_management/src/rabbit_stream_connection_mgmt.erl b/deps/rabbitmq_stream_management/src/rabbit_stream_connection_mgmt.erl
index db1a3e4a518d..22483391a7b4 100644
--- a/deps/rabbitmq_stream_management/src/rabbit_stream_connection_mgmt.erl
+++ b/deps/rabbitmq_stream_management/src/rabbit_stream_connection_mgmt.erl
@@ -156,5 +156,5 @@ force_close_connection(ReqData, Pid) ->
V ->
binary_to_list(V)
end,
- gen_server:call(Pid, {shutdown, Reason}, infinity),
+ Pid ! {shutdown, Reason},
ok.
diff --git a/deps/rabbitmq_web_mqtt/BUILD.bazel b/deps/rabbitmq_web_mqtt/BUILD.bazel
index 7210fd0ec120..6ac2d1251a5b 100644
--- a/deps/rabbitmq_web_mqtt/BUILD.bazel
+++ b/deps/rabbitmq_web_mqtt/BUILD.bazel
@@ -34,11 +34,11 @@ BUILD_DEPS = [
DEPS = [
"//deps/rabbit_common:erlang_app",
"@cowboy//:erlang_app",
+ "//deps/rabbitmq_mqtt:erlang_app",
]
RUNTIME_DEPS = [
"//deps/rabbit:erlang_app",
- "//deps/rabbitmq_mqtt:erlang_app",
]
rabbitmq_app(
@@ -64,12 +64,15 @@ dialyze(
plt = ":base_plt",
)
-broker_for_integration_suites()
+broker_for_integration_suites(
+ extra_plugins = [
+ "//deps/rabbitmq_management:erlang_app",
+ ],
+)
rabbitmq_test_helper(
name = "test_utils",
srcs = glob(["test/src/*.erl"]),
- hdrs = glob(["test/src/*.hrl"]),
)
rabbitmq_integration_suite(
@@ -79,7 +82,6 @@ rabbitmq_integration_suite(
rabbitmq_integration_suite(
name = "proxy_protocol_SUITE",
additional_beam = [":test_utils"],
- additional_hdrs = glob(["test/src/*.hrl"]),
erlc_opts = [
"-I",
"deps/rabbitmq_web_mqtt/test",
@@ -89,7 +91,12 @@ rabbitmq_integration_suite(
rabbitmq_integration_suite(
name = "system_SUITE",
additional_beam = [":test_utils"],
- additional_hdrs = glob(["test/src/*.hrl"]),
+ runtime_deps = [
+ "@emqtt//:erlang_app",
+ "@gun//:erlang_app",
+ "@cowlib//:erlang_app",
+ "//deps/rabbitmq_management_agent:erlang_app",
+ ],
)
assert_suites()
diff --git a/deps/rabbitmq_web_mqtt/Makefile b/deps/rabbitmq_web_mqtt/Makefile
index e31f164954af..25405eae04c3 100644
--- a/deps/rabbitmq_web_mqtt/Makefile
+++ b/deps/rabbitmq_web_mqtt/Makefile
@@ -13,13 +13,19 @@ define PROJECT_ENV
]
endef
+# We do not need QUIC as dependency of emqtt.
+BUILD_WITHOUT_QUIC=1
+export BUILD_WITHOUT_QUIC
+
DEPS = rabbit_common rabbit cowboy rabbitmq_mqtt
-TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers
+TEST_DEPS = emqtt rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_management
# FIXME: Add Ranch as a BUILD_DEPS to be sure the correct version is picked.
# See rabbitmq-components.mk.
BUILD_DEPS += ranch
+dep_emqtt = git https://github.com/emqx/emqtt.git 1.7.0-rc.2
+
DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-plugin.mk
DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk
diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl
index a30295756318..707012820519 100644
--- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl
+++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_app.erl
@@ -12,8 +12,7 @@
start/2,
prep_stop/1,
stop/1,
- list_connections/0,
- close_all_client_connections/1
+ list_connections/0
]).
%% Dummy supervisor - see Ulf Wiger's comment at
@@ -51,15 +50,8 @@ init([]) -> {ok, {{one_for_one, 1, 5}, []}}.
list_connections() ->
PlainPids = connection_pids_of_protocol(?TCP_PROTOCOL),
TLSPids = connection_pids_of_protocol(?TLS_PROTOCOL),
-
PlainPids ++ TLSPids.
--spec close_all_client_connections(string()) -> {'ok', non_neg_integer()}.
-close_all_client_connections(Reason) ->
- Connections = list_connections(),
- [rabbit_web_mqtt_handler:close_connection(Pid, Reason) || Pid <- Connections],
- {ok, length(Connections)}.
-
%%
%% Implementation
%%
@@ -80,14 +72,14 @@ cowboy_ws_connection_pid(RanchConnPid) ->
mqtt_init() ->
CowboyOpts0 = maps:from_list(get_env(cowboy_opts, [])),
CowboyWsOpts = maps:from_list(get_env(cowboy_ws_opts, [])),
-
Routes = cowboy_router:compile([{'_', [
{get_env(ws_path, "/ws"), rabbit_web_mqtt_handler, [{ws_opts, CowboyWsOpts}]}
]}]),
- CowboyOpts = CowboyOpts0#{env => #{dispatch => Routes},
- middlewares => [cowboy_router, rabbit_web_mqtt_middleware, cowboy_handler],
- proxy_header => get_env(proxy_protocol, false),
- stream_handlers => [rabbit_web_mqtt_stream_handler, cowboy_stream_h]},
+ CowboyOpts = CowboyOpts0#{
+ env => #{dispatch => Routes},
+ proxy_header => get_env(proxy_protocol, false),
+ stream_handlers => [rabbit_web_mqtt_stream_handler, cowboy_stream_h]
+ },
case get_env(tcp_config, []) of
[] -> ok;
TCPConf0 -> start_tcp_listener(TCPConf0, CowboyOpts)
@@ -99,59 +91,55 @@ mqtt_init() ->
ok.
start_tcp_listener(TCPConf0, CowboyOpts) ->
- {TCPConf, IpStr, Port} = get_tcp_conf(TCPConf0),
- RanchTransportOpts = #{
- socket_opts => TCPConf,
- connection_type => supervisor,
- max_connections => get_max_connections(),
- num_acceptors => get_env(num_tcp_acceptors, 10),
- num_conns_sups => get_env(num_conns_sup, 1)
- },
- case ranch:start_listener(rabbit_networking:ranch_ref(TCPConf),
- ranch_tcp,
- RanchTransportOpts,
- rabbit_web_mqtt_connection_sup,
- CowboyOpts) of
- {ok, _} -> ok;
- {error, {already_started, _}} -> ok;
- {error, ErrTCP} ->
- rabbit_log_connection:error(
- "Failed to start a WebSocket (HTTP) listener. Error: ~tp,"
- " listener settings: ~tp",
+ {TCPConf, IpStr, Port} = get_tcp_conf(TCPConf0),
+ RanchRef = rabbit_networking:ranch_ref(TCPConf),
+ RanchTransportOpts =
+ #{
+ socket_opts => TCPConf,
+ max_connections => get_max_connections(),
+ num_acceptors => get_env(num_tcp_acceptors, 10),
+ num_conns_sups => get_env(num_conns_sup, 1)
+ },
+ case cowboy:start_clear(RanchRef, RanchTransportOpts, CowboyOpts) of
+ {ok, _} ->
+ ok;
+ {error, {already_started, _}} ->
+ ok;
+ {error, ErrTCP} ->
+ rabbit_log:error(
+ "Failed to start a WebSocket (HTTP) listener. Error: ~p, listener settings: ~p",
[ErrTCP, TCPConf]),
- throw(ErrTCP)
- end,
- listener_started(?TCP_PROTOCOL, TCPConf),
- rabbit_log:info("rabbit_web_mqtt: listening for HTTP connections on ~ts:~w",
- [IpStr, Port]).
+ throw(ErrTCP)
+ end,
+ listener_started(?TCP_PROTOCOL, TCPConf),
+ rabbit_log:info("rabbit_web_mqtt: listening for HTTP connections on ~s:~w",
+ [IpStr, Port]).
start_tls_listener(TLSConf0, CowboyOpts) ->
- _ = rabbit_networking:ensure_ssl(),
- {TLSConf, TLSIpStr, TLSPort} = get_tls_conf(TLSConf0),
- RanchTransportOpts = #{
- socket_opts => TLSConf,
- connection_type => supervisor,
- max_connections => get_max_connections(),
- num_acceptors => get_env(num_ssl_acceptors, 10),
- num_conns_sups => get_env(num_conns_sup, 1)
- },
- case ranch:start_listener(rabbit_networking:ranch_ref(TLSConf),
- ranch_ssl,
- RanchTransportOpts,
- rabbit_web_mqtt_connection_sup,
- CowboyOpts) of
- {ok, _} -> ok;
- {error, {already_started, _}} -> ok;
- {error, ErrTLS} ->
- rabbit_log_connection:error(
- "Failed to start a TLS WebSocket (HTTPS) listener. Error: ~tp,"
- " listener settings: ~tp",
+ _ = rabbit_networking:ensure_ssl(),
+ {TLSConf, TLSIpStr, TLSPort} = get_tls_conf(TLSConf0),
+ RanchRef = rabbit_networking:ranch_ref(TLSConf),
+ RanchTransportOpts =
+ #{
+ socket_opts => TLSConf,
+ max_connections => get_max_connections(),
+ num_acceptors => get_env(num_ssl_acceptors, 10),
+ num_conns_sups => get_env(num_conns_sup, 1)
+ },
+ case cowboy:start_tls(RanchRef, RanchTransportOpts, CowboyOpts) of
+ {ok, _} ->
+ ok;
+ {error, {already_started, _}} ->
+ ok;
+ {error, ErrTLS} ->
+ rabbit_log:error(
+ "Failed to start a TLS WebSocket (HTTPS) listener. Error: ~p, listener settings: ~p",
[ErrTLS, TLSConf]),
- throw(ErrTLS)
- end,
- listener_started(?TLS_PROTOCOL, TLSConf),
- rabbit_log:info("rabbit_web_mqtt: listening for HTTPS connections on ~ts:~w",
- [TLSIpStr, TLSPort]).
+ throw(ErrTLS)
+ end,
+ listener_started(?TLS_PROTOCOL, TLSConf),
+ rabbit_log:info("rabbit_web_mqtt: listening for HTTPS connections on ~s:~w",
+ [TLSIpStr, TLSPort]).
listener_started(Protocol, Listener) ->
Port = rabbit_misc:pget(port, Listener),
diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_connection_info.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_connection_info.erl
deleted file mode 100644
index 9e0c17d1713b..000000000000
--- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_connection_info.erl
+++ /dev/null
@@ -1,26 +0,0 @@
-%% This Source Code Form is subject to the terms of the Mozilla Public
-%% License, v. 2.0. If a copy of the MPL was not distributed with this
-%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
-%%
-%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
-%%
-
--module(rabbit_web_mqtt_connection_info).
-
-%% Module to add the MQTT client ID to authentication properties
-
-%% API
--export([additional_authn_params/4]).
-
-additional_authn_params(_Creds, _VHost, _Pid, Infos) ->
- case proplists:get_value(variable_map, Infos, undefined) of
- VariableMap when is_map(VariableMap) ->
- case maps:get(<<"client_id">>, VariableMap, []) of
- ClientId when is_binary(ClientId)->
- [{client_id, ClientId}];
- [] ->
- []
- end;
- _ ->
- []
- end.
diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_connection_sup.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_connection_sup.erl
deleted file mode 100644
index c37488ecf4f5..000000000000
--- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_connection_sup.erl
+++ /dev/null
@@ -1,75 +0,0 @@
-%% This Source Code Form is subject to the terms of the Mozilla Public
-%% License, v. 2.0. If a copy of the MPL was not distributed with this
-%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
-%%
-%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
-%%
-
--module(rabbit_web_mqtt_connection_sup).
-
--behaviour(supervisor).
--behaviour(ranch_protocol).
-
--include_lib("rabbit_common/include/rabbit.hrl").
-
--export([start_link/3, start_keepalive_link/0]).
-
--export([init/1]).
-
-%%----------------------------------------------------------------------------
-
-start_link(Ref, Transport, CowboyOpts0) ->
- {ok, SupPid} = supervisor:start_link(?MODULE, []),
- {ok, KeepaliveSup} = supervisor:start_child(
- SupPid,
- #{
- id => rabbit_web_mqtt_keepalive_sup,
- start => {?MODULE, start_keepalive_link, []},
- restart => transient,
- significant => true,
- shutdown => infinity,
- type => supervisor,
- modules => [rabbit_keepalive_sup]
- }
- ),
-
- %% In order for the Websocket handler to receive the KeepaliveSup
- %% variable, we need to pass it first through the environment and
- %% then have the middleware rabbit_web_mqtt_middleware place it
- %% in the initial handler state.
- Env = maps:get(env, CowboyOpts0),
- CowboyOpts = CowboyOpts0#{env => Env#{keepalive_sup => KeepaliveSup}},
- Protocol = case Transport of
- ranch_tcp -> cowboy_clear;
- ranch_ssl -> cowboy_tls
- end,
- {ok, ReaderPid} = supervisor:start_child(
- SupPid,
- #{
- id => Protocol,
- start => {Protocol, start_link, [Ref, Transport, CowboyOpts]},
- restart => transient,
- significant => true,
- shutdown => ?WORKER_WAIT,
- type => worker,
- modules => [Protocol]
- }
- ),
- {ok, SupPid, ReaderPid}.
-
-start_keepalive_link() ->
- supervisor:start_link(?MODULE, []).
-
-%%----------------------------------------------------------------------------
-
-init([]) ->
- {ok,
- {
- #{
- strategy => one_for_all,
- intensity => 0,
- period => 1,
- auto_shutdown => any_significant
- },
- []
- }}.
diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl
index bc1b9abfbbbf..72f52b3977af 100644
--- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl
+++ b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_handler.erl
@@ -9,6 +9,10 @@
-behaviour(cowboy_websocket).
-behaviour(cowboy_sub_protocol).
+-include_lib("kernel/include/logger.hrl").
+-include_lib("rabbit_common/include/logging.hrl").
+-include_lib("rabbitmq_mqtt/include/rabbit_mqtt.hrl").
+
-export([
init/2,
websocket_init/1,
@@ -16,28 +20,35 @@
websocket_info/2,
terminate/3
]).
--export([close_connection/2]).
+
+-export([conserve_resources/3]).
%% cowboy_sub_protocol
-export([upgrade/4,
upgrade/5,
takeover/7]).
--include_lib("amqp_client/include/amqp_client.hrl").
+-type option(T) :: undefined | T.
-record(state, {
- conn_name,
- keepalive,
- keepalive_sup,
- parse_state,
- proc_state,
- state,
- conserve_resources,
- socket,
- peername,
- stats_timer,
- connection
-}).
+ socket :: {rabbit_proxy_socket, any(), any()} | rabbit_net:socket(),
+ parse_state = rabbit_mqtt_packet:initial_state() :: rabbit_mqtt_packet:state(),
+ proc_state :: undefined | rabbit_mqtt_processor:state(),
+ connection_state = running :: running | blocked,
+ conserve = false :: boolean(),
+ stats_timer :: option(rabbit_event:state()),
+ keepalive = rabbit_mqtt_keepalive:init() :: rabbit_mqtt_keepalive:state(),
+ conn_name :: option(binary()),
+ received_connect_packet = false :: boolean()
+ }).
+
+-type state() :: #state{}.
+
+%% Close frame status codes as defined in https://www.rfc-editor.org/rfc/rfc6455#section-7.4.1
+-define(CLOSE_NORMAL, 1000).
+-define(CLOSE_PROTOCOL_ERROR, 1002).
+-define(CLOSE_UNACCEPTABLE_DATA_TYPE, 1003).
+-define(PROTO_FAMILY, 'Web MQTT').
%% cowboy_sub_protcol
upgrade(Req, Env, Handler, HandlerState) ->
@@ -46,7 +57,7 @@ upgrade(Req, Env, Handler, HandlerState) ->
upgrade(Req, Env, Handler, HandlerState, Opts) ->
cowboy_websocket:upgrade(Req, Env, Handler, HandlerState, Opts).
-takeover(Parent, Ref, Socket, Transport, Opts, Buffer, {Handler, HandlerState}) ->
+takeover(Parent, Ref, Socket, Transport, Opts, Buffer, {Handler, {HandlerState, PeerAddr}}) ->
Sock = case HandlerState#state.socket of
undefined ->
Socket;
@@ -54,250 +65,348 @@ takeover(Parent, Ref, Socket, Transport, Opts, Buffer, {Handler, HandlerState})
{rabbit_proxy_socket, Socket, ProxyInfo}
end,
cowboy_websocket:takeover(Parent, Ref, Socket, Transport, Opts, Buffer,
- {Handler, HandlerState#state{socket = Sock}}).
+ {Handler, {HandlerState#state{socket = Sock}, PeerAddr}}).
%% cowboy_websocket
init(Req, Opts) ->
- {PeerAddr, _PeerPort} = maps:get(peer, Req),
- {_, KeepaliveSup} = lists:keyfind(keepalive_sup, 1, Opts),
- SockInfo = maps:get(proxy_header, Req, undefined),
- WsOpts0 = proplists:get_value(ws_opts, Opts, #{}),
- WsOpts = maps:merge(#{compress => true}, WsOpts0),
- Req2 = case cowboy_req:header(<<"sec-websocket-protocol">>, Req) of
- undefined -> Req;
- SecWsProtocol ->
- cowboy_req:set_resp_header(<<"sec-websocket-protocol">>, SecWsProtocol, Req)
- end,
- {?MODULE, Req2, #state{
- keepalive = {none, none},
- keepalive_sup = KeepaliveSup,
- parse_state = rabbit_mqtt_frame:initial_state(),
- state = running,
- conserve_resources = false,
- socket = SockInfo,
- peername = PeerAddr
- }, WsOpts}.
-
-websocket_init(State0 = #state{socket = Sock, peername = PeerAddr}) ->
+ case cowboy_req:parse_header(<<"sec-websocket-protocol">>, Req) of
+ undefined ->
+ no_supported_sub_protocol(undefined, Req);
+ Protocol ->
+ {PeerAddr, _PeerPort} = maps:get(peer, Req),
+ WsOpts0 = proplists:get_value(ws_opts, Opts, #{}),
+ WsOpts = maps:merge(#{compress => true}, WsOpts0),
+ case lists:member(<<"mqtt">>, Protocol) of
+ false ->
+ no_supported_sub_protocol(Protocol, Req);
+ true ->
+ {?MODULE,
+ cowboy_req:set_resp_header(<<"sec-websocket-protocol">>, <<"mqtt">>, Req),
+ {#state{socket = maps:get(proxy_header, Req, undefined)},
+ PeerAddr},
+ WsOpts}
+ end
+ end.
+
+-spec websocket_init({state(), PeerAddr :: binary()}) ->
+ {cowboy_websocket:commands(), state()} |
+ {cowboy_websocket:commands(), state(), hibernate}.
+websocket_init({State0 = #state{socket = Sock}, PeerAddr}) ->
+ logger:set_process_metadata(#{domain => ?RMQLOG_DOMAIN_CONN ++ [web_mqtt]}),
ok = file_handle_cache:obtain(),
case rabbit_net:connection_string(Sock, inbound) of
{ok, ConnStr} ->
- State = State0#state{
- conn_name = ConnStr,
- socket = Sock
- },
- rabbit_log_connection:info("accepting Web MQTT connection ~tp (~ts)", [self(), ConnStr]),
- AdapterInfo = amqp_connection:socket_adapter_info(Sock, {'Web MQTT', "N/A"}),
- RealSocket = rabbit_net:unwrap_socket(Sock),
- ProcessorState = rabbit_mqtt_processor:initial_state(Sock,
- rabbit_mqtt_reader:ssl_login_name(RealSocket),
- AdapterInfo,
- fun send_reply/2,
- PeerAddr),
+ ConnName = rabbit_data_coercion:to_binary(ConnStr),
+ ?LOG_INFO("Accepting Web MQTT connection ~s", [ConnName]),
+ _ = rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}),
+ PState = rabbit_mqtt_processor:initial_state(
+ rabbit_net:unwrap_socket(Sock),
+ ConnName,
+ fun send_reply/2,
+ PeerAddr),
+ State1 = State0#state{conn_name = ConnName,
+ proc_state = PState},
+ State = rabbit_event:init_stats_timer(State1, #state.stats_timer),
process_flag(trap_exit, true),
- {ok,
- rabbit_event:init_stats_timer(
- State#state{proc_state = ProcessorState},
- #state.stats_timer),
- hibernate};
- _ ->
- {stop, State0}
+ {[], State, hibernate};
+ {error, Reason} ->
+ {[{shutdown_reason, Reason}], State0}
end.
--spec close_connection(pid(), string()) -> 'ok'.
-close_connection(Pid, Reason) ->
- rabbit_log_connection:info("Web MQTT: will terminate connection process ~tp, reason: ~ts",
- [Pid, Reason]),
- sys:terminate(Pid, Reason),
+-spec conserve_resources(pid(),
+ rabbit_alarm:resource_alarm_source(),
+ rabbit_alarm:resource_alert()) -> ok.
+conserve_resources(Pid, _, {_, Conserve, _}) ->
+ Pid ! {conserve_resources, Conserve},
ok.
+-spec websocket_handle(ping | pong | {text | binary | ping | pong, binary()}, State) ->
+ {cowboy_websocket:commands(), State} |
+ {cowboy_websocket:commands(), State, hibernate}.
websocket_handle({binary, Data}, State) ->
handle_data(Data, State);
-%% Silently ignore ping and pong frames.
-websocket_handle({Ping, _}, State) when Ping =:= ping; Ping =:= pong ->
- {ok, State, hibernate};
-websocket_handle(Ping, State) when Ping =:= ping; Ping =:= pong ->
- {ok, State, hibernate};
-%% Log any other unexpected frames.
+%% Silently ignore ping and pong frames as Cowboy will automatically reply to ping frames.
+websocket_handle({Ping, _}, State)
+ when Ping =:= ping orelse Ping =:= pong ->
+ {[], State, hibernate};
+websocket_handle(Ping, State)
+ when Ping =:= ping orelse Ping =:= pong ->
+ {[], State, hibernate};
+%% Log and close connection when receiving any other unexpected frames.
websocket_handle(Frame, State) ->
- rabbit_log_connection:info("Web MQTT: unexpected WebSocket frame ~tp",
- [Frame]),
- {ok, State, hibernate}.
-
-%% `rabbit_mqtt_processor:amqp_callback/2` doesn't actually return
-%% {'error', _, _}, so this small function is a place to silence
-%% unmatched warning. This allows to keep currently-unused
-%% error-handling code.
--spec callback_reply(#state{}, {'ok', any()} | {'error', any(), any()}) -> {'ok', #state{}, 'hibernate'}.
--dialyzer({no_match, callback_reply/2}).
-callback_reply(State, {ok, ProcState}) ->
- {ok, State #state { proc_state = ProcState }, hibernate};
-callback_reply(State, {error, _Reason, _ProcState}) ->
- stop(State).
+ ?LOG_INFO("Web MQTT: unexpected WebSocket frame ~tp", [Frame]),
+ stop(State, ?CLOSE_UNACCEPTABLE_DATA_TYPE, <<"unexpected WebSocket frame">>).
+-spec websocket_info(any(), State) ->
+ {cowboy_websocket:commands(), State} |
+ {cowboy_websocket:commands(), State, hibernate}.
websocket_info({conserve_resources, Conserve}, State) ->
- NewState = State#state{conserve_resources = Conserve},
- handle_credits(control_throttle(NewState));
+ handle_credits(State#state{conserve = Conserve});
websocket_info({bump_credit, Msg}, State) ->
credit_flow:handle_bump_msg(Msg),
- handle_credits(control_throttle(State));
-websocket_info({#'basic.deliver'{}, #amqp_msg{}, _DeliveryCtx} = Delivery,
- State = #state{ proc_state = ProcState0 }) ->
- callback_reply(State, rabbit_mqtt_processor:amqp_callback(Delivery, ProcState0));
-websocket_info(#'basic.ack'{} = Ack, State = #state{ proc_state = ProcState0 }) ->
- callback_reply(State, rabbit_mqtt_processor:amqp_callback(Ack, ProcState0));
-websocket_info(#'basic.consume_ok'{}, State) ->
- {ok, State, hibernate};
-websocket_info(#'basic.cancel'{}, State) ->
- stop(State);
+ handle_credits(State);
websocket_info({reply, Data}, State) ->
- {reply, {binary, Data}, State, hibernate};
+ {[{binary, Data}], State, hibernate};
websocket_info({'EXIT', _, _}, State) ->
stop(State);
+websocket_info({'$gen_cast', QueueEvent = {queue_event, _, _}},
+ State = #state{proc_state = PState0}) ->
+ case rabbit_mqtt_processor:handle_queue_event(QueueEvent, PState0) of
+ {ok, PState} ->
+ handle_credits(State#state{proc_state = PState});
+ {error, Reason, PState} ->
+ ?LOG_ERROR("Web MQTT connection ~p failed to handle queue event: ~p",
+ [State#state.conn_name, Reason]),
+ stop(State#state{proc_state = PState})
+ end;
websocket_info({'$gen_cast', duplicate_id}, State = #state{ proc_state = ProcState,
- conn_name = ConnName }) ->
- rabbit_log_connection:warning("Web MQTT disconnecting a client with duplicate ID '~ts' (~tp)",
+ conn_name = ConnName }) ->
+ ?LOG_WARNING("Web MQTT disconnecting a client with duplicate ID '~s' (~p)",
[rabbit_mqtt_processor:info(client_id, ProcState), ConnName]),
stop(State);
websocket_info({'$gen_cast', {close_connection, Reason}}, State = #state{ proc_state = ProcState,
- conn_name = ConnName }) ->
- rabbit_log_connection:warning("Web MQTT disconnecting client with ID '~ts' (~tp), reason: ~ts",
+ conn_name = ConnName }) ->
+ ?LOG_WARNING("Web MQTT disconnecting client with ID '~s' (~p), reason: ~s",
[rabbit_mqtt_processor:info(client_id, ProcState), ConnName, Reason]),
stop(State);
-websocket_info({start_keepalives, Keepalive},
- State = #state{ socket = Sock, keepalive_sup = KeepaliveSup }) ->
- %% Only the client has the responsibility for sending keepalives
- SendFun = fun() -> ok end,
- Parent = self(),
- ReceiveFun = fun() -> Parent ! keepalive_timeout end,
- Heartbeater = rabbit_heartbeat:start(
- KeepaliveSup, Sock, 0, SendFun, Keepalive, ReceiveFun),
- {ok, State #state { keepalive = Heartbeater }, hibernate};
-websocket_info(keepalive_timeout, State = #state{conn_name = ConnStr}) ->
- rabbit_log_connection:error("closing Web MQTT connection ~tp (keepalive timeout)", [ConnStr]),
- stop(State);
+websocket_info({'$gen_cast', {force_event_refresh, Ref}}, State0) ->
+ Infos = infos(?CREATION_EVENT_KEYS, State0),
+ rabbit_event:notify(connection_created, Infos, Ref),
+ State = rabbit_event:init_stats_timer(State0, #state.stats_timer),
+ {[], State, hibernate};
+websocket_info({'$gen_cast', refresh_config},
+ State0 = #state{proc_state = PState0,
+ conn_name = ConnName}) ->
+ PState = rabbit_mqtt_processor:update_trace(ConnName, PState0),
+ State = State0#state{proc_state = PState},
+ {[], State, hibernate};
+websocket_info({keepalive, Req}, State = #state{keepalive = KState0,
+ conn_name = ConnName}) ->
+ case rabbit_mqtt_keepalive:handle(Req, KState0) of
+ {ok, KState} ->
+ {[], State#state{keepalive = KState}, hibernate};
+ {error, timeout} ->
+ ?LOG_ERROR("keepalive timeout in Web MQTT connection ~p", [ConnName]),
+ stop(State, ?CLOSE_NORMAL, <<"MQTT keepalive timeout">>);
+ {error, Reason} ->
+ ?LOG_ERROR("keepalive error in Web MQTT connection ~p: ~p",
+ [ConnName, Reason]),
+ stop(State)
+ end;
websocket_info(emit_stats, State) ->
- {ok, emit_stats(State), hibernate};
-websocket_info({ra_event, _, _}, State) ->
- {ok, State, hibernate};
+ {[], emit_stats(State), hibernate};
+websocket_info({ra_event, _From, Evt},
+ #state{proc_state = PState0} = State) ->
+ PState = rabbit_mqtt_processor:handle_ra_event(Evt, PState0),
+ {[], State#state{proc_state = PState}, hibernate};
+websocket_info({{'DOWN', _QName}, _MRef, process, _Pid, _Reason} = Evt,
+ State = #state{proc_state = PState0}) ->
+ case rabbit_mqtt_processor:handle_down(Evt, PState0) of
+ {ok, PState} ->
+ handle_credits(State#state{proc_state = PState});
+ {error, Reason} ->
+ stop(State, ?CLOSE_NORMAL, Reason)
+ end;
+websocket_info({'DOWN', _MRef, process, QPid, _Reason}, State) ->
+ rabbit_amqqueue_common:notify_sent_queue_down(QPid),
+ {[], State, hibernate};
+websocket_info({shutdown, Reason}, #state{conn_name = ConnName} = State) ->
+ %% rabbitmq_management plugin requests to close connection.
+ ?LOG_INFO("Web MQTT closing connection ~tp: ~tp", [ConnName, Reason]),
+ stop(State, ?CLOSE_NORMAL, Reason);
+websocket_info(connection_created, State) ->
+ Infos = infos(?CREATION_EVENT_KEYS, State),
+ rabbit_core_metrics:connection_created(self(), Infos),
+ rabbit_event:notify(connection_created, Infos),
+ {[], State, hibernate};
websocket_info(Msg, State) ->
- rabbit_log_connection:info("Web MQTT: unexpected message ~tp",
- [Msg]),
- {ok, State, hibernate}.
+ ?LOG_WARNING("Web MQTT: unexpected message ~tp", [Msg]),
+ {[], State, hibernate}.
-terminate(_, _, #state{ proc_state = undefined }) ->
+terminate(_Reason, _Req, #state{proc_state = undefined}) ->
ok;
-terminate(_, _, State) ->
- _ = stop_rabbit_mqtt_processor(State),
- ok.
+terminate(Reason, Request, #state{} = State) ->
+ terminate(Reason, Request, {true, State});
+terminate(_Reason, _Request,
+ {SendWill, #state{conn_name = ConnName,
+ proc_state = PState,
+ keepalive = KState} = State}) ->
+ ?LOG_INFO("Web MQTT closing connection ~ts", [ConnName]),
+ maybe_emit_stats(State),
+ _ = rabbit_mqtt_keepalive:cancel_timer(KState),
+ ok = file_handle_cache:release(),
+ rabbit_mqtt_processor:terminate(SendWill, ConnName, ?PROTO_FAMILY, PState).
%% Internal.
-handle_data(Data, State0 = #state{conn_name = ConnStr}) ->
+no_supported_sub_protocol(Protocol, Req) ->
+ %% The client MUST include “mqtt” in the list of WebSocket Sub Protocols it offers [MQTT-6.0.0-3].
+ ?LOG_ERROR("Web MQTT: 'mqtt' not included in client offered subprotocols: ~tp", [Protocol]),
+ {ok, cowboy_req:reply(400, #{<<"connection">> => <<"close">>}, Req), #state{}}.
+
+handle_data(Data, State0 = #state{}) ->
case handle_data1(Data, State0) of
- {ok, State1 = #state{state = blocked}, hibernate} ->
+ {ok, State1 = #state{connection_state = blocked}, hibernate} ->
{[{active, false}], State1, hibernate};
- {error, Error} ->
- stop_with_framing_error(State0, Error, ConnStr);
Other ->
Other
end.
+handle_data1(<<>>, State0 = #state{received_connect_packet = false,
+ proc_state = PState,
+ conn_name = ConnName}) ->
+ ?LOG_INFO("Accepted web MQTT connection ~p (~s, client ID: ~s)",
+ [self(), ConnName, rabbit_mqtt_processor:info(client_id, PState)]),
+ State = State0#state{received_connect_packet = true},
+ {ok, ensure_stats_timer(control_throttle(State)), hibernate};
handle_data1(<<>>, State) ->
{ok, ensure_stats_timer(control_throttle(State)), hibernate};
handle_data1(Data, State = #state{ parse_state = ParseState,
- proc_state = ProcState,
- conn_name = ConnStr }) ->
- case rabbit_mqtt_frame:parse(Data, ParseState) of
+ proc_state = ProcState,
+ conn_name = ConnName }) ->
+ case parse(Data, ParseState) of
{more, ParseState1} ->
{ok, ensure_stats_timer(control_throttle(
State #state{ parse_state = ParseState1 })), hibernate};
- {ok, Frame, Rest} ->
- case rabbit_mqtt_processor:process_frame(Frame, ProcState) of
- {ok, ProcState1, ConnPid} ->
- PS = rabbit_mqtt_frame:initial_state(),
+ {ok, Packet, Rest} ->
+ case rabbit_mqtt_processor:process_packet(Packet, ProcState) of
+ {ok, ProcState1} ->
handle_data1(
Rest,
- State #state{ parse_state = PS,
- proc_state = ProcState1,
- connection = ConnPid });
+ State#state{parse_state = rabbit_mqtt_packet:initial_state(),
+ proc_state = ProcState1});
{error, Reason, _} ->
- rabbit_log_connection:info("MQTT protocol error ~tp for connection ~tp",
- [Reason, ConnStr]),
- stop(State, 1002, Reason);
- {stop, _} ->
- stop(State)
+ stop_mqtt_protocol_error(State, Reason, ConnName);
+ {stop, disconnect, ProcState1} ->
+ stop({_SendWill = false, State#state{proc_state = ProcState1}})
end;
- Other ->
- Other
+ {error, Reason} ->
+ stop_mqtt_protocol_error(State, Reason, ConnName)
end.
+parse(Data, ParseState) ->
+ try
+ rabbit_mqtt_packet:parse(Data, ParseState)
+ catch
+ _:Reason:Stacktrace ->
+ ?LOG_ERROR("Web MQTT cannot parse a packet, reason: ~tp, stacktrace: ~tp, "
+ "payload (first 100 bytes): ~tp",
+ [Reason, Stacktrace, rabbit_mqtt_util:truncate_binary(Data, 100)]),
+ {error, cannot_parse}
+ end.
+
+stop_mqtt_protocol_error(State, Reason, ConnName) ->
+ ?LOG_INFO("MQTT protocol error ~tp for connection ~tp", [Reason, ConnName]),
+ stop(State, ?CLOSE_PROTOCOL_ERROR, Reason).
+
stop(State) ->
- stop(State, 1000, "MQTT died").
+ stop(State, ?CLOSE_NORMAL, "MQTT died").
stop(State, CloseCode, Error0) ->
- ok = file_handle_cache:release(),
- _ = stop_rabbit_mqtt_processor(State),
- Error1 = rabbit_data_coercion:to_binary(Error0),
- {[{close, CloseCode, Error1}], State}.
-
-stop_with_framing_error(State, Error0, ConnStr) ->
- Error1 = rabbit_misc:format("~tp", [Error0]),
- rabbit_log_connection:error("MQTT detected framing error '~ts' for connection ~tp",
- [Error1, ConnStr]),
- stop(State, 1007, Error1).
-
-stop_rabbit_mqtt_processor(State = #state{state = running,
- proc_state = ProcState,
- conn_name = ConnName}) ->
- maybe_emit_stats(State),
- rabbit_log_connection:info("closing Web MQTT connection ~tp (~ts)", [self(), ConnName]),
- _ = rabbit_mqtt_processor:send_will(ProcState),
- rabbit_mqtt_processor:close_connection(ProcState).
+ Error = rabbit_data_coercion:to_binary(Error0),
+ {[{close, CloseCode, Error}], State}.
handle_credits(State0) ->
- case control_throttle(State0) of
- State = #state{state = running} ->
- {[{active, true}], State};
- State ->
- {ok, State}
- end.
+ State = #state{connection_state = CS} = control_throttle(State0),
+ Active = case CS of
+ running -> true;
+ blocked -> false
+ end,
+ {[{active, Active}], State, hibernate}.
-control_throttle(State = #state{ state = CS,
- conserve_resources = Mem }) ->
- case {CS, Mem orelse credit_flow:blocked()} of
- {running, true} -> ok = rabbit_heartbeat:pause_monitor(
- State#state.keepalive),
- State #state{ state = blocked };
- {blocked, false} -> ok = rabbit_heartbeat:resume_monitor(
- State#state.keepalive),
- State #state{ state = running };
- {_, _} -> State
+control_throttle(State = #state{connection_state = ConnState,
+ conserve = Conserve,
+ received_connect_packet = Connected,
+ proc_state = PState,
+ keepalive = KState
+ }) ->
+ Throttle = rabbit_mqtt_processor:throttle(Conserve, Connected, PState),
+ case {ConnState, Throttle} of
+ {running, true} ->
+ State#state{connection_state = blocked,
+ keepalive = rabbit_mqtt_keepalive:cancel_timer(KState)};
+ {blocked,false} ->
+ State#state{connection_state = running,
+ keepalive = rabbit_mqtt_keepalive:start_timer(KState)};
+ {_, _} ->
+ State
end.
-send_reply(Frame, _) ->
- self() ! {reply, rabbit_mqtt_frame:serialise(Frame)}.
+send_reply(Packet, PState) ->
+ self() ! {reply, rabbit_mqtt_processor:serialise(Packet, PState)}.
ensure_stats_timer(State) ->
rabbit_event:ensure_stats_timer(State, #state.stats_timer, emit_stats).
+maybe_emit_stats(#state{stats_timer = undefined}) ->
+ ok;
maybe_emit_stats(State) ->
rabbit_event:if_enabled(State, #state.stats_timer,
fun() -> emit_stats(State) end).
-emit_stats(State=#state{connection = C}) when C == none; C == undefined ->
+emit_stats(State=#state{received_connect_packet = false}) ->
%% Avoid emitting stats on terminate when the connection has not yet been
%% established, as this causes orphan entries on the stats database
+ rabbit_event:reset_stats_timer(State, #state.stats_timer);
+emit_stats(State) ->
+ [{_, Pid},
+ {_, RecvOct},
+ {_, SendOct},
+ {_, Reductions}] = infos(?SIMPLE_METRICS, State),
+ Infos = infos(?OTHER_METRICS, State),
+ rabbit_core_metrics:connection_stats(Pid, Infos),
+ rabbit_core_metrics:connection_stats(Pid, RecvOct, SendOct, Reductions),
State1 = rabbit_event:reset_stats_timer(State, #state.stats_timer),
- State1;
-emit_stats(State=#state{socket=Sock, state=RunningState, connection=Conn}) ->
- SockInfos = case rabbit_net:getstat(Sock,
- [recv_oct, recv_cnt, send_oct, send_cnt, send_pend]) of
- {ok, SI} -> SI;
- {error, _} -> []
- end,
- Infos = [{pid, Conn}, {state, RunningState}|SockInfos],
- rabbit_core_metrics:connection_stats(Conn, Infos),
- rabbit_event:notify(connection_stats, Infos),
- State1 = rabbit_event:reset_stats_timer(State, #state.stats_timer),
- State1.
+ ensure_stats_timer(State1).
+
+infos(Items, State) ->
+ [{Item, i(Item, State)} || Item <- Items].
+
+i(pid, _) ->
+ self();
+i(SockStat, #state{socket = Sock})
+ when SockStat =:= recv_oct;
+ SockStat =:= recv_cnt;
+ SockStat =:= send_oct;
+ SockStat =:= send_cnt;
+ SockStat =:= send_pend ->
+ case rabbit_net:getstat(Sock, [SockStat]) of
+ {ok, [{_, N}]} when is_number(N) ->
+ N;
+ _ ->
+ 0
+ end;
+i(reductions, _) ->
+ {reductions, Reductions} = erlang:process_info(self(), reductions),
+ Reductions;
+i(garbage_collection, _) ->
+ rabbit_misc:get_gc_info(self());
+i(protocol, #state{proc_state = PState}) ->
+ {?PROTO_FAMILY, rabbit_mqtt_processor:proto_version_tuple(PState)};
+i(SSL, #state{socket = Sock})
+ when SSL =:= ssl;
+ SSL =:= ssl_protocol;
+ SSL =:= ssl_key_exchange;
+ SSL =:= ssl_cipher;
+ SSL =:= ssl_hash ->
+ rabbit_ssl:info(SSL, {rabbit_net:unwrap_socket(Sock),
+ rabbit_net:maybe_get_proxy_socket(Sock)});
+i(name, S) ->
+ i(conn_name, S);
+i(conn_name, #state{conn_name = Val}) ->
+ Val;
+i(Cert, #state{socket = Sock})
+ when Cert =:= peer_cert_issuer;
+ Cert =:= peer_cert_subject;
+ Cert =:= peer_cert_validity ->
+ rabbit_ssl:cert_info(Cert, rabbit_net:unwrap_socket(Sock));
+i(state, S) ->
+ i(connection_state, S);
+i(connection_state, #state{received_connect_packet = false}) ->
+ starting;
+i(connection_state, #state{connection_state = Val}) ->
+ Val;
+i(timeout, #state{keepalive = KState}) ->
+ rabbit_mqtt_keepalive:interval_secs(KState);
+i(Key, #state{proc_state = PState}) ->
+ rabbit_mqtt_processor:info(Key, PState).
diff --git a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_middleware.erl b/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_middleware.erl
deleted file mode 100644
index 23cfa04f16b1..000000000000
--- a/deps/rabbitmq_web_mqtt/src/rabbit_web_mqtt_middleware.erl
+++ /dev/null
@@ -1,20 +0,0 @@
-%% This Source Code Form is subject to the terms of the Mozilla Public
-%% License, v. 2.0. If a copy of the MPL was not distributed with this
-%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
-%%
-%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
-%%
-
--module(rabbit_web_mqtt_middleware).
--behavior(cowboy_middleware).
-
--export([execute/2]).
-
-execute(Req, Env) ->
- #{keepalive_sup := KeepaliveSup} = Env,
- case maps:get(handler_opts, Env, undefined) of
- undefined -> {ok, Req, Env};
- Opts when is_list(Opts) ->
- {ok, Req, Env#{handler_opts => [{keepalive_sup, KeepaliveSup}
- |Opts]}}
- end.
diff --git a/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl b/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl
index bd49cec10d8b..95cd3e5c444e 100644
--- a/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl
+++ b/deps/rabbitmq_web_mqtt/test/proxy_protocol_SUITE.erl
@@ -8,11 +8,10 @@
-module(proxy_protocol_SUITE).
--compile(export_all).
+-compile([export_all, nowarn_export_all]).
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
--include("src/emqttc_packet.hrl").
suite() ->
[
@@ -21,14 +20,14 @@ suite() ->
].
all() ->
- [{group, http_tests}].%,
-% {group, https_tests}].
+ [{group, http_tests},
+ {group, https_tests}].
groups() ->
Tests = [
proxy_protocol
],
- [%{https_tests, [], Tests},
+ [{https_tests, [], Tests},
{http_tests, [], Tests}].
init_per_suite(Config) ->
@@ -83,14 +82,9 @@ proxy_protocol(Config) ->
Protocol = ?config(protocol, Config),
WS = rfc6455_client:new(Protocol ++ "://127.0.0.1:" ++ PortStr ++ "/ws", self(),
- undefined, [], "PROXY TCP4 192.168.1.1 192.168.1.2 80 81\r\n"),
+ undefined, ["mqtt"], "PROXY TCP4 192.168.1.1 192.168.1.2 80 81\r\n"),
{ok, _} = rfc6455_client:open(WS),
- Frame = emqttc_serialiser:serialise(
- ?CONNECT_PACKET(#mqtt_packet_connect{
- client_id = <<"web-mqtt-tests-proxy-protocol">>,
- username = <<"guest">>,
- password = <<"guest">>})),
- rfc6455_client:send_binary(WS, Frame),
+ rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_packet()),
{binary, _P} = rfc6455_client:recv(WS),
ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0,
?MODULE, connection_name, []),
diff --git a/deps/rabbitmq_web_mqtt/test/src/emqttc_packet.hrl b/deps/rabbitmq_web_mqtt/test/src/emqttc_packet.hrl
deleted file mode 100644
index f1063fdcff43..000000000000
--- a/deps/rabbitmq_web_mqtt/test/src/emqttc_packet.hrl
+++ /dev/null
@@ -1,240 +0,0 @@
-%%%-----------------------------------------------------------------------------
-%%% @Copyright (C) 2012-2015, Feng Lee
-%%%
-%%% Permission is hereby granted, free of charge, to any person obtaining a copy
-%%% of this software and associated documentation files (the "Software"), to deal
-%%% in the Software without restriction, including without limitation the rights
-%%% to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-%%% copies of the Software, and to permit persons to whom the Software is
-%%% furnished to do so, subject to the following conditions:
-%%%
-%%% The above copyright notice and this permission notice shall be included in all
-%%% copies or substantial portions of the Software.
-%%%
-%%% THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-%%% IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-%%% FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-%%% AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-%%% LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
-%%% SOFTWARE.
-%%%-----------------------------------------------------------------------------
-%%% @doc
-%%% emqttc packet header.
-%%% @end
-%%%-----------------------------------------------------------------------------
-
-%%------------------------------------------------------------------------------
-%% MQTT Protocol Version and Levels
-%%------------------------------------------------------------------------------
--define(MQTT_PROTO_V31, 3).
--define(MQTT_PROTO_V311, 4).
-
--define(PROTOCOL_NAMES, [
- {?MQTT_PROTO_V31, <<"MQIsdp">>},
- {?MQTT_PROTO_V311, <<"MQTT">>}]).
-
--type mqtt_vsn() :: ?MQTT_PROTO_V31 | ?MQTT_PROTO_V311.
-
-%%------------------------------------------------------------------------------
-%% QoS Levels
-%%------------------------------------------------------------------------------
-
--define(QOS_0, 0).
--define(QOS_1, 1).
--define(QOS_2, 2).
-
--define(IS_QOS(I), (I >= ?QOS_0 andalso I =< ?QOS_2)).
-
--type mqtt_qos() :: ?QOS_0 | ?QOS_1 | ?QOS_2.
-
-%%------------------------------------------------------------------------------
-%% Default Keepalive Timeout(secs)
-%%------------------------------------------------------------------------------
--define(KEEPALIVE, 90).
-
-%%------------------------------------------------------------------------------
-%% Max ClientId Length. Why 1024? NiDongDe!
-%%------------------------------------------------------------------------------
--define(MAX_CLIENTID_LEN, 1024).
-
-%%------------------------------------------------------------------------------
-%% MQTT Control Packet Types
-%%------------------------------------------------------------------------------
--define(RESERVED, 0). %% Reserved
--define(CONNECT, 1). %% Client request to connect to Server
--define(CONNACK, 2). %% Server to Client: Connect acknowledgment
--define(PUBLISH, 3). %% Publish message
--define(PUBACK, 4). %% Publish acknowledgment
--define(PUBREC, 5). %% Publish received (assured delivery part 1)
--define(PUBREL, 6). %% Publish release (assured delivery part 2)
--define(PUBCOMP, 7). %% Publish complete (assured delivery part 3)
--define(SUBSCRIBE, 8). %% Client subscribe request
--define(SUBACK, 9). %% Server Subscribe acknowledgment
--define(UNSUBSCRIBE, 10). %% Unsubscribe request
--define(UNSUBACK, 11). %% Unsubscribe acknowledgment
--define(PINGREQ, 12). %% PING request
--define(PINGRESP, 13). %% PING response
--define(DISCONNECT, 14). %% Client is disconnecting
-
--define(TYPE_NAMES, [
- 'CONNECT',
- 'CONNACK',
- 'PUBLISH',
- 'PUBACK',
- 'PUBREC',
- 'PUBREL',
- 'PUBCOMP',
- 'SUBSCRIBE',
- 'SUBACK',
- 'UNSUBSCRIBE',
- 'UNSUBACK',
- 'PINGREQ',
- 'PINGRESP',
- 'DISCONNECT']).
-
--type mqtt_packet_type() :: ?RESERVED..?DISCONNECT.
-
-%%------------------------------------------------------------------------------
-%% MQTT Connect Return Codes
-%%------------------------------------------------------------------------------
--define(CONNACK_ACCEPT, 0). %% Connection accepted
--define(CONNACK_PROTO_VER, 1). %% Unacceptable protocol version
--define(CONNACK_INVALID_ID, 2). %% Client Identifier is correct UTF-8 but not allowed by the Server
--define(CONNACK_SERVER, 3). %% Server unavailable
--define(CONNACK_CREDENTIALS, 4). %% Username or password is malformed
--define(CONNACK_AUTH, 5). %% Client is not authorized to connect
-
--type mqtt_connack() :: ?CONNACK_ACCEPT..?CONNACK_AUTH.
-
-%%------------------------------------------------------------------------------
-%% MQTT Parser and Serialiser
-%%------------------------------------------------------------------------------
--define(MAX_LEN, 16#fffffff).
--define(HIGHBIT, 2#10000000).
--define(LOWBITS, 2#01111111).
-
-%%------------------------------------------------------------------------------
-%% MQTT Packet Fixed Header
-%%------------------------------------------------------------------------------
--record(mqtt_packet_header, {
- type = ?RESERVED :: mqtt_packet_type(),
- dup = false :: boolean(),
- qos = ?QOS_0 :: mqtt_qos(),
- retain = false :: boolean()}).
-
-%%------------------------------------------------------------------------------
-%% MQTT Packets
-%%------------------------------------------------------------------------------
--type mqtt_packet_id() :: 1..16#ffff | undefined.
-
--record(mqtt_packet_connect, {
- client_id = <<>> :: binary(),
- proto_ver = ?MQTT_PROTO_V311 :: mqtt_vsn(),
- proto_name = <<"MQTT">> :: binary(),
- will_retain = false :: boolean(),
- will_qos = ?QOS_0 :: mqtt_qos(),
- will_flag = false :: boolean(),
- clean_sess = false :: boolean(),
- keep_alive = 60 :: non_neg_integer(),
- will_topic = undefined :: undefined | binary(),
- will_msg = undefined :: undefined | binary(),
- username = undefined :: undefined | binary(),
- password = undefined :: undefined | binary()}).
-
--record(mqtt_packet_connack, {
- ack_flags = ?RESERVED :: 0 | 1,
- return_code :: mqtt_connack() }).
-
--record(mqtt_packet_publish, {
- topic_name :: binary(),
- packet_id :: mqtt_packet_id() }).
-
--record(mqtt_packet_puback, {
- packet_id :: mqtt_packet_id() }).
-
--record(mqtt_packet_subscribe, {
- packet_id :: mqtt_packet_id(),
- topic_table :: list({binary(), mqtt_qos()}) }).
-
--record(mqtt_packet_unsubscribe, {
- packet_id :: mqtt_packet_id(),
- topics :: list(binary()) }).
-
--record(mqtt_packet_suback, {
- packet_id :: mqtt_packet_id(),
- qos_table :: list(mqtt_qos() | 128) }).
-
--record(mqtt_packet_unsuback, {
- packet_id :: mqtt_packet_id() }).
-
-%%------------------------------------------------------------------------------
-%% MQTT Control Packet
-%%------------------------------------------------------------------------------
--record(mqtt_packet, {
- header :: #mqtt_packet_header{},
- variable :: #mqtt_packet_connect{} | #mqtt_packet_connack{}
- | #mqtt_packet_publish{} | #mqtt_packet_puback{}
- | #mqtt_packet_subscribe{} | #mqtt_packet_suback{}
- | #mqtt_packet_unsubscribe{} | #mqtt_packet_unsuback{}
- | mqtt_packet_id() | undefined,
- payload :: binary() | undefined }).
-
--type mqtt_packet() :: #mqtt_packet{}.
-
-%%------------------------------------------------------------------------------
-%% MQTT Packet Match
-%%------------------------------------------------------------------------------
--define(CONNECT_PACKET(Packet),
- #mqtt_packet{header = #mqtt_packet_header{type = ?CONNECT}, variable = Packet}).
-
--define(CONNACK_PACKET(ReturnCode),
- #mqtt_packet{header = #mqtt_packet_header{type = ?CONNACK},
- variable = #mqtt_packet_connack{return_code = ReturnCode}}).
-
--define(PUBLISH_PACKET(Qos, Topic, PacketId, Payload),
- #mqtt_packet{header = #mqtt_packet_header{type = ?PUBLISH,
- qos = Qos},
- variable = #mqtt_packet_publish{topic_name = Topic,
- packet_id = PacketId},
- payload = Payload}).
-
--define(PUBACK_PACKET(Type, PacketId),
- #mqtt_packet{header = #mqtt_packet_header{type = Type},
- variable = #mqtt_packet_puback{packet_id = PacketId}}).
-
--define(PUBREL_PACKET(PacketId),
- #mqtt_packet{header = #mqtt_packet_header{type = ?PUBREL, qos = ?QOS_1},
- variable = #mqtt_packet_puback{packet_id = PacketId}}).
-
--define(SUBSCRIBE_PACKET(PacketId, TopicTable),
- #mqtt_packet{header = #mqtt_packet_header{type = ?SUBSCRIBE, qos = ?QOS_1},
- variable = #mqtt_packet_subscribe{packet_id = PacketId,
- topic_table = TopicTable}}).
--define(SUBACK_PACKET(PacketId, QosTable),
- #mqtt_packet{header = #mqtt_packet_header{type = ?SUBACK},
- variable = #mqtt_packet_suback{packet_id = PacketId,
- qos_table = QosTable}}).
--define(UNSUBSCRIBE_PACKET(PacketId, Topics),
- #mqtt_packet{header = #mqtt_packet_header{type = ?UNSUBSCRIBE, qos = ?QOS_1},
- variable = #mqtt_packet_unsubscribe{packet_id = PacketId,
- topics = Topics}}).
--define(UNSUBACK_PACKET(PacketId),
- #mqtt_packet{header = #mqtt_packet_header{type = ?UNSUBACK},
- variable = #mqtt_packet_unsuback{packet_id = PacketId}}).
-
--define(PACKET(Type),
- #mqtt_packet{header = #mqtt_packet_header{type = Type}}).
-
-%%------------------------------------------------------------------------------
-%% MQTT Message
-%%------------------------------------------------------------------------------
--record(mqtt_message, {
- qos = ?QOS_0 :: mqtt_qos(),
- retain = false :: boolean(),
- dup = false :: boolean(),
- msgid :: mqtt_packet_id(),
- topic :: binary(),
- payload :: binary()}).
-
--type mqtt_message() :: #mqtt_message{}.
diff --git a/deps/rabbitmq_web_mqtt/test/src/emqttc_parser.erl b/deps/rabbitmq_web_mqtt/test/src/emqttc_parser.erl
deleted file mode 100644
index ff800c1e1ff6..000000000000
--- a/deps/rabbitmq_web_mqtt/test/src/emqttc_parser.erl
+++ /dev/null
@@ -1,215 +0,0 @@
-%%%-----------------------------------------------------------------------------
-%%% Copyright (c) 2012-2015 eMQTT.IO, All Rights Reserved.
-%%%
-%%% Permission is hereby granted, free of charge, to any person obtaining a copy
-%%% of this software and associated documentation files (the "Software"), to deal
-%%% in the Software without restriction, including without limitation the rights
-%%% to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-%%% copies of the Software, and to permit persons to whom the Software is
-%%% furnished to do so, subject to the following conditions:
-%%%
-%%% The above copyright notice and this permission notice shall be included in all
-%%% copies or substantial portions of the Software.
-%%%
-%%% THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-%%% IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-%%% FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-%%% AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-%%% LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
-%%% SOFTWARE.
-%%%-----------------------------------------------------------------------------
-%%% @doc
-%%% emqttc received packet parser.
-%%%
-%%% @end
-%%%-----------------------------------------------------------------------------
--module(emqttc_parser).
-
--author("feng@emqtt.io").
-
--include("emqttc_packet.hrl").
-
-%% API
--export([new/0, parse/2]).
-
-
-%%%-----------------------------------------------------------------------------
-%% @doc Initialize a parser.
-%% @end
-%%%-----------------------------------------------------------------------------
--spec new() -> none.
-new() -> none.
-
-%%%-----------------------------------------------------------------------------
-%% @doc Parse MQTT Packet.
-%% @end
-%%%-----------------------------------------------------------------------------
--spec parse(binary(), none | fun()) -> {ok, mqtt_packet()} | {error, any()} | {more, fun()}.
-parse(<<>>, none) ->
- {more, fun(Bin) -> parse(Bin, none) end};
-parse(<>, none) ->
- parse_remaining_len(Rest, #mqtt_packet_header{type = PacketType,
- dup = bool(Dup),
- qos = QoS,
- retain = bool(Retain)});
-parse(Bin, Cont) -> Cont(Bin).
-
-parse_remaining_len(<<>>, Header) ->
- {more, fun(Bin) -> parse_remaining_len(Bin, Header) end};
-parse_remaining_len(Rest, Header) ->
- parse_remaining_len(Rest, Header, 1, 0).
-
-parse_remaining_len(_Bin, _Header, _Multiplier, Length)
- when Length > ?MAX_LEN ->
- {error, invalid_mqtt_frame_len};
-parse_remaining_len(<<>>, Header, Multiplier, Length) ->
- {more, fun(Bin) -> parse_remaining_len(Bin, Header, Multiplier, Length) end};
-parse_remaining_len(<<1:1, Len:7, Rest/binary>>, Header, Multiplier, Value) ->
- parse_remaining_len(Rest, Header, Multiplier * ?HIGHBIT, Value + Len * Multiplier);
-parse_remaining_len(<<0:1, Len:7, Rest/binary>>, Header, Multiplier, Value) ->
- parse_frame(Rest, Header, Value + Len * Multiplier).
-
-parse_frame(Bin, #mqtt_packet_header{type = Type,
- qos = Qos} = Header, Length) ->
- case {Type, Bin} of
- %{?CONNECT, <>} ->
- % {ProtoName, Rest1} = parse_utf(FrameBin),
- % <> = Rest1,
- % <> = Rest2,
- % {ClientId, Rest4} = parse_utf(Rest3),
- % {WillTopic, Rest5} = parse_utf(Rest4, WillFlag),
- % {WillMsg, Rest6} = parse_msg(Rest5, WillFlag),
- % {UserName, Rest7} = parse_utf(Rest6, UsernameFlag),
- % {PasssWord, <<>>} = parse_utf(Rest7, PasswordFlag),
- % case protocol_name_approved(ProtoVersion, ProtoName) of
- % true ->
- % wrap(Header,
- % #mqtt_packet_connect{
- % proto_ver = ProtoVersion,
- % proto_name = ProtoName,
- % will_retain = bool(WillRetain),
- % will_qos = WillQos,
- % will_flag = bool(WillFlag),
- % clean_sess = bool(CleanSession),
- % keep_alive = KeepAlive,
- % client_id = ClientId,
- % will_topic = WillTopic,
- % will_msg = WillMsg,
- % username = UserName,
- % password = PasssWord}, Rest);
- % false ->
- % {error, protocol_header_corrupt}
- % end;
- {?CONNACK, <>} ->
- <<_Reserved:7, SP:1, ReturnCode:8>> = FrameBin,
- wrap(Header, #mqtt_packet_connack{ack_flags = SP,
- return_code = ReturnCode }, Rest);
- {?PUBLISH, <>} ->
- {TopicName, Rest1} = parse_utf(FrameBin),
- {PacketId, Payload} = case Qos of
- 0 -> {undefined, Rest1};
- _ -> <> = Rest1,
- {Id, R}
- end,
- wrap(Header, #mqtt_packet_publish{topic_name = TopicName,
- packet_id = PacketId},
- Payload, Rest);
- {?PUBACK, <>} ->
- <> = FrameBin,
- wrap(Header, #mqtt_packet_puback{packet_id = PacketId}, Rest);
- {?PUBREC, <>} ->
- <> = FrameBin,
- wrap(Header, #mqtt_packet_puback{packet_id = PacketId}, Rest);
- {?PUBREL, <>} ->
- 1 = Qos,
- <> = FrameBin,
- wrap(Header, #mqtt_packet_puback{packet_id = PacketId}, Rest);
- {?PUBCOMP, <>} ->
- <> = FrameBin,
- wrap(Header, #mqtt_packet_puback{packet_id = PacketId}, Rest);
- %{?SUBSCRIBE, <>} ->
- % 1 = Qos,
- % <> = FrameBin,
- % TopicTable = parse_topics(?SUBSCRIBE, Rest1, []),
- % wrap(Header, #mqtt_packet_subscribe{packet_id = PacketId,
- % topic_table = TopicTable}, Rest);
- {?SUBACK, <>} ->
- <> = FrameBin,
- wrap(Header, #mqtt_packet_suback{packet_id = PacketId,
- qos_table = parse_qos(Rest1, [])}, Rest);
- %{?UNSUBSCRIBE, <>} ->
- % 1 = Qos,
- % <> = FrameBin,
- % Topics = parse_topics(?UNSUBSCRIBE, Rest1, []),
- % wrap(Header, #mqtt_packet_unsubscribe{packet_id = PacketId,
- % topics = Topics}, Rest);
- {?UNSUBACK, <>} ->
- <> = FrameBin,
- wrap(Header, #mqtt_packet_unsuback{packet_id = PacketId}, Rest);
- %{?PINGREQ, Rest} ->
- % Length = 0,
- % wrap(Header, Rest);
- {?PINGRESP, Rest} ->
- Length = 0,
- wrap(Header, Rest);
- %{?DISCONNECT, Rest} ->
- % Length = 0,
- % wrap(Header, Rest);
- {_, TooShortBin} ->
- {more, fun(BinMore) ->
- parse_frame(<>,
- Header, Length)
- end}
- end.
-
-wrap(Header, Variable, Payload, Rest) ->
- {ok, #mqtt_packet{header = Header, variable = Variable, payload = Payload}, Rest}.
-wrap(Header, Variable, Rest) ->
- {ok, #mqtt_packet{header = Header, variable = Variable}, Rest}.
-wrap(Header, Rest) ->
- {ok, #mqtt_packet{header = Header}, Rest}.
-
-parse_qos(<<>>, Acc) ->
- lists:reverse(Acc);
-parse_qos(<>, Acc) ->
- parse_qos(Rest, [QoS | Acc]).
-
-% server function
-%parse_topics(_, <<>>, Topics) ->
-% Topics;
-%parse_topics(?SUBSCRIBE = Sub, Bin, Topics) ->
-% {Name, <<_:6, QoS:2, Rest/binary>>} = parse_utf(Bin),
-% parse_topics(Sub, Rest, [{Name, QoS}| Topics]);
-%parse_topics(?UNSUBSCRIBE = Sub, Bin, Topics) ->
-% {Name, <>} = parse_utf(Bin),
-% parse_topics(Sub, Rest, [Name | Topics]).
-
-%parse_utf(Bin, 0) ->
-% {undefined, Bin};
-%parse_utf(Bin, _) ->
-% parse_utf(Bin).
-
-parse_utf(<>) ->
- {Str, Rest}.
-
-% server function
-%parse_msg(Bin, 0) ->
-% {undefined, Bin};
-%parse_msg(<>, _) ->
-% {Msg, Rest}.
-
-bool(0) -> false;
-bool(1) -> true.
-
-%protocol_name_approved(Ver, Name) ->
-% lists:member({Ver, Name}, ?PROTOCOL_NAMES).
-
diff --git a/deps/rabbitmq_web_mqtt/test/src/emqttc_serialiser.erl b/deps/rabbitmq_web_mqtt/test/src/emqttc_serialiser.erl
deleted file mode 100644
index 5654de221233..000000000000
--- a/deps/rabbitmq_web_mqtt/test/src/emqttc_serialiser.erl
+++ /dev/null
@@ -1,157 +0,0 @@
-%%%-----------------------------------------------------------------------------
-%%% Copyright (c) 2012-2015 eMQTT.IO, All Rights Reserved.
-%%%
-%%% Permission is hereby granted, free of charge, to any person obtaining a copy
-%%% of this software and associated documentation files (the "Software"), to deal
-%%% in the Software without restriction, including without limitation the rights
-%%% to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-%%% copies of the Software, and to permit persons to whom the Software is
-%%% furnished to do so, subject to the following conditions:
-%%%
-%%% The above copyright notice and this permission notice shall be included in all
-%%% copies or substantial portions of the Software.
-%%%
-%%% THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-%%% IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-%%% FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-%%% AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-%%% LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
-%%% SOFTWARE.
-%%%-----------------------------------------------------------------------------
-%%% @doc
-%%% emqttc packet serialiser.
-%%%
-%%% @end
-%%%-----------------------------------------------------------------------------
--module(emqttc_serialiser).
-
--author("feng@emqtt.io").
-
--include("emqttc_packet.hrl").
-
-%% API
--export([serialise/1]).
-
-
-serialise(#mqtt_packet{header = Header = #mqtt_packet_header{type = Type},
- variable = Variable,
- payload = Payload}) ->
- serialise_header(Header,
- serialise_variable(Type, Variable,
- serialise_payload(Payload))).
-
-serialise_header(#mqtt_packet_header{type = Type,
- dup = Dup,
- qos = Qos,
- retain = Retain},
- {VariableBin, PayloadBin})
- when is_integer(Type) andalso ?CONNECT =< Type andalso Type =< ?DISCONNECT ->
- Len = size(VariableBin) + size(PayloadBin),
- true = (Len =< ?MAX_LEN),
- LenBin = serialise_len(Len),
- <>.
-
-serialise_variable(?CONNECT, #mqtt_packet_connect{client_id = ClientId,
- proto_ver = ProtoVer,
- proto_name = ProtoName,
- will_retain = WillRetain,
- will_qos = WillQos,
- will_flag = WillFlag,
- clean_sess = CleanSess,
- keep_alive = KeepAlive,
- will_topic = WillTopic,
- will_msg = WillMsg,
- username = Username,
- password = Password }, undefined) ->
- VariableBin = <<(size(ProtoName)):16/big-unsigned-integer,
- ProtoName/binary,
- ProtoVer:8,
- (opt(Username)):1,
- (opt(Password)):1,
- (opt(WillRetain)):1,
- WillQos:2,
- (opt(WillFlag)):1,
- (opt(CleanSess)):1,
- 0:1,
- KeepAlive:16/big-unsigned-integer>>,
- PayloadBin = serialise_utf(ClientId),
- PayloadBin1 = case WillFlag of
- true -> <>;
- false -> PayloadBin
- end,
- UserPasswd = << <<(serialise_utf(B))/binary>> || B <- [Username, Password], B =/= undefined >>,
- {VariableBin, <>};
-
-serialise_variable(?CONNACK, #mqtt_packet_connack{ack_flags = AckFlags,
- return_code = ReturnCode},
- undefined) ->
- {<>, <<>>};
-
-serialise_variable(?SUBSCRIBE, #mqtt_packet_subscribe{packet_id = PacketId,
- topic_table = Topics }, undefined) ->
- {<>, serialise_topics(Topics)};
-
-serialise_variable(?SUBACK, #mqtt_packet_suback {packet_id = PacketId,
- qos_table = QosTable},
- undefined) ->
- {<>, << <> || Q <- QosTable >>};
-
-serialise_variable(?UNSUBSCRIBE, #mqtt_packet_unsubscribe{
- packet_id = PacketId, topics = Topics }, undefined) ->
- {<>, serialise_topics(Topics)};
-
-serialise_variable(?UNSUBACK, #mqtt_packet_suback {packet_id = PacketId},
- undefined) ->
- {<>, <<>>};
-
-serialise_variable(?PUBLISH, #mqtt_packet_publish { topic_name = TopicName,
- packet_id = PacketId }, PayloadBin) ->
- TopicBin = serialise_utf(TopicName),
- PacketIdBin = if
- PacketId =:= undefined -> <<>>;
- true -> <>
- end,
- {<>, PayloadBin};
-
-serialise_variable(PubAck, #mqtt_packet_puback { packet_id = PacketId }, _Payload)
- when PubAck =:= ?PUBACK; PubAck =:= ?PUBREC; PubAck =:= ?PUBREL; PubAck =:= ?PUBCOMP ->
- {<>, <<>>};
-
-serialise_variable(?PINGREQ, undefined, undefined) ->
- {<<>>, <<>>};
-
-serialise_variable(?DISCONNECT, undefined, undefined) ->
- {<<>>, <<>>}.
-
-serialise_payload(undefined) ->
- undefined;
-serialise_payload(Bin) when is_binary(Bin) ->
- Bin.
-
-serialise_topics([{_Topic, _Qos}|_] = Topics) ->
- << <<(serialise_utf(Topic))/binary, ?RESERVED:6, Qos:2>> || {Topic, Qos} <- Topics >>;
-
-serialise_topics([H|_] = Topics) when is_binary(H) ->
- << <<(serialise_utf(Topic))/binary>> || Topic <- Topics >>.
-
-serialise_utf(String) ->
- StringBin = unicode:characters_to_binary(String),
- Len = size(StringBin),
- true = (Len =< 16#ffff),
- <>.
-
-serialise_len(N) when N =< ?LOWBITS ->
- <<0:1, N:7>>;
-serialise_len(N) ->
- <<1:1, (N rem ?HIGHBIT):7, (serialise_len(N div ?HIGHBIT))/binary>>.
-
-opt(undefined) -> ?RESERVED;
-opt(false) -> 0;
-opt(true) -> 1;
-opt(X) when is_integer(X) -> X;
-opt(B) when is_binary(B) -> 1.
diff --git a/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl b/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl
index 697370c583b4..9b1f73bf19ab 100644
--- a/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl
+++ b/deps/rabbitmq_web_mqtt/test/src/rabbit_ws_test_util.erl
@@ -7,7 +7,10 @@
-module(rabbit_ws_test_util).
--export([update_app_env/3, get_web_mqtt_port_str/1]).
+-include_lib("common_test/include/ct.hrl").
+
+-export([update_app_env/3, get_web_mqtt_port_str/1,
+ mqtt_3_1_1_connect_packet/0]).
update_app_env(Config, Key, Val) ->
ok = rabbit_ct_broker_helpers:rpc(Config, 0,
@@ -21,10 +24,38 @@ update_app_env(Config, Key, Val) ->
[rabbitmq_web_mqtt]).
get_web_mqtt_port_str(Config) ->
- Port = case rabbit_ct_helpers:get_config(Config, protocol) of
- "ws" ->
- rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt);
- "wss" ->
- rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt_tls)
- end,
+ Port = case ?config(protocol, Config) of
+ "ws" ->
+ rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt);
+ "wss" ->
+ rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt_tls)
+ end,
integer_to_list(Port).
+
+mqtt_3_1_1_connect_packet() ->
+ <<16,
+ 24,
+ 0,
+ 4,
+ 77,
+ 81,
+ 84,
+ 84,
+ 4,
+ 2,
+ 0,
+ 60,
+ 0,
+ 12,
+ 84,
+ 101,
+ 115,
+ 116,
+ 67,
+ 111,
+ 110,
+ 115,
+ 117,
+ 109,
+ 101,
+ 114>>.
diff --git a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl
index e4216d96cca1..19b0f873c46a 100644
--- a/deps/rabbitmq_web_mqtt/test/system_SUITE.erl
+++ b/deps/rabbitmq_web_mqtt/test/system_SUITE.erl
@@ -8,31 +8,33 @@
-module(system_SUITE).
-include_lib("eunit/include/eunit.hrl").
--include("src/emqttc_packet.hrl").
--compile(export_all).
+-compile([export_all, nowarn_export_all]).
+
+-import(rabbit_ct_helpers, [eventually/1]).
all() ->
- [
- {group, non_parallel_tests}
- ].
+ [{group, tests}].
groups() ->
[
- {non_parallel_tests, [],
- [connection
- , pubsub_shared_connection
- , pubsub_separate_connections
- , last_will_enabled
- , last_will_disabled
- , disconnect
- ]}
+ {tests, [],
+ [no_websocket_subprotocol
+ ,unsupported_websocket_subprotocol
+ ,unacceptable_data_type
+ ,handle_invalid_packets
+ ,duplicate_connect
+ ]}
].
+suite() ->
+ [{timetrap, {minutes, 2}}].
+
init_per_suite(Config) ->
rabbit_ct_helpers:log_environment(),
Config1 = rabbit_ct_helpers:set_config(Config, [
- {rmq_nodename_suffix, ?MODULE}
+ {rmq_nodename_suffix, ?MODULE},
+ {protocol, "ws"}
]),
rabbit_ct_helpers:run_setup_steps(Config1,
rabbit_ct_broker_helpers:setup_steps() ++
@@ -55,190 +57,67 @@ init_per_testcase(Testcase, Config) ->
end_per_testcase(Testcase, Config) ->
rabbit_ct_helpers:testcase_finished(Config, Testcase).
--define(DEFAULT_TIMEOUT, 15000).
-
-
-connection(Config) ->
- Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt),
- PortStr = integer_to_list(Port),
- WS = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()),
+%% -------------------------------------------------------------------
+%% Testsuite cases
+%% -------------------------------------------------------------------
+
+no_websocket_subprotocol(Config) ->
+ websocket_subprotocol(Config, []).
+
+unsupported_websocket_subprotocol(Config) ->
+ websocket_subprotocol(Config, ["not-mqtt-protocol"]).
+
+%% "The client MUST include “mqtt” in the list of WebSocket Sub Protocols it offers" [MQTT-6.0.0-3].
+websocket_subprotocol(Config, SubProtocol) ->
+ PortStr = rabbit_ws_test_util:get_web_mqtt_port_str(Config),
+ WS = rfc6455_client:new("ws://localhost:" ++ PortStr ++ "/ws", self(), undefined, SubProtocol),
+ {_, [{http_response, Res}]} = rfc6455_client:open(WS),
+ {'HTTP/1.1', 400, <<"Bad Request">>, _} = cow_http:parse_status_line(rabbit_data_coercion:to_binary(Res)),
+ rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_packet()),
+ {close, _} = rfc6455_client:recv(WS, timer:seconds(1)).
+
+%% "MQTT Control Packets MUST be sent in WebSocket binary data frames. If any other type
+%% of data frame is received the recipient MUST close the Network Connection" [MQTT-6.0.0-1].
+unacceptable_data_type(Config) ->
+ PortStr = rabbit_ws_test_util:get_web_mqtt_port_str(Config),
+ WS = rfc6455_client:new("ws://localhost:" ++ PortStr ++ "/ws", self(), undefined, ["mqtt"]),
{ok, _} = rfc6455_client:open(WS),
- {close, _} = rfc6455_client:close(WS),
- ok.
+ rfc6455_client:send(WS, "not-binary-data"),
+ {close, {1003, _}} = rfc6455_client:recv(WS, timer:seconds(1)).
-pubsub_shared_connection(Config) ->
- Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt),
- PortStr = integer_to_list(Port),
- WS = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()),
+handle_invalid_packets(Config) ->
+ PortStr = rabbit_ws_test_util:get_web_mqtt_port_str(Config),
+ WS = rfc6455_client:new("ws://localhost:" ++ PortStr ++ "/ws", self(), undefined, ["mqtt"]),
{ok, _} = rfc6455_client:open(WS),
- ok = raw_send(WS,
- ?CONNECT_PACKET(#mqtt_packet_connect{
- clean_sess = true,
- client_id = <<"web-mqtt-tests-pubsub">>,
- username = <<"guest">>,
- password = <<"guest">>})),
-
- {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS),
-
- Dst = <<"/topic/test-web-mqtt">>,
-
- ok = raw_send(WS, ?SUBSCRIBE_PACKET(1, [{Dst, ?QOS_1}])),
- {ok, ?SUBACK_PACKET(_, _), _} = raw_recv(WS),
-
- Payload = <<"a\x00a">>,
-
- ok = raw_send(WS, ?PUBLISH_PACKET(?QOS_1, Dst, 2, Payload)),
- {ok, ?PUBLISH_PACKET(_, Dst, _, Payload), _} = raw_recv(WS),
-
- {close, _} = rfc6455_client:close(WS),
- ok.
-
-pubsub_separate_connections(Config) ->
- Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt),
- PortStr = integer_to_list(Port),
- WS1 = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()),
- {ok, _} = rfc6455_client:open(WS1),
- ok = raw_send(WS1,
- ?CONNECT_PACKET(#mqtt_packet_connect{
- clean_sess = true,
- client_id = <<"web-mqtt-tests-publisher">>,
- username = <<"guest">>,
- password = <<"guest">>})),
- {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS1),
-
- WS2 = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()),
- {ok, _} = rfc6455_client:open(WS2),
- ok = raw_send(WS2,
- ?CONNECT_PACKET(#mqtt_packet_connect{
- clean_sess = true,
- client_id = <<"web-mqtt-tests-consumer">>,
- username = <<"guest">>,
- password = <<"guest">>})),
- {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS2),
-
- Dst = <<"/topic/test-web-mqtt">>,
- ok = raw_send(WS2, ?SUBSCRIBE_PACKET(1, [{Dst, ?QOS_1}])),
- {ok, ?SUBACK_PACKET(_, _), _} = raw_recv(WS2),
-
- Payload = <<"a\x00a">>,
- ok = raw_send(WS1, ?PUBLISH_PACKET(?QOS_1, Dst, 2, Payload)),
- {ok, ?PUBLISH_PACKET(_, Dst, _, Payload), _} = raw_recv(WS2),
-
- {close, _} = rfc6455_client:close(WS1),
- {close, _} = rfc6455_client:close(WS2),
- ok.
-
-last_will_enabled(Config) ->
- Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt),
- PortStr = integer_to_list(Port),
-
- LastWillDst = <<"/topic/web-mqtt-tests-ws1-last-will">>,
- LastWillMsg = <<"a last will and testament message">>,
-
- WS1 = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()),
- {ok, _} = rfc6455_client:open(WS1),
- ok = raw_send(WS1,
- ?CONNECT_PACKET(#mqtt_packet_connect{
- clean_sess = true,
- client_id = <<"web-mqtt-tests-last-will-ws1">>,
- will_flag = true,
- will_qos = ?QOS_1,
- will_topic = LastWillDst,
- will_msg = LastWillMsg,
- username = <<"guest">>,
- password = <<"guest">>})),
- {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS1),
-
- WS2 = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()),
- {ok, _} = rfc6455_client:open(WS2),
- ok = raw_send(WS2,
- ?CONNECT_PACKET(#mqtt_packet_connect{
- clean_sess = true,
- client_id = <<"web-mqtt-tests-last-will-ws2">>,
- username = <<"guest">>,
- password = <<"guest">>})),
- {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS2),
-
- ok = raw_send(WS2, ?SUBSCRIBE_PACKET(1, [{LastWillDst, ?QOS_1}])),
- {ok, ?SUBACK_PACKET(_, _), _} = raw_recv(WS2),
-
- {close, _} = rfc6455_client:close(WS1),
- ?assertMatch({ok, ?PUBLISH_PACKET(_, LastWillDst, _, LastWillMsg), _}, raw_recv(WS2, 5000)),
-
- {close, _} = rfc6455_client:close(WS2),
- ok.
-
-last_will_disabled(Config) ->
- Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt),
- PortStr = integer_to_list(Port),
-
- LastWillDst = <<"/topic/web-mqtt-tests-ws1-last-will-disabled">>,
- LastWillMsg = <<"a last will and testament message">>,
-
- WS1 = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()),
- {ok, _} = rfc6455_client:open(WS1),
- ok = raw_send(WS1,
- ?CONNECT_PACKET(#mqtt_packet_connect{
- clean_sess = true,
- client_id = <<"web-mqtt-tests-last-will-ws1-disabled">>,
- will_flag = false,
- will_qos = ?QOS_1,
- will_topic = LastWillDst,
- will_msg = LastWillMsg,
- username = <<"guest">>,
- password = <<"guest">>})),
- {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS1),
-
- WS2 = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()),
- {ok, _} = rfc6455_client:open(WS2),
- ok = raw_send(WS2,
- ?CONNECT_PACKET(#mqtt_packet_connect{
- clean_sess = true,
- client_id = <<"web-mqtt-tests-last-will-ws2-disabled">>,
- username = <<"guest">>,
- password = <<"guest">>})),
- {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS2),
-
- ok = raw_send(WS2, ?SUBSCRIBE_PACKET(1, [{LastWillDst, ?QOS_1}])),
- ?assertMatch({ok, ?SUBACK_PACKET(_, _), _}, raw_recv(WS2)),
-
- {close, _} = rfc6455_client:close(WS1),
- ?assertEqual({error, timeout}, raw_recv(WS2, 3000)),
-
- {close, _} = rfc6455_client:close(WS2),
- ok.
-
-disconnect(Config) ->
- Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_web_mqtt),
- PortStr = integer_to_list(Port),
- WS = rfc6455_client:new("ws://127.0.0.1:" ++ PortStr ++ "/ws", self()),
+ Bin = <<"GET / HTTP/1.1\r\nHost: www.rabbitmq.com\r\nUser-Agent: curl/7.43.0\r\nAccept: */*">>,
+ rfc6455_client:send_binary(WS, Bin),
+ {close, {1002, _}} = rfc6455_client:recv(WS, timer:seconds(1)).
+
+%% "A Client can only send the CONNECT Packet once over a Network Connection.
+%% The Server MUST process a second CONNECT Packet sent from a Client as a protocol
+%% violation and disconnect the Client [MQTT-3.1.0-2].
+duplicate_connect(Config) ->
+ Url = "ws://127.0.0.1:" ++ rabbit_ws_test_util:get_web_mqtt_port_str(Config) ++ "/ws",
+ WS = rfc6455_client:new(Url, self(), undefined, ["mqtt"]),
{ok, _} = rfc6455_client:open(WS),
- ok = raw_send(WS,
- ?CONNECT_PACKET(#mqtt_packet_connect{
- clean_sess = true,
- client_id = <<"web-mqtt-tests-disconnect">>,
- username = <<"guest">>,
- password = <<"guest">>})),
- {ok, ?CONNACK_PACKET(?CONNACK_ACCEPT), _} = raw_recv(WS),
-
- ok = raw_send(WS, ?PACKET(?DISCONNECT)),
- {close, {1000, _}} = rfc6455_client:recv(WS),
-
- ok.
-
-
-raw_send(WS, Packet) ->
- Frame = emqttc_serialiser:serialise(Packet),
- rfc6455_client:send_binary(WS, Frame).
+ %% 1st CONNECT should succeed.
+ rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_packet()),
+ {binary, _P} = rfc6455_client:recv(WS),
+ eventually(?_assertEqual(1, num_mqtt_connections(Config, 0))),
+
+ %% 2nd CONNECT should fail.
+ process_flag(trap_exit, true),
+ rfc6455_client:send_binary(WS, rabbit_ws_test_util:mqtt_3_1_1_connect_packet()),
+ eventually(?_assertEqual(0, num_mqtt_connections(Config, 0))),
+ receive {'EXIT', WS, _} -> ok
+ after 500 -> ct:fail("expected web socket to exit")
+ end.
-raw_recv(WS) ->
- raw_recv(WS, ?DEFAULT_TIMEOUT).
+%% -------------------------------------------------------------------
+%% Internal helpers
+%% -------------------------------------------------------------------
-raw_recv(WS, Timeout) ->
- case rfc6455_client:recv(WS, Timeout) of
- {binary, P} ->
- emqttc_parser:parse(P, emqttc_parser:new());
- {error, timeout} ->
- {error, timeout}
- end.
+%% Web mqtt connections are tracked together with mqtt connections
+num_mqtt_connections(Config, Node) ->
+ length(rabbit_ct_broker_helpers:rpc(Config, Node, rabbit_mqtt, local_connection_pids, [])).
diff --git a/rabbitmq.bzl b/rabbitmq.bzl
index bed896eb8f15..8955c4b1b748 100644
--- a/rabbitmq.bzl
+++ b/rabbitmq.bzl
@@ -275,7 +275,9 @@ def rabbitmq_integration_suite(
# user_limits
# Starting from 3.12.0:
# feature_flags_v2
- "RABBITMQ_FEATURE_FLAGS": "quorum_queue,implicit_default_bindings,virtual_host_metadata,maintenance_mode_status,user_limits,feature_flags_v2",
+ # stream_queue
+ # classic_queue_type_delivery_support
+ "RABBITMQ_FEATURE_FLAGS": "quorum_queue,implicit_default_bindings,virtual_host_metadata,maintenance_mode_status,user_limits,feature_flags_v2,stream_queue,classic_queue_type_delivery_support",
"RABBITMQ_RUN": "$TEST_SRCDIR/$TEST_WORKSPACE/{}/rabbitmq-for-tests-run".format(package),
"RABBITMQCTL": "$TEST_SRCDIR/$TEST_WORKSPACE/{}/broker-for-tests-home/sbin/rabbitmqctl".format(package),
"RABBITMQ_PLUGINS": "$TEST_SRCDIR/$TEST_WORKSPACE/{}/broker-for-tests-home/sbin/rabbitmq-plugins".format(package),