Skip to content

Commit

Permalink
Allow configuring the maximum number of checkpoints
Browse files Browse the repository at this point in the history
  • Loading branch information
the-mikedavis committed Feb 5, 2024
1 parent ddd38cf commit 7d808e4
Show file tree
Hide file tree
Showing 6 changed files with 26 additions and 18 deletions.
2 changes: 2 additions & 0 deletions src/ra.hrl
Original file line number Diff line number Diff line change
Expand Up @@ -230,6 +230,8 @@

-define(DEFAULT_SNAPSHOT_MODULE, ra_log_snapshot).

-define(DEFAULT_MAX_CHECKPOINTS, 10).

-define(RA_LOG_COUNTER_FIELDS,
[{write_ops, ?C_RA_LOG_WRITE_OPS, counter,
"Total number of write ops"},
Expand Down
6 changes: 4 additions & 2 deletions src/ra_log.erl
Original file line number Diff line number Diff line change
Expand Up @@ -136,10 +136,11 @@ pre_init(#{uid := UId,
system_config := #{data_dir := DataDir}} = Conf) ->
Dir = server_data_dir(DataDir, UId),
SnapModule = maps:get(snapshot_module, Conf, ?DEFAULT_SNAPSHOT_MODULE),
MaxCheckpoints = maps:get(max_checkpoints, Conf, ?DEFAULT_MAX_CHECKPOINTS),
SnapshotsDir = filename:join(Dir, "snapshots"),
CheckpointsDir = filename:join(Dir, "checkpoints"),
_ = ra_snapshot:init(UId, SnapModule, SnapshotsDir,
CheckpointsDir, undefined),
CheckpointsDir, undefined, MaxCheckpoints),
ok.

-spec init(ra_log_init_args()) -> state().
Expand All @@ -156,6 +157,7 @@ init(#{uid := UId,
ResendWindow = maps:get(resend_window, Conf, ?DEFAULT_RESEND_WINDOW_SEC),
SnapInterval = maps:get(snapshot_interval, Conf, ?SNAPSHOT_INTERVAL),
CPInterval = maps:get(checkpoint_interval, Conf, ?CHECKPOINT_INTERVAL),
MaxCheckpoints = maps:get(max_checkpoints, Conf, ?DEFAULT_MAX_CHECKPOINTS),
SnapshotsDir = filename:join(Dir, "snapshots"),
CheckpointsDir = filename:join(Dir, "checkpoints"),
Counter = maps:get(counter, Conf, undefined),
Expand All @@ -167,7 +169,7 @@ init(#{uid := UId,
% initialise metrics for this server
true = ets:insert(ra_log_metrics, {UId, 0, 0, 0, 0}),
SnapshotState = ra_snapshot:init(UId, SnapModule, SnapshotsDir,
CheckpointsDir, Counter),
CheckpointsDir, Counter, MaxCheckpoints),
{SnapIdx, SnapTerm} = case ra_snapshot:current(SnapshotState) of
undefined -> {-1, -1};
Curr -> Curr
Expand Down
22 changes: 11 additions & 11 deletions src/ra_snapshot.erl
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@
read_chunk/3,
delete/2,

init/5,
init/6,
init_ets/0,
current/1,
pending/1,
Expand Down Expand Up @@ -76,13 +76,11 @@
pending :: option({pid(), ra_idxterm(), kind()}),
accepting :: option(#accept{}),
current :: option(ra_idxterm()),
checkpoints = [] :: list(checkpoint())}).
checkpoints = [] :: list(checkpoint()),
max_checkpoints :: pos_integer()}).

-define(ETSTBL, ra_log_snapshot_state).

%% TODO: Make this constant configurable?
-define(MAX_CHECKPOINTS, 10).

-opaque state() :: #?MODULE{}.

-export_type([state/0]).
Expand Down Expand Up @@ -164,14 +162,15 @@
-callback context() -> map().

-spec init(ra_uid(), module(), file:filename(), file:filename(),
undefined | counters:counters_ref()) ->
undefined | counters:counters_ref(), pos_integer()) ->
state().
init(UId, Module, SnapshotsDir, CheckpointDir, Counter) ->
init(UId, Module, SnapshotsDir, CheckpointDir, Counter, MaxCheckpoints) ->
State = #?MODULE{uid = UId,
counter = Counter,
module = Module,
snapshot_directory = SnapshotsDir,
checkpoint_directory = CheckpointDir},
checkpoint_directory = CheckpointDir,
max_checkpoints = MaxCheckpoints},
State1 = find_snapshots(State),
find_checkpoints(State1).

