Merge pull request #13218 from keynslug/feat/EMQX-12468/wal-less

feat(dsrepl): enable WAL-less batch writes
This commit is contained in:
Andrew Mayorov 2024-06-27 14:52:22 +02:00 committed by GitHub
commit 3ff9440a01
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
16 changed files with 602 additions and 129 deletions

View File

@ -163,7 +163,7 @@ mk_clientid(Prefix, ID) ->
restart_node(Node, NodeSpec) -> restart_node(Node, NodeSpec) ->
?tp(will_restart_node, #{}), ?tp(will_restart_node, #{}),
emqx_cth_cluster:restart(Node, NodeSpec), emqx_cth_cluster:restart(NodeSpec),
wait_nodeup(Node), wait_nodeup(Node),
?tp(restarted_node, #{}), ?tp(restarted_node, #{}),
ok. ok.

View File

@ -38,7 +38,7 @@
%% in `end_per_suite/1` or `end_per_group/2`) with the result from step 2. %% in `end_per_suite/1` or `end_per_group/2`) with the result from step 2.
-module(emqx_cth_cluster). -module(emqx_cth_cluster).
-export([start/1, start/2, restart/1, restart/2]). -export([start/1, start/2, restart/1]).
-export([stop/1, stop_node/1]). -export([stop/1, stop_node/1]).
-export([start_bare_nodes/1, start_bare_nodes/2]). -export([start_bare_nodes/1, start_bare_nodes/2]).
@ -163,13 +163,13 @@ wait_clustered([Node | Nodes] = All, Check, Deadline) ->
wait_clustered(All, Check, Deadline) wait_clustered(All, Check, Deadline)
end. end.
restart(NodeSpec) -> restart(NodeSpecs = [_ | _]) ->
restart(maps:get(name, NodeSpec), NodeSpec). Nodes = [maps:get(name, Spec) || Spec <- NodeSpecs],
ct:pal("Stopping peer nodes: ~p", [Nodes]),
restart(Node, Spec) -> ok = stop(Nodes),
ct:pal("Stopping peer node ~p", [Node]), start([Spec#{boot_type => restart} || Spec <- NodeSpecs]);
ok = emqx_cth_peer:stop(Node), restart(NodeSpec = #{}) ->
start([Spec#{boot_type => restart}]). restart([NodeSpec]).
mk_nodespecs(Nodes, ClusterOpts) -> mk_nodespecs(Nodes, ClusterOpts) ->
NodeSpecs = lists:zipwith( NodeSpecs = lists:zipwith(

View File

@ -22,6 +22,7 @@
-export([start/2, start/3, start/4]). -export([start/2, start/3, start/4]).
-export([start_link/2, start_link/3, start_link/4]). -export([start_link/2, start_link/3, start_link/4]).
-export([stop/1]). -export([stop/1]).
-export([kill/1]).
start(Name, Args) -> start(Name, Args) ->
start(Name, Args, []). start(Name, Args, []).
@ -66,6 +67,32 @@ stop(Node) when is_atom(Node) ->
ok ok
end. end.
%% @doc Kill a node abruptly, through mechanisms provided by OS.
%% Relies on POSIX `kill`.
kill(Node) ->
try erpc:call(Node, os, getpid, []) of
OSPid ->
Pid = whereis(Node),
_ = is_pid(Pid) andalso unlink(Pid),
Result = kill_os_process(OSPid),
%% Either ensure control process stops, or try to stop if not killed.
_ = is_pid(Pid) andalso catch peer:stop(Pid),
Result
catch
error:{erpc, _} = Reason ->
{error, Reason}
end.
kill_os_process(OSPid) ->
Cmd = "kill -SIGKILL " ++ OSPid,
Port = erlang:open_port({spawn, Cmd}, [binary, exit_status, hide]),
receive
{Port, {exit_status, 0}} ->
ok;
{Port, {exit_status, EC}} ->
{error, EC}
end.
parse_node_name(NodeName) -> parse_node_name(NodeName) ->
case string:tokens(atom_to_list(NodeName), "@") of case string:tokens(atom_to_list(NodeName), "@") of
[Name, Host] -> [Name, Host] ->

View File

@ -57,11 +57,14 @@
ra_store_batch/3 ra_store_batch/3
]). ]).
-behaviour(ra_machine).
-export([ -export([
init/1, init/1,
apply/3, apply/3,
tick/2, tick/2,
state_enter/2,
snapshot_module/0 snapshot_module/0
]). ]).
@ -143,7 +146,12 @@
%% Core state of the replication, i.e. the state of ra machine. %% Core state of the replication, i.e. the state of ra machine.
-type ra_state() :: #{ -type ra_state() :: #{
%% Shard ID.
db_shard := {emqx_ds:db(), shard_id()}, db_shard := {emqx_ds:db(), shard_id()},
%% Unique timestamp tracking real time closely.
%% With microsecond granularity it should be nearly impossible for it to run
%% too far ahead of the real time clock.
latest := timestamp_us() latest := timestamp_us()
}. }.
@ -374,7 +382,7 @@ init_buffer(_DB, _Shard, _Options) ->
{ok, #bs{}}. {ok, #bs{}}.
-spec flush_buffer(emqx_ds:db(), shard_id(), [emqx_types:message()], egress_state()) -> -spec flush_buffer(emqx_ds:db(), shard_id(), [emqx_types:message()], egress_state()) ->
{egress_state(), ok | {error, recoverable | unrecoverable, _}}. {egress_state(), ok | emqx_ds:error(_)}.
flush_buffer(DB, Shard, Messages, State) -> flush_buffer(DB, Shard, Messages, State) ->
case ra_store_batch(DB, Shard, Messages) of case ra_store_batch(DB, Shard, Messages) of
{timeout, ServerId} -> {timeout, ServerId} ->
@ -574,6 +582,20 @@ list_nodes() ->
%% Too large for normal operation, need better backpressure mechanism. %% Too large for normal operation, need better backpressure mechanism.
-define(RA_TIMEOUT, 60 * 1000). -define(RA_TIMEOUT, 60 * 1000).
%% How often to release Raft logs?
%% Each time we written approximately this number of bytes.
%% Close to the RocksDB's default of 64 MiB.
-define(RA_RELEASE_LOG_APPROX_SIZE, 50_000_000).
%% ...Or at least each N log entries.
-define(RA_RELEASE_LOG_MIN_FREQ, 64_000).
-ifdef(TEST).
-undef(RA_RELEASE_LOG_APPROX_SIZE).
-undef(RA_RELEASE_LOG_MIN_FREQ).
-define(RA_RELEASE_LOG_APPROX_SIZE, 50_000).
-define(RA_RELEASE_LOG_MIN_FREQ, 1_000).
-endif.
-define(SAFE_ERPC(EXPR), -define(SAFE_ERPC(EXPR),
try try
EXPR EXPR
@ -603,18 +625,20 @@ list_nodes() ->
). ).
-spec ra_store_batch(emqx_ds:db(), emqx_ds_replication_layer:shard_id(), [emqx_types:message()]) -> -spec ra_store_batch(emqx_ds:db(), emqx_ds_replication_layer:shard_id(), [emqx_types:message()]) ->
ok | {timeout, _} | {error, recoverable | unrecoverable, _Err} | _Err. ok | {timeout, _} | {error, recoverable | unrecoverable, _Err}.
ra_store_batch(DB, Shard, Messages) -> ra_store_batch(DB, Shard, Messages) ->
Command = #{ Command = #{
?tag => ?BATCH, ?tag => ?BATCH,
?batch_messages => Messages ?batch_messages => Messages
}, },
Servers = emqx_ds_replication_layer_shard:servers(DB, Shard, leader_preferred), Servers = emqx_ds_replication_layer_shard:servers(DB, Shard, leader_preferred),
case ra:process_command(Servers, Command, ?RA_TIMEOUT) of case emqx_ds_replication_layer_shard:process_command(Servers, Command, ?RA_TIMEOUT) of
{ok, Result, _Leader} -> {ok, Result, _Leader} ->
Result; Result;
Error -> {timeout, _} = Timeout ->
Error Timeout;
{error, Reason = servers_unreachable} ->
{error, recoverable, Reason}
end. end.
ra_add_generation(DB, Shard) -> ra_add_generation(DB, Shard) ->
@ -741,6 +765,9 @@ ra_drop_shard(DB, Shard) ->
%% %%
-define(pd_ra_idx_need_release, '$emqx_ds_raft_idx_need_release').
-define(pd_ra_bytes_need_release, '$emqx_ds_raft_bytes_need_release').
-spec init(_Args :: map()) -> ra_state(). -spec init(_Args :: map()) -> ra_state().
init(#{db := DB, shard := Shard}) -> init(#{db := DB, shard := Shard}) ->
#{db_shard => {DB, Shard}, latest => 0}. #{db_shard => {DB, Shard}, latest => 0}.
@ -748,33 +775,29 @@ init(#{db := DB, shard := Shard}) ->
-spec apply(ra_machine:command_meta_data(), ra_command(), ra_state()) -> -spec apply(ra_machine:command_meta_data(), ra_command(), ra_state()) ->
{ra_state(), _Reply, _Effects}. {ra_state(), _Reply, _Effects}.
apply( apply(
#{index := RaftIdx}, RaftMeta,
#{ #{
?tag := ?BATCH, ?tag := ?BATCH,
?batch_messages := MessagesIn ?batch_messages := MessagesIn
}, },
#{db_shard := DBShard = {DB, Shard}, latest := Latest0} = State0 #{db_shard := DBShard = {DB, Shard}, latest := Latest0} = State0
) -> ) ->
%% NOTE ?tp(ds_ra_apply_batch, #{db => DB, shard => Shard, batch => MessagesIn, latest => Latest0}),
%% Unique timestamp tracking real time closely. {Stats, Latest, Messages} = assign_timestamps(Latest0, MessagesIn),
%% With microsecond granularity it should be nearly impossible for it to run Result = emqx_ds_storage_layer:store_batch(DBShard, Messages, #{durable => false}),
%% too far ahead than the real time clock.
?tp(ds_ra_apply_batch, #{db => DB, shard => Shard, batch => MessagesIn, ts => Latest0}),
{Latest, Messages} = assign_timestamps(Latest0, MessagesIn),
Result = emqx_ds_storage_layer:store_batch(DBShard, Messages, #{}),
State = State0#{latest := Latest}, State = State0#{latest := Latest},
set_ts(DBShard, Latest), set_ts(DBShard, Latest),
%% TODO: Need to measure effects of changing frequency of `release_cursor`. Effects = try_release_log(Stats, RaftMeta, State),
Effect = {release_cursor, RaftIdx, State}, Effects =/= [] andalso ?tp(ds_ra_effects, #{effects => Effects, meta => RaftMeta}),
{State, Result, Effect}; {State, Result, Effects};
apply( apply(
_RaftMeta, RaftMeta,
#{?tag := add_generation, ?since := Since}, #{?tag := add_generation, ?since := Since},
#{db_shard := DBShard, latest := Latest0} = State0 #{db_shard := DBShard, latest := Latest0} = State0
) -> ) ->
?tp( ?tp(
info, info,
ds_replication_layer_add_generation, ds_ra_add_generation,
#{ #{
shard => DBShard, shard => DBShard,
since => Since since => Since
@ -784,15 +807,17 @@ apply(
Result = emqx_ds_storage_layer:add_generation(DBShard, Timestamp), Result = emqx_ds_storage_layer:add_generation(DBShard, Timestamp),
State = State0#{latest := Latest}, State = State0#{latest := Latest},
set_ts(DBShard, Latest), set_ts(DBShard, Latest),
{State, Result}; Effects = release_log(RaftMeta, State),
Effects =/= [] andalso ?tp(ds_ra_effects, #{effects => Effects, meta => RaftMeta}),
{State, Result, Effects};
apply( apply(
_RaftMeta, RaftMeta,
#{?tag := update_config, ?since := Since, ?config := Opts}, #{?tag := update_config, ?since := Since, ?config := Opts},
#{db_shard := DBShard, latest := Latest0} = State0 #{db_shard := DBShard, latest := Latest0} = State0
) -> ) ->
?tp( ?tp(
notice, notice,
ds_replication_layer_update_config, ds_ra_update_config,
#{ #{
shard => DBShard, shard => DBShard,
config => Opts, config => Opts,
@ -802,7 +827,9 @@ apply(
{Timestamp, Latest} = ensure_monotonic_timestamp(Since, Latest0), {Timestamp, Latest} = ensure_monotonic_timestamp(Since, Latest0),
Result = emqx_ds_storage_layer:update_config(DBShard, Timestamp, Opts), Result = emqx_ds_storage_layer:update_config(DBShard, Timestamp, Opts),
State = State0#{latest := Latest}, State = State0#{latest := Latest},
{State, Result}; Effects = release_log(RaftMeta, State),
Effects =/= [] andalso ?tp(ds_ra_effects, #{effects => Effects, meta => RaftMeta}),
{State, Result, Effects};
apply( apply(
_RaftMeta, _RaftMeta,
#{?tag := drop_generation, ?generation := GenId}, #{?tag := drop_generation, ?generation := GenId},
@ -810,7 +837,7 @@ apply(
) -> ) ->
?tp( ?tp(
info, info,
ds_replication_layer_drop_generation, ds_ra_drop_generation,
#{ #{
shard => DBShard, shard => DBShard,
generation => GenId generation => GenId
@ -827,7 +854,7 @@ apply(
set_ts(DBShard, Latest), set_ts(DBShard, Latest),
?tp( ?tp(
debug, debug,
emqx_ds_replication_layer_storage_event, ds_ra_storage_event,
#{ #{
shard => DBShard, payload => CustomEvent, latest => Latest shard => DBShard, payload => CustomEvent, latest => Latest
} }
@ -835,27 +862,83 @@ apply(
Effects = handle_custom_event(DBShard, Latest, CustomEvent), Effects = handle_custom_event(DBShard, Latest, CustomEvent),
{State#{latest => Latest}, ok, Effects}. {State#{latest => Latest}, ok, Effects}.
try_release_log({_N, BatchSize}, RaftMeta = #{index := CurrentIdx}, State) ->
%% NOTE
%% Because cursor release means storage flush (see
%% `emqx_ds_replication_snapshot:write/3`), we should do that not too often
%% (so the storage is happy with L0 SST sizes) and not too rarely (so we don't
%% accumulate huge Raft logs).
case inc_bytes_need_release(BatchSize) of
AccSize when AccSize > ?RA_RELEASE_LOG_APPROX_SIZE ->
release_log(RaftMeta, State);
_NotYet ->
case get_log_need_release(RaftMeta) of
undefined ->
[];
PrevIdx when CurrentIdx - PrevIdx > ?RA_RELEASE_LOG_MIN_FREQ ->
%% Release everything up to the last log entry, but only if there were
%% more than %% `?RA_RELEASE_LOG_MIN_FREQ` new entries since the last
%% release.
release_log(RaftMeta, State);
_ ->
[]
end
end.
release_log(RaftMeta = #{index := CurrentIdx}, State) ->
%% NOTE
%% Release everything up to the last log entry. This is important: any log entries
%% following `CurrentIdx` should not contribute to `State` (that will be recovered
%% from a snapshot).
update_log_need_release(RaftMeta),
reset_bytes_need_release(),
{release_cursor, CurrentIdx, State}.
get_log_need_release(RaftMeta) ->
case erlang:get(?pd_ra_idx_need_release) of
undefined ->
update_log_need_release(RaftMeta),
undefined;
LastIdx ->
LastIdx
end.
update_log_need_release(#{index := CurrentIdx}) ->
erlang:put(?pd_ra_idx_need_release, CurrentIdx).
get_bytes_need_release() ->
emqx_maybe:define(erlang:get(?pd_ra_bytes_need_release), 0).
inc_bytes_need_release(Size) ->
Acc = get_bytes_need_release() + Size,
erlang:put(?pd_ra_bytes_need_release, Acc),
Acc.
reset_bytes_need_release() ->
erlang:put(?pd_ra_bytes_need_release, 0).
-spec tick(integer(), ra_state()) -> ra_machine:effects(). -spec tick(integer(), ra_state()) -> ra_machine:effects().
tick(TimeMs, #{db_shard := DBShard = {DB, Shard}, latest := Latest}) -> tick(TimeMs, #{db_shard := DBShard = {DB, Shard}, latest := Latest}) ->
%% Leader = emqx_ds_replication_layer_shard:lookup_leader(DB, Shard), %% Leader = emqx_ds_replication_layer_shard:lookup_leader(DB, Shard),
{Timestamp, _} = ensure_monotonic_timestamp(timestamp_to_timeus(TimeMs), Latest), {Timestamp, _} = ensure_monotonic_timestamp(timestamp_to_timeus(TimeMs), Latest),
?tp(emqx_ds_replication_layer_tick, #{db => DB, shard => Shard, ts => Timestamp}), ?tp(emqx_ds_replication_layer_tick, #{db => DB, shard => Shard, timestamp => Timestamp}),
handle_custom_event(DBShard, Timestamp, tick). handle_custom_event(DBShard, Timestamp, tick).
assign_timestamps(Latest, Messages) -> assign_timestamps(Latest, Messages) ->
assign_timestamps(Latest, Messages, []). assign_timestamps(Latest, Messages, [], 0, 0).
assign_timestamps(Latest, [MessageIn | Rest], Acc) -> assign_timestamps(Latest0, [Message0 | Rest], Acc, N, Sz) ->
case emqx_message:timestamp(MessageIn, microsecond) of case emqx_message:timestamp(Message0, microsecond) of
TimestampUs when TimestampUs > Latest -> TimestampUs when TimestampUs > Latest0 ->
Message = assign_timestamp(TimestampUs, MessageIn), Latest = TimestampUs,
assign_timestamps(TimestampUs, Rest, [Message | Acc]); Message = assign_timestamp(TimestampUs, Message0);
_Earlier -> _Earlier ->
Message = assign_timestamp(Latest + 1, MessageIn), Latest = Latest0 + 1,
assign_timestamps(Latest + 1, Rest, [Message | Acc]) Message = assign_timestamp(Latest, Message0)
end; end,
assign_timestamps(Latest, [], Acc) -> assign_timestamps(Latest, Rest, [Message | Acc], N + 1, Sz + approx_message_size(Message0));
{Latest, lists:reverse(Acc)}. assign_timestamps(Latest, [], Acc, N, Size) ->
{{N, Size}, Latest, lists:reverse(Acc)}.
assign_timestamp(TimestampUs, Message) -> assign_timestamp(TimestampUs, Message) ->
{TimestampUs, Message}. {TimestampUs, Message}.
@ -888,3 +971,26 @@ handle_custom_event(DBShard, Latest, Event) ->
set_ts({DB, Shard}, TS) -> set_ts({DB, Shard}, TS) ->
emqx_ds_builtin_raft_sup:set_gvar(DB, ?gv_timestamp(Shard), TS). emqx_ds_builtin_raft_sup:set_gvar(DB, ?gv_timestamp(Shard), TS).
%%
-spec state_enter(ra_server:ra_state() | eol, ra_state()) -> ra_machine:effects().
state_enter(MemberState, #{db_shard := {DB, Shard}, latest := Latest}) ->
?tp(
ds_ra_state_enter,
#{db => DB, shard => Shard, latest => Latest, state => MemberState}
),
[].
%%
approx_message_size(#message{from = ClientID, topic = Topic, payload = Payload}) ->
%% NOTE: Overhead here is basically few empty maps + 8-byte message id.
%% TODO: Probably need to ask the storage layer about the footprint.
MinOverhead = 40,
MinOverhead + clientid_size(ClientID) + byte_size(Topic) + byte_size(Payload).
clientid_size(ClientID) when is_binary(ClientID) ->
byte_size(ClientID);
clientid_size(ClientID) ->
erlang:external_size(ClientID).

View File

@ -4,6 +4,8 @@
-module(emqx_ds_replication_layer_shard). -module(emqx_ds_replication_layer_shard).
-include_lib("snabbkaffe/include/trace.hrl").
%% API: %% API:
-export([start_link/3]). -export([start_link/3]).
@ -19,6 +21,12 @@
servers/3 servers/3
]). ]).
%% Safe Process Command API
-export([
process_command/3,
try_servers/3
]).
%% Membership %% Membership
-export([ -export([
add_local_server/2, add_local_server/2,
@ -37,6 +45,12 @@
-type server() :: ra:server_id(). -type server() :: ra:server_id().
-type server_error() :: server_error(none()).
-type server_error(Reason) ::
{timeout, server()}
| {error, server(), Reason}
| {error, servers_unreachable}.
-define(MEMBERSHIP_CHANGE_TIMEOUT, 30_000). -define(MEMBERSHIP_CHANGE_TIMEOUT, 30_000).
%% %%
@ -146,6 +160,40 @@ local_site() ->
%% %%
-spec process_command([server()], _Command, timeout()) ->
{ok, _Result, _Leader :: server()} | server_error().
process_command(Servers, Command, Timeout) ->
try_servers(Servers, fun ra:process_command/3, [Command, Timeout]).
-spec try_servers([server()], function(), [_Arg]) ->
{ok, _Result, _Leader :: server()} | server_error(_Reason).
try_servers([Server | Rest], Fun, Args) ->
case is_server_online(Server) andalso erlang:apply(Fun, [Server | Args]) of
{ok, R, Leader} ->
{ok, R, Leader};
_Online = false ->
?tp(emqx_ds_replshard_try_next_servers, #{server => Server, reason => offline}),
try_servers(Rest, Fun, Args);
{error, Reason = noproc} ->
?tp(emqx_ds_replshard_try_next_servers, #{server => Server, reason => Reason}),
try_servers(Rest, Fun, Args);
{error, Reason} when Reason =:= nodedown orelse Reason =:= shutdown ->
%% NOTE
%% Conceptually, those error conditions basically mean the same as a plain
%% timeout: "it's impossible to tell if operation has succeeded or not".
?tp(emqx_ds_replshard_try_servers_timeout, #{server => Server, reason => Reason}),
{timeout, Server};
{timeout, _} = Timeout ->
?tp(emqx_ds_replshard_try_servers_timeout, #{server => Server, reason => timeout}),
Timeout;
{error, Reason} ->
{error, Server, Reason}
end;
try_servers([], _Fun, _Args) ->
{error, servers_unreachable}.
%%
%% @doc Add a local server to the shard cluster. %% @doc Add a local server to the shard cluster.
%% It's recommended to have the local server running before calling this function. %% It's recommended to have the local server running before calling this function.
%% This function is idempotent. %% This function is idempotent.
@ -174,10 +222,10 @@ add_local_server(DB, Shard) ->
} }
end, end,
Timeout = ?MEMBERSHIP_CHANGE_TIMEOUT, Timeout = ?MEMBERSHIP_CHANGE_TIMEOUT,
case ra_try_servers(ShardServers, fun ra:add_member/3, [ServerRecord, Timeout]) of case try_servers(ShardServers, fun ra:add_member/3, [ServerRecord, Timeout]) of
{ok, _, _Leader} -> {ok, _, _Leader} ->
ok; ok;
{error, already_member} -> {error, _Server, already_member} ->
ok; ok;
Error -> Error ->
{error, recoverable, Error} {error, recoverable, Error}
@ -208,10 +256,10 @@ drop_local_server(DB, Shard) ->
remove_server(DB, Shard, Server) -> remove_server(DB, Shard, Server) ->
ShardServers = shard_servers(DB, Shard), ShardServers = shard_servers(DB, Shard),
Timeout = ?MEMBERSHIP_CHANGE_TIMEOUT, Timeout = ?MEMBERSHIP_CHANGE_TIMEOUT,
case ra_try_servers(ShardServers, fun ra:remove_member/3, [Server, Timeout]) of case try_servers(ShardServers, fun ra:remove_member/3, [Server, Timeout]) of
{ok, _, _Leader} -> {ok, _, _Leader} ->
ok; ok;
{error, not_member} -> {error, _Server, not_member} ->
ok; ok;
Error -> Error ->
{error, recoverable, Error} {error, recoverable, Error}
@ -261,20 +309,6 @@ member_readiness(#{status := Status, voter_status := #{membership := Membership}
member_readiness(#{}) -> member_readiness(#{}) ->
unknown. unknown.
%%
ra_try_servers([Server | Rest], Fun, Args) ->
case erlang:apply(Fun, [Server | Args]) of
{ok, R, Leader} ->
{ok, R, Leader};
{error, Reason} when Reason == noproc; Reason == nodedown ->
ra_try_servers(Rest, Fun, Args);
ErrorOrTimeout ->
ErrorOrTimeout
end;
ra_try_servers([], _Fun, _Args) ->
{error, servers_unreachable}.
ra_overview(Server) -> ra_overview(Server) ->
case ra:member_overview(Server) of case ra:member_overview(Server) of
{ok, Overview, _Leader} -> {ok, Overview, _Leader} ->

View File

@ -69,6 +69,8 @@ prepare(Index, State) ->
-spec write(_SnapshotDir :: file:filename(), ra_snapshot:meta(), _State :: ra_state()) -> -spec write(_SnapshotDir :: file:filename(), ra_snapshot:meta(), _State :: ra_state()) ->
ok | {ok, _BytesWritten :: non_neg_integer()} | {error, ra_snapshot:file_err()}. ok | {ok, _BytesWritten :: non_neg_integer()} | {error, ra_snapshot:file_err()}.
write(Dir, Meta, MachineState) -> write(Dir, Meta, MachineState) ->
?tp(dsrepl_snapshot_write, #{meta => Meta, state => MachineState}),
ok = emqx_ds_storage_layer:flush(shard_id(MachineState)),
ra_log_snapshot:write(Dir, Meta, MachineState). ra_log_snapshot:write(Dir, Meta, MachineState).
%% Reading a snapshot. %% Reading a snapshot.
@ -165,6 +167,7 @@ complete_read(RS = #rs{reader = SnapReader, started_at = StartedAt}) ->
-spec begin_accept(_SnapshotDir :: file:filename(), ra_snapshot:meta()) -> -spec begin_accept(_SnapshotDir :: file:filename(), ra_snapshot:meta()) ->
{ok, ws()}. {ok, ws()}.
begin_accept(Dir, Meta) -> begin_accept(Dir, Meta) ->
?tp(dsrepl_snapshot_accept_started, #{meta => Meta}),
WS = #ws{ WS = #ws{
phase = machine_state, phase = machine_state,
started_at = erlang:monotonic_time(millisecond), started_at = erlang:monotonic_time(millisecond),
@ -207,7 +210,7 @@ complete_accept(Chunk, WS = #ws{phase = storage_snapshot, writer = SnapWriter0})
?tp(dsrepl_snapshot_write_complete, #{writer => SnapWriter}), ?tp(dsrepl_snapshot_write_complete, #{writer => SnapWriter}),
_ = emqx_ds_storage_snapshot:release_writer(SnapWriter), _ = emqx_ds_storage_snapshot:release_writer(SnapWriter),
Result = complete_accept(WS#ws{writer = SnapWriter}), Result = complete_accept(WS#ws{writer = SnapWriter}),
?tp(dsrepl_snapshot_accepted, #{shard => shard_id(WS)}), ?tp(dsrepl_snapshot_accepted, #{shard => shard_id(WS), state => WS#ws.state}),
Result; Result;
{error, Reason} -> {error, Reason} ->
?tp(dsrepl_snapshot_write_error, #{reason => Reason, writer => SnapWriter0}), ?tp(dsrepl_snapshot_write_error, #{reason => Reason, writer => SnapWriter0}),
@ -218,7 +221,7 @@ complete_accept(Chunk, WS = #ws{phase = storage_snapshot, writer = SnapWriter0})
complete_accept(WS = #ws{started_at = StartedAt, writer = SnapWriter}) -> complete_accept(WS = #ws{started_at = StartedAt, writer = SnapWriter}) ->
ShardId = shard_id(WS), ShardId = shard_id(WS),
logger:info(#{ logger:info(#{
msg => "dsrepl_snapshot_read_complete", msg => "dsrepl_snapshot_write_complete",
shard => ShardId, shard => ShardId,
duration_ms => erlang:monotonic_time(millisecond) - StartedAt, duration_ms => erlang:monotonic_time(millisecond) - StartedAt,
bytes_written => emqx_ds_storage_snapshot:writer_info(bytes_written, SnapWriter) bytes_written => emqx_ds_storage_snapshot:writer_info(bytes_written, SnapWriter)
@ -227,7 +230,7 @@ complete_accept(WS = #ws{started_at = StartedAt, writer = SnapWriter}) ->
write_machine_snapshot(WS). write_machine_snapshot(WS).
write_machine_snapshot(#ws{dir = Dir, meta = Meta, state = MachineState}) -> write_machine_snapshot(#ws{dir = Dir, meta = Meta, state = MachineState}) ->
write(Dir, Meta, MachineState). ra_log_snapshot:write(Dir, Meta, MachineState).
%% Restoring machine state from a snapshot. %% Restoring machine state from a snapshot.
%% This is equivalent to restoring from a log snapshot. %% This is equivalent to restoring from a log snapshot.

View File

@ -140,6 +140,7 @@ t_replication_transfers_snapshots(Config) ->
%% Stop the DB on the "offline" node. %% Stop the DB on the "offline" node.
ok = emqx_cth_cluster:stop_node(NodeOffline), ok = emqx_cth_cluster:stop_node(NodeOffline),
_ = ?block_until(#{?snk_kind := ds_ra_state_enter, state := leader}, 500, 0),
%% Fill the storage with messages and few additional generations. %% Fill the storage with messages and few additional generations.
emqx_ds_test_helpers:apply_stream(?DB, Nodes -- [NodeOffline], Stream), emqx_ds_test_helpers:apply_stream(?DB, Nodes -- [NodeOffline], Stream),
@ -232,14 +233,14 @@ t_rebalance(Config) ->
], ],
Stream1 = emqx_utils_stream:interleave( Stream1 = emqx_utils_stream:interleave(
[ [
{10, Stream0}, {20, Stream0},
emqx_utils_stream:const(add_generation) emqx_utils_stream:const(add_generation)
], ],
false false
), ),
Stream = emqx_utils_stream:interleave( Stream = emqx_utils_stream:interleave(
[ [
{50, Stream0}, {50, Stream1},
emqx_utils_stream:list(Sequence) emqx_utils_stream:list(Sequence)
], ],
true true
@ -604,7 +605,7 @@ t_drop_generation(Config) ->
after after
emqx_cth_cluster:stop(Nodes) emqx_cth_cluster:stop(Nodes)
end, end,
fun(Trace) -> fun(_Trace) ->
%% TODO: some idempotency errors still happen %% TODO: some idempotency errors still happen
%% ?assertMatch([], ?of_kind(ds_storage_layer_failed_to_drop_generation, Trace)), %% ?assertMatch([], ?of_kind(ds_storage_layer_failed_to_drop_generation, Trace)),
true true
@ -794,6 +795,118 @@ t_store_batch_fail(_Config) ->
] ]
). ).
t_crash_restart_recover(init, Config) ->
Apps = [appspec(emqx_durable_storage), emqx_ds_builtin_raft],
Specs = emqx_cth_cluster:mk_nodespecs(
[
{t_crash_stop_recover1, #{apps => Apps}},
{t_crash_stop_recover2, #{apps => Apps}},
{t_crash_stop_recover3, #{apps => Apps}}
],
#{work_dir => emqx_cth_suite:work_dir(?FUNCTION_NAME, Config)}
),
Nodes = emqx_cth_cluster:start(Specs),
[{nodes, Nodes}, {nodespecs, Specs} | Config];
t_crash_restart_recover('end', Config) ->
ok = emqx_cth_cluster:stop(?config(nodes, Config)).
t_crash_restart_recover(Config) ->
%% This testcase verifies that in the event of abrupt site failure message data is
%% correctly preserved.
Nodes = [N1, N2, N3] = ?config(nodes, Config),
_Specs = [_, NS2, NS3] = ?config(nodespecs, Config),
DBOpts = opts(#{n_shards => 16, n_sites => 3, replication_factor => 3}),
%% Prepare test event stream.
NMsgs = 400,
NClients = 8,
{Stream0, TopicStreams} =
emqx_ds_test_helpers:interleaved_topic_messages(?FUNCTION_NAME, NClients, NMsgs),
Stream1 = emqx_utils_stream:interleave(
[
{300, Stream0},
emqx_utils_stream:const(add_generation)
],
false
),
Stream = emqx_utils_stream:interleave(
[
{1000, Stream1},
emqx_utils_stream:list([
fun() -> kill_restart_node_async(N2, NS2, DBOpts) end,
fun() -> kill_restart_node_async(N3, NS3, DBOpts) end
])
],
true
),
?check_trace(
begin
%% Initialize DB on all nodes.
?assertEqual(
[{ok, ok} || _ <- Nodes],
erpc:multicall(Nodes, emqx_ds, open_db, [?DB, DBOpts])
),
%% Apply the test events, including simulated node crashes.
NodeStream = emqx_utils_stream:const(N1),
emqx_ds_test_helpers:apply_stream(?DB, NodeStream, Stream, 0),
%% It's expected to lose few messages when leaders are abruptly killed.
MatchFlushFailed = ?match_event(#{?snk_kind := emqx_ds_buffer_flush_failed}),
{ok, SubRef} = snabbkaffe:subscribe(MatchFlushFailed, NMsgs, _Timeout = 5000, infinity),
{timeout, Events} = snabbkaffe:receive_events(SubRef),
LostMessages = [M || #{batch := Messages} <- Events, M <- Messages],
ct:pal("Some messages were lost: ~p", [LostMessages]),
?assert(length(LostMessages) < NMsgs div 20),
%% Verify that all the successfully persisted messages are there.
VerifyClient = fun({ClientId, ExpectedStream}) ->
Topic = emqx_ds_test_helpers:client_topic(?FUNCTION_NAME, ClientId),
ClientNodes = nodes_of_clientid(ClientId, Nodes),
DSStream1 = ds_topic_stream(ClientId, Topic, hd(ClientNodes)),
%% Do nodes contain same messages for a client?
lists:foreach(
fun(ClientNode) ->
DSStream = ds_topic_stream(ClientId, Topic, ClientNode),
?defer_assert(emqx_ds_test_helpers:diff_messages(DSStream1, DSStream))
end,
tl(ClientNodes)
),
%% Does any messages were lost unexpectedly?
{_, DSMessages} = lists:unzip(emqx_utils_stream:consume(DSStream1)),
ExpectedMessages = emqx_utils_stream:consume(ExpectedStream),
MissingMessages = ExpectedMessages -- DSMessages,
?defer_assert(?assertEqual([], MissingMessages -- LostMessages, DSMessages))
end,
lists:foreach(VerifyClient, TopicStreams)
end,
[]
).
nodes_of_clientid(ClientId, Nodes) ->
emqx_ds_test_helpers:nodes_of_clientid(?DB, ClientId, Nodes).
ds_topic_stream(ClientId, ClientTopic, Node) ->
emqx_ds_test_helpers:ds_topic_stream(?DB, ClientId, ClientTopic, Node).
is_message_lost(Message, MessagesLost) ->
lists:any(
fun(ML) ->
emqx_ds_test_helpers:message_eq([clientid, topic, payload], Message, ML)
end,
MessagesLost
).
kill_restart_node_async(Node, Spec, DBOpts) ->
erlang:spawn_link(?MODULE, kill_restart_node, [Node, Spec, DBOpts]).
kill_restart_node(Node, Spec, DBOpts) ->
ok = emqx_cth_peer:kill(Node),
?tp(test_cluster_node_killed, #{node => Node}),
_ = emqx_cth_cluster:restart(Spec),
ok = erpc:call(Node, emqx_ds, open_db, [?DB, DBOpts]).
%% %%
shard_server_info(Node, DB, Shard, Site, Info) -> shard_server_info(Node, DB, Shard, Site, Info) ->

View File

@ -314,7 +314,7 @@ do_flush(
?tp( ?tp(
debug, debug,
emqx_ds_buffer_flush_failed, emqx_ds_buffer_flush_failed,
#{db => DB, shard => Shard, error => Err} #{db => DB, shard => Shard, batch => Messages, error => Err}
), ),
emqx_ds_builtin_metrics:inc_buffer_batches_failed(Metrics), emqx_ds_builtin_metrics:inc_buffer_batches_failed(Metrics),
Reply = Reply =

View File

@ -29,7 +29,7 @@
open/5, open/5,
drop/5, drop/5,
prepare_batch/4, prepare_batch/4,
commit_batch/3, commit_batch/4,
get_streams/4, get_streams/4,
get_delete_streams/4, get_delete_streams/4,
make_iterator/5, make_iterator/5,
@ -270,7 +270,7 @@ drop(_Shard, DBHandle, GenId, CFRefs, #s{trie = Trie, gvars = GVars}) ->
emqx_ds_storage_layer:shard_id(), emqx_ds_storage_layer:shard_id(),
s(), s(),
[{emqx_ds:time(), emqx_types:message()}, ...], [{emqx_ds:time(), emqx_types:message()}, ...],
emqx_ds:message_store_opts() emqx_ds_storage_layer:batch_store_opts()
) -> ) ->
{ok, cooked_batch()}. {ok, cooked_batch()}.
prepare_batch(_ShardId, S, Messages, _Options) -> prepare_batch(_ShardId, S, Messages, _Options) ->
@ -294,12 +294,14 @@ prepare_batch(_ShardId, S, Messages, _Options) ->
-spec commit_batch( -spec commit_batch(
emqx_ds_storage_layer:shard_id(), emqx_ds_storage_layer:shard_id(),
s(), s(),
cooked_batch() cooked_batch(),
emqx_ds_storage_layer:batch_store_opts()
) -> ok | emqx_ds:error(_). ) -> ok | emqx_ds:error(_).
commit_batch( commit_batch(
_ShardId, _ShardId,
_Data, _Data,
#{?cooked_payloads := [], ?cooked_lts_ops := LTS} #{?cooked_payloads := [], ?cooked_lts_ops := LTS},
_Options
) -> ) ->
%% Assert: %% Assert:
[] = LTS, [] = LTS,
@ -307,7 +309,8 @@ commit_batch(
commit_batch( commit_batch(
_ShardId, _ShardId,
#s{db = DB, data = DataCF, trie = Trie, trie_cf = TrieCF, gvars = Gvars}, #s{db = DB, data = DataCF, trie = Trie, trie_cf = TrieCF, gvars = Gvars},
#{?cooked_lts_ops := LtsOps, ?cooked_payloads := Payloads, ?cooked_ts := MaxTs} #{?cooked_lts_ops := LtsOps, ?cooked_payloads := Payloads, ?cooked_ts := MaxTs},
Options
) -> ) ->
{ok, Batch} = rocksdb:batch(), {ok, Batch} = rocksdb:batch(),
%% Commit LTS trie to the storage: %% Commit LTS trie to the storage:
@ -326,7 +329,7 @@ commit_batch(
end, end,
Payloads Payloads
), ),
Result = rocksdb:write_batch(DB, Batch, []), Result = rocksdb:write_batch(DB, Batch, write_batch_opts(Options)),
rocksdb:release_batch(Batch), rocksdb:release_batch(Batch),
ets:insert(Gvars, {?IDLE_DETECT, false, MaxTs}), ets:insert(Gvars, {?IDLE_DETECT, false, MaxTs}),
%% NOTE %% NOTE
@ -964,6 +967,13 @@ pop_lts_persist_ops() ->
L L
end. end.
-spec write_batch_opts(emqx_ds_storage_layer:batch_store_opts()) ->
_RocksDBOpts :: [{atom(), _}].
write_batch_opts(#{durable := false}) ->
[{disable_wal, true}];
write_batch_opts(#{}) ->
[].
-ifdef(TEST). -ifdef(TEST).
serialize(Msg) -> serialize(Msg) ->

View File

@ -27,7 +27,7 @@
%% Data %% Data
store_batch/3, store_batch/3,
prepare_batch/3, prepare_batch/3,
commit_batch/2, commit_batch/3,
get_streams/3, get_streams/3,
get_delete_streams/3, get_delete_streams/3,
@ -44,6 +44,7 @@
drop_generation/2, drop_generation/2,
%% Snapshotting %% Snapshotting
flush/1,
take_snapshot/1, take_snapshot/1,
accept_snapshot/1, accept_snapshot/1,
@ -69,7 +70,8 @@
shard_id/0, shard_id/0,
options/0, options/0,
prototype/0, prototype/0,
cooked_batch/0 cooked_batch/0,
batch_store_opts/0
]). ]).
-include_lib("snabbkaffe/include/snabbkaffe.hrl"). -include_lib("snabbkaffe/include/snabbkaffe.hrl").
@ -108,10 +110,28 @@
-type shard_id() :: {emqx_ds:db(), binary()}. -type shard_id() :: {emqx_ds:db(), binary()}.
-type cf_refs() :: [{string(), rocksdb:cf_handle()}]. -type cf_ref() :: {string(), rocksdb:cf_handle()}.
-type cf_refs() :: [cf_ref()].
-type gen_id() :: 0..16#ffff. -type gen_id() :: 0..16#ffff.
%% Options affecting how batches should be stored.
%% See also: `emqx_ds:message_store_opts()'.
-type batch_store_opts() ::
#{
%% Whether the whole batch given to `store_batch' should be inserted atomically as
%% a unit. Default: `false'.
atomic => boolean(),
%% Should the storage make sure that the batch is written durably? Non-durable
%% writes are in general unsafe but require much less resources, i.e. with RocksDB
%% non-durable (WAL-less) writes do not usually involve _any_ disk I/O.
%% Default: `true'.
durable => boolean()
}.
%% Options affecting how batches should be prepared.
-type batch_prepare_opts() :: #{}.
%% TODO: kept for BPAPI compatibility. Remove me on EMQX v5.6 %% TODO: kept for BPAPI compatibility. Remove me on EMQX v5.6
-opaque stream_v1() :: -opaque stream_v1() ::
#{ #{
@ -159,7 +179,7 @@
%% Module-specific data defined at generation creation time: %% Module-specific data defined at generation creation time:
data := Data, data := Data,
%% Column families used by this generation %% Column families used by this generation
cf_refs := cf_refs(), cf_names := [string()],
%% 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_message:timestamp(), created_at := emqx_message:timestamp(),
@ -225,14 +245,15 @@
shard_id(), shard_id(),
generation_data(), generation_data(),
[{emqx_ds:time(), emqx_types:message()}, ...], [{emqx_ds:time(), emqx_types:message()}, ...],
emqx_ds:message_store_opts() batch_store_opts()
) -> ) ->
{ok, term()} | emqx_ds:error(_). {ok, term()} | emqx_ds:error(_).
-callback commit_batch( -callback commit_batch(
shard_id(), shard_id(),
generation_data(), generation_data(),
_CookedBatch _CookedBatch,
batch_store_opts()
) -> ok | emqx_ds:error(_). ) -> ok | emqx_ds:error(_).
-callback get_streams( -callback get_streams(
@ -279,6 +300,7 @@
-record(call_update_config, {options :: emqx_ds:create_db_opts(), 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()}).
-record(call_flush, {}).
-record(call_take_snapshot, {}). -record(call_take_snapshot, {}).
-spec drop_shard(shard_id()) -> ok. -spec drop_shard(shard_id()) -> ok.
@ -288,16 +310,13 @@ drop_shard(Shard) ->
-spec store_batch( -spec store_batch(
shard_id(), shard_id(),
[{emqx_ds:time(), emqx_types:message()}], [{emqx_ds:time(), emqx_types:message()}],
emqx_ds:message_store_opts() batch_store_opts()
) -> ) ->
emqx_ds:store_batch_result(). emqx_ds:store_batch_result().
store_batch(Shard, Messages, Options) -> store_batch(Shard, Messages, Options) ->
?tp(emqx_ds_storage_layer_store_batch, #{ case prepare_batch(Shard, Messages, #{}) of
shard => Shard, messages => Messages, options => Options
}),
case prepare_batch(Shard, Messages, Options) of
{ok, CookedBatch} -> {ok, CookedBatch} ->
commit_batch(Shard, CookedBatch); commit_batch(Shard, CookedBatch, Options);
ignore -> ignore ->
ok; ok;
Error = {error, _, _} -> Error = {error, _, _} ->
@ -307,7 +326,7 @@ store_batch(Shard, Messages, Options) ->
-spec prepare_batch( -spec prepare_batch(
shard_id(), shard_id(),
[{emqx_ds:time(), emqx_types:message()}], [{emqx_ds:time(), emqx_types:message()}],
emqx_ds:message_store_opts() batch_prepare_opts()
) -> {ok, cooked_batch()} | ignore | emqx_ds:error(_). ) -> {ok, cooked_batch()} | ignore | emqx_ds:error(_).
prepare_batch(Shard, Messages = [{Time, _} | _], Options) -> prepare_batch(Shard, Messages = [{Time, _} | _], Options) ->
%% NOTE %% NOTE
@ -336,11 +355,15 @@ prepare_batch(Shard, Messages = [{Time, _} | _], Options) ->
prepare_batch(_Shard, [], _Options) -> prepare_batch(_Shard, [], _Options) ->
ignore. ignore.
-spec commit_batch(shard_id(), cooked_batch()) -> emqx_ds:store_batch_result(). -spec commit_batch(
commit_batch(Shard, #{?tag := ?COOKED_BATCH, ?generation := GenId, ?enc := CookedBatch}) -> shard_id(),
cooked_batch(),
batch_store_opts()
) -> emqx_ds:store_batch_result().
commit_batch(Shard, #{?tag := ?COOKED_BATCH, ?generation := GenId, ?enc := CookedBatch}, Options) ->
#{?GEN_KEY(GenId) := #{module := Mod, data := GenData}} = get_schema_runtime(Shard), #{?GEN_KEY(GenId) := #{module := Mod, data := GenData}} = get_schema_runtime(Shard),
T0 = erlang:monotonic_time(microsecond), T0 = erlang:monotonic_time(microsecond),
Result = Mod:commit_batch(Shard, GenData, CookedBatch), Result = Mod:commit_batch(Shard, GenData, CookedBatch, Options),
T1 = erlang:monotonic_time(microsecond), T1 = erlang:monotonic_time(microsecond),
emqx_ds_builtin_metrics:observe_store_batch_time(Shard, T1 - T0), emqx_ds_builtin_metrics:observe_store_batch_time(Shard, T1 - T0),
Result. Result.
@ -539,6 +562,10 @@ shard_info(ShardId, status) ->
error:badarg -> down error:badarg -> down
end. end.
-spec flush(shard_id()) -> ok | {error, _}.
flush(ShardId) ->
gen_server:call(?REF(ShardId), #call_flush{}, infinity).
-spec take_snapshot(shard_id()) -> {ok, emqx_ds_storage_snapshot:reader()} | {error, _Reason}. -spec take_snapshot(shard_id()) -> {ok, emqx_ds_storage_snapshot:reader()} | {error, _Reason}.
take_snapshot(ShardId) -> take_snapshot(ShardId) ->
case gen_server:call(?REF(ShardId), #call_take_snapshot{}, infinity) of case gen_server:call(?REF(ShardId), #call_take_snapshot{}, infinity) of
@ -566,6 +593,7 @@ start_link(Shard = {_, _}, Options) ->
shard_id :: shard_id(), shard_id :: shard_id(),
db :: rocksdb:db_handle(), db :: rocksdb:db_handle(),
cf_refs :: cf_refs(), cf_refs :: cf_refs(),
cf_need_flush :: gen_id(),
schema :: shard_schema(), schema :: shard_schema(),
shard :: shard() shard :: shard()
}). }).
@ -591,10 +619,12 @@ init({ShardId, Options}) ->
{Scm, CFRefs0} {Scm, CFRefs0}
end, end,
Shard = open_shard(ShardId, DB, CFRefs, Schema), Shard = open_shard(ShardId, DB, CFRefs, Schema),
CurrentGenId = maps:get(current_generation, Schema),
S = #s{ S = #s{
shard_id = ShardId, shard_id = ShardId,
db = DB, db = DB,
cf_refs = CFRefs, cf_refs = CFRefs,
cf_need_flush = CurrentGenId,
schema = Schema, schema = Schema,
shard = Shard shard = Shard
}, },
@ -635,6 +665,9 @@ handle_call(#call_list_generations_with_lifetimes{}, _From, S) ->
handle_call(#call_drop_generation{gen_id = GenId}, _From, S0) -> handle_call(#call_drop_generation{gen_id = GenId}, _From, S0) ->
{Reply, S} = handle_drop_generation(S0, GenId), {Reply, S} = handle_drop_generation(S0, GenId),
{reply, Reply, S}; {reply, Reply, S};
handle_call(#call_flush{}, _From, S0) ->
{Reply, S} = handle_flush(S0),
{reply, Reply, S};
handle_call(#call_take_snapshot{}, _From, S) -> handle_call(#call_take_snapshot{}, _From, S) ->
Snapshot = handle_take_snapshot(S), Snapshot = handle_take_snapshot(S),
{reply, Snapshot, S}; {reply, Snapshot, S};
@ -750,9 +783,9 @@ handle_drop_generation(S0, GenId) ->
#s{ #s{
shard_id = ShardId, shard_id = ShardId,
db = DB, db = DB,
schema = #{?GEN_KEY(GenId) := GenSchema} = OldSchema, schema = #{?GEN_KEY(GenId) := GenSchema} = Schema0,
shard = OldShard, shard = #{?GEN_KEY(GenId) := #{data := RuntimeData}} = Shard0,
cf_refs = OldCFRefs cf_refs = CFRefs0
} = S0, } = S0,
%% 1. Commit the metadata first, so other functions are less %% 1. Commit the metadata first, so other functions are less
%% likely to see stale data, and replicas don't end up %% likely to see stale data, and replicas don't end up
@ -761,16 +794,16 @@ handle_drop_generation(S0, GenId) ->
%% %%
%% Note: in theory, this operation may be interrupted in the %% Note: in theory, this operation may be interrupted in the
%% middle. This will leave column families hanging. %% middle. This will leave column families hanging.
Shard = maps:remove(?GEN_KEY(GenId), OldShard), Shard = maps:remove(?GEN_KEY(GenId), Shard0),
Schema = maps:remove(?GEN_KEY(GenId), OldSchema), Schema = maps:remove(?GEN_KEY(GenId), Schema0),
S1 = S0#s{ S1 = S0#s{
shard = Shard, shard = Shard,
schema = Schema schema = Schema
}, },
commit_metadata(S1), commit_metadata(S1),
%% 2. Now, actually drop the data from RocksDB: %% 2. Now, actually drop the data from RocksDB:
#{module := Mod, cf_refs := GenCFRefs} = GenSchema, #{module := Mod, cf_names := GenCFNames} = GenSchema,
#{?GEN_KEY(GenId) := #{data := RuntimeData}} = OldShard, GenCFRefs = [cf_ref(Name, CFRefs0) || Name <- GenCFNames],
try try
Mod:drop(ShardId, DB, GenId, GenCFRefs, RuntimeData) Mod:drop(ShardId, DB, GenId, GenCFRefs, RuntimeData)
catch catch
@ -787,7 +820,7 @@ handle_drop_generation(S0, GenId) ->
} }
) )
end, end,
CFRefs = OldCFRefs -- GenCFRefs, CFRefs = CFRefs0 -- GenCFRefs,
S = S1#s{cf_refs = CFRefs}, S = S1#s{cf_refs = CFRefs},
{ok, S}. {ok, S}.
@ -839,7 +872,7 @@ new_generation(ShardId, DB, Schema0, Shard0, Since) ->
GenSchema = #{ GenSchema = #{
module => Mod, module => Mod,
data => GenData, data => GenData,
cf_refs => NewCFRefs, cf_names => cf_names(NewCFRefs),
created_at => erlang:system_time(millisecond), created_at => erlang:system_time(millisecond),
since => Since, since => Since,
until => undefined until => undefined
@ -866,6 +899,10 @@ rocksdb_open(Shard, Options) ->
DBOptions = [ DBOptions = [
{create_if_missing, true}, {create_if_missing, true},
{create_missing_column_families, true}, {create_missing_column_families, true},
%% NOTE
%% With WAL-less writes, it's important to have CFs flushed atomically.
%% For example, bitfield-lts backend needs data + trie CFs to be consistent.
{atomic_flush, true},
{enable_write_thread_adaptive_yield, false} {enable_write_thread_adaptive_yield, false}
| maps:get(db_options, Options, []) | maps:get(db_options, Options, [])
], ],
@ -921,6 +958,34 @@ update_last_until(Schema = #{current_generation := GenId}, Until) ->
{error, overlaps_existing_generations} {error, overlaps_existing_generations}
end. end.
handle_flush(S = #s{db = DB, cf_refs = CFRefs, cf_need_flush = NeedFlushGenId, shard = Shard}) ->
%% NOTE
%% There could have been few generations added since the last time `flush/1` was
%% called. Strictly speaking, we don't need to flush them all at once as part of
%% a single atomic flush, but the error handling is a bit easier this way.
CurrentGenId = maps:get(current_generation, Shard),
GenIds = lists:seq(NeedFlushGenId, CurrentGenId),
CFHandles = lists:flatmap(
fun(GenId) ->
case Shard of
#{?GEN_KEY(GenId) := #{cf_names := CFNames}} ->
[cf_handle(N, CFRefs) || N <- CFNames];
#{} ->
%% Generation was probably dropped.
[]
end
end,
GenIds
),
case rocksdb:flush(DB, CFHandles, [{wait, true}]) of
ok ->
%% Current generation will always need a flush.
?tp(ds_storage_flush_complete, #{gens => GenIds, cfs => CFHandles}),
{ok, S#s{cf_need_flush = CurrentGenId}};
{error, _} = Error ->
{Error, S}
end.
handle_take_snapshot(#s{db = DB, shard_id = ShardId}) -> handle_take_snapshot(#s{db = DB, shard_id = ShardId}) ->
Name = integer_to_list(erlang:system_time(millisecond)), Name = integer_to_list(erlang:system_time(millisecond)),
Dir = checkpoint_dir(ShardId, Name), Dir = checkpoint_dir(ShardId, Name),
@ -954,6 +1019,21 @@ handle_event(Shard, Time, Event) ->
GenId = generation_current(Shard), GenId = generation_current(Shard),
handle_event(Shard, Time, ?mk_storage_event(GenId, Event)). handle_event(Shard, Time, ?mk_storage_event(GenId, Event)).
%%--------------------------------------------------------------------------------
-spec cf_names(cf_refs()) -> [string()].
cf_names(CFRefs) ->
{CFNames, _CFHandles} = lists:unzip(CFRefs),
CFNames.
-spec cf_ref(_Name :: string(), cf_refs()) -> cf_ref().
cf_ref(Name, CFRefs) ->
lists:keyfind(Name, 1, CFRefs).
-spec cf_handle(_Name :: string(), cf_refs()) -> rocksdb:cf_handle().
cf_handle(Name, CFRefs) ->
element(2, cf_ref(Name, CFRefs)).
%%-------------------------------------------------------------------------------- %%--------------------------------------------------------------------------------
%% Schema access %% Schema access
%%-------------------------------------------------------------------------------- %%--------------------------------------------------------------------------------
@ -1041,23 +1121,106 @@ erase_schema_runtime(Shard) ->
-undef(PERSISTENT_TERM). -undef(PERSISTENT_TERM).
-define(ROCKSDB_SCHEMA_KEY, <<"schema_v1">>). -define(ROCKSDB_SCHEMA_KEY(V), <<"schema_", V>>).
-define(ROCKSDB_SCHEMA_KEY, ?ROCKSDB_SCHEMA_KEY("v2")).
-define(ROCKSDB_SCHEMA_KEYS, [
?ROCKSDB_SCHEMA_KEY,
?ROCKSDB_SCHEMA_KEY("v1")
]).
-spec get_schema_persistent(rocksdb:db_handle()) -> shard_schema() | not_found. -spec get_schema_persistent(rocksdb:db_handle()) -> shard_schema() | not_found.
get_schema_persistent(DB) -> get_schema_persistent(DB) ->
case rocksdb:get(DB, ?ROCKSDB_SCHEMA_KEY, []) of get_schema_persistent(DB, ?ROCKSDB_SCHEMA_KEYS).
get_schema_persistent(DB, [Key | Rest]) ->
case rocksdb:get(DB, Key, []) of
{ok, Blob} -> {ok, Blob} ->
Schema = binary_to_term(Blob), deserialize_schema(Key, Blob);
%% Sanity check:
#{current_generation := _, prototype := _} = Schema,
Schema;
not_found -> not_found ->
not_found get_schema_persistent(DB, Rest)
end. end;
get_schema_persistent(_DB, []) ->
not_found.
-spec put_schema_persistent(rocksdb:db_handle(), shard_schema()) -> ok. -spec put_schema_persistent(rocksdb:db_handle(), shard_schema()) -> ok.
put_schema_persistent(DB, Schema) -> put_schema_persistent(DB, Schema) ->
Blob = term_to_binary(Schema), Blob = term_to_binary(Schema),
rocksdb:put(DB, ?ROCKSDB_SCHEMA_KEY, Blob, []). rocksdb:put(DB, ?ROCKSDB_SCHEMA_KEY, Blob, []).
-spec deserialize_schema(_SchemaVsn :: binary(), binary()) -> shard_schema().
deserialize_schema(SchemaVsn, Blob) ->
%% Sanity check:
Schema = #{current_generation := _, prototype := _} = binary_to_term(Blob),
decode_schema(SchemaVsn, Schema).
decode_schema(?ROCKSDB_SCHEMA_KEY, Schema) ->
Schema;
decode_schema(?ROCKSDB_SCHEMA_KEY("v1"), Schema) ->
maps:map(fun decode_schema_v1/2, Schema).
decode_schema_v1(?GEN_KEY(_), Generation = #{}) ->
decode_generation_schema_v1(Generation);
decode_schema_v1(_, V) ->
V.
decode_generation_schema_v1(SchemaV1 = #{cf_refs := CFRefs}) ->
%% Drop potentially dead CF references from the time generation was created.
Schema = maps:remove(cf_refs, SchemaV1),
Schema#{cf_names => cf_names(CFRefs)};
decode_generation_schema_v1(Schema = #{}) ->
Schema.
%%--------------------------------------------------------------------------------
-ifdef(TEST).
-include_lib("eunit/include/eunit.hrl").
decode_schema_v1_test() ->
SchemaV1 = #{
current_generation => 42,
prototype => {emqx_ds_storage_reference, #{}},
?GEN_KEY(41) => #{
module => emqx_ds_storage_reference,
data => {schema},
cf_refs => [{"emqx_ds_storage_reference41", erlang:make_ref()}],
created_at => 12345,
since => 0,
until => 123456
},
?GEN_KEY(42) => #{
module => emqx_ds_storage_reference,
data => {schema},
cf_refs => [{"emqx_ds_storage_reference42", erlang:make_ref()}],
created_at => 54321,
since => 123456,
until => undefined
}
},
?assertEqual(
#{
current_generation => 42,
prototype => {emqx_ds_storage_reference, #{}},
?GEN_KEY(41) => #{
module => emqx_ds_storage_reference,
data => {schema},
cf_names => ["emqx_ds_storage_reference41"],
created_at => 12345,
since => 0,
until => 123456
},
?GEN_KEY(42) => #{
module => emqx_ds_storage_reference,
data => {schema},
cf_names => ["emqx_ds_storage_reference42"],
created_at => 54321,
since => 123456,
until => undefined
}
},
deserialize_schema(?ROCKSDB_SCHEMA_KEY("v1"), term_to_binary(SchemaV1))
).
-endif.
-undef(ROCKSDB_SCHEMA_KEY). -undef(ROCKSDB_SCHEMA_KEY).

View File

@ -32,7 +32,7 @@
open/5, open/5,
drop/5, drop/5,
prepare_batch/4, prepare_batch/4,
commit_batch/3, commit_batch/4,
get_streams/4, get_streams/4,
get_delete_streams/4, get_delete_streams/4,
make_iterator/5, make_iterator/5,
@ -105,7 +105,7 @@ drop(_ShardId, DBHandle, _GenId, _CFRefs, #s{cf = CFHandle}) ->
prepare_batch(_ShardId, _Data, Messages, _Options) -> prepare_batch(_ShardId, _Data, Messages, _Options) ->
{ok, Messages}. {ok, Messages}.
commit_batch(_ShardId, #s{db = DB, cf = CF}, Messages) -> commit_batch(_ShardId, #s{db = DB, cf = CF}, Messages, Options) ->
{ok, Batch} = rocksdb:batch(), {ok, Batch} = rocksdb:batch(),
lists:foreach( lists:foreach(
fun({TS, Msg}) -> fun({TS, Msg}) ->
@ -115,7 +115,7 @@ commit_batch(_ShardId, #s{db = DB, cf = CF}, Messages) ->
end, end,
Messages Messages
), ),
Res = rocksdb:write_batch(DB, Batch, _WriteOptions = []), Res = rocksdb:write_batch(DB, Batch, write_batch_opts(Options)),
rocksdb:release_batch(Batch), rocksdb:release_batch(Batch),
Res. Res.
@ -284,3 +284,10 @@ do_delete_next(
-spec data_cf(emqx_ds_storage_layer:gen_id()) -> [char()]. -spec data_cf(emqx_ds_storage_layer:gen_id()) -> [char()].
data_cf(GenId) -> data_cf(GenId) ->
"emqx_ds_storage_reference" ++ integer_to_list(GenId). "emqx_ds_storage_reference" ++ integer_to_list(GenId).
-spec write_batch_opts(emqx_ds_storage_layer:batch_store_opts()) ->
_RocksDBOpts :: [{atom(), _}].
write_batch_opts(#{durable := false}) ->
[{disable_wal, true}];
write_batch_opts(#{}) ->
[].

View File

@ -64,7 +64,7 @@ t_iterate(_Config) ->
{PublishedAt, 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, #{}),
%% Iterate through individual topics: %% Iterate through individual topics:
[ [
begin begin
@ -94,7 +94,7 @@ t_delete(_Config) ->
{PublishedAt, 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, #{}),
%% Iterate through topics: %% Iterate through topics:
StartTime = 0, StartTime = 0,
@ -125,7 +125,7 @@ t_get_streams(_Config) ->
{PublishedAt, 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, #{}),
GetStream = fun(Topic) -> GetStream = fun(Topic) ->
StartTime = 0, StartTime = 0,
emqx_ds_storage_layer:get_streams(?SHARD, parse_topic(Topic), StartTime) emqx_ds_storage_layer:get_streams(?SHARD, parse_topic(Topic), StartTime)
@ -152,7 +152,7 @@ t_get_streams(_Config) ->
end end
|| I <- lists:seq(1, 200) || I <- lists:seq(1, 200)
], ],
ok = emqx_ds_storage_layer:store_batch(?SHARD, NewBatch, []), ok = emqx_ds_storage_layer:store_batch(?SHARD, NewBatch, #{}),
%% Check that "foo/bar/baz" topic now appears in two streams: %% Check that "foo/bar/baz" topic now appears in two streams:
%% "foo/bar/baz" and "foo/bar/+": %% "foo/bar/baz" and "foo/bar/+":
NewStreams = lists:sort(GetStream("foo/bar/baz")), NewStreams = lists:sort(GetStream("foo/bar/baz")),
@ -180,7 +180,7 @@ t_new_generation_inherit_trie(_Config) ->
|| I <- lists:seq(1, 200), || I <- lists:seq(1, 200),
Suffix <- [<<"foo">>, <<"bar">>] Suffix <- [<<"foo">>, <<"bar">>]
], ],
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch1, []), ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch1, #{}),
%% 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, _Since = 1_000), ok = emqx_ds_storage_layer:add_generation(?SHARD, _Since = 1_000),
@ -194,7 +194,7 @@ t_new_generation_inherit_trie(_Config) ->
|| I <- lists:seq(1, 200), || I <- lists:seq(1, 200),
Suffix <- [<<"foo">>, <<"bar">>] Suffix <- [<<"foo">>, <<"bar">>]
], ],
ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch2, []), ok = emqx_ds_storage_layer:store_batch(?SHARD, Batch2, #{}),
%% We should get only two streams for wildcard query, for "foo" and for "bar". %% We should get only two streams for wildcard query, for "foo" and for "bar".
?assertMatch( ?assertMatch(
[_Foo, _Bar], [_Foo, _Bar],
@ -217,13 +217,13 @@ t_replay(_Config) ->
{PublishedAt, 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 = [
{TS, make_message(TS, make_topic([wildcard, I, suffix, Suffix]), bin(TS))} {TS, make_message(TS, make_topic([wildcard, I, suffix, Suffix]), bin(TS))}
|| 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 = [M || {_TS, M} <- Batch1 ++ Batch2], Messages = [M || {_TS, M} <- Batch1 ++ Batch2],
%% Missing topics (no ghost messages): %% Missing topics (no ghost messages):

View File

@ -188,12 +188,14 @@ apply_stream(DB, NodeStream0, Stream0, N) ->
?ON(Node, emqx_ds:store_batch(DB, [Msg], #{sync => true})), ?ON(Node, emqx_ds:store_batch(DB, [Msg], #{sync => true})),
apply_stream(DB, NodeStream, Stream, N + 1); apply_stream(DB, NodeStream, Stream, N + 1);
[add_generation | Stream] -> [add_generation | Stream] ->
%% FIXME: ?tp(notice, test_add_generation, #{}),
[Node | NodeStream] = emqx_utils_stream:next(NodeStream0), [Node | NodeStream] = emqx_utils_stream:next(NodeStream0),
?ON(Node, emqx_ds:add_generation(DB)), ?ON(Node, emqx_ds:add_generation(DB)),
apply_stream(DB, NodeStream, Stream, N); apply_stream(DB, NodeStream, Stream, N);
[{Node, Operation, Arg} | Stream] when [{Node, Operation, Arg} | Stream] when
Operation =:= join_db_site; Operation =:= leave_db_site; Operation =:= assign_db_sites Operation =:= join_db_site;
Operation =:= leave_db_site;
Operation =:= assign_db_sites
-> ->
?tp(notice, test_apply_operation, #{node => Node, operation => Operation, arg => Arg}), ?tp(notice, test_apply_operation, #{node => Node, operation => Operation, arg => Arg}),
%% Apply the transition. %% Apply the transition.
@ -207,7 +209,12 @@ apply_stream(DB, NodeStream0, Stream0, N) ->
%% Give some time for at least one transition to complete. %% Give some time for at least one transition to complete.
Transitions = transitions(Node, DB), Transitions = transitions(Node, DB),
ct:pal("Transitions after ~p: ~p", [Operation, Transitions]), ct:pal("Transitions after ~p: ~p", [Operation, Transitions]),
?retry(200, 10, ?assertNotEqual(Transitions, transitions(Node, DB))), case Transitions of
[_ | _] ->
?retry(200, 10, ?assertNotEqual(Transitions, transitions(Node, DB)));
[] ->
ok
end,
apply_stream(DB, NodeStream0, Stream, N); apply_stream(DB, NodeStream0, Stream, N);
[Fun | Stream] when is_function(Fun) -> [Fun | Stream] when is_function(Fun) ->
Fun(), Fun(),
@ -259,15 +266,18 @@ verify_stream_effects(DB, TestCase, Node, ClientId, ExpectedStream) ->
ct:pal("Checking consistency of effects for ~p on ~p", [ClientId, Node]), ct:pal("Checking consistency of effects for ~p on ~p", [ClientId, Node]),
?defer_assert( ?defer_assert(
begin begin
snabbkaffe_diff:assert_lists_eq( diff_messages(
ExpectedStream, ExpectedStream,
ds_topic_stream(DB, ClientId, client_topic(TestCase, ClientId), Node), ds_topic_stream(DB, ClientId, client_topic(TestCase, ClientId), Node)
message_diff_options([id, qos, from, flags, headers, topic, payload, extra])
), ),
ct:pal("Data for client ~p on ~p is consistent.", [ClientId, Node]) ct:pal("Data for client ~p on ~p is consistent.", [ClientId, Node])
end end
). ).
diff_messages(Expected, Got) ->
Fields = [id, qos, from, flags, headers, topic, payload, extra],
diff_messages(Fields, Expected, Got).
diff_messages(Fields, Expected, Got) -> diff_messages(Fields, Expected, Got) ->
snabbkaffe_diff:assert_lists_eq(Expected, Got, message_diff_options(Fields)). snabbkaffe_diff:assert_lists_eq(Expected, Got, message_diff_options(Fields)).

View File

@ -190,7 +190,7 @@ transpose_tail(S, Tail) ->
%% @doc Make a stream by concatenating multiple streams. %% @doc Make a stream by concatenating multiple streams.
-spec chain([stream(X)]) -> stream(X). -spec chain([stream(X)]) -> stream(X).
chain(L) -> chain(L) ->
lists:foldl(fun chain/2, empty(), L). lists:foldr(fun chain/2, empty(), L).
%% @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.

View File

@ -54,7 +54,7 @@ defmodule EMQXUmbrella.MixProject do
{:jiffy, github: "emqx/jiffy", tag: "1.0.6", override: true}, {:jiffy, github: "emqx/jiffy", tag: "1.0.6", override: true},
{:cowboy, github: "emqx/cowboy", tag: "2.9.2", override: true}, {:cowboy, github: "emqx/cowboy", tag: "2.9.2", override: true},
{:esockd, github: "emqx/esockd", tag: "5.11.2", override: true}, {:esockd, github: "emqx/esockd", tag: "5.11.2", override: true},
{:rocksdb, github: "emqx/erlang-rocksdb", tag: "1.8.0-emqx-5", override: true}, {:rocksdb, github: "emqx/erlang-rocksdb", tag: "1.8.0-emqx-6", override: true},
{:ekka, github: "emqx/ekka", tag: "0.19.4", override: true}, {:ekka, github: "emqx/ekka", tag: "0.19.4", override: true},
{:gen_rpc, github: "emqx/gen_rpc", tag: "3.3.1", override: true}, {:gen_rpc, github: "emqx/gen_rpc", tag: "3.3.1", override: true},
{:grpc, github: "emqx/grpc-erl", tag: "0.6.12", override: true}, {:grpc, github: "emqx/grpc-erl", tag: "0.6.12", override: true},

View File

@ -82,7 +82,7 @@
{jiffy, {git, "https://github.com/emqx/jiffy", {tag, "1.0.6"}}}, {jiffy, {git, "https://github.com/emqx/jiffy", {tag, "1.0.6"}}},
{cowboy, {git, "https://github.com/emqx/cowboy", {tag, "2.9.2"}}}, {cowboy, {git, "https://github.com/emqx/cowboy", {tag, "2.9.2"}}},
{esockd, {git, "https://github.com/emqx/esockd", {tag, "5.11.2"}}}, {esockd, {git, "https://github.com/emqx/esockd", {tag, "5.11.2"}}},
{rocksdb, {git, "https://github.com/emqx/erlang-rocksdb", {tag, "1.8.0-emqx-5"}}}, {rocksdb, {git, "https://github.com/emqx/erlang-rocksdb", {tag, "1.8.0-emqx-6"}}},
{ekka, {git, "https://github.com/emqx/ekka", {tag, "0.19.4"}}}, {ekka, {git, "https://github.com/emqx/ekka", {tag, "0.19.4"}}},
{gen_rpc, {git, "https://github.com/emqx/gen_rpc", {tag, "3.3.1"}}}, {gen_rpc, {git, "https://github.com/emqx/gen_rpc", {tag, "3.3.1"}}},
{grpc, {git, "https://github.com/emqx/grpc-erl", {tag, "0.6.12"}}}, {grpc, {git, "https://github.com/emqx/grpc-erl", {tag, "0.6.12"}}},