refactor(emqx_trie): store only prefixes

This commit refactors emqx_trie implementation
with mainly two changes

1. Deleted the edge table.

   In the old implementation, trie consist of nodes and edges.
   e.g. for topic 'a/b/+/d', node 'a' with edge 'b'
   points to node 'a/b' and so on.

   However, edges can be computed at runtime,
   so there is no need to store them in a table.
This commit is contained in:
Zaiming Shi 2021-04-19 02:18:35 +02:00 committed by Zaiming (Stone) Shi
parent 7128bc9e6e
commit bc6a87946c
5 changed files with 294 additions and 205 deletions

View File

@ -89,31 +89,6 @@
dest :: node() | {binary(), node()} dest :: node() | {binary(), node()}
}). }).
%%--------------------------------------------------------------------
%% Trie
%%--------------------------------------------------------------------
-type(trie_node_id() :: binary() | atom()).
-record(trie_node, {
node_id :: trie_node_id(),
edge_count = 0 :: non_neg_integer(),
topic :: binary() | undefined,
flags :: list(atom()) | undefined
}).
-record(trie_edge, {
node_id :: trie_node_id(),
word :: binary() | atom()
}).
-record(trie, {
edge :: #trie_edge{},
node_id :: trie_node_id()
}).
-type(trie_node() :: #trie_node{}).
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% Plugin %% Plugin
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------

View File

@ -2264,6 +2264,18 @@ end}.
{datatype, {enum, [key, tab, global]}} {datatype, {enum, [key, tab, global]}}
]}. ]}.
%% @doc Enable trie path compaction.
%% Enabling it significantly improves wildcard topic subscribe rate,
%% if wildcard topics have unique prefixes like: 'sensor/{{id}}/+/',
%% where ID is unique per subscriber.
%%
%% Topic match performance (when publishing) may degrade if messages
%% are mostly published to topics with large number of levels.
{mapping, "broker.perf.trie_compaction", "emqx.trie_compaction", [
{default, true},
{datatype, {enum, [true, false]}}
]}.
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% System Monitor %% System Monitor
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------

View File

@ -133,12 +133,11 @@ match_routes(Topic) when is_binary(Topic) ->
lists:append([lookup_routes(To) || To <- [Topic | Matched]]) lists:append([lookup_routes(To) || To <- [Topic | Matched]])
end. end.
%% @private
%% Optimize: routing table will be replicated to all router nodes. %% Optimize: routing table will be replicated to all router nodes.
match_trie(Topic) -> match_trie(Topic) ->
case emqx_trie:empty() of case emqx_trie:empty() of
true -> []; true -> [];
false -> mnesia:ets(fun emqx_trie:match/1, [Topic]) false -> emqx_trie:match(Topic)
end. end.
-spec(lookup_routes(emqx_topic:topic()) -> [emqx_types:route()]). -spec(lookup_routes(emqx_topic:topic()) -> [emqx_types:route()]).

View File

