Merge pull request #13072 from ieQu1/dev/fix-drop-generation

Idempotent drop_generation
This commit is contained in:
ieQu1 2024-05-22 21:13:30 +02:00 committed by GitHub
commit bfd01c425d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
15 changed files with 385 additions and 184 deletions

View File

@ -246,7 +246,8 @@ fields(layout_builtin_reference) ->
reference, reference,
#{ #{
'readOnly' => true, 'readOnly' => true,
importance => ?IMPORTANCE_HIDDEN importance => ?IMPORTANCE_LOW,
desc => ?DESC(layout_builtin_reference_type)
} }
)} )}
]. ].
@ -257,6 +258,8 @@ desc(builtin_local_write_buffer) ->
?DESC(builtin_local_write_buffer); ?DESC(builtin_local_write_buffer);
desc(layout_builtin_wildcard_optimized) -> desc(layout_builtin_wildcard_optimized) ->
?DESC(layout_builtin_wildcard_optimized); ?DESC(layout_builtin_wildcard_optimized);
desc(layout_builtin_reference) ->
?DESC(layout_builtin_reference);
desc(_) -> desc(_) ->
undefined. undefined.
@ -273,17 +276,12 @@ ds_schema(Options) ->
Options Options
). ).
-ifndef(TEST).
builtin_layouts() ->
[ref(layout_builtin_wildcard_optimized)].
-else.
builtin_layouts() -> builtin_layouts() ->
%% Reference layout stores everything in one stream, so it's not %% Reference layout stores everything in one stream, so it's not
%% suitable for production use. However, it's very simple and %% suitable for production use. However, it's very simple and
%% produces a very predictabale replay order, which can be useful %% produces a very predictabale replay order, which can be useful
%% for testing and debugging: %% for testing and debugging:
[ref(layout_builtin_wildcard_optimized), ref(layout_builtin_reference)]. [ref(layout_builtin_wildcard_optimized), ref(layout_builtin_reference)].
-endif.
sc(Type, Meta) -> hoconsc:mk(Type, Meta). sc(Type, Meta) -> hoconsc:mk(Type, Meta).

View File

@ -124,6 +124,8 @@ The following application environment variables are available:
- `emqx_durable_storage.egress_flush_interval`: period at which the batches of messages are committed to the durable storage. - `emqx_durable_storage.egress_flush_interval`: period at which the batches of messages are committed to the durable storage.
- `emqx_durable_storage.reads`: `leader_preferred` | `local_preferred`.
Runtime settings for the durable storages can be modified via CLI as well as the REST API. Runtime settings for the durable storages can be modified via CLI as well as the REST API.
The following CLI commands are available: The following CLI commands are available:

View File

