Merge pull request #11720 from ieQu1/dev/refactor-persistent-session
Refactor emqx_durable storage application and introduce learned topic structure storage
This commit is contained in:
commit
788698f157
|
@ -14,9 +14,4 @@
|
||||||
%% limitations under the License.
|
%% limitations under the License.
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
|
|
||||||
-ifndef(EMQX_BPAPI_HRL).
|
-include_lib("emqx_utils/include/bpapi.hrl").
|
||||||
-define(EMQX_BPAPI_HRL, true).
|
|
||||||
|
|
||||||
-compile({parse_transform, emqx_bpapi_trans}).
|
|
||||||
|
|
||||||
-endif.
|
|
||||||
|
|
|
@ -55,29 +55,7 @@
|
||||||
|
|
||||||
-record(subscription, {topic, subid, subopts}).
|
-record(subscription, {topic, subid, subopts}).
|
||||||
|
|
||||||
%% See 'Application Message' in MQTT Version 5.0
|
-include_lib("emqx_utils/include/emqx_message.hrl").
|
||||||
-record(message, {
|
|
||||||
%% Global unique message ID
|
|
||||||
id :: binary(),
|
|
||||||
%% Message QoS
|
|
||||||
qos = 0,
|
|
||||||
%% Message from
|
|
||||||
from :: atom() | binary(),
|
|
||||||
%% Message flags
|
|
||||||
flags = #{} :: emqx_types:flags(),
|
|
||||||
%% Message headers. May contain any metadata. e.g. the
|
|
||||||
%% protocol version number, username, peerhost or
|
|
||||||
%% the PUBLISH properties (MQTT 5.0).
|
|
||||||
headers = #{} :: emqx_types:headers(),
|
|
||||||
%% Topic that the message is published to
|
|
||||||
topic :: emqx_types:topic(),
|
|
||||||
%% Message Payload
|
|
||||||
payload :: emqx_types:payload(),
|
|
||||||
%% Timestamp (Unit: millisecond)
|
|
||||||
timestamp :: integer(),
|
|
||||||
%% not used so far, for future extension
|
|
||||||
extra = [] :: term()
|
|
||||||
}).
|
|
||||||
|
|
||||||
-record(delivery, {
|
-record(delivery, {
|
||||||
%% Sender of the delivery
|
%% Sender of the delivery
|
||||||
|
|
|
@ -1,7 +1,7 @@
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
%% Copyright (c) 2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
%% Copyright (c) 2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
-module(emqx_ds_SUITE).
|
-module(emqx_persistent_session_ds_SUITE).
|
||||||
|
|
||||||
-compile(export_all).
|
-compile(export_all).
|
||||||
-compile(nowarn_export_all).
|
-compile(nowarn_export_all).
|
||||||
|
@ -14,7 +14,6 @@
|
||||||
-define(DEFAULT_KEYSPACE, default).
|
-define(DEFAULT_KEYSPACE, default).
|
||||||
-define(DS_SHARD_ID, <<"local">>).
|
-define(DS_SHARD_ID, <<"local">>).
|
||||||
-define(DS_SHARD, {?DEFAULT_KEYSPACE, ?DS_SHARD_ID}).
|
-define(DS_SHARD, {?DEFAULT_KEYSPACE, ?DS_SHARD_ID}).
|
||||||
-define(ITERATOR_REF_TAB, emqx_ds_iterator_ref).
|
|
||||||
|
|
||||||
-import(emqx_common_test_helpers, [on_exit/1]).
|
-import(emqx_common_test_helpers, [on_exit/1]).
|
||||||
|
|
||||||
|
@ -91,9 +90,6 @@ get_mqtt_port(Node, Type) ->
|
||||||
{_IP, Port} = erpc:call(Node, emqx_config, get, [[listeners, Type, default, bind]]),
|
{_IP, Port} = erpc:call(Node, emqx_config, get, [[listeners, Type, default, bind]]),
|
||||||
Port.
|
Port.
|
||||||
|
|
||||||
get_all_iterator_refs(Node) ->
|
|
||||||
erpc:call(Node, mnesia, dirty_all_keys, [?ITERATOR_REF_TAB]).
|
|
||||||
|
|
||||||
get_all_iterator_ids(Node) ->
|
get_all_iterator_ids(Node) ->
|
||||||
Fn = fun(K, _V, Acc) -> [K | Acc] end,
|
Fn = fun(K, _V, Acc) -> [K | Acc] end,
|
||||||
erpc:call(Node, fun() ->
|
erpc:call(Node, fun() ->
|
||||||
|
@ -126,6 +122,32 @@ start_client(Opts0 = #{}) ->
|
||||||
on_exit(fun() -> catch emqtt:stop(Client) end),
|
on_exit(fun() -> catch emqtt:stop(Client) end),
|
||||||
Client.
|
Client.
|
||||||
|
|
||||||
|
restart_node(Node, NodeSpec) ->
|
||||||
|
?tp(will_restart_node, #{}),
|
||||||
|
?tp(notice, "restarting node", #{node => Node}),
|
||||||
|
true = monitor_node(Node, true),
|
||||||
|
ok = erpc:call(Node, init, restart, []),
|
||||||
|
receive
|
||||||
|
{nodedown, Node} ->
|
||||||
|
ok
|
||||||
|
after 10_000 ->
|
||||||
|
ct:fail("node ~p didn't stop", [Node])
|
||||||
|
end,
|
||||||
|
?tp(notice, "waiting for nodeup", #{node => Node}),
|
||||||
|
wait_nodeup(Node),
|
||||||
|
wait_gen_rpc_down(NodeSpec),
|
||||||
|
?tp(notice, "restarting apps", #{node => Node}),
|
||||||
|
Apps = maps:get(apps, NodeSpec),
|
||||||
|
ok = erpc:call(Node, emqx_cth_suite, load_apps, [Apps]),
|
||||||
|
_ = erpc:call(Node, emqx_cth_suite, start_apps, [Apps, NodeSpec]),
|
||||||
|
%% have to re-inject this so that we may stop the node succesfully at the
|
||||||
|
%% end....
|
||||||
|
ok = emqx_cth_cluster:set_node_opts(Node, NodeSpec),
|
||||||
|
ok = snabbkaffe:forward_trace(Node),
|
||||||
|
?tp(notice, "node restarted", #{node => Node}),
|
||||||
|
?tp(restarted_node, #{}),
|
||||||
|
ok.
|
||||||
|
|
||||||
%%------------------------------------------------------------------------------
|
%%------------------------------------------------------------------------------
|
||||||
%% Testcases
|
%% Testcases
|
||||||
%%------------------------------------------------------------------------------
|
%%------------------------------------------------------------------------------
|
||||||
|
@ -143,24 +165,14 @@ t_non_persistent_session_subscription(_Config) ->
|
||||||
{ok, _} = emqtt:connect(Client),
|
{ok, _} = emqtt:connect(Client),
|
||||||
?tp(notice, "subscribing", #{}),
|
?tp(notice, "subscribing", #{}),
|
||||||
{ok, _, [?RC_GRANTED_QOS_2]} = emqtt:subscribe(Client, SubTopicFilter, qos2),
|
{ok, _, [?RC_GRANTED_QOS_2]} = emqtt:subscribe(Client, SubTopicFilter, qos2),
|
||||||
IteratorRefs = get_all_iterator_refs(node()),
|
|
||||||
IteratorIds = get_all_iterator_ids(node()),
|
|
||||||
|
|
||||||
ok = emqtt:stop(Client),
|
ok = emqtt:stop(Client),
|
||||||
|
|
||||||
#{
|
ok
|
||||||
iterator_refs => IteratorRefs,
|
|
||||||
iterator_ids => IteratorIds
|
|
||||||
}
|
|
||||||
end,
|
end,
|
||||||
fun(Res, Trace) ->
|
fun(Trace) ->
|
||||||
ct:pal("trace:\n ~p", [Trace]),
|
ct:pal("trace:\n ~p", [Trace]),
|
||||||
#{
|
?assertEqual([], ?of_kind(ds_session_subscription_added, Trace)),
|
||||||
iterator_refs := IteratorRefs,
|
|
||||||
iterator_ids := IteratorIds
|
|
||||||
} = Res,
|
|
||||||
?assertEqual([], IteratorRefs),
|
|
||||||
?assertEqual({ok, []}, IteratorIds),
|
|
||||||
ok
|
ok
|
||||||
end
|
end
|
||||||
),
|
),
|
||||||
|
@ -175,7 +187,7 @@ t_session_subscription_idempotency(Config) ->
|
||||||
?check_trace(
|
?check_trace(
|
||||||
begin
|
begin
|
||||||
?force_ordering(
|
?force_ordering(
|
||||||
#{?snk_kind := persistent_session_ds_iterator_added},
|
#{?snk_kind := persistent_session_ds_subscription_added},
|
||||||
_NEvents0 = 1,
|
_NEvents0 = 1,
|
||||||
#{?snk_kind := will_restart_node},
|
#{?snk_kind := will_restart_node},
|
||||||
_Guard0 = true
|
_Guard0 = true
|
||||||
|
@ -187,32 +199,7 @@ t_session_subscription_idempotency(Config) ->
|
||||||
_Guard1 = true
|
_Guard1 = true
|
||||||
),
|
),
|
||||||
|
|
||||||
spawn_link(fun() ->
|
spawn_link(fun() -> restart_node(Node1, Node1Spec) end),
|
||||||
?tp(will_restart_node, #{}),
|
|
||||||
?tp(notice, "restarting node", #{node => Node1}),
|
|
||||||
true = monitor_node(Node1, true),
|
|
||||||
ok = erpc:call(Node1, init, restart, []),
|
|
||||||
receive
|
|
||||||
{nodedown, Node1} ->
|
|
||||||
ok
|
|
||||||
after 10_000 ->
|
|
||||||
ct:fail("node ~p didn't stop", [Node1])
|
|
||||||
end,
|
|
||||||
?tp(notice, "waiting for nodeup", #{node => Node1}),
|
|
||||||
wait_nodeup(Node1),
|
|
||||||
wait_gen_rpc_down(Node1Spec),
|
|
||||||
?tp(notice, "restarting apps", #{node => Node1}),
|
|
||||||
Apps = maps:get(apps, Node1Spec),
|
|
||||||
ok = erpc:call(Node1, emqx_cth_suite, load_apps, [Apps]),
|
|
||||||
_ = erpc:call(Node1, emqx_cth_suite, start_apps, [Apps, Node1Spec]),
|
|
||||||
%% have to re-inject this so that we may stop the node succesfully at the
|
|
||||||
%% end....
|
|
||||||
ok = emqx_cth_cluster:set_node_opts(Node1, Node1Spec),
|
|
||||||
ok = snabbkaffe:forward_trace(Node1),
|
|
||||||
?tp(notice, "node restarted", #{node => Node1}),
|
|
||||||
?tp(restarted_node, #{}),
|
|
||||||
ok
|
|
||||||
end),
|
|
||||||
|
|
||||||
?tp(notice, "starting 1", #{}),
|
?tp(notice, "starting 1", #{}),
|
||||||
Client0 = start_client(#{port => Port, clientid => ClientId}),
|
Client0 = start_client(#{port => Port, clientid => ClientId}),
|
||||||
|
@ -223,7 +210,7 @@ t_session_subscription_idempotency(Config) ->
|
||||||
receive
|
receive
|
||||||
{'EXIT', {shutdown, _}} ->
|
{'EXIT', {shutdown, _}} ->
|
||||||
ok
|
ok
|
||||||
after 0 -> ok
|
after 100 -> ok
|
||||||
end,
|
end,
|
||||||
process_flag(trap_exit, false),
|
process_flag(trap_exit, false),
|
||||||
|
|
||||||
|
@ -240,10 +227,7 @@ t_session_subscription_idempotency(Config) ->
|
||||||
end,
|
end,
|
||||||
fun(Trace) ->
|
fun(Trace) ->
|
||||||
ct:pal("trace:\n ~p", [Trace]),
|
ct:pal("trace:\n ~p", [Trace]),
|
||||||
%% Exactly one iterator should have been opened.
|
|
||||||
SubTopicFilterWords = emqx_topic:words(SubTopicFilter),
|
SubTopicFilterWords = emqx_topic:words(SubTopicFilter),
|
||||||
?assertEqual([{ClientId, SubTopicFilterWords}], get_all_iterator_refs(Node1)),
|
|
||||||
?assertMatch({ok, [_]}, get_all_iterator_ids(Node1)),
|
|
||||||
?assertMatch(
|
?assertMatch(
|
||||||
{ok, #{}, #{SubTopicFilterWords := #{}}},
|
{ok, #{}, #{SubTopicFilterWords := #{}}},
|
||||||
erpc:call(Node1, emqx_persistent_session_ds, session_open, [ClientId])
|
erpc:call(Node1, emqx_persistent_session_ds, session_open, [ClientId])
|
||||||
|
@ -262,7 +246,10 @@ t_session_unsubscription_idempotency(Config) ->
|
||||||
?check_trace(
|
?check_trace(
|
||||||
begin
|
begin
|
||||||
?force_ordering(
|
?force_ordering(
|
||||||
#{?snk_kind := persistent_session_ds_close_iterators, ?snk_span := {complete, _}},
|
#{
|
||||||
|
?snk_kind := persistent_session_ds_subscription_delete,
|
||||||
|
?snk_span := {complete, _}
|
||||||
|
},
|
||||||
_NEvents0 = 1,
|
_NEvents0 = 1,
|
||||||
#{?snk_kind := will_restart_node},
|
#{?snk_kind := will_restart_node},
|
||||||
_Guard0 = true
|
_Guard0 = true
|
||||||
|
@ -270,36 +257,11 @@ t_session_unsubscription_idempotency(Config) ->
|
||||||
?force_ordering(
|
?force_ordering(
|
||||||
#{?snk_kind := restarted_node},
|
#{?snk_kind := restarted_node},
|
||||||
_NEvents1 = 1,
|
_NEvents1 = 1,
|
||||||
#{?snk_kind := persistent_session_ds_iterator_delete, ?snk_span := start},
|
#{?snk_kind := persistent_session_ds_subscription_route_delete, ?snk_span := start},
|
||||||
_Guard1 = true
|
_Guard1 = true
|
||||||
),
|
),
|
||||||
|
|
||||||
spawn_link(fun() ->
|
spawn_link(fun() -> restart_node(Node1, Node1Spec) end),
|
||||||
?tp(will_restart_node, #{}),
|
|
||||||
?tp(notice, "restarting node", #{node => Node1}),
|
|
||||||
true = monitor_node(Node1, true),
|
|
||||||
ok = erpc:call(Node1, init, restart, []),
|
|
||||||
receive
|
|
||||||
{nodedown, Node1} ->
|
|
||||||
ok
|
|
||||||
after 10_000 ->
|
|
||||||
ct:fail("node ~p didn't stop", [Node1])
|
|
||||||
end,
|
|
||||||
?tp(notice, "waiting for nodeup", #{node => Node1}),
|
|
||||||
wait_nodeup(Node1),
|
|
||||||
wait_gen_rpc_down(Node1Spec),
|
|
||||||
?tp(notice, "restarting apps", #{node => Node1}),
|
|
||||||
Apps = maps:get(apps, Node1Spec),
|
|
||||||
ok = erpc:call(Node1, emqx_cth_suite, load_apps, [Apps]),
|
|
||||||
_ = erpc:call(Node1, emqx_cth_suite, start_apps, [Apps, Node1Spec]),
|
|
||||||
%% have to re-inject this so that we may stop the node succesfully at the
|
|
||||||
%% end....
|
|
||||||
ok = emqx_cth_cluster:set_node_opts(Node1, Node1Spec),
|
|
||||||
ok = snabbkaffe:forward_trace(Node1),
|
|
||||||
?tp(notice, "node restarted", #{node => Node1}),
|
|
||||||
?tp(restarted_node, #{}),
|
|
||||||
ok
|
|
||||||
end),
|
|
||||||
|
|
||||||
?tp(notice, "starting 1", #{}),
|
?tp(notice, "starting 1", #{}),
|
||||||
Client0 = start_client(#{port => Port, clientid => ClientId}),
|
Client0 = start_client(#{port => Port, clientid => ClientId}),
|
||||||
|
@ -312,7 +274,7 @@ t_session_unsubscription_idempotency(Config) ->
|
||||||
receive
|
receive
|
||||||
{'EXIT', {shutdown, _}} ->
|
{'EXIT', {shutdown, _}} ->
|
||||||
ok
|
ok
|
||||||
after 0 -> ok
|
after 100 -> ok
|
||||||
end,
|
end,
|
||||||
process_flag(trap_exit, false),
|
process_flag(trap_exit, false),
|
||||||
|
|
||||||
|
@ -327,7 +289,7 @@ t_session_unsubscription_idempotency(Config) ->
|
||||||
?wait_async_action(
|
?wait_async_action(
|
||||||
emqtt:unsubscribe(Client1, SubTopicFilter),
|
emqtt:unsubscribe(Client1, SubTopicFilter),
|
||||||
#{
|
#{
|
||||||
?snk_kind := persistent_session_ds_iterator_delete,
|
?snk_kind := persistent_session_ds_subscription_route_delete,
|
||||||
?snk_span := {complete, _}
|
?snk_span := {complete, _}
|
||||||
},
|
},
|
||||||
15_000
|
15_000
|
||||||
|
@ -339,9 +301,10 @@ t_session_unsubscription_idempotency(Config) ->
|
||||||
end,
|
end,
|
||||||
fun(Trace) ->
|
fun(Trace) ->
|
||||||
ct:pal("trace:\n ~p", [Trace]),
|
ct:pal("trace:\n ~p", [Trace]),
|
||||||
%% No iterators remaining
|
?assertMatch(
|
||||||
?assertEqual([], get_all_iterator_refs(Node1)),
|
{ok, #{}, Subs = #{}} when map_size(Subs) =:= 0,
|
||||||
?assertEqual({ok, []}, get_all_iterator_ids(Node1)),
|
erpc:call(Node1, emqx_persistent_session_ds, session_open, [ClientId])
|
||||||
|
),
|
||||||
ok
|
ok
|
||||||
end
|
end
|
||||||
),
|
),
|
|
@ -18,6 +18,7 @@
|
||||||
{emqx_dashboard,1}.
|
{emqx_dashboard,1}.
|
||||||
{emqx_delayed,1}.
|
{emqx_delayed,1}.
|
||||||
{emqx_delayed,2}.
|
{emqx_delayed,2}.
|
||||||
|
{emqx_ds,1}.
|
||||||
{emqx_eviction_agent,1}.
|
{emqx_eviction_agent,1}.
|
||||||
{emqx_eviction_agent,2}.
|
{emqx_eviction_agent,2}.
|
||||||
{emqx_exhook,1}.
|
{emqx_exhook,1}.
|
||||||
|
|
|
@ -66,7 +66,8 @@
|
||||||
|
|
||||||
-export([
|
-export([
|
||||||
is_expired/1,
|
is_expired/1,
|
||||||
update_expiry/1
|
update_expiry/1,
|
||||||
|
timestamp_now/0
|
||||||
]).
|
]).
|
||||||
|
|
||||||
-export([
|
-export([
|
||||||
|
@ -113,14 +114,13 @@ make(From, Topic, Payload) ->
|
||||||
emqx_types:payload()
|
emqx_types:payload()
|
||||||
) -> emqx_types:message().
|
) -> emqx_types:message().
|
||||||
make(From, QoS, Topic, Payload) when ?QOS_0 =< QoS, QoS =< ?QOS_2 ->
|
make(From, QoS, Topic, Payload) when ?QOS_0 =< QoS, QoS =< ?QOS_2 ->
|
||||||
Now = erlang:system_time(millisecond),
|
|
||||||
#message{
|
#message{
|
||||||
id = emqx_guid:gen(),
|
id = emqx_guid:gen(),
|
||||||
qos = QoS,
|
qos = QoS,
|
||||||
from = From,
|
from = From,
|
||||||
topic = Topic,
|
topic = Topic,
|
||||||
payload = Payload,
|
payload = Payload,
|
||||||
timestamp = Now
|
timestamp = timestamp_now()
|
||||||
}.
|
}.
|
||||||
|
|
||||||
-spec make(
|
-spec make(
|
||||||
|
@ -137,7 +137,6 @@ make(From, QoS, Topic, Payload, Flags, Headers) when
|
||||||
is_map(Flags),
|
is_map(Flags),
|
||||||
is_map(Headers)
|
is_map(Headers)
|
||||||
->
|
->
|
||||||
Now = erlang:system_time(millisecond),
|
|
||||||
#message{
|
#message{
|
||||||
id = emqx_guid:gen(),
|
id = emqx_guid:gen(),
|
||||||
qos = QoS,
|
qos = QoS,
|
||||||
|
@ -146,7 +145,7 @@ make(From, QoS, Topic, Payload, Flags, Headers) when
|
||||||
headers = Headers,
|
headers = Headers,
|
||||||
topic = Topic,
|
topic = Topic,
|
||||||
payload = Payload,
|
payload = Payload,
|
||||||
timestamp = Now
|
timestamp = timestamp_now()
|
||||||
}.
|
}.
|
||||||
|
|
||||||
-spec make(
|
-spec make(
|
||||||
|
@ -164,7 +163,6 @@ make(MsgId, From, QoS, Topic, Payload, Flags, Headers) when
|
||||||
is_map(Flags),
|
is_map(Flags),
|
||||||
is_map(Headers)
|
is_map(Headers)
|
||||||
->
|
->
|
||||||
Now = erlang:system_time(millisecond),
|
|
||||||
#message{
|
#message{
|
||||||
id = MsgId,
|
id = MsgId,
|
||||||
qos = QoS,
|
qos = QoS,
|
||||||
|
@ -173,7 +171,7 @@ make(MsgId, From, QoS, Topic, Payload, Flags, Headers) when
|
||||||
headers = Headers,
|
headers = Headers,
|
||||||
topic = Topic,
|
topic = Topic,
|
||||||
payload = Payload,
|
payload = Payload,
|
||||||
timestamp = Now
|
timestamp = timestamp_now()
|
||||||
}.
|
}.
|
||||||
|
|
||||||
%% optimistic esitmation of a message size after serialization
|
%% optimistic esitmation of a message size after serialization
|
||||||
|
@ -403,6 +401,11 @@ from_map(#{
|
||||||
extra = Extra
|
extra = Extra
|
||||||
}.
|
}.
|
||||||
|
|
||||||
|
%% @doc Get current timestamp in milliseconds.
|
||||||
|
-spec timestamp_now() -> integer().
|
||||||
|
timestamp_now() ->
|
||||||
|
erlang:system_time(millisecond).
|
||||||
|
|
||||||
%% MilliSeconds
|
%% MilliSeconds
|
||||||
elapsed(Since) ->
|
elapsed(Since) ->
|
||||||
max(0, erlang:system_time(millisecond) - Since).
|
max(0, timestamp_now() - Since).
|
||||||
|
|
|
@ -23,16 +23,12 @@
|
||||||
|
|
||||||
%% Message persistence
|
%% Message persistence
|
||||||
-export([
|
-export([
|
||||||
persist/1,
|
persist/1
|
||||||
serialize/1,
|
|
||||||
deserialize/1
|
|
||||||
]).
|
]).
|
||||||
|
|
||||||
%% FIXME
|
-define(PERSISTENT_MESSAGE_DB, emqx_persistent_message).
|
||||||
-define(DS_SHARD_ID, <<"local">>).
|
|
||||||
-define(DEFAULT_KEYSPACE, default).
|
|
||||||
-define(DS_SHARD, {?DEFAULT_KEYSPACE, ?DS_SHARD_ID}).
|
|
||||||
|
|
||||||
|
%% FIXME
|
||||||
-define(WHEN_ENABLED(DO),
|
-define(WHEN_ENABLED(DO),
|
||||||
case is_store_enabled() of
|
case is_store_enabled() of
|
||||||
true -> DO;
|
true -> DO;
|
||||||
|
@ -44,18 +40,10 @@
|
||||||
|
|
||||||
init() ->
|
init() ->
|
||||||
?WHEN_ENABLED(begin
|
?WHEN_ENABLED(begin
|
||||||
ok = emqx_ds:ensure_shard(
|
ok = emqx_ds:open_db(?PERSISTENT_MESSAGE_DB, #{
|
||||||
?DS_SHARD,
|
backend => builtin,
|
||||||
#{
|
storage => {emqx_ds_storage_bitfield_lts, #{}}
|
||||||
dir => filename:join([
|
}),
|
||||||
emqx:data_dir(),
|
|
||||||
ds,
|
|
||||||
messages,
|
|
||||||
?DEFAULT_KEYSPACE,
|
|
||||||
?DS_SHARD_ID
|
|
||||||
])
|
|
||||||
}
|
|
||||||
),
|
|
||||||
ok = emqx_persistent_session_ds_router:init_tables(),
|
ok = emqx_persistent_session_ds_router:init_tables(),
|
||||||
ok = emqx_persistent_session_ds:create_tables(),
|
ok = emqx_persistent_session_ds:create_tables(),
|
||||||
ok
|
ok
|
||||||
|
@ -82,19 +70,11 @@ persist(Msg) ->
|
||||||
needs_persistence(Msg) ->
|
needs_persistence(Msg) ->
|
||||||
not (emqx_message:get_flag(dup, Msg) orelse emqx_message:is_sys(Msg)).
|
not (emqx_message:get_flag(dup, Msg) orelse emqx_message:is_sys(Msg)).
|
||||||
|
|
||||||
|
-spec store_message(emqx_types:message()) -> emqx_ds:store_batch_result().
|
||||||
store_message(Msg) ->
|
store_message(Msg) ->
|
||||||
ID = emqx_message:id(Msg),
|
emqx_ds:store_batch(?PERSISTENT_MESSAGE_DB, [Msg]).
|
||||||
Timestamp = emqx_guid:timestamp(ID),
|
|
||||||
Topic = emqx_topic:words(emqx_message:topic(Msg)),
|
|
||||||
emqx_ds_storage_layer:store(?DS_SHARD, ID, Timestamp, Topic, serialize(Msg)).
|
|
||||||
|
|
||||||
has_subscribers(#message{topic = Topic}) ->
|
has_subscribers(#message{topic = Topic}) ->
|
||||||
emqx_persistent_session_ds_router:has_any_route(Topic).
|
emqx_persistent_session_ds_router:has_any_route(Topic).
|
||||||
|
|
||||||
%%
|
%%
|
||||||
|
|
||||||
serialize(Msg) ->
|
|
||||||
term_to_binary(emqx_message:to_map(Msg)).
|
|
||||||
|
|
||||||
deserialize(Bin) ->
|
|
||||||
emqx_message:from_map(binary_to_term(Bin)).
|
|
||||||
|
|
|
@ -0,0 +1,213 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%
|
||||||
|
%% Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
%% you may not use this file except in compliance with the License.
|
||||||
|
%% You may obtain a copy of the License at
|
||||||
|
%%
|
||||||
|
%% http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
%%
|
||||||
|
%% Unless required by applicable law or agreed to in writing, software
|
||||||
|
%% distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
%% See the License for the specific language governing permissions and
|
||||||
|
%% limitations under the License.
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
|
||||||
|
%% @doc This module implements the routines for replaying streams of
|
||||||
|
%% messages.
|
||||||
|
-module(emqx_persistent_message_ds_replayer).
|
||||||
|
|
||||||
|
%% API:
|
||||||
|
-export([new/0, next_packet_id/1, replay/2, commit_offset/3, poll/3]).
|
||||||
|
|
||||||
|
%% internal exports:
|
||||||
|
-export([]).
|
||||||
|
|
||||||
|
-export_type([inflight/0]).
|
||||||
|
|
||||||
|
-include("emqx_persistent_session_ds.hrl").
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Type declarations
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
%% Note: sequence numbers are monotonic; they don't wrap around:
|
||||||
|
-type seqno() :: non_neg_integer().
|
||||||
|
|
||||||
|
-record(range, {
|
||||||
|
stream :: emqx_ds:stream(),
|
||||||
|
first :: seqno(),
|
||||||
|
last :: seqno(),
|
||||||
|
iterator_next :: emqx_ds:iterator() | undefined
|
||||||
|
}).
|
||||||
|
|
||||||
|
-type range() :: #range{}.
|
||||||
|
|
||||||
|
-record(inflight, {
|
||||||
|
next_seqno = 0 :: seqno(),
|
||||||
|
acked_seqno = 0 :: seqno(),
|
||||||
|
offset_ranges = [] :: [range()]
|
||||||
|
}).
|
||||||
|
|
||||||
|
-opaque inflight() :: #inflight{}.
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% API funcions
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-spec new() -> inflight().
|
||||||
|
new() ->
|
||||||
|
#inflight{}.
|
||||||
|
|
||||||
|
-spec next_packet_id(inflight()) -> {emqx_types:packet_id(), inflight()}.
|
||||||
|
next_packet_id(Inflight0 = #inflight{next_seqno = LastSeqno}) ->
|
||||||
|
Inflight = Inflight0#inflight{next_seqno = LastSeqno + 1},
|
||||||
|
{seqno_to_packet_id(LastSeqno), Inflight}.
|
||||||
|
|
||||||
|
-spec replay(emqx_persistent_session_ds:id(), inflight()) ->
|
||||||
|
emqx_session:replies().
|
||||||
|
replay(_SessionId, _Inflight = #inflight{offset_ranges = _Ranges}) ->
|
||||||
|
[].
|
||||||
|
|
||||||
|
-spec commit_offset(emqx_persistent_session_ds:id(), emqx_types:packet_id(), inflight()) ->
|
||||||
|
{_IsValidOffset :: boolean(), inflight()}.
|
||||||
|
commit_offset(
|
||||||
|
SessionId,
|
||||||
|
PacketId,
|
||||||
|
Inflight0 = #inflight{
|
||||||
|
acked_seqno = AckedSeqno0, next_seqno = NextSeqNo, offset_ranges = Ranges0
|
||||||
|
}
|
||||||
|
) ->
|
||||||
|
AckedSeqno = packet_id_to_seqno(NextSeqNo, PacketId),
|
||||||
|
true = AckedSeqno0 < AckedSeqno,
|
||||||
|
Ranges = lists:filter(
|
||||||
|
fun(#range{stream = Stream, last = LastSeqno, iterator_next = ItNext}) ->
|
||||||
|
case LastSeqno =< AckedSeqno of
|
||||||
|
true ->
|
||||||
|
%% This range has been fully
|
||||||
|
%% acked. Remove it and replace saved
|
||||||
|
%% iterator with the trailing iterator.
|
||||||
|
update_iterator(SessionId, Stream, ItNext),
|
||||||
|
false;
|
||||||
|
false ->
|
||||||
|
%% This range still has unacked
|
||||||
|
%% messages:
|
||||||
|
true
|
||||||
|
end
|
||||||
|
end,
|
||||||
|
Ranges0
|
||||||
|
),
|
||||||
|
Inflight = Inflight0#inflight{acked_seqno = AckedSeqno, offset_ranges = Ranges},
|
||||||
|
{true, Inflight}.
|
||||||
|
|
||||||
|
-spec poll(emqx_persistent_session_ds:id(), inflight(), pos_integer()) ->
|
||||||
|
{emqx_session:replies(), inflight()}.
|
||||||
|
poll(SessionId, Inflight0, WindowSize) when WindowSize > 0, WindowSize < 16#7fff ->
|
||||||
|
#inflight{next_seqno = NextSeqNo0, acked_seqno = AckedSeqno} =
|
||||||
|
Inflight0,
|
||||||
|
FetchThreshold = max(1, WindowSize div 2),
|
||||||
|
FreeSpace = AckedSeqno + WindowSize - NextSeqNo0,
|
||||||
|
case FreeSpace >= FetchThreshold of
|
||||||
|
false ->
|
||||||
|
%% TODO: this branch is meant to avoid fetching data from
|
||||||
|
%% the DB in chunks that are too small. However, this
|
||||||
|
%% logic is not exactly good for the latency. Can the
|
||||||
|
%% client get stuck even?
|
||||||
|
{[], Inflight0};
|
||||||
|
true ->
|
||||||
|
Streams = shuffle(get_streams(SessionId)),
|
||||||
|
fetch(SessionId, Inflight0, Streams, FreeSpace, [])
|
||||||
|
end.
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Internal exports
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Internal functions
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
fetch(_SessionId, Inflight, _Streams = [], _N, Acc) ->
|
||||||
|
{lists:reverse(Acc), Inflight};
|
||||||
|
fetch(_SessionId, Inflight, _Streams, 0, Acc) ->
|
||||||
|
{lists:reverse(Acc), Inflight};
|
||||||
|
fetch(SessionId, Inflight0, [#ds_stream{stream = Stream} | Streams], N, Publishes0) ->
|
||||||
|
#inflight{next_seqno = FirstSeqNo, offset_ranges = Ranges0} = Inflight0,
|
||||||
|
ItBegin = get_last_iterator(SessionId, Stream, Ranges0),
|
||||||
|
{ok, ItEnd, Messages} = emqx_ds:next(ItBegin, N),
|
||||||
|
{Publishes, Inflight1} =
|
||||||
|
lists:foldl(
|
||||||
|
fun(Msg, {PubAcc0, InflightAcc0}) ->
|
||||||
|
{PacketId, InflightAcc} = next_packet_id(InflightAcc0),
|
||||||
|
PubAcc = [{PacketId, Msg} | PubAcc0],
|
||||||
|
{PubAcc, InflightAcc}
|
||||||
|
end,
|
||||||
|
{Publishes0, Inflight0},
|
||||||
|
Messages
|
||||||
|
),
|
||||||
|
#inflight{next_seqno = LastSeqNo} = Inflight1,
|
||||||
|
NMessages = LastSeqNo - FirstSeqNo,
|
||||||
|
case NMessages > 0 of
|
||||||
|
true ->
|
||||||
|
Range = #range{
|
||||||
|
first = FirstSeqNo,
|
||||||
|
last = LastSeqNo - 1,
|
||||||
|
stream = Stream,
|
||||||
|
iterator_next = ItEnd
|
||||||
|
},
|
||||||
|
Inflight = Inflight1#inflight{offset_ranges = Ranges0 ++ [Range]},
|
||||||
|
fetch(SessionId, Inflight, Streams, N - NMessages, Publishes);
|
||||||
|
false ->
|
||||||
|
fetch(SessionId, Inflight1, Streams, N, Publishes)
|
||||||
|
end.
|
||||||
|
|
||||||
|
update_iterator(SessionId, Stream, Iterator) ->
|
||||||
|
mria:dirty_write(?SESSION_ITER_TAB, #ds_iter{id = {SessionId, Stream}, iter = Iterator}).
|
||||||
|
|
||||||
|
get_last_iterator(SessionId, Stream, Ranges) ->
|
||||||
|
case lists:keyfind(Stream, #range.stream, lists:reverse(Ranges)) of
|
||||||
|
false ->
|
||||||
|
get_iterator(SessionId, Stream);
|
||||||
|
#range{iterator_next = Next} ->
|
||||||
|
Next
|
||||||
|
end.
|
||||||
|
|
||||||
|
get_iterator(SessionId, Stream) ->
|
||||||
|
Id = {SessionId, Stream},
|
||||||
|
[#ds_iter{iter = It}] = mnesia:dirty_read(?SESSION_ITER_TAB, Id),
|
||||||
|
It.
|
||||||
|
|
||||||
|
get_streams(SessionId) ->
|
||||||
|
mnesia:dirty_read(?SESSION_STREAM_TAB, SessionId).
|
||||||
|
|
||||||
|
%% Packet ID as defined by MQTT protocol is a 16-bit integer in range
|
||||||
|
%% 1..FFFF. This function translates internal session sequence number
|
||||||
|
%% to MQTT packet ID by chopping off most significant bits and adding
|
||||||
|
%% 1. This assumes that there's never more FFFF in-flight packets at
|
||||||
|
%% any time:
|
||||||
|
-spec seqno_to_packet_id(non_neg_integer()) -> emqx_types:packet_id().
|
||||||
|
seqno_to_packet_id(Counter) ->
|
||||||
|
Counter rem 16#ffff + 1.
|
||||||
|
|
||||||
|
%% Reconstruct session counter by adding most significant bits from
|
||||||
|
%% the current counter to the packet id.
|
||||||
|
-spec packet_id_to_seqno(non_neg_integer(), emqx_types:packet_id()) -> non_neg_integer().
|
||||||
|
packet_id_to_seqno(NextSeqNo, PacketId) ->
|
||||||
|
N = ((NextSeqNo bsr 16) bsl 16) + PacketId,
|
||||||
|
case N > NextSeqNo of
|
||||||
|
true -> N - 16#10000;
|
||||||
|
false -> N
|
||||||
|
end.
|
||||||
|
|
||||||
|
-spec shuffle([A]) -> [A].
|
||||||
|
shuffle(L0) ->
|
||||||
|
L1 = lists:map(
|
||||||
|
fun(A) ->
|
||||||
|
{rand:uniform(), A}
|
||||||
|
end,
|
||||||
|
L0
|
||||||
|
),
|
||||||
|
L2 = lists:sort(L1),
|
||||||
|
{_, L} = lists:unzip(L2),
|
||||||
|
L.
|
|
@ -18,9 +18,12 @@
|
||||||
|
|
||||||
-include("emqx.hrl").
|
-include("emqx.hrl").
|
||||||
-include_lib("snabbkaffe/include/snabbkaffe.hrl").
|
-include_lib("snabbkaffe/include/snabbkaffe.hrl").
|
||||||
|
-include_lib("stdlib/include/ms_transform.hrl").
|
||||||
|
|
||||||
-include("emqx_mqtt.hrl").
|
-include("emqx_mqtt.hrl").
|
||||||
|
|
||||||
|
-include("emqx_persistent_session_ds.hrl").
|
||||||
|
|
||||||
%% Session API
|
%% Session API
|
||||||
-export([
|
-export([
|
||||||
create/3,
|
create/3,
|
||||||
|
@ -50,7 +53,7 @@
|
||||||
-export([
|
-export([
|
||||||
deliver/3,
|
deliver/3,
|
||||||
replay/3,
|
replay/3,
|
||||||
% handle_timeout/3,
|
handle_timeout/3,
|
||||||
disconnect/1,
|
disconnect/1,
|
||||||
terminate/2
|
terminate/2
|
||||||
]).
|
]).
|
||||||
|
@ -58,33 +61,27 @@
|
||||||
%% session table operations
|
%% session table operations
|
||||||
-export([create_tables/0]).
|
-export([create_tables/0]).
|
||||||
|
|
||||||
-ifdef(TEST).
|
%% Remove me later (satisfy checks for an unused BPAPI)
|
||||||
-export([session_open/1]).
|
|
||||||
-endif.
|
|
||||||
|
|
||||||
%% RPC
|
|
||||||
-export([
|
|
||||||
ensure_iterator_closed_on_all_shards/1,
|
|
||||||
ensure_all_iterators_closed/1
|
|
||||||
]).
|
|
||||||
-export([
|
-export([
|
||||||
do_open_iterator/3,
|
do_open_iterator/3,
|
||||||
do_ensure_iterator_closed/1,
|
do_ensure_iterator_closed/1,
|
||||||
do_ensure_all_iterators_closed/1
|
do_ensure_all_iterators_closed/1
|
||||||
]).
|
]).
|
||||||
|
|
||||||
%% FIXME
|
-ifdef(TEST).
|
||||||
-define(DS_SHARD_ID, <<"local">>).
|
-export([session_open/1]).
|
||||||
-define(DEFAULT_KEYSPACE, default).
|
-endif.
|
||||||
-define(DS_SHARD, {?DEFAULT_KEYSPACE, ?DS_SHARD_ID}).
|
|
||||||
|
|
||||||
%% Currently, this is the clientid. We avoid `emqx_types:clientid()' because that can be
|
%% Currently, this is the clientid. We avoid `emqx_types:clientid()' because that can be
|
||||||
%% an atom, in theory (?).
|
%% an atom, in theory (?).
|
||||||
-type id() :: binary().
|
-type id() :: binary().
|
||||||
-type iterator() :: emqx_ds:iterator().
|
|
||||||
-type iterator_id() :: emqx_ds:iterator_id().
|
|
||||||
-type topic_filter() :: emqx_ds:topic_filter().
|
-type topic_filter() :: emqx_ds:topic_filter().
|
||||||
-type iterators() :: #{topic_filter() => iterator()}.
|
-type subscription_id() :: {id(), topic_filter()}.
|
||||||
|
-type subscription() :: #{
|
||||||
|
start_time := emqx_ds:time(),
|
||||||
|
propts := map(),
|
||||||
|
extra := map()
|
||||||
|
}.
|
||||||
-type session() :: #{
|
-type session() :: #{
|
||||||
%% Client ID
|
%% Client ID
|
||||||
id := id(),
|
id := id(),
|
||||||
|
@ -93,11 +90,15 @@
|
||||||
%% When the session should expire
|
%% When the session should expire
|
||||||
expires_at := timestamp() | never,
|
expires_at := timestamp() | never,
|
||||||
%% Client’s Subscriptions.
|
%% Client’s Subscriptions.
|
||||||
iterators := #{topic() => iterator()},
|
iterators := #{topic() => subscription()},
|
||||||
|
%% Inflight messages
|
||||||
|
inflight := emqx_persistent_message_ds_replayer:inflight(),
|
||||||
%%
|
%%
|
||||||
props := map()
|
props := map()
|
||||||
}.
|
}.
|
||||||
|
|
||||||
|
%% -type session() :: #session{}.
|
||||||
|
|
||||||
-type timestamp() :: emqx_utils_calendar:epoch_millisecond().
|
-type timestamp() :: emqx_utils_calendar:epoch_millisecond().
|
||||||
-type topic() :: emqx_types:topic().
|
-type topic() :: emqx_types:topic().
|
||||||
-type clientinfo() :: emqx_types:clientinfo().
|
-type clientinfo() :: emqx_types:clientinfo().
|
||||||
|
@ -106,12 +107,15 @@
|
||||||
|
|
||||||
-export_type([id/0]).
|
-export_type([id/0]).
|
||||||
|
|
||||||
|
-define(PERSISTENT_MESSAGE_DB, emqx_persistent_message).
|
||||||
|
|
||||||
%%
|
%%
|
||||||
|
|
||||||
-spec create(clientinfo(), conninfo(), emqx_session:conf()) ->
|
-spec create(clientinfo(), conninfo(), emqx_session:conf()) ->
|
||||||
session().
|
session().
|
||||||
create(#{clientid := ClientID}, _ConnInfo, Conf) ->
|
create(#{clientid := ClientID}, _ConnInfo, Conf) ->
|
||||||
% TODO: expiration
|
% TODO: expiration
|
||||||
|
ensure_timers(),
|
||||||
ensure_session(ClientID, Conf).
|
ensure_session(ClientID, Conf).
|
||||||
|
|
||||||
-spec open(clientinfo(), conninfo()) ->
|
-spec open(clientinfo(), conninfo()) ->
|
||||||
|
@ -126,6 +130,7 @@ open(#{clientid := ClientID}, _ConnInfo) ->
|
||||||
ok = emqx_cm:discard_session(ClientID),
|
ok = emqx_cm:discard_session(ClientID),
|
||||||
case open_session(ClientID) of
|
case open_session(ClientID) of
|
||||||
Session = #{} ->
|
Session = #{} ->
|
||||||
|
ensure_timers(),
|
||||||
{true, Session, []};
|
{true, Session, []};
|
||||||
false ->
|
false ->
|
||||||
false
|
false
|
||||||
|
@ -137,17 +142,17 @@ ensure_session(ClientID, Conf) ->
|
||||||
|
|
||||||
open_session(ClientID) ->
|
open_session(ClientID) ->
|
||||||
case session_open(ClientID) of
|
case session_open(ClientID) of
|
||||||
{ok, Session, Iterators} ->
|
{ok, Session, Subscriptions} ->
|
||||||
Session#{iterators => prep_iterators(Iterators)};
|
Session#{iterators => prep_subscriptions(Subscriptions)};
|
||||||
false ->
|
false ->
|
||||||
false
|
false
|
||||||
end.
|
end.
|
||||||
|
|
||||||
prep_iterators(Iterators) ->
|
prep_subscriptions(Subscriptions) ->
|
||||||
maps:fold(
|
maps:fold(
|
||||||
fun(Topic, Iterator, Acc) -> Acc#{emqx_topic:join(Topic) => Iterator} end,
|
fun(Topic, Subscription, Acc) -> Acc#{emqx_topic:join(Topic) => Subscription} end,
|
||||||
#{},
|
#{},
|
||||||
Iterators
|
Subscriptions
|
||||||
).
|
).
|
||||||
|
|
||||||
-spec destroy(session() | clientinfo()) -> ok.
|
-spec destroy(session() | clientinfo()) -> ok.
|
||||||
|
@ -157,7 +162,6 @@ destroy(#{clientid := ClientID}) ->
|
||||||
destroy_session(ClientID).
|
destroy_session(ClientID).
|
||||||
|
|
||||||
destroy_session(ClientID) ->
|
destroy_session(ClientID) ->
|
||||||
_ = ensure_all_iterators_closed(ClientID),
|
|
||||||
session_drop(ClientID).
|
session_drop(ClientID).
|
||||||
|
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
|
@ -245,7 +249,7 @@ unsubscribe(
|
||||||
) when is_map_key(TopicFilter, Iters) ->
|
) when is_map_key(TopicFilter, Iters) ->
|
||||||
Iterator = maps:get(TopicFilter, Iters),
|
Iterator = maps:get(TopicFilter, Iters),
|
||||||
SubOpts = maps:get(props, Iterator),
|
SubOpts = maps:get(props, Iterator),
|
||||||
ok = del_subscription(TopicFilter, Iterator, ID),
|
ok = del_subscription(TopicFilter, ID),
|
||||||
{ok, Session#{iterators := maps:remove(TopicFilter, Iters)}, SubOpts};
|
{ok, Session#{iterators := maps:remove(TopicFilter, Iters)}, SubOpts};
|
||||||
unsubscribe(
|
unsubscribe(
|
||||||
_TopicFilter,
|
_TopicFilter,
|
||||||
|
@ -271,19 +275,29 @@ get_subscription(TopicFilter, #{iterators := Iters}) ->
|
||||||
{ok, emqx_types:publish_result(), replies(), session()}
|
{ok, emqx_types:publish_result(), replies(), session()}
|
||||||
| {error, emqx_types:reason_code()}.
|
| {error, emqx_types:reason_code()}.
|
||||||
publish(_PacketId, Msg, Session) ->
|
publish(_PacketId, Msg, Session) ->
|
||||||
% TODO: stub
|
%% TODO:
|
||||||
{ok, emqx_broker:publish(Msg), [], Session}.
|
Result = emqx_broker:publish(Msg),
|
||||||
|
{ok, Result, [], Session}.
|
||||||
|
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
%% Client -> Broker: PUBACK
|
%% Client -> Broker: PUBACK
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
|
|
||||||
|
%% FIXME: parts of the commit offset function are mocked
|
||||||
|
-dialyzer({nowarn_function, puback/3}).
|
||||||
|
|
||||||
-spec puback(clientinfo(), emqx_types:packet_id(), session()) ->
|
-spec puback(clientinfo(), emqx_types:packet_id(), session()) ->
|
||||||
{ok, emqx_types:message(), replies(), session()}
|
{ok, emqx_types:message(), replies(), session()}
|
||||||
| {error, emqx_types:reason_code()}.
|
| {error, emqx_types:reason_code()}.
|
||||||
puback(_ClientInfo, _PacketId, _Session = #{}) ->
|
puback(_ClientInfo, PacketId, Session = #{id := Id, inflight := Inflight0}) ->
|
||||||
% TODO: stub
|
case emqx_persistent_message_ds_replayer:commit_offset(Id, PacketId, Inflight0) of
|
||||||
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND}.
|
{true, Inflight} ->
|
||||||
|
%% TODO
|
||||||
|
Msg = #message{},
|
||||||
|
{ok, Msg, [], Session#{inflight => Inflight}};
|
||||||
|
{false, _} ->
|
||||||
|
{error, ?RC_PACKET_IDENTIFIER_NOT_FOUND}
|
||||||
|
end.
|
||||||
|
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
%% Client -> Broker: PUBREC
|
%% Client -> Broker: PUBREC
|
||||||
|
@ -320,10 +334,22 @@ pubcomp(_ClientInfo, _PacketId, _Session = #{}) ->
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
|
|
||||||
-spec deliver(clientinfo(), [emqx_types:deliver()], session()) ->
|
-spec deliver(clientinfo(), [emqx_types:deliver()], session()) ->
|
||||||
no_return().
|
{ok, replies(), session()}.
|
||||||
deliver(_ClientInfo, _Delivers, _Session = #{}) ->
|
deliver(_ClientInfo, _Delivers, Session) ->
|
||||||
% TODO: ensure it's unreachable somehow
|
%% TODO: QoS0 and system messages end up here.
|
||||||
error(unexpected).
|
{ok, [], Session}.
|
||||||
|
|
||||||
|
-spec handle_timeout(clientinfo(), _Timeout, session()) ->
|
||||||
|
{ok, replies(), session()} | {ok, replies(), timeout(), session()}.
|
||||||
|
handle_timeout(_ClientInfo, pull, Session = #{id := Id, inflight := Inflight0}) ->
|
||||||
|
WindowSize = 100,
|
||||||
|
{Publishes, Inflight} = emqx_persistent_message_ds_replayer:poll(Id, Inflight0, WindowSize),
|
||||||
|
ensure_timer(pull),
|
||||||
|
{ok, Publishes, Session#{inflight => Inflight}};
|
||||||
|
handle_timeout(_ClientInfo, get_streams, Session = #{id := Id}) ->
|
||||||
|
renew_streams(Id),
|
||||||
|
ensure_timer(get_streams),
|
||||||
|
{ok, [], Session}.
|
||||||
|
|
||||||
-spec replay(clientinfo(), [], session()) ->
|
-spec replay(clientinfo(), [], session()) ->
|
||||||
{ok, replies(), session()}.
|
{ok, replies(), session()}.
|
||||||
|
@ -344,151 +370,69 @@ terminate(_Reason, _Session = #{}) ->
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
|
|
||||||
-spec add_subscription(topic(), emqx_types:subopts(), id()) ->
|
-spec add_subscription(topic(), emqx_types:subopts(), id()) ->
|
||||||
emqx_ds:iterator().
|
subscription().
|
||||||
add_subscription(TopicFilterBin, SubOpts, DSSessionID) ->
|
add_subscription(TopicFilterBin, SubOpts, DSSessionID) ->
|
||||||
% N.B.: we chose to update the router before adding the subscription to the
|
%% N.B.: we chose to update the router before adding the subscription to the
|
||||||
% session/iterator table. The reasoning for this is as follows:
|
%% session/iterator table. The reasoning for this is as follows:
|
||||||
%
|
%%
|
||||||
% Messages matching this topic filter should start to be persisted as soon as
|
%% Messages matching this topic filter should start to be persisted as soon as
|
||||||
% possible to avoid missing messages. If this is the first such persistent
|
%% possible to avoid missing messages. If this is the first such persistent
|
||||||
% session subscription, it's important to do so early on.
|
%% session subscription, it's important to do so early on.
|
||||||
%
|
%%
|
||||||
% This could, in turn, lead to some inconsistency: if such a route gets
|
%% This could, in turn, lead to some inconsistency: if such a route gets
|
||||||
% created but the session/iterator data fails to be updated accordingly, we
|
%% created but the session/iterator data fails to be updated accordingly, we
|
||||||
% have a dangling route. To remove such dangling routes, we may have a
|
%% have a dangling route. To remove such dangling routes, we may have a
|
||||||
% periodic GC process that removes routes that do not have a matching
|
%% periodic GC process that removes routes that do not have a matching
|
||||||
% persistent subscription. Also, route operations use dirty mnesia
|
%% persistent subscription. Also, route operations use dirty mnesia
|
||||||
% operations, which inherently have room for inconsistencies.
|
%% operations, which inherently have room for inconsistencies.
|
||||||
%
|
%%
|
||||||
% In practice, we use the iterator reference table as a source of truth,
|
%% In practice, we use the iterator reference table as a source of truth,
|
||||||
% since it is guarded by a transaction context: we consider a subscription
|
%% since it is guarded by a transaction context: we consider a subscription
|
||||||
% operation to be successful if it ended up changing this table. Both router
|
%% operation to be successful if it ended up changing this table. Both router
|
||||||
% and iterator information can be reconstructed from this table, if needed.
|
%% and iterator information can be reconstructed from this table, if needed.
|
||||||
ok = emqx_persistent_session_ds_router:do_add_route(TopicFilterBin, DSSessionID),
|
ok = emqx_persistent_session_ds_router:do_add_route(TopicFilterBin, DSSessionID),
|
||||||
TopicFilter = emqx_topic:words(TopicFilterBin),
|
TopicFilter = emqx_topic:words(TopicFilterBin),
|
||||||
{ok, Iterator, IsNew} = session_add_iterator(
|
{ok, DSSubExt, IsNew} = session_add_subscription(
|
||||||
DSSessionID, TopicFilter, SubOpts
|
DSSessionID, TopicFilter, SubOpts
|
||||||
),
|
),
|
||||||
Ctx = #{iterator => Iterator, is_new => IsNew},
|
?tp(persistent_session_ds_subscription_added, #{sub => DSSubExt, is_new => IsNew}),
|
||||||
?tp(persistent_session_ds_iterator_added, Ctx),
|
%% we'll list streams and open iterators when implementing message replay.
|
||||||
?tp_span(
|
DSSubExt.
|
||||||
persistent_session_ds_open_iterators,
|
|
||||||
Ctx,
|
|
||||||
ok = open_iterator_on_all_shards(TopicFilter, Iterator)
|
|
||||||
),
|
|
||||||
Iterator.
|
|
||||||
|
|
||||||
-spec update_subscription(topic(), iterator(), emqx_types:subopts(), id()) ->
|
-spec update_subscription(topic(), subscription(), emqx_types:subopts(), id()) ->
|
||||||
iterator().
|
subscription().
|
||||||
update_subscription(TopicFilterBin, Iterator, SubOpts, DSSessionID) ->
|
update_subscription(TopicFilterBin, DSSubExt, SubOpts, DSSessionID) ->
|
||||||
TopicFilter = emqx_topic:words(TopicFilterBin),
|
TopicFilter = emqx_topic:words(TopicFilterBin),
|
||||||
{ok, NIterator, false} = session_add_iterator(
|
{ok, NDSSubExt, false} = session_add_subscription(
|
||||||
DSSessionID, TopicFilter, SubOpts
|
DSSessionID, TopicFilter, SubOpts
|
||||||
),
|
),
|
||||||
ok = ?tp(persistent_session_ds_iterator_updated, #{iterator => Iterator}),
|
ok = ?tp(persistent_session_ds_iterator_updated, #{sub => DSSubExt}),
|
||||||
NIterator.
|
NDSSubExt.
|
||||||
|
|
||||||
-spec open_iterator_on_all_shards(emqx_types:words(), emqx_ds:iterator()) -> ok.
|
-spec del_subscription(topic(), id()) ->
|
||||||
open_iterator_on_all_shards(TopicFilter, Iterator) ->
|
|
||||||
?tp(persistent_session_ds_will_open_iterators, #{iterator => Iterator}),
|
|
||||||
%% Note: currently, shards map 1:1 to nodes, but this will change in the future.
|
|
||||||
Nodes = emqx:running_nodes(),
|
|
||||||
Results = emqx_persistent_session_ds_proto_v1:open_iterator(
|
|
||||||
Nodes,
|
|
||||||
TopicFilter,
|
|
||||||
maps:get(start_time, Iterator),
|
|
||||||
maps:get(id, Iterator)
|
|
||||||
),
|
|
||||||
%% TODO
|
|
||||||
%% 1. Handle errors.
|
|
||||||
%% 2. Iterator handles are rocksdb resources, it's doubtful they survive RPC.
|
|
||||||
%% Even if they do, we throw them away here anyway. All in all, we probably should
|
|
||||||
%% hold each of them in a process on the respective node.
|
|
||||||
true = lists:all(fun(Res) -> element(1, Res) =:= ok end, Results),
|
|
||||||
ok.
|
ok.
|
||||||
|
del_subscription(TopicFilterBin, DSSessionId) ->
|
||||||
%% RPC target.
|
|
||||||
-spec do_open_iterator(emqx_types:words(), emqx_ds:time(), emqx_ds:iterator_id()) ->
|
|
||||||
{ok, emqx_ds_storage_layer:iterator()} | {error, _Reason}.
|
|
||||||
do_open_iterator(TopicFilter, StartMS, IteratorID) ->
|
|
||||||
Replay = {TopicFilter, StartMS},
|
|
||||||
emqx_ds_storage_layer:ensure_iterator(?DS_SHARD, IteratorID, Replay).
|
|
||||||
|
|
||||||
-spec del_subscription(topic(), iterator(), id()) ->
|
|
||||||
ok.
|
|
||||||
del_subscription(TopicFilterBin, #{id := IteratorID}, DSSessionID) ->
|
|
||||||
% N.B.: see comments in `?MODULE:add_subscription' for a discussion about the
|
|
||||||
% order of operations here.
|
|
||||||
TopicFilter = emqx_topic:words(TopicFilterBin),
|
TopicFilter = emqx_topic:words(TopicFilterBin),
|
||||||
Ctx = #{iterator_id => IteratorID},
|
|
||||||
?tp_span(
|
?tp_span(
|
||||||
persistent_session_ds_close_iterators,
|
persistent_session_ds_subscription_delete,
|
||||||
Ctx,
|
#{session_id => DSSessionId},
|
||||||
ok = ensure_iterator_closed_on_all_shards(IteratorID)
|
ok = session_del_subscription(DSSessionId, TopicFilter)
|
||||||
),
|
),
|
||||||
?tp_span(
|
?tp_span(
|
||||||
persistent_session_ds_iterator_delete,
|
persistent_session_ds_subscription_route_delete,
|
||||||
Ctx,
|
#{session_id => DSSessionId},
|
||||||
session_del_iterator(DSSessionID, TopicFilter)
|
ok = emqx_persistent_session_ds_router:do_delete_route(TopicFilterBin, DSSessionId)
|
||||||
),
|
).
|
||||||
ok = emqx_persistent_session_ds_router:do_delete_route(TopicFilterBin, DSSessionID).
|
|
||||||
|
|
||||||
-spec ensure_iterator_closed_on_all_shards(emqx_ds:iterator_id()) -> ok.
|
|
||||||
ensure_iterator_closed_on_all_shards(IteratorID) ->
|
|
||||||
%% Note: currently, shards map 1:1 to nodes, but this will change in the future.
|
|
||||||
Nodes = emqx:running_nodes(),
|
|
||||||
Results = emqx_persistent_session_ds_proto_v1:close_iterator(Nodes, IteratorID),
|
|
||||||
%% TODO: handle errors
|
|
||||||
true = lists:all(fun(Res) -> Res =:= {ok, ok} end, Results),
|
|
||||||
ok.
|
|
||||||
|
|
||||||
%% RPC target.
|
|
||||||
-spec do_ensure_iterator_closed(emqx_ds:iterator_id()) -> ok.
|
|
||||||
do_ensure_iterator_closed(IteratorID) ->
|
|
||||||
ok = emqx_ds_storage_layer:discard_iterator(?DS_SHARD, IteratorID),
|
|
||||||
ok.
|
|
||||||
|
|
||||||
-spec ensure_all_iterators_closed(id()) -> ok.
|
|
||||||
ensure_all_iterators_closed(DSSessionID) ->
|
|
||||||
%% Note: currently, shards map 1:1 to nodes, but this will change in the future.
|
|
||||||
Nodes = emqx:running_nodes(),
|
|
||||||
Results = emqx_persistent_session_ds_proto_v1:close_all_iterators(Nodes, DSSessionID),
|
|
||||||
%% TODO: handle errors
|
|
||||||
true = lists:all(fun(Res) -> Res =:= {ok, ok} end, Results),
|
|
||||||
ok.
|
|
||||||
|
|
||||||
%% RPC target.
|
|
||||||
-spec do_ensure_all_iterators_closed(id()) -> ok.
|
|
||||||
do_ensure_all_iterators_closed(DSSessionID) ->
|
|
||||||
ok = emqx_ds_storage_layer:discard_iterator_prefix(?DS_SHARD, DSSessionID),
|
|
||||||
ok.
|
|
||||||
|
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
%% Session tables operations
|
%% Session tables operations
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
|
|
||||||
-define(SESSION_TAB, emqx_ds_session).
|
|
||||||
-define(ITERATOR_REF_TAB, emqx_ds_iterator_ref).
|
|
||||||
-define(DS_MRIA_SHARD, emqx_ds_shard).
|
|
||||||
|
|
||||||
-record(session, {
|
|
||||||
%% same as clientid
|
|
||||||
id :: id(),
|
|
||||||
%% creation time
|
|
||||||
created_at :: _Millisecond :: non_neg_integer(),
|
|
||||||
expires_at = never :: _Millisecond :: non_neg_integer() | never,
|
|
||||||
%% for future usage
|
|
||||||
props = #{} :: map()
|
|
||||||
}).
|
|
||||||
|
|
||||||
-record(iterator_ref, {
|
|
||||||
ref_id :: {id(), emqx_ds:topic_filter()},
|
|
||||||
it_id :: emqx_ds:iterator_id(),
|
|
||||||
start_time :: emqx_ds:time(),
|
|
||||||
props = #{} :: map()
|
|
||||||
}).
|
|
||||||
|
|
||||||
create_tables() ->
|
create_tables() ->
|
||||||
|
ok = emqx_ds:open_db(?PERSISTENT_MESSAGE_DB, #{
|
||||||
|
backend => builtin,
|
||||||
|
storage => {emqx_ds_storage_bitfield_lts, #{}}
|
||||||
|
}),
|
||||||
ok = mria:create_table(
|
ok = mria:create_table(
|
||||||
?SESSION_TAB,
|
?SESSION_TAB,
|
||||||
[
|
[
|
||||||
|
@ -500,15 +444,38 @@ create_tables() ->
|
||||||
]
|
]
|
||||||
),
|
),
|
||||||
ok = mria:create_table(
|
ok = mria:create_table(
|
||||||
?ITERATOR_REF_TAB,
|
?SESSION_SUBSCRIPTIONS_TAB,
|
||||||
[
|
[
|
||||||
{rlog_shard, ?DS_MRIA_SHARD},
|
{rlog_shard, ?DS_MRIA_SHARD},
|
||||||
{type, ordered_set},
|
{type, ordered_set},
|
||||||
{storage, storage()},
|
{storage, storage()},
|
||||||
{record_name, iterator_ref},
|
{record_name, ds_sub},
|
||||||
{attributes, record_info(fields, iterator_ref)}
|
{attributes, record_info(fields, ds_sub)}
|
||||||
]
|
]
|
||||||
),
|
),
|
||||||
|
ok = mria:create_table(
|
||||||
|
?SESSION_STREAM_TAB,
|
||||||
|
[
|
||||||
|
{rlog_shard, ?DS_MRIA_SHARD},
|
||||||
|
{type, bag},
|
||||||
|
{storage, storage()},
|
||||||
|
{record_name, ds_stream},
|
||||||
|
{attributes, record_info(fields, ds_stream)}
|
||||||
|
]
|
||||||
|
),
|
||||||
|
ok = mria:create_table(
|
||||||
|
?SESSION_ITER_TAB,
|
||||||
|
[
|
||||||
|
{rlog_shard, ?DS_MRIA_SHARD},
|
||||||
|
{type, set},
|
||||||
|
{storage, storage()},
|
||||||
|
{record_name, ds_iter},
|
||||||
|
{attributes, record_info(fields, ds_iter)}
|
||||||
|
]
|
||||||
|
),
|
||||||
|
ok = mria:wait_for_tables([
|
||||||
|
?SESSION_TAB, ?SESSION_SUBSCRIPTIONS_TAB, ?SESSION_STREAM_TAB, ?SESSION_ITER_TAB
|
||||||
|
]),
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
-dialyzer({nowarn_function, storage/0}).
|
-dialyzer({nowarn_function, storage/0}).
|
||||||
|
@ -529,26 +496,26 @@ storage() ->
|
||||||
%% Note: session API doesn't handle session takeovers, it's the job of
|
%% Note: session API doesn't handle session takeovers, it's the job of
|
||||||
%% the broker.
|
%% the broker.
|
||||||
-spec session_open(id()) ->
|
-spec session_open(id()) ->
|
||||||
{ok, session(), iterators()} | false.
|
{ok, session(), #{topic() => subscription()}} | false.
|
||||||
session_open(SessionId) ->
|
session_open(SessionId) ->
|
||||||
transaction(fun() ->
|
transaction(fun() ->
|
||||||
case mnesia:read(?SESSION_TAB, SessionId, write) of
|
case mnesia:read(?SESSION_TAB, SessionId, write) of
|
||||||
[Record = #session{}] ->
|
[Record = #session{}] ->
|
||||||
Session = export_record(Record),
|
Session = export_session(Record),
|
||||||
IteratorRefs = session_read_iterators(SessionId),
|
DSSubs = session_read_subscriptions(SessionId),
|
||||||
Iterators = export_iterators(IteratorRefs),
|
Subscriptions = export_subscriptions(DSSubs),
|
||||||
{ok, Session, Iterators};
|
{ok, Session, Subscriptions};
|
||||||
[] ->
|
[] ->
|
||||||
false
|
false
|
||||||
end
|
end
|
||||||
end).
|
end).
|
||||||
|
|
||||||
-spec session_ensure_new(id(), _Props :: map()) ->
|
-spec session_ensure_new(id(), _Props :: map()) ->
|
||||||
{ok, session(), iterators()}.
|
{ok, session(), #{topic() => subscription()}}.
|
||||||
session_ensure_new(SessionId, Props) ->
|
session_ensure_new(SessionId, Props) ->
|
||||||
transaction(fun() ->
|
transaction(fun() ->
|
||||||
ok = session_drop_iterators(SessionId),
|
ok = session_drop_subscriptions(SessionId),
|
||||||
Session = export_record(session_create(SessionId, Props)),
|
Session = export_session(session_create(SessionId, Props)),
|
||||||
{ok, Session, #{}}
|
{ok, Session, #{}}
|
||||||
end).
|
end).
|
||||||
|
|
||||||
|
@ -557,7 +524,8 @@ session_create(SessionId, Props) ->
|
||||||
id = SessionId,
|
id = SessionId,
|
||||||
created_at = erlang:system_time(millisecond),
|
created_at = erlang:system_time(millisecond),
|
||||||
expires_at = never,
|
expires_at = never,
|
||||||
props = Props
|
props = Props,
|
||||||
|
inflight = emqx_persistent_message_ds_replayer:new()
|
||||||
},
|
},
|
||||||
ok = mnesia:write(?SESSION_TAB, Session, write),
|
ok = mnesia:write(?SESSION_TAB, Session, write),
|
||||||
Session.
|
Session.
|
||||||
|
@ -568,80 +536,143 @@ session_create(SessionId, Props) ->
|
||||||
session_drop(DSSessionId) ->
|
session_drop(DSSessionId) ->
|
||||||
transaction(fun() ->
|
transaction(fun() ->
|
||||||
%% TODO: ensure all iterators from this clientid are closed?
|
%% TODO: ensure all iterators from this clientid are closed?
|
||||||
ok = session_drop_iterators(DSSessionId),
|
ok = session_drop_subscriptions(DSSessionId),
|
||||||
ok = mnesia:delete(?SESSION_TAB, DSSessionId, write)
|
ok = mnesia:delete(?SESSION_TAB, DSSessionId, write)
|
||||||
end).
|
end).
|
||||||
|
|
||||||
session_drop_iterators(DSSessionId) ->
|
session_drop_subscriptions(DSSessionId) ->
|
||||||
IteratorRefs = session_read_iterators(DSSessionId),
|
IteratorRefs = session_read_subscriptions(DSSessionId),
|
||||||
ok = lists:foreach(fun session_del_iterator/1, IteratorRefs).
|
ok = lists:foreach(fun session_del_subscription/1, IteratorRefs).
|
||||||
|
|
||||||
%% @doc Called when a client subscribes to a topic. Idempotent.
|
%% @doc Called when a client subscribes to a topic. Idempotent.
|
||||||
-spec session_add_iterator(id(), topic_filter(), _Props :: map()) ->
|
-spec session_add_subscription(id(), topic_filter(), _Props :: map()) ->
|
||||||
{ok, iterator(), _IsNew :: boolean()}.
|
{ok, subscription(), _IsNew :: boolean()}.
|
||||||
session_add_iterator(DSSessionId, TopicFilter, Props) ->
|
session_add_subscription(DSSessionId, TopicFilter, Props) ->
|
||||||
IteratorRefId = {DSSessionId, TopicFilter},
|
DSSubId = {DSSessionId, TopicFilter},
|
||||||
transaction(fun() ->
|
transaction(fun() ->
|
||||||
case mnesia:read(?ITERATOR_REF_TAB, IteratorRefId, write) of
|
case mnesia:read(?SESSION_SUBSCRIPTIONS_TAB, DSSubId, write) of
|
||||||
[] ->
|
[] ->
|
||||||
IteratorRef = session_insert_iterator(DSSessionId, TopicFilter, Props),
|
DSSub = session_insert_subscription(DSSessionId, TopicFilter, Props),
|
||||||
Iterator = export_record(IteratorRef),
|
DSSubExt = export_subscription(DSSub),
|
||||||
?tp(
|
?tp(
|
||||||
ds_session_subscription_added,
|
ds_session_subscription_added,
|
||||||
#{iterator => Iterator, session_id => DSSessionId}
|
#{sub => DSSubExt, session_id => DSSessionId}
|
||||||
),
|
),
|
||||||
{ok, Iterator, _IsNew = true};
|
{ok, DSSubExt, _IsNew = true};
|
||||||
[#iterator_ref{} = IteratorRef] ->
|
[#ds_sub{} = DSSub] ->
|
||||||
NIteratorRef = session_update_iterator(IteratorRef, Props),
|
NDSSub = session_update_subscription(DSSub, Props),
|
||||||
NIterator = export_record(NIteratorRef),
|
NDSSubExt = export_subscription(NDSSub),
|
||||||
?tp(
|
?tp(
|
||||||
ds_session_subscription_present,
|
ds_session_subscription_present,
|
||||||
#{iterator => NIterator, session_id => DSSessionId}
|
#{sub => NDSSubExt, session_id => DSSessionId}
|
||||||
),
|
),
|
||||||
{ok, NIterator, _IsNew = false}
|
{ok, NDSSubExt, _IsNew = false}
|
||||||
end
|
end
|
||||||
end).
|
end).
|
||||||
|
|
||||||
session_insert_iterator(DSSessionId, TopicFilter, Props) ->
|
-spec session_insert_subscription(id(), topic_filter(), map()) -> ds_sub().
|
||||||
{IteratorId, StartMS} = new_iterator_id(DSSessionId),
|
session_insert_subscription(DSSessionId, TopicFilter, Props) ->
|
||||||
IteratorRef = #iterator_ref{
|
{DSSubId, StartMS} = new_subscription_id(DSSessionId, TopicFilter),
|
||||||
ref_id = {DSSessionId, TopicFilter},
|
DSSub = #ds_sub{
|
||||||
it_id = IteratorId,
|
id = DSSubId,
|
||||||
start_time = StartMS,
|
start_time = StartMS,
|
||||||
props = Props
|
props = Props,
|
||||||
|
extra = #{}
|
||||||
},
|
},
|
||||||
ok = mnesia:write(?ITERATOR_REF_TAB, IteratorRef, write),
|
ok = mnesia:write(?SESSION_SUBSCRIPTIONS_TAB, DSSub, write),
|
||||||
IteratorRef.
|
DSSub.
|
||||||
|
|
||||||
session_update_iterator(IteratorRef, Props) ->
|
-spec session_update_subscription(ds_sub(), map()) -> ds_sub().
|
||||||
NIteratorRef = IteratorRef#iterator_ref{props = Props},
|
session_update_subscription(DSSub, Props) ->
|
||||||
ok = mnesia:write(?ITERATOR_REF_TAB, NIteratorRef, write),
|
NDSSub = DSSub#ds_sub{props = Props},
|
||||||
NIteratorRef.
|
ok = mnesia:write(?SESSION_SUBSCRIPTIONS_TAB, NDSSub, write),
|
||||||
|
NDSSub.
|
||||||
|
|
||||||
%% @doc Called when a client unsubscribes from a topic.
|
session_del_subscription(DSSessionId, TopicFilter) ->
|
||||||
-spec session_del_iterator(id(), topic_filter()) -> ok.
|
DSSubId = {DSSessionId, TopicFilter},
|
||||||
session_del_iterator(DSSessionId, TopicFilter) ->
|
|
||||||
IteratorRefId = {DSSessionId, TopicFilter},
|
|
||||||
transaction(fun() ->
|
transaction(fun() ->
|
||||||
mnesia:delete(?ITERATOR_REF_TAB, IteratorRefId, write)
|
mnesia:delete(?SESSION_SUBSCRIPTIONS_TAB, DSSubId, write)
|
||||||
end).
|
end).
|
||||||
|
|
||||||
session_del_iterator(#iterator_ref{ref_id = IteratorRefId}) ->
|
session_del_subscription(#ds_sub{id = DSSubId}) ->
|
||||||
mnesia:delete(?ITERATOR_REF_TAB, IteratorRefId, write).
|
mnesia:delete(?SESSION_SUBSCRIPTIONS_TAB, DSSubId, write).
|
||||||
|
|
||||||
session_read_iterators(DSSessionId) ->
|
session_read_subscriptions(DSSessionId) ->
|
||||||
% NOTE: somewhat convoluted way to trick dialyzer
|
MS = ets:fun2ms(
|
||||||
Pat = erlang:make_tuple(record_info(size, iterator_ref), '_', [
|
fun(Sub = #ds_sub{id = {Sess, _}}) when Sess =:= DSSessionId ->
|
||||||
{1, iterator_ref},
|
Sub
|
||||||
{#iterator_ref.ref_id, {DSSessionId, '_'}}
|
end
|
||||||
]),
|
),
|
||||||
mnesia:match_object(?ITERATOR_REF_TAB, Pat, read).
|
mnesia:select(?SESSION_SUBSCRIPTIONS_TAB, MS, read).
|
||||||
|
|
||||||
-spec new_iterator_id(id()) -> {iterator_id(), emqx_ds:time()}.
|
-spec new_subscription_id(id(), topic_filter()) -> {subscription_id(), integer()}.
|
||||||
new_iterator_id(DSSessionId) ->
|
new_subscription_id(DSSessionId, TopicFilter) ->
|
||||||
NowMS = erlang:system_time(microsecond),
|
%% Note: here we use _milliseconds_ to match with the timestamp
|
||||||
IteratorId = <<DSSessionId/binary, (emqx_guid:gen())/binary>>,
|
%% field of `#message' record.
|
||||||
{IteratorId, NowMS}.
|
NowMS = erlang:system_time(millisecond),
|
||||||
|
DSSubId = {DSSessionId, TopicFilter},
|
||||||
|
{DSSubId, NowMS}.
|
||||||
|
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% RPC targets (v1)
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
|
||||||
|
%% RPC target.
|
||||||
|
-spec do_open_iterator(emqx_types:words(), emqx_ds:time(), emqx_ds:iterator_id()) ->
|
||||||
|
{ok, emqx_ds_storage_layer:iterator()} | {error, _Reason}.
|
||||||
|
do_open_iterator(_TopicFilter, _StartMS, _IteratorID) ->
|
||||||
|
{error, not_implemented}.
|
||||||
|
|
||||||
|
%% RPC target.
|
||||||
|
-spec do_ensure_iterator_closed(emqx_ds:iterator_id()) -> ok.
|
||||||
|
do_ensure_iterator_closed(_IteratorID) ->
|
||||||
|
ok.
|
||||||
|
|
||||||
|
%% RPC target.
|
||||||
|
-spec do_ensure_all_iterators_closed(id()) -> ok.
|
||||||
|
do_ensure_all_iterators_closed(_DSSessionID) ->
|
||||||
|
ok.
|
||||||
|
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Reading batches
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
|
||||||
|
renew_streams(Id) ->
|
||||||
|
Subscriptions = ro_transaction(fun() -> session_read_subscriptions(Id) end),
|
||||||
|
ExistingStreams = ro_transaction(fun() -> mnesia:read(?SESSION_STREAM_TAB, Id) end),
|
||||||
|
lists:foreach(
|
||||||
|
fun(#ds_sub{id = {_, TopicFilter}, start_time = StartTime}) ->
|
||||||
|
renew_streams(Id, ExistingStreams, TopicFilter, StartTime)
|
||||||
|
end,
|
||||||
|
Subscriptions
|
||||||
|
).
|
||||||
|
|
||||||
|
renew_streams(Id, ExistingStreams, TopicFilter, StartTime) ->
|
||||||
|
AllStreams = emqx_ds:get_streams(?PERSISTENT_MESSAGE_DB, TopicFilter, StartTime),
|
||||||
|
transaction(
|
||||||
|
fun() ->
|
||||||
|
lists:foreach(
|
||||||
|
fun({Rank, Stream}) ->
|
||||||
|
Rec = #ds_stream{
|
||||||
|
session = Id,
|
||||||
|
topic_filter = TopicFilter,
|
||||||
|
stream = Stream,
|
||||||
|
rank = Rank
|
||||||
|
},
|
||||||
|
case lists:member(Rec, ExistingStreams) of
|
||||||
|
true ->
|
||||||
|
ok;
|
||||||
|
false ->
|
||||||
|
mnesia:write(?SESSION_STREAM_TAB, Rec, write),
|
||||||
|
{ok, Iterator} = emqx_ds:make_iterator(Stream, TopicFilter, StartTime),
|
||||||
|
IterRec = #ds_iter{id = {Id, Stream}, iter = Iterator},
|
||||||
|
mnesia:write(?SESSION_ITER_TAB, IterRec, write)
|
||||||
|
end
|
||||||
|
end,
|
||||||
|
AllStreams
|
||||||
|
)
|
||||||
|
end
|
||||||
|
).
|
||||||
|
|
||||||
%%--------------------------------------------------------------------------------
|
%%--------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
@ -649,23 +680,39 @@ transaction(Fun) ->
|
||||||
{atomic, Res} = mria:transaction(?DS_MRIA_SHARD, Fun),
|
{atomic, Res} = mria:transaction(?DS_MRIA_SHARD, Fun),
|
||||||
Res.
|
Res.
|
||||||
|
|
||||||
|
ro_transaction(Fun) ->
|
||||||
|
{atomic, Res} = mria:ro_transaction(?DS_MRIA_SHARD, Fun),
|
||||||
|
Res.
|
||||||
|
|
||||||
%%--------------------------------------------------------------------------------
|
%%--------------------------------------------------------------------------------
|
||||||
|
|
||||||
export_iterators(IteratorRefs) ->
|
export_subscriptions(DSSubs) ->
|
||||||
lists:foldl(
|
lists:foldl(
|
||||||
fun(IteratorRef = #iterator_ref{ref_id = {_DSSessionId, TopicFilter}}, Acc) ->
|
fun(DSSub = #ds_sub{id = {_DSSessionId, TopicFilter}}, Acc) ->
|
||||||
Acc#{TopicFilter => export_record(IteratorRef)}
|
Acc#{TopicFilter => export_subscription(DSSub)}
|
||||||
end,
|
end,
|
||||||
#{},
|
#{},
|
||||||
IteratorRefs
|
DSSubs
|
||||||
).
|
).
|
||||||
|
|
||||||
export_record(#session{} = Record) ->
|
export_session(#session{} = Record) ->
|
||||||
export_record(Record, #session.id, [id, created_at, expires_at, props], #{});
|
export_record(Record, #session.id, [id, created_at, expires_at, inflight, props], #{}).
|
||||||
export_record(#iterator_ref{} = Record) ->
|
|
||||||
export_record(Record, #iterator_ref.it_id, [id, start_time, props], #{}).
|
export_subscription(#ds_sub{} = Record) ->
|
||||||
|
export_record(Record, #ds_sub.start_time, [start_time, props, extra], #{}).
|
||||||
|
|
||||||
export_record(Record, I, [Field | Rest], Acc) ->
|
export_record(Record, I, [Field | Rest], Acc) ->
|
||||||
export_record(Record, I + 1, Rest, Acc#{Field => element(I, Record)});
|
export_record(Record, I + 1, Rest, Acc#{Field => element(I, Record)});
|
||||||
export_record(_, _, [], Acc) ->
|
export_record(_, _, [], Acc) ->
|
||||||
Acc.
|
Acc.
|
||||||
|
|
||||||
|
%% TODO: find a more reliable way to perform actions that have side
|
||||||
|
%% effects. Add `CBM:init' callback to the session behavior?
|
||||||
|
ensure_timers() ->
|
||||||
|
ensure_timer(pull),
|
||||||
|
ensure_timer(get_streams).
|
||||||
|
|
||||||
|
-spec ensure_timer(pull | get_streams) -> ok.
|
||||||
|
ensure_timer(Type) ->
|
||||||
|
_ = emqx_utils:start_timer(100, {emqx_session, Type}),
|
||||||
|
ok.
|
||||||
|
|
|
@ -0,0 +1,56 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%
|
||||||
|
%% Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
%% you may not use this file except in compliance with the License.
|
||||||
|
%% You may obtain a copy of the License at
|
||||||
|
%%
|
||||||
|
%% http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
%%
|
||||||
|
%% Unless required by applicable law or agreed to in writing, software
|
||||||
|
%% distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
%% See the License for the specific language governing permissions and
|
||||||
|
%% limitations under the License.
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
-ifndef(EMQX_PERSISTENT_SESSION_DS_HRL_HRL).
|
||||||
|
-define(EMQX_PERSISTENT_SESSION_DS_HRL_HRL, true).
|
||||||
|
|
||||||
|
-define(SESSION_TAB, emqx_ds_session).
|
||||||
|
-define(SESSION_SUBSCRIPTIONS_TAB, emqx_ds_session_subscriptions).
|
||||||
|
-define(SESSION_STREAM_TAB, emqx_ds_stream_tab).
|
||||||
|
-define(SESSION_ITER_TAB, emqx_ds_iter_tab).
|
||||||
|
-define(DS_MRIA_SHARD, emqx_ds_session_shard).
|
||||||
|
|
||||||
|
-record(ds_sub, {
|
||||||
|
id :: emqx_persistent_session_ds:subscription_id(),
|
||||||
|
start_time :: emqx_ds:time(),
|
||||||
|
props = #{} :: map(),
|
||||||
|
extra = #{} :: map()
|
||||||
|
}).
|
||||||
|
-type ds_sub() :: #ds_sub{}.
|
||||||
|
|
||||||
|
-record(ds_stream, {
|
||||||
|
session :: emqx_persistent_session_ds:id(),
|
||||||
|
topic_filter :: emqx_ds:topic_filter(),
|
||||||
|
stream :: emqx_ds:stream(),
|
||||||
|
rank :: emqx_ds:stream_rank()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-record(ds_iter, {
|
||||||
|
id :: {emqx_persistent_session_ds:id(), emqx_ds:stream()},
|
||||||
|
iter :: emqx_ds:iterator()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-record(session, {
|
||||||
|
%% same as clientid
|
||||||
|
id :: emqx_persistent_session_ds:id(),
|
||||||
|
%% creation time
|
||||||
|
created_at :: _Millisecond :: non_neg_integer(),
|
||||||
|
expires_at = never :: _Millisecond :: non_neg_integer() | never,
|
||||||
|
inflight :: emqx_persistent_message_ds_replayer:inflight(),
|
||||||
|
%% for future usage
|
||||||
|
props = #{} :: map()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-endif.
|
|
@ -20,6 +20,7 @@
|
||||||
|
|
||||||
-export([
|
-export([
|
||||||
introduced_in/0,
|
introduced_in/0,
|
||||||
|
deprecated_since/0,
|
||||||
|
|
||||||
open_iterator/4,
|
open_iterator/4,
|
||||||
close_iterator/2,
|
close_iterator/2,
|
||||||
|
@ -31,9 +32,11 @@
|
||||||
-define(TIMEOUT, 30_000).
|
-define(TIMEOUT, 30_000).
|
||||||
|
|
||||||
introduced_in() ->
|
introduced_in() ->
|
||||||
%% FIXME
|
|
||||||
"5.3.0".
|
"5.3.0".
|
||||||
|
|
||||||
|
deprecated_since() ->
|
||||||
|
"5.4.0".
|
||||||
|
|
||||||
-spec open_iterator(
|
-spec open_iterator(
|
||||||
[node()],
|
[node()],
|
||||||
emqx_types:words(),
|
emqx_types:words(),
|
||||||
|
|
|
@ -26,9 +26,7 @@
|
||||||
|
|
||||||
-import(emqx_common_test_helpers, [on_exit/1]).
|
-import(emqx_common_test_helpers, [on_exit/1]).
|
||||||
|
|
||||||
-define(DEFAULT_KEYSPACE, default).
|
-define(PERSISTENT_MESSAGE_DB, emqx_persistent_message).
|
||||||
-define(DS_SHARD_ID, <<"local">>).
|
|
||||||
-define(DS_SHARD, {?DEFAULT_KEYSPACE, ?DS_SHARD_ID}).
|
|
||||||
|
|
||||||
all() ->
|
all() ->
|
||||||
emqx_common_test_helpers:all(?MODULE).
|
emqx_common_test_helpers:all(?MODULE).
|
||||||
|
@ -48,6 +46,7 @@ init_per_testcase(t_session_subscription_iterators = TestCase, Config) ->
|
||||||
Nodes = emqx_cth_cluster:start(Cluster, #{work_dir => emqx_cth_suite:work_dir(TestCase, Config)}),
|
Nodes = emqx_cth_cluster:start(Cluster, #{work_dir => emqx_cth_suite:work_dir(TestCase, Config)}),
|
||||||
[{nodes, Nodes} | Config];
|
[{nodes, Nodes} | Config];
|
||||||
init_per_testcase(TestCase, Config) ->
|
init_per_testcase(TestCase, Config) ->
|
||||||
|
ok = emqx_ds:drop_db(?PERSISTENT_MESSAGE_DB),
|
||||||
Apps = emqx_cth_suite:start(
|
Apps = emqx_cth_suite:start(
|
||||||
app_specs(),
|
app_specs(),
|
||||||
#{work_dir => emqx_cth_suite:work_dir(TestCase, Config)}
|
#{work_dir => emqx_cth_suite:work_dir(TestCase, Config)}
|
||||||
|
@ -58,10 +57,11 @@ end_per_testcase(t_session_subscription_iterators, Config) ->
|
||||||
Nodes = ?config(nodes, Config),
|
Nodes = ?config(nodes, Config),
|
||||||
emqx_common_test_helpers:call_janitor(60_000),
|
emqx_common_test_helpers:call_janitor(60_000),
|
||||||
ok = emqx_cth_cluster:stop(Nodes),
|
ok = emqx_cth_cluster:stop(Nodes),
|
||||||
ok;
|
end_per_testcase(common, Config);
|
||||||
end_per_testcase(_TestCase, Config) ->
|
end_per_testcase(_TestCase, Config) ->
|
||||||
Apps = ?config(apps, Config),
|
Apps = proplists:get_value(apps, Config, []),
|
||||||
emqx_common_test_helpers:call_janitor(60_000),
|
emqx_common_test_helpers:call_janitor(60_000),
|
||||||
|
clear_db(),
|
||||||
emqx_cth_suite:stop(Apps),
|
emqx_cth_suite:stop(Apps),
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
|
@ -95,14 +95,15 @@ t_messages_persisted(_Config) ->
|
||||||
Results = [emqtt:publish(CP, Topic, Payload, 1) || {Topic, Payload} <- Messages],
|
Results = [emqtt:publish(CP, Topic, Payload, 1) || {Topic, Payload} <- Messages],
|
||||||
|
|
||||||
ct:pal("Results = ~p", [Results]),
|
ct:pal("Results = ~p", [Results]),
|
||||||
|
timer:sleep(2000),
|
||||||
|
|
||||||
Persisted = consume(?DS_SHARD, {['#'], 0}),
|
Persisted = consume(['#'], 0),
|
||||||
|
|
||||||
ct:pal("Persisted = ~p", [Persisted]),
|
ct:pal("Persisted = ~p", [Persisted]),
|
||||||
|
|
||||||
?assertEqual(
|
?assertEqual(
|
||||||
[M1, M2, M5, M7, M9, M10],
|
lists:sort([M1, M2, M5, M7, M9, M10]),
|
||||||
[{emqx_message:topic(M), emqx_message:payload(M)} || M <- Persisted]
|
lists:sort([{emqx_message:topic(M), emqx_message:payload(M)} || M <- Persisted])
|
||||||
),
|
),
|
||||||
|
|
||||||
ok.
|
ok.
|
||||||
|
@ -139,23 +140,25 @@ t_messages_persisted_2(_Config) ->
|
||||||
{ok, #{reason_code := ?RC_NO_MATCHING_SUBSCRIBERS}} =
|
{ok, #{reason_code := ?RC_NO_MATCHING_SUBSCRIBERS}} =
|
||||||
emqtt:publish(CP, T(<<"client/2/topic">>), <<"8">>, 1),
|
emqtt:publish(CP, T(<<"client/2/topic">>), <<"8">>, 1),
|
||||||
|
|
||||||
Persisted = consume(?DS_SHARD, {['#'], 0}),
|
timer:sleep(2000),
|
||||||
|
|
||||||
|
Persisted = consume(['#'], 0),
|
||||||
|
|
||||||
ct:pal("Persisted = ~p", [Persisted]),
|
ct:pal("Persisted = ~p", [Persisted]),
|
||||||
|
|
||||||
?assertEqual(
|
?assertEqual(
|
||||||
[
|
lists:sort([
|
||||||
{T(<<"client/1/topic">>), <<"4">>},
|
{T(<<"client/1/topic">>), <<"4">>},
|
||||||
{T(<<"client/2/topic">>), <<"5">>}
|
{T(<<"client/2/topic">>), <<"5">>}
|
||||||
],
|
]),
|
||||||
[{emqx_message:topic(M), emqx_message:payload(M)} || M <- Persisted]
|
lists:sort([{emqx_message:topic(M), emqx_message:payload(M)} || M <- Persisted])
|
||||||
),
|
),
|
||||||
|
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
%% TODO: test quic and ws too
|
%% TODO: test quic and ws too
|
||||||
t_session_subscription_iterators(Config) ->
|
t_session_subscription_iterators(Config) ->
|
||||||
[Node1, Node2] = ?config(nodes, Config),
|
[Node1, _Node2] = ?config(nodes, Config),
|
||||||
Port = get_mqtt_port(Node1, tcp),
|
Port = get_mqtt_port(Node1, tcp),
|
||||||
Topic = <<"t/topic">>,
|
Topic = <<"t/topic">>,
|
||||||
SubTopicFilter = <<"t/+">>,
|
SubTopicFilter = <<"t/+">>,
|
||||||
|
@ -202,11 +205,8 @@ t_session_subscription_iterators(Config) ->
|
||||||
messages => [Message1, Message2, Message3, Message4]
|
messages => [Message1, Message2, Message3, Message4]
|
||||||
}
|
}
|
||||||
end,
|
end,
|
||||||
fun(Results, Trace) ->
|
fun(Trace) ->
|
||||||
ct:pal("trace:\n ~p", [Trace]),
|
ct:pal("trace:\n ~p", [Trace]),
|
||||||
#{
|
|
||||||
messages := [_Message1, Message2, Message3 | _]
|
|
||||||
} = Results,
|
|
||||||
case ?of_kind(ds_session_subscription_added, Trace) of
|
case ?of_kind(ds_session_subscription_added, Trace) of
|
||||||
[] ->
|
[] ->
|
||||||
%% Since `emqx_durable_storage' is a dependency of `emqx', it gets
|
%% Since `emqx_durable_storage' is a dependency of `emqx', it gets
|
||||||
|
@ -228,17 +228,6 @@ t_session_subscription_iterators(Config) ->
|
||||||
),
|
),
|
||||||
ok
|
ok
|
||||||
end,
|
end,
|
||||||
?assertMatch({ok, [_]}, get_all_iterator_ids(Node1)),
|
|
||||||
{ok, [IteratorId]} = get_all_iterator_ids(Node1),
|
|
||||||
?assertMatch({ok, [IteratorId]}, get_all_iterator_ids(Node2)),
|
|
||||||
ReplayMessages1 = erpc:call(Node1, fun() -> consume(?DS_SHARD, IteratorId) end),
|
|
||||||
ExpectedMessages = [Message2, Message3],
|
|
||||||
%% Note: it is expected that this will break after replayers are in place.
|
|
||||||
%% They might have consumed all the messages by this time.
|
|
||||||
?assertEqual(ExpectedMessages, ReplayMessages1),
|
|
||||||
%% Different DS shard
|
|
||||||
ReplayMessages2 = erpc:call(Node2, fun() -> consume(?DS_SHARD, IteratorId) end),
|
|
||||||
?assertEqual([], ReplayMessages2),
|
|
||||||
ok
|
ok
|
||||||
end
|
end
|
||||||
),
|
),
|
||||||
|
@ -263,33 +252,26 @@ connect(Opts0 = #{}) ->
|
||||||
{ok, _} = emqtt:connect(Client),
|
{ok, _} = emqtt:connect(Client),
|
||||||
Client.
|
Client.
|
||||||
|
|
||||||
consume(Shard, Replay = {_TopicFiler, _StartMS}) ->
|
consume(TopicFilter, StartMS) ->
|
||||||
{ok, It} = emqx_ds_storage_layer:make_iterator(Shard, Replay),
|
Streams = emqx_ds:get_streams(?PERSISTENT_MESSAGE_DB, TopicFilter, StartMS),
|
||||||
consume(It);
|
lists:flatmap(
|
||||||
consume(Shard, IteratorId) when is_binary(IteratorId) ->
|
fun({_Rank, Stream}) ->
|
||||||
{ok, It} = emqx_ds_storage_layer:restore_iterator(Shard, IteratorId),
|
{ok, It} = emqx_ds:make_iterator(Stream, TopicFilter, StartMS),
|
||||||
consume(It).
|
consume(It)
|
||||||
|
end,
|
||||||
|
Streams
|
||||||
|
).
|
||||||
|
|
||||||
consume(It) ->
|
consume(It) ->
|
||||||
case emqx_ds_storage_layer:next(It) of
|
case emqx_ds:next(It, 100) of
|
||||||
{value, Msg, NIt} ->
|
{ok, _NIt, _Msgs = []} ->
|
||||||
[emqx_persistent_message:deserialize(Msg) | consume(NIt)];
|
[];
|
||||||
none ->
|
{ok, NIt, Msgs} ->
|
||||||
|
Msgs ++ consume(NIt);
|
||||||
|
{ok, end_of_stream} ->
|
||||||
[]
|
[]
|
||||||
end.
|
end.
|
||||||
|
|
||||||
delete_all_messages() ->
|
|
||||||
Persisted = consume(?DS_SHARD, {['#'], 0}),
|
|
||||||
lists:foreach(
|
|
||||||
fun(Msg) ->
|
|
||||||
GUID = emqx_message:id(Msg),
|
|
||||||
Topic = emqx_topic:words(emqx_message:topic(Msg)),
|
|
||||||
Timestamp = emqx_guid:timestamp(GUID),
|
|
||||||
ok = emqx_ds_storage_layer:delete(?DS_SHARD, GUID, Timestamp, Topic)
|
|
||||||
end,
|
|
||||||
Persisted
|
|
||||||
).
|
|
||||||
|
|
||||||
receive_messages(Count) ->
|
receive_messages(Count) ->
|
||||||
receive_messages(Count, []).
|
receive_messages(Count, []).
|
||||||
|
|
||||||
|
@ -306,13 +288,6 @@ receive_messages(Count, Msgs) ->
|
||||||
publish(Node, Message) ->
|
publish(Node, Message) ->
|
||||||
erpc:call(Node, emqx, publish, [Message]).
|
erpc:call(Node, emqx, publish, [Message]).
|
||||||
|
|
||||||
get_iterator_ids(Node, ClientId) ->
|
|
||||||
Channel = erpc:call(Node, fun() ->
|
|
||||||
[ConnPid] = emqx_cm:lookup_channels(ClientId),
|
|
||||||
sys:get_state(ConnPid)
|
|
||||||
end),
|
|
||||||
emqx_connection:info({channel, {session, iterators}}, Channel).
|
|
||||||
|
|
||||||
app_specs() ->
|
app_specs() ->
|
||||||
[
|
[
|
||||||
emqx_durable_storage,
|
emqx_durable_storage,
|
||||||
|
@ -330,5 +305,6 @@ get_mqtt_port(Node, Type) ->
|
||||||
{_IP, Port} = erpc:call(Node, emqx_config, get, [[listeners, Type, default, bind]]),
|
{_IP, Port} = erpc:call(Node, emqx_config, get, [[listeners, Type, default, bind]]),
|
||||||
Port.
|
Port.
|
||||||
|
|
||||||
get_all_iterator_ids(Node) ->
|
clear_db() ->
|
||||||
erpc:call(Node, emqx_ds_storage_layer, list_iterator_prefix, [?DS_SHARD, <<>>]).
|
ok = emqx_ds:drop_db(?PERSISTENT_MESSAGE_DB),
|
||||||
|
ok.
|
||||||
|
|
|
@ -24,6 +24,8 @@
|
||||||
-compile(export_all).
|
-compile(export_all).
|
||||||
-compile(nowarn_export_all).
|
-compile(nowarn_export_all).
|
||||||
|
|
||||||
|
-define(PERSISTENT_MESSAGE_DB, emqx_persistent_message).
|
||||||
|
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
%% SUITE boilerplate
|
%% SUITE boilerplate
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
|
@ -131,6 +133,7 @@ get_listener_port(Type, Name) ->
|
||||||
end_per_group(Group, Config) when Group == tcp; Group == ws; Group == quic ->
|
end_per_group(Group, Config) when Group == tcp; Group == ws; Group == quic ->
|
||||||
ok = emqx_cth_suite:stop(?config(group_apps, Config));
|
ok = emqx_cth_suite:stop(?config(group_apps, Config));
|
||||||
end_per_group(_, _Config) ->
|
end_per_group(_, _Config) ->
|
||||||
|
ok = emqx_ds:drop_db(?PERSISTENT_MESSAGE_DB),
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
init_per_testcase(TestCase, Config) ->
|
init_per_testcase(TestCase, Config) ->
|
||||||
|
@ -188,7 +191,7 @@ receive_messages(Count, Msgs) ->
|
||||||
receive_messages(Count - 1, [Msg | Msgs]);
|
receive_messages(Count - 1, [Msg | Msgs]);
|
||||||
_Other ->
|
_Other ->
|
||||||
receive_messages(Count, Msgs)
|
receive_messages(Count, Msgs)
|
||||||
after 5000 ->
|
after 15000 ->
|
||||||
Msgs
|
Msgs
|
||||||
end.
|
end.
|
||||||
|
|
||||||
|
@ -227,11 +230,11 @@ wait_for_cm_unregister(ClientId, N) ->
|
||||||
end.
|
end.
|
||||||
|
|
||||||
publish(Topic, Payloads) ->
|
publish(Topic, Payloads) ->
|
||||||
publish(Topic, Payloads, false).
|
publish(Topic, Payloads, false, 2).
|
||||||
|
|
||||||
publish(Topic, Payloads, WaitForUnregister) ->
|
publish(Topic, Payloads, WaitForUnregister, QoS) ->
|
||||||
Fun = fun(Client, Payload) ->
|
Fun = fun(Client, Payload) ->
|
||||||
{ok, _} = emqtt:publish(Client, Topic, Payload, 2)
|
{ok, _} = emqtt:publish(Client, Topic, Payload, QoS)
|
||||||
end,
|
end,
|
||||||
do_publish(Payloads, Fun, WaitForUnregister).
|
do_publish(Payloads, Fun, WaitForUnregister).
|
||||||
|
|
||||||
|
@ -510,6 +513,48 @@ t_process_dies_session_expires(Config) ->
|
||||||
|
|
||||||
emqtt:disconnect(Client2).
|
emqtt:disconnect(Client2).
|
||||||
|
|
||||||
|
t_publish_while_client_is_gone_qos1(Config) ->
|
||||||
|
%% A persistent session should receive messages in its
|
||||||
|
%% subscription even if the process owning the session dies.
|
||||||
|
ConnFun = ?config(conn_fun, Config),
|
||||||
|
Topic = ?config(topic, Config),
|
||||||
|
STopic = ?config(stopic, Config),
|
||||||
|
Payload1 = <<"hello1">>,
|
||||||
|
Payload2 = <<"hello2">>,
|
||||||
|
ClientId = ?config(client_id, Config),
|
||||||
|
{ok, Client1} = emqtt:start_link([
|
||||||
|
{proto_ver, v5},
|
||||||
|
{clientid, ClientId},
|
||||||
|
{properties, #{'Session-Expiry-Interval' => 30}},
|
||||||
|
{clean_start, true}
|
||||||
|
| Config
|
||||||
|
]),
|
||||||
|
{ok, _} = emqtt:ConnFun(Client1),
|
||||||
|
{ok, _, [1]} = emqtt:subscribe(Client1, STopic, qos1),
|
||||||
|
|
||||||
|
ok = emqtt:disconnect(Client1),
|
||||||
|
maybe_kill_connection_process(ClientId, Config),
|
||||||
|
|
||||||
|
ok = publish(Topic, [Payload1, Payload2], false, 1),
|
||||||
|
|
||||||
|
{ok, Client2} = emqtt:start_link([
|
||||||
|
{proto_ver, v5},
|
||||||
|
{clientid, ClientId},
|
||||||
|
{properties, #{'Session-Expiry-Interval' => 30}},
|
||||||
|
{clean_start, false}
|
||||||
|
| Config
|
||||||
|
]),
|
||||||
|
{ok, _} = emqtt:ConnFun(Client2),
|
||||||
|
Msgs = receive_messages(2),
|
||||||
|
?assertMatch([_, _], Msgs),
|
||||||
|
[Msg2, Msg1] = Msgs,
|
||||||
|
?assertEqual({ok, iolist_to_binary(Payload1)}, maps:find(payload, Msg1)),
|
||||||
|
?assertEqual({ok, 1}, maps:find(qos, Msg1)),
|
||||||
|
?assertEqual({ok, iolist_to_binary(Payload2)}, maps:find(payload, Msg2)),
|
||||||
|
?assertEqual({ok, 1}, maps:find(qos, Msg2)),
|
||||||
|
|
||||||
|
ok = emqtt:disconnect(Client2).
|
||||||
|
|
||||||
t_publish_while_client_is_gone(init, Config) -> skip_ds_tc(Config);
|
t_publish_while_client_is_gone(init, Config) -> skip_ds_tc(Config);
|
||||||
t_publish_while_client_is_gone('end', _Config) -> ok.
|
t_publish_while_client_is_gone('end', _Config) -> ok.
|
||||||
t_publish_while_client_is_gone(Config) ->
|
t_publish_while_client_is_gone(Config) ->
|
||||||
|
|
|
@ -31,48 +31,6 @@ Read pattern: pseudoserial
|
||||||
|
|
||||||
Number of records: O(total write throughput * retention time)
|
Number of records: O(total write throughput * retention time)
|
||||||
|
|
||||||
## Session storage
|
|
||||||
|
|
||||||
Data there is updated when:
|
|
||||||
|
|
||||||
- A new client connects with clean session = false
|
|
||||||
- Client subscribes to a topic
|
|
||||||
- Client unsubscribes to a topic
|
|
||||||
- Garbage collection is performed
|
|
||||||
|
|
||||||
Write throughput: low
|
|
||||||
|
|
||||||
Data is read when a client connects and replay agents are started
|
|
||||||
|
|
||||||
Read throughput: low
|
|
||||||
|
|
||||||
Data format:
|
|
||||||
|
|
||||||
`#session{clientId = "foobar", iterators = [ItKey1, ItKey2, ItKey3, ...]}`
|
|
||||||
|
|
||||||
Number of records: O(N clients)
|
|
||||||
|
|
||||||
Size of record: O(N subscriptions per clients)
|
|
||||||
|
|
||||||
## Iterator storage
|
|
||||||
|
|
||||||
Data is written every time a client acks a message.
|
|
||||||
|
|
||||||
Data is read when a client reconnects and we restart replay agents.
|
|
||||||
|
|
||||||
`#iterator{key = IterKey, data = Blob}`
|
|
||||||
|
|
||||||
Number of records: O(N clients * N subscriptions per client)
|
|
||||||
|
|
||||||
Size of record: O(1)
|
|
||||||
|
|
||||||
Write throughput: high, lots of small updates
|
|
||||||
|
|
||||||
Write pattern: mostly key overwrite
|
|
||||||
|
|
||||||
Read throughput: low
|
|
||||||
|
|
||||||
Read pattern: random
|
|
||||||
|
|
||||||
# Push vs. Pull model
|
# Push vs. Pull model
|
||||||
|
|
||||||
|
|
|
@ -1,9 +1,10 @@
|
||||||
# EMQX Replay
|
# EMQX Replay
|
||||||
|
|
||||||
`emqx_ds` is a durable storage for MQTT messages within EMQX.
|
`emqx_ds` is a generic durable storage for MQTT messages within EMQX.
|
||||||
It implements the following scenarios:
|
|
||||||
- Persisting messages published by clients
|
Concepts:
|
||||||
-
|
|
||||||
|
|
||||||
|
|
||||||
> 0. App overview introduction
|
> 0. App overview introduction
|
||||||
> 1. let people know what your project can do specifically. Is it a base
|
> 1. let people know what your project can do specifically. Is it a base
|
||||||
|
|
|
@ -0,0 +1,19 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%
|
||||||
|
%% Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
%% you may not use this file except in compliance with the License.
|
||||||
|
%% You may obtain a copy of the License at
|
||||||
|
%%
|
||||||
|
%% http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
%%
|
||||||
|
%% Unless required by applicable law or agreed to in writing, software
|
||||||
|
%% distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
%% See the License for the specific language governing permissions and
|
||||||
|
%% limitations under the License.
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
-ifndef(EMQX_DS_HRL_HRL).
|
||||||
|
-define(EMQX_DS_HRL_HRL, true).
|
||||||
|
|
||||||
|
-endif.
|
|
@ -0,0 +1,3 @@
|
||||||
|
%% -*- mode:erlang -*-
|
||||||
|
{deps,
|
||||||
|
[{emqx_utils, {path, "../emqx_utils"}}]}.
|
|
@ -13,50 +13,48 @@
|
||||||
%% See the License for the specific language governing permissions and
|
%% See the License for the specific language governing permissions and
|
||||||
%% limitations under the License.
|
%% limitations under the License.
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
|
|
||||||
|
%% @doc Main interface module for `emqx_durable_storage' application.
|
||||||
|
%%
|
||||||
|
%% It takes care of forwarding calls to the underlying DBMS. Currently
|
||||||
|
%% only the embedded `emqx_ds_replication_layer' storage is supported,
|
||||||
|
%% so all the calls are simply passed through.
|
||||||
-module(emqx_ds).
|
-module(emqx_ds).
|
||||||
|
|
||||||
-include_lib("stdlib/include/ms_transform.hrl").
|
%% Management API:
|
||||||
-include_lib("snabbkaffe/include/snabbkaffe.hrl").
|
-export([open_db/2, drop_db/1]).
|
||||||
|
|
||||||
%% API:
|
%% Message storage API:
|
||||||
-export([ensure_shard/2]).
|
-export([store_batch/2, store_batch/3]).
|
||||||
%% Messages:
|
|
||||||
-export([message_store/2, message_store/1, message_stats/0]).
|
|
||||||
%% Iterator:
|
|
||||||
-export([iterator_update/2, iterator_next/1, iterator_stats/0]).
|
|
||||||
|
|
||||||
%% internal exports:
|
%% Message replay API:
|
||||||
|
-export([get_streams/3, make_iterator/3, next/2]).
|
||||||
|
|
||||||
|
%% Misc. API:
|
||||||
-export([]).
|
-export([]).
|
||||||
|
|
||||||
-export_type([
|
-export_type([
|
||||||
keyspace/0,
|
create_db_opts/0,
|
||||||
message_id/0,
|
builtin_db_opts/0,
|
||||||
message_stats/0,
|
db/0,
|
||||||
message_store_opts/0,
|
time/0,
|
||||||
replay/0,
|
|
||||||
replay_id/0,
|
|
||||||
iterator_id/0,
|
|
||||||
iterator/0,
|
|
||||||
shard/0,
|
|
||||||
shard_id/0,
|
|
||||||
topic/0,
|
|
||||||
topic_filter/0,
|
topic_filter/0,
|
||||||
time/0
|
topic/0,
|
||||||
|
stream/0,
|
||||||
|
stream_rank/0,
|
||||||
|
iterator/0,
|
||||||
|
message_id/0,
|
||||||
|
next_result/1, next_result/0,
|
||||||
|
store_batch_result/0,
|
||||||
|
make_iterator_result/1, make_iterator_result/0,
|
||||||
|
get_iterator_result/1
|
||||||
]).
|
]).
|
||||||
|
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
%% Type declarations
|
%% Type declarations
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
||||||
-type iterator() :: term().
|
-type db() :: atom().
|
||||||
|
|
||||||
-type iterator_id() :: binary().
|
|
||||||
|
|
||||||
-type message_store_opts() :: #{}.
|
|
||||||
|
|
||||||
-type message_stats() :: #{}.
|
|
||||||
|
|
||||||
-type message_id() :: binary().
|
|
||||||
|
|
||||||
%% Parsed topic.
|
%% Parsed topic.
|
||||||
-type topic() :: list(binary()).
|
-type topic() :: list(binary()).
|
||||||
|
@ -64,9 +62,22 @@
|
||||||
%% Parsed topic filter.
|
%% Parsed topic filter.
|
||||||
-type topic_filter() :: list(binary() | '+' | '#' | '').
|
-type topic_filter() :: list(binary() | '+' | '#' | '').
|
||||||
|
|
||||||
-type keyspace() :: atom().
|
-type stream_rank() :: {term(), integer()}.
|
||||||
-type shard_id() :: binary().
|
|
||||||
-type shard() :: {keyspace(), shard_id()}.
|
-opaque stream() :: emqx_ds_replication_layer:stream().
|
||||||
|
|
||||||
|
-opaque iterator() :: emqx_ds_replication_layer:iterator().
|
||||||
|
|
||||||
|
-type store_batch_result() :: ok | {error, _}.
|
||||||
|
|
||||||
|
-type make_iterator_result(Iterator) :: {ok, Iterator} | {error, _}.
|
||||||
|
|
||||||
|
-type make_iterator_result() :: make_iterator_result(iterator()).
|
||||||
|
|
||||||
|
-type next_result(Iterator) ::
|
||||||
|
{ok, Iterator, [emqx_types:message()]} | {ok, end_of_stream} | {error, _}.
|
||||||
|
|
||||||
|
-type next_result() :: next_result(iterator()).
|
||||||
|
|
||||||
%% Timestamp
|
%% Timestamp
|
||||||
%% Earliest possible timestamp is 0.
|
%% Earliest possible timestamp is 0.
|
||||||
|
@ -74,70 +85,102 @@
|
||||||
%% use in emqx_guid. Otherwise, the iterators won't match the message timestamps.
|
%% use in emqx_guid. Otherwise, the iterators won't match the message timestamps.
|
||||||
-type time() :: non_neg_integer().
|
-type time() :: non_neg_integer().
|
||||||
|
|
||||||
-type replay_id() :: binary().
|
-type message_store_opts() :: #{}.
|
||||||
|
|
||||||
-type replay() :: {
|
-type builtin_db_opts() ::
|
||||||
_TopicFilter :: topic_filter(),
|
#{
|
||||||
_StartTime :: time()
|
backend := builtin,
|
||||||
}.
|
storage := emqx_ds_storage_layer:prototype()
|
||||||
|
}.
|
||||||
|
|
||||||
|
-type create_db_opts() ::
|
||||||
|
builtin_db_opts().
|
||||||
|
|
||||||
|
-type message_id() :: emqx_ds_replication_layer:message_id().
|
||||||
|
|
||||||
|
-type get_iterator_result(Iterator) :: {ok, Iterator} | undefined.
|
||||||
|
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
%% API funcions
|
%% API funcions
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
||||||
-spec ensure_shard(shard(), emqx_ds_storage_layer:options()) ->
|
%% @doc Different DBs are completely independent from each other. They
|
||||||
ok | {error, _Reason}.
|
%% could represent something like different tenants.
|
||||||
ensure_shard(Shard, Options) ->
|
-spec open_db(db(), create_db_opts()) -> ok.
|
||||||
case emqx_ds_storage_layer_sup:start_shard(Shard, Options) of
|
open_db(DB, Opts = #{backend := builtin}) ->
|
||||||
{ok, _Pid} ->
|
emqx_ds_replication_layer:open_db(DB, Opts).
|
||||||
ok;
|
|
||||||
{error, {already_started, _Pid}} ->
|
|
||||||
ok;
|
|
||||||
{error, Reason} ->
|
|
||||||
{error, Reason}
|
|
||||||
end.
|
|
||||||
|
|
||||||
%%--------------------------------------------------------------------------------
|
%% @doc TODO: currently if one or a few shards are down, they won't be
|
||||||
%% Message
|
|
||||||
%%--------------------------------------------------------------------------------
|
|
||||||
-spec message_store([emqx_types:message()], message_store_opts()) ->
|
|
||||||
{ok, [message_id()]} | {error, _}.
|
|
||||||
message_store(_Msg, _Opts) ->
|
|
||||||
%% TODO
|
|
||||||
{error, not_implemented}.
|
|
||||||
|
|
||||||
-spec message_store([emqx_types:message()]) -> {ok, [message_id()]} | {error, _}.
|
%% deleted.
|
||||||
message_store(Msg) ->
|
-spec drop_db(db()) -> ok.
|
||||||
%% TODO
|
drop_db(DB) ->
|
||||||
message_store(Msg, #{}).
|
emqx_ds_replication_layer:drop_db(DB).
|
||||||
|
|
||||||
-spec message_stats() -> message_stats().
|
-spec store_batch(db(), [emqx_types:message()], message_store_opts()) -> store_batch_result().
|
||||||
message_stats() ->
|
store_batch(DB, Msgs, Opts) ->
|
||||||
#{}.
|
emqx_ds_replication_layer:store_batch(DB, Msgs, Opts).
|
||||||
|
|
||||||
%%--------------------------------------------------------------------------------
|
-spec store_batch(db(), [emqx_types:message()]) -> store_batch_result().
|
||||||
%% Session
|
store_batch(DB, Msgs) ->
|
||||||
%%--------------------------------------------------------------------------------
|
store_batch(DB, Msgs, #{}).
|
||||||
|
|
||||||
%%--------------------------------------------------------------------------------
|
%% @doc Get a list of streams needed for replaying a topic filter.
|
||||||
%% Iterator (pull API)
|
%%
|
||||||
%%--------------------------------------------------------------------------------
|
%% Motivation: under the hood, EMQX may store different topics at
|
||||||
|
%% different locations or even in different databases. A wildcard
|
||||||
|
%% topic filter may require pulling data from any number of locations.
|
||||||
|
%%
|
||||||
|
%% Stream is an abstraction exposed by `emqx_ds' that, on one hand,
|
||||||
|
%% reflects the notion that different topics can be stored
|
||||||
|
%% differently, but hides the implementation details.
|
||||||
|
%%
|
||||||
|
%% While having to work with multiple iterators to replay a topic
|
||||||
|
%% filter may be cumbersome, it opens up some possibilities:
|
||||||
|
%%
|
||||||
|
%% 1. It's possible to parallelize replays
|
||||||
|
%%
|
||||||
|
%% 2. Streams can be shared between different clients to implement
|
||||||
|
%% shared subscriptions
|
||||||
|
%%
|
||||||
|
%% IMPORTANT RULES:
|
||||||
|
%%
|
||||||
|
%% 0. There is no 1-to-1 mapping between MQTT topics and streams. One
|
||||||
|
%% stream can contain any number of MQTT topics.
|
||||||
|
%%
|
||||||
|
%% 1. New streams matching the topic filter and start time can appear
|
||||||
|
%% without notice, so the replayer must periodically call this
|
||||||
|
%% function to get the updated list of streams.
|
||||||
|
%%
|
||||||
|
%% 2. Streams may depend on one another. Therefore, care should be
|
||||||
|
%% taken while replaying them in parallel to avoid out-of-order
|
||||||
|
%% replay. This function returns stream together with its
|
||||||
|
%% "coordinate": `stream_rank()'.
|
||||||
|
%%
|
||||||
|
%% Stream rank is a tuple of two integers, let's call them X and Y. If
|
||||||
|
%% X coordinate of two streams is different, they are independent and
|
||||||
|
%% can be replayed in parallel. If it's the same, then the stream with
|
||||||
|
%% smaller Y coordinate should be replayed first. If Y coordinates are
|
||||||
|
%% equal, then the streams are independent.
|
||||||
|
%%
|
||||||
|
%% Stream is fully consumed when `next/3' function returns
|
||||||
|
%% `end_of_stream'. Then and only then the client can proceed to
|
||||||
|
%% replaying streams that depend on the given one.
|
||||||
|
-spec get_streams(db(), topic_filter(), time()) -> [{stream_rank(), stream()}].
|
||||||
|
get_streams(DB, TopicFilter, StartTime) ->
|
||||||
|
emqx_ds_replication_layer:get_streams(DB, TopicFilter, StartTime).
|
||||||
|
|
||||||
%% @doc Called when a client acks a message
|
-spec make_iterator(stream(), topic_filter(), time()) -> make_iterator_result().
|
||||||
-spec iterator_update(iterator_id(), iterator()) -> ok.
|
make_iterator(Stream, TopicFilter, StartTime) ->
|
||||||
iterator_update(_IterId, _Iter) ->
|
emqx_ds_replication_layer:make_iterator(Stream, TopicFilter, StartTime).
|
||||||
%% TODO
|
|
||||||
ok.
|
|
||||||
|
|
||||||
%% @doc Called when a client acks a message
|
-spec next(iterator(), pos_integer()) -> next_result().
|
||||||
-spec iterator_next(iterator()) -> {value, emqx_types:message(), iterator()} | none | {error, _}.
|
next(Iter, BatchSize) ->
|
||||||
iterator_next(_Iter) ->
|
emqx_ds_replication_layer:next(Iter, BatchSize).
|
||||||
%% TODO
|
|
||||||
none.
|
|
||||||
|
|
||||||
-spec iterator_stats() -> #{}.
|
%%================================================================================
|
||||||
iterator_stats() ->
|
%% Internal exports
|
||||||
#{}.
|
%%================================================================================
|
||||||
|
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
%% Internal functions
|
%% Internal functions
|
||||||
|
|
|
@ -0,0 +1,36 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2022-2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%
|
||||||
|
%% Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
%% you may not use this file except in compliance with the License.
|
||||||
|
%% You may obtain a copy of the License at
|
||||||
|
%%
|
||||||
|
%% http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
%%
|
||||||
|
%% Unless required by applicable law or agreed to in writing, software
|
||||||
|
%% distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
%% See the License for the specific language governing permissions and
|
||||||
|
%% limitations under the License.
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
-ifndef(EMQX_DS_BITMASK_HRL).
|
||||||
|
-define(EMQX_DS_BITMASK_HRL, true).
|
||||||
|
|
||||||
|
-record(filter_scan_action, {
|
||||||
|
offset :: emqx_ds_bitmask_keymapper:offset(),
|
||||||
|
size :: emqx_ds_bitmask_keymapper:bitsize(),
|
||||||
|
min :: non_neg_integer(),
|
||||||
|
max :: non_neg_integer()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-record(filter, {
|
||||||
|
size :: non_neg_integer(),
|
||||||
|
bitfilter :: non_neg_integer(),
|
||||||
|
bitmask :: non_neg_integer(),
|
||||||
|
%% Ranges (in _bitsource_ basis):
|
||||||
|
bitsource_ranges :: array:array(#filter_scan_action{}),
|
||||||
|
range_min :: non_neg_integer(),
|
||||||
|
range_max :: non_neg_integer()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-endif.
|
|
@ -0,0 +1,824 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2022-2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
-module(emqx_ds_bitmask_keymapper).
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% @doc This module is used to map N-dimensional coordinates to a
|
||||||
|
%% 1-dimensional space.
|
||||||
|
%%
|
||||||
|
%% Example:
|
||||||
|
%%
|
||||||
|
%% Let us assume that `T' is a topic and `t' is time. These are the two
|
||||||
|
%% dimensions used to index messages. They can be viewed as
|
||||||
|
%% "coordinates" of an MQTT message in a 2D space.
|
||||||
|
%%
|
||||||
|
%% Oftentimes, when wildcard subscription is used, keys must be
|
||||||
|
%% scanned in both dimensions simultaneously.
|
||||||
|
%%
|
||||||
|
%% Rocksdb allows to iterate over sorted keys very fast. This means we
|
||||||
|
%% need to map our two-dimentional keys to a single index that is
|
||||||
|
%% sorted in a way that helps to iterate over both time and topic
|
||||||
|
%% without having to do a lot of random seeks.
|
||||||
|
%%
|
||||||
|
%% == Mapping of 2D keys to rocksdb keys ==
|
||||||
|
%%
|
||||||
|
%% We use "zigzag" pattern to store messages, where rocksdb key is
|
||||||
|
%% composed like like this:
|
||||||
|
%%
|
||||||
|
%% |ttttt|TTTTTTTTT|tttt|
|
||||||
|
%% ^ ^ ^
|
||||||
|
%% | | |
|
||||||
|
%% +-------+ | +---------+
|
||||||
|
%% | | |
|
||||||
|
%% most significant topic hash least significant
|
||||||
|
%% bits of timestamp bits of timestamp
|
||||||
|
%% (a.k.a epoch) (a.k.a time offset)
|
||||||
|
%%
|
||||||
|
%% Topic hash is level-aware: each topic level is hashed separately
|
||||||
|
%% and the resulting hashes are bitwise-concatentated. This allows us
|
||||||
|
%% to map topics to fixed-length bitstrings while keeping some degree
|
||||||
|
%% of information about the hierarchy.
|
||||||
|
%%
|
||||||
|
%% Next important concept is what we call "epoch". Duration of the
|
||||||
|
%% epoch is determined by maximum time offset. Epoch is calculated by
|
||||||
|
%% shifting bits of the timestamp right.
|
||||||
|
%%
|
||||||
|
%% The resulting index is a space-filling curve that looks like
|
||||||
|
%% this in the topic-time 2D space:
|
||||||
|
%%
|
||||||
|
%% T ^ ---->------ |---->------ |---->------
|
||||||
|
%% | --/ / --/ / --/
|
||||||
|
%% | -<-/ | -<-/ | -<-/
|
||||||
|
%% | -/ | -/ | -/
|
||||||
|
%% | ---->------ | ---->------ | ---->------
|
||||||
|
%% | --/ / --/ / --/
|
||||||
|
%% | ---/ | ---/ | ---/
|
||||||
|
%% | -/ ^ -/ ^ -/
|
||||||
|
%% | ---->------ | ---->------ | ---->------
|
||||||
|
%% | --/ / --/ / --/
|
||||||
|
%% | -<-/ | -<-/ | -<-/
|
||||||
|
%% | -/ | -/ | -/
|
||||||
|
%% | ---->------| ---->------| ---------->
|
||||||
|
%% |
|
||||||
|
%% -+------------+-----------------------------> t
|
||||||
|
%% epoch
|
||||||
|
%%
|
||||||
|
%% This structure allows to quickly seek to a the first message that
|
||||||
|
%% was recorded in a certain epoch in a certain topic or a
|
||||||
|
%% group of topics matching filter like `foo/bar/#`.
|
||||||
|
%%
|
||||||
|
%% Due to its structure, for each pair of rocksdb keys K1 and K2, such
|
||||||
|
%% that K1 > K2 and topic(K1) = topic(K2), timestamp(K1) >
|
||||||
|
%% timestamp(K2).
|
||||||
|
%% That is, replay doesn't reorder messages published in each
|
||||||
|
%% individual topic.
|
||||||
|
%%
|
||||||
|
%% This property doesn't hold between different topics, but it's not deemed
|
||||||
|
%% a problem right now.
|
||||||
|
%%
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
%% API:
|
||||||
|
-export([
|
||||||
|
make_keymapper/1,
|
||||||
|
vector_to_key/2,
|
||||||
|
bin_vector_to_key/2,
|
||||||
|
key_to_vector/2,
|
||||||
|
bin_key_to_vector/2,
|
||||||
|
key_to_bitstring/2,
|
||||||
|
bitstring_to_key/2,
|
||||||
|
make_filter/2,
|
||||||
|
ratchet/2,
|
||||||
|
bin_increment/2,
|
||||||
|
bin_checkmask/2,
|
||||||
|
bitsize/1
|
||||||
|
]).
|
||||||
|
|
||||||
|
-export_type([vector/0, key/0, dimension/0, offset/0, bitsize/0, bitsource/0, keymapper/0]).
|
||||||
|
|
||||||
|
-compile(
|
||||||
|
{inline, [
|
||||||
|
ones/1,
|
||||||
|
extract/2,
|
||||||
|
extract_inv/2
|
||||||
|
]}
|
||||||
|
).
|
||||||
|
|
||||||
|
-elvis([{elvis_style, no_if_expression, disable}]).
|
||||||
|
|
||||||
|
-ifdef(TEST).
|
||||||
|
-include_lib("proper/include/proper.hrl").
|
||||||
|
-include_lib("eunit/include/eunit.hrl").
|
||||||
|
-endif.
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Type declarations
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-type scalar() :: integer().
|
||||||
|
|
||||||
|
-type vector() :: [scalar()].
|
||||||
|
|
||||||
|
%% N-th coordinate of a vector:
|
||||||
|
-type dimension() :: pos_integer().
|
||||||
|
|
||||||
|
-type offset() :: non_neg_integer().
|
||||||
|
|
||||||
|
-type bitsize() :: pos_integer().
|
||||||
|
|
||||||
|
%% The resulting 1D key:
|
||||||
|
-type key() :: non_neg_integer().
|
||||||
|
|
||||||
|
-type bitsource() ::
|
||||||
|
%% Consume `_Size` bits from timestamp starting at `_Offset`th
|
||||||
|
%% bit from N-th element of the input vector:
|
||||||
|
{dimension(), offset(), bitsize()}.
|
||||||
|
|
||||||
|
-record(scan_action, {
|
||||||
|
src_bitmask :: integer(),
|
||||||
|
src_offset :: offset(),
|
||||||
|
dst_offset :: offset()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-type scan_action() :: #scan_action{}.
|
||||||
|
|
||||||
|
-type scanner() :: [[scan_action()]].
|
||||||
|
|
||||||
|
-record(keymapper, {
|
||||||
|
schema :: [bitsource()],
|
||||||
|
scanner :: scanner(),
|
||||||
|
size :: non_neg_integer(),
|
||||||
|
dim_sizeof :: [non_neg_integer()]
|
||||||
|
}).
|
||||||
|
|
||||||
|
-opaque keymapper() :: #keymapper{}.
|
||||||
|
|
||||||
|
-type scalar_range() ::
|
||||||
|
any | {'=', scalar() | infinity} | {'>=', scalar()} | {scalar(), '..', scalar()}.
|
||||||
|
|
||||||
|
-include("emqx_ds_bitmask.hrl").
|
||||||
|
|
||||||
|
-type filter() :: #filter{}.
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% API functions
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
%% @doc Create a keymapper object that stores the "schema" of the
|
||||||
|
%% transformation from a list of bitsources.
|
||||||
|
%%
|
||||||
|
%% Note: Dimension is 1-based.
|
||||||
|
%%
|
||||||
|
%% Note: order of bitsources is important. First element of the list
|
||||||
|
%% is mapped to the _least_ significant bits of the key, and the last
|
||||||
|
%% element becomes most significant bits.
|
||||||
|
-spec make_keymapper([bitsource()]) -> keymapper().
|
||||||
|
make_keymapper(Bitsources) ->
|
||||||
|
Arr0 = array:new([{fixed, false}, {default, {0, []}}]),
|
||||||
|
{Size, Arr} = fold_bitsources(
|
||||||
|
fun(DestOffset, {Dim0, Offset, Size}, Acc) ->
|
||||||
|
Dim = Dim0 - 1,
|
||||||
|
Action = #scan_action{
|
||||||
|
src_bitmask = ones(Size), src_offset = Offset, dst_offset = DestOffset
|
||||||
|
},
|
||||||
|
{DimSizeof, Actions} = array:get(Dim, Acc),
|
||||||
|
array:set(Dim, {DimSizeof + Size, [Action | Actions]}, Acc)
|
||||||
|
end,
|
||||||
|
Arr0,
|
||||||
|
Bitsources
|
||||||
|
),
|
||||||
|
{DimSizeof, Scanner} = lists:unzip(array:to_list(Arr)),
|
||||||
|
#keymapper{
|
||||||
|
schema = Bitsources,
|
||||||
|
scanner = Scanner,
|
||||||
|
size = Size,
|
||||||
|
dim_sizeof = DimSizeof
|
||||||
|
}.
|
||||||
|
|
||||||
|
-spec bitsize(keymapper()) -> pos_integer().
|
||||||
|
bitsize(#keymapper{size = Size}) ->
|
||||||
|
Size.
|
||||||
|
|
||||||
|
%% @doc Map N-dimensional vector to a scalar key.
|
||||||
|
%%
|
||||||
|
%% Note: this function is not injective.
|
||||||
|
-spec vector_to_key(keymapper(), vector()) -> key().
|
||||||
|
vector_to_key(#keymapper{scanner = []}, []) ->
|
||||||
|
0;
|
||||||
|
vector_to_key(#keymapper{scanner = [Actions | Scanner]}, [Coord | Vector]) ->
|
||||||
|
do_vector_to_key(Actions, Scanner, Coord, Vector, 0).
|
||||||
|
|
||||||
|
%% @doc Same as `vector_to_key', but it works with binaries, and outputs a binary.
|
||||||
|
-spec bin_vector_to_key(keymapper(), [binary()]) -> binary().
|
||||||
|
bin_vector_to_key(Keymapper = #keymapper{dim_sizeof = DimSizeof, size = Size}, Binaries) ->
|
||||||
|
Vec = lists:zipwith(
|
||||||
|
fun(Bin, SizeOf) ->
|
||||||
|
<<Int:SizeOf>> = Bin,
|
||||||
|
Int
|
||||||
|
end,
|
||||||
|
Binaries,
|
||||||
|
DimSizeof
|
||||||
|
),
|
||||||
|
Key = vector_to_key(Keymapper, Vec),
|
||||||
|
<<Key:Size>>.
|
||||||
|
|
||||||
|
%% @doc Map key to a vector.
|
||||||
|
%%
|
||||||
|
%% Note: `vector_to_key(key_to_vector(K)) = K' but
|
||||||
|
%% `key_to_vector(vector_to_key(V)) = V' is not guaranteed.
|
||||||
|
-spec key_to_vector(keymapper(), key()) -> vector().
|
||||||
|
key_to_vector(#keymapper{scanner = Scanner}, Key) ->
|
||||||
|
lists:map(
|
||||||
|
fun(Actions) ->
|
||||||
|
lists:foldl(
|
||||||
|
fun(Action, Acc) ->
|
||||||
|
Acc bor extract_inv(Key, Action)
|
||||||
|
end,
|
||||||
|
0,
|
||||||
|
Actions
|
||||||
|
)
|
||||||
|
end,
|
||||||
|
Scanner
|
||||||
|
).
|
||||||
|
|
||||||
|
%% @doc Same as `key_to_vector', but it works with binaries.
|
||||||
|
-spec bin_key_to_vector(keymapper(), binary()) -> [binary()].
|
||||||
|
bin_key_to_vector(Keymapper = #keymapper{dim_sizeof = DimSizeof, size = Size}, BinKey) ->
|
||||||
|
<<Key:Size>> = BinKey,
|
||||||
|
Vector = key_to_vector(Keymapper, Key),
|
||||||
|
lists:zipwith(
|
||||||
|
fun(Elem, SizeOf) ->
|
||||||
|
<<Elem:SizeOf>>
|
||||||
|
end,
|
||||||
|
Vector,
|
||||||
|
DimSizeof
|
||||||
|
).
|
||||||
|
|
||||||
|
%% @doc Transform a bitstring to a key
|
||||||
|
-spec bitstring_to_key(keymapper(), bitstring()) -> key().
|
||||||
|
bitstring_to_key(#keymapper{size = Size}, Bin) ->
|
||||||
|
case Bin of
|
||||||
|
<<Key:Size>> ->
|
||||||
|
Key;
|
||||||
|
_ ->
|
||||||
|
error({invalid_key, Bin, Size})
|
||||||
|
end.
|
||||||
|
|
||||||
|
%% @doc Transform key to a fixed-size bistring
|
||||||
|
-spec key_to_bitstring(keymapper(), key()) -> bitstring().
|
||||||
|
key_to_bitstring(#keymapper{size = Size}, Key) ->
|
||||||
|
<<Key:Size>>.
|
||||||
|
|
||||||
|
%% @doc Create a filter object that facilitates range scans.
|
||||||
|
-spec make_filter(keymapper(), [scalar_range()]) -> filter().
|
||||||
|
make_filter(
|
||||||
|
KeyMapper = #keymapper{schema = Schema, dim_sizeof = DimSizeof, size = TotalSize}, Filter0
|
||||||
|
) ->
|
||||||
|
NDim = length(DimSizeof),
|
||||||
|
%% Transform "symbolic" constraints to ranges:
|
||||||
|
Filter1 = constraints_to_ranges(KeyMapper, Filter0),
|
||||||
|
{Bitmask, Bitfilter} = make_bitfilter(KeyMapper, Filter1),
|
||||||
|
%% Calculate maximum source offset as per bitsource specification:
|
||||||
|
MaxOffset = lists:foldl(
|
||||||
|
fun({Dim, Offset, _Size}, Acc) ->
|
||||||
|
maps:update_with(
|
||||||
|
Dim, fun(OldVal) -> max(OldVal, Offset) end, maps:merge(#{Dim => 0}, Acc)
|
||||||
|
)
|
||||||
|
end,
|
||||||
|
#{},
|
||||||
|
Schema
|
||||||
|
),
|
||||||
|
%% Adjust minimum and maximum values for each interval like this:
|
||||||
|
%%
|
||||||
|
%% Min: 110100|101011 -> 110100|00000
|
||||||
|
%% Max: 110101|001011 -> 110101|11111
|
||||||
|
%% ^
|
||||||
|
%% |
|
||||||
|
%% max offset
|
||||||
|
%%
|
||||||
|
%% This is needed so when we increment the vector, we always scan
|
||||||
|
%% the full range of least significant bits.
|
||||||
|
Filter2 = lists:zipwith(
|
||||||
|
fun
|
||||||
|
({Val, Val}, _Dim) ->
|
||||||
|
{Val, Val};
|
||||||
|
({Min0, Max0}, Dim) ->
|
||||||
|
Offset = maps:get(Dim, MaxOffset, 0),
|
||||||
|
%% Set least significant bits of Min to 0:
|
||||||
|
Min = (Min0 bsr Offset) bsl Offset,
|
||||||
|
%% Set least significant bits of Max to 1:
|
||||||
|
Max = Max0 bor ones(Offset),
|
||||||
|
{Min, Max}
|
||||||
|
end,
|
||||||
|
Filter1,
|
||||||
|
lists:seq(1, NDim)
|
||||||
|
),
|
||||||
|
%% Project the vector into "bitsource coordinate system":
|
||||||
|
{_, Filter} = fold_bitsources(
|
||||||
|
fun(DstOffset, {Dim, SrcOffset, Size}, Acc) ->
|
||||||
|
{Min0, Max0} = lists:nth(Dim, Filter2),
|
||||||
|
Min = (Min0 bsr SrcOffset) band ones(Size),
|
||||||
|
Max = (Max0 bsr SrcOffset) band ones(Size),
|
||||||
|
Action = #filter_scan_action{
|
||||||
|
offset = DstOffset,
|
||||||
|
size = Size,
|
||||||
|
min = Min,
|
||||||
|
max = Max
|
||||||
|
},
|
||||||
|
[Action | Acc]
|
||||||
|
end,
|
||||||
|
[],
|
||||||
|
Schema
|
||||||
|
),
|
||||||
|
Ranges = array:from_list(lists:reverse(Filter)),
|
||||||
|
%% Compute estimated upper and lower bounds of a _continous_
|
||||||
|
%% interval where all keys lie:
|
||||||
|
case Filter of
|
||||||
|
[] ->
|
||||||
|
RangeMin = 0,
|
||||||
|
RangeMax = 0;
|
||||||
|
[#filter_scan_action{offset = MSBOffset, min = MSBMin, max = MSBMax} | _] ->
|
||||||
|
RangeMin = MSBMin bsl MSBOffset,
|
||||||
|
RangeMax = MSBMax bsl MSBOffset bor ones(MSBOffset)
|
||||||
|
end,
|
||||||
|
%% Final value
|
||||||
|
#filter{
|
||||||
|
size = TotalSize,
|
||||||
|
bitmask = Bitmask,
|
||||||
|
bitfilter = Bitfilter,
|
||||||
|
bitsource_ranges = Ranges,
|
||||||
|
range_min = RangeMin,
|
||||||
|
range_max = RangeMax
|
||||||
|
}.
|
||||||
|
|
||||||
|
%% @doc Given a filter `F' and key `K0', return the smallest key `K'
|
||||||
|
%% that satisfies the following conditions:
|
||||||
|
%%
|
||||||
|
%% 1. `K >= K0'
|
||||||
|
%%
|
||||||
|
%% 2. `K' satisfies filter `F'.
|
||||||
|
%%
|
||||||
|
%% If these conditions cannot be satisfied, return `overflow'.
|
||||||
|
%%
|
||||||
|
%% Corollary: `K' may be equal to `K0'.
|
||||||
|
-spec ratchet(filter(), key()) -> key() | overflow.
|
||||||
|
ratchet(#filter{bitsource_ranges = Ranges, range_max = Max}, Key) when Key =< Max ->
|
||||||
|
%% This function works in two steps: first, it finds the position
|
||||||
|
%% of bitsource ("pivot point") corresponding to the part of the
|
||||||
|
%% key that should be incremented (or set to the _minimum_ value
|
||||||
|
%% of the range, in case the respective part of the original key
|
||||||
|
%% is less than the minimum). It also returns "increment": value
|
||||||
|
%% that should be added to the part of the key at the pivot point.
|
||||||
|
%% Increment can be 0 or 1.
|
||||||
|
%%
|
||||||
|
%% Then it transforms the key using the following operation:
|
||||||
|
%%
|
||||||
|
%% 1. Parts of the key that are less than the pivot point are
|
||||||
|
%% reset to their minimum values.
|
||||||
|
%%
|
||||||
|
%% 2. `Increment' is added to the part of the key at the pivot
|
||||||
|
%% point.
|
||||||
|
%%
|
||||||
|
%% 3. The rest of key stays the same
|
||||||
|
NDim = array:size(Ranges),
|
||||||
|
case ratchet_scan(Ranges, NDim, Key, 0, {_Pivot0 = -1, _Increment0 = 0}, _Carry = 0) of
|
||||||
|
overflow ->
|
||||||
|
overflow;
|
||||||
|
{Pivot, Increment} ->
|
||||||
|
ratchet_do(Ranges, Key, NDim - 1, Pivot, Increment)
|
||||||
|
end;
|
||||||
|
ratchet(_, _) ->
|
||||||
|
overflow.
|
||||||
|
|
||||||
|
%% @doc Given a binary representing a key and a filter, return the
|
||||||
|
%% next key matching the filter, or `overflow' if such key doesn't
|
||||||
|
%% exist.
|
||||||
|
-spec bin_increment(filter(), binary()) -> binary() | overflow.
|
||||||
|
bin_increment(Filter = #filter{size = Size}, <<>>) ->
|
||||||
|
Key = ratchet(Filter, 0),
|
||||||
|
<<Key:Size>>;
|
||||||
|
bin_increment(
|
||||||
|
Filter = #filter{size = Size, bitmask = Bitmask, bitfilter = Bitfilter, range_max = RangeMax},
|
||||||
|
KeyBin
|
||||||
|
) ->
|
||||||
|
%% The key may contain random suffix, skip it:
|
||||||
|
<<Key0:Size, _/binary>> = KeyBin,
|
||||||
|
Key1 = Key0 + 1,
|
||||||
|
if
|
||||||
|
Key1 band Bitmask =:= Bitfilter, Key1 =< RangeMax ->
|
||||||
|
<<Key1:Size>>;
|
||||||
|
true ->
|
||||||
|
case ratchet(Filter, Key1) of
|
||||||
|
overflow ->
|
||||||
|
overflow;
|
||||||
|
Key ->
|
||||||
|
<<Key:Size>>
|
||||||
|
end
|
||||||
|
end.
|
||||||
|
|
||||||
|
%% @doc Given a filter and a binary representation of a key, return
|
||||||
|
%% `false' if the key _doesn't_ match the fitler. This function
|
||||||
|
%% returning `true' is necessary, but not sufficient condition that
|
||||||
|
%% the key satisfies the filter.
|
||||||
|
-spec bin_checkmask(filter(), binary()) -> boolean().
|
||||||
|
bin_checkmask(#filter{size = Size, bitmask = Bitmask, bitfilter = Bitfilter}, Key) ->
|
||||||
|
case Key of
|
||||||
|
<<Int:Size>> ->
|
||||||
|
Int band Bitmask =:= Bitfilter;
|
||||||
|
_ ->
|
||||||
|
false
|
||||||
|
end.
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Internal functions
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
%% Note: this function operates in bitsource basis, scanning it from 0
|
||||||
|
%% to NDim (i.e. from the least significant bits to the most
|
||||||
|
%% significant bits)
|
||||||
|
ratchet_scan(_Ranges, NDim, _Key, NDim, Pivot, 0) ->
|
||||||
|
%% We've reached the end:
|
||||||
|
Pivot;
|
||||||
|
ratchet_scan(_Ranges, NDim, _Key, NDim, _Pivot, 1) ->
|
||||||
|
%% We've reached the end, but key is still not large enough:
|
||||||
|
overflow;
|
||||||
|
ratchet_scan(Ranges, NDim, Key, I, Pivot0, Carry) ->
|
||||||
|
#filter_scan_action{offset = Offset, size = Size, min = Min, max = Max} = array:get(I, Ranges),
|
||||||
|
%% Extract I-th element of the vector from the original key:
|
||||||
|
Elem = ((Key bsr Offset) band ones(Size)) + Carry,
|
||||||
|
if
|
||||||
|
Elem < Min ->
|
||||||
|
%% I-th coordinate is less than the specified minimum.
|
||||||
|
%%
|
||||||
|
%% We reset this coordinate to the minimum value. It means
|
||||||
|
%% we incremented this bitposition, the less significant
|
||||||
|
%% bits have to be reset to their respective minimum
|
||||||
|
%% values:
|
||||||
|
Pivot = {I + 1, 0},
|
||||||
|
ratchet_scan(Ranges, NDim, Key, I + 1, Pivot, 0);
|
||||||
|
Elem > Max ->
|
||||||
|
%% I-th coordinate is larger than the specified
|
||||||
|
%% minimum. We can only fix this problem by incrementing
|
||||||
|
%% the next coordinate (i.e. more significant bits).
|
||||||
|
%%
|
||||||
|
%% We reset this coordinate to the minimum value, and
|
||||||
|
%% increment the next coordinate (by setting `Carry' to
|
||||||
|
%% 1).
|
||||||
|
Pivot = {I + 1, 1},
|
||||||
|
ratchet_scan(Ranges, NDim, Key, I + 1, Pivot, 1);
|
||||||
|
true ->
|
||||||
|
%% Coordinate is within range:
|
||||||
|
ratchet_scan(Ranges, NDim, Key, I + 1, Pivot0, 0)
|
||||||
|
end.
|
||||||
|
|
||||||
|
%% Note: this function operates in bitsource basis, scanning it from
|
||||||
|
%% NDim to 0. It applies the transformation specified by
|
||||||
|
%% `ratchet_scan'.
|
||||||
|
ratchet_do(_Ranges, _Key, I, _Pivot, _Increment) when I < 0 ->
|
||||||
|
0;
|
||||||
|
ratchet_do(Ranges, Key, I, Pivot, Increment) ->
|
||||||
|
#filter_scan_action{offset = Offset, size = Size, min = Min} = array:get(I, Ranges),
|
||||||
|
Mask = ones(Offset + Size) bxor ones(Offset),
|
||||||
|
Elem =
|
||||||
|
if
|
||||||
|
I > Pivot ->
|
||||||
|
Mask band Key;
|
||||||
|
I =:= Pivot ->
|
||||||
|
(Mask band Key) + (Increment bsl Offset);
|
||||||
|
true ->
|
||||||
|
Min bsl Offset
|
||||||
|
end,
|
||||||
|
%% erlang:display(
|
||||||
|
%% {ratchet_do, I, integer_to_list(Key, 16), integer_to_list(Mask, 2),
|
||||||
|
%% integer_to_list(Elem, 16)}
|
||||||
|
%% ),
|
||||||
|
Elem bor ratchet_do(Ranges, Key, I - 1, Pivot, Increment).
|
||||||
|
|
||||||
|
-spec make_bitfilter(keymapper(), [{non_neg_integer(), non_neg_integer()}]) ->
|
||||||
|
{non_neg_integer(), non_neg_integer()}.
|
||||||
|
make_bitfilter(Keymapper = #keymapper{dim_sizeof = DimSizeof}, Ranges) ->
|
||||||
|
L = lists:zipwith(
|
||||||
|
fun
|
||||||
|
({N, N}, Bits) ->
|
||||||
|
%% For strict equality we can employ bitmask:
|
||||||
|
{ones(Bits), N};
|
||||||
|
(_, _) ->
|
||||||
|
{0, 0}
|
||||||
|
end,
|
||||||
|
Ranges,
|
||||||
|
DimSizeof
|
||||||
|
),
|
||||||
|
{Bitmask, Bitfilter} = lists:unzip(L),
|
||||||
|
{vector_to_key(Keymapper, Bitmask), vector_to_key(Keymapper, Bitfilter)}.
|
||||||
|
|
||||||
|
%% Transform constraints into a list of closed intervals that the
|
||||||
|
%% vector elements should lie in.
|
||||||
|
constraints_to_ranges(#keymapper{dim_sizeof = DimSizeof}, Filter) ->
|
||||||
|
lists:zipwith(
|
||||||
|
fun(Constraint, Bitsize) ->
|
||||||
|
Max = ones(Bitsize),
|
||||||
|
case Constraint of
|
||||||
|
any ->
|
||||||
|
{0, Max};
|
||||||
|
{'=', infinity} ->
|
||||||
|
{Max, Max};
|
||||||
|
{'=', Val} when Val =< Max ->
|
||||||
|
{Val, Val};
|
||||||
|
{'>=', Val} when Val =< Max ->
|
||||||
|
{Val, Max};
|
||||||
|
{A, '..', B} when A =< Max, B =< Max ->
|
||||||
|
{A, B}
|
||||||
|
end
|
||||||
|
end,
|
||||||
|
Filter,
|
||||||
|
DimSizeof
|
||||||
|
).
|
||||||
|
|
||||||
|
-spec fold_bitsources(fun((_DstOffset :: non_neg_integer(), bitsource(), Acc) -> Acc), Acc, [
|
||||||
|
bitsource()
|
||||||
|
]) -> {bitsize(), Acc}.
|
||||||
|
fold_bitsources(Fun, InitAcc, Bitsources) ->
|
||||||
|
lists:foldl(
|
||||||
|
fun(Bitsource = {_Dim, _Offset, Size}, {DstOffset, Acc0}) ->
|
||||||
|
Acc = Fun(DstOffset, Bitsource, Acc0),
|
||||||
|
{DstOffset + Size, Acc}
|
||||||
|
end,
|
||||||
|
{0, InitAcc},
|
||||||
|
Bitsources
|
||||||
|
).
|
||||||
|
|
||||||
|
do_vector_to_key([], [], _Coord, [], Acc) ->
|
||||||
|
Acc;
|
||||||
|
do_vector_to_key([], [NewActions | Scanner], _Coord, [NewCoord | Vector], Acc) ->
|
||||||
|
do_vector_to_key(NewActions, Scanner, NewCoord, Vector, Acc);
|
||||||
|
do_vector_to_key([Action | Actions], Scanner, Coord, Vector, Acc0) ->
|
||||||
|
Acc = Acc0 bor extract(Coord, Action),
|
||||||
|
do_vector_to_key(Actions, Scanner, Coord, Vector, Acc).
|
||||||
|
|
||||||
|
-spec extract(_Source :: scalar(), scan_action()) -> integer().
|
||||||
|
extract(Src, #scan_action{src_bitmask = SrcBitmask, src_offset = SrcOffset, dst_offset = DstOffset}) ->
|
||||||
|
((Src bsr SrcOffset) band SrcBitmask) bsl DstOffset.
|
||||||
|
|
||||||
|
%% extract^-1
|
||||||
|
-spec extract_inv(_Dest :: scalar(), scan_action()) -> integer().
|
||||||
|
extract_inv(Dest, #scan_action{
|
||||||
|
src_bitmask = SrcBitmask, src_offset = SrcOffset, dst_offset = DestOffset
|
||||||
|
}) ->
|
||||||
|
((Dest bsr DestOffset) band SrcBitmask) bsl SrcOffset.
|
||||||
|
|
||||||
|
ones(Bits) ->
|
||||||
|
1 bsl Bits - 1.
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Unit tests
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-ifdef(TEST).
|
||||||
|
|
||||||
|
make_keymapper0_test() ->
|
||||||
|
Schema = [],
|
||||||
|
?assertEqual(
|
||||||
|
#keymapper{
|
||||||
|
schema = Schema,
|
||||||
|
scanner = [],
|
||||||
|
size = 0,
|
||||||
|
dim_sizeof = []
|
||||||
|
},
|
||||||
|
make_keymapper(Schema)
|
||||||
|
).
|
||||||
|
|
||||||
|
make_keymapper1_test() ->
|
||||||
|
Schema = [{1, 0, 3}, {2, 0, 5}],
|
||||||
|
?assertEqual(
|
||||||
|
#keymapper{
|
||||||
|
schema = Schema,
|
||||||
|
scanner = [
|
||||||
|
[#scan_action{src_bitmask = 2#111, src_offset = 0, dst_offset = 0}],
|
||||||
|
[#scan_action{src_bitmask = 2#11111, src_offset = 0, dst_offset = 3}]
|
||||||
|
],
|
||||||
|
size = 8,
|
||||||
|
dim_sizeof = [3, 5]
|
||||||
|
},
|
||||||
|
make_keymapper(Schema)
|
||||||
|
).
|
||||||
|
|
||||||
|
make_keymapper2_test() ->
|
||||||
|
Schema = [{1, 0, 3}, {2, 0, 5}, {1, 3, 5}],
|
||||||
|
?assertEqual(
|
||||||
|
#keymapper{
|
||||||
|
schema = Schema,
|
||||||
|
scanner = [
|
||||||
|
[
|
||||||
|
#scan_action{src_bitmask = 2#11111, src_offset = 3, dst_offset = 8},
|
||||||
|
#scan_action{src_bitmask = 2#111, src_offset = 0, dst_offset = 0}
|
||||||
|
],
|
||||||
|
[#scan_action{src_bitmask = 2#11111, src_offset = 0, dst_offset = 3}]
|
||||||
|
],
|
||||||
|
size = 13,
|
||||||
|
dim_sizeof = [8, 5]
|
||||||
|
},
|
||||||
|
make_keymapper(Schema)
|
||||||
|
).
|
||||||
|
|
||||||
|
vector_to_key0_test() ->
|
||||||
|
Schema = [],
|
||||||
|
Vector = [],
|
||||||
|
?assertEqual(0, vec2key(Schema, Vector)).
|
||||||
|
|
||||||
|
vector_to_key1_test() ->
|
||||||
|
Schema = [{1, 0, 8}],
|
||||||
|
?assertEqual(16#ff, vec2key(Schema, [16#ff])),
|
||||||
|
?assertEqual(16#1a, vec2key(Schema, [16#1a])),
|
||||||
|
?assertEqual(16#ff, vec2key(Schema, [16#aaff])).
|
||||||
|
|
||||||
|
%% Test handling of source offset:
|
||||||
|
vector_to_key2_test() ->
|
||||||
|
Schema = [{1, 8, 8}],
|
||||||
|
?assertEqual(0, vec2key(Schema, [16#ff])),
|
||||||
|
?assertEqual(16#1a, vec2key(Schema, [16#1aff])),
|
||||||
|
?assertEqual(16#aa, vec2key(Schema, [16#11aaff])).
|
||||||
|
|
||||||
|
%% Basic test of 2D vector:
|
||||||
|
vector_to_key3_test() ->
|
||||||
|
Schema = [{1, 0, 8}, {2, 0, 8}],
|
||||||
|
?assertEqual(16#aaff, vec2key(Schema, [16#ff, 16#aa])),
|
||||||
|
?assertEqual(16#2211, vec2key(Schema, [16#aa11, 16#bb22])).
|
||||||
|
|
||||||
|
%% Advanced test with 2D vector:
|
||||||
|
vector_to_key4_test() ->
|
||||||
|
Schema = [{1, 0, 8}, {2, 0, 8}, {1, 8, 8}, {2, 16, 8}],
|
||||||
|
?assertEqual(16#bb112211, vec2key(Schema, [16#aa1111, 16#bb2222])).
|
||||||
|
|
||||||
|
%% Test with binaries:
|
||||||
|
vector_to_key_bin_test() ->
|
||||||
|
Schema = [{1, 0, 8 * 4}, {2, 0, 8 * 5}, {3, 0, 8 * 5}],
|
||||||
|
Keymapper = make_keymapper(lists:reverse(Schema)),
|
||||||
|
?assertMatch(
|
||||||
|
<<"wellhelloworld">>, bin_vector_to_key(Keymapper, [<<"well">>, <<"hello">>, <<"world">>])
|
||||||
|
).
|
||||||
|
|
||||||
|
key_to_vector0_test() ->
|
||||||
|
Schema = [],
|
||||||
|
key2vec(Schema, []).
|
||||||
|
|
||||||
|
key_to_vector1_test() ->
|
||||||
|
Schema = [{1, 0, 8}, {2, 0, 8}],
|
||||||
|
key2vec(Schema, [1, 1]),
|
||||||
|
key2vec(Schema, [255, 255]),
|
||||||
|
key2vec(Schema, [255, 1]),
|
||||||
|
key2vec(Schema, [0, 1]),
|
||||||
|
key2vec(Schema, [255, 0]).
|
||||||
|
|
||||||
|
key_to_vector2_test() ->
|
||||||
|
Schema = [{1, 0, 3}, {2, 0, 8}, {1, 3, 5}],
|
||||||
|
key2vec(Schema, [1, 1]),
|
||||||
|
key2vec(Schema, [255, 255]),
|
||||||
|
key2vec(Schema, [255, 1]),
|
||||||
|
key2vec(Schema, [0, 1]),
|
||||||
|
key2vec(Schema, [255, 0]).
|
||||||
|
|
||||||
|
make_bitmask0_test() ->
|
||||||
|
Keymapper = make_keymapper([]),
|
||||||
|
?assertMatch({0, 0}, mkbmask(Keymapper, [])).
|
||||||
|
|
||||||
|
make_bitmask1_test() ->
|
||||||
|
Keymapper = make_keymapper([{1, 0, 8}]),
|
||||||
|
?assertEqual({0, 0}, mkbmask(Keymapper, [any])),
|
||||||
|
?assertEqual({16#ff, 1}, mkbmask(Keymapper, [{'=', 1}])),
|
||||||
|
?assertEqual({16#ff, 255}, mkbmask(Keymapper, [{'=', 255}])),
|
||||||
|
?assertEqual({0, 0}, mkbmask(Keymapper, [{'>=', 0}])),
|
||||||
|
?assertEqual({0, 0}, mkbmask(Keymapper, [{'>=', 1}])),
|
||||||
|
?assertEqual({0, 0}, mkbmask(Keymapper, [{'>=', 16#f}])).
|
||||||
|
|
||||||
|
make_bitmask2_test() ->
|
||||||
|
Keymapper = make_keymapper([{1, 0, 3}, {2, 0, 4}, {3, 0, 2}]),
|
||||||
|
?assertEqual({2#00_0000_000, 2#00_0000_000}, mkbmask(Keymapper, [any, any, any])),
|
||||||
|
?assertEqual({2#11_0000_000, 2#00_0000_000}, mkbmask(Keymapper, [any, any, {'=', 0}])),
|
||||||
|
?assertEqual({2#00_1111_000, 2#00_0000_000}, mkbmask(Keymapper, [any, {'=', 0}, any])),
|
||||||
|
?assertEqual({2#00_0000_111, 2#00_0000_000}, mkbmask(Keymapper, [{'=', 0}, any, any])).
|
||||||
|
|
||||||
|
make_bitmask3_test() ->
|
||||||
|
%% Key format of type |TimeOffset|Topic|Epoch|:
|
||||||
|
Keymapper = make_keymapper([{1, 0, 8}, {2, 0, 8}, {1, 8, 8}]),
|
||||||
|
?assertEqual({2#00000000_00000000_00000000, 16#00_00_00}, mkbmask(Keymapper, [any, any])),
|
||||||
|
?assertEqual(
|
||||||
|
{2#11111111_11111111_11111111, 16#aa_cc_bb},
|
||||||
|
mkbmask(Keymapper, [{'=', 16#aabb}, {'=', 16#cc}])
|
||||||
|
),
|
||||||
|
?assertEqual(
|
||||||
|
{2#00000000_11111111_00000000, 16#00_bb_00}, mkbmask(Keymapper, [{'>=', 255}, {'=', 16#bb}])
|
||||||
|
).
|
||||||
|
|
||||||
|
make_filter_test() ->
|
||||||
|
KeyMapper = make_keymapper([]),
|
||||||
|
Filter = [],
|
||||||
|
?assertMatch(#filter{size = 0, bitmask = 0, bitfilter = 0}, make_filter(KeyMapper, Filter)).
|
||||||
|
|
||||||
|
ratchet1_test() ->
|
||||||
|
Bitsources = [{1, 0, 8}],
|
||||||
|
M = make_keymapper(Bitsources),
|
||||||
|
F = make_filter(M, [any]),
|
||||||
|
#filter{bitsource_ranges = Rarr} = F,
|
||||||
|
?assertMatch(
|
||||||
|
[
|
||||||
|
#filter_scan_action{
|
||||||
|
offset = 0,
|
||||||
|
size = 8,
|
||||||
|
min = 0,
|
||||||
|
max = 16#ff
|
||||||
|
}
|
||||||
|
],
|
||||||
|
array:to_list(Rarr)
|
||||||
|
),
|
||||||
|
?assertEqual(0, ratchet(F, 0)),
|
||||||
|
?assertEqual(16#fa, ratchet(F, 16#fa)),
|
||||||
|
?assertEqual(16#ff, ratchet(F, 16#ff)),
|
||||||
|
?assertEqual(overflow, ratchet(F, 16#100)).
|
||||||
|
|
||||||
|
%% erlfmt-ignore
|
||||||
|
ratchet2_test() ->
|
||||||
|
Bitsources = [{1, 0, 8}, %% Static topic index
|
||||||
|
{2, 8, 8}, %% Epoch
|
||||||
|
{3, 0, 8}, %% Varying topic hash
|
||||||
|
{2, 0, 8}], %% Timestamp offset
|
||||||
|
M = make_keymapper(lists:reverse(Bitsources)),
|
||||||
|
F1 = make_filter(M, [{'=', 16#aa}, any, {'=', 16#cc}]),
|
||||||
|
?assertEqual(16#aa00cc00, ratchet(F1, 0)),
|
||||||
|
?assertEqual(16#aa01cc00, ratchet(F1, 16#aa00cd00)),
|
||||||
|
?assertEqual(16#aa01cc11, ratchet(F1, 16#aa01cc11)),
|
||||||
|
?assertEqual(16#aa11cc00, ratchet(F1, 16#aa10cd00)),
|
||||||
|
?assertEqual(16#aa11cc00, ratchet(F1, 16#aa10dc11)),
|
||||||
|
?assertEqual(overflow, ratchet(F1, 16#ab000000)),
|
||||||
|
F2 = make_filter(M, [{'=', 16#aa}, {'>=', 16#dddd}, {'=', 16#cc}]),
|
||||||
|
%% TODO: note that it's `16#aaddcc00` instead of
|
||||||
|
%% `16#aaddccdd'. That is because currently ratchet function
|
||||||
|
%% doesn't take LSBs of an '>=' interval if it has a hole in the
|
||||||
|
%% middle (see `make_filter/2'). This only adds extra keys to the
|
||||||
|
%% very first interval, so it's not deemed a huge problem.
|
||||||
|
?assertEqual(16#aaddcc00, ratchet(F2, 0)),
|
||||||
|
?assertEqual(16#aa_de_cc_00, ratchet(F2, 16#aa_dd_cd_11)).
|
||||||
|
|
||||||
|
%% erlfmt-ignore
|
||||||
|
ratchet3_test_() ->
|
||||||
|
EpochBits = 4,
|
||||||
|
Bitsources = [{1, 0, 2}, %% Static topic index
|
||||||
|
{2, EpochBits, 4}, %% Epoch
|
||||||
|
{3, 0, 2}, %% Varying topic hash
|
||||||
|
{2, 0, EpochBits}], %% Timestamp offset
|
||||||
|
Keymapper = make_keymapper(lists:reverse(Bitsources)),
|
||||||
|
Filter1 = make_filter(Keymapper, [{'=', 2#10}, any, {'=', 2#01}]),
|
||||||
|
Filter2 = make_filter(Keymapper, [{'=', 2#01}, any, any]),
|
||||||
|
Filter3 = make_filter(Keymapper, [{'=', 2#01}, {'>=', 16#aa}, any]),
|
||||||
|
{timeout, 15,
|
||||||
|
[?_assert(test_iterate(Filter1, 0)),
|
||||||
|
?_assert(test_iterate(Filter2, 0)),
|
||||||
|
%% Not starting from 0 here for simplicity, since the beginning
|
||||||
|
%% of a >= interval can't be properly checked with a bitmask:
|
||||||
|
?_assert(test_iterate(Filter3, ratchet(Filter3, 1)))
|
||||||
|
]}.
|
||||||
|
|
||||||
|
%% Note: this function iterates through the full range of keys, so its
|
||||||
|
%% complexity grows _exponentially_ with the total size of the
|
||||||
|
%% keymapper.
|
||||||
|
test_iterate(_Filter, overflow) ->
|
||||||
|
true;
|
||||||
|
test_iterate(Filter, Key0) ->
|
||||||
|
Key = ratchet(Filter, Key0 + 1),
|
||||||
|
?assert(ratchet_prop(Filter, Key0, Key)),
|
||||||
|
test_iterate(Filter, Key).
|
||||||
|
|
||||||
|
ratchet_prop(#filter{bitfilter = Bitfilter, bitmask = Bitmask, size = Size}, Key0, Key) ->
|
||||||
|
%% Validate basic properties of the generated key. It must be
|
||||||
|
%% greater than the old key, and match the bitmask:
|
||||||
|
?assert(Key =:= overflow orelse (Key band Bitmask =:= Bitfilter)),
|
||||||
|
?assert(Key > Key0, {Key, '>=', Key0}),
|
||||||
|
IMax = ones(Size),
|
||||||
|
%% Iterate through all keys between `Key0 + 1' and `Key' and
|
||||||
|
%% validate that none of them match the bitmask. Ultimately, it
|
||||||
|
%% means that `ratchet' function doesn't skip over any valid keys:
|
||||||
|
CheckGaps = fun
|
||||||
|
F(I) when I >= Key; I > IMax ->
|
||||||
|
true;
|
||||||
|
F(I) ->
|
||||||
|
?assertNot(
|
||||||
|
I band Bitmask =:= Bitfilter,
|
||||||
|
{found_gap, Key0, I, Key}
|
||||||
|
),
|
||||||
|
F(I + 1)
|
||||||
|
end,
|
||||||
|
CheckGaps(Key0 + 1).
|
||||||
|
|
||||||
|
mkbmask(Keymapper, Filter0) ->
|
||||||
|
Filter = constraints_to_ranges(Keymapper, Filter0),
|
||||||
|
make_bitfilter(Keymapper, Filter).
|
||||||
|
|
||||||
|
key2vec(Schema, Vector) ->
|
||||||
|
Keymapper = make_keymapper(Schema),
|
||||||
|
Key = vector_to_key(Keymapper, Vector),
|
||||||
|
?assertEqual(Vector, key_to_vector(Keymapper, Key)).
|
||||||
|
|
||||||
|
vec2key(Schema, Vector) ->
|
||||||
|
vector_to_key(make_keymapper(Schema), Vector).
|
||||||
|
|
||||||
|
-endif.
|
|
@ -0,0 +1,619 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%
|
||||||
|
%% Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
%% you may not use this file except in compliance with the License.
|
||||||
|
%% You may obtain a copy of the License at
|
||||||
|
%%
|
||||||
|
%% http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
%%
|
||||||
|
%% Unless required by applicable law or agreed to in writing, software
|
||||||
|
%% distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
%% See the License for the specific language governing permissions and
|
||||||
|
%% limitations under the License.
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
|
||||||
|
-module(emqx_ds_lts).
|
||||||
|
|
||||||
|
%% API:
|
||||||
|
-export([
|
||||||
|
trie_create/1, trie_create/0, trie_restore/2, topic_key/3, match_topics/2, lookup_topic_key/2
|
||||||
|
]).
|
||||||
|
|
||||||
|
%% Debug:
|
||||||
|
-export([trie_next/3, trie_insert/3, dump_to_dot/2]).
|
||||||
|
|
||||||
|
-export_type([options/0, static_key/0, trie/0]).
|
||||||
|
|
||||||
|
-include_lib("stdlib/include/ms_transform.hrl").
|
||||||
|
|
||||||
|
-ifdef(TEST).
|
||||||
|
-include_lib("eunit/include/eunit.hrl").
|
||||||
|
-endif.
|
||||||
|
|
||||||
|
-elvis([{elvis_style, variable_naming_convention, disable}]).
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Type declarations
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
%% End Of Topic
|
||||||
|
-define(EOT, []).
|
||||||
|
-define(PLUS, '+').
|
||||||
|
|
||||||
|
-type edge() :: binary() | ?EOT | ?PLUS.
|
||||||
|
|
||||||
|
%% Fixed size binary
|
||||||
|
-type static_key() :: non_neg_integer().
|
||||||
|
|
||||||
|
-define(PREFIX, prefix).
|
||||||
|
-type state() :: static_key() | ?PREFIX.
|
||||||
|
|
||||||
|
-type varying() :: [binary() | ?PLUS].
|
||||||
|
|
||||||
|
-type msg_storage_key() :: {static_key(), varying()}.
|
||||||
|
|
||||||
|
-type threshold_fun() :: fun((non_neg_integer()) -> non_neg_integer()).
|
||||||
|
|
||||||
|
-type persist_callback() :: fun((_Key, _Val) -> ok).
|
||||||
|
|
||||||
|
-type options() ::
|
||||||
|
#{
|
||||||
|
persist_callback => persist_callback(),
|
||||||
|
static_key_size => pos_integer()
|
||||||
|
}.
|
||||||
|
|
||||||
|
-record(trie, {
|
||||||
|
persist :: persist_callback(),
|
||||||
|
static_key_size :: pos_integer(),
|
||||||
|
trie :: ets:tid(),
|
||||||
|
stats :: ets:tid()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-opaque trie() :: #trie{}.
|
||||||
|
|
||||||
|
-record(trans, {
|
||||||
|
key :: {state(), edge()},
|
||||||
|
next :: state()
|
||||||
|
}).
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% API funcions
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
%% @doc Create an empty trie
|
||||||
|
-spec trie_create(options()) -> trie().
|
||||||
|
trie_create(UserOpts) ->
|
||||||
|
Defaults = #{
|
||||||
|
persist_callback => fun(_, _) -> ok end,
|
||||||
|
static_key_size => 8
|
||||||
|
},
|
||||||
|
#{
|
||||||
|
persist_callback := Persist,
|
||||||
|
static_key_size := StaticKeySize
|
||||||
|
} = maps:merge(Defaults, UserOpts),
|
||||||
|
Trie = ets:new(trie, [{keypos, #trans.key}, set, public]),
|
||||||
|
Stats = ets:new(stats, [{keypos, 1}, set, public]),
|
||||||
|
#trie{
|
||||||
|
persist = Persist,
|
||||||
|
static_key_size = StaticKeySize,
|
||||||
|
trie = Trie,
|
||||||
|
stats = Stats
|
||||||
|
}.
|
||||||
|
|
||||||
|
-spec trie_create() -> trie().
|
||||||
|
trie_create() ->
|
||||||
|
trie_create(#{}).
|
||||||
|
|
||||||
|
%% @doc Restore trie from a dump
|
||||||
|
-spec trie_restore(options(), [{_Key, _Val}]) -> trie().
|
||||||
|
trie_restore(Options, Dump) ->
|
||||||
|
Trie = trie_create(Options),
|
||||||
|
lists:foreach(
|
||||||
|
fun({{StateFrom, Token}, StateTo}) ->
|
||||||
|
trie_insert(Trie, StateFrom, Token, StateTo)
|
||||||
|
end,
|
||||||
|
Dump
|
||||||
|
),
|
||||||
|
Trie.
|
||||||
|
|
||||||
|
%% @doc Lookup the topic key. Create a new one, if not found.
|
||||||
|
-spec topic_key(trie(), threshold_fun(), [binary()]) -> msg_storage_key().
|
||||||
|
topic_key(Trie, ThresholdFun, Tokens) ->
|
||||||
|
do_topic_key(Trie, ThresholdFun, 0, ?PREFIX, Tokens, []).
|
||||||
|
|
||||||
|
%% @doc Return an exisiting topic key if it exists.
|
||||||
|
-spec lookup_topic_key(trie(), [binary()]) -> {ok, msg_storage_key()} | undefined.
|
||||||
|
lookup_topic_key(Trie, Tokens) ->
|
||||||
|
do_lookup_topic_key(Trie, ?PREFIX, Tokens, []).
|
||||||
|
|
||||||
|
%% @doc Return list of keys of topics that match a given topic filter
|
||||||
|
-spec match_topics(trie(), [binary() | '+' | '#']) ->
|
||||||
|
[msg_storage_key()].
|
||||||
|
match_topics(Trie, TopicFilter) ->
|
||||||
|
do_match_topics(Trie, ?PREFIX, [], TopicFilter).
|
||||||
|
|
||||||
|
%% @doc Dump trie to graphviz format for debugging
|
||||||
|
-spec dump_to_dot(trie(), file:filename()) -> ok.
|
||||||
|
dump_to_dot(#trie{trie = Trie, stats = Stats}, Filename) ->
|
||||||
|
L = ets:tab2list(Trie),
|
||||||
|
{Nodes0, Edges} =
|
||||||
|
lists:foldl(
|
||||||
|
fun(#trans{key = {From, Label}, next = To}, {AccN, AccEdge}) ->
|
||||||
|
Edge = {From, To, Label},
|
||||||
|
{[From, To] ++ AccN, [Edge | AccEdge]}
|
||||||
|
end,
|
||||||
|
{[], []},
|
||||||
|
L
|
||||||
|
),
|
||||||
|
Nodes =
|
||||||
|
lists:map(
|
||||||
|
fun(Node) ->
|
||||||
|
case ets:lookup(Stats, Node) of
|
||||||
|
[{_, NChildren}] -> ok;
|
||||||
|
[] -> NChildren = 0
|
||||||
|
end,
|
||||||
|
{Node, NChildren}
|
||||||
|
end,
|
||||||
|
lists:usort(Nodes0)
|
||||||
|
),
|
||||||
|
{ok, FD} = file:open(Filename, [write]),
|
||||||
|
Print = fun
|
||||||
|
(?PREFIX) -> "prefix";
|
||||||
|
(NodeId) -> integer_to_binary(NodeId, 16)
|
||||||
|
end,
|
||||||
|
io:format(FD, "digraph {~n", []),
|
||||||
|
lists:foreach(
|
||||||
|
fun({Node, NChildren}) ->
|
||||||
|
Id = Print(Node),
|
||||||
|
io:format(FD, " \"~s\" [label=\"~s : ~p\"];~n", [Id, Id, NChildren])
|
||||||
|
end,
|
||||||
|
Nodes
|
||||||
|
),
|
||||||
|
lists:foreach(
|
||||||
|
fun({From, To, Label}) ->
|
||||||
|
io:format(FD, " \"~s\" -> \"~s\" [label=\"~s\"];~n", [Print(From), Print(To), Label])
|
||||||
|
end,
|
||||||
|
Edges
|
||||||
|
),
|
||||||
|
io:format(FD, "}~n", []),
|
||||||
|
file:close(FD).
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Internal exports
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-spec trie_next(trie(), state(), binary() | ?EOT) -> {Wildcard, state()} | undefined when
|
||||||
|
Wildcard :: boolean().
|
||||||
|
trie_next(#trie{trie = Trie}, State, ?EOT) ->
|
||||||
|
case ets:lookup(Trie, {State, ?EOT}) of
|
||||||
|
[#trans{next = Next}] -> {false, Next};
|
||||||
|
[] -> undefined
|
||||||
|
end;
|
||||||
|
trie_next(#trie{trie = Trie}, State, Token) ->
|
||||||
|
case ets:lookup(Trie, {State, Token}) of
|
||||||
|
[#trans{next = Next}] ->
|
||||||
|
{false, Next};
|
||||||
|
[] ->
|
||||||
|
case ets:lookup(Trie, {State, ?PLUS}) of
|
||||||
|
[#trans{next = Next}] -> {true, Next};
|
||||||
|
[] -> undefined
|
||||||
|
end
|
||||||
|
end.
|
||||||
|
|
||||||
|
-spec trie_insert(trie(), state(), edge()) -> {Updated, state()} when
|
||||||
|
NChildren :: non_neg_integer(),
|
||||||
|
Updated :: false | NChildren.
|
||||||
|
trie_insert(Trie, State, Token) ->
|
||||||
|
trie_insert(Trie, State, Token, get_id_for_key(Trie, State, Token)).
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Internal functions
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-spec trie_insert(trie(), state(), edge(), state()) -> {Updated, state()} when
|
||||||
|
NChildren :: non_neg_integer(),
|
||||||
|
Updated :: false | NChildren.
|
||||||
|
trie_insert(#trie{trie = Trie, stats = Stats, persist = Persist}, State, Token, NewState) ->
|
||||||
|
Key = {State, Token},
|
||||||
|
Rec = #trans{
|
||||||
|
key = Key,
|
||||||
|
next = NewState
|
||||||
|
},
|
||||||
|
case ets:insert_new(Trie, Rec) of
|
||||||
|
true ->
|
||||||
|
ok = Persist(Key, NewState),
|
||||||
|
Inc =
|
||||||
|
case Token of
|
||||||
|
?EOT -> 0;
|
||||||
|
?PLUS -> 0;
|
||||||
|
_ -> 1
|
||||||
|
end,
|
||||||
|
NChildren = ets:update_counter(Stats, State, {2, Inc}, {State, 0}),
|
||||||
|
{NChildren, NewState};
|
||||||
|
false ->
|
||||||
|
[#trans{next = NextState}] = ets:lookup(Trie, Key),
|
||||||
|
{false, NextState}
|
||||||
|
end.
|
||||||
|
|
||||||
|
-spec get_id_for_key(trie(), state(), edge()) -> static_key().
|
||||||
|
get_id_for_key(#trie{static_key_size = Size}, _State, _Token) ->
|
||||||
|
%% Requirements for the return value:
|
||||||
|
%%
|
||||||
|
%% It should be globally unique for the `{State, Token}` pair. Other
|
||||||
|
%% than that, there's no requirements. The return value doesn't even
|
||||||
|
%% have to be deterministic, since the states are saved in the trie.
|
||||||
|
%%
|
||||||
|
%% The generated value becomes the ID of the topic in the durable
|
||||||
|
%% storage. Its size should be relatively small to reduce the
|
||||||
|
%% overhead of storing messages.
|
||||||
|
%%
|
||||||
|
%% If we want to impress computer science crowd, sorry, I mean to
|
||||||
|
%% minimize storage requirements, we can even employ Huffman coding
|
||||||
|
%% based on the frequency of messages.
|
||||||
|
<<Int:(Size * 8)>> = crypto:strong_rand_bytes(Size),
|
||||||
|
Int.
|
||||||
|
|
||||||
|
%% erlfmt-ignore
|
||||||
|
-spec do_match_topics(trie(), state(), [binary() | '+'], [binary() | '+' | '#']) ->
|
||||||
|
list().
|
||||||
|
do_match_topics(Trie, State, Varying, []) ->
|
||||||
|
case trie_next(Trie, State, ?EOT) of
|
||||||
|
{false, Static} -> [{Static, lists:reverse(Varying)}];
|
||||||
|
undefined -> []
|
||||||
|
end;
|
||||||
|
do_match_topics(Trie, State, Varying, ['#']) ->
|
||||||
|
Emanating = emanating(Trie, State, ?PLUS),
|
||||||
|
lists:flatmap(
|
||||||
|
fun
|
||||||
|
({?EOT, Static}) ->
|
||||||
|
[{Static, lists:reverse(Varying)}];
|
||||||
|
({?PLUS, NextState}) ->
|
||||||
|
do_match_topics(Trie, NextState, [?PLUS | Varying], ['#']);
|
||||||
|
({_, NextState}) ->
|
||||||
|
do_match_topics(Trie, NextState, Varying, ['#'])
|
||||||
|
end,
|
||||||
|
Emanating
|
||||||
|
);
|
||||||
|
do_match_topics(Trie, State, Varying, [Level | Rest]) ->
|
||||||
|
Emanating = emanating(Trie, State, Level),
|
||||||
|
lists:flatmap(
|
||||||
|
fun
|
||||||
|
({?EOT, _NextState}) ->
|
||||||
|
[];
|
||||||
|
({?PLUS, NextState}) ->
|
||||||
|
do_match_topics(Trie, NextState, [Level | Varying], Rest);
|
||||||
|
({_, NextState}) ->
|
||||||
|
do_match_topics(Trie, NextState, Varying, Rest)
|
||||||
|
end,
|
||||||
|
Emanating
|
||||||
|
).
|
||||||
|
|
||||||
|
-spec do_lookup_topic_key(trie(), state(), [binary()], [binary()]) ->
|
||||||
|
{ok, msg_storage_key()} | undefined.
|
||||||
|
do_lookup_topic_key(Trie, State, [], Varying) ->
|
||||||
|
case trie_next(Trie, State, ?EOT) of
|
||||||
|
{false, Static} ->
|
||||||
|
{ok, {Static, lists:reverse(Varying)}};
|
||||||
|
undefined ->
|
||||||
|
undefined
|
||||||
|
end;
|
||||||
|
do_lookup_topic_key(Trie, State, [Tok | Rest], Varying) ->
|
||||||
|
case trie_next(Trie, State, Tok) of
|
||||||
|
{true, NextState} ->
|
||||||
|
do_lookup_topic_key(Trie, NextState, Rest, [Tok | Varying]);
|
||||||
|
{false, NextState} ->
|
||||||
|
do_lookup_topic_key(Trie, NextState, Rest, Varying);
|
||||||
|
undefined ->
|
||||||
|
undefined
|
||||||
|
end.
|
||||||
|
|
||||||
|
do_topic_key(Trie, _, _, State, [], Varying) ->
|
||||||
|
%% We reached the end of topic. Assert: Trie node that corresponds
|
||||||
|
%% to EOT cannot be a wildcard.
|
||||||
|
{_, false, Static} = trie_next_(Trie, State, ?EOT),
|
||||||
|
{Static, lists:reverse(Varying)};
|
||||||
|
do_topic_key(Trie, ThresholdFun, Depth, State, [Tok | Rest], Varying0) ->
|
||||||
|
% TODO: it's not necessary to call it every time.
|
||||||
|
Threshold = ThresholdFun(Depth),
|
||||||
|
Varying =
|
||||||
|
case trie_next_(Trie, State, Tok) of
|
||||||
|
{NChildren, _, NextState} when is_integer(NChildren), NChildren >= Threshold ->
|
||||||
|
%% Number of children for the trie node reached the
|
||||||
|
%% threshold, we need to insert wildcard here.
|
||||||
|
{_, _WildcardState} = trie_insert(Trie, State, ?PLUS),
|
||||||
|
Varying0;
|
||||||
|
{_, false, NextState} ->
|
||||||
|
Varying0;
|
||||||
|
{_, true, NextState} ->
|
||||||
|
%% This topic level is marked as wildcard in the trie,
|
||||||
|
%% we need to add it to the varying part of the key:
|
||||||
|
[Tok | Varying0]
|
||||||
|
end,
|
||||||
|
do_topic_key(Trie, ThresholdFun, Depth + 1, NextState, Rest, Varying).
|
||||||
|
|
||||||
|
%% @doc Has side effects! Inserts missing elements
|
||||||
|
-spec trie_next_(trie(), state(), binary() | ?EOT) -> {New, Wildcard, state()} when
|
||||||
|
New :: false | non_neg_integer(),
|
||||||
|
Wildcard :: boolean().
|
||||||
|
trie_next_(Trie, State, Token) ->
|
||||||
|
case trie_next(Trie, State, Token) of
|
||||||
|
{Wildcard, NextState} ->
|
||||||
|
{false, Wildcard, NextState};
|
||||||
|
undefined ->
|
||||||
|
{Updated, NextState} = trie_insert(Trie, State, Token),
|
||||||
|
{Updated, false, NextState}
|
||||||
|
end.
|
||||||
|
|
||||||
|
%% @doc Return all edges emanating from a node:
|
||||||
|
%% erlfmt-ignore
|
||||||
|
-spec emanating(trie(), state(), edge()) -> [{edge(), state()}].
|
||||||
|
emanating(#trie{trie = Tab}, State, ?PLUS) ->
|
||||||
|
ets:select(
|
||||||
|
Tab,
|
||||||
|
ets:fun2ms(
|
||||||
|
fun(#trans{key = {S, Edge}, next = Next}) when S == State ->
|
||||||
|
{Edge, Next}
|
||||||
|
end
|
||||||
|
)
|
||||||
|
);
|
||||||
|
emanating(#trie{trie = Tab}, State, ?EOT) ->
|
||||||
|
case ets:lookup(Tab, {State, ?EOT}) of
|
||||||
|
[#trans{next = Next}] -> [{?EOT, Next}];
|
||||||
|
[] -> []
|
||||||
|
end;
|
||||||
|
emanating(#trie{trie = Tab}, State, Bin) when is_binary(Bin) ->
|
||||||
|
[
|
||||||
|
{Edge, Next}
|
||||||
|
|| #trans{key = {_, Edge}, next = Next} <-
|
||||||
|
ets:lookup(Tab, {State, ?PLUS}) ++
|
||||||
|
ets:lookup(Tab, {State, Bin})
|
||||||
|
].
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Tests
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-ifdef(TEST).
|
||||||
|
|
||||||
|
trie_basic_test() ->
|
||||||
|
T = trie_create(),
|
||||||
|
?assertMatch(undefined, trie_next(T, ?PREFIX, <<"foo">>)),
|
||||||
|
{1, S1} = trie_insert(T, ?PREFIX, <<"foo">>),
|
||||||
|
?assertMatch({false, S1}, trie_insert(T, ?PREFIX, <<"foo">>)),
|
||||||
|
?assertMatch({false, S1}, trie_next(T, ?PREFIX, <<"foo">>)),
|
||||||
|
|
||||||
|
?assertMatch(undefined, trie_next(T, ?PREFIX, <<"bar">>)),
|
||||||
|
{2, S2} = trie_insert(T, ?PREFIX, <<"bar">>),
|
||||||
|
?assertMatch({false, S2}, trie_insert(T, ?PREFIX, <<"bar">>)),
|
||||||
|
|
||||||
|
?assertMatch(undefined, trie_next(T, S1, <<"foo">>)),
|
||||||
|
?assertMatch(undefined, trie_next(T, S1, <<"bar">>)),
|
||||||
|
{1, S11} = trie_insert(T, S1, <<"foo">>),
|
||||||
|
{2, S12} = trie_insert(T, S1, <<"bar">>),
|
||||||
|
?assertMatch({false, S11}, trie_next(T, S1, <<"foo">>)),
|
||||||
|
?assertMatch({false, S12}, trie_next(T, S1, <<"bar">>)),
|
||||||
|
|
||||||
|
?assertMatch(undefined, trie_next(T, S11, <<"bar">>)),
|
||||||
|
{1, S111} = trie_insert(T, S11, <<"bar">>),
|
||||||
|
?assertMatch({false, S111}, trie_next(T, S11, <<"bar">>)).
|
||||||
|
|
||||||
|
lookup_key_test() ->
|
||||||
|
T = trie_create(),
|
||||||
|
{_, S1} = trie_insert(T, ?PREFIX, <<"foo">>),
|
||||||
|
{_, S11} = trie_insert(T, S1, <<"foo">>),
|
||||||
|
%% Topics don't match until we insert ?EOT:
|
||||||
|
?assertMatch(
|
||||||
|
undefined,
|
||||||
|
lookup_topic_key(T, [<<"foo">>])
|
||||||
|
),
|
||||||
|
?assertMatch(
|
||||||
|
undefined,
|
||||||
|
lookup_topic_key(T, [<<"foo">>, <<"foo">>])
|
||||||
|
),
|
||||||
|
{_, S10} = trie_insert(T, S1, ?EOT),
|
||||||
|
{_, S110} = trie_insert(T, S11, ?EOT),
|
||||||
|
?assertMatch(
|
||||||
|
{ok, {S10, []}},
|
||||||
|
lookup_topic_key(T, [<<"foo">>])
|
||||||
|
),
|
||||||
|
?assertMatch(
|
||||||
|
{ok, {S110, []}},
|
||||||
|
lookup_topic_key(T, [<<"foo">>, <<"foo">>])
|
||||||
|
),
|
||||||
|
%% The rest of keys still don't match:
|
||||||
|
?assertMatch(
|
||||||
|
undefined,
|
||||||
|
lookup_topic_key(T, [<<"bar">>])
|
||||||
|
),
|
||||||
|
?assertMatch(
|
||||||
|
undefined,
|
||||||
|
lookup_topic_key(T, [<<"bar">>, <<"foo">>])
|
||||||
|
).
|
||||||
|
|
||||||
|
wildcard_lookup_test() ->
|
||||||
|
T = trie_create(),
|
||||||
|
{1, S1} = trie_insert(T, ?PREFIX, <<"foo">>),
|
||||||
|
%% Plus doesn't increase the number of children
|
||||||
|
{0, S11} = trie_insert(T, S1, ?PLUS),
|
||||||
|
{1, S111} = trie_insert(T, S11, <<"foo">>),
|
||||||
|
%% ?EOT doesn't increase the number of children
|
||||||
|
{0, S1110} = trie_insert(T, S111, ?EOT),
|
||||||
|
?assertMatch(
|
||||||
|
{ok, {S1110, [<<"bar">>]}},
|
||||||
|
lookup_topic_key(T, [<<"foo">>, <<"bar">>, <<"foo">>])
|
||||||
|
),
|
||||||
|
?assertMatch(
|
||||||
|
{ok, {S1110, [<<"quux">>]}},
|
||||||
|
lookup_topic_key(T, [<<"foo">>, <<"quux">>, <<"foo">>])
|
||||||
|
),
|
||||||
|
?assertMatch(
|
||||||
|
undefined,
|
||||||
|
lookup_topic_key(T, [<<"foo">>])
|
||||||
|
),
|
||||||
|
?assertMatch(
|
||||||
|
undefined,
|
||||||
|
lookup_topic_key(T, [<<"foo">>, <<"bar">>])
|
||||||
|
),
|
||||||
|
?assertMatch(
|
||||||
|
undefined,
|
||||||
|
lookup_topic_key(T, [<<"foo">>, <<"bar">>, <<"bar">>])
|
||||||
|
),
|
||||||
|
?assertMatch(
|
||||||
|
undefined,
|
||||||
|
lookup_topic_key(T, [<<"bar">>, <<"foo">>, <<"foo">>])
|
||||||
|
),
|
||||||
|
{_, S10} = trie_insert(T, S1, ?EOT),
|
||||||
|
?assertMatch(
|
||||||
|
{ok, {S10, []}},
|
||||||
|
lookup_topic_key(T, [<<"foo">>])
|
||||||
|
).
|
||||||
|
|
||||||
|
%% erlfmt-ignore
|
||||||
|
topic_key_test() ->
|
||||||
|
T = trie_create(),
|
||||||
|
try
|
||||||
|
Threshold = 4,
|
||||||
|
ThresholdFun = fun(0) -> 1000;
|
||||||
|
(_) -> Threshold
|
||||||
|
end,
|
||||||
|
%% Test that bottom layer threshold is high:
|
||||||
|
lists:foreach(
|
||||||
|
fun(I) ->
|
||||||
|
{_, []} = test_key(T, ThresholdFun, [I, 99999, 999999, 99999])
|
||||||
|
end,
|
||||||
|
lists:seq(1, 10)),
|
||||||
|
%% Test adding children on the 2nd level:
|
||||||
|
lists:foreach(
|
||||||
|
fun(I) ->
|
||||||
|
case test_key(T, ThresholdFun, [1, I, 1]) of
|
||||||
|
{_, []} ->
|
||||||
|
?assert(I < Threshold, {I, '<', Threshold}),
|
||||||
|
ok;
|
||||||
|
{_, [Var]} ->
|
||||||
|
?assert(I >= Threshold, {I, '>=', Threshold}),
|
||||||
|
?assertEqual(Var, integer_to_binary(I))
|
||||||
|
end
|
||||||
|
end,
|
||||||
|
lists:seq(1, 100)),
|
||||||
|
%% This doesn't affect 2nd level with a different prefix:
|
||||||
|
?assertMatch({_, []}, test_key(T, ThresholdFun, [2, 1, 1])),
|
||||||
|
?assertMatch({_, []}, test_key(T, ThresholdFun, [2, 10, 1])),
|
||||||
|
%% This didn't retroactively change the indexes that were
|
||||||
|
%% created prior to reaching the threshold:
|
||||||
|
?assertMatch({_, []}, test_key(T, ThresholdFun, [1, 1, 1])),
|
||||||
|
?assertMatch({_, []}, test_key(T, ThresholdFun, [1, 2, 1])),
|
||||||
|
%% Now create another level of +:
|
||||||
|
lists:foreach(
|
||||||
|
fun(I) ->
|
||||||
|
case test_key(T, ThresholdFun, [1, 42, 1, I, 42]) of
|
||||||
|
{_, [<<"42">>]} when I =< Threshold -> %% TODO: off by 1 error
|
||||||
|
ok;
|
||||||
|
{_, [<<"42">>, Var]} ->
|
||||||
|
?assertEqual(Var, integer_to_binary(I));
|
||||||
|
Ret ->
|
||||||
|
error({Ret, I})
|
||||||
|
end
|
||||||
|
end,
|
||||||
|
lists:seq(1, 100))
|
||||||
|
after
|
||||||
|
dump_to_dot(T, filename:join("_build", atom_to_list(?FUNCTION_NAME) ++ ".dot"))
|
||||||
|
end.
|
||||||
|
|
||||||
|
%% erlfmt-ignore
|
||||||
|
topic_match_test() ->
|
||||||
|
T = trie_create(),
|
||||||
|
try
|
||||||
|
Threshold = 2,
|
||||||
|
ThresholdFun = fun(0) -> 1000;
|
||||||
|
(_) -> Threshold
|
||||||
|
end,
|
||||||
|
{S1, []} = test_key(T, ThresholdFun, [1]),
|
||||||
|
{S11, []} = test_key(T, ThresholdFun, [1, 1]),
|
||||||
|
{S12, []} = test_key(T, ThresholdFun, [1, 2]),
|
||||||
|
{S111, []} = test_key(T, ThresholdFun, [1, 1, 1]),
|
||||||
|
%% Match concrete topics:
|
||||||
|
assert_match_topics(T, [1], [{S1, []}]),
|
||||||
|
assert_match_topics(T, [1, 1], [{S11, []}]),
|
||||||
|
assert_match_topics(T, [1, 1, 1], [{S111, []}]),
|
||||||
|
%% Match topics with +:
|
||||||
|
assert_match_topics(T, [1, '+'], [{S11, []}, {S12, []}]),
|
||||||
|
assert_match_topics(T, [1, '+', 1], [{S111, []}]),
|
||||||
|
%% Match topics with #:
|
||||||
|
assert_match_topics(T, [1, '#'],
|
||||||
|
[{S1, []},
|
||||||
|
{S11, []}, {S12, []},
|
||||||
|
{S111, []}]),
|
||||||
|
assert_match_topics(T, [1, 1, '#'],
|
||||||
|
[{S11, []},
|
||||||
|
{S111, []}]),
|
||||||
|
%% Now add learned wildcards:
|
||||||
|
{S21, []} = test_key(T, ThresholdFun, [2, 1]),
|
||||||
|
{S22, []} = test_key(T, ThresholdFun, [2, 2]),
|
||||||
|
{S2_, [<<"3">>]} = test_key(T, ThresholdFun, [2, 3]),
|
||||||
|
{S2_11, [<<"3">>]} = test_key(T, ThresholdFun, [2, 3, 1, 1]),
|
||||||
|
{S2_12, [<<"4">>]} = test_key(T, ThresholdFun, [2, 4, 1, 2]),
|
||||||
|
{S2_1_, [<<"3">>, <<"3">>]} = test_key(T, ThresholdFun, [2, 3, 1, 3]),
|
||||||
|
%% %% Check matching:
|
||||||
|
assert_match_topics(T, [2, 2],
|
||||||
|
[{S22, []}, {S2_, [<<"2">>]}]),
|
||||||
|
assert_match_topics(T, [2, '+'],
|
||||||
|
[{S22, []}, {S21, []}, {S2_, ['+']}]),
|
||||||
|
assert_match_topics(T, [2, '#'],
|
||||||
|
[{S21, []}, {S22, []},
|
||||||
|
{S2_, ['+']},
|
||||||
|
{S2_11, ['+']}, {S2_12, ['+']}, {S2_1_, ['+', '+']}]),
|
||||||
|
ok
|
||||||
|
after
|
||||||
|
dump_to_dot(T, filename:join("_build", atom_to_list(?FUNCTION_NAME) ++ ".dot"))
|
||||||
|
end.
|
||||||
|
|
||||||
|
-define(keys_history, topic_key_history).
|
||||||
|
|
||||||
|
%% erlfmt-ignore
|
||||||
|
assert_match_topics(Trie, Filter0, Expected) ->
|
||||||
|
Filter = lists:map(fun(I) when is_integer(I) -> integer_to_binary(I);
|
||||||
|
(I) -> I
|
||||||
|
end,
|
||||||
|
Filter0),
|
||||||
|
Matched = match_topics(Trie, Filter),
|
||||||
|
?assertMatch( #{missing := [], unexpected := []}
|
||||||
|
, #{ missing => Expected -- Matched
|
||||||
|
, unexpected => Matched -- Expected
|
||||||
|
}
|
||||||
|
, Filter
|
||||||
|
).
|
||||||
|
|
||||||
|
%% erlfmt-ignore
|
||||||
|
test_key(Trie, Threshold, Topic0) ->
|
||||||
|
Topic = [integer_to_binary(I) || I <- Topic0],
|
||||||
|
Ret = topic_key(Trie, Threshold, Topic),
|
||||||
|
%% Test idempotency:
|
||||||
|
Ret1 = topic_key(Trie, Threshold, Topic),
|
||||||
|
?assertEqual(Ret, Ret1, Topic),
|
||||||
|
%% Add new key to the history:
|
||||||
|
case get(?keys_history) of
|
||||||
|
undefined -> OldHistory = #{};
|
||||||
|
OldHistory -> ok
|
||||||
|
end,
|
||||||
|
%% Test that the generated keys are always unique for the topic:
|
||||||
|
History = maps:update_with(
|
||||||
|
Ret,
|
||||||
|
fun(Old) ->
|
||||||
|
case Old =:= Topic of
|
||||||
|
true -> Old;
|
||||||
|
false -> error(#{ '$msg' => "Duplicate key!"
|
||||||
|
, key => Ret
|
||||||
|
, old_topic => Old
|
||||||
|
, new_topic => Topic
|
||||||
|
})
|
||||||
|
end
|
||||||
|
end,
|
||||||
|
Topic,
|
||||||
|
OldHistory),
|
||||||
|
put(?keys_history, History),
|
||||||
|
{ok, Ret} = lookup_topic_key(Trie, Topic),
|
||||||
|
Ret.
|
||||||
|
|
||||||
|
-endif.
|
|
@ -1,742 +0,0 @@
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
%% Copyright (c) 2022-2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
|
|
||||||
-module(emqx_ds_message_storage_bitmask).
|
|
||||||
|
|
||||||
%%================================================================================
|
|
||||||
%% @doc Description of the schema
|
|
||||||
%%
|
|
||||||
%% Let us assume that `T' is a topic and `t' is time. These are the two
|
|
||||||
%% dimensions used to index messages. They can be viewed as
|
|
||||||
%% "coordinates" of an MQTT message in a 2D space.
|
|
||||||
%%
|
|
||||||
%% Oftentimes, when wildcard subscription is used, keys must be
|
|
||||||
%% scanned in both dimensions simultaneously.
|
|
||||||
%%
|
|
||||||
%% Rocksdb allows to iterate over sorted keys very fast. This means we
|
|
||||||
%% need to map our two-dimentional keys to a single index that is
|
|
||||||
%% sorted in a way that helps to iterate over both time and topic
|
|
||||||
%% without having to do a lot of random seeks.
|
|
||||||
%%
|
|
||||||
%% == Mapping of 2D keys to rocksdb keys ==
|
|
||||||
%%
|
|
||||||
%% We use "zigzag" pattern to store messages, where rocksdb key is
|
|
||||||
%% composed like like this:
|
|
||||||
%%
|
|
||||||
%% |ttttt|TTTTTTTTT|tttt|
|
|
||||||
%% ^ ^ ^
|
|
||||||
%% | | |
|
|
||||||
%% +-------+ | +---------+
|
|
||||||
%% | | |
|
|
||||||
%% most significant topic hash least significant
|
|
||||||
%% bits of timestamp bits of timestamp
|
|
||||||
%% (a.k.a epoch) (a.k.a time offset)
|
|
||||||
%%
|
|
||||||
%% Topic hash is level-aware: each topic level is hashed separately
|
|
||||||
%% and the resulting hashes are bitwise-concatentated. This allows us
|
|
||||||
%% to map topics to fixed-length bitstrings while keeping some degree
|
|
||||||
%% of information about the hierarchy.
|
|
||||||
%%
|
|
||||||
%% Next important concept is what we call "epoch". Duration of the
|
|
||||||
%% epoch is determined by maximum time offset. Epoch is calculated by
|
|
||||||
%% shifting bits of the timestamp right.
|
|
||||||
%%
|
|
||||||
%% The resulting index is a space-filling curve that looks like
|
|
||||||
%% this in the topic-time 2D space:
|
|
||||||
%%
|
|
||||||
%% T ^ ---->------ |---->------ |---->------
|
|
||||||
%% | --/ / --/ / --/
|
|
||||||
%% | -<-/ | -<-/ | -<-/
|
|
||||||
%% | -/ | -/ | -/
|
|
||||||
%% | ---->------ | ---->------ | ---->------
|
|
||||||
%% | --/ / --/ / --/
|
|
||||||
%% | ---/ | ---/ | ---/
|
|
||||||
%% | -/ ^ -/ ^ -/
|
|
||||||
%% | ---->------ | ---->------ | ---->------
|
|
||||||
%% | --/ / --/ / --/
|
|
||||||
%% | -<-/ | -<-/ | -<-/
|
|
||||||
%% | -/ | -/ | -/
|
|
||||||
%% | ---->------| ---->------| ---------->
|
|
||||||
%% |
|
|
||||||
%% -+------------+-----------------------------> t
|
|
||||||
%% epoch
|
|
||||||
%%
|
|
||||||
%% This structure allows to quickly seek to a the first message that
|
|
||||||
%% was recorded in a certain epoch in a certain topic or a
|
|
||||||
%% group of topics matching filter like `foo/bar/#`.
|
|
||||||
%%
|
|
||||||
%% Due to its structure, for each pair of rocksdb keys K1 and K2, such
|
|
||||||
%% that K1 > K2 and topic(K1) = topic(K2), timestamp(K1) >
|
|
||||||
%% timestamp(K2).
|
|
||||||
%% That is, replay doesn't reorder messages published in each
|
|
||||||
%% individual topic.
|
|
||||||
%%
|
|
||||||
%% This property doesn't hold between different topics, but it's not deemed
|
|
||||||
%% a problem right now.
|
|
||||||
%%
|
|
||||||
%%================================================================================
|
|
||||||
|
|
||||||
-behaviour(emqx_ds_storage_layer).
|
|
||||||
|
|
||||||
%% API:
|
|
||||||
-export([create_new/3, open/5]).
|
|
||||||
-export([make_keymapper/1]).
|
|
||||||
|
|
||||||
-export([store/5]).
|
|
||||||
-export([delete/4]).
|
|
||||||
-export([make_iterator/2]).
|
|
||||||
-export([make_iterator/3]).
|
|
||||||
-export([next/1]).
|
|
||||||
|
|
||||||
-export([preserve_iterator/1]).
|
|
||||||
-export([restore_iterator/3]).
|
|
||||||
-export([refresh_iterator/1]).
|
|
||||||
|
|
||||||
%% Debug/troubleshooting:
|
|
||||||
%% Keymappers
|
|
||||||
-export([
|
|
||||||
keymapper_info/1,
|
|
||||||
compute_bitstring/3,
|
|
||||||
compute_topic_bitmask/2,
|
|
||||||
compute_time_bitmask/1,
|
|
||||||
hash/2
|
|
||||||
]).
|
|
||||||
|
|
||||||
%% Keyspace filters
|
|
||||||
-export([
|
|
||||||
make_keyspace_filter/2,
|
|
||||||
compute_initial_seek/1,
|
|
||||||
compute_next_seek/2,
|
|
||||||
compute_time_seek/3,
|
|
||||||
compute_topic_seek/4
|
|
||||||
]).
|
|
||||||
|
|
||||||
-export_type([db/0, iterator/0, schema/0]).
|
|
||||||
|
|
||||||
-export_type([options/0]).
|
|
||||||
-export_type([iteration_options/0]).
|
|
||||||
|
|
||||||
-compile(
|
|
||||||
{inline, [
|
|
||||||
bitwise_concat/3,
|
|
||||||
ones/1,
|
|
||||||
successor/1,
|
|
||||||
topic_hash_matches/3,
|
|
||||||
time_matches/3
|
|
||||||
]}
|
|
||||||
).
|
|
||||||
|
|
||||||
%%================================================================================
|
|
||||||
%% Type declarations
|
|
||||||
%%================================================================================
|
|
||||||
|
|
||||||
-type topic() :: emqx_ds:topic().
|
|
||||||
-type topic_filter() :: emqx_ds:topic_filter().
|
|
||||||
-type time() :: emqx_ds:time().
|
|
||||||
|
|
||||||
%% Number of bits
|
|
||||||
-type bits() :: non_neg_integer().
|
|
||||||
|
|
||||||
%% Key of a RocksDB record.
|
|
||||||
-type key() :: binary().
|
|
||||||
|
|
||||||
%% Distribution of entropy among topic levels.
|
|
||||||
%% Example: [4, 8, 16] means that level 1 gets 4 bits, level 2 gets 8 bits,
|
|
||||||
%% and _rest of levels_ (if any) get 16 bits.
|
|
||||||
-type bits_per_level() :: [bits(), ...].
|
|
||||||
|
|
||||||
-type options() :: #{
|
|
||||||
%% Number of bits in a message timestamp.
|
|
||||||
timestamp_bits := bits(),
|
|
||||||
%% Number of bits in a key allocated to each level in a message topic.
|
|
||||||
topic_bits_per_level := bits_per_level(),
|
|
||||||
%% Maximum granularity of iteration over time.
|
|
||||||
epoch := time(),
|
|
||||||
|
|
||||||
iteration => iteration_options(),
|
|
||||||
|
|
||||||
cf_options => emqx_ds_storage_layer:db_cf_options()
|
|
||||||
}.
|
|
||||||
|
|
||||||
-type iteration_options() :: #{
|
|
||||||
%% Request periodic iterator refresh.
|
|
||||||
%% This might be helpful during replays taking a lot of time (e.g. tens of seconds).
|
|
||||||
%% Note that `{every, 1000}` means 1000 _operations_ with the iterator which is not
|
|
||||||
%% the same as 1000 replayed messages.
|
|
||||||
iterator_refresh => {every, _NumOperations :: pos_integer()}
|
|
||||||
}.
|
|
||||||
|
|
||||||
%% Persistent configuration of the generation, it is used to create db
|
|
||||||
%% record when the database is reopened
|
|
||||||
-record(schema, {keymapper :: keymapper()}).
|
|
||||||
|
|
||||||
-opaque schema() :: #schema{}.
|
|
||||||
|
|
||||||
-record(db, {
|
|
||||||
shard :: emqx_ds:shard(),
|
|
||||||
handle :: rocksdb:db_handle(),
|
|
||||||
cf :: rocksdb:cf_handle(),
|
|
||||||
keymapper :: keymapper(),
|
|
||||||
write_options = [{sync, true}] :: emqx_ds_storage_layer:db_write_options(),
|
|
||||||
read_options = [] :: emqx_ds_storage_layer:db_read_options()
|
|
||||||
}).
|
|
||||||
|
|
||||||
-record(it, {
|
|
||||||
handle :: rocksdb:itr_handle(),
|
|
||||||
filter :: keyspace_filter(),
|
|
||||||
cursor :: binary() | undefined,
|
|
||||||
next_action :: {seek, binary()} | next,
|
|
||||||
refresh_counter :: {non_neg_integer(), pos_integer()} | undefined
|
|
||||||
}).
|
|
||||||
|
|
||||||
-record(filter, {
|
|
||||||
keymapper :: keymapper(),
|
|
||||||
topic_filter :: topic_filter(),
|
|
||||||
start_time :: integer(),
|
|
||||||
hash_bitfilter :: integer(),
|
|
||||||
hash_bitmask :: integer(),
|
|
||||||
time_bitfilter :: integer(),
|
|
||||||
time_bitmask :: integer()
|
|
||||||
}).
|
|
||||||
|
|
||||||
% NOTE
|
|
||||||
% Keymapper decides how to map messages into RocksDB column family keyspace.
|
|
||||||
-record(keymapper, {
|
|
||||||
source :: [bitsource(), ...],
|
|
||||||
bitsize :: bits(),
|
|
||||||
epoch :: non_neg_integer()
|
|
||||||
}).
|
|
||||||
|
|
||||||
-type bitsource() ::
|
|
||||||
%% Consume `_Size` bits from timestamp starting at `_Offset`th bit.
|
|
||||||
%% TODO consistency
|
|
||||||
{timestamp, _Offset :: bits(), _Size :: bits()}
|
|
||||||
%% Consume next topic level (either one or all of them) and compute `_Size` bits-wide hash.
|
|
||||||
| {hash, level | levels, _Size :: bits()}.
|
|
||||||
|
|
||||||
-opaque db() :: #db{}.
|
|
||||||
-opaque iterator() :: #it{}.
|
|
||||||
-type keymapper() :: #keymapper{}.
|
|
||||||
-type keyspace_filter() :: #filter{}.
|
|
||||||
|
|
||||||
%%================================================================================
|
|
||||||
%% API funcions
|
|
||||||
%%================================================================================
|
|
||||||
|
|
||||||
%% Create a new column family for the generation and a serializable representation of the schema
|
|
||||||
-spec create_new(rocksdb:db_handle(), emqx_ds_storage_layer:gen_id(), options()) ->
|
|
||||||
{schema(), emqx_ds_storage_layer:cf_refs()}.
|
|
||||||
create_new(DBHandle, GenId, Options) ->
|
|
||||||
CFName = data_cf(GenId),
|
|
||||||
CFOptions = maps:get(cf_options, Options, []),
|
|
||||||
{ok, CFHandle} = rocksdb:create_column_family(DBHandle, CFName, CFOptions),
|
|
||||||
Schema = #schema{keymapper = make_keymapper(Options)},
|
|
||||||
{Schema, [{CFName, CFHandle}]}.
|
|
||||||
|
|
||||||
%% Reopen the database
|
|
||||||
-spec open(
|
|
||||||
emqx_ds:shard(),
|
|
||||||
rocksdb:db_handle(),
|
|
||||||
emqx_ds_storage_layer:gen_id(),
|
|
||||||
emqx_ds_storage_layer:cf_refs(),
|
|
||||||
schema()
|
|
||||||
) ->
|
|
||||||
db().
|
|
||||||
open(Shard, DBHandle, GenId, CFs, #schema{keymapper = Keymapper}) ->
|
|
||||||
{value, {_, CFHandle}} = lists:keysearch(data_cf(GenId), 1, CFs),
|
|
||||||
#db{
|
|
||||||
shard = Shard,
|
|
||||||
handle = DBHandle,
|
|
||||||
cf = CFHandle,
|
|
||||||
keymapper = Keymapper
|
|
||||||
}.
|
|
||||||
|
|
||||||
-spec make_keymapper(options()) -> keymapper().
|
|
||||||
make_keymapper(#{
|
|
||||||
timestamp_bits := TimestampBits,
|
|
||||||
topic_bits_per_level := BitsPerLevel,
|
|
||||||
epoch := MaxEpoch
|
|
||||||
}) ->
|
|
||||||
TimestampLSBs = min(TimestampBits, floor(math:log2(MaxEpoch))),
|
|
||||||
TimestampMSBs = TimestampBits - TimestampLSBs,
|
|
||||||
NLevels = length(BitsPerLevel),
|
|
||||||
{LevelBits, [TailLevelsBits]} = lists:split(NLevels - 1, BitsPerLevel),
|
|
||||||
Source = lists:flatten([
|
|
||||||
[{timestamp, TimestampLSBs, TimestampMSBs} || TimestampMSBs > 0],
|
|
||||||
[{hash, level, Bits} || Bits <- LevelBits],
|
|
||||||
{hash, levels, TailLevelsBits},
|
|
||||||
[{timestamp, 0, TimestampLSBs} || TimestampLSBs > 0]
|
|
||||||
]),
|
|
||||||
#keymapper{
|
|
||||||
source = Source,
|
|
||||||
bitsize = lists:sum([S || {_, _, S} <- Source]),
|
|
||||||
epoch = 1 bsl TimestampLSBs
|
|
||||||
}.
|
|
||||||
|
|
||||||
-spec store(db(), emqx_guid:guid(), emqx_ds:time(), topic(), binary()) ->
|
|
||||||
ok | {error, _TODO}.
|
|
||||||
store(DB = #db{handle = DBHandle, cf = CFHandle}, MessageID, PublishedAt, Topic, MessagePayload) ->
|
|
||||||
Key = make_message_key(Topic, PublishedAt, MessageID, DB#db.keymapper),
|
|
||||||
Value = make_message_value(Topic, MessagePayload),
|
|
||||||
rocksdb:put(DBHandle, CFHandle, Key, Value, DB#db.write_options).
|
|
||||||
|
|
||||||
-spec delete(db(), emqx_guid:guid(), emqx_ds:time(), topic()) ->
|
|
||||||
ok | {error, _TODO}.
|
|
||||||
delete(DB = #db{handle = DBHandle, cf = CFHandle}, MessageID, PublishedAt, Topic) ->
|
|
||||||
Key = make_message_key(Topic, PublishedAt, MessageID, DB#db.keymapper),
|
|
||||||
rocksdb:delete(DBHandle, CFHandle, Key, DB#db.write_options).
|
|
||||||
|
|
||||||
-spec make_iterator(db(), emqx_ds:replay()) ->
|
|
||||||
{ok, iterator()} | {error, _TODO}.
|
|
||||||
make_iterator(DB, Replay) ->
|
|
||||||
{Keyspace, _ShardId} = DB#db.shard,
|
|
||||||
Options = emqx_ds_conf:iteration_options(Keyspace),
|
|
||||||
make_iterator(DB, Replay, Options).
|
|
||||||
|
|
||||||
-spec make_iterator(db(), emqx_ds:replay(), iteration_options()) ->
|
|
||||||
% {error, invalid_start_time}? might just start from the beginning of time
|
|
||||||
% and call it a day: client violated the contract anyway.
|
|
||||||
{ok, iterator()} | {error, _TODO}.
|
|
||||||
make_iterator(DB = #db{handle = DBHandle, cf = CFHandle}, Replay, Options) ->
|
|
||||||
case rocksdb:iterator(DBHandle, CFHandle, DB#db.read_options) of
|
|
||||||
{ok, ITHandle} ->
|
|
||||||
Filter = make_keyspace_filter(Replay, DB#db.keymapper),
|
|
||||||
InitialSeek = combine(compute_initial_seek(Filter), <<>>, DB#db.keymapper),
|
|
||||||
RefreshCounter = make_refresh_counter(maps:get(iterator_refresh, Options, undefined)),
|
|
||||||
{ok, #it{
|
|
||||||
handle = ITHandle,
|
|
||||||
filter = Filter,
|
|
||||||
next_action = {seek, InitialSeek},
|
|
||||||
refresh_counter = RefreshCounter
|
|
||||||
}};
|
|
||||||
Err ->
|
|
||||||
Err
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec next(iterator()) -> {value, binary(), iterator()} | none | {error, closed}.
|
|
||||||
next(It0 = #it{filter = #filter{keymapper = Keymapper}}) ->
|
|
||||||
It = maybe_refresh_iterator(It0),
|
|
||||||
case rocksdb:iterator_move(It#it.handle, It#it.next_action) of
|
|
||||||
% spec says `{ok, Key}` is also possible but the implementation says it's not
|
|
||||||
{ok, Key, Value} ->
|
|
||||||
% Preserve last seen key in the iterator so it could be restored / refreshed later.
|
|
||||||
ItNext = It#it{cursor = Key},
|
|
||||||
Bitstring = extract(Key, Keymapper),
|
|
||||||
case match_next(Bitstring, Value, It#it.filter) of
|
|
||||||
{_Topic, Payload} ->
|
|
||||||
{value, Payload, ItNext#it{next_action = next}};
|
|
||||||
next ->
|
|
||||||
next(ItNext#it{next_action = next});
|
|
||||||
NextBitstring when is_integer(NextBitstring) ->
|
|
||||||
NextSeek = combine(NextBitstring, <<>>, Keymapper),
|
|
||||||
next(ItNext#it{next_action = {seek, NextSeek}});
|
|
||||||
none ->
|
|
||||||
stop_iteration(ItNext)
|
|
||||||
end;
|
|
||||||
{error, invalid_iterator} ->
|
|
||||||
stop_iteration(It);
|
|
||||||
{error, iterator_closed} ->
|
|
||||||
{error, closed}
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec preserve_iterator(iterator()) -> binary().
|
|
||||||
preserve_iterator(#it{cursor = Cursor}) ->
|
|
||||||
State = #{
|
|
||||||
v => 1,
|
|
||||||
cursor => Cursor
|
|
||||||
},
|
|
||||||
term_to_binary(State).
|
|
||||||
|
|
||||||
-spec restore_iterator(db(), emqx_ds:replay(), binary()) ->
|
|
||||||
{ok, iterator()} | {error, _TODO}.
|
|
||||||
restore_iterator(DB, Replay, Serial) when is_binary(Serial) ->
|
|
||||||
State = binary_to_term(Serial),
|
|
||||||
restore_iterator(DB, Replay, State);
|
|
||||||
restore_iterator(DB, Replay, #{
|
|
||||||
v := 1,
|
|
||||||
cursor := Cursor
|
|
||||||
}) ->
|
|
||||||
case make_iterator(DB, Replay) of
|
|
||||||
{ok, It} when Cursor == undefined ->
|
|
||||||
% Iterator was preserved right after it has been made.
|
|
||||||
{ok, It};
|
|
||||||
{ok, It} ->
|
|
||||||
% Iterator was preserved mid-replay, seek right past the last seen key.
|
|
||||||
{ok, It#it{cursor = Cursor, next_action = {seek, successor(Cursor)}}};
|
|
||||||
Err ->
|
|
||||||
Err
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec refresh_iterator(iterator()) -> iterator().
|
|
||||||
refresh_iterator(It = #it{handle = Handle, cursor = Cursor, next_action = Action}) ->
|
|
||||||
case rocksdb:iterator_refresh(Handle) of
|
|
||||||
ok when Action =:= next ->
|
|
||||||
% Now the underlying iterator is invalid, need to seek instead.
|
|
||||||
It#it{next_action = {seek, successor(Cursor)}};
|
|
||||||
ok ->
|
|
||||||
% Now the underlying iterator is invalid, but will seek soon anyway.
|
|
||||||
It;
|
|
||||||
{error, _} ->
|
|
||||||
% Implementation could in theory return an {error, ...} tuple.
|
|
||||||
% Supposedly our best bet is to ignore it.
|
|
||||||
% TODO logging?
|
|
||||||
It
|
|
||||||
end.
|
|
||||||
|
|
||||||
%%================================================================================
|
|
||||||
%% Internal exports
|
|
||||||
%%================================================================================
|
|
||||||
|
|
||||||
-spec keymapper_info(keymapper()) ->
|
|
||||||
#{source := [bitsource()], bitsize := bits(), epoch := time()}.
|
|
||||||
keymapper_info(#keymapper{source = Source, bitsize = Bitsize, epoch = Epoch}) ->
|
|
||||||
#{source => Source, bitsize => Bitsize, epoch => Epoch}.
|
|
||||||
|
|
||||||
make_message_key(Topic, PublishedAt, MessageID, Keymapper) ->
|
|
||||||
combine(compute_bitstring(Topic, PublishedAt, Keymapper), MessageID, Keymapper).
|
|
||||||
|
|
||||||
make_message_value(Topic, MessagePayload) ->
|
|
||||||
term_to_binary({Topic, MessagePayload}).
|
|
||||||
|
|
||||||
unwrap_message_value(Binary) ->
|
|
||||||
binary_to_term(Binary).
|
|
||||||
|
|
||||||
-spec combine(_Bitstring :: integer(), emqx_guid:guid() | <<>>, keymapper()) ->
|
|
||||||
key().
|
|
||||||
combine(Bitstring, MessageID, #keymapper{bitsize = Size}) ->
|
|
||||||
<<Bitstring:Size/integer, MessageID/binary>>.
|
|
||||||
|
|
||||||
-spec extract(key(), keymapper()) ->
|
|
||||||
_Bitstring :: integer().
|
|
||||||
extract(Key, #keymapper{bitsize = Size}) ->
|
|
||||||
<<Bitstring:Size/integer, _MessageID/binary>> = Key,
|
|
||||||
Bitstring.
|
|
||||||
|
|
||||||
-spec compute_bitstring(topic_filter(), time(), keymapper()) -> integer().
|
|
||||||
compute_bitstring(TopicFilter, Timestamp, #keymapper{source = Source}) ->
|
|
||||||
compute_bitstring(TopicFilter, Timestamp, Source, 0).
|
|
||||||
|
|
||||||
-spec compute_topic_bitmask(topic_filter(), keymapper()) -> integer().
|
|
||||||
compute_topic_bitmask(TopicFilter, #keymapper{source = Source}) ->
|
|
||||||
compute_topic_bitmask(TopicFilter, Source, 0).
|
|
||||||
|
|
||||||
-spec compute_time_bitmask(keymapper()) -> integer().
|
|
||||||
compute_time_bitmask(#keymapper{source = Source}) ->
|
|
||||||
compute_time_bitmask(Source, 0).
|
|
||||||
|
|
||||||
-spec hash(term(), bits()) -> integer().
|
|
||||||
hash(Input, Bits) ->
|
|
||||||
% at most 32 bits
|
|
||||||
erlang:phash2(Input, 1 bsl Bits).
|
|
||||||
|
|
||||||
-spec make_keyspace_filter(emqx_ds:replay(), keymapper()) -> keyspace_filter().
|
|
||||||
make_keyspace_filter({TopicFilter, StartTime}, Keymapper) ->
|
|
||||||
Bitstring = compute_bitstring(TopicFilter, StartTime, Keymapper),
|
|
||||||
HashBitmask = compute_topic_bitmask(TopicFilter, Keymapper),
|
|
||||||
TimeBitmask = compute_time_bitmask(Keymapper),
|
|
||||||
HashBitfilter = Bitstring band HashBitmask,
|
|
||||||
TimeBitfilter = Bitstring band TimeBitmask,
|
|
||||||
#filter{
|
|
||||||
keymapper = Keymapper,
|
|
||||||
topic_filter = TopicFilter,
|
|
||||||
start_time = StartTime,
|
|
||||||
hash_bitfilter = HashBitfilter,
|
|
||||||
hash_bitmask = HashBitmask,
|
|
||||||
time_bitfilter = TimeBitfilter,
|
|
||||||
time_bitmask = TimeBitmask
|
|
||||||
}.
|
|
||||||
|
|
||||||
-spec compute_initial_seek(keyspace_filter()) -> integer().
|
|
||||||
compute_initial_seek(#filter{hash_bitfilter = HashBitfilter, time_bitfilter = TimeBitfilter}) ->
|
|
||||||
% Should be the same as `compute_initial_seek(0, Filter)`.
|
|
||||||
HashBitfilter bor TimeBitfilter.
|
|
||||||
|
|
||||||
-spec compute_next_seek(integer(), keyspace_filter()) -> integer().
|
|
||||||
compute_next_seek(
|
|
||||||
Bitstring,
|
|
||||||
Filter = #filter{
|
|
||||||
hash_bitfilter = HashBitfilter,
|
|
||||||
hash_bitmask = HashBitmask,
|
|
||||||
time_bitfilter = TimeBitfilter,
|
|
||||||
time_bitmask = TimeBitmask
|
|
||||||
}
|
|
||||||
) ->
|
|
||||||
HashMatches = topic_hash_matches(Bitstring, HashBitfilter, HashBitmask),
|
|
||||||
TimeMatches = time_matches(Bitstring, TimeBitfilter, TimeBitmask),
|
|
||||||
compute_next_seek(HashMatches, TimeMatches, Bitstring, Filter).
|
|
||||||
|
|
||||||
%%================================================================================
|
|
||||||
%% Internal functions
|
|
||||||
%%================================================================================
|
|
||||||
|
|
||||||
compute_bitstring(Topic, Timestamp, [{timestamp, Offset, Size} | Rest], Acc) ->
|
|
||||||
I = (Timestamp bsr Offset) band ones(Size),
|
|
||||||
compute_bitstring(Topic, Timestamp, Rest, bitwise_concat(Acc, I, Size));
|
|
||||||
compute_bitstring([], Timestamp, [{hash, level, Size} | Rest], Acc) ->
|
|
||||||
I = hash(<<"/">>, Size),
|
|
||||||
compute_bitstring([], Timestamp, Rest, bitwise_concat(Acc, I, Size));
|
|
||||||
compute_bitstring([Level | Tail], Timestamp, [{hash, level, Size} | Rest], Acc) ->
|
|
||||||
I = hash(Level, Size),
|
|
||||||
compute_bitstring(Tail, Timestamp, Rest, bitwise_concat(Acc, I, Size));
|
|
||||||
compute_bitstring(Tail, Timestamp, [{hash, levels, Size} | Rest], Acc) ->
|
|
||||||
I = hash(Tail, Size),
|
|
||||||
compute_bitstring(Tail, Timestamp, Rest, bitwise_concat(Acc, I, Size));
|
|
||||||
compute_bitstring(_, _, [], Acc) ->
|
|
||||||
Acc.
|
|
||||||
|
|
||||||
compute_topic_bitmask(Filter, [{timestamp, _, Size} | Rest], Acc) ->
|
|
||||||
compute_topic_bitmask(Filter, Rest, bitwise_concat(Acc, 0, Size));
|
|
||||||
compute_topic_bitmask(['#'], [{hash, _, Size} | Rest], Acc) ->
|
|
||||||
compute_topic_bitmask(['#'], Rest, bitwise_concat(Acc, 0, Size));
|
|
||||||
compute_topic_bitmask(['+' | Tail], [{hash, _, Size} | Rest], Acc) ->
|
|
||||||
compute_topic_bitmask(Tail, Rest, bitwise_concat(Acc, 0, Size));
|
|
||||||
compute_topic_bitmask([], [{hash, level, Size} | Rest], Acc) ->
|
|
||||||
compute_topic_bitmask([], Rest, bitwise_concat(Acc, ones(Size), Size));
|
|
||||||
compute_topic_bitmask([_ | Tail], [{hash, level, Size} | Rest], Acc) ->
|
|
||||||
compute_topic_bitmask(Tail, Rest, bitwise_concat(Acc, ones(Size), Size));
|
|
||||||
compute_topic_bitmask(Tail, [{hash, levels, Size} | Rest], Acc) ->
|
|
||||||
Mask =
|
|
||||||
case lists:member('+', Tail) orelse lists:member('#', Tail) of
|
|
||||||
true -> 0;
|
|
||||||
false -> ones(Size)
|
|
||||||
end,
|
|
||||||
compute_topic_bitmask([], Rest, bitwise_concat(Acc, Mask, Size));
|
|
||||||
compute_topic_bitmask(_, [], Acc) ->
|
|
||||||
Acc.
|
|
||||||
|
|
||||||
compute_time_bitmask([{timestamp, _, Size} | Rest], Acc) ->
|
|
||||||
compute_time_bitmask(Rest, bitwise_concat(Acc, ones(Size), Size));
|
|
||||||
compute_time_bitmask([{hash, _, Size} | Rest], Acc) ->
|
|
||||||
compute_time_bitmask(Rest, bitwise_concat(Acc, 0, Size));
|
|
||||||
compute_time_bitmask([], Acc) ->
|
|
||||||
Acc.
|
|
||||||
|
|
||||||
bitwise_concat(Acc, Item, ItemSize) ->
|
|
||||||
(Acc bsl ItemSize) bor Item.
|
|
||||||
|
|
||||||
ones(Bits) ->
|
|
||||||
1 bsl Bits - 1.
|
|
||||||
|
|
||||||
-spec successor(key()) -> key().
|
|
||||||
successor(Key) ->
|
|
||||||
<<Key/binary, 0:8>>.
|
|
||||||
|
|
||||||
%% |123|345|678|
|
|
||||||
%% foo bar baz
|
|
||||||
|
|
||||||
%% |123|000|678| - |123|fff|678|
|
|
||||||
|
|
||||||
%% foo + baz
|
|
||||||
|
|
||||||
%% |fff|000|fff|
|
|
||||||
|
|
||||||
%% |123|000|678|
|
|
||||||
|
|
||||||
%% |123|056|678| & |fff|000|fff| = |123|000|678|.
|
|
||||||
|
|
||||||
match_next(
|
|
||||||
Bitstring,
|
|
||||||
Value,
|
|
||||||
Filter = #filter{
|
|
||||||
topic_filter = TopicFilter,
|
|
||||||
hash_bitfilter = HashBitfilter,
|
|
||||||
hash_bitmask = HashBitmask,
|
|
||||||
time_bitfilter = TimeBitfilter,
|
|
||||||
time_bitmask = TimeBitmask
|
|
||||||
}
|
|
||||||
) ->
|
|
||||||
HashMatches = topic_hash_matches(Bitstring, HashBitfilter, HashBitmask),
|
|
||||||
TimeMatches = time_matches(Bitstring, TimeBitfilter, TimeBitmask),
|
|
||||||
case HashMatches and TimeMatches of
|
|
||||||
true ->
|
|
||||||
Message = {Topic, _Payload} = unwrap_message_value(Value),
|
|
||||||
case emqx_topic:match(Topic, TopicFilter) of
|
|
||||||
true ->
|
|
||||||
Message;
|
|
||||||
false ->
|
|
||||||
next
|
|
||||||
end;
|
|
||||||
false ->
|
|
||||||
compute_next_seek(HashMatches, TimeMatches, Bitstring, Filter)
|
|
||||||
end.
|
|
||||||
|
|
||||||
%% `Bitstring` is out of the hash space defined by `HashBitfilter`.
|
|
||||||
compute_next_seek(
|
|
||||||
_HashMatches = false,
|
|
||||||
_TimeMatches,
|
|
||||||
Bitstring,
|
|
||||||
Filter = #filter{
|
|
||||||
keymapper = Keymapper,
|
|
||||||
hash_bitfilter = HashBitfilter,
|
|
||||||
hash_bitmask = HashBitmask,
|
|
||||||
time_bitfilter = TimeBitfilter,
|
|
||||||
time_bitmask = TimeBitmask
|
|
||||||
}
|
|
||||||
) ->
|
|
||||||
NextBitstring = compute_topic_seek(Bitstring, HashBitfilter, HashBitmask, Keymapper),
|
|
||||||
case NextBitstring of
|
|
||||||
none ->
|
|
||||||
none;
|
|
||||||
_ ->
|
|
||||||
TimeMatches = time_matches(NextBitstring, TimeBitfilter, TimeBitmask),
|
|
||||||
compute_next_seek(true, TimeMatches, NextBitstring, Filter)
|
|
||||||
end;
|
|
||||||
%% `Bitstring` is out of the time range defined by `TimeBitfilter`.
|
|
||||||
compute_next_seek(
|
|
||||||
_HashMatches = true,
|
|
||||||
_TimeMatches = false,
|
|
||||||
Bitstring,
|
|
||||||
#filter{
|
|
||||||
time_bitfilter = TimeBitfilter,
|
|
||||||
time_bitmask = TimeBitmask
|
|
||||||
}
|
|
||||||
) ->
|
|
||||||
compute_time_seek(Bitstring, TimeBitfilter, TimeBitmask);
|
|
||||||
compute_next_seek(true, true, Bitstring, _It) ->
|
|
||||||
Bitstring.
|
|
||||||
|
|
||||||
topic_hash_matches(Bitstring, HashBitfilter, HashBitmask) ->
|
|
||||||
(Bitstring band HashBitmask) == HashBitfilter.
|
|
||||||
|
|
||||||
time_matches(Bitstring, TimeBitfilter, TimeBitmask) ->
|
|
||||||
(Bitstring band TimeBitmask) >= TimeBitfilter.
|
|
||||||
|
|
||||||
compute_time_seek(Bitstring, TimeBitfilter, TimeBitmask) ->
|
|
||||||
% Replace the bits of the timestamp in `Bistring` with bits from `Timebitfilter`.
|
|
||||||
(Bitstring band (bnot TimeBitmask)) bor TimeBitfilter.
|
|
||||||
|
|
||||||
%% Find the closest bitstring which is:
|
|
||||||
%% * greater than `Bitstring`,
|
|
||||||
%% * and falls into the hash space defined by `HashBitfilter`.
|
|
||||||
%% Note that the result can end up "back" in time and out of the time range.
|
|
||||||
compute_topic_seek(Bitstring, HashBitfilter, HashBitmask, Keymapper) ->
|
|
||||||
Sources = Keymapper#keymapper.source,
|
|
||||||
Size = Keymapper#keymapper.bitsize,
|
|
||||||
compute_topic_seek(Bitstring, HashBitfilter, HashBitmask, Sources, Size).
|
|
||||||
|
|
||||||
compute_topic_seek(Bitstring, HashBitfilter, HashBitmask, Sources, Size) ->
|
|
||||||
% NOTE
|
|
||||||
% We're iterating through `Substring` here, in lockstep with `HashBitfilter`
|
|
||||||
% and `HashBitmask`, starting from least signigicant bits. Each bitsource in
|
|
||||||
% `Sources` has a bitsize `S` and, accordingly, gives us a sub-bitstring `S`
|
|
||||||
% bits long which we interpret as a "digit". There are 2 flavors of those
|
|
||||||
% "digits":
|
|
||||||
% * regular digit with 2^S possible values,
|
|
||||||
% * degenerate digit with exactly 1 possible value U (represented with 0).
|
|
||||||
% Our goal here is to find a successor of `Bistring` and perform a kind of
|
|
||||||
% digit-by-digit addition operation with carry propagation.
|
|
||||||
NextSeek = zipfoldr3(
|
|
||||||
fun(Source, Substring, Filter, LBitmask, Offset, Acc) ->
|
|
||||||
case Source of
|
|
||||||
{hash, _, S} when LBitmask =:= 0 ->
|
|
||||||
% Regular case
|
|
||||||
bitwise_add_digit(Substring, Acc, S, Offset);
|
|
||||||
{hash, _, _} when LBitmask =/= 0, Substring < Filter ->
|
|
||||||
% Degenerate case, I_digit < U, no overflow.
|
|
||||||
% Successor is `U bsl Offset` which is equivalent to 0.
|
|
||||||
0;
|
|
||||||
{hash, _, S} when LBitmask =/= 0, Substring > Filter ->
|
|
||||||
% Degenerate case, I_digit > U, overflow.
|
|
||||||
% Successor is `(1 bsl Size + U) bsl Offset`.
|
|
||||||
overflow_digit(S, Offset);
|
|
||||||
{hash, _, S} when LBitmask =/= 0 ->
|
|
||||||
% Degenerate case, I_digit = U
|
|
||||||
% Perform digit addition with I_digit = 0, assuming "digit" has
|
|
||||||
% 0 bits of information (but is `S` bits long at the same time).
|
|
||||||
% This will overflow only if the result of previous iteration
|
|
||||||
% was an overflow.
|
|
||||||
bitwise_add_digit(0, Acc, 0, S, Offset);
|
|
||||||
{timestamp, _, S} ->
|
|
||||||
% Regular case
|
|
||||||
bitwise_add_digit(Substring, Acc, S, Offset)
|
|
||||||
end
|
|
||||||
end,
|
|
||||||
0,
|
|
||||||
Bitstring,
|
|
||||||
HashBitfilter,
|
|
||||||
HashBitmask,
|
|
||||||
Size,
|
|
||||||
Sources
|
|
||||||
),
|
|
||||||
case NextSeek bsr Size of
|
|
||||||
_Carry = 0 ->
|
|
||||||
% Found the successor.
|
|
||||||
% We need to recover values of those degenerate digits which we
|
|
||||||
% represented with 0 during digit-by-digit iteration.
|
|
||||||
NextSeek bor (HashBitfilter band HashBitmask);
|
|
||||||
_Carry = 1 ->
|
|
||||||
% We got "carried away" past the range, time to stop iteration.
|
|
||||||
none
|
|
||||||
end.
|
|
||||||
|
|
||||||
bitwise_add_digit(Digit, Number, Width, Offset) ->
|
|
||||||
bitwise_add_digit(Digit, Number, Width, Width, Offset).
|
|
||||||
|
|
||||||
%% Add "digit" (represented with integer `Digit`) to the `Number` assuming
|
|
||||||
%% this digit starts at `Offset` bits in `Number` and is `Width` bits long.
|
|
||||||
%% Perform an overflow if the result of addition would not fit into `Bits`
|
|
||||||
%% bits.
|
|
||||||
bitwise_add_digit(Digit, Number, Bits, Width, Offset) ->
|
|
||||||
Sum = (Digit bsl Offset) + Number,
|
|
||||||
case (Sum bsr Offset) < (1 bsl Bits) of
|
|
||||||
true -> Sum;
|
|
||||||
false -> overflow_digit(Width, Offset)
|
|
||||||
end.
|
|
||||||
|
|
||||||
%% Constuct a number which denotes an overflow of digit that starts at
|
|
||||||
%% `Offset` bits and is `Width` bits long.
|
|
||||||
overflow_digit(Width, Offset) ->
|
|
||||||
(1 bsl Width) bsl Offset.
|
|
||||||
|
|
||||||
%% Iterate through sub-bitstrings of 3 integers in lockstep, starting from least
|
|
||||||
%% significant bits first.
|
|
||||||
%%
|
|
||||||
%% Each integer is assumed to be `Size` bits long. Lengths of sub-bitstring are
|
|
||||||
%% specified in `Sources` list, in order from most significant bits to least
|
|
||||||
%% significant. Each iteration calls `FoldFun` with:
|
|
||||||
%% * bitsource that was used to extract sub-bitstrings,
|
|
||||||
%% * 3 sub-bitstrings in integer representation,
|
|
||||||
%% * bit offset into integers,
|
|
||||||
%% * current accumulator.
|
|
||||||
-spec zipfoldr3(FoldFun, Acc, integer(), integer(), integer(), _Size :: bits(), [bitsource()]) ->
|
|
||||||
Acc
|
|
||||||
when
|
|
||||||
FoldFun :: fun((bitsource(), integer(), integer(), integer(), _Offset :: bits(), Acc) -> Acc).
|
|
||||||
zipfoldr3(_FoldFun, Acc, _, _, _, 0, []) ->
|
|
||||||
Acc;
|
|
||||||
zipfoldr3(FoldFun, Acc, I1, I2, I3, Offset, [Source = {_, _, S} | Rest]) ->
|
|
||||||
OffsetNext = Offset - S,
|
|
||||||
AccNext = zipfoldr3(FoldFun, Acc, I1, I2, I3, OffsetNext, Rest),
|
|
||||||
FoldFun(
|
|
||||||
Source,
|
|
||||||
substring(I1, OffsetNext, S),
|
|
||||||
substring(I2, OffsetNext, S),
|
|
||||||
substring(I3, OffsetNext, S),
|
|
||||||
OffsetNext,
|
|
||||||
AccNext
|
|
||||||
).
|
|
||||||
|
|
||||||
substring(I, Offset, Size) ->
|
|
||||||
(I bsr Offset) band ones(Size).
|
|
||||||
|
|
||||||
%% @doc Generate a column family ID for the MQTT messages
|
|
||||||
-spec data_cf(emqx_ds_storage_layer:gen_id()) -> [char()].
|
|
||||||
data_cf(GenId) ->
|
|
||||||
?MODULE_STRING ++ integer_to_list(GenId).
|
|
||||||
|
|
||||||
make_refresh_counter({every, N}) when is_integer(N), N > 0 ->
|
|
||||||
{0, N};
|
|
||||||
make_refresh_counter(undefined) ->
|
|
||||||
undefined.
|
|
||||||
|
|
||||||
maybe_refresh_iterator(It = #it{refresh_counter = {N, N}}) ->
|
|
||||||
refresh_iterator(It#it{refresh_counter = {0, N}});
|
|
||||||
maybe_refresh_iterator(It = #it{refresh_counter = {M, N}}) ->
|
|
||||||
It#it{refresh_counter = {M + 1, N}};
|
|
||||||
maybe_refresh_iterator(It = #it{refresh_counter = undefined}) ->
|
|
||||||
It.
|
|
||||||
|
|
||||||
stop_iteration(It) ->
|
|
||||||
ok = rocksdb:iterator_close(It#it.handle),
|
|
||||||
none.
|
|
|
@ -0,0 +1,217 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%
|
||||||
|
%% Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
%% you may not use this file except in compliance with the License.
|
||||||
|
%% You may obtain a copy of the License at
|
||||||
|
%%
|
||||||
|
%% http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
%%
|
||||||
|
%% Unless required by applicable law or agreed to in writing, software
|
||||||
|
%% distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
%% See the License for the specific language governing permissions and
|
||||||
|
%% limitations under the License.
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
|
||||||
|
%% @doc Replication layer for DS backends that don't support
|
||||||
|
%% replication on their own.
|
||||||
|
-module(emqx_ds_replication_layer).
|
||||||
|
|
||||||
|
-export([
|
||||||
|
list_shards/1,
|
||||||
|
open_db/2,
|
||||||
|
drop_db/1,
|
||||||
|
store_batch/3,
|
||||||
|
get_streams/3,
|
||||||
|
make_iterator/3,
|
||||||
|
next/2
|
||||||
|
]).
|
||||||
|
|
||||||
|
%% internal exports:
|
||||||
|
-export([
|
||||||
|
do_open_shard_v1/2,
|
||||||
|
do_drop_shard_v1/1,
|
||||||
|
do_get_streams_v1/3,
|
||||||
|
do_make_iterator_v1/4,
|
||||||
|
do_next_v1/3
|
||||||
|
]).
|
||||||
|
|
||||||
|
-export_type([shard_id/0, stream/0, iterator/0, message_id/0]).
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Type declarations
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-type db() :: emqx_ds:db().
|
||||||
|
|
||||||
|
-type shard_id() :: {db(), atom()}.
|
||||||
|
|
||||||
|
%% This record enapsulates the stream entity from the replication
|
||||||
|
%% level.
|
||||||
|
%%
|
||||||
|
%% TODO: currently the stream is hardwired to only support the
|
||||||
|
%% internal rocksdb storage. In the future we want to add another
|
||||||
|
%% implementations for emqx_ds, so this type has to take this into
|
||||||
|
%% account.
|
||||||
|
-record(stream, {
|
||||||
|
shard :: emqx_ds_replication_layer:shard_id(),
|
||||||
|
enc :: emqx_ds_storage_layer:stream()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-opaque stream() :: #stream{}.
|
||||||
|
|
||||||
|
-record(iterator, {
|
||||||
|
shard :: emqx_ds_replication_layer:shard_id(),
|
||||||
|
enc :: enqx_ds_storage_layer:iterator()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-opaque iterator() :: #iterator{}.
|
||||||
|
|
||||||
|
-type message_id() :: emqx_ds_storage_layer:message_id().
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% API functions
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-spec list_shards(db()) -> [shard_id()].
|
||||||
|
list_shards(DB) ->
|
||||||
|
%% TODO: milestone 5
|
||||||
|
lists:map(
|
||||||
|
fun(Node) ->
|
||||||
|
shard_id(DB, Node)
|
||||||
|
end,
|
||||||
|
list_nodes()
|
||||||
|
).
|
||||||
|
|
||||||
|
-spec open_db(db(), emqx_ds:create_db_opts()) -> ok | {error, _}.
|
||||||
|
open_db(DB, Opts) ->
|
||||||
|
%% TODO: improve error reporting, don't just crash
|
||||||
|
lists:foreach(
|
||||||
|
fun(Node) ->
|
||||||
|
Shard = shard_id(DB, Node),
|
||||||
|
ok = emqx_ds_proto_v1:open_shard(Node, Shard, Opts)
|
||||||
|
end,
|
||||||
|
list_nodes()
|
||||||
|
).
|
||||||
|
|
||||||
|
-spec drop_db(db()) -> ok | {error, _}.
|
||||||
|
drop_db(DB) ->
|
||||||
|
lists:foreach(
|
||||||
|
fun(Node) ->
|
||||||
|
Shard = shard_id(DB, Node),
|
||||||
|
ok = emqx_ds_proto_v1:drop_shard(Node, Shard)
|
||||||
|
end,
|
||||||
|
list_nodes()
|
||||||
|
).
|
||||||
|
|
||||||
|
-spec store_batch(db(), [emqx_types:message()], emqx_ds:message_store_opts()) ->
|
||||||
|
emqx_ds:store_batch_result().
|
||||||
|
store_batch(DB, Msg, Opts) ->
|
||||||
|
%% TODO: Currently we store messages locally.
|
||||||
|
Shard = shard_id(DB, node()),
|
||||||
|
emqx_ds_storage_layer:store_batch(Shard, Msg, Opts).
|
||||||
|
|
||||||
|
-spec get_streams(db(), emqx_ds:topic_filter(), emqx_ds:time()) ->
|
||||||
|
[{emqx_ds:stream_rank(), stream()}].
|
||||||
|
get_streams(DB, TopicFilter, StartTime) ->
|
||||||
|
Shards = list_shards(DB),
|
||||||
|
lists:flatmap(
|
||||||
|
fun(Shard) ->
|
||||||
|
Node = node_of_shard(Shard),
|
||||||
|
Streams = emqx_ds_proto_v1:get_streams(Node, Shard, TopicFilter, StartTime),
|
||||||
|
lists:map(
|
||||||
|
fun({RankY, Stream}) ->
|
||||||
|
RankX = Shard,
|
||||||
|
Rank = {RankX, RankY},
|
||||||
|
{Rank, #stream{
|
||||||
|
shard = Shard,
|
||||||
|
enc = Stream
|
||||||
|
}}
|
||||||
|
end,
|
||||||
|
Streams
|
||||||
|
)
|
||||||
|
end,
|
||||||
|
Shards
|
||||||
|
).
|
||||||
|
|
||||||
|
-spec make_iterator(stream(), emqx_ds:topic_filter(), emqx_ds:time()) ->
|
||||||
|
emqx_ds:make_iterator_result(iterator()).
|
||||||
|
make_iterator(Stream, TopicFilter, StartTime) ->
|
||||||
|
#stream{shard = Shard, enc = StorageStream} = Stream,
|
||||||
|
Node = node_of_shard(Shard),
|
||||||
|
case emqx_ds_proto_v1:make_iterator(Node, Shard, StorageStream, TopicFilter, StartTime) of
|
||||||
|
{ok, Iter} ->
|
||||||
|
{ok, #iterator{shard = Shard, enc = Iter}};
|
||||||
|
Err = {error, _} ->
|
||||||
|
Err
|
||||||
|
end.
|
||||||
|
|
||||||
|
-spec next(iterator(), pos_integer()) -> emqx_ds:next_result(iterator()).
|
||||||
|
next(Iter0, BatchSize) ->
|
||||||
|
#iterator{shard = Shard, enc = StorageIter0} = Iter0,
|
||||||
|
Node = node_of_shard(Shard),
|
||||||
|
%% TODO: iterator can contain information that is useful for
|
||||||
|
%% reconstructing messages sent over the network. For example,
|
||||||
|
%% when we send messages with the learned topic index, we could
|
||||||
|
%% send the static part of topic once, and append it to the
|
||||||
|
%% messages on the receiving node, hence saving some network.
|
||||||
|
%%
|
||||||
|
%% This kind of trickery should be probably done here in the
|
||||||
|
%% replication layer. Or, perhaps, in the logic layer.
|
||||||
|
case emqx_ds_proto_v1:next(Node, Shard, StorageIter0, BatchSize) of
|
||||||
|
{ok, StorageIter, Batch} ->
|
||||||
|
Iter = #iterator{shard = Shard, enc = StorageIter},
|
||||||
|
{ok, Iter, Batch};
|
||||||
|
Other ->
|
||||||
|
Other
|
||||||
|
end.
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% behavior callbacks
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Internal exports (RPC targets)
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-spec do_open_shard_v1(shard_id(), emqx_ds:create_db_opts()) -> ok.
|
||||||
|
do_open_shard_v1(Shard, Opts) ->
|
||||||
|
emqx_ds_storage_layer:open_shard(Shard, Opts).
|
||||||
|
|
||||||
|
-spec do_drop_shard_v1(shard_id()) -> ok.
|
||||||
|
do_drop_shard_v1(Shard) ->
|
||||||
|
emqx_ds_storage_layer:drop_shard(Shard).
|
||||||
|
|
||||||
|
-spec do_get_streams_v1(shard_id(), emqx_ds:topic_filter(), emqx_ds:time()) ->
|
||||||
|
[{integer(), _Stream}].
|
||||||
|
do_get_streams_v1(Shard, TopicFilter, StartTime) ->
|
||||||
|
emqx_ds_storage_layer:get_streams(Shard, TopicFilter, StartTime).
|
||||||
|
|
||||||
|
-spec do_make_iterator_v1(
|
||||||
|
shard_id(), emqx_ds_storage_layer:stream(), emqx_ds:topic_filter(), emqx_ds:time()
|
||||||
|
) ->
|
||||||
|
{ok, iterator()} | {error, _}.
|
||||||
|
do_make_iterator_v1(Shard, Stream, TopicFilter, StartTime) ->
|
||||||
|
emqx_ds_storage_layer:make_iterator(Shard, Stream, TopicFilter, StartTime).
|
||||||
|
|
||||||
|
-spec do_next_v1(shard_id(), emqx_ds_storage_layer:iterator(), pos_integer()) ->
|
||||||
|
emqx_ds:next_result(emqx_ds_storage_layer:iterator()).
|
||||||
|
do_next_v1(Shard, Iter, BatchSize) ->
|
||||||
|
emqx_ds_storage_layer:next(Shard, Iter, BatchSize).
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Internal functions
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
shard_id(DB, Node) ->
|
||||||
|
%% TODO: don't bake node name into the schema, don't repeat the
|
||||||
|
%% Mnesia's 1M$ mistake.
|
||||||
|
{DB, Node}.
|
||||||
|
|
||||||
|
-spec node_of_shard(shard_id()) -> node().
|
||||||
|
node_of_shard({_DB, Node}) ->
|
||||||
|
Node.
|
||||||
|
|
||||||
|
list_nodes() ->
|
||||||
|
mria:running_nodes().
|
|
@ -0,0 +1,418 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%
|
||||||
|
%% Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
%% you may not use this file except in compliance with the License.
|
||||||
|
%% You may obtain a copy of the License at
|
||||||
|
%%
|
||||||
|
%% http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
%%
|
||||||
|
%% Unless required by applicable law or agreed to in writing, software
|
||||||
|
%% distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
%% See the License for the specific language governing permissions and
|
||||||
|
%% limitations under the License.
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
|
||||||
|
%% @doc A storage layout based on learned topic structure and using
|
||||||
|
%% bitfield mapping for the varying topic layers.
|
||||||
|
-module(emqx_ds_storage_bitfield_lts).
|
||||||
|
|
||||||
|
-behaviour(emqx_ds_storage_layer).
|
||||||
|
|
||||||
|
%% API:
|
||||||
|
-export([]).
|
||||||
|
|
||||||
|
%% behavior callbacks:
|
||||||
|
-export([create/4, open/5, store_batch/4, get_streams/4, make_iterator/5, next/4]).
|
||||||
|
|
||||||
|
%% internal exports:
|
||||||
|
-export([format_key/2]).
|
||||||
|
|
||||||
|
-export_type([options/0]).
|
||||||
|
|
||||||
|
-include_lib("emqx_utils/include/emqx_message.hrl").
|
||||||
|
-include_lib("snabbkaffe/include/trace.hrl").
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Type declarations
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-type options() ::
|
||||||
|
#{
|
||||||
|
bits_per_wildcard_level => pos_integer(),
|
||||||
|
topic_index_bytes => pos_integer(),
|
||||||
|
epoch_bits => non_neg_integer()
|
||||||
|
}.
|
||||||
|
|
||||||
|
%% Permanent state:
|
||||||
|
-type schema() ::
|
||||||
|
#{
|
||||||
|
bits_per_wildcard_level := pos_integer(),
|
||||||
|
topic_index_bytes := pos_integer(),
|
||||||
|
ts_bits := non_neg_integer(),
|
||||||
|
ts_offset_bits := non_neg_integer()
|
||||||
|
}.
|
||||||
|
|
||||||
|
%% Runtime state:
|
||||||
|
-record(s, {
|
||||||
|
db :: rocksdb:db_handle(),
|
||||||
|
data :: rocksdb:cf_handle(),
|
||||||
|
trie :: emqx_ds_lts:trie(),
|
||||||
|
keymappers :: array:array(emqx_ds_bitmask_keymapper:keymapper()),
|
||||||
|
ts_offset :: non_neg_integer()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-type s() :: #s{}.
|
||||||
|
|
||||||
|
-record(stream, {
|
||||||
|
storage_key :: emqx_ds_lts:msg_storage_key()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-record(it, {
|
||||||
|
topic_filter :: emqx_ds:topic_filter(),
|
||||||
|
start_time :: emqx_ds:time(),
|
||||||
|
storage_key :: emqx_ds_lts:msg_storage_key(),
|
||||||
|
last_seen_key = <<>> :: binary()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-type iterator() :: #it{}.
|
||||||
|
|
||||||
|
-define(COUNTER, emqx_ds_storage_bitfield_lts_counter).
|
||||||
|
|
||||||
|
%% Limit on the number of wildcard levels in the learned topic trie:
|
||||||
|
-define(WILDCARD_LIMIT, 10).
|
||||||
|
|
||||||
|
-include("emqx_ds_bitmask.hrl").
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% API funcions
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% behavior callbacks
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-spec create(
|
||||||
|
emqx_ds_replication_layer:shard_id(),
|
||||||
|
rocksdb:db_handle(),
|
||||||
|
emqx_ds_storage_layer:gen_id(),
|
||||||
|
options()
|
||||||
|
) ->
|
||||||
|
{schema(), emqx_ds_storage_layer:cf_refs()}.
|
||||||
|
create(_ShardId, DBHandle, GenId, Options) ->
|
||||||
|
%% Get options:
|
||||||
|
BitsPerTopicLevel = maps:get(bits_per_wildcard_level, Options, 64),
|
||||||
|
TopicIndexBytes = maps:get(topic_index_bytes, Options, 4),
|
||||||
|
%% 10 bits -> 1024 ms -> ~1 sec
|
||||||
|
TSOffsetBits = maps:get(epoch_bits, Options, 10),
|
||||||
|
%% Create column families:
|
||||||
|
DataCFName = data_cf(GenId),
|
||||||
|
TrieCFName = trie_cf(GenId),
|
||||||
|
{ok, DataCFHandle} = rocksdb:create_column_family(DBHandle, DataCFName, []),
|
||||||
|
{ok, TrieCFHandle} = rocksdb:create_column_family(DBHandle, TrieCFName, []),
|
||||||
|
%% Create schema:
|
||||||
|
Schema = #{
|
||||||
|
bits_per_wildcard_level => BitsPerTopicLevel,
|
||||||
|
topic_index_bytes => TopicIndexBytes,
|
||||||
|
ts_bits => 64,
|
||||||
|
ts_offset_bits => TSOffsetBits
|
||||||
|
},
|
||||||
|
{Schema, [{DataCFName, DataCFHandle}, {TrieCFName, TrieCFHandle}]}.
|
||||||
|
|
||||||
|
-spec open(
|
||||||
|
emqx_ds_replication_layer:shard_id(),
|
||||||
|
rocksdb:db_handle(),
|
||||||
|
emqx_ds_storage_layer:gen_id(),
|
||||||
|
emqx_ds_storage_layer:cf_refs(),
|
||||||
|
schema()
|
||||||
|
) ->
|
||||||
|
s().
|
||||||
|
open(_Shard, DBHandle, GenId, CFRefs, Schema) ->
|
||||||
|
#{
|
||||||
|
bits_per_wildcard_level := BitsPerTopicLevel,
|
||||||
|
topic_index_bytes := TopicIndexBytes,
|
||||||
|
ts_bits := TSBits,
|
||||||
|
ts_offset_bits := TSOffsetBits
|
||||||
|
} = Schema,
|
||||||
|
{_, DataCF} = lists:keyfind(data_cf(GenId), 1, CFRefs),
|
||||||
|
{_, TrieCF} = lists:keyfind(trie_cf(GenId), 1, CFRefs),
|
||||||
|
Trie = restore_trie(TopicIndexBytes, DBHandle, TrieCF),
|
||||||
|
%% If user's topics have more than learned 10 wildcard levels
|
||||||
|
%% (more than 2, really), then it's total carnage; learned topic
|
||||||
|
%% structure won't help.
|
||||||
|
MaxWildcardLevels = ?WILDCARD_LIMIT,
|
||||||
|
KeymapperCache = array:from_list(
|
||||||
|
[
|
||||||
|
make_keymapper(TopicIndexBytes, BitsPerTopicLevel, TSBits, TSOffsetBits, N)
|
||||||
|
|| N <- lists:seq(0, MaxWildcardLevels)
|
||||||
|
]
|
||||||
|
),
|
||||||
|
#s{
|
||||||
|
db = DBHandle,
|
||||||
|
data = DataCF,
|
||||||
|
trie = Trie,
|
||||||
|
keymappers = KeymapperCache,
|
||||||
|
ts_offset = TSOffsetBits
|
||||||
|
}.
|
||||||
|
|
||||||
|
-spec store_batch(
|
||||||
|
emqx_ds_replication_layer:shard_id(), s(), [emqx_types:message()], emqx_ds:message_store_opts()
|
||||||
|
) ->
|
||||||
|
emqx_ds:store_batch_result().
|
||||||
|
store_batch(_ShardId, S = #s{db = DB, data = Data}, Messages, _Options) ->
|
||||||
|
lists:foreach(
|
||||||
|
fun(Msg) ->
|
||||||
|
{Key, _} = make_key(S, Msg),
|
||||||
|
Val = serialize(Msg),
|
||||||
|
rocksdb:put(DB, Data, Key, Val, [])
|
||||||
|
end,
|
||||||
|
Messages
|
||||||
|
).
|
||||||
|
|
||||||
|
get_streams(_Shard, #s{trie = Trie}, TopicFilter, _StartTime) ->
|
||||||
|
Indexes = emqx_ds_lts:match_topics(Trie, TopicFilter),
|
||||||
|
[#stream{storage_key = I} || I <- Indexes].
|
||||||
|
|
||||||
|
make_iterator(_Shard, _Data, #stream{storage_key = StorageKey}, TopicFilter, StartTime) ->
|
||||||
|
%% Note: it's a good idea to keep the iterator structure lean,
|
||||||
|
%% since it can be stored on a remote node that could update its
|
||||||
|
%% code independently from us.
|
||||||
|
{ok, #it{
|
||||||
|
topic_filter = TopicFilter,
|
||||||
|
start_time = StartTime,
|
||||||
|
storage_key = StorageKey
|
||||||
|
}}.
|
||||||
|
|
||||||
|
next(_Shard, Schema = #s{ts_offset = TSOffset}, It, BatchSize) ->
|
||||||
|
%% Compute safe cutoff time.
|
||||||
|
%% It's the point in time where the last complete epoch ends, so we need to know
|
||||||
|
%% the current time to compute it.
|
||||||
|
Now = emqx_message:timestamp_now(),
|
||||||
|
SafeCutoffTime = (Now bsr TSOffset) bsl TSOffset,
|
||||||
|
next_until(Schema, It, SafeCutoffTime, BatchSize).
|
||||||
|
|
||||||
|
next_until(_Schema, It, SafeCutoffTime, _BatchSize) when It#it.start_time >= SafeCutoffTime ->
|
||||||
|
%% We're in the middle of the current epoch, so we can't yet iterate over it.
|
||||||
|
%% It would be unsafe otherwise: messages can be stored in the current epoch
|
||||||
|
%% concurrently with iterating over it. They can end up earlier (in the iteration
|
||||||
|
%% order) due to the nature of keymapping, potentially causing us to miss them.
|
||||||
|
{ok, It, []};
|
||||||
|
next_until(#s{db = DB, data = CF, keymappers = Keymappers}, It, SafeCutoffTime, BatchSize) ->
|
||||||
|
#it{
|
||||||
|
start_time = StartTime,
|
||||||
|
storage_key = {TopicIndex, Varying}
|
||||||
|
} = It,
|
||||||
|
%% Make filter:
|
||||||
|
Inequations = [
|
||||||
|
{'=', TopicIndex},
|
||||||
|
{StartTime, '..', SafeCutoffTime - 1},
|
||||||
|
%% Unique integer:
|
||||||
|
any
|
||||||
|
%% Varying topic levels:
|
||||||
|
| lists:map(
|
||||||
|
fun
|
||||||
|
('+') ->
|
||||||
|
any;
|
||||||
|
(TopicLevel) when is_binary(TopicLevel) ->
|
||||||
|
{'=', hash_topic_level(TopicLevel)}
|
||||||
|
end,
|
||||||
|
Varying
|
||||||
|
)
|
||||||
|
],
|
||||||
|
%% Obtain a keymapper for the current number of varying levels.
|
||||||
|
NVarying = length(Varying),
|
||||||
|
%% Assert:
|
||||||
|
NVarying =< ?WILDCARD_LIMIT orelse
|
||||||
|
error({too_many_varying_topic_levels, NVarying}),
|
||||||
|
Keymapper = array:get(NVarying, Keymappers),
|
||||||
|
Filter =
|
||||||
|
#filter{range_min = LowerBound, range_max = UpperBound} = emqx_ds_bitmask_keymapper:make_filter(
|
||||||
|
Keymapper, Inequations
|
||||||
|
),
|
||||||
|
{ok, ITHandle} = rocksdb:iterator(DB, CF, [
|
||||||
|
{iterate_lower_bound, emqx_ds_bitmask_keymapper:key_to_bitstring(Keymapper, LowerBound)},
|
||||||
|
{iterate_upper_bound, emqx_ds_bitmask_keymapper:key_to_bitstring(Keymapper, UpperBound + 1)}
|
||||||
|
]),
|
||||||
|
try
|
||||||
|
put(?COUNTER, 0),
|
||||||
|
next_loop(ITHandle, Keymapper, Filter, SafeCutoffTime, It, [], BatchSize)
|
||||||
|
after
|
||||||
|
rocksdb:iterator_close(ITHandle),
|
||||||
|
erase(?COUNTER)
|
||||||
|
end.
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Internal functions
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
next_loop(_ITHandle, _KeyMapper, _Filter, _Cutoff, It, Acc, 0) ->
|
||||||
|
{ok, It, lists:reverse(Acc)};
|
||||||
|
next_loop(ITHandle, KeyMapper, Filter, Cutoff, It0, Acc0, N0) ->
|
||||||
|
inc_counter(),
|
||||||
|
#it{last_seen_key = Key0} = It0,
|
||||||
|
case emqx_ds_bitmask_keymapper:bin_increment(Filter, Key0) of
|
||||||
|
overflow ->
|
||||||
|
{ok, It0, lists:reverse(Acc0)};
|
||||||
|
Key1 ->
|
||||||
|
%% assert
|
||||||
|
true = Key1 > Key0,
|
||||||
|
case rocksdb:iterator_move(ITHandle, {seek, Key1}) of
|
||||||
|
{ok, Key, Val} ->
|
||||||
|
{N, It, Acc} =
|
||||||
|
traverse_interval(ITHandle, Filter, Cutoff, Key, Val, It0, Acc0, N0),
|
||||||
|
next_loop(ITHandle, KeyMapper, Filter, Cutoff, It, Acc, N);
|
||||||
|
{error, invalid_iterator} ->
|
||||||
|
{ok, It0, lists:reverse(Acc0)}
|
||||||
|
end
|
||||||
|
end.
|
||||||
|
|
||||||
|
traverse_interval(ITHandle, Filter, Cutoff, Key, Val, It0, Acc0, N) ->
|
||||||
|
It = It0#it{last_seen_key = Key},
|
||||||
|
case emqx_ds_bitmask_keymapper:bin_checkmask(Filter, Key) of
|
||||||
|
true ->
|
||||||
|
Msg = deserialize(Val),
|
||||||
|
case check_message(Cutoff, It, Msg) of
|
||||||
|
true ->
|
||||||
|
Acc = [Msg | Acc0],
|
||||||
|
traverse_interval(ITHandle, Filter, Cutoff, It, Acc, N - 1);
|
||||||
|
false ->
|
||||||
|
traverse_interval(ITHandle, Filter, Cutoff, It, Acc0, N);
|
||||||
|
overflow ->
|
||||||
|
{0, It0, Acc0}
|
||||||
|
end;
|
||||||
|
false ->
|
||||||
|
{N, It, Acc0}
|
||||||
|
end.
|
||||||
|
|
||||||
|
traverse_interval(_ITHandle, _Filter, _Cutoff, It, Acc, 0) ->
|
||||||
|
{0, It, Acc};
|
||||||
|
traverse_interval(ITHandle, Filter, Cutoff, It, Acc, N) ->
|
||||||
|
inc_counter(),
|
||||||
|
case rocksdb:iterator_move(ITHandle, next) of
|
||||||
|
{ok, Key, Val} ->
|
||||||
|
traverse_interval(ITHandle, Filter, Cutoff, Key, Val, It, Acc, N);
|
||||||
|
{error, invalid_iterator} ->
|
||||||
|
{0, It, Acc}
|
||||||
|
end.
|
||||||
|
|
||||||
|
-spec check_message(emqx_ds:time(), iterator(), emqx_types:message()) ->
|
||||||
|
true | false | overflow.
|
||||||
|
check_message(
|
||||||
|
Cutoff,
|
||||||
|
_It,
|
||||||
|
#message{timestamp = Timestamp}
|
||||||
|
) when Timestamp >= Cutoff ->
|
||||||
|
%% We hit the current epoch, we can't continue iterating over it yet.
|
||||||
|
%% It would be unsafe otherwise: messages can be stored in the current epoch
|
||||||
|
%% concurrently with iterating over it. They can end up earlier (in the iteration
|
||||||
|
%% order) due to the nature of keymapping, potentially causing us to miss them.
|
||||||
|
overflow;
|
||||||
|
check_message(
|
||||||
|
_Cutoff,
|
||||||
|
#it{start_time = StartTime, topic_filter = TopicFilter},
|
||||||
|
#message{timestamp = Timestamp, topic = Topic}
|
||||||
|
) when Timestamp >= StartTime ->
|
||||||
|
emqx_topic:match(emqx_topic:words(Topic), TopicFilter);
|
||||||
|
check_message(_Cutoff, _It, _Msg) ->
|
||||||
|
false.
|
||||||
|
|
||||||
|
format_key(KeyMapper, Key) ->
|
||||||
|
Vec = [integer_to_list(I, 16) || I <- emqx_ds_bitmask_keymapper:key_to_vector(KeyMapper, Key)],
|
||||||
|
lists:flatten(io_lib:format("~.16B (~s)", [Key, string:join(Vec, ",")])).
|
||||||
|
|
||||||
|
-spec make_key(s(), emqx_types:message()) -> {binary(), [binary()]}.
|
||||||
|
make_key(#s{keymappers = KeyMappers, trie = Trie}, #message{timestamp = Timestamp, topic = TopicBin}) ->
|
||||||
|
Tokens = emqx_topic:tokens(TopicBin),
|
||||||
|
{TopicIndex, Varying} = emqx_ds_lts:topic_key(Trie, fun threshold_fun/1, Tokens),
|
||||||
|
VaryingHashes = [hash_topic_level(I) || I <- Varying],
|
||||||
|
KeyMapper = array:get(length(Varying), KeyMappers),
|
||||||
|
KeyBin = make_key(KeyMapper, TopicIndex, Timestamp, VaryingHashes),
|
||||||
|
{KeyBin, Varying}.
|
||||||
|
|
||||||
|
-spec make_key(emqx_ds_bitmask_keymapper:keymapper(), emqx_ds_lts:static_key(), emqx_ds:time(), [
|
||||||
|
non_neg_integer()
|
||||||
|
]) ->
|
||||||
|
binary().
|
||||||
|
make_key(KeyMapper, TopicIndex, Timestamp, Varying) ->
|
||||||
|
UniqueInteger = erlang:unique_integer([monotonic, positive]),
|
||||||
|
emqx_ds_bitmask_keymapper:key_to_bitstring(
|
||||||
|
KeyMapper,
|
||||||
|
emqx_ds_bitmask_keymapper:vector_to_key(KeyMapper, [
|
||||||
|
TopicIndex, Timestamp, UniqueInteger | Varying
|
||||||
|
])
|
||||||
|
).
|
||||||
|
|
||||||
|
%% TODO: don't hardcode the thresholds
|
||||||
|
threshold_fun(0) ->
|
||||||
|
100;
|
||||||
|
threshold_fun(_) ->
|
||||||
|
20.
|
||||||
|
|
||||||
|
hash_topic_level(TopicLevel) ->
|
||||||
|
<<Int:64, _/binary>> = erlang:md5(TopicLevel),
|
||||||
|
Int.
|
||||||
|
|
||||||
|
serialize(Msg) ->
|
||||||
|
term_to_binary(Msg).
|
||||||
|
|
||||||
|
deserialize(Blob) ->
|
||||||
|
binary_to_term(Blob).
|
||||||
|
|
||||||
|
-define(BYTE_SIZE, 8).
|
||||||
|
|
||||||
|
%% erlfmt-ignore
|
||||||
|
make_keymapper(TopicIndexBytes, BitsPerTopicLevel, TSBits, TSOffsetBits, N) ->
|
||||||
|
Bitsources =
|
||||||
|
%% Dimension Offset Bitsize
|
||||||
|
[{1, 0, TopicIndexBytes * ?BYTE_SIZE}, %% Topic index
|
||||||
|
{2, TSOffsetBits, TSBits - TSOffsetBits }] ++ %% Timestamp epoch
|
||||||
|
[{3 + I, 0, BitsPerTopicLevel } %% Varying topic levels
|
||||||
|
|| I <- lists:seq(1, N)] ++
|
||||||
|
[{2, 0, TSOffsetBits }, %% Timestamp offset
|
||||||
|
{3, 0, 64 }], %% Unique integer
|
||||||
|
Keymapper = emqx_ds_bitmask_keymapper:make_keymapper(lists:reverse(Bitsources)),
|
||||||
|
%% Assert:
|
||||||
|
case emqx_ds_bitmask_keymapper:bitsize(Keymapper) rem 8 of
|
||||||
|
0 ->
|
||||||
|
ok;
|
||||||
|
_ ->
|
||||||
|
error(#{'$msg' => "Non-even key size", bitsources => Bitsources})
|
||||||
|
end,
|
||||||
|
Keymapper.
|
||||||
|
|
||||||
|
-spec restore_trie(pos_integer(), rocksdb:db_handle(), rocksdb:cf_handle()) -> emqx_ds_lts:trie().
|
||||||
|
restore_trie(TopicIndexBytes, DB, CF) ->
|
||||||
|
PersistCallback = fun(Key, Val) ->
|
||||||
|
rocksdb:put(DB, CF, term_to_binary(Key), term_to_binary(Val), [])
|
||||||
|
end,
|
||||||
|
{ok, IT} = rocksdb:iterator(DB, CF, []),
|
||||||
|
try
|
||||||
|
Dump = read_persisted_trie(IT, rocksdb:iterator_move(IT, first)),
|
||||||
|
TrieOpts = #{persist_callback => PersistCallback, static_key_size => TopicIndexBytes},
|
||||||
|
emqx_ds_lts:trie_restore(TrieOpts, Dump)
|
||||||
|
after
|
||||||
|
rocksdb:iterator_close(IT)
|
||||||
|
end.
|
||||||
|
|
||||||
|
read_persisted_trie(IT, {ok, KeyB, ValB}) ->
|
||||||
|
[
|
||||||
|
{binary_to_term(KeyB), binary_to_term(ValB)}
|
||||||
|
| read_persisted_trie(IT, rocksdb:iterator_move(IT, next))
|
||||||
|
];
|
||||||
|
read_persisted_trie(_IT, {error, invalid_iterator}) ->
|
||||||
|
[].
|
||||||
|
|
||||||
|
inc_counter() ->
|
||||||
|
N = get(?COUNTER),
|
||||||
|
put(?COUNTER, N + 1).
|
||||||
|
|
||||||
|
%% @doc Generate a column family ID for the MQTT messages
|
||||||
|
-spec data_cf(emqx_ds_storage_layer:gen_id()) -> [char()].
|
||||||
|
data_cf(GenId) ->
|
||||||
|
"emqx_ds_storage_bitfield_lts_data" ++ integer_to_list(GenId).
|
||||||
|
|
||||||
|
%% @doc Generate a column family ID for the trie
|
||||||
|
-spec trie_cf(emqx_ds_storage_layer:gen_id()) -> [char()].
|
||||||
|
trie_cf(GenId) ->
|
||||||
|
"emqx_ds_storage_bitfield_lts_trie" ++ integer_to_list(GenId).
|
|
@ -1,277 +1,255 @@
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
%% Copyright (c) 2022-2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
%% Copyright (c) 2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%
|
||||||
|
%% Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
%% you may not use this file except in compliance with the License.
|
||||||
|
%% You may obtain a copy of the License at
|
||||||
|
%%
|
||||||
|
%% http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
%%
|
||||||
|
%% Unless required by applicable law or agreed to in writing, software
|
||||||
|
%% distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
%% See the License for the specific language governing permissions and
|
||||||
|
%% limitations under the License.
|
||||||
%%--------------------------------------------------------------------
|
%%--------------------------------------------------------------------
|
||||||
-module(emqx_ds_storage_layer).
|
-module(emqx_ds_storage_layer).
|
||||||
|
|
||||||
-behaviour(gen_server).
|
-behaviour(gen_server).
|
||||||
|
|
||||||
%% API:
|
%% Replication layer API:
|
||||||
-export([start_link/2]).
|
-export([open_shard/2, drop_shard/1, store_batch/3, get_streams/3, make_iterator/4, next/3]).
|
||||||
-export([create_generation/3]).
|
|
||||||
|
|
||||||
-export([store/5]).
|
%% gen_server
|
||||||
-export([delete/4]).
|
-export([start_link/2, init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2]).
|
||||||
|
|
||||||
-export([make_iterator/2, next/1]).
|
%% internal exports:
|
||||||
|
-export([db_dir/1]).
|
||||||
|
|
||||||
-export([
|
-export_type([gen_id/0, generation/0, cf_refs/0, stream/0, iterator/0]).
|
||||||
preserve_iterator/2,
|
|
||||||
restore_iterator/2,
|
|
||||||
discard_iterator/2,
|
|
||||||
ensure_iterator/3,
|
|
||||||
discard_iterator_prefix/2,
|
|
||||||
list_iterator_prefix/2,
|
|
||||||
foldl_iterator_prefix/4
|
|
||||||
]).
|
|
||||||
|
|
||||||
%% behaviour callbacks:
|
-include_lib("snabbkaffe/include/snabbkaffe.hrl").
|
||||||
-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2]).
|
|
||||||
|
|
||||||
-export_type([cf_refs/0, gen_id/0, options/0, state/0, iterator/0]).
|
|
||||||
-export_type([db_options/0, db_write_options/0, db_read_options/0]).
|
|
||||||
|
|
||||||
-compile({inline, [meta_lookup/2]}).
|
|
||||||
|
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
%% Type declarations
|
%% Type declarations
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
||||||
-type options() :: #{
|
-type prototype() ::
|
||||||
dir => file:filename()
|
{emqx_ds_storage_reference, emqx_ds_storage_reference:options()}
|
||||||
}.
|
| {emqx_ds_storage_bitfield_lts, emqx_ds_storage_bitfield_lts:options()}.
|
||||||
|
|
||||||
%% see rocksdb:db_options()
|
-type shard_id() :: emqx_ds_replication_layer:shard_id().
|
||||||
-type db_options() :: proplists:proplist().
|
|
||||||
%% see rocksdb:write_options()
|
|
||||||
-type db_write_options() :: proplists:proplist().
|
|
||||||
%% see rocksdb:read_options()
|
|
||||||
-type db_read_options() :: proplists:proplist().
|
|
||||||
|
|
||||||
-type cf_refs() :: [{string(), rocksdb:cf_handle()}].
|
-type cf_refs() :: [{string(), rocksdb:cf_handle()}].
|
||||||
|
|
||||||
%% Message storage generation
|
-type gen_id() :: 0..16#ffff.
|
||||||
%% Keep in mind that instances of this type are persisted in long-term storage.
|
|
||||||
-type generation() :: #{
|
%% Note: this record might be stored permanently on a remote node.
|
||||||
%% Module that handles data for the generation
|
-record(stream, {
|
||||||
|
generation :: gen_id(),
|
||||||
|
enc :: _EncapsulatedData,
|
||||||
|
misc = #{} :: map()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-opaque stream() :: #stream{}.
|
||||||
|
|
||||||
|
%% Note: this record might be stored permanently on a remote node.
|
||||||
|
-record(it, {
|
||||||
|
generation :: gen_id(),
|
||||||
|
enc :: _EncapsulatedData,
|
||||||
|
misc = #{} :: map()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-opaque iterator() :: #it{}.
|
||||||
|
|
||||||
|
%%%% Generation:
|
||||||
|
|
||||||
|
-type generation(Data) :: #{
|
||||||
|
%% Module that handles data for the generation:
|
||||||
module := module(),
|
module := module(),
|
||||||
%% Module-specific data defined at generation creation time
|
%% Module-specific data defined at generation creation time:
|
||||||
data := term(),
|
data := Data,
|
||||||
%% When should this generation become active?
|
%% When should this generation become active?
|
||||||
%% This generation should only contain messages timestamped no earlier than that.
|
%% This generation should only contain messages timestamped no earlier than that.
|
||||||
%% The very first generation will have `since` equal 0.
|
%% The very first generation will have `since` equal 0.
|
||||||
since := emqx_ds:time()
|
since := emqx_ds:time(),
|
||||||
|
until := emqx_ds:time() | undefined
|
||||||
}.
|
}.
|
||||||
|
|
||||||
|
%% Schema for a generation. Persistent term.
|
||||||
|
-type generation_schema() :: generation(term()).
|
||||||
|
|
||||||
|
%% Runtime view of generation:
|
||||||
|
-type generation() :: generation(term()).
|
||||||
|
|
||||||
|
%%%% Shard:
|
||||||
|
|
||||||
|
-type shard(GenData) :: #{
|
||||||
|
%% ID of the current generation (where the new data is written):
|
||||||
|
current_generation := gen_id(),
|
||||||
|
%% This data is used to create new generation:
|
||||||
|
prototype := prototype(),
|
||||||
|
%% Generations:
|
||||||
|
{generation, gen_id()} => GenData
|
||||||
|
}.
|
||||||
|
|
||||||
|
%% Shard schema (persistent):
|
||||||
|
-type shard_schema() :: shard(generation_schema()).
|
||||||
|
|
||||||
|
%% Shard (runtime):
|
||||||
|
-type shard() :: shard(generation()).
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Generation callbacks
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
%% Create the new schema given generation id and the options.
|
||||||
|
%% Create rocksdb column families.
|
||||||
|
-callback create(shard_id(), rocksdb:db_handle(), gen_id(), _Options) ->
|
||||||
|
{_Schema, cf_refs()}.
|
||||||
|
|
||||||
|
%% Open the existing schema
|
||||||
|
-callback open(shard_id(), rocsdb:db_handle(), gen_id(), cf_refs(), _Schema) ->
|
||||||
|
_Data.
|
||||||
|
|
||||||
|
-callback store_batch(shard_id(), _Data, [emqx_types:message()], emqx_ds:message_store_opts()) ->
|
||||||
|
emqx_ds:store_batch_result().
|
||||||
|
|
||||||
|
-callback get_streams(shard_id(), _Data, emqx_ds:topic_filter(), emqx_ds:time()) ->
|
||||||
|
[_Stream].
|
||||||
|
|
||||||
|
-callback make_iterator(shard_id(), _Data, _Stream, emqx_ds:topic_filter(), emqx_ds:time()) ->
|
||||||
|
emqx_ds:make_iterator_result(_Iterator).
|
||||||
|
|
||||||
|
-callback next(shard_id(), _Data, Iter, pos_integer()) ->
|
||||||
|
{ok, Iter, [emqx_types:message()]} | {error, _}.
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% API for the replication layer
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-spec open_shard(shard_id(), emqx_ds:builtin_db_opts()) -> ok.
|
||||||
|
open_shard(Shard, Options) ->
|
||||||
|
emqx_ds_storage_layer_sup:ensure_shard(Shard, Options).
|
||||||
|
|
||||||
|
-spec drop_shard(shard_id()) -> ok.
|
||||||
|
drop_shard(Shard) ->
|
||||||
|
catch emqx_ds_storage_layer_sup:stop_shard(Shard),
|
||||||
|
ok = rocksdb:destroy(db_dir(Shard), []).
|
||||||
|
|
||||||
|
-spec store_batch(shard_id(), [emqx_types:message()], emqx_ds:message_store_opts()) ->
|
||||||
|
emqx_ds:store_batch_result().
|
||||||
|
store_batch(Shard, Messages, Options) ->
|
||||||
|
%% We always store messages in the current generation:
|
||||||
|
GenId = generation_current(Shard),
|
||||||
|
#{module := Mod, data := GenData} = generation_get(Shard, GenId),
|
||||||
|
Mod:store_batch(Shard, GenData, Messages, Options).
|
||||||
|
|
||||||
|
-spec get_streams(shard_id(), emqx_ds:topic_filter(), emqx_ds:time()) ->
|
||||||
|
[{integer(), stream()}].
|
||||||
|
get_streams(Shard, TopicFilter, StartTime) ->
|
||||||
|
Gens = generations_since(Shard, StartTime),
|
||||||
|
lists:flatmap(
|
||||||
|
fun(GenId) ->
|
||||||
|
#{module := Mod, data := GenData} = generation_get(Shard, GenId),
|
||||||
|
Streams = Mod:get_streams(Shard, GenData, TopicFilter, StartTime),
|
||||||
|
[
|
||||||
|
{GenId, #stream{
|
||||||
|
generation = GenId,
|
||||||
|
enc = Stream
|
||||||
|
}}
|
||||||
|
|| Stream <- Streams
|
||||||
|
]
|
||||||
|
end,
|
||||||
|
Gens
|
||||||
|
).
|
||||||
|
|
||||||
|
-spec make_iterator(shard_id(), stream(), emqx_ds:topic_filter(), emqx_ds:time()) ->
|
||||||
|
emqx_ds:make_iterator_result(iterator()).
|
||||||
|
make_iterator(Shard, #stream{generation = GenId, enc = Stream}, TopicFilter, StartTime) ->
|
||||||
|
#{module := Mod, data := GenData} = generation_get(Shard, GenId),
|
||||||
|
case Mod:make_iterator(Shard, GenData, Stream, TopicFilter, StartTime) of
|
||||||
|
{ok, Iter} ->
|
||||||
|
{ok, #it{
|
||||||
|
generation = GenId,
|
||||||
|
enc = Iter
|
||||||
|
}};
|
||||||
|
{error, _} = Err ->
|
||||||
|
Err
|
||||||
|
end.
|
||||||
|
|
||||||
|
-spec next(shard_id(), iterator(), pos_integer()) ->
|
||||||
|
emqx_ds:next_result(iterator()).
|
||||||
|
next(Shard, Iter = #it{generation = GenId, enc = GenIter0}, BatchSize) ->
|
||||||
|
#{module := Mod, data := GenData} = generation_get(Shard, GenId),
|
||||||
|
Current = generation_current(Shard),
|
||||||
|
case Mod:next(Shard, GenData, GenIter0, BatchSize) of
|
||||||
|
{ok, _GenIter, []} when GenId < Current ->
|
||||||
|
%% This is a past generation. Storage layer won't write
|
||||||
|
%% any more messages here. The iterator reached the end:
|
||||||
|
%% the stream has been fully replayed.
|
||||||
|
{ok, end_of_stream};
|
||||||
|
{ok, GenIter, Batch} ->
|
||||||
|
{ok, Iter#it{enc = GenIter}, Batch};
|
||||||
|
Error = {error, _} ->
|
||||||
|
Error
|
||||||
|
end.
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% gen_server for the shard
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-define(REF(ShardId), {via, gproc, {n, l, {?MODULE, ShardId}}}).
|
||||||
|
|
||||||
|
-spec start_link(shard_id(), emqx_ds:builtin_db_opts()) ->
|
||||||
|
{ok, pid()}.
|
||||||
|
start_link(Shard, Options) ->
|
||||||
|
gen_server:start_link(?REF(Shard), ?MODULE, {Shard, Options}, []).
|
||||||
|
|
||||||
-record(s, {
|
-record(s, {
|
||||||
shard :: emqx_ds:shard(),
|
shard_id :: emqx_ds:shard_id(),
|
||||||
db :: rocksdb:db_handle(),
|
db :: rocksdb:db_handle(),
|
||||||
cf_iterator :: rocksdb:cf_handle(),
|
cf_refs :: cf_refs(),
|
||||||
cf_generations :: cf_refs()
|
schema :: shard_schema(),
|
||||||
|
shard :: shard()
|
||||||
}).
|
}).
|
||||||
|
|
||||||
-record(it, {
|
%% Note: we specify gen_server requests as records to make use of Dialyzer:
|
||||||
shard :: emqx_ds:shard(),
|
-record(call_create_generation, {since :: emqx_ds:time()}).
|
||||||
gen :: gen_id(),
|
|
||||||
replay :: emqx_ds:replay(),
|
|
||||||
module :: module(),
|
|
||||||
data :: term()
|
|
||||||
}).
|
|
||||||
|
|
||||||
-type gen_id() :: 0..16#ffff.
|
-type server_state() :: #s{}.
|
||||||
|
|
||||||
-opaque state() :: #s{}.
|
|
||||||
-opaque iterator() :: #it{}.
|
|
||||||
|
|
||||||
%% Contents of the default column family:
|
|
||||||
%%
|
|
||||||
%% [{<<"genNN">>, #generation{}}, ...,
|
|
||||||
%% {<<"current">>, GenID}]
|
|
||||||
|
|
||||||
-define(DEFAULT_CF, "default").
|
-define(DEFAULT_CF, "default").
|
||||||
-define(DEFAULT_CF_OPTS, []).
|
-define(DEFAULT_CF_OPTS, []).
|
||||||
|
|
||||||
-define(ITERATOR_CF, "$iterators").
|
init({ShardId, Options}) ->
|
||||||
|
|
||||||
%% TODO
|
|
||||||
%% 1. CuckooTable might be of use here / `OptimizeForPointLookup(...)`.
|
|
||||||
%% 2. Supposedly might be compressed _very_ effectively.
|
|
||||||
%% 3. `inplace_update_support`?
|
|
||||||
-define(ITERATOR_CF_OPTS, []).
|
|
||||||
|
|
||||||
-define(REF(Keyspace, ShardId), {via, gproc, {n, l, {?MODULE, Keyspace, ShardId}}}).
|
|
||||||
|
|
||||||
%%================================================================================
|
|
||||||
%% Callbacks
|
|
||||||
%%================================================================================
|
|
||||||
|
|
||||||
-callback create_new(rocksdb:db_handle(), gen_id(), _Options :: term()) ->
|
|
||||||
{_Schema, cf_refs()}.
|
|
||||||
|
|
||||||
-callback open(
|
|
||||||
emqx_ds:shard(),
|
|
||||||
rocksdb:db_handle(),
|
|
||||||
gen_id(),
|
|
||||||
cf_refs(),
|
|
||||||
_Schema
|
|
||||||
) ->
|
|
||||||
term().
|
|
||||||
|
|
||||||
-callback store(
|
|
||||||
_Schema,
|
|
||||||
_MessageID :: binary(),
|
|
||||||
emqx_ds:time(),
|
|
||||||
emqx_ds:topic(),
|
|
||||||
_Payload :: binary()
|
|
||||||
) ->
|
|
||||||
ok | {error, _}.
|
|
||||||
|
|
||||||
-callback delete(_Schema, _MessageID :: binary(), emqx_ds:time(), emqx_ds:topic()) ->
|
|
||||||
ok | {error, _}.
|
|
||||||
|
|
||||||
-callback make_iterator(_Schema, emqx_ds:replay()) ->
|
|
||||||
{ok, _It} | {error, _}.
|
|
||||||
|
|
||||||
-callback restore_iterator(_Schema, emqx_ds:replay(), binary()) -> {ok, _It} | {error, _}.
|
|
||||||
|
|
||||||
-callback preserve_iterator(_It) -> term().
|
|
||||||
|
|
||||||
-callback next(It) -> {value, binary(), It} | none | {error, closed}.
|
|
||||||
|
|
||||||
%%================================================================================
|
|
||||||
%% API funcions
|
|
||||||
%%================================================================================
|
|
||||||
|
|
||||||
-spec start_link(emqx_ds:shard(), emqx_ds_storage_layer:options()) ->
|
|
||||||
{ok, pid()}.
|
|
||||||
start_link(Shard = {Keyspace, ShardId}, Options) ->
|
|
||||||
gen_server:start_link(?REF(Keyspace, ShardId), ?MODULE, {Shard, Options}, []).
|
|
||||||
|
|
||||||
-spec create_generation(
|
|
||||||
emqx_ds:shard(), emqx_ds:time(), emqx_ds_conf:backend_config()
|
|
||||||
) ->
|
|
||||||
{ok, gen_id()} | {error, nonmonotonic}.
|
|
||||||
create_generation({Keyspace, ShardId}, Since, Config = {_Module, _Options}) ->
|
|
||||||
gen_server:call(?REF(Keyspace, ShardId), {create_generation, Since, Config}).
|
|
||||||
|
|
||||||
-spec store(emqx_ds:shard(), emqx_guid:guid(), emqx_ds:time(), emqx_ds:topic(), binary()) ->
|
|
||||||
ok | {error, _}.
|
|
||||||
store(Shard, GUID, Time, Topic, Msg) ->
|
|
||||||
{_GenId, #{module := Mod, data := Data}} = meta_lookup_gen(Shard, Time),
|
|
||||||
Mod:store(Data, GUID, Time, Topic, Msg).
|
|
||||||
|
|
||||||
-spec delete(emqx_ds:shard(), emqx_guid:guid(), emqx_ds:time(), emqx_ds:topic()) ->
|
|
||||||
ok | {error, _}.
|
|
||||||
delete(Shard, GUID, Time, Topic) ->
|
|
||||||
{_GenId, #{module := Mod, data := Data}} = meta_lookup_gen(Shard, Time),
|
|
||||||
Mod:delete(Data, GUID, Time, Topic).
|
|
||||||
|
|
||||||
-spec make_iterator(emqx_ds:shard(), emqx_ds:replay()) ->
|
|
||||||
{ok, iterator()} | {error, _TODO}.
|
|
||||||
make_iterator(Shard, Replay = {_, StartTime}) ->
|
|
||||||
{GenId, Gen} = meta_lookup_gen(Shard, StartTime),
|
|
||||||
open_iterator(Gen, #it{
|
|
||||||
shard = Shard,
|
|
||||||
gen = GenId,
|
|
||||||
replay = Replay
|
|
||||||
}).
|
|
||||||
|
|
||||||
-spec next(iterator()) -> {value, binary(), iterator()} | none | {error, closed}.
|
|
||||||
next(It = #it{module = Mod, data = ItData}) ->
|
|
||||||
case Mod:next(ItData) of
|
|
||||||
{value, Val, ItDataNext} ->
|
|
||||||
{value, Val, It#it{data = ItDataNext}};
|
|
||||||
{error, _} = Error ->
|
|
||||||
Error;
|
|
||||||
none ->
|
|
||||||
case open_next_iterator(It) of
|
|
||||||
{ok, ItNext} ->
|
|
||||||
next(ItNext);
|
|
||||||
{error, _} = Error ->
|
|
||||||
Error;
|
|
||||||
none ->
|
|
||||||
none
|
|
||||||
end
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec preserve_iterator(iterator(), emqx_ds:iterator_id()) ->
|
|
||||||
ok | {error, _TODO}.
|
|
||||||
preserve_iterator(It = #it{}, IteratorID) ->
|
|
||||||
iterator_put_state(IteratorID, It).
|
|
||||||
|
|
||||||
-spec restore_iterator(emqx_ds:shard(), emqx_ds:replay_id()) ->
|
|
||||||
{ok, iterator()} | {error, _TODO}.
|
|
||||||
restore_iterator(Shard, ReplayID) ->
|
|
||||||
case iterator_get_state(Shard, ReplayID) of
|
|
||||||
{ok, Serial} ->
|
|
||||||
restore_iterator_state(Shard, Serial);
|
|
||||||
not_found ->
|
|
||||||
{error, not_found};
|
|
||||||
{error, _Reason} = Error ->
|
|
||||||
Error
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec ensure_iterator(emqx_ds:shard(), emqx_ds:iterator_id(), emqx_ds:replay()) ->
|
|
||||||
{ok, iterator()} | {error, _TODO}.
|
|
||||||
ensure_iterator(Shard, IteratorID, Replay = {_TopicFilter, _StartMS}) ->
|
|
||||||
case restore_iterator(Shard, IteratorID) of
|
|
||||||
{ok, It} ->
|
|
||||||
{ok, It};
|
|
||||||
{error, not_found} ->
|
|
||||||
{ok, It} = make_iterator(Shard, Replay),
|
|
||||||
ok = emqx_ds_storage_layer:preserve_iterator(It, IteratorID),
|
|
||||||
{ok, It};
|
|
||||||
Error ->
|
|
||||||
Error
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec discard_iterator(emqx_ds:shard(), emqx_ds:replay_id()) ->
|
|
||||||
ok | {error, _TODO}.
|
|
||||||
discard_iterator(Shard, ReplayID) ->
|
|
||||||
iterator_delete(Shard, ReplayID).
|
|
||||||
|
|
||||||
-spec discard_iterator_prefix(emqx_ds:shard(), binary()) ->
|
|
||||||
ok | {error, _TODO}.
|
|
||||||
discard_iterator_prefix(Shard, KeyPrefix) ->
|
|
||||||
case do_discard_iterator_prefix(Shard, KeyPrefix) of
|
|
||||||
{ok, _} -> ok;
|
|
||||||
Error -> Error
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec list_iterator_prefix(
|
|
||||||
emqx_ds:shard(),
|
|
||||||
binary()
|
|
||||||
) -> {ok, [emqx_ds:iterator_id()]} | {error, _TODO}.
|
|
||||||
list_iterator_prefix(Shard, KeyPrefix) ->
|
|
||||||
do_list_iterator_prefix(Shard, KeyPrefix).
|
|
||||||
|
|
||||||
-spec foldl_iterator_prefix(
|
|
||||||
emqx_ds:shard(),
|
|
||||||
binary(),
|
|
||||||
fun((_Key :: binary(), _Value :: binary(), Acc) -> Acc),
|
|
||||||
Acc
|
|
||||||
) -> {ok, Acc} | {error, _TODO} when
|
|
||||||
Acc :: term().
|
|
||||||
foldl_iterator_prefix(Shard, KeyPrefix, Fn, Acc) ->
|
|
||||||
do_foldl_iterator_prefix(Shard, KeyPrefix, Fn, Acc).
|
|
||||||
|
|
||||||
%%================================================================================
|
|
||||||
%% behaviour callbacks
|
|
||||||
%%================================================================================
|
|
||||||
|
|
||||||
init({Shard, Options}) ->
|
|
||||||
process_flag(trap_exit, true),
|
process_flag(trap_exit, true),
|
||||||
{ok, S0} = open_db(Shard, Options),
|
logger:set_process_metadata(#{shard_id => ShardId, domain => [ds, storage_layer, shard]}),
|
||||||
S = ensure_current_generation(S0),
|
erase_schema_runtime(ShardId),
|
||||||
ok = populate_metadata(S),
|
{ok, DB, CFRefs0} = rocksdb_open(ShardId, Options),
|
||||||
|
{Schema, CFRefs} =
|
||||||
|
case get_schema_persistent(DB) of
|
||||||
|
not_found ->
|
||||||
|
Prototype = maps:get(storage, Options),
|
||||||
|
create_new_shard_schema(ShardId, DB, CFRefs0, Prototype);
|
||||||
|
Scm ->
|
||||||
|
{Scm, CFRefs0}
|
||||||
|
end,
|
||||||
|
Shard = open_shard(ShardId, DB, CFRefs, Schema),
|
||||||
|
S = #s{
|
||||||
|
shard_id = ShardId,
|
||||||
|
db = DB,
|
||||||
|
cf_refs = CFRefs,
|
||||||
|
schema = Schema,
|
||||||
|
shard = Shard
|
||||||
|
},
|
||||||
|
commit_metadata(S),
|
||||||
{ok, S}.
|
{ok, S}.
|
||||||
|
|
||||||
handle_call({create_generation, Since, Config}, _From, S) ->
|
handle_call(#call_create_generation{since = Since}, _From, S0) ->
|
||||||
case create_new_gen(Since, Config, S) of
|
S = add_generation(S0, Since),
|
||||||
{ok, GenId, NS} ->
|
commit_metadata(S),
|
||||||
{reply, {ok, GenId}, NS};
|
{reply, ok, S};
|
||||||
{error, _} = Error ->
|
|
||||||
{reply, Error, S}
|
|
||||||
end;
|
|
||||||
handle_call(_Call, _From, S) ->
|
handle_call(_Call, _From, S) ->
|
||||||
{reply, {error, unknown_call}, S}.
|
{reply, {error, unknown_call}, S}.
|
||||||
|
|
||||||
|
@ -281,346 +259,182 @@ handle_cast(_Cast, S) ->
|
||||||
handle_info(_Info, S) ->
|
handle_info(_Info, S) ->
|
||||||
{noreply, S}.
|
{noreply, S}.
|
||||||
|
|
||||||
terminate(_Reason, #s{db = DB, shard = Shard}) ->
|
terminate(_Reason, #s{db = DB, shard_id = ShardId}) ->
|
||||||
meta_erase(Shard),
|
erase_schema_runtime(ShardId),
|
||||||
ok = rocksdb:close(DB).
|
ok = rocksdb:close(DB).
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Internal exports
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
%% Internal functions
|
%% Internal functions
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
||||||
-record(db, {handle :: rocksdb:db_handle(), cf_iterator :: rocksdb:cf_handle()}).
|
-spec open_shard(shard_id(), rocksdb:db_handle(), cf_refs(), shard_schema()) ->
|
||||||
|
shard().
|
||||||
|
open_shard(ShardId, DB, CFRefs, ShardSchema) ->
|
||||||
|
%% Transform generation schemas to generation runtime data:
|
||||||
|
maps:map(
|
||||||
|
fun
|
||||||
|
({generation, GenId}, GenSchema) ->
|
||||||
|
open_generation(ShardId, DB, CFRefs, GenId, GenSchema);
|
||||||
|
(_K, Val) ->
|
||||||
|
Val
|
||||||
|
end,
|
||||||
|
ShardSchema
|
||||||
|
).
|
||||||
|
|
||||||
-spec populate_metadata(state()) -> ok.
|
-spec add_generation(server_state(), emqx_ds:time()) -> server_state().
|
||||||
populate_metadata(S = #s{shard = Shard, db = DBHandle, cf_iterator = CFIterator}) ->
|
add_generation(S0, Since) ->
|
||||||
ok = meta_put(Shard, db, #db{handle = DBHandle, cf_iterator = CFIterator}),
|
#s{shard_id = ShardId, db = DB, schema = Schema0, shard = Shard0, cf_refs = CFRefs0} = S0,
|
||||||
Current = schema_get_current(DBHandle),
|
{GenId, Schema, NewCFRefs} = new_generation(ShardId, DB, Schema0, Since),
|
||||||
lists:foreach(fun(GenId) -> populate_metadata(GenId, S) end, lists:seq(0, Current)).
|
CFRefs = NewCFRefs ++ CFRefs0,
|
||||||
|
Key = {generation, GenId},
|
||||||
|
Generation = open_generation(ShardId, DB, CFRefs, GenId, maps:get(Key, Schema)),
|
||||||
|
Shard = Shard0#{Key => Generation},
|
||||||
|
S0#s{
|
||||||
|
cf_refs = CFRefs,
|
||||||
|
schema = Schema,
|
||||||
|
shard = Shard
|
||||||
|
}.
|
||||||
|
|
||||||
-spec populate_metadata(gen_id(), state()) -> ok.
|
-spec open_generation(shard_id(), rocksdb:db_handle(), cf_refs(), gen_id(), generation_schema()) ->
|
||||||
populate_metadata(GenId, S = #s{shard = Shard, db = DBHandle}) ->
|
generation().
|
||||||
Gen = open_gen(GenId, schema_get_gen(DBHandle, GenId), S),
|
open_generation(ShardId, DB, CFRefs, GenId, GenSchema) ->
|
||||||
meta_register_gen(Shard, GenId, Gen).
|
?tp(debug, ds_open_generation, #{gen_id => GenId, schema => GenSchema}),
|
||||||
|
#{module := Mod, data := Schema} = GenSchema,
|
||||||
|
RuntimeData = Mod:open(ShardId, DB, GenId, CFRefs, Schema),
|
||||||
|
GenSchema#{data => RuntimeData}.
|
||||||
|
|
||||||
-spec ensure_current_generation(state()) -> state().
|
-spec create_new_shard_schema(shard_id(), rocksdb:db_handle(), cf_refs(), prototype()) ->
|
||||||
ensure_current_generation(S = #s{shard = {Keyspace, _ShardId}, db = DBHandle}) ->
|
{shard_schema(), cf_refs()}.
|
||||||
case schema_get_current(DBHandle) of
|
create_new_shard_schema(ShardId, DB, CFRefs, Prototype) ->
|
||||||
undefined ->
|
?tp(notice, ds_create_new_shard_schema, #{shard => ShardId, prototype => Prototype}),
|
||||||
Config = emqx_ds_conf:keyspace_config(Keyspace),
|
%% TODO: read prototype from options/config
|
||||||
{ok, _, NS} = create_new_gen(0, Config, S),
|
Schema0 = #{
|
||||||
NS;
|
current_generation => 0,
|
||||||
_GenId ->
|
prototype => Prototype
|
||||||
S
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec create_new_gen(emqx_ds:time(), emqx_ds_conf:backend_config(), state()) ->
|
|
||||||
{ok, gen_id(), state()} | {error, nonmonotonic}.
|
|
||||||
create_new_gen(Since, Config, S = #s{shard = Shard, db = DBHandle}) ->
|
|
||||||
GenId = get_next_id(meta_get_current(Shard)),
|
|
||||||
GenId = get_next_id(schema_get_current(DBHandle)),
|
|
||||||
case is_gen_valid(Shard, GenId, Since) of
|
|
||||||
ok ->
|
|
||||||
{ok, Gen, NS} = create_gen(GenId, Since, Config, S),
|
|
||||||
%% TODO: Transaction? Column family creation can't be transactional, anyway.
|
|
||||||
ok = schema_put_gen(DBHandle, GenId, Gen),
|
|
||||||
ok = schema_put_current(DBHandle, GenId),
|
|
||||||
ok = meta_register_gen(Shard, GenId, open_gen(GenId, Gen, NS)),
|
|
||||||
{ok, GenId, NS};
|
|
||||||
{error, _} = Error ->
|
|
||||||
Error
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec create_gen(gen_id(), emqx_ds:time(), emqx_ds_conf:backend_config(), state()) ->
|
|
||||||
{ok, generation(), state()}.
|
|
||||||
create_gen(GenId, Since, {Module, Options}, S = #s{db = DBHandle, cf_generations = CFs}) ->
|
|
||||||
% TODO: Backend implementation should ensure idempotency.
|
|
||||||
{Schema, NewCFs} = Module:create_new(DBHandle, GenId, Options),
|
|
||||||
Gen = #{
|
|
||||||
module => Module,
|
|
||||||
data => Schema,
|
|
||||||
since => Since
|
|
||||||
},
|
},
|
||||||
{ok, Gen, S#s{cf_generations = NewCFs ++ CFs}}.
|
{_NewGenId, Schema, NewCFRefs} = new_generation(ShardId, DB, Schema0, _Since = 0),
|
||||||
|
{Schema, NewCFRefs ++ CFRefs}.
|
||||||
|
|
||||||
-spec open_db(emqx_ds:shard(), options()) -> {ok, state()} | {error, _TODO}.
|
-spec new_generation(shard_id(), rocksdb:db_handle(), shard_schema(), emqx_ds:time()) ->
|
||||||
open_db(Shard = {Keyspace, ShardId}, Options) ->
|
{gen_id(), shard_schema(), cf_refs()}.
|
||||||
DefaultDir = filename:join([atom_to_binary(Keyspace), ShardId]),
|
new_generation(ShardId, DB, Schema0, Since) ->
|
||||||
DBDir = unicode:characters_to_list(maps:get(dir, Options, DefaultDir)),
|
#{current_generation := PrevGenId, prototype := {Mod, ModConf}} = Schema0,
|
||||||
|
GenId = PrevGenId + 1,
|
||||||
|
{GenData, NewCFRefs} = Mod:create(ShardId, DB, GenId, ModConf),
|
||||||
|
GenSchema = #{module => Mod, data => GenData, since => Since, until => undefined},
|
||||||
|
Schema = Schema0#{
|
||||||
|
current_generation => GenId,
|
||||||
|
{generation, GenId} => GenSchema
|
||||||
|
},
|
||||||
|
{GenId, Schema, NewCFRefs}.
|
||||||
|
|
||||||
|
%% @doc Commit current state of the server to both rocksdb and the persistent term
|
||||||
|
-spec commit_metadata(server_state()) -> ok.
|
||||||
|
commit_metadata(#s{shard_id = ShardId, schema = Schema, shard = Runtime, db = DB}) ->
|
||||||
|
ok = put_schema_persistent(DB, Schema),
|
||||||
|
put_schema_runtime(ShardId, Runtime).
|
||||||
|
|
||||||
|
-spec rocksdb_open(shard_id(), emqx_ds:builtin_db_opts()) ->
|
||||||
|
{ok, rocksdb:db_handle(), cf_refs()} | {error, _TODO}.
|
||||||
|
rocksdb_open(Shard, Options) ->
|
||||||
DBOptions = [
|
DBOptions = [
|
||||||
{create_if_missing, true},
|
{create_if_missing, true},
|
||||||
{create_missing_column_families, true}
|
{create_missing_column_families, true}
|
||||||
| emqx_ds_conf:db_options(Keyspace)
|
| maps:get(db_options, Options, [])
|
||||||
],
|
],
|
||||||
|
DBDir = db_dir(Shard),
|
||||||
_ = filelib:ensure_dir(DBDir),
|
_ = filelib:ensure_dir(DBDir),
|
||||||
ExistingCFs =
|
ExistingCFs =
|
||||||
case rocksdb:list_column_families(DBDir, DBOptions) of
|
case rocksdb:list_column_families(DBDir, DBOptions) of
|
||||||
{ok, CFs} ->
|
{ok, CFs} ->
|
||||||
[{Name, []} || Name <- CFs, Name /= ?DEFAULT_CF, Name /= ?ITERATOR_CF];
|
[{Name, []} || Name <- CFs, Name /= ?DEFAULT_CF];
|
||||||
% DB is not present. First start
|
% DB is not present. First start
|
||||||
{error, {db_open, _}} ->
|
{error, {db_open, _}} ->
|
||||||
[]
|
[]
|
||||||
end,
|
end,
|
||||||
ColumnFamilies = [
|
ColumnFamilies = [
|
||||||
{?DEFAULT_CF, ?DEFAULT_CF_OPTS},
|
{?DEFAULT_CF, ?DEFAULT_CF_OPTS}
|
||||||
{?ITERATOR_CF, ?ITERATOR_CF_OPTS}
|
|
||||||
| ExistingCFs
|
| ExistingCFs
|
||||||
],
|
],
|
||||||
case rocksdb:open(DBDir, DBOptions, ColumnFamilies) of
|
case rocksdb:open(DBDir, DBOptions, ColumnFamilies) of
|
||||||
{ok, DBHandle, [_CFDefault, CFIterator | CFRefs]} ->
|
{ok, DBHandle, [_CFDefault | CFRefs]} ->
|
||||||
{CFNames, _} = lists:unzip(ExistingCFs),
|
{CFNames, _} = lists:unzip(ExistingCFs),
|
||||||
{ok, #s{
|
{ok, DBHandle, lists:zip(CFNames, CFRefs)};
|
||||||
shard = Shard,
|
|
||||||
db = DBHandle,
|
|
||||||
cf_iterator = CFIterator,
|
|
||||||
cf_generations = lists:zip(CFNames, CFRefs)
|
|
||||||
}};
|
|
||||||
Error ->
|
Error ->
|
||||||
Error
|
Error
|
||||||
end.
|
end.
|
||||||
|
|
||||||
-spec open_gen(gen_id(), generation(), state()) -> generation().
|
-spec db_dir(shard_id()) -> file:filename().
|
||||||
open_gen(
|
db_dir({DB, ShardId}) ->
|
||||||
GenId,
|
filename:join([emqx:data_dir(), atom_to_list(DB), atom_to_list(ShardId)]).
|
||||||
Gen = #{module := Mod, data := Data},
|
|
||||||
#s{shard = Shard, db = DBHandle, cf_generations = CFs}
|
|
||||||
) ->
|
|
||||||
DB = Mod:open(Shard, DBHandle, GenId, CFs, Data),
|
|
||||||
Gen#{data := DB}.
|
|
||||||
|
|
||||||
-spec open_next_iterator(iterator()) -> {ok, iterator()} | {error, _Reason} | none.
|
%%--------------------------------------------------------------------------------
|
||||||
open_next_iterator(It = #it{shard = Shard, gen = GenId}) ->
|
%% Schema access
|
||||||
open_next_iterator(meta_get_gen(Shard, GenId + 1), It#it{gen = GenId + 1}).
|
%%--------------------------------------------------------------------------------
|
||||||
|
|
||||||
open_next_iterator(undefined, _It) ->
|
-spec generation_current(shard_id()) -> gen_id().
|
||||||
none;
|
generation_current(Shard) ->
|
||||||
open_next_iterator(Gen = #{}, It) ->
|
#{current_generation := Current} = get_schema_runtime(Shard),
|
||||||
open_iterator(Gen, It).
|
Current.
|
||||||
|
|
||||||
-spec open_iterator(generation(), iterator()) -> {ok, iterator()} | {error, _Reason}.
|
-spec generation_get(shard_id(), gen_id()) -> generation().
|
||||||
open_iterator(#{module := Mod, data := Data}, It = #it{}) ->
|
generation_get(Shard, GenId) ->
|
||||||
case Mod:make_iterator(Data, It#it.replay) of
|
#{{generation, GenId} := GenData} = get_schema_runtime(Shard),
|
||||||
{ok, ItData} ->
|
GenData.
|
||||||
{ok, It#it{module = Mod, data = ItData}};
|
|
||||||
Err ->
|
|
||||||
Err
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec open_restore_iterator(generation(), iterator(), binary()) ->
|
-spec generations_since(shard_id(), emqx_ds:time()) -> [gen_id()].
|
||||||
{ok, iterator()} | {error, _Reason}.
|
generations_since(Shard, Since) ->
|
||||||
open_restore_iterator(#{module := Mod, data := Data}, It = #it{replay = Replay}, Serial) ->
|
Schema = get_schema_runtime(Shard),
|
||||||
case Mod:restore_iterator(Data, Replay, Serial) of
|
maps:fold(
|
||||||
{ok, ItData} ->
|
fun
|
||||||
{ok, It#it{module = Mod, data = ItData}};
|
({generation, GenId}, #{until := Until}, Acc) when Until >= Since ->
|
||||||
Err ->
|
[GenId | Acc];
|
||||||
Err
|
(_K, _V, Acc) ->
|
||||||
end.
|
Acc
|
||||||
|
|
||||||
%%
|
|
||||||
|
|
||||||
-define(KEY_REPLAY_STATE(IteratorId), <<(IteratorId)/binary, "rs">>).
|
|
||||||
-define(KEY_REPLAY_STATE_PAT(KeyReplayState), begin
|
|
||||||
<<IteratorId:(size(KeyReplayState) - 2)/binary, "rs">> = (KeyReplayState),
|
|
||||||
IteratorId
|
|
||||||
end).
|
|
||||||
|
|
||||||
-define(ITERATION_WRITE_OPTS, []).
|
|
||||||
-define(ITERATION_READ_OPTS, []).
|
|
||||||
|
|
||||||
iterator_get_state(Shard, ReplayID) ->
|
|
||||||
#db{handle = Handle, cf_iterator = CF} = meta_lookup(Shard, db),
|
|
||||||
rocksdb:get(Handle, CF, ?KEY_REPLAY_STATE(ReplayID), ?ITERATION_READ_OPTS).
|
|
||||||
|
|
||||||
iterator_put_state(ID, It = #it{shard = Shard}) ->
|
|
||||||
#db{handle = Handle, cf_iterator = CF} = meta_lookup(Shard, db),
|
|
||||||
Serial = preserve_iterator_state(It),
|
|
||||||
rocksdb:put(Handle, CF, ?KEY_REPLAY_STATE(ID), Serial, ?ITERATION_WRITE_OPTS).
|
|
||||||
|
|
||||||
iterator_delete(Shard, ID) ->
|
|
||||||
#db{handle = Handle, cf_iterator = CF} = meta_lookup(Shard, db),
|
|
||||||
rocksdb:delete(Handle, CF, ?KEY_REPLAY_STATE(ID), ?ITERATION_WRITE_OPTS).
|
|
||||||
|
|
||||||
preserve_iterator_state(#it{
|
|
||||||
gen = Gen,
|
|
||||||
replay = {TopicFilter, StartTime},
|
|
||||||
module = Mod,
|
|
||||||
data = ItData
|
|
||||||
}) ->
|
|
||||||
term_to_binary(#{
|
|
||||||
v => 1,
|
|
||||||
gen => Gen,
|
|
||||||
filter => TopicFilter,
|
|
||||||
start => StartTime,
|
|
||||||
st => Mod:preserve_iterator(ItData)
|
|
||||||
}).
|
|
||||||
|
|
||||||
restore_iterator_state(Shard, Serial) when is_binary(Serial) ->
|
|
||||||
restore_iterator_state(Shard, binary_to_term(Serial));
|
|
||||||
restore_iterator_state(
|
|
||||||
Shard,
|
|
||||||
#{
|
|
||||||
v := 1,
|
|
||||||
gen := Gen,
|
|
||||||
filter := TopicFilter,
|
|
||||||
start := StartTime,
|
|
||||||
st := State
|
|
||||||
}
|
|
||||||
) ->
|
|
||||||
It = #it{shard = Shard, gen = Gen, replay = {TopicFilter, StartTime}},
|
|
||||||
open_restore_iterator(meta_get_gen(Shard, Gen), It, State).
|
|
||||||
|
|
||||||
do_list_iterator_prefix(Shard, KeyPrefix) ->
|
|
||||||
Fn = fun(K0, _V, Acc) ->
|
|
||||||
K = ?KEY_REPLAY_STATE_PAT(K0),
|
|
||||||
[K | Acc]
|
|
||||||
end,
|
end,
|
||||||
do_foldl_iterator_prefix(Shard, KeyPrefix, Fn, []).
|
[],
|
||||||
|
Schema
|
||||||
|
).
|
||||||
|
|
||||||
do_discard_iterator_prefix(Shard, KeyPrefix) ->
|
-define(PERSISTENT_TERM(SHARD), {emqx_ds_storage_layer, SHARD}).
|
||||||
#db{handle = DBHandle, cf_iterator = CF} = meta_lookup(Shard, db),
|
|
||||||
Fn = fun(K, _V, _Acc) -> ok = rocksdb:delete(DBHandle, CF, K, ?ITERATION_WRITE_OPTS) end,
|
|
||||||
do_foldl_iterator_prefix(Shard, KeyPrefix, Fn, ok).
|
|
||||||
|
|
||||||
do_foldl_iterator_prefix(Shard, KeyPrefix, Fn, Acc) ->
|
-spec get_schema_runtime(shard_id()) -> shard().
|
||||||
#db{handle = Handle, cf_iterator = CF} = meta_lookup(Shard, db),
|
get_schema_runtime(Shard) ->
|
||||||
case rocksdb:iterator(Handle, CF, ?ITERATION_READ_OPTS) of
|
persistent_term:get(?PERSISTENT_TERM(Shard)).
|
||||||
{ok, It} ->
|
|
||||||
NextAction = {seek, KeyPrefix},
|
|
||||||
do_foldl_iterator_prefix(Handle, CF, It, KeyPrefix, NextAction, Fn, Acc);
|
|
||||||
Error ->
|
|
||||||
Error
|
|
||||||
end.
|
|
||||||
|
|
||||||
do_foldl_iterator_prefix(DBHandle, CF, It, KeyPrefix, NextAction, Fn, Acc) ->
|
-spec put_schema_runtime(shard_id(), shard()) -> ok.
|
||||||
case rocksdb:iterator_move(It, NextAction) of
|
put_schema_runtime(Shard, RuntimeSchema) ->
|
||||||
{ok, K = <<KeyPrefix:(size(KeyPrefix))/binary, _/binary>>, V} ->
|
persistent_term:put(?PERSISTENT_TERM(Shard), RuntimeSchema),
|
||||||
NewAcc = Fn(K, V, Acc),
|
ok.
|
||||||
do_foldl_iterator_prefix(DBHandle, CF, It, KeyPrefix, next, Fn, NewAcc);
|
|
||||||
{ok, _K, _V} ->
|
|
||||||
ok = rocksdb:iterator_close(It),
|
|
||||||
{ok, Acc};
|
|
||||||
{error, invalid_iterator} ->
|
|
||||||
ok = rocksdb:iterator_close(It),
|
|
||||||
{ok, Acc};
|
|
||||||
Error ->
|
|
||||||
ok = rocksdb:iterator_close(It),
|
|
||||||
Error
|
|
||||||
end.
|
|
||||||
|
|
||||||
%% Functions for dealing with the metadata stored persistently in rocksdb
|
-spec erase_schema_runtime(shard_id()) -> ok.
|
||||||
|
erase_schema_runtime(Shard) ->
|
||||||
-define(CURRENT_GEN, <<"current">>).
|
persistent_term:erase(?PERSISTENT_TERM(Shard)),
|
||||||
-define(SCHEMA_WRITE_OPTS, []).
|
|
||||||
-define(SCHEMA_READ_OPTS, []).
|
|
||||||
|
|
||||||
-spec schema_get_gen(rocksdb:db_handle(), gen_id()) -> generation().
|
|
||||||
schema_get_gen(DBHandle, GenId) ->
|
|
||||||
{ok, Bin} = rocksdb:get(DBHandle, schema_gen_key(GenId), ?SCHEMA_READ_OPTS),
|
|
||||||
binary_to_term(Bin).
|
|
||||||
|
|
||||||
-spec schema_put_gen(rocksdb:db_handle(), gen_id(), generation()) -> ok | {error, _}.
|
|
||||||
schema_put_gen(DBHandle, GenId, Gen) ->
|
|
||||||
rocksdb:put(DBHandle, schema_gen_key(GenId), term_to_binary(Gen), ?SCHEMA_WRITE_OPTS).
|
|
||||||
|
|
||||||
-spec schema_get_current(rocksdb:db_handle()) -> gen_id() | undefined.
|
|
||||||
schema_get_current(DBHandle) ->
|
|
||||||
case rocksdb:get(DBHandle, ?CURRENT_GEN, ?SCHEMA_READ_OPTS) of
|
|
||||||
{ok, Bin} ->
|
|
||||||
binary_to_integer(Bin);
|
|
||||||
not_found ->
|
|
||||||
undefined
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec schema_put_current(rocksdb:db_handle(), gen_id()) -> ok | {error, _}.
|
|
||||||
schema_put_current(DBHandle, GenId) ->
|
|
||||||
rocksdb:put(DBHandle, ?CURRENT_GEN, integer_to_binary(GenId), ?SCHEMA_WRITE_OPTS).
|
|
||||||
|
|
||||||
-spec schema_gen_key(integer()) -> binary().
|
|
||||||
schema_gen_key(N) ->
|
|
||||||
<<"gen", N:32>>.
|
|
||||||
|
|
||||||
-undef(CURRENT_GEN).
|
|
||||||
-undef(SCHEMA_WRITE_OPTS).
|
|
||||||
-undef(SCHEMA_READ_OPTS).
|
|
||||||
|
|
||||||
%% Functions for dealing with the runtime shard metadata:
|
|
||||||
|
|
||||||
-define(PERSISTENT_TERM(SHARD, GEN), {?MODULE, SHARD, GEN}).
|
|
||||||
|
|
||||||
-spec meta_register_gen(emqx_ds:shard(), gen_id(), generation()) -> ok.
|
|
||||||
meta_register_gen(Shard, GenId, Gen) ->
|
|
||||||
Gs =
|
|
||||||
case GenId > 0 of
|
|
||||||
true -> meta_lookup(Shard, GenId - 1);
|
|
||||||
false -> []
|
|
||||||
end,
|
|
||||||
ok = meta_put(Shard, GenId, [Gen | Gs]),
|
|
||||||
ok = meta_put(Shard, current, GenId).
|
|
||||||
|
|
||||||
-spec meta_lookup_gen(emqx_ds:shard(), emqx_ds:time()) -> {gen_id(), generation()}.
|
|
||||||
meta_lookup_gen(Shard, Time) ->
|
|
||||||
% TODO
|
|
||||||
% Is cheaper persistent term GC on update here worth extra lookup? I'm leaning
|
|
||||||
% towards a "no".
|
|
||||||
Current = meta_lookup(Shard, current),
|
|
||||||
Gens = meta_lookup(Shard, Current),
|
|
||||||
find_gen(Time, Current, Gens).
|
|
||||||
|
|
||||||
find_gen(Time, GenId, [Gen = #{since := Since} | _]) when Time >= Since ->
|
|
||||||
{GenId, Gen};
|
|
||||||
find_gen(Time, GenId, [_Gen | Rest]) ->
|
|
||||||
find_gen(Time, GenId - 1, Rest).
|
|
||||||
|
|
||||||
-spec meta_get_gen(emqx_ds:shard(), gen_id()) -> generation() | undefined.
|
|
||||||
meta_get_gen(Shard, GenId) ->
|
|
||||||
case meta_lookup(Shard, GenId, []) of
|
|
||||||
[Gen | _Older] -> Gen;
|
|
||||||
[] -> undefined
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec meta_get_current(emqx_ds:shard()) -> gen_id() | undefined.
|
|
||||||
meta_get_current(Shard) ->
|
|
||||||
meta_lookup(Shard, current, undefined).
|
|
||||||
|
|
||||||
-spec meta_lookup(emqx_ds:shard(), _K) -> _V.
|
|
||||||
meta_lookup(Shard, K) ->
|
|
||||||
persistent_term:get(?PERSISTENT_TERM(Shard, K)).
|
|
||||||
|
|
||||||
-spec meta_lookup(emqx_ds:shard(), _K, Default) -> _V | Default.
|
|
||||||
meta_lookup(Shard, K, Default) ->
|
|
||||||
persistent_term:get(?PERSISTENT_TERM(Shard, K), Default).
|
|
||||||
|
|
||||||
-spec meta_put(emqx_ds:shard(), _K, _V) -> ok.
|
|
||||||
meta_put(Shard, K, V) ->
|
|
||||||
persistent_term:put(?PERSISTENT_TERM(Shard, K), V).
|
|
||||||
|
|
||||||
-spec meta_erase(emqx_ds:shard()) -> ok.
|
|
||||||
meta_erase(Shard) ->
|
|
||||||
[
|
|
||||||
persistent_term:erase(K)
|
|
||||||
|| {K = ?PERSISTENT_TERM(Z, _), _} <- persistent_term:get(), Z =:= Shard
|
|
||||||
],
|
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
-undef(PERSISTENT_TERM).
|
-undef(PERSISTENT_TERM).
|
||||||
|
|
||||||
get_next_id(undefined) -> 0;
|
-define(ROCKSDB_SCHEMA_KEY, <<"schema_v1">>).
|
||||||
get_next_id(GenId) -> GenId + 1.
|
|
||||||
|
|
||||||
is_gen_valid(Shard, GenId, Since) when GenId > 0 ->
|
-spec get_schema_persistent(rocksdb:db_handle()) -> shard_schema() | not_found.
|
||||||
[GenPrev | _] = meta_lookup(Shard, GenId - 1),
|
get_schema_persistent(DB) ->
|
||||||
case GenPrev of
|
case rocksdb:get(DB, ?ROCKSDB_SCHEMA_KEY, []) of
|
||||||
#{since := SincePrev} when Since > SincePrev ->
|
{ok, Blob} ->
|
||||||
ok;
|
Schema = binary_to_term(Blob),
|
||||||
#{} ->
|
%% Sanity check:
|
||||||
{error, nonmonotonic}
|
#{current_generation := _, prototype := _} = Schema,
|
||||||
end;
|
Schema;
|
||||||
is_gen_valid(_Shard, 0, 0) ->
|
not_found ->
|
||||||
ok.
|
not_found
|
||||||
|
end.
|
||||||
|
|
||||||
%% -spec store_cfs(rocksdb:db_handle(), [{string(), rocksdb:cf_handle()}]) -> ok.
|
-spec put_schema_persistent(rocksdb:db_handle(), shard_schema()) -> ok.
|
||||||
%% store_cfs(DBHandle, CFRefs) ->
|
put_schema_persistent(DB, Schema) ->
|
||||||
%% lists:foreach(
|
Blob = term_to_binary(Schema),
|
||||||
%% fun({CFName, CFRef}) ->
|
rocksdb:put(DB, ?ROCKSDB_SCHEMA_KEY, Blob, []).
|
||||||
%% persistent_term:put({self(), CFName}, {DBHandle, CFRef})
|
|
||||||
%% end,
|
-undef(ROCKSDB_SCHEMA_KEY).
|
||||||
%% CFRefs).
|
|
||||||
|
|
|
@ -6,7 +6,7 @@
|
||||||
-behaviour(supervisor).
|
-behaviour(supervisor).
|
||||||
|
|
||||||
%% API:
|
%% API:
|
||||||
-export([start_link/0, start_shard/2, stop_shard/1]).
|
-export([start_link/0, start_shard/2, stop_shard/1, ensure_shard/2]).
|
||||||
|
|
||||||
%% behaviour callbacks:
|
%% behaviour callbacks:
|
||||||
-export([init/1]).
|
-export([init/1]).
|
||||||
|
@ -25,7 +25,7 @@
|
||||||
start_link() ->
|
start_link() ->
|
||||||
supervisor:start_link({local, ?SUP}, ?MODULE, []).
|
supervisor:start_link({local, ?SUP}, ?MODULE, []).
|
||||||
|
|
||||||
-spec start_shard(emqx_ds:shard(), emqx_ds_storage_layer:options()) ->
|
-spec start_shard(emqx_ds_replication_layer:shard_id(), emqx_ds:create_db_opts()) ->
|
||||||
supervisor:startchild_ret().
|
supervisor:startchild_ret().
|
||||||
start_shard(Shard, Options) ->
|
start_shard(Shard, Options) ->
|
||||||
supervisor:start_child(?SUP, shard_child_spec(Shard, Options)).
|
supervisor:start_child(?SUP, shard_child_spec(Shard, Options)).
|
||||||
|
@ -35,6 +35,17 @@ stop_shard(Shard) ->
|
||||||
ok = supervisor:terminate_child(?SUP, Shard),
|
ok = supervisor:terminate_child(?SUP, Shard),
|
||||||
ok = supervisor:delete_child(?SUP, Shard).
|
ok = supervisor:delete_child(?SUP, Shard).
|
||||||
|
|
||||||
|
-spec ensure_shard(emqx_ds:shard(), emqx_ds_storage_layer:options()) -> ok | {error, _Reason}.
|
||||||
|
ensure_shard(Shard, Options) ->
|
||||||
|
case start_shard(Shard, Options) of
|
||||||
|
{ok, _Pid} ->
|
||||||
|
ok;
|
||||||
|
{error, {already_started, _Pid}} ->
|
||||||
|
ok;
|
||||||
|
{error, Reason} ->
|
||||||
|
{error, Reason}
|
||||||
|
end.
|
||||||
|
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
%% behaviour callbacks
|
%% behaviour callbacks
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
@ -52,7 +63,7 @@ init([]) ->
|
||||||
%% Internal functions
|
%% Internal functions
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
||||||
-spec shard_child_spec(emqx_ds:shard(), emqx_ds_storage_layer:options()) ->
|
-spec shard_child_spec(emqx_ds_replication_layer:shard_id(), emqx_ds:create_db_opts()) ->
|
||||||
supervisor:child_spec().
|
supervisor:child_spec().
|
||||||
shard_child_spec(Shard, Options) ->
|
shard_child_spec(Shard, Options) ->
|
||||||
#{
|
#{
|
||||||
|
|
|
@ -0,0 +1,139 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%
|
||||||
|
%% Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
%% you may not use this file except in compliance with the License.
|
||||||
|
%% You may obtain a copy of the License at
|
||||||
|
%%
|
||||||
|
%% http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
%%
|
||||||
|
%% Unless required by applicable law or agreed to in writing, software
|
||||||
|
%% distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
%% See the License for the specific language governing permissions and
|
||||||
|
%% limitations under the License.
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
|
||||||
|
%% @doc Reference implementation of the storage.
|
||||||
|
%%
|
||||||
|
%% Trivial, extremely slow and inefficient. It also doesn't handle
|
||||||
|
%% restart of the Erlang node properly, so obviously it's only to be
|
||||||
|
%% used for testing.
|
||||||
|
-module(emqx_ds_storage_reference).
|
||||||
|
|
||||||
|
-behaviour(emqx_ds_storage_layer).
|
||||||
|
|
||||||
|
%% API:
|
||||||
|
-export([]).
|
||||||
|
|
||||||
|
%% behavior callbacks:
|
||||||
|
-export([create/4, open/5, store_batch/4, get_streams/4, make_iterator/5, next/4]).
|
||||||
|
|
||||||
|
%% internal exports:
|
||||||
|
-export([]).
|
||||||
|
|
||||||
|
-export_type([options/0]).
|
||||||
|
|
||||||
|
-include_lib("emqx_utils/include/emqx_message.hrl").
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Type declarations
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-type options() :: #{}.
|
||||||
|
|
||||||
|
%% Permanent state:
|
||||||
|
-record(schema, {}).
|
||||||
|
|
||||||
|
%% Runtime state:
|
||||||
|
-record(s, {
|
||||||
|
db :: rocksdb:db_handle(),
|
||||||
|
cf :: rocksdb:cf_handle()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-record(stream, {}).
|
||||||
|
|
||||||
|
-record(it, {
|
||||||
|
topic_filter :: emqx_ds:topic_filter(),
|
||||||
|
start_time :: emqx_ds:time(),
|
||||||
|
last_seen_message_key = first :: binary() | first
|
||||||
|
}).
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% API funcions
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% behavior callbacks
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
create(_ShardId, DBHandle, GenId, _Options) ->
|
||||||
|
CFName = data_cf(GenId),
|
||||||
|
{ok, CFHandle} = rocksdb:create_column_family(DBHandle, CFName, []),
|
||||||
|
Schema = #schema{},
|
||||||
|
{Schema, [{CFName, CFHandle}]}.
|
||||||
|
|
||||||
|
open(_Shard, DBHandle, GenId, CFRefs, #schema{}) ->
|
||||||
|
{_, CF} = lists:keyfind(data_cf(GenId), 1, CFRefs),
|
||||||
|
#s{db = DBHandle, cf = CF}.
|
||||||
|
|
||||||
|
store_batch(_ShardId, #s{db = DB, cf = CF}, Messages, _Options) ->
|
||||||
|
lists:foreach(
|
||||||
|
fun(Msg) ->
|
||||||
|
Id = erlang:unique_integer([monotonic]),
|
||||||
|
Key = <<Id:64>>,
|
||||||
|
Val = term_to_binary(Msg),
|
||||||
|
rocksdb:put(DB, CF, Key, Val, [])
|
||||||
|
end,
|
||||||
|
Messages
|
||||||
|
).
|
||||||
|
|
||||||
|
get_streams(_Shard, _Data, _TopicFilter, _StartTime) ->
|
||||||
|
[#stream{}].
|
||||||
|
|
||||||
|
make_iterator(_Shard, _Data, #stream{}, TopicFilter, StartTime) ->
|
||||||
|
{ok, #it{
|
||||||
|
topic_filter = TopicFilter,
|
||||||
|
start_time = StartTime
|
||||||
|
}}.
|
||||||
|
|
||||||
|
next(_Shard, #s{db = DB, cf = CF}, It0, BatchSize) ->
|
||||||
|
#it{topic_filter = TopicFilter, start_time = StartTime, last_seen_message_key = Key0} = It0,
|
||||||
|
{ok, ITHandle} = rocksdb:iterator(DB, CF, []),
|
||||||
|
Action =
|
||||||
|
case Key0 of
|
||||||
|
first ->
|
||||||
|
first;
|
||||||
|
_ ->
|
||||||
|
_ = rocksdb:iterator_move(ITHandle, Key0),
|
||||||
|
next
|
||||||
|
end,
|
||||||
|
{Key, Messages} = do_next(TopicFilter, StartTime, ITHandle, Action, BatchSize, Key0, []),
|
||||||
|
rocksdb:iterator_close(ITHandle),
|
||||||
|
It = It0#it{last_seen_message_key = Key},
|
||||||
|
{ok, It, lists:reverse(Messages)}.
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% Internal functions
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
do_next(_, _, _, _, 0, Key, Acc) ->
|
||||||
|
{Key, Acc};
|
||||||
|
do_next(TopicFilter, StartTime, IT, Action, NLeft, Key0, Acc) ->
|
||||||
|
case rocksdb:iterator_move(IT, Action) of
|
||||||
|
{ok, Key, Blob} ->
|
||||||
|
Msg = #message{topic = Topic, timestamp = TS} = binary_to_term(Blob),
|
||||||
|
case emqx_topic:match(Topic, TopicFilter) andalso TS >= StartTime of
|
||||||
|
true ->
|
||||||
|
do_next(TopicFilter, StartTime, IT, next, NLeft - 1, Key, [Msg | Acc]);
|
||||||
|
false ->
|
||||||
|
do_next(TopicFilter, StartTime, IT, next, NLeft, Key, Acc)
|
||||||
|
end;
|
||||||
|
{error, invalid_iterator} ->
|
||||||
|
{Key0, Acc}
|
||||||
|
end.
|
||||||
|
|
||||||
|
%% @doc Generate a column family ID for the MQTT messages
|
||||||
|
-spec data_cf(emqx_ds_storage_layer:gen_id()) -> [char()].
|
||||||
|
data_cf(GenId) ->
|
||||||
|
"emqx_ds_storage_reference" ++ integer_to_list(GenId).
|
|
@ -30,7 +30,7 @@ start_link() ->
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
||||||
init([]) ->
|
init([]) ->
|
||||||
Children = [shard_sup()],
|
Children = [storage_layer_sup()],
|
||||||
SupFlags = #{
|
SupFlags = #{
|
||||||
strategy => one_for_all,
|
strategy => one_for_all,
|
||||||
intensity => 0,
|
intensity => 0,
|
||||||
|
@ -42,7 +42,7 @@ init([]) ->
|
||||||
%% Internal functions
|
%% Internal functions
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
||||||
shard_sup() ->
|
storage_layer_sup() ->
|
||||||
#{
|
#{
|
||||||
id => local_store_shard_sup,
|
id => local_store_shard_sup,
|
||||||
start => {emqx_ds_storage_layer_sup, start_link, []},
|
start => {emqx_ds_storage_layer_sup, start_link, []},
|
||||||
|
|
|
@ -5,7 +5,7 @@
|
||||||
{vsn, "0.1.6"},
|
{vsn, "0.1.6"},
|
||||||
{modules, []},
|
{modules, []},
|
||||||
{registered, []},
|
{registered, []},
|
||||||
{applications, [kernel, stdlib, rocksdb, gproc, mria]},
|
{applications, [kernel, stdlib, rocksdb, gproc, mria, emqx_utils]},
|
||||||
{mod, {emqx_ds_app, []}},
|
{mod, {emqx_ds_app, []}},
|
||||||
{env, []}
|
{env, []}
|
||||||
]}.
|
]}.
|
||||||
|
|
|
@ -0,0 +1,75 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%
|
||||||
|
%% Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
%% you may not use this file except in compliance with the License.
|
||||||
|
%% You may obtain a copy of the License at
|
||||||
|
%%
|
||||||
|
%% http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
%%
|
||||||
|
%% Unless required by applicable law or agreed to in writing, software
|
||||||
|
%% distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
%% See the License for the specific language governing permissions and
|
||||||
|
%% limitations under the License.
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
-module(emqx_ds_proto_v1).
|
||||||
|
|
||||||
|
-behavior(emqx_bpapi).
|
||||||
|
|
||||||
|
-include_lib("emqx_utils/include/bpapi.hrl").
|
||||||
|
%% API:
|
||||||
|
-export([open_shard/3, drop_shard/2, get_streams/4, make_iterator/5, next/4]).
|
||||||
|
|
||||||
|
%% behavior callbacks:
|
||||||
|
-export([introduced_in/0]).
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% API funcions
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
-spec open_shard(node(), emqx_ds_replication_layer:shard_id(), emqx_ds:create_db_opts()) ->
|
||||||
|
ok.
|
||||||
|
open_shard(Node, Shard, Opts) ->
|
||||||
|
erpc:call(Node, emqx_ds_replication_layer, do_open_shard_v1, [Shard, Opts]).
|
||||||
|
|
||||||
|
-spec drop_shard(node(), emqx_ds_replication_layer:shard_id()) ->
|
||||||
|
ok.
|
||||||
|
drop_shard(Node, Shard) ->
|
||||||
|
erpc:call(Node, emqx_ds_replication_layer, do_drop_shard_v1, [Shard]).
|
||||||
|
|
||||||
|
-spec get_streams(
|
||||||
|
node(), emqx_ds_replication_layer:shard_id(), emqx_ds:topic_filter(), emqx_ds:time()
|
||||||
|
) ->
|
||||||
|
[{integer(), emqx_ds_replication_layer:stream()}].
|
||||||
|
get_streams(Node, Shard, TopicFilter, Time) ->
|
||||||
|
erpc:call(Node, emqx_ds_replication_layer, do_get_streams_v1, [Shard, TopicFilter, Time]).
|
||||||
|
|
||||||
|
-spec make_iterator(
|
||||||
|
node(),
|
||||||
|
emqx_ds_replication_layer:shard_id(),
|
||||||
|
emqx_ds_storage_layer:stream(),
|
||||||
|
emqx_ds:topic_filter(),
|
||||||
|
emqx_ds:time()
|
||||||
|
) ->
|
||||||
|
{ok, emqx_ds_replication_layer:iterator()} | {error, _}.
|
||||||
|
make_iterator(Node, Shard, Stream, TopicFilter, StartTime) ->
|
||||||
|
erpc:call(Node, emqx_ds_replication_layer, do_make_iterator_v1, [
|
||||||
|
Shard, Stream, TopicFilter, StartTime
|
||||||
|
]).
|
||||||
|
|
||||||
|
-spec next(
|
||||||
|
node(), emqx_ds_replication_layer:shard_id(), emqx_ds_storage_layer:iterator(), pos_integer()
|
||||||
|
) ->
|
||||||
|
{ok, emqx_ds_storage_layer:iterator(), [emqx_types:messages()]}
|
||||||
|
| {ok, end_of_stream}
|
||||||
|
| {error, _}.
|
||||||
|
next(Node, Shard, Iter, BatchSize) ->
|
||||||
|
erpc:call(Node, emqx_ds_replication_layer, do_next_v1, [Shard, Iter, BatchSize]).
|
||||||
|
|
||||||
|
%%================================================================================
|
||||||
|
%% behavior callbacks
|
||||||
|
%%================================================================================
|
||||||
|
|
||||||
|
introduced_in() ->
|
||||||
|
"5.4.0".
|
|
@ -0,0 +1,146 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%
|
||||||
|
%% Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
%% you may not use this file except in compliance with the License.
|
||||||
|
%% You may obtain a copy of the License at
|
||||||
|
%%
|
||||||
|
%% http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
%%
|
||||||
|
%% Unless required by applicable law or agreed to in writing, software
|
||||||
|
%% distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
%% See the License for the specific language governing permissions and
|
||||||
|
%% limitations under the License.
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
-module(emqx_ds_SUITE).
|
||||||
|
|
||||||
|
-compile(export_all).
|
||||||
|
-compile(nowarn_export_all).
|
||||||
|
|
||||||
|
-include_lib("emqx/include/emqx.hrl").
|
||||||
|
-include_lib("common_test/include/ct.hrl").
|
||||||
|
-include_lib("stdlib/include/assert.hrl").
|
||||||
|
-include_lib("snabbkaffe/include/snabbkaffe.hrl").
|
||||||
|
|
||||||
|
opts() ->
|
||||||
|
#{
|
||||||
|
backend => builtin,
|
||||||
|
storage => {emqx_ds_storage_reference, #{}}
|
||||||
|
}.
|
||||||
|
|
||||||
|
%% A simple smoke test that verifies that opening/closing the DB
|
||||||
|
%% doesn't crash, and not much else
|
||||||
|
t_00_smoke_open_drop(_Config) ->
|
||||||
|
DB = 'DB',
|
||||||
|
?assertMatch(ok, emqx_ds:open_db(DB, opts())),
|
||||||
|
?assertMatch(ok, emqx_ds:open_db(DB, opts())),
|
||||||
|
?assertMatch(ok, emqx_ds:drop_db(DB)).
|
||||||
|
|
||||||
|
%% A simple smoke test that verifies that storing the messages doesn't
|
||||||
|
%% crash
|
||||||
|
t_01_smoke_store(_Config) ->
|
||||||
|
DB = default,
|
||||||
|
?assertMatch(ok, emqx_ds:open_db(DB, opts())),
|
||||||
|
Msg = message(<<"foo/bar">>, <<"foo">>, 0),
|
||||||
|
?assertMatch(ok, emqx_ds:store_batch(DB, [Msg])).
|
||||||
|
|
||||||
|
%% A simple smoke test that verifies that getting the list of streams
|
||||||
|
%% doesn't crash and that iterators can be opened.
|
||||||
|
t_02_smoke_get_streams_start_iter(_Config) ->
|
||||||
|
DB = ?FUNCTION_NAME,
|
||||||
|
?assertMatch(ok, emqx_ds:open_db(DB, opts())),
|
||||||
|
StartTime = 0,
|
||||||
|
TopicFilter = ['#'],
|
||||||
|
[{Rank, Stream}] = emqx_ds:get_streams(DB, TopicFilter, StartTime),
|
||||||
|
?assertMatch({_, _}, Rank),
|
||||||
|
?assertMatch({ok, _Iter}, emqx_ds:make_iterator(Stream, TopicFilter, StartTime)).
|
||||||
|
|
||||||
|
%% A simple smoke test that verifies that it's possible to iterate
|
||||||
|
%% over messages.
|
||||||
|
t_03_smoke_iterate(_Config) ->
|
||||||
|
DB = ?FUNCTION_NAME,
|
||||||
|
?assertMatch(ok, emqx_ds:open_db(DB, opts())),
|
||||||
|
StartTime = 0,
|
||||||
|
TopicFilter = ['#'],
|
||||||
|
Msgs = [
|
||||||
|
message(<<"foo/bar">>, <<"1">>, 0),
|
||||||
|
message(<<"foo">>, <<"2">>, 1),
|
||||||
|
message(<<"bar/bar">>, <<"3">>, 2)
|
||||||
|
],
|
||||||
|
?assertMatch(ok, emqx_ds:store_batch(DB, Msgs)),
|
||||||
|
[{_, Stream}] = emqx_ds:get_streams(DB, TopicFilter, StartTime),
|
||||||
|
{ok, Iter0} = emqx_ds:make_iterator(Stream, TopicFilter, StartTime),
|
||||||
|
{ok, Iter, Batch} = iterate(Iter0, 1),
|
||||||
|
?assertEqual(Msgs, Batch, {Iter0, Iter}).
|
||||||
|
|
||||||
|
%% Verify that iterators survive restart of the application. This is
|
||||||
|
%% an important property, since the lifetime of the iterators is tied
|
||||||
|
%% to the external resources, such as clients' sessions, and they
|
||||||
|
%% should always be able to continue replaying the topics from where
|
||||||
|
%% they are left off.
|
||||||
|
t_04_restart(_Config) ->
|
||||||
|
DB = ?FUNCTION_NAME,
|
||||||
|
?assertMatch(ok, emqx_ds:open_db(DB, opts())),
|
||||||
|
TopicFilter = ['#'],
|
||||||
|
StartTime = 0,
|
||||||
|
Msgs = [
|
||||||
|
message(<<"foo/bar">>, <<"1">>, 0),
|
||||||
|
message(<<"foo">>, <<"2">>, 1),
|
||||||
|
message(<<"bar/bar">>, <<"3">>, 2)
|
||||||
|
],
|
||||||
|
?assertMatch(ok, emqx_ds:store_batch(DB, Msgs)),
|
||||||
|
[{_, Stream}] = emqx_ds:get_streams(DB, TopicFilter, StartTime),
|
||||||
|
{ok, Iter0} = emqx_ds:make_iterator(Stream, TopicFilter, StartTime),
|
||||||
|
%% Restart the application:
|
||||||
|
?tp(warning, emqx_ds_SUITE_restart_app, #{}),
|
||||||
|
ok = application:stop(emqx_durable_storage),
|
||||||
|
{ok, _} = application:ensure_all_started(emqx_durable_storage),
|
||||||
|
ok = emqx_ds:open_db(DB, opts()),
|
||||||
|
%% The old iterator should be still operational:
|
||||||
|
{ok, Iter, Batch} = iterate(Iter0, 1),
|
||||||
|
?assertEqual(Msgs, Batch, {Iter0, Iter}).
|
||||||
|
|
||||||
|
message(Topic, Payload, PublishedAt) ->
|
||||||
|
#message{
|
||||||
|
topic = Topic,
|
||||||
|
payload = Payload,
|
||||||
|
timestamp = PublishedAt,
|
||||||
|
id = emqx_guid:gen()
|
||||||
|
}.
|
||||||
|
|
||||||
|
iterate(It, BatchSize) ->
|
||||||
|
iterate(It, BatchSize, []).
|
||||||
|
|
||||||
|
iterate(It0, BatchSize, Acc) ->
|
||||||
|
case emqx_ds:next(It0, BatchSize) of
|
||||||
|
{ok, It, []} ->
|
||||||
|
{ok, It, Acc};
|
||||||
|
{ok, It, Msgs} ->
|
||||||
|
iterate(It, BatchSize, Acc ++ Msgs);
|
||||||
|
Ret ->
|
||||||
|
Ret
|
||||||
|
end.
|
||||||
|
|
||||||
|
%% CT callbacks
|
||||||
|
|
||||||
|
all() -> emqx_common_test_helpers:all(?MODULE).
|
||||||
|
|
||||||
|
init_per_suite(Config) ->
|
||||||
|
Apps = emqx_cth_suite:start(
|
||||||
|
[mria, emqx_durable_storage],
|
||||||
|
#{work_dir => ?config(priv_dir, Config)}
|
||||||
|
),
|
||||||
|
[{apps, Apps} | Config].
|
||||||
|
|
||||||
|
end_per_suite(Config) ->
|
||||||
|
ok = emqx_cth_suite:stop(?config(apps, Config)),
|
||||||
|
ok.
|
||||||
|
|
||||||
|
init_per_testcase(_TC, Config) ->
|
||||||
|
%% snabbkaffe:fix_ct_logging(),
|
||||||
|
application:ensure_all_started(emqx_durable_storage),
|
||||||
|
Config.
|
||||||
|
|
||||||
|
end_per_testcase(_TC, _Config) ->
|
||||||
|
ok = application:stop(emqx_durable_storage).
|
|
@ -1,188 +0,0 @@
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
%% Copyright (c) 2022-2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
-module(emqx_ds_message_storage_bitmask_SUITE).
|
|
||||||
|
|
||||||
-compile(export_all).
|
|
||||||
-compile(nowarn_export_all).
|
|
||||||
|
|
||||||
-include_lib("stdlib/include/assert.hrl").
|
|
||||||
|
|
||||||
-import(emqx_ds_message_storage_bitmask, [
|
|
||||||
make_keymapper/1,
|
|
||||||
keymapper_info/1,
|
|
||||||
compute_topic_bitmask/2,
|
|
||||||
compute_time_bitmask/1,
|
|
||||||
compute_topic_seek/4
|
|
||||||
]).
|
|
||||||
|
|
||||||
all() -> emqx_common_test_helpers:all(?MODULE).
|
|
||||||
|
|
||||||
t_make_keymapper(_) ->
|
|
||||||
?assertMatch(
|
|
||||||
#{
|
|
||||||
source := [
|
|
||||||
{timestamp, 9, 23},
|
|
||||||
{hash, level, 2},
|
|
||||||
{hash, level, 4},
|
|
||||||
{hash, levels, 8},
|
|
||||||
{timestamp, 0, 9}
|
|
||||||
],
|
|
||||||
bitsize := 46,
|
|
||||||
epoch := 512
|
|
||||||
},
|
|
||||||
keymapper_info(
|
|
||||||
make_keymapper(#{
|
|
||||||
timestamp_bits => 32,
|
|
||||||
topic_bits_per_level => [2, 4, 8],
|
|
||||||
epoch => 1000
|
|
||||||
})
|
|
||||||
)
|
|
||||||
).
|
|
||||||
|
|
||||||
t_make_keymapper_single_hash_level(_) ->
|
|
||||||
?assertMatch(
|
|
||||||
#{
|
|
||||||
source := [
|
|
||||||
{timestamp, 0, 32},
|
|
||||||
{hash, levels, 16}
|
|
||||||
],
|
|
||||||
bitsize := 48,
|
|
||||||
epoch := 1
|
|
||||||
},
|
|
||||||
keymapper_info(
|
|
||||||
make_keymapper(#{
|
|
||||||
timestamp_bits => 32,
|
|
||||||
topic_bits_per_level => [16],
|
|
||||||
epoch => 1
|
|
||||||
})
|
|
||||||
)
|
|
||||||
).
|
|
||||||
|
|
||||||
t_make_keymapper_no_timestamp(_) ->
|
|
||||||
?assertMatch(
|
|
||||||
#{
|
|
||||||
source := [
|
|
||||||
{hash, level, 4},
|
|
||||||
{hash, level, 8},
|
|
||||||
{hash, levels, 16}
|
|
||||||
],
|
|
||||||
bitsize := 28,
|
|
||||||
epoch := 1
|
|
||||||
},
|
|
||||||
keymapper_info(
|
|
||||||
make_keymapper(#{
|
|
||||||
timestamp_bits => 0,
|
|
||||||
topic_bits_per_level => [4, 8, 16],
|
|
||||||
epoch => 42
|
|
||||||
})
|
|
||||||
)
|
|
||||||
).
|
|
||||||
|
|
||||||
t_compute_topic_bitmask(_) ->
|
|
||||||
KM = make_keymapper(#{topic_bits_per_level => [3, 4, 5, 2], timestamp_bits => 0, epoch => 1}),
|
|
||||||
?assertEqual(
|
|
||||||
2#111_1111_11111_11,
|
|
||||||
compute_topic_bitmask([<<"foo">>, <<"bar">>], KM)
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
2#111_0000_11111_11,
|
|
||||||
compute_topic_bitmask([<<"foo">>, '+'], KM)
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
2#111_0000_00000_11,
|
|
||||||
compute_topic_bitmask([<<"foo">>, '+', '+'], KM)
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
2#111_0000_11111_00,
|
|
||||||
compute_topic_bitmask([<<"foo">>, '+', <<"bar">>, '+'], KM)
|
|
||||||
).
|
|
||||||
|
|
||||||
t_compute_topic_bitmask_wildcard(_) ->
|
|
||||||
KM = make_keymapper(#{topic_bits_per_level => [3, 4, 5, 2], timestamp_bits => 0, epoch => 1}),
|
|
||||||
?assertEqual(
|
|
||||||
2#000_0000_00000_00,
|
|
||||||
compute_topic_bitmask(['#'], KM)
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
2#111_0000_00000_00,
|
|
||||||
compute_topic_bitmask([<<"foo">>, '#'], KM)
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
2#111_1111_11111_00,
|
|
||||||
compute_topic_bitmask([<<"foo">>, <<"bar">>, <<"baz">>, '#'], KM)
|
|
||||||
).
|
|
||||||
|
|
||||||
t_compute_topic_bitmask_wildcard_long_tail(_) ->
|
|
||||||
KM = make_keymapper(#{topic_bits_per_level => [3, 4, 5, 2], timestamp_bits => 0, epoch => 1}),
|
|
||||||
?assertEqual(
|
|
||||||
2#111_1111_11111_11,
|
|
||||||
compute_topic_bitmask([<<"foo">>, <<"bar">>, <<"baz">>, <<>>, <<"xyzzy">>], KM)
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
2#111_1111_11111_00,
|
|
||||||
compute_topic_bitmask([<<"foo">>, <<"bar">>, <<"baz">>, <<>>, '#'], KM)
|
|
||||||
).
|
|
||||||
|
|
||||||
t_compute_time_bitmask(_) ->
|
|
||||||
KM = make_keymapper(#{topic_bits_per_level => [1, 2, 3], timestamp_bits => 10, epoch => 200}),
|
|
||||||
?assertEqual(2#111_000000_1111111, compute_time_bitmask(KM)).
|
|
||||||
|
|
||||||
t_compute_time_bitmask_epoch_only(_) ->
|
|
||||||
KM = make_keymapper(#{topic_bits_per_level => [1, 2, 3], timestamp_bits => 10, epoch => 1}),
|
|
||||||
?assertEqual(2#1111111111_000000, compute_time_bitmask(KM)).
|
|
||||||
|
|
||||||
%% Filter = |123|***|678|***|
|
|
||||||
%% Mask = |123|***|678|***|
|
|
||||||
%% Key1 = |123|011|108|121| → Seek = 0 |123|011|678|000|
|
|
||||||
%% Key2 = |123|011|679|919| → Seek = 0 |123|012|678|000|
|
|
||||||
%% Key3 = |123|999|679|001| → Seek = 1 |123|000|678|000| → eos
|
|
||||||
%% Key4 = |125|011|179|017| → Seek = 1 |123|000|678|000| → eos
|
|
||||||
|
|
||||||
t_compute_next_topic_seek(_) ->
|
|
||||||
KM = make_keymapper(#{topic_bits_per_level => [8, 8, 16, 12], timestamp_bits => 0, epoch => 1}),
|
|
||||||
?assertMatch(
|
|
||||||
none,
|
|
||||||
compute_topic_seek(
|
|
||||||
16#FD_42_4242_043,
|
|
||||||
16#FD_42_4242_042,
|
|
||||||
16#FF_FF_FFFF_FFF,
|
|
||||||
KM
|
|
||||||
)
|
|
||||||
),
|
|
||||||
?assertMatch(
|
|
||||||
16#FD_11_0678_000,
|
|
||||||
compute_topic_seek(
|
|
||||||
16#FD_11_0108_121,
|
|
||||||
16#FD_00_0678_000,
|
|
||||||
16#FF_00_FFFF_000,
|
|
||||||
KM
|
|
||||||
)
|
|
||||||
),
|
|
||||||
?assertMatch(
|
|
||||||
16#FD_12_0678_000,
|
|
||||||
compute_topic_seek(
|
|
||||||
16#FD_11_0679_919,
|
|
||||||
16#FD_00_0678_000,
|
|
||||||
16#FF_00_FFFF_000,
|
|
||||||
KM
|
|
||||||
)
|
|
||||||
),
|
|
||||||
?assertMatch(
|
|
||||||
none,
|
|
||||||
compute_topic_seek(
|
|
||||||
16#FD_FF_0679_001,
|
|
||||||
16#FD_00_0678_000,
|
|
||||||
16#FF_00_FFFF_000,
|
|
||||||
KM
|
|
||||||
)
|
|
||||||
),
|
|
||||||
?assertMatch(
|
|
||||||
none,
|
|
||||||
compute_topic_seek(
|
|
||||||
16#FE_11_0179_017,
|
|
||||||
16#FD_00_0678_000,
|
|
||||||
16#FF_00_FFFF_000,
|
|
||||||
KM
|
|
||||||
)
|
|
||||||
).
|
|
|
@ -0,0 +1,396 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2022-2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
-module(emqx_ds_storage_bitfield_lts_SUITE).
|
||||||
|
|
||||||
|
-compile(export_all).
|
||||||
|
-compile(nowarn_export_all).
|
||||||
|
|
||||||
|
-include_lib("emqx/include/emqx.hrl").
|
||||||
|
-include_lib("common_test/include/ct.hrl").
|
||||||
|
-include_lib("snabbkaffe/include/snabbkaffe.hrl").
|
||||||
|
-include_lib("stdlib/include/assert.hrl").
|
||||||
|
|
||||||
|
-define(SHARD, shard(?FUNCTION_NAME)).
|
||||||
|
|
||||||
|
-define(DEFAULT_CONFIG, #{
|
||||||
|
backend => builtin,
|
||||||
|
storage => {emqx_ds_storage_bitfield_lts, #{}}
|
||||||
|
}).
|
||||||
|
|
||||||
|
-define(COMPACT_CONFIG, #{
|
||||||
|
backend => builtin,
|
||||||
|
storage =>
|
||||||
|
{emqx_ds_storage_bitfield_lts, #{
|
||||||
|
bits_per_wildcard_level => 8
|
||||||
|
}}
|
||||||
|
}).
|
||||||
|
|
||||||
|
%% Smoke test for opening and reopening the database
|
||||||
|
t_open(_Config) ->
|
||||||
|
ok = emqx_ds_storage_layer_sup:stop_shard(?SHARD),
|
||||||
|
{ok, _} = emqx_ds_storage_layer_sup:start_shard(?SHARD, #{}).
|
||||||
|
|
||||||
|
%% Smoke test of store function
|
||||||
|
t_store(_Config) ->
|
||||||
|
MessageID = emqx_guid:gen(),
|
||||||
|
PublishedAt = 1000,
|
||||||
|
Topic = <<"foo/bar">>,
|
||||||
|
Payload = <<"message">>,
|
||||||
|
Msg = #message{
|
||||||
|
id = MessageID,
|
||||||
|
topic = Topic,
|
||||||
|
payload = Payload,
|
||||||
|
timestamp = PublishedAt
|
||||||
|
},
|
||||||
|
?assertMatch(ok, emqx_ds_storage_layer:store_batch(?SHARD, [Msg], #{})).
|
||||||
|
|
||||||
|
%% Smoke test for iteration through a concrete topic
|
||||||
|
t_iterate(_Config) ->
|
||||||
|
%% Prepare data:
|
||||||
|
Topics = [<<"foo/bar">>, <<"foo/bar/baz">>, <<"a">>],
|
||||||
|
Timestamps = lists:seq(1, 10),
|
||||||
|
Batch = [
|
||||||
|
make_message(PublishedAt, Topic, integer_to_binary(PublishedAt))
|
||||||
|
|| Topic <- Topics, PublishedAt <- Timestamps
|
||||||
|
],
|
||||||
|
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch, []),
|
||||||
|
%% Iterate through individual topics:
|
||||||
|
[
|
||||||
|
begin
|
||||||
|
[{_Rank, Stream}] = emqx_ds_storage_layer:get_streams(?SHARD, parse_topic(Topic), 0),
|
||||||
|
{ok, It} = emqx_ds_storage_layer:make_iterator(?SHARD, Stream, parse_topic(Topic), 0),
|
||||||
|
{ok, NextIt, Messages} = emqx_ds_storage_layer:next(?SHARD, It, 100),
|
||||||
|
?assertEqual(
|
||||||
|
lists:map(fun integer_to_binary/1, Timestamps),
|
||||||
|
payloads(Messages)
|
||||||
|
),
|
||||||
|
{ok, _, []} = emqx_ds_storage_layer:next(?SHARD, NextIt, 100)
|
||||||
|
end
|
||||||
|
|| Topic <- Topics
|
||||||
|
],
|
||||||
|
ok.
|
||||||
|
|
||||||
|
-define(assertSameSet(A, B), ?assertEqual(lists:sort(A), lists:sort(B))).
|
||||||
|
|
||||||
|
%% Smoke test that verifies that concrete topics are mapped to
|
||||||
|
%% individual streams, unless there's too many of them.
|
||||||
|
t_get_streams(_Config) ->
|
||||||
|
%% Prepare data (without wildcards):
|
||||||
|
Topics = [<<"foo/bar">>, <<"foo/bar/baz">>, <<"a">>],
|
||||||
|
Timestamps = lists:seq(1, 10),
|
||||||
|
Batch = [
|
||||||
|
make_message(PublishedAt, Topic, integer_to_binary(PublishedAt))
|
||||||
|
|| Topic <- Topics, PublishedAt <- Timestamps
|
||||||
|
],
|
||||||
|
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch, []),
|
||||||
|
GetStream = fun(Topic) ->
|
||||||
|
StartTime = 0,
|
||||||
|
emqx_ds_storage_layer:get_streams(?SHARD, parse_topic(Topic), StartTime)
|
||||||
|
end,
|
||||||
|
%% Get streams for individual topics to use as a reference for later:
|
||||||
|
[FooBar = {_, _}] = GetStream(<<"foo/bar">>),
|
||||||
|
[FooBarBaz] = GetStream(<<"foo/bar/baz">>),
|
||||||
|
[A] = GetStream(<<"a">>),
|
||||||
|
%% Restart shard to make sure trie is persisted and restored:
|
||||||
|
ok = emqx_ds_storage_layer_sup:stop_shard(?SHARD),
|
||||||
|
{ok, _} = emqx_ds_storage_layer_sup:start_shard(?SHARD, #{}),
|
||||||
|
%% Verify that there are no "ghost streams" for topics that don't
|
||||||
|
%% have any messages:
|
||||||
|
[] = GetStream(<<"bar/foo">>),
|
||||||
|
%% Test some wildcard patterns:
|
||||||
|
?assertEqual([FooBar], GetStream("+/+")),
|
||||||
|
?assertSameSet([FooBar, FooBarBaz], GetStream(<<"foo/#">>)),
|
||||||
|
?assertSameSet([FooBar, FooBarBaz, A], GetStream(<<"#">>)),
|
||||||
|
%% Now insert a bunch of messages with different topics to create wildcards:
|
||||||
|
NewBatch = [
|
||||||
|
begin
|
||||||
|
B = integer_to_binary(I),
|
||||||
|
make_message(100, <<"foo/bar/", B/binary>>, <<"filler", B/binary>>)
|
||||||
|
end
|
||||||
|
|| I <- lists:seq(1, 200)
|
||||||
|
],
|
||||||
|
ok = emqx_ds_storage_layer:store_batch(?SHARD, NewBatch, []),
|
||||||
|
%% Check that "foo/bar/baz" topic now appears in two streams:
|
||||||
|
%% "foo/bar/baz" and "foo/bar/+":
|
||||||
|
NewStreams = lists:sort(GetStream("foo/bar/baz")),
|
||||||
|
?assertMatch([_, _], NewStreams),
|
||||||
|
?assert(lists:member(FooBarBaz, NewStreams)),
|
||||||
|
%% Verify that size of the trie is still relatively small, even
|
||||||
|
%% after processing 200+ topics:
|
||||||
|
AllStreams = GetStream("#"),
|
||||||
|
NTotal = length(AllStreams),
|
||||||
|
?assert(NTotal < 30, {NTotal, '<', 30}),
|
||||||
|
?assert(lists:member(FooBar, AllStreams)),
|
||||||
|
?assert(lists:member(FooBarBaz, AllStreams)),
|
||||||
|
?assert(lists:member(A, AllStreams)),
|
||||||
|
ok.
|
||||||
|
|
||||||
|
t_replay(_Config) ->
|
||||||
|
%% Create concrete topics:
|
||||||
|
Topics = [<<"foo/bar">>, <<"foo/bar/baz">>],
|
||||||
|
Timestamps = lists:seq(1, 10_000, 100),
|
||||||
|
Batch1 = [
|
||||||
|
make_message(PublishedAt, Topic, integer_to_binary(PublishedAt))
|
||||||
|
|| Topic <- Topics, PublishedAt <- Timestamps
|
||||||
|
],
|
||||||
|
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch1, []),
|
||||||
|
%% Create wildcard topics `wildcard/+/suffix/foo' and `wildcard/+/suffix/bar':
|
||||||
|
Batch2 = [
|
||||||
|
begin
|
||||||
|
B = integer_to_binary(I),
|
||||||
|
make_message(
|
||||||
|
TS, <<"wildcard/", B/binary, "/suffix/", Suffix/binary>>, integer_to_binary(TS)
|
||||||
|
)
|
||||||
|
end
|
||||||
|
|| I <- lists:seq(1, 200), TS <- Timestamps, Suffix <- [<<"foo">>, <<"bar">>]
|
||||||
|
],
|
||||||
|
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch2, []),
|
||||||
|
%% Check various topic filters:
|
||||||
|
Messages = Batch1 ++ Batch2,
|
||||||
|
%% Missing topics (no ghost messages):
|
||||||
|
?assertNot(check(?SHARD, <<"missing/foo/bar">>, 0, Messages)),
|
||||||
|
%% Regular topics:
|
||||||
|
?assert(check(?SHARD, <<"foo/bar">>, 0, Messages)),
|
||||||
|
?assert(check(?SHARD, <<"foo/bar/baz">>, 0, Messages)),
|
||||||
|
?assert(check(?SHARD, <<"foo/#">>, 0, Messages)),
|
||||||
|
?assert(check(?SHARD, <<"foo/+">>, 0, Messages)),
|
||||||
|
?assert(check(?SHARD, <<"foo/+/+">>, 0, Messages)),
|
||||||
|
?assert(check(?SHARD, <<"+/+/+">>, 0, Messages)),
|
||||||
|
?assert(check(?SHARD, <<"+/+/baz">>, 0, Messages)),
|
||||||
|
%% Restart shard to make sure trie is persisted and restored:
|
||||||
|
ok = emqx_ds_storage_layer_sup:stop_shard(?SHARD),
|
||||||
|
{ok, _} = emqx_ds_storage_layer_sup:start_shard(?SHARD, #{}),
|
||||||
|
%% Learned wildcard topics:
|
||||||
|
?assertNot(check(?SHARD, <<"wildcard/1000/suffix/foo">>, 0, [])),
|
||||||
|
?assert(check(?SHARD, <<"wildcard/1/suffix/foo">>, 0, Messages)),
|
||||||
|
?assert(check(?SHARD, <<"wildcard/100/suffix/foo">>, 0, Messages)),
|
||||||
|
?assert(check(?SHARD, <<"wildcard/+/suffix/foo">>, 0, Messages)),
|
||||||
|
?assert(check(?SHARD, <<"wildcard/1/suffix/+">>, 0, Messages)),
|
||||||
|
?assert(check(?SHARD, <<"wildcard/100/suffix/+">>, 0, Messages)),
|
||||||
|
?assert(check(?SHARD, <<"wildcard/#">>, 0, Messages)),
|
||||||
|
?assert(check(?SHARD, <<"wildcard/1/#">>, 0, Messages)),
|
||||||
|
?assert(check(?SHARD, <<"wildcard/100/#">>, 0, Messages)),
|
||||||
|
?assert(check(?SHARD, <<"#">>, 0, Messages)),
|
||||||
|
ok.
|
||||||
|
|
||||||
|
check(Shard, TopicFilter, StartTime, ExpectedMessages) ->
|
||||||
|
ExpectedFiltered = lists:filter(
|
||||||
|
fun(#message{topic = Topic, timestamp = TS}) ->
|
||||||
|
emqx_topic:match(Topic, TopicFilter) andalso TS >= StartTime
|
||||||
|
end,
|
||||||
|
ExpectedMessages
|
||||||
|
),
|
||||||
|
?check_trace(
|
||||||
|
#{timetrap => 10_000},
|
||||||
|
begin
|
||||||
|
Dump = dump_messages(Shard, TopicFilter, StartTime),
|
||||||
|
verify_dump(TopicFilter, StartTime, Dump),
|
||||||
|
Missing = ExpectedFiltered -- Dump,
|
||||||
|
Extras = Dump -- ExpectedFiltered,
|
||||||
|
?assertMatch(
|
||||||
|
#{missing := [], unexpected := []},
|
||||||
|
#{
|
||||||
|
missing => Missing,
|
||||||
|
unexpected => Extras,
|
||||||
|
topic_filter => TopicFilter,
|
||||||
|
start_time => StartTime
|
||||||
|
}
|
||||||
|
)
|
||||||
|
end,
|
||||||
|
[]
|
||||||
|
),
|
||||||
|
length(ExpectedFiltered) > 0.
|
||||||
|
|
||||||
|
verify_dump(TopicFilter, StartTime, Dump) ->
|
||||||
|
lists:foldl(
|
||||||
|
fun(#message{topic = Topic, timestamp = TS}, Acc) ->
|
||||||
|
%% Verify that the topic of the message returned by the
|
||||||
|
%% iterator matches the expected topic filter:
|
||||||
|
?assert(emqx_topic:match(Topic, TopicFilter), {unexpected_topic, Topic, TopicFilter}),
|
||||||
|
%% Verify that timestamp of the message is greater than
|
||||||
|
%% the StartTime of the iterator:
|
||||||
|
?assert(TS >= StartTime, {start_time, TopicFilter, TS, StartTime}),
|
||||||
|
%% Verify that iterator didn't reorder messages
|
||||||
|
%% (timestamps for each topic are growing):
|
||||||
|
LastTopicTs = maps:get(Topic, Acc, -1),
|
||||||
|
?assert(TS >= LastTopicTs, {topic_ts_reordering, Topic, TS, LastTopicTs}),
|
||||||
|
Acc#{Topic => TS}
|
||||||
|
end,
|
||||||
|
#{},
|
||||||
|
Dump
|
||||||
|
).
|
||||||
|
|
||||||
|
dump_messages(Shard, TopicFilter, StartTime) ->
|
||||||
|
Streams = emqx_ds_storage_layer:get_streams(Shard, parse_topic(TopicFilter), StartTime),
|
||||||
|
lists:flatmap(
|
||||||
|
fun({_Rank, Stream}) ->
|
||||||
|
dump_stream(Shard, Stream, TopicFilter, StartTime)
|
||||||
|
end,
|
||||||
|
Streams
|
||||||
|
).
|
||||||
|
|
||||||
|
dump_stream(Shard, Stream, TopicFilter, StartTime) ->
|
||||||
|
BatchSize = 100,
|
||||||
|
{ok, Iterator} = emqx_ds_storage_layer:make_iterator(
|
||||||
|
Shard, Stream, parse_topic(TopicFilter), StartTime
|
||||||
|
),
|
||||||
|
Loop = fun
|
||||||
|
F(It, 0) ->
|
||||||
|
error({too_many_iterations, It});
|
||||||
|
F(It, N) ->
|
||||||
|
case emqx_ds_storage_layer:next(Shard, It, BatchSize) of
|
||||||
|
end_of_stream ->
|
||||||
|
[];
|
||||||
|
{ok, _NextIt, []} ->
|
||||||
|
[];
|
||||||
|
{ok, NextIt, Batch} ->
|
||||||
|
Batch ++ F(NextIt, N - 1)
|
||||||
|
end
|
||||||
|
end,
|
||||||
|
MaxIterations = 1000000,
|
||||||
|
Loop(Iterator, MaxIterations).
|
||||||
|
|
||||||
|
%% t_create_gen(_Config) ->
|
||||||
|
%% {ok, 1} = emqx_ds_storage_layer:create_generation(?SHARD, 5, ?DEFAULT_CONFIG),
|
||||||
|
%% ?assertEqual(
|
||||||
|
%% {error, nonmonotonic},
|
||||||
|
%% emqx_ds_storage_layer:create_generation(?SHARD, 1, ?DEFAULT_CONFIG)
|
||||||
|
%% ),
|
||||||
|
%% ?assertEqual(
|
||||||
|
%% {error, nonmonotonic},
|
||||||
|
%% emqx_ds_storage_layer:create_generation(?SHARD, 5, ?DEFAULT_CONFIG)
|
||||||
|
%% ),
|
||||||
|
%% {ok, 2} = emqx_ds_storage_layer:create_generation(?SHARD, 10, ?COMPACT_CONFIG),
|
||||||
|
%% Topics = ["foo/bar", "foo/bar/baz"],
|
||||||
|
%% Timestamps = lists:seq(1, 100),
|
||||||
|
%% [
|
||||||
|
%% ?assertMatch({ok, [_]}, store(?SHARD, PublishedAt, Topic, <<>>))
|
||||||
|
%% || Topic <- Topics, PublishedAt <- Timestamps
|
||||||
|
%% ].
|
||||||
|
|
||||||
|
%% t_iterate_multigen(_Config) ->
|
||||||
|
%% {ok, 1} = emqx_ds_storage_layer:create_generation(?SHARD, 10, ?COMPACT_CONFIG),
|
||||||
|
%% {ok, 2} = emqx_ds_storage_layer:create_generation(?SHARD, 50, ?DEFAULT_CONFIG),
|
||||||
|
%% {ok, 3} = emqx_ds_storage_layer:create_generation(?SHARD, 1000, ?DEFAULT_CONFIG),
|
||||||
|
%% Topics = ["foo/bar", "foo/bar/baz", "a", "a/bar"],
|
||||||
|
%% Timestamps = lists:seq(1, 100),
|
||||||
|
%% _ = [
|
||||||
|
%% store(?SHARD, PublishedAt, Topic, term_to_binary({Topic, PublishedAt}))
|
||||||
|
%% || Topic <- Topics, PublishedAt <- Timestamps
|
||||||
|
%% ],
|
||||||
|
%% ?assertEqual(
|
||||||
|
%% lists:sort([
|
||||||
|
%% {Topic, PublishedAt}
|
||||||
|
%% || Topic <- ["foo/bar", "foo/bar/baz"], PublishedAt <- Timestamps
|
||||||
|
%% ]),
|
||||||
|
%% lists:sort([binary_to_term(Payload) || Payload <- iterate(?SHARD, "foo/#", 0)])
|
||||||
|
%% ),
|
||||||
|
%% ?assertEqual(
|
||||||
|
%% lists:sort([
|
||||||
|
%% {Topic, PublishedAt}
|
||||||
|
%% || Topic <- ["a", "a/bar"], PublishedAt <- lists:seq(60, 100)
|
||||||
|
%% ]),
|
||||||
|
%% lists:sort([binary_to_term(Payload) || Payload <- iterate(?SHARD, "a/#", 60)])
|
||||||
|
%% ).
|
||||||
|
|
||||||
|
%% t_iterate_multigen_preserve_restore(_Config) ->
|
||||||
|
%% ReplayID = atom_to_binary(?FUNCTION_NAME),
|
||||||
|
%% {ok, 1} = emqx_ds_storage_layer:create_generation(?SHARD, 10, ?COMPACT_CONFIG),
|
||||||
|
%% {ok, 2} = emqx_ds_storage_layer:create_generation(?SHARD, 50, ?DEFAULT_CONFIG),
|
||||||
|
%% {ok, 3} = emqx_ds_storage_layer:create_generation(?SHARD, 100, ?DEFAULT_CONFIG),
|
||||||
|
%% Topics = ["foo/bar", "foo/bar/baz", "a/bar"],
|
||||||
|
%% Timestamps = lists:seq(1, 100),
|
||||||
|
%% TopicFilter = "foo/#",
|
||||||
|
%% TopicsMatching = ["foo/bar", "foo/bar/baz"],
|
||||||
|
%% _ = [
|
||||||
|
%% store(?SHARD, TS, Topic, term_to_binary({Topic, TS}))
|
||||||
|
%% || Topic <- Topics, TS <- Timestamps
|
||||||
|
%% ],
|
||||||
|
%% It0 = iterator(?SHARD, TopicFilter, 0),
|
||||||
|
%% {It1, Res10} = iterate(It0, 10),
|
||||||
|
%% % preserve mid-generation
|
||||||
|
%% ok = emqx_ds_storage_layer:preserve_iterator(It1, ReplayID),
|
||||||
|
%% {ok, It2} = emqx_ds_storage_layer:restore_iterator(?SHARD, ReplayID),
|
||||||
|
%% {It3, Res100} = iterate(It2, 88),
|
||||||
|
%% % preserve on the generation boundary
|
||||||
|
%% ok = emqx_ds_storage_layer:preserve_iterator(It3, ReplayID),
|
||||||
|
%% {ok, It4} = emqx_ds_storage_layer:restore_iterator(?SHARD, ReplayID),
|
||||||
|
%% {It5, Res200} = iterate(It4, 1000),
|
||||||
|
%% ?assertEqual({end_of_stream, []}, iterate(It5, 1)),
|
||||||
|
%% ?assertEqual(
|
||||||
|
%% lists:sort([{Topic, TS} || Topic <- TopicsMatching, TS <- Timestamps]),
|
||||||
|
%% lists:sort([binary_to_term(Payload) || Payload <- Res10 ++ Res100 ++ Res200])
|
||||||
|
%% ),
|
||||||
|
%% ?assertEqual(
|
||||||
|
%% ok,
|
||||||
|
%% emqx_ds_storage_layer:discard_iterator(?SHARD, ReplayID)
|
||||||
|
%% ),
|
||||||
|
%% ?assertEqual(
|
||||||
|
%% {error, not_found},
|
||||||
|
%% emqx_ds_storage_layer:restore_iterator(?SHARD, ReplayID)
|
||||||
|
%% ).
|
||||||
|
|
||||||
|
make_message(PublishedAt, Topic, Payload) when is_list(Topic) ->
|
||||||
|
make_message(PublishedAt, list_to_binary(Topic), Payload);
|
||||||
|
make_message(PublishedAt, Topic, Payload) when is_binary(Topic) ->
|
||||||
|
ID = emqx_guid:gen(),
|
||||||
|
#message{
|
||||||
|
id = ID,
|
||||||
|
topic = Topic,
|
||||||
|
timestamp = PublishedAt,
|
||||||
|
payload = Payload
|
||||||
|
}.
|
||||||
|
|
||||||
|
store(Shard, PublishedAt, TopicL, Payload) when is_list(TopicL) ->
|
||||||
|
store(Shard, PublishedAt, list_to_binary(TopicL), Payload);
|
||||||
|
store(Shard, PublishedAt, Topic, Payload) ->
|
||||||
|
ID = emqx_guid:gen(),
|
||||||
|
Msg = #message{
|
||||||
|
id = ID,
|
||||||
|
topic = Topic,
|
||||||
|
timestamp = PublishedAt,
|
||||||
|
payload = Payload
|
||||||
|
},
|
||||||
|
emqx_ds_storage_layer:message_store(Shard, [Msg], #{}).
|
||||||
|
|
||||||
|
payloads(Messages) ->
|
||||||
|
lists:map(
|
||||||
|
fun(#message{payload = P}) ->
|
||||||
|
P
|
||||||
|
end,
|
||||||
|
Messages
|
||||||
|
).
|
||||||
|
|
||||||
|
parse_topic(Topic = [L | _]) when is_binary(L); is_atom(L) ->
|
||||||
|
Topic;
|
||||||
|
parse_topic(Topic) ->
|
||||||
|
emqx_topic:words(iolist_to_binary(Topic)).
|
||||||
|
|
||||||
|
%% CT callbacks
|
||||||
|
|
||||||
|
all() -> emqx_common_test_helpers:all(?MODULE).
|
||||||
|
suite() -> [{timetrap, {seconds, 20}}].
|
||||||
|
|
||||||
|
init_per_suite(Config) ->
|
||||||
|
{ok, _} = application:ensure_all_started(emqx_durable_storage),
|
||||||
|
Config.
|
||||||
|
|
||||||
|
end_per_suite(_Config) ->
|
||||||
|
ok = application:stop(emqx_durable_storage).
|
||||||
|
|
||||||
|
init_per_testcase(TC, Config) ->
|
||||||
|
{ok, _} = emqx_ds_storage_layer_sup:start_shard(shard(TC), ?DEFAULT_CONFIG),
|
||||||
|
Config.
|
||||||
|
|
||||||
|
end_per_testcase(TC, _Config) ->
|
||||||
|
ok = emqx_ds_storage_layer_sup:stop_shard(shard(TC)).
|
||||||
|
|
||||||
|
shard(TC) ->
|
||||||
|
{?MODULE, TC}.
|
||||||
|
|
||||||
|
keyspace(TC) ->
|
||||||
|
TC.
|
||||||
|
|
||||||
|
set_keyspace_config(Keyspace, Config) ->
|
||||||
|
ok = application:set_env(emqx_ds, keyspace_config, #{Keyspace => Config}).
|
|
@ -1,282 +0,0 @@
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
%% Copyright (c) 2022-2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
-module(emqx_ds_storage_layer_SUITE).
|
|
||||||
|
|
||||||
-compile(export_all).
|
|
||||||
-compile(nowarn_export_all).
|
|
||||||
|
|
||||||
-include_lib("common_test/include/ct.hrl").
|
|
||||||
-include_lib("stdlib/include/assert.hrl").
|
|
||||||
|
|
||||||
-define(SHARD, shard(?FUNCTION_NAME)).
|
|
||||||
|
|
||||||
-define(DEFAULT_CONFIG,
|
|
||||||
{emqx_ds_message_storage_bitmask, #{
|
|
||||||
timestamp_bits => 64,
|
|
||||||
topic_bits_per_level => [8, 8, 32, 16],
|
|
||||||
epoch => 5,
|
|
||||||
iteration => #{
|
|
||||||
iterator_refresh => {every, 5}
|
|
||||||
}
|
|
||||||
}}
|
|
||||||
).
|
|
||||||
|
|
||||||
-define(COMPACT_CONFIG,
|
|
||||||
{emqx_ds_message_storage_bitmask, #{
|
|
||||||
timestamp_bits => 16,
|
|
||||||
topic_bits_per_level => [16, 16],
|
|
||||||
epoch => 10
|
|
||||||
}}
|
|
||||||
).
|
|
||||||
|
|
||||||
%% Smoke test for opening and reopening the database
|
|
||||||
t_open(_Config) ->
|
|
||||||
ok = emqx_ds_storage_layer_sup:stop_shard(?SHARD),
|
|
||||||
{ok, _} = emqx_ds_storage_layer_sup:start_shard(?SHARD, #{}).
|
|
||||||
|
|
||||||
%% Smoke test of store function
|
|
||||||
t_store(_Config) ->
|
|
||||||
MessageID = emqx_guid:gen(),
|
|
||||||
PublishedAt = 1000,
|
|
||||||
Topic = [<<"foo">>, <<"bar">>],
|
|
||||||
Payload = <<"message">>,
|
|
||||||
?assertMatch(ok, emqx_ds_storage_layer:store(?SHARD, MessageID, PublishedAt, Topic, Payload)).
|
|
||||||
|
|
||||||
%% Smoke test for iteration through a concrete topic
|
|
||||||
t_iterate(_Config) ->
|
|
||||||
%% Prepare data:
|
|
||||||
Topics = [[<<"foo">>, <<"bar">>], [<<"foo">>, <<"bar">>, <<"baz">>], [<<"a">>]],
|
|
||||||
Timestamps = lists:seq(1, 10),
|
|
||||||
[
|
|
||||||
emqx_ds_storage_layer:store(
|
|
||||||
?SHARD,
|
|
||||||
emqx_guid:gen(),
|
|
||||||
PublishedAt,
|
|
||||||
Topic,
|
|
||||||
integer_to_binary(PublishedAt)
|
|
||||||
)
|
|
||||||
|| Topic <- Topics, PublishedAt <- Timestamps
|
|
||||||
],
|
|
||||||
%% Iterate through individual topics:
|
|
||||||
[
|
|
||||||
begin
|
|
||||||
{ok, It} = emqx_ds_storage_layer:make_iterator(?SHARD, {Topic, 0}),
|
|
||||||
Values = iterate(It),
|
|
||||||
?assertEqual(lists:map(fun integer_to_binary/1, Timestamps), Values)
|
|
||||||
end
|
|
||||||
|| Topic <- Topics
|
|
||||||
],
|
|
||||||
ok.
|
|
||||||
|
|
||||||
%% Smoke test for iteration with wildcard topic filter
|
|
||||||
t_iterate_wildcard(_Config) ->
|
|
||||||
%% Prepare data:
|
|
||||||
Topics = ["foo/bar", "foo/bar/baz", "a", "a/bar"],
|
|
||||||
Timestamps = lists:seq(1, 10),
|
|
||||||
_ = [
|
|
||||||
store(?SHARD, PublishedAt, Topic, term_to_binary({Topic, PublishedAt}))
|
|
||||||
|| Topic <- Topics, PublishedAt <- Timestamps
|
|
||||||
],
|
|
||||||
?assertEqual(
|
|
||||||
lists:sort([{Topic, PublishedAt} || Topic <- Topics, PublishedAt <- Timestamps]),
|
|
||||||
lists:sort([binary_to_term(Payload) || Payload <- iterate(?SHARD, "#", 0)])
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
[],
|
|
||||||
lists:sort([binary_to_term(Payload) || Payload <- iterate(?SHARD, "#", 10 + 1)])
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
lists:sort([{Topic, PublishedAt} || Topic <- Topics, PublishedAt <- lists:seq(5, 10)]),
|
|
||||||
lists:sort([binary_to_term(Payload) || Payload <- iterate(?SHARD, "#", 5)])
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
lists:sort([
|
|
||||||
{Topic, PublishedAt}
|
|
||||||
|| Topic <- ["foo/bar", "foo/bar/baz"], PublishedAt <- Timestamps
|
|
||||||
]),
|
|
||||||
lists:sort([binary_to_term(Payload) || Payload <- iterate(?SHARD, "foo/#", 0)])
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
lists:sort([{"foo/bar", PublishedAt} || PublishedAt <- Timestamps]),
|
|
||||||
lists:sort([binary_to_term(Payload) || Payload <- iterate(?SHARD, "foo/+", 0)])
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
[],
|
|
||||||
lists:sort([binary_to_term(Payload) || Payload <- iterate(?SHARD, "foo/+/bar", 0)])
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
lists:sort([
|
|
||||||
{Topic, PublishedAt}
|
|
||||||
|| Topic <- ["foo/bar", "foo/bar/baz", "a/bar"], PublishedAt <- Timestamps
|
|
||||||
]),
|
|
||||||
lists:sort([binary_to_term(Payload) || Payload <- iterate(?SHARD, "+/bar/#", 0)])
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
lists:sort([{Topic, PublishedAt} || Topic <- ["a", "a/bar"], PublishedAt <- Timestamps]),
|
|
||||||
lists:sort([binary_to_term(Payload) || Payload <- iterate(?SHARD, "a/#", 0)])
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
[],
|
|
||||||
lists:sort([binary_to_term(Payload) || Payload <- iterate(?SHARD, "a/+/+", 0)])
|
|
||||||
),
|
|
||||||
ok.
|
|
||||||
|
|
||||||
t_iterate_long_tail_wildcard(_Config) ->
|
|
||||||
Topic = "b/c/d/e/f/g",
|
|
||||||
TopicFilter = "b/c/d/e/+/+",
|
|
||||||
Timestamps = lists:seq(1, 100),
|
|
||||||
_ = [
|
|
||||||
store(?SHARD, PublishedAt, Topic, term_to_binary({Topic, PublishedAt}))
|
|
||||||
|| PublishedAt <- Timestamps
|
|
||||||
],
|
|
||||||
?assertEqual(
|
|
||||||
lists:sort([{"b/c/d/e/f/g", PublishedAt} || PublishedAt <- lists:seq(50, 100)]),
|
|
||||||
lists:sort([binary_to_term(Payload) || Payload <- iterate(?SHARD, TopicFilter, 50)])
|
|
||||||
).
|
|
||||||
|
|
||||||
t_create_gen(_Config) ->
|
|
||||||
{ok, 1} = emqx_ds_storage_layer:create_generation(?SHARD, 5, ?DEFAULT_CONFIG),
|
|
||||||
?assertEqual(
|
|
||||||
{error, nonmonotonic},
|
|
||||||
emqx_ds_storage_layer:create_generation(?SHARD, 1, ?DEFAULT_CONFIG)
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
{error, nonmonotonic},
|
|
||||||
emqx_ds_storage_layer:create_generation(?SHARD, 5, ?DEFAULT_CONFIG)
|
|
||||||
),
|
|
||||||
{ok, 2} = emqx_ds_storage_layer:create_generation(?SHARD, 10, ?COMPACT_CONFIG),
|
|
||||||
Topics = ["foo/bar", "foo/bar/baz"],
|
|
||||||
Timestamps = lists:seq(1, 100),
|
|
||||||
[
|
|
||||||
?assertEqual(ok, store(?SHARD, PublishedAt, Topic, <<>>))
|
|
||||||
|| Topic <- Topics, PublishedAt <- Timestamps
|
|
||||||
].
|
|
||||||
|
|
||||||
t_iterate_multigen(_Config) ->
|
|
||||||
{ok, 1} = emqx_ds_storage_layer:create_generation(?SHARD, 10, ?COMPACT_CONFIG),
|
|
||||||
{ok, 2} = emqx_ds_storage_layer:create_generation(?SHARD, 50, ?DEFAULT_CONFIG),
|
|
||||||
{ok, 3} = emqx_ds_storage_layer:create_generation(?SHARD, 1000, ?DEFAULT_CONFIG),
|
|
||||||
Topics = ["foo/bar", "foo/bar/baz", "a", "a/bar"],
|
|
||||||
Timestamps = lists:seq(1, 100),
|
|
||||||
_ = [
|
|
||||||
store(?SHARD, PublishedAt, Topic, term_to_binary({Topic, PublishedAt}))
|
|
||||||
|| Topic <- Topics, PublishedAt <- Timestamps
|
|
||||||
],
|
|
||||||
?assertEqual(
|
|
||||||
lists:sort([
|
|
||||||
{Topic, PublishedAt}
|
|
||||||
|| Topic <- ["foo/bar", "foo/bar/baz"], PublishedAt <- Timestamps
|
|
||||||
]),
|
|
||||||
lists:sort([binary_to_term(Payload) || Payload <- iterate(?SHARD, "foo/#", 0)])
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
lists:sort([
|
|
||||||
{Topic, PublishedAt}
|
|
||||||
|| Topic <- ["a", "a/bar"], PublishedAt <- lists:seq(60, 100)
|
|
||||||
]),
|
|
||||||
lists:sort([binary_to_term(Payload) || Payload <- iterate(?SHARD, "a/#", 60)])
|
|
||||||
).
|
|
||||||
|
|
||||||
t_iterate_multigen_preserve_restore(_Config) ->
|
|
||||||
ReplayID = atom_to_binary(?FUNCTION_NAME),
|
|
||||||
{ok, 1} = emqx_ds_storage_layer:create_generation(?SHARD, 10, ?COMPACT_CONFIG),
|
|
||||||
{ok, 2} = emqx_ds_storage_layer:create_generation(?SHARD, 50, ?DEFAULT_CONFIG),
|
|
||||||
{ok, 3} = emqx_ds_storage_layer:create_generation(?SHARD, 100, ?DEFAULT_CONFIG),
|
|
||||||
Topics = ["foo/bar", "foo/bar/baz", "a/bar"],
|
|
||||||
Timestamps = lists:seq(1, 100),
|
|
||||||
TopicFilter = "foo/#",
|
|
||||||
TopicsMatching = ["foo/bar", "foo/bar/baz"],
|
|
||||||
_ = [
|
|
||||||
store(?SHARD, TS, Topic, term_to_binary({Topic, TS}))
|
|
||||||
|| Topic <- Topics, TS <- Timestamps
|
|
||||||
],
|
|
||||||
It0 = iterator(?SHARD, TopicFilter, 0),
|
|
||||||
{It1, Res10} = iterate(It0, 10),
|
|
||||||
% preserve mid-generation
|
|
||||||
ok = emqx_ds_storage_layer:preserve_iterator(It1, ReplayID),
|
|
||||||
{ok, It2} = emqx_ds_storage_layer:restore_iterator(?SHARD, ReplayID),
|
|
||||||
{It3, Res100} = iterate(It2, 88),
|
|
||||||
% preserve on the generation boundary
|
|
||||||
ok = emqx_ds_storage_layer:preserve_iterator(It3, ReplayID),
|
|
||||||
{ok, It4} = emqx_ds_storage_layer:restore_iterator(?SHARD, ReplayID),
|
|
||||||
{It5, Res200} = iterate(It4, 1000),
|
|
||||||
?assertEqual(none, It5),
|
|
||||||
?assertEqual(
|
|
||||||
lists:sort([{Topic, TS} || Topic <- TopicsMatching, TS <- Timestamps]),
|
|
||||||
lists:sort([binary_to_term(Payload) || Payload <- Res10 ++ Res100 ++ Res200])
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
ok,
|
|
||||||
emqx_ds_storage_layer:discard_iterator(?SHARD, ReplayID)
|
|
||||||
),
|
|
||||||
?assertEqual(
|
|
||||||
{error, not_found},
|
|
||||||
emqx_ds_storage_layer:restore_iterator(?SHARD, ReplayID)
|
|
||||||
).
|
|
||||||
|
|
||||||
store(Shard, PublishedAt, Topic, Payload) ->
|
|
||||||
ID = emqx_guid:gen(),
|
|
||||||
emqx_ds_storage_layer:store(Shard, ID, PublishedAt, parse_topic(Topic), Payload).
|
|
||||||
|
|
||||||
iterate(DB, TopicFilter, StartTime) ->
|
|
||||||
iterate(iterator(DB, TopicFilter, StartTime)).
|
|
||||||
|
|
||||||
iterate(It) ->
|
|
||||||
case emqx_ds_storage_layer:next(It) of
|
|
||||||
{value, Payload, ItNext} ->
|
|
||||||
[Payload | iterate(ItNext)];
|
|
||||||
none ->
|
|
||||||
[]
|
|
||||||
end.
|
|
||||||
|
|
||||||
iterate(It, 0) ->
|
|
||||||
{It, []};
|
|
||||||
iterate(It, N) ->
|
|
||||||
case emqx_ds_storage_layer:next(It) of
|
|
||||||
{value, Payload, ItNext} ->
|
|
||||||
{ItFinal, Ps} = iterate(ItNext, N - 1),
|
|
||||||
{ItFinal, [Payload | Ps]};
|
|
||||||
none ->
|
|
||||||
{none, []}
|
|
||||||
end.
|
|
||||||
|
|
||||||
iterator(DB, TopicFilter, StartTime) ->
|
|
||||||
{ok, It} = emqx_ds_storage_layer:make_iterator(DB, {parse_topic(TopicFilter), StartTime}),
|
|
||||||
It.
|
|
||||||
|
|
||||||
parse_topic(Topic = [L | _]) when is_binary(L); is_atom(L) ->
|
|
||||||
Topic;
|
|
||||||
parse_topic(Topic) ->
|
|
||||||
emqx_topic:words(iolist_to_binary(Topic)).
|
|
||||||
|
|
||||||
%% CT callbacks
|
|
||||||
|
|
||||||
all() -> emqx_common_test_helpers:all(?MODULE).
|
|
||||||
|
|
||||||
init_per_suite(Config) ->
|
|
||||||
{ok, _} = application:ensure_all_started(emqx_durable_storage),
|
|
||||||
Config.
|
|
||||||
|
|
||||||
end_per_suite(_Config) ->
|
|
||||||
ok = application:stop(emqx_durable_storage).
|
|
||||||
|
|
||||||
init_per_testcase(TC, Config) ->
|
|
||||||
ok = set_keyspace_config(keyspace(TC), ?DEFAULT_CONFIG),
|
|
||||||
{ok, _} = emqx_ds_storage_layer_sup:start_shard(shard(TC), #{}),
|
|
||||||
Config.
|
|
||||||
|
|
||||||
end_per_testcase(TC, _Config) ->
|
|
||||||
ok = emqx_ds_storage_layer_sup:stop_shard(shard(TC)).
|
|
||||||
|
|
||||||
keyspace(TC) ->
|
|
||||||
list_to_atom(lists:concat([?MODULE, "_", TC])).
|
|
||||||
|
|
||||||
shard_id(_TC) ->
|
|
||||||
<<"shard">>.
|
|
||||||
|
|
||||||
shard(TC) ->
|
|
||||||
{keyspace(TC), shard_id(TC)}.
|
|
||||||
|
|
||||||
set_keyspace_config(Keyspace, Config) ->
|
|
||||||
ok = application:set_env(emqx_ds, keyspace_config, #{Keyspace => Config}).
|
|
|
@ -4,9 +4,11 @@
|
||||||
|
|
||||||
-module(emqx_ds_message_storage_bitmask_shim).
|
-module(emqx_ds_message_storage_bitmask_shim).
|
||||||
|
|
||||||
|
-include_lib("emqx/include/emqx.hrl").
|
||||||
|
|
||||||
-export([open/0]).
|
-export([open/0]).
|
||||||
-export([close/1]).
|
-export([close/1]).
|
||||||
-export([store/5]).
|
-export([store/2]).
|
||||||
-export([iterate/2]).
|
-export([iterate/2]).
|
||||||
|
|
||||||
-type topic() :: list(binary()).
|
-type topic() :: list(binary()).
|
||||||
|
@ -25,20 +27,21 @@ close(Tab) ->
|
||||||
true = ets:delete(Tab),
|
true = ets:delete(Tab),
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
-spec store(t(), emqx_guid:guid(), time(), topic(), binary()) ->
|
-spec store(t(), emqx_types:message()) ->
|
||||||
ok | {error, _TODO}.
|
ok | {error, _TODO}.
|
||||||
store(Tab, MessageID, PublishedAt, Topic, Payload) ->
|
store(Tab, Msg = #message{id = MessageID, timestamp = PublishedAt}) ->
|
||||||
true = ets:insert(Tab, {{PublishedAt, MessageID}, Topic, Payload}),
|
true = ets:insert(Tab, {{PublishedAt, MessageID}, Msg}),
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
-spec iterate(t(), emqx_ds:replay()) ->
|
-spec iterate(t(), emqx_ds:replay()) ->
|
||||||
[binary()].
|
[binary()].
|
||||||
iterate(Tab, {TopicFilter, StartTime}) ->
|
iterate(Tab, {TopicFilter0, StartTime}) ->
|
||||||
|
TopicFilter = iolist_to_binary(lists:join("/", TopicFilter0)),
|
||||||
ets:foldr(
|
ets:foldr(
|
||||||
fun({{PublishedAt, _}, Topic, Payload}, Acc) ->
|
fun({{PublishedAt, _}, Msg = #message{topic = Topic}}, Acc) ->
|
||||||
case emqx_topic:match(Topic, TopicFilter) of
|
case emqx_topic:match(Topic, TopicFilter) of
|
||||||
true when PublishedAt >= StartTime ->
|
true when PublishedAt >= StartTime ->
|
||||||
[Payload | Acc];
|
[Msg | Acc];
|
||||||
_ ->
|
_ ->
|
||||||
Acc
|
Acc
|
||||||
end
|
end
|
||||||
|
|
|
@ -1,466 +0,0 @@
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
%% Copyright (c) 2020-2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
|
|
||||||
-module(prop_replay_message_storage).
|
|
||||||
|
|
||||||
-include_lib("proper/include/proper.hrl").
|
|
||||||
-include_lib("eunit/include/eunit.hrl").
|
|
||||||
|
|
||||||
-define(WORK_DIR, ["_build", "test"]).
|
|
||||||
-define(RUN_ID, {?MODULE, testrun_id}).
|
|
||||||
|
|
||||||
-define(KEYSPACE, ?MODULE).
|
|
||||||
-define(SHARD_ID, <<"shard">>).
|
|
||||||
-define(SHARD, {?KEYSPACE, ?SHARD_ID}).
|
|
||||||
-define(GEN_ID, 42).
|
|
||||||
|
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
%% Properties
|
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
|
|
||||||
prop_bitstring_computes() ->
|
|
||||||
?FORALL(
|
|
||||||
Keymapper,
|
|
||||||
keymapper(),
|
|
||||||
?FORALL({Topic, Timestamp}, {topic(), integer()}, begin
|
|
||||||
BS = emqx_ds_message_storage_bitmask:compute_bitstring(Topic, Timestamp, Keymapper),
|
|
||||||
is_integer(BS) andalso (BS < (1 bsl get_keymapper_bitsize(Keymapper)))
|
|
||||||
end)
|
|
||||||
).
|
|
||||||
|
|
||||||
prop_topic_bitmask_computes() ->
|
|
||||||
Keymapper = make_keymapper(16, [8, 12, 16], 100),
|
|
||||||
?FORALL(TopicFilter, topic_filter(), begin
|
|
||||||
Mask = emqx_ds_message_storage_bitmask:compute_topic_bitmask(TopicFilter, Keymapper),
|
|
||||||
% topic bits + timestamp LSBs
|
|
||||||
is_integer(Mask) andalso (Mask < (1 bsl (36 + 6)))
|
|
||||||
end).
|
|
||||||
|
|
||||||
prop_next_seek_monotonic() ->
|
|
||||||
?FORALL(
|
|
||||||
{TopicFilter, StartTime, Keymapper},
|
|
||||||
{topic_filter(), pos_integer(), keymapper()},
|
|
||||||
begin
|
|
||||||
Filter = emqx_ds_message_storage_bitmask:make_keyspace_filter(
|
|
||||||
{TopicFilter, StartTime},
|
|
||||||
Keymapper
|
|
||||||
),
|
|
||||||
?FORALL(
|
|
||||||
Bitstring,
|
|
||||||
bitstr(get_keymapper_bitsize(Keymapper)),
|
|
||||||
emqx_ds_message_storage_bitmask:compute_next_seek(Bitstring, Filter) >= Bitstring
|
|
||||||
)
|
|
||||||
end
|
|
||||||
).
|
|
||||||
|
|
||||||
prop_next_seek_eq_initial_seek() ->
|
|
||||||
?FORALL(
|
|
||||||
Filter,
|
|
||||||
keyspace_filter(),
|
|
||||||
emqx_ds_message_storage_bitmask:compute_initial_seek(Filter) =:=
|
|
||||||
emqx_ds_message_storage_bitmask:compute_next_seek(0, Filter)
|
|
||||||
).
|
|
||||||
|
|
||||||
prop_iterate_messages() ->
|
|
||||||
TBPL = [4, 8, 12],
|
|
||||||
Options = #{
|
|
||||||
timestamp_bits => 32,
|
|
||||||
topic_bits_per_level => TBPL,
|
|
||||||
epoch => 200
|
|
||||||
},
|
|
||||||
% TODO
|
|
||||||
% Shrinking is too unpredictable and leaves a LOT of garbage in the scratch dit.
|
|
||||||
?FORALL(Stream, noshrink(non_empty(messages(topic(TBPL)))), begin
|
|
||||||
Filepath = make_filepath(?FUNCTION_NAME, erlang:system_time(microsecond)),
|
|
||||||
{DB, Handle} = open_db(Filepath, Options),
|
|
||||||
Shim = emqx_ds_message_storage_bitmask_shim:open(),
|
|
||||||
ok = store_db(DB, Stream),
|
|
||||||
ok = store_shim(Shim, Stream),
|
|
||||||
?FORALL(
|
|
||||||
{
|
|
||||||
{Topic, _},
|
|
||||||
Pattern,
|
|
||||||
StartTime
|
|
||||||
},
|
|
||||||
{
|
|
||||||
nth(Stream),
|
|
||||||
topic_filter_pattern(),
|
|
||||||
start_time()
|
|
||||||
},
|
|
||||||
begin
|
|
||||||
TopicFilter = make_topic_filter(Pattern, Topic),
|
|
||||||
Iteration = {TopicFilter, StartTime},
|
|
||||||
Messages = iterate_db(DB, Iteration),
|
|
||||||
Reference = iterate_shim(Shim, Iteration),
|
|
||||||
ok = close_db(Handle),
|
|
||||||
ok = emqx_ds_message_storage_bitmask_shim:close(Shim),
|
|
||||||
?WHENFAIL(
|
|
||||||
begin
|
|
||||||
io:format(user, " *** Filepath = ~s~n", [Filepath]),
|
|
||||||
io:format(user, " *** TopicFilter = ~p~n", [TopicFilter]),
|
|
||||||
io:format(user, " *** StartTime = ~p~n", [StartTime])
|
|
||||||
end,
|
|
||||||
is_list(Messages) andalso equals(Messages -- Reference, Reference -- Messages)
|
|
||||||
)
|
|
||||||
end
|
|
||||||
)
|
|
||||||
end).
|
|
||||||
|
|
||||||
prop_iterate_eq_iterate_with_preserve_restore() ->
|
|
||||||
TBPL = [4, 8, 16, 12],
|
|
||||||
Options = #{
|
|
||||||
timestamp_bits => 32,
|
|
||||||
topic_bits_per_level => TBPL,
|
|
||||||
epoch => 500
|
|
||||||
},
|
|
||||||
{DB, _Handle} = open_db(make_filepath(?FUNCTION_NAME), Options),
|
|
||||||
?FORALL(Stream, non_empty(messages(topic(TBPL))), begin
|
|
||||||
% TODO
|
|
||||||
% This proptest is impure because messages from testruns assumed to be
|
|
||||||
% independent of each other are accumulated in the same storage. This
|
|
||||||
% would probably confuse shrinker in the event a testrun fails.
|
|
||||||
ok = store_db(DB, Stream),
|
|
||||||
?FORALL(
|
|
||||||
{
|
|
||||||
{Topic, _},
|
|
||||||
Pat,
|
|
||||||
StartTime,
|
|
||||||
Commands
|
|
||||||
},
|
|
||||||
{
|
|
||||||
nth(Stream),
|
|
||||||
topic_filter_pattern(),
|
|
||||||
start_time(),
|
|
||||||
shuffled(flat([non_empty(list({preserve, restore})), list(iterate)]))
|
|
||||||
},
|
|
||||||
begin
|
|
||||||
Replay = {make_topic_filter(Pat, Topic), StartTime},
|
|
||||||
Iterator = make_iterator(DB, Replay),
|
|
||||||
Ctx = #{db => DB, replay => Replay},
|
|
||||||
Messages = run_iterator_commands(Commands, Iterator, Ctx),
|
|
||||||
equals(Messages, iterate_db(DB, Replay))
|
|
||||||
end
|
|
||||||
)
|
|
||||||
end).
|
|
||||||
|
|
||||||
prop_iterate_eq_iterate_with_refresh() ->
|
|
||||||
TBPL = [4, 8, 16, 12],
|
|
||||||
Options = #{
|
|
||||||
timestamp_bits => 32,
|
|
||||||
topic_bits_per_level => TBPL,
|
|
||||||
epoch => 500
|
|
||||||
},
|
|
||||||
{DB, _Handle} = open_db(make_filepath(?FUNCTION_NAME), Options),
|
|
||||||
?FORALL(Stream, non_empty(messages(topic(TBPL))), begin
|
|
||||||
% TODO
|
|
||||||
% This proptest is also impure, see above.
|
|
||||||
ok = store_db(DB, Stream),
|
|
||||||
?FORALL(
|
|
||||||
{
|
|
||||||
{Topic, _},
|
|
||||||
Pat,
|
|
||||||
StartTime,
|
|
||||||
RefreshEvery
|
|
||||||
},
|
|
||||||
{
|
|
||||||
nth(Stream),
|
|
||||||
topic_filter_pattern(),
|
|
||||||
start_time(),
|
|
||||||
pos_integer()
|
|
||||||
},
|
|
||||||
?TIMEOUT(5000, begin
|
|
||||||
Replay = {make_topic_filter(Pat, Topic), StartTime},
|
|
||||||
IterationOptions = #{iterator_refresh => {every, RefreshEvery}},
|
|
||||||
Iterator = make_iterator(DB, Replay, IterationOptions),
|
|
||||||
Messages = iterate_db(Iterator),
|
|
||||||
equals(Messages, iterate_db(DB, Replay))
|
|
||||||
end)
|
|
||||||
)
|
|
||||||
end).
|
|
||||||
|
|
||||||
% store_message_stream(DB, [{Topic, {Payload, ChunkNum, _ChunkCount}} | Rest]) ->
|
|
||||||
% MessageID = emqx_guid:gen(),
|
|
||||||
% PublishedAt = ChunkNum,
|
|
||||||
% MessageID, PublishedAt, Topic
|
|
||||||
% ]),
|
|
||||||
% ok = emqx_ds_message_storage_bitmask:store(DB, MessageID, PublishedAt, Topic, Payload),
|
|
||||||
% store_message_stream(DB, payload_gen:next(Rest));
|
|
||||||
% store_message_stream(_Zone, []) ->
|
|
||||||
% ok.
|
|
||||||
|
|
||||||
store_db(DB, Messages) ->
|
|
||||||
lists:foreach(
|
|
||||||
fun({Topic, Payload = {MessageID, Timestamp, _}}) ->
|
|
||||||
Bin = term_to_binary(Payload),
|
|
||||||
emqx_ds_message_storage_bitmask:store(DB, MessageID, Timestamp, Topic, Bin)
|
|
||||||
end,
|
|
||||||
Messages
|
|
||||||
).
|
|
||||||
|
|
||||||
iterate_db(DB, Iteration) ->
|
|
||||||
iterate_db(make_iterator(DB, Iteration)).
|
|
||||||
|
|
||||||
iterate_db(It) ->
|
|
||||||
case emqx_ds_message_storage_bitmask:next(It) of
|
|
||||||
{value, Payload, ItNext} ->
|
|
||||||
[binary_to_term(Payload) | iterate_db(ItNext)];
|
|
||||||
none ->
|
|
||||||
[]
|
|
||||||
end.
|
|
||||||
|
|
||||||
make_iterator(DB, Replay) ->
|
|
||||||
{ok, It} = emqx_ds_message_storage_bitmask:make_iterator(DB, Replay),
|
|
||||||
It.
|
|
||||||
|
|
||||||
make_iterator(DB, Replay, Options) ->
|
|
||||||
{ok, It} = emqx_ds_message_storage_bitmask:make_iterator(DB, Replay, Options),
|
|
||||||
It.
|
|
||||||
|
|
||||||
run_iterator_commands([iterate | Rest], It, Ctx) ->
|
|
||||||
case emqx_ds_message_storage_bitmask:next(It) of
|
|
||||||
{value, Payload, ItNext} ->
|
|
||||||
[binary_to_term(Payload) | run_iterator_commands(Rest, ItNext, Ctx)];
|
|
||||||
none ->
|
|
||||||
[]
|
|
||||||
end;
|
|
||||||
run_iterator_commands([{preserve, restore} | Rest], It, Ctx) ->
|
|
||||||
#{
|
|
||||||
db := DB,
|
|
||||||
replay := Replay
|
|
||||||
} = Ctx,
|
|
||||||
Serial = emqx_ds_message_storage_bitmask:preserve_iterator(It),
|
|
||||||
{ok, ItNext} = emqx_ds_message_storage_bitmask:restore_iterator(DB, Replay, Serial),
|
|
||||||
run_iterator_commands(Rest, ItNext, Ctx);
|
|
||||||
run_iterator_commands([], It, _Ctx) ->
|
|
||||||
iterate_db(It).
|
|
||||||
|
|
||||||
store_shim(Shim, Messages) ->
|
|
||||||
lists:foreach(
|
|
||||||
fun({Topic, Payload = {MessageID, Timestamp, _}}) ->
|
|
||||||
Bin = term_to_binary(Payload),
|
|
||||||
emqx_ds_message_storage_bitmask_shim:store(Shim, MessageID, Timestamp, Topic, Bin)
|
|
||||||
end,
|
|
||||||
Messages
|
|
||||||
).
|
|
||||||
|
|
||||||
iterate_shim(Shim, Iteration) ->
|
|
||||||
lists:map(
|
|
||||||
fun binary_to_term/1,
|
|
||||||
emqx_ds_message_storage_bitmask_shim:iterate(Shim, Iteration)
|
|
||||||
).
|
|
||||||
|
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
%% Setup / teardown
|
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
|
|
||||||
open_db(Filepath, Options) ->
|
|
||||||
{ok, Handle} = rocksdb:open(Filepath, [{create_if_missing, true}]),
|
|
||||||
{Schema, CFRefs} = emqx_ds_message_storage_bitmask:create_new(Handle, ?GEN_ID, Options),
|
|
||||||
DB = emqx_ds_message_storage_bitmask:open(?SHARD, Handle, ?GEN_ID, CFRefs, Schema),
|
|
||||||
{DB, Handle}.
|
|
||||||
|
|
||||||
close_db(Handle) ->
|
|
||||||
rocksdb:close(Handle).
|
|
||||||
|
|
||||||
make_filepath(TC) ->
|
|
||||||
make_filepath(TC, 0).
|
|
||||||
|
|
||||||
make_filepath(TC, InstID) ->
|
|
||||||
Name = io_lib:format("~0p.~0p", [TC, InstID]),
|
|
||||||
Path = filename:join(?WORK_DIR ++ ["proper", "runs", get_run_id(), ?MODULE_STRING, Name]),
|
|
||||||
ok = filelib:ensure_dir(Path),
|
|
||||||
Path.
|
|
||||||
|
|
||||||
get_run_id() ->
|
|
||||||
case persistent_term:get(?RUN_ID, undefined) of
|
|
||||||
RunID when RunID /= undefined ->
|
|
||||||
RunID;
|
|
||||||
undefined ->
|
|
||||||
RunID = make_run_id(),
|
|
||||||
ok = persistent_term:put(?RUN_ID, RunID),
|
|
||||||
RunID
|
|
||||||
end.
|
|
||||||
|
|
||||||
make_run_id() ->
|
|
||||||
calendar:system_time_to_rfc3339(erlang:system_time(second), [{offset, "Z"}]).
|
|
||||||
|
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
%% Type generators
|
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
|
|
||||||
topic() ->
|
|
||||||
non_empty(list(topic_level())).
|
|
||||||
|
|
||||||
topic(EntropyWeights) ->
|
|
||||||
?LET(L, scaled(1 / 4, list(1)), begin
|
|
||||||
EWs = lists:sublist(EntropyWeights ++ L, length(L)),
|
|
||||||
?SIZED(S, [oneof([topic_level(S * EW), topic_level_fixed()]) || EW <- EWs])
|
|
||||||
end).
|
|
||||||
|
|
||||||
topic_filter() ->
|
|
||||||
?SUCHTHAT(
|
|
||||||
L,
|
|
||||||
non_empty(
|
|
||||||
list(
|
|
||||||
frequency([
|
|
||||||
{5, topic_level()},
|
|
||||||
{2, '+'},
|
|
||||||
{1, '#'}
|
|
||||||
])
|
|
||||||
)
|
|
||||||
),
|
|
||||||
not lists:member('#', L) orelse lists:last(L) == '#'
|
|
||||||
).
|
|
||||||
|
|
||||||
topic_level_pattern() ->
|
|
||||||
frequency([
|
|
||||||
{5, level},
|
|
||||||
{2, '+'},
|
|
||||||
{1, '#'}
|
|
||||||
]).
|
|
||||||
|
|
||||||
topic_filter_pattern() ->
|
|
||||||
list(topic_level_pattern()).
|
|
||||||
|
|
||||||
topic_filter(Topic) ->
|
|
||||||
?LET({T, Pat}, {Topic, topic_filter_pattern()}, make_topic_filter(Pat, T)).
|
|
||||||
|
|
||||||
make_topic_filter([], _) ->
|
|
||||||
[];
|
|
||||||
make_topic_filter(_, []) ->
|
|
||||||
[];
|
|
||||||
make_topic_filter(['#' | _], _) ->
|
|
||||||
['#'];
|
|
||||||
make_topic_filter(['+' | Rest], [_ | Levels]) ->
|
|
||||||
['+' | make_topic_filter(Rest, Levels)];
|
|
||||||
make_topic_filter([level | Rest], [L | Levels]) ->
|
|
||||||
[L | make_topic_filter(Rest, Levels)].
|
|
||||||
|
|
||||||
% topic() ->
|
|
||||||
% ?LAZY(?SIZED(S, frequency([
|
|
||||||
% {S, [topic_level() | topic()]},
|
|
||||||
% {1, []}
|
|
||||||
% ]))).
|
|
||||||
|
|
||||||
% topic_filter() ->
|
|
||||||
% ?LAZY(?SIZED(S, frequency([
|
|
||||||
% {round(S / 3 * 2), [topic_level() | topic_filter()]},
|
|
||||||
% {round(S / 3 * 1), ['+' | topic_filter()]},
|
|
||||||
% {1, []},
|
|
||||||
% {1, ['#']}
|
|
||||||
% ]))).
|
|
||||||
|
|
||||||
topic_level() ->
|
|
||||||
?LET(L, list(oneof([range($a, $z), range($0, $9)])), iolist_to_binary(L)).
|
|
||||||
|
|
||||||
topic_level(Entropy) ->
|
|
||||||
S = floor(1 + math:log2(Entropy) / 4),
|
|
||||||
?LET(I, range(1, Entropy), iolist_to_binary(io_lib:format("~*.16.0B", [S, I]))).
|
|
||||||
|
|
||||||
topic_level_fixed() ->
|
|
||||||
oneof([
|
|
||||||
<<"foo">>,
|
|
||||||
<<"bar">>,
|
|
||||||
<<"baz">>,
|
|
||||||
<<"xyzzy">>
|
|
||||||
]).
|
|
||||||
|
|
||||||
keymapper() ->
|
|
||||||
?LET(
|
|
||||||
{TimestampBits, TopicBits, Epoch},
|
|
||||||
{
|
|
||||||
range(0, 128),
|
|
||||||
non_empty(list(range(1, 32))),
|
|
||||||
pos_integer()
|
|
||||||
},
|
|
||||||
make_keymapper(TimestampBits, TopicBits, Epoch * 100)
|
|
||||||
).
|
|
||||||
|
|
||||||
keyspace_filter() ->
|
|
||||||
?LET(
|
|
||||||
{TopicFilter, StartTime, Keymapper},
|
|
||||||
{topic_filter(), pos_integer(), keymapper()},
|
|
||||||
emqx_ds_message_storage_bitmask:make_keyspace_filter({TopicFilter, StartTime}, Keymapper)
|
|
||||||
).
|
|
||||||
|
|
||||||
messages(Topic) ->
|
|
||||||
?LET(
|
|
||||||
Ts,
|
|
||||||
list(Topic),
|
|
||||||
interleaved(
|
|
||||||
?LET(Messages, vector(length(Ts), scaled(4, list(message()))), lists:zip(Ts, Messages))
|
|
||||||
)
|
|
||||||
).
|
|
||||||
|
|
||||||
message() ->
|
|
||||||
?LET({Timestamp, Payload}, {timestamp(), binary()}, {emqx_guid:gen(), Timestamp, Payload}).
|
|
||||||
|
|
||||||
message_streams(Topic) ->
|
|
||||||
?LET(Topics, list(Topic), [{T, payload_gen:binary_stream_gen(64)} || T <- Topics]).
|
|
||||||
|
|
||||||
timestamp() ->
|
|
||||||
scaled(20, pos_integer()).
|
|
||||||
|
|
||||||
start_time() ->
|
|
||||||
scaled(10, pos_integer()).
|
|
||||||
|
|
||||||
bitstr(Size) ->
|
|
||||||
?LET(B, binary(1 + (Size div 8)), binary:decode_unsigned(B) band (1 bsl Size - 1)).
|
|
||||||
|
|
||||||
nth(L) ->
|
|
||||||
?LET(I, range(1, length(L)), lists:nth(I, L)).
|
|
||||||
|
|
||||||
scaled(Factor, T) ->
|
|
||||||
?SIZED(S, resize(ceil(S * Factor), T)).
|
|
||||||
|
|
||||||
interleaved(T) ->
|
|
||||||
?LET({L, Seed}, {T, integer()}, interleave(L, rand:seed_s(exsss, Seed))).
|
|
||||||
|
|
||||||
shuffled(T) ->
|
|
||||||
?LET({L, Seed}, {T, integer()}, shuffle(L, rand:seed_s(exsss, Seed))).
|
|
||||||
|
|
||||||
flat(T) ->
|
|
||||||
?LET(L, T, lists:flatten(L)).
|
|
||||||
|
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
%% Internal functions
|
|
||||||
%%--------------------------------------------------------------------
|
|
||||||
|
|
||||||
make_keymapper(TimestampBits, TopicBits, MaxEpoch) ->
|
|
||||||
emqx_ds_message_storage_bitmask:make_keymapper(#{
|
|
||||||
timestamp_bits => TimestampBits,
|
|
||||||
topic_bits_per_level => TopicBits,
|
|
||||||
epoch => MaxEpoch
|
|
||||||
}).
|
|
||||||
|
|
||||||
get_keymapper_bitsize(Keymapper) ->
|
|
||||||
maps:get(bitsize, emqx_ds_message_storage_bitmask:keymapper_info(Keymapper)).
|
|
||||||
|
|
||||||
-spec interleave(list({Tag, list(E)}), rand:state()) -> list({Tag, E}).
|
|
||||||
interleave(Seqs, Rng) ->
|
|
||||||
interleave(Seqs, length(Seqs), Rng).
|
|
||||||
|
|
||||||
interleave(Seqs, L, Rng) when L > 0 ->
|
|
||||||
{N, RngNext} = rand:uniform_s(L, Rng),
|
|
||||||
{SeqHead, SeqTail} = lists:split(N - 1, Seqs),
|
|
||||||
case SeqTail of
|
|
||||||
[{Tag, [M | Rest]} | SeqRest] ->
|
|
||||||
[{Tag, M} | interleave(SeqHead ++ [{Tag, Rest} | SeqRest], L, RngNext)];
|
|
||||||
[{_, []} | SeqRest] ->
|
|
||||||
interleave(SeqHead ++ SeqRest, L - 1, RngNext)
|
|
||||||
end;
|
|
||||||
interleave([], 0, _) ->
|
|
||||||
[].
|
|
||||||
|
|
||||||
-spec shuffle(list(E), rand:state()) -> list(E).
|
|
||||||
shuffle(L, Rng) ->
|
|
||||||
{Rands, _} = randoms(length(L), Rng),
|
|
||||||
[E || {_, E} <- lists:sort(lists:zip(Rands, L))].
|
|
||||||
|
|
||||||
randoms(N, Rng) when N > 0 ->
|
|
||||||
{Rand, RngNext} = rand:uniform_s(Rng),
|
|
||||||
{Tail, RngFinal} = randoms(N - 1, RngNext),
|
|
||||||
{[Rand | Tail], RngFinal};
|
|
||||||
randoms(_, Rng) ->
|
|
||||||
{[], Rng}.
|
|
|
@ -0,0 +1,22 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2017-2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%
|
||||||
|
%% Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
%% you may not use this file except in compliance with the License.
|
||||||
|
%% You may obtain a copy of the License at
|
||||||
|
%%
|
||||||
|
%% http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
%%
|
||||||
|
%% Unless required by applicable law or agreed to in writing, software
|
||||||
|
%% distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
%% See the License for the specific language governing permissions and
|
||||||
|
%% limitations under the License.
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
|
||||||
|
-ifndef(EMQX_BPAPI_HRL).
|
||||||
|
-define(EMQX_BPAPI_HRL, true).
|
||||||
|
|
||||||
|
-compile({parse_transform, emqx_bpapi_trans}).
|
||||||
|
|
||||||
|
-endif.
|
|
@ -0,0 +1,43 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2022-2023 EMQ Technologies Co., Ltd. All Rights Reserved.
|
||||||
|
%%
|
||||||
|
%% Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
%% you may not use this file except in compliance with the License.
|
||||||
|
%% You may obtain a copy of the License at
|
||||||
|
%%
|
||||||
|
%% http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
%%
|
||||||
|
%% Unless required by applicable law or agreed to in writing, software
|
||||||
|
%% distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
%% See the License for the specific language governing permissions and
|
||||||
|
%% limitations under the License.
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
-ifndef(EMQX_MESSAGE_HRL).
|
||||||
|
-define(EMQX_MESSAGE_HRL, true).
|
||||||
|
|
||||||
|
%% See 'Application Message' in MQTT Version 5.0
|
||||||
|
-record(message, {
|
||||||
|
%% Global unique message ID
|
||||||
|
id :: binary(),
|
||||||
|
%% Message QoS
|
||||||
|
qos = 0,
|
||||||
|
%% Message from
|
||||||
|
from :: atom() | binary(),
|
||||||
|
%% Message flags
|
||||||
|
flags = #{} :: emqx_types:flags(),
|
||||||
|
%% Message headers. May contain any metadata. e.g. the
|
||||||
|
%% protocol version number, username, peerhost or
|
||||||
|
%% the PUBLISH properties (MQTT 5.0).
|
||||||
|
headers = #{} :: emqx_types:headers(),
|
||||||
|
%% Topic that the message is published to
|
||||||
|
topic :: emqx_types:topic(),
|
||||||
|
%% Message Payload
|
||||||
|
payload :: emqx_types:payload(),
|
||||||
|
%% Timestamp (Unit: millisecond)
|
||||||
|
timestamp :: integer(),
|
||||||
|
%% not used so far, for future extension
|
||||||
|
extra = [] :: term()
|
||||||
|
}).
|
||||||
|
|
||||||
|
-endif.
|
|
@ -106,6 +106,10 @@
|
||||||
emqx_exproto_pb % generated code for protobuf
|
emqx_exproto_pb % generated code for protobuf
|
||||||
]}.
|
]}.
|
||||||
|
|
||||||
|
{eunit_opts,
|
||||||
|
[ verbose
|
||||||
|
]}.
|
||||||
|
|
||||||
{project_plugins,
|
{project_plugins,
|
||||||
[ erlfmt,
|
[ erlfmt,
|
||||||
{rebar3_hex, "7.0.2"},
|
{rebar3_hex, "7.0.2"},
|
||||||
|
|
Loading…
Reference in New Issue