Merge pull request #11592 from keynslug/ft/EMQX-10827/session-impl

feat(session): introduce session implementation concept
This commit is contained in:
Andrew Mayorov 2023-09-21 00:18:13 +04:00 committed by GitHub
commit d799b3a785
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
52 changed files with 3105 additions and 4279 deletions

View File

@ -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(),
%% Clients 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_map_key(id, S))).
-endif.

View File

@ -0,0 +1,55 @@
%%--------------------------------------------------------------------
%% 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(),
%% Clients 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()
}).
-endif.

View File

@ -72,30 +72,19 @@ end_per_testcase(_TestCase, _Config) ->
%%------------------------------------------------------------------------------
cluster(#{n := N}) ->
Node1 = ds_SUITE1,
Spec = #{
role => core,
join_to => emqx_cth_cluster:node_name(Node1),
apps => app_specs()
},
[
{Node1, Spec}
| lists:map(
fun(M) ->
Name = binary_to_atom(<<"ds_SUITE", (integer_to_binary(M))/binary>>),
{Name, Spec}
end,
lists:seq(2, N)
)
].
Spec = #{role => core, apps => app_specs()},
lists:map(
fun(M) ->
Name = list_to_atom("ds_SUITE" ++ integer_to_list(M)),
{Name, Spec}
end,
lists:seq(1, N)
).
app_specs() ->
[
emqx_durable_storage,
{emqx, #{
config => #{persistent_session_store => #{ds => true}},
override_env => [{boot_modules, [broker, listeners]}]
}}
{emqx, "persistent_session_store = {ds = true}"}
].
get_mqtt_port(Node, Type) ->
@ -256,10 +245,9 @@ t_session_subscription_idempotency(Config) ->
?assertEqual([{ClientId, SubTopicFilterWords}], get_all_iterator_refs(Node1)),
?assertMatch({ok, [_]}, get_all_iterator_ids(Node1)),
?assertMatch(
{_IsNew = false, ClientId},
{ok, #{}, #{SubTopicFilterWords := #{}}},
erpc:call(Node1, emqx_ds, session_open, [ClientId])
),
ok
)
end
),
ok.

View File

@ -46,7 +46,6 @@
{emqx_node_rebalance_purge,1}.
{emqx_node_rebalance_status,1}.
{emqx_node_rebalance_status,2}.
{emqx_persistent_session,1}.
{emqx_persistent_session_ds,1}.
{emqx_plugins,1}.
{emqx_prometheus,1}.

View File

@ -38,8 +38,7 @@
start(_Type, _Args) ->
ok = maybe_load_config(),
ok = emqx_persistent_session:init_db_backend(),
_ = emqx_persistent_session_ds:init(),
_ = emqx_persistent_message:init(),
ok = maybe_start_quicer(),
ok = emqx_bpapi:start(),
ok = emqx_alarm_handler:load(),

View File

@ -224,8 +224,17 @@ publish(Msg) when is_record(Msg, message) ->
}),
[];
Msg1 = #message{topic = Topic} ->
_ = emqx_persistent_session_ds:persist_message(Msg1),
route(aggre(emqx_router:match_routes(Topic)), delivery(Msg1))
PersistRes = persist_publish(Msg1),
PersistRes ++ route(aggre(emqx_router:match_routes(Topic)), delivery(Msg1))
end.
persist_publish(Msg) ->
case emqx_persistent_message:persist(Msg) of
ok ->
[persisted];
{_SkipOrError, _Reason} ->
% TODO: log errors?
[]
end.
%% Called internally

View File