Expand Down Expand Up @@ -570,13 +569,14 @@ take_older_checkpoints(Idx, #?MODULE{checkpoints = Checkpoints0} = State0) ->

-spec take_extra_checkpoints(state()) ->
{state(), [checkpoint()]}.
take_extra_checkpoints(#?MODULE{checkpoints = Checkpoints0} = State0) ->
take_extra_checkpoints(#?MODULE{checkpoints = Checkpoints0,
max_checkpoints = MaxCheckpoints} = State0) ->
Len = erlang:length(Checkpoints0),
case Len - ?MAX_CHECKPOINTS of
case Len - MaxCheckpoints of
ToDelete when ToDelete > 0 ->
%% Take `ToDelete' checkpoints from the list randomly without
%% ever taking the first or last checkpoint.
IdxsToTake = random_idxs_to_take(?MAX_CHECKPOINTS, ToDelete),
IdxsToTake = random_idxs_to_take(MaxCheckpoints, ToDelete),
{Checkpoints, Extras} = lists_take_idxs(Checkpoints0, IdxsToTake),
{State0#?MODULE{checkpoints = Checkpoints}, Extras};
_ ->
Expand Down
6 changes: 4 additions & 2 deletions test/ra_checkpoint_SUITE.erl
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@

-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
-include("src/ra.hrl").

%%%===================================================================
%%% Common Test callbacks
Expand Down Expand Up @@ -61,7 +62,8 @@ init_per_testcase(TestCase, Config) ->
ok = ra_lib:make_dir(CheckpointDir),
[{uid, ra_lib:to_binary(TestCase)},
{snap_dir, SnapDir},
{checkpoint_dir, CheckpointDir} | Config].
{checkpoint_dir, CheckpointDir},
{max_checkpoints, ?DEFAULT_MAX_CHECKPOINTS} | Config].

end_per_testcase(_TestCase, _Config) ->
ok.
Expand Down Expand Up @@ -338,7 +340,7 @@ init_state(Config) ->
ra_log_snapshot,
?config(snap_dir, Config),
?config(checkpoint_dir, Config),
undefined).
undefined, ?config(max_checkpoints, Config)).

meta(Idx, Term, Cluster) ->
#{index => Idx,
Expand Down
2 changes: 1 addition & 1 deletion test/ra_log_2_SUITE.erl
Original file line number Diff line number Diff line change
Expand Up @@ -1357,7 +1357,7 @@ create_snapshot_chunk(Config, #{index := Idx} = Meta, Context) ->
ok = ra_lib:make_dir(OthDir),
ok = ra_lib:make_dir(CPDir),
Sn0 = ra_snapshot:init(<<"someotheruid_adsfasdf">>, ra_log_snapshot,
OthDir, CPDir, undefined),
OthDir, CPDir, undefined, ?DEFAULT_MAX_CHECKPOINTS),
MacRef = <<"9">>,
{Sn1, _} = ra_snapshot:begin_snapshot(Meta, MacRef, snapshot, Sn0),
Sn2 =
Expand Down
6 changes: 4 additions & 2 deletions test/ra_snapshot_SUITE.erl
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@

-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
-include("src/ra.hrl").

%%%===================================================================
%%% Common Test callbacks
Expand Down Expand Up @@ -68,7 +69,8 @@ init_per_testcase(TestCase, Config) ->
ok = ra_lib:make_dir(CheckpointDir),
[{uid, ra_lib:to_binary(TestCase)},
{snap_dir, SnapDir},
{checkpoint_dir, CheckpointDir} | Config].
{checkpoint_dir, CheckpointDir},
{max_checkpoints, ?DEFAULT_MAX_CHECKPOINTS} | Config].

end_per_testcase(_TestCase, _Config) ->
ok.
Expand Down Expand Up @@ -472,7 +474,7 @@ init_state(Config) ->
ra_snapshot:init(?config(uid, Config), ra_log_snapshot,
?config(snap_dir, Config),
?config(checkpoint_dir, Config),
undefined).
undefined, ?config(max_checkpoints, Config)).

meta(Idx, Term, Cluster) ->
#{index => Idx,
Expand Down

0 comments on commit 7d808e4

Please sign in to comment.