Merge pull request #403 from emqtt/0.14

0.14 - New PubSub, Router Design
This commit is contained in:
Feng Lee 2015-12-10 15:30:41 +08:00
commit 82119afd88
73 changed files with 1852 additions and 1416 deletions

20
TODO Normal file
View File

@ -0,0 +1,20 @@
DONE TODO 1. refactor gproc_pool usage
DONE TODO 2. emqttd_router, emqttd_pubsub to route message
DONE TODO 3. sup, pool_sup, manager......
DONE TODO 4. route ageing...
TODO 5. dashboard
TODO 6. emqttd_ctl
DONE TODO 7. transaction on route, and topic?
TODO 8. topics, subscriptions CLI
DONE TODO 9. LOG.....
DONE TODO 10. emqttd_sm.erl to remove mnesia:index_read...

7
doc/pool.md Normal file
View File

@ -0,0 +1,7 @@
sup(one_for_all)
manager
pool_sup(one_for_one)
worker1
worker2
...
workerN

View File

@ -60,26 +60,15 @@
-type mqtt_topic() :: #mqtt_topic{}.
%%------------------------------------------------------------------------------
%% MQTT Subscriber
%% MQTT Subscription
%%------------------------------------------------------------------------------
-record(mqtt_subscriber, {
-record(mqtt_subscription, {
subid :: binary() | atom(),
topic :: binary(),
subpid :: pid(),
qos = 0 :: 0 | 1 | 2
}).
-type mqtt_subscriber() :: #mqtt_subscriber{}.
%%------------------------------------------------------------------------------
%% P2P Queue Subscriber
%%------------------------------------------------------------------------------
-record(mqtt_queue, {
name :: binary(),
qpid :: pid(),
qos = 0 :: 0 | 1 | 2
}).
-type mqtt_queue() :: #mqtt_queue{}.
-type mqtt_subscription() :: #mqtt_subscription{}.
%%------------------------------------------------------------------------------
%% MQTT Client

View File

@ -19,12 +19,18 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% MQTT Internal Header.
%%% @doc Internal Header File
%%%
%%% @end
%%%-----------------------------------------------------------------------------
-define(GPROC_POOL(JoinOrLeave, Pool, I),
(begin
case JoinOrLeave of
join -> gproc_pool:connect_worker(Pool, {Pool, Id});
leave -> gproc_pool:disconnect_worker(Pool, {Pool, I})
end
end)).
-define(record_to_proplist(Def, Rec),
lists:zip(record_info(fields, Def),
tl(tuple_to_list(Rec)))).
@ -33,3 +39,21 @@
[{K, V} || {K, V} <- ?record_to_proplist(Def, Rec),
lists:member(K, Fields)]).
-define(UNEXPECTED_REQ(Req, State),
(begin
lager:error("Unexpected Request: ~p", [Req]),
{reply, {error, unexpected_request}, State}
end)).
-define(UNEXPECTED_MSG(Msg, State),
(begin
lager:error("Unexpected Message: ~p", [Msg]),
{noreply, State}
end)).
-define(UNEXPECTED_INFO(Info, State),
(begin
lager:error("Unexpected Info: ~p", [Info]),
{noreply, State}
end)).

View File

@ -17,6 +17,8 @@
{crash_log, "log/emqttd_crash.log"},
{handlers, [
{lager_console_backend, info},
%%NOTICE: Level >= error
%%{lager_emqtt_backend, error},
{lager_file_backend, [
{formatter_config, [time, " ", pid, " [",severity,"] ", message, "\n"]},
{file, "log/emqttd_info.log"},
@ -141,11 +143,25 @@
%% Max Payload Size of retained message
{max_playload_size, 65536}
]},
%% PubSub
%% PubSub and Router
{pubsub, [
%% default should be scheduler numbers
%% {pool_size, 8}
%% Default should be scheduler numbers
%% {pool_size, 8},
%% Subscription: disc | ram | false
{subscription, ram},
%% Route shard
{route_shard, false},
%% Route delay, false | integer
{route_delay, false},
%% Route aging time(seconds)
{route_aging, 5}
]},
%% Bridge
{bridge, [
%%TODO: bridge queue size
@ -262,12 +278,11 @@
%% Erlang System Monitor
{sysmon, [
%% Long GC
{long_gc, 100},
%% Long Schedule(ms)
{long_schedule, 50},
{long_schedule, 100},
%% 8M words. 32MB on 32-bit VM, 64MB on 64-bit VM.
%% 8 * 1024 * 1024

View File

@ -17,6 +17,8 @@
{crash_log, "log/emqttd_crash.log"},
{handlers, [
%%{lager_console_backend, info},
%%NOTICE: Level >= error
%%{lager_emqtt_backend, error},
{lager_file_backend, [
{formatter_config, [time, " ", pid, " [",severity,"] ", message, "\n"]},
{file, "log/emqttd_error.log"},
@ -133,11 +135,25 @@
%% Max Payload Size of retained message
{max_playload_size, 65536}
]},
%% PubSub
%% PubSub and Router
{pubsub, [
%% default should be scheduler numbers
%% {pool_size, 8}
%% Default should be scheduler numbers
%% {pool_size, 8},
%% Subscription: disc | ram | false
{subscription, ram},
%% Route shard
{route_shard, false},
%% Route delay, false | integer
{route_delay, false},
%% Route aging time(seconds)
{route_aging, 5}
]},
%% Bridge
{bridge, [
%%TODO: bridge queue size
@ -254,20 +270,19 @@
%% Erlang System Monitor
{sysmon, [
%% Long GC, don't monitor in production mode for:
%% https://github.com/erlang/otp/blob/feb45017da36be78d4c5784d758ede619fa7bfd3/erts/emulator/beam/erl_gc.c#L421
{long_gc, false},
%% Long Schedule(ms)
{long_schedule, 100},
{long_schedule, 240},
%% 8M words. 32MB on 32-bit VM, 64MB on 64-bit VM.
%% 8 * 1024 * 1024
{large_heap, 8388608},
%% Busy Port
{busy_port, true},
{busy_port, false},
%% Busy Dist Port
{busy_dist_port, true}

View File

@ -34,6 +34,9 @@
## Valid range is 1-2097151. Default is 1024.
## +zdbbl 8192
## CPU Schedulers
## +sbt db
##-------------------------------------------------------------------------
## Env
##-------------------------------------------------------------------------

View File

@ -1,14 +1,15 @@
{application, emqttd,
[
{id, "emqttd"},
{vsn, "0.13.1"},
{vsn, "0.14.0"},
{description, "Erlang MQTT Broker"},
{modules, []},
{registered, []},
{applications, [kernel,
stdlib,
gproc,
esockd]},
esockd,
mochiweb]},
{mod, {emqttd_app, []}},
{env, []}
]}.

View File

@ -19,17 +19,14 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd main module.
%%% @doc emqttd main module.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd).
-author("Feng Lee <feng@emqtt.io>").
-export([start/0, env/1, env/2,
open_listeners/1, close_listeners/1,
start_listeners/0, stop_listeners/0,
load_all_mods/0, is_mod_enabled/1,
is_running/1]).
@ -38,8 +35,9 @@
{packet, raw},
{reuseaddr, true},
{backlog, 512},
{nodelay, true}
]).
{nodelay, true}]).
-define(APP, ?MODULE).
-type listener() :: {atom(), inet:port_number(), [esockd:option()]}.
@ -49,7 +47,7 @@
%%------------------------------------------------------------------------------
-spec start() -> ok | {error, any()}.
start() ->
application:start(emqttd).
application:start(?APP).
%%------------------------------------------------------------------------------
%% @doc Get environment
@ -57,39 +55,41 @@ start() ->
%%------------------------------------------------------------------------------
-spec env(atom()) -> list().
env(Group) ->
application:get_env(emqttd, Group, []).
application:get_env(?APP, Group, []).
-spec env(atom(), atom()) -> undefined | any().
env(Group, Name) ->
proplists:get_value(Name, env(Group)).
%%------------------------------------------------------------------------------
%% @doc Open Listeners
%% @doc Start Listeners
%% @end
%%------------------------------------------------------------------------------
-spec open_listeners([listener()]) -> any().
open_listeners(Listeners) when is_list(Listeners) ->
[open_listener(Listener) || Listener <- Listeners].
-spec start_listeners() -> any().
start_listeners() ->
{ok, Listeners} = application:get_env(?APP, listeners),
lists:foreach(fun start_listener/1, Listeners).
%% open mqtt port
open_listener({mqtt, Port, Options}) ->
open_listener(mqtt, Port, Options);
%% Start mqtt listener
-spec start_listener(listener()) -> any().
start_listener({mqtt, Port, Options}) ->
start_listener(mqtt, Port, Options);
%% open mqtt(SSL) port
open_listener({mqtts, Port, Options}) ->
open_listener(mqtts, Port, Options);
%% Start mqtt(SSL) listener
start_listener({mqtts, Port, Options}) ->
start_listener(mqtts, Port, Options);
%% open http port
open_listener({http, Port, Options}) ->
%% Start http listener
start_listener({http, Port, Options}) ->
MFArgs = {emqttd_http, handle_request, []},
mochiweb:start_http(Port, Options, MFArgs);
%% open https port
open_listener({https, Port, Options}) ->
%% Start https listener
start_listener({https, Port, Options}) ->
MFArgs = {emqttd_http, handle_request, []},
mochiweb:start_http(Port, Options, MFArgs).
open_listener(Protocol, Port, Options) ->
start_listener(Protocol, Port, Options) ->
MFArgs = {emqttd_client, start_link, [env(mqtt)]},
esockd:open(Protocol, Port, merge_sockopts(Options) , MFArgs).
@ -99,22 +99,25 @@ merge_sockopts(Options) ->
emqttd_opts:merge(Options, [{sockopts, SockOpts}]).
%%------------------------------------------------------------------------------
%% @doc Close Listeners
%% @doc Stop Listeners
%% @end
%%------------------------------------------------------------------------------
-spec close_listeners([listener()]) -> any().
close_listeners(Listeners) when is_list(Listeners) ->
[close_listener(Listener) || Listener <- Listeners].
stop_listeners() ->
{ok, Listeners} = application:get_env(?APP, listeners),
lists:foreach(fun stop_listener/1, Listeners).
close_listener({Protocol, Port, _Options}) ->
stop_listener({Protocol, Port, _Options}) ->
esockd:close({Protocol, Port}).
load_all_mods() ->
lists:foreach(fun({Name, Opts}) ->
lists:foreach(fun load_mod/1, env(modules)).
load_mod({Name, Opts}) ->
Mod = list_to_atom("emqttd_mod_" ++ atom_to_list(Name)),
Mod:load(Opts),
lager:info("load module ~s successfully", [Name])
end, env(modules)).
case catch Mod:load(Opts) of
{ok, _State} -> lager:info("load module ~s successfully", [Name]);
{'EXIT', Reason} -> lager:error("load module ~s error: ~p", [Name, Reason])
end.
is_mod_enabled(Name) ->
env(modules, Name) =/= undefined.
@ -124,7 +127,7 @@ is_mod_enabled(Name) ->
%% @end
%%------------------------------------------------------------------------------
is_running(Node) ->
case rpc:call(Node, erlang, whereis, [emqttd]) of
case rpc:call(Node, erlang, whereis, [?APP]) of
{badrpc, _} -> false;
undefined -> false;
Pid when is_pid(Pid) -> true

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd authentication and ACL server.
%%% @doc Authentication and ACL Control Server
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_access_control).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-behaviour(gen_server).
@ -62,9 +59,9 @@
start_link() ->
start_link(emqttd:env(access)).
-spec start_link(AcOpts :: list()) -> {ok, pid()} | ignore | {error, any()}.
start_link(AcOpts) ->
gen_server:start_link({local, ?SERVER}, ?MODULE, [AcOpts], []).
-spec start_link(Opts :: list()) -> {ok, pid()} | ignore | {error, any()}.
start_link(Opts) ->
gen_server:start_link({local, ?SERVER}, ?MODULE, [Opts], []).
%%------------------------------------------------------------------------------
%% @doc Authenticate MQTT Client
@ -76,10 +73,11 @@ auth(Client, Password) when is_record(Client, mqtt_client) ->
auth(_Client, _Password, []) ->
{error, "No auth module to check!"};
auth(Client, Password, [{Mod, State, _Seq} | Mods]) ->
case Mod:check(Client, Password, State) of
case catch Mod:check(Client, Password, State) of
ok -> ok;
ignore -> auth(Client, Password, Mods);
{error, Reason} -> {error, Reason};
ignore -> auth(Client, Password, Mods)
{'EXIT', Error} -> {error, Error}
end.
%%------------------------------------------------------------------------------
@ -117,7 +115,7 @@ reload_acl() ->
%% @doc Register authentication or ACL module
%% @end
%%------------------------------------------------------------------------------
-spec register_mod(Type :: auth | acl, Mod :: atom(), Opts :: list()) -> ok | {error, any()}.
-spec register_mod(auth | acl, atom(), list()) -> ok | {error, any()}.
register_mod(Type, Mod, Opts) when Type =:= auth; Type =:= acl->
register_mod(Type, Mod, Opts, 0).
@ -143,10 +141,9 @@ lookup_mods(Type) ->
[] -> [];
[{_, Mods}] -> Mods
end.
tab_key(auth) ->
auth_modules;
tab_key(acl) ->
acl_modules.
tab_key(auth) -> auth_modules;
tab_key(acl) -> acl_modules.
%%------------------------------------------------------------------------------
%% @doc Stop access control server
@ -159,10 +156,11 @@ stop() ->
%%% gen_server callbacks
%%%=============================================================================
init([AcOpts]) ->
init([Opts]) ->
ets:new(?ACCESS_CONTROL_TAB, [set, named_table, protected, {read_concurrency, true}]),
ets:insert(?ACCESS_CONTROL_TAB, {auth_modules, init_mods(auth, proplists:get_value(auth, AcOpts))}),
ets:insert(?ACCESS_CONTROL_TAB, {acl_modules, init_mods(acl, proplists:get_value(acl, AcOpts))}),
ets:insert(?ACCESS_CONTROL_TAB, {auth_modules, init_mods(auth, proplists:get_value(auth, Opts))}),
ets:insert(?ACCESS_CONTROL_TAB, {acl_modules, init_mods(acl, proplists:get_value(acl, Opts))}),
{ok, state}.
init_mods(auth, AuthMods) ->
@ -233,8 +231,11 @@ code_change(_OldVsn, State, _Extra) ->
%%%=============================================================================
authmod(Name) when is_atom(Name) ->
list_to_atom(lists:concat(["emqttd_auth_", Name])).
mod(emqttd_auth_, Name).
aclmod(Name) when is_atom(Name) ->
list_to_atom(lists:concat(["emqttd_acl_", Name])).
mod(emqttd_acl_, Name).
mod(Prefix, Name) ->
list_to_atom(lists:concat([Prefix, Name])).

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd ACL rule.
%%% @doc emqttd ACL Rule
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_access_rule).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-type who() :: all | binary() |

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% Internal ACL that load rules from etc/acl.config
%%% @doc Internal ACL that load rules from etc/acl.config
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_acl_internal).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-export([all_rules/0]).
@ -142,6 +139,5 @@ reload_acl(State) ->
%% @end
%%------------------------------------------------------------------------------
-spec description() -> string().
description() ->
"Internal ACL with etc/acl.config".
description() -> "Internal ACL with etc/acl.config".

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% ACL module behaviour.
%%% @doc ACL module behaviour
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_acl_mod).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
%%%=============================================================================

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% copy alarm_handler.
%%% @doc Copy alarm_handler
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_alarm).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-behaviour(gen_event).
@ -111,7 +108,7 @@ handle_event({set_alarm, Alarm = #mqtt_alarm{id = AlarmId,
handle_event({clear_alarm, AlarmId}, Alarms) ->
Json = mochijson2:encode([{id, AlarmId}, {ts, emqttd_util:now_to_secs()}]),
emqttd_pubsub:publish(alarm_msg(clear, AlarmId, Json)),
{ok, lists:keydelete(AlarmId, 2, Alarms)};
{ok, lists:keydelete(AlarmId, 2, Alarms), hibernate};
handle_event(_, Alarms)->
{ok, Alarms}.

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd application.
%%% @doc emqttd application.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_app).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd_cli.hrl").
-behaviour(application).
@ -52,7 +49,7 @@ start(_StartType, _StartArgs) ->
emqttd_cli:load(),
emqttd:load_all_mods(),
emqttd_plugins:load(),
start_listeners(),
emqttd:start_listeners(),
register(emqttd, self()),
print_vsn(),
{ok, Sup}.
@ -65,18 +62,14 @@ print_vsn() ->
{ok, Desc} = application:get_key(description),
?PRINT("~s ~s is running now~n", [Desc, Vsn]).
start_listeners() ->
{ok, Listeners} = application:get_env(listeners),
emqttd:open_listeners(Listeners).
start_servers(Sup) ->
Servers = [{"emqttd ctl", emqttd_ctl},
{"emqttd trace", emqttd_trace},
{"emqttd trace", {supervisor, emqttd_trace_sup}},
{"emqttd pubsub", {supervisor, emqttd_pubsub_sup}},
{"emqttd stats", emqttd_stats},
{"emqttd metrics", emqttd_metrics},
{"emqttd retained", emqttd_retained},
{"emqttd pooler", {supervisor, emqttd_pooler_sup}},
{"emqttd retainer", emqttd_retainer},
{"emqttd pooler", {supervisor, emqttd_pooler}},
{"emqttd client manager", {supervisor, emqttd_cm_sup}},
{"emqttd session manager", {supervisor, emqttd_sm_sup}},
{"emqttd session supervisor", {supervisor, emqttd_session_sup}},
@ -85,7 +78,7 @@ start_servers(Sup) ->
{"emqttd mod supervisor", emqttd_mod_sup},
{"emqttd bridge supervisor", {supervisor, emqttd_bridge_sup}},
{"emqttd access control", emqttd_access_control},
{"emqttd system monitor", emqttd_sysmon, emqttd:env(sysmon)}],
{"emqttd system monitor", {supervisor, emqttd_sysmon_sup}}],
[start_server(Sup, Server) || Server <- Servers].
start_server(_Sup, {Name, F}) when is_function(F) ->
@ -135,15 +128,5 @@ worker_spec(M, F, A) ->
-spec stop(State :: term()) -> term().
stop(_State) ->
stop_listeners().
stop_listeners() ->
%% ensure that esockd applications is started?
case lists:keyfind(esockd, 1, application:which_applications()) of
false ->
ignore;
_Tuple ->
{ok, Listeners} = application:get_env(listeners),
emqttd:close_listeners(Listeners)
end.
catch emqttd:stop_listeners().

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% Anonymous authentication module.
%%% @doc Anonymous Authentication Module
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_auth_anonymous).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(emqttd_auth_mod).
-export([init/1, check/3, description/0]).

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% ClientId Authentication Module.
%%% @doc ClientId Authentication Module
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_auth_clientid).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-export([add_clientid/1, add_clientid/2,

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% LDAP Authentication Module
%%% @doc LDAP Authentication Module
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_auth_ldap).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-import(proplists, [get_value/2, get_value/3]).
@ -84,6 +81,5 @@ ldap_bind(LDAP, UserDn, Password) ->
fill(Username, UserDn) ->
re:replace(UserDn, "\\$u", Username, [global, {return, list}]).
description() ->
"LDAP Authentication Module".
description() -> "LDAP Authentication Module".

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd authentication behaviour.
%%% @doc emqttd Authentication Behaviour
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_auth_mod).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
%%%=============================================================================

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% Authentication with username and password.
%%% @doc Authentication with username and password
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_auth_username).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-include("emqttd_cli.hrl").