@ -84,7 +84,7 @@
%% MQTT ClientInfo
clientinfo :: emqx_types:clientinfo(),
%% MQTT Session
session :: maybe(emqx_session:session()),
session :: maybe(emqx_session:t()),
%% Keepalive
keepalive :: maybe(emqx_keepalive:keepalive()),
%% MQTT Will Msg
@ -104,7 +104,7 @@
%% Takeover
takeover :: boolean(),
%% Resume
resuming :: boolean(),
resuming :: false | _ReplayContext,
%% Pending delivers when takeovering
pendings :: list()
}).
@ -130,14 +130,9 @@
-define(IS_MQTT_V5, #channel{conninfo = #{proto_ver := ?MQTT_PROTO_V5}}).
-define(TIMER_TABLE, #{
alive_timer => keepalive,
retry_timer => retry_delivery,
await_timer => expire_awaiting_rel,
expire_timer => expire_session,
will_timer => will_message,
quota_timer => expire_quota_limit
}).
-define(IS_COMMON_SESSION_TIMER(N),
((N == retry_delivery) orelse (N == expire_awaiting_rel))
).
-define(LIMITER_ROUTING, message_routing).
@ -412,7 +407,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} ->
@ -469,7 +464,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});
@ -602,12 +597,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} ->
@ -734,7 +727,7 @@ do_publish(
{ok, PubRes, NSession} ->
RC = pubrec_reason_code(PubRes),
NChannel0 = Channel#channel{session = NSession},
NChannel1 = ensure_timer(await_timer, NChannel0),
NChannel1 = ensure_timer(expire_awaiting_rel, NChannel0),
NChannel2 = ensure_quota(PubRes, NChannel1),
handle_out(pubrec, {PacketId, RC}, NChannel2);
{error, RC = ?RC_PACKET_IDENTIFIER_IN_USE} ->
@ -765,7 +758,7 @@ ensure_quota(PubRes, Channel = #channel{quota = Limiter}) ->
{ok, NLimiter} ->
Channel#channel{quota = NLimiter};
{_, Intv, NLimiter} ->
ensure_timer(quota_timer, Intv, Channel#channel{quota = NLimiter})
ensure_timer(expire_quota_limit, Intv, Channel#channel{quota = NLimiter})
end.
-compile({inline, [pubrec_reason_code/1]}).
@ -909,8 +902,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
@ -946,10 +939,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{
@ -959,11 +954,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_timer, NChannel));
{ok, NSession} ->
{ok, Channel#channel{session = NSession}}
handle_out(publish, Publishes, ensure_timer(retry_delivery, NChannel))
end.
%% Nack delivers from shared subscription
@ -1077,6 +1072,10 @@ return_connack(AckPacket, Channel) ->
},
{Packets, NChannel2} = do_deliver(Publishes, NChannel1),
Outgoing = [?REPLY_OUTGOING(Packets) || length(Packets) > 0],
%% NOTE
%% Session timers are not restored here, so there's a tiny chance that
%% the session becomes stuck, when it already has no place to track new
%% messages.
{ok, Replies ++ Outgoing, NChannel2}
end.
@ -1173,10 +1172,12 @@ 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),
AllPendings = lists:append(Pendings, maybe_nack(Delivers)),
?tp(
debug,
emqx_channel_takeover_end,
@ -1199,7 +1200,7 @@ handle_call(
SockInfo = maps:get(sockinfo, emqx_cm:get_chan_info(ClientId), #{}),
ChanInfo1 = info(NChannel),
emqx_cm:set_chan_info(ClientId, ChanInfo1#{sockinfo => SockInfo}),
reply(ok, reset_timer(alive_timer, NChannel));
reply(ok, reset_timer(keepalive, NChannel));
handle_call(Req, Channel) ->
?SLOG(error, #{msg => "unexpected_call", call => Req}),
reply(ignored, Channel).
@ -1231,14 +1232,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}) ->
@ -1305,66 +1310,54 @@ handle_timeout(
case emqx_keepalive:check(StatVal, Keepalive) of
{ok, NKeepalive} ->
NChannel = Channel#channel{keepalive = NKeepalive},
{ok, reset_timer(alive_timer, NChannel)};
{ok, reset_timer(keepalive, NChannel)};
{error, timeout} ->
handle_out(disconnect, ?RC_KEEP_ALIVE_TIMEOUT, Channel)
end;
handle_timeout(
_TRef,
retry_delivery,
TimerName,
Channel = #channel{conn_state = disconnected}
) ->
) when ?IS_COMMON_SESSION_TIMER(TimerName) ->
{ok, Channel};
handle_timeout(
_TRef,
retry_delivery,
TimerName,
Channel = #channel{session = Session, clientinfo = ClientInfo}
) ->
case emqx_session:retry(ClientInfo, Session) of
{ok, NSession} ->
) when ?IS_COMMON_SESSION_TIMER(TimerName) ->
%% NOTE
%% Responsibility for these timers is smeared across both this module and the
%% `emqx_session` module: the latter holds configured timer intervals, and is
%% responsible for the actual timeout logic. Yet they are managed here, since
%% they are kind of common to all session implementations.
case emqx_session:handle_timeout(ClientInfo, TimerName, Session) of
{ok, Publishes, NSession} ->
NChannel = Channel#channel{session = NSession},
{ok, clean_timer(retry_timer, NChannel)};
handle_out(publish, Publishes, clean_timer(TimerName, NChannel));
{ok, Publishes, Timeout, NSession} ->
NChannel = Channel#channel{session = NSession},
handle_out(publish, Publishes, reset_timer(retry_timer, Timeout, NChannel))
end;
handle_timeout(
_TRef,
expire_awaiting_rel,
Channel = #channel{conn_state = disconnected}
) ->
{ok, Channel};
handle_timeout(
_TRef,
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(await_timer, NChannel)};
{ok, Timeout, NSession} ->
NChannel = Channel#channel{session = NSession},
{ok, reset_timer(await_timer, Timeout, NChannel)}
handle_out(publish, Publishes, reset_timer(TimerName, Timeout, NChannel))
end;
handle_timeout(_TRef, expire_session, Channel) ->
shutdown(expired, Channel);
handle_timeout(
_TRef, will_message, Channel = #channel{clientinfo = ClientInfo, will_msg = WillMsg}
_TRef,
will_message = TimerName,
Channel = #channel{clientinfo = ClientInfo, will_msg = WillMsg}
) ->
(WillMsg =/= undefined) andalso publish_will_msg(ClientInfo, WillMsg),
{ok, clean_timer(will_timer, Channel#channel{will_msg = undefined})};
{ok, clean_timer(TimerName, Channel#channel{will_msg = undefined})};
handle_timeout(
_TRef,
expire_quota_limit,
expire_quota_limit = TimerName,
#channel{quota = Quota} = Channel
) ->
case emqx_limiter_container:retry(?LIMITER_ROUTING, Quota) of
{_, Intv, Quota2} ->
Channel2 = ensure_timer(quota_timer, Intv, Channel#channel{quota = Quota2}),
Channel2 = ensure_timer(TimerName, Intv, Channel#channel{quota = Quota2}),
{ok, Channel2};
{_, Quota2} ->
{ok, clean_timer(quota_timer, Channel#channel{quota = Quota2})}
{ok, clean_timer(TimerName, Channel#channel{quota = Quota2})}
end;
handle_timeout(TRef, Msg, Channel) ->
case emqx_hooks:run_fold('client.timeout', [TRef, Msg], []) of
@ -1392,8 +1385,7 @@ ensure_timer(Name, Channel = #channel{timers = Timers}) ->
end.
ensure_timer(Name, Time, Channel = #channel{timers = Timers}) ->
Msg = maps:get(Name, ?TIMER_TABLE),
TRef = emqx_utils:start_timer(Time, Msg),
TRef = emqx_utils:start_timer(Time, Name),
Channel#channel{timers = Timers#{Name => TRef}}.
reset_timer(Name, Channel) ->
@ -1405,15 +1397,15 @@ reset_timer(Name, Time, Channel) ->
clean_timer(Name, Channel = #channel{timers = Timers}) ->
Channel#channel{timers = maps:remove(Name, Timers)}.
interval(alive_timer, #channel{keepalive = KeepAlive}) ->
interval(keepalive, #channel{keepalive = KeepAlive}) ->
emqx_keepalive:info(interval, KeepAlive);
interval(retry_timer, #channel{session = Session}) ->
interval(retry_delivery, #channel{session = Session}) ->
emqx_session:info(retry_interval, Session);
interval(await_timer, #channel{session = Session}) ->
interval(expire_awaiting_rel, #channel{session = Session}) ->
emqx_session:info(await_rel_timeout, Session);
interval(expire_timer, #channel{conninfo = ConnInfo}) ->
interval(expire_session, #channel{conninfo = ConnInfo}) ->
maps:get(expiry_interval, ConnInfo);
interval(will_timer, #channel{will_msg = WillMsg}) ->
interval(will_message, #channel{will_msg = WillMsg}) ->
timer:seconds(will_delay_interval(WillMsg)).
%%--------------------------------------------------------------------
@ -1783,7 +1775,7 @@ packing_alias(Packet, Channel) ->
%% Check quota state
check_quota_exceeded(_, #channel{timers = Timers}) ->
case maps:get(quota_timer, Timers, undefined) of
case maps:get(expire_quota_limit, Timers, undefined) of
undefined -> ok;
_ -> {error, ?RC_QUOTA_EXCEEDED}
end.
@ -2044,15 +2036,15 @@ ensure_keepalive_timer(Interval, Channel = #channel{clientinfo = #{zone := Zone}
Multiplier = get_mqtt_conf(Zone, keepalive_multiplier),
RecvCnt = emqx_pd:get_counter(recv_pkt),
Keepalive = emqx_keepalive:init(RecvCnt, round(timer:seconds(Interval) * Multiplier)),
ensure_timer(alive_timer, Channel#channel{keepalive = Keepalive}).
ensure_timer(keepalive, Channel#channel{keepalive = Keepalive}).
clear_keepalive(Channel = #channel{timers = Timers}) ->
case maps:get(alive_timer, Timers, undefined) of
case maps:get(keepalive, Timers, undefined) of
undefined ->
Channel;
TRef ->
emqx_utils:cancel_timer(TRef),
Channel#channel{timers = maps:without([alive_timer], Timers)}
Channel#channel{timers = maps:without([keepalive], Timers)}
end.
%%--------------------------------------------------------------------
%% Maybe Resume Session
@ -2061,30 +2053,25 @@ 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};
I when I > 0 ->
{ok, ensure_timer(expire_timer, I, Channel)};
{ok, ensure_timer(expire_session, I, Channel)};
_ ->
shutdown(Reason, Channel)
end.
end;
maybe_shutdown(Reason, _Intent = shutdown, Channel) ->
shutdown(Reason, Channel).
%%--------------------------------------------------------------------
%% Parse Topic Filters
@ -2120,7 +2107,7 @@ maybe_publish_will_msg(Channel = #channel{clientinfo = ClientInfo, will_msg = Wi
ok = publish_will_msg(ClientInfo, WillMsg),
Channel#channel{will_msg = undefined};
I ->
ensure_timer(will_timer, timer:seconds(I), Channel)
ensure_timer(will_message, timer:seconds(I), Channel)
end.
will_delay_interval(WillMsg) ->

View File

@ -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,12 @@
set_chan_stats/2
]).
-export([get_chann_conn_mod/2]).
-export([
open_session/3,
discard_session/1,
discard_session/2,
takeover_session/1,
takeover_session/2,
takeover_session_begin/1,
takeover_session_end/1,
kick_session/1,
kick_session/2
]).
@ -63,15 +59,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 +91,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
]).
@ -118,6 +117,8 @@
_Stats :: emqx_types:stats()
}.
-type takeover_state() :: {_ConnMod :: module(), _ChanPid :: pid()}.
-define(CHAN_STATS, [
{?CHAN_TAB, 'channels.count', 'channels.max'},
{?CHAN_TAB, 'sessions.count', 'sessions.max'},
@ -259,98 +260,71 @@ set_chan_stats(ClientId, ChanPid, Stats) ->
%% @doc Open a session.
-spec open_session(boolean(), emqx_types:clientinfo(), emqx_types:conninfo()) ->
{ok, #{
session := emqx_session:session(),
session := emqx_session:t(),
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 ->
create_register_session(ClientInfo, ConnInfo, Self)
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, Session} ->
ok = register_channel(ClientId, Self, ConnInfo),
{ok, #{session => Session, present => false}}
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.
-spec takeover_session_begin(emqx_types:clientid()) ->
{ok, emqx_session_mem:session(), takeover_state()} | none.
takeover_session_begin(ClientId) ->
takeover_session_begin(ClientId, pick_channel(ClientId)).
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_session_begin(ClientId, ChanPid) when is_pid(ChanPid) ->
case takeover_session(ClientId, ChanPid) of
{living, ConnMod, Session} ->
{ok, Session, {ConnMod, ChanPid}};
none ->
none
end;
takeover_session_begin(_ClientId, undefined) ->
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)
}.
%% @doc Conclude the session takeover process.
-spec takeover_session_end(takeover_state()) ->
{ok, _ReplayContext} | {error, _Reason}.
takeover_session_end({ConnMod, ChanPid}) ->
case wrap_rpc(emqx_cm_proto_v2:takeover_finish(ConnMod, ChanPid)) of
{ok, Pendings} ->
{ok, Pendings};
{error, _} = Error ->
Error
end.
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 +334,7 @@ takeover_session(ClientId) ->
end,
StalePids
),
takeover_session(ClientId, ChanPid)
ChanPid
end.
takeover_finish(ConnMod, ChanPid) ->
@ -370,9 +344,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 +355,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.
@ -415,7 +390,7 @@ discard_session(ClientId) when is_binary(ClientId) ->
%% benefits nobody.
-spec request_stepdown(Action, module(), pid()) ->
ok
| {ok, emqx_session:session() | list(emqx_types:deliver())}
| {ok, emqx_session:t() | _ReplayContext}
| {error, term()}
when
Action :: kick | discard | {takeover, 'begin'} | {takeover, 'end'}.
@ -488,9 +463,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 +701,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 +717,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).

View File

@ -28,6 +28,7 @@
update/3,
resize/2,
delete/2,
fold/3,
values/1,
to_list/1,
to_list/2,
@ -77,6 +78,18 @@ delete(Key, ?INFLIGHT(MaxSize, Tree)) ->
update(Key, Val, ?INFLIGHT(MaxSize, Tree)) ->
?INFLIGHT(MaxSize, gb_trees:update(Key, Val, Tree)).
-spec fold(fun((key(), Val :: term(), Acc) -> Acc), Acc, inflight()) -> Acc.
fold(FoldFun, AccIn, ?INFLIGHT(Tree)) ->
fold_iterator(FoldFun, AccIn, gb_trees:iterator(Tree)).
fold_iterator(FoldFun, Acc, It) ->
case gb_trees:next(It) of
{Key, Val, ItNext} ->
fold_iterator(FoldFun, FoldFun(Key, Val, Acc), ItNext);
none ->
Acc
end.
-spec resize(integer(), inflight()) -> inflight().
resize(MaxSize, ?INFLIGHT(Tree)) ->
?INFLIGHT(MaxSize, Tree).

View File

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

View File

@ -19,18 +19,40 @@
-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
create/3,
open/2,
destroy/1
]).
-export([
serialize_message/1,
deserialize_message/1
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,
replay/3,
% handle_timeout/3,
disconnect/1,
terminate/2
]).
%% RPC
@ -49,157 +71,353 @@
-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
).
-type id() :: emqx_ds:session_id().
-type iterator() :: emqx_ds:iterator().
-type session() :: #{
%% Client ID
id := id(),
%% When the session was created
created_at := timestamp(),
%% When the session should expire
expires_at := timestamp() | never,
%% Clients Subscriptions.
iterators := #{topic() => iterator()},
%%
props := map()
}.
-type timestamp() :: emqx_utils_calendar:epoch_millisecond().
-type topic() :: emqx_types:topic().
-type clientinfo() :: emqx_types:clientinfo().
-type conninfo() :: emqx_session: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) ->
% TODO: expiration
ensure_session(ClientID, Conf).
%%
-spec open(clientinfo(), conninfo()) ->
{_IsPresent :: true, session(), []} | false.
open(#{clientid := ClientID}, _ConnInfo) ->
%% NOTE
%% The fact that we need to concern about discarding all live channels here
%% is essentially a consequence of the in-memory session design, where we
%% have disconnected channels holding onto session state. Ideally, we should
%% somehow isolate those idling not-yet-expired sessions into a separate process
%% space, and move this call back into `emqx_cm` where it belongs.
ok = emqx_cm:discard_session(ClientID),
case open_session(ClientID) of
Session = #{} ->
{true, Session, []};
false ->
false
end.
-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
).
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_message(Msg)
).
has_subscribers(#message{topic = Topic}) ->
emqx_persistent_session_ds_router:has_any_route(Topic).
ensure_session(ClientID, Conf) ->
{ok, Session, #{}} = emqx_ds:session_ensure_new(ClientID, Conf),
Session#{iterators => #{}}.
open_session(ClientID) ->
?WHEN_ENABLED(emqx_ds:session_open(ClientID)).
case emqx_ds:session_open(ClientID) of
{ok, Session, Iterators} ->
Session#{iterators => prep_iterators(Iterators)};
false ->
false
end.
-spec add_subscription(emqx_types:topic(), emqx_ds:session_id()) ->
{ok, emqx_ds:iterator_id(), IsNew :: boolean()} | {skipped, disabled}.
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
prep_iterators(Iterators) ->
maps:fold(
fun(Topic, Iterator, Acc) -> Acc#{emqx_topic:join(Topic) => Iterator} end,
#{},
Iterators
).
-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) ->
?tp(persistent_session_ds_will_open_iterators, #{
iterator_id => IteratorID,
start_time => StartMS
}),
-spec destroy(session() | clientinfo()) -> ok.
destroy(#{id := ClientID}) ->
destroy_session(ClientID);
destroy(#{clientid := ClientID}) ->
destroy_session(ClientID).
destroy_session(ClientID) ->
_ = ensure_all_iterators_closed(ClientID),
emqx_ds:session_drop(ClientID).
%%--------------------------------------------------------------------
%% Info, Stats
%%--------------------------------------------------------------------
info(Keys, Session) when is_list(Keys) ->
[{Key, info(Key, Session)} || Key <- Keys];
info(id, #{id := ClientID}) ->
ClientID;
info(clientid, #{id := ClientID}) ->
ClientID;
info(created_at, #{created_at := CreatedAt}) ->
CreatedAt;
info(is_persistent, #{}) ->
true;
info(subscriptions, #{iterators := Iters}) ->
maps:map(fun(_, #{props := SubOpts}) -> SubOpts end, Iters);
info(subscriptions_cnt, #{iterators := Iters}) ->
maps:size(Iters);
info(subscriptions_max, #{props := Conf}) ->
maps:get(max_subscriptions, Conf);
info(upgrade_qos, #{props := 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, #{props := 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, #{}) ->
_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, #{props := Conf}) ->
maps:get(max_awaiting_rel, Conf);
info(await_rel_timeout, #{props := 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(topic(), emqx_types:subopts(), session()) ->
{ok, session()} | {error, emqx_types:reason_code()}.
subscribe(
TopicFilter,
SubOpts,
Session = #{id := ID, iterators := Iters}
) when is_map_key(TopicFilter, Iters) ->
Iterator = maps:get(TopicFilter, Iters),
NIterator = update_subscription(TopicFilter, Iterator, SubOpts, ID),
{ok, Session#{iterators := Iters#{TopicFilter => NIterator}}};
subscribe(
TopicFilter,
SubOpts,
Session = #{id := ID, iterators := Iters}
) ->
% TODO: max_subscriptions
Iterator = add_subscription(TopicFilter, SubOpts, ID),
{ok, Session#{iterators := Iters#{TopicFilter => Iterator}}}.
-spec unsubscribe(topic(), session()) ->
{ok, session(), emqx_types:subopts()} | {error, emqx_types:reason_code()}.
unsubscribe(
TopicFilter,
Session = #{id := ID, iterators := Iters}
) when is_map_key(TopicFilter, Iters) ->
Iterator = maps:get(TopicFilter, Iters),
SubOpts = maps:get(props, Iterator),
ok = del_subscription(TopicFilter, Iterator, ID),
{ok, Session#{iterators := maps:remove(TopicFilter, Iters)}, SubOpts};
unsubscribe(
_TopicFilter,
_Session = #{}
) ->
{error, ?RC_NO_SUBSCRIPTION_EXISTED}.
-spec get_subscription(topic(), session()) ->
emqx_types:subopts() | undefined.
get_subscription(TopicFilter, #{iterators := Iters}) ->
case maps:get(TopicFilter, Iters, undefined) of
Iterator = #{} ->
maps:get(props, Iterator);
undefined ->
undefined
end.
%%--------------------------------------------------------------------
%% 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 = #{}) ->
% 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 = #{}) ->
% 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 = #{}) ->
% 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 = #{}) ->
% TODO: stub
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND}.
%%--------------------------------------------------------------------
-spec deliver(clientinfo(), [emqx_types:deliver()], session()) ->
no_return().
deliver(_ClientInfo, _Delivers, _Session = #{}) ->
% TODO: ensure it's unreachable somehow
error(unexpected).
-spec replay(clientinfo(), [], session()) ->
{ok, replies(), session()}.
replay(_ClientInfo, [], Session = #{}) ->
{ok, [], Session}.
%%--------------------------------------------------------------------
-spec disconnect(session()) -> {shutdown, session()}.
disconnect(Session = #{}) ->
{shutdown, Session}.
-spec terminate(Reason :: term(), session()) -> ok.
terminate(_Reason, _Session = #{}) ->
% TODO: close iterators
ok.
%%--------------------------------------------------------------------
-spec add_subscription(topic(), emqx_types:subopts(), id()) ->
emqx_ds:iterator().
add_subscription(TopicFilterBin, SubOpts, DSSessionID) ->
% 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, Iterator, IsNew} = emqx_ds:session_add_iterator(
DSSessionID, TopicFilter, SubOpts
),
Ctx = #{iterator => Iterator, 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, Iterator)
),
Iterator.
-spec update_subscription(topic(), iterator(), emqx_types:subopts(), id()) ->
iterator().
update_subscription(TopicFilterBin, Iterator, SubOpts, DSSessionID) ->
TopicFilter = emqx_topic:words(TopicFilterBin),
{ok, NIterator, false} = emqx_ds:session_add_iterator(
DSSessionID, TopicFilter, SubOpts
),
ok = ?tp(persistent_session_ds_iterator_updated, #{iterator => Iterator}),
NIterator.
-spec open_iterator_on_all_shards(emqx_types:words(), emqx_ds:iterator()) -> ok.
open_iterator_on_all_shards(TopicFilter, Iterator) ->
?tp(persistent_session_ds_will_open_iterators, #{iterator => Iterator}),
%% Note: currently, shards map 1:1 to nodes, but this will change in the future.
Nodes = emqx:running_nodes(),
Results = emqx_persistent_session_ds_proto_v1:open_iterator(
Nodes, TopicFilter, StartMS, IteratorID
Nodes,
TopicFilter,
maps:get(start_time, Iterator),
maps:get(id, Iterator)
),
%% 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.
-spec do_open_iterator(emqx_types:words(), emqx_ds:time(), emqx_ds:iterator_id()) ->
{ok, emqx_ds_storage_layer:iterator()} | {error, _Reason}.
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(topic(), iterator(), id()) ->
ok.
del_subscription(TopicFilterBin, #{id := IteratorID}, 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) ->
@ -216,7 +434,7 @@ do_ensure_iterator_closed(IteratorID) ->
ok = emqx_ds_storage_layer:discard_iterator(?DS_SHARD, IteratorID),
ok.
-spec ensure_all_iterators_closed(emqx_ds:session_id()) -> ok.
-spec ensure_all_iterators_closed(id()) -> ok.
ensure_all_iterators_closed(DSSessionID) ->
%% Note: currently, shards map 1:1 to nodes, but this will change in the future.
Nodes = emqx:running_nodes(),
@ -226,20 +444,7 @@ ensure_all_iterators_closed(DSSessionID) ->
ok.
%% RPC target.
-spec do_ensure_all_iterators_closed(emqx_ds:session_id()) -> ok.
-spec do_ensure_all_iterators_closed(id()) -> ok.
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

View File

@ -0,0 +1,92 @@
%%--------------------------------------------------------------------
%% 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 message() :: emqx_types:message().
-type event_expired() :: {expired, message()}.
-type event_dropped() :: {dropped, message(), _Reason :: atom() | #{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, 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, {dropped, Msg, #{reason := qos0_msg, logctx := Ctx}}) ->
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,
Ctx#{
msg => "dropped_qos0_msg",
payload => Msg#message.payload
},
#{topic => Msg#message.topic}
);
handle_event(ClientInfo, {dropped, Msg, #{reason := queue_full, logctx := Ctx}}) ->
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,
Ctx#{
msg => "dropped_msg_due_to_mqueue_is_full",
payload => Msg#message.payload
},
#{topic => Msg#message.topic}
);
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.

View File

@ -0,0 +1,800 @@
%%--------------------------------------------------------------------
%% 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([
create/3,
open/2,
destroy/1
]).
-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,
dedup/4
]).
%% 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 replayctx() :: [emqx_types:message()].
-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 = #{},
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 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(), conninfo()) ->
{_IsPresent :: true, session(), replayctx()} | _IsPresent :: false.
open(ClientInfo = #{clientid := ClientId}, _ConnInfo) ->
case emqx_cm:takeover_session_begin(ClientId) of
{ok, SessionRemote, TakeoverState} ->
Session = resume(ClientInfo, SessionRemote),
case emqx_cm:takeover_session_end(TakeoverState) of
{ok, Pendings} ->
clean_session(ClientInfo, Session, Pendings);
{error, _} ->
% TODO log error?
false
end;
none ->
false
end.
clean_session(ClientInfo, Session = #session{mqueue = Q}, Pendings) ->
Q1 = emqx_mqueue:filter(fun emqx_session:should_keep/1, Q),
Session1 = Session#session{mqueue = Q1},
Pendings1 = emqx_session:enrich_delivers(ClientInfo, Pendings, Session),
Pendings2 = lists:filter(fun emqx_session:should_keep/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}
) ->
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),
{ok, Results, Session#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, 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, [], 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), 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, NQ} = emqx_mqueue:in(Msg, Q),
case Dropped of
undefined ->
Session#session{mqueue = NQ};
_Msg ->
NQInfo = emqx_mqueue:info(NQ),
Reason =
case NQInfo of
#{store_qos0 := false} when QOS =:= ?QOS_0 -> qos0_msg;
_ -> queue_full
end,
_ = emqx_session_events:handle_event(
ClientInfo,
{dropped, Dropped, #{reason => Reason, logctx => #{queue => NQInfo}}}
),
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
%%--------------------------------------------------------------------
%% @doc Handle timeout events
-spec handle_timeout(clientinfo(), emqx_session:common_timer_name(), session()) ->
{ok, replies(), session()} | {ok, replies(), timeout(), session()}.
handle_timeout(ClientInfo, retry_delivery, Session) ->
retry(ClientInfo, Session);
handle_timeout(ClientInfo, expire_awaiting_rel, Session) ->
expire(ClientInfo, 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, _, Session = #session{retry_interval = Interval}) ->
{ok, lists:reverse(Acc), Interval, 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 ->
{ok, lists:reverse(Acc), Interval - max(0, Age), Session}
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),
expire_awaiting_rel(ClientInfo, Now, Session)
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}),
Session1 = Session#session{awaiting_rel = AwaitingRel1},
case maps:size(AwaitingRel1) of
0 -> {ok, [], Session1};
_ -> {ok, [], Timeout, Session1}
end.
%%--------------------------------------------------------------------
%% 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.
-spec replay(emqx_types:clientinfo(), replayctx(), session()) ->
{ok, replies(), session()}.
replay(ClientInfo, Pendings, Session) ->
%% NOTE
%% Here, `Pendings` is a list messages that were pending delivery in the remote
%% session, see `clean_session/3`. It's a replay context that gets passed back
%% here after the remote session is taken over by `open/2`. When we have a set
%% of remote deliveries and a set of local deliveries, some publishes might actually
%% be in both sets, because there's a tiny amount of time when both remote and local
%% sessions were subscribed to the same set of topics simultaneously (i.e. after
%% local session calls `resume/2` but before remote session calls `takeover/1`
%% through `emqx_channel:handle_call({takeover, 'end'}, Channel)`).
%% We basically need to:
%% 1. Combine and deduplicate remote and local pending messages, so that no message
%% is delivered twice.
%% 2. Replay deliveries of the inflight messages, this time to the new channel.
%% 3. Deliver the combined pending messages, following the same logic as `deliver/3`.
PendingsAll = dedup(ClientInfo, Pendings, emqx_utils:drain_deliver(), Session),
{ok, PubsResendQueued, Session1} = replay(ClientInfo, Session),
{ok, PubsPending, Session2} = deliver(ClientInfo, PendingsAll, Session1),
{ok, append(PubsResendQueued, PubsPending), Session2}.
-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), Session1}.
-spec dedup(clientinfo(), [emqx_types:message()], [emqx_types:deliver()], session()) ->
[emqx_types:message()].
dedup(ClientInfo, Pendings, DeliversLocal, Session) ->
PendingsLocal1 = emqx_session:enrich_delivers(ClientInfo, DeliversLocal, Session),
PendingsLocal2 = lists:filter(
fun(Msg) -> not lists:keymember(Msg#message.id, #message.id, Pendings) end,
PendingsLocal1
),
append(Pendings, PendingsLocal2).
append(L1, []) -> L1;
append(L1, L2) -> L1 ++ L2.
%%--------------------------------------------------------------------
-spec disconnect(session()) -> {idle, session()}.
disconnect(Session = #session{}) ->
% TODO: isolate expiry timer / timeout handling here?
{idle, 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.
%%--------------------------------------------------------------------
%% For CT tests
%%--------------------------------------------------------------------
set_field(Name, Value, Session) ->
Pos = emqx_utils:index_of(Name, record_info(fields, session)),
setelement(Pos + 1, Session, Value).

View File

@ -1,306 +0,0 @@
%%--------------------------------------------------------------------
%% 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_session_router).
-behaviour(gen_server).
-include("emqx.hrl").
-include("logger.hrl").
-include("types.hrl").
-include("persistent_session/emqx_persistent_session.hrl").
-include_lib("snabbkaffe/include/snabbkaffe.hrl").
-export([
create_init_tab/0,
create_router_tab/1,
start_link/2
]).
%% Route APIs
-export([
delete_routes/2,
do_add_route/2,
do_delete_route/2,
match_routes/1
]).
-export([
buffer/3,
pending/2,
resume_begin/2,
resume_end/2
]).
-export([print_routes/1]).
%% gen_server callbacks
-export([
init/1,
handle_call/3,
handle_cast/2,
handle_info/2,
terminate/2,
code_change/3
]).
-type dest() :: node() | {emqx_types:group(), node()}.
-define(ROUTE_RAM_TAB, emqx_session_route_ram).
-define(ROUTE_DISC_TAB, emqx_session_route_disc).
-define(SESSION_INIT_TAB, session_init_tab).
%%--------------------------------------------------------------------
%% Mnesia bootstrap
%%--------------------------------------------------------------------
create_router_tab(disc) ->
create_table(?ROUTE_DISC_TAB, disc_copies);
create_router_tab(ram) ->
create_table(?ROUTE_RAM_TAB, ram_copies).
create_table(Tab, Storage) ->
ok = mria:create_table(Tab, [
{type, bag},
{rlog_shard, ?ROUTE_SHARD},
{storage, Storage},
{record_name, route},
{attributes, record_info(fields, route)},
{storage_properties, [
{ets, [
{read_concurrency, true},
{write_concurrency, true}
]}
]}
]).
%%--------------------------------------------------------------------
%% Start a router
%%--------------------------------------------------------------------
create_init_tab() ->
emqx_utils_ets:new(?SESSION_INIT_TAB, [
public,
{read_concurrency, true},
{write_concurrency, true}
]).
-spec start_link(atom(), pos_integer()) -> startlink_ret().
start_link(Pool, Id) ->
gen_server:start_link(
{local, emqx_utils:proc_name(?MODULE, Id)},
?MODULE,
[Pool, Id],
[{hibernate_after, 1000}]
).
%%--------------------------------------------------------------------
%% Route APIs
%%--------------------------------------------------------------------
-spec do_add_route(emqx_types:topic(), dest()) -> ok | {error, term()}.
do_add_route(Topic, SessionID) when is_binary(Topic) ->
Route = #route{topic = Topic, dest = SessionID},
case lists:member(Route, lookup_routes(Topic)) of
true ->
ok;
false ->
case emqx_topic:wildcard(Topic) of
true ->
Fun = fun emqx_router_utils:insert_session_trie_route/2,
emqx_router_utils:maybe_trans(
Fun,
[route_tab(), Route],
?PERSISTENT_SESSION_SHARD
);
false ->
emqx_router_utils:insert_direct_route(route_tab(), Route)
end
end.
%% @doc Match routes
-spec match_routes(emqx_types:topic()) -> [emqx_types:route()].
match_routes(Topic) when is_binary(Topic) ->
case match_trie(Topic) of
[] -> lookup_routes(Topic);
Matched -> lists:append([lookup_routes(To) || To <- [Topic | Matched]])
end.
%% Optimize: routing table will be replicated to all router nodes.
match_trie(Topic) ->
case emqx_trie:empty_session() of
true -> [];
false -> emqx_trie:match_session(Topic)
end.
%% Async
delete_routes(SessionID, Subscriptions) ->
cast(pick(SessionID), {delete_routes, SessionID, Subscriptions}).
-spec do_delete_route(emqx_types:topic(), dest()) -> ok | {error, term()}.
do_delete_route(Topic, SessionID) ->
Route = #route{topic = Topic, dest = SessionID},
case emqx_topic:wildcard(Topic) of
true ->
Fun = fun emqx_router_utils:delete_session_trie_route/2,
emqx_router_utils:maybe_trans(Fun, [route_tab(), Route], ?PERSISTENT_SESSION_SHARD);
false ->
emqx_router_utils:delete_direct_route(route_tab(), Route)
end.
%% @doc Print routes to a topic
-spec print_routes(emqx_types:topic()) -> ok.
print_routes(Topic) ->
lists:foreach(
fun(#route{topic = To, dest = SessionID}) ->
io:format("~s -> ~p~n", [To, SessionID])
end,
match_routes(Topic)
).
%%--------------------------------------------------------------------
%% Session APIs
%%--------------------------------------------------------------------
pending(SessionID, MarkerIDs) ->
call(pick(SessionID), {pending, SessionID, MarkerIDs}).
buffer(SessionID, STopic, Msg) ->
case emqx_utils_ets:lookup_value(?SESSION_INIT_TAB, SessionID) of
undefined -> ok;
Worker -> emqx_session_router_worker:buffer(Worker, STopic, Msg)
end.
-spec resume_begin(pid(), binary()) -> [{node(), emqx_guid:guid()}].
resume_begin(From, SessionID) when is_pid(From), is_binary(SessionID) ->
call(pick(SessionID), {resume_begin, From, SessionID}).
-spec resume_end(pid(), binary()) ->
{'ok', [emqx_types:message()]} | {'error', term()}.
resume_end(From, SessionID) when is_pid(From), is_binary(SessionID) ->
case emqx_utils_ets:lookup_value(?SESSION_INIT_TAB, SessionID) of
undefined ->
?tp(ps_session_not_found, #{sid => SessionID}),
{error, not_found};
Pid ->
Res = emqx_session_router_worker:resume_end(From, Pid, SessionID),
cast(pick(SessionID), {resume_end, SessionID, Pid}),
Res
end.
%%--------------------------------------------------------------------
%% Worker internals
%%--------------------------------------------------------------------
call(Router, Msg) ->
gen_server:call(Router, Msg, infinity).
cast(Router, Msg) ->
gen_server:cast(Router, Msg).
pick(#route{dest = SessionID}) ->
gproc_pool:pick_worker(session_router_pool, SessionID);
pick(SessionID) when is_binary(SessionID) ->
gproc_pool:pick_worker(session_router_pool, SessionID).
%%--------------------------------------------------------------------
%% gen_server callbacks
%%--------------------------------------------------------------------
init([Pool, Id]) ->
true = gproc_pool:connect_worker(Pool, {Pool, Id}),
{ok, #{pool => Pool, id => Id, pmon => emqx_pmon:new()}}.
handle_call({resume_begin, RemotePid, SessionID}, _From, State) ->
case init_resume_worker(RemotePid, SessionID, State) of
error ->
{reply, error, State};
{ok, Pid, State1} ->
ets:insert(?SESSION_INIT_TAB, {SessionID, Pid}),
MarkerID = emqx_persistent_session:mark_resume_begin(SessionID),
{reply, {ok, MarkerID}, State1}
end;
handle_call({pending, SessionID, MarkerIDs}, _From, State) ->
Res = emqx_persistent_session:pending_messages_in_db(SessionID, MarkerIDs),
{reply, Res, State};
handle_call(Req, _From, State) ->
?SLOG(error, #{msg => "unexpected_call", req => Req}),
{reply, ignored, State}.
handle_cast({delete_routes, SessionID, Subscriptions}, State) ->
%% TODO: Make a batch for deleting all routes.
Fun = fun(Topic, _) -> do_delete_route(Topic, SessionID) end,
ok = maps:foreach(Fun, Subscriptions),
{noreply, State};
handle_cast({resume_end, SessionID, Pid}, State) ->
case emqx_utils_ets:lookup_value(?SESSION_INIT_TAB, SessionID) of
undefined -> skip;
P when P =:= Pid -> ets:delete(?SESSION_INIT_TAB, SessionID);
P when is_pid(P) -> skip
end,
Pmon = emqx_pmon:demonitor(Pid, maps:get(pmon, State)),
_ = emqx_session_router_worker_sup:abort_worker(Pid),
{noreply, State#{pmon => Pmon}};
handle_cast(Msg, State) ->
?SLOG(error, #{msg => "unexpected_cast", cast => Msg}),
{noreply, State}.
handle_info(Info, State) ->
?SLOG(error, #{msg => "unexpected_info", info => Info}),
{noreply, State}.
terminate(_Reason, #{pool := Pool, id := Id}) ->
gproc_pool:disconnect_worker(Pool, {Pool, Id}).
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
%%--------------------------------------------------------------------
%% Resume worker. A process that buffers the persisted messages during
%% initialisation of a resuming session.
%%--------------------------------------------------------------------
init_resume_worker(RemotePid, SessionID, #{pmon := Pmon} = State) ->
case emqx_session_router_worker_sup:start_worker(SessionID, RemotePid) of
{error, What} ->
?SLOG(error, #{msg => "failed_to_start_resume_worker", reason => What}),
error;
{ok, Pid} ->
Pmon1 = emqx_pmon:monitor(Pid, Pmon),
case emqx_utils_ets:lookup_value(?SESSION_INIT_TAB, SessionID) of
undefined ->
{ok, Pid, State#{pmon => Pmon1}};
{_, OldPid} ->
Pmon2 = emqx_pmon:demonitor(OldPid, Pmon1),
_ = emqx_session_router_worker_sup:abort_worker(OldPid),
{ok, Pid, State#{pmon => Pmon2}}
end
end.
%%--------------------------------------------------------------------
%% Internal functions
%%--------------------------------------------------------------------
lookup_routes(Topic) ->
ets:lookup(route_tab(), Topic).
route_tab() ->
case emqx_persistent_session:storage_type() of
disc -> ?ROUTE_DISC_TAB;
ram -> ?ROUTE_RAM_TAB
end.

View File

@ -67,13 +67,11 @@ init([]) ->
KernelSup = child_spec(emqx_kernel_sup, supervisor),
RouterSup = child_spec(emqx_router_sup, supervisor),
BrokerSup = child_spec(emqx_broker_sup, supervisor),
SessionSup = child_spec(emqx_persistent_session_sup, supervisor),
CMSup = child_spec(emqx_cm_sup, supervisor),
SysSup = child_spec(emqx_sys_sup, supervisor),
Limiter = child_spec(emqx_limiter_sup, supervisor),
Children =
[KernelSup] ++
[SessionSup || emqx_persistent_session:is_store_enabled()] ++
[RouterSup || emqx_boot:is_enabled(broker)] ++
[BrokerSup || emqx_boot:is_enabled(broker)] ++
[CMSup || emqx_boot:is_enabled(broker)] ++

View File

@ -244,6 +244,7 @@
-type publish_result() :: [
{node(), topic(), deliver_result()}
| {share, topic(), deliver_result()}
| persisted
].
-type route() :: #route{}.
-type route_entry() :: {topic(), node()} | {topic, group()}.

View File

@ -1,562 +0,0 @@
%%--------------------------------------------------------------------
%% 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_session).
-export([
is_store_enabled/0,
init_db_backend/0,
storage_backend/0,
storage_type/0
]).
-export([
discard/2,
discard_if_present/1,
lookup/1,
persist/3,
persist_message/1,
pending/1,
pending/2,
resume/3
]).
-export([
add_subscription/3,
remove_subscription/3
]).
-export([
mark_as_delivered/2,
mark_resume_begin/1
]).
-export([
pending_messages_in_db/2,
delete_session_message/1,
gc_session_messages/1,
session_message_info/2
]).
-export([
delete_message/1,
first_message_id/0,
next_message_id/1
]).
-export_type([sess_msg_key/0]).
-include("emqx.hrl").
-include("emqx_channel.hrl").
-include("emqx_persistent_session.hrl").
-include_lib("snabbkaffe/include/snabbkaffe.hrl").
-compile({inline, [is_store_enabled/0]}).
%% NOTE: Order is significant because of traversal order of the table.
-define(MARKER, 3).
-define(DELIVERED, 2).
-define(UNDELIVERED, 1).
-define(ABANDONED, 0).
-type bin_timestamp() :: <<_:64>>.
-opaque sess_msg_key() ::
{emqx_guid:guid(), emqx_guid:guid(), emqx_types:topic(), ?UNDELIVERED | ?DELIVERED}
| {emqx_guid:guid(), emqx_guid:guid(), <<>>, ?MARKER}
| {emqx_guid:guid(), <<>>, bin_timestamp(), ?ABANDONED}.
-type gc_traverse_fun() :: fun(('delete' | 'marker' | 'abandoned', sess_msg_key()) -> 'ok').
%% EMQX configuration keys
-define(conf_storage_backend, [persistent_session_store, backend, type]).
%%--------------------------------------------------------------------
%% Init
%%--------------------------------------------------------------------
init_db_backend() ->
case is_store_enabled() of
true ->
StorageType = storage_type(),
ok = emqx_trie:create_session_trie(StorageType),
ok = emqx_session_router:create_router_tab(StorageType),
case storage_backend() of
builtin ->
emqx_persistent_session_backend_builtin:create_tables(),
persistent_term:put(?db_backend_key, emqx_persistent_session_backend_builtin)
end,
ok;
false ->
persistent_term:put(?db_backend_key, emqx_persistent_session_backend_dummy),
ok
end.
is_store_enabled() ->
emqx_config:get(?is_enabled_key).
-spec storage_backend() -> builtin.
storage_backend() ->
emqx_config:get(?conf_storage_backend).
%%--------------------------------------------------------------------
%% Session message ADT API
%%--------------------------------------------------------------------
-spec session_message_info('timestamp' | 'session_id', sess_msg_key()) -> term().
session_message_info(timestamp, {_, <<>>, <<TS:64>>, ?ABANDONED}) -> TS;
session_message_info(timestamp, {_, GUID, _, _}) -> emqx_guid:timestamp(GUID);
session_message_info(session_id, {SessionID, _, _, _}) -> SessionID.
%%--------------------------------------------------------------------
%% DB API
%%--------------------------------------------------------------------
first_message_id() ->
?db_backend:first_message_id().
next_message_id(Key) ->
?db_backend:next_message_id(Key).
delete_message(Key) ->
?db_backend:delete_message(Key).
first_session_message() ->
?db_backend:first_session_message().
next_session_message(Key) ->
?db_backend:next_session_message(Key).
delete_session_message(Key) ->
?db_backend:delete_session_message(Key).
put_session_store(#session_store{} = SS) ->
?db_backend:put_session_store(SS).
delete_session_store(ClientID) ->
?db_backend:delete_session_store(ClientID).
lookup_session_store(ClientID) ->
?db_backend:lookup_session_store(ClientID).
put_session_message({_, _, _, _} = Key) ->
?db_backend:put_session_message(#session_msg{key = Key}).
put_message(Msg) ->
?db_backend:put_message(Msg).
get_message(MsgId) ->
?db_backend:get_message(MsgId).
pending_messages_in_db(SessionID, MarkerIds) ->
?db_backend:ro_transaction(pending_messages_fun(SessionID, MarkerIds)).
%%--------------------------------------------------------------------
%% Session API
%%--------------------------------------------------------------------
%% The timestamp (TS) is the last time a client interacted with the session,
%% or when the client disconnected.
-spec persist(
emqx_types:clientinfo(),
emqx_types:conninfo(),
emqx_session:session()
) -> emqx_session:session().
persist(#{clientid := ClientID}, ConnInfo, Session) ->
case ClientID == undefined orelse not emqx_session:info(is_persistent, Session) of
true ->
Session;
false ->
SS = #session_store{
client_id = ClientID,
expiry_interval = maps:get(expiry_interval, ConnInfo),
ts = timestamp_from_conninfo(ConnInfo),
session = Session
},
case persistent_session_status(SS) of
not_persistent ->
Session;
expired ->
discard(ClientID, Session);
persistent ->
put_session_store(SS),
Session
end
end.
timestamp_from_conninfo(ConnInfo) ->
case maps:get(disconnected_at, ConnInfo, undefined) of
undefined -> erlang:system_time(millisecond);
Disconnect -> Disconnect
end.
lookup(ClientID) when is_binary(ClientID) ->
case is_store_enabled() of
false ->
none;
true ->
case lookup_session_store(ClientID) of
none ->
none;
{value, #session_store{session = S} = SS} ->
case persistent_session_status(SS) of
expired -> {expired, S};
persistent -> {persistent, S}
end
end
end.
-spec discard_if_present(binary()) -> 'ok'.
discard_if_present(ClientID) ->
case lookup(ClientID) of
none ->
ok;
{Tag, Session} when Tag =:= persistent; Tag =:= expired ->
_ = discard(ClientID, Session),
ok
end.
-spec discard(binary(), emqx_session:session()) -> emqx_session:session().
discard(ClientID, Session) ->
discard_opt(is_store_enabled(), ClientID, Session).
discard_opt(false, _ClientID, Session) ->
emqx_session:set_field(is_persistent, false, Session);
discard_opt(true, ClientID, Session) ->
delete_session_store(ClientID),
SessionID = emqx_session:info(id, Session),
put_session_message({SessionID, <<>>, <<(erlang:system_time(microsecond)):64>>, ?ABANDONED}),
Subscriptions = emqx_session:info(subscriptions, Session),
emqx_session_router:delete_routes(SessionID, Subscriptions),
emqx_session:set_field(is_persistent, false, Session).
-spec mark_resume_begin(emqx_session:session_id()) -> emqx_guid:guid().
mark_resume_begin(SessionID) ->
MarkerID = emqx_guid:gen(),
put_session_message({SessionID, MarkerID, <<>>, ?MARKER}),
MarkerID.
add_subscription(TopicFilter, SessionID, true = _IsPersistent) ->
case is_store_enabled() of
true -> emqx_session_router:do_add_route(TopicFilter, SessionID);
false -> ok
end;
add_subscription(_TopicFilter, _SessionID, false = _IsPersistent) ->
ok.
remove_subscription(TopicFilter, SessionID, true = _IsPersistent) ->
case is_store_enabled() of
true -> emqx_session_router:do_delete_route(TopicFilter, SessionID);
false -> ok
end;
remove_subscription(_TopicFilter, _SessionID, false = _IsPersistent) ->
ok.
%%--------------------------------------------------------------------
%% Resuming from DB state
%%--------------------------------------------------------------------
%% Must be called inside a emqx_cm_locker transaction.
-spec resume(emqx_types:clientinfo(), emqx_types:conninfo(), emqx_session:session()) ->
{emqx_session:session(), [emqx_types:deliver()]}.
resume(ClientInfo, ConnInfo, Session) ->
SessionID = emqx_session:info(id, Session),
?tp(ps_resuming, #{from => db, sid => SessionID}),
%% NOTE: Order is important!
%% 1. Get pending messages from DB.
?tp(ps_initial_pendings, #{sid => SessionID}),
Pendings1 = pending(SessionID),
?tp(ps_got_initial_pendings, #{
sid => SessionID,
msgs => Pendings1
}),
%% 2. Enqueue messages to mimic that the process was alive
%% when the messages were delivered.
?tp(ps_persist_pendings, #{sid => SessionID}),
Session1 = emqx_session:enqueue(ClientInfo, Pendings1, Session),
Session2 = persist(ClientInfo, ConnInfo, Session1),
mark_as_delivered(SessionID, Pendings1),
?tp(ps_persist_pendings_msgs, #{
msgs => Pendings1,
sid => SessionID
}),
%% 3. Notify writers that we are resuming.
%% They will buffer new messages.
?tp(ps_notify_writers, #{sid => SessionID}),
Nodes = mria:running_nodes(),
NodeMarkers = resume_begin(Nodes, SessionID),
?tp(ps_node_markers, #{sid => SessionID, markers => NodeMarkers}),
%% 4. Subscribe to topics.
?tp(ps_resume_session, #{sid => SessionID}),
ok = emqx_session:resume(ClientInfo, Session2),
%% 5. Get pending messages from DB until we find all markers.
?tp(ps_marker_pendings, #{sid => SessionID}),
MarkerIDs = [Marker || {_, Marker} <- NodeMarkers],
Pendings2 = pending(SessionID, MarkerIDs),
?tp(ps_marker_pendings_msgs, #{
sid => SessionID,
msgs => Pendings2
}),
%% 6. Get pending messages from writers.
?tp(ps_resume_end, #{sid => SessionID}),
WriterPendings = resume_end(Nodes, SessionID),
?tp(ps_writer_pendings, #{
msgs => WriterPendings,
sid => SessionID
}),
%% 7. Drain the inbox and usort the messages
%% with the pending messages. (Should be done by caller.)
{Session2, Pendings2 ++ WriterPendings}.
resume_begin(Nodes, SessionID) ->
Res = emqx_persistent_session_proto_v1:resume_begin(Nodes, self(), SessionID),
[{Node, Marker} || {{ok, {ok, Marker}}, Node} <- lists:zip(Res, Nodes)].
resume_end(Nodes, SessionID) ->
Res = emqx_persistent_session_proto_v1:resume_end(Nodes, self(), SessionID),
?tp(ps_erpc_multical_result, #{res => Res, sid => SessionID}),
%% TODO: Should handle the errors
[
{deliver, STopic, M}
|| {ok, {ok, Messages}} <- Res,
{{M, STopic}} <- Messages
].
%%--------------------------------------------------------------------
%% Messages API
%%--------------------------------------------------------------------
persist_message(Msg) ->
case is_store_enabled() of
true -> do_persist_message(Msg);
false -> ok
end.
do_persist_message(Msg) ->
case emqx_message:get_flag(dup, Msg) orelse emqx_message:is_sys(Msg) of
true ->
ok;
false ->
case emqx_session_router:match_routes(emqx_message:topic(Msg)) of
[] ->
ok;
Routes ->
put_message(Msg),
MsgId = emqx_message:id(Msg),
persist_message_routes(Routes, MsgId, Msg)
end
end.
persist_message_routes([#route{dest = SessionID, topic = STopic} | Left], MsgId, Msg) ->
?tp(ps_persist_msg, #{sid => SessionID, payload => emqx_message:payload(Msg)}),
put_session_message({SessionID, MsgId, STopic, ?UNDELIVERED}),
emqx_session_router:buffer(SessionID, STopic, Msg),
persist_message_routes(Left, MsgId, Msg);
persist_message_routes([], _MsgId, _Msg) ->
ok.
mark_as_delivered(SessionID, List) ->
case is_store_enabled() of
true -> do_mark_as_delivered(SessionID, List);
false -> ok
end.
do_mark_as_delivered(SessionID, [{deliver, STopic, Msg} | Left]) ->
MsgID = emqx_message:id(Msg),
case next_session_message({SessionID, MsgID, STopic, ?ABANDONED}) of
{SessionID, MsgID, STopic, ?UNDELIVERED} = Key ->
%% We can safely delete this entry
%% instead of marking it as delivered.
delete_session_message(Key);
_ ->
put_session_message({SessionID, MsgID, STopic, ?DELIVERED})
end,
do_mark_as_delivered(SessionID, Left);
do_mark_as_delivered(_SessionID, []) ->
ok.
-spec pending(emqx_session:session_id()) ->
[{emqx_types:message(), STopic :: binary()}].
pending(SessionID) ->
pending_messages_in_db(SessionID, []).
-spec pending(emqx_session:session_id(), MarkerIDs :: [emqx_guid:guid()]) ->
[{emqx_types:message(), STopic :: binary()}].
pending(SessionID, MarkerIds) ->
%% TODO: Handle lost MarkerIDs
case emqx_session_router:pending(SessionID, MarkerIds) of
incomplete ->
timer:sleep(10),
pending(SessionID, MarkerIds);
Delivers ->
Delivers
end.
%%--------------------------------------------------------------------
%% Session internal functions
%%--------------------------------------------------------------------
%% @private [MQTT-3.1.2-23]
persistent_session_status(#session_store{expiry_interval = 0}) ->
not_persistent;
persistent_session_status(#session_store{expiry_interval = ?EXPIRE_INTERVAL_INFINITE}) ->
persistent;
persistent_session_status(#session_store{expiry_interval = E, ts = TS}) ->
case E + TS > erlang:system_time(millisecond) of
true -> persistent;
false -> expired
end.
%%--------------------------------------------------------------------
%% Pending messages internal functions
%%--------------------------------------------------------------------
pending_messages_fun(SessionID, MarkerIds) ->
fun() ->
case pending_messages({SessionID, <<>>, <<>>, ?DELIVERED}, [], MarkerIds) of
{Pending, []} -> read_pending_msgs(Pending, []);
{_Pending, [_ | _]} -> incomplete
end
end.
read_pending_msgs([{MsgId, STopic} | Left], Acc) ->
Acc1 =
try
[{deliver, STopic, get_message(MsgId)} | Acc]
catch
error:{msg_not_found, _} ->
HighwaterMark =
erlang:system_time(microsecond) -
emqx_config:get(?msg_retain) * 1000,
case emqx_guid:timestamp(MsgId) < HighwaterMark of
%% Probably cleaned by GC
true -> Acc;
false -> error({msg_not_found, MsgId})
end
end,
read_pending_msgs(Left, Acc1);
read_pending_msgs([], Acc) ->
lists:reverse(Acc).
%% The keys are ordered by
%% {session_id(), <<>>, bin_timestamp(), ?ABANDONED} For abandoned sessions (clean started or expired).
%% {session_id(), emqx_guid:guid(), STopic :: binary(), ?DELIVERED | ?UNDELIVERED | ?MARKER}
%% where
%% <<>> < emqx_guid:guid()
%% <<>> < bin_timestamp()
%% emqx_guid:guid() is ordered in ts() and by node()
%% ?ABANDONED < ?UNDELIVERED < ?DELIVERED < ?MARKER
%%
%% We traverse the table until we reach another session.
%% TODO: Garbage collect the delivered messages.
pending_messages({SessionID, PrevMsgId, PrevSTopic, PrevTag} = PrevKey, Acc, MarkerIds) ->
case next_session_message(PrevKey) of
{S, <<>>, _TS, ?ABANDONED} when S =:= SessionID ->
{[], []};
{S, MsgId, <<>>, ?MARKER} = Key when S =:= SessionID ->
MarkerIds1 = MarkerIds -- [MsgId],
case PrevTag =:= ?UNDELIVERED of
false -> pending_messages(Key, Acc, MarkerIds1);
true -> pending_messages(Key, [{PrevMsgId, PrevSTopic} | Acc], MarkerIds1)
end;
{S, MsgId, STopic, ?DELIVERED} = Key when
S =:= SessionID,
MsgId =:= PrevMsgId,
STopic =:= PrevSTopic
->
pending_messages(Key, Acc, MarkerIds);
{S, _MsgId, _STopic, _Tag} = Key when S =:= SessionID ->
case PrevTag =:= ?UNDELIVERED of
false -> pending_messages(Key, Acc, MarkerIds);
true -> pending_messages(Key, [{PrevMsgId, PrevSTopic} | Acc], MarkerIds)
end;
%% Next session_id or '$end_of_table'
_What ->
case PrevTag =:= ?UNDELIVERED of
false -> {lists:reverse(Acc), MarkerIds};
true -> {lists:reverse([{PrevMsgId, PrevSTopic} | Acc]), MarkerIds}
end
end.
%%--------------------------------------------------------------------
%% Garbage collection
%%--------------------------------------------------------------------
-spec gc_session_messages(gc_traverse_fun()) -> 'ok'.
gc_session_messages(Fun) ->
gc_traverse(first_session_message(), <<>>, false, Fun).
gc_traverse('$end_of_table', _SessionID, _Abandoned, _Fun) ->
ok;
gc_traverse({S, <<>>, _TS, ?ABANDONED} = Key, _SessionID, _Abandoned, Fun) ->
%% Only report the abandoned session if it has no messages.
%% We want to keep the abandoned marker to last to make the GC reentrant.
case next_session_message(Key) of
'$end_of_table' = NextKey ->
ok = Fun(abandoned, Key),
gc_traverse(NextKey, S, true, Fun);
{S2, _, _, _} = NextKey when S =:= S2 ->
gc_traverse(NextKey, S, true, Fun);
{_, _, _, _} = NextKey ->
ok = Fun(abandoned, Key),
gc_traverse(NextKey, S, true, Fun)
end;
gc_traverse({S, _MsgID, <<>>, ?MARKER} = Key, SessionID, Abandoned, Fun) ->
ok = Fun(marker, Key),
NewAbandoned = S =:= SessionID andalso Abandoned,
gc_traverse(next_session_message(Key), S, NewAbandoned, Fun);
gc_traverse({S, _MsgID, _STopic, _Tag} = Key, SessionID, Abandoned, Fun) when
Abandoned andalso
S =:= SessionID
->
%% Delete all messages from an abandoned session.
ok = Fun(delete, Key),
gc_traverse(next_session_message(Key), S, Abandoned, Fun);
gc_traverse({S, MsgID, STopic, ?UNDELIVERED} = Key, SessionID, Abandoned, Fun) ->
case next_session_message(Key) of
{S1, M, ST, ?DELIVERED} = NextKey when
S1 =:= S andalso
MsgID =:= M andalso
STopic =:= ST
->
%% We have both markers for the same message/topic so it is safe to delete both.
ok = Fun(delete, Key),
ok = Fun(delete, NextKey),
gc_traverse(next_session_message(NextKey), S, Abandoned, Fun);
NextKey ->
%% Something else is here, so let's just loop.
NewAbandoned = S =:= SessionID andalso Abandoned,
gc_traverse(NextKey, SessionID, NewAbandoned, Fun)
end;
gc_traverse({S, _MsgID, _STopic, ?DELIVERED} = Key, SessionID, Abandoned, Fun) ->
%% We have a message that is marked as ?DELIVERED, but the ?UNDELIVERED is missing.
NewAbandoned = S =:= SessionID andalso Abandoned,
gc_traverse(next_session_message(Key), S, NewAbandoned, Fun).
-spec storage_type() -> ram | disc.
storage_type() ->
case emqx_config:get(?on_disc_key) of
true -> disc;
false -> ram
end.

View File

@ -1,41 +0,0 @@
%%--------------------------------------------------------------------
%% 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.
%%--------------------------------------------------------------------
-define(PERSISTENT_SESSION_SHARD, emqx_persistent_session_shard).
-record(session_store, {
client_id :: binary(),
expiry_interval :: non_neg_integer(),
ts :: non_neg_integer(),
session :: emqx_session:session()
}).
-record(session_msg, {
key :: emqx_persistent_session:sess_msg_key(),
val = [] :: []
}).
-define(cfg_root, persistent_session_store).
-define(db_backend_key, [?cfg_root, db_backend]).
-define(is_enabled_key, [?cfg_root, enabled]).
-define(msg_retain, [?cfg_root, max_retain_undelivered]).
-define(on_disc_key, [?cfg_root, on_disc]).
-define(SESSION_STORE, emqx_session_store).
-define(SESS_MSG_TAB, emqx_session_msg).
-define(MSG_TAB, emqx_persistent_msg).
-define(db_backend, (persistent_term:get(?db_backend_key))).

View File

@ -1,157 +0,0 @@
%%--------------------------------------------------------------------
%% 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_session_backend_builtin).
-include("emqx.hrl").
-include_lib("typerefl/include/types.hrl").
-include("emqx_persistent_session.hrl").
-export([
create_tables/0,
first_message_id/0,
next_message_id/1,
delete_message/1,
first_session_message/0,
next_session_message/1,
delete_session_message/1,
put_session_store/1,
delete_session_store/1,
lookup_session_store/1,
put_session_message/1,
put_message/1,
get_message/1,
ro_transaction/1
]).
-type mria_table_type() :: ram_copies | disc_copies | rocksdb_copies.
-define(IS_ETS(BACKEND), (BACKEND =:= ram_copies orelse BACKEND =:= disc_copies)).
create_tables() ->
SessStoreBackend = table_type(session),
ok = mria:create_table(?SESSION_STORE, [
{type, set},
{rlog_shard, ?PERSISTENT_SESSION_SHARD},
{storage, SessStoreBackend},
{record_name, session_store},
{attributes, record_info(fields, session_store)},
{storage_properties, storage_properties(?SESSION_STORE, SessStoreBackend)}
]),
SessMsgBackend = table_type(session_messages),
ok = mria:create_table(?SESS_MSG_TAB, [
{type, ordered_set},
{rlog_shard, ?PERSISTENT_SESSION_SHARD},
{storage, SessMsgBackend},
{record_name, session_msg},
{attributes, record_info(fields, session_msg)},
{storage_properties, storage_properties(?SESS_MSG_TAB, SessMsgBackend)}
]),
MsgBackend = table_type(messages),
ok = mria:create_table(?MSG_TAB, [
{type, ordered_set},
{rlog_shard, ?PERSISTENT_SESSION_SHARD},
{storage, MsgBackend},
{record_name, message},
{attributes, record_info(fields, message)},
{storage_properties, storage_properties(?MSG_TAB, MsgBackend)}
]).
first_session_message() ->
mnesia:dirty_first(?SESS_MSG_TAB).
next_session_message(Key) ->
mnesia:dirty_next(?SESS_MSG_TAB, Key).
first_message_id() ->
mnesia:dirty_first(?MSG_TAB).
next_message_id(Key) ->
mnesia:dirty_next(?MSG_TAB, Key).
delete_message(Key) ->
mria:dirty_delete(?MSG_TAB, Key).
delete_session_message(Key) ->
mria:dirty_delete(?SESS_MSG_TAB, Key).
put_session_store(SS) ->
mria:dirty_write(?SESSION_STORE, SS).
delete_session_store(ClientID) ->
mria:dirty_delete(?SESSION_STORE, ClientID).
lookup_session_store(ClientID) ->
case mnesia:dirty_read(?SESSION_STORE, ClientID) of
[] -> none;
[SS] -> {value, SS}
end.
put_session_message(SessMsg) ->
mria:dirty_write(?SESS_MSG_TAB, SessMsg).
put_message(Msg) ->
mria:dirty_write(?MSG_TAB, Msg).
get_message(MsgId) ->
case mnesia:read(?MSG_TAB, MsgId) of
[] -> error({msg_not_found, MsgId});
[Msg] -> Msg
end.
ro_transaction(Fun) ->
{atomic, Res} = mria:ro_transaction(?PERSISTENT_SESSION_SHARD, Fun),
Res.
-spec storage_properties(?SESSION_STORE | ?SESS_MSG_TAB | ?MSG_TAB, mria_table_type()) -> term().
storage_properties(?SESSION_STORE, Backend) when ?IS_ETS(Backend) ->
[{ets, [{read_concurrency, true}]}];
storage_properties(_, Backend) when ?IS_ETS(Backend) ->
[
{ets, [
{read_concurrency, true},
{write_concurrency, true}
]}
];
storage_properties(_, _) ->
[].
%% Dialyzer sees the compiled literal in
%% `mria:rocksdb_backend_available/0' and complains about the
%% complementar match arm...
-dialyzer({no_match, table_type/1}).
-spec table_type(atom()) -> mria_table_type().
table_type(Table) ->
DiscPersistence = emqx_config:get([?cfg_root, on_disc]),
RamCache = get_overlayed(Table, ram_cache),
RocksDBAvailable = mria:rocksdb_backend_available(),
case {DiscPersistence, RamCache, RocksDBAvailable} of
{true, true, _} ->
disc_copies;
{true, false, true} ->
rocksdb_copies;
{true, false, false} ->
disc_copies;
{false, _, _} ->
ram_copies
end.
-spec get_overlayed(atom(), on_disc | ram_cache) -> boolean().
get_overlayed(Table, Suffix) ->
Default = emqx_config:get([?cfg_root, Suffix]),
emqx_config:get([?cfg_root, backend, Table, Suffix], Default).

View File

@ -1,76 +0,0 @@
%%--------------------------------------------------------------------
%% 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_session_backend_dummy).
-include("emqx_persistent_session.hrl").
-export([
first_message_id/0,
next_message_id/1,
delete_message/1,
first_session_message/0,
next_session_message/1,
delete_session_message/1,
put_session_store/1,
delete_session_store/1,
lookup_session_store/1,
put_session_message/1,
put_message/1,
get_message/1,
ro_transaction/1
]).
first_message_id() ->
'$end_of_table'.
next_message_id(_) ->
'$end_of_table'.
-spec delete_message(binary()) -> no_return().
delete_message(_Key) ->
error(should_not_be_called).
first_session_message() ->
'$end_of_table'.
next_session_message(_Key) ->
'$end_of_table'.
delete_session_message(_Key) ->
ok.
put_session_store(#session_store{}) ->
ok.
delete_session_store(_ClientID) ->
ok.
lookup_session_store(_ClientID) ->
none.
put_session_message({_, _, _, _}) ->
ok.
put_message(_Msg) ->
ok.
-spec get_message(binary()) -> no_return().
get_message(_MsgId) ->
error(should_not_be_called).
ro_transaction(Fun) ->
Fun().

View File

@ -1,163 +0,0 @@
%%--------------------------------------------------------------------
%% 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_session_gc).
-behaviour(gen_server).
-include("emqx_persistent_session.hrl").
%% API
-export([start_link/0]).
%% gen_server callbacks
-export([
init/1,
handle_call/3,
handle_cast/2,
handle_info/2,
terminate/2
]).
-ifdef(TEST).
-export([
session_gc_worker/2,
message_gc_worker/0
]).
-endif.
-define(SERVER, ?MODULE).
%% TODO: Maybe these should be configurable?
-define(MARKER_GRACE_PERIOD, 60000000).
-define(ABANDONED_GRACE_PERIOD, 300000000).
%%--------------------------------------------------------------------
%% API
%%--------------------------------------------------------------------
start_link() ->
gen_server:start_link({local, ?SERVER}, ?MODULE, [], []).
%%--------------------------------------------------------------------
%% gen_server callbacks
%%--------------------------------------------------------------------
init([]) ->
process_flag(trap_exit, true),
mria_rlog:ensure_shard(?PERSISTENT_SESSION_SHARD),
{ok, start_message_gc_timer(start_session_gc_timer(#{}))}.
handle_call(_Request, _From, State) ->
Reply = ok,
{reply, Reply, State}.
handle_cast(_Request, State) ->
{noreply, State}.
handle_info({timeout, Ref, session_gc_timeout}, State) ->
State1 = session_gc_timeout(Ref, State),
{noreply, State1};
handle_info({timeout, Ref, message_gc_timeout}, State) ->
State1 = message_gc_timeout(Ref, State),
{noreply, State1};
handle_info(_Info, State) ->
{noreply, State}.
terminate(_Reason, _State) ->
ok.
%%--------------------------------------------------------------------
%% Internal functions
%%--------------------------------------------------------------------
%%--------------------------------------------------------------------
%% Session messages GC
%%--------------------------------------------------------------------
start_session_gc_timer(State) ->
Interval = emqx_config:get([persistent_session_store, session_message_gc_interval]),
State#{session_gc_timer => erlang:start_timer(Interval, self(), session_gc_timeout)}.
session_gc_timeout(Ref, #{session_gc_timer := R} = State) when R =:= Ref ->
%% Prevent overlapping processes.
GCPid = maps:get(session_gc_pid, State, undefined),
case GCPid =/= undefined andalso erlang:is_process_alive(GCPid) of
true ->
start_session_gc_timer(State);
false ->
start_session_gc_timer(State#{
session_gc_pid => proc_lib:spawn_link(fun session_gc_worker/0)
})
end;
session_gc_timeout(_Ref, State) ->
State.
session_gc_worker() ->
ok = emqx_persistent_session:gc_session_messages(fun session_gc_worker/2).
session_gc_worker(delete, Key) ->
emqx_persistent_session:delete_session_message(Key);
session_gc_worker(marker, Key) ->
TS = emqx_persistent_session:session_message_info(timestamp, Key),
case TS + ?MARKER_GRACE_PERIOD < erlang:system_time(microsecond) of
true -> emqx_persistent_session:delete_session_message(Key);
false -> ok
end;
session_gc_worker(abandoned, Key) ->
TS = emqx_persistent_session:session_message_info(timestamp, Key),
case TS + ?ABANDONED_GRACE_PERIOD < erlang:system_time(microsecond) of
true -> emqx_persistent_session:delete_session_message(Key);
false -> ok
end.
%%--------------------------------------------------------------------
%% Message GC
%% --------------------------------------------------------------------
%% The message GC simply removes all messages older than the retain
%% period. A more exact GC would either involve treating the session
%% message table as root set, or some kind of reference counting.
%% We sacrifice space for simplicity at this point.
start_message_gc_timer(State) ->
Interval = emqx_config:get([persistent_session_store, session_message_gc_interval]),
State#{message_gc_timer => erlang:start_timer(Interval, self(), message_gc_timeout)}.
message_gc_timeout(Ref, #{message_gc_timer := R} = State) when R =:= Ref ->
%% Prevent overlapping processes.
GCPid = maps:get(message_gc_pid, State, undefined),
case GCPid =/= undefined andalso erlang:is_process_alive(GCPid) of
true ->
start_message_gc_timer(State);
false ->
start_message_gc_timer(State#{
message_gc_pid => proc_lib:spawn_link(fun message_gc_worker/0)
})
end;
message_gc_timeout(_Ref, State) ->
State.
message_gc_worker() ->
HighWaterMark = erlang:system_time(microsecond) - emqx_config:get(?msg_retain) * 1000,
message_gc_worker(emqx_persistent_session:first_message_id(), HighWaterMark).
message_gc_worker('$end_of_table', _HighWaterMark) ->
ok;
message_gc_worker(MsgId, HighWaterMark) ->
case emqx_guid:timestamp(MsgId) < HighWaterMark of
true ->
emqx_persistent_session:delete_message(MsgId),
message_gc_worker(emqx_persistent_session:next_message_id(MsgId), HighWaterMark);
false ->
ok
end.

View File

@ -1,69 +0,0 @@
%%--------------------------------------------------------------------
%% 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_session_sup).
-behaviour(supervisor).
-export([start_link/0]).
-export([init/1]).
start_link() ->
supervisor:start_link({local, ?MODULE}, ?MODULE, []).
init([]) ->
%% We want this supervisor to own the table for restarts
SessionTab = emqx_session_router:create_init_tab(),
%% Resume worker sup
ResumeSup = #{
id => router_worker_sup,
start => {emqx_session_router_worker_sup, start_link, [SessionTab]},
restart => permanent,
shutdown => 2000,
type => supervisor,
modules => [emqx_session_router_worker_sup]
},
SessionRouterPool = emqx_pool_sup:spec(
session_router_pool,
[
session_router_pool,
hash,
{emqx_session_router, start_link, []}
]
),
GCWorker = child_spec(emqx_persistent_session_gc, worker),
Spec = #{
strategy => one_for_all,
intensity => 0,
period => 1
},
{ok, {Spec, [ResumeSup, SessionRouterPool, GCWorker]}}.
child_spec(Mod, worker) ->
#{
id => Mod,
start => {Mod, start_link, []},
restart => permanent,
shutdown => 15000,
type => worker,
modules => [Mod]
}.

View File

@ -48,11 +48,13 @@ kickout_client(Node, ClientId) ->
lookup_client(Node, Key) ->
rpc:call(Node, emqx_cm, lookup_client, [Key]).
-spec get_chan_stats(emqx_types:clientid(), emqx_cm:chan_pid()) -> emqx_types:stats() | {badrpc, _}.
-spec get_chan_stats(emqx_types:clientid(), emqx_cm:chan_pid()) ->
emqx_types:stats() | undefined | {badrpc, _}.
get_chan_stats(ClientId, ChanPid) ->
rpc:call(node(ChanPid), emqx_cm, do_get_chan_stats, [ClientId, ChanPid], ?T_GET_INFO * 2).
-spec get_chan_info(emqx_types:clientid(), emqx_cm:chan_pid()) -> emqx_types:infos() | {badrpc, _}.
-spec get_chan_info(emqx_types:clientid(), emqx_cm:chan_pid()) ->
emqx_types:infos() | undefined | {badrpc, _}.
get_chan_info(ClientId, ChanPid) ->
rpc:call(node(ChanPid), emqx_cm, do_get_chan_info, [ClientId, ChanPid], ?T_GET_INFO * 2).

View File

@ -36,7 +36,7 @@ introduced_in() ->
-spec open_iterator(
[node()],
emqx_topic:words(),
emqx_types:words(),
emqx_ds:time(),
emqx_ds:iterator_id()
) ->

View File

@ -1,41 +0,0 @@
%%--------------------------------------------------------------------
%% 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.
%%--------------------------------------------------------------------
-module(emqx_persistent_session_proto_v1).
-behaviour(emqx_bpapi).
-export([
introduced_in/0,
resume_begin/3,
resume_end/3
]).
-include("bpapi.hrl").
-include("emqx.hrl").
introduced_in() ->
"5.0.0".
-spec resume_begin([node()], pid(), binary()) ->
emqx_rpc:erpc_multicall([{node(), emqx_guid:guid()}]).
resume_begin(Nodes, Pid, SessionID) when is_pid(Pid), is_binary(SessionID) ->
erpc:multicall(Nodes, emqx_session_router, resume_begin, [Pid, SessionID]).
-spec resume_end([node()], pid(), binary()) ->
emqx_rpc:erpc_multicall({'ok', [emqx_types:message()]} | {'error', term()}).
resume_end(Nodes, Pid, SessionID) when is_pid(Pid), is_binary(SessionID) ->
erpc:multicall(Nodes, emqx_session_router, resume_end, [Pid, SessionID]).

View File

@ -53,11 +53,13 @@
-define(IGNORED_MODULES, "emqx_rpc").
-define(FORCE_DELETED_MODULES, [
emqx_statsd,
emqx_statsd_proto_v1
emqx_statsd_proto_v1,
emqx_persistent_session_proto_v1
]).
-define(FORCE_DELETED_APIS, [
{emqx_statsd, 1},
{emqx_plugin_libs, 1}
{emqx_plugin_libs, 1},
{emqx_persistent_session, 1}
]).
%% List of known RPC backend modules:
-define(RPC_MODULES, "gen_rpc, erpc, rpc, emqx_rpc").

View File

@ -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()).
@ -768,32 +729,24 @@ t_handle_info_sock_closed(_) ->
%% Test cases for handle_timeout
%%--------------------------------------------------------------------
t_handle_timeout_emit_stats(_) ->
TRef = make_ref(),
ok = meck:expect(emqx_cm, set_chan_stats, fun(_, _) -> ok end),
Channel = emqx_channel:set_field(timers, #{stats_timer => TRef}, channel()),
{ok, _Chan} = emqx_channel:handle_timeout(TRef, {emit_stats, []}, Channel).
t_handle_timeout_keepalive(_) ->
TRef = make_ref(),
Channel = emqx_channel:set_field(timers, #{alive_timer => TRef}, channel()),
Channel = emqx_channel:set_field(timers, #{keepalive => TRef}, channel()),
{ok, _Chan} = emqx_channel:handle_timeout(make_ref(), {keepalive, 10}, Channel).
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_timer => TRef}, channel()),
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, #{await_timer => TRef}, channel()),
Channel = emqx_channel:set_field(timers, #{expire_awaiting_rel => TRef}, channel()),
{ok, _Chan} = emqx_channel:handle_timeout(TRef, expire_awaiting_rel, Channel).
t_handle_timeout_expire_session(_) ->
TRef = make_ref(),
Channel = emqx_channel:set_field(timers, #{expire_timer => TRef}, channel()),
Channel = emqx_channel:set_field(timers, #{expire_session => TRef}, channel()),
{shutdown, expired, _Chan} = emqx_channel:handle_timeout(TRef, expire_session, Channel).
t_handle_timeout_will_message(_) ->
@ -983,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
@ -1000,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},
@ -1010,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
@ -1029,8 +988,8 @@ channel(InitFields) ->
),
maps:merge(
#{
clientinfo => clientinfo(),
session => session(),
clientinfo => ClientInfo,
session => session(ClientInfo, #{}),
conn_state => connected
},
InitFields
@ -1045,6 +1004,7 @@ clientinfo(InitProps) ->
listener => {tcp, default},
protocol => mqtt,
peerhost => {127, 0, 0, 1},
sockport => 3456,
clientid => <<"clientid">>,
username => <<"username">>,
is_superuser => false,
@ -1073,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

View File

@ -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_session_begin(ClientId),
ok
end,
case Reason =:= timeout orelse Reason =:= noproc of
@ -381,30 +381,35 @@ t_discard_session_race(_) ->
t_takeover_session(_) ->
#{conninfo := ConnInfo} = ?ChanInfo,
none = emqx_cm:takeover_session(<<"clientid">>),
ClientId = <<"clientid">>,
none = emqx_cm:takeover_session_begin(ClientId),
Parent = self(),
erlang:spawn_link(fun() ->
ok = emqx_cm:register_channel(<<"clientid">>, self(), ConnInfo),
ChanPid = 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, _Pendings = [])
end
end
end),
receive
registered -> ok
end,
{living, emqx_connection, _, test} = emqx_cm:takeover_session(<<"clientid">>),
emqx_cm:unregister_channel(<<"clientid">>).
{ok, test, State = {emqx_connection, ChanPid}} = emqx_cm:takeover_session_begin(ClientId),
{ok, []} = emqx_cm:takeover_session_end(State),
emqx_cm:unregister_channel(ClientId).
t_takeover_session_process_gone(_) ->
#{conninfo := ConnInfo} = ?ChanInfo,
ClientIDTcp = <<"clientidTCP">>,
ClientIDWs = <<"clientidWs">>,
ClientIDRpc = <<"clientidRPC">>,
none = emqx_cm:takeover_session(ClientIDTcp),
none = emqx_cm:takeover_session(ClientIDWs),
none = emqx_cm:takeover_session_begin(ClientIDTcp),
none = emqx_cm:takeover_session_begin(ClientIDWs),
meck:new(emqx_connection, [passthrough, no_history]),
meck:expect(
emqx_connection,
@ -417,7 +422,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_session_begin(ClientIDTcp),
meck:expect(
emqx_connection,
call,
@ -429,7 +434,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_session_begin(ClientIDWs),
meck:expect(
emqx_connection,
call,
@ -441,7 +446,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_session_begin(ClientIDRpc),
emqx_cm:unregister_channel(ClientIDTcp),
emqx_cm:unregister_channel(ClientIDWs),
emqx_cm:unregister_channel(ClientIDRpc),

View File

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

View File

@ -340,7 +340,6 @@ default_appspec(emqx_conf, SuiteOpts) ->
node => #{
name => node(),
cookie => erlang:get_cookie(),
% FIXME
data_dir => unicode:characters_to_binary(maps:get(work_dir, SuiteOpts, "data"))
}
},

View File

@ -76,6 +76,17 @@ t_values(_) ->
?assertEqual([1, 2], emqx_inflight:values(Inflight)),
?assertEqual([{a, 1}, {b, 2}], emqx_inflight:to_list(Inflight)).
t_fold(_) ->
Inflight = maps:fold(
fun emqx_inflight:insert/3,
emqx_inflight:new(),
#{a => 1, b => 2, c => 42}
),
?assertEqual(
emqx_inflight:fold(fun(_, V, S) -> S + V end, 0, Inflight),
lists:foldl(fun({_, V}, S) -> S + V end, 0, emqx_inflight:to_list(Inflight))
).
t_is_full(_) ->
Inflight = emqx_inflight:insert(k, v, emqx_inflight:new()),
?assertNot(emqx_inflight:is_full(Inflight)),

View File

@ -186,13 +186,14 @@ t_session_subscription_iterators(Config) ->
ct:pal("publishing 2"),
Message2 = emqx_message:make(Topic, Payload2),
publish(Node1, Message2),
[_] = receive_messages(1),
% TODO: no incoming publishes at the moment
% [_] = receive_messages(1),
ct:pal("subscribing 2"),
{ok, _, [1]} = emqtt:subscribe(Client, SubTopicFilter, qos1),
ct:pal("publishing 3"),
Message3 = emqx_message:make(Topic, Payload3),
publish(Node1, Message3),
[_] = receive_messages(1),
% [_] = receive_messages(1),
ct:pal("publishing 4"),
Message4 = emqx_message:make(AnotherTopic, Payload4),
publish(Node1, Message4),
@ -272,7 +273,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.
@ -315,21 +316,13 @@ get_iterator_ids(Node, ClientId) ->
app_specs() ->
[
emqx_durable_storage,
{emqx, #{
config => #{persistent_session_store => #{ds => true}},
override_env => [{boot_modules, [broker, listeners]}]
}}
{emqx, "persistent_session_store {ds = true}"}
].
cluster() ->
Node1 = persistent_messages_SUITE1,
Spec = #{
role => core,
join_to => emqx_cth_cluster:node_name(Node1),
apps => app_specs()
},
Spec = #{role => core, apps => app_specs()},
[
{Node1, Spec},
{persistent_messages_SUITE1, Spec},
{persistent_messages_SUITE2, Spec}
].

View File

@ -19,8 +19,7 @@
-include_lib("stdlib/include/assert.hrl").
-include_lib("common_test/include/ct.hrl").
-include_lib("snabbkaffe/include/snabbkaffe.hrl").
-include_lib("../include/emqx.hrl").
-include("../src/persistent_session/emqx_persistent_session.hrl").
-include_lib("emqx/include/emqx_mqtt.hrl").
-compile(export_all).
-compile(nowarn_export_all).
@ -34,8 +33,8 @@ all() ->
% NOTE
% Tests are disabled while existing session persistence impl is being
% phased out.
% {group, persistent_store_enabled},
{group, persistent_store_disabled}
{group, persistent_store_disabled},
{group, persistent_store_ds}
].
%% A persistent session can be resumed in two ways:
@ -51,177 +50,92 @@ all() ->
groups() ->
TCs = emqx_common_test_helpers:all(?MODULE),
SnabbkaffeTCs = [TC || TC <- TCs, is_snabbkaffe_tc(TC)],
GCTests = [TC || TC <- TCs, is_gc_tc(TC)],
OtherTCs = (TCs -- SnabbkaffeTCs) -- GCTests,
TCsNonGeneric = [t_choose_impl],
[
{persistent_store_enabled, [
{group, ram_tables},
{group, disc_tables}
]},
{persistent_store_disabled, [{group, no_kill_connection_process}]},
{ram_tables, [], [
{group, no_kill_connection_process},
{group, kill_connection_process},
{group, snabbkaffe},
{group, gc_tests}
]},
{disc_tables, [], [
{group, no_kill_connection_process},
{group, kill_connection_process},
{group, snabbkaffe},
{group, gc_tests}
]},
{persistent_store_ds, [{group, no_kill_connection_process}]},
{no_kill_connection_process, [], [{group, tcp}, {group, quic}, {group, ws}]},
{kill_connection_process, [], [{group, tcp}, {group, quic}, {group, ws}]},
{snabbkaffe, [], [
{group, tcp_snabbkaffe}, {group, quic_snabbkaffe}, {group, ws_snabbkaffe}
]},
{tcp, [], OtherTCs},
{quic, [], OtherTCs},
{ws, [], OtherTCs},
{tcp_snabbkaffe, [], SnabbkaffeTCs},
{quic_snabbkaffe, [], SnabbkaffeTCs},
{ws_snabbkaffe, [], SnabbkaffeTCs},
{gc_tests, [], GCTests}
{tcp, [], TCs},
{quic, [], TCs -- TCsNonGeneric},
{ws, [], TCs -- TCsNonGeneric}
].
is_snabbkaffe_tc(TC) ->
re:run(atom_to_list(TC), "^t_snabbkaffe_") /= nomatch.
is_gc_tc(TC) ->
re:run(atom_to_list(TC), "^t_gc_") /= nomatch.
init_per_group(persistent_store_enabled, Config) ->
[{persistent_store_enabled, true} | Config];
init_per_group(Group, Config) when Group =:= ram_tables; Group =:= disc_tables ->
%% Start Apps
Reply =
case Group =:= ram_tables of
true -> ram;
false -> disc
end,
emqx_common_test_helpers:boot_modules(all),
meck:new(emqx_config, [non_strict, passthrough, no_history, no_link]),
meck:expect(emqx_config, get, fun
(?on_disc_key) -> Reply =:= disc;
(?is_enabled_key) -> true;
(Other) -> meck:passthrough([Other])
end),
emqx_common_test_helpers:start_apps([], fun set_special_confs/1),
?assertEqual(true, emqx_persistent_session:is_store_enabled()),
Config;
init_per_group(persistent_store_disabled, Config) ->
%% Start Apps
emqx_common_test_helpers:boot_modules(all),
meck:new(emqx_config, [non_strict, passthrough, no_history, no_link]),
meck:expect(emqx_config, get, fun
(?is_enabled_key) -> false;
(Other) -> meck:passthrough([Other])
end),
emqx_common_test_helpers:start_apps([], fun set_special_confs/1),
?assertEqual(false, emqx_persistent_session:is_store_enabled()),
[{persistent_store_enabled, false} | Config];
init_per_group(Group, Config) when Group == ws; Group == ws_snabbkaffe ->
[
{emqx_config, "persistent_session_store { enabled = false }"},
{persistent_store, false}
| Config
];
init_per_group(persistent_store_ds, Config) ->
[
{emqx_config, "persistent_session_store { ds = true }"},
{persistent_store, ds}
| Config
];
init_per_group(Group, Config) when Group == tcp ->
Apps = emqx_cth_suite:start(
[{emqx, ?config(emqx_config, Config)}],
#{work_dir => emqx_cth_suite:work_dir(Config)}
),
[
{port, get_listener_port(tcp, default)},
{conn_fun, connect},
{group_apps, Apps}
| Config
];
init_per_group(Group, Config) when Group == ws ->
Apps = emqx_cth_suite:start(
[{emqx, ?config(emqx_config, Config)}],
#{work_dir => emqx_cth_suite:work_dir(Config)}
),
[
{ssl, false},
{host, "localhost"},
{enable_websocket, true},
{port, 8083},
{conn_fun, ws_connect}
{port, get_listener_port(ws, default)},
{conn_fun, ws_connect},
{group_apps, Apps}
| Config
];
init_per_group(Group, Config) when Group == quic ->
Apps = emqx_cth_suite:start(
[
{emqx,
?config(emqx_config, Config) ++
"\n listeners.quic.test { enable = true }"}
],
#{work_dir => emqx_cth_suite:work_dir(Config)}
),
[
{port, get_listener_port(quic, test)},
{conn_fun, quic_connect},
{group_apps, Apps}
| Config
];
init_per_group(Group, Config) when Group == tcp; Group == tcp_snabbkaffe ->
[{port, 1883}, {conn_fun, connect} | Config];
init_per_group(Group, Config) when Group == quic; Group == quic_snabbkaffe ->
UdpPort = 1883,
emqx_common_test_helpers:ensure_quic_listener(?MODULE, UdpPort),
[{port, UdpPort}, {conn_fun, quic_connect} | Config];
init_per_group(no_kill_connection_process, Config) ->
[{kill_connection_process, false} | Config];
init_per_group(kill_connection_process, Config) ->
[{kill_connection_process, true} | Config];
init_per_group(snabbkaffe, Config) ->
[{kill_connection_process, true} | Config];
init_per_group(gc_tests, Config) ->
%% We need to make sure the system does not interfere with this test group.
lists:foreach(
fun(ClientId) ->
maybe_kill_connection_process(ClientId, [{kill_connection_process, true}])
end,
emqx_cm:all_client_ids()
),
emqx_common_test_helpers:stop_apps([]),
SessionMsgEts = gc_tests_session_store,
MsgEts = gc_tests_msg_store,
Pid = spawn(fun() ->
ets:new(SessionMsgEts, [named_table, public, ordered_set]),
ets:new(MsgEts, [named_table, public, ordered_set, {keypos, 2}]),
receive
stop -> ok
end
end),
meck:new(mnesia, [non_strict, passthrough, no_history, no_link]),
meck:expect(mnesia, dirty_first, fun
(?SESS_MSG_TAB) -> ets:first(SessionMsgEts);
(?MSG_TAB) -> ets:first(MsgEts);
(X) -> meck:passthrough([X])
end),
meck:expect(mnesia, dirty_next, fun
(?SESS_MSG_TAB, X) -> ets:next(SessionMsgEts, X);
(?MSG_TAB, X) -> ets:next(MsgEts, X);
(Tab, X) -> meck:passthrough([Tab, X])
end),
meck:expect(mnesia, dirty_delete, fun
(?MSG_TAB, X) -> ets:delete(MsgEts, X);
(Tab, X) -> meck:passthrough([Tab, X])
end),
[{store_owner, Pid}, {session_msg_store, SessionMsgEts}, {msg_store, MsgEts} | Config].
[{kill_connection_process, true} | Config].
init_per_suite(Config) ->
Config.
get_listener_port(Type, Name) ->
case emqx_config:get([listeners, Type, Name, bind]) of
{_, Port} -> Port;
Port -> Port
end.
set_special_confs(_) ->
ok.
end_per_suite(_Config) ->
emqx_common_test_helpers:ensure_mnesia_stopped(),
ok.
end_per_group(gc_tests, Config) ->
meck:unload(mnesia),
?config(store_owner, Config) ! stop,
ok;
end_per_group(Group, _Config) when
Group =:= ram_tables; Group =:= disc_tables
->
meck:unload(emqx_config),
emqx_common_test_helpers:stop_apps([]);
end_per_group(persistent_store_disabled, _Config) ->
meck:unload(emqx_config),
emqx_common_test_helpers:stop_apps([]);
end_per_group(_Group, _Config) ->
end_per_group(Group, Config) when Group == tcp; Group == ws; Group == quic ->
ok = emqx_cth_suite:stop(?config(group_apps, Config));
end_per_group(_, _Config) ->
ok.
init_per_testcase(TestCase, Config) ->
Config1 = preconfig_per_testcase(TestCase, Config),
case is_gc_tc(TestCase) of
true ->
ets:delete_all_objects(?config(msg_store, Config)),
ets:delete_all_objects(?config(session_msg_store, Config));
false ->
skip
end,
case erlang:function_exported(?MODULE, TestCase, 2) of
true -> ?MODULE:TestCase(init, Config1);
_ -> Config1
end.
end_per_testcase(TestCase, Config) ->
case is_snabbkaffe_tc(TestCase) of
true -> snabbkaffe:stop();
false -> skip
end,
case erlang:function_exported(?MODULE, TestCase, 2) of
true -> ?MODULE:TestCase('end', Config);
false -> ok
@ -307,20 +221,6 @@ wait_for_cm_unregister(ClientId, N) ->
wait_for_cm_unregister(ClientId, N - 1)
end.
snabbkaffe_sync_publish(Topic, Payloads) ->
Fun = fun(Client, Payload) ->
?check_trace(
begin
?wait_async_action(
{ok, _} = emqtt:publish(Client, Topic, Payload, 2),
#{?snk_kind := ps_persist_msg, payload := Payload}
)
end,
fun(_, _Trace) -> ok end
)
end,
do_publish(Payloads, Fun, true).
publish(Topic, Payloads) ->
publish(Topic, Payloads, false).
@ -377,7 +277,55 @@ do_publish(Payload, PublishFun, WaitForUnregister) ->
%% Test Cases
%%--------------------------------------------------------------------
t_choose_impl(Config) ->
ClientId = ?config(client_id, Config),
ConnFun = ?config(conn_fun, Config),
{ok, Client} = emqtt:start_link([
{clientid, ClientId},
{proto_ver, v5},
{properties, #{'Session-Expiry-Interval' => 30}}
| Config
]),
{ok, _} = emqtt:ConnFun(Client),
[ChanPid] = emqx_cm:lookup_channels(ClientId),
?assertEqual(
case ?config(persistent_store, Config) of
false -> emqx_session_mem;
ds -> emqx_persistent_session_ds
end,
emqx_connection:info({channel, {session, impl}}, sys:get_state(ChanPid))
).
t_connect_discards_existing_client(Config) ->
ClientId = ?config(client_id, Config),
ConnFun = ?config(conn_fun, Config),
ClientOpts = [
{clientid, ClientId},
{proto_ver, v5},
{properties, #{'Session-Expiry-Interval' => 30}}
| Config
],
{ok, Client1} = emqtt:start_link(ClientOpts),
true = unlink(Client1),
MRef = erlang:monitor(process, Client1),
{ok, _} = emqtt:ConnFun(Client1),
{ok, Client2} = emqtt:start_link(ClientOpts),
{ok, _} = emqtt:ConnFun(Client2),
receive
{'DOWN', MRef, process, Client1, Reason} ->
ok = ?assertMatch({disconnected, ?RC_SESSION_TAKEN_OVER, _}, Reason),
ok = emqtt:stop(Client2),
ok
after 1000 ->
error({client_still_connected, Client1})
end.
%% [MQTT-3.1.2-23]
t_connect_session_expiry_interval(init, Config) -> skip_ds_tc(Config);
t_connect_session_expiry_interval('end', _Config) -> ok.
t_connect_session_expiry_interval(Config) ->
ConnFun = ?config(conn_fun, Config),
Topic = ?config(topic, Config),
@ -514,20 +462,8 @@ t_persist_on_disconnect(Config) ->
?assertEqual(0, client_info(session_present, Client2)),
ok = emqtt:disconnect(Client2).
wait_for_pending(SId) ->
wait_for_pending(SId, 100).
wait_for_pending(_SId, 0) ->
error(exhausted_wait_for_pending);
wait_for_pending(SId, N) ->
case emqx_persistent_session:pending(SId) of
[] ->
timer:sleep(1),
wait_for_pending(SId, N - 1);
[_ | _] = Pending ->
Pending
end.
t_process_dies_session_expires(init, Config) -> skip_ds_tc(Config);
t_process_dies_session_expires('end', _Config) -> ok.
t_process_dies_session_expires(Config) ->
%% Emulate an error in the connect process,
%% or that the node of the process goes down.
@ -552,36 +488,8 @@ t_process_dies_session_expires(Config) ->
ok = publish(Topic, [Payload]),
SessionId =
case ?config(persistent_store_enabled, Config) of
false ->
undefined;
true ->
%% The session should not be marked as expired.
{Tag, Session} = emqx_persistent_session:lookup(ClientId),
?assertEqual(persistent, Tag),
SId = emqx_session:info(id, Session),
case ?config(kill_connection_process, Config) of
true ->
%% The session should have a pending message
?assertMatch([_], wait_for_pending(SId));
false ->
skip
end,
SId
end,
timer:sleep(1100),
%% The session should now be marked as expired.
case
(?config(kill_connection_process, Config) andalso
?config(persistent_store_enabled, Config))
of
true -> ?assertMatch({expired, _}, emqx_persistent_session:lookup(ClientId));
false -> skip
end,
{ok, Client2} = emqtt:start_link([
{proto_ver, v5},
{clientid, ClientId},
@ -592,26 +500,13 @@ t_process_dies_session_expires(Config) ->
{ok, _} = emqtt:ConnFun(Client2),
?assertEqual(0, client_info(session_present, Client2)),
case
(?config(kill_connection_process, Config) andalso
?config(persistent_store_enabled, Config))
of
true ->
%% The session should be a fresh one
{persistent, NewSession} = emqx_persistent_session:lookup(ClientId),
?assertNotEqual(SessionId, emqx_session:info(id, NewSession)),
%% The old session should now either
%% be marked as abandoned or already be garbage collected.
?assertMatch([], emqx_persistent_session:pending(SessionId));
false ->
skip
end,
%% We should not receive the pending message
?assertEqual([], receive_messages(1)),
emqtt:disconnect(Client2).
t_publish_while_client_is_gone(init, Config) -> skip_ds_tc(Config);
t_publish_while_client_is_gone('end', _Config) -> ok.
t_publish_while_client_is_gone(Config) ->
%% A persistent session should receive messages in its
%% subscription even if the process owning the session dies.
@ -654,6 +549,8 @@ t_publish_while_client_is_gone(Config) ->
ok = emqtt:disconnect(Client2).
t_clean_start_drops_subscriptions(init, Config) -> skip_ds_tc(Config);
t_clean_start_drops_subscriptions('end', _Config) -> ok.
t_clean_start_drops_subscriptions(Config) ->
%% 1. A persistent session is started and disconnected.
%% 2. While disconnected, a message is published and persisted.
@ -724,7 +621,6 @@ t_clean_start_drops_subscriptions(Config) ->
t_unsubscribe(Config) ->
ConnFun = ?config(conn_fun, Config),
Topic = ?config(topic, Config),
STopic = ?config(stopic, Config),
ClientId = ?config(client_id, Config),
{ok, Client} = emqtt:start_link([
@ -735,24 +631,13 @@ t_unsubscribe(Config) ->
]),
{ok, _} = emqtt:ConnFun(Client),
{ok, _, [2]} = emqtt:subscribe(Client, STopic, qos2),
case emqx_persistent_session:is_store_enabled() of
true ->
{persistent, Session} = emqx_persistent_session:lookup(ClientId),
SessionID = emqx_session:info(id, Session),
SessionIDs = [SId || #route{dest = SId} <- emqx_session_router:match_routes(Topic)],
?assert(lists:member(SessionID, SessionIDs)),
?assertMatch([_], [Sub || {ST, _} = Sub <- emqtt:subscriptions(Client), ST =:= STopic]),
{ok, _, _} = emqtt:unsubscribe(Client, STopic),
?assertMatch([], [Sub || {ST, _} = Sub <- emqtt:subscriptions(Client), ST =:= STopic]),
SessionIDs2 = [SId || #route{dest = SId} <- emqx_session_router:match_routes(Topic)],
?assert(not lists:member(SessionID, SessionIDs2));
false ->
?assertMatch([_], [Sub || {ST, _} = Sub <- emqtt:subscriptions(Client), ST =:= STopic]),
{ok, _, _} = emqtt:unsubscribe(Client, STopic),
?assertMatch([], [Sub || {ST, _} = Sub <- emqtt:subscriptions(Client), ST =:= STopic])
end,
?assertMatch([_], [Sub || {ST, _} = Sub <- emqtt:subscriptions(Client), ST =:= STopic]),
{ok, _, _} = emqtt:unsubscribe(Client, STopic),
?assertMatch([], [Sub || {ST, _} = Sub <- emqtt:subscriptions(Client), ST =:= STopic]),
ok = emqtt:disconnect(Client).
t_multiple_subscription_matches(init, Config) -> skip_ds_tc(Config);
t_multiple_subscription_matches('end', _Config) -> ok.
t_multiple_subscription_matches(Config) ->
ConnFun = ?config(conn_fun, Config),
Topic = ?config(topic, Config),
@ -795,514 +680,10 @@ t_multiple_subscription_matches(Config) ->
?assertEqual({ok, 2}, maps:find(qos, Msg2)),
ok = emqtt:disconnect(Client2).
t_lost_messages_because_of_gc(init, Config) ->
case
(emqx_persistent_session:is_store_enabled() andalso
?config(kill_connection_process, Config))
of
true ->
Retain = 1000,
OldRetain = emqx_config:get(?msg_retain, Retain),
emqx_config:put(?msg_retain, Retain),
[{retain, Retain}, {old_retain, OldRetain} | Config];
false ->
{skip, only_relevant_with_store_and_kill_process}
end;
t_lost_messages_because_of_gc('end', Config) ->
OldRetain = ?config(old_retain, Config),
emqx_config:put(?msg_retain, OldRetain),
ok.
t_lost_messages_because_of_gc(Config) ->
ConnFun = ?config(conn_fun, Config),
Topic = ?config(topic, Config),
STopic = ?config(stopic, Config),
ClientId = ?config(client_id, Config),
Retain = ?config(retain, Config),
Payload1 = <<"hello1">>,
Payload2 = <<"hello2">>,
{ok, Client1} = emqtt:start_link([
{clientid, ClientId},
{proto_ver, v5},
{properties, #{'Session-Expiry-Interval' => 30}}
| Config
]),
{ok, _} = emqtt:ConnFun(Client1),
{ok, _, [2]} = emqtt:subscribe(Client1, STopic, qos2),
emqtt:disconnect(Client1),
maybe_kill_connection_process(ClientId, Config),
publish(Topic, Payload1),
timer:sleep(2 * Retain),
publish(Topic, Payload2),
emqx_persistent_session_gc:message_gc_worker(),
{ok, Client2} = emqtt:start_link([
{clientid, ClientId},
{clean_start, false},
{proto_ver, v5},
{properties, #{'Session-Expiry-Interval' => 30}}
| Config
]),
{ok, _} = emqtt:ConnFun(Client2),
Msgs = receive_messages(2),
?assertMatch([_], Msgs),
?assertEqual({ok, iolist_to_binary(Payload2)}, maps:find(payload, hd(Msgs))),
emqtt:disconnect(Client2),
ok.
%%--------------------------------------------------------------------
%% Snabbkaffe helpers
%%--------------------------------------------------------------------
check_snabbkaffe_vanilla(Trace) ->
ResumeTrace = [
T
|| #{?snk_kind := K} = T <- Trace,
re:run(to_list(K), "^ps_") /= nomatch
],
?assertMatch([_ | _], ResumeTrace),
[_Sid] = lists:usort(?projection(sid, ResumeTrace)),
%% Check internal flow of the emqx_cm resuming
?assert(
?strict_causality(
#{?snk_kind := ps_resuming},
#{?snk_kind := ps_initial_pendings},
ResumeTrace
)
),
?assert(
?strict_causality(
#{?snk_kind := ps_initial_pendings},
#{?snk_kind := ps_persist_pendings},
ResumeTrace
)
),
?assert(
?strict_causality(
#{?snk_kind := ps_persist_pendings},
#{?snk_kind := ps_notify_writers},
ResumeTrace
)
),
?assert(
?strict_causality(
#{?snk_kind := ps_notify_writers},
#{?snk_kind := ps_node_markers},
ResumeTrace
)
),
?assert(
?strict_causality(
#{?snk_kind := ps_node_markers},
#{?snk_kind := ps_resume_session},
ResumeTrace
)
),
?assert(
?strict_causality(
#{?snk_kind := ps_resume_session},
#{?snk_kind := ps_marker_pendings},
ResumeTrace
)
),
?assert(
?strict_causality(
#{?snk_kind := ps_marker_pendings},
#{?snk_kind := ps_marker_pendings_msgs},
ResumeTrace
)
),
?assert(
?strict_causality(
#{?snk_kind := ps_marker_pendings_msgs},
#{?snk_kind := ps_resume_end},
ResumeTrace
)
),
%% Check flow between worker and emqx_cm
?assert(
?strict_causality(
#{?snk_kind := ps_notify_writers},
#{?snk_kind := ps_worker_started},
ResumeTrace
)
),
?assert(
?strict_causality(
#{?snk_kind := ps_marker_pendings},
#{?snk_kind := ps_worker_resume_end},
ResumeTrace
)
),
?assert(
?strict_causality(
#{?snk_kind := ps_worker_resume_end},
#{?snk_kind := ps_worker_shutdown},
ResumeTrace
)
),
[Markers] = ?projection(markers, ?of_kind(ps_node_markers, Trace)),
?assertMatch([_], Markers).
to_list(L) when is_list(L) -> L;
to_list(A) when is_atom(A) -> atom_to_list(A);
to_list(B) when is_binary(B) -> binary_to_list(B).
%%--------------------------------------------------------------------
%% Snabbkaffe tests
%%--------------------------------------------------------------------
t_snabbkaffe_vanilla_stages(Config) ->
%% Test that all stages of session resume works ok in the simplest case
ConnFun = ?config(conn_fun, Config),
ClientId = ?config(client_id, Config),
EmqttOpts = [
{proto_ver, v5},
{clientid, ClientId},
{properties, #{'Session-Expiry-Interval' => 30}}
| Config
],
{ok, Client1} = emqtt:start_link([{clean_start, true} | EmqttOpts]),
{ok, _} = emqtt:ConnFun(Client1),
ok = emqtt:disconnect(Client1),
maybe_kill_connection_process(ClientId, Config),
?check_trace(
begin
{ok, Client2} = emqtt:start_link([{clean_start, false} | EmqttOpts]),
{ok, _} = emqtt:ConnFun(Client2),
ok = emqtt:disconnect(Client2)
end,
fun(ok, Trace) ->
check_snabbkaffe_vanilla(Trace)
end
),
ok.
t_snabbkaffe_pending_messages(Config) ->
%% Make sure there are pending messages are fetched during the init stage.
ConnFun = ?config(conn_fun, Config),
ClientId = ?config(client_id, Config),
Topic = ?config(topic, Config),
STopic = ?config(stopic, Config),
Payloads = [<<"test", (integer_to_binary(X))/binary>> || X <- [1, 2, 3, 4, 5]],
EmqttOpts = [
{proto_ver, v5},
{clientid, ClientId},
{properties, #{'Session-Expiry-Interval' => 30}}
| Config
],
{ok, Client1} = emqtt:start_link([{clean_start, true} | EmqttOpts]),
{ok, _} = emqtt:ConnFun(Client1),
{ok, _, [2]} = emqtt:subscribe(Client1, STopic, qos2),
ok = emqtt:disconnect(Client1),
maybe_kill_connection_process(ClientId, Config),
?check_trace(
begin
snabbkaffe_sync_publish(Topic, Payloads),
{ok, Client2} = emqtt:start_link([{clean_start, false} | EmqttOpts]),
{ok, _} = emqtt:ConnFun(Client2),
Msgs = receive_messages(length(Payloads)),
ReceivedPayloads = [P || #{payload := P} <- Msgs],
?assertEqual(lists:sort(ReceivedPayloads), lists:sort(Payloads)),
ok = emqtt:disconnect(Client2)
end,
fun(ok, Trace) ->
check_snabbkaffe_vanilla(Trace),
%% Check that all messages was delivered from the DB
[Delivers1] = ?projection(msgs, ?of_kind(ps_persist_pendings_msgs, Trace)),
[Delivers2] = ?projection(msgs, ?of_kind(ps_marker_pendings_msgs, Trace)),
Delivers = Delivers1 ++ Delivers2,
?assertEqual(length(Payloads), length(Delivers)),
%% Check for no duplicates
?assertEqual(lists:usort(Delivers), lists:sort(Delivers))
end
),
ok.
t_snabbkaffe_buffered_messages(Config) ->
%% Make sure to buffer messages during startup.
ConnFun = ?config(conn_fun, Config),
ClientId = ?config(client_id, Config),
Topic = ?config(topic, Config),
STopic = ?config(stopic, Config),
Payloads1 = [<<"test", (integer_to_binary(X))/binary>> || X <- [1, 2, 3]],
Payloads2 = [<<"test", (integer_to_binary(X))/binary>> || X <- [4, 5, 6]],
EmqttOpts = [
{proto_ver, v5},
{clientid, ClientId},
{properties, #{'Session-Expiry-Interval' => 30}}
| Config
],
{ok, Client1} = emqtt:start_link([{clean_start, true} | EmqttOpts]),
{ok, _} = emqtt:ConnFun(Client1),
{ok, _, [2]} = emqtt:subscribe(Client1, STopic, qos2),
ok = emqtt:disconnect(Client1),
maybe_kill_connection_process(ClientId, Config),
publish(Topic, Payloads1),
?check_trace(
begin
%% Make the resume init phase wait until the first message is delivered.
?force_ordering(
#{?snk_kind := ps_worker_deliver},
#{?snk_kind := ps_resume_end}
),
Parent = self(),
spawn_link(fun() ->
?block_until(#{?snk_kind := ps_marker_pendings_msgs}, infinity, 5000),
publish(Topic, Payloads2, true),
Parent ! publish_done,
ok
end),
{ok, Client2} = emqtt:start_link([{clean_start, false} | EmqttOpts]),
{ok, _} = emqtt:ConnFun(Client2),
receive
publish_done -> ok
after 10000 -> error(too_long_to_publish)
end,
Msgs = receive_messages(length(Payloads1) + length(Payloads2) + 1),
ReceivedPayloads = [P || #{payload := P} <- Msgs],
?assertEqual(
lists:sort(Payloads1 ++ Payloads2),
lists:sort(ReceivedPayloads)
),
ok = emqtt:disconnect(Client2)
end,
fun(ok, Trace) ->
check_snabbkaffe_vanilla(Trace),
%% Check that some messages was buffered in the writer process
[Msgs] = ?projection(msgs, ?of_kind(ps_writer_pendings, Trace)),
?assertMatch(
X when 0 < X andalso X =< length(Payloads2),
length(Msgs)
)
end
),
ok.
%%--------------------------------------------------------------------
%% GC tests
%%--------------------------------------------------------------------
-define(MARKER, 3).
-define(DELIVERED, 2).
-define(UNDELIVERED, 1).
-define(ABANDONED, 0).
msg_id() ->
emqx_guid:gen().
delivered_msg(MsgId, SessionID, STopic) ->
{SessionID, MsgId, STopic, ?DELIVERED}.
undelivered_msg(MsgId, SessionID, STopic) ->
{SessionID, MsgId, STopic, ?UNDELIVERED}.
marker_msg(MarkerID, SessionID) ->
{SessionID, MarkerID, <<>>, ?MARKER}.
guid(MicrosecondsAgo) ->
%% Make a fake GUID and set a timestamp.
<<TS:64, Tail:64>> = emqx_guid:gen(),
<<(TS - MicrosecondsAgo):64, Tail:64>>.
abandoned_session_msg(SessionID) ->
abandoned_session_msg(SessionID, 0).
abandoned_session_msg(SessionID, MicrosecondsAgo) ->
TS = erlang:system_time(microsecond),
{SessionID, <<>>, <<(TS - MicrosecondsAgo):64>>, ?ABANDONED}.
fresh_gc_delete_fun() ->
Ets = ets:new(gc_collect, [ordered_set]),
fun
(delete, Key) ->
ets:insert(Ets, {Key}),
ok;
(collect, <<>>) ->
List = ets:match(Ets, {'$1'}),
ets:delete(Ets),
lists:append(List);
(_, _Key) ->
ok
skip_ds_tc(Config) ->
case ?config(persistent_store, Config) of
ds ->
{skip, "Testcase not yet supported under 'emqx_persistent_session_ds' implementation"};
_ ->
Config
end.
fresh_gc_callbacks_fun() ->
Ets = ets:new(gc_collect, [ordered_set]),
fun
(collect, <<>>) ->
List = ets:match(Ets, {'$1'}),
ets:delete(Ets),
lists:append(List);
(Tag, Key) ->
ets:insert(Ets, {{Key, Tag}}),
ok
end.
get_gc_delete_messages() ->
Fun = fresh_gc_delete_fun(),
emqx_persistent_session:gc_session_messages(Fun),
Fun(collect, <<>>).
get_gc_callbacks() ->
Fun = fresh_gc_callbacks_fun(),
emqx_persistent_session:gc_session_messages(Fun),
Fun(collect, <<>>).
t_gc_all_delivered(Config) ->
Store = ?config(session_msg_store, Config),
STopic = ?config(stopic, Config),
SessionId = emqx_guid:gen(),
MsgIds = [msg_id() || _ <- lists:seq(1, 5)],
Delivered = [delivered_msg(X, SessionId, STopic) || X <- MsgIds],
Undelivered = [undelivered_msg(X, SessionId, STopic) || X <- MsgIds],
SortedContent = lists:usort(Delivered ++ Undelivered),
ets:insert(Store, [{X, <<>>} || X <- SortedContent]),
GCMessages = get_gc_delete_messages(),
?assertEqual(SortedContent, GCMessages),
ok.
t_gc_some_undelivered(Config) ->
Store = ?config(session_msg_store, Config),
STopic = ?config(stopic, Config),
SessionId = emqx_guid:gen(),
MsgIds = [msg_id() || _ <- lists:seq(1, 10)],
Delivered = [delivered_msg(X, SessionId, STopic) || X <- MsgIds],
{Delivered1, _Delivered2} = split(Delivered),
Undelivered = [undelivered_msg(X, SessionId, STopic) || X <- MsgIds],
{Undelivered1, Undelivered2} = split(Undelivered),
Content = Delivered1 ++ Undelivered1 ++ Undelivered2,
ets:insert(Store, [{X, <<>>} || X <- Content]),
Expected = lists:usort(Delivered1 ++ Undelivered1),
GCMessages = get_gc_delete_messages(),
?assertEqual(Expected, GCMessages),
ok.
t_gc_with_markers(Config) ->
Store = ?config(session_msg_store, Config),
STopic = ?config(stopic, Config),
SessionId = emqx_guid:gen(),
MsgIds1 = [msg_id() || _ <- lists:seq(1, 10)],
MarkerId = msg_id(),
MsgIds = [msg_id() || _ <- lists:seq(1, 4)] ++ MsgIds1,
Delivered = [delivered_msg(X, SessionId, STopic) || X <- MsgIds],
{Delivered1, _Delivered2} = split(Delivered),
Undelivered = [undelivered_msg(X, SessionId, STopic) || X <- MsgIds],
{Undelivered1, Undelivered2} = split(Undelivered),
Markers = [marker_msg(MarkerId, SessionId)],
Content = Delivered1 ++ Undelivered1 ++ Undelivered2 ++ Markers,
ets:insert(Store, [{X, <<>>} || X <- Content]),
Expected = lists:usort(Delivered1 ++ Undelivered1),
GCMessages = get_gc_delete_messages(),
?assertEqual(Expected, GCMessages),
ok.
t_gc_abandoned_some_undelivered(Config) ->
Store = ?config(session_msg_store, Config),
STopic = ?config(stopic, Config),
SessionId = emqx_guid:gen(),
MsgIds = [msg_id() || _ <- lists:seq(1, 10)],
Delivered = [delivered_msg(X, SessionId, STopic) || X <- MsgIds],
{Delivered1, _Delivered2} = split(Delivered),
Undelivered = [undelivered_msg(X, SessionId, STopic) || X <- MsgIds],
{Undelivered1, Undelivered2} = split(Undelivered),
Abandoned = abandoned_session_msg(SessionId),
Content = Delivered1 ++ Undelivered1 ++ Undelivered2 ++ [Abandoned],
ets:insert(Store, [{X, <<>>} || X <- Content]),
Expected = lists:usort(Delivered1 ++ Undelivered1 ++ Undelivered2),
GCMessages = get_gc_delete_messages(),
?assertEqual(Expected, GCMessages),
ok.
t_gc_abandoned_only_called_on_empty_session(Config) ->
Store = ?config(session_msg_store, Config),
STopic = ?config(stopic, Config),
SessionId = emqx_guid:gen(),
MsgIds = [msg_id() || _ <- lists:seq(1, 10)],
Delivered = [delivered_msg(X, SessionId, STopic) || X <- MsgIds],
Undelivered = [undelivered_msg(X, SessionId, STopic) || X <- MsgIds],
Abandoned = abandoned_session_msg(SessionId),
Content = Delivered ++ Undelivered ++ [Abandoned],
ets:insert(Store, [{X, <<>>} || X <- Content]),
GCMessages = get_gc_callbacks(),
%% Since we had messages to delete, we don't expect to get the
%% callback on the abandoned session
?assertEqual([], [X || {X, abandoned} <- GCMessages]),
%% But if we have only the abandoned session marker for this
%% session, it should be called.
ets:delete_all_objects(Store),
UndeliveredOtherSession = undelivered_msg(msg_id(), emqx_guid:gen(), <<"topic">>),
ets:insert(Store, [{X, <<>>} || X <- [Abandoned, UndeliveredOtherSession]]),
GCMessages2 = get_gc_callbacks(),
?assertEqual([Abandoned], [X || {X, abandoned} <- GCMessages2]),
ok.
t_gc_session_gc_worker(init, Config) ->
meck:new(emqx_persistent_session, [passthrough, no_link]),
Config;
t_gc_session_gc_worker('end', _Config) ->
meck:unload(emqx_persistent_session),
ok.
t_gc_session_gc_worker(Config) ->
STopic = ?config(stopic, Config),
SessionID = emqx_guid:gen(),
MsgDeleted = delivered_msg(msg_id(), SessionID, STopic),
MarkerNotDeleted = marker_msg(msg_id(), SessionID),
MarkerDeleted = marker_msg(guid(120 * 1000 * 1000), SessionID),
AbandonedNotDeleted = abandoned_session_msg(SessionID),
AbandonedDeleted = abandoned_session_msg(SessionID, 500 * 1000 * 1000),
meck:expect(emqx_persistent_session, delete_session_message, fun(_Key) -> ok end),
emqx_persistent_session_gc:session_gc_worker(delete, MsgDeleted),
emqx_persistent_session_gc:session_gc_worker(marker, MarkerNotDeleted),
emqx_persistent_session_gc:session_gc_worker(marker, MarkerDeleted),
emqx_persistent_session_gc:session_gc_worker(abandoned, AbandonedDeleted),
emqx_persistent_session_gc:session_gc_worker(abandoned, AbandonedNotDeleted),
History = meck:history(emqx_persistent_session, self()),
DeleteCalls = [
Key
|| {_Pid, {_, delete_session_message, [Key]}, _Result} <-
History
],
?assertEqual(
lists:sort([MsgDeleted, AbandonedDeleted, MarkerDeleted]),
lists:sort(DeleteCalls)
),
ok.
t_gc_message_gc(Config) ->
Topic = ?config(topic, Config),
ClientID = ?config(client_id, Config),
Store = ?config(msg_store, Config),
NewMsgs = [
emqx_message:make(ClientID, Topic, integer_to_binary(P))
|| P <- lists:seq(6, 10)
],
Retain = 60 * 1000,
emqx_config:put(?msg_retain, Retain),
Msgs1 = [
emqx_message:make(ClientID, Topic, integer_to_binary(P))
|| P <- lists:seq(1, 5)
],
OldMsgs = [M#message{id = guid(Retain * 1000)} || M <- Msgs1],
ets:insert(Store, NewMsgs ++ OldMsgs),
?assertEqual(lists:sort(OldMsgs ++ NewMsgs), ets:tab2list(Store)),
ok = emqx_persistent_session_gc:message_gc_worker(),
?assertEqual(lists:sort(NewMsgs), ets:tab2list(Store)),
ok.
split(List) ->
split(List, [], []).
split([], L1, L2) ->
{L1, L2};
split([H], L1, L2) ->
{[H | L1], L2};
split([H1, H2 | Left], L1, L2) ->
split(Left, [H1 | L1], [H2 | L2]).

View File

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

View File

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

View File

@ -0,0 +1,599 @@
%%--------------------------------------------------------------------
%% 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("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
%%--------------------------------------------------------------------
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]).
%%--------------------------------------------------------------------
%% 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()),
?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),
PacketId1 = 1,
Session = session(#{max_awaiting_rel => 2, awaiting_rel => #{PacketId1 => ts(millisecond)}}),
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,
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,
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),
?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),
?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),
?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),
?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),
?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
),
?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),
?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),
?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(_) ->
RetryIntervalMs = 1000,
Session = session(#{retry_interval => RetryIntervalMs}),
Delivers = enrich(
[
delivery(?QOS_1, <<"t1">>, <<"expiressoon">>, _Expiry = 1),
delivery(?QOS_2, <<"t2">>),
delivery(?QOS_0, <<"t3">>),
delivery(?QOS_1, <<"t4">>)
],
Session
),
{ok, Pubs, Session1} = emqx_session_mem:deliver(clientinfo(), Delivers, Session),
[_Pub1, Pub2, _Pub3, Pub4] = Pubs,
{ok, _Msg, Session2} = emqx_session_mem:pubrec(get_packet_id(Pub2), Session1),
ElapseMs = 1500,
ok = timer:sleep(ElapseMs),
{ok, PubsRetry, RetryIntervalMs, Session3} = emqx_session_mem:handle_timeout(
clientinfo(), retry_delivery, Session2
),
?assertEqual(
[
% Pub1 is expired
{pubrel, get_packet_id(Pub2)},
% Pub3 is QoS0
set_duplicate_pub(Pub4)
],
remove_deliver_flag(PubsRetry)
),
?assertEqual(
2,
emqx_session_mem:info(inflight_cnt, Session3)
).
%%--------------------------------------------------------------------
%% 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(_) ->
Now = ts(millisecond),
AwaitRelTimeout = 10000,
Session = session(#{await_rel_timeout => AwaitRelTimeout}),
Ts1 = Now - 1000,
Ts2 = Now - 20000,
{ok, [], Session1} = emqx_session_mem:expire(clientinfo(), Session),
Session2 = emqx_session_mem:set_field(awaiting_rel, #{1 => Ts1, 2 => Ts2}, Session1),
{ok, [], Timeout, Session3} = emqx_session_mem:expire(clientinfo(), Session2),
?assertEqual(#{1 => Ts1}, emqx_session_mem:info(awaiting_rel, Session3)),
?assert(Timeout =< AwaitRelTimeout).
t_expire_awaiting_rel_all(_) ->
Session = session(#{awaiting_rel => #{1 => 1, 2 => 2}}),
{ok, [], Session1} = emqx_session_mem:expire(clientinfo(), Session),
?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) ->
Payload = emqx_guid:to_hexstr(emqx_guid:gen()),
{deliver, Topic, emqx_message:make(test, QoS, Topic, Payload)}.
delivery(QoS, Topic, Payload, ExpiryInterval) ->
Headers = #{properties => #{'Message-Expiry-Interval' => ExpiryInterval}},
{deliver, Topic, emqx_message:make(test, QoS, Topic, Payload, #{}, Headers)}.
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({pubrel, Id}) ->
{pubrel, Id};
remove_deliver_flag({Id, Data}) ->
{Id, remove_deliver_flag(Data)};
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).
set_duplicate_pub({Id, Msg}) ->
{Id, emqx_message:set_flag(dup, Msg)}.
get_packet_id({Id, _}) ->
Id.

View File

@ -160,4 +160,15 @@ assert_messages_order([Msg | Ls1], [{publish, #{payload := No}} | Ls2]) ->
end.
messages(Cnt) ->
[emqx_message:make(ct, 1, ?TOPIC, integer_to_binary(I)) || I <- lists:seq(1, Cnt)].
[emqx_message:make(ct, 1, ?TOPIC, payload(I)) || I <- lists:seq(1, Cnt)].
payload(I) ->
% NOTE
% Introduce randomness so that natural order is not the same as arrival order.
iolist_to_binary(
io_lib:format("~4.16.0B [~B] [~s]", [
rand:uniform(16#10000) - 1,
I,
emqx_utils_calendar:now_to_rfc3339(millisecond)
])
).

View File

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

View File

@ -27,9 +27,10 @@
%% Session:
-export([
session_open/1,
session_ensure_new/2,
session_drop/1,
session_suspend/1,
session_add_iterator/2,
session_add_iterator/3,
session_get_iterator_id/2,
session_del_iterator/2,
session_stats/0
@ -51,6 +52,7 @@
shard/0,
shard_id/0,
topic/0,
topic_filter/0,
time/0
]).
@ -60,6 +62,17 @@
%% Type declarations
%%================================================================================
%% Session
%% See also: `#session{}`.
-type session() :: #{
id := emqx_ds:session_id(),
created_at := _Millisecond :: non_neg_integer(),
expires_at := _Millisecond :: non_neg_integer() | never,
props := map()
}.
-type iterators() :: #{topic_filter() => iterator()}.
%% Currently, this is the clientid. We avoid `emqx_types:clientid()' because that can be
%% an atom, in theory (?).
-type session_id() :: binary().
@ -68,17 +81,18 @@
-type iterator_id() :: binary().
%%-type session() :: #session{}.
-type message_store_opts() :: #{}.
-type message_stats() :: #{}.
-type message_id() :: binary().
%% Parsed topic:
%% Parsed topic.
-type topic() :: list(binary()).
%% Parsed topic filter.
-type topic_filter() :: list(binary() | '+' | '#' | '').
-type keyspace() :: atom().
-type shard_id() :: binary().
-type shard() :: {keyspace(), shard_id()}.
@ -92,7 +106,7 @@
-type replay_id() :: binary().
-type replay() :: {
_TopicFilter :: emqx_topic:words(),
_TopicFilter :: topic_filter(),
_StartTime :: time()
}.
@ -135,39 +149,59 @@ message_stats() ->
%%--------------------------------------------------------------------------------
%% @doc Called when a client connects. This function looks up a
%% session or creates a new one if previous one couldn't be found.
%% session or returns `false` if previous one couldn't be found.
%%
%% This function also spawns replay agents for each iterator.
%%
%% Note: session API doesn't handle session takeovers, it's the job of
%% the broker.
-spec session_open(emqx_types:clientid()) -> {_New :: boolean(), session_id()}.
session_open(ClientID) ->
{atomic, Res} =
mria:transaction(?DS_MRIA_SHARD, fun() ->
case mnesia:read(?SESSION_TAB, ClientID, write) of
[#session{}] ->
{false, ClientID};
[] ->
Session = #session{id = ClientID},
mnesia:write(?SESSION_TAB, Session, write),
{true, ClientID}
end
end),
Res.
-spec session_open(session_id()) ->
{ok, session(), iterators()} | false.
session_open(SessionId) ->
transaction(fun() ->
case mnesia:read(?SESSION_TAB, SessionId, write) of
[Record = #session{}] ->
Session = export_record(Record),
IteratorRefs = session_read_iterators(SessionId),
Iterators = export_iterators(IteratorRefs),
{ok, Session, Iterators};
[] ->
false
end
end).
-spec session_ensure_new(session_id(), _Props :: map()) ->
{ok, session(), iterators()}.
session_ensure_new(SessionId, Props) ->
transaction(fun() ->
ok = session_drop_iterators(SessionId),
Session = export_record(session_create(SessionId, Props)),
{ok, Session, #{}}
end).
session_create(SessionId, Props) ->
Session = #session{
id = SessionId,
created_at = erlang:system_time(millisecond),
expires_at = never,
props = Props
},
ok = mnesia:write(?SESSION_TAB, Session, write),
Session.
%% @doc Called when a client reconnects with `clean session=true' or
%% during session GC
-spec session_drop(emqx_types:clientid()) -> ok.
session_drop(ClientID) ->
{atomic, ok} = mria:transaction(
?DS_MRIA_SHARD,
fun() ->
%% TODO: ensure all iterators from this clientid are closed?
mnesia:delete({?SESSION_TAB, ClientID})
end
),
ok.
-spec session_drop(session_id()) -> ok.
session_drop(DSSessionId) ->
transaction(fun() ->
%% TODO: ensure all iterators from this clientid are closed?
ok = session_drop_iterators(DSSessionId),
ok = mnesia:delete(?SESSION_TAB, DSSessionId, write)
end).
session_drop_iterators(DSSessionId) ->
IteratorRefs = session_read_iterators(DSSessionId),
ok = lists:foreach(fun session_del_iterator/1, IteratorRefs).
%% @doc Called when a client disconnects. This function terminates all
%% active processes related to the session.
@ -177,39 +211,48 @@ session_suspend(_SessionId) ->
ok.
%% @doc Called when a client subscribes to a topic. Idempotent.
-spec session_add_iterator(session_id(), emqx_topic:words()) ->
{ok, iterator_id(), time(), _IsNew :: boolean()}.
session_add_iterator(DSSessionId, TopicFilter) ->
-spec session_add_iterator(session_id(), topic_filter(), _Props :: map()) ->
{ok, iterator(), _IsNew :: boolean()}.
session_add_iterator(DSSessionId, TopicFilter, Props) ->
IteratorRefId = {DSSessionId, TopicFilter},
{atomic, Res} =
mria:transaction(?DS_MRIA_SHARD, fun() ->
case mnesia:read(?ITERATOR_REF_TAB, IteratorRefId, write) of
[] ->
{IteratorId, StartMS} = new_iterator_id(DSSessionId),
IteratorRef = #iterator_ref{
ref_id = IteratorRefId,
it_id = IteratorId,
start_time = StartMS
},
ok = mnesia:write(?ITERATOR_REF_TAB, IteratorRef, write),
?tp(
ds_session_subscription_added,
#{iterator_id => IteratorId, session_id => DSSessionId}
),
IsNew = true,
{ok, IteratorId, StartMS, IsNew};
[#iterator_ref{it_id = IteratorId, start_time = StartMS}] ->
?tp(
ds_session_subscription_present,
#{iterator_id => IteratorId, session_id => DSSessionId}
),
IsNew = false,
{ok, IteratorId, StartMS, IsNew}
end
end),
Res.
transaction(fun() ->
case mnesia:read(?ITERATOR_REF_TAB, IteratorRefId, write) of
[] ->
IteratorRef = session_insert_iterator(DSSessionId, TopicFilter, Props),
Iterator = export_record(IteratorRef),
?tp(
ds_session_subscription_added,
#{iterator => Iterator, session_id => DSSessionId}
),
{ok, Iterator, _IsNew = true};
[#iterator_ref{} = IteratorRef] ->
NIteratorRef = session_update_iterator(IteratorRef, Props),
NIterator = export_record(NIteratorRef),
?tp(
ds_session_subscription_present,
#{iterator => NIterator, session_id => DSSessionId}
),
{ok, NIterator, _IsNew = false}
end
end).
-spec session_get_iterator_id(session_id(), emqx_topic:words()) ->
session_insert_iterator(DSSessionId, TopicFilter, Props) ->
{IteratorId, StartMS} = new_iterator_id(DSSessionId),
IteratorRef = #iterator_ref{
ref_id = {DSSessionId, TopicFilter},
it_id = IteratorId,
start_time = StartMS,
props = Props
},
ok = mnesia:write(?ITERATOR_REF_TAB, IteratorRef, write),
IteratorRef.
session_update_iterator(IteratorRef, Props) ->
NIteratorRef = IteratorRef#iterator_ref{props = Props},
ok = mnesia:write(?ITERATOR_REF_TAB, NIteratorRef, write),
NIteratorRef.
-spec session_get_iterator_id(session_id(), topic_filter()) ->
{ok, iterator_id()} | {error, not_found}.
session_get_iterator_id(DSSessionId, TopicFilter) ->
IteratorRefId = {DSSessionId, TopicFilter},
@ -221,14 +264,23 @@ session_get_iterator_id(DSSessionId, TopicFilter) ->
end.
%% @doc Called when a client unsubscribes from a topic.
-spec session_del_iterator(session_id(), emqx_topic:words()) -> ok.
-spec session_del_iterator(session_id(), topic_filter()) -> ok.
session_del_iterator(DSSessionId, TopicFilter) ->
IteratorRefId = {DSSessionId, TopicFilter},
{atomic, ok} =
mria:transaction(?DS_MRIA_SHARD, fun() ->
mnesia:delete(?ITERATOR_REF_TAB, IteratorRefId, write)
end),
ok.
transaction(fun() ->
mnesia:delete(?ITERATOR_REF_TAB, IteratorRefId, write)
end).
session_del_iterator(#iterator_ref{ref_id = IteratorRefId}) ->
mnesia:delete(?ITERATOR_REF_TAB, IteratorRefId, write).
session_read_iterators(DSSessionId) ->
% NOTE: somewhat convoluted way to trick dialyzer
Pat = erlang:make_tuple(record_info(size, iterator_ref), '_', [
{1, iterator_ref},
{#iterator_ref.ref_id, {DSSessionId, '_'}}
]),
mnesia:match_object(?ITERATOR_REF_TAB, Pat, read).
-spec session_stats() -> #{}.
session_stats() ->
@ -263,3 +315,30 @@ new_iterator_id(DSSessionId) ->
NowMS = erlang:system_time(microsecond),
IteratorId = <<DSSessionId/binary, (emqx_guid:gen())/binary>>,
{IteratorId, NowMS}.
%%--------------------------------------------------------------------------------
transaction(Fun) ->
{atomic, Res} = mria:transaction(?DS_MRIA_SHARD, Fun),
Res.
%%--------------------------------------------------------------------------------
export_iterators(IteratorRefs) ->
lists:foldl(
fun(IteratorRef = #iterator_ref{ref_id = {_DSSessionId, TopicFilter}}, Acc) ->
Acc#{TopicFilter => export_record(IteratorRef)}
end,
#{},
IteratorRefs
).
export_record(#session{} = Record) ->
export_record(Record, #session.id, [id, created_at, expires_at, props], #{});
export_record(#iterator_ref{} = Record) ->
export_record(Record, #iterator_ref.it_id, [id, start_time, props], #{}).
export_record(Record, I, [Field | Rest], Acc) ->
export_record(Record, I + 1, Rest, Acc#{Field => element(I, Record)});
export_record(_, _, [], Acc) ->
Acc.

View File

@ -23,14 +23,18 @@
-record(session, {
%% same as clientid
id :: emqx_ds:session_id(),
%% creation time
created_at :: _Millisecond :: non_neg_integer(),
expires_at = never :: _Millisecond :: non_neg_integer() | never,
%% for future usage
props = #{} :: map()
}).
-record(iterator_ref, {
ref_id :: {emqx_ds:session_id(), emqx_topic:words()},
ref_id :: {emqx_ds:session_id(), emqx_ds:topic_filter()},
it_id :: emqx_ds:iterator_id(),
start_time :: emqx_ds:time()
start_time :: emqx_ds:time(),
props = #{} :: map()
}).
-endif.

View File

@ -132,6 +132,7 @@
%%================================================================================
-type topic() :: emqx_ds:topic().
-type topic_filter() :: emqx_ds:topic_filter().
-type time() :: emqx_ds:time().
%% Number of bits
@ -191,7 +192,7 @@
-record(filter, {
keymapper :: keymapper(),
topic_filter :: emqx_topic:words(),
topic_filter :: topic_filter(),
start_time :: integer(),
hash_bitfilter :: integer(),
hash_bitmask :: integer(),
@ -412,11 +413,11 @@ extract(Key, #keymapper{bitsize = Size}) ->
<<Bitstring:Size/integer, _MessageID/binary>> = Key,
Bitstring.
-spec compute_bitstring(topic(), time(), keymapper()) -> integer().
compute_bitstring(Topic, Timestamp, #keymapper{source = Source}) ->
compute_bitstring(Topic, Timestamp, Source, 0).
-spec compute_bitstring(topic_filter(), time(), keymapper()) -> integer().
compute_bitstring(TopicFilter, Timestamp, #keymapper{source = Source}) ->
compute_bitstring(TopicFilter, Timestamp, Source, 0).
-spec compute_topic_bitmask(emqx_topic:words(), keymapper()) -> integer().
-spec compute_topic_bitmask(topic_filter(), keymapper()) -> integer().
compute_topic_bitmask(TopicFilter, #keymapper{source = Source}) ->
compute_topic_bitmask(TopicFilter, Source, 0).

View File

@ -7,7 +7,6 @@
-include_lib("emqx/include/emqx.hrl").
-include_lib("emqx/include/emqx_channel.hrl").
-include_lib("emqx/include/emqx_mqtt.hrl").
-include_lib("emqx/include/logger.hrl").
-include_lib("emqx/include/types.hrl").
@ -122,7 +121,9 @@ handle_call(
pendings := Pendings
} = Channel
) ->
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),
@ -196,8 +197,11 @@ handle_deliver(
clientinfo := ClientInfo
} = Channel
) ->
% NOTE
% This is essentially part of `emqx_session_mem` logic, thus call it directly.
Delivers1 = emqx_channel:maybe_nack(Delivers),
NSession = emqx_session:enqueue(ClientInfo, Delivers1, Session),
Messages = emqx_session:enrich_delivers(ClientInfo, Delivers1, Session),
NSession = emqx_session_mem:enqueue(ClientInfo, Messages, Session),
Channel#{session := NSession}.
cancel_expiry_timer(#{expiry_timer := TRef}) when is_reference(TRef) ->
@ -230,7 +234,7 @@ open_session(ConnInfo, #{clientid := ClientId} = ClientInfo) ->
}
),
{error, no_session};
{ok, #{session := Session, present := true, pendings := Pendings0}} ->
{ok, #{session := Session, present := true, replay := Pendings}} ->
?SLOG(
info,
#{
@ -239,12 +243,15 @@ open_session(ConnInfo, #{clientid := ClientId} = ClientInfo) ->
node => node()
}
),
Pendings1 = lists:usort(lists:append(Pendings0, emqx_utils:drain_deliver())),
NSession = emqx_session:enqueue(
ClientInfo,
emqx_channel:maybe_nack(Pendings1),
Session
),
% NOTE
% Here we aggregate and deduplicate remote and local pending deliveries,
% throwing away any local deliveries that are part of some shared
% subscription. Remote deliviries pertaining to shared subscriptions should
% already have been thrown away by `emqx_channel:handle_deliver/2`.
% See also: `emqx_channel:maybe_resume_session/1`, `emqx_session_mem:replay/3`.
DeliversLocal = emqx_channel:maybe_nack(emqx_utils:drain_deliver()),
PendingsAll = emqx_session_mem:dedup(ClientInfo, Pendings, DeliversLocal, Session),
NSession = emqx_session_mem:enqueue(ClientInfo, PendingsAll, Session),
NChannel = Channel#{session => NSession},
ok = emqx_cm:insert_channel_info(ClientId, info(NChannel), stats(NChannel)),
?SLOG(

View File

@ -29,16 +29,6 @@ init_per_suite(Config) ->
end_per_suite(_Config) ->
emqx_common_test_helpers:stop_apps([emqx_eviction_agent, emqx_conf]).
init_per_testcase(t_persistence, _Config) ->
{skip, "Existing session persistence implementation is being phased out"};
init_per_testcase(_TestCase, Config) ->
Config.
end_per_testcase(t_persistence, Config) ->
Config;
end_per_testcase(_TestCase, _Config) ->
ok.
%%--------------------------------------------------------------------
%% Tests
%%--------------------------------------------------------------------
@ -199,40 +189,6 @@ t_get_connected_client_count(_Config) ->
emqx_cm:get_connected_client_count()
).
t_persistence(_Config) ->
erlang:process_flag(trap_exit, true),
Topic = <<"t1">>,
Message = <<"message_to_persist">>,
{ok, C0} = emqtt_connect(?CLIENT_ID, false),
{ok, _, _} = emqtt:subscribe(C0, Topic, 0),
Opts = evict_session_opts(?CLIENT_ID),
{ok, Pid} = emqx_eviction_agent_channel:start_supervised(Opts),
{ok, C1} = emqtt_connect(),
{ok, _} = emqtt:publish(C1, Topic, Message, 1),
ok = emqtt:disconnect(C1),
%% Kill channel so that the session is only persisted
ok = emqx_eviction_agent_channel:call(Pid, kick),
%% Should restore session from persistents storage and receive messages
{ok, C2} = emqtt_connect(?CLIENT_ID, false),
receive
{publish, #{
payload := Message,
topic := Topic
}} ->
ok
after 1000 ->
ct:fail("message not received")
end,
ok = emqtt:disconnect(C2).
%%--------------------------------------------------------------------
%% Helpers
%%--------------------------------------------------------------------

View File

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

View File

@ -1,6 +1,6 @@
{application, emqx_gateway_coap, [
{description, "CoAP Gateway"},
{vsn, "0.1.2"},
{vsn, "0.1.3"},
{registered, []},
{applications, [kernel, stdlib, emqx, emqx_gateway]},
{env, []},

View File

@ -74,12 +74,6 @@
-type replies() :: emqx_types:packet() | reply() | [reply()].
-define(TIMER_TABLE, #{
alive_timer => keepalive,
force_timer => force_close,
idle_timer => force_close_idle
}).
-define(INFO_KEYS, [conninfo, conn_state, clientinfo, session, will_msg]).
%%--------------------------------------------------------------------
@ -224,7 +218,7 @@ address({Host, Port}) ->
%% avoid udp connection process leak
start_idle_checking_timer(Channel = #channel{conninfo = #{socktype := udp}}) ->
ensure_timer(idle_timer, Channel);
ensure_timer(force_close_idle, Channel);
start_idle_checking_timer(Channel) ->
Channel.
@ -293,10 +287,10 @@ handle_timeout(
case emqx_keepalive:check(StatVal, Keepalive) of
{ok, NKeepalive} ->
NChannel = Channel#channel{keepalive = NKeepalive},
{ok, reset_timer(alive_timer, NChannel)};
{ok, reset_timer(keepalive, NChannel)};
{error, timeout} ->
Req = #{type => 'KEEPALIVE'},
NChannel = remove_timer_ref(alive_timer, Channel),
NChannel = remove_timer_ref(keepalive, Channel),
%% close connection if keepalive timeout
Replies = [{event, disconnected}, {close, keepalive_timeout}],
NChannel1 = dispatch(on_timer_timeout, Req, NChannel#channel{
@ -419,7 +413,7 @@ handle_call(
NConnInfo = ConnInfo#{keepalive => Interval},
NClientInfo = ClientInfo#{keepalive => Interval},
NChannel = Channel#channel{conninfo = NConnInfo, clientinfo = NClientInfo},
{reply, ok, [{event, updated}], ensure_keepalive(cancel_timer(idle_timer, NChannel))};
{reply, ok, [{event, updated}], ensure_keepalive(cancel_timer(force_close_idle, NChannel))};
handle_call(
{subscribe_from_client, TopicFilter, Qos},
_From,
@ -529,7 +523,7 @@ handle_info(
_ ->
Channel
end,
Channel2 = ensure_timer(force_timer, Channel1),
Channel2 = ensure_timer(force_close, Channel1),
{ok, ensure_disconnected(Reason, Channel2)}
end;
handle_info(
@ -547,13 +541,13 @@ handle_info(
ShutdownNow =
emqx_exproto_gcli:is_empty(GClient) andalso
maps:get(force_timer, Timers, undefined) =/= undefined,
maps:get(force_close, Timers, undefined) =/= undefined,
case Result of
ok when not ShutdownNow ->
GClient1 = emqx_exproto_gcli:maybe_shoot(GClient),
{ok, Channel#channel{gcli = GClient1}};
ok when ShutdownNow ->
Channel1 = cancel_timer(force_timer, Channel),
Channel1 = cancel_timer(force_close, Channel),
{shutdown, Channel1#channel.closed_reason, Channel1};
{error, Reason} ->
{shutdown, {error, {FunName, Reason}}, Channel}
@ -711,7 +705,7 @@ ensure_keepalive_timer(Interval, Channel) when Interval =< 0 ->
ensure_keepalive_timer(Interval, Channel) ->
StatVal = emqx_gateway_conn:keepalive_stats(recv),
Keepalive = emqx_keepalive:init(StatVal, timer:seconds(Interval)),
ensure_timer(alive_timer, Channel#channel{keepalive = Keepalive}).
ensure_timer(keepalive, Channel#channel{keepalive = Keepalive}).
ensure_timer(Name, Channel = #channel{timers = Timers}) ->
TRef = maps:get(Name, Timers, undefined),
@ -723,8 +717,7 @@ ensure_timer(Name, Channel = #channel{timers = Timers}) ->
end.
ensure_timer(Name, Time, Channel = #channel{timers = Timers}) ->
Msg = maps:get(Name, ?TIMER_TABLE),
TRef = emqx_utils:start_timer(Time, Msg),
TRef = emqx_utils:start_timer(Time, Name),
Channel#channel{timers = Timers#{Name => TRef}}.
reset_timer(Name, Channel) ->
@ -737,11 +730,11 @@ cancel_timer(Name, Channel = #channel{timers = Timers}) ->
remove_timer_ref(Name, Channel = #channel{timers = Timers}) ->
Channel#channel{timers = maps:remove(Name, Timers)}.
interval(idle_timer, #channel{conninfo = #{idle_timeout := IdleTimeout}}) ->
interval(force_close_idle, #channel{conninfo = #{idle_timeout := IdleTimeout}}) ->
IdleTimeout;
interval(force_timer, _) ->
interval(force_close, _) ->
15000;
interval(alive_timer, #channel{keepalive = Keepalive}) ->
interval(keepalive, #channel{keepalive = Keepalive}) ->
emqx_keepalive:info(interval, Keepalive).
%%--------------------------------------------------------------------

View File

@ -1,6 +1,6 @@
{application, emqx_gateway_exproto, [
{description, "ExProto Gateway"},
{vsn, "0.1.3"},
{vsn, "0.1.4"},
{registered, []},
{applications, [kernel, stdlib, grpc, emqx, emqx_gateway]},
{env, []},

View File

@ -1,6 +1,6 @@
{application, emqx_gateway_mqttsn, [
{description, "MQTT-SN Gateway"},
{vsn, "0.1.3"},
{vsn, "0.1.4"},
{registered, []},
{applications, [kernel, stdlib, emqx, emqx_gateway]},
{env, []},

View File

@ -104,15 +104,6 @@
-type replies() :: reply() | [reply()].
-define(TIMER_TABLE, #{
alive_timer => keepalive,
retry_timer => retry_delivery,
await_timer => expire_awaiting_rel,
expire_timer => expire_session,
asleep_timer => expire_asleep,
register_timer => retry_register
}).
-define(DEFAULT_OVERRIDE, #{
clientid => <<"${ConnInfo.clientid}">>
%, username => <<"${ConnInfo.clientid}">>
@ -431,7 +422,7 @@ ensure_keepalive_timer(0, Channel) ->
Channel;
ensure_keepalive_timer(Interval, Channel) ->
Keepalive = emqx_keepalive:init(round(timer:seconds(Interval))),
ensure_timer(alive_timer, Channel#channel{keepalive = Keepalive}).
ensure_timer(keepalive, Channel#channel{keepalive = Keepalive}).
%%--------------------------------------------------------------------
%% Handle incoming packet
@ -669,7 +660,7 @@ handle_in(
topic_name => TopicName
}),
NChannel = cancel_timer(
register_timer,
retry_register,
Channel#channel{register_inflight = undefined}
),
send_next_register_or_replay_publish(TopicName, NChannel);
@ -692,7 +683,7 @@ handle_in(
topic_name => TopicName
}),
NChannel = cancel_timer(
register_timer,
retry_register,
Channel#channel{register_inflight = undefined}
),
send_next_register_or_replay_publish(TopicName, NChannel)
@ -1165,7 +1156,7 @@ do_publish(
case emqx_mqttsn_session:publish(ClientInfo, MsgId, Msg, Session) of
{ok, _PubRes, NSession} ->
NChannel1 = ensure_timer(
await_timer,
expire_awaiting_rel,
Channel#channel{session = NSession}
),
handle_out(pubrec, MsgId, NChannel1);
@ -1178,10 +1169,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.
@ -1439,18 +1426,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,
Channel1 = cancel_timer(asleep_timer, Channel),
{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
}
@ -1499,7 +1479,7 @@ asleep(Duration, Channel = #channel{conn_state = asleep}) ->
msg => "update_asleep_timer",
new_duration => Duration
}),
ensure_asleep_timer(Duration, cancel_timer(asleep_timer, Channel));
ensure_asleep_timer(Duration, cancel_timer(expire_asleep, Channel));
asleep(Duration, Channel = #channel{conn_state = connected}) ->
?SLOG(info, #{
msg => "goto_asleep_state",
@ -1907,7 +1887,7 @@ maybe_shutdown(Reason, Channel = #channel{conninfo = ConnInfo}) ->
?UINT_MAX ->
{ok, Channel};
I when I > 0 ->
{ok, ensure_timer(expire_timer, I, Channel)};
{ok, ensure_timer(expire_session, I, Channel)};
_ ->
shutdown(Reason, Channel)
end.
@ -2007,7 +1987,7 @@ handle_deliver(
handle_out(
publish,
Publishes,
ensure_timer(retry_timer, NChannel)
ensure_timer(retry_delivery, NChannel)
);
{ok, NSession} ->
{ok, Channel#channel{session = NSession}}
@ -2068,7 +2048,7 @@ handle_timeout(
case emqx_keepalive:check(StatVal, Keepalive) of
{ok, NKeepalive} ->
NChannel = Channel#channel{keepalive = NKeepalive},
{ok, reset_timer(alive_timer, NChannel)};
{ok, reset_timer(keepalive, NChannel)};
{error, timeout} ->
handle_out(disconnect, ?SN_RC2_KEEPALIVE_TIMEOUT, Channel)
end;
@ -2080,23 +2060,10 @@ handle_timeout(
{ok, Channel};
handle_timeout(
_TRef,
retry_delivery,
retry_delivery = TimerName,
Channel = #channel{conn_state = asleep}
) ->
{ok, reset_timer(retry_timer, Channel)};
handle_timeout(
_TRef,
retry_delivery,
Channel = #channel{session = Session, clientinfo = ClientInfo}
) ->
case emqx_mqttsn_session:retry(ClientInfo, Session) of
{ok, NSession} ->
{ok, clean_timer(retry_timer, Channel#channel{session = NSession})};
{ok, Publishes, Timeout, NSession} ->
NChannel = Channel#channel{session = NSession},
%% XXX: These replay messages should awaiting register acked?
handle_out(publish, Publishes, reset_timer(retry_timer, Timeout, NChannel))
end;
{ok, reset_timer(TimerName, Channel)};
handle_timeout(
_TRef,
expire_awaiting_rel,
@ -2105,20 +2072,23 @@ handle_timeout(
{ok, Channel};
handle_timeout(
_TRef,
expire_awaiting_rel,
expire_awaiting_rel = TimerName,
Channel = #channel{conn_state = asleep}
) ->
{ok, reset_timer(await_timer, Channel)};
{ok, reset_timer(TimerName, Channel)};
handle_timeout(
_TRef,
expire_awaiting_rel,
TimerName,
Channel = #channel{session = Session, clientinfo = ClientInfo}
) ->
case emqx_mqttsn_session:expire(ClientInfo, awaiting_rel, Session) of
{ok, NSession} ->
{ok, clean_timer(await_timer, Channel#channel{session = NSession})};
{ok, Timeout, NSession} ->
{ok, reset_timer(await_timer, Timeout, Channel#channel{session = NSession})}
) when TimerName == retry_delivery; TimerName == expire_awaiting_rel ->
case emqx_mqttsn_session:handle_timeout(ClientInfo, TimerName, Session) of
{ok, Publishes, NSession} ->
NChannel = Channel#channel{session = NSession},
handle_out(publish, Publishes, clean_timer(TimerName, NChannel));
{ok, Publishes, Timeout, NSession} ->
NChannel = Channel#channel{session = NSession},
%% XXX: These replay messages should awaiting register acked?
handle_out(publish, Publishes, reset_timer(TimerName, Timeout, NChannel))
end;
handle_timeout(
_TRef,
@ -2210,7 +2180,7 @@ ensure_asleep_timer(Channel = #channel{asleep_timer_duration = Duration}) when
ensure_asleep_timer(Durtion, Channel) ->
ensure_timer(
asleep_timer,
expire_asleep,
timer:seconds(Durtion),
Channel#channel{asleep_timer_duration = Durtion}
).
@ -2219,9 +2189,8 @@ ensure_register_timer(Channel) ->
ensure_register_timer(0, Channel).
ensure_register_timer(RetryTimes, Channel = #channel{timers = Timers}) ->
Msg = maps:get(register_timer, ?TIMER_TABLE),
TRef = emqx_utils:start_timer(?REGISTER_TIMEOUT, {Msg, RetryTimes}),
Channel#channel{timers = Timers#{register_timer => TRef}}.
TRef = emqx_utils:start_timer(?REGISTER_TIMEOUT, {retry_register, RetryTimes}),
Channel#channel{timers = Timers#{retry_register => TRef}}.
cancel_timer(Name, Channel = #channel{timers = Timers}) ->
case maps:get(Name, Timers, undefined) of
@ -2242,8 +2211,7 @@ ensure_timer(Name, Channel = #channel{timers = Timers}) ->
end.
ensure_timer(Name, Time, Channel = #channel{timers = Timers}) ->
Msg = maps:get(Name, ?TIMER_TABLE),
TRef = emqx_utils:start_timer(Time, Msg),
TRef = emqx_utils:start_timer(Time, Name),
Channel#channel{timers = Timers#{Name => TRef}}.
reset_timer(Name, Channel) ->
@ -2255,13 +2223,12 @@ reset_timer(Name, Time, Channel) ->
clean_timer(Name, Channel = #channel{timers = Timers}) ->
Channel#channel{timers = maps:remove(Name, Timers)}.
interval(alive_timer, #channel{keepalive = KeepAlive}) ->
interval(keepalive, #channel{keepalive = KeepAlive}) ->
emqx_keepalive:info(interval, KeepAlive);
interval(retry_timer, #channel{session = Session}) ->
interval(retry_delivery, #channel{session = Session}) ->
emqx_mqttsn_session:info(retry_interval, Session);
interval(await_timer, #channel{session = Session}) ->
interval(expire_awaiting_rel, #channel{session = Session}) ->
emqx_mqttsn_session:info(await_rel_timeout, Session).
%%--------------------------------------------------------------------
%% Helper functions
%%--------------------------------------------------------------------

View File

@ -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 puback
%% for publish / pubrec / pubcomp / deliver
{ok, ResultReplies, Sess1} ->
{ok, ResultReplies, Session#{session := Sess1}};
%% for puback / handle_timeout
{ok, Msgs, Replies, Sess1} ->
{ok, Msgs, Replies, Session#{session := Sess1}}
{ok, Msgs, Replies, Session#{session := Sess1}};
%% for any errors
{error, Reason} ->
{error, Reason}
end.

View File

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