Merge pull request #12361 from keynslug/ft/EMQX-11756/emqx-ds-replication
feat(ds): implement raft-based replication
This commit is contained in:
commit
e10d43cdce
|
@ -34,7 +34,8 @@
|
||||||
{emqx_http_lib, {git, "https://github.com/emqx/emqx_http_lib.git", {tag, "0.5.3"}}},
|
{emqx_http_lib, {git, "https://github.com/emqx/emqx_http_lib.git", {tag, "0.5.3"}}},
|
||||||
{pbkdf2, {git, "https://github.com/emqx/erlang-pbkdf2.git", {tag, "2.0.4"}}},
|
{pbkdf2, {git, "https://github.com/emqx/erlang-pbkdf2.git", {tag, "2.0.4"}}},
|
||||||
{recon, {git, "https://github.com/ferd/recon", {tag, "2.5.1"}}},
|
{recon, {git, "https://github.com/ferd/recon", {tag, "2.5.1"}}},
|
||||||
{snabbkaffe, {git, "https://github.com/kafka4beam/snabbkaffe.git", {tag, "1.0.8"}}}
|
{snabbkaffe, {git, "https://github.com/kafka4beam/snabbkaffe.git", {tag, "1.0.8"}}},
|
||||||
|
{ra, "2.7.3"}
|
||||||
]}.
|
]}.
|
||||||
|
|
||||||
{plugins, [{rebar3_proper, "0.12.1"}, rebar3_path_deps]}.
|
{plugins, [{rebar3_proper, "0.12.1"}, rebar3_path_deps]}.
|
||||||
|
|
|
@ -36,12 +36,15 @@
|
||||||
%% API
|
%% API
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
||||||
translate_builtin(#{
|
translate_builtin(
|
||||||
backend := builtin,
|
Backend = #{
|
||||||
n_shards := NShards,
|
backend := builtin,
|
||||||
replication_factor := ReplFactor,
|
n_shards := NShards,
|
||||||
layout := Layout
|
n_sites := NSites,
|
||||||
}) ->
|
replication_factor := ReplFactor,
|
||||||
|
layout := Layout
|
||||||
|
}
|
||||||
|
) ->
|
||||||
Storage =
|
Storage =
|
||||||
case Layout of
|
case Layout of
|
||||||
#{
|
#{
|
||||||
|
@ -61,7 +64,9 @@ translate_builtin(#{
|
||||||
#{
|
#{
|
||||||
backend => builtin,
|
backend => builtin,
|
||||||
n_shards => NShards,
|
n_shards => NShards,
|
||||||
|
n_sites => NSites,
|
||||||
replication_factor => ReplFactor,
|
replication_factor => ReplFactor,
|
||||||
|
replication_options => maps:get(replication_options, Backend, #{}),
|
||||||
storage => Storage
|
storage => Storage
|
||||||
}.
|
}.
|
||||||
|
|
||||||
|
@ -126,6 +131,16 @@ fields(builtin) ->
|
||||||
desc => ?DESC(builtin_n_shards)
|
desc => ?DESC(builtin_n_shards)
|
||||||
}
|
}
|
||||||
)},
|
)},
|
||||||
|
%% TODO: Deprecate once cluster management and rebalancing is implemented.
|
||||||
|
{"n_sites",
|
||||||
|
sc(
|
||||||
|
pos_integer(),
|
||||||
|
#{
|
||||||
|
default => 1,
|
||||||
|
importance => ?IMPORTANCE_HIDDEN,
|
||||||
|
desc => ?DESC(builtin_n_sites)
|
||||||
|
}
|
||||||
|
)},
|
||||||
{replication_factor,
|
{replication_factor,
|
||||||
sc(
|
sc(
|
||||||
pos_integer(),
|
pos_integer(),
|
||||||
|
@ -134,6 +149,15 @@ fields(builtin) ->
|
||||||
importance => ?IMPORTANCE_HIDDEN
|
importance => ?IMPORTANCE_HIDDEN
|
||||||
}
|
}
|
||||||
)},
|
)},
|
||||||
|
%% TODO: Elaborate.
|
||||||
|
{"replication_options",
|
||||||
|
sc(
|
||||||
|
hoconsc:map(name, any()),
|
||||||
|
#{
|
||||||
|
default => #{},
|
||||||
|
importance => ?IMPORTANCE_HIDDEN
|
||||||
|
}
|
||||||
|
)},
|
||||||
{local_write_buffer,
|
{local_write_buffer,
|
||||||
sc(
|
sc(
|
||||||
ref(builtin_local_write_buffer),
|
ref(builtin_local_write_buffer),
|
||||||
|
@ -201,7 +225,7 @@ fields(layout_builtin_wildcard_optimized) ->
|
||||||
sc(
|
sc(
|
||||||
range(0, 64),
|
range(0, 64),
|
||||||
#{
|
#{
|
||||||
default => 10,
|
default => 20,
|
||||||
importance => ?IMPORTANCE_HIDDEN,
|
importance => ?IMPORTANCE_HIDDEN,
|
||||||
desc => ?DESC(wildcard_optimized_epoch_bits)
|
desc => ?DESC(wildcard_optimized_epoch_bits)
|
||||||
}
|
}
|
||||||
|
|
|
@ -38,7 +38,8 @@
|
||||||
from/1,
|
from/1,
|
||||||
topic/1,
|
topic/1,
|
||||||
payload/1,
|
payload/1,
|
||||||
timestamp/1
|
timestamp/1,
|
||||||
|
timestamp/2
|
||||||
]).
|
]).
|
||||||
|
|
||||||
%% Flags
|
%% Flags
|
||||||
|
@ -79,7 +80,10 @@
|
||||||
estimate_size/1
|
estimate_size/1
|
||||||
]).
|
]).
|
||||||
|
|
||||||
-export_type([message_map/0]).
|
-export_type([
|
||||||
|
timestamp/0,
|
||||||
|
message_map/0
|
||||||
|
]).
|
||||||
|
|
||||||
-type message_map() :: #{
|
-type message_map() :: #{
|
||||||
id := binary(),
|
id := binary(),
|
||||||
|
@ -89,10 +93,14 @@
|
||||||
headers := emqx_types:headers(),
|
headers := emqx_types:headers(),
|
||||||
topic := emqx_types:topic(),
|
topic := emqx_types:topic(),
|
||||||
payload := emqx_types:payload(),
|
payload := emqx_types:payload(),
|
||||||
timestamp := integer(),
|
timestamp := timestamp(),
|
||||||
extra := _
|
extra := _
|
||||||
}.
|
}.
|
||||||
|
|
||||||
|
%% Message timestamp
|
||||||
|
%% Granularity: milliseconds.
|
||||||
|
-type timestamp() :: non_neg_integer().
|
||||||
|
|
||||||
-elvis([{elvis_style, god_modules, disable}]).
|
-elvis([{elvis_style, god_modules, disable}]).
|
||||||
|
|
||||||
-spec make(emqx_types:topic(), emqx_types:payload()) -> emqx_types:message().
|
-spec make(emqx_types:topic(), emqx_types:payload()) -> emqx_types:message().
|
||||||
|
@ -201,9 +209,14 @@ topic(#message{topic = Topic}) -> Topic.
|
||||||
-spec payload(emqx_types:message()) -> emqx_types:payload().
|
-spec payload(emqx_types:message()) -> emqx_types:payload().
|
||||||
payload(#message{payload = Payload}) -> Payload.
|
payload(#message{payload = Payload}) -> Payload.
|
||||||
|
|
||||||
-spec timestamp(emqx_types:message()) -> integer().
|
-spec timestamp(emqx_types:message()) -> timestamp().
|
||||||
timestamp(#message{timestamp = TS}) -> TS.
|
timestamp(#message{timestamp = TS}) -> TS.
|
||||||
|
|
||||||
|
-spec timestamp(emqx_types:message(), second | millisecond | microsecond) -> non_neg_integer().
|
||||||
|
timestamp(#message{timestamp = TS}, second) -> TS div 1000;
|
||||||
|
timestamp(#message{timestamp = TS}, millisecond) -> TS;
|
||||||
|
timestamp(#message{timestamp = TS}, microsecond) -> TS * 1000.
|
||||||
|
|
||||||
-spec is_sys(emqx_types:message()) -> boolean().
|
-spec is_sys(emqx_types:message()) -> boolean().
|
||||||
is_sys(#message{flags = #{sys := true}}) ->
|
is_sys(#message{flags = #{sys := true}}) ->
|
||||||
true;
|
true;
|
||||||
|
@ -416,7 +429,7 @@ from_map(#{
|
||||||
}.
|
}.
|
||||||
|
|
||||||
%% @doc Get current timestamp in milliseconds.
|
%% @doc Get current timestamp in milliseconds.
|
||||||
-spec timestamp_now() -> integer().
|
-spec timestamp_now() -> timestamp().
|
||||||
timestamp_now() ->
|
timestamp_now() ->
|
||||||
erlang:system_time(millisecond).
|
erlang:system_time(millisecond).
|
||||||
|
|
||||||
|
|
|
@ -37,7 +37,6 @@
|
||||||
badrpc/0,
|
badrpc/0,
|
||||||
call_result/1,
|
call_result/1,
|
||||||
call_result/0,
|
call_result/0,
|
||||||
cast_result/0,
|
|
||||||
multicall_result/1,
|
multicall_result/1,
|
||||||
multicall_result/0,
|
multicall_result/0,
|
||||||
erpc/1,
|
erpc/1,
|
||||||
|
|
|
@ -48,7 +48,7 @@
|
||||||
|
|
||||||
%% Applications and modules we wish to ignore in the analysis:
|
%% Applications and modules we wish to ignore in the analysis:
|
||||||
-define(IGNORED_APPS,
|
-define(IGNORED_APPS,
|
||||||
"gen_rpc, recon, redbug, observer_cli, snabbkaffe, ekka, mria, amqp_client, rabbit_common, esaml"
|
"gen_rpc, recon, redbug, observer_cli, snabbkaffe, ekka, mria, amqp_client, rabbit_common, esaml, ra"
|
||||||
).
|
).
|
||||||
-define(IGNORED_MODULES, "emqx_rpc").
|
-define(IGNORED_MODULES, "emqx_rpc").
|
||||||
-define(FORCE_DELETED_MODULES, [
|
-define(FORCE_DELETED_MODULES, [
|
||||||
|
|
|
@ -33,10 +33,6 @@ all() ->
|
||||||
emqx_common_test_helpers:all(?MODULE).
|
emqx_common_test_helpers:all(?MODULE).
|
||||||
|
|
||||||
init_per_suite(Config) ->
|
init_per_suite(Config) ->
|
||||||
%% avoid inter-suite flakiness...
|
|
||||||
%% TODO: remove after other suites start to use `emx_cth_suite'
|
|
||||||
application:stop(emqx),
|
|
||||||
application:stop(emqx_durable_storage),
|
|
||||||
Config.
|
Config.
|
||||||
|
|
||||||
end_per_suite(_Config) ->
|
end_per_suite(_Config) ->
|
||||||
|
@ -45,19 +41,33 @@ end_per_suite(_Config) ->
|
||||||
init_per_testcase(t_session_subscription_iterators = TestCase, Config) ->
|
init_per_testcase(t_session_subscription_iterators = TestCase, Config) ->
|
||||||
Cluster = cluster(),
|
Cluster = cluster(),
|
||||||
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)}),
|
||||||
|
_ = wait_shards_online(Nodes),
|
||||||
[{nodes, Nodes} | Config];
|
[{nodes, Nodes} | Config];
|
||||||
init_per_testcase(t_message_gc = TestCase, Config) ->
|
init_per_testcase(t_message_gc = TestCase, Config) ->
|
||||||
Opts = #{
|
Opts = #{
|
||||||
extra_emqx_conf =>
|
extra_emqx_conf =>
|
||||||
"\n session_persistence.message_retention_period = 1s"
|
"\n session_persistence.message_retention_period = 3s"
|
||||||
"\n durable_storage.messages.n_shards = 3"
|
"\n durable_storage.messages.n_shards = 3"
|
||||||
},
|
},
|
||||||
common_init_per_testcase(TestCase, [{n_shards, 3} | Config], Opts);
|
common_init_per_testcase(TestCase, [{n_shards, 3} | Config], Opts);
|
||||||
|
init_per_testcase(t_replication_options = TestCase, Config) ->
|
||||||
|
Opts = #{
|
||||||
|
extra_emqx_conf =>
|
||||||
|
"\n durable_storage.messages.replication_options {"
|
||||||
|
"\n wal_max_size_bytes = 16000000"
|
||||||
|
"\n wal_max_batch_size = 1024"
|
||||||
|
"\n wal_write_strategy = o_sync"
|
||||||
|
"\n wal_sync_method = datasync"
|
||||||
|
"\n wal_compute_checksums = false"
|
||||||
|
"\n snapshot_interval = 64"
|
||||||
|
"\n resend_window = 60"
|
||||||
|
"\n}"
|
||||||
|
},
|
||||||
|
common_init_per_testcase(TestCase, Config, Opts);
|
||||||
init_per_testcase(TestCase, Config) ->
|
init_per_testcase(TestCase, Config) ->
|
||||||
common_init_per_testcase(TestCase, Config, _Opts = #{}).
|
common_init_per_testcase(TestCase, Config, _Opts = #{}).
|
||||||
|
|
||||||
common_init_per_testcase(TestCase, Config, Opts) ->
|
common_init_per_testcase(TestCase, Config, Opts) ->
|
||||||
ok = emqx_ds:drop_db(?PERSISTENT_MESSAGE_DB),
|
|
||||||
Apps = emqx_cth_suite:start(
|
Apps = emqx_cth_suite:start(
|
||||||
app_specs(Opts),
|
app_specs(Opts),
|
||||||
#{work_dir => emqx_cth_suite:work_dir(TestCase, Config)}
|
#{work_dir => emqx_cth_suite:work_dir(TestCase, Config)}
|
||||||
|
@ -67,14 +77,11 @@ common_init_per_testcase(TestCase, Config, Opts) ->
|
||||||
end_per_testcase(t_session_subscription_iterators, Config) ->
|
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);
|
||||||
end_per_testcase(common, Config);
|
|
||||||
end_per_testcase(_TestCase, Config) ->
|
end_per_testcase(_TestCase, Config) ->
|
||||||
Apps = proplists:get_value(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(),
|
ok = emqx_cth_suite:stop(Apps).
|
||||||
emqx_cth_suite:stop(Apps),
|
|
||||||
ok.
|
|
||||||
|
|
||||||
t_messages_persisted(_Config) ->
|
t_messages_persisted(_Config) ->
|
||||||
C1 = connect(<<?MODULE_STRING "1">>, true, 30),
|
C1 = connect(<<?MODULE_STRING "1">>, true, 30),
|
||||||
|
@ -390,7 +397,7 @@ t_message_gc(Config) ->
|
||||||
message(<<"foo/bar">>, <<"1">>, 0),
|
message(<<"foo/bar">>, <<"1">>, 0),
|
||||||
message(<<"foo/baz">>, <<"2">>, 1)
|
message(<<"foo/baz">>, <<"2">>, 1)
|
||||||
],
|
],
|
||||||
ok = emqx_ds:store_batch(?PERSISTENT_MESSAGE_DB, Msgs0),
|
ok = emqx_ds:store_batch(?PERSISTENT_MESSAGE_DB, Msgs0, #{sync => true}),
|
||||||
?tp(inserted_batch, #{}),
|
?tp(inserted_batch, #{}),
|
||||||
{ok, _} = ?block_until(#{?snk_kind := ps_message_gc_added_gen}),
|
{ok, _} = ?block_until(#{?snk_kind := ps_message_gc_added_gen}),
|
||||||
|
|
||||||
|
@ -399,7 +406,7 @@ t_message_gc(Config) ->
|
||||||
message(<<"foo/bar">>, <<"3">>, Now + 100),
|
message(<<"foo/bar">>, <<"3">>, Now + 100),
|
||||||
message(<<"foo/baz">>, <<"4">>, Now + 101)
|
message(<<"foo/baz">>, <<"4">>, Now + 101)
|
||||||
],
|
],
|
||||||
ok = emqx_ds:store_batch(?PERSISTENT_MESSAGE_DB, Msgs1),
|
ok = emqx_ds:store_batch(?PERSISTENT_MESSAGE_DB, Msgs1, #{sync => true}),
|
||||||
|
|
||||||
{ok, _} = snabbkaffe:block_until(
|
{ok, _} = snabbkaffe:block_until(
|
||||||
?match_n_events(NShards, #{?snk_kind := message_gc_generation_dropped}),
|
?match_n_events(NShards, #{?snk_kind := message_gc_generation_dropped}),
|
||||||
|
@ -455,6 +462,33 @@ t_metrics_not_dropped(_Config) ->
|
||||||
|
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
|
t_replication_options(_Config) ->
|
||||||
|
?assertMatch(
|
||||||
|
#{
|
||||||
|
backend := builtin,
|
||||||
|
replication_options := #{
|
||||||
|
wal_max_size_bytes := 16000000,
|
||||||
|
wal_max_batch_size := 1024,
|
||||||
|
wal_write_strategy := o_sync,
|
||||||
|
wal_sync_method := datasync,
|
||||||
|
wal_compute_checksums := false,
|
||||||
|
snapshot_interval := 64,
|
||||||
|
resend_window := 60
|
||||||
|
}
|
||||||
|
},
|
||||||
|
emqx_ds_replication_layer_meta:get_options(?PERSISTENT_MESSAGE_DB)
|
||||||
|
),
|
||||||
|
?assertMatch(
|
||||||
|
#{
|
||||||
|
wal_max_size_bytes := 16000000,
|
||||||
|
wal_max_batch_size := 1024,
|
||||||
|
wal_write_strategy := o_sync,
|
||||||
|
wal_compute_checksums := false,
|
||||||
|
wal_sync_method := datasync
|
||||||
|
},
|
||||||
|
ra_system:fetch(?PERSISTENT_MESSAGE_DB)
|
||||||
|
).
|
||||||
|
|
||||||
%%
|
%%
|
||||||
|
|
||||||
connect(ClientId, CleanStart, EI) ->
|
connect(ClientId, CleanStart, EI) ->
|
||||||
|
@ -524,22 +558,24 @@ app_specs(Opts) ->
|
||||||
].
|
].
|
||||||
|
|
||||||
cluster() ->
|
cluster() ->
|
||||||
Spec = #{role => core, apps => app_specs()},
|
ExtraConf = "\n durable_storage.messages.n_sites = 2",
|
||||||
|
Spec = #{role => core, apps => app_specs(#{extra_emqx_conf => ExtraConf})},
|
||||||
[
|
[
|
||||||
{persistent_messages_SUITE1, Spec},
|
{persistent_messages_SUITE1, Spec},
|
||||||
{persistent_messages_SUITE2, Spec}
|
{persistent_messages_SUITE2, Spec}
|
||||||
].
|
].
|
||||||
|
|
||||||
|
wait_shards_online(Nodes = [Node | _]) ->
|
||||||
|
NShards = erpc:call(Node, emqx_ds_replication_layer_meta, n_shards, [?PERSISTENT_MESSAGE_DB]),
|
||||||
|
?retry(500, 10, [?assertEqual(NShards, shards_online(N)) || N <- Nodes]).
|
||||||
|
|
||||||
|
shards_online(Node) ->
|
||||||
|
length(erpc:call(Node, emqx_ds_builtin_db_sup, which_shards, [?PERSISTENT_MESSAGE_DB])).
|
||||||
|
|
||||||
get_mqtt_port(Node, Type) ->
|
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.
|
||||||
|
|
||||||
clear_db() ->
|
|
||||||
ok = emqx_ds:drop_db(?PERSISTENT_MESSAGE_DB),
|
|
||||||
mria:stop(),
|
|
||||||
ok = mnesia:delete_schema([node()]),
|
|
||||||
ok.
|
|
||||||
|
|
||||||
message(Topic, Payload, PublishedAt) ->
|
message(Topic, Payload, PublishedAt) ->
|
||||||
#message{
|
#message{
|
||||||
topic = Topic,
|
topic = Topic,
|
||||||
|
|
|
@ -43,6 +43,7 @@
|
||||||
|
|
||||||
%% Misc. API:
|
%% Misc. API:
|
||||||
-export([count/1]).
|
-export([count/1]).
|
||||||
|
-export([timestamp_us/0]).
|
||||||
|
|
||||||
-export_type([
|
-export_type([
|
||||||
create_db_opts/0,
|
create_db_opts/0,
|
||||||
|
@ -147,9 +148,8 @@
|
||||||
-type error(Reason) :: {error, recoverable | unrecoverable, Reason}.
|
-type error(Reason) :: {error, recoverable | unrecoverable, Reason}.
|
||||||
|
|
||||||
%% Timestamp
|
%% Timestamp
|
||||||
|
%% Each message must have unique timestamp.
|
||||||
%% Earliest possible timestamp is 0.
|
%% Earliest possible timestamp is 0.
|
||||||
%% TODO granularity? Currently, we should always use milliseconds, as that's the unit we
|
|
||||||
%% use in emqx_guid. Otherwise, the iterators won't match the message timestamps.
|
|
||||||
-type time() :: non_neg_integer().
|
-type time() :: non_neg_integer().
|
||||||
|
|
||||||
-type message_store_opts() ::
|
-type message_store_opts() ::
|
||||||
|
@ -295,6 +295,7 @@ drop_db(DB) ->
|
||||||
undefined ->
|
undefined ->
|
||||||
ok;
|
ok;
|
||||||
Module ->
|
Module ->
|
||||||
|
_ = persistent_term:erase(?persistent_term(DB)),
|
||||||
Module:drop_db(DB)
|
Module:drop_db(DB)
|
||||||
end.
|
end.
|
||||||
|
|
||||||
|
@ -394,6 +395,10 @@ count(DB) ->
|
||||||
%% Internal exports
|
%% Internal exports
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
||||||
|
-spec timestamp_us() -> time().
|
||||||
|
timestamp_us() ->
|
||||||
|
erlang:system_time(microsecond).
|
||||||
|
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
%% Internal functions
|
%% Internal functions
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
|
@ -112,7 +112,9 @@
|
||||||
vector_to_key/2,
|
vector_to_key/2,
|
||||||
bin_vector_to_key/2,
|
bin_vector_to_key/2,
|
||||||
key_to_vector/2,
|
key_to_vector/2,
|
||||||
|
key_to_coord/3,
|
||||||
bin_key_to_vector/2,
|
bin_key_to_vector/2,
|
||||||
|
bin_key_to_coord/3,
|
||||||
key_to_bitstring/2,
|
key_to_bitstring/2,
|
||||||
bitstring_to_key/2,
|
bitstring_to_key/2,
|
||||||
make_filter/2,
|
make_filter/2,
|
||||||
|
@ -297,13 +299,7 @@ bin_vector_to_key(Keymapper = #keymapper{vec_coord_size = DimSizeof, key_size =
|
||||||
key_to_vector(#keymapper{vec_scanner = Scanner}, Key) ->
|
key_to_vector(#keymapper{vec_scanner = Scanner}, Key) ->
|
||||||
lists:map(
|
lists:map(
|
||||||
fun(Actions) ->
|
fun(Actions) ->
|
||||||
lists:foldl(
|
extract_coord(Actions, Key)
|
||||||
fun(Action, Acc) ->
|
|
||||||
Acc bor extract_inv(Key, Action)
|
|
||||||
end,
|
|
||||||
0,
|
|
||||||
Actions
|
|
||||||
)
|
|
||||||
end,
|
end,
|
||||||
Scanner
|
Scanner
|
||||||
).
|
).
|
||||||
|
@ -324,6 +320,16 @@ bin_key_to_vector(Keymapper = #keymapper{vec_coord_size = DimSizeof, key_size =
|
||||||
DimSizeof
|
DimSizeof
|
||||||
).
|
).
|
||||||
|
|
||||||
|
-spec key_to_coord(keymapper(), scalar(), dimension()) -> coord().
|
||||||
|
key_to_coord(#keymapper{vec_scanner = Scanner}, Key, Dim) ->
|
||||||
|
Actions = lists:nth(Dim, Scanner),
|
||||||
|
extract_coord(Actions, Key).
|
||||||
|
|
||||||
|
-spec bin_key_to_coord(keymapper(), key(), dimension()) -> coord().
|
||||||
|
bin_key_to_coord(Keymapper = #keymapper{key_size = Size}, BinKey, Dim) ->
|
||||||
|
<<Key:Size>> = BinKey,
|
||||||
|
key_to_coord(Keymapper, Key, Dim).
|
||||||
|
|
||||||
%% @doc Transform a bitstring to a key
|
%% @doc Transform a bitstring to a key
|
||||||
-spec bitstring_to_key(keymapper(), bitstring()) -> scalar().
|
-spec bitstring_to_key(keymapper(), bitstring()) -> scalar().
|
||||||
bitstring_to_key(#keymapper{key_size = Size}, Bin) ->
|
bitstring_to_key(#keymapper{key_size = Size}, Bin) ->
|
||||||
|
@ -680,6 +686,15 @@ extract_inv(Dest, #scan_action{
|
||||||
}) ->
|
}) ->
|
||||||
((Dest bsr DestOffset) band SrcBitmask) bsl SrcOffset.
|
((Dest bsr DestOffset) band SrcBitmask) bsl SrcOffset.
|
||||||
|
|
||||||
|
extract_coord(Actions, Key) ->
|
||||||
|
lists:foldl(
|
||||||
|
fun(Action, Acc) ->
|
||||||
|
Acc bor extract_inv(Key, Action)
|
||||||
|
end,
|
||||||
|
0,
|
||||||
|
Actions
|
||||||
|
).
|
||||||
|
|
||||||
ones(Bits) ->
|
ones(Bits) ->
|
||||||
1 bsl Bits - 1.
|
1 bsl Bits - 1.
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,8 @@
|
||||||
-behaviour(supervisor).
|
-behaviour(supervisor).
|
||||||
|
|
||||||
%% API:
|
%% API:
|
||||||
-export([start_db/2, start_shard/1, start_egress/1, stop_shard/1, ensure_shard/1]).
|
-export([start_db/2, start_shard/1, start_egress/1, stop_shard/1, ensure_shard/1, ensure_egress/1]).
|
||||||
|
-export([which_shards/1]).
|
||||||
|
|
||||||
%% behaviour callbacks:
|
%% behaviour callbacks:
|
||||||
-export([init/1]).
|
-export([init/1]).
|
||||||
|
@ -36,12 +37,14 @@
|
||||||
-define(via(REC), {via, gproc, {n, l, REC}}).
|
-define(via(REC), {via, gproc, {n, l, REC}}).
|
||||||
|
|
||||||
-define(db_sup, ?MODULE).
|
-define(db_sup, ?MODULE).
|
||||||
-define(shard_sup, emqx_ds_builtin_db_shard_sup).
|
-define(shards_sup, emqx_ds_builtin_db_shards_sup).
|
||||||
-define(egress_sup, emqx_ds_builtin_db_egress_sup).
|
-define(egress_sup, emqx_ds_builtin_db_egress_sup).
|
||||||
|
-define(shard_sup, emqx_ds_builtin_db_shard_sup).
|
||||||
|
|
||||||
-record(?db_sup, {db}).
|
-record(?db_sup, {db}).
|
||||||
-record(?shard_sup, {db}).
|
-record(?shards_sup, {db}).
|
||||||
-record(?egress_sup, {db}).
|
-record(?egress_sup, {db}).
|
||||||
|
-record(?shard_sup, {db, shard}).
|
||||||
|
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
%% API funcions
|
%% API funcions
|
||||||
|
@ -53,8 +56,8 @@ start_db(DB, Opts) ->
|
||||||
|
|
||||||
-spec start_shard(emqx_ds_storage_layer:shard_id()) ->
|
-spec start_shard(emqx_ds_storage_layer:shard_id()) ->
|
||||||
supervisor:startchild_ret().
|
supervisor:startchild_ret().
|
||||||
start_shard(Shard = {DB, _}) ->
|
start_shard({DB, Shard}) ->
|
||||||
supervisor:start_child(?via(#?shard_sup{db = DB}), shard_spec(DB, Shard)).
|
supervisor:start_child(?via(#?shards_sup{db = DB}), shard_spec(DB, Shard)).
|
||||||
|
|
||||||
-spec start_egress(emqx_ds_storage_layer:shard_id()) ->
|
-spec start_egress(emqx_ds_storage_layer:shard_id()) ->
|
||||||
supervisor:startchild_ret().
|
supervisor:startchild_ret().
|
||||||
|
@ -63,21 +66,24 @@ start_egress({DB, Shard}) ->
|
||||||
|
|
||||||
-spec stop_shard(emqx_ds_storage_layer:shard_id()) -> ok | {error, _}.
|
-spec stop_shard(emqx_ds_storage_layer:shard_id()) -> ok | {error, _}.
|
||||||
stop_shard(Shard = {DB, _}) ->
|
stop_shard(Shard = {DB, _}) ->
|
||||||
Sup = ?via(#?shard_sup{db = DB}),
|
Sup = ?via(#?shards_sup{db = DB}),
|
||||||
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_storage_layer:shard_id()) ->
|
-spec ensure_shard(emqx_ds_storage_layer:shard_id()) ->
|
||||||
ok | {error, _Reason}.
|
ok | {error, _Reason}.
|
||||||
ensure_shard(Shard) ->
|
ensure_shard(Shard) ->
|
||||||
case start_shard(Shard) of
|
ensure_started(start_shard(Shard)).
|
||||||
{ok, _Pid} ->
|
|
||||||
ok;
|
-spec ensure_egress(emqx_ds_storage_layer:shard_id()) ->
|
||||||
{error, {already_started, _Pid}} ->
|
ok | {error, _Reason}.
|
||||||
ok;
|
ensure_egress(Shard) ->
|
||||||
{error, Reason} ->
|
ensure_started(start_egress(Shard)).
|
||||||
{error, Reason}
|
|
||||||
end.
|
-spec which_shards(emqx_ds:db()) ->
|
||||||
|
[_Child].
|
||||||
|
which_shards(DB) ->
|
||||||
|
supervisor:which_children(?via(#?shards_sup{db = DB})).
|
||||||
|
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
%% behaviour callbacks
|
%% behaviour callbacks
|
||||||
|
@ -86,45 +92,78 @@ ensure_shard(Shard) ->
|
||||||
init({#?db_sup{db = DB}, DefaultOpts}) ->
|
init({#?db_sup{db = DB}, DefaultOpts}) ->
|
||||||
%% Spec for the top-level supervisor for the database:
|
%% Spec for the top-level supervisor for the database:
|
||||||
logger:notice("Starting DS DB ~p", [DB]),
|
logger:notice("Starting DS DB ~p", [DB]),
|
||||||
_ = emqx_ds_replication_layer_meta:open_db(DB, DefaultOpts),
|
Opts = emqx_ds_replication_layer_meta:open_db(DB, DefaultOpts),
|
||||||
%% TODO: before the leader election is implemented, we set ourselves as the leader for all shards:
|
ok = start_ra_system(DB, Opts),
|
||||||
MyShards = emqx_ds_replication_layer_meta:my_shards(DB),
|
Children = [
|
||||||
lists:foreach(
|
sup_spec(#?shards_sup{db = DB}, []),
|
||||||
fun(Shard) ->
|
sup_spec(#?egress_sup{db = DB}, []),
|
||||||
emqx_ds_replication_layer:maybe_set_myself_as_leader(DB, Shard)
|
shard_allocator_spec(DB, Opts)
|
||||||
end,
|
],
|
||||||
MyShards
|
|
||||||
),
|
|
||||||
Children = [sup_spec(#?shard_sup{db = DB}, []), sup_spec(#?egress_sup{db = DB}, [])],
|
|
||||||
SupFlags = #{
|
SupFlags = #{
|
||||||
strategy => one_for_all,
|
strategy => one_for_all,
|
||||||
intensity => 0,
|
intensity => 0,
|
||||||
period => 1
|
period => 1
|
||||||
},
|
},
|
||||||
{ok, {SupFlags, Children}};
|
{ok, {SupFlags, Children}};
|
||||||
init({#?shard_sup{db = DB}, _}) ->
|
init({#?shards_sup{db = _DB}, _}) ->
|
||||||
%% Spec for the supervisor that manages the worker processes for
|
%% Spec for the supervisor that manages the supervisors for
|
||||||
%% each local shard of the DB:
|
%% each local shard of the DB:
|
||||||
MyShards = emqx_ds_replication_layer_meta:my_shards(DB),
|
|
||||||
Children = [shard_spec(DB, Shard) || Shard <- MyShards],
|
|
||||||
SupFlags = #{
|
SupFlags = #{
|
||||||
strategy => one_for_one,
|
strategy => one_for_one,
|
||||||
intensity => 10,
|
intensity => 10,
|
||||||
period => 1
|
period => 1
|
||||||
},
|
},
|
||||||
{ok, {SupFlags, Children}};
|
{ok, {SupFlags, []}};
|
||||||
init({#?egress_sup{db = DB}, _}) ->
|
init({#?egress_sup{db = _DB}, _}) ->
|
||||||
%% Spec for the supervisor that manages the egress proxy processes
|
%% Spec for the supervisor that manages the egress proxy processes
|
||||||
%% managing traffic towards each of the shards of the DB:
|
%% managing traffic towards each of the shards of the DB:
|
||||||
Shards = emqx_ds_replication_layer_meta:shards(DB),
|
|
||||||
Children = [egress_spec(DB, Shard) || Shard <- Shards],
|
|
||||||
SupFlags = #{
|
SupFlags = #{
|
||||||
strategy => one_for_one,
|
strategy => one_for_one,
|
||||||
intensity => 0,
|
intensity => 0,
|
||||||
period => 1
|
period => 1
|
||||||
},
|
},
|
||||||
|
{ok, {SupFlags, []}};
|
||||||
|
init({#?shard_sup{db = DB, shard = Shard}, _}) ->
|
||||||
|
SupFlags = #{
|
||||||
|
strategy => rest_for_one,
|
||||||
|
intensity => 10,
|
||||||
|
period => 100
|
||||||
|
},
|
||||||
|
Opts = emqx_ds_replication_layer_meta:get_options(DB),
|
||||||
|
Children = [
|
||||||
|
shard_storage_spec(DB, Shard, Opts),
|
||||||
|
shard_replication_spec(DB, Shard, Opts)
|
||||||
|
],
|
||||||
{ok, {SupFlags, Children}}.
|
{ok, {SupFlags, Children}}.
|
||||||
|
|
||||||
|
start_ra_system(DB, #{replication_options := ReplicationOpts}) ->
|
||||||
|
DataDir = filename:join([emqx_ds:base_dir(), DB, dsrepl]),
|
||||||
|
Config = lists:foldr(fun maps:merge/2, #{}, [
|
||||||
|
ra_system:default_config(),
|
||||||
|
#{
|
||||||
|
name => DB,
|
||||||
|
data_dir => DataDir,
|
||||||
|
wal_data_dir => DataDir,
|
||||||
|
names => ra_system:derive_names(DB)
|
||||||
|
},
|
||||||
|
maps:with(
|
||||||
|
[
|
||||||
|
wal_max_size_bytes,
|
||||||
|
wal_max_batch_size,
|
||||||
|
wal_write_strategy,
|
||||||
|
wal_sync_method,
|
||||||
|
wal_compute_checksums
|
||||||
|
],
|
||||||
|
ReplicationOpts
|
||||||
|
)
|
||||||
|
]),
|
||||||
|
case ra_system:start(Config) of
|
||||||
|
{ok, _System} ->
|
||||||
|
ok;
|
||||||
|
{error, {already_started, _System}} ->
|
||||||
|
ok
|
||||||
|
end.
|
||||||
|
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
%% Internal exports
|
%% Internal exports
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
@ -145,15 +184,39 @@ sup_spec(Id, Options) ->
|
||||||
}.
|
}.
|
||||||
|
|
||||||
shard_spec(DB, Shard) ->
|
shard_spec(DB, Shard) ->
|
||||||
Options = emqx_ds_replication_layer_meta:get_options(DB),
|
|
||||||
#{
|
#{
|
||||||
id => Shard,
|
id => {shard, Shard},
|
||||||
start => {emqx_ds_storage_layer, start_link, [{DB, Shard}, Options]},
|
start => {?MODULE, start_link_sup, [#?shard_sup{db = DB, shard = Shard}, []]},
|
||||||
|
shutdown => infinity,
|
||||||
|
restart => permanent,
|
||||||
|
type => supervisor
|
||||||
|
}.
|
||||||
|
|
||||||
|
shard_storage_spec(DB, Shard, Opts) ->
|
||||||
|
#{
|
||||||
|
id => {Shard, storage},
|
||||||
|
start => {emqx_ds_storage_layer, start_link, [{DB, Shard}, Opts]},
|
||||||
shutdown => 5_000,
|
shutdown => 5_000,
|
||||||
restart => permanent,
|
restart => permanent,
|
||||||
type => worker
|
type => worker
|
||||||
}.
|
}.
|
||||||
|
|
||||||
|
shard_replication_spec(DB, Shard, Opts) ->
|
||||||
|
#{
|
||||||
|
id => {Shard, replication},
|
||||||
|
start => {emqx_ds_replication_layer_shard, start_link, [DB, Shard, Opts]},
|
||||||
|
restart => transient,
|
||||||
|
type => worker
|
||||||
|
}.
|
||||||
|
|
||||||
|
shard_allocator_spec(DB, Opts) ->
|
||||||
|
#{
|
||||||
|
id => shard_allocator,
|
||||||
|
start => {emqx_ds_replication_shard_allocator, start_link, [DB, Opts]},
|
||||||
|
restart => permanent,
|
||||||
|
type => worker
|
||||||
|
}.
|
||||||
|
|
||||||
egress_spec(DB, Shard) ->
|
egress_spec(DB, Shard) ->
|
||||||
#{
|
#{
|
||||||
id => Shard,
|
id => Shard,
|
||||||
|
@ -162,3 +225,13 @@ egress_spec(DB, Shard) ->
|
||||||
restart => permanent,
|
restart => permanent,
|
||||||
type => worker
|
type => worker
|
||||||
}.
|
}.
|
||||||
|
|
||||||
|
ensure_started(Res) ->
|
||||||
|
case Res of
|
||||||
|
{ok, _Pid} ->
|
||||||
|
ok;
|
||||||
|
{error, {already_started, _Pid}} ->
|
||||||
|
ok;
|
||||||
|
{error, Reason} ->
|
||||||
|
{error, Reason}
|
||||||
|
end.
|
||||||
|
|
|
@ -36,27 +36,36 @@
|
||||||
update_iterator/3,
|
update_iterator/3,
|
||||||
next/3,
|
next/3,
|
||||||
delete_next/4,
|
delete_next/4,
|
||||||
node_of_shard/2,
|
shard_of_message/3
|
||||||
shard_of_message/3,
|
|
||||||
maybe_set_myself_as_leader/2
|
|
||||||
]).
|
]).
|
||||||
|
|
||||||
%% internal exports:
|
%% internal exports:
|
||||||
-export([
|
-export([
|
||||||
|
%% RPC Targets:
|
||||||
do_drop_db_v1/1,
|
do_drop_db_v1/1,
|
||||||
do_store_batch_v1/4,
|
do_store_batch_v1/4,
|
||||||
do_get_streams_v1/4,
|
do_get_streams_v1/4,
|
||||||
do_get_streams_v2/4,
|
do_get_streams_v2/4,
|
||||||
do_make_iterator_v1/5,
|
|
||||||
do_make_iterator_v2/5,
|
do_make_iterator_v2/5,
|
||||||
do_update_iterator_v2/4,
|
do_update_iterator_v2/4,
|
||||||
do_next_v1/4,
|
do_next_v1/4,
|
||||||
do_add_generation_v2/1,
|
|
||||||
do_list_generations_with_lifetimes_v3/2,
|
do_list_generations_with_lifetimes_v3/2,
|
||||||
do_drop_generation_v3/3,
|
|
||||||
do_get_delete_streams_v4/4,
|
do_get_delete_streams_v4/4,
|
||||||
do_make_delete_iterator_v4/5,
|
do_make_delete_iterator_v4/5,
|
||||||
do_delete_next_v4/5
|
do_delete_next_v4/5,
|
||||||
|
%% Unused:
|
||||||
|
do_drop_generation_v3/3,
|
||||||
|
%% Obsolete:
|
||||||
|
do_make_iterator_v1/5,
|
||||||
|
do_add_generation_v2/1,
|
||||||
|
|
||||||
|
%% Egress API:
|
||||||
|
ra_store_batch/3
|
||||||
|
]).
|
||||||
|
|
||||||
|
-export([
|
||||||
|
init/1,
|
||||||
|
apply/3
|
||||||
]).
|
]).
|
||||||
|
|
||||||
-export_type([
|
-export_type([
|
||||||
|
@ -85,7 +94,9 @@
|
||||||
backend := builtin,
|
backend := builtin,
|
||||||
storage := emqx_ds_storage_layer:prototype(),
|
storage := emqx_ds_storage_layer:prototype(),
|
||||||
n_shards => pos_integer(),
|
n_shards => pos_integer(),
|
||||||
replication_factor => pos_integer()
|
n_sites => pos_integer(),
|
||||||
|
replication_factor => pos_integer(),
|
||||||
|
replication_options => _TODO :: #{}
|
||||||
}.
|
}.
|
||||||
|
|
||||||
%% This enapsulates the stream entity from the replication level.
|
%% This enapsulates the stream entity from the replication level.
|
||||||
|
@ -150,13 +161,19 @@ open_db(DB, CreateOpts) ->
|
||||||
|
|
||||||
-spec add_generation(emqx_ds:db()) -> ok | {error, _}.
|
-spec add_generation(emqx_ds:db()) -> ok | {error, _}.
|
||||||
add_generation(DB) ->
|
add_generation(DB) ->
|
||||||
Nodes = emqx_ds_replication_layer_meta:leader_nodes(DB),
|
foreach_shard(
|
||||||
_ = emqx_ds_proto_v4:add_generation(Nodes, DB),
|
DB,
|
||||||
ok.
|
fun(Shard) -> ok = ra_add_generation(DB, Shard) end
|
||||||
|
).
|
||||||
|
|
||||||
-spec update_db_config(emqx_ds:db(), builtin_db_opts()) -> ok | {error, _}.
|
-spec update_db_config(emqx_ds:db(), builtin_db_opts()) -> ok | {error, _}.
|
||||||
update_db_config(DB, CreateOpts) ->
|
update_db_config(DB, CreateOpts) ->
|
||||||
emqx_ds_replication_layer_meta:update_db_config(DB, CreateOpts).
|
ok = emqx_ds_replication_layer_meta:update_db_config(DB, CreateOpts),
|
||||||
|
Opts = emqx_ds_replication_layer_meta:get_options(DB),
|
||||||
|
foreach_shard(
|
||||||
|
DB,
|
||||||
|
fun(Shard) -> ok = ra_update_config(DB, Shard, Opts) end
|
||||||
|
).
|
||||||
|
|
||||||
-spec list_generations_with_lifetimes(emqx_ds:db()) ->
|
-spec list_generations_with_lifetimes(emqx_ds:db()) ->
|
||||||
#{generation_rank() => emqx_ds:generation_info()}.
|
#{generation_rank() => emqx_ds:generation_info()}.
|
||||||
|
@ -164,13 +181,12 @@ list_generations_with_lifetimes(DB) ->
|
||||||
Shards = list_shards(DB),
|
Shards = list_shards(DB),
|
||||||
lists:foldl(
|
lists:foldl(
|
||||||
fun(Shard, GensAcc) ->
|
fun(Shard, GensAcc) ->
|
||||||
Node = node_of_shard(DB, Shard),
|
|
||||||
maps:fold(
|
maps:fold(
|
||||||
fun(GenId, Data, AccInner) ->
|
fun(GenId, Data, AccInner) ->
|
||||||
AccInner#{{Shard, GenId} => Data}
|
AccInner#{{Shard, GenId} => Data}
|
||||||
end,
|
end,
|
||||||
GensAcc,
|
GensAcc,
|
||||||
emqx_ds_proto_v4:list_generations_with_lifetimes(Node, DB, Shard)
|
ra_list_generations_with_lifetimes(DB, Shard)
|
||||||
)
|
)
|
||||||
end,
|
end,
|
||||||
#{},
|
#{},
|
||||||
|
@ -179,18 +195,15 @@ list_generations_with_lifetimes(DB) ->
|
||||||
|
|
||||||
-spec drop_generation(emqx_ds:db(), generation_rank()) -> ok | {error, _}.
|
-spec drop_generation(emqx_ds:db(), generation_rank()) -> ok | {error, _}.
|
||||||
drop_generation(DB, {Shard, GenId}) ->
|
drop_generation(DB, {Shard, GenId}) ->
|
||||||
%% TODO: drop generation in all nodes in the replica set, not only in the leader,
|
ra_drop_generation(DB, Shard, GenId).
|
||||||
%% after we have proper replication in place.
|
|
||||||
Node = node_of_shard(DB, Shard),
|
|
||||||
emqx_ds_proto_v4:drop_generation(Node, DB, Shard, GenId).
|
|
||||||
|
|
||||||
-spec drop_db(emqx_ds:db()) -> ok | {error, _}.
|
-spec drop_db(emqx_ds:db()) -> ok | {error, _}.
|
||||||
drop_db(DB) ->
|
drop_db(DB) ->
|
||||||
Nodes = list_nodes(),
|
foreach_shard(DB, fun(Shard) ->
|
||||||
_ = emqx_ds_proto_v4:drop_db(Nodes, DB),
|
{ok, _} = ra_drop_shard(DB, Shard)
|
||||||
_ = emqx_ds_replication_layer_meta:drop_db(DB),
|
end),
|
||||||
emqx_ds_builtin_sup:stop_db(DB),
|
_ = emqx_ds_proto_v4:drop_db(list_nodes(), DB),
|
||||||
ok.
|
emqx_ds_replication_layer_meta:drop_db(DB).
|
||||||
|
|
||||||
-spec store_batch(emqx_ds:db(), [emqx_types:message(), ...], emqx_ds:message_store_opts()) ->
|
-spec store_batch(emqx_ds:db(), [emqx_types:message(), ...], emqx_ds:message_store_opts()) ->
|
||||||
emqx_ds:store_batch_result().
|
emqx_ds:store_batch_result().
|
||||||
|
@ -208,10 +221,9 @@ get_streams(DB, TopicFilter, StartTime) ->
|
||||||
Shards = list_shards(DB),
|
Shards = list_shards(DB),
|
||||||
lists:flatmap(
|
lists:flatmap(
|
||||||
fun(Shard) ->
|
fun(Shard) ->
|
||||||
Node = node_of_shard(DB, Shard),
|
|
||||||
Streams =
|
Streams =
|
||||||
try
|
try
|
||||||
emqx_ds_proto_v4:get_streams(Node, DB, Shard, TopicFilter, StartTime)
|
ra_get_streams(DB, Shard, TopicFilter, StartTime)
|
||||||
catch
|
catch
|
||||||
error:{erpc, _} ->
|
error:{erpc, _} ->
|
||||||
%% TODO: log?
|
%% TODO: log?
|
||||||
|
@ -235,8 +247,7 @@ get_delete_streams(DB, TopicFilter, StartTime) ->
|
||||||
Shards = list_shards(DB),
|
Shards = list_shards(DB),
|
||||||
lists:flatmap(
|
lists:flatmap(
|
||||||
fun(Shard) ->
|
fun(Shard) ->
|
||||||
Node = node_of_shard(DB, Shard),
|
Streams = ra_get_delete_streams(DB, Shard, TopicFilter, StartTime),
|
||||||
Streams = emqx_ds_proto_v4:get_delete_streams(Node, DB, Shard, TopicFilter, StartTime),
|
|
||||||
lists:map(
|
lists:map(
|
||||||
fun(StorageLayerStream) ->
|
fun(StorageLayerStream) ->
|
||||||
?delete_stream(Shard, StorageLayerStream)
|
?delete_stream(Shard, StorageLayerStream)
|
||||||
|
@ -251,8 +262,7 @@ get_delete_streams(DB, TopicFilter, StartTime) ->
|
||||||
emqx_ds:make_iterator_result(iterator()).
|
emqx_ds:make_iterator_result(iterator()).
|
||||||
make_iterator(DB, Stream, TopicFilter, StartTime) ->
|
make_iterator(DB, Stream, TopicFilter, StartTime) ->
|
||||||
?stream_v2(Shard, StorageStream) = Stream,
|
?stream_v2(Shard, StorageStream) = Stream,
|
||||||
Node = node_of_shard(DB, Shard),
|
try ra_make_iterator(DB, Shard, StorageStream, TopicFilter, StartTime) of
|
||||||
try emqx_ds_proto_v4:make_iterator(Node, DB, Shard, StorageStream, TopicFilter, StartTime) of
|
|
||||||
{ok, Iter} ->
|
{ok, Iter} ->
|
||||||
{ok, #{?tag => ?IT, ?shard => Shard, ?enc => Iter}};
|
{ok, #{?tag => ?IT, ?shard => Shard, ?enc => Iter}};
|
||||||
Error = {error, _, _} ->
|
Error = {error, _, _} ->
|
||||||
|
@ -266,12 +276,7 @@ make_iterator(DB, Stream, TopicFilter, StartTime) ->
|
||||||
emqx_ds:make_delete_iterator_result(delete_iterator()).
|
emqx_ds:make_delete_iterator_result(delete_iterator()).
|
||||||
make_delete_iterator(DB, Stream, TopicFilter, StartTime) ->
|
make_delete_iterator(DB, Stream, TopicFilter, StartTime) ->
|
||||||
?delete_stream(Shard, StorageStream) = Stream,
|
?delete_stream(Shard, StorageStream) = Stream,
|
||||||
Node = node_of_shard(DB, Shard),
|
case ra_make_delete_iterator(DB, Shard, StorageStream, TopicFilter, StartTime) of
|
||||||
case
|
|
||||||
emqx_ds_proto_v4:make_delete_iterator(
|
|
||||||
Node, DB, Shard, StorageStream, TopicFilter, StartTime
|
|
||||||
)
|
|
||||||
of
|
|
||||||
{ok, Iter} ->
|
{ok, Iter} ->
|
||||||
{ok, #{?tag => ?DELETE_IT, ?shard => Shard, ?enc => Iter}};
|
{ok, #{?tag => ?DELETE_IT, ?shard => Shard, ?enc => Iter}};
|
||||||
Err = {error, _} ->
|
Err = {error, _} ->
|
||||||
|
@ -282,8 +287,7 @@ make_delete_iterator(DB, Stream, TopicFilter, StartTime) ->
|
||||||
emqx_ds:make_iterator_result(iterator()).
|
emqx_ds:make_iterator_result(iterator()).
|
||||||
update_iterator(DB, OldIter, DSKey) ->
|
update_iterator(DB, OldIter, DSKey) ->
|
||||||
#{?tag := ?IT, ?shard := Shard, ?enc := StorageIter} = OldIter,
|
#{?tag := ?IT, ?shard := Shard, ?enc := StorageIter} = OldIter,
|
||||||
Node = node_of_shard(DB, Shard),
|
try ra_update_iterator(DB, Shard, StorageIter, DSKey) of
|
||||||
try emqx_ds_proto_v4:update_iterator(Node, DB, Shard, StorageIter, DSKey) of
|
|
||||||
{ok, Iter} ->
|
{ok, Iter} ->
|
||||||
{ok, #{?tag => ?IT, ?shard => Shard, ?enc => Iter}};
|
{ok, #{?tag => ?IT, ?shard => Shard, ?enc => Iter}};
|
||||||
Error = {error, _, _} ->
|
Error = {error, _, _} ->
|
||||||
|
@ -296,7 +300,6 @@ update_iterator(DB, OldIter, DSKey) ->
|
||||||
-spec next(emqx_ds:db(), iterator(), pos_integer()) -> emqx_ds:next_result(iterator()).
|
-spec next(emqx_ds:db(), iterator(), pos_integer()) -> emqx_ds:next_result(iterator()).
|
||||||
next(DB, Iter0, BatchSize) ->
|
next(DB, Iter0, BatchSize) ->
|
||||||
#{?tag := ?IT, ?shard := Shard, ?enc := StorageIter0} = Iter0,
|
#{?tag := ?IT, ?shard := Shard, ?enc := StorageIter0} = Iter0,
|
||||||
Node = node_of_shard(DB, Shard),
|
|
||||||
%% TODO: iterator can contain information that is useful for
|
%% TODO: iterator can contain information that is useful for
|
||||||
%% reconstructing messages sent over the network. For example,
|
%% reconstructing messages sent over the network. For example,
|
||||||
%% when we send messages with the learned topic index, we could
|
%% when we send messages with the learned topic index, we could
|
||||||
|
@ -305,7 +308,7 @@ next(DB, Iter0, BatchSize) ->
|
||||||
%%
|
%%
|
||||||
%% This kind of trickery should be probably done here in the
|
%% This kind of trickery should be probably done here in the
|
||||||
%% replication layer. Or, perhaps, in the logic layer.
|
%% replication layer. Or, perhaps, in the logic layer.
|
||||||
case emqx_ds_proto_v4:next(Node, DB, Shard, StorageIter0, BatchSize) of
|
case ra_next(DB, Shard, StorageIter0, BatchSize) of
|
||||||
{ok, StorageIter, Batch} ->
|
{ok, StorageIter, Batch} ->
|
||||||
Iter = Iter0#{?enc := StorageIter},
|
Iter = Iter0#{?enc := StorageIter},
|
||||||
{ok, Iter, Batch};
|
{ok, Iter, Batch};
|
||||||
|
@ -321,8 +324,7 @@ next(DB, Iter0, BatchSize) ->
|
||||||
emqx_ds:delete_next_result(delete_iterator()).
|
emqx_ds:delete_next_result(delete_iterator()).
|
||||||
delete_next(DB, Iter0, Selector, BatchSize) ->
|
delete_next(DB, Iter0, Selector, BatchSize) ->
|
||||||
#{?tag := ?DELETE_IT, ?shard := Shard, ?enc := StorageIter0} = Iter0,
|
#{?tag := ?DELETE_IT, ?shard := Shard, ?enc := StorageIter0} = Iter0,
|
||||||
Node = node_of_shard(DB, Shard),
|
case ra_delete_next(DB, Shard, StorageIter0, Selector, BatchSize) of
|
||||||
case emqx_ds_proto_v4:delete_next(Node, DB, Shard, StorageIter0, Selector, BatchSize) of
|
|
||||||
{ok, StorageIter, NumDeleted} ->
|
{ok, StorageIter, NumDeleted} ->
|
||||||
Iter = Iter0#{?enc := StorageIter},
|
Iter = Iter0#{?enc := StorageIter},
|
||||||
{ok, Iter, NumDeleted};
|
{ok, Iter, NumDeleted};
|
||||||
|
@ -330,21 +332,10 @@ delete_next(DB, Iter0, Selector, BatchSize) ->
|
||||||
Other
|
Other
|
||||||
end.
|
end.
|
||||||
|
|
||||||
-spec node_of_shard(emqx_ds:db(), shard_id()) -> node().
|
|
||||||
node_of_shard(DB, Shard) ->
|
|
||||||
case emqx_ds_replication_layer_meta:shard_leader(DB, Shard) of
|
|
||||||
{ok, Leader} ->
|
|
||||||
Leader;
|
|
||||||
{error, no_leader_for_shard} ->
|
|
||||||
%% TODO: use optvar
|
|
||||||
timer:sleep(500),
|
|
||||||
node_of_shard(DB, Shard)
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec shard_of_message(emqx_ds:db(), emqx_types:message(), clientid | topic) ->
|
-spec shard_of_message(emqx_ds:db(), emqx_types:message(), clientid | topic) ->
|
||||||
emqx_ds_replication_layer:shard_id().
|
emqx_ds_replication_layer:shard_id().
|
||||||
shard_of_message(DB, #message{from = From, topic = Topic}, SerializeBy) ->
|
shard_of_message(DB, #message{from = From, topic = Topic}, SerializeBy) ->
|
||||||
N = emqx_ds_replication_layer_meta:n_shards(DB),
|
N = emqx_ds_replication_shard_allocator:n_shards(DB),
|
||||||
Hash =
|
Hash =
|
||||||
case SerializeBy of
|
case SerializeBy of
|
||||||
clientid -> erlang:phash2(From, N);
|
clientid -> erlang:phash2(From, N);
|
||||||
|
@ -352,18 +343,8 @@ shard_of_message(DB, #message{from = From, topic = Topic}, SerializeBy) ->
|
||||||
end,
|
end,
|
||||||
integer_to_binary(Hash).
|
integer_to_binary(Hash).
|
||||||
|
|
||||||
%% TODO: there's no real leader election right now
|
foreach_shard(DB, Fun) ->
|
||||||
-spec maybe_set_myself_as_leader(emqx_ds:db(), shard_id()) -> ok.
|
lists:foreach(Fun, list_shards(DB)).
|
||||||
maybe_set_myself_as_leader(DB, Shard) ->
|
|
||||||
Site = emqx_ds_replication_layer_meta:this_site(),
|
|
||||||
case emqx_ds_replication_layer_meta:in_sync_replicas(DB, Shard) of
|
|
||||||
[Site | _] ->
|
|
||||||
%% Currently the first in-sync replica always becomes the
|
|
||||||
%% leader
|
|
||||||
ok = emqx_ds_replication_layer_meta:set_leader(DB, Shard, node());
|
|
||||||
_Sites ->
|
|
||||||
ok
|
|
||||||
end.
|
|
||||||
|
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
%% behavior callbacks
|
%% behavior callbacks
|
||||||
|
@ -392,7 +373,8 @@ do_drop_db_v1(DB) ->
|
||||||
) ->
|
) ->
|
||||||
emqx_ds:store_batch_result().
|
emqx_ds:store_batch_result().
|
||||||
do_store_batch_v1(DB, Shard, #{?tag := ?BATCH, ?batch_messages := Messages}, Options) ->
|
do_store_batch_v1(DB, Shard, #{?tag := ?BATCH, ?batch_messages := Messages}, Options) ->
|
||||||
emqx_ds_storage_layer:store_batch({DB, Shard}, Messages, Options).
|
Batch = [{emqx_message:timestamp(Message), Message} || Message <- Messages],
|
||||||
|
emqx_ds_storage_layer:store_batch({DB, Shard}, Batch, Options).
|
||||||
|
|
||||||
%% Remove me in EMQX 5.6
|
%% Remove me in EMQX 5.6
|
||||||
-dialyzer({nowarn_function, do_get_streams_v1/4}).
|
-dialyzer({nowarn_function, do_get_streams_v1/4}).
|
||||||
|
@ -477,15 +459,9 @@ do_next_v1(DB, Shard, Iter, BatchSize) ->
|
||||||
do_delete_next_v4(DB, Shard, Iter, Selector, BatchSize) ->
|
do_delete_next_v4(DB, Shard, Iter, Selector, BatchSize) ->
|
||||||
emqx_ds_storage_layer:delete_next({DB, Shard}, Iter, Selector, BatchSize).
|
emqx_ds_storage_layer:delete_next({DB, Shard}, Iter, Selector, BatchSize).
|
||||||
|
|
||||||
-spec do_add_generation_v2(emqx_ds:db()) -> ok | {error, _}.
|
-spec do_add_generation_v2(emqx_ds:db()) -> no_return().
|
||||||
do_add_generation_v2(DB) ->
|
do_add_generation_v2(_DB) ->
|
||||||
MyShards = emqx_ds_replication_layer_meta:my_owned_shards(DB),
|
error(obsolete_api).
|
||||||
lists:foreach(
|
|
||||||
fun(ShardId) ->
|
|
||||||
emqx_ds_storage_layer:add_generation({DB, ShardId})
|
|
||||||
end,
|
|
||||||
MyShards
|
|
||||||
).
|
|
||||||
|
|
||||||
-spec do_list_generations_with_lifetimes_v3(emqx_ds:db(), shard_id()) ->
|
-spec do_list_generations_with_lifetimes_v3(emqx_ds:db(), shard_id()) ->
|
||||||
#{emqx_ds:ds_specific_generation_rank() => emqx_ds:generation_info()}.
|
#{emqx_ds:ds_specific_generation_rank() => emqx_ds:generation_info()}.
|
||||||
|
@ -510,3 +486,188 @@ do_get_delete_streams_v4(DB, Shard, TopicFilter, StartTime) ->
|
||||||
|
|
||||||
list_nodes() ->
|
list_nodes() ->
|
||||||
mria:running_nodes().
|
mria:running_nodes().
|
||||||
|
|
||||||
|
%% TODO
|
||||||
|
%% Too large for normal operation, need better backpressure mechanism.
|
||||||
|
-define(RA_TIMEOUT, 60 * 1000).
|
||||||
|
|
||||||
|
ra_store_batch(DB, Shard, Messages) ->
|
||||||
|
Command = #{
|
||||||
|
?tag => ?BATCH,
|
||||||
|
?batch_messages => Messages
|
||||||
|
},
|
||||||
|
Servers = emqx_ds_replication_layer_shard:servers(DB, Shard, leader_preferred),
|
||||||
|
case ra:process_command(Servers, Command, ?RA_TIMEOUT) of
|
||||||
|
{ok, Result, _Leader} ->
|
||||||
|
Result;
|
||||||
|
Error ->
|
||||||
|
Error
|
||||||
|
end.
|
||||||
|
|
||||||
|
ra_add_generation(DB, Shard) ->
|
||||||
|
Command = #{
|
||||||
|
?tag => add_generation,
|
||||||
|
?since => emqx_ds:timestamp_us()
|
||||||
|
},
|
||||||
|
Servers = emqx_ds_replication_layer_shard:servers(DB, Shard, leader_preferred),
|
||||||
|
case ra:process_command(Servers, Command, ?RA_TIMEOUT) of
|
||||||
|
{ok, Result, _Leader} ->
|
||||||
|
Result;
|
||||||
|
Error ->
|
||||||
|
error(Error, [DB, Shard])
|
||||||
|
end.
|
||||||
|
|
||||||
|
ra_update_config(DB, Shard, Opts) ->
|
||||||
|
Command = #{
|
||||||
|
?tag => update_config,
|
||||||
|
?config => Opts,
|
||||||
|
?since => emqx_ds:timestamp_us()
|
||||||
|
},
|
||||||
|
Servers = emqx_ds_replication_layer_shard:servers(DB, Shard, leader_preferred),
|
||||||
|
case ra:process_command(Servers, Command, ?RA_TIMEOUT) of
|
||||||
|
{ok, Result, _Leader} ->
|
||||||
|
Result;
|
||||||
|
Error ->
|
||||||
|
error(Error, [DB, Shard])
|
||||||
|
end.
|
||||||
|
|
||||||
|
ra_drop_generation(DB, Shard, GenId) ->
|
||||||
|
Command = #{?tag => drop_generation, ?generation => GenId},
|
||||||
|
Servers = emqx_ds_replication_layer_shard:servers(DB, Shard, leader_preferred),
|
||||||
|
case ra:process_command(Servers, Command, ?RA_TIMEOUT) of
|
||||||
|
{ok, Result, _Leader} ->
|
||||||
|
Result;
|
||||||
|
Error ->
|
||||||
|
error(Error, [DB, Shard])
|
||||||
|
end.
|
||||||
|
|
||||||
|
ra_get_streams(DB, Shard, TopicFilter, Time) ->
|
||||||
|
{_, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred),
|
||||||
|
TimestampUs = timestamp_to_timeus(Time),
|
||||||
|
emqx_ds_proto_v4:get_streams(Node, DB, Shard, TopicFilter, TimestampUs).
|
||||||
|
|
||||||
|
ra_get_delete_streams(DB, Shard, TopicFilter, Time) ->
|
||||||
|
{_Name, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred),
|
||||||
|
emqx_ds_proto_v4:get_delete_streams(Node, DB, Shard, TopicFilter, Time).
|
||||||
|
|
||||||
|
ra_make_iterator(DB, Shard, Stream, TopicFilter, StartTime) ->
|
||||||
|
{_, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred),
|
||||||
|
TimestampUs = timestamp_to_timeus(StartTime),
|
||||||
|
emqx_ds_proto_v4:make_iterator(Node, DB, Shard, Stream, TopicFilter, TimestampUs).
|
||||||
|
|
||||||
|
ra_make_delete_iterator(DB, Shard, Stream, TopicFilter, StartTime) ->
|
||||||
|
{_Name, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred),
|
||||||
|
emqx_ds_proto_v4:make_delete_iterator(Node, DB, Shard, Stream, TopicFilter, StartTime).
|
||||||
|
|
||||||
|
ra_update_iterator(DB, Shard, Iter, DSKey) ->
|
||||||
|
{_Name, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred),
|
||||||
|
emqx_ds_proto_v4:update_iterator(Node, DB, Shard, Iter, DSKey).
|
||||||
|
|
||||||
|
ra_next(DB, Shard, Iter, BatchSize) ->
|
||||||
|
{_Name, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred),
|
||||||
|
emqx_ds_proto_v4:next(Node, DB, Shard, Iter, BatchSize).
|
||||||
|
|
||||||
|
ra_delete_next(DB, Shard, Iter, Selector, BatchSize) ->
|
||||||
|
{_Name, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred),
|
||||||
|
emqx_ds_proto_v4:delete_next(Node, DB, Shard, Iter, Selector, BatchSize).
|
||||||
|
|
||||||
|
ra_list_generations_with_lifetimes(DB, Shard) ->
|
||||||
|
{_Name, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred),
|
||||||
|
Gens = emqx_ds_proto_v4:list_generations_with_lifetimes(Node, DB, Shard),
|
||||||
|
maps:map(
|
||||||
|
fun(_GenId, Data = #{since := Since, until := Until}) ->
|
||||||
|
Data#{
|
||||||
|
since := timeus_to_timestamp(Since),
|
||||||
|
until := emqx_maybe:apply(fun timeus_to_timestamp/1, Until)
|
||||||
|
}
|
||||||
|
end,
|
||||||
|
Gens
|
||||||
|
).
|
||||||
|
|
||||||
|
ra_drop_shard(DB, Shard) ->
|
||||||
|
ra:delete_cluster(emqx_ds_replication_layer_shard:shard_servers(DB, Shard), ?RA_TIMEOUT).
|
||||||
|
|
||||||
|
%%
|
||||||
|
|
||||||
|
init(#{db := DB, shard := Shard}) ->
|
||||||
|
#{db_shard => {DB, Shard}, latest => 0}.
|
||||||
|
|
||||||
|
apply(
|
||||||
|
#{index := RaftIdx},
|
||||||
|
#{
|
||||||
|
?tag := ?BATCH,
|
||||||
|
?batch_messages := MessagesIn
|
||||||
|
},
|
||||||
|
#{db_shard := DBShard, latest := Latest} = State
|
||||||
|
) ->
|
||||||
|
%% NOTE
|
||||||
|
%% Unique timestamp tracking real time closely.
|
||||||
|
%% With microsecond granularity it should be nearly impossible for it to run
|
||||||
|
%% too far ahead than the real time clock.
|
||||||
|
{NLatest, Messages} = assign_timestamps(Latest, MessagesIn),
|
||||||
|
%% TODO
|
||||||
|
%% Batch is now reversed, but it should not make a lot of difference.
|
||||||
|
%% Even if it would be in order, it's still possible to write messages far away
|
||||||
|
%% in the past, i.e. when replica catches up with the leader. Storage layer
|
||||||
|
%% currently relies on wall clock time to decide if it's safe to iterate over
|
||||||
|
%% next epoch, this is likely wrong. Ideally it should rely on consensus clock
|
||||||
|
%% time instead.
|
||||||
|
Result = emqx_ds_storage_layer:store_batch(DBShard, Messages, #{}),
|
||||||
|
NState = State#{latest := NLatest},
|
||||||
|
%% TODO: Need to measure effects of changing frequency of `release_cursor`.
|
||||||
|
Effect = {release_cursor, RaftIdx, NState},
|
||||||
|
{NState, Result, Effect};
|
||||||
|
apply(
|
||||||
|
_RaftMeta,
|
||||||
|
#{?tag := add_generation, ?since := Since},
|
||||||
|
#{db_shard := DBShard, latest := Latest} = State
|
||||||
|
) ->
|
||||||
|
{Timestamp, NLatest} = ensure_monotonic_timestamp(Since, Latest),
|
||||||
|
Result = emqx_ds_storage_layer:add_generation(DBShard, Timestamp),
|
||||||
|
NState = State#{latest := NLatest},
|
||||||
|
{NState, Result};
|
||||||
|
apply(
|
||||||
|
_RaftMeta,
|
||||||
|
#{?tag := update_config, ?since := Since, ?config := Opts},
|
||||||
|
#{db_shard := DBShard, latest := Latest} = State
|
||||||
|
) ->
|
||||||
|
{Timestamp, NLatest} = ensure_monotonic_timestamp(Since, Latest),
|
||||||
|
Result = emqx_ds_storage_layer:update_config(DBShard, Timestamp, Opts),
|
||||||
|
NState = State#{latest := NLatest},
|
||||||
|
{NState, Result};
|
||||||
|
apply(
|
||||||
|
_RaftMeta,
|
||||||
|
#{?tag := drop_generation, ?generation := GenId},
|
||||||
|
#{db_shard := DBShard} = State
|
||||||
|
) ->
|
||||||
|
Result = emqx_ds_storage_layer:drop_generation(DBShard, GenId),
|
||||||
|
{State, Result}.
|
||||||
|
|
||||||
|
assign_timestamps(Latest, Messages) ->
|
||||||
|
assign_timestamps(Latest, Messages, []).
|
||||||
|
|
||||||
|
assign_timestamps(Latest, [MessageIn | Rest], Acc) ->
|
||||||
|
case emqx_message:timestamp(MessageIn, microsecond) of
|
||||||
|
TimestampUs when TimestampUs > Latest ->
|
||||||
|
Message = assign_timestamp(TimestampUs, MessageIn),
|
||||||
|
assign_timestamps(TimestampUs, Rest, [Message | Acc]);
|
||||||
|
_Earlier ->
|
||||||
|
Message = assign_timestamp(Latest + 1, MessageIn),
|
||||||
|
assign_timestamps(Latest + 1, Rest, [Message | Acc])
|
||||||
|
end;
|
||||||
|
assign_timestamps(Latest, [], Acc) ->
|
||||||
|
{Latest, Acc}.
|
||||||
|
|
||||||
|
assign_timestamp(TimestampUs, Message) ->
|
||||||
|
{TimestampUs, Message}.
|
||||||
|
|
||||||
|
ensure_monotonic_timestamp(TimestampUs, Latest) when TimestampUs > Latest ->
|
||||||
|
{TimestampUs, TimestampUs + 1};
|
||||||
|
ensure_monotonic_timestamp(_TimestampUs, Latest) ->
|
||||||
|
{Latest, Latest + 1}.
|
||||||
|
|
||||||
|
timestamp_to_timeus(TimestampMs) ->
|
||||||
|
TimestampMs * 1000.
|
||||||
|
|
||||||
|
timeus_to_timestamp(TimestampUs) ->
|
||||||
|
TimestampUs div 1000.
|
||||||
|
|
|
@ -29,6 +29,16 @@
|
||||||
-define(tag, 1).
|
-define(tag, 1).
|
||||||
-define(shard, 2).
|
-define(shard, 2).
|
||||||
-define(enc, 3).
|
-define(enc, 3).
|
||||||
|
|
||||||
|
%% ?BATCH
|
||||||
-define(batch_messages, 2).
|
-define(batch_messages, 2).
|
||||||
|
-define(timestamp, 3).
|
||||||
|
|
||||||
|
%% add_generation / update_config
|
||||||
|
-define(config, 2).
|
||||||
|
-define(since, 3).
|
||||||
|
|
||||||
|
%% drop_generation
|
||||||
|
-define(generation, 2).
|
||||||
|
|
||||||
-endif.
|
-endif.
|
||||||
|
|
|
@ -40,7 +40,6 @@
|
||||||
|
|
||||||
-export_type([]).
|
-export_type([]).
|
||||||
|
|
||||||
-include("emqx_ds_replication_layer.hrl").
|
|
||||||
-include_lib("snabbkaffe/include/trace.hrl").
|
-include_lib("snabbkaffe/include/trace.hrl").
|
||||||
|
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
@ -109,7 +108,6 @@ store_batch(DB, Messages, Opts) ->
|
||||||
-record(s, {
|
-record(s, {
|
||||||
db :: emqx_ds:db(),
|
db :: emqx_ds:db(),
|
||||||
shard :: emqx_ds_replication_layer:shard_id(),
|
shard :: emqx_ds_replication_layer:shard_id(),
|
||||||
leader :: node(),
|
|
||||||
n = 0 :: non_neg_integer(),
|
n = 0 :: non_neg_integer(),
|
||||||
tref :: reference(),
|
tref :: reference(),
|
||||||
batch = [] :: [emqx_types:message()],
|
batch = [] :: [emqx_types:message()],
|
||||||
|
@ -119,12 +117,9 @@ store_batch(DB, Messages, Opts) ->
|
||||||
init([DB, Shard]) ->
|
init([DB, Shard]) ->
|
||||||
process_flag(trap_exit, true),
|
process_flag(trap_exit, true),
|
||||||
process_flag(message_queue_data, off_heap),
|
process_flag(message_queue_data, off_heap),
|
||||||
%% TODO: adjust leader dynamically
|
|
||||||
Leader = shard_leader(DB, Shard),
|
|
||||||
S = #s{
|
S = #s{
|
||||||
db = DB,
|
db = DB,
|
||||||
shard = Shard,
|
shard = Shard,
|
||||||
leader = Leader,
|
|
||||||
tref = start_timer()
|
tref = start_timer()
|
||||||
},
|
},
|
||||||
{ok, S}.
|
{ok, S}.
|
||||||
|
@ -156,16 +151,32 @@ terminate(_Reason, _S) ->
|
||||||
%% Internal functions
|
%% Internal functions
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
||||||
|
-define(COOLDOWN_MIN, 1000).
|
||||||
|
-define(COOLDOWN_MAX, 5000).
|
||||||
|
|
||||||
do_flush(S = #s{batch = []}) ->
|
do_flush(S = #s{batch = []}) ->
|
||||||
S#s{tref = start_timer()};
|
S#s{tref = start_timer()};
|
||||||
do_flush(
|
do_flush(
|
||||||
S = #s{batch = Messages, pending_replies = Replies, db = DB, shard = Shard, leader = Leader}
|
S = #s{batch = Messages, pending_replies = Replies, db = DB, shard = Shard}
|
||||||
) ->
|
) ->
|
||||||
Batch = #{?tag => ?BATCH, ?batch_messages => lists:reverse(Messages)},
|
case emqx_ds_replication_layer:ra_store_batch(DB, Shard, lists:reverse(Messages)) of
|
||||||
ok = emqx_ds_proto_v2:store_batch(Leader, DB, Shard, Batch, #{}),
|
ok ->
|
||||||
[gen_server:reply(From, ok) || From <- lists:reverse(Replies)],
|
lists:foreach(fun(From) -> gen_server:reply(From, ok) end, Replies),
|
||||||
?tp(emqx_ds_replication_layer_egress_flush, #{db => DB, shard => Shard, batch => Messages}),
|
true = erlang:garbage_collect(),
|
||||||
erlang:garbage_collect(),
|
?tp(
|
||||||
|
emqx_ds_replication_layer_egress_flush,
|
||||||
|
#{db => DB, shard => Shard, batch => Messages}
|
||||||
|
);
|
||||||
|
Error ->
|
||||||
|
true = erlang:garbage_collect(),
|
||||||
|
?tp(
|
||||||
|
warning,
|
||||||
|
emqx_ds_replication_layer_egress_flush_failed,
|
||||||
|
#{db => DB, shard => Shard, reason => Error}
|
||||||
|
),
|
||||||
|
Cooldown = ?COOLDOWN_MIN + rand:uniform(?COOLDOWN_MAX - ?COOLDOWN_MIN),
|
||||||
|
ok = timer:sleep(Cooldown)
|
||||||
|
end,
|
||||||
S#s{
|
S#s{
|
||||||
n = 0,
|
n = 0,
|
||||||
batch = [],
|
batch = [],
|
||||||
|
@ -212,13 +223,3 @@ do_enqueue(From, Sync, MsgOrBatch, S0 = #s{n = N, batch = Batch, pending_replies
|
||||||
start_timer() ->
|
start_timer() ->
|
||||||
Interval = application:get_env(emqx_durable_storage, egress_flush_interval, 100),
|
Interval = application:get_env(emqx_durable_storage, egress_flush_interval, 100),
|
||||||
erlang:send_after(Interval, self(), ?flush).
|
erlang:send_after(Interval, self(), ?flush).
|
||||||
|
|
||||||
shard_leader(DB, Shard) ->
|
|
||||||
%% TODO: use optvar
|
|
||||||
case emqx_ds_replication_layer_meta:shard_leader(DB, Shard) of
|
|
||||||
{ok, Leader} ->
|
|
||||||
Leader;
|
|
||||||
{error, no_leader_for_shard} ->
|
|
||||||
timer:sleep(500),
|
|
||||||
shard_leader(DB, Shard)
|
|
||||||
end.
|
|
||||||
|
|
|
@ -29,19 +29,15 @@
|
||||||
-export([
|
-export([
|
||||||
shards/1,
|
shards/1,
|
||||||
my_shards/1,
|
my_shards/1,
|
||||||
my_owned_shards/1,
|
allocate_shards/2,
|
||||||
leader_nodes/1,
|
|
||||||
replica_set/2,
|
replica_set/2,
|
||||||
in_sync_replicas/2,
|
|
||||||
sites/0,
|
sites/0,
|
||||||
|
node/1,
|
||||||
open_db/2,
|
open_db/2,
|
||||||
get_options/1,
|
get_options/1,
|
||||||
update_db_config/2,
|
update_db_config/2,
|
||||||
drop_db/1,
|
drop_db/1,
|
||||||
shard_leader/2,
|
|
||||||
this_site/0,
|
this_site/0,
|
||||||
set_leader/3,
|
|
||||||
is_leader/1,
|
|
||||||
print_status/0
|
print_status/0
|
||||||
]).
|
]).
|
||||||
|
|
||||||
|
@ -51,12 +47,10 @@
|
||||||
%% internal exports:
|
%% internal exports:
|
||||||
-export([
|
-export([
|
||||||
open_db_trans/2,
|
open_db_trans/2,
|
||||||
|
allocate_shards_trans/2,
|
||||||
update_db_config_trans/2,
|
update_db_config_trans/2,
|
||||||
drop_db_trans/1,
|
drop_db_trans/1,
|
||||||
claim_site/2,
|
claim_site/2,
|
||||||
in_sync_replicas_trans/2,
|
|
||||||
set_leader_trans/3,
|
|
||||||
is_leader_trans/1,
|
|
||||||
n_shards/1
|
n_shards/1
|
||||||
]).
|
]).
|
||||||
|
|
||||||
|
@ -95,9 +89,6 @@
|
||||||
%% Sites that should contain the data when the cluster is in the
|
%% Sites that should contain the data when the cluster is in the
|
||||||
%% stable state (no nodes are being added or removed from it):
|
%% stable state (no nodes are being added or removed from it):
|
||||||
replica_set :: [site()],
|
replica_set :: [site()],
|
||||||
%% Sites that contain the actual data:
|
|
||||||
in_sync_replicas :: [site()],
|
|
||||||
leader :: node() | undefined,
|
|
||||||
misc = #{} :: map()
|
misc = #{} :: map()
|
||||||
}).
|
}).
|
||||||
|
|
||||||
|
@ -107,13 +98,24 @@
|
||||||
%% Peristent term key:
|
%% Peristent term key:
|
||||||
-define(emqx_ds_builtin_site, emqx_ds_builtin_site).
|
-define(emqx_ds_builtin_site, emqx_ds_builtin_site).
|
||||||
|
|
||||||
|
%% Make Dialyzer happy
|
||||||
|
-define(NODE_PAT(),
|
||||||
|
%% Equivalent of `#?NODE_TAB{_ = '_'}`:
|
||||||
|
erlang:make_tuple(record_info(size, ?NODE_TAB), '_')
|
||||||
|
).
|
||||||
|
|
||||||
|
-define(SHARD_PAT(SHARD),
|
||||||
|
%% Equivalent of `#?SHARD_TAB{shard = SHARD, _ = '_'}`
|
||||||
|
erlang:make_tuple(record_info(size, ?SHARD_TAB), '_', [{#?SHARD_TAB.shard, SHARD}])
|
||||||
|
).
|
||||||
|
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
%% API funcions
|
%% API funcions
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
||||||
-spec print_status() -> ok.
|
-spec print_status() -> ok.
|
||||||
print_status() ->
|
print_status() ->
|
||||||
io:format("THIS SITE:~n~s~n", [base64:encode(this_site())]),
|
io:format("THIS SITE:~n~s~n", [this_site()]),
|
||||||
io:format("~nSITES:~n", []),
|
io:format("~nSITES:~n", []),
|
||||||
Nodes = [node() | nodes()],
|
Nodes = [node() | nodes()],
|
||||||
lists:foreach(
|
lists:foreach(
|
||||||
|
@ -123,28 +125,18 @@ print_status() ->
|
||||||
true -> up;
|
true -> up;
|
||||||
false -> down
|
false -> down
|
||||||
end,
|
end,
|
||||||
io:format("~s ~p ~p~n", [base64:encode(Site), Node, Status])
|
io:format("~s ~p ~p~n", [Site, Node, Status])
|
||||||
end,
|
end,
|
||||||
eval_qlc(mnesia:table(?NODE_TAB))
|
eval_qlc(mnesia:table(?NODE_TAB))
|
||||||
),
|
),
|
||||||
io:format(
|
io:format(
|
||||||
"~nSHARDS:~nId Leader Status~n", []
|
"~nSHARDS:~nId Replicas~n", []
|
||||||
),
|
),
|
||||||
lists:foreach(
|
lists:foreach(
|
||||||
fun(#?SHARD_TAB{shard = {DB, Shard}, leader = Leader}) ->
|
fun(#?SHARD_TAB{shard = {DB, Shard}, replica_set = RS}) ->
|
||||||
ShardStr = string:pad(io_lib:format("~p/~s", [DB, Shard]), 30),
|
ShardStr = string:pad(io_lib:format("~p/~s", [DB, Shard]), 30),
|
||||||
LeaderStr = string:pad(atom_to_list(Leader), 33),
|
ReplicasStr = string:pad(io_lib:format("~p", [RS]), 40),
|
||||||
Status =
|
io:format("~s ~s~n", [ShardStr, ReplicasStr])
|
||||||
case lists:member(Leader, Nodes) of
|
|
||||||
true ->
|
|
||||||
case node() of
|
|
||||||
Leader -> "up *";
|
|
||||||
_ -> "up"
|
|
||||||
end;
|
|
||||||
false ->
|
|
||||||
"down"
|
|
||||||
end,
|
|
||||||
io:format("~s ~s ~s~n", [ShardStr, LeaderStr, Status])
|
|
||||||
end,
|
end,
|
||||||
eval_qlc(mnesia:table(?SHARD_TAB))
|
eval_qlc(mnesia:table(?SHARD_TAB))
|
||||||
).
|
).
|
||||||
|
@ -169,30 +161,19 @@ shards(DB) ->
|
||||||
-spec my_shards(emqx_ds:db()) -> [emqx_ds_replication_layer:shard_id()].
|
-spec my_shards(emqx_ds:db()) -> [emqx_ds_replication_layer:shard_id()].
|
||||||
my_shards(DB) ->
|
my_shards(DB) ->
|
||||||
Site = this_site(),
|
Site = this_site(),
|
||||||
filter_shards(DB, fun(#?SHARD_TAB{replica_set = ReplicaSet, in_sync_replicas = InSync}) ->
|
filter_shards(DB, fun(#?SHARD_TAB{replica_set = ReplicaSet}) ->
|
||||||
lists:member(Site, ReplicaSet) orelse lists:member(Site, InSync)
|
lists:member(Site, ReplicaSet)
|
||||||
end).
|
end).
|
||||||
|
|
||||||
-spec my_owned_shards(emqx_ds:db()) -> [emqx_ds_replication_layer:shard_id()].
|
allocate_shards(DB, Opts) ->
|
||||||
my_owned_shards(DB) ->
|
case mria:transaction(?SHARD, fun ?MODULE:allocate_shards_trans/2, [DB, Opts]) of
|
||||||
Self = node(),
|
{atomic, Shards} ->
|
||||||
filter_shards(DB, fun(#?SHARD_TAB{leader = Leader}) ->
|
{ok, Shards};
|
||||||
Self =:= Leader
|
{aborted, {shards_already_allocated, Shards}} ->
|
||||||
end).
|
{ok, Shards};
|
||||||
|
{aborted, {insufficient_sites_online, Needed, Sites}} ->
|
||||||
-spec leader_nodes(emqx_ds:db()) -> [node()].
|
{error, #{reason => insufficient_sites_online, needed => Needed, sites => Sites}}
|
||||||
leader_nodes(DB) ->
|
end.
|
||||||
lists:uniq(
|
|
||||||
filter_shards(
|
|
||||||
DB,
|
|
||||||
fun(#?SHARD_TAB{leader = Leader}) ->
|
|
||||||
Leader =/= undefined
|
|
||||||
end,
|
|
||||||
fun(#?SHARD_TAB{leader = Leader}) ->
|
|
||||||
Leader
|
|
||||||
end
|
|
||||||
)
|
|
||||||
).
|
|
||||||
|
|
||||||
-spec replica_set(emqx_ds:db(), emqx_ds_replication_layer:shard_id()) ->
|
-spec replica_set(emqx_ds:db(), emqx_ds_replication_layer:shard_id()) ->
|
||||||
{ok, [site()]} | {error, _}.
|
{ok, [site()]} | {error, _}.
|
||||||
|
@ -204,46 +185,27 @@ replica_set(DB, Shard) ->
|
||||||
{error, no_shard}
|
{error, no_shard}
|
||||||
end.
|
end.
|
||||||
|
|
||||||
-spec in_sync_replicas(emqx_ds:db(), emqx_ds_replication_layer:shard_id()) ->
|
|
||||||
[site()].
|
|
||||||
in_sync_replicas(DB, ShardId) ->
|
|
||||||
{atomic, Result} = mria:transaction(?SHARD, fun ?MODULE:in_sync_replicas_trans/2, [DB, ShardId]),
|
|
||||||
case Result of
|
|
||||||
{ok, InSync} ->
|
|
||||||
InSync;
|
|
||||||
{error, _} ->
|
|
||||||
[]
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec sites() -> [site()].
|
-spec sites() -> [site()].
|
||||||
sites() ->
|
sites() ->
|
||||||
eval_qlc(qlc:q([Site || #?NODE_TAB{site = Site} <- mnesia:table(?NODE_TAB)])).
|
eval_qlc(qlc:q([Site || #?NODE_TAB{site = Site} <- mnesia:table(?NODE_TAB)])).
|
||||||
|
|
||||||
-spec shard_leader(emqx_ds:db(), emqx_ds_replication_layer:shard_id()) ->
|
-spec node(site()) -> node() | undefined.
|
||||||
{ok, node()} | {error, no_leader_for_shard}.
|
node(Site) ->
|
||||||
shard_leader(DB, Shard) ->
|
case mnesia:dirty_read(?NODE_TAB, Site) of
|
||||||
case mnesia:dirty_read(?SHARD_TAB, {DB, Shard}) of
|
[#?NODE_TAB{node = Node}] ->
|
||||||
[#?SHARD_TAB{leader = Leader}] when Leader =/= undefined ->
|
Node;
|
||||||
{ok, Leader};
|
[] ->
|
||||||
_ ->
|
undefined
|
||||||
{error, no_leader_for_shard}
|
|
||||||
end.
|
end.
|
||||||
|
|
||||||
-spec set_leader(emqx_ds:db(), emqx_ds_replication_layer:shard_id(), node()) ->
|
|
||||||
ok.
|
|
||||||
set_leader(DB, Shard, Node) ->
|
|
||||||
{atomic, _} = mria:transaction(?SHARD, fun ?MODULE:set_leader_trans/3, [DB, Shard, Node]),
|
|
||||||
ok.
|
|
||||||
|
|
||||||
-spec is_leader(node()) -> boolean().
|
|
||||||
is_leader(Node) ->
|
|
||||||
{atomic, Result} = mria:transaction(?SHARD, fun ?MODULE:is_leader_trans/1, [Node]),
|
|
||||||
Result.
|
|
||||||
|
|
||||||
-spec get_options(emqx_ds:db()) -> emqx_ds_replication_layer:builtin_db_opts().
|
-spec get_options(emqx_ds:db()) -> emqx_ds_replication_layer:builtin_db_opts().
|
||||||
get_options(DB) ->
|
get_options(DB) ->
|
||||||
{atomic, Opts} = mria:transaction(?SHARD, fun ?MODULE:open_db_trans/2, [DB, undefined]),
|
case mnesia:dirty_read(?META_TAB, DB) of
|
||||||
Opts.
|
[#?META_TAB{db_props = Opts}] ->
|
||||||
|
Opts;
|
||||||
|
[] ->
|
||||||
|
#{}
|
||||||
|
end.
|
||||||
|
|
||||||
-spec open_db(emqx_ds:db(), emqx_ds_replication_layer:builtin_db_opts()) ->
|
-spec open_db(emqx_ds:db(), emqx_ds_replication_layer:builtin_db_opts()) ->
|
||||||
emqx_ds_replication_layer:builtin_db_opts().
|
emqx_ds_replication_layer:builtin_db_opts().
|
||||||
|
@ -275,7 +237,6 @@ init([]) ->
|
||||||
logger:set_process_metadata(#{domain => [ds, meta]}),
|
logger:set_process_metadata(#{domain => [ds, meta]}),
|
||||||
ensure_tables(),
|
ensure_tables(),
|
||||||
ensure_site(),
|
ensure_site(),
|
||||||
{ok, _} = mnesia:subscribe({table, ?META_TAB, detailed}),
|
|
||||||
S = #s{},
|
S = #s{},
|
||||||
{ok, S}.
|
{ok, S}.
|
||||||
|
|
||||||
|
@ -285,18 +246,6 @@ handle_call(_Call, _From, S) ->
|
||||||
handle_cast(_Cast, S) ->
|
handle_cast(_Cast, S) ->
|
||||||
{noreply, S}.
|
{noreply, S}.
|
||||||
|
|
||||||
handle_info(
|
|
||||||
{mnesia_table_event, {write, ?META_TAB, #?META_TAB{db = DB, db_props = Options}, [_], _}}, S
|
|
||||||
) ->
|
|
||||||
MyShards = my_owned_shards(DB),
|
|
||||||
|
|
||||||
lists:foreach(
|
|
||||||
fun(ShardId) ->
|
|
||||||
emqx_ds_storage_layer:update_config({DB, ShardId}, Options)
|
|
||||||
end,
|
|
||||||
MyShards
|
|
||||||
),
|
|
||||||
{noreply, S};
|
|
||||||
handle_info(_Info, S) ->
|
handle_info(_Info, S) ->
|
||||||
{noreply, S}.
|
{noreply, S}.
|
||||||
|
|
||||||
|
@ -308,20 +257,60 @@ terminate(_Reason, #s{}) ->
|
||||||
%% Internal exports
|
%% Internal exports
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
||||||
-spec open_db_trans(emqx_ds:db(), emqx_ds_replication_layer:builtin_db_opts() | undefined) ->
|
-spec open_db_trans(emqx_ds:db(), emqx_ds_replication_layer:builtin_db_opts()) ->
|
||||||
emqx_ds_replication_layer:builtin_db_opts().
|
emqx_ds_replication_layer:builtin_db_opts().
|
||||||
open_db_trans(DB, CreateOpts) ->
|
open_db_trans(DB, CreateOpts) ->
|
||||||
case mnesia:wread({?META_TAB, DB}) of
|
case mnesia:wread({?META_TAB, DB}) of
|
||||||
[] when is_map(CreateOpts) ->
|
[] ->
|
||||||
NShards = maps:get(n_shards, CreateOpts),
|
|
||||||
ReplicationFactor = maps:get(replication_factor, CreateOpts),
|
|
||||||
mnesia:write(#?META_TAB{db = DB, db_props = CreateOpts}),
|
mnesia:write(#?META_TAB{db = DB, db_props = CreateOpts}),
|
||||||
create_shards(DB, NShards, ReplicationFactor),
|
|
||||||
CreateOpts;
|
CreateOpts;
|
||||||
[#?META_TAB{db_props = Opts}] ->
|
[#?META_TAB{db_props = Opts}] ->
|
||||||
Opts
|
Opts
|
||||||
end.
|
end.
|
||||||
|
|
||||||
|
-spec allocate_shards_trans(emqx_ds:db(), emqx_ds_replication_layer:builtin_db_opts()) -> [_Shard].
|
||||||
|
allocate_shards_trans(DB, Opts) ->
|
||||||
|
NShards = maps:get(n_shards, Opts),
|
||||||
|
NSites = maps:get(n_sites, Opts),
|
||||||
|
ReplicationFactor = maps:get(replication_factor, Opts),
|
||||||
|
NReplicas = min(NSites, ReplicationFactor),
|
||||||
|
Shards = [integer_to_binary(I) || I <- lists:seq(0, NShards - 1)],
|
||||||
|
AllSites = mnesia:match_object(?NODE_TAB, ?NODE_PAT(), read),
|
||||||
|
case length(AllSites) of
|
||||||
|
N when N >= NSites ->
|
||||||
|
ok;
|
||||||
|
_ ->
|
||||||
|
mnesia:abort({insufficient_sites_online, NSites, AllSites})
|
||||||
|
end,
|
||||||
|
case mnesia:match_object(?SHARD_TAB, ?SHARD_PAT({DB, '_'}), write) of
|
||||||
|
[] ->
|
||||||
|
ok;
|
||||||
|
Records ->
|
||||||
|
ShardsAllocated = [Shard || #?SHARD_TAB{shard = {_DB, Shard}} <- Records],
|
||||||
|
mnesia:abort({shards_already_allocated, ShardsAllocated})
|
||||||
|
end,
|
||||||
|
{Allocation, _} = lists:mapfoldl(
|
||||||
|
fun(Shard, SSites) ->
|
||||||
|
{Sites, _} = emqx_utils_stream:consume(NReplicas, SSites),
|
||||||
|
{_, SRest} = emqx_utils_stream:consume(1, SSites),
|
||||||
|
{{Shard, Sites}, SRest}
|
||||||
|
end,
|
||||||
|
emqx_utils_stream:repeat(emqx_utils_stream:list(AllSites)),
|
||||||
|
Shards
|
||||||
|
),
|
||||||
|
lists:map(
|
||||||
|
fun({Shard, Sites}) ->
|
||||||
|
ReplicaSet = [Site || #?NODE_TAB{site = Site} <- Sites],
|
||||||
|
Record = #?SHARD_TAB{
|
||||||
|
shard = {DB, Shard},
|
||||||
|
replica_set = ReplicaSet
|
||||||
|
},
|
||||||
|
ok = mnesia:write(Record),
|
||||||
|
Shard
|
||||||
|
end,
|
||||||
|
Allocation
|
||||||
|
).
|
||||||
|
|
||||||
-spec update_db_config_trans(emqx_ds:db(), emqx_ds_replication_layer:builtin_db_opts()) ->
|
-spec update_db_config_trans(emqx_ds:db(), emqx_ds_replication_layer:builtin_db_opts()) ->
|
||||||
ok | {error, database}.
|
ok | {error, database}.
|
||||||
update_db_config_trans(DB, CreateOpts) ->
|
update_db_config_trans(DB, CreateOpts) ->
|
||||||
|
@ -357,51 +346,13 @@ drop_db_trans(DB) ->
|
||||||
claim_site(Site, Node) ->
|
claim_site(Site, Node) ->
|
||||||
mnesia:write(#?NODE_TAB{site = Site, node = Node}).
|
mnesia:write(#?NODE_TAB{site = Site, node = Node}).
|
||||||
|
|
||||||
-spec in_sync_replicas_trans(emqx_ds:db(), emqx_ds_replication_layer:shard_id()) ->
|
|
||||||
{ok, [site()]} | {error, no_shard}.
|
|
||||||
in_sync_replicas_trans(DB, Shard) ->
|
|
||||||
case mnesia:read(?SHARD_TAB, {DB, Shard}) of
|
|
||||||
[#?SHARD_TAB{in_sync_replicas = InSync}] ->
|
|
||||||
{ok, InSync};
|
|
||||||
[] ->
|
|
||||||
{error, no_shard}
|
|
||||||
end.
|
|
||||||
|
|
||||||
-spec set_leader_trans(emqx_ds:db(), emqx_ds_replication_layer:shard_id(), node()) ->
|
|
||||||
ok.
|
|
||||||
set_leader_trans(DB, Shard, Node) ->
|
|
||||||
[Record0] = mnesia:wread({?SHARD_TAB, {DB, Shard}}),
|
|
||||||
Record = Record0#?SHARD_TAB{leader = Node},
|
|
||||||
mnesia:write(Record).
|
|
||||||
|
|
||||||
-spec is_leader_trans(node) -> boolean().
|
|
||||||
is_leader_trans(Node) ->
|
|
||||||
case
|
|
||||||
mnesia:select(
|
|
||||||
?SHARD_TAB,
|
|
||||||
ets:fun2ms(fun(#?SHARD_TAB{leader = Leader}) ->
|
|
||||||
Leader =:= Node
|
|
||||||
end),
|
|
||||||
1,
|
|
||||||
read
|
|
||||||
)
|
|
||||||
of
|
|
||||||
{[_ | _], _Cont} ->
|
|
||||||
true;
|
|
||||||
_ ->
|
|
||||||
false
|
|
||||||
end.
|
|
||||||
|
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
%% Internal functions
|
%% Internal functions
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
||||||
ensure_tables() ->
|
ensure_tables() ->
|
||||||
%% TODO: seems like it may introduce flakiness
|
|
||||||
Majority = false,
|
|
||||||
ok = mria:create_table(?META_TAB, [
|
ok = mria:create_table(?META_TAB, [
|
||||||
{rlog_shard, ?SHARD},
|
{rlog_shard, ?SHARD},
|
||||||
{majority, Majority},
|
|
||||||
{type, ordered_set},
|
{type, ordered_set},
|
||||||
{storage, disc_copies},
|
{storage, disc_copies},
|
||||||
{record_name, ?META_TAB},
|
{record_name, ?META_TAB},
|
||||||
|
@ -409,7 +360,6 @@ ensure_tables() ->
|
||||||
]),
|
]),
|
||||||
ok = mria:create_table(?NODE_TAB, [
|
ok = mria:create_table(?NODE_TAB, [
|
||||||
{rlog_shard, ?SHARD},
|
{rlog_shard, ?SHARD},
|
||||||
{majority, Majority},
|
|
||||||
{type, ordered_set},
|
{type, ordered_set},
|
||||||
{storage, disc_copies},
|
{storage, disc_copies},
|
||||||
{record_name, ?NODE_TAB},
|
{record_name, ?NODE_TAB},
|
||||||
|
@ -417,7 +367,6 @@ ensure_tables() ->
|
||||||
]),
|
]),
|
||||||
ok = mria:create_table(?SHARD_TAB, [
|
ok = mria:create_table(?SHARD_TAB, [
|
||||||
{rlog_shard, ?SHARD},
|
{rlog_shard, ?SHARD},
|
||||||
{majority, Majority},
|
|
||||||
{type, ordered_set},
|
{type, ordered_set},
|
||||||
{storage, disc_copies},
|
{storage, disc_copies},
|
||||||
{record_name, ?SHARD_TAB},
|
{record_name, ?SHARD_TAB},
|
||||||
|
@ -431,8 +380,8 @@ ensure_site() ->
|
||||||
{ok, [Site]} ->
|
{ok, [Site]} ->
|
||||||
ok;
|
ok;
|
||||||
_ ->
|
_ ->
|
||||||
Site = crypto:strong_rand_bytes(8),
|
Site = binary:encode_hex(crypto:strong_rand_bytes(8)),
|
||||||
logger:notice("Creating a new site with ID=~s", [base64:encode(Site)]),
|
logger:notice("Creating a new site with ID=~s", [Site]),
|
||||||
ok = filelib:ensure_dir(Filename),
|
ok = filelib:ensure_dir(Filename),
|
||||||
{ok, FD} = file:open(Filename, [write]),
|
{ok, FD} = file:open(Filename, [write]),
|
||||||
io:format(FD, "~p.", [Site]),
|
io:format(FD, "~p.", [Site]),
|
||||||
|
@ -442,30 +391,6 @@ ensure_site() ->
|
||||||
persistent_term:put(?emqx_ds_builtin_site, Site),
|
persistent_term:put(?emqx_ds_builtin_site, Site),
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
-spec create_shards(emqx_ds:db(), pos_integer(), pos_integer()) -> ok.
|
|
||||||
create_shards(DB, NShards, ReplicationFactor) ->
|
|
||||||
Shards = [integer_to_binary(I) || I <- lists:seq(0, NShards - 1)],
|
|
||||||
AllSites = sites(),
|
|
||||||
lists:foreach(
|
|
||||||
fun(Shard) ->
|
|
||||||
Hashes0 = [{hash(Shard, Site), Site} || Site <- AllSites],
|
|
||||||
Hashes = lists:sort(Hashes0),
|
|
||||||
{_, Sites} = lists:unzip(Hashes),
|
|
||||||
[First | ReplicaSet] = lists:sublist(Sites, 1, ReplicationFactor),
|
|
||||||
Record = #?SHARD_TAB{
|
|
||||||
shard = {DB, Shard},
|
|
||||||
replica_set = ReplicaSet,
|
|
||||||
in_sync_replicas = [First]
|
|
||||||
},
|
|
||||||
mnesia:write(Record)
|
|
||||||
end,
|
|
||||||
Shards
|
|
||||||
).
|
|
||||||
|
|
||||||
-spec hash(emqx_ds_replication_layer:shard_id(), site()) -> any().
|
|
||||||
hash(Shard, Site) ->
|
|
||||||
erlang:phash2({Shard, Site}).
|
|
||||||
|
|
||||||
eval_qlc(Q) ->
|
eval_qlc(Q) ->
|
||||||
case mnesia:is_transaction() of
|
case mnesia:is_transaction() of
|
||||||
true ->
|
true ->
|
||||||
|
|
|
@ -0,0 +1,207 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2024 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_replication_layer_shard).
|
||||||
|
|
||||||
|
-export([start_link/3]).
|
||||||
|
|
||||||
|
%% Static server configuration
|
||||||
|
-export([
|
||||||
|
shard_servers/2,
|
||||||
|
local_server/2
|
||||||
|
]).
|
||||||
|
|
||||||
|
%% Dynamic server location API
|
||||||
|
-export([
|
||||||
|
servers/3,
|
||||||
|
server/3
|
||||||
|
]).
|
||||||
|
|
||||||
|
-behaviour(gen_server).
|
||||||
|
-export([
|
||||||
|
init/1,
|
||||||
|
handle_call/3,
|
||||||
|
handle_cast/2,
|
||||||
|
terminate/2
|
||||||
|
]).
|
||||||
|
|
||||||
|
%%
|
||||||
|
|
||||||
|
start_link(DB, Shard, Opts) ->
|
||||||
|
gen_server:start_link(?MODULE, {DB, Shard, Opts}, []).
|
||||||
|
|
||||||
|
shard_servers(DB, Shard) ->
|
||||||
|
{ok, ReplicaSet} = emqx_ds_replication_layer_meta:replica_set(DB, Shard),
|
||||||
|
[
|
||||||
|
{server_name(DB, Shard, Site), emqx_ds_replication_layer_meta:node(Site)}
|
||||||
|
|| Site <- ReplicaSet
|
||||||
|
].
|
||||||
|
|
||||||
|
local_server(DB, Shard) ->
|
||||||
|
Site = emqx_ds_replication_layer_meta:this_site(),
|
||||||
|
{server_name(DB, Shard, Site), node()}.
|
||||||
|
|
||||||
|
cluster_name(DB, Shard) ->
|
||||||
|
iolist_to_binary(io_lib:format("~s_~s", [DB, Shard])).
|
||||||
|
|
||||||
|
server_name(DB, Shard, Site) ->
|
||||||
|
DBBin = atom_to_binary(DB),
|
||||||
|
binary_to_atom(<<"ds_", DBBin/binary, Shard/binary, "_", Site/binary>>).
|
||||||
|
|
||||||
|
%%
|
||||||
|
|
||||||
|
servers(DB, Shard, _Order = leader_preferred) ->
|
||||||
|
get_servers_leader_preferred(DB, Shard);
|
||||||
|
servers(DB, Shard, _Order = undefined) ->
|
||||||
|
get_shard_servers(DB, Shard).
|
||||||
|
|
||||||
|
server(DB, Shard, _Which = local_preferred) ->
|
||||||
|
get_server_local_preferred(DB, Shard).
|
||||||
|
|
||||||
|
get_servers_leader_preferred(DB, Shard) ->
|
||||||
|
%% NOTE: Contact last known leader first, then rest of shard servers.
|
||||||
|
ClusterName = get_cluster_name(DB, Shard),
|
||||||
|
case ra_leaderboard:lookup_leader(ClusterName) of
|
||||||
|
Leader when Leader /= undefined ->
|
||||||
|
Servers = ra_leaderboard:lookup_members(ClusterName),
|
||||||
|
[Leader | lists:delete(Leader, Servers)];
|
||||||
|
undefined ->
|
||||||
|
%% TODO: Dynamic membership.
|
||||||
|
get_shard_servers(DB, Shard)
|
||||||
|
end.
|
||||||
|
|
||||||
|
get_server_local_preferred(DB, Shard) ->
|
||||||
|
%% NOTE: Contact random replica that is not a known leader.
|
||||||
|
%% TODO: Replica may be down, so we may need to retry.
|
||||||
|
ClusterName = get_cluster_name(DB, Shard),
|
||||||
|
case ra_leaderboard:lookup_members(ClusterName) of
|
||||||
|
Servers when is_list(Servers) ->
|
||||||
|
pick_local(Servers);
|
||||||
|
undefined ->
|
||||||
|
%% TODO
|
||||||
|
%% Leader is unkonwn if there are no servers of this group on the
|
||||||
|
%% local node. We want to pick a replica in that case as well.
|
||||||
|
%% TODO: Dynamic membership.
|
||||||
|
pick_random(get_shard_servers(DB, Shard))
|
||||||
|
end.
|
||||||
|
|
||||||
|
pick_local(Servers) ->
|
||||||
|
case lists:dropwhile(fun({_Name, Node}) -> Node =/= node() end, Servers) of
|
||||||
|
[Local | _] ->
|
||||||
|
Local;
|
||||||
|
[] ->
|
||||||
|
pick_random(Servers)
|
||||||
|
end.
|
||||||
|
|
||||||
|
pick_random(Servers) ->
|
||||||
|
lists:nth(rand:uniform(length(Servers)), Servers).
|
||||||
|
|
||||||
|
get_cluster_name(DB, Shard) ->
|
||||||
|
memoize(fun cluster_name/2, [DB, Shard]).
|
||||||
|
|
||||||
|
get_local_server(DB, Shard) ->
|
||||||
|
memoize(fun local_server/2, [DB, Shard]).
|
||||||
|
|
||||||
|
get_shard_servers(DB, Shard) ->
|
||||||
|
maps:get(servers, emqx_ds_replication_shard_allocator:shard_meta(DB, Shard)).
|
||||||
|
|
||||||
|
%%
|
||||||
|
|
||||||
|
init({DB, Shard, Opts}) ->
|
||||||
|
_ = process_flag(trap_exit, true),
|
||||||
|
_Meta = start_shard(DB, Shard, Opts),
|
||||||
|
{ok, {DB, Shard}}.
|
||||||
|
|
||||||
|
handle_call(_Call, _From, State) ->
|
||||||
|
{reply, ignored, State}.
|
||||||
|
|
||||||
|
handle_cast(_Msg, State) ->
|
||||||
|
{noreply, State}.
|
||||||
|
|
||||||
|
terminate(_Reason, {DB, Shard}) ->
|
||||||
|
LocalServer = get_local_server(DB, Shard),
|
||||||
|
ok = ra:stop_server(DB, LocalServer).
|
||||||
|
|
||||||
|
%%
|
||||||
|
|
||||||
|
start_shard(DB, Shard, #{replication_options := ReplicationOpts}) ->
|
||||||
|
Site = emqx_ds_replication_layer_meta:this_site(),
|
||||||
|
ClusterName = cluster_name(DB, Shard),
|
||||||
|
LocalServer = local_server(DB, Shard),
|
||||||
|
Servers = shard_servers(DB, Shard),
|
||||||
|
case ra:restart_server(DB, LocalServer) of
|
||||||
|
ok ->
|
||||||
|
Bootstrap = false;
|
||||||
|
{error, name_not_registered} ->
|
||||||
|
Bootstrap = true,
|
||||||
|
ok = ra:start_server(DB, #{
|
||||||
|
id => LocalServer,
|
||||||
|
uid => <<ClusterName/binary, "_", Site/binary>>,
|
||||||
|
cluster_name => ClusterName,
|
||||||
|
initial_members => Servers,
|
||||||
|
machine => {module, emqx_ds_replication_layer, #{db => DB, shard => Shard}},
|
||||||
|
log_init_args => maps:with(
|
||||||
|
[
|
||||||
|
snapshot_interval,
|
||||||
|
resend_window
|
||||||
|
],
|
||||||
|
ReplicationOpts
|
||||||
|
)
|
||||||
|
})
|
||||||
|
end,
|
||||||
|
case Servers of
|
||||||
|
[LocalServer | _] ->
|
||||||
|
%% TODO
|
||||||
|
%% Not super robust, but we probably don't expect nodes to be down
|
||||||
|
%% when we bring up a fresh consensus group. Triggering election
|
||||||
|
%% is not really required otherwise.
|
||||||
|
%% TODO
|
||||||
|
%% Ensure that doing that on node restart does not disrupt consensus.
|
||||||
|
%% Edit: looks like it doesn't, this could actually be quite useful
|
||||||
|
%% to "steal" leadership from nodes that have too much leader load.
|
||||||
|
%% TODO
|
||||||
|
%% It doesn't really work that way. There's `ra:transfer_leadership/2`
|
||||||
|
%% for that.
|
||||||
|
try
|
||||||
|
ra:trigger_election(LocalServer, _Timeout = 1_000)
|
||||||
|
catch
|
||||||
|
%% TODO
|
||||||
|
%% Tolerating exceptions because server might be occupied with log
|
||||||
|
%% replay for a while.
|
||||||
|
exit:{timeout, _} when not Bootstrap ->
|
||||||
|
ok
|
||||||
|
end;
|
||||||
|
_ ->
|
||||||
|
ok
|
||||||
|
end,
|
||||||
|
#{
|
||||||
|
cluster_name => ClusterName,
|
||||||
|
servers => Servers,
|
||||||
|
local_server => LocalServer
|
||||||
|
}.
|
||||||
|
|
||||||
|
%%
|
||||||
|
|
||||||
|
memoize(Fun, Args) ->
|
||||||
|
%% NOTE: Assuming that the function is pure and never returns `undefined`.
|
||||||
|
case persistent_term:get([Fun | Args], undefined) of
|
||||||
|
undefined ->
|
||||||
|
Result = erlang:apply(Fun, Args),
|
||||||
|
_ = persistent_term:put([Fun | Args], Result),
|
||||||
|
Result;
|
||||||
|
Result ->
|
||||||
|
Result
|
||||||
|
end.
|
|
@ -0,0 +1,154 @@
|
||||||
|
%%--------------------------------------------------------------------
|
||||||
|
%% Copyright (c) 2024 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_replication_shard_allocator).
|
||||||
|
|
||||||
|
-export([start_link/2]).
|
||||||
|
|
||||||
|
-export([n_shards/1]).
|
||||||
|
-export([shard_meta/2]).
|
||||||
|
|
||||||
|
-behaviour(gen_server).
|
||||||
|
-export([
|
||||||
|
init/1,
|
||||||
|
handle_call/3,
|
||||||
|
handle_cast/2,
|
||||||
|
handle_info/2,
|
||||||
|
terminate/2
|
||||||
|
]).
|
||||||
|
|
||||||
|
-define(db_meta(DB), {?MODULE, DB}).
|
||||||
|
-define(shard_meta(DB, SHARD), {?MODULE, DB, SHARD}).
|
||||||
|
|
||||||
|
%%
|
||||||
|
|
||||||
|
start_link(DB, Opts) ->
|
||||||
|
gen_server:start_link(?MODULE, {DB, Opts}, []).
|
||||||
|
|
||||||
|
n_shards(DB) ->
|
||||||
|
Meta = persistent_term:get(?db_meta(DB)),
|
||||||
|
maps:get(n_shards, Meta).
|
||||||
|
|
||||||
|
shard_meta(DB, Shard) ->
|
||||||
|
persistent_term:get(?shard_meta(DB, Shard)).
|
||||||
|
|
||||||
|
%%
|
||||||
|
|
||||||
|
-define(ALLOCATE_RETRY_TIMEOUT, 1_000).
|
||||||
|
|
||||||
|
init({DB, Opts}) ->
|
||||||
|
_ = erlang:process_flag(trap_exit, true),
|
||||||
|
_ = logger:set_process_metadata(#{db => DB, domain => [ds, db, shard_allocator]}),
|
||||||
|
State = #{db => DB, opts => Opts, status => allocating},
|
||||||
|
case allocate_shards(State) of
|
||||||
|
{ok, NState} ->
|
||||||
|
{ok, NState};
|
||||||
|
{error, Data} ->
|
||||||
|
_ = logger:notice(
|
||||||
|
Data#{
|
||||||
|
msg => "Shard allocation still in progress",
|
||||||
|
retry_in => ?ALLOCATE_RETRY_TIMEOUT
|
||||||
|
}
|
||||||
|
),
|
||||||
|
{ok, State, ?ALLOCATE_RETRY_TIMEOUT}
|
||||||
|
end.
|
||||||
|
|
||||||
|
handle_call(_Call, _From, State) ->
|
||||||
|
{reply, ignored, State}.
|
||||||
|
|
||||||
|
handle_cast(_Cast, State) ->
|
||||||
|
{noreply, State}.
|
||||||
|
|
||||||
|
handle_info(timeout, State) ->
|
||||||
|
case allocate_shards(State) of
|
||||||
|
{ok, NState} ->
|
||||||
|
{noreply, NState};
|
||||||
|
{error, Data} ->
|
||||||
|
_ = logger:notice(
|
||||||
|
Data#{
|
||||||
|
msg => "Shard allocation still in progress",
|
||||||
|
retry_in => ?ALLOCATE_RETRY_TIMEOUT
|
||||||
|
}
|
||||||
|
),
|
||||||
|
{noreply, State, ?ALLOCATE_RETRY_TIMEOUT}
|
||||||
|
end;
|
||||||
|
handle_info(_Info, State) ->
|
||||||
|
{noreply, State}.
|
||||||
|
|
||||||
|
terminate(_Reason, #{db := DB, shards := Shards}) ->
|
||||||
|
erase_db_meta(DB),
|
||||||
|
erase_shards_meta(DB, Shards);
|
||||||
|
terminate(_Reason, #{}) ->
|
||||||
|
ok.
|
||||||
|
|
||||||
|
%%
|
||||||
|
|
||||||
|
allocate_shards(State = #{db := DB, opts := Opts}) ->
|
||||||
|
case emqx_ds_replication_layer_meta:allocate_shards(DB, Opts) of
|
||||||
|
{ok, Shards} ->
|
||||||
|
logger:notice(#{msg => "Shards allocated", shards => Shards}),
|
||||||
|
ok = start_shards(DB, emqx_ds_replication_layer_meta:my_shards(DB)),
|
||||||
|
ok = start_egresses(DB, Shards),
|
||||||
|
ok = save_db_meta(DB, Shards),
|
||||||
|
ok = save_shards_meta(DB, Shards),
|
||||||
|
{ok, State#{shards => Shards, status := ready}};
|
||||||
|
{error, Reason} ->
|
||||||
|
{error, Reason}
|
||||||
|
end.
|
||||||
|
|
||||||
|
start_shards(DB, Shards) ->
|
||||||
|
ok = lists:foreach(
|
||||||
|
fun(Shard) ->
|
||||||
|
ok = emqx_ds_builtin_db_sup:ensure_shard({DB, Shard})
|
||||||
|
end,
|
||||||
|
Shards
|
||||||
|
),
|
||||||
|
ok = logger:info(#{msg => "Shards started", shards => Shards}),
|
||||||
|
ok.
|
||||||
|
|
||||||
|
start_egresses(DB, Shards) ->
|
||||||
|
ok = lists:foreach(
|
||||||
|
fun(Shard) ->
|
||||||
|
ok = emqx_ds_builtin_db_sup:ensure_egress({DB, Shard})
|
||||||
|
end,
|
||||||
|
Shards
|
||||||
|
),
|
||||||
|
logger:info(#{msg => "Egresses started", shards => Shards}),
|
||||||
|
ok.
|
||||||
|
|
||||||
|
save_db_meta(DB, Shards) ->
|
||||||
|
persistent_term:put(?db_meta(DB), #{
|
||||||
|
shards => Shards,
|
||||||
|
n_shards => length(Shards)
|
||||||
|
}).
|
||||||
|
|
||||||
|
save_shards_meta(DB, Shards) ->
|
||||||
|
lists:foreach(fun(Shard) -> save_shard_meta(DB, Shard) end, Shards).
|
||||||
|
|
||||||
|
save_shard_meta(DB, Shard) ->
|
||||||
|
Servers = emqx_ds_replication_layer_shard:shard_servers(DB, Shard),
|
||||||
|
persistent_term:put(?shard_meta(DB, Shard), #{
|
||||||
|
servers => Servers
|
||||||
|
}).
|
||||||
|
|
||||||
|
erase_db_meta(DB) ->
|
||||||
|
persistent_term:erase(?db_meta(DB)).
|
||||||
|
|
||||||
|
erase_shards_meta(DB, Shards) ->
|
||||||
|
lists:foreach(fun(Shard) -> erase_shard_meta(DB, Shard) end, Shards).
|
||||||
|
|
||||||
|
erase_shard_meta(DB, Shard) ->
|
||||||
|
persistent_term:erase(?shard_meta(DB, Shard)).
|
|
@ -137,6 +137,9 @@
|
||||||
|
|
||||||
-include("emqx_ds_bitmask.hrl").
|
-include("emqx_ds_bitmask.hrl").
|
||||||
|
|
||||||
|
-define(DIM_TOPIC, 1).
|
||||||
|
-define(DIM_TS, 2).
|
||||||
|
|
||||||
-ifdef(TEST).
|
-ifdef(TEST).
|
||||||
-include_lib("eunit/include/eunit.hrl").
|
-include_lib("eunit/include/eunit.hrl").
|
||||||
-endif.
|
-endif.
|
||||||
|
@ -160,8 +163,8 @@ create(_ShardId, DBHandle, GenId, Options) ->
|
||||||
%% Get options:
|
%% Get options:
|
||||||
BitsPerTopicLevel = maps:get(bits_per_wildcard_level, Options, 64),
|
BitsPerTopicLevel = maps:get(bits_per_wildcard_level, Options, 64),
|
||||||
TopicIndexBytes = maps:get(topic_index_bytes, Options, 4),
|
TopicIndexBytes = maps:get(topic_index_bytes, Options, 4),
|
||||||
%% 10 bits -> 1024 ms -> ~1 sec
|
%% 20 bits -> 1048576 us -> ~1 sec
|
||||||
TSOffsetBits = maps:get(epoch_bits, Options, 10),
|
TSOffsetBits = maps:get(epoch_bits, Options, 20),
|
||||||
%% Create column families:
|
%% Create column families:
|
||||||
DataCFName = data_cf(GenId),
|
DataCFName = data_cf(GenId),
|
||||||
TrieCFName = trie_cf(GenId),
|
TrieCFName = trie_cf(GenId),
|
||||||
|
@ -242,16 +245,19 @@ drop(_Shard, DBHandle, GenId, CFRefs, #s{}) ->
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
-spec store_batch(
|
-spec store_batch(
|
||||||
emqx_ds_storage_layer:shard_id(), s(), [emqx_types:message()], emqx_ds:message_store_opts()
|
emqx_ds_storage_layer:shard_id(),
|
||||||
|
s(),
|
||||||
|
[{emqx_ds:time(), emqx_types:message()}],
|
||||||
|
emqx_ds:message_store_opts()
|
||||||
) ->
|
) ->
|
||||||
emqx_ds:store_batch_result().
|
emqx_ds:store_batch_result().
|
||||||
store_batch(_ShardId, S = #s{db = DB, data = Data}, Messages, _Options) ->
|
store_batch(_ShardId, S = #s{db = DB, data = Data}, Messages, _Options) ->
|
||||||
{ok, Batch} = rocksdb:batch(),
|
{ok, Batch} = rocksdb:batch(),
|
||||||
lists:foreach(
|
lists:foreach(
|
||||||
fun(Msg) ->
|
fun({Timestamp, Msg}) ->
|
||||||
{Key, _} = make_key(S, Msg),
|
{Key, _} = make_key(S, Timestamp, Msg),
|
||||||
Val = serialize(Msg),
|
Val = serialize(Msg),
|
||||||
rocksdb:batch_put(Batch, Data, Key, Val)
|
rocksdb:put(DB, Data, Key, Val, [])
|
||||||
end,
|
end,
|
||||||
Messages
|
Messages
|
||||||
),
|
),
|
||||||
|
@ -345,7 +351,7 @@ next(_Shard, Schema = #s{ts_offset = TSOffset}, It, BatchSize) ->
|
||||||
%% Compute safe cutoff time.
|
%% Compute safe cutoff time.
|
||||||
%% It's the point in time where the last complete epoch ends, so we need to know
|
%% It's the point in time where the last complete epoch ends, so we need to know
|
||||||
%% the current time to compute it.
|
%% the current time to compute it.
|
||||||
Now = emqx_message:timestamp_now(),
|
Now = emqx_ds:timestamp_us(),
|
||||||
SafeCutoffTime = (Now bsr TSOffset) bsl TSOffset,
|
SafeCutoffTime = (Now bsr TSOffset) bsl TSOffset,
|
||||||
next_until(Schema, It, SafeCutoffTime, BatchSize).
|
next_until(Schema, It, SafeCutoffTime, BatchSize).
|
||||||
|
|
||||||
|
@ -436,9 +442,7 @@ prepare_loop_context(DB, CF, TopicIndex, StartTime, SafeCutoffTime, Varying, Key
|
||||||
%% Make filter:
|
%% Make filter:
|
||||||
Inequations = [
|
Inequations = [
|
||||||
{'=', TopicIndex},
|
{'=', TopicIndex},
|
||||||
{StartTime, '..', SafeCutoffTime - 1},
|
{StartTime, '..', SafeCutoffTime - 1}
|
||||||
%% Unique integer:
|
|
||||||
any
|
|
||||||
%% Varying topic levels:
|
%% Varying topic levels:
|
||||||
| lists:map(
|
| lists:map(
|
||||||
fun
|
fun
|
||||||
|
@ -483,39 +487,44 @@ next_loop(ITHandle, KeyMapper, Filter, Cutoff, It0, Acc0, N0) ->
|
||||||
true = Key1 > Key0,
|
true = Key1 > Key0,
|
||||||
case rocksdb:iterator_move(ITHandle, {seek, Key1}) of
|
case rocksdb:iterator_move(ITHandle, {seek, Key1}) of
|
||||||
{ok, Key, Val} ->
|
{ok, Key, Val} ->
|
||||||
{N, It, Acc} =
|
{N, It, Acc} = traverse_interval(
|
||||||
traverse_interval(ITHandle, Filter, Cutoff, Key, Val, It0, Acc0, N0),
|
ITHandle, KeyMapper, Filter, Cutoff, Key, Val, It0, Acc0, N0
|
||||||
|
),
|
||||||
next_loop(ITHandle, KeyMapper, Filter, Cutoff, It, Acc, N);
|
next_loop(ITHandle, KeyMapper, Filter, Cutoff, It, Acc, N);
|
||||||
{error, invalid_iterator} ->
|
{error, invalid_iterator} ->
|
||||||
{ok, It0, lists:reverse(Acc0)}
|
{ok, It0, lists:reverse(Acc0)}
|
||||||
end
|
end
|
||||||
end.
|
end.
|
||||||
|
|
||||||
traverse_interval(ITHandle, Filter, Cutoff, Key, Val, It0, Acc0, N) ->
|
traverse_interval(ITHandle, KeyMapper, Filter, Cutoff, Key, Val, It0, Acc0, N) ->
|
||||||
It = It0#{?last_seen_key := Key},
|
It = It0#{?last_seen_key := Key},
|
||||||
case emqx_ds_bitmask_keymapper:bin_checkmask(Filter, Key) of
|
Timestamp = emqx_ds_bitmask_keymapper:bin_key_to_coord(KeyMapper, Key, ?DIM_TS),
|
||||||
|
case
|
||||||
|
emqx_ds_bitmask_keymapper:bin_checkmask(Filter, Key) andalso
|
||||||
|
check_timestamp(Cutoff, It, Timestamp)
|
||||||
|
of
|
||||||
true ->
|
true ->
|
||||||
Msg = deserialize(Val),
|
Msg = deserialize(Val),
|
||||||
case check_message(Cutoff, It, Msg) of
|
case check_message(It, Msg) of
|
||||||
true ->
|
true ->
|
||||||
Acc = [{Key, Msg} | Acc0],
|
Acc = [{Key, Msg} | Acc0],
|
||||||
traverse_interval(ITHandle, Filter, Cutoff, It, Acc, N - 1);
|
traverse_interval(ITHandle, KeyMapper, Filter, Cutoff, It, Acc, N - 1);
|
||||||
false ->
|
false ->
|
||||||
traverse_interval(ITHandle, Filter, Cutoff, It, Acc0, N);
|
traverse_interval(ITHandle, KeyMapper, Filter, Cutoff, It, Acc0, N)
|
||||||
overflow ->
|
|
||||||
{0, It0, Acc0}
|
|
||||||
end;
|
end;
|
||||||
|
overflow ->
|
||||||
|
{0, It0, Acc0};
|
||||||
false ->
|
false ->
|
||||||
{N, It, Acc0}
|
{N, It, Acc0}
|
||||||
end.
|
end.
|
||||||
|
|
||||||
traverse_interval(_ITHandle, _Filter, _Cutoff, It, Acc, 0) ->
|
traverse_interval(_ITHandle, _KeyMapper, _Filter, _Cutoff, It, Acc, 0) ->
|
||||||
{0, It, Acc};
|
{0, It, Acc};
|
||||||
traverse_interval(ITHandle, Filter, Cutoff, It, Acc, N) ->
|
traverse_interval(ITHandle, KeyMapper, Filter, Cutoff, It, Acc, N) ->
|
||||||
inc_counter(),
|
inc_counter(),
|
||||||
case rocksdb:iterator_move(ITHandle, next) of
|
case rocksdb:iterator_move(ITHandle, next) of
|
||||||
{ok, Key, Val} ->
|
{ok, Key, Val} ->
|
||||||
traverse_interval(ITHandle, Filter, Cutoff, Key, Val, It, Acc, N);
|
traverse_interval(ITHandle, KeyMapper, Filter, Cutoff, Key, Val, It, Acc, N);
|
||||||
{error, invalid_iterator} ->
|
{error, invalid_iterator} ->
|
||||||
{0, It, Acc}
|
{0, It, Acc}
|
||||||
end.
|
end.
|
||||||
|
@ -564,6 +573,7 @@ delete_traverse_interval(LoopContext0) ->
|
||||||
storage_iter := It0,
|
storage_iter := It0,
|
||||||
current_key := Key,
|
current_key := Key,
|
||||||
current_val := Val,
|
current_val := Val,
|
||||||
|
keymapper := KeyMapper,
|
||||||
filter := Filter,
|
filter := Filter,
|
||||||
safe_cutoff_time := Cutoff,
|
safe_cutoff_time := Cutoff,
|
||||||
selector := Selector,
|
selector := Selector,
|
||||||
|
@ -574,10 +584,14 @@ delete_traverse_interval(LoopContext0) ->
|
||||||
remaining := Remaining0
|
remaining := Remaining0
|
||||||
} = LoopContext0,
|
} = LoopContext0,
|
||||||
It = It0#{?last_seen_key := Key},
|
It = It0#{?last_seen_key := Key},
|
||||||
case emqx_ds_bitmask_keymapper:bin_checkmask(Filter, Key) of
|
Timestamp = emqx_ds_bitmask_keymapper:bin_key_to_coord(KeyMapper, Key, ?DIM_TS),
|
||||||
|
case
|
||||||
|
emqx_ds_bitmask_keymapper:bin_checkmask(Filter, Key) andalso
|
||||||
|
check_timestamp(Cutoff, It, Timestamp)
|
||||||
|
of
|
||||||
true ->
|
true ->
|
||||||
Msg = deserialize(Val),
|
Msg = deserialize(Val),
|
||||||
case check_message(Cutoff, It, Msg) of
|
case check_message(It, Msg) of
|
||||||
true ->
|
true ->
|
||||||
case Selector(Msg) of
|
case Selector(Msg) of
|
||||||
true ->
|
true ->
|
||||||
|
@ -590,10 +604,10 @@ delete_traverse_interval(LoopContext0) ->
|
||||||
delete_traverse_interval1(LoopContext0#{remaining := Remaining0 - 1})
|
delete_traverse_interval1(LoopContext0#{remaining := Remaining0 - 1})
|
||||||
end;
|
end;
|
||||||
false ->
|
false ->
|
||||||
delete_traverse_interval1(LoopContext0);
|
delete_traverse_interval1(LoopContext0)
|
||||||
overflow ->
|
|
||||||
{0, It0, AccDel0, AccIter0}
|
|
||||||
end;
|
end;
|
||||||
|
overflow ->
|
||||||
|
{0, It0, AccDel0, AccIter0};
|
||||||
false ->
|
false ->
|
||||||
{Remaining0, It, AccDel0, AccIter0}
|
{Remaining0, It, AccDel0, AccIter0}
|
||||||
end.
|
end.
|
||||||
|
@ -621,39 +635,28 @@ delete_traverse_interval1(LoopContext0) ->
|
||||||
{0, It, AccDel, AccIter}
|
{0, It, AccDel, AccIter}
|
||||||
end.
|
end.
|
||||||
|
|
||||||
-spec check_message(emqx_ds:time(), iterator() | delete_iterator(), emqx_types:message()) ->
|
-spec check_timestamp(emqx_ds:time(), iterator() | delete_iterator(), emqx_ds:time()) ->
|
||||||
true | false | overflow.
|
true | false | overflow.
|
||||||
check_message(
|
check_timestamp(Cutoff, _It, Timestamp) when Timestamp >= Cutoff ->
|
||||||
Cutoff,
|
|
||||||
_It,
|
|
||||||
#message{timestamp = Timestamp}
|
|
||||||
) when Timestamp >= Cutoff ->
|
|
||||||
%% We hit the current epoch, we can't continue iterating over it yet.
|
%% 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
|
%% 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
|
%% 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.
|
%% order) due to the nature of keymapping, potentially causing us to miss them.
|
||||||
overflow;
|
overflow;
|
||||||
check_message(
|
check_timestamp(_Cutoff, #{?start_time := StartTime}, Timestamp) ->
|
||||||
_Cutoff,
|
Timestamp >= StartTime.
|
||||||
#{?tag := ?IT, ?start_time := StartTime, ?topic_filter := TopicFilter},
|
|
||||||
#message{timestamp = Timestamp, topic = Topic}
|
-spec check_message(iterator() | delete_iterator(), emqx_types:message()) ->
|
||||||
) when Timestamp >= StartTime ->
|
true | false.
|
||||||
emqx_topic:match(emqx_topic:tokens(Topic), TopicFilter);
|
check_message(#{?topic_filter := TopicFilter}, #message{topic = Topic}) ->
|
||||||
check_message(
|
emqx_topic:match(emqx_topic:tokens(Topic), TopicFilter).
|
||||||
_Cutoff,
|
|
||||||
#{?tag := ?DELETE_IT, ?start_time := StartTime, ?topic_filter := TopicFilter},
|
|
||||||
#message{timestamp = Timestamp, topic = Topic}
|
|
||||||
) when Timestamp >= StartTime ->
|
|
||||||
emqx_topic:match(emqx_topic:tokens(Topic), TopicFilter);
|
|
||||||
check_message(_Cutoff, _It, _Msg) ->
|
|
||||||
false.
|
|
||||||
|
|
||||||
format_key(KeyMapper, Key) ->
|
format_key(KeyMapper, Key) ->
|
||||||
Vec = [integer_to_list(I, 16) || I <- emqx_ds_bitmask_keymapper:key_to_vector(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, ",")])).
|
lists:flatten(io_lib:format("~.16B (~s)", [Key, string:join(Vec, ",")])).
|
||||||
|
|
||||||
-spec make_key(s(), emqx_types:message()) -> {binary(), [binary()]}.
|
-spec make_key(s(), emqx_ds:time(), emqx_types:message()) -> {binary(), [binary()]}.
|
||||||
make_key(#s{keymappers = KeyMappers, trie = Trie}, #message{timestamp = Timestamp, topic = TopicBin}) ->
|
make_key(#s{keymappers = KeyMappers, trie = Trie}, Timestamp, #message{topic = TopicBin}) ->
|
||||||
Tokens = emqx_topic:words(TopicBin),
|
Tokens = emqx_topic:words(TopicBin),
|
||||||
{TopicIndex, Varying} = emqx_ds_lts:topic_key(Trie, fun threshold_fun/1, Tokens),
|
{TopicIndex, Varying} = emqx_ds_lts:topic_key(Trie, fun threshold_fun/1, Tokens),
|
||||||
VaryingHashes = [hash_topic_level(I) || I <- Varying],
|
VaryingHashes = [hash_topic_level(I) || I <- Varying],
|
||||||
|
@ -666,11 +669,10 @@ make_key(#s{keymappers = KeyMappers, trie = Trie}, #message{timestamp = Timestam
|
||||||
]) ->
|
]) ->
|
||||||
binary().
|
binary().
|
||||||
make_key(KeyMapper, TopicIndex, Timestamp, Varying) ->
|
make_key(KeyMapper, TopicIndex, Timestamp, Varying) ->
|
||||||
UniqueInteger = erlang:unique_integer([monotonic, positive]),
|
|
||||||
emqx_ds_bitmask_keymapper:key_to_bitstring(
|
emqx_ds_bitmask_keymapper:key_to_bitstring(
|
||||||
KeyMapper,
|
KeyMapper,
|
||||||
emqx_ds_bitmask_keymapper:vector_to_key(KeyMapper, [
|
emqx_ds_bitmask_keymapper:vector_to_key(KeyMapper, [
|
||||||
TopicIndex, Timestamp, UniqueInteger | Varying
|
TopicIndex, Timestamp | Varying
|
||||||
])
|
])
|
||||||
).
|
).
|
||||||
|
|
||||||
|
@ -723,13 +725,12 @@ deserialize(Blob) ->
|
||||||
%% erlfmt-ignore
|
%% erlfmt-ignore
|
||||||
make_keymapper(TopicIndexBytes, BitsPerTopicLevel, TSBits, TSOffsetBits, N) ->
|
make_keymapper(TopicIndexBytes, BitsPerTopicLevel, TSBits, TSOffsetBits, N) ->
|
||||||
Bitsources =
|
Bitsources =
|
||||||
%% Dimension Offset Bitsize
|
%% Dimension Offset Bitsize
|
||||||
[{1, 0, TopicIndexBytes * ?BYTE_SIZE}, %% Topic index
|
[{?DIM_TOPIC, 0, TopicIndexBytes * ?BYTE_SIZE}, %% Topic index
|
||||||
{2, TSOffsetBits, TSBits - TSOffsetBits }] ++ %% Timestamp epoch
|
{?DIM_TS, TSOffsetBits, TSBits - TSOffsetBits }] ++ %% Timestamp epoch
|
||||||
[{3 + I, 0, BitsPerTopicLevel } %% Varying topic levels
|
[{?DIM_TS + I, 0, BitsPerTopicLevel } %% Varying topic levels
|
||||||
|| I <- lists:seq(1, N)] ++
|
|| I <- lists:seq(1, N)] ++
|
||||||
[{2, 0, TSOffsetBits }, %% Timestamp offset
|
[{?DIM_TS, 0, TSOffsetBits }], %% Timestamp offset
|
||||||
{3, 0, 64 }], %% Unique integer
|
|
||||||
Keymapper = emqx_ds_bitmask_keymapper:make_keymapper(lists:reverse(Bitsources)),
|
Keymapper = emqx_ds_bitmask_keymapper:make_keymapper(lists:reverse(Bitsources)),
|
||||||
%% Assert:
|
%% Assert:
|
||||||
case emqx_ds_bitmask_keymapper:bitsize(Keymapper) rem 8 of
|
case emqx_ds_bitmask_keymapper:bitsize(Keymapper) rem 8 of
|
||||||
|
|
|
@ -29,8 +29,8 @@
|
||||||
update_iterator/3,
|
update_iterator/3,
|
||||||
next/3,
|
next/3,
|
||||||
delete_next/4,
|
delete_next/4,
|
||||||
update_config/2,
|
update_config/3,
|
||||||
add_generation/1,
|
add_generation/2,
|
||||||
list_generations_with_lifetimes/1,
|
list_generations_with_lifetimes/1,
|
||||||
drop_generation/2
|
drop_generation/2
|
||||||
]).
|
]).
|
||||||
|
@ -133,7 +133,7 @@
|
||||||
cf_refs := cf_refs(),
|
cf_refs := cf_refs(),
|
||||||
%% Time at which this was created. Might differ from `since', in particular for the
|
%% Time at which this was created. Might differ from `since', in particular for the
|
||||||
%% first generation.
|
%% first generation.
|
||||||
created_at := emqx_ds:time(),
|
created_at := emqx_message:timestamp(),
|
||||||
%% 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.
|
||||||
|
@ -194,7 +194,12 @@
|
||||||
-callback drop(shard_id(), rocksdb:db_handle(), gen_id(), cf_refs(), _RuntimeData) ->
|
-callback drop(shard_id(), rocksdb:db_handle(), gen_id(), cf_refs(), _RuntimeData) ->
|
||||||
ok | {error, _Reason}.
|
ok | {error, _Reason}.
|
||||||
|
|
||||||
-callback store_batch(shard_id(), _Data, [emqx_types:message()], emqx_ds:message_store_opts()) ->
|
-callback store_batch(
|
||||||
|
shard_id(),
|
||||||
|
_Data,
|
||||||
|
[{emqx_ds:time(), emqx_types:message()}],
|
||||||
|
emqx_ds:message_store_opts()
|
||||||
|
) ->
|
||||||
emqx_ds:store_batch_result().
|
emqx_ds:store_batch_result().
|
||||||
|
|
||||||
-callback get_streams(shard_id(), _Data, emqx_ds:topic_filter(), emqx_ds:time()) ->
|
-callback get_streams(shard_id(), _Data, emqx_ds:topic_filter(), emqx_ds:time()) ->
|
||||||
|
@ -219,6 +224,9 @@
|
||||||
%% API for the replication layer
|
%% API for the replication layer
|
||||||
%%================================================================================
|
%%================================================================================
|
||||||
|
|
||||||
|
%% Note: we specify gen_server requests as records to make use of Dialyzer:
|
||||||
|
-record(call_add_generation, {since :: emqx_ds:time()}).
|
||||||
|
-record(call_update_config, {options :: emqx_ds:create_db_opts(), since :: emqx_ds:time()}).
|
||||||
-record(call_list_generations_with_lifetimes, {}).
|
-record(call_list_generations_with_lifetimes, {}).
|
||||||
-record(call_drop_generation, {gen_id :: gen_id()}).
|
-record(call_drop_generation, {gen_id :: gen_id()}).
|
||||||
|
|
||||||
|
@ -230,7 +238,11 @@ open_shard(Shard, Options) ->
|
||||||
drop_shard(Shard) ->
|
drop_shard(Shard) ->
|
||||||
ok = rocksdb:destroy(db_dir(Shard), []).
|
ok = rocksdb:destroy(db_dir(Shard), []).
|
||||||
|
|
||||||
-spec store_batch(shard_id(), [emqx_types:message()], emqx_ds:message_store_opts()) ->
|
-spec store_batch(
|
||||||
|
shard_id(),
|
||||||
|
[{emqx_ds:time(), emqx_types:message()}],
|
||||||
|
emqx_ds:message_store_opts()
|
||||||
|
) ->
|
||||||
emqx_ds:store_batch_result().
|
emqx_ds:store_batch_result().
|
||||||
store_batch(Shard, Messages, Options) ->
|
store_batch(Shard, Messages, Options) ->
|
||||||
%% We always store messages in the current generation:
|
%% We always store messages in the current generation:
|
||||||
|
@ -398,13 +410,16 @@ delete_next(
|
||||||
{ok, end_of_stream}
|
{ok, end_of_stream}
|
||||||
end.
|
end.
|
||||||
|
|
||||||
-spec update_config(shard_id(), emqx_ds:create_db_opts()) -> ok.
|
-spec update_config(shard_id(), emqx_ds:time(), emqx_ds:create_db_opts()) ->
|
||||||
update_config(ShardId, Options) ->
|
ok | {error, overlaps_existing_generations}.
|
||||||
gen_server:call(?REF(ShardId), {?FUNCTION_NAME, Options}, infinity).
|
update_config(ShardId, Since, Options) ->
|
||||||
|
Call = #call_update_config{since = Since, options = Options},
|
||||||
|
gen_server:call(?REF(ShardId), Call, infinity).
|
||||||
|
|
||||||
-spec add_generation(shard_id()) -> ok.
|
-spec add_generation(shard_id(), emqx_ds:time()) ->
|
||||||
add_generation(ShardId) ->
|
ok | {error, overlaps_existing_generations}.
|
||||||
gen_server:call(?REF(ShardId), add_generation, infinity).
|
add_generation(ShardId, Since) ->
|
||||||
|
gen_server:call(?REF(ShardId), #call_add_generation{since = Since}, infinity).
|
||||||
|
|
||||||
-spec list_generations_with_lifetimes(shard_id()) ->
|
-spec list_generations_with_lifetimes(shard_id()) ->
|
||||||
#{
|
#{
|
||||||
|
@ -438,9 +453,6 @@ start_link(Shard = {_, _}, Options) ->
|
||||||
shard :: shard()
|
shard :: shard()
|
||||||
}).
|
}).
|
||||||
|
|
||||||
%% Note: we specify gen_server requests as records to make use of Dialyzer:
|
|
||||||
-record(call_create_generation, {since :: emqx_ds:time()}).
|
|
||||||
|
|
||||||
-type server_state() :: #s{}.
|
-type server_state() :: #s{}.
|
||||||
|
|
||||||
-define(DEFAULT_CF, "default").
|
-define(DEFAULT_CF, "default").
|
||||||
|
@ -470,18 +482,22 @@ init({ShardId, Options}) ->
|
||||||
commit_metadata(S),
|
commit_metadata(S),
|
||||||
{ok, S}.
|
{ok, S}.
|
||||||
|
|
||||||
handle_call({update_config, Options}, _From, #s{schema = Schema} = S0) ->
|
handle_call(#call_update_config{since = Since, options = Options}, _From, S0) ->
|
||||||
Prototype = maps:get(storage, Options),
|
case handle_update_config(S0, Since, Options) of
|
||||||
S1 = S0#s{schema = Schema#{prototype := Prototype}},
|
S = #s{} ->
|
||||||
Since = emqx_message:timestamp_now(),
|
commit_metadata(S),
|
||||||
S = add_generation(S1, Since),
|
{reply, ok, S};
|
||||||
commit_metadata(S),
|
Error = {error, _} ->
|
||||||
{reply, ok, S};
|
{reply, Error, S0}
|
||||||
handle_call(add_generation, _From, S0) ->
|
end;
|
||||||
Since = emqx_message:timestamp_now(),
|
handle_call(#call_add_generation{since = Since}, _From, S0) ->
|
||||||
S = add_generation(S0, Since),
|
case handle_add_generation(S0, Since) of
|
||||||
commit_metadata(S),
|
S = #s{} ->
|
||||||
{reply, ok, S};
|
commit_metadata(S),
|
||||||
|
{reply, ok, S};
|
||||||
|
Error = {error, _} ->
|
||||||
|
{reply, Error, S0}
|
||||||
|
end;
|
||||||
handle_call(#call_list_generations_with_lifetimes{}, _From, S) ->
|
handle_call(#call_list_generations_with_lifetimes{}, _From, S) ->
|
||||||
Generations = handle_list_generations_with_lifetimes(S),
|
Generations = handle_list_generations_with_lifetimes(S),
|
||||||
{reply, Generations, S};
|
{reply, Generations, S};
|
||||||
|
@ -489,10 +505,6 @@ handle_call(#call_drop_generation{gen_id = GenId}, _From, S0) ->
|
||||||
{Reply, S} = handle_drop_generation(S0, GenId),
|
{Reply, S} = handle_drop_generation(S0, GenId),
|
||||||
commit_metadata(S),
|
commit_metadata(S),
|
||||||
{reply, Reply, S};
|
{reply, Reply, S};
|
||||||
handle_call(#call_create_generation{since = Since}, _From, S0) ->
|
|
||||||
S = add_generation(S0, Since),
|
|
||||||
commit_metadata(S),
|
|
||||||
{reply, ok, S};
|
|
||||||
handle_call(_Call, _From, S) ->
|
handle_call(_Call, _From, S) ->
|
||||||
{reply, {error, unknown_call}, S}.
|
{reply, {error, unknown_call}, S}.
|
||||||
|
|
||||||
|
@ -528,11 +540,10 @@ open_shard(ShardId, DB, CFRefs, ShardSchema) ->
|
||||||
ShardSchema
|
ShardSchema
|
||||||
).
|
).
|
||||||
|
|
||||||
-spec add_generation(server_state(), emqx_ds:time()) -> server_state().
|
-spec handle_add_generation(server_state(), emqx_ds:time()) ->
|
||||||
add_generation(S0, Since) ->
|
server_state() | {error, overlaps_existing_generations}.
|
||||||
|
handle_add_generation(S0, Since) ->
|
||||||
#s{shard_id = ShardId, db = DB, schema = Schema0, shard = Shard0, cf_refs = CFRefs0} = S0,
|
#s{shard_id = ShardId, db = DB, schema = Schema0, shard = Shard0, cf_refs = CFRefs0} = S0,
|
||||||
Schema1 = update_last_until(Schema0, Since),
|
|
||||||
Shard1 = update_last_until(Shard0, Since),
|
|
||||||
|
|
||||||
#{current_generation := OldGenId, prototype := {CurrentMod, _ModConf}} = Schema0,
|
#{current_generation := OldGenId, prototype := {CurrentMod, _ModConf}} = Schema0,
|
||||||
OldKey = ?GEN_KEY(OldGenId),
|
OldKey = ?GEN_KEY(OldGenId),
|
||||||
|
@ -540,39 +551,53 @@ add_generation(S0, Since) ->
|
||||||
#{cf_refs := OldCFRefs} = OldGenSchema,
|
#{cf_refs := OldCFRefs} = OldGenSchema,
|
||||||
#{OldKey := #{module := OldMod, data := OldGenData}} = Shard0,
|
#{OldKey := #{module := OldMod, data := OldGenData}} = Shard0,
|
||||||
|
|
||||||
{GenId, Schema, NewCFRefs} = new_generation(ShardId, DB, Schema1, Since),
|
Schema1 = update_last_until(Schema0, Since),
|
||||||
|
Shard1 = update_last_until(Shard0, Since),
|
||||||
|
|
||||||
CFRefs = NewCFRefs ++ CFRefs0,
|
case Schema1 of
|
||||||
Key = ?GEN_KEY(GenId),
|
_Updated = #{} ->
|
||||||
Generation0 =
|
{GenId, Schema, NewCFRefs} = new_generation(ShardId, DB, Schema1, Since),
|
||||||
#{data := NewGenData0} =
|
CFRefs = NewCFRefs ++ CFRefs0,
|
||||||
open_generation(ShardId, DB, CFRefs, GenId, maps:get(Key, Schema)),
|
Key = ?GEN_KEY(GenId),
|
||||||
|
Generation0 =
|
||||||
|
#{data := NewGenData0} =
|
||||||
|
open_generation(ShardId, DB, CFRefs, GenId, maps:get(Key, Schema)),
|
||||||
|
%% When the new generation's module is the same as the last one, we might want to
|
||||||
|
%% perform actions like inheriting some of the previous (meta)data.
|
||||||
|
NewGenData =
|
||||||
|
run_post_creation_actions(
|
||||||
|
#{
|
||||||
|
shard_id => ShardId,
|
||||||
|
db => DB,
|
||||||
|
new_gen_id => GenId,
|
||||||
|
old_gen_id => OldGenId,
|
||||||
|
new_cf_refs => NewCFRefs,
|
||||||
|
old_cf_refs => OldCFRefs,
|
||||||
|
new_gen_runtime_data => NewGenData0,
|
||||||
|
old_gen_runtime_data => OldGenData,
|
||||||
|
new_module => CurrentMod,
|
||||||
|
old_module => OldMod
|
||||||
|
}
|
||||||
|
),
|
||||||
|
Generation = Generation0#{data := NewGenData},
|
||||||
|
Shard = Shard1#{current_generation := GenId, Key => Generation},
|
||||||
|
S0#s{
|
||||||
|
cf_refs = CFRefs,
|
||||||
|
schema = Schema,
|
||||||
|
shard = Shard
|
||||||
|
};
|
||||||
|
{error, exists} ->
|
||||||
|
S0;
|
||||||
|
{error, Reason} ->
|
||||||
|
{error, Reason}
|
||||||
|
end.
|
||||||
|
|
||||||
%% When the new generation's module is the same as the last one, we might want to
|
-spec handle_update_config(server_state(), emqx_ds:time(), emqx_ds:create_db_opts()) ->
|
||||||
%% perform actions like inheriting some of the previous (meta)data.
|
server_state() | {error, overlaps_existing_generations}.
|
||||||
NewGenData =
|
handle_update_config(S0 = #s{schema = Schema}, Since, Options) ->
|
||||||
run_post_creation_actions(
|
Prototype = maps:get(storage, Options),
|
||||||
#{
|
S = S0#s{schema = Schema#{prototype := Prototype}},
|
||||||
shard_id => ShardId,
|
handle_add_generation(S, Since).
|
||||||
db => DB,
|
|
||||||
new_gen_id => GenId,
|
|
||||||
old_gen_id => OldGenId,
|
|
||||||
new_cf_refs => NewCFRefs,
|
|
||||||
old_cf_refs => OldCFRefs,
|
|
||||||
new_gen_runtime_data => NewGenData0,
|
|
||||||
old_gen_runtime_data => OldGenData,
|
|
||||||
new_module => CurrentMod,
|
|
||||||
old_module => OldMod
|
|
||||||
}
|
|
||||||
),
|
|
||||||
Generation = Generation0#{data := NewGenData},
|
|
||||||
|
|
||||||
Shard = Shard1#{current_generation := GenId, Key => Generation},
|
|
||||||
S0#s{
|
|
||||||
cf_refs = CFRefs,
|
|
||||||
schema = Schema,
|
|
||||||
shard = Shard
|
|
||||||
}.
|
|
||||||
|
|
||||||
-spec handle_list_generations_with_lifetimes(server_state()) -> #{gen_id() => map()}.
|
-spec handle_list_generations_with_lifetimes(server_state()) -> #{gen_id() => map()}.
|
||||||
handle_list_generations_with_lifetimes(#s{schema = ShardSchema}) ->
|
handle_list_generations_with_lifetimes(#s{schema = ShardSchema}) ->
|
||||||
|
@ -652,7 +677,7 @@ new_generation(ShardId, DB, Schema0, Since) ->
|
||||||
module => Mod,
|
module => Mod,
|
||||||
data => GenData,
|
data => GenData,
|
||||||
cf_refs => NewCFRefs,
|
cf_refs => NewCFRefs,
|
||||||
created_at => emqx_message:timestamp_now(),
|
created_at => erlang:system_time(millisecond),
|
||||||
since => Since,
|
since => Since,
|
||||||
until => undefined
|
until => undefined
|
||||||
},
|
},
|
||||||
|
@ -703,12 +728,19 @@ rocksdb_open(Shard, Options) ->
|
||||||
db_dir({DB, ShardId}) ->
|
db_dir({DB, ShardId}) ->
|
||||||
filename:join([emqx_ds:base_dir(), atom_to_list(DB), binary_to_list(ShardId)]).
|
filename:join([emqx_ds:base_dir(), atom_to_list(DB), binary_to_list(ShardId)]).
|
||||||
|
|
||||||
-spec update_last_until(Schema, emqx_ds:time()) -> Schema when Schema :: shard_schema() | shard().
|
-spec update_last_until(Schema, emqx_ds:time()) ->
|
||||||
update_last_until(Schema, Until) ->
|
Schema | {error, exists | overlaps_existing_generations}
|
||||||
#{current_generation := GenId} = Schema,
|
when
|
||||||
GenData0 = maps:get(?GEN_KEY(GenId), Schema),
|
Schema :: shard_schema() | shard().
|
||||||
GenData = GenData0#{until := Until},
|
update_last_until(Schema = #{current_generation := GenId}, Until) ->
|
||||||
Schema#{?GEN_KEY(GenId) := GenData}.
|
case maps:get(?GEN_KEY(GenId), Schema) of
|
||||||
|
GenData = #{since := CurrentSince} when CurrentSince < Until ->
|
||||||
|
Schema#{?GEN_KEY(GenId) := GenData#{until := Until}};
|
||||||
|
#{since := Until} ->
|
||||||
|
{error, exists};
|
||||||
|
#{since := CurrentSince} when CurrentSince > Until ->
|
||||||
|
{error, overlaps_existing_generations}
|
||||||
|
end.
|
||||||
|
|
||||||
run_post_creation_actions(
|
run_post_creation_actions(
|
||||||
#{
|
#{
|
||||||
|
|
|
@ -117,9 +117,8 @@ store_batch(_ShardId, #s{db = DB, cf = CF}, Messages, _Options = #{atomic := tru
|
||||||
Res;
|
Res;
|
||||||
store_batch(_ShardId, #s{db = DB, cf = CF}, Messages, _Options) ->
|
store_batch(_ShardId, #s{db = DB, cf = CF}, Messages, _Options) ->
|
||||||
lists:foreach(
|
lists:foreach(
|
||||||
fun(Msg) ->
|
fun({Timestamp, Msg}) ->
|
||||||
Id = erlang:unique_integer([monotonic]),
|
Key = <<Timestamp:64>>,
|
||||||
Key = <<Id:64>>,
|
|
||||||
Val = term_to_binary(Msg),
|
Val = term_to_binary(Msg),
|
||||||
rocksdb:put(DB, CF, Key, Val, [])
|
rocksdb:put(DB, CF, Key, Val, [])
|
||||||
end,
|
end,
|
||||||
|
@ -210,8 +209,8 @@ do_next(_, _, _, _, 0, Key, Acc) ->
|
||||||
{Key, Acc};
|
{Key, Acc};
|
||||||
do_next(TopicFilter, StartTime, IT, Action, NLeft, Key0, Acc) ->
|
do_next(TopicFilter, StartTime, IT, Action, NLeft, Key0, Acc) ->
|
||||||
case rocksdb:iterator_move(IT, Action) of
|
case rocksdb:iterator_move(IT, Action) of
|
||||||
{ok, Key, Blob} ->
|
{ok, Key = <<TS:64>>, Blob} ->
|
||||||
Msg = #message{topic = Topic, timestamp = TS} = binary_to_term(Blob),
|
Msg = #message{topic = Topic} = binary_to_term(Blob),
|
||||||
TopicWords = emqx_topic:words(Topic),
|
TopicWords = emqx_topic:words(Topic),
|
||||||
case emqx_topic:match(TopicWords, TopicFilter) andalso TS >= StartTime of
|
case emqx_topic:match(TopicWords, TopicFilter) andalso TS >= StartTime of
|
||||||
true ->
|
true ->
|
||||||
|
|
|
@ -5,7 +5,7 @@
|
||||||
{vsn, "0.1.12"},
|
{vsn, "0.1.12"},
|
||||||
{modules, []},
|
{modules, []},
|
||||||
{registered, []},
|
{registered, []},
|
||||||
{applications, [kernel, stdlib, rocksdb, gproc, mria, emqx_utils]},
|
{applications, [kernel, stdlib, rocksdb, gproc, mria, ra, emqx_utils]},
|
||||||
{mod, {emqx_ds_app, []}},
|
{mod, {emqx_ds_app, []}},
|
||||||
{env, []}
|
{env, []}
|
||||||
]}.
|
]}.
|
||||||
|
|
|
@ -31,7 +31,9 @@ opts() ->
|
||||||
backend => builtin,
|
backend => builtin,
|
||||||
storage => {emqx_ds_storage_reference, #{}},
|
storage => {emqx_ds_storage_reference, #{}},
|
||||||
n_shards => ?N_SHARDS,
|
n_shards => ?N_SHARDS,
|
||||||
replication_factor => 3
|
n_sites => 1,
|
||||||
|
replication_factor => 3,
|
||||||
|
replication_options => #{}
|
||||||
}.
|
}.
|
||||||
|
|
||||||
%% A simple smoke test that verifies that opening/closing the DB
|
%% A simple smoke test that verifies that opening/closing the DB
|
||||||
|
@ -51,13 +53,8 @@ t_00_smoke_open_drop(_Config) ->
|
||||||
lists:foreach(
|
lists:foreach(
|
||||||
fun(Shard) ->
|
fun(Shard) ->
|
||||||
?assertEqual(
|
?assertEqual(
|
||||||
{ok, []}, emqx_ds_replication_layer_meta:replica_set(DB, Shard)
|
{ok, [Site]}, emqx_ds_replication_layer_meta:replica_set(DB, Shard)
|
||||||
),
|
)
|
||||||
?assertEqual(
|
|
||||||
[Site], emqx_ds_replication_layer_meta:in_sync_replicas(DB, Shard)
|
|
||||||
),
|
|
||||||
%% Check that the leader is eleected;
|
|
||||||
?assertEqual({ok, node()}, emqx_ds_replication_layer_meta:shard_leader(DB, Shard))
|
|
||||||
end,
|
end,
|
||||||
Shards
|
Shards
|
||||||
),
|
),
|
||||||
|
|
|
@ -29,7 +29,9 @@
|
||||||
backend => builtin,
|
backend => builtin,
|
||||||
storage => {emqx_ds_storage_bitfield_lts, #{}},
|
storage => {emqx_ds_storage_bitfield_lts, #{}},
|
||||||
n_shards => 1,
|
n_shards => 1,
|
||||||
replication_factor => 1
|
n_sites => 1,
|
||||||
|
replication_factor => 1,
|
||||||
|
replication_options => #{}
|
||||||
}).
|
}).
|
||||||
|
|
||||||
-define(COMPACT_CONFIG, #{
|
-define(COMPACT_CONFIG, #{
|
||||||
|
@ -54,7 +56,7 @@ t_store(_Config) ->
|
||||||
payload = Payload,
|
payload = Payload,
|
||||||
timestamp = PublishedAt
|
timestamp = PublishedAt
|
||||||
},
|
},
|
||||||
?assertMatch(ok, emqx_ds_storage_layer:store_batch(?SHARD, [Msg], #{})).
|
?assertMatch(ok, emqx_ds_storage_layer:store_batch(?SHARD, [{PublishedAt, Msg}], #{})).
|
||||||
|
|
||||||
%% Smoke test for iteration through a concrete topic
|
%% Smoke test for iteration through a concrete topic
|
||||||
t_iterate(_Config) ->
|
t_iterate(_Config) ->
|
||||||
|
@ -62,7 +64,7 @@ t_iterate(_Config) ->
|
||||||
Topics = [<<"foo/bar">>, <<"foo/bar/baz">>, <<"a">>],
|
Topics = [<<"foo/bar">>, <<"foo/bar/baz">>, <<"a">>],
|
||||||
Timestamps = lists:seq(1, 10),
|
Timestamps = lists:seq(1, 10),
|
||||||
Batch = [
|
Batch = [
|
||||||
make_message(PublishedAt, Topic, integer_to_binary(PublishedAt))
|
{PublishedAt, make_message(PublishedAt, Topic, integer_to_binary(PublishedAt))}
|
||||||
|| Topic <- Topics, PublishedAt <- Timestamps
|
|| Topic <- Topics, PublishedAt <- Timestamps
|
||||||
],
|
],
|
||||||
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch, []),
|
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch, []),
|
||||||
|
@ -90,7 +92,7 @@ t_delete(_Config) ->
|
||||||
Topics = [<<"foo/bar">>, TopicToDelete, <<"a">>],
|
Topics = [<<"foo/bar">>, TopicToDelete, <<"a">>],
|
||||||
Timestamps = lists:seq(1, 10),
|
Timestamps = lists:seq(1, 10),
|
||||||
Batch = [
|
Batch = [
|
||||||
make_message(PublishedAt, Topic, integer_to_binary(PublishedAt))
|
{PublishedAt, make_message(PublishedAt, Topic, integer_to_binary(PublishedAt))}
|
||||||
|| Topic <- Topics, PublishedAt <- Timestamps
|
|| Topic <- Topics, PublishedAt <- Timestamps
|
||||||
],
|
],
|
||||||
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch, []),
|
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch, []),
|
||||||
|
@ -121,7 +123,7 @@ t_get_streams(_Config) ->
|
||||||
Topics = [<<"foo/bar">>, <<"foo/bar/baz">>, <<"a">>],
|
Topics = [<<"foo/bar">>, <<"foo/bar/baz">>, <<"a">>],
|
||||||
Timestamps = lists:seq(1, 10),
|
Timestamps = lists:seq(1, 10),
|
||||||
Batch = [
|
Batch = [
|
||||||
make_message(PublishedAt, Topic, integer_to_binary(PublishedAt))
|
{PublishedAt, make_message(PublishedAt, Topic, integer_to_binary(PublishedAt))}
|
||||||
|| Topic <- Topics, PublishedAt <- Timestamps
|
|| Topic <- Topics, PublishedAt <- Timestamps
|
||||||
],
|
],
|
||||||
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch, []),
|
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch, []),
|
||||||
|
@ -147,7 +149,7 @@ t_get_streams(_Config) ->
|
||||||
NewBatch = [
|
NewBatch = [
|
||||||
begin
|
begin
|
||||||
B = integer_to_binary(I),
|
B = integer_to_binary(I),
|
||||||
make_message(100, <<"foo/bar/", B/binary>>, <<"filler", B/binary>>)
|
{100, make_message(100, <<"foo/bar/", B/binary>>, <<"filler", B/binary>>)}
|
||||||
end
|
end
|
||||||
|| I <- lists:seq(1, 200)
|
|| I <- lists:seq(1, 200)
|
||||||
],
|
],
|
||||||
|
@ -176,12 +178,8 @@ t_new_generation_inherit_trie(_Config) ->
|
||||||
Timestamps = lists:seq(1, 10_000, 100),
|
Timestamps = lists:seq(1, 10_000, 100),
|
||||||
Batch = [
|
Batch = [
|
||||||
begin
|
begin
|
||||||
B = integer_to_binary(I),
|
Topic = emqx_topic:join(["wildcard", integer_to_binary(I), "suffix", Suffix]),
|
||||||
make_message(
|
{TS, make_message(TS, Topic, integer_to_binary(TS))}
|
||||||
TS,
|
|
||||||
<<"wildcard/", B/binary, "/suffix/", Suffix/binary>>,
|
|
||||||
integer_to_binary(TS)
|
|
||||||
)
|
|
||||||
end
|
end
|
||||||
|| I <- lists:seq(1, 200),
|
|| I <- lists:seq(1, 200),
|
||||||
TS <- Timestamps,
|
TS <- Timestamps,
|
||||||
|
@ -190,7 +188,7 @@ t_new_generation_inherit_trie(_Config) ->
|
||||||
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch, []),
|
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch, []),
|
||||||
%% Now we create a new generation with the same LTS module. It should inherit the
|
%% Now we create a new generation with the same LTS module. It should inherit the
|
||||||
%% learned trie.
|
%% learned trie.
|
||||||
ok = emqx_ds_storage_layer:add_generation(?SHARD),
|
ok = emqx_ds_storage_layer:add_generation(?SHARD, _Since = 1000),
|
||||||
ok
|
ok
|
||||||
end,
|
end,
|
||||||
fun(Trace) ->
|
fun(Trace) ->
|
||||||
|
@ -205,23 +203,21 @@ t_replay(_Config) ->
|
||||||
Topics = [<<"foo/bar">>, <<"foo/bar/baz">>],
|
Topics = [<<"foo/bar">>, <<"foo/bar/baz">>],
|
||||||
Timestamps = lists:seq(1, 10_000, 100),
|
Timestamps = lists:seq(1, 10_000, 100),
|
||||||
Batch1 = [
|
Batch1 = [
|
||||||
make_message(PublishedAt, Topic, integer_to_binary(PublishedAt))
|
{PublishedAt, make_message(PublishedAt, Topic, integer_to_binary(PublishedAt))}
|
||||||
|| Topic <- Topics, PublishedAt <- Timestamps
|
|| Topic <- Topics, PublishedAt <- Timestamps
|
||||||
],
|
],
|
||||||
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch1, []),
|
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch1, []),
|
||||||
%% Create wildcard topics `wildcard/+/suffix/foo' and `wildcard/+/suffix/bar':
|
%% Create wildcard topics `wildcard/+/suffix/foo' and `wildcard/+/suffix/bar':
|
||||||
Batch2 = [
|
Batch2 = [
|
||||||
begin
|
begin
|
||||||
B = integer_to_binary(I),
|
Topic = emqx_topic:join(["wildcard", integer_to_list(I), "suffix", Suffix]),
|
||||||
make_message(
|
{TS, make_message(TS, Topic, integer_to_binary(TS))}
|
||||||
TS, <<"wildcard/", B/binary, "/suffix/", Suffix/binary>>, integer_to_binary(TS)
|
|
||||||
)
|
|
||||||
end
|
end
|
||||||
|| I <- lists:seq(1, 200), TS <- Timestamps, Suffix <- [<<"foo">>, <<"bar">>]
|
|| I <- lists:seq(1, 200), TS <- Timestamps, Suffix <- [<<"foo">>, <<"bar">>]
|
||||||
],
|
],
|
||||||
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch2, []),
|
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch2, []),
|
||||||
%% Check various topic filters:
|
%% Check various topic filters:
|
||||||
Messages = Batch1 ++ Batch2,
|
Messages = [M || {_TS, M} <- Batch1 ++ Batch2],
|
||||||
%% Missing topics (no ghost messages):
|
%% Missing topics (no ghost messages):
|
||||||
?assertNot(check(?SHARD, <<"missing/foo/bar">>, 0, Messages)),
|
?assertNot(check(?SHARD, <<"missing/foo/bar">>, 0, Messages)),
|
||||||
%% Regular topics:
|
%% Regular topics:
|
||||||
|
@ -479,18 +475,6 @@ make_message(PublishedAt, Topic, Payload) when is_binary(Topic) ->
|
||||||
payload = Payload
|
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) ->
|
payloads(Messages) ->
|
||||||
lists:map(
|
lists:map(
|
||||||
fun(#message{payload = P}) ->
|
fun(#message{payload = P}) ->
|
||||||
|
|
|
@ -31,7 +31,7 @@
|
||||||
-endif.
|
-endif.
|
||||||
|
|
||||||
%% These apps are always (re)started by emqx_machine:
|
%% These apps are always (re)started by emqx_machine:
|
||||||
-define(BASIC_REBOOT_APPS, [gproc, esockd, ranch, cowboy, emqx]).
|
-define(BASIC_REBOOT_APPS, [gproc, esockd, ranch, cowboy, emqx_durable_storage, emqx]).
|
||||||
|
|
||||||
%% If any of these applications crash, the entire EMQX node shuts down:
|
%% If any of these applications crash, the entire EMQX node shuts down:
|
||||||
-define(BASIC_PERMANENT_APPS, [mria, ekka, esockd, emqx]).
|
-define(BASIC_PERMANENT_APPS, [mria, ekka, esockd, emqx]).
|
||||||
|
|
|
@ -22,7 +22,9 @@
|
||||||
list/1,
|
list/1,
|
||||||
mqueue/1,
|
mqueue/1,
|
||||||
map/2,
|
map/2,
|
||||||
chain/2
|
transpose/1,
|
||||||
|
chain/2,
|
||||||
|
repeat/1
|
||||||
]).
|
]).
|
||||||
|
|
||||||
%% Evaluating
|
%% Evaluating
|
||||||
|
@ -91,6 +93,31 @@ map(F, S) ->
|
||||||
end
|
end
|
||||||
end.
|
end.
|
||||||
|
|
||||||
|
%% @doc Transpose a list of streams into a stream producing lists of their respective values.
|
||||||
|
%% The resulting stream is as long as the shortest of the input streams.
|
||||||
|
-spec transpose([stream(X)]) -> stream([X]).
|
||||||
|
transpose([S]) ->
|
||||||
|
map(fun(X) -> [X] end, S);
|
||||||
|
transpose([S | Streams]) ->
|
||||||
|
transpose_tail(S, transpose(Streams));
|
||||||
|
transpose([]) ->
|
||||||
|
empty().
|
||||||
|
|
||||||
|
transpose_tail(S, Tail) ->
|
||||||
|
fun() ->
|
||||||
|
case next(S) of
|
||||||
|
[X | SRest] ->
|
||||||
|
case next(Tail) of
|
||||||
|
[Xs | TailRest] ->
|
||||||
|
[[X | Xs] | transpose_tail(SRest, TailRest)];
|
||||||
|
[] ->
|
||||||
|
[]
|
||||||
|
end;
|
||||||
|
[] ->
|
||||||
|
[]
|
||||||
|
end
|
||||||
|
end.
|
||||||
|
|
||||||
%% @doc Make a stream by chaining (concatenating) two streams.
|
%% @doc Make a stream by chaining (concatenating) two streams.
|
||||||
%% The second stream begins to produce values only after the first one is exhausted.
|
%% The second stream begins to produce values only after the first one is exhausted.
|
||||||
-spec chain(stream(X), stream(Y)) -> stream(X | Y).
|
-spec chain(stream(X), stream(Y)) -> stream(X | Y).
|
||||||
|
@ -104,6 +131,19 @@ chain(SFirst, SThen) ->
|
||||||
end
|
end
|
||||||
end.
|
end.
|
||||||
|
|
||||||
|
%% @doc Make an infinite stream out of repeats of given stream.
|
||||||
|
%% If the given stream is empty, the resulting stream is also empty.
|
||||||
|
-spec repeat(stream(X)) -> stream(X).
|
||||||
|
repeat(S) ->
|
||||||
|
fun() ->
|
||||||
|
case next(S) of
|
||||||
|
[X | SRest] ->
|
||||||
|
[X | chain(SRest, repeat(S))];
|
||||||
|
[] ->
|
||||||
|
[]
|
||||||
|
end
|
||||||
|
end.
|
||||||
|
|
||||||
%%
|
%%
|
||||||
|
|
||||||
%% @doc Produce the next value from the stream.
|
%% @doc Produce the next value from the stream.
|
||||||
|
|
|
@ -74,6 +74,80 @@ chain_list_map_test() ->
|
||||||
emqx_utils_stream:consume(S)
|
emqx_utils_stream:consume(S)
|
||||||
).
|
).
|
||||||
|
|
||||||
|
transpose_test() ->
|
||||||
|
S = emqx_utils_stream:transpose([
|
||||||
|
emqx_utils_stream:list([1, 2, 3]),
|
||||||
|
emqx_utils_stream:list([4, 5, 6, 7])
|
||||||
|
]),
|
||||||
|
?assertEqual(
|
||||||
|
[[1, 4], [2, 5], [3, 6]],
|
||||||
|
emqx_utils_stream:consume(S)
|
||||||
|
).
|
||||||
|
|
||||||
|
transpose_none_test() ->
|
||||||
|
?assertEqual(
|
||||||
|
[],
|
||||||
|
emqx_utils_stream:consume(emqx_utils_stream:transpose([]))
|
||||||
|
).
|
||||||
|
|
||||||
|
transpose_one_test() ->
|
||||||
|
S = emqx_utils_stream:transpose([emqx_utils_stream:list([1, 2, 3])]),
|
||||||
|
?assertEqual(
|
||||||
|
[[1], [2], [3]],
|
||||||
|
emqx_utils_stream:consume(S)
|
||||||
|
).
|
||||||
|
|
||||||
|
transpose_many_test() ->
|
||||||
|
S = emqx_utils_stream:transpose([
|
||||||
|
emqx_utils_stream:list([1, 2, 3]),
|
||||||
|
emqx_utils_stream:list([4, 5, 6, 7]),
|
||||||
|
emqx_utils_stream:list([8, 9])
|
||||||
|
]),
|
||||||
|
?assertEqual(
|
||||||
|
[[1, 4, 8], [2, 5, 9]],
|
||||||
|
emqx_utils_stream:consume(S)
|
||||||
|
).
|
||||||
|
|
||||||
|
transpose_many_empty_test() ->
|
||||||
|
S = emqx_utils_stream:transpose([
|
||||||
|
emqx_utils_stream:list([1, 2, 3]),
|
||||||
|
emqx_utils_stream:list([4, 5, 6, 7]),
|
||||||
|
emqx_utils_stream:empty()
|
||||||
|
]),
|
||||||
|
?assertEqual(
|
||||||
|
[],
|
||||||
|
emqx_utils_stream:consume(S)
|
||||||
|
).
|
||||||
|
|
||||||
|
repeat_test() ->
|
||||||
|
S = emqx_utils_stream:repeat(emqx_utils_stream:list([1, 2, 3])),
|
||||||
|
?assertMatch(
|
||||||
|
{[1, 2, 3, 1, 2, 3, 1, 2], _},
|
||||||
|
emqx_utils_stream:consume(8, S)
|
||||||
|
),
|
||||||
|
{_, SRest} = emqx_utils_stream:consume(8, S),
|
||||||
|
?assertMatch(
|
||||||
|
{[3, 1, 2, 3, 1, 2, 3, 1], _},
|
||||||
|
emqx_utils_stream:consume(8, SRest)
|
||||||
|
).
|
||||||
|
|
||||||
|
repeat_empty_test() ->
|
||||||
|
S = emqx_utils_stream:repeat(emqx_utils_stream:list([])),
|
||||||
|
?assertEqual(
|
||||||
|
[],
|
||||||
|
emqx_utils_stream:consume(8, S)
|
||||||
|
).
|
||||||
|
|
||||||
|
transpose_repeat_test() ->
|
||||||
|
S = emqx_utils_stream:transpose([
|
||||||
|
emqx_utils_stream:repeat(emqx_utils_stream:list([1, 2])),
|
||||||
|
emqx_utils_stream:list([4, 5, 6, 7, 8])
|
||||||
|
]),
|
||||||
|
?assertEqual(
|
||||||
|
[[1, 4], [2, 5], [1, 6], [2, 7], [1, 8]],
|
||||||
|
emqx_utils_stream:consume(S)
|
||||||
|
).
|
||||||
|
|
||||||
mqueue_test() ->
|
mqueue_test() ->
|
||||||
_ = erlang:send_after(1, self(), 1),
|
_ = erlang:send_after(1, self(), 1),
|
||||||
_ = erlang:send_after(100, self(), 2),
|
_ = erlang:send_after(100, self(), 2),
|
||||||
|
|
3
mix.exs
3
mix.exs
|
@ -100,7 +100,8 @@ defmodule EMQXUmbrella.MixProject do
|
||||||
{:rfc3339, github: "emqx/rfc3339", tag: "0.2.3", override: true},
|
{:rfc3339, github: "emqx/rfc3339", tag: "0.2.3", override: true},
|
||||||
{:bcrypt, github: "emqx/erlang-bcrypt", tag: "0.6.2", override: true},
|
{:bcrypt, github: "emqx/erlang-bcrypt", tag: "0.6.2", override: true},
|
||||||
{:uuid, github: "okeuday/uuid", tag: "v2.0.6", override: true},
|
{:uuid, github: "okeuday/uuid", tag: "v2.0.6", override: true},
|
||||||
{:quickrand, github: "okeuday/quickrand", tag: "v2.0.6", override: true}
|
{:quickrand, github: "okeuday/quickrand", tag: "v2.0.6", override: true},
|
||||||
|
{:ra, "2.7.3", override: true}
|
||||||
] ++
|
] ++
|
||||||
emqx_apps(profile_info, version) ++
|
emqx_apps(profile_info, version) ++
|
||||||
enterprise_deps(profile_info) ++ jq_dep() ++ quicer_dep()
|
enterprise_deps(profile_info) ++ jq_dep() ++ quicer_dep()
|
||||||
|
|
|
@ -110,7 +110,8 @@
|
||||||
{uuid, {git, "https://github.com/okeuday/uuid.git", {tag, "v2.0.6"}}},
|
{uuid, {git, "https://github.com/okeuday/uuid.git", {tag, "v2.0.6"}}},
|
||||||
{ssl_verify_fun, "1.1.7"},
|
{ssl_verify_fun, "1.1.7"},
|
||||||
{rfc3339, {git, "https://github.com/emqx/rfc3339.git", {tag, "0.2.3"}}},
|
{rfc3339, {git, "https://github.com/emqx/rfc3339.git", {tag, "0.2.3"}}},
|
||||||
{bcrypt, {git, "https://github.com/emqx/erlang-bcrypt.git", {tag, "0.6.2"}}}
|
{bcrypt, {git, "https://github.com/emqx/erlang-bcrypt.git", {tag, "0.6.2"}}},
|
||||||
|
{ra, "2.7.3"}
|
||||||
]}.
|
]}.
|
||||||
|
|
||||||
{xref_ignores,
|
{xref_ignores,
|
||||||
|
|
|
@ -30,6 +30,15 @@ builtin_n_shards.desc:
|
||||||
Please note that it takes effect only during the initialization of the durable storage database.
|
Please note that it takes effect only during the initialization of the durable storage database.
|
||||||
Changing this configuration parameter after the database has been already created won't take any effect.~"""
|
Changing this configuration parameter after the database has been already created won't take any effect.~"""
|
||||||
|
|
||||||
|
builtin_n_sites.label: "Initial number of sites"
|
||||||
|
builtin_n_sites.desc:
|
||||||
|
"""~
|
||||||
|
Number of storage sites that need to share responsibility over the set of storage shards.
|
||||||
|
In this context, sites are essentially EMQX nodes that have message durability enabled.
|
||||||
|
Please note that it takes effect only during the initialization of the durable storage database.
|
||||||
|
During this phase at least that many sites should come online to distribute shards between them, otherwise message storage will be unavailable until then.
|
||||||
|
After the initialization is complete, sites may be offline, which will affect availability depending on the number of offline sites and replication factor.~"""
|
||||||
|
|
||||||
builtin_local_write_buffer.label: "Local write buffer"
|
builtin_local_write_buffer.label: "Local write buffer"
|
||||||
builtin_local_write_buffer.desc:
|
builtin_local_write_buffer.desc:
|
||||||
"""~
|
"""~
|
||||||
|
|
Loading…
Reference in New Issue