@ -27,8 +27,9 @@
%% Trie APIs %% Trie APIs
-export([ insert/1 -export([ insert/1
, match/1 , match/1
, lookup/1
, delete/1 , delete/1
, put_compaction_flag/1
, put_default_compaction_flag/0
]). ]).
-export([ empty/0 -export([ empty/0
@ -40,177 +41,270 @@
-compile(nowarn_export_all). -compile(nowarn_export_all).
-endif. -endif.
-type(triple() :: {root | binary(), emqx_topic:word(), binary()}). -define(PREFIX(Prefix), {Prefix, 0}).
-define(TOPIC(Topic), {Topic, 1}).
%% Mnesia tables -record(emqx_topic,
-define(TRIE_TAB, emqx_trie). { key :: ?TOPIC(binary()) | ?PREFIX(binary())
-define(TRIE_NODE_TAB, emqx_trie_node). , count = 0 :: non_neg_integer()
}).
-elvis([{elvis_style, function_naming_convention, disable}]). -define(TOPICS_TAB, emqx_topic).
-define(IS_COMPACT, true).
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% Mnesia bootstrap %% Mnesia bootstrap
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% @doc Create or replicate trie tables. put_compaction_flag(Bool) when is_boolean(Bool) ->
_ = persistent_term:put({?MODULE, compaction}, Bool),
ok.
put_default_compaction_flag() ->
ok = put_compaction_flag(?IS_COMPACT).
%% @doc Create or replicate topics table.
-spec(mnesia(boot | copy) -> ok). -spec(mnesia(boot | copy) -> ok).
mnesia(boot) -> mnesia(boot) ->
%% Optimize storage %% Optimize storage
StoreProps = [{ets, [{read_concurrency, true}, StoreProps = [{ets, [{read_concurrency, true},
{write_concurrency, true}]}], {write_concurrency, true}]}],
%% Trie table ok = ekka_mnesia:create_table(?TOPICS_TAB, [
ok = ekka_mnesia:create_table(?TRIE_TAB, [
{ram_copies, [node()]}, {ram_copies, [node()]},
{record_name, trie}, {record_name, emqx_topic},
{attributes, record_info(fields, trie)}, {attributes, record_info(fields, emqx_topic)},
{storage_properties, StoreProps}]),
%% Trie node table
ok = ekka_mnesia:create_table(?TRIE_NODE_TAB, [
{ram_copies, [node()]},
{record_name, trie_node},
{attributes, record_info(fields, trie_node)},
{storage_properties, StoreProps}]); {storage_properties, StoreProps}]);
mnesia(copy) -> mnesia(copy) ->
%% Copy trie table %% Copy topics table
ok = ekka_mnesia:copy_table(?TRIE_TAB, ram_copies), ok = ekka_mnesia:copy_table(?TOPICS_TAB, ram_copies).
%% Copy trie_node table
ok = ekka_mnesia:copy_table(?TRIE_NODE_TAB, ram_copies).
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% Trie APIs %% Topics APIs
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% @doc Insert a topic filter into the trie. %% @doc Insert a topic filter into the trie.
-spec(insert(emqx_topic:topic()) -> ok). -spec(insert(emqx_topic:topic()) -> ok).
insert(Topic) when is_binary(Topic) -> insert(Topic) when is_binary(Topic) ->
case mnesia:wread({?TRIE_NODE_TAB, Topic}) of {TopicKey, PrefixKeys} = make_keys(Topic),
[#trie_node{topic = Topic}] -> case mnesia:wread({?TOPICS_TAB, TopicKey}) of
ok; [_] -> ok; %% already inserted
[TrieNode = #trie_node{topic = undefined}] -> [] -> lists:foreach(fun insert_key/1, [TopicKey | PrefixKeys])
write_trie_node(TrieNode#trie_node{topic = Topic});
[] ->
%% Add trie path
ok = lists:foreach(fun add_path/1, triples(Topic)),
%% Add last node
write_trie_node(#trie_node{node_id = Topic, topic = Topic})
end. end.
%% @doc Find trie nodes that match the topic name.
-spec(match(emqx_topic:topic()) -> list(emqx_topic:topic())).
match(Topic) when is_binary(Topic) ->
TrieNodes = match_node(root, emqx_topic:words(Topic)),
[Name || #trie_node{topic = Name} <- TrieNodes, Name =/= undefined].
%% @doc Lookup a trie node.
-spec(lookup(NodeId :: binary()) -> [trie_node()]).
lookup(NodeId) ->
mnesia:read(?TRIE_NODE_TAB, NodeId).
%% @doc Delete a topic filter from the trie. %% @doc Delete a topic filter from the trie.
-spec(delete(emqx_topic:topic()) -> ok). -spec(delete(emqx_topic:topic()) -> ok).
delete(Topic) when is_binary(Topic) -> delete(Topic) when is_binary(Topic) ->
case mnesia:wread({?TRIE_NODE_TAB, Topic}) of {TopicKey, PrefixKeys} = make_keys(Topic),
[#trie_node{edge_count = 0}] -> case [] =/= mnesia:wread({?TOPICS_TAB, TopicKey}) of
ok = mnesia:delete({?TRIE_NODE_TAB, Topic}), true -> lists:foreach(fun delete_key/1, [TopicKey | PrefixKeys]);
delete_path(lists:reverse(triples(Topic))); false -> ok
[TrieNode] ->
write_trie_node(TrieNode#trie_node{topic = undefined});
[] -> ok
end. end.
%% @doc Find trie nodes that matchs the topic name.
-spec(match(emqx_topic:topic()) -> list(emqx_topic:topic())).
match(Topic) when is_binary(Topic) ->
Words = emqx_topic:words(Topic),
false = emqx_topic:wildcard(Words), %% assert
do_match(Words).
%% @doc Is the trie empty? %% @doc Is the trie empty?
-spec(empty() -> boolean()). -spec(empty() -> boolean()).
empty() -> empty() -> ets:info(?TOPICS_TAB, size) == 0.
ets:info(?TRIE_TAB, size) == 0.
-spec lock_tables() -> ok. -spec lock_tables() -> ok.
lock_tables() -> lock_tables() ->
mnesia:write_lock_table(?TRIE_TAB), mnesia:write_lock_table(?TOPICS_TAB).
mnesia:write_lock_table(?TRIE_NODE_TAB).
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% Internal functions %% Internal functions
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% Topic to triples. make_keys(Topic) ->
-spec(triples(emqx_topic:topic()) -> list(triple())). Words = emqx_topic:words(Topic),
triples(Topic) when is_binary(Topic) -> {?TOPIC(Topic), [?PREFIX(Prefix) || Prefix <- make_prefixes(Words)]}.
triples(emqx_topic:words(Topic), root, []).
triples([], _Parent, Acc) -> compact(Words) ->
lists:reverse(Acc); case is_compact() of
triples([W|Words], Parent, Acc) -> true -> do_compact(Words);
Node = join(Parent, W), false -> Words
triples(Words, Node, [{Parent, W, Node}|Acc]).
join(root, W) ->
emqx_topic:join([W]);
join(Parent, W) ->
emqx_topic:join([Parent, W]).
%% Add a path to the trie.
add_path({Node, Word, Child}) ->
Edge = #trie_edge{node_id = Node, word = Word},
case mnesia:wread({?TRIE_NODE_TAB, Node}) of
[TrieNode = #trie_node{edge_count = Count}] ->
case mnesia:wread({?TRIE_TAB, Edge}) of
[] ->
ok = write_trie_node(TrieNode#trie_node{edge_count = Count + 1}),
write_trie(#trie{edge = Edge, node_id = Child});
[_] -> ok
end;
[] ->
ok = write_trie_node(#trie_node{node_id = Node, edge_count = 1}),
write_trie(#trie{edge = Edge, node_id = Child})
end. end.
%% Match node with word or '+'. %% a/b/c/+/d/# => [a/b/c/+, d/#]
match_node(root, [NodeId = <<$$, _/binary>>|Words]) -> %% a/+/+/b => [a/+, +, b]
match_node(NodeId, Words, []); do_compact(Words) ->
do_compact(Words, empty, []).
match_node(NodeId, Words) -> do_compact([], empty, Acc) -> lists:reverse(Acc);
match_node(NodeId, Words, []). do_compact([], Seg, Acc) -> lists:reverse([Seg | Acc]);
do_compact([Word | Words], Seg, Acc) when Word =:= '+' orelse Word =:= '#' ->
do_compact(Words, empty, [join(Seg, Word) | Acc]);
do_compact([Word | Words], Seg, Acc) ->
do_compact(Words, join(Seg, Word), Acc).
match_node(NodeId, [], ResAcc) -> join(empty, '+') -> <<"+">>;
mnesia:read(?TRIE_NODE_TAB, NodeId) ++ 'match_#'(NodeId, ResAcc); join(empty, '#') -> <<"#">>;
join(empty, '') -> <<>>;
join(empty, Word) -> Word;
join(Prefix, Word) -> emqx_topic:join([Prefix, Word]).
match_node(NodeId, [W|Words], ResAcc) -> make_prefixes(Words) ->
lists:foldl(fun(WArg, Acc) -> lists:map(fun emqx_topic:join/1,
case mnesia:read(?TRIE_TAB, #trie_edge{node_id = NodeId, word = WArg}) of make_prefixes(compact(Words), [], [])).
[#trie{node_id = ChildId}] -> match_node(ChildId, Words, Acc);
[] -> Acc
end
end, 'match_#'(NodeId, ResAcc), [W, '+']).
%% Match node with '#'. make_prefixes([_LastWord], _Prefix, Acc) ->
'match_#'(NodeId, ResAcc) -> lists:map(fun lists:reverse/1, Acc);
case mnesia:read(?TRIE_TAB, #trie_edge{node_id = NodeId, word = '#'}) of make_prefixes([H | T], Prefix0, Acc0) ->
[#trie{node_id = ChildId}] -> Prefix = [H | Prefix0],
mnesia:read(?TRIE_NODE_TAB, ChildId) ++ ResAcc; Acc = [Prefix | Acc0],
[] -> ResAcc make_prefixes(T, Prefix, Acc).
end.
%% Delete paths from the trie. insert_key(Key) ->
delete_path([]) -> T = case mnesia:wread({?TOPICS_TAB, Key}) of
ok; [#emqx_topic{count = C} = T1] ->
delete_path([{NodeId, Word, _} | RestPath]) -> T1#emqx_topic{count = C + 1};
ok = mnesia:delete({?TRIE_TAB, #trie_edge{node_id = NodeId, word = Word}}),
case mnesia:wread({?TRIE_NODE_TAB, NodeId}) of
[#trie_node{edge_count = 1, topic = undefined}] ->
ok = mnesia:delete({?TRIE_NODE_TAB, NodeId}),
delete_path(RestPath);
[TrieNode = #trie_node{edge_count = 1, topic = _}] ->
write_trie_node(TrieNode#trie_node{edge_count = 0});
[TrieNode = #trie_node{edge_count = C}] ->
write_trie_node(TrieNode#trie_node{edge_count = C-1});
[] -> [] ->
mnesia:abort({node_not_found, NodeId}) #emqx_topic{key = Key, count = 1}
end,
ok = mnesia:write(T).
delete_key(Key) ->
case mnesia:wread({?TOPICS_TAB, Key}) of
[#emqx_topic{count = C} = T] when C > 1 ->
ok = mnesia:write(T#emqx_topic{count = C - 1});
[_] ->
ok = mnesia:delete(?TOPICS_TAB, Key, write);
[] ->
ok
end. end.
write_trie(Trie) -> lookup_topic(Topic) when is_binary(Topic) ->
mnesia:write(?TRIE_TAB, Trie, write). case ets:lookup(?TOPICS_TAB, ?TOPIC(Topic)) of
[#emqx_topic{count = C}] -> [Topic || C > 0];
[] -> []
end.
write_trie_node(TrieNode) -> has_prefix(empty) -> true; %% this is the virtual tree root
mnesia:write(?TRIE_NODE_TAB, TrieNode, write). has_prefix(Prefix) ->
case ets:lookup(?TOPICS_TAB, ?PREFIX(Prefix)) of
[#emqx_topic{count = C}] -> C > 0;
[] -> false
end.
do_match([<<"$", _/binary>> = Prefix | Words]) ->
%% For topics having dollar sign prefix,
%% we do not match root level + or #,
%% fast forward to the next level.
case Words =:= [] of
true -> lookup_topic(Prefix);
false -> []
end ++ do_match(Words, Prefix);
do_match(Words) ->
do_match(Words, empty).
do_match(Words, Prefix) ->
match(is_compact(), Words, Prefix, []).
match(_IsCompact, [], Topic, Acc) ->
match_any_level(Topic) ++ %% try match foo/bar/#
lookup_topic(Topic) ++ %% try match foo/bar
Acc;
match(IsCompact, [Word | Words], Prefix, Acc0) ->
case {has_prefix(Prefix), IsCompact} of
{false, false} ->
%% non-compact paths in database
%% if there is no prefix matches the current topic prefix
%% we can simpliy return from here
%% e.g. a/b/c/+ results in
%% - a
%% - a/b
%% - a/b/c
%% - a/b/c/+
%% if the input topic is to match 'a/x/y',
%% then at the second level, we lookup prefix a/x,
%% no such prefix to be found, meaning there is no point
%% searching for 'a/x/y', 'a/x/+' or 'a/x/#'
Acc0;
_ ->
%% compact paths in database
%% we have to enumerate all possible prefixes
%% e.g. a/+/b/# results with below entries in database
%% - a/+
%% - a/+/b/#
%% when matching a/x/y, we need to enumerate
%% - a
%% - a/x
%% - a/x/y
%% *with '+', '#' replaced at each level
Acc1 = match_any_level(Prefix) ++ Acc0,
Acc = match(IsCompact, Words, join(Prefix, '+'), Acc1),
match(IsCompact, Words, join(Prefix, Word), Acc)
end.
match_any_level(Prefix) ->
MlTopic = join(Prefix, '#'),
lookup_topic(MlTopic).
is_compact() ->
case persistent_term:get({?MODULE, compaction}, undefined) of
undefined ->
Default = ?IS_COMPACT,
FromEnv = emqx:get_env(trie_compaction, Default),
_ = put_compaction_flag(FromEnv),
true = is_boolean(FromEnv),
FromEnv;
Value when is_boolean(Value) ->
Value
end.
-ifdef(TEST).
-include_lib("eunit/include/eunit.hrl").
make_keys_test_() ->
[{"no compact", fun() -> with_compact_flag(false, fun make_keys_no_compact/0) end},
{"compact", fun() -> with_compact_flag(true, fun make_keys_compact/0) end}
].
make_keys_no_compact() ->
?assertEqual({?TOPIC(<<"#">>), []}, make_keys(<<"#">>)),
?assertEqual({?TOPIC(<<"a/+">>),
[?PREFIX(<<"a">>)]}, make_keys(<<"a/+">>)),
?assertEqual({?TOPIC(<<"+">>), []}, make_keys(<<"+">>)).
make_keys_compact() ->
?assertEqual({?TOPIC(<<"#">>), []}, make_keys(<<"#">>)),
?assertEqual({?TOPIC(<<"a/+">>), []}, make_keys(<<"a/+">>)),
?assertEqual({?TOPIC(<<"+">>), []}, make_keys(<<"+">>)),
?assertEqual({?TOPIC(<<"a/+/c">>),
[?PREFIX(<<"a/+">>)]}, make_keys(<<"a/+/c">>)).
words(T) -> emqx_topic:words(T).
make_prefixes_t(Topic) -> make_prefixes(words(Topic)).
with_compact_flag(IsCmopact, F) ->
put_compaction_flag(IsCmopact),
try F()
after put_default_compaction_flag()
end.
make_prefixes_test_() ->
[{"no compact", fun() -> with_compact_flag(false, fun make_prefixes_no_compact/0) end},
{"compact", fun() -> with_compact_flag(true, fun make_prefixes_compact/0) end}
].
make_prefixes_no_compact() ->
?assertEqual([<<"a/b">>, <<"a">>], make_prefixes_t(<<"a/b/+">>)),
?assertEqual([<<"a/b/+/c">>, <<"a/b/+">>, <<"a/b">>, <<"a">>],
make_prefixes_t(<<"a/b/+/c/#">>)).
make_prefixes_compact() ->
?assertEqual([], make_prefixes_t(<<"a/b/+">>)),
?assertEqual([<<"a/b/+">>], make_prefixes_t(<<"a/b/+/c/#">>)).
do_compact_test() ->
?assertEqual([<<"/+">>], do_compact(words(<<"/+">>))),
?assertEqual([<<"/#">>], do_compact(words(<<"/#">>))),
?assertEqual([<<"a/b/+">>, <<"c">>], do_compact(words(<<"a/b/+/c">>))),
?assertEqual([<<"a/+">>, <<"+">>, <<"b">>], do_compact(words(<<"a/+/+/b">>))).
-endif. % TEST

View File

@ -23,9 +23,26 @@
-include_lib("eunit/include/eunit.hrl"). -include_lib("eunit/include/eunit.hrl").
-define(TRIE, emqx_trie). -define(TRIE, emqx_trie).
-define(TRIE_TABS, [emqx_trie, emqx_trie_node]). -define(TRIE_TABS, [emqx_topic]).
all() -> emqx_ct:all(?MODULE). all() ->
[{group, compact},
{group, not_compact}
].
groups() ->
Cases = emqx_ct:all(?MODULE),
[{compact, Cases}, {not_compact, Cases}].
init_per_group(compact, Config) ->
emqx_trie:put_compaction_flag(true),
Config;
init_per_group(not_compact, Config) ->
emqx_trie:put_compaction_flag(false),
Config.
end_per_group(_, _) ->
emqx_trie:put_default_compaction_flag().
init_per_suite(Config) -> init_per_suite(Config) ->
application:load(emqx), application:load(emqx),
@ -38,6 +55,7 @@ end_per_suite(_Config) ->
ekka_mnesia:delete_schema(). ekka_mnesia:delete_schema().
init_per_testcase(_TestCase, Config) -> init_per_testcase(_TestCase, Config) ->
clear_tables(),
Config. Config.
end_per_testcase(_TestCase, _Config) -> end_per_testcase(_TestCase, _Config) ->
@ -47,50 +65,46 @@ t_mnesia(_) ->
ok = ?TRIE:mnesia(copy). ok = ?TRIE:mnesia(copy).
t_insert(_) -> t_insert(_) ->
TN = #trie_node{node_id = <<"sensor">>,
edge_count = 3,
topic = <<"sensor">>,
flags = undefined
},
Fun = fun() -> Fun = fun() ->
?TRIE:insert(<<"sensor/1/metric/2">>), ?TRIE:insert(<<"sensor/1/metric/2">>),
?TRIE:insert(<<"sensor/+/#">>), ?TRIE:insert(<<"sensor/+/#">>),
?TRIE:insert(<<"sensor/#">>), ?TRIE:insert(<<"sensor/#">>)
?TRIE:insert(<<"sensor">>),
?TRIE:insert(<<"sensor">>),
?TRIE:lookup(<<"sensor">>)
end, end,
?assertEqual({atomic, [TN]}, trans(Fun)). ?assertEqual({atomic, ok}, trans(Fun)),
?assertEqual([<<"sensor/#">>], ?TRIE:match(<<"sensor">>)).
t_match(_) -> t_match(_) ->
Machted = [<<"sensor/+/#">>, <<"sensor/#">>], Machted = [<<"sensor/#">>, <<"sensor/+/#">>],
Fun = fun() -> trans(fun() ->
?TRIE:insert(<<"sensor/1/metric/2">>), ?TRIE:insert(<<"sensor/1/metric/2">>),
?TRIE:insert(<<"sensor/+/#">>), ?TRIE:insert(<<"sensor/+/#">>),
?TRIE:insert(<<"sensor/#">>), ?TRIE:insert(<<"sensor/#">>)
?TRIE:match(<<"sensor/1">>) end),
end, ?assertEqual(Machted, lists:sort(?TRIE:match(<<"sensor/1">>))).
?assertEqual({atomic, Machted}, trans(Fun)).
t_match2(_) -> t_match2(_) ->
Matched = {[<<"+/+/#">>, <<"+/#">>, <<"#">>], []}, Matched = [<<"#">>, <<"+/#">>, <<"+/+/#">>],
Fun = fun() -> trans(fun() ->
?TRIE:insert(<<"#">>), ?TRIE:insert(<<"#">>),
?TRIE:insert(<<"+/#">>), ?TRIE:insert(<<"+/#">>),
?TRIE:insert(<<"+/+/#">>), ?TRIE:insert(<<"+/+/#">>)
{?TRIE:match(<<"a/b/c">>), end),
?TRIE:match(<<"$SYS/broker/zenmq">>)} ?assertEqual(Matched, lists:sort(?TRIE:match(<<"a/b/c">>))),
end, ?assertEqual([], ?TRIE:match(<<"$SYS/broker/zenmq">>)).
?assertEqual({atomic, Matched}, trans(Fun)).
t_match3(_) -> t_match3(_) ->
Topics = [<<"d/#">>, <<"a/b/c">>, <<"a/b/+">>, <<"a/#">>, <<"#">>, <<"$SYS/#">>], Topics = [<<"d/#">>, <<"a/b/c">>, <<"a/b/+">>, <<"a/#">>, <<"#">>, <<"$SYS/#">>],
trans(fun() -> [emqx_trie:insert(Topic) || Topic <- Topics] end), trans(fun() -> [emqx_trie:insert(Topic) || Topic <- Topics] end),
Matched = mnesia:async_dirty(fun emqx_trie:match/1, [<<"a/b/c">>]), Matched = mnesia:async_dirty(fun emqx_trie:match/1, [<<"a/b/c">>]),
?assertEqual(4, length(Matched)), ?assertEqual(4, length(Matched)),
SysMatched = mnesia:async_dirty(fun emqx_trie:match/1, [<<"$SYS/a/b/c">>]), SysMatched = emqx_trie:match(<<"$SYS/a/b/c">>),
?assertEqual([<<"$SYS/#">>], SysMatched). ?assertEqual([<<"$SYS/#">>], SysMatched).
t_match4(_) ->
Topics = [<<"/#">>, <<"/+">>, <<"/+/a/b/c">>],
trans(fun() -> lists:foreach(fun emqx_trie:insert/1, Topics) end),
?assertEqual([<<"/#">>, <<"/+/a/b/c">>], lists:sort(emqx_trie:match(<<"/0/a/b/c">>))).
t_empty(_) -> t_empty(_) ->
?assert(?TRIE:empty()), ?assert(?TRIE:empty()),
trans(fun ?TRIE:insert/1, [<<"topic/x/#">>]), trans(fun ?TRIE:insert/1, [<<"topic/x/#">>]),
@ -99,53 +113,48 @@ t_empty(_) ->
?assert(?TRIE:empty()). ?assert(?TRIE:empty()).
t_delete(_) -> t_delete(_) ->
TN = #trie_node{node_id = <<"sensor/1">>, trans(fun() ->
edge_count = 2,
topic = undefined,
flags = undefined},
Fun = fun() ->
?TRIE:insert(<<"sensor/1/#">>), ?TRIE:insert(<<"sensor/1/#">>),
?TRIE:insert(<<"sensor/1/metric/2">>), ?TRIE:insert(<<"sensor/1/metric/2">>),
?TRIE:insert(<<"sensor/1/metric/3">>), ?TRIE:insert(<<"sensor/1/metric/3">>)
end),
trans(fun() ->
?TRIE:delete(<<"sensor/1/metric/2">>), ?TRIE:delete(<<"sensor/1/metric/2">>),
?TRIE:delete(<<"sensor/1/metric">>), ?TRIE:delete(<<"sensor/1/metric">>),
?TRIE:delete(<<"sensor/1/metric">>), ?TRIE:delete(<<"sensor/1/metric">>)
?TRIE:lookup(<<"sensor/1">>) end),
end, ?assertEqual([<<"sensor/1/#">>], ?TRIE:match(<<"sensor/1/x">>)).
?assertEqual({atomic, [TN]}, trans(Fun)).
t_delete2(_) -> t_delete2(_) ->
Fun = fun() -> trans(fun() ->
?TRIE:insert(<<"sensor">>), ?TRIE:insert(<<"sensor">>),
?TRIE:insert(<<"sensor/1/metric/2">>), ?TRIE:insert(<<"sensor/1/metric/2">>),
?TRIE:insert(<<"sensor/+/metric/3">>), ?TRIE:insert(<<"sensor/+/metric/3">>)
end),
trans(fun() ->
?TRIE:delete(<<"sensor">>), ?TRIE:delete(<<"sensor">>),
?TRIE:delete(<<"sensor/1/metric/2">>), ?TRIE:delete(<<"sensor/1/metric/2">>),
?TRIE:delete(<<"sensor/+/metric/3">>), ?TRIE:delete(<<"sensor/+/metric/3">>),
?TRIE:delete(<<"sensor/+/metric/3">>), ?TRIE:delete(<<"sensor/+/metric/3">>)
{?TRIE:lookup(<<"sensor">>), ?TRIE:lookup(<<"sensor/1">>)} end),
end, ?assertEqual([], ?TRIE:match(<<"sensor">>)),
?assertEqual({atomic, {[], []}}, trans(Fun)). ?assertEqual([], ?TRIE:match(<<"sensor/1">>)).
t_delete3(_) -> t_delete3(_) ->
Fun = fun() -> trans(fun() ->
?TRIE:insert(<<"sensor/+">>), ?TRIE:insert(<<"sensor/+">>),
?TRIE:insert(<<"sensor/+/metric/2">>), ?TRIE:insert(<<"sensor/+/metric/2">>),
?TRIE:insert(<<"sensor/+/metric/3">>), ?TRIE:insert(<<"sensor/+/metric/3">>)
end),
trans(fun() ->
?TRIE:delete(<<"sensor/+/metric/2">>), ?TRIE:delete(<<"sensor/+/metric/2">>),
?TRIE:delete(<<"sensor/+/metric/3">>), ?TRIE:delete(<<"sensor/+/metric/3">>),
?TRIE:delete(<<"sensor">>), ?TRIE:delete(<<"sensor">>),
?TRIE:delete(<<"sensor/+">>), ?TRIE:delete(<<"sensor/+">>),
?TRIE:delete(<<"sensor/+/unknown">>), ?TRIE:delete(<<"sensor/+/unknown">>)
{?TRIE:lookup(<<"sensor">>), ?TRIE:lookup(<<"sensor/+">>)} end),
end, ?assertEqual([], ?TRIE:match(<<"sensor">>)),
?assertEqual({atomic, {[], []}}, trans(Fun)). ?assertEqual([], ?TRIE:lookup_topic(<<"sensor/+">>)).
t_triples(_) ->
Triples = [{root,<<"a">>,<<"a">>},
{<<"a">>,<<"b">>,<<"a/b">>},
{<<"a/b">>,<<"c">>,<<"a/b/c">>}],
?assertEqual(Triples, emqx_trie:triples(<<"a/b/c">>)).
clear_tables() -> clear_tables() ->
lists:foreach(fun mnesia:clear_table/1, ?TRIE_TABS). lists:foreach(fun mnesia:clear_table/1, ?TRIE_TABS).