View File

@ -19,19 +19,18 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd bridge.
%%% @doc emqttd bridge
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_bridge).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-include("emqttd_protocol.hrl").
-include("emqttd_internal.hrl").
%% API Function Exports
-export([start_link/3]).
@ -85,7 +84,7 @@ init([Node, SubTopic, Options]) ->
MQueue = emqttd_mqueue:new(qname(Node, SubTopic),
[{max_len, State#state.max_queue_len}],
emqttd_alarm:alarm_fun()),
emqttd_pubsub:subscribe(SubTopic, State#state.qos),
emqttd_pubsub:subscribe({SubTopic, State#state.qos}),
{ok, State#state{mqueue = MQueue}};
false ->
{stop, {cannot_connect, Node}}
@ -111,23 +110,23 @@ qname(Node, SubTopic) when is_atom(Node) ->
qname(Node, SubTopic) ->
list_to_binary(["Bridge:", Node, ":", SubTopic]).
handle_call(_Request, _From, State) ->
{reply, error, State}.
handle_call(Req, _From, State) ->
?UNEXPECTED_REQ(Req, State).
handle_cast(_Msg, State) ->
{noreply, State}.
handle_cast(Msg, State) ->
?UNEXPECTED_MSG(Msg, State).
handle_info({dispatch, Msg}, State = #state{mqueue = MQ, status = down}) ->
handle_info({dispatch, _Topic, Msg}, State = #state{mqueue = MQ, status = down}) ->
{noreply, State#state{mqueue = emqttd_mqueue:in(Msg, MQ)}};
handle_info({dispatch, Msg}, State = #state{node = Node, status = up}) ->
handle_info({dispatch, _Topic, Msg}, State = #state{node = Node, status = up}) ->
rpc:cast(Node, emqttd_pubsub, publish, [transform(Msg, State)]),
{noreply, State};
{noreply, State, hibernate};
handle_info({nodedown, Node}, State = #state{node = Node, ping_down_interval = Interval}) ->
lager:warning("Bridge Node Down: ~p", [Node]),
erlang:send_after(Interval, self(), ping_down_node),
{noreply, State#state{status = down}};
{noreply, State#state{status = down}, hibernate};
handle_info({nodeup, Node}, State = #state{node = Node}) ->
%% TODO: Really fast??
@ -156,8 +155,7 @@ handle_info({'EXIT', _Pid, normal}, State) ->
{noreply, State};
handle_info(Info, State) ->
lager:error("Unexpected Info: ~p", [Info]),
{noreply, State}.
?UNEXPECTED_INFO(Info, State).
terminate(_Reason, _State) ->
ok.

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd bridge supervisor.
%%% @doc Bridge Supervisor
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_bridge_sup).
-author("Feng Lee <feng@emqtt.io>").
-behavior(supervisor).
-export([start_link/0,

View File

@ -19,17 +19,18 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd broker.
%%% @doc emqttd broker
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_broker).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(gen_server).
-include("emqttd.hrl").
-include("emqttd_internal.hrl").
%% API Function Exports
-export([start_link/0]).
@ -48,18 +49,16 @@
%% Tick API
-export([start_tick/1, stop_tick/1]).
-behaviour(gen_server).
-define(SERVER, ?MODULE).
%% gen_server Function Exports
-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
terminate/2, code_change/3]).
-define(BROKER_TAB, mqtt_broker).
-record(state, {started_at, sys_interval, heartbeat, tick_tref}).
-define(SERVER, ?MODULE).
-define(BROKER_TAB, mqtt_broker).
%% $SYS Topics of Broker
-define(SYSTOP_BROKERS, [
version, % Broker version
@ -219,10 +218,10 @@ stop_tick(TRef) ->
%%%=============================================================================
init([]) ->
random:seed(now()),
random:seed(os:timestamp()),
ets:new(?BROKER_TAB, [set, public, named_table]),
% Create $SYS Topics
emqttd_pubsub:create(<<"$SYS/brokers">>),
emqttd_pubsub:create(topic, <<"$SYS/brokers">>),
[ok = create_topic(Topic) || Topic <- ?SYSTOP_BROKERS],
% Tick
{ok, #state{started_at = os:timestamp(),
@ -258,11 +257,10 @@ handle_call({unhook, Hook, Name}, _From, State) ->
{reply, Reply, State};
handle_call(Req, _From, State) ->
lager:error("Unexpected request: ~p", [Req]),
{reply, {error, badreq}, State}.
?UNEXPECTED_REQ(Req, State).
handle_cast(_Msg, State) ->
{noreply, State}.
handle_cast(Msg, State) ->
?UNEXPECTED_MSG(Msg, State).
handle_info(heartbeat, State) ->
publish(uptime, list_to_binary(uptime(State))),
@ -273,10 +271,10 @@ handle_info(tick, State) ->
retain(brokers),
retain(version, list_to_binary(version())),
retain(sysdescr, list_to_binary(sysdescr())),
{noreply, State};
{noreply, State, hibernate};
handle_info(_Info, State) ->
{noreply, State}.
handle_info(Info, State) ->
?UNEXPECTED_INFO(Info, State).
terminate(_Reason, #state{heartbeat = Hb, tick_tref = TRef}) ->
stop_tick(Hb),
@ -291,7 +289,7 @@ code_change(_OldVsn, State, _Extra) ->
%%%=============================================================================
create_topic(Topic) ->
emqttd_pubsub:create(emqttd_topic:systop(Topic)).
emqttd_pubsub:create(topic, emqttd_topic:systop(Topic)).
retain(brokers) ->
Payload = list_to_binary(string:join([atom_to_list(N) || N <- running_nodes()], ",")),

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd cli.
%%% @doc emqttd cli
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_cli).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-include("emqttd_cli.hrl").
@ -42,6 +39,8 @@
clients/1, sessions/1, plugins/1, listeners/1,
vm/1, mnesia/1, trace/1]).
%% TODO: topics, subscriptions...
-define(PROC_INFOKEYS, [status,
memory,
message_queue_len,
@ -50,6 +49,8 @@
stack_size,
reductions]).
-define(APP, emqttd).
load() ->
Cmds = [Fun || {Fun, _} <- ?MODULE:module_info(exports), is_cmd(Fun)],
[emqttd_ctl:register_cmd(Cmd, {?MODULE, Cmd}, []) || Cmd <- Cmds].
@ -68,10 +69,10 @@ is_cmd(Fun) ->
status([]) ->
{InternalStatus, _ProvidedStatus} = init:get_status(),
?PRINT("Node ~p is ~p~n", [node(), InternalStatus]),
case lists:keysearch(emqttd, 1, application:which_applications()) of
case lists:keysearch(?APP, 1, application:which_applications()) of
false ->
?PRINT_MSG("emqttd is not running~n");
{value, {emqttd, _Desc, Vsn}} ->
{value, {?APP, _Desc, Vsn}} ->
?PRINT("emqttd ~s is running~n", [Vsn])
end;
status(_) ->
@ -132,13 +133,9 @@ cluster([SNode]) ->
false ->
cluster(Node, fun() ->
emqttd_plugins:unload(),
application:stop(emqttd),
application:stop(esockd),
application:stop(gproc),
stop_apps(),
emqttd_mnesia:cluster(Node),
application:start(gproc),
application:start(esockd),
application:start(emqttd)
start_apps()
end)
end;
@ -160,6 +157,12 @@ cluster(pong, Node, DoCluster) ->
cluster(pang, Node, _DoCluster) ->
?PRINT("Cannot connect to ~s~n", [Node]).
stop_apps() ->
[application:stop(App) || App <- [emqttd, esockd, gproc]].
start_apps() ->
[application:start(App) || App <- [gproc, esockd, emqttd]].
%%------------------------------------------------------------------------------
%% @doc Query clients
%% @end

View File

@ -19,14 +19,13 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% MQTT Client Connection.
%%% @doc MQTT Client Connection
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_client).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(gen_server).
-include("emqttd.hrl").
@ -34,12 +33,10 @@
-include("emqttd_internal.hrl").
-behaviour(gen_server).
%% API Function Exports
-export([start_link/2, session/1, info/1, kick/1]).
%% SUB/UNSUB Asynchronously, called by plugins.
%% SUB/UNSUB Asynchronously. Called by plugins.
-export([subscribe/2, unsubscribe/2]).
%% gen_server Function Exports
@ -131,8 +128,7 @@ handle_call(kick, _From, State) ->
{stop, {shutdown, kick}, ok, State};
handle_call(Req, _From, State) ->
?LOG(critical, "Unexpected request: ~p", [Req], State),
{reply, {error, unsupported_request}, State}.
?UNEXPECTED_REQ(Req, State).
handle_cast({subscribe, TopicTable}, State) ->
with_session(fun(SessPid) ->
@ -145,8 +141,7 @@ handle_cast({unsubscribe, Topics}, State) ->
end, State);
handle_cast(Msg, State) ->
?LOG(critical, "Unexpected msg: ~p", [Msg], State),
noreply(State).
?UNEXPECTED_MSG(Msg, State).
handle_info(timeout, State) ->
shutdown(idle_timeout, State);
@ -214,8 +209,7 @@ handle_info({keepalive, check}, State = #client_state{keepalive = KeepAlive}) ->
end;
handle_info(Info, State) ->
?LOG(critical, "Unexpected info: ~p", [Info], State),
noreply(State).
?UNEXPECTED_INFO(Info, State).
terminate(Reason, #client_state{connection = Connection,
keepalive = KeepAlive,
@ -246,7 +240,7 @@ with_session(Fun, State = #client_state{proto_state = ProtoState}) ->
Fun(emqttd_protocol:session(ProtoState)),
hibernate(State).
%% receive and parse tcp data
%% Receive and parse tcp data
received(<<>>, State) ->
hibernate(State);

View File

@ -19,19 +19,18 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% MQTT Client Manager
%%% @doc MQTT Client Manager
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_cm).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-include("emqttd_internal.hrl").
%% API Exports
-export([start_link/2, pool/0]).
-export([start_link/3]).
-export([lookup/1, lookup_proc/1, register/1, unregister/1]).
@ -44,28 +43,27 @@
%% gen_server2 priorities
-export([prioritise_call/4, prioritise_cast/3, prioritise_info/3]).
-record(state, {id, statsfun, monitors}).
-record(state, {pool, id, statsfun, monitors}).
-define(CM_POOL, ?MODULE).
-define(POOL, ?MODULE).
%%%=============================================================================
%%% API
%%%=============================================================================
%%------------------------------------------------------------------------------
%% @doc Start client manager
%% @doc Start Client Manager
%% @end
%%------------------------------------------------------------------------------
-spec start_link(Id, StatsFun) -> {ok, pid()} | ignore | {error, any()} when
-spec start_link(Pool, Id, StatsFun) -> {ok, pid()} | ignore | {error, any()} when
Pool :: atom(),
Id :: pos_integer(),
StatsFun :: fun().
start_link(Id, StatsFun) ->
gen_server2:start_link(?MODULE, [Id, StatsFun], []).
pool() -> ?CM_POOL.
start_link(Pool, Id, StatsFun) ->
gen_server2:start_link(?MODULE, [Pool, Id, StatsFun], []).
%%------------------------------------------------------------------------------
%% @doc Lookup client by clientId
%% @doc Lookup Client by ClientId
%% @end
%%------------------------------------------------------------------------------
-spec lookup(ClientId :: binary()) -> mqtt_client() | undefined.
@ -81,19 +79,18 @@ lookup(ClientId) when is_binary(ClientId) ->
%%------------------------------------------------------------------------------
-spec lookup_proc(ClientId :: binary()) -> pid() | undefined.
lookup_proc(ClientId) when is_binary(ClientId) ->
try ets:lookup_element(mqtt_client, ClientId, #mqtt_client.client_pid) of
Pid -> Pid
try ets:lookup_element(mqtt_client, ClientId, #mqtt_client.client_pid)
catch
error:badarg -> undefined
end.
%%------------------------------------------------------------------------------
%% @doc Register clientId with pid.
%% @doc Register ClientId with Pid.
%% @end
%%------------------------------------------------------------------------------
-spec register(Client :: mqtt_client()) -> ok.
register(Client = #mqtt_client{client_id = ClientId}) ->
CmPid = gproc_pool:pick_worker(?CM_POOL, ClientId),
CmPid = gproc_pool:pick_worker(?POOL, ClientId),
gen_server2:cast(CmPid, {register, Client}).
%%------------------------------------------------------------------------------
@ -102,16 +99,18 @@ register(Client = #mqtt_client{client_id = ClientId}) ->
%%------------------------------------------------------------------------------
-spec unregister(ClientId :: binary()) -> ok.
unregister(ClientId) when is_binary(ClientId) ->
CmPid = gproc_pool:pick_worker(?CM_POOL, ClientId),
CmPid = gproc_pool:pick_worker(?POOL, ClientId),
gen_server2:cast(CmPid, {unregister, ClientId, self()}).
%%%=============================================================================
%%% gen_server callbacks
%%%=============================================================================
init([Id, StatsFun]) ->
gproc_pool:connect_worker(?CM_POOL, {?MODULE, Id}),
{ok, #state{id = Id, statsfun = StatsFun, monitors = dict:new()}}.
init([Pool, Id, StatsFun]) ->
?GPROC_POOL(join, Pool, Id),
{ok, #state{pool = Pool, id = Id,
statsfun = StatsFun,
monitors = dict:new()}}.
prioritise_call(_Req, _From, _Len, _State) ->
1.
@ -127,8 +126,7 @@ prioritise_info(_Msg, _Len, _State) ->
3.
handle_call(Req, _From, State) ->
lager:error("Unexpected request: ~p", [Req]),
{reply, {error, unsupported_req}, State}.
?UNEXPECTED_REQ(Req, State).
handle_cast({register, Client = #mqtt_client{client_id = ClientId,
client_pid = Pid}}, State) ->
@ -150,8 +148,7 @@ handle_cast({unregister, ClientId, Pid}, State) ->
end;
handle_cast(Msg, State) ->
lager:error("Unexpected Msg: ~p", [Msg]),
{noreply, State}.
?UNEXPECTED_MSG(Msg, State).
handle_info({'DOWN', MRef, process, DownPid, _Reason}, State) ->
case dict:find(MRef, State#state.monitors) of
@ -169,12 +166,10 @@ handle_info({'DOWN', MRef, process, DownPid, _Reason}, State) ->
end;
handle_info(Info, State) ->
lager:error("Unexpected Info: ~p", [Info]),
{noreply, State}.
?UNEXPECTED_INFO(Info, State).
terminate(_Reason, #state{id = Id}) ->
gproc_pool:disconnect_worker(?CM_POOL, {?MODULE, Id}),
ok.
terminate(_Reason, #state{pool = Pool, id = Id}) ->
?GPROC_POOL(leave, Pool, Id), ok.
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
@ -183,12 +178,12 @@ code_change(_OldVsn, State, _Extra) ->
%%% Internal functions
%%%=============================================================================
monitor_client(ClientId, Pid, State = #state{monitors = Monintors}) ->
monitor_client(ClientId, Pid, State = #state{monitors = Monitors}) ->
MRef = erlang:monitor(process, Pid),
State#state{monitors = dict:store(MRef, {ClientId, Pid}, Monintors)}.
State#state{monitors = dict:store(MRef, {ClientId, Pid}, Monitors)}.
erase_monitor(MRef, State = #state{monitors = Monintors}) ->
State#state{monitors = dict:erase(MRef, Monintors)}.
erase_monitor(MRef, State = #state{monitors = Monitors}) ->
State#state{monitors = dict:erase(MRef, Monitors)}.
setstats(State = #state{statsfun = StatsFun}) ->
StatsFun(ets:info(mqtt_client, size)), State.

View File

@ -19,41 +19,45 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd client manager supervisor.
%%% @doc Client Manager Supervisor.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_cm_sup).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(supervisor).
-include("emqttd.hrl").
-behaviour(supervisor).
%% API
-export([start_link/0]).
%% Supervisor callbacks
-export([init/1]).
-define(CM, emqttd_cm).
-define(TAB, mqtt_client).
start_link() ->
supervisor:start_link({local, ?MODULE}, ?MODULE, []).
init([]) ->
ets:new(mqtt_client, [ordered_set, named_table, public,
{keypos, 2}, {write_concurrency, true}]),
Schedulers = erlang:system_info(schedulers),
gproc_pool:new(emqttd_cm:pool(), hash, [{size, Schedulers}]),
StatsFun = emqttd_stats:statsfun('clients/count', 'clients/max'),
Children = lists:map(
fun(I) ->
Name = {emqttd_cm, I},
gproc_pool:add_worker(emqttd_cm:pool(), Name, I),
{Name, {emqttd_cm, start_link, [I, StatsFun]},
permanent, 10000, worker, [emqttd_cm]}
end, lists:seq(1, Schedulers)),
{ok, {{one_for_all, 10, 100}, Children}}.
%% Create client table
create_client_tab(),
%% CM Pool Sup
MFA = {?CM, start_link, [emqttd_stats:statsfun('clients/count', 'clients/max')]},
PoolSup = emqttd_pool_sup:spec([?CM, hash, erlang:system_info(schedulers), MFA]),
{ok, {{one_for_all, 10, 3600}, [PoolSup]}}.
create_client_tab() ->
case ets:info(?TAB, name) of
undefined ->
ets:new(?TAB, [ordered_set, named_table, public,
{keypos, 2}, {write_concurrency, true}]);
_ ->
ok
end.

View File

@ -19,21 +19,18 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd control.
%%% @doc emqttd control
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_ctl).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(gen_server).
-include("emqttd.hrl").
-include("emqttd_cli.hrl").
-behaviour(gen_server).
-define(SERVER, ?MODULE).
%% API Function Exports

View File

@ -19,10 +19,9 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd distribution functions.
%%% @doc emqttd distribution functions
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_dist).

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd gen_mod behaviour
%%% @doc emqttd gen_mod behaviour
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_gen_mod).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-ifdef(use_specs).

View File

@ -34,11 +34,11 @@
%%% 4. Sequence: 2 bytes sequence in one process
%%%
%%% @end
%%%
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_guid).
-author("Feng Lee <feng@emqtt.io>").
-export([gen/0, new/0, timestamp/1]).
-define(MAX_SEQ, 16#FFFF).

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd http publish API and websocket client.
%%% @doc emqttd http publish API and websocket client.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_http).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-include("emqttd_protocol.hrl").
@ -44,7 +41,7 @@ handle_request('GET', "/status", Req) ->
AppStatus =
case lists:keysearch(emqttd, 1, application:which_applications()) of
false -> not_running;
{value, _Ver} -> running
{value, _Val} -> running
end,
Status = io_lib:format("Node ~s is ~s~nemqttd is ~s",
[node(), InternalStatus, AppStatus]),
@ -81,7 +78,7 @@ handle_request('POST', "/mqtt/publish", Req) ->
%% MQTT Over WebSocket
%%------------------------------------------------------------------------------
handle_request('GET', "/mqtt", Req) ->
lager:info("Websocket Connection from: ~s", [Req:get(peer)]),
lager:info("WebSocket Connection from: ~s", [Req:get(peer)]),
Upgrade = Req:get_header_value("Upgrade"),
Proto = Req:get_header_value("Sec-WebSocket-Protocol"),
case {is_websocket(Upgrade), Proto} of

View File

@ -21,12 +21,10 @@
%%%-----------------------------------------------------------------------------
%%% @doc client keepalive
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_keepalive).
-author("Feng Lee <feng@emqtt.io>").
-export([start/3, check/1, cancel/1]).
-record(keepalive, {statfun, statval,

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% MQTT Message Functions
%%% @doc MQTT Message Functions
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_message).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-include("emqttd_protocol.hrl").

View File

@ -19,21 +19,18 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd metrics. responsible for collecting broker metrics.
%%% @doc emqttd metrics. responsible for collecting broker metrics
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_metrics).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(gen_server).
-include("emqttd.hrl").
-include("emqttd_protocol.hrl").
-behaviour(gen_server).
-define(SERVER, ?MODULE).
%% API Function Exports
@ -286,14 +283,14 @@ key(counter, Metric) ->
%%%=============================================================================
init([]) ->
random:seed(now()),
random:seed(os:timestamp()),
Metrics = ?SYSTOP_BYTES ++ ?SYSTOP_PACKETS ++ ?SYSTOP_MESSAGES,
% Create metrics table
ets:new(?METRIC_TAB, [set, public, named_table, {write_concurrency, true}]),
% Init metrics
[create_metric(Metric) || Metric <- Metrics],
% $SYS Topics for metrics
[ok = emqttd_pubsub:create(metric_topic(Topic)) || {_, Topic} <- Metrics],
[ok = emqttd_pubsub:create(topic, metric_topic(Topic)) || {_, Topic} <- Metrics],
% Tick to publish metrics
{ok, #state{tick_tref = emqttd_broker:start_tick(tick)}, hibernate}.

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd mnesia.
%%% @doc emqttd mnesia
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_mnesia).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-export([start/0, cluster/1]).
@ -122,8 +119,7 @@ copy_table(Table) ->
%% @end
%%------------------------------------------------------------------------------
wait_for_tables() ->
%%TODO: is not right?
%%lager:info("local_tables: ~p", [mnesia:system_info(local_tables)]),
%% io:format("mnesia wait_for_tables: ~p~n", [mnesia:system_info(local_tables)]),
mnesia:wait_for_tables(mnesia:system_info(local_tables), infinity).
%%------------------------------------------------------------------------------

View File

@ -19,21 +19,18 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd auto subscribe module.
%%% @doc emqttd auto subscribe module.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_mod_autosub).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(emqttd_gen_mod).
-include("emqttd.hrl").
-include("emqttd_protocol.hrl").
-behaviour(emqttd_gen_mod).
-export([load/1, client_connected/3, unload/1]).
-record(state, {topics}).

View File

@ -19,19 +19,16 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd presence management module.
%%% @doc emqttd presence management module
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_mod_presence).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(emqttd_gen_mod).
-include("emqttd.hrl").
-behaviour(emqttd_gen_mod).
-export([load/1, unload/1]).
-export([client_connected/3, client_disconnected/3]).

View File

@ -19,19 +19,16 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd rewrite module.
%%% @doc emqttd rewrite module
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_mod_rewrite).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(emqttd_gen_mod).
-include("emqttd.hrl").
-behaviour(emqttd_gen_mod).
-export([load/1, reload/1, unload/1]).
-export([rewrite/3, rewrite/4]).

View File

@ -19,19 +19,16 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd module supervisor.
%%% @doc emqttd module supervisor.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_mod_sup).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(supervisor).
-include("emqttd.hrl").
-behaviour(supervisor).
%% API
-export([start_link/0, start_child/1, start_child/2]).

View File

@ -46,11 +46,11 @@
%%% otherwise dropped the oldest pending one.
%%%
%%% @end
%%%
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_mqueue).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-include("emqttd_protocol.hrl").

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd net utility functions. some functions copied from rabbitmq.
%%% @doc emqttd net utility functions. some functions copied from rabbitmq.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_net).
-author("Feng Lee <feng@emqtt.io>").
-include_lib("kernel/include/inet.hrl").
-export([tcp_name/3, tcp_host/1, getopts/2, setopts/2,

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd options handler.
%%% @doc emqttd options handler.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_opts).
-author("Feng Lee <feng@emqtt.io>").
-export([merge/2, g/2, g/3]).
%%------------------------------------------------------------------------------
@ -38,10 +35,8 @@ merge(Defaults, Options) ->
lists:foldl(
fun({Opt, Val}, Acc) ->
case lists:keymember(Opt, 1, Acc) of
true ->
lists:keyreplace(Opt, 1, Acc, {Opt, Val});
false ->
[{Opt, Val}|Acc]
true -> lists:keyreplace(Opt, 1, Acc, {Opt, Val});
false -> [{Opt, Val}|Acc]
end;
(Opt, Acc) ->
case lists:member(Opt, Acc) of

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% MQTT Packet Functions
%%% @doc MQTT Packet Functions
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_packet).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-include("emqttd_protocol.hrl").

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% MQTT Packet Parser.
%%% @doc MQTT Packet Parser
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_parser).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-include("emqttd_protocol.hrl").

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd plugins.
%%% @doc emqttd plugins.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_plugins).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-export([load/0, unload/0]).

77
src/emqttd_pool_sup.erl Normal file
View File

@ -0,0 +1,77 @@
%%%-----------------------------------------------------------------------------
%%% Copyright (c) 2012-2015 eMQTT.IO, All Rights Reserved.
%%%
%%% Permission is hereby granted, free of charge, to any person obtaining a copy
%%% of this software and associated documentation files (the "Software"), to deal
%%% in the Software without restriction, including without limitation the rights
%%% to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
%%% copies of the Software, and to permit persons to whom the Software is
%%% furnished to do so, subject to the following conditions:
%%%
%%% The above copyright notice and this permission notice shall be included in all
%%% copies or substantial portions of the Software.
%%%
%%% THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
%%% IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
%%% FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
%%% AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
%%% LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc Common Pool Supervisor
%%%
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_pool_sup).
-behaviour(supervisor).
%% API
-export([spec/1, spec/2, start_link/3, start_link/4]).
%% Supervisor callbacks
-export([init/1]).
-spec spec(list()) -> supervisor:child_spec().
spec(Args) ->
spec(pool_sup, Args).
-spec spec(any(), list()) -> supervisor:child_spec().
spec(ChildId, Args) ->
{ChildId, {?MODULE, start_link, Args},
transient, infinity, supervisor, [?MODULE]}.
-spec start_link(atom(), atom(), mfa()) -> {ok, pid()} | {error, any()}.
start_link(Pool, Type, MFA) ->
Schedulers = erlang:system_info(schedulers),
start_link(Pool, Type, Schedulers, MFA).
-spec start_link(atom(), atom(), pos_integer(), mfa()) -> {ok, pid()} | {error, any()}.
start_link(Pool, Type, Size, MFA) ->
supervisor:start_link({local, sup_name(Pool)}, ?MODULE, [Pool, Type, Size, MFA]).
sup_name(Pool) when is_atom(Pool) ->
list_to_atom(atom_to_list(Pool) ++ "_pool_sup").
init([Pool, Type, Size, {M, F, Args}]) ->
ensure_pool(Pool, Type, [{size, Size}]),
{ok, {{one_for_one, 10, 3600}, [
begin
ensure_pool_worker(Pool, {Pool, I}, I),
{{M, I}, {M, F, [Pool, I | Args]},
transient, 5000, worker, [M]}
end || I <- lists:seq(1, Size)]}}.
ensure_pool(Pool, Type, Opts) ->
try gproc_pool:new(Pool, Type, Opts)
catch
error:exists -> ok
end.
ensure_pool_worker(Pool, Name, Slot) ->
try gproc_pool:add_worker(Pool, Name, Slot)
catch
error:exists -> ok
end.

View File

@ -19,32 +19,41 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd pooler.
%%% @doc emqttd pooler.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_pooler).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(gen_server).
-include("emqttd_internal.hrl").
%% Start the pool supervisor
-export([start_link/0]).
%% API Exports
-export([start_link/1, submit/1, async_submit/1]).
-export([start_link/2, submit/1, async_submit/1]).
%% gen_server Function Exports
-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
terminate/2, code_change/3]).
-record(state, {id}).
-record(state, {pool, id}).
%%------------------------------------------------------------------------------
%% @doc Start Pooler Supervisor.
%% @end
%%------------------------------------------------------------------------------
start_link() ->
emqttd_pool_sup:start_link(pooler, random, {?MODULE, start_link, []}).
%%%=============================================================================
%%% API
%%%=============================================================================
-spec start_link(Id :: pos_integer()) -> {ok, pid()} | ignore | {error, any()}.
start_link(Id) ->
gen_server:start_link({local, name(Id)}, ?MODULE, [Id], []).
-spec start_link(atom(), pos_integer()) -> {ok, pid()} | ignore | {error, any()}.
start_link(Pool, Id) ->
gen_server:start_link({local, name(Id)}, ?MODULE, [Pool, Id], []).
name(Id) ->
list_to_atom(lists:concat([?MODULE, "_", integer_to_list(Id)])).
@ -54,22 +63,25 @@ name(Id) ->
%% @end
%%------------------------------------------------------------------------------
submit(Fun) ->
gen_server:call(gproc_pool:pick_worker(pooler), {submit, Fun}, infinity).
gen_server:call(worker(), {submit, Fun}, infinity).
%%------------------------------------------------------------------------------
%% @doc Submit work to pooler asynchronously
%% @end
%%------------------------------------------------------------------------------
async_submit(Fun) ->
gen_server:cast(gproc_pool:pick_worker(pooler), {async_submit, Fun}).
gen_server:cast(worker(), {async_submit, Fun}).
worker() ->
gproc_pool:pick_worker(pooler).
%%%=============================================================================
%%% gen_server callbacks
%%%=============================================================================
init([Id]) ->
gproc_pool:connect_worker(pooler, {pooler, Id}),
{ok, #state{id = Id}}.
init([Pool, Id]) ->
?GPROC_POOL(join, Pool, Id),
{ok, #state{pool = Pool, id = Id}}.
handle_call({submit, Fun}, _From, State) ->
{reply, run(Fun), State};
@ -90,8 +102,8 @@ handle_cast(_Msg, State) ->
handle_info(_Info, State) ->
{noreply, State}.
terminate(_Reason, #state{id = I}) ->
gproc_pool:disconnect_worker(pooler, {pooler, I}), ok.
terminate(_Reason, #state{pool = Pool, id = Id}) ->
?GPROC_POOL(leave, Pool, Id), ok.
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
@ -105,4 +117,3 @@ run({M, F, A}) ->
run(Fun) when is_function(Fun) ->
Fun().

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd protocol.
%%% @doc emqttd protocol.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_protocol).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-include("emqttd_protocol.hrl").
@ -263,7 +260,7 @@ send(Packet, State = #proto_state{sendfun = SendFun})
when is_record(Packet, mqtt_packet) ->
trace(send, Packet, State),
emqttd_metrics:sent(Packet),
Data = emqttd_serialiser:serialise(Packet),
Data = emqttd_serializer:serialize(Packet),
?LOG(debug, "SEND ~p", [Data], State),
emqttd_metrics:inc('bytes/sent', size(Data)),
SendFun(Data),

View File

@ -19,19 +19,20 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd pubsub.
%%% @doc emqttd pubsub
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_pubsub).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(gen_server2).
-include("emqttd.hrl").
-include("emqttd_protocol.hrl").
-include("emqttd_internal.hrl").
%% Mnesia Callbacks
-export([mnesia/1]).
@ -39,59 +40,82 @@
-copy_mnesia({mnesia, [copy]}).
%% API Exports
-export([start_link/2]).
-export([start_link/4]).
-export([create/1,
subscribe/1, subscribe/2,
unsubscribe/1,
publish/1]).
-export([create/2, subscribe/1, subscribe/2,
unsubscribe/1, unsubscribe/2, publish/1]).
%% Local node
-export([dispatch/2, match/1]).
-behaviour(gen_server2).
-export([match/1]).
%% gen_server Function Exports
-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
terminate/2, code_change/3]).
%% gen_server2 priorities
-export([prioritise_call/4, prioritise_cast/3, prioritise_info/3]).
-ifdef(TEST).
-compile(export_all).
-endif.
-define(POOL, pubsub).
-record(state, {pool, id, statsfun}).
-record(state, {id, submap :: map()}).
-define(ROUTER, emqttd_router).
-define(HELPER, emqttd_pubsub_helper).
%%%=============================================================================
%%% Mnesia callbacks
%%%=============================================================================
mnesia(boot) ->
%% p2p queue table
ok = emqttd_mnesia:create_table(queue, [
{type, set},
{ram_copies, [node()]},
{record_name, mqtt_queue},
{attributes, record_info(fields, mqtt_queue)}]),
%% topic table
ok = emqttd_mnesia:create_table(topic, [
{type, bag},
{ram_copies, [node()]},
{record_name, mqtt_topic},
{attributes, record_info(fields, mqtt_topic)}]),
%% local subscriber table, not shared with other nodes
ok = emqttd_mnesia:create_table(subscriber, [
{type, bag},
{ram_copies, [node()]},
{record_name, mqtt_subscriber},
{attributes, record_info(fields, mqtt_subscriber)},
{index, [subpid]},
{local_content, true}]);
ok = create_table(topic, ram_copies),
if_subscription(fun(RamOrDisc) ->
ok = create_table(subscription, RamOrDisc)
end);
mnesia(copy) ->
ok = emqttd_mnesia:copy_table(queue),
ok = emqttd_mnesia:copy_table(topic),
ok = emqttd_mnesia:copy_table(subscriber).
%% Only one disc_copy???
if_subscription(fun(_RamOrDisc) ->
ok = emqttd_mnesia:copy_table(subscription)
end).
%% Topic Table
create_table(topic, RamOrDisc) ->
emqttd_mnesia:create_table(topic, [
{type, bag},
{RamOrDisc, [node()]},
{record_name, mqtt_topic},
{attributes, record_info(fields, mqtt_topic)}]);
%% Subscription Table
create_table(subscription, RamOrDisc) ->
emqttd_mnesia:create_table(subscription, [
{type, bag},
{RamOrDisc, [node()]},
{record_name, mqtt_subscription},
{attributes, record_info(fields, mqtt_subscription)},
{storage_properties, [{ets, [compressed]},
{dets, [{auto_save, 5000}]}]}]).
if_subscription(Fun) ->
case env(subscription) of
disc -> Fun(disc_copies);
ram -> Fun(ram_copies);
false -> ok;
undefined -> ok
end.
env(Key) ->
case get({pubsub, Key}) of
undefined ->
cache_env(Key);
Val ->
Val
end.
cache_env(Key) ->
Val = emqttd_opts:g(Key, emqttd_broker:env(pubsub)),
put({pubsub, Key}, Val),
Val.
%%%=============================================================================
%%% API
@ -101,52 +125,64 @@ mnesia(copy) ->
%% @doc Start one pubsub server
%% @end
%%------------------------------------------------------------------------------
-spec start_link(Id, Opts) -> {ok, pid()} | ignore | {error, any()} when
-spec start_link(Pool, Id, StatsFun, Opts) -> {ok, pid()} | ignore | {error, any()} when
Pool :: atom(),
Id :: pos_integer(),
Opts :: list().
start_link(Id, Opts) ->
gen_server2:start_link({local, name(Id)}, ?MODULE, [Id, Opts], []).
StatsFun :: fun(),
Opts :: list(tuple()).
start_link(Pool, Id, StatsFun, Opts) ->
gen_server2:start_link({local, name(Id)}, ?MODULE, [Pool, Id, StatsFun, Opts], []).
name(Id) ->
list_to_atom("emqttd_pubsub_" ++ integer_to_list(Id)).
%%------------------------------------------------------------------------------
%% @doc Create topic. Notice That this transaction is not protected by pubsub pool
%% @doc Create Topic or Subscription.
%% @end
%%------------------------------------------------------------------------------
-spec create(Topic :: binary()) -> ok | {error, Error :: any()}.
create(<<"$Q/", _Queue/binary>>) ->
%% protecte from queue
{error, cannot_create_queue};
-spec create(topic | subscription, binary()) -> ok | {error, any()}.
create(topic, Topic) when is_binary(Topic) ->
Record = #mqtt_topic{topic = Topic, node = node()},
case mnesia:transaction(fun add_topic/1, [Record]) of
{atomic, ok} -> ok;
{aborted, Error} -> {error, Error}
end;
create(Topic) when is_binary(Topic) ->
TopicR = #mqtt_topic{topic = Topic, node = node()},
case mnesia:transaction(fun add_topic/1, [TopicR]) of
{atomic, ok} ->
setstats(topics), ok;
{aborted, Error} ->
{error, Error}
create(subscription, {SubId, Topic, Qos}) ->
case mnesia:transaction(fun add_subscription/2, [SubId, {Topic, Qos}]) of
{atomic, ok} -> ok;
{aborted, Error} -> {error, Error}
end.
%%------------------------------------------------------------------------------
%% @doc Subscribe topic
%% @doc Subscribe Topics
%% @end
%%------------------------------------------------------------------------------
-spec subscribe({Topic, Qos} | list({Topic, Qos})) ->
{ok, Qos | list(Qos)} | {error, any()} when
Topic :: binary(),
Qos :: mqtt_qos() | mqtt_qos_name().
subscribe({Topic, Qos}) when is_binary(Topic) andalso (?IS_QOS(Qos) orelse is_atom(Qos)) ->
call({subscribe, self(), Topic, ?QOS_I(Qos)});
subscribe({Topic, Qos}) ->
subscribe([{Topic, Qos}]);
subscribe(TopicTable) when is_list(TopicTable) ->
call({subscribe, {undefined, self()}, fixqos(TopicTable)}).
subscribe(Topics = [{_Topic, _Qos} | _]) ->
call({subscribe, self(), [{Topic, ?QOS_I(Qos)} || {Topic, Qos} <- Topics]}).
-spec subscribe(Topic, Qos) -> {ok, Qos} when
-spec subscribe(ClientId, {Topic, Qos} | list({Topic, Qos})) ->
{ok, Qos | list(Qos)} | {error, any()} when
ClientId :: binary(),
Topic :: binary(),
Qos :: mqtt_qos() | mqtt_qos_name().
subscribe(Topic, Qos) ->
subscribe({Topic, Qos}).
subscribe(ClientId, {Topic, Qos}) when is_binary(ClientId) ->
subscribe(ClientId, [{Topic, Qos}]);
subscribe(ClientId, TopicTable) when is_binary(ClientId) andalso is_list(TopicTable) ->
call({subscribe, {ClientId, self()}, fixqos(TopicTable)}).
fixqos(TopicTable) ->
[{Topic, ?QOS_I(Qos)} || {Topic, Qos} <- TopicTable].
call(Request) ->
PubSub = gproc_pool:pick_worker(pubsub, self()),
gen_server2:call(PubSub, Request, infinity).
%%------------------------------------------------------------------------------
%% @doc Unsubscribe Topic or Topics
@ -154,18 +190,19 @@ subscribe(Topic, Qos) ->
%%------------------------------------------------------------------------------
-spec unsubscribe(binary() | list(binary())) -> ok.
unsubscribe(Topic) when is_binary(Topic) ->
cast({unsubscribe, self(), Topic});
unsubscribe([Topic]);
unsubscribe(Topics = [Topic|_]) when is_binary(Topic) ->
cast({unsubscribe, self(), Topics}).
cast({unsubscribe, {undefined, self()}, Topics}).
call(Req) ->
Pid = gproc_pool:pick_worker(?POOL, self()),
gen_server2:call(Pid, Req, infinity).
-spec unsubscribe(binary(), binary() | list(binary())) -> ok.
unsubscribe(ClientId, Topic) when is_binary(ClientId) andalso is_binary(Topic) ->
unsubscribe(ClientId, [Topic]);
unsubscribe(ClientId, Topics = [Topic|_]) when is_binary(Topic) ->
cast({unsubscribe, {ClientId, self()}, Topics}).
cast(Msg) ->
Pid = gproc_pool:pick_worker(?POOL, self()),
gen_server2:cast(Pid, Msg).
PubSub = gproc_pool:pick_worker(pubsub, self()),
gen_server2:cast(PubSub, Msg).
%%------------------------------------------------------------------------------
%% @doc Publish to cluster nodes
@ -174,229 +211,107 @@ cast(Msg) ->
-spec publish(Msg :: mqtt_message()) -> ok.
publish(Msg = #mqtt_message{from = From}) ->
trace(publish, From, Msg),
Msg1 = #mqtt_message{topic = Topic}
Msg1 = #mqtt_message{topic = To}
= emqttd_broker:foldl_hooks('message.publish', [], Msg),
%% Retain message first. Don't create retained topic.
case emqttd_retained:retain(Msg1) of
case emqttd_retainer:retain(Msg1) of
ok ->
%% TODO: why unset 'retain' flag?
publish(Topic, emqttd_message:unset_flag(Msg1));
publish(To, emqttd_message:unset_flag(Msg1));
ignore ->
publish(Topic, Msg1)
publish(To, Msg1)
end.
publish(Queue = <<"$Q/", _/binary>>, Msg = #mqtt_message{qos = Qos}) ->
lists:foreach(
fun(#mqtt_queue{qpid = QPid, qos = SubQos}) ->
Msg1 = if
Qos > SubQos -> Msg#mqtt_message{qos = SubQos};
true -> Msg
end,
QPid ! {dispatch, Msg1}
end, mnesia:dirty_read(queue, Queue));
publish(Topic, Msg) when is_binary(Topic) ->
lists:foreach(fun(#mqtt_topic{topic=Name, node=Node}) ->
publish(To, Msg) ->
lists:foreach(fun(#mqtt_topic{topic = Topic, node = Node}) ->
case Node =:= node() of
true -> dispatch(Name, Msg);
false -> rpc:cast(Node, ?MODULE, dispatch, [Name, Msg])
true -> ?ROUTER:route(Topic, Msg);
false -> rpc:cast(Node, ?ROUTER, route, [Topic, Msg])
end
end, match(Topic)).
end, match(To)).
%%------------------------------------------------------------------------------
%% @doc Dispatch message locally. should only be called by publish.
%% @doc Match Topic Name with Topic Filters
%% @end
%%------------------------------------------------------------------------------
-spec dispatch(Topic :: binary(), Msg :: mqtt_message()) -> non_neg_integer().
dispatch(Topic, Msg = #mqtt_message{qos = Qos}) when is_binary(Topic) ->
Subscribers = mnesia:dirty_read(subscriber, Topic),
setstats(dropped, Subscribers =:= []),
lists:foreach(
fun(#mqtt_subscriber{subpid=SubPid, qos = SubQos}) ->
Msg1 = if
Qos > SubQos -> Msg#mqtt_message{qos = SubQos};
true -> Msg
end,
SubPid ! {dispatch, Msg1}
end, Subscribers),
length(Subscribers).
-spec match(Topic :: binary()) -> [mqtt_topic()].
match(Topic) when is_binary(Topic) ->
MatchedTopics = mnesia:async_dirty(fun emqttd_trie:match/1, [Topic]),
lists:append([mnesia:dirty_read(topic, Name) || Name <- MatchedTopics]).
-spec match(binary()) -> [mqtt_topic()].
match(To) ->
MatchedTopics = mnesia:async_dirty(fun emqttd_trie:match/1, [To]),
%% ets:lookup for topic table will be replicated.
lists:append([ets:lookup(topic, Topic) || Topic <- MatchedTopics]).
%%%=============================================================================
%%% gen_server callbacks
%%%=============================================================================
init([Id, _Opts]) ->
%%process_flag(priority, high),
%%process_flag(min_heap_size, 1024*1024),
gproc_pool:connect_worker(pubsub, {?MODULE, Id}),
{ok, #state{id = Id, submap = maps:new()}}.
init([Pool, Id, StatsFun, Opts]) ->
?ROUTER:init(Opts),
?GPROC_POOL(join, Pool, Id),
{ok, #state{pool = Pool, id = Id, statsfun = StatsFun}}.
prioritise_call(Msg, _From, _Len, _State) ->
case Msg of
{subscribe, _, _} -> 1;
_ -> 0
end.
handle_call({subscribe, {SubId, SubPid}, TopicTable}, _From,
State = #state{statsfun = StatsFun}) ->
prioritise_cast(Msg, _Len, _State) ->
case Msg of
{unsubscribe, _, _} -> 2;
_ -> 0
end.
Topics = [Topic || {Topic, _Qos} <- TopicTable],
prioritise_info(Msg, _Len, _State) ->
case Msg of
{'DOWN', _, _, _, _} -> 3;
_ -> 0
end.
%% Add routes first
?ROUTER:add_routes(Topics, SubPid),
handle_call({subscribe, SubPid, Topics}, _From, State) ->
TopicSubs = lists:map(fun({<<"$Q/", _/binary>> = Queue, Qos}) ->
#mqtt_queue{name = Queue, qpid = SubPid, qos = Qos};
({Topic, Qos}) ->
{#mqtt_topic{topic = Topic, node = node()},
#mqtt_subscriber{topic = Topic, subpid = SubPid, qos = Qos}}
end, Topics),
F = fun() ->
lists:map(fun(QueueR) when is_record(QueueR, mqtt_queue) ->
add_queue(QueueR);
(TopicSub) ->
add_subscriber(TopicSub)
end, TopicSubs)
end,
case mnesia:transaction(F) of
{atomic, _Result} ->
setstats(all),
NewState = monitor_subscriber(SubPid, State),
%%TODO: grant all qos
{reply, {ok, [Qos || {_Topic, Qos} <- Topics]}, NewState};
{aborted, Error} ->
{reply, {error, Error}, State}
end;
%% Insert topic records to global topic table
Records = [#mqtt_topic{topic = Topic, node = node()} || Topic <- Topics],
handle_call({subscribe, SubPid, <<"$Q/", _/binary>> = Queue, Qos}, _From, State) ->
case mnesia:dirty_read(queue, Queue) of
[OldQueueR] -> lager:error("Queue is overwrited by ~p: ~p", [SubPid, OldQueueR]);
[] -> ok
end,
QueueR = #mqtt_queue{name = Queue, qpid = SubPid, qos = Qos},
case mnesia:transaction(fun add_queue/1, [QueueR]) of
{atomic, ok} ->
setstats(queues),
{reply, {ok, Qos}, monitor_subscriber(SubPid, State)};
{aborted, Error} ->
{reply, {error, Error}, State}
end;
handle_call({subscribe, SubPid, Topic, Qos}, _From, State) ->
TopicR = #mqtt_topic{topic = Topic, node = node()},
Subscriber = #mqtt_subscriber{topic = Topic, subpid = SubPid, qos = Qos},
case mnesia:transaction(fun add_subscriber/1, [{TopicR, Subscriber}]) of
{atomic, ok} ->
setstats(all),
{reply, {ok, Qos}, monitor_subscriber(SubPid, State)};
case mnesia:transaction(fun add_topics/1, [Records]) of
{atomic, _} ->
StatsFun(topic),
if_subscription(
fun(_) ->
%% Add subscriptions
Args = [fun add_subscriptions/2, [SubId, TopicTable]],
emqttd_pooler:async_submit({mnesia, async_dirty, Args}),
StatsFun(subscription)
end),
%% Grant all qos...
{reply, {ok, [Qos || {_Topic, Qos} <- TopicTable]}, State};
{aborted, Error} ->
{reply, {error, Error}, State}
end;
handle_call(Req, _From, State) ->
lager:error("Bad Request: ~p", [Req]),
{reply, {error, badreq}, State}.
?UNEXPECTED_REQ(Req, State).
handle_cast({unsubscribe, SubPid, Topics}, State) when is_list(Topics) ->
handle_cast({unsubscribe, {SubId, SubPid}, Topics}, State = #state{statsfun = StatsFun}) ->
%% Delete routes first
?ROUTER:delete_routes(Topics, SubPid),
TopicSubs = lists:map(fun(<<"$Q/", _/binary>> = Queue) ->
#mqtt_queue{name = Queue, qpid = SubPid};
(Topic) ->
{#mqtt_topic{topic = Topic, node = node()},
#mqtt_subscriber{topic = Topic, subpid = SubPid, _ = '_'}}
end, Topics),
F = fun() ->
lists:foreach(
fun(QueueR) when is_record(QueueR, mqtt_queue) ->
remove_queue(QueueR);
(TopicSub) ->
remove_subscriber(TopicSub)
end, TopicSubs)
end,
case mnesia:transaction(F) of
{atomic, _} -> ok;
{aborted, Error} -> lager:error("unsubscribe ~p error: ~p", [Topics, Error])
end,
setstats(all),
{noreply, State};
handle_cast({unsubscribe, SubPid, <<"$Q/", _/binary>> = Queue}, State) ->
QueueR = #mqtt_queue{name = Queue, qpid = SubPid},
case mnesia:transaction(fun remove_queue/1, [QueueR]) of
{atomic, _} ->
setstats(queues);
{aborted, Error} ->
lager:error("unsubscribe queue ~s error: ~p", [Queue, Error])
end,
{noreply, State};
handle_cast({unsubscribe, SubPid, Topic}, State) ->
TopicR = #mqtt_topic{topic = Topic, node = node()},
Subscriber = #mqtt_subscriber{topic = Topic, subpid = SubPid, _ = '_'},
case mnesia:transaction(fun remove_subscriber/1, [{TopicR, Subscriber}]) of
{atomic, _} -> ok;
{aborted, Error} -> lager:error("unsubscribe ~s error: ~p", [Topic, Error])
end,
setstats(all),
%% Remove subscriptions
if_subscription(
fun(_) ->
Args = [fun remove_subscriptions/2, [SubId, Topics]],
emqttd_pooler:async_submit({mnesia, async_dirty, Args}),
StatsFun(subscription)
end),
{noreply, State};
handle_cast(Msg, State) ->
lager:error("Bad Msg: ~p", [Msg]),
{noreply, State}.
?UNEXPECTED_MSG(Msg, State).
handle_info({'DOWN', _Mon, _Type, DownPid, _Info}, State = #state{submap = SubMap}) ->
case maps:is_key(DownPid, SubMap) of
true ->
Node = node(),
F = fun() ->
%% remove queue...
Queues = mnesia:match_object(queue, #mqtt_queue{qpid = DownPid, _ = '_'}, write),
lists:foreach(fun(QueueR) ->
mnesia:delete_object(queue, QueueR, write)
end, Queues),
handle_info({'DOWN', _Mon, _Type, DownPid, _Info}, State) ->
%% remove subscribers...
Subscribers = mnesia:index_read(subscriber, DownPid, #mqtt_subscriber.subpid),
lists:foreach(fun(Sub = #mqtt_subscriber{topic = Topic}) ->
mnesia:delete_object(subscriber, Sub, write),
try_remove_topic(#mqtt_topic{topic = Topic, node = Node})
end, Subscribers)
end,
case catch mnesia:transaction(F) of
{atomic, _} -> ok;
{aborted, Reason} ->
lager:error("Failed to delete 'DOWN' subscriber ~p: ~p", [DownPid, Reason])
end,
setstats(all),
{noreply, State#state{submap = maps:remove(DownPid, SubMap)}};
false ->
lager:error("Unexpected 'DOWN' from ~p", [DownPid]),
{noreply, State}
end;
Routes = ?ROUTER:lookup_routes(DownPid),
%% Delete all routes of the process
?ROUTER:delete_routes(DownPid),
?HELPER:aging([Topic || Topic <- Routes, not ?ROUTER:has_route(Topic)]),
{noreply, State, hibernate};
handle_info(Info, State) ->
lager:error("Unexpected Info: ~p", [Info]),
{noreply, State}.
?UNEXPECTED_INFO(Info, State).
terminate(_Reason, _State) ->
TopicR = #mqtt_topic{_ = '_', node = node()},
F = fun() ->
[mnesia:delete_object(topic, R, write) || R <- mnesia:match_object(topic, TopicR, write)]
%%TODO: remove trie??
end,
mnesia:transaction(F),
setstats(all).
terminate(_Reason, #state{pool = Pool, id = Id}) ->
?GPROC_POOL(leave, Pool, Id).
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
@ -405,8 +320,8 @@ code_change(_OldVsn, State, _Extra) ->
%%% Internal functions
%%%=============================================================================
add_queue(QueueR) ->
mnesia:write(queue, QueueR, write).
add_topics(Records) ->
lists:foreach(fun add_topic/1, Records).
add_topic(TopicR = #mqtt_topic{topic = Topic}) ->
case mnesia:wread({topic, Topic}) of
@ -420,92 +335,46 @@ add_topic(TopicR = #mqtt_topic{topic = Topic}) ->
end
end.
%% Fix issue #53 - Remove Overlapping Subscriptions
add_subscriber({TopicR, Subscriber = #mqtt_subscriber{topic = Topic, subpid = SubPid, qos = Qos}})
when is_record(TopicR, mqtt_topic) ->
case add_topic(TopicR) of
ok ->
OverlapSubs = [Sub || Sub = #mqtt_subscriber{topic = SubTopic, qos = SubQos}
<- mnesia:index_read(subscriber, SubPid, #mqtt_subscriber.subpid),
SubTopic =:= Topic, SubQos =/= Qos],
add_subscriptions(undefined, _TopicTable) ->
ok;
add_subscriptions(SubId, TopicTable) ->
lists:foreach(fun({Topic, Qos}) ->
add_subscription(SubId, {Topic, Qos})
end,TopicTable).
%% remove overlapping subscribers
if
length(OverlapSubs) =:= 0 -> ok;
add_subscription(SubId, {Topic, Qos}) ->
Subscription = #mqtt_subscription{subid = SubId, topic = Topic, qos = Qos},
Pattern = #mqtt_subscription{subid = SubId, topic = Topic, qos = '_'},
Records = mnesia:match_object(subscription, Pattern, write),
case lists:member(Subscription, Records) of
true ->
lager:warning("Remove overlapping subscribers: ~p", [OverlapSubs]),
[mnesia:delete_object(subscriber, OverlapSub, write) || OverlapSub <- OverlapSubs]
end,
%% insert subscriber
mnesia:write(subscriber, Subscriber, write);
Error ->
Error
end.
monitor_subscriber(SubPid, State = #state{submap = SubMap}) ->
NewSubMap = case maps:is_key(SubPid, SubMap) of
ok;
false ->
maps:put(SubPid, erlang:monitor(process, SubPid), SubMap);
true ->
SubMap
end,
State#state{submap = NewSubMap}.
remove_queue(#mqtt_queue{name = Name, qpid = Pid}) ->
case mnesia:wread({queue, Name}) of
[R = #mqtt_queue{qpid = Pid}] ->
mnesia:delete(queue, R, write);
_ ->
ok
[delete_subscription(Record) || Record <- Records],
insert_subscription(Subscription)
end.
remove_subscriber({TopicR, Subscriber}) when is_record(TopicR, mqtt_topic) ->
[mnesia:delete_object(subscriber, Sub, write) ||
Sub <- mnesia:match_object(subscriber, Subscriber, write)],
try_remove_topic(TopicR).
insert_subscription(Record) ->
mnesia:write(subscription, Record, write).
try_remove_topic(TopicR = #mqtt_topic{topic = Topic}) ->
case mnesia:read({subscriber, Topic}) of
[] ->
mnesia:delete_object(topic, TopicR, write),
case mnesia:read(topic, Topic) of
[] -> emqttd_trie:delete(Topic);
_ -> ok
end;
_ ->
ok
end.
remove_subscriptions(undefined, _Topics) ->
ok;
remove_subscriptions(SubId, Topics) ->
lists:foreach(fun(Topic) ->
Pattern = #mqtt_subscription{subid = SubId, topic = Topic, qos = '_'},
Records = mnesia:match_object(subscription, Pattern, write),
[delete_subscription(Record) || Record <- Records]
end, Topics).
delete_subscription(Record) ->
mnesia:delete_object(subscription, Record, write).
%%%=============================================================================
%%% Stats functions
%%%=============================================================================
setstats(all) ->
[setstats(Stat) || Stat <- [queues, topics, subscribers]];
setstats(queues) ->
emqttd_stats:setstats('queues/count', 'queues/max',
mnesia:table_info(queue, size));
setstats(topics) ->
emqttd_stats:setstats('topics/count', 'topics/max',
mnesia:table_info(topic, size));
setstats(subscribers) ->
emqttd_stats:setstats('subscribers/count', 'subscribers/max',
mnesia:table_info(subscriber, size)).
setstats(dropped, false) ->
ignore;
setstats(dropped, true) ->
emqttd_metrics:inc('messages/dropped').
%%%=============================================================================
%%% Trace functions
%%% Trace Functions
%%%=============================================================================
trace(publish, From, _Msg) when is_atom(From) ->
%%dont' trace broker publish
%% Dont' trace '$SYS' publish
ignore;
trace(publish, From, #mqtt_message{topic = Topic, payload = Payload}) ->

View File

@ -0,0 +1,194 @@
%%%-----------------------------------------------------------------------------
%%% Copyright (c) 2012-2015 eMQTT.IO, All Rights Reserved.
%%%
%%% Permission is hereby granted, free of charge, to any person obtaining a copy
%%% of this software and associated documentation files (the "Software"), to deal
%%% in the Software without restriction, including without limitation the rights
%%% to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
%%% copies of the Software, and to permit persons to whom the Software is
%%% furnished to do so, subject to the following conditions:
%%%
%%% The above copyright notice and this permission notice shall be included in all
%%% copies or substantial portions of the Software.
%%%
%%% THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
%%% IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
%%% FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
%%% AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
%%% LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc PubSub Route Aging Helper
%%%
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_pubsub_helper).
-behaviour(gen_server2).
-include("emqttd.hrl").
-include("emqttd_internal.hrl").
%% API Function Exports
-export([start_link/2, aging/1]).
%% gen_server Function Exports
-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
terminate/2, code_change/3]).
-ifdef(TEST).
-compile(export_all).
-endif.
-record(aging, {topics, time, tref}).
-record(state, {aging :: #aging{}, statsfun}).
-define(SERVER, ?MODULE).
-define(ROUTER, emqttd_router).
%%%=============================================================================
%%% API
%%%=============================================================================
%%------------------------------------------------------------------------------
%% @doc Start pubsub helper.
%% @end
%%------------------------------------------------------------------------------
-spec start_link(fun(), list(tuple())) -> {ok, pid()} | ignore | {error, any()}.
start_link(StatsFun, Opts) ->
gen_server2:start_link({local, ?SERVER}, ?MODULE, [StatsFun, Opts], []).
%%------------------------------------------------------------------------------
%% @doc Aging topics
%% @end
%%------------------------------------------------------------------------------
-spec aging(list(binary())) -> ok.
aging(Topics) ->
gen_server2:cast(?SERVER, {aging, Topics}).
%%%=============================================================================
%%% gen_server callbacks
%%%=============================================================================
init([StatsFun, Opts]) ->
mnesia:subscribe(system),
AgingSecs = proplists:get_value(route_aging, Opts, 5),
%% Aging Timer
{ok, AgingTref} = start_tick(AgingSecs div 2),
{ok, #state{aging = #aging{topics = dict:new(),
time = AgingSecs,
tref = AgingTref},
statsfun = StatsFun}}.
start_tick(Secs) ->
timer:send_interval(timer:seconds(Secs), {clean, aged}).
handle_call(Req, _From, State) ->
?UNEXPECTED_REQ(Req, State).
handle_cast({aging, Topics}, State = #state{aging = Aging}) ->
#aging{topics = Dict} = Aging,
TS = emqttd_util:now_to_secs(),
Dict1 =
lists:foldl(fun(Topic, Acc) ->
case dict:find(Topic, Acc) of
{ok, _} -> Acc;
error -> dict:store(Topic, TS, Acc)
end
end, Dict, Topics),
{noreply, State#state{aging = Aging#aging{topics = Dict1}}};
handle_cast(Msg, State) ->
?UNEXPECTED_MSG(Msg, State).
handle_info({clean, aged}, State = #state{aging = Aging}) ->
#aging{topics = Dict, time = Time} = Aging,
ByTime = emqttd_util:now_to_secs() - Time,
Dict1 = try_clean(ByTime, dict:to_list(Dict)),
NewAging = Aging#aging{topics = dict:from_list(Dict1)},
noreply(State#state{aging = NewAging});
handle_info({mnesia_system_event, {mnesia_down, Node}}, State) ->
%% mnesia master?
Pattern = #mqtt_topic{_ = '_', node = Node},
F = fun() ->
[mnesia:delete_object(topic, R, write) ||
R <- mnesia:match_object(topic, Pattern, write)]
end,
mnesia:async_dirty(F),
noreply(State);
handle_info(Info, State) ->
?UNEXPECTED_INFO(Info, State).
terminate(_Reason, #state{aging = #aging{tref = TRef}}) ->
timer:cancel(TRef).
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
%%%=============================================================================
%%% Internal Functions
%%%=============================================================================
noreply(State = #state{statsfun = StatsFun}) ->
StatsFun(topic),
{noreply, State, hibernate}.
try_clean(ByTime, List) ->
try_clean(ByTime, List, []).
try_clean(_ByTime, [], Acc) ->
Acc;
try_clean(ByTime, [{Topic, TS} | Left], Acc) ->
case ?ROUTER:has_route(Topic) of
false ->
try_clean2(ByTime, {Topic, TS}, Left, Acc);
true ->
try_clean(ByTime, Left, Acc)
end.
try_clean2(ByTime, {Topic, TS}, Left, Acc) when TS > ByTime ->
try_clean(ByTime, Left, [{Topic, TS}|Acc]);
try_clean2(ByTime, {Topic, _TS}, Left, Acc) ->
TopicR = #mqtt_topic{topic = Topic, node = node()},
mnesia:transaction(fun try_remove_topic/1, [TopicR]),
try_clean(ByTime, Left, Acc).
try_remove_topic(TopicR = #mqtt_topic{topic = Topic}) ->
%% Lock topic first
case mnesia:wread({topic, Topic}) of
[] -> ok;
[TopicR] ->
if_no_route(Topic, fun() ->
%% Remove topic and trie
mnesia:delete_object(topic, TopicR, write),
emqttd_trie:delete(Topic)
end);
_More ->
if_no_route(Topic, fun() ->
%% Remove topic
mnesia:delete_object(topic, TopicR, write)
end)
end.
if_no_route(Topic, Fun) ->
case ?ROUTER:has_route(Topic) of
true -> ok;
false -> Fun()
end.

View File

@ -19,18 +19,17 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd pubsub supervisor.
%%% @doc PubSub Supervisor
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_pubsub_sup).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(supervisor).
-include("emqttd.hrl").
-behaviour(supervisor).
-define(HELPER, emqttd_pubsub_helper).
%% API
-export([start_link/0]).
@ -39,19 +38,26 @@
-export([init/1]).
start_link() ->
supervisor:start_link({local, ?MODULE}, ?MODULE, []).
supervisor:start_link({local, ?MODULE}, ?MODULE, [emqttd_broker:env(pubsub)]).
init([]) ->
Opts = emqttd_broker:env(pubsub),
init([Opts]) ->
%% PubSub Helper
Helper = {helper, {?HELPER, start_link, [fun stats/1, Opts]},
permanent, infinity, worker, [?HELPER]},
%% PubSub Pool Sup
MFA = {emqttd_pubsub, start_link, [fun stats/1, Opts]},
PoolSup = emqttd_pool_sup:spec([pubsub, hash, pool_size(Opts), MFA]),
{ok, {{one_for_all, 10, 60}, [Helper, PoolSup]}}.
pool_size(Opts) ->
Schedulers = erlang:system_info(schedulers),
PoolSize = proplists:get_value(pool_size, Opts, Schedulers),
gproc_pool:new(pubsub, hash, [{size, PoolSize}]),
Children = lists:map(
fun(I) ->
Name = {emqttd_pubsub, I},
gproc_pool:add_worker(pubsub, Name, I),
{Name, {emqttd_pubsub, start_link, [I, Opts]},
permanent, 10000, worker, [emqttd_pubsub]}
end, lists:seq(1, PoolSize)),
{ok, {{one_for_all, 10, 100}, Children}}.
proplists:get_value(pool_size, Opts, Schedulers).
stats(topic) ->
emqttd_stats:setstats('topics/count', 'topics/max',
mnesia:table_info(topic, size));
stats(subscription) ->
emqttd_stats:setstats('subscriptions/count', 'subscriptions/max',
mnesia:table_info(subscription, size)).

View File

@ -19,19 +19,22 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% MQTT retained message storage.
%%% @doc MQTT retained message storage.
%%%
%%% TODO: should match topic tree
%%%
%%% @end
%%%
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_retained).
-module(emqttd_retainer).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(gen_server).
-include("emqttd.hrl").
-include("emqttd_internal.hrl").
-include_lib("stdlib/include/ms_transform.hrl").
%% Mnesia callbacks
@ -46,8 +49,6 @@
%% API Function Exports
-export([start_link/0, expire/1]).
-behaviour(gen_server).
%% gen_server Function Exports
-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
terminate/2, code_change/3]).
@ -66,7 +67,6 @@ mnesia(boot) ->
{ram_copies, [node()]},
{record_name, mqtt_retained},
{attributes, record_info(fields, mqtt_retained)}]);
mnesia(copy) ->
ok = emqttd_mnesia:copy_table(retained).
@ -142,7 +142,7 @@ dispatch(Topic, CPid) when is_binary(Topic) ->
end,
mnesia:async_dirty(fun mnesia:foldl/3, [Fun, [], retained])
end,
lists:foreach(fun(Msg) -> CPid ! {dispatch, Msg} end, lists:reverse(Msgs)).
lists:foreach(fun(Msg) -> CPid ! {dispatch, Topic, Msg} end, lists:reverse(Msgs)).
%%%=============================================================================
%%% gen_server callbacks
@ -159,12 +159,11 @@ init([]) ->
stats_timer = StatsTimer,
expire_timer = ExpireTimer}}.
handle_call(_Request, _From, State) ->
{reply, ok, State}.
handle_call(Req, _From, State) ->
?UNEXPECTED_REQ(Req, State).
handle_cast(Msg, State) ->
lager:error("Unexpected Msg: ~p", [Msg]),
{noreply, State}.
?UNEXPECTED_MSG(Msg, State).
handle_info(stats, State = #state{stats_fun = StatsFun}) ->
StatsFun(mnesia:table_info(retained, size)),
@ -179,8 +178,7 @@ handle_info(expire, State = #state{expired_after = ExpiredAfter}) ->
{noreply, State, hibernate};
handle_info(Info, State) ->
lager:error("Unexpected Info: ~p", [Info]),
{noreply, State}.
?UNEXPECTED_INFO(Info, State).
terminate(_Reason, _State = #state{stats_timer = TRef1, expire_timer = TRef2}) ->
timer:cancel(TRef1),

179
src/emqttd_router.erl Normal file
View File

@ -0,0 +1,179 @@
%%%-----------------------------------------------------------------------------
%%% Copyright (c) 2012-2015 eMQTT.IO, All Rights Reserved.
%%%
%%% Permission is hereby granted, free of charge, to any person obtaining a copy
%%% of this software and associated documentation files (the "Software"), to deal
%%% in the Software without restriction, including without limitation the rights
%%% to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
%%% copies of the Software, and to permit persons to whom the Software is
%%% furnished to do so, subject to the following conditions:
%%%
%%% The above copyright notice and this permission notice shall be included in all
%%% copies or substantial portions of the Software.
%%%
%%% THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
%%% IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
%%% FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
%%% AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
%%% LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc MQTT Message Router on Local Node
%%%
%%% Route Table:
%%%
%%% Topic -> Pid1, Pid2, ...
%%%
%%% Reverse Route Table:
%%%
%%% Pid -> Topic1, Topic2, ...
%%%
%%% @end
%%%
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_router).
-include("emqttd.hrl").
-include("emqttd_protocol.hrl").
-export([init/1, route/2, lookup_routes/1, has_route/1,
add_routes/2, delete_routes/1, delete_routes/2]).
-ifdef(TEST).
-compile(export_all).
-endif.
%%------------------------------------------------------------------------------
%% @doc Create route tables.
%% @end
%%------------------------------------------------------------------------------
init(_Opts) ->
TabOpts = [bag, public, named_table,
{write_concurrency, true}],
%% Route Table: Topic -> {Pid, QoS}
%% Route Shard: {Topic, Shard} -> {Pid, QoS}
ensure_tab(route, TabOpts),
%% Reverse Route Table: Pid -> {Topic, QoS}
ensure_tab(reverse_route, TabOpts).
ensure_tab(Tab, Opts) ->
case ets:info(Tab, name) of
undefined ->
ets:new(Tab, Opts);
_ ->
ok
end.
%%------------------------------------------------------------------------------
%% @doc Add Routes.
%% @end
%%------------------------------------------------------------------------------
-spec add_routes(list(binary()), pid()) -> ok.
add_routes(Topics, Pid) when is_pid(Pid) ->
with_stats(fun() ->
case lookup_routes(Pid) of
[] ->
erlang:monitor(process, Pid),
insert_routes(Topics, Pid);
InEts ->
insert_routes(Topics -- InEts, Pid)
end
end).
%%------------------------------------------------------------------------------
%% @doc Lookup Routes
%% @end
%%------------------------------------------------------------------------------
-spec lookup_routes(pid()) -> list(binary()).
lookup_routes(Pid) when is_pid(Pid) ->
[Topic || {_, Topic} <- ets:lookup(reverse_route, Pid)].
%%------------------------------------------------------------------------------
%% @doc Has Route?
%% @end
%%------------------------------------------------------------------------------
-spec has_route(binary()) -> boolean().
has_route(Topic) ->
ets:member(route, Topic).
%%------------------------------------------------------------------------------
%% @doc Delete Routes
%% @end
%%------------------------------------------------------------------------------
-spec delete_routes(list(binary()), pid()) -> ok.
delete_routes(Topics, Pid) ->
with_stats(fun() ->
Routes = [{Topic, Pid} || Topic <- Topics],
lists:foreach(fun delete_route/1, Routes)
end).
-spec delete_routes(pid()) -> ok.
delete_routes(Pid) when is_pid(Pid) ->
with_stats(fun() ->
Routes = [{Topic, Pid} || Topic <- lookup_routes(Pid)],
ets:delete(reverse_route, Pid),
lists:foreach(fun delete_route_only/1, Routes)
end).
%%------------------------------------------------------------------------------
%% @doc Route Message on Local Node.
%% @end
%%------------------------------------------------------------------------------
-spec route(binary(), mqtt_message()) -> non_neg_integer().
route(Queue = <<"$Q/", _Q>>, Msg) ->
case ets:lookup(route, Queue) of
[] ->
emqttd_metrics:inc('messages/dropped');
Routes ->
Idx = crypto:rand_uniform(1, length(Routes) + 1),
{_, SubPid} = lists:nth(Idx, Routes),
dispatch(SubPid, Queue, Msg)
end;
route(Topic, Msg) ->
case ets:lookup(route, Topic) of
[] ->
emqttd_metrics:inc('messages/dropped');
Routes ->
lists:foreach(fun({_Topic, SubPid}) ->
dispatch(SubPid, Topic, Msg)
end, Routes)
end.
dispatch(SubPid, Topic, Msg) -> SubPid ! {dispatch, Topic, Msg}.
%%%=============================================================================
%%% Internal Functions
%%%=============================================================================
insert_routes([], _Pid) ->
ok;
insert_routes(Topics, Pid) ->
{Routes, ReverseRoutes} = routes(Topics, Pid),
ets:insert(route, Routes),
ets:insert(reverse_route, ReverseRoutes).
routes(Topics, Pid) ->
lists:unzip([{{Topic, Pid}, {Pid, Topic}} || Topic <- Topics]).
delete_route({Topic, Pid}) ->
ets:delete_object(reverse_route, {Pid, Topic}),
ets:delete_object(route, {Topic, Pid}).
delete_route_only({Topic, Pid}) ->
ets:delete_object(route, {Topic, Pid}).
with_stats(Fun) ->
Ok = Fun(), setstats(), Ok.
setstats() ->
lists:foreach(fun setstat/1, [{route, 'routes/count'},
{reverse_route, 'routes/reverse'}]).
setstat({Tab, Stat}) ->
emqttd_stats:setstat(Stat, ets:info(Tab, size)).

View File

@ -19,48 +19,45 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% MQTT Packet Serialiser.
%%% @doc MQTT Packet Serializer
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_serialiser).
-author("Feng Lee <feng@emqtt.io>").
-module(emqttd_serializer).
-include("emqttd.hrl").
-include("emqttd_protocol.hrl").
%% API
-export([serialise/1]).
-export([serialize/1]).
%%------------------------------------------------------------------------------
%% @doc Serialise MQTT Packet
%% @end
%%------------------------------------------------------------------------------
-spec serialise(mqtt_packet()) -> binary().
serialise(#mqtt_packet{header = Header = #mqtt_packet_header{type = Type},
-spec serialize(mqtt_packet()) -> binary().
serialize(#mqtt_packet{header = Header = #mqtt_packet_header{type = Type},
variable = Variable,
payload = Payload}) ->
serialise_header(Header,
serialise_variable(Type, Variable,
serialise_payload(Payload))).
serialize_header(Header,
serialize_variable(Type, Variable,
serialize_payload(Payload))).
serialise_header(#mqtt_packet_header{type = Type,
serialize_header(#mqtt_packet_header{type = Type,
dup = Dup,
qos = Qos,
retain = Retain},
{VariableBin, PayloadBin}) when ?CONNECT =< Type andalso Type =< ?DISCONNECT ->
Len = size(VariableBin) + size(PayloadBin),
true = (Len =< ?MAX_LEN),
LenBin = serialise_len(Len),
LenBin = serialize_len(Len),
<<Type:4, (opt(Dup)):1, (opt(Qos)):2, (opt(Retain)):1,
LenBin/binary,
VariableBin/binary,
PayloadBin/binary>>.
serialise_variable(?CONNECT, #mqtt_packet_connect{client_id = ClientId,
serialize_variable(?CONNECT, #mqtt_packet_connect{client_id = ClientId,
proto_ver = ProtoVer,
proto_name = ProtoName,
will_retain = WillRetain,
@ -83,79 +80,79 @@ serialise_variable(?CONNECT, #mqtt_packet_connect{client_id = ClientId,
(opt(CleanSess)):1,
0:1,
KeepAlive:16/big-unsigned-integer>>,
PayloadBin = serialise_utf(ClientId),
PayloadBin = serialize_utf(ClientId),
PayloadBin1 = case WillFlag of
true -> <<PayloadBin/binary,
(serialise_utf(WillTopic))/binary,
(serialize_utf(WillTopic))/binary,
(size(WillMsg)):16/big-unsigned-integer,
WillMsg/binary>>;
false -> PayloadBin
end,
UserPasswd = << <<(serialise_utf(B))/binary>> || B <- [Username, Password], B =/= undefined >>,
UserPasswd = << <<(serialize_utf(B))/binary>> || B <- [Username, Password], B =/= undefined >>,
{VariableBin, <<PayloadBin1/binary, UserPasswd/binary>>};
serialise_variable(?CONNACK, #mqtt_packet_connack{ack_flags = AckFlags,
serialize_variable(?CONNACK, #mqtt_packet_connack{ack_flags = AckFlags,
return_code = ReturnCode}, undefined) ->
{<<AckFlags:8, ReturnCode:8>>, <<>>};
serialise_variable(?SUBSCRIBE, #mqtt_packet_subscribe{packet_id = PacketId,
serialize_variable(?SUBSCRIBE, #mqtt_packet_subscribe{packet_id = PacketId,
topic_table = Topics }, undefined) ->
{<<PacketId:16/big>>, serialise_topics(Topics)};
{<<PacketId:16/big>>, serialize_topics(Topics)};
serialise_variable(?SUBACK, #mqtt_packet_suback{packet_id = PacketId,
serialize_variable(?SUBACK, #mqtt_packet_suback{packet_id = PacketId,
qos_table = QosTable}, undefined) ->
{<<PacketId:16/big>>, << <<Q:8>> || Q <- QosTable >>};
serialise_variable(?UNSUBSCRIBE, #mqtt_packet_unsubscribe{packet_id = PacketId,
serialize_variable(?UNSUBSCRIBE, #mqtt_packet_unsubscribe{packet_id = PacketId,
topics = Topics }, undefined) ->
{<<PacketId:16/big>>, serialise_topics(Topics)};
{<<PacketId:16/big>>, serialize_topics(Topics)};
serialise_variable(?UNSUBACK, #mqtt_packet_unsuback{packet_id = PacketId}, undefined) ->
serialize_variable(?UNSUBACK, #mqtt_packet_unsuback{packet_id = PacketId}, undefined) ->
{<<PacketId:16/big>>, <<>>};
serialise_variable(?PUBLISH, #mqtt_packet_publish{topic_name = TopicName,
serialize_variable(?PUBLISH, #mqtt_packet_publish{topic_name = TopicName,
packet_id = PacketId }, PayloadBin) ->
TopicBin = serialise_utf(TopicName),
TopicBin = serialize_utf(TopicName),
PacketIdBin = if
PacketId =:= undefined -> <<>>;
true -> <<PacketId:16/big>>
end,
{<<TopicBin/binary, PacketIdBin/binary>>, PayloadBin};
serialise_variable(PubAck, #mqtt_packet_puback{packet_id = PacketId}, _Payload)
serialize_variable(PubAck, #mqtt_packet_puback{packet_id = PacketId}, _Payload)
when PubAck =:= ?PUBACK; PubAck =:= ?PUBREC; PubAck =:= ?PUBREL; PubAck =:= ?PUBCOMP ->
{<<PacketId:16/big>>, <<>>};
serialise_variable(?PINGREQ, undefined, undefined) ->
serialize_variable(?PINGREQ, undefined, undefined) ->
{<<>>, <<>>};
serialise_variable(?PINGRESP, undefined, undefined) ->
serialize_variable(?PINGRESP, undefined, undefined) ->
{<<>>, <<>>};
serialise_variable(?DISCONNECT, undefined, undefined) ->
serialize_variable(?DISCONNECT, undefined, undefined) ->
{<<>>, <<>>}.
serialise_payload(undefined) ->
serialize_payload(undefined) ->
undefined;
serialise_payload(Bin) when is_binary(Bin) ->
serialize_payload(Bin) when is_binary(Bin) ->
Bin.
serialise_topics([{_Topic, _Qos}|_] = Topics) ->
<< <<(serialise_utf(Topic))/binary, ?RESERVED:6, Qos:2>> || {Topic, Qos} <- Topics >>;
serialize_topics([{_Topic, _Qos}|_] = Topics) ->
<< <<(serialize_utf(Topic))/binary, ?RESERVED:6, Qos:2>> || {Topic, Qos} <- Topics >>;
serialise_topics([H|_] = Topics) when is_binary(H) ->
<< <<(serialise_utf(Topic))/binary>> || Topic <- Topics >>.
serialize_topics([H|_] = Topics) when is_binary(H) ->
<< <<(serialize_utf(Topic))/binary>> || Topic <- Topics >>.
serialise_utf(String) ->
serialize_utf(String) ->
StringBin = unicode:characters_to_binary(String),
Len = size(StringBin),
true = (Len =< 16#ffff),
<<Len:16/big, StringBin/binary>>.
serialise_len(N) when N =< ?LOWBITS ->
serialize_len(N) when N =< ?LOWBITS ->
<<0:1, N:7>>;
serialise_len(N) ->
<<1:1, (N rem ?HIGHBIT):7, (serialise_len(N div ?HIGHBIT))/binary>>.
serialize_len(N) ->
<<1:1, (N rem ?HIGHBIT):7, (serialize_len(N div ?HIGHBIT))/binary>>.
opt(undefined) -> ?RESERVED;
opt(false) -> 0;

View File

@ -19,9 +19,7 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%%
%%% Session for persistent MQTT client.
%%% @doc Session for persistent MQTT client.
%%%
%%% Session State in the broker consists of:
%%%
@ -43,15 +41,17 @@
%%% State of Message: newcome, inflight, pending
%%%
%%% @end
%%%
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_session).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-include("emqttd_protocol.hrl").
-include("emqttd_internal.hrl").
-behaviour(gen_server2).
%% Session API
@ -86,7 +86,7 @@
packet_id = 1,
%% Clients subscriptions.
subscriptions :: list(),
subscriptions :: dict:dict(),
%% Inflight qos1, qos2 messages sent to the client but unacked,
%% QoS 1 and QoS 2 messages which have been sent to the Client,
@ -245,7 +245,7 @@ init([CleanSess, ClientId, ClientPid]) ->
clean_sess = CleanSess,
client_id = ClientId,
client_pid = ClientPid,
subscriptions = [],
subscriptions = dict:new(),
inflight_queue = [],
max_inflight = emqttd_opts:g(max_inflight, SessEnv, 0),
message_queue = emqttd_mqueue:new(ClientId, QEnv, emqttd_alarm:alarm_fun()),
@ -287,7 +287,7 @@ prioritise_info(Msg, _Len, _State) ->
expired -> 10;
{timeout, _, _} -> 5;
collect_info -> 2;
{dispatch, _} -> 1;
{dispatch, _, _} -> 1;
_ -> 0
end.
@ -308,21 +308,20 @@ handle_call({publish, Msg = #mqtt_message{qos = ?QOS_2, pktid = PktId}},
end;
handle_call(Req, _From, State) ->
?LOG(critical, "Unexpected Request: ~p", [Req], State),
{reply, {error, unsupported_req}, State, hibernate}.
?UNEXPECTED_REQ(Req, State).
handle_cast({subscribe, TopicTable0, AckFun}, Session = #session{client_id = ClientId,
subscriptions = Subscriptions}) ->
TopicTable = emqttd_broker:foldl_hooks('client.subscribe', [ClientId], TopicTable0),
case TopicTable -- Subscriptions of
case TopicTable -- dict:to_list(Subscriptions) of
[] ->
AckFun([Qos || {_, Qos} <- TopicTable]),
hibernate(Session);
_ ->
%% subscribe first and don't care if the subscriptions have been existed
{ok, GrantedQos} = emqttd_pubsub:subscribe(TopicTable),
{ok, GrantedQos} = emqttd_pubsub:subscribe(ClientId, TopicTable),
AckFun(GrantedQos),
@ -331,21 +330,22 @@ handle_cast({subscribe, TopicTable0, AckFun}, Session = #session{client_id = Cli
?LOG(info, "Subscribe ~p, Granted QoS: ~p", [TopicTable, GrantedQos], Session),
Subscriptions1 =
lists:foldl(fun({Topic, Qos}, Acc) ->
case lists:keyfind(Topic, 1, Acc) of
{Topic, Qos} ->
lists:foldl(fun({Topic, Qos}, Dict) ->
case dict:find(Topic, Dict) of
{ok, Qos} ->
?LOG(warning, "resubscribe ~s, qos = ~w", [Topic, Qos], Session),
Acc;
{Topic, OldQos} ->
Dict;
{ok, OldQos} ->
?LOG(warning, "resubscribe ~s, old qos=~w, new qos=~w", [Topic, OldQos, Qos], Session),
lists:keyreplace(Topic, 1, Acc, {Topic, Qos});
false ->
dict:store(Topic, Qos, Dict);
error ->
%%TODO: the design is ugly, rewrite later...:(
%% <MQTT V3.1.1>: 3.8.4
%% Where the Topic Filter is not identical to any existing Subscriptions filter,
%% a new Subscription is created and all matching retained messages are sent.
emqttd_retained:dispatch(Topic, self()),
[{Topic, Qos} | Acc]
emqttd_retainer:dispatch(Topic, self()),
dict:store(Topic, Qos, Dict)
end
end, Subscriptions, TopicTable),
hibernate(Session#session{subscriptions = Subscriptions1})
@ -362,13 +362,8 @@ handle_cast({unsubscribe, Topics0}, Session = #session{client_id = ClientId,
?LOG(info, "unsubscribe ~p", [Topics], Session),
Subscriptions1 =
lists:foldl(fun(Topic, Acc) ->
case lists:keyfind(Topic, 1, Acc) of
{Topic, _Qos} ->
lists:keydelete(Topic, 1, Acc);
false ->
Acc
end
lists:foldl(fun(Topic, Dict) ->
dict:erase(Topic, Dict)
end, Subscriptions, Topics),
hibernate(Session#session{subscriptions = Subscriptions1});
@ -483,31 +478,12 @@ handle_cast({pubcomp, PktId}, Session = #session{awaiting_comp = AwaitingComp})
end;
handle_cast(Msg, State) ->
?LOG(critical, "Unexpected Msg: ~p", [Msg], State),
hibernate(State).
?UNEXPECTED_MSG(Msg, State).
%% Queue messages when client is offline
handle_info({dispatch, Msg}, Session = #session{client_pid = undefined,
message_queue = Q})
%% Dispatch Message
handle_info({dispatch, Topic, Msg}, Session = #session{subscriptions = Subscriptions})
when is_record(Msg, mqtt_message) ->
hibernate(Session#session{message_queue = emqttd_mqueue:in(Msg, Q)});
%% Dispatch qos0 message directly to client
handle_info({dispatch, Msg = #mqtt_message{qos = ?QOS_0}},
Session = #session{client_pid = ClientPid}) ->
ClientPid ! {deliver, Msg},
hibernate(Session);
handle_info({dispatch, Msg = #mqtt_message{qos = QoS}},
Session = #session{message_queue = MsgQ})
when QoS =:= ?QOS_1 orelse QoS =:= ?QOS_2 ->
case check_inflight(Session) of
true ->
noreply(deliver(Msg, Session));
false ->
hibernate(Session#session{message_queue = emqttd_mqueue:in(Msg, MsgQ)})
end;
dispatch(fixqos(Topic, Msg, Subscriptions), Session);
handle_info({timeout, awaiting_ack, PktId}, Session = #session{client_pid = undefined,
awaiting_ack = AwaitingAck}) ->
@ -580,8 +556,7 @@ handle_info(expired, Session) ->
shutdown(expired, Session);
handle_info(Info, Session) ->
?LOG(critical, "Unexpected info: ~p", [Info], Session),
hibernate(Session).
?UNEXPECTED_INFO(Info, Session).
terminate(_Reason, #session{clean_sess = CleanSess, client_id = ClientId}) ->
emqttd_sm:unregister_session(CleanSess, ClientId).
@ -606,6 +581,38 @@ kick(ClientId, OldPid, Pid) ->
%% Clean noproc
receive {'EXIT', OldPid, _} -> ok after 0 -> ok end.
%%------------------------------------------------------------------------------
%% Dispatch Messages
%%------------------------------------------------------------------------------
%% Queue message if client disconnected
dispatch(Msg, Session = #session{client_pid = undefined, message_queue = Q}) ->
hibernate(Session#session{message_queue = emqttd_mqueue:in(Msg, Q)});
%% Deliver qos0 message directly to client
dispatch(Msg = #mqtt_message{qos = ?QOS0}, Session = #session{client_pid = ClientPid}) ->
ClientPid ! {deliver, Msg},
hibernate(Session);
dispatch(Msg = #mqtt_message{qos = QoS}, Session = #session{message_queue = MsgQ})
when QoS =:= ?QOS1 orelse QoS =:= ?QOS2 ->
case check_inflight(Session) of
true ->
noreply(deliver(Msg, Session));
false ->
hibernate(Session#session{message_queue = emqttd_mqueue:in(Msg, MsgQ)})
end.
fixqos(Topic, Msg = #mqtt_message{qos = PubQos}, Subscriptions) ->
case dict:find(Topic, Subscriptions) of
{ok, SubQos} when PubQos > SubQos ->
Msg#mqtt_message{qos = SubQos};
{ok, _SubQos} ->
Msg;
error ->
Msg
end.
%%------------------------------------------------------------------------------
%% Check inflight and awaiting_rel
%%------------------------------------------------------------------------------
@ -725,7 +732,7 @@ sess_info(#session{clean_sess = CleanSess,
timestamp = CreatedAt}) ->
Stats = emqttd_mqueue:stats(MessageQueue),
[{clean_sess, CleanSess},
{subscriptions, Subscriptions},
{subscriptions, dict:to_list(Subscriptions)},
{max_inflight, MaxInflight},
{inflight_queue, length(InflightQueue)},
{message_queue, proplists:get_value(len, Stats)},

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd session supervisor.
%%% @doc emqttd session supervisor.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_session_sup).
-author("Feng Lee <feng@emqtt.io>").
-behavior(supervisor).
-export([start_link/0, start_session/3]).

View File

@ -19,17 +19,16 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd session manager.
%%% @doc Session Manager
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_sm).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-include("emqttd_internal.hrl").
%% Mnesia Callbacks
-export([mnesia/1]).
@ -37,7 +36,7 @@
-copy_mnesia({mnesia, [copy]}).
%% API Function Exports
-export([start_link/1, pool/0]).
-export([start_link/2]).
-export([start_session/2, lookup_session/1]).
@ -52,11 +51,11 @@
%% gen_server2 priorities
-export([prioritise_call/4, prioritise_cast/3, prioritise_info/3]).
-record(state, {id}).
-record(state, {pool, id, monitors}).
-define(SM_POOL, ?MODULE).
-define(POOL, ?MODULE).
-define(TIMEOUT, 60000).
-define(TIMEOUT, 120000).
-define(LOG(Level, Format, Args, Session),
lager:Level("SM(~s): " ++ Format, [Session#mqtt_session.client_id | Args])).
@ -66,13 +65,12 @@
%%%=============================================================================
mnesia(boot) ->
%% global session...
%% Global Session Table
ok = emqttd_mnesia:create_table(session, [
{type, ordered_set},
{type, set},
{ram_copies, [node()]},
{record_name, mqtt_session},
{attributes, record_info(fields, mqtt_session)},
{index, [sess_pid]}]);
{attributes, record_info(fields, mqtt_session)}]);
mnesia(copy) ->
ok = emqttd_mnesia:copy_table(session).
@ -85,26 +83,20 @@ mnesia(copy) ->
%% @doc Start a session manager
%% @end
%%------------------------------------------------------------------------------
-spec start_link(Id :: pos_integer()) -> {ok, pid()} | ignore | {error, any()}.
start_link(Id) ->
gen_server2:start_link({local, name(Id)}, ?MODULE, [Id], []).
-spec start_link(atom(), pos_integer()) -> {ok, pid()} | ignore | {error, any()}.
start_link(Pool, Id) ->
gen_server2:start_link({local, name(Id)}, ?MODULE, [Pool, Id], []).
name(Id) ->
list_to_atom("emqttd_sm_" ++ integer_to_list(Id)).
%%------------------------------------------------------------------------------
%% @doc Pool name.
%% @end
%%------------------------------------------------------------------------------
pool() -> ?SM_POOL.
%%------------------------------------------------------------------------------
%% @doc Start a session
%% @end
%%------------------------------------------------------------------------------
-spec start_session(CleanSess :: boolean(), binary()) -> {ok, pid(), boolean()} | {error, any()}.
start_session(CleanSess, ClientId) ->
SM = gproc_pool:pick_worker(?SM_POOL, ClientId),
SM = gproc_pool:pick_worker(?POOL, ClientId),
call(SM, {start_session, {CleanSess, ClientId, self()}}).
%%------------------------------------------------------------------------------
@ -149,9 +141,10 @@ call(SM, Req) ->
%%% gen_server callbacks
%%%=============================================================================
init([Id]) ->
gproc_pool:connect_worker(?SM_POOL, {?MODULE, Id}),
{ok, #state{id = Id}}.
init([Pool, Id]) ->
?GPROC_POOL(join, Pool, Id),
{ok, #state{pool = Pool, id = Id,
monitors = dict:new()}}.
prioritise_call(_Msg, _From, _Len, _State) ->
1.
@ -162,50 +155,63 @@ prioritise_cast(_Msg, _Len, _State) ->
prioritise_info(_Msg, _Len, _State) ->
2.
%% persistent session
handle_call({start_session, {false, ClientId, ClientPid}}, _From, State) ->
%% Persistent Session
handle_call({start_session, Client = {false, ClientId, ClientPid}}, _From, State) ->
case lookup_session(ClientId) of
undefined ->
%% create session locally
reply(create_session(false, ClientId, ClientPid), false, State);
%% Create session locally
create_session(Client, State);
Session ->
reply(resume_session(Session, ClientPid), true, State)
case resume_session(Session, ClientPid) of
{ok, SessPid} ->
{reply, {ok, SessPid, true}, State};
{error, Erorr} ->
{reply, {error, Erorr}, State}
end
end;
%% transient session
handle_call({start_session, {true, ClientId, ClientPid}}, _From, State) ->
%% Transient Session
handle_call({start_session, Client = {true, ClientId, _ClientPid}}, _From, State) ->
case lookup_session(ClientId) of
undefined ->
reply(create_session(true, ClientId, ClientPid), false, State);
create_session(Client, State);
Session ->
case destroy_session(Session) of
ok ->
reply(create_session(true, ClientId, ClientPid), false, State);
create_session(Client, State);
{error, Error} ->
{reply, {error, Error}, State}
end
end;
handle_call(_Request, _From, State) ->
{reply, ok, State}.
handle_call(Req, _From, State) ->
?UNEXPECTED_REQ(Req, State).
handle_cast(Msg, State) ->
lager:error("Unexpected Msg: ~p", [Msg]),
{noreply, State}.
?UNEXPECTED_MSG(Msg, State).
handle_info({'DOWN', _MRef, process, DownPid, _Reason}, State) ->
handle_info({'DOWN', MRef, process, DownPid, _Reason}, State) ->
case dict:find(MRef, State#state.monitors) of
{ok, ClientId} ->
mnesia:transaction(fun() ->
[mnesia:delete_object(session, Sess, write) || Sess
<- mnesia:index_read(session, DownPid, #mqtt_session.sess_pid)]
case mnesia:wread({session, ClientId}) of
[] -> ok;
[Sess = #mqtt_session{sess_pid = DownPid}] ->
mnesia:delete_object(session, Sess, write);
[_Sess] -> ok
end
end),
{noreply, State};
{noreply, erase_monitor(MRef, State)};
error ->
lager:error("MRef of session ~p not found", [DownPid]),
{noreply, State}
end;
handle_info(Info, State) ->
lager:error("Unexpected Info: ~p", [Info]),
{noreply, State}.
?UNEXPECTED_INFO(Info, State).
terminate(_Reason, #state{id = Id}) ->
gproc_pool:disconnect_worker(?SM_POOL, {?MODULE, Id}), ok.
terminate(_Reason, #state{pool = Pool, id = Id}) ->
?GPROC_POOL(leave, Pool, Id).
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
@ -214,6 +220,16 @@ code_change(_OldVsn, State, _Extra) ->
%%% Internal functions
%%%=============================================================================
%% Create Session Locally
create_session({CleanSess, ClientId, ClientPid}, State) ->
case create_session(CleanSess, ClientId, ClientPid) of
{ok, SessPid} ->
{reply, {ok, SessPid, false},
monitor_session(ClientId, SessPid, State)};
{error, Error} ->
{reply, {error, Error}, State}
end.
create_session(CleanSess, ClientId, ClientPid) ->
case emqttd_session_sup:start_session(CleanSess, ClientId, ClientPid) of
{ok, SessPid} ->
@ -226,7 +242,6 @@ create_session(CleanSess, ClientId, ClientPid) ->
lager:error("SM(~s): Conflict with ~p", [ClientId, ConflictPid]),
{error, mnesia_conflict};
{atomic, ok} ->
erlang:monitor(process, SessPid),
{ok, SessPid}
end;
{error, Error} ->
@ -301,8 +316,10 @@ remove_session(Session) ->
{aborted, Error} -> {error, Error}
end.
reply({ok, SessPid}, SP, State) ->
{reply, {ok, SessPid, SP}, State};
reply({error, Error}, _SP, State) ->
{reply, {error, Error}, State}.
monitor_session(ClientId, SessPid, State = #state{monitors = Monitors}) ->
MRef = erlang:monitor(process, SessPid),
State#state{monitors = dict:store(MRef, ClientId, Monitors)}.
erase_monitor(MRef, State = #state{monitors = Monitors}) ->
State#state{monitors = dict:erase(MRef, Monitors)}.

View File

@ -19,23 +19,22 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd session helper.
%%% @doc Session Helper.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_sm_helper).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(gen_server).
-include("emqttd.hrl").
-include("emqttd_internal.hrl").
-include_lib("stdlib/include/ms_transform.hrl").
%% API Function Exports
-export([start_link/0]).
-behaviour(gen_server).
-export([start_link/1]).
%% gen_server Function Exports
-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
@ -47,22 +46,20 @@
%% @doc Start a session helper
%% @end
%%------------------------------------------------------------------------------
-spec start_link() -> {ok, pid()} | ignore | {error, any()}.
start_link() ->
gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).
-spec start_link(fun()) -> {ok, pid()} | ignore | {error, any()}.
start_link(StatsFun) ->
gen_server:start_link({local, ?MODULE}, ?MODULE, [StatsFun], []).
init([]) ->
init([StatsFun]) ->
mnesia:subscribe(system),
{ok, TRef} = timer:send_interval(timer:seconds(1), tick),
StatsFun = emqttd_stats:statsfun('sessions/count', 'sessions/max'),
{ok, #state{stats_fun = StatsFun, tick_tref = TRef}}.
handle_call(_Request, _From, State) ->
{reply, ok, State}.
handle_call(Req, _From, State) ->
?UNEXPECTED_REQ(Req, State).
handle_cast(Msg, State) ->
lager:error("Unexpected Msg: ~p", [Msg]),
{noreply, State}.
?UNEXPECTED_MSG(Msg, State).
handle_info({mnesia_system_event, {mnesia_down, Node}}, State) ->
lager:error("!!!Mnesia node down: ~s", [Node]),
@ -83,8 +80,7 @@ handle_info(tick, State) ->
{noreply, setstats(State), hibernate};
handle_info(Info, State) ->
lager:error("Unexpected Info: ~p", [Info]),
{noreply, State}.
?UNEXPECTED_INFO(Info, State).
terminate(_Reason, _State = #state{tick_tref = TRef}) ->
timer:cancel(TRef),

View File

@ -19,25 +19,26 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd session manager supervisor.
%%% @doc Session Manager Supervisor.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_sm_sup).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(supervisor).
-include("emqttd.hrl").
-define(CHILD(Mod), {Mod, {Mod, start_link, []},
permanent, 5000, worker, [Mod]}).
-define(SM, emqttd_sm).
-define(HELPER, emqttd_sm_helper).
-define(TABS, [mqtt_transient_session,
mqtt_persistent_session]).
%% API
-export([start_link/0]).
-behaviour(supervisor).
%% Supervisor callbacks
-export([init/1]).
@ -45,20 +46,22 @@ start_link() ->
supervisor:start_link({local, ?MODULE}, ?MODULE, []).
init([]) ->
init_session_ets(),
Schedulers = erlang:system_info(schedulers),
gproc_pool:new(emqttd_sm:pool(), hash, [{size, Schedulers}]),
Managers = lists:map(
fun(I) ->
Name = {emqttd_sm, I},
gproc_pool:add_worker(emqttd_sm:pool(), Name, I),
{Name, {emqttd_sm, start_link, [I]},
permanent, 10000, worker, [emqttd_sm]}
end, lists:seq(1, Schedulers)),
{ok, {{one_for_all, 10, 100}, [?CHILD(emqttd_sm_helper) | Managers]}}.
%% Create session tables
create_session_tabs(),
init_session_ets() ->
Tables = [mqtt_transient_session, mqtt_persistent_session],
Attrs = [ordered_set, named_table, public, {write_concurrency, true}],
lists:foreach(fun(Tab) -> ets:new(Tab, Attrs) end, Tables).
%% Helper
StatsFun = emqttd_stats:statsfun('sessions/count', 'sessions/max'),
Helper = {?HELPER, {?HELPER, start_link, [StatsFun]},
permanent, 5000, worker, [?HELPER]},
%% SM Pool Sup
MFA = {?SM, start_link, []},
PoolSup = emqttd_pool_sup:spec([?SM, hash, erlang:system_info(schedulers), MFA]),
{ok, {{one_for_all, 10, 3600}, [Helper, PoolSup]}}.
create_session_tabs() ->
Opts = [ordered_set, named_table, public,
{write_concurrency, true}],
[ets:new(Tab, Opts) || Tab <- ?TABS].

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd statistics.
%%% @doc emqttd statistics
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_stats).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-behaviour(gen_server).
@ -63,10 +60,12 @@
%% $SYS Topics for Subscribers
-define(SYSTOP_PUBSUB, [
'routes/count', % ...
'routes/reverse', % ...
'topics/count', % ...
'topics/max', % ...
'subscribers/count', % ...
'subscribers/max', % ...
'subscriptions/count', % ...
'subscriptions/max', % ...
'queues/count', % ...
'queues/max' % ...
]).
@ -141,12 +140,12 @@ setstats(Stat, MaxStat, Val) ->
%%%=============================================================================
init([]) ->
random:seed(now()),
random:seed(os:timestamp()),
ets:new(?STATS_TAB, [set, public, named_table, {write_concurrency, true}]),
Topics = ?SYSTOP_CLIENTS ++ ?SYSTOP_SESSIONS ++ ?SYSTOP_PUBSUB ++ ?SYSTOP_RETAINED,
ets:insert(?STATS_TAB, [{Topic, 0} || Topic <- Topics]),
% Create $SYS Topics
[ok = emqttd_pubsub:create(stats_topic(Topic)) || Topic <- Topics],
[ok = emqttd_pubsub:create(topic, stats_topic(Topic)) || Topic <- Topics],
% Tick to publish stats
{ok, #state{tick_tref = emqttd_broker:start_tick(tick)}, hibernate}.

View File

@ -19,19 +19,16 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd supervisor.
%%% @doc emqttd top supervisor.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_sup).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(supervisor).
-include("emqttd.hrl").
-behaviour(supervisor).
%% API
-export([start_link/0, start_child/1, start_child/2]).

View File

@ -19,23 +19,30 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd system monitor.
%%% @doc VM System Monitor
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_sysmon).
-author("Feng Lee <feng@emqtt.io>").
-behavior(gen_server).
-include("emqttd_internal.hrl").
-export([start_link/1]).
-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
terminate/2, code_change/3]).
-record(state, {tick_tref, events = []}).
-record(state, {tickref, events = [], tracelog}).
-define(LOG_FMT, [{formatter_config, [time, " ", message, "\n"]}]).
-define(LOG(Msg, ProcInfo),
lager:warning([{sysmon, true}], "~s~n~p", [WarnMsg, ProcInfo])).
-define(LOG(Msg, ProcInfo, PortInfo),
lager:warning([{sysmon, true}], "~s~n~p~n~p", [WarnMsg, ProcInfo, PortInfo])).
%%------------------------------------------------------------------------------
%% @doc Start system monitor
@ -53,7 +60,9 @@ start_link(Opts) ->
init([Opts]) ->
erlang:system_monitor(self(), parse_opt(Opts)),
{ok, TRef} = timer:send_interval(timer:seconds(1), reset),
{ok, #state{tick_tref = TRef}}.
%%TODO: don't trace for performance issue.
%%{ok, TraceLog} = start_tracelog(proplists:get_value(logfile, Opts)),
{ok, #state{tickref = TRef}}.
parse_opt(Opts) ->
parse_opt(Opts, []).
@ -63,6 +72,8 @@ parse_opt([{long_gc, false}|Opts], Acc) ->
parse_opt(Opts, Acc);
parse_opt([{long_gc, Ms}|Opts], Acc) when is_integer(Ms) ->
parse_opt(Opts, [{long_gc, Ms}|Acc]);
parse_opt([{long_schedule, false}|Opts], Acc) ->
parse_opt(Opts, Acc);
parse_opt([{long_schedule, Ms}|Opts], Acc) when is_integer(Ms) ->
parse_opt(Opts, [{long_schedule, Ms}|Acc]);
parse_opt([{large_heap, Size}|Opts], Acc) when is_integer(Size) ->
@ -74,55 +85,55 @@ parse_opt([{busy_port, false}|Opts], Acc) ->
parse_opt([{busy_dist_port, true}|Opts], Acc) ->
parse_opt(Opts, [busy_dist_port|Acc]);
parse_opt([{busy_dist_port, false}|Opts], Acc) ->
parse_opt(Opts, Acc);
parse_opt([_Opt|Opts], Acc) ->
parse_opt(Opts, Acc).
handle_call(Request, _From, State) ->
lager:error("Unexpected request: ~p", [Request]),
{reply, {error, unexpected_request}, State}.
handle_call(Req, _From, State) ->
?UNEXPECTED_REQ(Req, State).
handle_cast(Msg, State) ->
lager:error("Unexpected msg: ~p", [Msg]),
{noreply, State}.
?UNEXPECTED_MSG(Msg, State).
handle_info({monitor, Pid, long_gc, Info}, State) ->
suppress({long_gc, Pid}, fun() ->
WarnMsg = io_lib:format("long_gc: pid = ~p, info: ~p", [Pid, Info]),
lager:error("~s~n~p", [WarnMsg, procinfo(Pid)]),
WarnMsg = io_lib:format("long_gc warning: pid = ~p, info: ~p", [Pid, Info]),
?LOG(WarnMsg, procinfo(Pid)),
publish(long_gc, WarnMsg)
end, State);
handle_info({monitor, Pid, long_schedule, Info}, State) when is_pid(Pid) ->
suppress({long_schedule, Pid}, fun() ->
WarnMsg = io_lib:format("long_schedule warning: pid = ~p, info: ~p", [Pid, Info]),
lager:error("~s~n~p", [WarnMsg, procinfo(Pid)]),
?LOG(WarnMsg, procinfo(Pid)),
publish(long_schedule, WarnMsg)
end, State);
handle_info({monitor, Port, long_schedule, Info}, State) when is_port(Port) ->
suppress({long_schedule, Port}, fun() ->
WarnMsg = io_lib:format("long_schedule warning: port = ~p, info: ~p", [Port, Info]),
lager:error("~s~n~p", [WarnMsg, erlang:port_info(Port)]),
?LOG(WarnMsg, erlang:port_info(Port)),
publish(long_schedule, WarnMsg)
end, State);
handle_info({monitor, Pid, large_heap, Info}, State) ->
suppress({large_heap, Pid}, fun() ->
WarnMsg = io_lib:format("large_heap warning: pid = ~p, info: ~p", [Pid, Info]),
lager:error("~s~n~p", [WarnMsg, procinfo(Pid)]),
?LOG(WarnMsg, procinfo(Pid)),
publish(large_heap, WarnMsg)
end, State);
handle_info({monitor, SusPid, busy_port, Port}, State) ->
suppress({busy_port, Port}, fun() ->
WarnMsg = io_lib:format("busy_port warning: suspid = ~p, port = ~p", [SusPid, Port]),
lager:error("~s~n~p~n~p", [WarnMsg, procinfo(SusPid), erlang:port_info(Port)]),
?LOG(WarnMsg, procinfo(SusPid), erlang:port_info(Port)),
publish(busy_port, WarnMsg)
end, State);
handle_info({monitor, SusPid, busy_dist_port, Port}, State) ->
suppress({busy_dist_port, Port}, fun() ->
WarnMsg = io_lib:format("busy_dist_port warning: suspid = ~p, port = ~p", [SusPid, Port]),
lager:error("~s~n~p~n~p", [WarnMsg, procinfo(SusPid), erlang:port_info(Port)]),
?LOG(WarnMsg, procinfo(SusPid), erlang:port_info(Port)),
publish(busy_dist_port, WarnMsg)
end, State);
@ -130,11 +141,11 @@ handle_info(reset, State) ->
{noreply, State#state{events = []}};
handle_info(Info, State) ->
lager:error("Unexpected info: ~p", [Info]),
{noreply, State}.
?UNEXPECTED_INFO(Info, State).
terminate(_Reason, #state{tick_tref = TRef}) ->
timer:cancel(TRef).
terminate(_Reason, #state{tickref = TRef, tracelog = TraceLog}) ->
timer:cancel(TRef),
cancel_tracelog(TraceLog).
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
@ -162,3 +173,13 @@ publish(Sysmon, WarnMsg) ->
topic(Sysmon) ->
emqttd_topic:systop(list_to_binary(lists:concat(['sysmon/', Sysmon]))).
start_tracelog(undefined) ->
{ok, undefined};
start_tracelog(LogFile) ->
lager:trace_file(LogFile, [{sysmon, true}], info, ?LOG_FMT).
cancel_tracelog(undefined) ->
ok;
cancel_tracelog(TraceLog) ->
lager:stop_trace(TraceLog).

View File

@ -19,40 +19,26 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd pooler supervisor.
%%% @doc emqttd sysmon supervisor.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_pooler_sup).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-module(emqttd_sysmon_sup).
-behaviour(supervisor).
%% API
-export([start_link/0, start_link/1]).
-export([start_link/0]).
%% Supervisor callbacks
-export([init/1]).
start_link() ->
start_link(erlang:system_info(schedulers)).
start_link(PoolSize) ->
supervisor:start_link({local, ?MODULE}, ?MODULE, [PoolSize]).
init([PoolSize]) ->
gproc_pool:new(pooler, random, [{size, PoolSize}]),
Children = lists:map(
fun(I) ->
gproc_pool:add_worker(pooler, {pooler, I}, I),
{{emqttd_pooler, I},
{emqttd_pooler, start_link, [I]},
permanent, 5000, worker, [emqttd_pooler]}
end, lists:seq(1, PoolSize)),
{ok, {{one_for_all, 10, 100}, Children}}.
supervisor:start_link({local, ?MODULE}, ?MODULE, []).
init([]) ->
Env = emqttd:env(sysmon),
{ok, {{one_for_one, 10, 100},
[{sysmon, {emqttd_sysmon, start_link, [Env]},
permanent, 5000, worker, [emqttd_sysmon]}]}}.

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% MQTT Topic Functions
%%% @doc MQTT Topic Functions
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_topic).
-author("Feng Lee <feng@emqtt.io>").
-import(lists, [reverse/1]).
-export([match/2, validate/1, triples/1, words/1, wildcard/1]).

View File

@ -19,27 +19,26 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% Trace MQTT packets/messages by clientid or topic.
%%% @doc Trace MQTT packets/messages by ClientID or Topic.
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_trace).
-author("Feng Lee <feng@emqtt.io>").
-behaviour(gen_server).
-include("emqttd_internal.hrl").
%% API Function Exports
-export([start_link/0]).
-export([start_trace/2, stop_trace/1, all_traces/0]).
-behaviour(gen_server).
%% gen_server Function Exports
-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
terminate/2, code_change/3]).
-record(state, {level, trace_map}).
-record(state, {level, traces}).
-type trace_who() :: {client | topic, binary()}.
@ -85,41 +84,42 @@ all_traces() ->
gen_server:call(?MODULE, all_traces).
init([]) ->
{ok, #state{level = info, trace_map = #{}}}.
{ok, #state{level = info, traces = #{}}}.
handle_call({start_trace, Who, LogFile}, _From, State = #state{level = Level, trace_map = TraceMap}) ->
handle_call({start_trace, Who, LogFile}, _From, State = #state{level = Level, traces = Traces}) ->
case lager:trace_file(LogFile, [Who], Level, ?TRACE_OPTIONS) of
{ok, exists} ->
{reply, {error, existed}, State};
{ok, Trace} ->
{reply, ok, State#state{trace_map = maps:put(Who, {Trace, LogFile}, TraceMap)}};
{reply, ok, State#state{traces = maps:put(Who, {Trace, LogFile}, Traces)}};
{error, Error} ->
{reply, {error, Error}, State}
end;
handle_call({stop_trace, Who}, _From, State = #state{trace_map = TraceMap}) ->
case maps:find(Who, TraceMap) of
handle_call({stop_trace, Who}, _From, State = #state{traces = Traces}) ->
case maps:find(Who, Traces) of
{ok, {Trace, _LogFile}} ->
case lager:stop_trace(Trace) of
ok -> ok;
{error, Error} -> lager:error("Stop trace ~p error: ~p", [Who, Error])
end,
{reply, ok, State#state{trace_map = maps:remove(Who, TraceMap)}};
{reply, ok, State#state{traces = maps:remove(Who, Traces)}};
error ->
{reply, {error, not_found}, State}
end;
handle_call(all_traces, _From, State = #state{trace_map = TraceMap}) ->
{reply, [{Who, LogFile} || {Who, {_Trace, LogFile}} <- maps:to_list(TraceMap)], State};
handle_call(all_traces, _From, State = #state{traces = Traces}) ->
{reply, [{Who, LogFile} || {Who, {_Trace, LogFile}}
<- maps:to_list(Traces)], State};
handle_call(_Req, _From, State) ->
{reply, error, State}.
handle_call(Req, _From, State) ->
?UNEXPECTED_REQ(Req, State).
handle_cast(_Msg, State) ->
{noreply, State}.
handle_cast(Msg, State) ->
?UNEXPECTED_MSG(Msg, State).
handle_info(_Info, State) ->
{noreply, State}.
handle_info(Info, State) ->
?UNEXPECTED_INFO(Info, State).
terminate(_Reason, _State) ->
ok.

43
src/emqttd_trace_sup.erl Normal file
View File

@ -0,0 +1,43 @@
%%%-----------------------------------------------------------------------------
%%% Copyright (c) 2012-2015 eMQTT.IO, All Rights Reserved.
%%%
%%% Permission is hereby granted, free of charge, to any person obtaining a copy
%%% of this software and associated documentation files (the "Software"), to deal
%%% in the Software without restriction, including without limitation the rights
%%% to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
%%% copies of the Software, and to permit persons to whom the Software is
%%% furnished to do so, subject to the following conditions:
%%%
%%% The above copyright notice and this permission notice shall be included in all
%%% copies or substantial portions of the Software.
%%%
%%% THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
%%% IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
%%% FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
%%% AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
%%% LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc emqttd trace supervisor.
%%%
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_trace_sup).
-behaviour(supervisor).
%% API
-export([start_link/0]).
%% Supervisor callbacks
-export([init/1]).
start_link() ->
supervisor:start_link({local, ?MODULE}, ?MODULE, []).
init([]) ->
{ok, {{one_for_one, 10, 100},
[{trace, {emqttd_trace, start_link, []},
permanent, 5000, worker, [emqttd_trace]}]}}.

View File

@ -20,16 +20,16 @@
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% MQTT Topic Trie Tree.
%%% MQTT Topic Trie.
%%%
%%% [Trie](http://en.wikipedia.org/wiki/Trie)
%%%
%%% @end
%%%
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_trie).
-author("Feng Lee <feng@emqtt.io>").
%% Mnesia Callbacks
-export([mnesia/1]).
@ -62,16 +62,17 @@
%%%=============================================================================
%%------------------------------------------------------------------------------
%% @doc Create trie tables
%% @doc Create Trie Tables
%% @end
%%------------------------------------------------------------------------------
-spec mnesia(boot | copy) -> ok.
mnesia(boot) ->
%% trie tree tables
%% Trie Table
ok = emqttd_mnesia:create_table(trie, [
{ram_copies, [node()]},
{record_name, trie},
{attributes, record_info(fields, trie)}]),
%% Trie Node Table
ok = emqttd_mnesia:create_table(trie_node, [
{ram_copies, [node()]},
{record_name, trie_node},
@ -82,7 +83,9 @@ mnesia(boot) ->
%% @end
%%------------------------------------------------------------------------------
mnesia(copy) ->
%% Copy Trie Table
ok = emqttd_mnesia:copy_table(trie),
%% Copy Trie Node Table
ok = emqttd_mnesia:copy_table(trie_node).
%%%=============================================================================
@ -117,7 +120,7 @@ match(Topic) when is_binary(Topic) ->
[Name || #trie_node{topic=Name} <- TrieNodes, Name =/= undefined].
%%------------------------------------------------------------------------------
%% @doc Delete topic from trie tree
%% @doc Delete topic from trie
%% @end
%%------------------------------------------------------------------------------
-spec delete(Topic :: binary()) -> ok.

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd utility functions.
%%% @doc emqttd utility functions
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_util).
-author("Feng Lee <feng@emqtt.io>").
-export([apply_module_attributes/1,
all_module_attributes/1,
cancel_timer/1,

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd erlang vm.
%%% @doc emqttd erlang vm.
%%%
%%% @end
%%% @author huangdan
%%%-----------------------------------------------------------------------------
-module(emqttd_vm).
-author('huangdan').
-export([schedulers/0]).
-export([microsecs/0]).
@ -36,23 +33,15 @@
-export([get_memory/0]).
-export([get_process_list/0,
get_process_info/0,
get_process_info/1,
get_process_gc/0,
get_process_gc/1,
-export([get_process_list/0, get_process_info/0, get_process_info/1,
get_process_gc/0, get_process_gc/1,
get_process_group_leader_info/1,
get_process_limit/0]).
-export([get_ets_list/0,
get_ets_info/0,
get_ets_info/1,
get_ets_object/0,
get_ets_object/1]).
-export([get_ets_list/0, get_ets_info/0, get_ets_info/1,
get_ets_object/0, get_ets_object/1]).
-export([get_port_types/0,
get_port_info/0,
get_port_info/1]).
-export([get_port_types/0, get_port_info/0, get_port_info/1]).
-define(UTIL_ALLOCATORS, [temp_alloc,
eheap_alloc,
@ -195,8 +184,7 @@ format_system_info(allocator, {_,_,_,List}) ->
List;
format_system_info(dist_ctrl, List) ->
lists:map(fun({Node, Socket}) ->
{ok, Stats} = inet:getstat(Socket),
{Node, Stats}
{ok, Stats} = inet:getstat(Socket), {Node, Stats}
end, List);
format_system_info(driver_version, Value) ->
list_to_binary(Value);
@ -241,10 +229,9 @@ scheduler_usage(Interval) when is_integer(Interval) ->
scheduler_usage_diff(First, Last).
scheduler_usage_diff(First, Last) ->
lists:map(
fun({{I, A0, T0},{I, A1, T1}}) ->{I, (A1 - A0)/(T1 - T0)}end,
lists:zip(lists:sort(First), lists:sort(Last))
).
lists:map(fun({{I, A0, T0},{I, A1, T1}}) ->
{I, (A1 - A0)/(T1 - T0)}
end, lists:zip(lists:sort(First), lists:sort(Last))).
get_memory()->
[{Key, get_memory(Key, current)} || Key <- [used, allocated, unused, usage]] ++ erlang:memory().
@ -278,10 +265,8 @@ allocators() ->
UtilAllocators = erlang:system_info(alloc_util_allocators),
Allocators = [sys_alloc, mseg_alloc|UtilAllocators],
[{{A, N},lists:sort(proplists:delete(versions, Props))} ||
A <- Allocators,
Allocs <- [erlang:system_info({allocator, A})],
Allocs =/= false,
{_, N, Props} <- Allocs].
A <- Allocators, Allocs <- [erlang:system_info({allocator, A})],
Allocs =/= false, {_, N, Props} <- Allocs].
container_size(Prop, Keyword, Container) ->
Sbcs = container_value(Prop, Keyword, sbcs, Container),
@ -294,7 +279,8 @@ container_value(Props, Pos, mbcs = Type, Container) when is_integer(Pos)->
Pool = case proplists:get_value(mbcs_pool, Props) of
PoolProps when PoolProps =/= undefined ->
element(Pos, lists:keyfind(Container, 1, PoolProps));
_ -> 0
_ ->
0
end,
TypeProps = proplists:get_value(Type, Props),
Pool + element(Pos, lists:keyfind(Container, 1, TypeProps));
@ -388,7 +374,6 @@ port_info(PortTerm, specific) ->
case catch inet:getstat(Port) of
{ok, Stats} -> [{statistics, Stats}];
_ -> []
end ++
case catch inet:peername(Port) of
{ok, Peer} -> [{peername, Peer}];
@ -404,7 +389,8 @@ case catch inet:getopts(Port, ?SOCKET_OPTS ) of
end;
{_, "efile"} ->
[];
_ ->[]
_ ->
[]
end,
{specific, Props};
port_info(PortTerm, Keys) when is_list(Keys) ->
@ -426,11 +412,7 @@ transform_port(N) when is_integer(N) ->
Name = iolist_to_binary(atom_to_list(node())),
NameLen = iolist_size(Name),
Vsn = binary:last(term_to_binary(self())),
Bin = <<131, 102, 100,
NameLen:2/unit:8,
Name:NameLen/binary,
N:4/unit:8,
Vsn:8>>,
Bin = <<131, 102, 100, NameLen:2/unit:8, Name:NameLen/binary, N:4/unit:8, Vsn:8>>,
binary_to_term(Bin).
ports_type_list() ->

View File

@ -19,15 +19,12 @@
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd websocket client.
%%% @doc emqttd websocket client
%%%
%%% @end
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(emqttd_ws_client).
-author("Feng Lee <feng@emqtt.io>").
-include("emqttd.hrl").
-include("emqttd_protocol.hrl").

View File

@ -0,0 +1,96 @@
%%%-----------------------------------------------------------------------------
%%% Copyright (c) 2012-2015 eMQTT.IO, All Rights Reserved.
%%%
%%% Permission is hereby granted, free of charge, to any person obtaining a copy
%%% of this software and associated documentation files (the "Software"), to deal
%%% in the Software without restriction, including without limitation the rights
%%% to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
%%% copies of the Software, and to permit persons to whom the Software is
%%% furnished to do so, subject to the following conditions:
%%%
%%% The above copyright notice and this permission notice shall be included in all
%%% copies or substantial portions of the Software.
%%%
%%% THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
%%% IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
%%% FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
%%% AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
%%% LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
%%% OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc emqtt lager backend
%%%
%%% @author Feng Lee <feng@emqtt.io>
%%%-----------------------------------------------------------------------------
-module(lager_emqtt_backend).
-behaviour(gen_event).
-include_lib("lager/include/lager.hrl").
-export([init/1, handle_call/2, handle_event/2, handle_info/2,
terminate/2, code_change/3]).
-record(state, {level :: {'mask', integer()},
formatter :: atom(),
format_config :: any()}).
-define(DEFAULT_FORMAT, [time, " ", pid, " [",severity, "] ", message]).
init([Level]) when is_atom(Level) ->
init(Level);
init(Level) when is_atom(Level) ->
init([Level,{lager_default_formatter, ?DEFAULT_FORMAT}]);
init([Level,{Formatter, FormatterConfig}]) when is_atom(Formatter) ->
Levels = lager_util:config_to_mask(Level),
{ok, #state{level = Levels, formatter = Formatter,
format_config = FormatterConfig}}.
handle_call(get_loglevel, #state{level = Level} = State) ->
{ok, Level, State};
handle_call({set_loglevel, Level}, State) ->
try lager_util:config_to_mask(Level) of
Levels -> {ok, ok, State#state{level = Levels}}
catch
_:_ -> {ok, {error, bad_log_level}, State}
end;
handle_call(_Request, State) ->
{ok, ok, State}.
handle_event({log, Message}, State = #state{level = L}) ->
case lager_util:is_loggable(Message, L, ?MODULE) of
true ->
publish_log(Message, State);
false ->
{ok, State}
end;
handle_event(_Event, State) ->
{ok, State}.
handle_info(_Info, State) ->
{ok, State}.
terminate(_Reason, _State) ->
ok.
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
publish_log(Message, State = #state{formatter = Formatter,
format_config = FormatConfig}) ->
Severity = lager_msg:severity(Message),
Payload = Formatter:format(Message, FormatConfig),
emqttd_pubsub:publish(
emqttd_message:make(
log, topic(Severity), iolist_to_binary(Payload))),
{ok, State}.
topic(Severity) ->
emqttd_topic:systop(list_to_binary(lists:concat(['logs/', Severity]))).

View File

@ -20,11 +20,11 @@
%%% SOFTWARE.
%%%-----------------------------------------------------------------------------
%%% @doc
%%% emqttd_serialiser tests.
%%% emqttd_serializer tests.
%%%
%%% @end
%%%-----------------------------------------------------------------------------
-module(emqttd_serialiser_tests).
-module(emqttd_serializer_tests).
-ifdef(TEST).
@ -32,47 +32,47 @@
-include_lib("eunit/include/eunit.hrl").
-import(emqttd_serialiser, [serialise/1]).
-import(emqttd_serializer, [serialize/1]).
serialise_connect_test() ->
serialise(?CONNECT_PACKET(#mqtt_packet_connect{})).
serilize_connect_test() ->
serilize(?CONNECT_PACKET(#mqtt_packet_connect{})).
serialise_connack_test() ->
serilize_connack_test() ->
ConnAck = #mqtt_packet{header = #mqtt_packet_header{type = ?CONNACK},
variable = #mqtt_packet_connack{ack_flags = 0, return_code = 0}},
?assertEqual(<<32,2,0,0>>, serialise(ConnAck)).
?assertEqual(<<32,2,0,0>>, serilize(ConnAck)).
serialise_publish_test() ->
serialise(?PUBLISH_PACKET(?QOS_0, <<"Topic">>, undefined, <<"Payload">>)),
serialise(?PUBLISH_PACKET(?QOS_1, <<"Topic">>, 938, <<"Payload">>)).
serilize_publish_test() ->
serilize(?PUBLISH_PACKET(?QOS_0, <<"Topic">>, undefined, <<"Payload">>)),
serilize(?PUBLISH_PACKET(?QOS_1, <<"Topic">>, 938, <<"Payload">>)).
serialise_puback_test() ->
serialise(?PUBACK_PACKET(?PUBACK, 10384)).
serilize_puback_test() ->
serilize(?PUBACK_PACKET(?PUBACK, 10384)).
serialise_pubrel_test() ->
serialise(?PUBREL_PACKET(10384)).
serilize_pubrel_test() ->
serilize(?PUBREL_PACKET(10384)).
serialise_subscribe_test() ->
serilize_subscribe_test() ->
TopicTable = [{<<"TopicQos0">>, ?QOS_0}, {<<"TopicQos1">>, ?QOS_1}, {<<"TopicQos2">>, ?QOS_2}],
serialise(?SUBSCRIBE_PACKET(10, TopicTable)).
serilize(?SUBSCRIBE_PACKET(10, TopicTable)).
serialise_suback_test() ->
serialise(?SUBACK_PACKET(10, [?QOS_0, ?QOS_1, 128])).
serilize_suback_test() ->
serilize(?SUBACK_PACKET(10, [?QOS_0, ?QOS_1, 128])).
serialise_unsubscribe_test() ->
serialise(?UNSUBSCRIBE_PACKET(10, [<<"Topic1">>, <<"Topic2">>])).
serilize_unsubscribe_test() ->
serilize(?UNSUBSCRIBE_PACKET(10, [<<"Topic1">>, <<"Topic2">>])).
serialise_unsuback_test() ->
serialise(?UNSUBACK_PACKET(10)).
serilize_unsuback_test() ->
serilize(?UNSUBACK_PACKET(10)).
serialise_pingreq_test() ->
serialise(?PACKET(?PINGREQ)).
serilize_pingreq_test() ->
serilize(?PACKET(?PINGREQ)).
serialise_pingresp_test() ->
serialise(?PACKET(?PINGRESP)).
serilize_pingresp_test() ->
serilize(?PACKET(?PINGRESP)).
serialise_disconnect_test() ->
serialise(?PACKET(?DISCONNECT)).
serilize_disconnect_test() ->
serilize(?PACKET(?DISCONNECT)).
-endif.