@ -33,6 +33,12 @@
]). ]).
-export([which_dbs/0, which_shards/1]). -export([which_dbs/0, which_shards/1]).
%% Debug:
-export([
get_egress_workers/1,
get_shard_workers/1
]).
%% behaviour callbacks: %% behaviour callbacks:
-export([init/1]). -export([init/1]).
@ -111,6 +117,28 @@ which_dbs() ->
Key = {n, l, #?db_sup{_ = '_', db = '$1'}}, Key = {n, l, #?db_sup{_ = '_', db = '$1'}},
gproc:select({local, names}, [{{Key, '_', '_'}, [], ['$1']}]). gproc:select({local, names}, [{{Key, '_', '_'}, [], ['$1']}]).
%% @doc Get pids of all local egress servers for the given DB.
-spec get_egress_workers(emqx_ds:db()) -> #{_Shard => pid()}.
get_egress_workers(DB) ->
Children = supervisor:which_children(?via(#?egress_sup{db = DB})),
L = [{Shard, Child} || {Shard, Child, _, _} <- Children, is_pid(Child)],
maps:from_list(L).
%% @doc Get pids of all local shard servers for the given DB.
-spec get_shard_workers(emqx_ds:db()) -> #{_Shard => pid()}.
get_shard_workers(DB) ->
Shards = supervisor:which_children(?via(#?shards_sup{db = DB})),
L = lists:flatmap(
fun
({_Shard, Sup, _, _}) when is_pid(Sup) ->
[{Id, Pid} || {Id, Pid, _, _} <- supervisor:which_children(Sup), is_pid(Pid)];
(_) ->
[]
end,
Shards
),
maps:from_list(L).
%%================================================================================ %%================================================================================
%% behaviour callbacks %% behaviour callbacks
%%================================================================================ %%================================================================================

View File

@ -561,12 +561,27 @@ 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).
-define(SAFERPC(EXPR), -define(SAFE_ERPC(EXPR),
try try
EXPR EXPR
catch catch
error:RPCError = {erpc, _} -> error:RPCError__ = {erpc, _} ->
{error, recoverable, RPCError} {error, recoverable, RPCError__}
end
).
-define(SHARD_RPC(DB, SHARD, NODE, BODY),
case
emqx_ds_replication_layer_shard:servers(
DB, SHARD, application:get_env(emqx_durable_storage, reads, leader_preferred)
)
of
[{_, NODE} | _] ->
begin
BODY
end;
[] ->
{error, recoverable, replica_offline}
end end
). ).
@ -623,44 +638,79 @@ ra_drop_generation(DB, Shard, GenId) ->
end. end.
ra_get_streams(DB, Shard, TopicFilter, Time) -> ra_get_streams(DB, Shard, TopicFilter, Time) ->
{_, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred),
TimestampUs = timestamp_to_timeus(Time), TimestampUs = timestamp_to_timeus(Time),
?SAFERPC(emqx_ds_proto_v4:get_streams(Node, DB, Shard, TopicFilter, TimestampUs)). ?SHARD_RPC(
DB,
Shard,
Node,
?SAFE_ERPC(emqx_ds_proto_v4:get_streams(Node, DB, Shard, TopicFilter, TimestampUs))
).
ra_get_delete_streams(DB, Shard, TopicFilter, Time) -> ra_get_delete_streams(DB, Shard, TopicFilter, Time) ->
{_Name, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred), ?SHARD_RPC(
?SAFERPC(emqx_ds_proto_v4:get_delete_streams(Node, DB, Shard, TopicFilter, Time)). DB,
Shard,
Node,
?SAFE_ERPC(emqx_ds_proto_v4:get_delete_streams(Node, DB, Shard, TopicFilter, Time))
).
ra_make_iterator(DB, Shard, Stream, TopicFilter, StartTime) -> ra_make_iterator(DB, Shard, Stream, TopicFilter, StartTime) ->
{_, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred),
TimeUs = timestamp_to_timeus(StartTime), TimeUs = timestamp_to_timeus(StartTime),
?SAFERPC(emqx_ds_proto_v4:make_iterator(Node, DB, Shard, Stream, TopicFilter, TimeUs)). ?SHARD_RPC(
DB,
Shard,
Node,
?SAFE_ERPC(emqx_ds_proto_v4:make_iterator(Node, DB, Shard, Stream, TopicFilter, TimeUs))
).
ra_make_delete_iterator(DB, Shard, Stream, TopicFilter, StartTime) -> ra_make_delete_iterator(DB, Shard, Stream, TopicFilter, StartTime) ->
{_Name, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred),
TimeUs = timestamp_to_timeus(StartTime), TimeUs = timestamp_to_timeus(StartTime),
?SAFERPC(emqx_ds_proto_v4:make_delete_iterator(Node, DB, Shard, Stream, TopicFilter, TimeUs)). ?SHARD_RPC(
DB,
Shard,
Node,
?SAFE_ERPC(
emqx_ds_proto_v4:make_delete_iterator(Node, DB, Shard, Stream, TopicFilter, TimeUs)
)
).
ra_update_iterator(DB, Shard, Iter, DSKey) -> ra_update_iterator(DB, Shard, Iter, DSKey) ->
{_Name, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred), ?SHARD_RPC(
?SAFERPC(emqx_ds_proto_v4:update_iterator(Node, DB, Shard, Iter, DSKey)). DB,
Shard,
Node,
?SAFE_ERPC(emqx_ds_proto_v4:update_iterator(Node, DB, Shard, Iter, DSKey))
).
ra_next(DB, Shard, Iter, BatchSize) -> ra_next(DB, Shard, Iter, BatchSize) ->
{_Name, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred), ?SHARD_RPC(
DB,
Shard,
Node,
case emqx_ds_proto_v4:next(Node, DB, Shard, Iter, BatchSize) of case emqx_ds_proto_v4:next(Node, DB, Shard, Iter, BatchSize) of
RPCError = {badrpc, _} -> Err = {badrpc, _} ->
{error, recoverable, RPCError}; {error, recoverable, Err};
Other -> Ret ->
Other Ret
end. end
).
ra_delete_next(DB, Shard, Iter, Selector, BatchSize) -> ra_delete_next(DB, Shard, Iter, Selector, BatchSize) ->
{_Name, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred), ?SHARD_RPC(
emqx_ds_proto_v4:delete_next(Node, DB, Shard, Iter, Selector, BatchSize). DB,
Shard,
Node,
?SAFE_ERPC(emqx_ds_proto_v4:delete_next(Node, DB, Shard, Iter, Selector, BatchSize))
).
ra_list_generations_with_lifetimes(DB, Shard) -> ra_list_generations_with_lifetimes(DB, Shard) ->
{_Name, Node} = emqx_ds_replication_layer_shard:server(DB, Shard, local_preferred), Reply = ?SHARD_RPC(
case ?SAFERPC(emqx_ds_proto_v4:list_generations_with_lifetimes(Node, DB, Shard)) of DB,
Shard,
Node,
?SAFE_ERPC(emqx_ds_proto_v4:list_generations_with_lifetimes(Node, DB, Shard))
),
case Reply of
Gens = #{} -> Gens = #{} ->
maps:map( maps:map(
fun(_GenId, Data = #{since := Since, until := Until}) -> fun(_GenId, Data = #{since := Since, until := Until}) ->
@ -711,6 +761,14 @@ apply(
#{?tag := add_generation, ?since := Since}, #{?tag := add_generation, ?since := Since},
#{db_shard := DBShard, latest := Latest0} = State0 #{db_shard := DBShard, latest := Latest0} = State0
) -> ) ->
?tp(
info,
ds_replication_layer_add_generation,
#{
shard => DBShard,
since => Since
}
),
{Timestamp, Latest} = ensure_monotonic_timestamp(Since, Latest0), {Timestamp, Latest} = ensure_monotonic_timestamp(Since, Latest0),
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},
@ -721,6 +779,15 @@ apply(
#{?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(
notice,
ds_replication_layer_update_config,
#{
shard => DBShard,
config => Opts,
since => Since
}
),
{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},
@ -730,6 +797,14 @@ apply(
#{?tag := drop_generation, ?generation := GenId}, #{?tag := drop_generation, ?generation := GenId},
#{db_shard := DBShard} = State #{db_shard := DBShard} = State
) -> ) ->
?tp(
info,
ds_replication_layer_drop_generation,
#{
shard => DBShard,
generation => GenId
}
),
Result = emqx_ds_storage_layer:drop_generation(DBShard, GenId), Result = emqx_ds_storage_layer:drop_generation(DBShard, GenId),
{State, Result}; {State, Result};
apply( apply(

View File

@ -33,7 +33,7 @@
-export([start_link/2, store_batch/3]). -export([start_link/2, store_batch/3]).
%% behavior callbacks: %% behavior callbacks:
-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2]). -export([init/1, format_status/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2]).
%% internal exports: %% internal exports:
-export([]). -export([]).
@ -129,6 +129,21 @@ init([DB, Shard]) ->
}, },
{ok, S}. {ok, S}.
format_status(Status) ->
maps:map(
fun
(state, #s{db = DB, shard = Shard, queue = Q}) ->
#{
db => DB,
shard => Shard,
queue => queue:len(Q)
};
(_, Val) ->
Val
end,
Status
).
handle_call( handle_call(
#enqueue_req{ #enqueue_req{
messages = Msgs, messages = Msgs,

View File

@ -28,8 +28,7 @@
%% Dynamic server location API %% Dynamic server location API
-export([ -export([
servers/3, servers/3
server/3
]). ]).
%% Membership %% Membership
@ -83,16 +82,15 @@ server_name(DB, Shard, Site) ->
%% %%
-spec servers(emqx_ds:db(), emqx_ds_replication_layer:shard_id(), Order) -> [server(), ...] when -spec servers(emqx_ds:db(), emqx_ds_replication_layer:shard_id(), Order) -> [server()] when
Order :: leader_preferred | undefined. Order :: leader_preferred | local_preferred | undefined.
servers(DB, Shard, _Order = leader_preferred) -> servers(DB, Shard, leader_preferred) ->
get_servers_leader_preferred(DB, Shard); get_servers_leader_preferred(DB, Shard);
servers(DB, Shard, local_preferred) ->
get_servers_local_preferred(DB, Shard);
servers(DB, Shard, _Order = undefined) -> servers(DB, Shard, _Order = undefined) ->
get_shard_servers(DB, Shard). get_shard_servers(DB, Shard).
server(DB, Shard, _Which = local_preferred) ->
get_server_local_preferred(DB, Shard).
get_servers_leader_preferred(DB, Shard) -> get_servers_leader_preferred(DB, Shard) ->
%% NOTE: Contact last known leader first, then rest of shard servers. %% NOTE: Contact last known leader first, then rest of shard servers.
ClusterName = get_cluster_name(DB, Shard), ClusterName = get_cluster_name(DB, Shard),
@ -104,17 +102,24 @@ get_servers_leader_preferred(DB, Shard) ->
get_online_servers(DB, Shard) get_online_servers(DB, Shard)
end. end.
get_server_local_preferred(DB, Shard) -> get_servers_local_preferred(DB, Shard) ->
%% NOTE: Contact either local server or a random replica. %% Return list of servers, where the local replica (if exists) is
%% the first element. Note: result is _NOT_ shuffled. This can be
%% bad for the load balancing, but it makes results more
%% deterministic. Caller that doesn't care about that can shuffle
%% the results by itself.
ClusterName = get_cluster_name(DB, Shard), ClusterName = get_cluster_name(DB, Shard),
case ra_leaderboard:lookup_members(ClusterName) of case ra_leaderboard:lookup_members(ClusterName) of
Servers when is_list(Servers) ->
pick_local(Servers);
undefined -> undefined ->
%% TODO Servers = get_online_servers(DB, Shard);
%% Leader is unkonwn if there are no servers of this group on the Servers when is_list(Servers) ->
%% local node. We want to pick a replica in that case as well. ok
pick_random(get_online_servers(DB, Shard)) end,
case lists:keytake(node(), 2, Servers) of
false ->
Servers;
{value, Local, Rest} ->
[Local | Rest]
end. end.
lookup_leader(DB, Shard) -> lookup_leader(DB, Shard) ->
@ -139,17 +144,6 @@ filter_online(Servers) ->
is_server_online({_Name, Node}) -> is_server_online({_Name, Node}) ->
Node == node() orelse lists:member(Node, nodes()). Node == node() orelse lists:member(Node, nodes()).
pick_local(Servers) ->
case lists:keyfind(node(), 2, Servers) of
Local when is_tuple(Local) ->
Local;
false ->
pick_random(Servers)
end.
pick_random(Servers) ->
lists:nth(rand:uniform(length(Servers)), Servers).
get_cluster_name(DB, Shard) -> get_cluster_name(DB, Shard) ->
memoize(fun cluster_name/2, [DB, Shard]). memoize(fun cluster_name/2, [DB, Shard]).

View File

@ -35,7 +35,7 @@
make_iterator/5, make_iterator/5,
make_delete_iterator/5, make_delete_iterator/5,
update_iterator/4, update_iterator/4,
next/5, next/6,
delete_next/6, delete_next/6,
post_creation_actions/1, post_creation_actions/1,
@ -161,7 +161,7 @@
%% GVar used for idle detection: %% GVar used for idle detection:
-define(IDLE_DETECT, idle_detect). -define(IDLE_DETECT, idle_detect).
-define(EPOCH(S, TS), (TS bsl S#s.ts_bits)). -define(EPOCH(S, TS), (TS bsr S#s.ts_offset)).
-ifdef(TEST). -ifdef(TEST).
-include_lib("eunit/include/eunit.hrl"). -include_lib("eunit/include/eunit.hrl").
@ -424,23 +424,21 @@ next(
Schema = #s{ts_offset = TSOffset, ts_bits = TSBits}, Schema = #s{ts_offset = TSOffset, ts_bits = TSBits},
It = #{?storage_key := Stream}, It = #{?storage_key := Stream},
BatchSize, BatchSize,
Now Now,
IsCurrent
) -> ) ->
init_counters(), init_counters(),
%% Compute safe cutoff time. It's the point in time where the last %% Compute safe cutoff time. It's the point in time where the last
%% complete epoch ends, so we need to know the current time to %% complete epoch ends, so we need to know the current time to
%% compute it. This is needed because new keys can be added before %% compute it. This is needed because new keys can be added before
%% the iterator. %% the iterator.
IsWildcard = %%
%% This is needed to avoid situations when the iterator advances
%% to position k1, and then a new message with k2, such that k2 <
%% k1 is inserted. k2 would be missed.
HasCutoff =
case Stream of case Stream of
{_StaticKey, []} -> false; {_StaticKey, []} ->
_ -> true
end,
SafeCutoffTime =
case IsWildcard of
true ->
(Now bsr TSOffset) bsl TSOffset;
false ->
%% Iterators scanning streams without varying topic %% Iterators scanning streams without varying topic
%% levels can operate on incomplete epochs, since new %% levels can operate on incomplete epochs, since new
%% matching keys for the single topic are added in %% matching keys for the single topic are added in
@ -450,10 +448,27 @@ next(
%% filters operating on streams with varying parts: %% filters operating on streams with varying parts:
%% iterator can jump to the next topic and then it %% iterator can jump to the next topic and then it
%% won't backtrack. %% won't backtrack.
false;
_ ->
%% New batches are only added to the current
%% generation. We can ignore cutoff time for old
%% generations:
IsCurrent
end,
SafeCutoffTime =
case HasCutoff of
true ->
?EPOCH(Schema, Now) bsl TSOffset;
false ->
1 bsl TSBits - 1 1 bsl TSBits - 1
end, end,
try try
next_until(Schema, It, SafeCutoffTime, BatchSize) case next_until(Schema, It, SafeCutoffTime, BatchSize) of
{ok, _, []} when not IsCurrent ->
{ok, end_of_stream};
Result ->
Result
end
after after
report_counters(Shard) report_counters(Shard)
end. end.
@ -538,6 +553,17 @@ delete_next_until(
end. end.
handle_event(_ShardId, State = #s{gvars = Gvars}, Time, tick) -> handle_event(_ShardId, State = #s{gvars = Gvars}, Time, tick) ->
%% If the last message was published more than one epoch ago, and
%% the shard remains idle, we need to advance safety cutoff
%% interval to make sure the last epoch becomes visible to the
%% readers.
%%
%% We do so by emitting a dummy event that will be persisted by
%% the replication layer. Processing it will advance the
%% replication layer's clock.
%%
%% This operation is latched to avoid publishing events on every
%% tick.
case ets:lookup(Gvars, ?IDLE_DETECT) of case ets:lookup(Gvars, ?IDLE_DETECT) of
[{?IDLE_DETECT, Latch, LastWrittenTs}] -> [{?IDLE_DETECT, Latch, LastWrittenTs}] ->
ok; ok;
@ -546,13 +572,17 @@ handle_event(_ShardId, State = #s{gvars = Gvars}, Time, tick) ->
LastWrittenTs = 0 LastWrittenTs = 0
end, end,
case Latch of case Latch of
false when ?EPOCH(State, Time) > ?EPOCH(State, LastWrittenTs) -> false when ?EPOCH(State, Time) > ?EPOCH(State, LastWrittenTs) + 1 ->
%% Note: + 1 above delays the event by one epoch to add a
%% safety margin.
ets:insert(Gvars, {?IDLE_DETECT, true, LastWrittenTs}), ets:insert(Gvars, {?IDLE_DETECT, true, LastWrittenTs}),
[dummy_event]; [dummy_event];
_ -> _ ->
[] []
end; end;
handle_event(_ShardId, _Data, _Time, _Event) -> handle_event(_ShardId, _Data, _Time, _Event) ->
%% `dummy_event' goes here and does nothing. But it forces update
%% of `Time' in the replication layer.
[]. [].
%%================================================================================ %%================================================================================

View File

@ -52,7 +52,7 @@
]). ]).
%% gen_server %% gen_server
-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2]). -export([init/1, format_status/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2]).
%% internal exports: %% internal exports:
-export([db_dir/1]). -export([db_dir/1]).
@ -80,6 +80,10 @@
-define(stream_v2(GENERATION, INNER), [GENERATION | INNER]). -define(stream_v2(GENERATION, INNER), [GENERATION | INNER]).
-define(delete_stream(GENERATION, INNER), [GENERATION | INNER]). -define(delete_stream(GENERATION, INNER), [GENERATION | INNER]).
%% Wrappers for the storage events:
-define(storage_event(GEN_ID, PAYLOAD), #{0 := 3333, 1 := GEN_ID, 2 := PAYLOAD}).
-define(mk_storage_event(GEN_ID, PAYLOAD), #{0 => 3333, 1 => GEN_ID, 2 => PAYLOAD}).
%%================================================================================ %%================================================================================
%% Type declarations %% Type declarations
%%================================================================================ %%================================================================================
@ -244,8 +248,8 @@
) -> ) ->
emqx_ds:make_delete_iterator_result(_Iterator). emqx_ds:make_delete_iterator_result(_Iterator).
-callback next(shard_id(), _Data, Iter, pos_integer(), emqx_ds:time()) -> -callback next(shard_id(), _Data, Iter, pos_integer(), emqx_ds:time(), _IsCurrent :: boolean()) ->
{ok, Iter, [emqx_types:message()]} | {error, _}. {ok, Iter, [emqx_types:message()]} | {ok, end_of_stream} | {error, _}.
-callback delete_next( -callback delete_next(
shard_id(), _Data, DeleteIterator, emqx_ds:delete_selector(), pos_integer(), emqx_ds:time() shard_id(), _Data, DeleteIterator, emqx_ds:delete_selector(), pos_integer(), emqx_ds:time()
@ -297,13 +301,15 @@ store_batch(Shard, Messages, Options) ->
[{emqx_ds:time(), emqx_types:message()}], [{emqx_ds:time(), emqx_types:message()}],
emqx_ds:message_store_opts() emqx_ds:message_store_opts()
) -> {ok, cooked_batch()} | ignore | emqx_ds:error(_). ) -> {ok, cooked_batch()} | ignore | emqx_ds:error(_).
prepare_batch(Shard, Messages = [{Time, _Msg} | _], Options) -> prepare_batch(Shard, Messages = [{Time, _} | _], Options) ->
%% NOTE %% NOTE
%% We assume that batches do not span generations. Callers should enforce this. %% We assume that batches do not span generations. Callers should enforce this.
?tp(emqx_ds_storage_layer_prepare_batch, #{ ?tp(emqx_ds_storage_layer_prepare_batch, #{
shard => Shard, messages => Messages, options => Options shard => Shard, messages => Messages, options => Options
}), }),
{GenId, #{module := Mod, data := GenData}} = generation_at(Shard, Time), %% FIXME: always store messages in the current generation
case generation_at(Shard, Time) of
{GenId, #{module := Mod, data := GenData}} ->
T0 = erlang:monotonic_time(microsecond), T0 = erlang:monotonic_time(microsecond),
Result = Result =
case Mod:prepare_batch(Shard, GenData, Messages, Options) of case Mod:prepare_batch(Shard, GenData, Messages, Options) of
@ -316,6 +322,9 @@ prepare_batch(Shard, Messages = [{Time, _Msg} | _], Options) ->
%% TODO store->prepare %% TODO store->prepare
emqx_ds_builtin_metrics:observe_store_batch_time(Shard, T1 - T0), emqx_ds_builtin_metrics:observe_store_batch_time(Shard, T1 - T0),
Result; Result;
not_found ->
ignore
end;
prepare_batch(_Shard, [], _Options) -> prepare_batch(_Shard, [], _Options) ->
ignore. ignore.
@ -444,15 +453,12 @@ update_iterator(
next(Shard, Iter = #{?tag := ?IT, ?generation := GenId, ?enc := GenIter0}, BatchSize, Now) -> next(Shard, Iter = #{?tag := ?IT, ?generation := GenId, ?enc := GenIter0}, BatchSize, Now) ->
case generation_get(Shard, GenId) of case generation_get(Shard, GenId) of
#{module := Mod, data := GenData} -> #{module := Mod, data := GenData} ->
Current = generation_current(Shard), IsCurrent = GenId =:= generation_current(Shard),
case Mod:next(Shard, GenData, GenIter0, BatchSize, Now) of case Mod:next(Shard, GenData, GenIter0, BatchSize, Now, IsCurrent) of
{ok, _GenIter, []} when GenId < Current ->
%% This is a past generation. Storage layer won't write
%% any more messages here. The iterator reached the end:
%% the stream has been fully replayed.
{ok, end_of_stream};
{ok, GenIter, Batch} -> {ok, GenIter, Batch} ->
{ok, Iter#{?enc := GenIter}, Batch}; {ok, Iter#{?enc := GenIter}, Batch};
{ok, end_of_stream} ->
{ok, end_of_stream};
Error = {error, _, _} -> Error = {error, _, _} ->
Error Error
end; end;
@ -513,7 +519,7 @@ add_generation(ShardId, Since) ->
list_generations_with_lifetimes(ShardId) -> list_generations_with_lifetimes(ShardId) ->
gen_server:call(?REF(ShardId), #call_list_generations_with_lifetimes{}, infinity). gen_server:call(?REF(ShardId), #call_list_generations_with_lifetimes{}, infinity).
-spec drop_generation(shard_id(), gen_id()) -> ok. -spec drop_generation(shard_id(), gen_id()) -> ok | {error, _}.
drop_generation(ShardId, GenId) -> drop_generation(ShardId, GenId) ->
gen_server:call(?REF(ShardId), #call_drop_generation{gen_id = GenId}, infinity). gen_server:call(?REF(ShardId), #call_drop_generation{gen_id = GenId}, infinity).
@ -563,6 +569,7 @@ start_link(Shard = {_, _}, Options) ->
init({ShardId, Options}) -> init({ShardId, Options}) ->
process_flag(trap_exit, true), process_flag(trap_exit, true),
?tp(info, ds_storage_init, #{shard => ShardId}),
logger:set_process_metadata(#{shard_id => ShardId, domain => [ds, storage_layer, shard]}), logger:set_process_metadata(#{shard_id => ShardId, domain => [ds, storage_layer, shard]}),
erase_schema_runtime(ShardId), erase_schema_runtime(ShardId),
clear_all_checkpoints(ShardId), clear_all_checkpoints(ShardId),
@ -586,6 +593,17 @@ init({ShardId, Options}) ->
commit_metadata(S), commit_metadata(S),
{ok, S}. {ok, S}.
format_status(Status) ->
maps:map(
fun
(state, State) ->
format_state(State);
(_, Val) ->
Val
end,
Status
).
handle_call(#call_update_config{since = Since, options = Options}, _From, S0) -> handle_call(#call_update_config{since = Since, options = Options}, _From, S0) ->
case handle_update_config(S0, Since, Options) of case handle_update_config(S0, Since, Options) of
S = #s{} -> S = #s{} ->
@ -758,8 +776,22 @@ handle_drop_generation(S0, GenId) ->
} = S0, } = S0,
#{module := Mod, cf_refs := GenCFRefs} = GenSchema, #{module := Mod, cf_refs := GenCFRefs} = GenSchema,
#{?GEN_KEY(GenId) := #{data := RuntimeData}} = OldShard, #{?GEN_KEY(GenId) := #{data := RuntimeData}} = OldShard,
case Mod:drop(ShardId, DB, GenId, GenCFRefs, RuntimeData) of try
ok -> Mod:drop(ShardId, DB, GenId, GenCFRefs, RuntimeData)
catch
EC:Err:Stack ->
?tp(
error,
ds_storage_layer_failed_to_drop_generation,
#{
shard => ShardId,
EC => Err,
stacktrace => Stack,
generation => GenId,
s => format_state(S0)
}
)
end,
CFRefs = OldCFRefs -- GenCFRefs, CFRefs = OldCFRefs -- GenCFRefs,
Shard = maps:remove(?GEN_KEY(GenId), OldShard), Shard = maps:remove(?GEN_KEY(GenId), OldShard),
Schema = maps:remove(?GEN_KEY(GenId), OldSchema), Schema = maps:remove(?GEN_KEY(GenId), OldSchema),
@ -768,8 +800,7 @@ handle_drop_generation(S0, GenId) ->
shard = Shard, shard = Shard,
schema = Schema schema = Schema
}, },
{ok, S} {ok, S}.
end.
-spec open_generation(shard_id(), rocksdb:db_handle(), cf_refs(), gen_id(), generation_schema()) -> -spec open_generation(shard_id(), rocksdb:db_handle(), cf_refs(), gen_id(), generation_schema()) ->
generation(). generation().
@ -815,10 +846,6 @@ new_generation(ShardId, DB, Schema0, Since) ->
next_generation_id(GenId) -> next_generation_id(GenId) ->
GenId + 1. GenId + 1.
-spec prev_generation_id(gen_id()) -> gen_id().
prev_generation_id(GenId) when GenId > 0 ->
GenId - 1.
%% @doc Commit current state of the server to both rocksdb and the persistent term %% @doc Commit current state of the server to both rocksdb and the persistent term
-spec commit_metadata(server_state()) -> ok. -spec commit_metadata(server_state()) -> ok.
commit_metadata(#s{shard_id = ShardId, schema = Schema, shard = Runtime, db = DB}) -> commit_metadata(#s{shard_id = ShardId, schema = Schema, shard = Runtime, db = DB}) ->
@ -914,23 +941,23 @@ handle_accept_snapshot(ShardId) ->
Dir = db_dir(ShardId), Dir = db_dir(ShardId),
emqx_ds_storage_snapshot:new_writer(Dir). emqx_ds_storage_snapshot:new_writer(Dir).
%% FIXME: currently this interface is a hack to handle safe cutoff -spec handle_event(shard_id(), emqx_ds:time(), Event) -> [Event].
%% timestamp in LTS. It has many shortcomings (can lead to infinite handle_event(Shard, Time, ?storage_event(GenId, Event)) ->
%% loops if the CBM is not careful; events from one generation may be case generation_get(Shard, GenId) of
%% sent to the next one, etc.) and the API is not well thought out in not_found ->
%% general. [];
%% #{module := Mod, data := GenData} ->
%% The mechanism of storage layer events should be refined later.
-spec handle_event(shard_id(), emqx_ds:time(), CustomEvent | tick) -> [CustomEvent].
handle_event(Shard, Time, Event) ->
{_GenId, #{module := Mod, data := GenData}} = generation_at(Shard, Time),
?tp(emqx_ds_storage_layer_event, #{mod => Mod, time => Time, event => Event}),
case erlang:function_exported(Mod, handle_event, 4) of case erlang:function_exported(Mod, handle_event, 4) of
true -> true ->
Mod:handle_event(Shard, GenData, Time, Event); NewEvents = Mod:handle_event(Shard, GenData, Time, Event),
[?mk_storage_event(GenId, E) || E <- NewEvents];
false -> false ->
[] []
end. end
end;
handle_event(Shard, Time, Event) ->
GenId = generation_current(Shard),
handle_event(Shard, Time, ?mk_storage_event(GenId, Event)).
%%-------------------------------------------------------------------------------- %%--------------------------------------------------------------------------------
%% Schema access %% Schema access
@ -941,6 +968,25 @@ generation_current(Shard) ->
#{current_generation := Current} = get_schema_runtime(Shard), #{current_generation := Current} = get_schema_runtime(Shard),
Current. Current.
%% TODO: remove me
-spec generation_at(shard_id(), emqx_ds:time()) -> {gen_id(), generation()} | not_found.
generation_at(Shard, Time) ->
Schema = #{current_generation := Current} = get_schema_runtime(Shard),
generation_at(Time, Current, Schema).
generation_at(Time, GenId, Schema) ->
case Schema of
#{?GEN_KEY(GenId) := Gen} ->
case Gen of
#{since := Since} when Time < Since andalso GenId > 0 ->
generation_at(Time, GenId - 1, Schema);
_ ->
{GenId, Gen}
end;
_ ->
not_found
end.
-spec generation_get(shard_id(), gen_id()) -> generation() | not_found. -spec generation_get(shard_id(), gen_id()) -> generation() | not_found.
generation_get(Shard, GenId) -> generation_get(Shard, GenId) ->
case get_schema_runtime(Shard) of case get_schema_runtime(Shard) of
@ -964,19 +1010,23 @@ generations_since(Shard, Since) ->
Schema Schema
). ).
-spec generation_at(shard_id(), emqx_ds:time()) -> {gen_id(), generation()}. format_state(#s{shard_id = ShardId, db = DB, cf_refs = CFRefs, schema = Schema, shard = Shard}) ->
generation_at(Shard, Time) -> #{
Schema = #{current_generation := Current} = get_schema_runtime(Shard), id => ShardId,
generation_at(Time, Current, Schema). db => DB,
cf_refs => CFRefs,
generation_at(Time, GenId, Schema) -> schema => Schema,
#{?GEN_KEY(GenId) := Gen} = Schema, shard =>
case Gen of maps:map(
#{since := Since} when Time < Since andalso GenId > 0 -> fun
generation_at(Time, prev_generation_id(GenId), Schema); (?GEN_KEY(_), _Schema) ->
_ -> '...';
{GenId, Gen} (_K, Val) ->
end. Val
end,
Shard
)
}.
-define(PERSISTENT_TERM(SHARD), {emqx_ds_storage_layer, SHARD}). -define(PERSISTENT_TERM(SHARD), {emqx_ds_storage_layer, SHARD}).

View File

@ -38,7 +38,7 @@
make_iterator/5, make_iterator/5,
make_delete_iterator/5, make_delete_iterator/5,
update_iterator/4, update_iterator/4,
next/5, next/6,
delete_next/6 delete_next/6
]). ]).
@ -148,7 +148,7 @@ update_iterator(_Shard, _Data, OldIter, DSKey) ->
last_seen_message_key = DSKey last_seen_message_key = DSKey
}}. }}.
next(_Shard, #s{db = DB, cf = CF}, It0, BatchSize, _Now) -> next(_Shard, #s{db = DB, cf = CF}, It0, BatchSize, _Now, IsCurrent) ->
#it{topic_filter = TopicFilter, start_time = StartTime, last_seen_message_key = Key0} = It0, #it{topic_filter = TopicFilter, start_time = StartTime, last_seen_message_key = Key0} = It0,
{ok, ITHandle} = rocksdb:iterator(DB, CF, []), {ok, ITHandle} = rocksdb:iterator(DB, CF, []),
Action = Action =
@ -162,7 +162,12 @@ next(_Shard, #s{db = DB, cf = CF}, It0, BatchSize, _Now) ->
{Key, Messages} = do_next(TopicFilter, StartTime, ITHandle, Action, BatchSize, Key0, []), {Key, Messages} = do_next(TopicFilter, StartTime, ITHandle, Action, BatchSize, Key0, []),
rocksdb:iterator_close(ITHandle), rocksdb:iterator_close(ITHandle),
It = It0#it{last_seen_message_key = Key}, It = It0#it{last_seen_message_key = Key},
{ok, It, lists:reverse(Messages)}. case Messages of
[] when not IsCurrent ->
{ok, end_of_stream};
_ ->
{ok, It, lists:reverse(Messages)}
end.
delete_next(_Shard, #s{db = DB, cf = CF}, It0, Selector, BatchSize, _Now) -> delete_next(_Shard, #s{db = DB, cf = CF}, It0, Selector, BatchSize, _Now) ->
#delete_it{ #delete_it{

View File

@ -179,8 +179,7 @@ make_delete_iterator(Node, DB, Shard, Stream, TopicFilter, StartTime) ->
| {ok, end_of_stream} | {ok, end_of_stream}
| {error, _}. | {error, _}.
delete_next(Node, DB, Shard, Iter, Selector, BatchSize) -> delete_next(Node, DB, Shard, Iter, Selector, BatchSize) ->
emqx_rpc:call( erpc:call(
Shard,
Node, Node,
emqx_ds_replication_layer, emqx_ds_replication_layer,
do_delete_next_v4, do_delete_next_v4,

View File

@ -67,10 +67,16 @@ t_00_smoke_open_drop(_Config) ->
%% A simple smoke test that verifies that storing the messages doesn't %% A simple smoke test that verifies that storing the messages doesn't
%% crash %% crash
t_01_smoke_store(_Config) -> t_01_smoke_store(_Config) ->
?check_trace(
#{timetrap => 10_000},
begin
DB = default, DB = default,
?assertMatch(ok, emqx_ds:open_db(DB, opts())), ?assertMatch(ok, emqx_ds:open_db(DB, opts())),
Msg = message(<<"foo/bar">>, <<"foo">>, 0), Msg = message(<<"foo/bar">>, <<"foo">>, 0),
?assertMatch(ok, emqx_ds:store_batch(DB, [Msg])). ?assertMatch(ok, emqx_ds:store_batch(DB, [Msg]))
end,
[]
).
%% A simple smoke test that verifies that getting the list of streams %% A simple smoke test that verifies that getting the list of streams
%% doesn't crash and that iterators can be opened. %% doesn't crash and that iterators can be opened.

View File

@ -183,7 +183,7 @@ t_rebalance(Config) ->
], ],
Stream1 = emqx_utils_stream:interleave( Stream1 = emqx_utils_stream:interleave(
[ [
{50, Stream0}, {10, Stream0},
emqx_utils_stream:const(add_generation) emqx_utils_stream:const(add_generation)
], ],
false false
@ -479,11 +479,13 @@ t_rebalance_offline_restarts(Config) ->
%% %%
shard_server_info(Node, DB, Shard, Site, Info) -> shard_server_info(Node, DB, Shard, Site, Info) ->
Server = shard_server(Node, DB, Shard, Site), ?ON(
{Server, ds_repl_shard(Node, server_info, [Info, Server])}. Node,
begin
shard_server(Node, DB, Shard, Site) -> Server = emqx_ds_replication_layer_shard:shard_server(DB, Shard, Site),
ds_repl_shard(Node, shard_server, [DB, Shard, Site]). {Server, emqx_ds_replication_layer_shard:server_info(Info, Server)}
end
).
ds_repl_meta(Node, Fun) -> ds_repl_meta(Node, Fun) ->
ds_repl_meta(Node, Fun, []). ds_repl_meta(Node, Fun, []).
@ -499,9 +501,6 @@ ds_repl_meta(Node, Fun, Args) ->
error(meta_op_failed) error(meta_op_failed)
end. end.
ds_repl_shard(Node, Fun, Args) ->
erpc:call(Node, emqx_ds_replication_layer_shard, Fun, Args).
shards(Node, DB) -> shards(Node, DB) ->
erpc:call(Node, emqx_ds_replication_layer_meta, shards, [DB]). erpc:call(Node, emqx_ds_replication_layer_meta, shards, [DB]).

View File

@ -27,25 +27,6 @@ opts() ->
%% %%
t_idempotent_store_batch(_Config) ->
Shard = {?FUNCTION_NAME, _ShardId = <<"42">>},
{ok, Pid} = emqx_ds_storage_layer:start_link(Shard, opts()),
%% Push some messages to the shard.
Msgs1 = [gen_message(N) || N <- lists:seq(10, 20)],
GenTs = 30,
Msgs2 = [gen_message(N) || N <- lists:seq(40, 50)],
?assertEqual(ok, emqx_ds_storage_layer:store_batch(Shard, batch(Msgs1), #{})),
%% Add new generation and push the same batch + some more.
?assertEqual(ok, emqx_ds_storage_layer:add_generation(Shard, GenTs)),
?assertEqual(ok, emqx_ds_storage_layer:store_batch(Shard, batch(Msgs1), #{})),
?assertEqual(ok, emqx_ds_storage_layer:store_batch(Shard, batch(Msgs2), #{})),
%% First batch should have been handled idempotently.
?assertEqual(
Msgs1 ++ Msgs2,
lists:keysort(#message.timestamp, emqx_ds_test_helpers:storage_consume(Shard, ['#']))
),
ok = stop_shard(Pid).
t_snapshot_take_restore(_Config) -> t_snapshot_take_restore(_Config) ->
Shard = {?FUNCTION_NAME, _ShardId = <<"42">>}, Shard = {?FUNCTION_NAME, _ShardId = <<"42">>},
{ok, Pid} = emqx_ds_storage_layer:start_link(Shard, opts()), {ok, Pid} = emqx_ds_storage_layer:start_link(Shard, opts()),
@ -77,7 +58,7 @@ t_snapshot_take_restore(_Config) ->
%% Verify that the restored shard contains the messages up until the snapshot. %% Verify that the restored shard contains the messages up until the snapshot.
{ok, _Pid} = emqx_ds_storage_layer:start_link(Shard, opts()), {ok, _Pid} = emqx_ds_storage_layer:start_link(Shard, opts()),
?assertEqual( snabbkaffe_diff:assert_lists_eq(
Msgs1 ++ Msgs2, Msgs1 ++ Msgs2,
lists:keysort(#message.timestamp, emqx_ds_test_helpers:storage_consume(Shard, ['#'])) lists:keysort(#message.timestamp, emqx_ds_test_helpers:storage_consume(Shard, ['#']))
). ).

View File

@ -0,0 +1,10 @@
Various fixes related to the `durable_sessions` feature:
- Add an option to execute read operations on the leader.
- `drop_generation` operation can be replayed multiple times by the replication layer, but it's not idempotent. This PR adds a workaround that avoids a crash when `drop_generation` doesn't succeed. In the future, however, we want to make `drop_generation` idempotent in a nicer way.
- Wrap storage layer events in a small structure containing the generation ID, to make sure events are handled by the same layout CBM & context that produced them.
- Fix crash when storage event arrives to the dropped generation (now removed `storage_layer:generation_at` function didn't handle the case of dropped generations).
- Implement `format_status` callback for several workers to minimize log spam
- Move the responsibility of `end_of_stream` detection to the layout CBM. Previously storage layer used a heuristic: old generations that return an empty batch won't produce more data. This was, obviously, incorrect: for example, bitfield-LTS layout MAY return empty batch while waiting for safe cutoff time.
- `reference` layout has been enabled in prod build. It could be useful for integration testing.
- Fix incorrect epoch calculation in `bitfield_lts:handle_event` callback that lead to missed safe cutoff time updates, and effectively, subscribers being unable to fetch messages until a fresh batch was published.

View File

@ -90,11 +90,20 @@ wildcard_optimized_epoch_bits.desc:
Time span covered by each epoch grows exponentially with the value of `epoch_bits`: Time span covered by each epoch grows exponentially with the value of `epoch_bits`:
- `epoch_bits = 1`: epoch time = 1 millisecond - `epoch_bits = 1`: epoch time = 2 microseconds
- `epoch_bits = 2`: 2 milliseconds - `epoch_bits = 2`: 4 microseconds
... ...
- `epoch_bits = 10`: 1024 milliseconds - `epoch_bits = 20`: ~1s
- `epoch_bits = 13`: ~8 seconds
...~""" ...~"""
layout_builtin_reference.label: "Reference layout"
layout_builtin_reference.desc:
"""~
A simplistic layout type that stores all messages from all topics in chronological order in a single stream.
Not recommended for production use.~"""
layout_builtin_reference_type.label: "Layout type"
layout_builtin_reference_type.desc: "Reference layout type."
} }