feat(session): introduce session implementation concept
This commit is contained in:
parent
780ca15298
commit
bf16417513
|
@ -1,5 +1,5 @@
|
|||
%%--------------------------------------------------------------------
|
||||
%% Copyright (c) 2017-2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||
%% Copyright (c) 2022-2023 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.
|
||||
|
@ -17,39 +17,7 @@
|
|||
-ifndef(EMQX_SESSION_HRL).
|
||||
-define(EMQX_SESSION_HRL, true).
|
||||
|
||||
-record(session, {
|
||||
%% Client's id
|
||||
clientid :: emqx_types:clientid(),
|
||||
id :: emqx_session:session_id(),
|
||||
%% Is this session a persistent session i.e. was it started with Session-Expiry > 0
|
||||
is_persistent :: boolean(),
|
||||
%% Client’s Subscriptions.
|
||||
subscriptions :: map(),
|
||||
%% Max subscriptions allowed
|
||||
max_subscriptions :: non_neg_integer() | infinity,
|
||||
%% Upgrade QoS?
|
||||
upgrade_qos :: boolean(),
|
||||
%% Client <- Broker: QoS1/2 messages sent to the client but
|
||||
%% have not been unacked.
|
||||
inflight :: emqx_inflight:inflight(),
|
||||
%% All QoS1/2 messages published to when client is disconnected,
|
||||
%% or QoS1/2 messages pending transmission to the Client.
|
||||
%%
|
||||
%% Optionally, QoS0 messages pending transmission to the Client.
|
||||
mqueue :: emqx_mqueue:mqueue(),
|
||||
%% Next packet id of the session
|
||||
next_pkt_id = 1 :: emqx_types:packet_id(),
|
||||
%% Retry interval for redelivering QoS1/2 messages (Unit: millisecond)
|
||||
retry_interval :: timeout(),
|
||||
%% Client -> Broker: QoS2 messages received from the client, but
|
||||
%% have not been completely acknowledged
|
||||
awaiting_rel :: map(),
|
||||
%% Maximum number of awaiting QoS2 messages allowed
|
||||
max_awaiting_rel :: non_neg_integer() | infinity,
|
||||
%% Awaiting PUBREL Timeout (Unit: millisecond)
|
||||
await_rel_timeout :: timeout(),
|
||||
%% Created at
|
||||
created_at :: pos_integer()
|
||||
}).
|
||||
-define(IS_SESSION_IMPL_MEM(S), (is_tuple(S) andalso element(1, S) =:= session)).
|
||||
-define(IS_SESSION_IMPL_DS(S), (is_tuple(S) andalso element(1, S) =:= sessionds)).
|
||||
|
||||
-endif.
|
||||
|
|
|
@ -0,0 +1,58 @@
|
|||
%%--------------------------------------------------------------------
|
||||
%% Copyright (c) 2022-2023 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.
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-ifndef(EMQX_SESSION_MEM_HRL).
|
||||
-define(EMQX_SESSION_MEM_HRL, true).
|
||||
|
||||
-record(session, {
|
||||
%% Client's id
|
||||
clientid :: emqx_types:clientid(),
|
||||
id :: emqx_session:session_id(),
|
||||
%% Is this session a persistent session i.e. was it started with Session-Expiry > 0
|
||||
is_persistent :: boolean(),
|
||||
%% Client’s Subscriptions.
|
||||
subscriptions :: map(),
|
||||
%% Max subscriptions allowed
|
||||
max_subscriptions :: non_neg_integer() | infinity,
|
||||
%% Upgrade QoS?
|
||||
upgrade_qos :: boolean(),
|
||||
%% Client <- Broker: QoS1/2 messages sent to the client but
|
||||
%% have not been unacked.
|
||||
inflight :: emqx_inflight:inflight(),
|
||||
%% All QoS1/2 messages published to when client is disconnected,
|
||||
%% or QoS1/2 messages pending transmission to the Client.
|
||||
%%
|
||||
%% Optionally, QoS0 messages pending transmission to the Client.
|
||||
mqueue :: emqx_mqueue:mqueue(),
|
||||
%% Next packet id of the session
|
||||
next_pkt_id = 1 :: emqx_types:packet_id(),
|
||||
%% Retry interval for redelivering QoS1/2 messages (Unit: millisecond)
|
||||
retry_interval :: timeout(),
|
||||
%% Client -> Broker: QoS2 messages received from the client, but
|
||||
%% have not been completely acknowledged
|
||||
awaiting_rel :: map(),
|
||||
%% Maximum number of awaiting QoS2 messages allowed
|
||||
max_awaiting_rel :: non_neg_integer() | infinity,
|
||||
%% Awaiting PUBREL Timeout (Unit: millisecond)
|
||||
await_rel_timeout :: timeout(),
|
||||
%% Created at
|
||||
created_at :: pos_integer(),
|
||||
|
||||
%% Timers
|
||||
timers :: #{_Name => reference()}
|
||||
}).
|
||||
|
||||
-endif.
|
|
@ -38,7 +38,7 @@
|
|||
|
||||
start(_Type, _Args) ->
|
||||
ok = maybe_load_config(),
|
||||
_ = emqx_persistent_session_ds:init(),
|
||||
_ = emqx_persistent_message:init(),
|
||||
ok = maybe_start_quicer(),
|
||||
ok = emqx_bpapi:start(),
|
||||
ok = emqx_alarm_handler:load(),
|
||||
|
|
|
@ -224,7 +224,7 @@ publish(Msg) when is_record(Msg, message) ->
|
|||
}),
|
||||
[];
|
||||
Msg1 = #message{topic = Topic} ->
|
||||
_ = emqx_persistent_session_ds:persist_message(Msg1),
|
||||
_ = emqx_persistent_message:persist(Msg1),
|
||||
route(aggre(emqx_router:match_routes(Topic)), delivery(Msg1))
|
||||
end.
|
||||
|
||||
|
|
|
@ -104,7 +104,7 @@
|
|||
%% Takeover
|
||||
takeover :: boolean(),
|
||||
%% Resume
|
||||
resuming :: boolean(),
|
||||
resuming :: false | _ReplayContext,
|
||||
%% Pending delivers when takeovering
|
||||
pendings :: list()
|
||||
}).
|
||||
|
@ -403,7 +403,7 @@ handle_in(
|
|||
#channel{clientinfo = ClientInfo, session = Session}
|
||||
) ->
|
||||
case emqx_session:puback(ClientInfo, PacketId, Session) of
|
||||
{ok, Msg, NSession} ->
|
||||
{ok, Msg, [], NSession} ->
|
||||
ok = after_message_acked(ClientInfo, Msg, Properties),
|
||||
{ok, Channel#channel{session = NSession}};
|
||||
{ok, Msg, Publishes, NSession} ->
|
||||
|
@ -460,7 +460,7 @@ handle_in(
|
|||
}
|
||||
) ->
|
||||
case emqx_session:pubcomp(ClientInfo, PacketId, Session) of
|
||||
{ok, NSession} ->
|
||||
{ok, [], NSession} ->
|
||||
{ok, Channel#channel{session = NSession}};
|
||||
{ok, Publishes, NSession} ->
|
||||
handle_out(publish, Publishes, Channel#channel{session = NSession});
|
||||
|
@ -593,12 +593,10 @@ process_connect(
|
|||
{ok, #{session := Session, present := false}} ->
|
||||
NChannel = Channel#channel{session = Session},
|
||||
handle_out(connack, {?RC_SUCCESS, sp(false), AckProps}, ensure_connected(NChannel));
|
||||
{ok, #{session := Session, present := true, pendings := Pendings}} ->
|
||||
Pendings1 = lists:usort(lists:append(Pendings, emqx_utils:drain_deliver())),
|
||||
{ok, #{session := Session, present := true, replay := ReplayContext}} ->
|
||||
NChannel = Channel#channel{
|
||||
session = Session,
|
||||
resuming = true,
|
||||
pendings = Pendings1
|
||||
resuming = ReplayContext
|
||||
},
|
||||
handle_out(connack, {?RC_SUCCESS, sp(true), AckProps}, ensure_connected(NChannel));
|
||||
{error, client_id_unavailable} ->
|
||||
|
@ -725,9 +723,8 @@ do_publish(
|
|||
{ok, PubRes, NSession} ->
|
||||
RC = pubrec_reason_code(PubRes),
|
||||
NChannel0 = Channel#channel{session = NSession},
|
||||
NChannel1 = ensure_timer(expire_awaiting_rel, NChannel0),
|
||||
NChannel2 = ensure_quota(PubRes, NChannel1),
|
||||
handle_out(pubrec, {PacketId, RC}, NChannel2);
|
||||
NChannel1 = ensure_quota(PubRes, NChannel0),
|
||||
handle_out(pubrec, {PacketId, RC}, NChannel1);
|
||||
{error, RC = ?RC_PACKET_IDENTIFIER_IN_USE} ->
|
||||
ok = emqx_metrics:inc('packets.publish.inuse'),
|
||||
handle_out(pubrec, {PacketId, RC}, Channel);
|
||||
|
@ -900,8 +897,8 @@ maybe_update_expiry_interval(
|
|||
%% Check if the client turns off persistence (turning it on is disallowed)
|
||||
case EI =:= 0 andalso OldEI > 0 of
|
||||
true ->
|
||||
NSession = emqx_session:unpersist(NChannel#channel.session),
|
||||
NChannel#channel{session = NSession};
|
||||
ok = emqx_session:destroy(NChannel#channel.session),
|
||||
NChannel#channel{session = undefined};
|
||||
false ->
|
||||
NChannel
|
||||
end
|
||||
|
@ -937,10 +934,12 @@ handle_deliver(
|
|||
clientinfo = ClientInfo
|
||||
}
|
||||
) ->
|
||||
% NOTE
|
||||
% This is essentially part of `emqx_session_mem` logic, thus call it directly.
|
||||
Delivers1 = maybe_nack(Delivers),
|
||||
NSession = emqx_session:enqueue(ClientInfo, Delivers1, Session),
|
||||
NChannel = Channel#channel{session = NSession},
|
||||
{ok, NChannel};
|
||||
Messages = emqx_session:enrich_delivers(ClientInfo, Delivers1, Session),
|
||||
NSession = emqx_session_mem:enqueue(ClientInfo, Messages, Session),
|
||||
{ok, Channel#channel{session = NSession}};
|
||||
handle_deliver(
|
||||
Delivers,
|
||||
Channel = #channel{
|
||||
|
@ -950,11 +949,11 @@ handle_deliver(
|
|||
}
|
||||
) ->
|
||||
case emqx_session:deliver(ClientInfo, Delivers, Session) of
|
||||
{ok, [], NSession} ->
|
||||
{ok, Channel#channel{session = NSession}};
|
||||
{ok, Publishes, NSession} ->
|
||||
NChannel = Channel#channel{session = NSession},
|
||||
handle_out(publish, Publishes, ensure_timer(retry_delivery, NChannel));
|
||||
{ok, NSession} ->
|
||||
{ok, Channel#channel{session = NSession}}
|
||||
handle_out(publish, Publishes, NChannel)
|
||||
end.
|
||||
|
||||
%% Nack delivers from shared subscription
|
||||
|
@ -1164,7 +1163,9 @@ handle_call(
|
|||
conninfo = #{clientid := ClientId}
|
||||
}
|
||||
) ->
|
||||
ok = emqx_session:takeover(Session),
|
||||
% NOTE
|
||||
% This is essentially part of `emqx_session_mem` logic, thus call it directly.
|
||||
ok = emqx_session_mem:takeover(Session),
|
||||
%% TODO: Should not drain deliver here (side effect)
|
||||
Delivers = emqx_utils:drain_deliver(),
|
||||
AllPendings = lists:append(Delivers, Pendings),
|
||||
|
@ -1222,14 +1223,18 @@ handle_info(
|
|||
{sock_closed, Reason},
|
||||
Channel =
|
||||
#channel{
|
||||
conn_state = ConnState
|
||||
conn_state = ConnState,
|
||||
clientinfo = ClientInfo,
|
||||
session = Session
|
||||
}
|
||||
) when
|
||||
ConnState =:= connected orelse ConnState =:= reauthenticating
|
||||
->
|
||||
{Intent, Session1} = emqx_session:disconnect(ClientInfo, Session),
|
||||
Channel1 = ensure_disconnected(Reason, maybe_publish_will_msg(Channel)),
|
||||
case maybe_shutdown(Reason, Channel1) of
|
||||
{ok, Channel2} -> {ok, ?REPLY_EVENT(disconnected), Channel2};
|
||||
Channel2 = Channel1#channel{session = Session1},
|
||||
case maybe_shutdown(Reason, Intent, Channel2) of
|
||||
{ok, Channel3} -> {ok, ?REPLY_EVENT(disconnected), Channel3};
|
||||
Shutdown -> Shutdown
|
||||
end;
|
||||
handle_info({sock_closed, Reason}, Channel = #channel{conn_state = disconnected}) ->
|
||||
|
@ -1302,41 +1307,14 @@ handle_timeout(
|
|||
end;
|
||||
handle_timeout(
|
||||
_TRef,
|
||||
_Name = retry_delivery,
|
||||
Channel = #channel{conn_state = disconnected}
|
||||
) ->
|
||||
{ok, Channel};
|
||||
handle_timeout(
|
||||
_TRef,
|
||||
Name = retry_delivery,
|
||||
{emqx_session, Name},
|
||||
Channel = #channel{session = Session, clientinfo = ClientInfo}
|
||||
) ->
|
||||
case emqx_session:retry(ClientInfo, Session) of
|
||||
{ok, NSession} ->
|
||||
NChannel = Channel#channel{session = NSession},
|
||||
{ok, clean_timer(Name, NChannel)};
|
||||
{ok, Publishes, Timeout, NSession} ->
|
||||
NChannel = Channel#channel{session = NSession},
|
||||
handle_out(publish, Publishes, reset_timer(Name, Timeout, NChannel))
|
||||
end;
|
||||
handle_timeout(
|
||||
_TRef,
|
||||
_Name = expire_awaiting_rel,
|
||||
Channel = #channel{conn_state = disconnected}
|
||||
) ->
|
||||
{ok, Channel};
|
||||
handle_timeout(
|
||||
_TRef,
|
||||
Name = expire_awaiting_rel,
|
||||
Channel = #channel{session = Session, clientinfo = ClientInfo}
|
||||
) ->
|
||||
case emqx_session:expire(ClientInfo, awaiting_rel, Session) of
|
||||
{ok, NSession} ->
|
||||
NChannel = Channel#channel{session = NSession},
|
||||
{ok, clean_timer(Name, NChannel)};
|
||||
{ok, Timeout, NSession} ->
|
||||
NChannel = Channel#channel{session = NSession},
|
||||
{ok, reset_timer(Name, Timeout, NChannel)}
|
||||
case emqx_session:handle_timeout(ClientInfo, Name, Session) of
|
||||
{ok, [], NSession} ->
|
||||
{ok, Channel#channel{session = NSession}};
|
||||
{ok, Replies, NSession} ->
|
||||
handle_out(publish, Replies, Channel#channel{session = NSession})
|
||||
end;
|
||||
handle_timeout(_TRef, expire_session, Channel) ->
|
||||
shutdown(expired, Channel);
|
||||
|
@ -1391,18 +1369,11 @@ ensure_timer(Name, Time, Channel = #channel{timers = Timers}) ->
|
|||
reset_timer(Name, Channel) ->
|
||||
ensure_timer(Name, clean_timer(Name, Channel)).
|
||||
|
||||
reset_timer(Name, Time, Channel) ->
|
||||
ensure_timer(Name, Time, clean_timer(Name, Channel)).
|
||||
|
||||
clean_timer(Name, Channel = #channel{timers = Timers}) ->
|
||||
Channel#channel{timers = maps:remove(Name, Timers)}.
|
||||
|
||||
interval(keepalive, #channel{keepalive = KeepAlive}) ->
|
||||
emqx_keepalive:info(interval, KeepAlive);
|
||||
interval(retry_delivery, #channel{session = Session}) ->
|
||||
emqx_session:info(retry_interval, Session);
|
||||
interval(expire_awaiting_rel, #channel{session = Session}) ->
|
||||
emqx_session:info(await_rel_timeout, Session);
|
||||
interval(expire_session, #channel{conninfo = ConnInfo}) ->
|
||||
maps:get(expiry_interval, ConnInfo);
|
||||
interval(will_message, #channel{will_msg = WillMsg}) ->
|
||||
|
@ -2053,22 +2024,15 @@ maybe_resume_session(#channel{resuming = false}) ->
|
|||
ignore;
|
||||
maybe_resume_session(#channel{
|
||||
session = Session,
|
||||
resuming = true,
|
||||
pendings = Pendings,
|
||||
resuming = ReplayContext,
|
||||
clientinfo = ClientInfo
|
||||
}) ->
|
||||
{ok, Publishes, Session1} = emqx_session:replay(ClientInfo, Session),
|
||||
case emqx_session:deliver(ClientInfo, Pendings, Session1) of
|
||||
{ok, Session2} ->
|
||||
{ok, Publishes, Session2};
|
||||
{ok, More, Session2} ->
|
||||
{ok, lists:append(Publishes, More), Session2}
|
||||
end.
|
||||
emqx_session:replay(ClientInfo, ReplayContext, Session).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Maybe Shutdown the Channel
|
||||
|
||||
maybe_shutdown(Reason, Channel = #channel{conninfo = ConnInfo}) ->
|
||||
maybe_shutdown(Reason, _Intent = idle, Channel = #channel{conninfo = ConnInfo}) ->
|
||||
case maps:get(expiry_interval, ConnInfo) of
|
||||
?EXPIRE_INTERVAL_INFINITE ->
|
||||
{ok, Channel};
|
||||
|
@ -2076,7 +2040,9 @@ maybe_shutdown(Reason, Channel = #channel{conninfo = ConnInfo}) ->
|
|||
{ok, ensure_timer(expire_session, I, Channel)};
|
||||
_ ->
|
||||
shutdown(Reason, Channel)
|
||||
end.
|
||||
end;
|
||||
maybe_shutdown(Reason, _Intent = shutdown, Channel) ->
|
||||
shutdown(Reason, Channel).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Parse Topic Filters
|
||||
|
|
|
@ -19,9 +19,7 @@
|
|||
|
||||
-behaviour(gen_server).
|
||||
|
||||
-include("emqx.hrl").
|
||||
-include("emqx_cm.hrl").
|
||||
-include("emqx_session.hrl").
|
||||
-include("logger.hrl").
|
||||
-include("types.hrl").
|
||||
-include_lib("snabbkaffe/include/snabbkaffe.hrl").
|
||||
|
@ -48,14 +46,13 @@
|
|||
set_chan_stats/2
|
||||
]).
|
||||
|
||||
-export([get_chann_conn_mod/2]).
|
||||
% -export([get_chann_conn_mod/2]).
|
||||
|
||||
-export([
|
||||
open_session/3,
|
||||
discard_session/1,
|
||||
discard_session/2,
|
||||
takeover_session/1,
|
||||
takeover_session/2,
|
||||
takeover_channel_session/2,
|
||||
kick_session/1,
|
||||
kick_session/2
|
||||
]).
|
||||
|
@ -63,15 +60,14 @@
|
|||
-export([
|
||||
lookup_channels/1,
|
||||
lookup_channels/2,
|
||||
|
||||
lookup_client/1
|
||||
lookup_client/1,
|
||||
pick_channel/1
|
||||
]).
|
||||
|
||||
%% Test/debug interface
|
||||
-export([
|
||||
all_channels/0,
|
||||
all_client_ids/0,
|
||||
get_session_confs/2
|
||||
all_client_ids/0
|
||||
]).
|
||||
|
||||
%% Client management
|
||||
|
@ -96,12 +92,16 @@
|
|||
clean_down/1,
|
||||
mark_channel_connected/1,
|
||||
mark_channel_disconnected/1,
|
||||
get_connected_client_count/0,
|
||||
takeover_finish/2,
|
||||
get_connected_client_count/0
|
||||
]).
|
||||
|
||||
%% RPC targets
|
||||
-export([
|
||||
takeover_session/2,
|
||||
takeover_finish/2,
|
||||
do_kick_session/3,
|
||||
do_get_chan_stats/2,
|
||||
do_get_chan_info/2,
|
||||
do_get_chan_stats/2,
|
||||
do_get_chann_conn_mod/2
|
||||
]).
|
||||
|
||||
|
@ -261,96 +261,64 @@ set_chan_stats(ClientId, ChanPid, Stats) ->
|
|||
{ok, #{
|
||||
session := emqx_session:session(),
|
||||
present := boolean(),
|
||||
pendings => list()
|
||||
replay => _ReplayContext
|
||||
}}
|
||||
| {error, Reason :: term()}.
|
||||
open_session(true, ClientInfo = #{clientid := ClientId}, ConnInfo) ->
|
||||
Self = self(),
|
||||
CleanStart = fun(_) ->
|
||||
emqx_cm_locker:trans(ClientId, fun(_) ->
|
||||
ok = discard_session(ClientId),
|
||||
ok = emqx_session:destroy(ClientId),
|
||||
ok = emqx_session:destroy(ClientInfo, ConnInfo),
|
||||
create_register_session(ClientInfo, ConnInfo, Self)
|
||||
end,
|
||||
emqx_cm_locker:trans(ClientId, CleanStart);
|
||||
end);
|
||||
open_session(false, ClientInfo = #{clientid := ClientId}, ConnInfo) ->
|
||||
Self = self(),
|
||||
ResumeStart = fun(_) ->
|
||||
case takeover_session(ClientId) of
|
||||
{living, ConnMod, ChanPid, Session} ->
|
||||
ok = emqx_session:resume(ClientInfo, Session),
|
||||
case wrap_rpc(emqx_cm_proto_v2:takeover_finish(ConnMod, ChanPid)) of
|
||||
{ok, Pendings} ->
|
||||
clean_register_session(Session, Pendings, ClientInfo, ConnInfo, Self);
|
||||
{error, _} ->
|
||||
create_register_session(ClientInfo, ConnInfo, Self)
|
||||
end;
|
||||
none ->
|
||||
emqx_cm_locker:trans(ClientId, fun(_) ->
|
||||
case emqx_session:open(ClientInfo, ConnInfo) of
|
||||
{true, Session, ReplayContext} ->
|
||||
ok = register_channel(ClientId, Self, ConnInfo),
|
||||
{ok, #{session => Session, present => true, replay => ReplayContext}};
|
||||
false ->
|
||||
create_register_session(ClientInfo, ConnInfo, Self)
|
||||
end
|
||||
end,
|
||||
emqx_cm_locker:trans(ClientId, ResumeStart).
|
||||
|
||||
create_session(ClientInfo, ConnInfo) ->
|
||||
Options = get_session_confs(ClientInfo, ConnInfo),
|
||||
Session = emqx_session:init_and_open(Options),
|
||||
ok = emqx_metrics:inc('session.created'),
|
||||
ok = emqx_hooks:run('session.created', [ClientInfo, emqx_session:info(Session)]),
|
||||
Session.
|
||||
end).
|
||||
|
||||
create_register_session(ClientInfo = #{clientid := ClientId}, ConnInfo, ChanPid) ->
|
||||
Session = create_session(ClientInfo, ConnInfo),
|
||||
Session = emqx_session:create(ClientInfo, ConnInfo),
|
||||
ok = register_channel(ClientId, ChanPid, ConnInfo),
|
||||
{ok, #{session => Session, present => false}}.
|
||||
|
||||
clean_register_session(Session, Pendings, #{clientid := ClientId}, ConnInfo, ChanPid) ->
|
||||
ok = register_channel(ClientId, ChanPid, ConnInfo),
|
||||
{ok, #{
|
||||
session => clean_session(Session),
|
||||
present => true,
|
||||
pendings => clean_pendings(Pendings)
|
||||
}}.
|
||||
%% @doc Try to takeover a session from existing channel.
|
||||
%% Naming is wierd, because `takeover_session/2` is an RPC target and cannot be renamed.
|
||||
-spec takeover_channel_session(emqx_types:clientid(), _TODO) ->
|
||||
{ok, emqx_session:session(), _ReplayContext} | none | {error, _Reason}.
|
||||
takeover_channel_session(ClientId, OnTakeover) ->
|
||||
takeover_channel_session(ClientId, pick_channel(ClientId), OnTakeover).
|
||||
|
||||
get_session_confs(#{zone := Zone, clientid := ClientId}, #{
|
||||
receive_maximum := MaxInflight, expiry_interval := EI
|
||||
}) ->
|
||||
#{
|
||||
clientid => ClientId,
|
||||
max_subscriptions => get_mqtt_conf(Zone, max_subscriptions),
|
||||
upgrade_qos => get_mqtt_conf(Zone, upgrade_qos),
|
||||
max_inflight => MaxInflight,
|
||||
retry_interval => get_mqtt_conf(Zone, retry_interval),
|
||||
await_rel_timeout => get_mqtt_conf(Zone, await_rel_timeout),
|
||||
max_awaiting_rel => get_mqtt_conf(Zone, max_awaiting_rel),
|
||||
mqueue => mqueue_confs(Zone),
|
||||
%% TODO: Add conf for allowing/disallowing persistent sessions.
|
||||
%% Note that the connection info is already enriched to have
|
||||
%% default config values for session expiry.
|
||||
is_persistent => EI > 0
|
||||
}.
|
||||
takeover_channel_session(ClientId, ChanPid, OnTakeover) when is_pid(ChanPid) ->
|
||||
case takeover_session(ClientId, ChanPid) of
|
||||
{living, ConnMod, Session} ->
|
||||
Session1 = OnTakeover(Session),
|
||||
case wrap_rpc(emqx_cm_proto_v2:takeover_finish(ConnMod, ChanPid)) of
|
||||
{ok, Pendings} ->
|
||||
{ok, Session1, Pendings};
|
||||
{error, _} = Error ->
|
||||
Error
|
||||
end;
|
||||
none ->
|
||||
none
|
||||
end;
|
||||
takeover_channel_session(_ClientId, undefined, _OnTakeover) ->
|
||||
none.
|
||||
|
||||
mqueue_confs(Zone) ->
|
||||
#{
|
||||
max_len => get_mqtt_conf(Zone, max_mqueue_len),
|
||||
store_qos0 => get_mqtt_conf(Zone, mqueue_store_qos0),
|
||||
priorities => get_mqtt_conf(Zone, mqueue_priorities),
|
||||
default_priority => get_mqtt_conf(Zone, mqueue_default_priority)
|
||||
}.
|
||||
|
||||
get_mqtt_conf(Zone, Key) ->
|
||||
emqx_config:get_zone_conf(Zone, [mqtt, Key]).
|
||||
|
||||
%% @doc Try to takeover a session.
|
||||
-spec takeover_session(emqx_types:clientid()) ->
|
||||
none
|
||||
| {living, atom(), pid(), emqx_session:session()}
|
||||
| {persistent, emqx_session:session()}
|
||||
| {expired, emqx_session:session()}.
|
||||
takeover_session(ClientId) ->
|
||||
-spec pick_channel(emqx_types:clientid()) ->
|
||||
maybe(pid()).
|
||||
pick_channel(ClientId) ->
|
||||
case lookup_channels(ClientId) of
|
||||
[] ->
|
||||
emqx_session:lookup(ClientId);
|
||||
undefined;
|
||||
[ChanPid] ->
|
||||
takeover_session(ClientId, ChanPid);
|
||||
ChanPid;
|
||||
ChanPids ->
|
||||
[ChanPid | StalePids] = lists:reverse(ChanPids),
|
||||
?SLOG(warning, #{msg => "more_than_one_channel_found", chan_pids => ChanPids}),
|
||||
|
@ -360,7 +328,7 @@ takeover_session(ClientId) ->
|
|||
end,
|
||||
StalePids
|
||||
),
|
||||
takeover_session(ClientId, ChanPid)
|
||||
ChanPid
|
||||
end.
|
||||
|
||||
takeover_finish(ConnMod, ChanPid) ->
|
||||
|
@ -370,9 +338,10 @@ takeover_finish(ConnMod, ChanPid) ->
|
|||
ChanPid
|
||||
).
|
||||
|
||||
%% @doc RPC Target @ emqx_cm_proto_v2:takeover_session/2
|
||||
takeover_session(ClientId, Pid) ->
|
||||
try
|
||||
do_takeover_session(ClientId, Pid)
|
||||
do_takeover_begin(ClientId, Pid)
|
||||
catch
|
||||
_:R when
|
||||
R == noproc;
|
||||
|
@ -380,25 +349,25 @@ takeover_session(ClientId, Pid) ->
|
|||
%% request_stepdown/3
|
||||
R == unexpected_exception
|
||||
->
|
||||
emqx_session:lookup(ClientId);
|
||||
none;
|
||||
% rpc_call/3
|
||||
_:{'EXIT', {noproc, _}} ->
|
||||
emqx_session:lookup(ClientId)
|
||||
none
|
||||
end.
|
||||
|
||||
do_takeover_session(ClientId, ChanPid) when node(ChanPid) == node() ->
|
||||
case get_chann_conn_mod(ClientId, ChanPid) of
|
||||
do_takeover_begin(ClientId, ChanPid) when node(ChanPid) == node() ->
|
||||
case do_get_chann_conn_mod(ClientId, ChanPid) of
|
||||
undefined ->
|
||||
emqx_session:lookup(ClientId);
|
||||
none;
|
||||
ConnMod when is_atom(ConnMod) ->
|
||||
case request_stepdown({takeover, 'begin'}, ConnMod, ChanPid) of
|
||||
{ok, Session} ->
|
||||
{living, ConnMod, ChanPid, Session};
|
||||
{living, ConnMod, Session};
|
||||
{error, Reason} ->
|
||||
error(Reason)
|
||||
end
|
||||
end;
|
||||
do_takeover_session(ClientId, ChanPid) ->
|
||||
do_takeover_begin(ClientId, ChanPid) ->
|
||||
wrap_rpc(emqx_cm_proto_v2:takeover_session(ClientId, ChanPid)).
|
||||
|
||||
%% @doc Discard all the sessions identified by the ClientId.
|
||||
|
@ -488,9 +457,10 @@ discard_session(ClientId, ChanPid) ->
|
|||
kick_session(ClientId, ChanPid) ->
|
||||
kick_session(kick, ClientId, ChanPid).
|
||||
|
||||
%% @doc RPC Target @ emqx_cm_proto_v2:kick_session/3
|
||||
-spec do_kick_session(kick | discard, emqx_types:clientid(), chan_pid()) -> ok.
|
||||
do_kick_session(Action, ClientId, ChanPid) ->
|
||||
case get_chann_conn_mod(ClientId, ChanPid) of
|
||||
do_kick_session(Action, ClientId, ChanPid) when node(ChanPid) =:= node() ->
|
||||
case do_get_chann_conn_mod(ClientId, ChanPid) of
|
||||
undefined ->
|
||||
%% already deregistered
|
||||
ok;
|
||||
|
@ -725,9 +695,6 @@ do_get_chann_conn_mod(ClientId, ChanPid) ->
|
|||
error:badarg -> undefined
|
||||
end.
|
||||
|
||||
get_chann_conn_mod(ClientId, ChanPid) ->
|
||||
wrap_rpc(emqx_cm_proto_v2:get_chann_conn_mod(ClientId, ChanPid)).
|
||||
|
||||
mark_channel_connected(ChanPid) ->
|
||||
?tp(emqx_cm_connected_client_count_inc, #{chan_pid => ChanPid}),
|
||||
ets:insert_new(?CHAN_LIVE_TAB, {ChanPid, true}),
|
||||
|
@ -744,14 +711,3 @@ get_connected_client_count() ->
|
|||
undefined -> 0;
|
||||
Size -> Size
|
||||
end.
|
||||
|
||||
clean_session(Session) ->
|
||||
emqx_session:filter_queue(fun is_banned_msg/1, Session).
|
||||
|
||||
clean_pendings(Pendings) ->
|
||||
lists:filter(fun is_banned_msg/1, Pendings).
|
||||
|
||||
is_banned_msg(#message{from = ClientId}) ->
|
||||
[] =:= emqx_banned:look_up({clientid, ClientId});
|
||||
is_banned_msg({deliver, _Topic, Msg}) ->
|
||||
is_banned_msg(Msg).
|
||||
|
|
|
@ -0,0 +1,99 @@
|
|||
%%--------------------------------------------------------------------
|
||||
%% Copyright (c) 2021-2023 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.
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-module(emqx_persistent_message).
|
||||
|
||||
-include("emqx.hrl").
|
||||
|
||||
-export([init/0]).
|
||||
-export([is_store_enabled/0]).
|
||||
|
||||
%% Message persistence
|
||||
-export([
|
||||
persist/1,
|
||||
serialize/1,
|
||||
deserialize/1
|
||||
]).
|
||||
|
||||
%% FIXME
|
||||
-define(DS_SHARD_ID, <<"local">>).
|
||||
-define(DEFAULT_KEYSPACE, default).
|
||||
-define(DS_SHARD, {?DEFAULT_KEYSPACE, ?DS_SHARD_ID}).
|
||||
|
||||
-define(WHEN_ENABLED(DO),
|
||||
case is_store_enabled() of
|
||||
true -> DO;
|
||||
false -> {skipped, disabled}
|
||||
end
|
||||
).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
init() ->
|
||||
?WHEN_ENABLED(begin
|
||||
ok = emqx_ds:ensure_shard(
|
||||
?DS_SHARD,
|
||||
#{
|
||||
dir => filename:join([
|
||||
emqx:data_dir(),
|
||||
ds,
|
||||
messages,
|
||||
?DEFAULT_KEYSPACE,
|
||||
?DS_SHARD_ID
|
||||
])
|
||||
}
|
||||
),
|
||||
ok = emqx_persistent_session_ds_router:init_tables(),
|
||||
ok
|
||||
end).
|
||||
|
||||
-spec is_store_enabled() -> boolean().
|
||||
is_store_enabled() ->
|
||||
emqx_config:get([persistent_session_store, ds]).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec persist(emqx_types:message()) ->
|
||||
ok | {skipped, _Reason} | {error, _TODO}.
|
||||
persist(Msg) ->
|
||||
?WHEN_ENABLED(
|
||||
case needs_persistence(Msg) andalso has_subscribers(Msg) of
|
||||
true ->
|
||||
store_message(Msg);
|
||||
false ->
|
||||
{skipped, needs_no_persistence}
|
||||
end
|
||||
).
|
||||
|
||||
needs_persistence(Msg) ->
|
||||
not (emqx_message:get_flag(dup, Msg) orelse emqx_message:is_sys(Msg)).
|
||||
|
||||
store_message(Msg) ->
|
||||
ID = emqx_message:id(Msg),
|
||||
Timestamp = emqx_guid:timestamp(ID),
|
||||
Topic = emqx_topic:words(emqx_message:topic(Msg)),
|
||||
emqx_ds_storage_layer:store(?DS_SHARD, ID, Timestamp, Topic, serialize(Msg)).
|
||||
|
||||
has_subscribers(#message{topic = Topic}) ->
|
||||
emqx_persistent_session_ds_router:has_any_route(Topic).
|
||||
|
||||
%%
|
||||
|
||||
serialize(Msg) ->
|
||||
term_to_binary(emqx_message:to_map(Msg)).
|
||||
|
||||
deserialize(Bin) ->
|
||||
emqx_message:from_map(binary_to_term(Bin)).
|
|
@ -19,18 +19,43 @@
|
|||
-include("emqx.hrl").
|
||||
-include_lib("snabbkaffe/include/snabbkaffe.hrl").
|
||||
|
||||
-export([init/0]).
|
||||
-include("emqx_mqtt.hrl").
|
||||
|
||||
%% Session API
|
||||
-export([
|
||||
persist_message/1,
|
||||
open_session/1,
|
||||
add_subscription/2,
|
||||
del_subscription/2
|
||||
lookup/1,
|
||||
destroy/1
|
||||
]).
|
||||
|
||||
-export([
|
||||
serialize_message/1,
|
||||
deserialize_message/1
|
||||
create/3,
|
||||
open/2
|
||||
]).
|
||||
|
||||
-export([
|
||||
info/2,
|
||||
stats/1
|
||||
]).
|
||||
|
||||
-export([
|
||||
subscribe/3,
|
||||
unsubscribe/2,
|
||||
get_subscription/2
|
||||
]).
|
||||
|
||||
-export([
|
||||
publish/3,
|
||||
puback/3,
|
||||
pubrec/2,
|
||||
pubrel/2,
|
||||
pubcomp/3
|
||||
]).
|
||||
|
||||
-export([
|
||||
deliver/3,
|
||||
% handle_timeout/3,
|
||||
disconnect/1,
|
||||
terminate/2
|
||||
]).
|
||||
|
||||
%% RPC
|
||||
|
@ -49,106 +74,265 @@
|
|||
-define(DEFAULT_KEYSPACE, default).
|
||||
-define(DS_SHARD, {?DEFAULT_KEYSPACE, ?DS_SHARD_ID}).
|
||||
|
||||
-define(WHEN_ENABLED(DO),
|
||||
case is_store_enabled() of
|
||||
true -> DO;
|
||||
false -> {skipped, disabled}
|
||||
end
|
||||
).
|
||||
-record(sessionds, {
|
||||
%% Client ID
|
||||
id :: binary(),
|
||||
%% Client’s Subscriptions.
|
||||
subscriptions :: map(),
|
||||
iterators :: map(),
|
||||
%%
|
||||
conf
|
||||
}).
|
||||
|
||||
-type session() :: #sessionds{}.
|
||||
|
||||
-type clientinfo() :: emqx_types:clientinfo().
|
||||
-type conninfo() :: emqx_types:conninfo().
|
||||
-type replies() :: emqx_session:replies().
|
||||
|
||||
%%
|
||||
|
||||
init() ->
|
||||
?WHEN_ENABLED(begin
|
||||
ok = emqx_ds:ensure_shard(
|
||||
?DS_SHARD,
|
||||
#{
|
||||
dir => filename:join([
|
||||
emqx:data_dir(),
|
||||
ds,
|
||||
messages,
|
||||
?DEFAULT_KEYSPACE,
|
||||
?DS_SHARD_ID
|
||||
])
|
||||
}
|
||||
),
|
||||
ok = emqx_persistent_session_ds_router:init_tables(),
|
||||
ok
|
||||
end).
|
||||
-spec create(clientinfo(), conninfo(), emqx_session:conf()) ->
|
||||
session().
|
||||
create(#{clientid := ClientID}, _ConnInfo, Conf) ->
|
||||
#sessionds{
|
||||
id = ClientID,
|
||||
subscriptions = #{},
|
||||
conf = Conf
|
||||
}.
|
||||
|
||||
%%
|
||||
-spec open(clientinfo(), conninfo()) ->
|
||||
{true, session()} | false.
|
||||
open(#{clientid := ClientID}, _ConnInfo) ->
|
||||
open_session(ClientID).
|
||||
|
||||
-spec persist_message(emqx_types:message()) ->
|
||||
ok | {skipped, _Reason} | {error, _TODO}.
|
||||
persist_message(Msg) ->
|
||||
?WHEN_ENABLED(
|
||||
case needs_persistence(Msg) andalso has_subscribers(Msg) of
|
||||
true ->
|
||||
store_message(Msg);
|
||||
false ->
|
||||
{skipped, needs_no_persistence}
|
||||
end
|
||||
).
|
||||
-spec lookup(emqx_types:clientinfo()) -> none.
|
||||
lookup(_ClientInfo) ->
|
||||
'TODO'.
|
||||
|
||||
needs_persistence(Msg) ->
|
||||
not (emqx_message:get_flag(dup, Msg) orelse emqx_message:is_sys(Msg)).
|
||||
-spec destroy(session() | clientinfo()) -> ok.
|
||||
destroy(#{clientid := ClientID}) ->
|
||||
emqx_ds:session_drop(ClientID).
|
||||
|
||||
store_message(Msg) ->
|
||||
ID = emqx_message:id(Msg),
|
||||
Timestamp = emqx_guid:timestamp(ID),
|
||||
Topic = emqx_topic:words(emqx_message:topic(Msg)),
|
||||
emqx_ds_storage_layer:store(
|
||||
?DS_SHARD, ID, Timestamp, Topic, serialize_message(Msg)
|
||||
).
|
||||
%%--------------------------------------------------------------------
|
||||
%% Info, Stats
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
has_subscribers(#message{topic = Topic}) ->
|
||||
emqx_persistent_session_ds_router:has_any_route(Topic).
|
||||
info(Keys, Session) when is_list(Keys) ->
|
||||
[{Key, info(Key, Session)} || Key <- Keys];
|
||||
info(id, #sessionds{id = ClientID}) ->
|
||||
ClientID;
|
||||
info(clientid, #sessionds{id = ClientID}) ->
|
||||
ClientID;
|
||||
% info(created_at, #sessionds{created_at = CreatedAt}) ->
|
||||
% CreatedAt;
|
||||
info(is_persistent, #sessionds{}) ->
|
||||
true;
|
||||
info(subscriptions, #sessionds{subscriptions = Subs}) ->
|
||||
Subs;
|
||||
info(subscriptions_cnt, #sessionds{subscriptions = Subs}) ->
|
||||
maps:size(Subs);
|
||||
info(subscriptions_max, #sessionds{conf = Conf}) ->
|
||||
maps:get(max_subscriptions, Conf);
|
||||
info(upgrade_qos, #sessionds{conf = Conf}) ->
|
||||
maps:get(upgrade_qos, Conf);
|
||||
% info(inflight, #sessmem{inflight = Inflight}) ->
|
||||
% Inflight;
|
||||
% info(inflight_cnt, #sessmem{inflight = Inflight}) ->
|
||||
% emqx_inflight:size(Inflight);
|
||||
% info(inflight_max, #sessmem{inflight = Inflight}) ->
|
||||
% emqx_inflight:max_size(Inflight);
|
||||
info(retry_interval, #sessionds{conf = Conf}) ->
|
||||
maps:get(retry_interval, Conf);
|
||||
% info(mqueue, #sessmem{mqueue = MQueue}) ->
|
||||
% MQueue;
|
||||
% info(mqueue_len, #sessmem{mqueue = MQueue}) ->
|
||||
% emqx_mqueue:len(MQueue);
|
||||
% info(mqueue_max, #sessmem{mqueue = MQueue}) ->
|
||||
% emqx_mqueue:max_len(MQueue);
|
||||
% info(mqueue_dropped, #sessmem{mqueue = MQueue}) ->
|
||||
% emqx_mqueue:dropped(MQueue);
|
||||
info(next_pkt_id, #sessionds{}) ->
|
||||
_PacketId = 'TODO';
|
||||
% info(awaiting_rel, #sessmem{awaiting_rel = AwaitingRel}) ->
|
||||
% AwaitingRel;
|
||||
% info(awaiting_rel_cnt, #sessmem{awaiting_rel = AwaitingRel}) ->
|
||||
% maps:size(AwaitingRel);
|
||||
info(awaiting_rel_max, #sessionds{conf = Conf}) ->
|
||||
maps:get(max_awaiting_rel, Conf);
|
||||
info(await_rel_timeout, #sessionds{conf = Conf}) ->
|
||||
maps:get(await_rel_timeout, Conf).
|
||||
|
||||
-spec stats(session()) -> emqx_types:stats().
|
||||
stats(Session) ->
|
||||
% TODO: stub
|
||||
info([], Session).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Client -> Broker: SUBSCRIBE / UNSUBSCRIBE
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec subscribe(emqx_types:topic(), emqx_types:subopts(), session()) ->
|
||||
{ok, session()} | {error, emqx_types:reason_code()}.
|
||||
subscribe(
|
||||
TopicFilter,
|
||||
SubOpts,
|
||||
Session = #sessionds{subscriptions = Subs}
|
||||
) when is_map_key(TopicFilter, Subs) ->
|
||||
{ok, Session#sessionds{
|
||||
subscriptions = Subs#{TopicFilter => SubOpts}
|
||||
}};
|
||||
subscribe(
|
||||
TopicFilter,
|
||||
SubOpts,
|
||||
Session = #sessionds{id = ClientID, subscriptions = Subs, iterators = Iters}
|
||||
) ->
|
||||
% TODO: max_subscriptions
|
||||
IteratorID = add_subscription(TopicFilter, ClientID),
|
||||
{ok, Session#sessionds{
|
||||
subscriptions = Subs#{TopicFilter => SubOpts},
|
||||
iterators = Iters#{TopicFilter => IteratorID}
|
||||
}}.
|
||||
|
||||
-spec unsubscribe(emqx_types:topic(), session()) ->
|
||||
{ok, session(), emqx_types:subopts()} | {error, emqx_types:reason_code()}.
|
||||
unsubscribe(
|
||||
TopicFilter,
|
||||
Session = #sessionds{id = ClientID, subscriptions = Subs, iterators = Iters}
|
||||
) when is_map_key(TopicFilter, Subs) ->
|
||||
IteratorID = maps:get(TopicFilter, Iters),
|
||||
ok = del_subscription(IteratorID, TopicFilter, ClientID),
|
||||
{ok, Session#sessionds{
|
||||
subscriptions = maps:remove(TopicFilter, Subs),
|
||||
iterators = maps:remove(TopicFilter, Iters)
|
||||
}};
|
||||
unsubscribe(
|
||||
_TopicFilter,
|
||||
_Session = #sessionds{}
|
||||
) ->
|
||||
{error, ?RC_NO_SUBSCRIPTION_EXISTED}.
|
||||
|
||||
-spec get_subscription(emqx_types:topic(), session()) ->
|
||||
emqx_types:subopts() | undefined.
|
||||
get_subscription(TopicFilter, #sessionds{subscriptions = Subs}) ->
|
||||
maps:get(TopicFilter, Subs, undefined).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Client -> Broker: PUBLISH
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec publish(emqx_types:packet_id(), emqx_types:message(), session()) ->
|
||||
{ok, emqx_types:publish_result(), replies(), session()}
|
||||
| {error, emqx_types:reason_code()}.
|
||||
publish(_PacketId, Msg, Session) ->
|
||||
% TODO: stub
|
||||
{ok, emqx_broker:publish(Msg), [], Session}.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Client -> Broker: PUBACK
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec puback(clientinfo(), emqx_types:packet_id(), session()) ->
|
||||
{ok, emqx_types:message(), replies(), session()}
|
||||
| {error, emqx_types:reason_code()}.
|
||||
puback(_ClientInfo, _PacketId, _Session = #sessionds{}) ->
|
||||
% TODO: stub
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND}.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Client -> Broker: PUBREC
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec pubrec(emqx_types:packet_id(), session()) ->
|
||||
{ok, emqx_types:message(), session()}
|
||||
| {error, emqx_types:reason_code()}.
|
||||
pubrec(_PacketId, _Session = #sessionds{}) ->
|
||||
% TODO: stub
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND}.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Client -> Broker: PUBREL
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec pubrel(emqx_types:packet_id(), session()) ->
|
||||
{ok, session()} | {error, emqx_types:reason_code()}.
|
||||
pubrel(_PacketId, Session = #sessionds{}) ->
|
||||
% TODO: stub
|
||||
{ok, Session}.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Client -> Broker: PUBCOMP
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec pubcomp(clientinfo(), emqx_types:packet_id(), session()) ->
|
||||
{ok, emqx_types:message(), replies(), session()}
|
||||
| {error, emqx_types:reason_code()}.
|
||||
pubcomp(_ClientInfo, _PacketId, _Session = #sessionds{}) ->
|
||||
% TODO: stub
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND}.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec deliver(clientinfo(), [emqx_types:deliver()], session()) ->
|
||||
{ok, replies(), session()}.
|
||||
deliver(_ClientInfo, _Delivers, _Session = #sessionds{}) ->
|
||||
% TODO: ensure it's unreachable somehow
|
||||
error(unexpected).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec disconnect(session()) -> {shutdown, session()}.
|
||||
disconnect(Session = #sessionds{}) ->
|
||||
{shutdown, Session}.
|
||||
|
||||
-spec terminate(Reason :: term(), session()) -> ok.
|
||||
terminate(_Reason, _Session = #sessionds{}) ->
|
||||
% TODO: close iterators
|
||||
ok.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
open_session(ClientID) ->
|
||||
?WHEN_ENABLED(emqx_ds:session_open(ClientID)).
|
||||
emqx_ds:session_open(ClientID).
|
||||
|
||||
-spec add_subscription(emqx_types:topic(), emqx_ds:session_id()) ->
|
||||
{ok, emqx_ds:iterator_id(), IsNew :: boolean()} | {skipped, disabled}.
|
||||
emqx_ds:iterator_id().
|
||||
add_subscription(TopicFilterBin, DSSessionID) ->
|
||||
?WHEN_ENABLED(
|
||||
begin
|
||||
%% N.B.: we chose to update the router before adding the subscription to the
|
||||
%% session/iterator table. The reasoning for this is as follows:
|
||||
%%
|
||||
%% Messages matching this topic filter should start to be persisted as soon as
|
||||
%% possible to avoid missing messages. If this is the first such persistent
|
||||
%% session subscription, it's important to do so early on.
|
||||
%%
|
||||
%% This could, in turn, lead to some inconsistency: if such a route gets
|
||||
%% created but the session/iterator data fails to be updated accordingly, we
|
||||
%% have a dangling route. To remove such dangling routes, we may have a
|
||||
%% periodic GC process that removes routes that do not have a matching
|
||||
%% persistent subscription. Also, route operations use dirty mnesia
|
||||
%% operations, which inherently have room for inconsistencies.
|
||||
%%
|
||||
%% In practice, we use the iterator reference table as a source of truth,
|
||||
%% since it is guarded by a transaction context: we consider a subscription
|
||||
%% operation to be successful if it ended up changing this table. Both router
|
||||
%% and iterator information can be reconstructed from this table, if needed.
|
||||
ok = emqx_persistent_session_ds_router:do_add_route(TopicFilterBin, DSSessionID),
|
||||
TopicFilter = emqx_topic:words(TopicFilterBin),
|
||||
{ok, IteratorID, StartMS, IsNew} = emqx_ds:session_add_iterator(
|
||||
DSSessionID, TopicFilter
|
||||
),
|
||||
Ctx = #{
|
||||
iterator_id => IteratorID,
|
||||
start_time => StartMS,
|
||||
is_new => IsNew
|
||||
},
|
||||
?tp(persistent_session_ds_iterator_added, Ctx),
|
||||
?tp_span(
|
||||
persistent_session_ds_open_iterators,
|
||||
Ctx,
|
||||
ok = open_iterator_on_all_shards(TopicFilter, StartMS, IteratorID)
|
||||
),
|
||||
{ok, IteratorID, IsNew}
|
||||
end
|
||||
).
|
||||
% N.B.: we chose to update the router before adding the subscription to the
|
||||
% session/iterator table. The reasoning for this is as follows:
|
||||
%
|
||||
% Messages matching this topic filter should start to be persisted as soon as
|
||||
% possible to avoid missing messages. If this is the first such persistent
|
||||
% session subscription, it's important to do so early on.
|
||||
%
|
||||
% This could, in turn, lead to some inconsistency: if such a route gets
|
||||
% created but the session/iterator data fails to be updated accordingly, we
|
||||
% have a dangling route. To remove such dangling routes, we may have a
|
||||
% periodic GC process that removes routes that do not have a matching
|
||||
% persistent subscription. Also, route operations use dirty mnesia
|
||||
% operations, which inherently have room for inconsistencies.
|
||||
%
|
||||
% In practice, we use the iterator reference table as a source of truth,
|
||||
% since it is guarded by a transaction context: we consider a subscription
|
||||
% operation to be successful if it ended up changing this table. Both router
|
||||
% and iterator information can be reconstructed from this table, if needed.
|
||||
ok = emqx_persistent_session_ds_router:do_add_route(TopicFilterBin, DSSessionID),
|
||||
TopicFilter = emqx_topic:words(TopicFilterBin),
|
||||
{ok, IteratorID, StartMS, IsNew} = emqx_ds:session_add_iterator(
|
||||
DSSessionID, TopicFilter
|
||||
),
|
||||
Ctx = #{
|
||||
iterator_id => IteratorID,
|
||||
start_time => StartMS,
|
||||
is_new => IsNew
|
||||
},
|
||||
?tp(persistent_session_ds_iterator_added, Ctx),
|
||||
?tp_span(
|
||||
persistent_session_ds_open_iterators,
|
||||
Ctx,
|
||||
ok = open_iterator_on_all_shards(TopicFilter, StartMS, IteratorID)
|
||||
),
|
||||
IteratorID.
|
||||
|
||||
-spec open_iterator_on_all_shards(emqx_topic:words(), emqx_ds:time(), emqx_ds:iterator_id()) -> ok.
|
||||
open_iterator_on_all_shards(TopicFilter, StartMS, IteratorID) ->
|
||||
|
@ -161,45 +345,38 @@ open_iterator_on_all_shards(TopicFilter, StartMS, IteratorID) ->
|
|||
Results = emqx_persistent_session_ds_proto_v1:open_iterator(
|
||||
Nodes, TopicFilter, StartMS, IteratorID
|
||||
),
|
||||
%% TODO: handle errors
|
||||
true = lists:all(fun(Res) -> Res =:= {ok, ok} end, Results),
|
||||
%% TODO
|
||||
%% 1. Handle errors.
|
||||
%% 2. Iterator handles are rocksdb resources, it's doubtful they survive RPC.
|
||||
%% Even if they do, we throw them away here anyway. All in all, we probably should
|
||||
%% hold each of them in a process on the respective node.
|
||||
true = lists:all(fun(Res) -> element(1, Res) =:= ok end, Results),
|
||||
ok.
|
||||
|
||||
%% RPC target.
|
||||
-spec do_open_iterator(emqx_topic:words(), emqx_ds:time(), emqx_ds:iterator_id()) -> ok.
|
||||
do_open_iterator(TopicFilter, StartMS, IteratorID) ->
|
||||
Replay = {TopicFilter, StartMS},
|
||||
{ok, _It} = emqx_ds_storage_layer:ensure_iterator(?DS_SHARD, IteratorID, Replay),
|
||||
ok.
|
||||
emqx_ds_storage_layer:ensure_iterator(?DS_SHARD, IteratorID, Replay).
|
||||
|
||||
-spec del_subscription(emqx_types:topic(), emqx_ds:session_id()) ->
|
||||
ok | {skipped, disabled}.
|
||||
del_subscription(TopicFilterBin, DSSessionID) ->
|
||||
?WHEN_ENABLED(
|
||||
begin
|
||||
%% N.B.: see comments in `?MODULE:add_subscription' for a discussion about the
|
||||
%% order of operations here.
|
||||
TopicFilter = emqx_topic:words(TopicFilterBin),
|
||||
case emqx_ds:session_get_iterator_id(DSSessionID, TopicFilter) of
|
||||
{error, not_found} ->
|
||||
%% already gone
|
||||
ok;
|
||||
{ok, IteratorID} ->
|
||||
?tp_span(
|
||||
persistent_session_ds_close_iterators,
|
||||
#{iterator_id => IteratorID},
|
||||
ok = ensure_iterator_closed_on_all_shards(IteratorID)
|
||||
)
|
||||
end,
|
||||
?tp_span(
|
||||
persistent_session_ds_iterator_delete,
|
||||
#{},
|
||||
emqx_ds:session_del_iterator(DSSessionID, TopicFilter)
|
||||
),
|
||||
ok = emqx_persistent_session_ds_router:do_delete_route(TopicFilterBin, DSSessionID),
|
||||
ok
|
||||
end
|
||||
).
|
||||
-spec del_subscription(emqx_ds:iterator_id() | undefined, emqx_types:topic(), emqx_ds:session_id()) ->
|
||||
ok.
|
||||
del_subscription(IteratorID, TopicFilterBin, DSSessionID) ->
|
||||
% N.B.: see comments in `?MODULE:add_subscription' for a discussion about the
|
||||
% order of operations here.
|
||||
TopicFilter = emqx_topic:words(TopicFilterBin),
|
||||
Ctx = #{iterator_id => IteratorID},
|
||||
?tp_span(
|
||||
persistent_session_ds_close_iterators,
|
||||
Ctx,
|
||||
ok = ensure_iterator_closed_on_all_shards(IteratorID)
|
||||
),
|
||||
?tp_span(
|
||||
persistent_session_ds_iterator_delete,
|
||||
Ctx,
|
||||
emqx_ds:session_del_iterator(DSSessionID, TopicFilter)
|
||||
),
|
||||
ok = emqx_persistent_session_ds_router:do_delete_route(TopicFilterBin, DSSessionID).
|
||||
|
||||
-spec ensure_iterator_closed_on_all_shards(emqx_ds:iterator_id()) -> ok.
|
||||
ensure_iterator_closed_on_all_shards(IteratorID) ->
|
||||
|
@ -230,16 +407,3 @@ ensure_all_iterators_closed(DSSessionID) ->
|
|||
do_ensure_all_iterators_closed(DSSessionID) ->
|
||||
ok = emqx_ds_storage_layer:discard_iterator_prefix(?DS_SHARD, DSSessionID),
|
||||
ok.
|
||||
|
||||
%%
|
||||
|
||||
serialize_message(Msg) ->
|
||||
term_to_binary(emqx_message:to_map(Msg)).
|
||||
|
||||
deserialize_message(Bin) ->
|
||||
emqx_message:from_map(binary_to_term(Bin)).
|
||||
|
||||
%%
|
||||
|
||||
is_store_enabled() ->
|
||||
emqx_config:get([persistent_session_store, ds]).
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,94 @@
|
|||
%%--------------------------------------------------------------------
|
||||
%% Copyright (c) 2023 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.
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-module(emqx_session_events).
|
||||
|
||||
-include("emqx.hrl").
|
||||
-include("logger.hrl").
|
||||
|
||||
-export([handle_event/2]).
|
||||
|
||||
-type event_expired() :: {expired, emqx_types:message()}.
|
||||
-type event_dropped() :: {dropped, emqx_types:message(), _Reason :: atom()}.
|
||||
-type event_expire_rel() :: {expired_rel, non_neg_integer()}.
|
||||
|
||||
-type event() ::
|
||||
event_expired()
|
||||
| event_dropped()
|
||||
| event_expire_rel().
|
||||
|
||||
%%
|
||||
|
||||
-spec handle_event(emqx_session:client_info(), event()) ->
|
||||
ok.
|
||||
handle_event(ClientInfo, {expired, Msg}) ->
|
||||
ok = emqx_hooks:run('delivery.dropped', [ClientInfo, Msg, expired]),
|
||||
ok = inc_delivery_expired_cnt(1);
|
||||
handle_event(ClientInfo, {dropped, Msg, qos0_msg}) ->
|
||||
ok = emqx_hooks:run('delivery.dropped', [ClientInfo, Msg, qos0_msg]),
|
||||
ok = emqx_metrics:inc('delivery.dropped'),
|
||||
ok = emqx_metrics:inc('delivery.dropped.qos0_msg'),
|
||||
ok = inc_pd('send_msg.dropped', 1),
|
||||
?SLOG(
|
||||
warning,
|
||||
#{
|
||||
msg => "dropped_qos0_msg",
|
||||
% FIXME
|
||||
% queue => QueueInfo,
|
||||
payload => Msg#message.payload
|
||||
},
|
||||
#{topic => Msg#message.topic}
|
||||
);
|
||||
handle_event(ClientInfo, {dropped, Msg, queue_full}) ->
|
||||
ok = emqx_hooks:run('delivery.dropped', [ClientInfo, Msg, queue_full]),
|
||||
ok = emqx_metrics:inc('delivery.dropped'),
|
||||
ok = emqx_metrics:inc('delivery.dropped.queue_full'),
|
||||
ok = inc_pd('send_msg.dropped', 1),
|
||||
ok = inc_pd('send_msg.dropped.queue_full', 1),
|
||||
?SLOG(
|
||||
warning,
|
||||
#{
|
||||
msg => "dropped_msg_due_to_mqueue_is_full",
|
||||
% FIXME
|
||||
% queue => QueueInfo,
|
||||
payload => Msg#message.payload
|
||||
},
|
||||
#{topic => Msg#message.topic}
|
||||
);
|
||||
handle_event(ClientInfo, {dropped, Msg, no_local}) ->
|
||||
ok = emqx_hooks:run('delivery.dropped', [ClientInfo, Msg, no_local]),
|
||||
ok = emqx_metrics:inc('delivery.dropped'),
|
||||
ok = emqx_metrics:inc('delivery.dropped.no_local');
|
||||
handle_event(_ClientInfo, {expired_rel, 0}) ->
|
||||
ok;
|
||||
handle_event(_ClientInfo, {expired_rel, ExpiredCnt}) ->
|
||||
inc_await_pubrel_timeout(ExpiredCnt).
|
||||
|
||||
inc_delivery_expired_cnt(N) ->
|
||||
ok = inc_pd('send_msg.dropped', N),
|
||||
ok = inc_pd('send_msg.dropped.expired', N),
|
||||
ok = emqx_metrics:inc('delivery.dropped', N),
|
||||
emqx_metrics:inc('delivery.dropped.expired', N).
|
||||
|
||||
inc_await_pubrel_timeout(N) ->
|
||||
ok = inc_pd('recv_msg.dropped', N),
|
||||
ok = inc_pd('recv_msg.dropped.await_pubrel_timeout', N),
|
||||
ok = emqx_metrics:inc('messages.dropped', N),
|
||||
emqx_metrics:inc('messages.dropped.await_pubrel_timeout', N).
|
||||
|
||||
inc_pd(Key, Inc) ->
|
||||
_ = emqx_pd:inc_counter(Key, Inc),
|
||||
ok.
|
|
@ -0,0 +1,823 @@
|
|||
%%--------------------------------------------------------------------
|
||||
%% Copyright (c) 2017-2023 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.
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% @doc
|
||||
%% A stateful interaction between a Client and a Server. Some Sessions
|
||||
%% last only as long as the Network Connection, others can span multiple
|
||||
%% consecutive Network Connections between a Client and a Server.
|
||||
%%
|
||||
%% The Session State in the Server consists of:
|
||||
%%
|
||||
%% The existence of a Session, even if the rest of the Session State is empty.
|
||||
%%
|
||||
%% The Clients subscriptions, including any Subscription Identifiers.
|
||||
%%
|
||||
%% QoS 1 and QoS 2 messages which have been sent to the Client, but have not
|
||||
%% been completely acknowledged.
|
||||
%%
|
||||
%% QoS 1 and QoS 2 messages pending transmission to the Client and OPTIONALLY
|
||||
%% QoS 0 messages pending transmission to the Client.
|
||||
%%
|
||||
%% QoS 2 messages which have been received from the Client, but have not been
|
||||
%% completely acknowledged.The Will Message and the Will Delay Interval
|
||||
%%
|
||||
%% If the Session is currently not connected, the time at which the Session
|
||||
%% will end and Session State will be discarded.
|
||||
%% @end
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
%% MQTT Session implementation
|
||||
%% State is stored in-memory in the process heap.
|
||||
-module(emqx_session_mem).
|
||||
|
||||
-include("emqx.hrl").
|
||||
-include("emqx_mqtt.hrl").
|
||||
-include("emqx_session_mem.hrl").
|
||||
-include("logger.hrl").
|
||||
-include("types.hrl").
|
||||
|
||||
-ifdef(TEST).
|
||||
-compile(export_all).
|
||||
-compile(nowarn_export_all).
|
||||
-endif.
|
||||
|
||||
-export([
|
||||
lookup/1,
|
||||
destroy/1
|
||||
]).
|
||||
|
||||
-export([
|
||||
create/3,
|
||||
open/2
|
||||
]).
|
||||
|
||||
-export([
|
||||
info/2,
|
||||
stats/1,
|
||||
obtain_next_pkt_id/1
|
||||
]).
|
||||
|
||||
-export([
|
||||
subscribe/3,
|
||||
unsubscribe/2,
|
||||
get_subscription/2
|
||||
]).
|
||||
|
||||
-export([
|
||||
publish/3,
|
||||
puback/3,
|
||||
pubrec/2,
|
||||
pubrel/2,
|
||||
pubcomp/3
|
||||
]).
|
||||
|
||||
-export([
|
||||
deliver/3,
|
||||
replay/3,
|
||||
handle_timeout/3,
|
||||
disconnect/1,
|
||||
terminate/2
|
||||
]).
|
||||
|
||||
-export([
|
||||
retry/2,
|
||||
expire/2
|
||||
]).
|
||||
|
||||
%% Part of takeover sequence
|
||||
-export([
|
||||
takeover/1,
|
||||
resume/2,
|
||||
enqueue/3,
|
||||
dequeue/2,
|
||||
replay/2
|
||||
]).
|
||||
|
||||
%% Export for CT
|
||||
-export([set_field/3]).
|
||||
|
||||
-type session_id() :: emqx_guid:guid().
|
||||
|
||||
-export_type([
|
||||
session/0,
|
||||
session_id/0
|
||||
]).
|
||||
|
||||
-type inflight_data_phase() :: wait_ack | wait_comp.
|
||||
|
||||
-record(inflight_data, {
|
||||
phase :: inflight_data_phase(),
|
||||
message :: emqx_types:message(),
|
||||
timestamp :: non_neg_integer()
|
||||
}).
|
||||
|
||||
-type session() :: #session{}.
|
||||
|
||||
-type clientinfo() :: emqx_types:clientinfo().
|
||||
-type conninfo() :: emqx_session:conninfo().
|
||||
-type replies() :: emqx_session:replies().
|
||||
|
||||
-define(STATS_KEYS, [
|
||||
subscriptions_cnt,
|
||||
subscriptions_max,
|
||||
inflight_cnt,
|
||||
inflight_max,
|
||||
mqueue_len,
|
||||
mqueue_max,
|
||||
mqueue_dropped,
|
||||
next_pkt_id,
|
||||
awaiting_rel_cnt,
|
||||
awaiting_rel_max
|
||||
]).
|
||||
|
||||
-define(DEFAULT_BATCH_N, 1000).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Init a Session
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec create(clientinfo(), conninfo(), emqx_session:conf()) ->
|
||||
session().
|
||||
create(#{zone := Zone, clientid := ClientId}, #{expiry_interval := EI}, Conf) ->
|
||||
QueueOpts = get_mqueue_conf(Zone),
|
||||
#session{
|
||||
id = emqx_guid:gen(),
|
||||
clientid = ClientId,
|
||||
created_at = erlang:system_time(millisecond),
|
||||
is_persistent = EI > 0,
|
||||
subscriptions = #{},
|
||||
inflight = emqx_inflight:new(maps:get(max_inflight, Conf)),
|
||||
mqueue = emqx_mqueue:init(QueueOpts),
|
||||
next_pkt_id = 1,
|
||||
awaiting_rel = #{},
|
||||
timers = #{},
|
||||
max_subscriptions = maps:get(max_subscriptions, Conf),
|
||||
max_awaiting_rel = maps:get(max_awaiting_rel, Conf),
|
||||
upgrade_qos = maps:get(upgrade_qos, Conf),
|
||||
retry_interval = maps:get(retry_interval, Conf),
|
||||
await_rel_timeout = maps:get(await_rel_timeout, Conf)
|
||||
}.
|
||||
|
||||
get_mqueue_conf(Zone) ->
|
||||
#{
|
||||
max_len => get_mqtt_conf(Zone, max_mqueue_len, 1000),
|
||||
store_qos0 => get_mqtt_conf(Zone, mqueue_store_qos0),
|
||||
priorities => get_mqtt_conf(Zone, mqueue_priorities),
|
||||
default_priority => get_mqtt_conf(Zone, mqueue_default_priority)
|
||||
}.
|
||||
|
||||
get_mqtt_conf(Zone, Key) ->
|
||||
emqx_config:get_zone_conf(Zone, [mqtt, Key]).
|
||||
|
||||
get_mqtt_conf(Zone, Key, Default) ->
|
||||
emqx_config:get_zone_conf(Zone, [mqtt, Key], Default).
|
||||
|
||||
-spec lookup(emqx_types:clientinfo()) -> none.
|
||||
lookup(_ClientInfo) ->
|
||||
% NOTE
|
||||
% This is a stub. This session impl has no backing store, thus always `none`.
|
||||
none.
|
||||
|
||||
-spec destroy(session() | clientinfo()) -> ok.
|
||||
destroy(_Session) ->
|
||||
% NOTE
|
||||
% This is a stub. This session impl has no backing store, thus always `ok`.
|
||||
ok.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Open a (possibly existing) Session
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec open(clientinfo(), emqx_types:conninfo()) ->
|
||||
{true, session(), _ReplayContext :: [emqx_types:message()]} | false.
|
||||
open(ClientInfo = #{clientid := ClientId}, _ConnInfo) ->
|
||||
case
|
||||
emqx_cm:takeover_channel_session(
|
||||
ClientId,
|
||||
fun(Session) -> resume(ClientInfo, Session) end
|
||||
)
|
||||
of
|
||||
{ok, Session, Pendings} ->
|
||||
clean_session(ClientInfo, Session, Pendings);
|
||||
{error, _} ->
|
||||
% TODO log error?
|
||||
false;
|
||||
none ->
|
||||
false
|
||||
end.
|
||||
|
||||
clean_session(ClientInfo, Session = #session{mqueue = Q}, Pendings) ->
|
||||
Q1 = emqx_mqueue:filter(fun emqx_session:should_discard/1, Q),
|
||||
Session1 = Session#session{mqueue = Q1},
|
||||
Pendings1 = emqx_session:enrich_delivers(ClientInfo, Pendings, Session),
|
||||
Pendings2 = lists:filter(fun emqx_session:should_discard/1, Pendings1),
|
||||
{true, Session1, Pendings2}.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Info, Stats
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
%% @doc Get infos of the session.
|
||||
info(Keys, Session) when is_list(Keys) ->
|
||||
[{Key, info(Key, Session)} || Key <- Keys];
|
||||
info(id, #session{id = Id}) ->
|
||||
Id;
|
||||
info(clientid, #session{clientid = ClientId}) ->
|
||||
ClientId;
|
||||
info(created_at, #session{created_at = CreatedAt}) ->
|
||||
CreatedAt;
|
||||
info(is_persistent, #session{is_persistent = IsPersistent}) ->
|
||||
IsPersistent;
|
||||
info(subscriptions, #session{subscriptions = Subs}) ->
|
||||
Subs;
|
||||
info(subscriptions_cnt, #session{subscriptions = Subs}) ->
|
||||
maps:size(Subs);
|
||||
info(subscriptions_max, #session{max_subscriptions = MaxSubs}) ->
|
||||
MaxSubs;
|
||||
info(upgrade_qos, #session{upgrade_qos = UpgradeQoS}) ->
|
||||
UpgradeQoS;
|
||||
info(inflight, #session{inflight = Inflight}) ->
|
||||
Inflight;
|
||||
info(inflight_cnt, #session{inflight = Inflight}) ->
|
||||
emqx_inflight:size(Inflight);
|
||||
info(inflight_max, #session{inflight = Inflight}) ->
|
||||
emqx_inflight:max_size(Inflight);
|
||||
info(retry_interval, #session{retry_interval = Interval}) ->
|
||||
Interval;
|
||||
info(mqueue, #session{mqueue = MQueue}) ->
|
||||
MQueue;
|
||||
info(mqueue_len, #session{mqueue = MQueue}) ->
|
||||
emqx_mqueue:len(MQueue);
|
||||
info(mqueue_max, #session{mqueue = MQueue}) ->
|
||||
emqx_mqueue:max_len(MQueue);
|
||||
info(mqueue_dropped, #session{mqueue = MQueue}) ->
|
||||
emqx_mqueue:dropped(MQueue);
|
||||
info(next_pkt_id, #session{next_pkt_id = PacketId}) ->
|
||||
PacketId;
|
||||
info(awaiting_rel, #session{awaiting_rel = AwaitingRel}) ->
|
||||
AwaitingRel;
|
||||
info(awaiting_rel_cnt, #session{awaiting_rel = AwaitingRel}) ->
|
||||
maps:size(AwaitingRel);
|
||||
info(awaiting_rel_max, #session{max_awaiting_rel = Max}) ->
|
||||
Max;
|
||||
info(await_rel_timeout, #session{await_rel_timeout = Timeout}) ->
|
||||
Timeout.
|
||||
|
||||
%% @doc Get stats of the session.
|
||||
-spec stats(session()) -> emqx_types:stats().
|
||||
stats(Session) -> info(?STATS_KEYS, Session).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Client -> Broker: SUBSCRIBE / UNSUBSCRIBE
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec subscribe(emqx_types:topic(), emqx_types:subopts(), session()) ->
|
||||
{ok, session()} | {error, emqx_types:reason_code()}.
|
||||
subscribe(
|
||||
TopicFilter,
|
||||
SubOpts,
|
||||
Session = #session{clientid = ClientId, subscriptions = Subs}
|
||||
) ->
|
||||
IsNew = not maps:is_key(TopicFilter, Subs),
|
||||
case IsNew andalso is_subscriptions_full(Session) of
|
||||
false ->
|
||||
ok = emqx_broker:subscribe(TopicFilter, ClientId, SubOpts),
|
||||
Session1 = Session#session{subscriptions = maps:put(TopicFilter, SubOpts, Subs)},
|
||||
{ok, Session1};
|
||||
true ->
|
||||
{error, ?RC_QUOTA_EXCEEDED}
|
||||
end.
|
||||
|
||||
is_subscriptions_full(#session{max_subscriptions = infinity}) ->
|
||||
false;
|
||||
is_subscriptions_full(#session{
|
||||
subscriptions = Subs,
|
||||
max_subscriptions = MaxLimit
|
||||
}) ->
|
||||
maps:size(Subs) >= MaxLimit.
|
||||
|
||||
-spec unsubscribe(emqx_types:topic(), session()) ->
|
||||
{ok, session(), emqx_types:subopts()} | {error, emqx_types:reason_code()}.
|
||||
unsubscribe(
|
||||
TopicFilter,
|
||||
Session = #session{subscriptions = Subs}
|
||||
) ->
|
||||
case maps:find(TopicFilter, Subs) of
|
||||
{ok, SubOpts} ->
|
||||
ok = emqx_broker:unsubscribe(TopicFilter),
|
||||
{ok, Session#session{subscriptions = maps:remove(TopicFilter, Subs)}, SubOpts};
|
||||
error ->
|
||||
{error, ?RC_NO_SUBSCRIPTION_EXISTED}
|
||||
end.
|
||||
|
||||
-spec get_subscription(emqx_types:topic(), session()) ->
|
||||
emqx_types:subopts() | undefined.
|
||||
get_subscription(Topic, #session{subscriptions = Subs}) ->
|
||||
maps:get(Topic, Subs, undefined).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Client -> Broker: PUBLISH
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec publish(emqx_types:packet_id(), emqx_types:message(), session()) ->
|
||||
{ok, emqx_types:publish_result(), session()}
|
||||
| {error, emqx_types:reason_code()}.
|
||||
publish(
|
||||
PacketId,
|
||||
Msg = #message{qos = ?QOS_2, timestamp = Ts},
|
||||
Session = #session{awaiting_rel = AwaitingRel, await_rel_timeout = Timeout}
|
||||
) ->
|
||||
case is_awaiting_full(Session) of
|
||||
false ->
|
||||
case maps:is_key(PacketId, AwaitingRel) of
|
||||
false ->
|
||||
Results = emqx_broker:publish(Msg),
|
||||
AwaitingRel1 = maps:put(PacketId, Ts, AwaitingRel),
|
||||
Session1 = ensure_timer(expire_awaiting_rel, Timeout, Session),
|
||||
{ok, Results, Session1#session{awaiting_rel = AwaitingRel1}};
|
||||
true ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_IN_USE}
|
||||
end;
|
||||
true ->
|
||||
{error, ?RC_RECEIVE_MAXIMUM_EXCEEDED}
|
||||
end;
|
||||
%% Publish QoS0/1 directly
|
||||
publish(_PacketId, Msg, Session) ->
|
||||
{ok, emqx_broker:publish(Msg), [], Session}.
|
||||
|
||||
is_awaiting_full(#session{max_awaiting_rel = infinity}) ->
|
||||
false;
|
||||
is_awaiting_full(#session{
|
||||
awaiting_rel = AwaitingRel,
|
||||
max_awaiting_rel = MaxLimit
|
||||
}) ->
|
||||
maps:size(AwaitingRel) >= MaxLimit.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Client -> Broker: PUBACK
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec puback(clientinfo(), emqx_types:packet_id(), session()) ->
|
||||
{ok, emqx_types:message(), replies(), session()}
|
||||
| {error, emqx_types:reason_code()}.
|
||||
puback(ClientInfo, PacketId, Session = #session{inflight = Inflight}) ->
|
||||
case emqx_inflight:lookup(PacketId, Inflight) of
|
||||
{value, #inflight_data{phase = wait_ack, message = Msg}} ->
|
||||
Inflight1 = emqx_inflight:delete(PacketId, Inflight),
|
||||
Session1 = Session#session{inflight = Inflight1},
|
||||
{ok, Replies, Session2} = dequeue(ClientInfo, Session1),
|
||||
{ok, Msg, Replies, Session2};
|
||||
{value, _} ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_IN_USE};
|
||||
none ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND}
|
||||
end.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Client -> Broker: PUBREC
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec pubrec(emqx_types:packet_id(), session()) ->
|
||||
{ok, emqx_types:message(), session()}
|
||||
| {error, emqx_types:reason_code()}.
|
||||
pubrec(PacketId, Session = #session{inflight = Inflight}) ->
|
||||
case emqx_inflight:lookup(PacketId, Inflight) of
|
||||
{value, #inflight_data{phase = wait_ack, message = Msg} = Data} ->
|
||||
Update = Data#inflight_data{phase = wait_comp},
|
||||
Inflight1 = emqx_inflight:update(PacketId, Update, Inflight),
|
||||
{ok, Msg, Session#session{inflight = Inflight1}};
|
||||
{value, _} ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_IN_USE};
|
||||
none ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND}
|
||||
end.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Client -> Broker: PUBREL
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec pubrel(emqx_types:packet_id(), session()) ->
|
||||
{ok, session()}
|
||||
| {error, emqx_types:reason_code()}.
|
||||
pubrel(PacketId, Session = #session{awaiting_rel = AwaitingRel}) ->
|
||||
case maps:take(PacketId, AwaitingRel) of
|
||||
{_Ts, AwaitingRel1} ->
|
||||
NSession = Session#session{awaiting_rel = AwaitingRel1},
|
||||
{ok, reconcile_expire_timer(NSession)};
|
||||
error ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND}
|
||||
end.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Client -> Broker: PUBCOMP
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec pubcomp(clientinfo(), emqx_types:packet_id(), session()) ->
|
||||
{ok, emqx_types:message(), replies(), session()}
|
||||
| {error, emqx_types:reason_code()}.
|
||||
pubcomp(ClientInfo, PacketId, Session = #session{inflight = Inflight}) ->
|
||||
case emqx_inflight:lookup(PacketId, Inflight) of
|
||||
{value, #inflight_data{phase = wait_comp, message = Msg}} ->
|
||||
Inflight1 = emqx_inflight:delete(PacketId, Inflight),
|
||||
Session1 = Session#session{inflight = Inflight1},
|
||||
{ok, Replies, Session2} = dequeue(ClientInfo, Session1),
|
||||
{ok, Msg, Replies, Session2};
|
||||
{value, _Other} ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_IN_USE};
|
||||
none ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND}
|
||||
end.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Dequeue Msgs
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
dequeue(ClientInfo, Session = #session{inflight = Inflight, mqueue = Q}) ->
|
||||
case emqx_mqueue:is_empty(Q) of
|
||||
true ->
|
||||
{ok, [], reconcile_retry_timer(Session)};
|
||||
false ->
|
||||
{Msgs, Q1} = dequeue(ClientInfo, batch_n(Inflight), [], Q),
|
||||
do_deliver(ClientInfo, Msgs, [], Session#session{mqueue = Q1})
|
||||
end.
|
||||
|
||||
dequeue(_ClientInfo, 0, Msgs, Q) ->
|
||||
{lists:reverse(Msgs), Q};
|
||||
dequeue(ClientInfo, Cnt, Msgs, Q) ->
|
||||
case emqx_mqueue:out(Q) of
|
||||
{empty, _Q} ->
|
||||
dequeue(ClientInfo, 0, Msgs, Q);
|
||||
{{value, Msg}, Q1} ->
|
||||
case emqx_message:is_expired(Msg) of
|
||||
true ->
|
||||
_ = emqx_session_events:handle_event(ClientInfo, {expired, Msg}),
|
||||
dequeue(ClientInfo, Cnt, Msgs, Q1);
|
||||
false ->
|
||||
dequeue(ClientInfo, acc_cnt(Msg, Cnt), [Msg | Msgs], Q1)
|
||||
end
|
||||
end.
|
||||
|
||||
acc_cnt(#message{qos = ?QOS_0}, Cnt) -> Cnt;
|
||||
acc_cnt(_Msg, Cnt) -> Cnt - 1.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Broker -> Client: Deliver
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec deliver(clientinfo(), [emqx_types:deliver()], session()) ->
|
||||
{ok, replies(), session()}.
|
||||
deliver(ClientInfo, Msgs, Session) ->
|
||||
do_deliver(ClientInfo, Msgs, [], Session).
|
||||
|
||||
do_deliver(_ClientInfo, [], Publishes, Session) ->
|
||||
{ok, lists:reverse(Publishes), reconcile_retry_timer(Session)};
|
||||
do_deliver(ClientInfo, [Msg | More], Acc, Session) ->
|
||||
case deliver_msg(ClientInfo, Msg, Session) of
|
||||
{ok, [], Session1} ->
|
||||
do_deliver(ClientInfo, More, Acc, Session1);
|
||||
{ok, [Publish], Session1} ->
|
||||
do_deliver(ClientInfo, More, [Publish | Acc], Session1)
|
||||
end.
|
||||
|
||||
deliver_msg(_ClientInfo, Msg = #message{qos = ?QOS_0}, Session) ->
|
||||
{ok, [{undefined, maybe_ack(Msg)}], Session};
|
||||
deliver_msg(
|
||||
ClientInfo,
|
||||
Msg = #message{qos = QoS},
|
||||
Session = #session{next_pkt_id = PacketId, inflight = Inflight}
|
||||
) when
|
||||
QoS =:= ?QOS_1 orelse QoS =:= ?QOS_2
|
||||
->
|
||||
case emqx_inflight:is_full(Inflight) of
|
||||
true ->
|
||||
Session1 =
|
||||
case maybe_nack(Msg) of
|
||||
true -> Session;
|
||||
false -> enqueue_msg(ClientInfo, Msg, Session)
|
||||
end,
|
||||
{ok, [], Session1};
|
||||
false ->
|
||||
%% Note that we publish message without shared ack header
|
||||
%% But add to inflight with ack headers
|
||||
%% This ack header is required for redispatch-on-terminate feature to work
|
||||
Publish = {PacketId, maybe_ack(Msg)},
|
||||
MarkedMsg = mark_begin_deliver(Msg),
|
||||
Inflight1 = emqx_inflight:insert(PacketId, with_ts(MarkedMsg), Inflight),
|
||||
{ok, [Publish], next_pkt_id(Session#session{inflight = Inflight1})}
|
||||
end.
|
||||
|
||||
-spec enqueue(clientinfo(), [emqx_types:message()], session()) ->
|
||||
session().
|
||||
enqueue(ClientInfo, Msgs, Session) when is_list(Msgs) ->
|
||||
lists:foldl(
|
||||
fun(Msg, Session0) -> enqueue_msg(ClientInfo, Msg, Session0) end,
|
||||
Session,
|
||||
Msgs
|
||||
).
|
||||
|
||||
enqueue_msg(ClientInfo, #message{qos = QOS} = Msg, Session = #session{mqueue = Q}) ->
|
||||
{Dropped, NewQ} = emqx_mqueue:in(Msg, Q),
|
||||
case Dropped of
|
||||
undefined ->
|
||||
Session#session{mqueue = NewQ};
|
||||
_Msg ->
|
||||
Reason =
|
||||
case emqx_mqueue:info(store_qos0, Q) of
|
||||
false when QOS =:= ?QOS_0 -> qos0_msg;
|
||||
_ -> queue_full
|
||||
end,
|
||||
_ = emqx_session_events:handle_event(ClientInfo, {dropped, Dropped, Reason}),
|
||||
Session
|
||||
end.
|
||||
|
||||
maybe_ack(Msg) ->
|
||||
emqx_shared_sub:maybe_ack(Msg).
|
||||
|
||||
maybe_nack(Msg) ->
|
||||
emqx_shared_sub:maybe_nack_dropped(Msg).
|
||||
|
||||
mark_begin_deliver(Msg) ->
|
||||
emqx_message:set_header(deliver_begin_at, erlang:system_time(millisecond), Msg).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Timeouts
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec handle_timeout(clientinfo(), atom(), session()) ->
|
||||
{ok, replies(), session()}.
|
||||
handle_timeout(ClientInfo, retry_delivery = Name, Session) ->
|
||||
retry(ClientInfo, clean_timer(Name, Session));
|
||||
handle_timeout(ClientInfo, expire_awaiting_rel = Name, Session) ->
|
||||
expire(ClientInfo, clean_timer(Name, Session)).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Retry Delivery
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec retry(clientinfo(), session()) ->
|
||||
{ok, replies(), session()}.
|
||||
retry(ClientInfo, Session = #session{inflight = Inflight}) ->
|
||||
case emqx_inflight:is_empty(Inflight) of
|
||||
true ->
|
||||
{ok, [], Session};
|
||||
false ->
|
||||
Now = erlang:system_time(millisecond),
|
||||
retry_delivery(
|
||||
ClientInfo,
|
||||
emqx_inflight:to_list(fun sort_fun/2, Inflight),
|
||||
[],
|
||||
Now,
|
||||
Session
|
||||
)
|
||||
end.
|
||||
|
||||
retry_delivery(_ClientInfo, [], Acc, _Now, Session) ->
|
||||
{ok, lists:reverse(Acc), reconcile_retry_timer(Session)};
|
||||
retry_delivery(
|
||||
ClientInfo,
|
||||
[{PacketId, #inflight_data{timestamp = Ts} = Data} | More],
|
||||
Acc,
|
||||
Now,
|
||||
Session = #session{retry_interval = Interval, inflight = Inflight}
|
||||
) ->
|
||||
case (Age = age(Now, Ts)) >= Interval of
|
||||
true ->
|
||||
{Acc1, Inflight1} = do_retry_delivery(ClientInfo, PacketId, Data, Now, Acc, Inflight),
|
||||
retry_delivery(ClientInfo, More, Acc1, Now, Session#session{inflight = Inflight1});
|
||||
false ->
|
||||
NSession = ensure_timer(retry_delivery, Interval - max(0, Age), Session),
|
||||
{ok, lists:reverse(Acc), NSession}
|
||||
end.
|
||||
|
||||
do_retry_delivery(
|
||||
ClientInfo,
|
||||
PacketId,
|
||||
#inflight_data{phase = wait_ack, message = Msg} = Data,
|
||||
Now,
|
||||
Acc,
|
||||
Inflight
|
||||
) ->
|
||||
case emqx_message:is_expired(Msg) of
|
||||
true ->
|
||||
_ = emqx_session_events:handle_event(ClientInfo, {expired, Msg}),
|
||||
{Acc, emqx_inflight:delete(PacketId, Inflight)};
|
||||
false ->
|
||||
Msg1 = emqx_message:set_flag(dup, true, Msg),
|
||||
Update = Data#inflight_data{message = Msg1, timestamp = Now},
|
||||
Inflight1 = emqx_inflight:update(PacketId, Update, Inflight),
|
||||
{[{PacketId, Msg1} | Acc], Inflight1}
|
||||
end;
|
||||
do_retry_delivery(_ClientInfo, PacketId, Data, Now, Acc, Inflight) ->
|
||||
Update = Data#inflight_data{timestamp = Now},
|
||||
Inflight1 = emqx_inflight:update(PacketId, Update, Inflight),
|
||||
{[{pubrel, PacketId} | Acc], Inflight1}.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Expire Awaiting Rel
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec expire(clientinfo(), session()) ->
|
||||
{ok, replies(), session()}.
|
||||
expire(ClientInfo, Session = #session{awaiting_rel = AwaitingRel}) ->
|
||||
case maps:size(AwaitingRel) of
|
||||
0 ->
|
||||
{ok, [], Session};
|
||||
_ ->
|
||||
Now = erlang:system_time(millisecond),
|
||||
NSession = expire_awaiting_rel(ClientInfo, Now, Session),
|
||||
{ok, [], reconcile_expire_timer(NSession)}
|
||||
end.
|
||||
|
||||
expire_awaiting_rel(
|
||||
ClientInfo,
|
||||
Now,
|
||||
Session = #session{awaiting_rel = AwaitingRel, await_rel_timeout = Timeout}
|
||||
) ->
|
||||
NotExpired = fun(_PacketId, Ts) -> age(Now, Ts) < Timeout end,
|
||||
AwaitingRel1 = maps:filter(NotExpired, AwaitingRel),
|
||||
ExpiredCnt = maps:size(AwaitingRel) - maps:size(AwaitingRel1),
|
||||
_ = emqx_session_events:handle_event(ClientInfo, {expired_rel, ExpiredCnt}),
|
||||
Session#session{awaiting_rel = AwaitingRel1}.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Takeover, Resume and Replay
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec takeover(session()) ->
|
||||
ok.
|
||||
takeover(#session{subscriptions = Subs}) ->
|
||||
lists:foreach(fun emqx_broker:unsubscribe/1, maps:keys(Subs)).
|
||||
|
||||
-spec resume(emqx_types:clientinfo(), session()) ->
|
||||
session().
|
||||
resume(ClientInfo = #{clientid := ClientId}, Session = #session{subscriptions = Subs}) ->
|
||||
ok = maps:foreach(
|
||||
fun(TopicFilter, SubOpts) ->
|
||||
ok = emqx_broker:subscribe(TopicFilter, ClientId, SubOpts)
|
||||
end,
|
||||
Subs
|
||||
),
|
||||
ok = emqx_metrics:inc('session.resumed'),
|
||||
ok = emqx_hooks:run('session.resumed', [ClientInfo, emqx_session:info(Session)]),
|
||||
Session#session{timers = #{}}.
|
||||
|
||||
-spec replay(emqx_types:clientinfo(), [emqx_types:message()], session()) ->
|
||||
{ok, replies(), session()}.
|
||||
replay(ClientInfo, Pendings, Session) ->
|
||||
PendingsLocal = emqx_session:enrich_delivers(
|
||||
ClientInfo,
|
||||
emqx_utils:drain_deliver(),
|
||||
Session
|
||||
),
|
||||
PendingsLocal1 = lists:filter(
|
||||
fun(Msg) -> not lists:keymember(Msg#message.id, #message.id, Pendings) end,
|
||||
PendingsLocal
|
||||
),
|
||||
{ok, PubsResendQueued, Session1} = replay(ClientInfo, Session),
|
||||
{ok, Pubs1, Session2} = deliver(ClientInfo, Pendings, Session1),
|
||||
{ok, Pubs2, Session3} = deliver(ClientInfo, PendingsLocal1, Session2),
|
||||
{ok, append(append(PubsResendQueued, Pubs1), Pubs2), Session3}.
|
||||
|
||||
-spec replay(emqx_types:clientinfo(), session()) ->
|
||||
{ok, replies(), session()}.
|
||||
replay(ClientInfo, Session) ->
|
||||
PubsResend = lists:map(
|
||||
fun
|
||||
({PacketId, #inflight_data{phase = wait_comp}}) ->
|
||||
{pubrel, PacketId};
|
||||
({PacketId, #inflight_data{message = Msg}}) ->
|
||||
{PacketId, emqx_message:set_flag(dup, true, Msg)}
|
||||
end,
|
||||
emqx_inflight:to_list(Session#session.inflight)
|
||||
),
|
||||
{ok, More, Session1} = dequeue(ClientInfo, Session),
|
||||
{ok, append(PubsResend, More), reconcile_expire_timer(Session1)}.
|
||||
|
||||
append(L1, []) -> L1;
|
||||
append(L1, L2) -> L1 ++ L2.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-spec disconnect(session()) -> {idle, session()}.
|
||||
disconnect(Session = #session{}) ->
|
||||
% TODO: isolate expiry timer / timeout handling here?
|
||||
{idle, cancel_timers(Session)}.
|
||||
|
||||
-spec terminate(Reason :: term(), session()) -> ok.
|
||||
terminate(Reason, Session) ->
|
||||
maybe_redispatch_shared_messages(Reason, Session),
|
||||
ok.
|
||||
|
||||
maybe_redispatch_shared_messages(takenover, _Session) ->
|
||||
ok;
|
||||
maybe_redispatch_shared_messages(kicked, _Session) ->
|
||||
ok;
|
||||
maybe_redispatch_shared_messages(_Reason, Session) ->
|
||||
redispatch_shared_messages(Session).
|
||||
|
||||
redispatch_shared_messages(#session{inflight = Inflight, mqueue = Q}) ->
|
||||
AllInflights = emqx_inflight:to_list(fun sort_fun/2, Inflight),
|
||||
F = fun
|
||||
({_PacketId, #inflight_data{message = #message{qos = ?QOS_1} = Msg}}) ->
|
||||
%% For QoS 2, here is what the spec says:
|
||||
%% If the Client's Session terminates before the Client reconnects,
|
||||
%% the Server MUST NOT send the Application Message to any other
|
||||
%% subscribed Client [MQTT-4.8.2-5].
|
||||
{true, Msg};
|
||||
({_PacketId, #inflight_data{}}) ->
|
||||
false
|
||||
end,
|
||||
InflightList = lists:filtermap(F, AllInflights),
|
||||
emqx_shared_sub:redispatch(InflightList ++ emqx_mqueue:to_list(Q)).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Next Packet Id
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
obtain_next_pkt_id(Session) ->
|
||||
{Session#session.next_pkt_id, next_pkt_id(Session)}.
|
||||
|
||||
next_pkt_id(Session = #session{next_pkt_id = ?MAX_PACKET_ID}) ->
|
||||
Session#session{next_pkt_id = 1};
|
||||
next_pkt_id(Session = #session{next_pkt_id = Id}) ->
|
||||
Session#session{next_pkt_id = Id + 1}.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Helper functions
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-compile({inline, [sort_fun/2, batch_n/1, with_ts/1, age/2]}).
|
||||
|
||||
sort_fun({_, A}, {_, B}) ->
|
||||
A#inflight_data.timestamp =< B#inflight_data.timestamp.
|
||||
|
||||
batch_n(Inflight) ->
|
||||
case emqx_inflight:max_size(Inflight) of
|
||||
0 -> ?DEFAULT_BATCH_N;
|
||||
Sz -> Sz - emqx_inflight:size(Inflight)
|
||||
end.
|
||||
|
||||
with_ts(Msg) ->
|
||||
#inflight_data{
|
||||
phase = wait_ack,
|
||||
message = Msg,
|
||||
timestamp = erlang:system_time(millisecond)
|
||||
}.
|
||||
|
||||
age(Now, Ts) -> Now - Ts.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
reconcile_retry_timer(Session = #session{inflight = Inflight}) ->
|
||||
case emqx_inflight:is_empty(Inflight) of
|
||||
false ->
|
||||
ensure_timer(retry_delivery, Session#session.retry_interval, Session);
|
||||
true ->
|
||||
cancel_timer(retry_delivery, Session)
|
||||
end.
|
||||
|
||||
reconcile_expire_timer(Session = #session{awaiting_rel = AwaitingRel}) ->
|
||||
case maps:size(AwaitingRel) of
|
||||
0 ->
|
||||
cancel_timer(expire_awaiting_rel, Session);
|
||||
_ ->
|
||||
ensure_timer(expire_awaiting_rel, Session#session.await_rel_timeout, Session)
|
||||
end.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
ensure_timer(Name, Timeout, Session = #session{timers = Timers}) ->
|
||||
NTimers = emqx_session:ensure_timer(Name, Timeout, Timers),
|
||||
Session#session{timers = NTimers}.
|
||||
|
||||
clean_timer(Name, Session = #session{timers = Timers}) ->
|
||||
Session#session{timers = maps:remove(Name, Timers)}.
|
||||
|
||||
cancel_timers(Session = #session{timers = Timers}) ->
|
||||
ok = maps:foreach(fun(_Name, TRef) -> emqx_utils:cancel_timer(TRef) end, Timers),
|
||||
Session#session{timers = #{}}.
|
||||
|
||||
cancel_timer(Name, Session = #session{timers = Timers}) ->
|
||||
Session#session{timers = emqx_session:cancel_timer(Name, Timers)}.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% For CT tests
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
set_field(Name, Value, Session) ->
|
||||
Pos = emqx_utils:index_of(Name, record_info(fields, session)),
|
||||
setelement(Pos + 1, Session, Value).
|
|
@ -38,48 +38,30 @@ init_per_suite(Config) ->
|
|||
ok = meck:expect(emqx_cm, mark_channel_disconnected, fun(_) -> ok end),
|
||||
%% Broker Meck
|
||||
ok = meck:new(emqx_broker, [passthrough, no_history, no_link]),
|
||||
%% Hooks Meck
|
||||
ok = meck:new(emqx_hooks, [passthrough, no_history, no_link]),
|
||||
ok = meck:expect(emqx_hooks, run, fun(_Hook, _Args) -> ok end),
|
||||
ok = meck:expect(emqx_hooks, run_fold, fun(_Hook, _Args, Acc) -> Acc end),
|
||||
%% Session Meck
|
||||
ok = meck:new(emqx_session, [passthrough, no_history, no_link]),
|
||||
%% Metrics
|
||||
ok = meck:new(emqx_metrics, [passthrough, no_history, no_link]),
|
||||
ok = meck:expect(emqx_metrics, inc, fun(_) -> ok end),
|
||||
ok = meck:expect(emqx_metrics, inc, fun(_, _) -> ok end),
|
||||
%% Ban
|
||||
meck:new(emqx_banned, [passthrough, no_history, no_link]),
|
||||
ok = meck:expect(emqx_banned, check, fun(_ConnInfo) -> false end),
|
||||
Config.
|
||||
Apps = emqx_cth_suite:start(
|
||||
[
|
||||
{emqx, #{
|
||||
override_env => [{boot_modules, [broker]}]
|
||||
}}
|
||||
],
|
||||
#{work_dir => emqx_cth_suite:work_dir(Config)}
|
||||
),
|
||||
[{suite_apps, Apps} | Config].
|
||||
|
||||
end_per_suite(_Config) ->
|
||||
end_per_suite(Config) ->
|
||||
ok = emqx_cth_suite:stop(?config(suite_apps, Config)),
|
||||
meck:unload([
|
||||
emqx_metrics,
|
||||
emqx_session,
|
||||
emqx_broker,
|
||||
emqx_hooks,
|
||||
emqx_cm,
|
||||
emqx_banned
|
||||
]).
|
||||
|
||||
init_per_testcase(_TestCase, Config) ->
|
||||
%% Access Control Meck
|
||||
ok = meck:new(emqx_access_control, [passthrough, no_history, no_link]),
|
||||
ok = meck:expect(
|
||||
emqx_access_control,
|
||||
authenticate,
|
||||
fun(_) -> {ok, #{is_superuser => false}} end
|
||||
),
|
||||
ok = meck:expect(emqx_access_control, authorize, fun(_, _, _) -> allow end),
|
||||
emqx_common_test_helpers:start_apps([]),
|
||||
Config.
|
||||
|
||||
end_per_testcase(_TestCase, Config) ->
|
||||
meck:unload([emqx_access_control]),
|
||||
emqx_common_test_helpers:stop_apps([]),
|
||||
Config.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Test cases for channel info/stats/caps
|
||||
%%--------------------------------------------------------------------
|
||||
|
@ -111,14 +93,7 @@ t_chan_caps(_) ->
|
|||
%% Test cases for channel handle_in
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
t_handle_in_connect_packet_sucess(_) ->
|
||||
ok = meck:expect(
|
||||
emqx_cm,
|
||||
open_session,
|
||||
fun(true, _ClientInfo, _ConnInfo) ->
|
||||
{ok, #{session => session(), present => false}}
|
||||
end
|
||||
),
|
||||
t_handle_in_connect_packet_success(_) ->
|
||||
IdleChannel = channel(#{conn_state => idle}),
|
||||
{ok, [{event, connected}, {connack, ?CONNACK_PACKET(?RC_SUCCESS, 0, _)}], Channel} =
|
||||
emqx_channel:handle_in(?CONNECT_PACKET(connpkt()), IdleChannel),
|
||||
|
@ -242,7 +217,6 @@ t_handle_in_qos2_publish(_) ->
|
|||
?assertEqual(2, proplists:get_value(awaiting_rel_cnt, emqx_channel:stats(Channel2))).
|
||||
|
||||
t_handle_in_qos2_publish_with_error_return(_) ->
|
||||
ok = meck:expect(emqx_metrics, inc, fun(_) -> ok end),
|
||||
ok = meck:expect(emqx_broker, publish, fun(_) -> [] end),
|
||||
Session = session(#{max_awaiting_rel => 2, awaiting_rel => #{1 => 1}}),
|
||||
Channel = channel(#{conn_state => connected, session => Session}),
|
||||
|
@ -268,7 +242,7 @@ t_handle_in_puback_ok(_) ->
|
|||
ok = meck:expect(
|
||||
emqx_session,
|
||||
puback,
|
||||
fun(_, _PacketId, Session) -> {ok, Msg, Session} end
|
||||
fun(_, _PacketId, Session) -> {ok, Msg, [], Session} end
|
||||
),
|
||||
Channel = channel(#{conn_state => connected}),
|
||||
{ok, _NChannel} = emqx_channel:handle_in(?PUBACK_PACKET(1, ?RC_SUCCESS), Channel).
|
||||
|
@ -379,7 +353,7 @@ t_handle_in_pubrel_not_found_error(_) ->
|
|||
emqx_channel:handle_in(?PUBREL_PACKET(1, ?RC_SUCCESS), channel()).
|
||||
|
||||
t_handle_in_pubcomp_ok(_) ->
|
||||
ok = meck:expect(emqx_session, pubcomp, fun(_, _, Session) -> {ok, Session} end),
|
||||
ok = meck:expect(emqx_session, pubcomp, fun(_, _, Session) -> {ok, [], Session} end),
|
||||
{ok, _Channel} = emqx_channel:handle_in(?PUBCOMP_PACKET(1, ?RC_SUCCESS), channel()).
|
||||
% ?assertEqual(#{pubcomp_in => 1}, emqx_channel:info(pub_stats, Channel)).
|
||||
|
||||
|
@ -491,18 +465,7 @@ t_process_unsubscribe(_) ->
|
|||
t_quota_qos0(_) ->
|
||||
esockd_limiter:start_link(),
|
||||
add_bucket(),
|
||||
Cnter = counters:new(1, []),
|
||||
ok = meck:expect(emqx_broker, publish, fun(_) -> [{node(), <<"topic">>, {ok, 4}}] end),
|
||||
ok = meck:expect(
|
||||
emqx_metrics,
|
||||
inc,
|
||||
fun('packets.publish.dropped') -> counters:add(Cnter, 1, 1) end
|
||||
),
|
||||
ok = meck:expect(
|
||||
emqx_metrics,
|
||||
val,
|
||||
fun('packets.publish.dropped') -> counters:get(Cnter, 1) end
|
||||
),
|
||||
Chann = channel(#{conn_state => connected, quota => quota()}),
|
||||
Pub = ?PUBLISH_PACKET(?QOS_0, <<"topic">>, undefined, <<"payload">>),
|
||||
|
||||
|
@ -515,8 +478,6 @@ t_quota_qos0(_) ->
|
|||
{ok, _} = emqx_channel:handle_in(Pub, Chann3),
|
||||
M1 = emqx_metrics:val('packets.publish.dropped') - 1,
|
||||
|
||||
ok = meck:expect(emqx_metrics, inc, fun(_) -> ok end),
|
||||
ok = meck:expect(emqx_metrics, inc, fun(_, _) -> ok end),
|
||||
del_bucket(),
|
||||
esockd_limiter:stop().
|
||||
|
||||
|
@ -741,7 +702,7 @@ t_handle_call_takeover_begin(_) ->
|
|||
{reply, _Session, _Chan} = emqx_channel:handle_call({takeover, 'begin'}, channel()).
|
||||
|
||||
t_handle_call_takeover_end(_) ->
|
||||
ok = meck:expect(emqx_session, takeover, fun(_) -> ok end),
|
||||
ok = meck:expect(emqx_broker, unsubscribe, fun(_) -> ok end),
|
||||
{shutdown, takenover, [], _, _Chan} =
|
||||
emqx_channel:handle_call({takeover, 'end'}, channel()).
|
||||
|
||||
|
@ -775,13 +736,11 @@ t_handle_timeout_keepalive(_) ->
|
|||
|
||||
t_handle_timeout_retry_delivery(_) ->
|
||||
TRef = make_ref(),
|
||||
ok = meck:expect(emqx_session, retry, fun(_, Session) -> {ok, Session} end),
|
||||
Channel = emqx_channel:set_field(timers, #{retry_delivery => TRef}, channel()),
|
||||
{ok, _Chan} = emqx_channel:handle_timeout(TRef, retry_delivery, Channel).
|
||||
|
||||
t_handle_timeout_expire_awaiting_rel(_) ->
|
||||
TRef = make_ref(),
|
||||
ok = meck:expect(emqx_session, expire, fun(_, _, Session) -> {ok, Session} end),
|
||||
Channel = emqx_channel:set_field(timers, #{expire_awaiting_rel => TRef}, channel()),
|
||||
{ok, _Chan} = emqx_channel:handle_timeout(TRef, expire_awaiting_rel, Channel).
|
||||
|
||||
|
@ -977,9 +936,14 @@ t_flapping_detect(_) ->
|
|||
{ok, #{session => session(), present => false}}
|
||||
end
|
||||
),
|
||||
ok = meck:expect(emqx_access_control, authenticate, fun(_) -> {error, not_authorized} end),
|
||||
ok = meck:expect(emqx_flapping, detect, fun(_) -> Parent ! flapping_detect end),
|
||||
IdleChannel = channel(#{conn_state => idle}),
|
||||
IdleChannel = channel(
|
||||
clientinfo(#{
|
||||
username => <<>>,
|
||||
enable_authn => quick_deny_anonymous
|
||||
}),
|
||||
#{conn_state => idle}
|
||||
),
|
||||
{shutdown, not_authorized, _ConnAck, _Channel} =
|
||||
emqx_channel:handle_in(?CONNECT_PACKET(connpkt()), IdleChannel),
|
||||
receive
|
||||
|
@ -994,7 +958,8 @@ t_flapping_detect(_) ->
|
|||
%%--------------------------------------------------------------------
|
||||
|
||||
channel() -> channel(#{}).
|
||||
channel(InitFields) ->
|
||||
channel(InitFields) -> channel(clientinfo(), InitFields).
|
||||
channel(ClientInfo, InitFields) ->
|
||||
ConnInfo = #{
|
||||
peername => {{127, 0, 0, 1}, 3456},
|
||||
sockname => {{127, 0, 0, 1}, 1883},
|
||||
|
@ -1004,7 +969,7 @@ channel(InitFields) ->
|
|||
clean_start => true,
|
||||
keepalive => 30,
|
||||
clientid => <<"clientid">>,
|
||||
username => <<"username">>,
|
||||
username => maps:get(username, ClientInfo, <<"username">>),
|
||||
conn_props => #{},
|
||||
receive_maximum => 100,
|
||||
expiry_interval => 0
|
||||
|
@ -1023,8 +988,8 @@ channel(InitFields) ->
|
|||
),
|
||||
maps:merge(
|
||||
#{
|
||||
clientinfo => clientinfo(),
|
||||
session => session(),
|
||||
clientinfo => ClientInfo,
|
||||
session => session(ClientInfo, #{}),
|
||||
conn_state => connected
|
||||
},
|
||||
InitFields
|
||||
|
@ -1039,6 +1004,7 @@ clientinfo(InitProps) ->
|
|||
listener => {tcp, default},
|
||||
protocol => mqtt,
|
||||
peerhost => {127, 0, 0, 1},
|
||||
sockport => 3456,
|
||||
clientid => <<"clientid">>,
|
||||
username => <<"username">>,
|
||||
is_superuser => false,
|
||||
|
@ -1067,17 +1033,17 @@ connpkt(Props) ->
|
|||
session() -> session(#{zone => default, clientid => <<"fake-test">>}, #{}).
|
||||
session(InitFields) -> session(#{zone => default, clientid => <<"fake-test">>}, InitFields).
|
||||
session(ClientInfo, InitFields) when is_map(InitFields) ->
|
||||
Conf = emqx_cm:get_session_confs(
|
||||
Session = emqx_session:create(
|
||||
ClientInfo,
|
||||
#{
|
||||
receive_maximum => 0,
|
||||
expiry_interval => 0
|
||||
}
|
||||
),
|
||||
Session = emqx_session:init(Conf),
|
||||
maps:fold(
|
||||
fun(Field, Value, SessionAcc) ->
|
||||
emqx_session:set_field(Field, Value, SessionAcc)
|
||||
% TODO: assuming specific session implementation
|
||||
emqx_session_mem:set_field(Field, Value, SessionAcc)
|
||||
end,
|
||||
Session,
|
||||
InitFields
|
||||
|
|
|
@ -321,7 +321,7 @@ test_stepdown_session(Action, Reason) ->
|
|||
discard ->
|
||||
emqx_cm:discard_session(ClientId);
|
||||
{takeover, _} ->
|
||||
none = emqx_cm:takeover_session(ClientId),
|
||||
none = emqx_cm:takeover_channel_session(ClientId, fun ident/1),
|
||||
ok
|
||||
end,
|
||||
case Reason =:= timeout orelse Reason =:= noproc of
|
||||
|
@ -381,21 +381,24 @@ t_discard_session_race(_) ->
|
|||
|
||||
t_takeover_session(_) ->
|
||||
#{conninfo := ConnInfo} = ?ChanInfo,
|
||||
none = emqx_cm:takeover_session(<<"clientid">>),
|
||||
none = emqx_cm:takeover_channel_session(<<"clientid">>, fun ident/1),
|
||||
Parent = self(),
|
||||
erlang:spawn_link(fun() ->
|
||||
ok = emqx_cm:register_channel(<<"clientid">>, self(), ConnInfo),
|
||||
Parent ! registered,
|
||||
receive
|
||||
{'$gen_call', From, {takeover, 'begin'}} ->
|
||||
gen_server:reply(From, test),
|
||||
ok
|
||||
{'$gen_call', From1, {takeover, 'begin'}} ->
|
||||
gen_server:reply(From1, test),
|
||||
receive
|
||||
{'$gen_call', From2, {takeover, 'end'}} ->
|
||||
gen_server:reply(From2, [])
|
||||
end
|
||||
end
|
||||
end),
|
||||
receive
|
||||
registered -> ok
|
||||
end,
|
||||
{living, emqx_connection, _, test} = emqx_cm:takeover_session(<<"clientid">>),
|
||||
{ok, test, []} = emqx_cm:takeover_channel_session(<<"clientid">>, fun ident/1),
|
||||
emqx_cm:unregister_channel(<<"clientid">>).
|
||||
|
||||
t_takeover_session_process_gone(_) ->
|
||||
|
@ -403,8 +406,8 @@ t_takeover_session_process_gone(_) ->
|
|||
ClientIDTcp = <<"clientidTCP">>,
|
||||
ClientIDWs = <<"clientidWs">>,
|
||||
ClientIDRpc = <<"clientidRPC">>,
|
||||
none = emqx_cm:takeover_session(ClientIDTcp),
|
||||
none = emqx_cm:takeover_session(ClientIDWs),
|
||||
none = emqx_cm:takeover_channel_session(ClientIDTcp, fun ident/1),
|
||||
none = emqx_cm:takeover_channel_session(ClientIDWs, fun ident/1),
|
||||
meck:new(emqx_connection, [passthrough, no_history]),
|
||||
meck:expect(
|
||||
emqx_connection,
|
||||
|
@ -417,7 +420,7 @@ t_takeover_session_process_gone(_) ->
|
|||
end
|
||||
),
|
||||
ok = emqx_cm:register_channel(ClientIDTcp, self(), ConnInfo),
|
||||
none = emqx_cm:takeover_session(ClientIDTcp),
|
||||
none = emqx_cm:takeover_channel_session(ClientIDTcp, fun ident/1),
|
||||
meck:expect(
|
||||
emqx_connection,
|
||||
call,
|
||||
|
@ -429,7 +432,7 @@ t_takeover_session_process_gone(_) ->
|
|||
end
|
||||
),
|
||||
ok = emqx_cm:register_channel(ClientIDWs, self(), ConnInfo),
|
||||
none = emqx_cm:takeover_session(ClientIDWs),
|
||||
none = emqx_cm:takeover_channel_session(ClientIDWs, fun ident/1),
|
||||
meck:expect(
|
||||
emqx_connection,
|
||||
call,
|
||||
|
@ -441,7 +444,7 @@ t_takeover_session_process_gone(_) ->
|
|||
end
|
||||
),
|
||||
ok = emqx_cm:register_channel(ClientIDRpc, self(), ConnInfo),
|
||||
none = emqx_cm:takeover_session(ClientIDRpc),
|
||||
none = emqx_cm:takeover_channel_session(ClientIDRpc, fun ident/1),
|
||||
emqx_cm:unregister_channel(ClientIDTcp),
|
||||
emqx_cm:unregister_channel(ClientIDWs),
|
||||
emqx_cm:unregister_channel(ClientIDRpc),
|
||||
|
@ -460,3 +463,8 @@ t_message(_) ->
|
|||
?CM ! testing,
|
||||
gen_server:cast(?CM, testing),
|
||||
gen_server:call(?CM, testing).
|
||||
|
||||
%%
|
||||
|
||||
ident(V) ->
|
||||
V.
|
||||
|
|
|
@ -676,10 +676,10 @@ channel(InitFields) ->
|
|||
is_superuser => false,
|
||||
mountpoint => undefined
|
||||
},
|
||||
Conf = emqx_cm:get_session_confs(ClientInfo, #{
|
||||
receive_maximum => 0, expiry_interval => 1000
|
||||
}),
|
||||
Session = emqx_session:init(Conf),
|
||||
Session = emqx_session:create(
|
||||
ClientInfo,
|
||||
#{receive_maximum => 0, expiry_interval => 1000}
|
||||
),
|
||||
maps:fold(
|
||||
fun(Field, Value, Channel) ->
|
||||
emqx_channel:set_field(Field, Value, Channel)
|
||||
|
|
|
@ -272,7 +272,7 @@ consume(Shard, IteratorId) when is_binary(IteratorId) ->
|
|||
consume(It) ->
|
||||
case emqx_ds_storage_layer:next(It) of
|
||||
{value, Msg, NIt} ->
|
||||
[emqx_persistent_session_ds:deserialize_message(Msg) | consume(NIt)];
|
||||
[emqx_persistent_message:deserialize(Msg) | consume(NIt)];
|
||||
none ->
|
||||
[]
|
||||
end.
|
||||
|
|
|
@ -20,7 +20,7 @@
|
|||
|
||||
-include_lib("proper/include/proper.hrl").
|
||||
-include("emqx.hrl").
|
||||
-include("emqx_session.hrl").
|
||||
-include("emqx_session_mem.hrl").
|
||||
-include("emqx_access_control.hrl").
|
||||
|
||||
%% High level Types
|
||||
|
@ -147,7 +147,8 @@ sessioninfo() ->
|
|||
awaiting_rel = awaiting_rel(),
|
||||
max_awaiting_rel = non_neg_integer(),
|
||||
await_rel_timeout = safty_timeout(),
|
||||
created_at = timestamp()
|
||||
created_at = timestamp(),
|
||||
timers = #{}
|
||||
},
|
||||
emqx_session:info(Session)
|
||||
).
|
||||
|
|
|
@ -1,527 +0,0 @@
|
|||
%%--------------------------------------------------------------------
|
||||
%% Copyright (c) 2018-2023 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.
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-module(emqx_session_SUITE).
|
||||
|
||||
-compile(export_all).
|
||||
-compile(nowarn_export_all).
|
||||
|
||||
-include_lib("emqx/include/emqx_mqtt.hrl").
|
||||
-include_lib("eunit/include/eunit.hrl").
|
||||
|
||||
all() -> emqx_common_test_helpers:all(?MODULE).
|
||||
|
||||
-define(NOW, erlang:system_time(millisecond)).
|
||||
|
||||
-type inflight_data_phase() :: wait_ack | wait_comp.
|
||||
|
||||
-record(inflight_data, {
|
||||
phase :: inflight_data_phase(),
|
||||
message :: emqx_types:message(),
|
||||
timestamp :: non_neg_integer()
|
||||
}).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% CT callbacks
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
init_per_suite(Config) ->
|
||||
emqx_common_test_helpers:start_apps([]),
|
||||
ok = meck:new(
|
||||
[emqx_hooks, emqx_metrics, emqx_broker],
|
||||
[passthrough, no_history, no_link]
|
||||
),
|
||||
ok = meck:expect(emqx_metrics, inc, fun(_) -> ok end),
|
||||
ok = meck:expect(emqx_metrics, inc, fun(_K, _V) -> ok end),
|
||||
ok = meck:expect(emqx_hooks, run, fun(_Hook, _Args) -> ok end),
|
||||
Config.
|
||||
|
||||
end_per_suite(_Config) ->
|
||||
meck:unload([emqx_broker, emqx_hooks, emqx_metrics]).
|
||||
|
||||
init_per_testcase(_TestCase, Config) ->
|
||||
Config.
|
||||
|
||||
end_per_testcase(_TestCase, Config) ->
|
||||
Config.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Test cases for session init
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
t_session_init(_) ->
|
||||
Conf = emqx_cm:get_session_confs(
|
||||
#{zone => default, clientid => <<"fake-test">>}, #{
|
||||
receive_maximum => 64, expiry_interval => 0
|
||||
}
|
||||
),
|
||||
Session = emqx_session:init(Conf),
|
||||
?assertEqual(#{}, emqx_session:info(subscriptions, Session)),
|
||||
?assertEqual(0, emqx_session:info(subscriptions_cnt, Session)),
|
||||
?assertEqual(infinity, emqx_session:info(subscriptions_max, Session)),
|
||||
?assertEqual(false, emqx_session:info(upgrade_qos, Session)),
|
||||
?assertEqual(0, emqx_session:info(inflight_cnt, Session)),
|
||||
?assertEqual(64, emqx_session:info(inflight_max, Session)),
|
||||
?assertEqual(1, emqx_session:info(next_pkt_id, Session)),
|
||||
?assertEqual(30000, emqx_session:info(retry_interval, Session)),
|
||||
?assertEqual(0, emqx_mqueue:len(emqx_session:info(mqueue, Session))),
|
||||
?assertEqual(0, emqx_session:info(awaiting_rel_cnt, Session)),
|
||||
?assertEqual(100, emqx_session:info(awaiting_rel_max, Session)),
|
||||
?assertEqual(300000, emqx_session:info(await_rel_timeout, Session)),
|
||||
?assert(is_integer(emqx_session:info(created_at, Session))).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Test cases for session info/stats
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
t_session_info(_) ->
|
||||
?assertMatch(
|
||||
#{
|
||||
subscriptions := #{},
|
||||
upgrade_qos := false,
|
||||
retry_interval := 30000,
|
||||
await_rel_timeout := 300000
|
||||
},
|
||||
emqx_session:info(session())
|
||||
).
|
||||
|
||||
t_session_stats(_) ->
|
||||
Stats = emqx_session:stats(session()),
|
||||
?assertMatch(
|
||||
#{
|
||||
subscriptions_max := infinity,
|
||||
inflight_max := 0,
|
||||
mqueue_len := 0,
|
||||
mqueue_max := 1000,
|
||||
mqueue_dropped := 0,
|
||||
next_pkt_id := 1,
|
||||
awaiting_rel_cnt := 0,
|
||||
awaiting_rel_max := 100
|
||||
},
|
||||
maps:from_list(Stats)
|
||||
).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Test cases for sub/unsub
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
t_subscribe(_) ->
|
||||
ok = meck:expect(emqx_broker, subscribe, fun(_, _, _) -> ok end),
|
||||
{ok, Session} = emqx_session:subscribe(
|
||||
clientinfo(), <<"#">>, subopts(), session()
|
||||
),
|
||||
?assertEqual(1, emqx_session:info(subscriptions_cnt, Session)).
|
||||
|
||||
t_is_subscriptions_full_false(_) ->
|
||||
Session = session(#{max_subscriptions => infinity}),
|
||||
?assertNot(emqx_session:is_subscriptions_full(Session)).
|
||||
|
||||
t_is_subscriptions_full_true(_) ->
|
||||
ok = meck:expect(emqx_broker, subscribe, fun(_, _, _) -> ok end),
|
||||
Session = session(#{max_subscriptions => 1}),
|
||||
?assertNot(emqx_session:is_subscriptions_full(Session)),
|
||||
{ok, Session1} = emqx_session:subscribe(
|
||||
clientinfo(), <<"t1">>, subopts(), Session
|
||||
),
|
||||
?assert(emqx_session:is_subscriptions_full(Session1)),
|
||||
{error, ?RC_QUOTA_EXCEEDED} =
|
||||
emqx_session:subscribe(clientinfo(), <<"t2">>, subopts(), Session1).
|
||||
|
||||
t_unsubscribe(_) ->
|
||||
ok = meck:expect(emqx_broker, unsubscribe, fun(_) -> ok end),
|
||||
Session = session(#{subscriptions => #{<<"#">> => subopts()}}),
|
||||
{ok, Session1} = emqx_session:unsubscribe(clientinfo(), <<"#">>, #{}, Session),
|
||||
{error, ?RC_NO_SUBSCRIPTION_EXISTED} =
|
||||
emqx_session:unsubscribe(clientinfo(), <<"#">>, #{}, Session1).
|
||||
|
||||
t_publish_qos0(_) ->
|
||||
ok = meck:expect(emqx_broker, publish, fun(_) -> [] end),
|
||||
Msg = emqx_message:make(clientid, ?QOS_0, <<"t">>, <<"payload">>),
|
||||
{ok, [], Session} = emqx_session:publish(clientinfo(), 1, Msg, Session = session()),
|
||||
{ok, [], Session} = emqx_session:publish(clientinfo(), undefined, Msg, Session).
|
||||
|
||||
t_publish_qos1(_) ->
|
||||
ok = meck:expect(emqx_broker, publish, fun(_) -> [] end),
|
||||
Msg = emqx_message:make(clientid, ?QOS_1, <<"t">>, <<"payload">>),
|
||||
{ok, [], Session} = emqx_session:publish(clientinfo(), 1, Msg, Session = session()),
|
||||
{ok, [], Session} = emqx_session:publish(clientinfo(), 2, Msg, Session).
|
||||
|
||||
t_publish_qos2(_) ->
|
||||
ok = meck:expect(emqx_broker, publish, fun(_) -> [] end),
|
||||
Msg = emqx_message:make(clientid, ?QOS_2, <<"t">>, <<"payload">>),
|
||||
{ok, [], Session} = emqx_session:publish(clientinfo(), 1, Msg, session()),
|
||||
?assertEqual(1, emqx_session:info(awaiting_rel_cnt, Session)),
|
||||
{ok, Session1} = emqx_session:pubrel(clientinfo(), 1, Session),
|
||||
?assertEqual(0, emqx_session:info(awaiting_rel_cnt, Session1)),
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND} = emqx_session:pubrel(clientinfo(), 1, Session1).
|
||||
|
||||
t_publish_qos2_with_error_return(_) ->
|
||||
ok = meck:expect(emqx_broker, publish, fun(_) -> [] end),
|
||||
ok = meck:expect(emqx_hooks, run, fun
|
||||
('message.dropped', [Msg, _By, ReasonName]) ->
|
||||
self() ! {'message.dropped', ReasonName, Msg},
|
||||
ok;
|
||||
(_Hook, _Arg) ->
|
||||
ok
|
||||
end),
|
||||
|
||||
Session = session(#{max_awaiting_rel => 2, awaiting_rel => #{PacketId1 = 1 => ts(millisecond)}}),
|
||||
begin
|
||||
Msg1 = emqx_message:make(clientid, ?QOS_2, <<"t">>, <<"payload1">>),
|
||||
{error, RC1 = ?RC_PACKET_IDENTIFIER_IN_USE} = emqx_session:publish(
|
||||
clientinfo(), PacketId1, Msg1, Session
|
||||
),
|
||||
receive
|
||||
{'message.dropped', Reason1, RecMsg1} ->
|
||||
?assertEqual(Reason1, emqx_reason_codes:name(RC1)),
|
||||
?assertEqual(RecMsg1, Msg1)
|
||||
after 1000 ->
|
||||
ct:fail(?FUNCTION_NAME)
|
||||
end
|
||||
end,
|
||||
|
||||
begin
|
||||
Msg2 = emqx_message:make(clientid, ?QOS_2, <<"t">>, <<"payload2">>),
|
||||
{ok, [], Session1} = emqx_session:publish(clientinfo(), _PacketId2 = 2, Msg2, Session),
|
||||
?assertEqual(2, emqx_session:info(awaiting_rel_cnt, Session1)),
|
||||
{error, RC2 = ?RC_RECEIVE_MAXIMUM_EXCEEDED} = emqx_session:publish(
|
||||
clientinfo(), _PacketId3 = 3, Msg2, Session1
|
||||
),
|
||||
receive
|
||||
{'message.dropped', Reason2, RecMsg2} ->
|
||||
?assertEqual(Reason2, emqx_reason_codes:name(RC2)),
|
||||
?assertEqual(RecMsg2, Msg2)
|
||||
after 1000 ->
|
||||
ct:fail(?FUNCTION_NAME)
|
||||
end
|
||||
end,
|
||||
ok = meck:expect(emqx_hooks, run, fun(_Hook, _Args) -> ok end).
|
||||
|
||||
t_is_awaiting_full_false(_) ->
|
||||
Session = session(#{max_awaiting_rel => infinity}),
|
||||
?assertNot(emqx_session:is_awaiting_full(Session)).
|
||||
|
||||
t_is_awaiting_full_true(_) ->
|
||||
Session = session(#{
|
||||
max_awaiting_rel => 1,
|
||||
awaiting_rel => #{1 => ts(millisecond)}
|
||||
}),
|
||||
?assert(emqx_session:is_awaiting_full(Session)).
|
||||
|
||||
t_puback(_) ->
|
||||
Msg = emqx_message:make(test, ?QOS_1, <<"t">>, <<>>),
|
||||
Inflight = emqx_inflight:insert(1, with_ts(wait_ack, Msg), emqx_inflight:new()),
|
||||
Session = session(#{inflight => Inflight, mqueue => mqueue()}),
|
||||
{ok, Msg, Session1} = emqx_session:puback(clientinfo(), 1, Session),
|
||||
?assertEqual(0, emqx_session:info(inflight_cnt, Session1)).
|
||||
|
||||
t_puback_with_dequeue(_) ->
|
||||
Msg1 = emqx_message:make(clientid, ?QOS_1, <<"t1">>, <<"payload1">>),
|
||||
Inflight = emqx_inflight:insert(1, with_ts(wait_ack, Msg1), emqx_inflight:new()),
|
||||
Msg2 = emqx_message:make(clientid, ?QOS_1, <<"t2">>, <<"payload2">>),
|
||||
{_, Q} = emqx_mqueue:in(Msg2, mqueue(#{max_len => 10})),
|
||||
Session = session(#{inflight => Inflight, mqueue => Q}),
|
||||
{ok, Msg1, [{_, Msg3}], Session1} = emqx_session:puback(clientinfo(), 1, Session),
|
||||
?assertEqual(1, emqx_session:info(inflight_cnt, Session1)),
|
||||
?assertEqual(0, emqx_session:info(mqueue_len, Session1)),
|
||||
?assertEqual(<<"t2">>, emqx_message:topic(Msg3)).
|
||||
|
||||
t_puback_error_packet_id_in_use(_) ->
|
||||
Inflight = emqx_inflight:insert(1, with_ts(wait_comp, undefined), emqx_inflight:new()),
|
||||
{error, ?RC_PACKET_IDENTIFIER_IN_USE} =
|
||||
emqx_session:puback(clientinfo(), 1, session(#{inflight => Inflight})).
|
||||
|
||||
t_puback_error_packet_id_not_found(_) ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND} = emqx_session:puback(clientinfo(), 1, session()).
|
||||
|
||||
t_pubrec(_) ->
|
||||
Msg = emqx_message:make(test, ?QOS_2, <<"t">>, <<>>),
|
||||
Inflight = emqx_inflight:insert(2, with_ts(wait_ack, Msg), emqx_inflight:new()),
|
||||
Session = session(#{inflight => Inflight}),
|
||||
{ok, Msg, Session1} = emqx_session:pubrec(clientinfo(), 2, Session),
|
||||
?assertMatch(
|
||||
[#inflight_data{phase = wait_comp}],
|
||||
emqx_inflight:values(emqx_session:info(inflight, Session1))
|
||||
).
|
||||
|
||||
t_pubrec_packet_id_in_use_error(_) ->
|
||||
Inflight = emqx_inflight:insert(1, with_ts(wait_comp, undefined), emqx_inflight:new()),
|
||||
{error, ?RC_PACKET_IDENTIFIER_IN_USE} =
|
||||
emqx_session:pubrec(clientinfo(), 1, session(#{inflight => Inflight})).
|
||||
|
||||
t_pubrec_packet_id_not_found_error(_) ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND} = emqx_session:pubrec(clientinfo(), 1, session()).
|
||||
|
||||
t_pubrel(_) ->
|
||||
Session = session(#{awaiting_rel => #{1 => ts(millisecond)}}),
|
||||
{ok, Session1} = emqx_session:pubrel(clientinfo(), 1, Session),
|
||||
?assertEqual(#{}, emqx_session:info(awaiting_rel, Session1)).
|
||||
|
||||
t_pubrel_error_packetid_not_found(_) ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND} = emqx_session:pubrel(clientinfo(), 1, session()).
|
||||
|
||||
t_pubcomp(_) ->
|
||||
Inflight = emqx_inflight:insert(1, with_ts(wait_comp, undefined), emqx_inflight:new()),
|
||||
Session = session(#{inflight => Inflight}),
|
||||
{ok, Session1} = emqx_session:pubcomp(clientinfo(), 1, Session),
|
||||
?assertEqual(0, emqx_session:info(inflight_cnt, Session1)).
|
||||
|
||||
t_pubcomp_error_packetid_in_use(_) ->
|
||||
Msg = emqx_message:make(test, ?QOS_2, <<"t">>, <<>>),
|
||||
Inflight = emqx_inflight:insert(1, {Msg, ts(millisecond)}, emqx_inflight:new()),
|
||||
Session = session(#{inflight => Inflight}),
|
||||
{error, ?RC_PACKET_IDENTIFIER_IN_USE} = emqx_session:pubcomp(clientinfo(), 1, Session).
|
||||
|
||||
t_pubcomp_error_packetid_not_found(_) ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND} = emqx_session:pubcomp(clientinfo(), 1, session()).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Test cases for deliver/retry
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
t_dequeue(_) ->
|
||||
Q = mqueue(#{store_qos0 => true}),
|
||||
{ok, Session} = emqx_session:dequeue(clientinfo(), session(#{mqueue => Q})),
|
||||
Msgs = [
|
||||
emqx_message:make(clientid, ?QOS_0, <<"t0">>, <<"payload">>),
|
||||
emqx_message:make(clientid, ?QOS_1, <<"t1">>, <<"payload">>),
|
||||
emqx_message:make(clientid, ?QOS_2, <<"t2">>, <<"payload">>)
|
||||
],
|
||||
Session1 = lists:foldl(
|
||||
fun(Msg, S) ->
|
||||
emqx_session:enqueue(clientinfo(), Msg, S)
|
||||
end,
|
||||
Session,
|
||||
Msgs
|
||||
),
|
||||
{ok, [{undefined, Msg0}, {1, Msg1}, {2, Msg2}], Session2} =
|
||||
emqx_session:dequeue(clientinfo(), Session1),
|
||||
?assertEqual(0, emqx_session:info(mqueue_len, Session2)),
|
||||
?assertEqual(2, emqx_session:info(inflight_cnt, Session2)),
|
||||
?assertEqual(<<"t0">>, emqx_message:topic(Msg0)),
|
||||
?assertEqual(<<"t1">>, emqx_message:topic(Msg1)),
|
||||
?assertEqual(<<"t2">>, emqx_message:topic(Msg2)).
|
||||
|
||||
t_deliver_qos0(_) ->
|
||||
ok = meck:expect(emqx_broker, subscribe, fun(_, _, _) -> ok end),
|
||||
{ok, Session} = emqx_session:subscribe(
|
||||
clientinfo(), <<"t0">>, subopts(), session()
|
||||
),
|
||||
{ok, Session1} = emqx_session:subscribe(
|
||||
clientinfo(), <<"t1">>, subopts(), Session
|
||||
),
|
||||
Deliveries = [delivery(?QOS_0, T) || T <- [<<"t0">>, <<"t1">>]],
|
||||
{ok, [{undefined, Msg1}, {undefined, Msg2}], Session1} =
|
||||
emqx_session:deliver(clientinfo(), Deliveries, Session1),
|
||||
?assertEqual(<<"t0">>, emqx_message:topic(Msg1)),
|
||||
?assertEqual(<<"t1">>, emqx_message:topic(Msg2)).
|
||||
|
||||
t_deliver_qos1(_) ->
|
||||
ok = meck:expect(emqx_broker, subscribe, fun(_, _, _) -> ok end),
|
||||
{ok, Session} = emqx_session:subscribe(
|
||||
clientinfo(), <<"t1">>, subopts(#{qos => ?QOS_1}), session()
|
||||
),
|
||||
Delivers = [delivery(?QOS_1, T) || T <- [<<"t1">>, <<"t2">>]],
|
||||
{ok, [{1, Msg1}, {2, Msg2}], Session1} = emqx_session:deliver(clientinfo(), Delivers, Session),
|
||||
?assertEqual(2, emqx_session:info(inflight_cnt, Session1)),
|
||||
?assertEqual(<<"t1">>, emqx_message:topic(Msg1)),
|
||||
?assertEqual(<<"t2">>, emqx_message:topic(Msg2)),
|
||||
{ok, Msg1T, Session2} = emqx_session:puback(clientinfo(), 1, Session1),
|
||||
?assertEqual(Msg1, remove_deliver_flag(Msg1T)),
|
||||
?assertEqual(1, emqx_session:info(inflight_cnt, Session2)),
|
||||
{ok, Msg2T, Session3} = emqx_session:puback(clientinfo(), 2, Session2),
|
||||
?assertEqual(Msg2, remove_deliver_flag(Msg2T)),
|
||||
?assertEqual(0, emqx_session:info(inflight_cnt, Session3)).
|
||||
|
||||
t_deliver_qos2(_) ->
|
||||
ok = meck:expect(emqx_broker, subscribe, fun(_, _, _) -> ok end),
|
||||
Delivers = [delivery(?QOS_2, <<"t0">>), delivery(?QOS_2, <<"t1">>)],
|
||||
{ok, [{1, Msg1}, {2, Msg2}], Session} =
|
||||
emqx_session:deliver(clientinfo(), Delivers, session()),
|
||||
?assertEqual(2, emqx_session:info(inflight_cnt, Session)),
|
||||
?assertEqual(<<"t0">>, emqx_message:topic(Msg1)),
|
||||
?assertEqual(<<"t1">>, emqx_message:topic(Msg2)).
|
||||
|
||||
t_deliver_one_msg(_) ->
|
||||
{ok, [{1, Msg}], Session} =
|
||||
emqx_session:deliver(clientinfo(), [delivery(?QOS_1, <<"t1">>)], session()),
|
||||
?assertEqual(1, emqx_session:info(inflight_cnt, Session)),
|
||||
?assertEqual(<<"t1">>, emqx_message:topic(Msg)).
|
||||
|
||||
t_deliver_when_inflight_is_full(_) ->
|
||||
Delivers = [delivery(?QOS_1, <<"t1">>), delivery(?QOS_2, <<"t2">>)],
|
||||
Session = session(#{inflight => emqx_inflight:new(1)}),
|
||||
{ok, Publishes, Session1} = emqx_session:deliver(clientinfo(), Delivers, Session),
|
||||
?assertEqual(1, length(Publishes)),
|
||||
?assertEqual(1, emqx_session:info(inflight_cnt, Session1)),
|
||||
?assertEqual(1, emqx_session:info(mqueue_len, Session1)),
|
||||
{ok, Msg1, [{2, Msg2}], Session2} = emqx_session:puback(clientinfo(), 1, Session1),
|
||||
?assertEqual(1, emqx_session:info(inflight_cnt, Session2)),
|
||||
?assertEqual(0, emqx_session:info(mqueue_len, Session2)),
|
||||
?assertEqual(<<"t1">>, emqx_message:topic(Msg1)),
|
||||
?assertEqual(<<"t2">>, emqx_message:topic(Msg2)).
|
||||
|
||||
t_enqueue(_) ->
|
||||
%% store_qos0 = true
|
||||
Session = emqx_session:enqueue(clientinfo(), [delivery(?QOS_0, <<"t0">>)], session()),
|
||||
Session1 = emqx_session:enqueue(
|
||||
clientinfo(),
|
||||
[
|
||||
delivery(?QOS_1, <<"t1">>),
|
||||
delivery(?QOS_2, <<"t2">>)
|
||||
],
|
||||
Session
|
||||
),
|
||||
?assertEqual(3, emqx_session:info(mqueue_len, Session1)).
|
||||
|
||||
t_retry(_) ->
|
||||
Delivers = [delivery(?QOS_1, <<"t1">>), delivery(?QOS_2, <<"t2">>)],
|
||||
%% 0.1s
|
||||
RetryIntervalMs = 100,
|
||||
Session = session(#{retry_interval => RetryIntervalMs}),
|
||||
{ok, Pubs, Session1} = emqx_session:deliver(clientinfo(), Delivers, Session),
|
||||
%% 0.2s
|
||||
ElapseMs = 200,
|
||||
ok = timer:sleep(ElapseMs),
|
||||
Msgs1 = [{I, with_ts(wait_ack, emqx_message:set_flag(dup, Msg))} || {I, Msg} <- Pubs],
|
||||
{ok, Msgs1T, 100, Session2} = emqx_session:retry(clientinfo(), Session1),
|
||||
?assertEqual(inflight_data_to_msg(Msgs1), remove_deliver_flag(Msgs1T)),
|
||||
?assertEqual(2, emqx_session:info(inflight_cnt, Session2)).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Test cases for takeover/resume
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
t_takeover(_) ->
|
||||
ok = meck:expect(emqx_broker, unsubscribe, fun(_) -> ok end),
|
||||
Session = session(#{subscriptions => #{<<"t">> => ?DEFAULT_SUBOPTS}}),
|
||||
ok = emqx_session:takeover(Session).
|
||||
|
||||
t_resume(_) ->
|
||||
ok = meck:expect(emqx_broker, subscribe, fun(_, _, _) -> ok end),
|
||||
Session = session(#{subscriptions => #{<<"t">> => ?DEFAULT_SUBOPTS}}),
|
||||
ok = emqx_session:resume(#{clientid => <<"clientid">>}, Session).
|
||||
|
||||
t_replay(_) ->
|
||||
Delivers = [delivery(?QOS_1, <<"t1">>), delivery(?QOS_2, <<"t2">>)],
|
||||
{ok, Pubs, Session1} = emqx_session:deliver(clientinfo(), Delivers, session()),
|
||||
Msg = emqx_message:make(clientid, ?QOS_1, <<"t1">>, <<"payload">>),
|
||||
Session2 = emqx_session:enqueue(clientinfo(), Msg, Session1),
|
||||
Pubs1 = [{I, emqx_message:set_flag(dup, M)} || {I, M} <- Pubs],
|
||||
{ok, ReplayPubs, Session3} = emqx_session:replay(clientinfo(), Session2),
|
||||
?assertEqual(Pubs1 ++ [{3, Msg}], remove_deliver_flag(ReplayPubs)),
|
||||
?assertEqual(3, emqx_session:info(inflight_cnt, Session3)).
|
||||
|
||||
t_expire_awaiting_rel(_) ->
|
||||
{ok, Session} = emqx_session:expire(clientinfo(), awaiting_rel, session()),
|
||||
Timeout = emqx_session:info(await_rel_timeout, Session),
|
||||
Session1 = emqx_session:set_field(awaiting_rel, #{1 => Ts = ts(millisecond)}, Session),
|
||||
{ok, Timeout, Session2} = emqx_session:expire(clientinfo(), awaiting_rel, Session1),
|
||||
?assertEqual(#{1 => Ts}, emqx_session:info(awaiting_rel, Session2)).
|
||||
|
||||
t_expire_awaiting_rel_all(_) ->
|
||||
Session = session(#{awaiting_rel => #{1 => 1, 2 => 2}}),
|
||||
{ok, Session1} = emqx_session:expire(clientinfo(), awaiting_rel, Session),
|
||||
?assertEqual(#{}, emqx_session:info(awaiting_rel, Session1)).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% CT for utility functions
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
t_next_pakt_id(_) ->
|
||||
Session = session(#{next_pkt_id => 16#FFFF}),
|
||||
Session1 = emqx_session:next_pkt_id(Session),
|
||||
?assertEqual(1, emqx_session:info(next_pkt_id, Session1)),
|
||||
Session2 = emqx_session:next_pkt_id(Session1),
|
||||
?assertEqual(2, emqx_session:info(next_pkt_id, Session2)).
|
||||
|
||||
t_obtain_next_pkt_id(_) ->
|
||||
Session = session(#{next_pkt_id => 16#FFFF}),
|
||||
{16#FFFF, Session1} = emqx_session:obtain_next_pkt_id(Session),
|
||||
?assertEqual(1, emqx_session:info(next_pkt_id, Session1)),
|
||||
{1, Session2} = emqx_session:obtain_next_pkt_id(Session1),
|
||||
?assertEqual(2, emqx_session:info(next_pkt_id, Session2)).
|
||||
|
||||
%% Helper functions
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
mqueue() -> mqueue(#{}).
|
||||
mqueue(Opts) ->
|
||||
emqx_mqueue:init(maps:merge(#{max_len => 0, store_qos0 => false}, Opts)).
|
||||
|
||||
session() -> session(#{}).
|
||||
session(InitFields) when is_map(InitFields) ->
|
||||
Conf = emqx_cm:get_session_confs(
|
||||
#{zone => default, clientid => <<"fake-test">>}, #{
|
||||
receive_maximum => 0, expiry_interval => 0
|
||||
}
|
||||
),
|
||||
Session = emqx_session:init(Conf),
|
||||
maps:fold(
|
||||
fun(Field, Value, SessionAcc) ->
|
||||
emqx_session:set_field(Field, Value, SessionAcc)
|
||||
end,
|
||||
Session,
|
||||
InitFields
|
||||
).
|
||||
|
||||
clientinfo() -> clientinfo(#{}).
|
||||
clientinfo(Init) ->
|
||||
maps:merge(
|
||||
#{
|
||||
clientid => <<"clientid">>,
|
||||
username => <<"username">>
|
||||
},
|
||||
Init
|
||||
).
|
||||
|
||||
subopts() -> subopts(#{}).
|
||||
subopts(Init) ->
|
||||
maps:merge(?DEFAULT_SUBOPTS, Init).
|
||||
|
||||
delivery(QoS, Topic) ->
|
||||
{deliver, Topic, emqx_message:make(test, QoS, Topic, <<"payload">>)}.
|
||||
|
||||
ts(second) ->
|
||||
erlang:system_time(second);
|
||||
ts(millisecond) ->
|
||||
erlang:system_time(millisecond).
|
||||
|
||||
with_ts(Phase, Msg) ->
|
||||
with_ts(Phase, Msg, erlang:system_time(millisecond)).
|
||||
|
||||
with_ts(Phase, Msg, Ts) ->
|
||||
#inflight_data{
|
||||
phase = Phase,
|
||||
message = Msg,
|
||||
timestamp = Ts
|
||||
}.
|
||||
|
||||
remove_deliver_flag({Id, Data}) ->
|
||||
{Id, remove_deliver_flag(Data)};
|
||||
remove_deliver_flag(#inflight_data{message = Msg} = Data) ->
|
||||
Data#inflight_data{message = remove_deliver_flag(Msg)};
|
||||
remove_deliver_flag(List) when is_list(List) ->
|
||||
lists:map(fun remove_deliver_flag/1, List);
|
||||
remove_deliver_flag(Msg) ->
|
||||
emqx_message:remove_header(deliver_begin_at, Msg).
|
||||
|
||||
inflight_data_to_msg({Id, Data}) ->
|
||||
{Id, inflight_data_to_msg(Data)};
|
||||
inflight_data_to_msg(#inflight_data{message = Msg}) ->
|
||||
Msg;
|
||||
inflight_data_to_msg(List) when is_list(List) ->
|
||||
lists:map(fun inflight_data_to_msg/1, List).
|
|
@ -0,0 +1,613 @@
|
|||
%%--------------------------------------------------------------------
|
||||
%% Copyright (c) 2018-2023 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.
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-module(emqx_session_mem_SUITE).
|
||||
|
||||
-compile(export_all).
|
||||
-compile(nowarn_export_all).
|
||||
|
||||
-include_lib("emqx/include/emqx_mqtt.hrl").
|
||||
-include_lib("emqx/include/asserts.hrl").
|
||||
-include_lib("eunit/include/eunit.hrl").
|
||||
-include_lib("common_test/include/ct.hrl").
|
||||
|
||||
all() -> emqx_common_test_helpers:all(?MODULE).
|
||||
|
||||
-type inflight_data_phase() :: wait_ack | wait_comp.
|
||||
|
||||
-record(inflight_data, {
|
||||
phase :: inflight_data_phase(),
|
||||
message :: emqx_types:message(),
|
||||
timestamp :: non_neg_integer()
|
||||
}).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% CT callbacks
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
-define(assertTimerSet(NAME, TIMEOUT),
|
||||
?assertReceive({timer, NAME, TIMEOUT} when is_integer(TIMEOUT))
|
||||
).
|
||||
-define(assertTimerCancel(NAME),
|
||||
?assertReceive({timer, NAME, cancel})
|
||||
).
|
||||
|
||||
init_per_suite(Config) ->
|
||||
ok = meck:new(
|
||||
[emqx_broker, emqx_hooks, emqx_session],
|
||||
[passthrough, no_history, no_link]
|
||||
),
|
||||
ok = meck:expect(emqx_hooks, run, fun(_Hook, _Args) -> ok end),
|
||||
Apps = emqx_cth_suite:start(
|
||||
[
|
||||
{emqx, #{
|
||||
override_env => [{boot_modules, [broker]}]
|
||||
}}
|
||||
],
|
||||
#{work_dir => emqx_cth_suite:work_dir(Config)}
|
||||
),
|
||||
[{suite_apps, Apps} | Config].
|
||||
|
||||
end_per_suite(Config) ->
|
||||
ok = emqx_cth_suite:stop(?config(suite_apps, Config)),
|
||||
meck:unload([emqx_broker, emqx_hooks]).
|
||||
|
||||
init_per_testcase(_TestCase, Config) ->
|
||||
Pid = self(),
|
||||
ok = meck:expect(
|
||||
emqx_session, ensure_timer, fun(Name, Timeout, Timers) ->
|
||||
_ = Pid ! {timer, Name, Timeout},
|
||||
meck:passthrough([Name, Timeout, Timers])
|
||||
end
|
||||
),
|
||||
ok = meck:expect(
|
||||
emqx_session, cancel_timer, fun(Name, Timers) ->
|
||||
_ = Pid ! {timer, Name, cancel},
|
||||
meck:passthrough([Name, Timers])
|
||||
end
|
||||
),
|
||||
Config.
|
||||
|
||||
end_per_testcase(_TestCase, Config) ->
|
||||
ok = meck:delete(emqx_session, ensure_timer, 3),
|
||||
Config.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Test cases for session init
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
t_session_init(_) ->
|
||||
ClientInfo = #{zone => default, clientid => <<"fake-test">>},
|
||||
ConnInfo = #{receive_maximum => 64, expiry_interval => 0},
|
||||
Session = emqx_session_mem:create(
|
||||
ClientInfo,
|
||||
ConnInfo,
|
||||
emqx_session:get_session_conf(ClientInfo, ConnInfo)
|
||||
),
|
||||
?assertEqual(#{}, emqx_session_mem:info(subscriptions, Session)),
|
||||
?assertEqual(0, emqx_session_mem:info(subscriptions_cnt, Session)),
|
||||
?assertEqual(infinity, emqx_session_mem:info(subscriptions_max, Session)),
|
||||
?assertEqual(false, emqx_session_mem:info(upgrade_qos, Session)),
|
||||
?assertEqual(0, emqx_session_mem:info(inflight_cnt, Session)),
|
||||
?assertEqual(64, emqx_session_mem:info(inflight_max, Session)),
|
||||
?assertEqual(1, emqx_session_mem:info(next_pkt_id, Session)),
|
||||
?assertEqual(30000, emqx_session_mem:info(retry_interval, Session)),
|
||||
?assertEqual(0, emqx_mqueue:len(emqx_session_mem:info(mqueue, Session))),
|
||||
?assertEqual(0, emqx_session_mem:info(awaiting_rel_cnt, Session)),
|
||||
?assertEqual(100, emqx_session_mem:info(awaiting_rel_max, Session)),
|
||||
?assertEqual(300000, emqx_session_mem:info(await_rel_timeout, Session)),
|
||||
?assert(is_integer(emqx_session_mem:info(created_at, Session))).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Test cases for session info/stats
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
t_session_info(_) ->
|
||||
Keys = [subscriptions, upgrade_qos, retry_interval, await_rel_timeout],
|
||||
?assertMatch(
|
||||
#{
|
||||
subscriptions := #{},
|
||||
upgrade_qos := false,
|
||||
retry_interval := 30000,
|
||||
await_rel_timeout := 300000
|
||||
},
|
||||
maps:from_list(emqx_session_mem:info(Keys, session()))
|
||||
).
|
||||
|
||||
t_session_stats(_) ->
|
||||
Stats = emqx_session_mem:stats(session()),
|
||||
?assertMatch(
|
||||
#{
|
||||
subscriptions_max := infinity,
|
||||
inflight_max := 0,
|
||||
mqueue_len := 0,
|
||||
mqueue_max := 1000,
|
||||
mqueue_dropped := 0,
|
||||
next_pkt_id := 1,
|
||||
awaiting_rel_cnt := 0,
|
||||
awaiting_rel_max := 100
|
||||
},
|
||||
maps:from_list(Stats)
|
||||
).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Test cases for sub/unsub
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
t_subscribe(_) ->
|
||||
ok = meck:expect(emqx_broker, subscribe, fun(_, _, _) -> ok end),
|
||||
{ok, Session} = emqx_session_mem:subscribe(<<"#">>, subopts(), session()),
|
||||
?assertEqual(1, emqx_session_mem:info(subscriptions_cnt, Session)).
|
||||
|
||||
t_is_subscriptions_full_false(_) ->
|
||||
Session = session(#{max_subscriptions => infinity}),
|
||||
?assertNot(emqx_session_mem:is_subscriptions_full(Session)).
|
||||
|
||||
t_is_subscriptions_full_true(_) ->
|
||||
ok = meck:expect(emqx_broker, subscribe, fun(_, _, _) -> ok end),
|
||||
Session = session(#{max_subscriptions => 1}),
|
||||
?assertNot(emqx_session_mem:is_subscriptions_full(Session)),
|
||||
{ok, Session1} = emqx_session_mem:subscribe(
|
||||
<<"t1">>, subopts(), Session
|
||||
),
|
||||
?assert(emqx_session_mem:is_subscriptions_full(Session1)),
|
||||
{error, ?RC_QUOTA_EXCEEDED} = emqx_session_mem:subscribe(
|
||||
<<"t2">>, subopts(), Session1
|
||||
).
|
||||
|
||||
t_unsubscribe(_) ->
|
||||
ok = meck:expect(emqx_broker, unsubscribe, fun(_) -> ok end),
|
||||
SubOpts = subopts(),
|
||||
Session = session(#{subscriptions => #{<<"#">> => SubOpts}}),
|
||||
{ok, Session1, SubOpts} = emqx_session_mem:unsubscribe(<<"#">>, Session),
|
||||
{error, ?RC_NO_SUBSCRIPTION_EXISTED} = emqx_session_mem:unsubscribe(<<"#">>, Session1).
|
||||
|
||||
t_publish_qos0(_) ->
|
||||
ok = meck:expect(emqx_broker, publish, fun(_) -> [] end),
|
||||
Msg = emqx_message:make(clientid, ?QOS_0, <<"t">>, <<"payload">>),
|
||||
{ok, [], [], Session} = emqx_session_mem:publish(1, Msg, Session = session()),
|
||||
{ok, [], [], Session} = emqx_session_mem:publish(undefined, Msg, Session).
|
||||
|
||||
t_publish_qos1(_) ->
|
||||
ok = meck:expect(emqx_broker, publish, fun(_) -> [] end),
|
||||
Msg = emqx_message:make(clientid, ?QOS_1, <<"t">>, <<"payload">>),
|
||||
{ok, [], [], Session} = emqx_session_mem:publish(1, Msg, Session = session()),
|
||||
{ok, [], [], Session} = emqx_session_mem:publish(2, Msg, Session).
|
||||
|
||||
t_publish_qos2(_) ->
|
||||
ok = meck:expect(emqx_broker, publish, fun(_) -> [] end),
|
||||
Msg = emqx_message:make(clientid, ?QOS_2, <<"t">>, <<"payload">>),
|
||||
{ok, [], Session} = emqx_session_mem:publish(1, Msg, session()),
|
||||
?assertTimerSet(expire_awaiting_rel, _Timeout),
|
||||
?assertEqual(1, emqx_session_mem:info(awaiting_rel_cnt, Session)),
|
||||
{ok, Session1} = emqx_session_mem:pubrel(1, Session),
|
||||
?assertEqual(0, emqx_session_mem:info(awaiting_rel_cnt, Session1)),
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND} = emqx_session_mem:pubrel(1, Session1).
|
||||
|
||||
t_publish_qos2_with_error_return(_) ->
|
||||
ok = meck:expect(emqx_broker, publish, fun(_) -> [] end),
|
||||
ok = meck:expect(emqx_hooks, run, fun
|
||||
('message.dropped', [Msg, _By, ReasonName]) ->
|
||||
self() ! {'message.dropped', ReasonName, Msg},
|
||||
ok;
|
||||
(_Hook, _Arg) ->
|
||||
ok
|
||||
end),
|
||||
|
||||
Session = session(#{max_awaiting_rel => 2, awaiting_rel => #{PacketId1 = 1 => ts(millisecond)}}),
|
||||
begin
|
||||
Msg1 = emqx_message:make(clientid, ?QOS_2, <<"t">>, <<"payload1">>),
|
||||
{error, RC1 = ?RC_PACKET_IDENTIFIER_IN_USE} = emqx_session:publish(
|
||||
clientinfo(), PacketId1, Msg1, Session
|
||||
),
|
||||
receive
|
||||
{'message.dropped', Reason1, RecMsg1} ->
|
||||
?assertEqual(Reason1, emqx_reason_codes:name(RC1)),
|
||||
?assertEqual(RecMsg1, Msg1)
|
||||
after 1000 ->
|
||||
ct:fail(?FUNCTION_NAME)
|
||||
end
|
||||
end,
|
||||
|
||||
begin
|
||||
Msg2 = emqx_message:make(clientid, ?QOS_2, <<"t">>, <<"payload2">>),
|
||||
{ok, [], Session1} = emqx_session:publish(
|
||||
clientinfo(), _PacketId2 = 2, Msg2, Session
|
||||
),
|
||||
?assertEqual(2, emqx_session_mem:info(awaiting_rel_cnt, Session1)),
|
||||
{error, RC2 = ?RC_RECEIVE_MAXIMUM_EXCEEDED} = emqx_session:publish(
|
||||
clientinfo(), _PacketId3 = 3, Msg2, Session1
|
||||
),
|
||||
receive
|
||||
{'message.dropped', Reason2, RecMsg2} ->
|
||||
?assertEqual(Reason2, emqx_reason_codes:name(RC2)),
|
||||
?assertEqual(RecMsg2, Msg2)
|
||||
after 1000 ->
|
||||
ct:fail(?FUNCTION_NAME)
|
||||
end
|
||||
end,
|
||||
ok = meck:expect(emqx_hooks, run, fun(_Hook, _Args) -> ok end).
|
||||
|
||||
t_is_awaiting_full_false(_) ->
|
||||
Session = session(#{max_awaiting_rel => infinity}),
|
||||
?assertNot(emqx_session_mem:is_awaiting_full(Session)).
|
||||
|
||||
t_is_awaiting_full_true(_) ->
|
||||
Session = session(#{
|
||||
max_awaiting_rel => 1,
|
||||
awaiting_rel => #{1 => ts(millisecond)}
|
||||
}),
|
||||
?assert(emqx_session_mem:is_awaiting_full(Session)).
|
||||
|
||||
t_puback(_) ->
|
||||
Msg = emqx_message:make(test, ?QOS_1, <<"t">>, <<>>),
|
||||
Inflight = emqx_inflight:insert(1, with_ts(wait_ack, Msg), emqx_inflight:new()),
|
||||
Session = session(#{inflight => Inflight, mqueue => mqueue()}),
|
||||
{ok, Msg, [], Session1} = emqx_session_mem:puback(clientinfo(), 1, Session),
|
||||
?assertEqual(0, emqx_session_mem:info(inflight_cnt, Session1)).
|
||||
|
||||
t_puback_with_dequeue(_) ->
|
||||
Msg1 = emqx_message:make(clientid, ?QOS_1, <<"t1">>, <<"payload1">>),
|
||||
Inflight = emqx_inflight:insert(1, with_ts(wait_ack, Msg1), emqx_inflight:new()),
|
||||
Msg2 = emqx_message:make(clientid, ?QOS_1, <<"t2">>, <<"payload2">>),
|
||||
{_, Q} = emqx_mqueue:in(Msg2, mqueue(#{max_len => 10})),
|
||||
Session = session(#{inflight => Inflight, mqueue => Q}),
|
||||
{ok, Msg1, [{_, Msg3}], Session1} = emqx_session_mem:puback(clientinfo(), 1, Session),
|
||||
?assertTimerSet(retry_delivery, _Timeout),
|
||||
?assertEqual(1, emqx_session_mem:info(inflight_cnt, Session1)),
|
||||
?assertEqual(0, emqx_session_mem:info(mqueue_len, Session1)),
|
||||
?assertEqual(<<"t2">>, emqx_message:topic(Msg3)).
|
||||
|
||||
t_puback_error_packet_id_in_use(_) ->
|
||||
Inflight = emqx_inflight:insert(1, with_ts(wait_comp, undefined), emqx_inflight:new()),
|
||||
{error, ?RC_PACKET_IDENTIFIER_IN_USE} =
|
||||
emqx_session_mem:puback(clientinfo(), 1, session(#{inflight => Inflight})).
|
||||
|
||||
t_puback_error_packet_id_not_found(_) ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND} = emqx_session_mem:puback(clientinfo(), 1, session()).
|
||||
|
||||
t_pubrec(_) ->
|
||||
Msg = emqx_message:make(test, ?QOS_2, <<"t">>, <<>>),
|
||||
Inflight = emqx_inflight:insert(2, with_ts(wait_ack, Msg), emqx_inflight:new()),
|
||||
Session = session(#{inflight => Inflight}),
|
||||
{ok, Msg, Session1} = emqx_session_mem:pubrec(2, Session),
|
||||
?assertMatch(
|
||||
[#inflight_data{phase = wait_comp}],
|
||||
emqx_inflight:values(emqx_session_mem:info(inflight, Session1))
|
||||
).
|
||||
|
||||
t_pubrec_packet_id_in_use_error(_) ->
|
||||
Inflight = emqx_inflight:insert(1, with_ts(wait_comp, undefined), emqx_inflight:new()),
|
||||
Session = session(#{inflight => Inflight}),
|
||||
{error, ?RC_PACKET_IDENTIFIER_IN_USE} = emqx_session_mem:pubrec(1, Session).
|
||||
|
||||
t_pubrec_packet_id_not_found_error(_) ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND} = emqx_session_mem:pubrec(1, session()).
|
||||
|
||||
t_pubrel(_) ->
|
||||
Session = session(#{awaiting_rel => #{1 => ts(millisecond)}}),
|
||||
{ok, Session1} = emqx_session_mem:pubrel(1, Session),
|
||||
?assertEqual(#{}, emqx_session_mem:info(awaiting_rel, Session1)).
|
||||
|
||||
t_pubrel_error_packetid_not_found(_) ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND} = emqx_session_mem:pubrel(1, session()).
|
||||
|
||||
t_pubcomp(_) ->
|
||||
Inflight = emqx_inflight:insert(1, with_ts(wait_comp, undefined), emqx_inflight:new()),
|
||||
Session = session(#{inflight => Inflight}),
|
||||
{ok, undefined, [], Session1} = emqx_session_mem:pubcomp(clientinfo(), 1, Session),
|
||||
?assertEqual(0, emqx_session_mem:info(inflight_cnt, Session1)).
|
||||
|
||||
t_pubcomp_error_packetid_in_use(_) ->
|
||||
Msg = emqx_message:make(test, ?QOS_2, <<"t">>, <<>>),
|
||||
Inflight = emqx_inflight:insert(1, {Msg, ts(millisecond)}, emqx_inflight:new()),
|
||||
Session = session(#{inflight => Inflight}),
|
||||
{error, ?RC_PACKET_IDENTIFIER_IN_USE} = emqx_session_mem:pubcomp(clientinfo(), 1, Session).
|
||||
|
||||
t_pubcomp_error_packetid_not_found(_) ->
|
||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND} = emqx_session_mem:pubcomp(clientinfo(), 1, session()).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Test cases for deliver/retry
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
t_dequeue(_) ->
|
||||
Q = mqueue(#{store_qos0 => true}),
|
||||
{ok, [], Session} = emqx_session_mem:dequeue(clientinfo(), session(#{mqueue => Q})),
|
||||
Msgs = [
|
||||
emqx_message:make(clientid, ?QOS_0, <<"t0">>, <<"payload">>),
|
||||
emqx_message:make(clientid, ?QOS_1, <<"t1">>, <<"payload">>),
|
||||
emqx_message:make(clientid, ?QOS_2, <<"t2">>, <<"payload">>)
|
||||
],
|
||||
Session1 = emqx_session_mem:enqueue(clientinfo(), Msgs, Session),
|
||||
{ok, [{undefined, Msg0}, {1, Msg1}, {2, Msg2}], Session2} =
|
||||
emqx_session_mem:dequeue(clientinfo(), Session1),
|
||||
?assertTimerSet(retry_delivery, _Timeout),
|
||||
?assertEqual(0, emqx_session_mem:info(mqueue_len, Session2)),
|
||||
?assertEqual(2, emqx_session_mem:info(inflight_cnt, Session2)),
|
||||
?assertEqual(<<"t0">>, emqx_message:topic(Msg0)),
|
||||
?assertEqual(<<"t1">>, emqx_message:topic(Msg1)),
|
||||
?assertEqual(<<"t2">>, emqx_message:topic(Msg2)).
|
||||
|
||||
t_deliver_qos0(_) ->
|
||||
ok = meck:expect(emqx_broker, subscribe, fun(_, _, _) -> ok end),
|
||||
{ok, Session} = emqx_session_mem:subscribe(<<"t0">>, subopts(), session()),
|
||||
{ok, Session1} = emqx_session_mem:subscribe(<<"t1">>, subopts(), Session),
|
||||
Deliveries = enrich([delivery(?QOS_0, T) || T <- [<<"t0">>, <<"t1">>]], Session1),
|
||||
{ok, [{undefined, Msg1}, {undefined, Msg2}], Session1} =
|
||||
emqx_session_mem:deliver(clientinfo(), Deliveries, Session1),
|
||||
?assertTimerCancel(retry_delivery),
|
||||
?assertEqual(<<"t0">>, emqx_message:topic(Msg1)),
|
||||
?assertEqual(<<"t1">>, emqx_message:topic(Msg2)).
|
||||
|
||||
t_deliver_qos1(_) ->
|
||||
ok = meck:expect(emqx_broker, subscribe, fun(_, _, _) -> ok end),
|
||||
{ok, Session} = emqx_session_mem:subscribe(
|
||||
<<"t1">>, subopts(#{qos => ?QOS_1}), session()
|
||||
),
|
||||
Delivers = enrich([delivery(?QOS_1, T) || T <- [<<"t1">>, <<"t2">>]], Session),
|
||||
{ok, [{1, Msg1}, {2, Msg2}], Session1} =
|
||||
emqx_session_mem:deliver(clientinfo(), Delivers, Session),
|
||||
?assertTimerSet(retry_delivery, _Timeout),
|
||||
?assertEqual(2, emqx_session_mem:info(inflight_cnt, Session1)),
|
||||
?assertEqual(<<"t1">>, emqx_message:topic(Msg1)),
|
||||
?assertEqual(<<"t2">>, emqx_message:topic(Msg2)),
|
||||
{ok, Msg1T, [], Session2} = emqx_session_mem:puback(clientinfo(), 1, Session1),
|
||||
?assertEqual(Msg1, remove_deliver_flag(Msg1T)),
|
||||
?assertEqual(1, emqx_session_mem:info(inflight_cnt, Session2)),
|
||||
{ok, Msg2T, [], Session3} = emqx_session_mem:puback(clientinfo(), 2, Session2),
|
||||
?assertEqual(Msg2, remove_deliver_flag(Msg2T)),
|
||||
?assertEqual(0, emqx_session_mem:info(inflight_cnt, Session3)).
|
||||
|
||||
t_deliver_qos2(_) ->
|
||||
ok = meck:expect(emqx_broker, subscribe, fun(_, _, _) -> ok end),
|
||||
Session = session(),
|
||||
Delivers = enrich([delivery(?QOS_2, <<"t0">>), delivery(?QOS_2, <<"t1">>)], Session),
|
||||
{ok, [{1, Msg1}, {2, Msg2}], Session1} =
|
||||
emqx_session_mem:deliver(clientinfo(), Delivers, Session),
|
||||
?assertTimerSet(retry_delivery, _Timeout),
|
||||
?assertEqual(2, emqx_session_mem:info(inflight_cnt, Session1)),
|
||||
?assertEqual(<<"t0">>, emqx_message:topic(Msg1)),
|
||||
?assertEqual(<<"t1">>, emqx_message:topic(Msg2)).
|
||||
|
||||
t_deliver_one_msg(_) ->
|
||||
Session = session(),
|
||||
{ok, [{1, Msg}], Session1} = emqx_session_mem:deliver(
|
||||
clientinfo(),
|
||||
enrich(delivery(?QOS_1, <<"t1">>), Session),
|
||||
Session
|
||||
),
|
||||
?assertTimerSet(retry_delivery, _Timeout),
|
||||
?assertEqual(1, emqx_session_mem:info(inflight_cnt, Session1)),
|
||||
?assertEqual(<<"t1">>, emqx_message:topic(Msg)).
|
||||
|
||||
t_deliver_when_inflight_is_full(_) ->
|
||||
Session = session(#{inflight => emqx_inflight:new(1)}),
|
||||
Delivers = enrich([delivery(?QOS_1, <<"t1">>), delivery(?QOS_2, <<"t2">>)], Session),
|
||||
{ok, Publishes, Session1} =
|
||||
emqx_session_mem:deliver(clientinfo(), Delivers, Session),
|
||||
{timer, _, Timeout} = ?assertTimerSet(retry_delivery, _Timeout),
|
||||
?assertEqual(1, length(Publishes)),
|
||||
?assertEqual(1, emqx_session_mem:info(inflight_cnt, Session1)),
|
||||
?assertEqual(1, emqx_session_mem:info(mqueue_len, Session1)),
|
||||
{ok, Msg1, [{2, Msg2}], Session2} =
|
||||
emqx_session_mem:puback(clientinfo(), 1, Session1),
|
||||
?assertTimerSet(retry_delivery, Timeout),
|
||||
?assertEqual(1, emqx_session_mem:info(inflight_cnt, Session2)),
|
||||
?assertEqual(0, emqx_session_mem:info(mqueue_len, Session2)),
|
||||
?assertEqual(<<"t1">>, emqx_message:topic(Msg1)),
|
||||
?assertEqual(<<"t2">>, emqx_message:topic(Msg2)).
|
||||
|
||||
t_enqueue(_) ->
|
||||
Session = session(#{mqueue => mqueue(#{max_len => 3, store_qos0 => true})}),
|
||||
Session1 = emqx_session_mem:enqueue(
|
||||
clientinfo(),
|
||||
emqx_session:enrich_delivers(
|
||||
clientinfo(),
|
||||
[
|
||||
delivery(?QOS_0, <<"t0">>),
|
||||
delivery(?QOS_1, <<"t1">>),
|
||||
delivery(?QOS_2, <<"t2">>)
|
||||
],
|
||||
Session
|
||||
),
|
||||
Session
|
||||
),
|
||||
?assertEqual(3, emqx_session_mem:info(mqueue_len, Session1)),
|
||||
Session2 = emqx_session_mem:enqueue(
|
||||
clientinfo(),
|
||||
emqx_session:enrich_delivers(clientinfo(), [delivery(?QOS_1, <<"drop">>)], Session1),
|
||||
Session1
|
||||
),
|
||||
?assertEqual(3, emqx_session_mem:info(mqueue_len, Session2)).
|
||||
|
||||
t_enqueue_qos0(_) ->
|
||||
Session = session(#{mqueue => mqueue(#{store_qos0 => false})}),
|
||||
Session1 = emqx_session_mem:enqueue(
|
||||
clientinfo(),
|
||||
emqx_session:enrich_delivers(
|
||||
clientinfo(),
|
||||
[
|
||||
delivery(?QOS_0, <<"t0">>),
|
||||
delivery(?QOS_1, <<"t1">>),
|
||||
delivery(?QOS_2, <<"t2">>)
|
||||
],
|
||||
Session
|
||||
),
|
||||
Session
|
||||
),
|
||||
?assertEqual(2, emqx_session_mem:info(mqueue_len, Session1)).
|
||||
|
||||
t_retry(_) ->
|
||||
%% 0.1s
|
||||
RetryIntervalMs = 100,
|
||||
Session = session(#{retry_interval => RetryIntervalMs}),
|
||||
Delivers = enrich([delivery(?QOS_1, <<"t1">>), delivery(?QOS_2, <<"t2">>)], Session),
|
||||
{ok, Pubs, Session1} = emqx_session_mem:deliver(clientinfo(), Delivers, Session),
|
||||
{timer, Name, _} = ?assertTimerSet(_Name, RetryIntervalMs),
|
||||
%% 0.2s
|
||||
ElapseMs = 200,
|
||||
ok = timer:sleep(ElapseMs),
|
||||
Msgs1 = [{I, with_ts(wait_ack, emqx_message:set_flag(dup, Msg))} || {I, Msg} <- Pubs],
|
||||
{ok, Msgs1T, Session2} = emqx_session_mem:handle_timeout(clientinfo(), Name, Session1),
|
||||
?assertTimerSet(Name, RetryIntervalMs),
|
||||
?assertEqual(inflight_data_to_msg(Msgs1), remove_deliver_flag(Msgs1T)),
|
||||
?assertEqual(2, emqx_session_mem:info(inflight_cnt, Session2)).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Test cases for takeover/resume
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
t_takeover(_) ->
|
||||
ok = meck:expect(emqx_broker, unsubscribe, fun(_) -> ok end),
|
||||
Session = session(#{subscriptions => #{<<"t">> => ?DEFAULT_SUBOPTS}}),
|
||||
ok = emqx_session_mem:takeover(Session).
|
||||
|
||||
t_resume(_) ->
|
||||
ok = meck:expect(emqx_broker, subscribe, fun(_, _, _) -> ok end),
|
||||
Session = session(#{subscriptions => #{<<"t">> => ?DEFAULT_SUBOPTS}}),
|
||||
_ = emqx_session_mem:resume(#{clientid => <<"clientid">>}, Session).
|
||||
|
||||
t_replay(_) ->
|
||||
Session = session(),
|
||||
Messages = enrich([delivery(?QOS_1, <<"t1">>), delivery(?QOS_2, <<"t2">>)], Session),
|
||||
{ok, Pubs, Session1} = emqx_session_mem:deliver(clientinfo(), Messages, Session),
|
||||
Msg = emqx_message:make(clientid, ?QOS_1, <<"t1">>, <<"payload">>),
|
||||
Session2 = emqx_session_mem:enqueue(clientinfo(), [Msg], Session1),
|
||||
Pubs1 = [{I, emqx_message:set_flag(dup, M)} || {I, M} <- Pubs],
|
||||
Pendings =
|
||||
[Msg4, Msg5] = enrich(
|
||||
[_D4 = delivery(?QOS_1, <<"t4">>), D5 = delivery(?QOS_2, <<"t5">>)],
|
||||
Session1
|
||||
),
|
||||
_ = self() ! D5,
|
||||
_ = self() ! D6 = delivery(?QOS_1, <<"t6">>),
|
||||
[Msg6] = enrich([D6], Session1),
|
||||
{ok, ReplayPubs, Session3} = emqx_session_mem:replay(clientinfo(), Pendings, Session2),
|
||||
?assertEqual(
|
||||
Pubs1 ++ [{3, Msg}, {4, Msg4}, {5, Msg5}, {6, Msg6}],
|
||||
remove_deliver_flag(ReplayPubs)
|
||||
),
|
||||
?assertEqual(6, emqx_session_mem:info(inflight_cnt, Session3)).
|
||||
|
||||
t_expire_awaiting_rel(_) ->
|
||||
{ok, [], Session} = emqx_session_mem:expire(clientinfo(), session()),
|
||||
Timeout = emqx_session_mem:info(await_rel_timeout, Session),
|
||||
Session1 = emqx_session_mem:set_field(awaiting_rel, #{1 => Ts = ts(millisecond)}, Session),
|
||||
{ok, [], Session2} = emqx_session_mem:expire(clientinfo(), Session1),
|
||||
?assertTimerSet(expire_awaiting_rel, Timeout),
|
||||
?assertEqual(#{1 => Ts}, emqx_session_mem:info(awaiting_rel, Session2)).
|
||||
|
||||
t_expire_awaiting_rel_all(_) ->
|
||||
Session = session(#{awaiting_rel => #{1 => 1, 2 => 2}}),
|
||||
{ok, [], Session1} = emqx_session_mem:expire(clientinfo(), Session),
|
||||
?assertTimerCancel(expire_awaiting_rel),
|
||||
?assertEqual(#{}, emqx_session_mem:info(awaiting_rel, Session1)).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% CT for utility functions
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
t_next_pakt_id(_) ->
|
||||
Session = session(#{next_pkt_id => 16#FFFF}),
|
||||
Session1 = emqx_session_mem:next_pkt_id(Session),
|
||||
?assertEqual(1, emqx_session_mem:info(next_pkt_id, Session1)),
|
||||
Session2 = emqx_session_mem:next_pkt_id(Session1),
|
||||
?assertEqual(2, emqx_session_mem:info(next_pkt_id, Session2)).
|
||||
|
||||
t_obtain_next_pkt_id(_) ->
|
||||
Session = session(#{next_pkt_id => 16#FFFF}),
|
||||
{16#FFFF, Session1} = emqx_session_mem:obtain_next_pkt_id(Session),
|
||||
?assertEqual(1, emqx_session_mem:info(next_pkt_id, Session1)),
|
||||
{1, Session2} = emqx_session_mem:obtain_next_pkt_id(Session1),
|
||||
?assertEqual(2, emqx_session_mem:info(next_pkt_id, Session2)).
|
||||
|
||||
%% Helper functions
|
||||
%%--------------------------------------------------------------------
|
||||
|
||||
mqueue() -> mqueue(#{}).
|
||||
mqueue(Opts) ->
|
||||
emqx_mqueue:init(maps:merge(#{max_len => 0, store_qos0 => false}, Opts)).
|
||||
|
||||
session() -> session(#{}).
|
||||
session(InitFields) when is_map(InitFields) ->
|
||||
ClientInfo = #{zone => default, clientid => <<"fake-test">>},
|
||||
ConnInfo = #{receive_maximum => 0, expiry_interval => 0},
|
||||
Session = emqx_session_mem:create(
|
||||
ClientInfo,
|
||||
ConnInfo,
|
||||
emqx_session:get_session_conf(ClientInfo, ConnInfo)
|
||||
),
|
||||
maps:fold(
|
||||
fun(Field, Value, SessionAcc) ->
|
||||
emqx_session_mem:set_field(Field, Value, SessionAcc)
|
||||
end,
|
||||
Session,
|
||||
InitFields
|
||||
).
|
||||
|
||||
clientinfo() -> clientinfo(#{}).
|
||||
clientinfo(Init) ->
|
||||
maps:merge(
|
||||
#{
|
||||
clientid => <<"clientid">>,
|
||||
username => <<"username">>
|
||||
},
|
||||
Init
|
||||
).
|
||||
|
||||
subopts() -> subopts(#{}).
|
||||
subopts(Init) ->
|
||||
maps:merge(?DEFAULT_SUBOPTS, Init).
|
||||
|
||||
delivery(QoS, Topic) ->
|
||||
{deliver, Topic, emqx_message:make(test, QoS, Topic, <<"payload">>)}.
|
||||
|
||||
enrich(Delivers, Session) when is_list(Delivers) ->
|
||||
emqx_session:enrich_delivers(clientinfo(), Delivers, Session);
|
||||
enrich(Delivery, Session) when is_tuple(Delivery) ->
|
||||
enrich([Delivery], Session).
|
||||
|
||||
ts(second) ->
|
||||
erlang:system_time(second);
|
||||
ts(millisecond) ->
|
||||
erlang:system_time(millisecond).
|
||||
|
||||
with_ts(Phase, Msg) ->
|
||||
with_ts(Phase, Msg, erlang:system_time(millisecond)).
|
||||
|
||||
with_ts(Phase, Msg, Ts) ->
|
||||
#inflight_data{
|
||||
phase = Phase,
|
||||
message = Msg,
|
||||
timestamp = Ts
|
||||
}.
|
||||
|
||||
remove_deliver_flag({Id, Data}) ->
|
||||
{Id, remove_deliver_flag(Data)};
|
||||
remove_deliver_flag(#inflight_data{message = Msg} = Data) ->
|
||||
Data#inflight_data{message = remove_deliver_flag(Msg)};
|
||||
remove_deliver_flag(List) when is_list(List) ->
|
||||
lists:map(fun remove_deliver_flag/1, List);
|
||||
remove_deliver_flag(Msg) ->
|
||||
emqx_message:remove_header(deliver_begin_at, Msg).
|
||||
|
||||
inflight_data_to_msg({Id, Data}) ->
|
||||
{Id, inflight_data_to_msg(Data)};
|
||||
inflight_data_to_msg(#inflight_data{message = Msg}) ->
|
||||
Msg;
|
||||
inflight_data_to_msg(List) when is_list(List) ->
|
||||
lists:map(fun inflight_data_to_msg/1, List).
|
|
@ -611,10 +611,10 @@ channel(InitFields) ->
|
|||
is_superuser => false,
|
||||
mountpoint => undefined
|
||||
},
|
||||
Conf = emqx_cm:get_session_confs(ClientInfo, #{
|
||||
receive_maximum => 0, expiry_interval => 0
|
||||
}),
|
||||
Session = emqx_session:init(Conf),
|
||||
Session = emqx_session:create(
|
||||
ClientInfo,
|
||||
#{receive_maximum => 0, expiry_interval => 0}
|
||||
),
|
||||
maps:fold(
|
||||
fun(Field, Value, Channel) ->
|
||||
emqx_channel:set_field(Field, Value, Channel)
|
||||
|
|
|
@ -388,8 +388,8 @@ open_session(
|
|||
{ok, #{session => Session, present => false}}
|
||||
end,
|
||||
case takeover_session(GwName, ClientId) of
|
||||
{ok, ConnMod, ChanPid, Session} ->
|
||||
ok = SessionMod:resume(ClientInfo, Session),
|
||||
{ok, ConnMod, ChanPid, SessionIn} ->
|
||||
Session = SessionMod:resume(ClientInfo, SessionIn),
|
||||
case request_stepdown({takeover, 'end'}, ConnMod, ChanPid) of
|
||||
{ok, Pendings} ->
|
||||
register_channel(
|
||||
|
|
|
@ -1155,11 +1155,7 @@ do_publish(
|
|||
) ->
|
||||
case emqx_mqttsn_session:publish(ClientInfo, MsgId, Msg, Session) of
|
||||
{ok, _PubRes, NSession} ->
|
||||
NChannel1 = ensure_timer(
|
||||
expire_awaiting_rel,
|
||||
Channel#channel{session = NSession}
|
||||
),
|
||||
handle_out(pubrec, MsgId, NChannel1);
|
||||
handle_out(pubrec, MsgId, Channel#channel{session = NSession});
|
||||
{error, ?RC_PACKET_IDENTIFIER_IN_USE} ->
|
||||
ok = metrics_inc(Ctx, 'packets.publish.inuse'),
|
||||
%% XXX: Use PUBACK to reply a PUBLISH Error Code
|
||||
|
@ -1169,10 +1165,6 @@ do_publish(
|
|||
Channel
|
||||
);
|
||||
{error, ?RC_RECEIVE_MAXIMUM_EXCEEDED} ->
|
||||
?SLOG(warning, #{
|
||||
msg => "dropped_the_qos2_packet_due_to_awaiting_rel_full",
|
||||
msg_id => MsgId
|
||||
}),
|
||||
ok = metrics_inc(Ctx, 'packets.publish.dropped'),
|
||||
handle_out(puback, {TopicId, MsgId, ?SN_RC_CONGESTION}, Channel)
|
||||
end.
|
||||
|
@ -1430,18 +1422,11 @@ awake(
|
|||
clientid => ClientId,
|
||||
previous_state => ConnState
|
||||
}),
|
||||
{ok, Publishes, Session1} = emqx_mqttsn_session:replay(ClientInfo, Session),
|
||||
{NPublishes, NSession} =
|
||||
case emqx_mqttsn_session:deliver(ClientInfo, [], Session1) of
|
||||
{ok, Session2} ->
|
||||
{Publishes, Session2};
|
||||
{ok, More, Session2} ->
|
||||
{lists:append(Publishes, More), Session2}
|
||||
end,
|
||||
{ok, Publishes, NSession} = emqx_mqttsn_session:replay(ClientInfo, Session),
|
||||
Channel1 = cancel_timer(expire_asleep, Channel),
|
||||
{Replies0, NChannel0} = outgoing_deliver_and_register(
|
||||
do_deliver(
|
||||
NPublishes,
|
||||
Publishes,
|
||||
Channel1#channel{
|
||||
conn_state = awake, session = NSession
|
||||
}
|
||||
|
@ -1995,11 +1980,7 @@ handle_deliver(
|
|||
of
|
||||
{ok, Publishes, NSession} ->
|
||||
NChannel = Channel#channel{session = NSession},
|
||||
handle_out(
|
||||
publish,
|
||||
Publishes,
|
||||
ensure_timer(retry_delivery, NChannel)
|
||||
);
|
||||
handle_out(publish, Publishes, NChannel);
|
||||
{ok, NSession} ->
|
||||
{ok, Channel#channel{session = NSession}}
|
||||
end.
|
||||
|
@ -2065,51 +2046,27 @@ handle_timeout(
|
|||
end;
|
||||
handle_timeout(
|
||||
_TRef,
|
||||
_Name = retry_delivery,
|
||||
{emqx_session, _Name},
|
||||
Channel = #channel{conn_state = disconnected}
|
||||
) ->
|
||||
{ok, Channel};
|
||||
handle_timeout(
|
||||
_TRef,
|
||||
retry_delivery,
|
||||
{emqx_session, _Name},
|
||||
Channel = #channel{conn_state = asleep}
|
||||
) ->
|
||||
{ok, reset_timer(retry_delivery, Channel)};
|
||||
{ok, Channel};
|
||||
handle_timeout(
|
||||
_TRef,
|
||||
Name = retry_delivery,
|
||||
{emqx_session, Name},
|
||||
Channel = #channel{session = Session, clientinfo = ClientInfo}
|
||||
) ->
|
||||
case emqx_mqttsn_session:retry(ClientInfo, Session) of
|
||||
{ok, NSession} ->
|
||||
{ok, clean_timer(Name, Channel#channel{session = NSession})};
|
||||
{ok, Publishes, Timeout, NSession} ->
|
||||
NChannel = Channel#channel{session = NSession},
|
||||
case emqx_mqttsn_session:handle_timeout(ClientInfo, Name, Session) of
|
||||
{ok, [], NSession} ->
|
||||
{ok, Channel#channel{session = NSession}};
|
||||
{ok, Publishes, NSession} ->
|
||||
%% XXX: These replay messages should awaiting register acked?
|
||||
handle_out(publish, Publishes, reset_timer(Name, Timeout, NChannel))
|
||||
end;
|
||||
handle_timeout(
|
||||
_TRef,
|
||||
_Name = expire_awaiting_rel,
|
||||
Channel = #channel{conn_state = disconnected}
|
||||
) ->
|
||||
{ok, Channel};
|
||||
handle_timeout(
|
||||
_TRef,
|
||||
Name = expire_awaiting_rel,
|
||||
Channel = #channel{conn_state = asleep}
|
||||
) ->
|
||||
{ok, reset_timer(Name, Channel)};
|
||||
handle_timeout(
|
||||
_TRef,
|
||||
Name = expire_awaiting_rel,
|
||||
Channel = #channel{session = Session, clientinfo = ClientInfo}
|
||||
) ->
|
||||
case emqx_mqttsn_session:expire(ClientInfo, awaiting_rel, Session) of
|
||||
{ok, NSession} ->
|
||||
{ok, clean_timer(Name, Channel#channel{session = NSession})};
|
||||
{ok, Timeout, NSession} ->
|
||||
{ok, reset_timer(Name, Timeout, Channel#channel{session = NSession})}
|
||||
handle_out(publish, Publishes, Channel#channel{session = NSession})
|
||||
end;
|
||||
handle_timeout(
|
||||
_TRef,
|
||||
|
@ -2238,18 +2195,11 @@ ensure_timer(Name, Time, Channel = #channel{timers = Timers}) ->
|
|||
reset_timer(Name, Channel) ->
|
||||
ensure_timer(Name, clean_timer(Name, Channel)).
|
||||
|
||||
reset_timer(Name, Time, Channel) ->
|
||||
ensure_timer(Name, Time, clean_timer(Name, Channel)).
|
||||
|
||||
clean_timer(Name, Channel = #channel{timers = Timers}) ->
|
||||
Channel#channel{timers = maps:remove(Name, Timers)}.
|
||||
|
||||
interval(keepalive, #channel{keepalive = KeepAlive}) ->
|
||||
emqx_keepalive:info(interval, KeepAlive);
|
||||
interval(retry_delivery, #channel{session = Session}) ->
|
||||
emqx_mqttsn_session:info(retry_interval, Session);
|
||||
interval(expire_awaiting_rel, #channel{session = Session}) ->
|
||||
emqx_mqttsn_session:info(await_rel_timeout, Session).
|
||||
emqx_keepalive:info(interval, KeepAlive).
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% Helper functions
|
||||
|
|
|
@ -22,8 +22,7 @@
|
|||
init/1,
|
||||
info/1,
|
||||
info/2,
|
||||
stats/1,
|
||||
resume/2
|
||||
stats/1
|
||||
]).
|
||||
|
||||
-export([
|
||||
|
@ -39,11 +38,11 @@
|
|||
-export([
|
||||
replay/2,
|
||||
deliver/3,
|
||||
handle_timeout/3,
|
||||
obtain_next_pkt_id/1,
|
||||
takeover/1,
|
||||
enqueue/3,
|
||||
retry/2,
|
||||
expire/3
|
||||
resume/2,
|
||||
enqueue/3
|
||||
]).
|
||||
|
||||
-type session() :: #{
|
||||
|
@ -54,12 +53,11 @@
|
|||
-export_type([session/0]).
|
||||
|
||||
init(ClientInfo) ->
|
||||
Conf = emqx_cm:get_session_confs(
|
||||
ClientInfo, #{receive_maximum => 1, expiry_interval => 0}
|
||||
),
|
||||
ConnInfo = #{receive_maximum => 1, expiry_interval => 0},
|
||||
SessionConf = emqx_session:get_session_conf(ClientInfo, ConnInfo),
|
||||
#{
|
||||
registry => emqx_mqttsn_registry:init(),
|
||||
session => emqx_session:init(Conf)
|
||||
session => emqx_session_mem:create(ClientInfo, ConnInfo, SessionConf)
|
||||
}.
|
||||
|
||||
registry(#{registry := Registry}) ->
|
||||
|
@ -98,47 +96,45 @@ subscribe(ClientInfo, Topic, SubOpts, Session) ->
|
|||
unsubscribe(ClientInfo, Topic, SubOpts, Session) ->
|
||||
with_sess(?FUNCTION_NAME, [ClientInfo, Topic, SubOpts], Session).
|
||||
|
||||
replay(ClientInfo, Session) ->
|
||||
with_sess(?FUNCTION_NAME, [ClientInfo], Session).
|
||||
deliver(ClientInfo, Delivers, Session) ->
|
||||
with_sess(?FUNCTION_NAME, [ClientInfo, Delivers], Session).
|
||||
|
||||
deliver(ClientInfo, Delivers, Session1) ->
|
||||
with_sess(?FUNCTION_NAME, [ClientInfo, Delivers], Session1).
|
||||
handle_timeout(ClientInfo, Name, Session) ->
|
||||
with_sess(?FUNCTION_NAME, [ClientInfo, Name], Session).
|
||||
|
||||
obtain_next_pkt_id(Session = #{session := Sess}) ->
|
||||
{Id, Sess1} = emqx_session:obtain_next_pkt_id(Sess),
|
||||
{Id, Sess1} = emqx_session_mem:obtain_next_pkt_id(Sess),
|
||||
{Id, Session#{session := Sess1}}.
|
||||
|
||||
takeover(_Session = #{session := Sess}) ->
|
||||
emqx_session:takeover(Sess).
|
||||
emqx_session_mem:takeover(Sess).
|
||||
|
||||
resume(ClientInfo, Session = #{session := Sess}) ->
|
||||
Session#{session := emqx_session_mem:resume(ClientInfo, Sess)}.
|
||||
|
||||
replay(ClientInfo, Session = #{session := Sess}) ->
|
||||
{ok, Replies, NSess} = emqx_session_mem:replay(ClientInfo, Sess),
|
||||
{ok, Replies, Session#{session := NSess}}.
|
||||
|
||||
enqueue(ClientInfo, Delivers, Session = #{session := Sess}) ->
|
||||
Sess1 = emqx_session:enqueue(ClientInfo, Delivers, Sess),
|
||||
Session#{session := Sess1}.
|
||||
|
||||
retry(ClientInfo, Session) ->
|
||||
with_sess(?FUNCTION_NAME, [ClientInfo], Session).
|
||||
|
||||
expire(ClientInfo, awaiting_rel, Session) ->
|
||||
with_sess(?FUNCTION_NAME, [ClientInfo, awaiting_rel], Session).
|
||||
|
||||
resume(ClientInfo, #{session := Sess}) ->
|
||||
emqx_session:resume(ClientInfo, Sess).
|
||||
Msgs = emqx_session:enrich_delivers(ClientInfo, Delivers, Sess),
|
||||
Session#{session := emqx_session_mem:enqueue(ClientInfo, Msgs, Sess)}.
|
||||
|
||||
%%--------------------------------------------------------------------
|
||||
%% internal funcs
|
||||
|
||||
with_sess(Fun, Args, Session = #{session := Sess}) ->
|
||||
case apply(emqx_session, Fun, Args ++ [Sess]) of
|
||||
%% for subscribe
|
||||
{error, Reason} ->
|
||||
{error, Reason};
|
||||
%% for pubrel
|
||||
%% for subscribe / unsubscribe / pubrel
|
||||
{ok, Sess1} ->
|
||||
{ok, Session#{session := Sess1}};
|
||||
%% for publish and puback
|
||||
{ok, Result, Sess1} ->
|
||||
{ok, Result, Session#{session := Sess1}};
|
||||
%% for publish / pubrec / pubcomp / deliver
|
||||
{ok, ResultReplies, Sess1} ->
|
||||
{ok, ResultReplies, Session#{session := Sess1}};
|
||||
%% for puback
|
||||
{ok, Msgs, Replies, Sess1} ->
|
||||
{ok, Msgs, Replies, Session#{session := Sess1}}
|
||||
{ok, Msgs, Replies, Session#{session := Sess1}};
|
||||
%% for any errors
|
||||
{error, Reason} ->
|
||||
{error, Reason}
|
||||
end.
|
||||
|
|
|
@ -40,11 +40,6 @@
|
|||
-define(HOST, {127, 0, 0, 1}).
|
||||
-define(PORT, 1884).
|
||||
|
||||
-define(FLAG_DUP(X), X).
|
||||
-define(FLAG_QOS(X), X).
|
||||
-define(FLAG_RETAIN(X), X).
|
||||
-define(FLAG_SESSION(X), X).
|
||||
|
||||
-define(LOG(Format, Args), ct:log("TEST: " ++ Format, Args)).
|
||||
|
||||
-define(MAX_PRED_TOPIC_ID, ?SN_MAX_PREDEF_TOPIC_ID).
|
||||
|
@ -1381,14 +1376,14 @@ t_asleep_test01_timeout(_) ->
|
|||
|
||||
t_asleep_test02_to_awake_and_back(_) ->
|
||||
QoS = 1,
|
||||
Keepalive_Duration = 1,
|
||||
KeepaliveDuration = 1,
|
||||
SleepDuration = 5,
|
||||
WillTopic = <<"dead">>,
|
||||
WillPayload = <<10, 11, 12, 13, 14>>,
|
||||
{ok, Socket} = gen_udp:open(0, [binary]),
|
||||
|
||||
ClientId = ?CLIENTID,
|
||||
send_connect_msg_with_will(Socket, Keepalive_Duration, ClientId),
|
||||
send_connect_msg_with_will(Socket, KeepaliveDuration, ClientId),
|
||||
?assertEqual(<<2, ?SN_WILLTOPICREQ>>, receive_response(Socket)),
|
||||
send_willtopic_msg(Socket, WillTopic, QoS),
|
||||
?assertEqual(<<2, ?SN_WILLMSGREQ>>, receive_response(Socket)),
|
||||
|
|
Loading…
Reference in New Issue