diff --git a/.gitignore b/.gitignore
index 75c436da..49d2e3bd 100644
--- a/.gitignore
+++ b/.gitignore
@@ -30,3 +30,4 @@ config
doc/*
/.vscode/
+.DS_store
diff --git a/Makefile b/Makefile
index 07a8dfc2..1ca9e08f 100644
--- a/Makefile
+++ b/Makefile
@@ -13,7 +13,7 @@ dep_aten = hex 0.6.0
dep_seshat = hex 0.6.0
DEPS = aten gen_batch_server seshat
-TEST_DEPS = proper meck eunit_formatters inet_tcp_proxy
+TEST_DEPS = proper meck inet_tcp_proxy
BUILD_DEPS = elvis_mk
diff --git a/README.md b/README.md
index 5b747a26..5ecbdff2 100644
--- a/README.md
+++ b/README.md
@@ -348,20 +348,6 @@ is available in a separate repository.
wal_sync_method |
diff --git a/docs/internals/COMPACTION.md b/docs/internals/COMPACTION.md
new file mode 100644
index 00000000..0704a4be
--- /dev/null
+++ b/docs/internals/COMPACTION.md
@@ -0,0 +1,296 @@
+# Ra log compaction
+
+This is a living document capturing current work on log compaction.
+
+## Overview
+
+
+Compaction in Ra is intrinsically linked to the snapshotting
+feature. Standard Raft snapshotting removes all entries in the Ra log
+that precedes the snapshot index where the snapshot is a full representation of
+the state machine state.
+
+The high level idea of compacting in Ra is that instead of deleting all
+segment data that precedes the snapshot index the snapshot data can emit a list
+of live raft indexes which will be kept, either in their original segments
+or written to new compacted segments. the data for these indexes can then
+be omitted from the snapshot to reduce its size and write amplification.
+
+
+### Log sections
+
+Two named sections of the log then emerge.
+
+#### Normal log section
+
+The normal log section is the contiguous log that follows the last snapshot.
+
+#### Compacting log section
+
+The compacting log section consists of all live raft indexes that are lower
+than or equal to the last snapshot taken.
+
+
+
+
+## Compaction phases
+
+### Phase 1
+
+Delete whole segments. This is the easiest and most efficient form of "compaction"
+and will run immediately after each snapshot is taken.
+
+The run will start with the oldest segment and move towards the newest segment
+in the compacting log section. Every segment that has no entries in the live
+indexes list returned by the snapshot state will be deleted. Standard Raft
+log truncation is achieved by returning and empty list of live indexes.
+
+### Compacted segments: naming (phase 3 compaction)
+
+Segment files in a Ra log have numeric names incremented as they are written.
+This is essential as the order is required to ensure log integrity.
+
+Desired Properties of phase 3 compaction:
+
+* Retain immutability, entries will never be deleted from a segment. Instead they
+will be written to a new segment.
+* lexicographic sorting of file names needs to be consistent with order of writes
+* Compaction walks from the old segment to new
+* Easy to recover after unclean shutdown
+
+Segments will be compacted when 2 or more adjacent segments fit into a single
+segment.
+
+The new segment will have the naming format `OLD-NEW.segment`
+
+This means that a single segment can only be compacted once e.g
+`001.segment -> 001-001.segment` as after this there is no new name available
+and it has to wait until it can be compacted with the adjacent segment. Single
+segment compaction could be optional and only triggered when a substantial,
+say 75% or more entries / data can be deleted.
+
+This naming format means it is easy to identify dead segments after an unclean
+exit.
+
+During compaction a different extension will be used: `002-004.compacting` and
+after an unclean shutdown any such files will be removed. Once synced it will be
+renamed to `.segment` and some time after the source files will be deleted (Once
+the Ra server has updated its list of segments).
+
+#### When does phase 3 compaction run?
+
+Options:
+
+* On a timer
+* After phase 1 if needed based on a ratio of live to dead indexes in the compacting section
+* After phase 1 if needed based on disk use / ratio of live data to dead.
+
+
+
+### Ra Server log worker responsibilities
+
+* Write checkpoints and snapshots
+* Perform compaction runs
+* Report segments to be deleted back to the ra server (NB: the worker does
+not perform the segment deletion itself, it needs to report changes back to the
+ra server first). The ra server log worker maintains its own list of segments
+to avoid double processing
+
+
+```mermaid
+sequenceDiagram
+ participant segment-writer
+ participant ra-server
+ participant ra-server-log
+
+ segment-writer--)ra-server: new segments
+ ra-server-)+ra-server-log: new segments
+ ra-server-log->>ra-server-log: phase 1 compaction
+ ra-server-log-)-ra-server: segment changes (new, to be deleted)
+ ra-server-)+ra-server-log: new snapshot
+ ra-server-log->>ra-server-log: write snapshot
+ ra-server-log->>ra-server-log: phase 1 compaction
+ ra-server-log-)-ra-server: snapshot written, segment changes
+```
+
+#### Impact on segment writer process
+
+The segment writer process as well as the WAL relies heavily on the
+`ra_log_snapshot_state` table to avoid writing data that is no longer
+needed. This table contains the latest snapshot index for every
+ra server in the system. In order to do the same for a compacting state machine
+the segment writer would need access to the list of live indexes when flushing
+the WAL to segments.
+
+Options:
+
+* It could read the latest snapshot to find out the live indexes
+* Live indexes are stored in the `ra_log_snapshot_state` table for easy access.
+
+Snapshots can be taken ahead of the segment part of the log meaning that the
+segment writer and log worker may modify the log at different times. To allow
+this there needs to be an invariant that the log worker never marks the last
+segment for deletion as it may have been appended to after or concurrently
+to when the log worker evaluated it's state.
+
+The segment writer can query the `ra_log_snapshot_table` to see if the server
+is using compaction (i.e. have preceding live entries) and if so read the
+live indexes from the snapshot directory (however it is stored). Then it
+can proceed writing any live indexes in the compacting section as well as
+contiguous entries in the normal log section.
+
+
+Segment range: (1, 1000)
+
+Memtable range: (1001, 2000)
+
+Snapshot: 1500, live indexes `[1501, 1999]`,
+
+
+Alt: if the log worker / Ra server is alive the segment writer could call into
+the log worker and ask it to do the log flush and thus make easy use of the
+live indexes list. If the Ra server is not running but is still registered
+the segment writer will flush all entries (if compacting), even those preceding
+last snapshot index. This option minimises changes to segment writer but could
+delay flush _if_ log worker is busy (doing compaction perhaps) when
+the flush request comes in.
+
+
+
+### Snapshot replication
+
+With the snapshot now defined as the snapshot state + live preceding raft indexes
+the default snapshot replication approach will need to change.
+
+The snapshot sender (Ra log worker??) needs to negotiate with the follower to
+discover which preceding raft indexes the follower does not yet have. Then it would
+go on and replicate these before or after (??) sending the snapshot itself.
+
+T: probably before as if a new snapshot has been taken locally we'd most likely
+skip some raft index replication on the second attempt.
+
+Q: How will the follower write the live indexes preceding the snapshot?
+If the main Ra process does it this introduces a 3rd modifier of the Ra log
+and there may be concurrent Ra log writes from the snapshot writer at this point.
+
+It can't write them to the WAL as they are not contiguous unless we allow
+such writes.
+
+The Ra process can write them to a set of temporary segment files then call into
+the segment writer to rename into the set of segments.
+No this can't work with the
+live indexes logic the segment writer applies as it relies on the mem tables
+ranges to decide which indexes to flush.
+
+having pending segment flushes when receiving
+
+the ra process truncates the mem table on snapshot installations
+so that the segment writer avoids
+writing any of the live index preceding the snapshot.
+
+If this is done before live indexes are replicated if the Ra process then waits
+for the mt delete to complete then
+
+Idea: the ra server could truncate the mt as soon as a snapshot installation
+starts to minimise subsequent mem table flushes. Typically this means emptying
+the memtable completely (the ra server could execute the delete perhaps to ensure).
+
+Scenario: pending mem table flushes when snapshot installation comes in.
+
+Need to ensure the old pending data in the WAL / memtable isn't flushed _after_
+the received live indexes are written to a segment and that segment is renamed
+into the list of segments.
+
+Options:
+
+1. Call into segment writer to rename the temporary named segment(s) into the
+main sequence of segments. This command will return the full list of new segments.
+If the memtable has been truncated before this is done by the time the rename
+returns we know there wont be any more segment records being written.
+We can't update the `ra_log_snapshot_state` table until _after_ this as the
+segment writer may assume the live indexes should be in the memtable.
+Also need to handle concurrent compaction?
+Downside: BLOCKING, segwriter can at at times run a backlog.
+
+2.
+
+
+##### Steps when receiving a new valid `install_snapshot_rpc`
+
+* Once the `last` part of the snapshot has been installed and recovered as the
+mem table state we can calculated which live indexes are needed to complete
+the snapshot. The Ra follower remains in `receiving_snapshot` state and replies
+back with a `ra_seq` of the required indexes.
+ * these are received and written to the log as normal
+process itself
+
+
+#### Alternative snapshot install procedure
+
+* Sender begins with sending negotiating which live indexes are needed. It is
+probably sufficient for the receiver to return it's `last_applied` index and the
+sender will send all sparse entries after that index
+* Then it proceeds to send the live indexes _before_ the snapshot (so in it's
+natural order if you like).
+* The receiving ra process then writes these commands to the WAL as normal but
+using a special command / flag to avoid the WAL triggering its' gap detection.
+Ideally the specialised command would include the previous idx so that we can
+still do gap detection in the sparse sequence (should all sends include prior
+sequence so this is the only mode?).
+* The sparse writes are written to a new memtable using a new `ra_mt:sparse_write/2`
+API that bypasses gap validation and stores a sparse sequence instead of range
+* Alt the live indexes replication could be done after the snapshot is complete
+as it is easy for the follower to work out which live indexes it needs.
+when it receives the `last` snapshot chunk it then replies with a special
+continuation command instead of `install_snapshot_result{}` which will initiate
+the live index replication. NB the snapshot sender process may need to call
+into the leader process to get read plans as entries _could_ be in the memtable.
+
+#### How to work out which live indexes the follower needs
+WA
+Gnarly example:
+
+Follower term indexes:
+
+`{{100, 1000}, 1}, {{1001, 1500}, 2}`
+
+Incoming snapshot at 2000 in term 3
+
+live indexes: `[100, 600, 1200, 1777]`
+
+Follower needs all live indexes greater than it's `last_applied` index.
+This is the only safe way to ensure that the live indexes are not stale entries.
+
+If follower `last applied` is: 1500 then follower needs: `[1777]` only.
+If follower `last_applied` is: 1100 then follower needs `[1200, 1777]`
+
+
+#### How to store live indexes with snapshot
+
+* New section in snapshot file format.
+* Separate file (that can be rebuilt if needed from the snapshot).
+
+
+### WAL impact
+
+The WAL will use the `ra_log_snapshot_state` to avoid writing entries that are
+lower than a server's last snapshot index. This is an important optimisation
+that can help the WAL catch up in cases where it is running a longer mailbox
+backlog.
+
+`ra_log_snapshot_state` is going to have to be extended to not just store
+the snapshot index but also the machine's smallest live raft index (at time of
+snapshot) such that the WAL can use that to reduce write workload instead of
+the snapshot index.
+
+Ideally the WAL would only write indexes the precede the snapshot if they're in
+the live indexes, however this would no doubt be performance impacting so in this
+case it is probably better just to have a secondary lowest live index to use
+for decision making.
+
+
+WAL needs to accept sparse writes without a higher snapshot idx (snap install)
+WAL needs to accept contiguous writes with a higher snap idx with and without live indexes
+WAL will send ra_seq of entries written in a WAL
+SegWriter needs to flush the live indexes preceeding the snapshot index which
+_should_ be covered in the sparse sequence of written indexes.
diff --git a/docs/internals/compaction1.jpg b/docs/internals/compaction1.jpg
new file mode 100644
index 00000000..57ae3d2c
Binary files /dev/null and b/docs/internals/compaction1.jpg differ
diff --git a/docs/internals/compaction2.jpg b/docs/internals/compaction2.jpg
new file mode 100644
index 00000000..738222cf
Binary files /dev/null and b/docs/internals/compaction2.jpg differ
diff --git a/src/ra.erl b/src/ra.erl
index 92a594a0..ebf3d50f 100644
--- a/src/ra.erl
+++ b/src/ra.erl
@@ -44,16 +44,10 @@
delete_cluster/1,
delete_cluster/2,
% server management
- % deprecated
- start_server/1,
start_server/2,
start_server/5,
- % deprecated
- restart_server/1,
restart_server/2,
restart_server/3,
- % deprecated
- stop_server/1,
stop_server/2,
force_delete_server/2,
trigger_election/1,
@@ -68,8 +62,6 @@
leave_and_delete_server/3,
leave_and_delete_server/4,
%% troubleshooting
- % deprecated
- overview/0,
overview/1,
%% helpers
new_uid/1,
@@ -79,20 +71,12 @@
aux_command/2,
aux_command/3,
cast_aux_command/2,
- register_external_log_reader/1,
member_overview/1,
member_overview/2,
key_metrics/1,
key_metrics/2
]).
-%% xref should pick these up
--deprecated({start_server, 1}).
--deprecated({restart_server, 1}).
--deprecated({stop_server, 1}).
--deprecated({overview, 0}).
--deprecated({register_external_log_reader, 1}).
-
-define(START_TIMEOUT, ?DEFAULT_TIMEOUT).
-type ra_cmd_ret() :: ra_server_proc:ra_cmd_ret().
@@ -102,8 +86,7 @@
{wal_data_dir, file:filename()} |
{segment_max_entries, non_neg_integer()} |
{wal_max_size_bytes, non_neg_integer()} |
- {wal_compute_checksums, boolean()} |
- {wal_write_strategy, default | o_sync}.
+ {wal_compute_checksums, boolean()}.
-type query_fun() :: fun((term()) -> term()) |
{M :: module(), F :: atom(), A :: list()}.
@@ -171,19 +154,6 @@ start(Params) when is_list(Params) ->
start_in(DataDir) ->
start([{data_dir, DataDir}]).
-%% @doc Restarts a previously successfully started ra server in the default system
-%% @param ServerId the ra_server_id() of the server
-%% @returns `{ok | error, Error}' where error can be
-%% `not_found', `system_not_started' or `name_not_registered' when the
-%% ra server has never before been started on the Erlang node.
-%% DEPRECATED: use restart_server/2
-%% @end
--spec restart_server(ra_server_id()) ->
- ok | {error, term()}.
-restart_server(ServerId) ->
- %% TODO: this is a bad overload
- restart_server(default, ServerId).
-
%% @doc Restarts a previously successfully started ra server
%% @param System the system identifier
%% @param ServerId the ra_server_id() of the server
@@ -227,16 +197,6 @@ restart_server(System, ServerId, AddConfig)
{'EXIT', Err} -> {error, Err}
end.
-%% @doc Stops a ra server in the default system
-%% @param ServerId the ra_server_id() of the server
-%% @returns `{ok | error, nodedown}'
-%% DEPRECATED: use stop_server/2
-%% @end
--spec stop_server(ra_server_id()) ->
- ok | {error, nodedown | system_not_started}.
-stop_server(ServerId) ->
- stop_server(default, ServerId).
-
%% @doc Stops a ra server
%% @param System the system name
%% @param ServerId the ra_server_id() of the server
@@ -509,16 +469,6 @@ start_server(System, ClusterName, #{id := {_, _}} = Conf0, Machine, ServerIds)
machine => Machine},
start_server(System, maps:merge(Conf0, Conf)).
-%% @doc Starts a ra server in the default system
-%% @param Conf a ra_server_config() configuration map.
-%% @returns `{ok | error, Error}'
-%% DEPRECATED: use start_server/2
-%% @end
--spec start_server(ra_server:ra_server_config()) ->
- ok | {error, term()}.
-start_server(Conf) ->
- start_server(default, Conf).
-
%% @doc Starts a ra server
%% @param System the system name
%% @param Conf a ra_server_config() configuration map.
@@ -742,14 +692,6 @@ new_uid(Source) when is_binary(Source) ->
Prefix = ra_lib:derive_safe_string(Source, 6),
ra_lib:make_uid(string:uppercase(Prefix)).
-%% @doc Returns a map of overview data of the default Ra system on the current Erlang
-%% node.
-%% DEPRECATED: use overview/1
-%% @end
--spec overview() -> map() | system_not_started.
-overview() ->
- overview(default).
-
%% @doc Returns a map of overview data of the Ra system on the current Erlang
%% node.
%% @end
@@ -1191,18 +1133,6 @@ aux_command(ServerRef, Cmd, Timeout) ->
cast_aux_command(ServerRef, Cmd) ->
gen_statem:cast(ServerRef, {aux_command, Cmd}).
-%% @doc Registers an external log reader. ServerId needs to be local to the node.
-%% Returns an initiated ra_log_reader:state() state.
-%% Deprecated. Now only reads log data stored in segments, not log data
-%% in mem tables.
-%% @end
--spec register_external_log_reader(ra_server_id()) ->
- ra_log_reader:state().
-register_external_log_reader({_, Node} = ServerId)
- when Node =:= node() ->
- {ok, Reader} = gen_statem:call(ServerId, {register_external_log_reader, self()}),
- Reader.
-
%% @doc Returns a overview map of the internal server state
%%
%% The keys and values will typically remain stable but may
diff --git a/src/ra.hrl b/src/ra.hrl
index 539f1931..89b6948d 100644
--- a/src/ra.hrl
+++ b/src/ra.hrl
@@ -94,6 +94,8 @@
%% A member of the cluster from which replies should be sent.
-type ra_reply_from() :: leader | local | {member, ra_server_id()}.
+-type mfargs() :: {M :: module(), F :: atom(), A :: [term()]}.
+
-define(RA_PROTO_VERSION, 1).
%% the protocol version should be incremented whenever extensions need to be
%% done to the core protocol records (below). It is only ever exchanged by the
diff --git a/src/ra_bench.erl b/src/ra_bench.erl
index 56fff922..85ca9e95 100644
--- a/src/ra_bench.erl
+++ b/src/ra_bench.erl
@@ -24,6 +24,7 @@
% profile/0,
% stop_profile/0
+ start/2,
prepare/0,
run/3,
diff --git a/src/ra_directory.erl b/src/ra_directory.erl
index caf81ecf..bdb47c89 100644
--- a/src/ra_directory.erl
+++ b/src/ra_directory.erl
@@ -191,7 +191,10 @@ overview(System) when is_atom(System) ->
end,
ets:tab2list(ra_state)),
States = maps:from_list(Rows),
- Snaps = maps:from_list(ets:tab2list(ra_log_snapshot_state)),
+ Snaps = lists:foldl(
+ fun (T, Acc) ->
+ Acc#{element(1, T) => erlang:delete_element(1, T)}
+ end, #{}, ets:tab2list(ra_log_snapshot_state)),
lists:foldl(fun ({UId, Pid, Parent, ServerName, ClusterName}, Acc) ->
{S, V} = maps:get(ServerName, States, {undefined, undefined}),
Acc#{ServerName =>
diff --git a/src/ra_flru.erl b/src/ra_flru.erl
index 386d92b6..b9305a9a 100644
--- a/src/ra_flru.erl
+++ b/src/ra_flru.erl
@@ -16,7 +16,8 @@
insert/3,
evict/2,
evict_all/1,
- size/1
+ size/1,
+ max_size/1
]).
-define(MAX_SIZE, 5).
@@ -91,6 +92,10 @@ evict_all(#?MODULE{items = Items,
size(#?MODULE{items = Items}) ->
length(Items).
+-spec max_size(state()) -> non_neg_integer().
+max_size(#?MODULE{max_size = MaxSize}) ->
+ MaxSize.
+
-ifdef(TEST).
-include_lib("eunit/include/eunit.hrl").
diff --git a/src/ra_log.erl b/src/ra_log.erl
index 53129fe7..3443fc71 100644
--- a/src/ra_log.erl
+++ b/src/ra_log.erl
@@ -16,6 +16,7 @@
commit_tx/1,
append/2,
write/2,
+ write_sparse/3,
append_sync/2,
write_sync/2,
fold/5,
@@ -32,7 +33,7 @@
next_index/1,
snapshot_state/1,
set_snapshot_state/2,
- install_snapshot/3,
+ install_snapshot/4,
recover_snapshot/1,
snapshot_index_term/1,
update_release_cursor/5,
@@ -49,9 +50,6 @@
delete_everything/1,
release_resources/3,
- % external reader
- register_reader/2,
- readers/1,
tick/2
]).
@@ -65,10 +63,12 @@
-type ra_meta_key() :: atom().
-type segment_ref() :: {ra_range:range(), File :: file:filename_all()}.
--type event_body() :: {written, ra_term(), ra:range()} |
+-type event_body() :: {written, ra_term(), ra_seq:state()} |
{segments, [{ets:tid(), ra:range()}], [segment_ref()]} |
{resend_write, ra_index()} |
- {snapshot_written, ra_idxterm(), ra_snapshot:kind()} |
+ {snapshot_written, ra_idxterm(),
+ LiveIndexes :: ra_seq:state(),
+ ra_snapshot:kind()} |
{down, pid(), term()}.
-type event() :: {ra_log_event, event_body()}.
@@ -108,9 +108,10 @@
last_resend_time :: option({integer(), WalPid :: pid() | undefined}),
last_wal_write :: {pid(), Ms :: integer()},
reader :: ra_log_reader:state(),
- readers = [] :: [pid()],
mem_table :: ra_mt:state(),
- tx = false :: boolean()
+ tx = false :: boolean(),
+ pending = [] :: ra_seq:state(),
+ live_indexes = [] :: ra_seq:state()
}).
-record(read_plan, {dir :: file:filename_all(),
@@ -209,6 +210,12 @@ init(#{uid := UId,
undefined -> {-1, -1};
Curr -> Curr
end,
+ %% TODO: error handling
+ %% TODO: the "indexes" file isn't authoritative when it comes to live
+ %% indexes, we need to recover the snapshot and query it for live indexes
+ %% to get the actual valua
+ {ok, LiveIndexes} = ra_snapshot:indexes(
+ ra_snapshot:current_snapshot_dir(SnapshotState)),
AccessPattern = maps:get(initial_access_pattern, Conf, sequential),
{ok, Mt0} = ra_log_ets:mem_table_please(Names, UId),
@@ -253,13 +260,15 @@ init(#{uid := UId,
reader = Reader,
mem_table = Mt,
snapshot_state = SnapshotState,
- last_wal_write = {whereis(Wal), now_ms()}
+ last_wal_write = {whereis(Wal), now_ms()},
+ live_indexes = LiveIndexes
},
put_counter(Cfg, ?C_RA_SVR_METRIC_SNAPSHOT_INDEX, SnapIdx),
LastIdx = State0#?MODULE.last_index,
put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, LastIdx),
put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, LastIdx),
- put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS, ra_log_reader:segment_ref_count(Reader)),
+ put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS,
+ ra_log_reader:segment_ref_count(Reader)),
case ra_snapshot:latest_checkpoint(SnapshotState) of
undefined ->
ok;
@@ -315,7 +324,8 @@ init(#{uid := UId,
{SnapIdx, SnapTerm},
State#?MODULE.last_written_index_term
]),
- element(1, delete_segments(SnapIdx, State)).
+ State.
+ % element(1, delete_segments(SnapIdx, State)).
-spec close(state()) -> ok.
close(#?MODULE{cfg = #cfg{uid = _UId},
@@ -340,7 +350,7 @@ commit_tx(#?MODULE{cfg = #cfg{uid = UId,
{WalCommands, Num} =
lists:foldl(fun ({Idx, Term, Cmd0}, {WC, N}) ->
Cmd = {ttb, term_to_iovec(Cmd0)},
- WalC = {append, WriterId, Tid, Idx, Term, Cmd},
+ WalC = {append, WriterId, Tid, Idx-1, Idx, Term, Cmd},
{[WalC | WC], N+1}
end, {[], 0}, Entries),
@@ -365,6 +375,7 @@ append({Idx, Term, Cmd0} = Entry,
wal = Wal} = Cfg,
last_index = LastIdx,
tx = false,
+ pending = Pend0,
mem_table = Mt0} = State)
when Idx =:= LastIdx + 1 ->
case ra_mt:insert(Entry, Mt0) of
@@ -373,11 +384,13 @@ append({Idx, Term, Cmd0} = Entry,
case ra_log_wal:write(Wal, {UId, self()}, ra_mt:tid(Mt),
Idx, Term, Cmd) of
{ok, Pid} ->
+ Pend = ra_seq:limit(Idx - 1, Pend0),
ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, 1),
put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx),
State#?MODULE{last_index = Idx,
last_term = Term,
last_wal_write = {Pid, now_ms()},
+ pending = ra_seq:append(Idx, Pend),
mem_table = Mt};
{error, wal_down} ->
error(wal_down)
@@ -395,6 +408,7 @@ append({Idx, Term, _Cmd} = Entry,
#?MODULE{cfg = Cfg,
last_index = LastIdx,
tx = true,
+ pending = Pend0,
mem_table = Mt0} = State)
when Idx =:= LastIdx + 1 ->
case ra_mt:stage(Entry, Mt0) of
@@ -402,6 +416,7 @@ append({Idx, Term, _Cmd} = Entry,
put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx),
State#?MODULE{last_index = Idx,
last_term = Term,
+ pending = ra_seq:append(Idx, Pend0),
mem_table = Mt};
{error, Reason} ->
?DEBUG("~ts: mem table ~s detected appending index ~b, tx=true "
@@ -423,12 +438,15 @@ append({Idx, _, _}, #?MODULE{last_index = LastIdx}) ->
write([{FstIdx, _, _} | _Rest] = Entries,
#?MODULE{cfg = Cfg,
last_index = LastIdx,
+ pending = Pend0,
mem_table = Mt0} = State0)
when FstIdx =< LastIdx + 1 andalso
FstIdx >= 0 ->
case stage_entries(Cfg, Entries, Mt0) of
{ok, Mt} ->
- wal_write_batch(State0#?MODULE{mem_table = Mt}, Entries);
+ Pend = ra_seq:limit(FstIdx - 1, Pend0),
+ wal_write_batch(State0#?MODULE{mem_table = Mt,
+ pending = Pend}, Entries);
Error ->
Error
end;
@@ -441,6 +459,39 @@ write([{Idx, _, _} | _], #?MODULE{cfg = #cfg{uid = UId},
[UId, Idx, LastIdx+1])),
{error, {integrity_error, Msg}}.
+-spec write_sparse(log_entry(), ra:index(), state()) ->
+ {ok, state()} | {error, wal_down | gap_detected}.
+write_sparse({Idx, Term, _} = Entry, PrevIdx0,
+ #?MODULE{cfg = #cfg{uid = UId,
+ wal = Wal} = Cfg,
+ last_index = LastIdx,
+ mem_table = Mt0} = State0)
+ when PrevIdx0 == undefined orelse
+ (PrevIdx0 == LastIdx) ->
+ {ok, Mt} = ra_mt:insert_sparse(Entry, PrevIdx0, Mt0),
+ % al_write_batch(State0#?MODULE{mem_table = Mt}, [Entry]).
+ put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, LastIdx),
+ ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, 1),
+ Tid = ra_mt:tid(Mt),
+ PrevIdx = case PrevIdx0 of
+ undefined ->
+ Idx - 1;
+ _ ->
+ PrevIdx0
+ end,
+ case ra_log_wal:write(Wal, {UId, self()}, Tid, PrevIdx, Idx,
+ Term, Entry) of
+ {ok, Pid} ->
+ ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, 1),
+ put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx),
+ {ok, State0#?MODULE{last_index = Idx,
+ last_term = Term,
+ mem_table = Mt,
+ last_wal_write = {Pid, now_ms()}}};
+ {error, wal_down} = Err->
+ Err
+ end.
+
-spec fold(FromIdx :: ra_index(), ToIdx :: ra_index(),
fun((log_entry(), Acc) -> Acc), Acc, state()) ->
{Acc, state()} when Acc :: term().
@@ -459,10 +510,12 @@ fold(From0, To0, Fun, Acc0,
MtOverlap = ra_mt:range_overlap({From, To}, Mt),
case MtOverlap of
{undefined, {RemStart, RemEnd}} ->
- {Reader, Acc} = ra_log_reader:fold(RemStart, RemEnd, Fun, Acc0, Reader0),
+ {Reader, Acc} = ra_log_reader:fold(RemStart, RemEnd, Fun,
+ Acc0, Reader0),
{Acc, State#?MODULE{reader = Reader}};
{{MtStart, MtEnd}, {RemStart, RemEnd}} ->
- {Reader, Acc1} = ra_log_reader:fold(RemStart, RemEnd, Fun, Acc0, Reader0),
+ {Reader, Acc1} = ra_log_reader:fold(RemStart, RemEnd, Fun,
+ Acc0, Reader0),
Acc = ra_mt:fold(MtStart, MtEnd, Fun, Acc1, Mt),
NumRead = MtEnd - MtStart + 1,
ok = incr_counter(Cfg, ?C_RA_LOG_READ_MEM_TBL, NumRead),
@@ -483,7 +536,9 @@ fold(_From, _To, _Fun, Acc, State) ->
{[log_entry()], state()}.
sparse_read(Indexes0, #?MODULE{cfg = Cfg,
reader = Reader0,
+ first_index = FstIdx,
last_index = LastIdx,
+ live_indexes = LiveIndexes,
mem_table = Mt} = State) ->
ok = incr_counter(Cfg, ?C_RA_LOG_READ_OPS, 1),
%% indexes need to be sorted high -> low for correct and efficient reading
@@ -499,7 +554,20 @@ sparse_read(Indexes0, #?MODULE{cfg = Cfg,
end,
%% drop any indexes that are larger than the last index available
- Indexes2 = lists:dropwhile(fun (I) -> I > LastIdx end, Indexes1),
+ %% or smaller than first index and not in live indexes
+ Indexes2 = lists:filter(fun (I) ->
+ I =< LastIdx andalso
+ (case I < FstIdx of
+ true ->
+ %% TODO: this can be slow if
+ %% live indexes are mostly
+ %% non-contiguous. Create
+ %% BST for live index lookups
+ ra_seq:in(I, LiveIndexes);
+ false ->
+ true
+ end)
+ end, Indexes1),
{Entries0, MemTblNumRead, Indexes} = ra_mt:get_items(Indexes2, Mt),
ok = incr_counter(Cfg, ?C_RA_LOG_READ_MEM_TBL, MemTblNumRead),
{Entries1, Reader} = ra_log_reader:sparse_read(Reader0, Indexes, Entries0),
@@ -523,8 +591,9 @@ sparse_read(Indexes0, #?MODULE{cfg = Cfg,
%% read a list of indexes,
%% found indexes be returned in the same order as the input list of indexes
-spec partial_read([ra_index()], state(),
- fun ((ra_index(), ra_term(), ra_server:command()) -> term())
- ) ->
+ fun ((ra_index(),
+ ra_term(),
+ ra_server:command()) -> term())) ->
read_plan().
partial_read(Indexes0, #?MODULE{cfg = Cfg,
reader = Reader0,
@@ -614,115 +683,198 @@ set_last_index(Idx, #?MODULE{cfg = Cfg,
-spec handle_event(event_body(), state()) ->
{state(), [effect()]}.
-handle_event({written, _Term, {FromIdx, _ToIdx}},
- #?MODULE{last_index = LastIdx} = State)
- when FromIdx > LastIdx ->
- %% we must have reverted back, either by explicit reset or by a snapshot
- %% installation taking place whilst the WAL was processing the write
- %% Just drop the event in this case as it is stale
- {State, []};
-handle_event({written, Term, {FromIdx, ToIdx}},
+handle_event({written, Term, WrittenSeq},
#?MODULE{cfg = Cfg,
- last_written_index_term = {LastWrittenIdx0,
- _LastWrittenTerm0},
- first_index = FirstIdx} = State0)
- when FromIdx =< LastWrittenIdx0 + 1 ->
- % We need to ignore any written events for the same index
- % but in a prior term if we do not we may end up confirming
- % to a leader writes that have not yet
- % been fully flushed
- case fetch_term(ToIdx, State0) of
+ first_index = FirstIdx,
+ pending = Pend0} = State0) ->
+ %% gap detection
+ %% 1. pending has lower indexes than the ra_seq:first index in WrittenSeq
+ %% 2.
+ LastWrittenIdx = ra_seq:last(WrittenSeq),
+ case fetch_term(LastWrittenIdx, State0) of
{Term, State} when is_integer(Term) ->
- ok = put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, ToIdx),
- {State#?MODULE{last_written_index_term = {ToIdx, Term}}, []};
- {undefined, State} when ToIdx < FirstIdx ->
+ ok = put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX,
+ LastWrittenIdx),
+
+ case ra_seq:remove_prefix(WrittenSeq, Pend0) of
+ {ok, Pend} ->
+ {State#?MODULE{last_written_index_term = {LastWrittenIdx, Term},
+ pending = Pend}, []};
+ {error, not_prefix} ->
+ ?DEBUG("~ts: ~p not prefix of ~p",
+ [Cfg#cfg.log_id, WrittenSeq, Pend0]),
+ {resend_pending(State0), []}
+ end;
+ {undefined, State} when LastWrittenIdx < FirstIdx ->
% A snapshot happened before the written event came in
% This can only happen on a leader when consensus is achieved by
% followers returning appending the entry and the leader committing
% and processing a snapshot before the written event comes in.
- {State, []};
+ %
+ % At this point the items may still be in pending so need to
+ % remove them
+ {ok, Pend} = ra_seq:remove_prefix(WrittenSeq, Pend0),
+ {State#?MODULE{pending = Pend}, []};
{OtherTerm, State} ->
- %% term mismatch, let's reduce the range and try again to see
+ %% term mismatch, let's reduce the seq and try again to see
%% if any entries in the range are valid
- case ra_range:new(FromIdx, ToIdx-1) of
- undefined ->
+ case ra_seq:limit(LastWrittenIdx - 1, WrittenSeq) of
+ [] ->
?DEBUG("~ts: written event did not find term ~b for index ~b "
"found ~w",
- [State#?MODULE.cfg#cfg.log_id, Term, ToIdx, OtherTerm]),
+ [Cfg#cfg.log_id, Term, LastWrittenIdx, OtherTerm]),
{State, []};
- NextWrittenRange ->
+ NewWrittenSeq ->
%% retry with a reduced range
- handle_event({written, Term, NextWrittenRange}, State0)
+ handle_event({written, Term, NewWrittenSeq}, State0)
end
end;
-handle_event({written, _Term, {FromIdx, _}} = Evt,
- #?MODULE{cfg = #cfg{log_id = LogId},
- mem_table = Mt,
- last_written_index_term = {LastWrittenIdx, _}} = State0)
- when FromIdx > LastWrittenIdx + 1 ->
- % leaving a gap is not ok - may need to resend from mem table
- Expected = LastWrittenIdx + 1,
- MtRange = ra_mt:range(Mt),
- case ra_range:in(Expected, MtRange) of
- true ->
- ?INFO("~ts: ra_log: written gap detected at ~b expected ~b!",
- [LogId, FromIdx, Expected]),
- {resend_from(Expected, State0), []};
- false ->
- ?DEBUG("~ts: ra_log: written gap detected at ~b but is outside
- of mem table range ~w. Updating last written index to ~b!",
- [LogId, FromIdx, MtRange, Expected]),
- %% if the entry is not in the mem table we may have missed a
- %% written event due to wal crash. Accept written event by updating
- %% last written index term and recursing
- {Term, State} = fetch_term(Expected, State0),
- handle_event(Evt,
- State#?MODULE{last_written_index_term = {Expected, Term}})
- end;
+% handle_event({written, _Term, [{FromIdx, _ToIdx}]},
+% #?MODULE{last_index = LastIdx} = State)
+% when FromIdx > LastIdx ->
+% %% we must have reverted back, either by explicit reset or by a snapshot
+% %% installation taking place whilst the WAL was processing the write
+% %% Just drop the event in this case as it is stale
+% {State, []};
+% handle_event({written, Term, [{FromIdx, ToIdx}]},
+% #?MODULE{cfg = Cfg,
+% last_written_index_term = {LastWrittenIdx0,
+% _LastWrittenTerm0},
+% first_index = FirstIdx} = State0)
+% when FromIdx =< LastWrittenIdx0 + 1 ->
+% % We need to ignore any written events for the same index
+% % but in a prior term if we do not we may end up confirming
+% % to a leader writes that have not yet
+% % been fully flushed
+% case fetch_term(ToIdx, State0) of
+% {Term, State} when is_integer(Term) ->
+% ok = put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, ToIdx),
+% {State#?MODULE{last_written_index_term = {ToIdx, Term}}, []};
+% {undefined, State} when ToIdx < FirstIdx ->
+% % A snapshot happened before the written event came in
+% % This can only happen on a leader when consensus is achieved by
+% % followers returning appending the entry and the leader committing
+% % and processing a snapshot before the written event comes in.
+% {State, []};
+% {OtherTerm, State} ->
+% %% term mismatch, let's reduce the range and try again to see
+% %% if any entries in the range are valid
+% case ra_range:new(FromIdx, ToIdx-1) of
+% undefined ->
+% ?DEBUG("~ts: written event did not find term ~b for index ~b "
+% "found ~w",
+% [Cfg#cfg.log_id, Term, ToIdx, OtherTerm]),
+% {State, []};
+% NextWrittenRange ->
+% %% retry with a reduced range
+% handle_event({written, Term, [NextWrittenRange]}, State0)
+% end
+% end;
+% handle_event({written, _Term, [{FromIdx, _}]} = Evt,
+% #?MODULE{cfg = #cfg{log_id = LogId},
+% mem_table = Mt,
+% last_written_index_term = {LastWrittenIdx, _}} = State0)
+% when FromIdx > LastWrittenIdx + 1 ->
+% % leaving a gap is not ok - may need to resend from mem table
+% Expected = LastWrittenIdx + 1,
+% MtRange = ra_mt:range(Mt),
+% case ra_range:in(Expected, MtRange) of
+% true ->
+% ?INFO("~ts: ra_log: written gap detected at ~b expected ~b!",
+% [LogId, FromIdx, Expected]),
+% {resend_from(Expected, State0), []};
+% false ->
+% ?DEBUG("~ts: ra_log: written gap detected at ~b but is outside
+% of mem table range ~w. Updating last written index to ~b!",
+% [LogId, FromIdx, MtRange, Expected]),
+% %% if the entry is not in the mem table we may have missed a
+% %% written event due to wal crash. Accept written event by updating
+% %% last written index term and recursing
+% {Term, State} = fetch_term(Expected, State0),
+% handle_event(Evt,
+% State#?MODULE{last_written_index_term = {Expected, Term}})
+% end;
+% handle_event({written, Term, Written}, State) ->
+% %% simple handling of ra_seqs for now
+% case Written of
+% [I] when is_integer(I) ->
+% handle_event({written, Term, [{I, I}]}, State);
+% [I2, I] when is_integer(I) andalso
+% I + 1 == I2 ->
+% handle_event({written, Term, [{I, I2}]}, State);
+% _ ->
+% exit({sparse_written_events_not_implemented, Written})
+% end;
handle_event({segments, TidRanges, NewSegs},
#?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg,
reader = Reader0,
- mem_table = Mt0,
- readers = Readers
- } = State0) ->
+ pending = Pend0,
+ mem_table = Mt0} = State0) ->
Reader = ra_log_reader:update_segments(NewSegs, Reader0),
put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS,
ra_log_reader:segment_ref_count(Reader)),
%% the tid ranges arrive in the reverse order they were written
%% (new -> old) so we need to foldr here to process the oldest first
Mt = lists:foldr(
- fun ({Tid, Range}, Acc0) ->
- {Spec, Acc} = ra_mt:record_flushed(Tid, Range, Acc0),
- ok = ra_log_ets:execute_delete(Names, UId, Spec),
- Acc
+ fun ({Tid, Seq}, Acc0) ->
+ {Spec, Acc} = ra_mt:record_flushed(Tid, Seq, Acc0),
+ ok = ra_log_ets:execute_delete(Names, UId, Spec),
+ Acc
end, Mt0, TidRanges),
+ %% it is theoretically possible that the segment writer flush _could_
+ %% over take WAL notifications
+ %%
+ FstPend = ra_seq:first(Pend0),
+ Pend = case ra_mt:range(Mt) of
+ {Start, _End} when Start > FstPend ->
+ ra_seq:floor(Start, Pend0);
+ _ ->
+ Pend0
+ end,
State = State0#?MODULE{reader = Reader,
+ pending = Pend,
mem_table = Mt},
- case Readers of
- [] ->
- {State, []};
- _ ->
- %% HACK: but this feature is deprecated anyway
- %% Dummy pid to swallow update notifications
- Pid = spawn(fun () -> ok end),
- {State, log_update_effects(Readers, Pid, State)}
- end;
-handle_event({snapshot_written, {SnapIdx, _} = Snap, SnapKind},
+ {State, []};
+handle_event({segments_to_be_deleted, SegRefs},
+ #?MODULE{cfg = #cfg{uid = UId,
+ log_id = LogId,
+ directory = Dir,
+ counter = Counter,
+ names = Names},
+ reader = Reader} = State) ->
+ Fun = fun () ->
+ [prim_file:delete(filename:join(Dir, F))
+ || {_, F} <- SegRefs],
+ ok
+ end,
+ ActiveSegs = ra_log_reader:segment_refs(Reader) -- SegRefs,
+ #{max_size := MaxOpenSegments} = ra_log_reader:info(Reader),
+ % close all open segments
+ ok = ra_log_reader:close(Reader),
+ ?DEBUG("~ts: ~b obsolete segments - remaining: ~b",
+ [LogId, length(SegRefs), length(ActiveSegs)]),
+ %% open a new segment with the new max open segment value
+ {State#?MODULE{reader = ra_log_reader:init(UId, Dir, MaxOpenSegments,
+ random,
+ ActiveSegs, Names, Counter)},
+
+ [{bg_work, Fun, fun (_Err) -> ok end}]};
+handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind},
#?MODULE{cfg = #cfg{uid = UId,
names = Names} = Cfg,
first_index = FstIdx,
last_index = LstIdx,
mem_table = Mt0,
+ pending = Pend0,
last_written_index_term = {LastWrittenIdx, _} = LWIdxTerm0,
snapshot_state = SnapState0} = State0)
%% only update snapshot if it is newer than the last snapshot
when SnapIdx >= FstIdx ->
- SnapState1 = ra_snapshot:complete_snapshot(Snap, SnapKind, SnapState0),
+ SnapState1 = ra_snapshot:complete_snapshot(Snap, SnapKind, LiveIndexes,
+ SnapState0),
case SnapKind of
snapshot ->
put_counter(Cfg, ?C_RA_SVR_METRIC_SNAPSHOT_INDEX, SnapIdx),
- % delete any segments outside of first_index
- {State, Effects0} = delete_segments(SnapIdx, State0),
%% Delete old snapshot files. This is done as an effect
%% so that if an old snapshot is still being replicated
%% the cleanup can be delayed until it is safe.
@@ -735,25 +887,46 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, SnapKind},
CPEffects = [{delete_snapshot,
ra_snapshot:directory(SnapState, checkpoint),
Checkpoint} || Checkpoint <- Checkpoints],
- Effects1 = [DeleteCurrentSnap | CPEffects] ++ Effects0,
+ Effects0 = [DeleteCurrentSnap | CPEffects],
- {LWIdxTerm, Effects} =
+ LWIdxTerm =
case LastWrittenIdx > SnapIdx of
true ->
- {LWIdxTerm0, Effects1};
+ LWIdxTerm0;
false ->
- {Snap, Effects1}
+ Snap
end,
+
+ CompEffs = schedule_compaction(SnapIdx, State0),
+ State = State0,
+
+
+ %% remove all pending below smallest live index as the wal
+ %% may not write them
+ %% TODO: test that a written even can still be processed if it
+ %% contains lower indexes than pending
+ SmallestLiveIdx = case ra_seq:first(LiveIndexes) of
+ undefined ->
+ SnapIdx + 1;
+ I ->
+ I
+ end,
+ %$% TODO: optimise - ra_seq:floor/2 is O(n),
+ Pend = ra_seq:floor(SmallestLiveIdx, Pend0),
+ %% delete from mem table
%% this will race with the segment writer but if the
%% segwriter detects a missing index it will query the snaphost
%% state and if that is higher it will resume flush
- {Spec, Mt1} = ra_mt:set_first(SnapIdx + 1, Mt0),
+ {Spec, Mt1} = ra_mt:set_first(SmallestLiveIdx, Mt0),
ok = exec_mem_table_delete(Names, UId, Spec),
+ Effects = CompEffs ++ Effects0, % ++ Effects1,
{State#?MODULE{first_index = SnapIdx + 1,
last_index = max(LstIdx, SnapIdx),
last_written_index_term = LWIdxTerm,
mem_table = Mt1,
+ pending = Pend,
+ live_indexes = LiveIndexes,
snapshot_state = SnapState}, Effects};
checkpoint ->
put_counter(Cfg, ?C_RA_SVR_METRIC_CHECKPOINT_INDEX, SnapIdx),
@@ -765,7 +938,7 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, SnapKind},
CP} || CP <- CPs],
{State0#?MODULE{snapshot_state = SnapState}, Effects}
end;
-handle_event({snapshot_written, {Idx, Term} = Snap, SnapKind},
+handle_event({snapshot_written, {Idx, Term} = Snap, _Indexes, SnapKind},
#?MODULE{cfg =#cfg{log_id = LogId},
snapshot_state = SnapState} = State0) ->
%% if the snapshot/checkpoint is stale we just want to delete it
@@ -777,16 +950,20 @@ handle_event({snapshot_written, {Idx, Term} = Snap, SnapKind},
ra_snapshot:directory(SnapState, SnapKind),
Snap}],
{State0, Effects};
+handle_event({snapshot_error, Snap, SnapKind, Error},
+ #?MODULE{cfg =#cfg{log_id = LogId},
+ snapshot_state = SnapState0} = State0) ->
+ ?INFO("~ts: snapshot error for ~w ~s ", [LogId, Snap, SnapKind]),
+ SnapState = ra_snapshot:handle_error(Snap, Error, SnapState0),
+ {State0#?MODULE{snapshot_state = SnapState}, []};
handle_event({resend_write, Idx},
- #?MODULE{cfg =#cfg{log_id = LogId}} = State) ->
+ #?MODULE{cfg = #cfg{log_id = LogId}} = State) ->
% resend missing entries from mem tables.
?INFO("~ts: ra_log: wal requested resend from ~b",
[LogId, Idx]),
{resend_from(Idx, State), []};
-handle_event({down, Pid, _Info},
- #?MODULE{readers = Readers} =
- State) ->
- {State#?MODULE{readers = lists:delete(Pid, Readers)}, []}.
+handle_event({down, _Pid, _Info}, #?MODULE{} = State) ->
+ {State, []}.
-spec next_index(state()) -> ra_index().
next_index(#?MODULE{last_index = LastIdx}) ->
@@ -795,11 +972,11 @@ next_index(#?MODULE{last_index = LastIdx}) ->
-spec fetch(ra_index(), state()) ->
{option(log_entry()), state()}.
fetch(Idx, State0) ->
- case fold(Idx, Idx, fun(E, Acc) -> [E | Acc] end, [], State0) of
- {[], State} ->
- {undefined, State};
+ case sparse_read([Idx], State0) of
{[Entry], State} ->
- {Entry, State}
+ {Entry, State};
+ {[], State} ->
+ {undefined, State}
end.
-spec fetch_term(ra_index(), state()) ->
@@ -825,32 +1002,45 @@ snapshot_state(State) ->
set_snapshot_state(SnapState, State) ->
State#?MODULE{snapshot_state = SnapState}.
--spec install_snapshot(ra_idxterm(), ra_snapshot:state(), state()) ->
- {state(), effects()}.
-install_snapshot({SnapIdx, SnapTerm} = IdxTerm, SnapState0,
+-spec install_snapshot(ra_idxterm(), ra_snapshot:state(), module(), state()) ->
+ {ra_snapshot:meta(), MacState :: term(), state(), effects()}.
+install_snapshot({SnapIdx, SnapTerm} = IdxTerm, SnapState0, MacMod,
#?MODULE{cfg = #cfg{uid = UId,
names = Names} = Cfg,
- mem_table = Mt0
- } = State0) ->
+ mem_table = Mt0} = State0)
+ when is_atom(MacMod) ->
ok = incr_counter(Cfg, ?C_RA_LOG_SNAPSHOTS_INSTALLED, 1),
ok = put_counter(Cfg, ?C_RA_SVR_METRIC_SNAPSHOT_INDEX, SnapIdx),
put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, SnapIdx),
put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, SnapIdx),
- {State, Effs} = delete_segments(SnapIdx, State0),
+
+ CompEffs = schedule_compaction(SnapIdx, State0),
+
{SnapState, Checkpoints} =
ra_snapshot:take_older_checkpoints(SnapIdx, SnapState0),
CPEffects = [{delete_snapshot,
ra_snapshot:directory(SnapState, checkpoint),
Checkpoint} || Checkpoint <- Checkpoints],
- {Spec, Mt} = ra_mt:set_first(SnapIdx, Mt0),
+ %% TODO: can't really do this as the mem table may contain live indexes
+ %% below the snap idx
+ {Spec, Mt} = ra_mt:set_first(SnapIdx + 1, Mt0),
ok = exec_mem_table_delete(Names, UId, Spec),
- {State#?MODULE{snapshot_state = SnapState,
- first_index = SnapIdx + 1,
- last_index = SnapIdx,
- last_term = SnapTerm,
- mem_table = Mt,
- last_written_index_term = IdxTerm},
- Effs ++ CPEffects}.
+ State = State0#?MODULE{snapshot_state = SnapState,
+ first_index = SnapIdx + 1,
+ last_index = SnapIdx,
+ last_term = SnapTerm,
+ mem_table = Mt,
+ last_written_index_term = IdxTerm},
+ {Meta, MacState} = recover_snapshot(State),
+ LiveIndexes = ra_machine:live_indexes(MacMod, MacState),
+ %% TODO: it is not safe to write the indexes _after_ if we then treat
+ %% the persisted indexes as authoritative as if we crash in between
+ %% it may compact segments that still contain live indexes
+ SnapDir = ra_snapshot:current_snapshot_dir(SnapState),
+ ok = ra_snapshot:write_indexes(SnapDir, LiveIndexes),
+ {Meta, MacState, State#?MODULE{live_indexes = LiveIndexes},
+ CompEffs ++ CPEffects}.
+
-spec recover_snapshot(State :: state()) ->
option({ra_snapshot:meta(), term()}).
@@ -866,25 +1056,28 @@ recover_snapshot(#?MODULE{snapshot_state = SnapState}) ->
snapshot_index_term(#?MODULE{snapshot_state = SS}) ->
ra_snapshot:current(SS).
--spec update_release_cursor(Idx :: ra_index(), Cluster :: ra_cluster(),
- MacVersion :: ra_machine:version(),
+-spec update_release_cursor(Idx :: ra_index(),
+ Cluster :: ra_cluster(),
+ MacModule :: module(),
MacState :: term(), State :: state()) ->
{state(), effects()}.
-update_release_cursor(Idx, Cluster, MacVersion, MacState, State) ->
- suggest_snapshot(snapshot, Idx, Cluster, MacVersion, MacState, State).
+update_release_cursor(Idx, Cluster, MacModule, MacState, State)
+ when is_atom(MacModule) ->
+ suggest_snapshot(snapshot, Idx, Cluster, MacModule, MacState, State).
-spec checkpoint(Idx :: ra_index(), Cluster :: ra_cluster(),
- MacVersion :: ra_machine:version(),
+ MacModule :: module(),
MacState :: term(), State :: state()) ->
{state(), effects()}.
-checkpoint(Idx, Cluster, MacVersion, MacState, State) ->
- suggest_snapshot(checkpoint, Idx, Cluster, MacVersion, MacState, State).
+checkpoint(Idx, Cluster, MacModule, MacState, State)
+ when is_atom(MacModule) ->
+ suggest_snapshot(checkpoint, Idx, Cluster, MacModule, MacState, State).
-suggest_snapshot(SnapKind, Idx, Cluster, MacVersion, MacState,
+suggest_snapshot(SnapKind, Idx, Cluster, MacModule, MacState,
#?MODULE{snapshot_state = SnapshotState} = State) ->
case ra_snapshot:pending(SnapshotState) of
undefined ->
- suggest_snapshot0(SnapKind, Idx, Cluster, MacVersion, MacState, State);
+ suggest_snapshot0(SnapKind, Idx, Cluster, MacModule, MacState, State);
_ ->
%% Only one snapshot or checkpoint may be written at a time to
%% prevent excessive I/O usage.
@@ -894,7 +1087,7 @@ suggest_snapshot(SnapKind, Idx, Cluster, MacVersion, MacState,
promote_checkpoint(Idx, #?MODULE{cfg = Cfg,
snapshot_state = SnapState0} = State) ->
case ra_snapshot:pending(SnapState0) of
- {_WriterPid, _IdxTerm, snapshot} ->
+ {_IdxTerm, snapshot} ->
%% If we're currently writing a snapshot, skip promoting a
%% checkpoint.
{State, []};
@@ -930,13 +1123,7 @@ tick(Now, #?MODULE{cfg = #cfg{wal = Wal},
State
end.
-suggest_snapshot0(SnapKind, Idx, Cluster, MacVersion, MacState, State0) ->
- ClusterServerIds = maps:map(fun (_, V) ->
- maps:with([voter_status], V)
- end, Cluster),
- Meta = #{index => Idx,
- cluster => ClusterServerIds,
- machine_version => MacVersion},
+suggest_snapshot0(SnapKind, Idx, Cluster, MacModule, MacState, State0) ->
case should_snapshot(SnapKind, Idx, State0) of
true ->
% TODO: here we use the current cluster configuration in
@@ -954,7 +1141,15 @@ suggest_snapshot0(SnapKind, Idx, Cluster, MacVersion, MacState, State0) ->
{undefined, _} ->
{State0, []};
{Term, State} ->
- write_snapshot(Meta#{term => Term}, MacState,
+ ClusterServerIds =
+ maps:map(fun (_, V) ->
+ maps:with([voter_status], V)
+ end, Cluster),
+ MachineVersion = ra_machine:version(MacModule),
+ Meta = #{index => Idx,
+ cluster => ClusterServerIds,
+ machine_version => MachineVersion},
+ write_snapshot(Meta#{term => Term}, MacModule, MacState,
SnapKind, State)
end;
false ->
@@ -1032,7 +1227,8 @@ overview(#?MODULE{last_index = LastIndex,
snapshot_state = SnapshotState,
reader = Reader,
last_wal_write = {_LastPid, LastMs},
- mem_table = Mt
+ mem_table = Mt,
+ pending = Pend
}) ->
CurrSnap = ra_snapshot:current(SnapshotState),
#{type => ?MODULE,
@@ -1048,9 +1244,9 @@ overview(#?MODULE{last_index = LastIndex,
{I, _} -> I
end,
snapshot_term => case CurrSnap of
- undefined -> undefined;
- {_, T} -> T
- end,
+ undefined -> undefined;
+ {_, T} -> T
+ end,
latest_checkpoint_index =>
case ra_snapshot:latest_checkpoint(SnapshotState) of
undefined -> undefined;
@@ -1058,7 +1254,8 @@ overview(#?MODULE{last_index = LastIndex,
end,
mem_table_range => ra_mt:range(Mt),
mem_table_info => ra_mt:info(Mt),
- last_wal_write => LastMs
+ last_wal_write => LastMs,
+ num_pending => ra_seq:length(Pend)
}.
-spec write_config(ra_server:config(), state()) -> ok.
@@ -1089,24 +1286,12 @@ read_config(Dir) ->
delete_everything(#?MODULE{cfg = #cfg{uid = UId,
names = Names,
directory = Dir},
- snapshot_state = SnapState} = Log) ->
+ snapshot_state = _SnapState} = Log) ->
_ = close(Log),
%% if there is a snapshot process pending it could cause the directory
%% deletion to fail, best kill the snapshot process first
ok = ra_log_ets:delete_mem_tables(Names, UId),
- catch ets:delete(ra_log_snapshot_state, UId),
- case ra_snapshot:pending(SnapState) of
- {Pid, _, _} ->
- case is_process_alive(Pid) of
- true ->
- exit(Pid, kill),
- ok;
- false ->
- ok
- end;
- _ ->
- ok
- end,
+ catch ra_log_snapshot_state:delete(ra_log_snapshot_state, UId),
try ra_lib:recursive_delete(Dir) of
ok -> ok
catch
@@ -1134,57 +1319,67 @@ release_resources(MaxOpenSegments,
AccessPattern,
ActiveSegs, Names, Counter)}.
--spec register_reader(pid(), state()) ->
- {state(), effects()}.
-register_reader(Pid, #?MODULE{cfg = #cfg{uid = UId,
- directory = Dir,
- names = Names},
- reader = Reader,
- readers = Readers} = State) ->
- SegRefs = ra_log_reader:segment_refs(Reader),
- NewReader = ra_log_reader:init(UId, Dir, 1, SegRefs, Names),
- {State#?MODULE{readers = [Pid | Readers]},
- [{reply, {ok, NewReader}},
- {monitor, process, log, Pid}]}.
-
-readers(#?MODULE{readers = Readers}) ->
- Readers.
-
%%% Local functions
-log_update_effects(Pids, ReplyPid, #?MODULE{first_index = Idx,
- reader = Reader}) ->
- SegRefs = ra_log_reader:segment_refs(Reader),
- [{send_msg, P, {ra_log_update, ReplyPid, Idx, SegRefs},
- [ra_event, local]} || P <- Pids].
+schedule_compaction(SnapIdx, #?MODULE{cfg = #cfg{},
+ snapshot_state = SnapState,
+ reader = Reader0}) ->
+ case ra_log_reader:segment_refs(Reader0) of
+ [] ->
+ [];
+ [_ | Compactable] ->
+ %% never compact the current segment
+ %% only take those who have a range lower than the snapshot index as
+ %% we never want to compact more than that
+ SegRefs = lists:takewhile(fun ({{_Start, End}, _}) ->
+ End =< SnapIdx
+ end, lists:reverse(Compactable)),
+ SnapDir = ra_snapshot:current_snapshot_dir(SnapState),
+
+ Self = self(),
+ Fun = fun () ->
+ {ok, Indexes} = ra_snapshot:indexes(SnapDir),
+ {Delete, _} = lists:foldl(
+ fun ({Range, _} = S, {Del, Keep}) ->
+ case ra_seq:in_range(Range, Indexes) of
+ [] ->
+ {[S | Del], Keep};
+ _ ->
+ {Del, [S | Keep]}
+ end
+ end, {[], []}, SegRefs),
+ %% need to update the ra_servers list of seg refs _before_
+ %% the segments can actually be deleted
+ Self ! {ra_log_event,
+ {segments_to_be_deleted, Delete}},
+ ok
+ end,
+
+ [{bg_work, Fun, fun (_Err) -> ok end}]
+ end.
%% deletes all segments where the last index is lower than
%% the Idx argument
-delete_segments(SnapIdx, #?MODULE{cfg = #cfg{log_id = LogId,
- segment_writer = SegWriter,
- uid = UId} = Cfg,
- readers = Readers,
- reader = Reader0} = State0) ->
- case ra_log_reader:update_first_index(SnapIdx + 1, Reader0) of
- {Reader, []} ->
- State = State0#?MODULE{reader = Reader},
- {State, log_update_effects(Readers, undefined, State)};
- {Reader, [Pivot | _] = Obsolete} ->
- Pid = spawn(
- fun () ->
- ok = log_update_wait_n(length(Readers)),
- ok = ra_log_segment_writer:truncate_segments(SegWriter,
- UId, Pivot)
- end),
- NumActive = ra_log_reader:segment_ref_count(Reader),
- ?DEBUG("~ts: ~b obsolete segments at ~b - remaining: ~b, pivot ~0p",
- [LogId, length(Obsolete), SnapIdx, NumActive, Pivot]),
- put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS, NumActive),
- State = State0#?MODULE{reader = Reader},
- {State, log_update_effects(Readers, Pid, State)}
- end.
+% delete_segments(SnapIdx, #?MODULE{cfg = #cfg{log_id = LogId,
+% segment_writer = SegWriter,
+% uid = UId} = Cfg,
+% reader = Reader0} = State0) ->
+% case ra_log_reader:update_first_index(SnapIdx + 1, Reader0) of
+% {Reader, []} ->
+% State = State0#?MODULE{reader = Reader},
+% {State, []};
+% {Reader, [Pivot | _] = Obsolete} ->
+% ok = ra_log_segment_writer:truncate_segments(SegWriter,
+% UId, Pivot),
+% NumActive = ra_log_reader:segment_ref_count(Reader),
+% ?DEBUG("~ts: ~b obsolete segments at ~b - remaining: ~b, pivot ~0p",
+% [LogId, length(Obsolete), SnapIdx, NumActive, Pivot]),
+% put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS, NumActive),
+% State = State0#?MODULE{reader = Reader},
+% {State, []}
+% end.
%% unly used by resend to wal functionality and doesn't update the mem table
wal_rewrite(#?MODULE{cfg = #cfg{uid = UId,
@@ -1204,19 +1399,20 @@ wal_rewrite(#?MODULE{cfg = #cfg{uid = UId,
wal_write_batch(#?MODULE{cfg = #cfg{uid = UId,
wal = Wal} = Cfg,
+ pending = Pend0,
mem_table = Mt0} = State,
Entries) ->
WriterId = {UId, self()},
%% all entries in a transaction are written to the same tid
Tid = ra_mt:tid(Mt0),
- {WalCommands, Num} =
- lists:foldl(fun ({Idx, Term, Cmd0}, {WC, N}) ->
+ {WalCommands, Num, Pend} =
+ lists:foldl(fun ({Idx, Term, Cmd0}, {WC, N, P}) ->
Cmd = {ttb, term_to_iovec(Cmd0)},
- WalC = {append, WriterId, Tid, Idx, Term, Cmd},
- {[WalC | WC], N+1}
- end, {[], 0}, Entries),
+ WalC = {append, WriterId, Tid, Idx-1, Idx, Term, Cmd},
+ {[WalC | WC], N+1, ra_seq:append(Idx, P)}
+ end, {[], 0, Pend0}, Entries),
- [{_, _, _, LastIdx, LastTerm, _} | _] = WalCommands,
+ [{_, _, _, _PrevIdx, LastIdx, LastTerm, _} | _] = WalCommands,
{_, Mt} = ra_mt:commit(Mt0),
put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, LastIdx),
ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, Num),
@@ -1225,7 +1421,8 @@ wal_write_batch(#?MODULE{cfg = #cfg{uid = UId,
{ok, State#?MODULE{last_index = LastIdx,
last_term = LastTerm,
last_wal_write = {Pid, now_ms()},
- mem_table = Mt}};
+ mem_table = Mt,
+ pending = Pend}};
{error, wal_down} = Err ->
%% if we get there the entry has already been inserted
%% into the mem table but never reached the wal
@@ -1234,15 +1431,15 @@ wal_write_batch(#?MODULE{cfg = #cfg{uid = UId,
end.
maybe_append_first_entry(State0 = #?MODULE{last_index = -1}) ->
- State = append({0, 0, undefined}, State0),
+ State1 = append({0, 0, undefined}, State0),
receive
- {ra_log_event, {written, 0, {0, 0}}} ->
- ok
+ {ra_log_event, {written, 0, [0]} = Evt} ->
+ State2 = State1#?MODULE{first_index = 0},
+ {State, _Effs} = handle_event(Evt, State2),
+ State
after 60000 ->
exit({?FUNCTION_NAME, timeout})
- end,
- State#?MODULE{first_index = 0,
- last_written_index_term = {0, 0}};
+ end;
maybe_append_first_entry(State) ->
State.
@@ -1256,6 +1453,23 @@ resend_from(Idx, #?MODULE{cfg = #cfg{uid = UId}} = State0) ->
State0
end.
+resend_pending(#?MODULE{cfg = Cfg,
+ last_resend_time = undefined,
+ pending = Pend,
+ mem_table = Mt} = State) ->
+ ?DEBUG("~ts: ra_log: resending from ~b to ~b mt ~p",
+ [State#?MODULE.cfg#cfg.log_id, ra_seq:first(Pend),
+ ra_seq:last(Pend), ra_mt:range(Mt)]),
+ ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_RESENDS, ra_seq:length(Pend)),
+ ra_seq:fold(fun (I, Acc) ->
+ {I, T, C} = ra_mt:lookup(I, Mt),
+ Tid = ra_mt:tid_for(I, T, Mt),
+ wal_rewrite(Acc, Tid, {I, T, C})
+ end,
+ State#?MODULE{last_resend_time = {erlang:system_time(seconds),
+ whereis(Cfg#cfg.wal)}},
+ Pend).
+
resend_from0(Idx, #?MODULE{cfg = Cfg,
last_index = LastIdx,
last_resend_time = undefined,
@@ -1314,7 +1528,7 @@ stage_entries0(Cfg, [Entry | Rem], Mt0) ->
-write_snapshot(Meta, MacRef, SnapKind,
+write_snapshot(Meta, MacModule, MacState, SnapKind,
#?MODULE{cfg = Cfg,
snapshot_state = SnapState0} = State) ->
Counter = case SnapKind of
@@ -1322,8 +1536,8 @@ write_snapshot(Meta, MacRef, SnapKind,
checkpoint -> ?C_RA_LOG_CHECKPOINTS_WRITTEN
end,
ok = incr_counter(Cfg, Counter, 1),
- {SnapState, Effects} = ra_snapshot:begin_snapshot(Meta, MacRef, SnapKind,
- SnapState0),
+ {SnapState, Effects} = ra_snapshot:begin_snapshot(Meta, MacModule, MacState,
+ SnapKind, SnapState0),
{State#?MODULE{snapshot_state = SnapState}, Effects}.
recover_ranges(UId, MtRange, SegWriter) ->
@@ -1359,28 +1573,29 @@ pick_range([{Fst, _Lst} | Tail], {CurFst, CurLst}) ->
%% TODO: implement synchronous writes using gen_batch_server:call/3
await_written_idx(Idx, Term, Log0) ->
- IDX = Idx,
receive
- {ra_log_event, {written, Term, {_, IDX}} = Evt} ->
+ {ra_log_event, {written, Term, _Seq} = Evt} ->
{Log, _} = handle_event(Evt, Log0),
- Log;
- {ra_log_event, {written, _, _} = Evt} ->
- {Log, _} = handle_event(Evt, Log0),
- await_written_idx(Idx, Term, Log)
+ case last_written(Log) of
+ {Idx, Term} ->
+ Log;
+ _ ->
+ await_written_idx(Idx, Term, Log)
+ end
after ?LOG_APPEND_TIMEOUT ->
throw(ra_log_append_timeout)
end.
-log_update_wait_n(0) ->
- ok;
-log_update_wait_n(N) ->
- receive
- ra_log_update_processed ->
- log_update_wait_n(N - 1)
- after 1500 ->
- %% just go ahead anyway
- ok
- end.
+% log_update_wait_n(0) ->
+% ok;
+% log_update_wait_n(N) ->
+% receive
+% ra_log_update_processed ->
+% log_update_wait_n(N - 1)
+% after 1500 ->
+% %% just go ahead anyway
+% ok
+% end.
incr_counter(#cfg{counter = Cnt}, Ix, N) when Cnt =/= undefined ->
counters:add(Cnt, Ix, N);
diff --git a/src/ra_log_reader.erl b/src/ra_log_reader.erl
index ea171e69..f87e7f36 100644
--- a/src/ra_log_reader.erl
+++ b/src/ra_log_reader.erl
@@ -13,7 +13,7 @@
init/7,
close/1,
update_segments/2,
- handle_log_update/2,
+ handle_compaction/2,
segment_refs/1,
segment_ref_count/1,
range/1,
@@ -23,7 +23,8 @@
sparse_read/3,
read_plan/2,
exec_read_plan/6,
- fetch_term/2
+ fetch_term/2,
+ info/1
]).
-include("ra.hrl").
@@ -109,7 +110,7 @@ update_segments(NewSegmentRefs,
SegmentRefs0 = ra_lol:to_list(SegRefs0),
SegmentRefsComp = compact_segrefs(NewSegmentRefs, SegmentRefs0),
- SegmentRefsCompRev = lists:reverse(SegmentRefsComp ),
+ SegmentRefsCompRev = lists:reverse(SegmentRefsComp),
SegRefs = ra_lol:from_list(fun seg_ref_gt/2, SegmentRefsCompRev),
Range = case SegmentRefsComp of
[{{_, L}, _} | _] ->
@@ -131,19 +132,20 @@ update_segments(NewSegmentRefs,
range = Range,
open_segments = Open}.
--spec handle_log_update({ra_log_update, undefined | pid(), ra_index(),
- [segment_ref()]}, state()) -> state().
-handle_log_update({ra_log_update, From, _FstIdx, SegRefs},
- #?STATE{open_segments = Open0} = State) ->
+-record(log_compaction_result,
+ {%range :: ra:range(),
+ deleted :: [segment_ref()],
+ new :: [segment_ref()]}).
+-spec handle_compaction(#log_compaction_result{}, state()) -> state().
+handle_compaction(#log_compaction_result{deleted = Deleted,
+ new = New},
+ #?STATE{open_segments = Open0,
+ segment_refs = SegRefs0} = State) ->
+ SegmentRefs0 = ra_lol:to_list(SegRefs0),
+ SegmentRefs = lists:sort((SegmentRefs0 -- Deleted) ++ New),
Open = ra_flru:evict_all(Open0),
- case From of
- undefined -> ok;
- _ ->
- %% reply to the updater process
- From ! ra_log_update_processed
- end,
State#?MODULE{segment_refs = ra_lol:from_list(fun seg_ref_gt/2,
- lists:reverse(SegRefs)),
+ lists:reverse(SegmentRefs)),
open_segments = Open}.
-spec update_first_index(ra_index(), state()) ->
@@ -257,6 +259,12 @@ fetch_term(Idx, #?STATE{cfg = #cfg{} = Cfg} = State0) ->
incr_counter(Cfg, ?C_RA_LOG_FETCH_TERM, 1),
segment_term_query(Idx, State0).
+-spec info(state()) -> map().
+info(#?STATE{cfg = #cfg{} = _Cfg,
+ open_segments = Open} = State) ->
+ #{max_size => ra_flru:max_size(Open),
+ num_segments => segment_ref_count(State)
+ }.
%% LOCAL
segment_read_plan(_SegRefs, [], Acc) ->
diff --git a/src/ra_log_segment_writer.erl b/src/ra_log_segment_writer.erl
index 7e94a778..d2e64e18 100644
--- a/src/ra_log_segment_writer.erl
+++ b/src/ra_log_segment_writer.erl
@@ -65,15 +65,14 @@ start_link(#{name := Name} = Config) ->
gen_server:start_link({local, Name}, ?MODULE, [Config], []).
-spec accept_mem_tables(atom() | pid(),
- #{ra_uid() => [{ets:tid(), ra:range()}]},
- string()) ->
- ok.
+ #{ra_uid() => [{ets:tid(), ra_seq:state()}]},
+ string()) -> ok.
accept_mem_tables(_SegmentWriter, Tables, undefined)
when map_size(Tables) == 0 ->
ok;
-accept_mem_tables(SegmentWriter, Tables, WalFile)
- when is_map(Tables) ->
- gen_server:cast(SegmentWriter, {mem_tables, Tables, WalFile}).
+accept_mem_tables(SegmentWriter, UIdTidRanges, WalFile)
+ when is_map(UIdTidRanges) ->
+ gen_server:cast(SegmentWriter, {mem_tables, UIdTidRanges, WalFile}).
-spec truncate_segments(atom() | pid(), ra_uid(), ra_log:segment_ref()) -> ok.
truncate_segments(SegWriter, Who, SegRef) ->
@@ -135,10 +134,11 @@ segments_for(UId, #state{data_dir = DataDir}) ->
Dir = filename:join(DataDir, ra_lib:to_list(UId)),
segment_files(Dir).
-handle_cast({mem_tables, Ranges, WalFile}, #state{data_dir = Dir,
- system = System} = State) ->
+handle_cast({mem_tables, UIdTidRanges, WalFile},
+ #state{data_dir = Dir,
+ system = System} = State) ->
T1 = erlang:monotonic_time(),
- ok = counters:add(State#state.counter, ?C_MEM_TABLES, map_size(Ranges)),
+ ok = counters:add(State#state.counter, ?C_MEM_TABLES, map_size(UIdTidRanges)),
#{names := Names} = ra_system:fetch(System),
Degree = erlang:system_info(schedulers),
%% TODO: refactor to make better use of time where each uid has an
@@ -157,7 +157,7 @@ handle_cast({mem_tables, Ranges, WalFile}, #state{data_dir = Dir,
ok = ra_log_ets:delete_mem_tables(Names, UId),
Acc
end
- end, [], Ranges),
+ end, [], UIdTidRanges),
_ = [begin
{_, Failures} = ra_lib:partition_parallel(
@@ -261,40 +261,41 @@ get_overview(#state{data_dir = Dir,
#{data_dir => Dir,
segment_conf => Conf}.
-flush_mem_table_ranges({ServerUId, TidRanges0},
+flush_mem_table_ranges({ServerUId, TidSeqs0},
#state{system = System} = State) ->
- SnapIdx = snap_idx(ServerUId),
+ SmallestIdx = smallest_live_idx(ServerUId),
%% TidRanges arrive here sorted new -> old.
%% truncate and limit all ranges to create a contiguous non-overlapping
%% list of tid ranges to flush to disk
%% now TidRanges are sorted old -> new, i.e the correct order of
%% processing
- TidRanges = lists:foldl(
- fun ({T, Range0}, []) ->
- case ra_range:truncate(SnapIdx, Range0) of
- undefined ->
- [];
- Range ->
- [{T, Range}]
- end;
- ({T, Range0}, [{_T, {Start, _}} | _] = Acc) ->
- Range1 = ra_range:truncate(SnapIdx, Range0),
- case ra_range:limit(Start, Range1) of
- undefined ->
- Acc;
- Range ->
- [{T, Range} | Acc]
- end
- end, [], TidRanges0),
+ TidSeqs = lists:foldl(
+ fun ({T, Seq0}, []) ->
+ case ra_seq:floor(SmallestIdx, Seq0) of
+ [] ->
+ [];
+ Seq ->
+ [{T, Seq}]
+ end;
+ ({T, Seq0}, [{_T, PrevSeq} | _] = Acc) ->
+ Start = ra_seq:first(PrevSeq),
+ Seq1 = ra_seq:floor(SmallestIdx, Seq0),
+ case ra_seq:limit(Start, Seq1) of
+ [] ->
+ Acc;
+ Seq ->
+ [{T, Seq} | Acc]
+ end
+ end, [], TidSeqs0),
SegRefs0 = lists:append(
lists:reverse(
%% segrefs are returned in appended order so new -> old
%% so we need to reverse them so that the final appended list
%% of segrefs is in the old -> new order
- [flush_mem_table_range(ServerUId, TidRange, State)
- || TidRange <- TidRanges])),
+ [flush_mem_table_range(ServerUId, TidSeq, State)
+ || TidSeq <- TidSeqs])),
%% compact cases where a segment was appended in a subsequent call to
%% flush_mem_table_range
@@ -308,14 +309,14 @@ flush_mem_table_ranges({ServerUId, TidRanges0},
[Seg | Acc]
end, [], SegRefs0)),
- ok = send_segments(System, ServerUId, TidRanges0, SegRefs),
+ ok = send_segments(System, ServerUId, TidSeqs0, SegRefs),
ok.
-flush_mem_table_range(ServerUId, {Tid, {StartIdx0, EndIdx}},
+flush_mem_table_range(ServerUId, {Tid, Seq},
#state{data_dir = DataDir,
segment_conf = SegConf} = State) ->
Dir = filename:join(DataDir, binary_to_list(ServerUId)),
- StartIdx = start_index(ServerUId, StartIdx0),
+ % StartIdx = start_index(ServerUId, StartIdx0),
case open_file(Dir, SegConf) of
enoent ->
?DEBUG("segment_writer: skipping segment as directory ~ts does "
@@ -324,8 +325,7 @@ flush_mem_table_range(ServerUId, {Tid, {StartIdx0, EndIdx}},
%% clean up the tables for this process
[];
Segment0 ->
- case append_to_segment(ServerUId, Tid, StartIdx, EndIdx,
- Segment0, State) of
+ case append_to_segment(ServerUId, Tid, Seq, Segment0, State) of
undefined ->
?WARN("segment_writer: skipping segments for ~w as
directory ~ts disappeared whilst writing",
@@ -352,15 +352,10 @@ flush_mem_table_range(ServerUId, {Tid, {StartIdx0, EndIdx}},
end.
start_index(ServerUId, StartIdx0) ->
- max(snap_idx(ServerUId) + 1, StartIdx0).
+ max(smallest_live_idx(ServerUId), StartIdx0).
-snap_idx(ServerUId) ->
- case ets:lookup(ra_log_snapshot_state, ServerUId) of
- [{_, SnapIdx}] ->
- SnapIdx;
- _ ->
- -1
- end.
+smallest_live_idx(ServerUId) ->
+ ra_log_snapshot_state:smallest(ra_log_snapshot_state, ServerUId).
send_segments(System, ServerUId, TidRanges, SegRefs) ->
case ra_directory:pid_of(System, ServerUId) of
@@ -371,23 +366,33 @@ send_segments(System, ServerUId, TidRanges, SegRefs) ->
[ServerUId, "No Pid"]),
%% delete from the memtable on the non-running server's behalf
[begin
- _ = catch ra_mt:delete({range, Tid, Range})
- end || {Tid, Range} <- TidRanges],
+ %% TODO: HACK: this is a hack to get a full range out of a
+ %% sequent, ideally the mt should take the ra_seq and
+ %% delete from that
+ _ = catch ra_mt:delete({indexes, Tid, Seq})
+ end || {Tid, Seq} <- TidRanges],
ok;
Pid ->
Pid ! {ra_log_event, {segments, TidRanges, SegRefs}},
ok
end.
-append_to_segment(UId, Tid, StartIdx0, EndIdx, Seg, State) ->
- StartIdx = start_index(UId, StartIdx0),
- % EndIdx + 1 because FP
- append_to_segment(UId, Tid, StartIdx, EndIdx+1, Seg, [], State).
-
-append_to_segment(_, _, StartIdx, EndIdx, Seg, Closed, _State)
- when StartIdx >= EndIdx ->
+append_to_segment(UId, Tid, Seq0, Seg, State) ->
+ FirstIdx = case ra_seq:first(Seq0) of
+ undefined ->
+ 0;
+ Fst ->
+ Fst
+ end,
+ StartIdx = start_index(UId, FirstIdx),
+ %% TODO combine flor and iterator into one operation
+ Seq = ra_seq:floor(StartIdx, Seq0),
+ SeqIter = ra_seq:iterator(Seq),
+ append_to_segment(UId, Tid, ra_seq:next(SeqIter), Seg, [], State).
+
+append_to_segment(_, _, end_of_seq, Seg, Closed, _State) ->
{Seg, Closed};
-append_to_segment(UId, Tid, Idx, EndIdx, Seg0, Closed, State) ->
+append_to_segment(UId, Tid, {Idx, SeqIter} = Cur, Seg0, Closed, State) ->
try ets:lookup(Tid, Idx) of
[] ->
StartIdx = start_index(UId, Idx),
@@ -395,7 +400,7 @@ append_to_segment(UId, Tid, Idx, EndIdx, Seg0, Closed, State) ->
true ->
%% a snapshot must have been completed after we last checked
%% the start idx, continue flush from new start index.
- append_to_segment(UId, Tid, StartIdx, EndIdx, Seg0,
+ append_to_segment(UId, Tid, next_ra_seq(StartIdx, SeqIter), Seg0,
Closed, State);
false ->
%% oh dear, an expected index was not found in the mem table.
@@ -424,7 +429,7 @@ append_to_segment(UId, Tid, Idx, EndIdx, Seg0, Closed, State) ->
%% the segment index but is probably good enough to get comparative
%% data rates for different Ra components
ok = counters:add(State#state.counter, ?C_BYTES_WRITTEN, DataSize),
- append_to_segment(UId, Tid, Idx+1, EndIdx, Seg, Closed, State);
+ append_to_segment(UId, Tid, ra_seq:next(SeqIter), Seg, Closed, State);
{error, full} ->
% close and open a new segment
case open_successor_segment(Seg0, State#state.segment_conf) of
@@ -437,8 +442,14 @@ append_to_segment(UId, Tid, Idx, EndIdx, Seg0, Closed, State) ->
%% re-evaluate snapshot state for the server in case
%% a snapshot has completed during segment flush
StartIdx = start_index(UId, Idx),
- append_to_segment(UId, Tid, StartIdx, EndIdx, Seg,
- [Seg0 | Closed], State)
+ Next = case StartIdx == Idx of
+ true ->
+ Cur;
+ false ->
+ next_ra_seq(StartIdx, SeqIter)
+ end,
+ append_to_segment(UId, Tid, Next,
+ Seg, [Seg0 | Closed], State)
end;
{error, Posix} ->
FileName = ra_log_segment:filename(Seg0),
@@ -554,3 +565,13 @@ maybe_upgrade_segment_file_names(System, DataDir) ->
ok
end.
+next_ra_seq(Idx, Iter0) ->
+ case ra_seq:next(Iter0) of
+ end_of_seq ->
+ end_of_seq;
+ {I, _} = Next
+ when I >= Idx ->
+ Next;
+ {_, Iter} ->
+ next_ra_seq(Idx, Iter)
+ end.
diff --git a/src/ra_log_snapshot.erl b/src/ra_log_snapshot.erl
index 21acee4c..0d6ab070 100644
--- a/src/ra_log_snapshot.erl
+++ b/src/ra_log_snapshot.erl
@@ -159,7 +159,7 @@ read_chunk({Pos, Eof, Fd}, Size, _Dir) ->
{error, unexpected_eof}
end.
--spec recover(file:filename()) ->
+-spec recover(file:filename_all()) ->
{ok, meta(), term()} |
{error, invalid_format |
{invalid_version, integer()} |
@@ -178,6 +178,7 @@ recover(Dir) ->
Err
end.
+
validate(Dir) ->
case recover(Dir) of
{ok, _, _} -> ok;
@@ -209,7 +210,6 @@ read_meta(Dir) ->
context() ->
#{can_accept_full_file => true}.
-
%% Internal
read_meta_internal(Fd) ->
diff --git a/src/ra_log_snapshot_state.erl b/src/ra_log_snapshot_state.erl
new file mode 100644
index 00000000..2f3e5f58
--- /dev/null
+++ b/src/ra_log_snapshot_state.erl
@@ -0,0 +1,50 @@
+-module(ra_log_snapshot_state).
+
+-export([
+ insert/5,
+ delete/2,
+ smallest/2,
+ snapshot/2
+ ]).
+
+-spec insert(ets:table(), ra:uid(), -1 | ra:index(), ra:index(), ra_seq:state()) ->
+ ok.
+insert(Table, UId, SnapIdx, SmallestIdx, LiveIndexes)
+ when is_binary(UId) andalso
+ is_integer(SnapIdx) andalso
+ is_integer(SmallestIdx) andalso
+ is_list(LiveIndexes) ->
+ true = ets:insert(Table, {UId, SnapIdx, SmallestIdx, LiveIndexes}),
+ ok.
+
+delete(Table, UId) ->
+ true = ets:delete(Table, UId),
+ ok.
+
+-spec smallest(ets:table(), ra:uid()) ->
+ ra:index().
+smallest(Table, UId) when is_binary(UId) ->
+ ets:lookup_element(Table, UId, 3, 0).
+
+-spec snapshot(ets:table(), ra:uid()) ->
+ ra:index() | -1.
+snapshot(Table, UId) when is_binary(UId) ->
+ ets:lookup_element(Table, UId, 2, -1).
+
+%%% ===================
+%%% Internal unit tests
+%%% ===================
+
+-ifdef(TEST).
+-include_lib("eunit/include/eunit.hrl").
+
+basics_test() ->
+
+ UId = atom_to_binary(?FUNCTION_NAME, utf8),
+ T = ets:new(?FUNCTION_NAME, [set]),
+ ok = insert(T, UId, 50, 51, []),
+ ?assertEqual(51, smallest(T, UId)),
+ ?assertEqual(50, snapshot(T, UId)),
+ ok.
+
+-endif.
diff --git a/src/ra_log_sup.erl b/src/ra_log_sup.erl
index b2850a2f..117febdb 100644
--- a/src/ra_log_sup.erl
+++ b/src/ra_log_sup.erl
@@ -56,9 +56,8 @@ init([#{data_dir := DataDir,
make_wal_conf(#{data_dir := DataDir,
- name := _System,
- names := #{wal := WalName,
- segment_writer := SegWriterName} = Names} = Cfg) ->
+ name := System,
+ names := #{} = Names} = Cfg) ->
WalDir = case Cfg of
#{wal_data_dir := D} -> D;
_ -> DataDir
@@ -69,7 +68,6 @@ make_wal_conf(#{data_dir := DataDir,
MaxBatchSize = maps:get(wal_max_batch_size, Cfg,
?WAL_DEFAULT_MAX_BATCH_SIZE),
MaxEntries = maps:get(wal_max_entries, Cfg, undefined),
- Strategy = maps:get(wal_write_strategy, Cfg, default),
SyncMethod = maps:get(wal_sync_method, Cfg, datasync),
HibAfter = maps:get(wal_hibernate_after, Cfg, undefined),
Gc = maps:get(wal_garbage_collect, Cfg, false),
@@ -77,12 +75,10 @@ make_wal_conf(#{data_dir := DataDir,
MinBinVheapSize = maps:get(wal_min_bin_vheap_size, Cfg,
?MIN_BIN_VHEAP_SIZE),
MinHeapSize = maps:get(wal_min_heap_size, Cfg, ?MIN_HEAP_SIZE),
- #{name => WalName,
- names => Names,
+ #{names => Names,
+ system => System,
dir => WalDir,
- segment_writer => SegWriterName,
compute_checksums => ComputeChecksums,
- write_strategy => Strategy,
max_size_bytes => MaxSizeBytes,
max_entries => MaxEntries,
sync_method => SyncMethod,
diff --git a/src/ra_log_wal.erl b/src/ra_log_wal.erl
index 87ca98d3..bd8b8d26 100644
--- a/src/ra_log_wal.erl
+++ b/src/ra_log_wal.erl
@@ -17,6 +17,7 @@
-export([
write/6,
+ write/7,
write_batch/2,
last_writer_seq/2,
force_roll_over/1]).
@@ -54,10 +55,10 @@
% tables and segment notification
-type writer_id() :: {ra_uid(), pid()}.
--record(batch_writer, {snap_idx :: ra_index(),
+-record(batch_writer, {smallest_live_idx :: ra_index(),
tid :: ets:tid(),
uid :: term(),
- range :: ra:range(),
+ seq :: ra_seq:state(),
term :: ra_term(),
old :: undefined | #batch_writer{}
}).
@@ -67,26 +68,16 @@
pending = [] :: iolist()
}).
--type wal_write_strategy() ::
- % writes all pending in one write(2) call then calls fsync(1)
- default |
- % like default but tries to open the file using synchronous io
- % (O_SYNC) rather than a write(2) followed by an fsync.
- o_sync |
- %% low latency mode where writers are notifies _before_ syncing
- %% but after writing.
- sync_after_notify.
-
-type writer_name_cache() :: {NextIntId :: non_neg_integer(),
#{writer_id() => binary()}}.
-record(conf, {dir :: file:filename_all(),
+ system :: atom(),
segment_writer = ra_log_segment_writer :: atom() | pid(),
compute_checksums = false :: boolean(),
max_size_bytes :: non_neg_integer(),
max_entries :: undefined | non_neg_integer(),
recovery_chunk_size = ?WAL_RECOVERY_CHUNK_SIZE :: non_neg_integer(),
- write_strategy = default :: wal_write_strategy(),
sync_method = datasync :: sync | datasync | none,
counter :: counters:counters_ref(),
mem_tables_tid :: ets:tid(),
@@ -108,7 +99,7 @@
-record(recovery, {mode :: initial | post_boot,
ranges = #{} :: #{ra_uid() =>
- [{ets:tid(), {ra:index(), ra:index()}}]},
+ [{ets:tid(), ra_seq:state()}]},
tables = #{} :: #{ra_uid() => ra_mt:state()},
writers = #{} :: #{ra_uid() => {in_seq, ra:index()}}
}).
@@ -131,15 +122,13 @@
}).
-type state() :: #state{}.
--type wal_conf() :: #{name := atom(), %% the name to register the wal as
- names := ra_system:names(),
+-type wal_conf() :: #{names := ra_system:names(),
+ system := atom(),
dir := file:filename_all(),
max_size_bytes => non_neg_integer(),
max_entries => non_neg_integer(),
- segment_writer => atom() | pid(),
compute_checksums => boolean(),
pre_allocate => boolean(),
- write_strategy => wal_write_strategy(),
sync_method => sync | datasync,
recovery_chunk_size => non_neg_integer(),
hibernate_after => non_neg_integer(),
@@ -149,11 +138,11 @@
min_bin_vheap_size => non_neg_integer()
}.
--export_type([wal_conf/0,
- wal_write_strategy/0]).
+-export_type([wal_conf/0]).
-type wal_command() ::
- {append | truncate, writer_id(), ra_index(), ra_term(), term()}.
+ {append, writer_id(), PrevIndex :: ra:index() | -1,
+ Index :: ra:index(), Term :: ra_term(), wal_cmd()}.
-type wal_op() :: {cast, wal_command()} |
{call, from(), wal_command()}.
@@ -162,10 +151,23 @@
-spec write(atom() | pid(), writer_id(), ets:tid(), ra_index(), ra_term(),
wal_cmd()) ->
{ok, pid()} | {error, wal_down}.
-write(Wal, {_, _} = From, MtTid, Idx, Term, Cmd)
+write(Wal, From, MtTid, Idx, Term, Cmd) ->
+ %% "normal opereation where we assume a contigious sequence
+ %% this may be removed at some point
+ write(Wal, From, MtTid, Idx-1, Idx, Term, Cmd).
+
+-spec write(atom() | pid(), writer_id(), ets:tid(),
+ PrevIndex :: ra:index() | -1,
+ Index :: ra_index(),
+ Term :: ra_term(),
+ wal_cmd()) ->
+ {ok, pid()} | {error, wal_down}.
+write(Wal, {_, _} = From, MtTid, PrevIdx, Idx, Term, Cmd)
when is_integer(Idx) andalso
- is_integer(Term) ->
- named_cast(Wal, {append, From, MtTid, Idx, Term, Cmd}).
+ is_integer(PrevIdx) andalso
+ is_integer(Term) andalso
+ PrevIdx < Idx ->
+ named_cast(Wal, {append, From, MtTid, PrevIdx, Idx, Term, Cmd}).
-spec write_batch(Wal :: atom() | pid(), [wal_command()]) ->
{ok, pid()} | {error, wal_down}.
@@ -235,7 +237,9 @@ force_roll_over(Wal) ->
{ok, pid()} |
{error, {already_started, pid()}} |
{error, wal_checksum_validation_failure}.
-start_link(#{name := Name} = Config)
+start_link(#{dir := _,
+ system := _,
+ names := #{wal := Name}} = Config)
when is_atom(Name) ->
WalMaxBatchSize = maps:get(max_batch_size, Config,
?WAL_DEFAULT_MAX_BATCH_SIZE),
@@ -254,23 +258,22 @@ start_link(#{name := Name} = Config)
-spec init(wal_conf()) ->
{ok, state()} |
{stop, wal_checksum_validation_failure} | {stop, term()}.
-init(#{dir := Dir} = Conf0) ->
+init(#{system := System,
+ dir := Dir} = Conf0) ->
#{max_size_bytes := MaxWalSize,
max_entries := MaxEntries,
recovery_chunk_size := RecoveryChunkSize,
- segment_writer := SegWriter,
compute_checksums := ComputeChecksums,
pre_allocate := PreAllocate,
- write_strategy := WriteStrategy,
sync_method := SyncMethod,
garbage_collect := Gc,
min_heap_size := MinHeapSize,
min_bin_vheap_size := MinBinVheapSize,
names := #{wal := WalName,
+ segment_writer := SegWriter,
open_mem_tbls := MemTablesName} = Names} =
merge_conf_defaults(Conf0),
- ?NOTICE("WAL: ~ts init, mem-tables table name: ~w",
- [WalName, MemTablesName]),
+ ?NOTICE("WAL in ~ts initialising with name ~ts", [System, WalName]),
process_flag(trap_exit, true),
% given ra_log_wal is effectively a fan-in sink it is likely that it will
% at times receive large number of messages from a large number of
@@ -280,12 +283,12 @@ init(#{dir := Dir} = Conf0) ->
process_flag(min_heap_size, MinHeapSize),
CRef = ra_counters:new(WalName, ?COUNTER_FIELDS),
Conf = #conf{dir = Dir,
+ system = System,
segment_writer = SegWriter,
compute_checksums = ComputeChecksums,
max_size_bytes = max(?WAL_MIN_SIZE, MaxWalSize),
max_entries = MaxEntries,
recovery_chunk_size = RecoveryChunkSize,
- write_strategy = WriteStrategy,
sync_method = SyncMethod,
counter = CRef,
mem_tables_tid = ets:whereis(MemTablesName),
@@ -299,7 +302,9 @@ init(#{dir := Dir} = Conf0) ->
% generated during recovery
ok = ra_log_segment_writer:await(SegWriter),
{ok, Result}
- catch _:Err:_Stack ->
+ catch _:Err:Stack ->
+ ?ERROR("WAL in ~ts failed to initialise with ~p, stack ~p",
+ [System, Err, Stack]),
{stop, Err}
end.
@@ -317,21 +322,20 @@ handle_batch(Ops, #state{conf = #conf{explicit_gc = Gc}} = State0) ->
%% process all ops
{ok, Actions, complete_batch(State)}.
-terminate(Reason, State) ->
- ?DEBUG("wal: terminating with ~W", [Reason, 20]),
+terminate(Reason, #state{conf = #conf{system = System}} = State) ->
+
+ ?DEBUG("WAL in ~ts: terminating with ~0P", [System, Reason, 20]),
_ = cleanup(State),
ok.
-format_status(#state{conf = #conf{write_strategy = Strat,
- sync_method = SyncMeth,
+format_status(#state{conf = #conf{sync_method = SyncMeth,
compute_checksums = Cs,
names = #{wal := WalName},
max_size_bytes = MaxSize},
writers = Writers,
wal = #wal{file_size = FSize,
filename = Fn}}) ->
- #{write_strategy => Strat,
- sync_method => SyncMeth,
+ #{sync_method => SyncMeth,
compute_checksums => Cs,
writers => maps:size(Writers),
filename => filename:basename(Fn),
@@ -352,7 +356,8 @@ handle_op({info, {'EXIT', _, Reason}}, _State) ->
%% this is here for testing purposes only
throw({stop, Reason}).
-recover_wal(Dir, #conf{segment_writer = SegWriter,
+recover_wal(Dir, #conf{system = System,
+ segment_writer = SegWriter,
mem_tables_tid = MemTblsTid} = Conf) ->
% ensure configured directory exists
ok = ra_lib:make_dir(Dir),
@@ -376,27 +381,32 @@ recover_wal(Dir, #conf{segment_writer = SegWriter,
end || File <- Files0,
filename:extension(File) == ".wal"],
WalFiles = lists:sort(Files),
- AllWriters =
- [begin
- ?DEBUG("wal: recovering ~ts, Mode ~s", [F, Mode]),
- Fd = open_at_first_record(filename:join(Dir, F)),
- {Time, #recovery{ranges = Ranges,
- writers = Writers}} =
- timer:tc(fun () -> recover_wal_chunks(Conf, Fd, Mode) end),
-
- ok = ra_log_segment_writer:accept_mem_tables(SegWriter, Ranges, F),
-
- close_existing(Fd),
- ?DEBUG("wal: recovered ~ts time taken ~bms - recovered ~b writers",
- [F, Time div 1000, map_size(Writers)]),
- Writers
- end || F <- WalFiles],
-
- FinalWriters = lists:foldl(fun (New, Acc) ->
- maps:merge(Acc, New)
- end, #{}, AllWriters),
-
- ?DEBUG("wal: recovered ~b writers", [map_size(FinalWriters)]),
+ FinalWriters =
+ lists:foldl(fun (F, Writers0) ->
+ ?DEBUG("WAL in ~ts: recovering ~ts, Mode ~s",
+ [System, F, Mode]),
+ Fd = open_at_first_record(filename:join(Dir, F)),
+ {Time, #recovery{ranges = Ranges,
+ writers = Writers}} =
+ timer:tc(fun () ->
+ recover_wal_chunks(Conf, Fd,
+ Writers0, Mode)
+ end),
+
+ ok = ra_log_segment_writer:accept_mem_tables(SegWriter,
+ Ranges, F),
+ close_existing(Fd),
+ ?DEBUG("WAL in ~ts: recovered ~ts time taken ~bms - recovered ~b writers",
+ [System, F, Time div 1000, map_size(Writers)]),
+ Writers
+ end, #{}, WalFiles),
+
+ % FinalWriters = lists:foldl(fun (New, Acc) ->
+ % maps:merge(Acc, New)
+ % end, #{}, AllWriters),
+
+ ?DEBUG("WAL in ~ts: final writers recovered ~b",
+ [System, map_size(FinalWriters)]),
FileNum = extract_file_num(lists:reverse(WalFiles)),
State = roll_over(#state{conf = Conf,
@@ -436,7 +446,7 @@ serialize_header(UId, Trunc, {Next, Cache} = WriterCache) ->
{Next + 1, Cache#{UId => BinId}}}
end.
-write_data({UId, Pid} = Id, MtTid, Idx, Term, Data0, Trunc, SnapIdx,
+write_data({UId, Pid} = Id, MtTid, Idx, Term, Data0, Trunc, SmallestIndex,
#state{conf = #conf{counter = Counter,
compute_checksums = ComputeChecksum} = _Cfg,
batch = Batch0,
@@ -449,7 +459,7 @@ write_data({UId, Pid} = Id, MtTid, Idx, Term, Data0, Trunc, SnapIdx,
case should_roll_wal(State0) of
true ->
State = complete_batch_and_roll(State0),
- write_data(Id, MtTid, Idx, Term, Data0, Trunc, SnapIdx, State);
+ write_data(Id, MtTid, Idx, Term, Data0, Trunc, SmallestIndex, State);
false ->
EntryData = case Data0 of
{ttb, Bin} ->
@@ -474,7 +484,7 @@ write_data({UId, Pid} = Id, MtTid, Idx, Term, Data0, Trunc, SnapIdx,
<> |
Entry],
Batch = incr_batch(Batch0, UId, Pid, MtTid,
- Idx, Term, Record, SnapIdx),
+ Idx, Term, Record, SmallestIndex),
counters:add(Counter, ?C_BYTES_WRITTEN, DataSize),
State0#state{batch = Batch,
wal = Wal#wal{writer_name_cache = Cache,
@@ -484,39 +494,42 @@ write_data({UId, Pid} = Id, MtTid, Idx, Term, Data0, Trunc, SnapIdx,
end.
-handle_msg({append, {UId, Pid} = Id, MtTid, Idx, Term, Entry},
+handle_msg({append, {UId, Pid} = Id, MtTid, ExpectedPrevIdx, Idx, Term, Entry},
#state{conf = Conf,
writers = Writers} = State0) ->
- SnapIdx = snap_idx(Conf, UId),
+ SmallestIdx = smallest_live_index(Conf, UId),
%% detect if truncating flag should be set
- Trunc = Idx == SnapIdx + 1,
- case maps:find(UId, Writers) of
- _ when Idx =< SnapIdx ->
- %% a snapshot already exists that is higher - just drop the write
- State0#state{writers = Writers#{UId => {in_seq, SnapIdx}}};
- {ok, {_, PrevIdx}}
- when Idx =< PrevIdx + 1 orelse
+ Trunc = Idx == SmallestIdx,
+
+ case maps:get(UId, Writers, undefined) of
+ _ when Idx < SmallestIdx ->
+ %% the smallest live index for the last snapshot is higher than
+ %% this index, just drop it
+ LastIdx = SmallestIdx - 1,
+ State0#state{writers = Writers#{UId => {in_seq, LastIdx}}};
+ {_, PrevIdx}
+ when ExpectedPrevIdx =< PrevIdx orelse
Trunc ->
- write_data(Id, MtTid, Idx, Term, Entry, Trunc, SnapIdx, State0);
- error ->
- write_data(Id, MtTid, Idx, Term, Entry, false, SnapIdx, State0);
- {ok, {out_of_seq, _}} ->
+ %% if the passed in previous index is less than the last written
+ %% index (gap detection) _or_ it is a truncation
+ %% then we can proceed and write the entry
+ write_data(Id, MtTid, Idx, Term, Entry, Trunc, SmallestIdx, State0);
+ undefined ->
+ %% no state for the UId is known so go ahead and write
+ write_data(Id, MtTid, Idx, Term, Entry, false, SmallestIdx, State0);
+ {out_of_seq, _} ->
% writer is out of seq simply ignore drop the write
% TODO: capture metric for dropped writes?
State0;
- {ok, {in_seq, PrevIdx}} ->
+ {in_seq, PrevIdx} ->
% writer was in seq but has sent an out of seq entry
% notify writer
- ?DEBUG("WAL: requesting resend from `~w`, "
- "last idx ~b idx received ~b",
- [UId, PrevIdx, Idx]),
+ ?DEBUG("WAL in ~ts: requesting resend from `~w`, "
+ "last idx ~b idx received (~b,~b)",
+ [Conf#conf.system, UId, PrevIdx, ExpectedPrevIdx, Idx]),
Pid ! {ra_log_event, {resend_write, PrevIdx + 1}},
State0#state{writers = Writers#{UId => {out_of_seq, PrevIdx}}}
end;
-handle_msg({truncate, Id, MtTid, Idx, Term, Entry},
- #state{conf = Conf} = State0) ->
- SnapIdx = snap_idx(Conf, Id),
- write_data(Id, MtTid, Idx, Term, Entry, true, SnapIdx, State0);
handle_msg({query, Fun}, State) ->
%% for testing
_ = catch Fun(State),
@@ -528,29 +541,28 @@ incr_batch(#batch{num_writes = Writes,
waiting = Waiting0,
pending = Pend} = Batch,
UId, Pid, MT_TID = MtTid,
- Idx, TERM = Term, Data, SnapIdx) ->
+ Idx, TERM = Term, Data, SmallestLiveIdx) ->
Waiting = case Waiting0 of
#{Pid := #batch_writer{term = TERM,
tid = MT_TID,
- range = Range0
- } = W} ->
+ seq = Seq0} = W} ->
%% The Tid and term is the same so add to current batch_writer
- Range = ra_range:extend(Idx, ra_range:truncate(SnapIdx, Range0)),
- Waiting0#{Pid => W#batch_writer{range = Range,
- snap_idx = SnapIdx,
- term = Term
- }};
+ Range = ra_seq:append(Idx, Seq0),
+ %% TODO: range nees to become a ra_seq so that we can
+ %% capture sparse writes correctly
+ Waiting0#{Pid => W#batch_writer{seq = Range,
+ smallest_live_idx = SmallestLiveIdx,
+ term = Term}};
_ ->
%% The tid is different, open a new batch writer for the
%% new tid and term
PrevBatchWriter = maps:get(Pid, Waiting0, undefined),
- Writer = #batch_writer{snap_idx = SnapIdx,
+ Writer = #batch_writer{smallest_live_idx = SmallestLiveIdx,
tid = MtTid,
- range = ra_range:new(Idx),
+ seq = [Idx],
uid = UId,
term = Term,
- old = PrevBatchWriter
- },
+ old = PrevBatchWriter},
Waiting0#{Pid => Writer}
end,
@@ -564,51 +576,45 @@ complete_batch_and_roll(#state{} = State0) ->
roll_over(#state{wal = Wal0, file_num = Num0,
conf = #conf{dir = Dir,
+ system = System,
segment_writer = SegWriter,
max_size_bytes = MaxBytes} = Conf0} = State0) ->
counters:add(Conf0#conf.counter, ?C_WAL_FILES, 1),
Num = Num0 + 1,
Fn = ra_lib:zpad_filename("", "wal", Num),
NextFile = filename:join(Dir, Fn),
- ?DEBUG("wal: opening new file ~ts", [Fn]),
+ ?DEBUG("WAL in ~ts: opening new file ~ts", [System, Fn]),
%% if this is the first wal since restart randomise the first
%% max wal size to reduce the likelihood that each erlang node will
%% flush mem tables at the same time
- NextMaxBytes = case Wal0 of
- undefined ->
- Half = MaxBytes div 2,
- Half + rand:uniform(Half);
- #wal{ranges = Ranges,
- filename = Filename} ->
- _ = file:advise(Wal0#wal.fd, 0, 0, dont_need),
- ok = close_file(Wal0#wal.fd),
- MemTables = Ranges,
- %% TODO: only keep base name in state
- Basename = filename:basename(Filename),
- ok = ra_log_segment_writer:accept_mem_tables(SegWriter,
- MemTables,
- Basename),
- MaxBytes
- end,
+ NextMaxBytes =
+ case Wal0 of
+ undefined ->
+ Half = MaxBytes div 2,
+ Half + rand:uniform(Half);
+ #wal{ranges = Ranges,
+ filename = Filename} ->
+ _ = file:advise(Wal0#wal.fd, 0, 0, dont_need),
+ ok = close_file(Wal0#wal.fd),
+ %% floor all sequences
+ MemTables = maps:map(
+ fun (UId, TidRanges) ->
+ SmallestIdx = smallest_live_index(Conf0, UId),
+ [{Tid, ra_seq:floor(SmallestIdx, Seq)}
+ || {Tid, Seq} <- TidRanges]
+ end, Ranges),
+ %% TODO: only keep base name in state
+ Basename = filename:basename(Filename),
+ ok = ra_log_segment_writer:accept_mem_tables(SegWriter,
+ MemTables,
+ Basename),
+ MaxBytes
+ end,
{Conf, Wal} = open_wal(NextFile, NextMaxBytes, Conf0),
State0#state{conf = Conf,
wal = Wal,
file_num = Num}.
-open_wal(File, Max, #conf{write_strategy = o_sync} = Conf) ->
- Modes = [sync | ?FILE_MODES],
- case prepare_file(File, Modes) of
- {ok, Fd} ->
- % many platforms implement O_SYNC a bit like O_DSYNC
- % perform a manual sync here to ensure metadata is flushed
- {Conf, #wal{fd = Fd,
- max_size = Max,
- filename = File}};
- {error, enotsup} ->
- ?WARN("wal: o_sync write strategy not supported. "
- "Reverting back to default strategy.", []),
- open_wal(File, Max, Conf#conf{write_strategy = default})
- end;
open_wal(File, Max, #conf{} = Conf0) ->
{ok, Fd} = prepare_file(File, ?FILE_MODES),
Conf = maybe_pre_allocate(Conf0, Fd, Max),
@@ -637,9 +643,8 @@ make_tmp(File) ->
ok = file:close(Fd),
Tmp.
-maybe_pre_allocate(#conf{pre_allocate = true,
- write_strategy = Strat} = Conf, Fd, Max0)
- when Strat /= o_sync ->
+maybe_pre_allocate(#conf{system = System,
+ pre_allocate = true} = Conf, Fd, Max0) ->
Max = Max0 - ?HEADER_SIZE,
case file:allocate(Fd, ?HEADER_SIZE, Max) of
ok ->
@@ -650,11 +655,12 @@ maybe_pre_allocate(#conf{pre_allocate = true,
{error, _} ->
%% fallocate may not be supported, fall back to fsync instead
%% of fdatasync
- ?INFO("wal: preallocation may not be supported by the file system"
- " falling back to fsync instead of fdatasync", []),
+ ?INFO("WAL in ~ts: preallocation may not be supported by the file system"
+ " falling back to fsync instead of fdatasync",
+ [System]),
Conf#conf{pre_allocate = false}
end;
-maybe_pre_allocate(Conf, _Fd, _Max) ->
+maybe_pre_allocate(Conf, _Fd, _Max0) ->
Conf.
close_file(undefined) ->
@@ -666,26 +672,11 @@ start_batch(#state{conf = #conf{counter = CRef}} = State) ->
ok = counters:add(CRef, ?C_BATCHES, 1),
State#state{batch = #batch{}}.
-
-post_notify_flush(#state{wal = #wal{fd = Fd},
- conf = #conf{write_strategy = sync_after_notify,
- sync_method = SyncMeth}}) ->
- sync(Fd, SyncMeth);
-post_notify_flush(_State) ->
- ok.
-
flush_pending(#state{wal = #wal{fd = Fd},
batch = #batch{pending = Pend},
- conf = #conf{write_strategy = WriteStrategy,
- sync_method = SyncMeth}} = State0) ->
-
- case WriteStrategy of
- default ->
- ok = file:write(Fd, Pend),
- sync(Fd, SyncMeth);
- _ ->
- ok = file:write(Fd, Pend)
- end,
+ conf = #conf{sync_method = SyncMeth}} = State0) ->
+ ok = file:write(Fd, Pend),
+ sync(Fd, SyncMeth),
State0#state{batch = undefined}.
sync(_Fd, none) ->
@@ -709,18 +700,17 @@ complete_batch(#state{batch = #batch{waiting = Waiting,
Ranges = maps:fold(fun (Pid, BatchWriter, Acc) ->
complete_batch_writer(Pid, BatchWriter, Acc)
end, Wal#wal.ranges, Waiting),
- ok = post_notify_flush(State),
State#state{wal = Wal#wal{ranges = Ranges}}.
-complete_batch_writer(Pid, #batch_writer{snap_idx = SnapIdx,
- tid = MtTid,
- uid = UId,
- range = Range,
- term = Term,
- old = undefined
- }, Ranges) ->
- Pid ! {ra_log_event, {written, Term, Range}},
- update_ranges(Ranges, UId, MtTid, SnapIdx, Range);
+complete_batch_writer(Pid, #batch_writer{smallest_live_idx = SmallestIdx,
+ tid = MtTid,
+ uid = UId,
+ seq = Seq0,
+ term = Term,
+ old = undefined}, Ranges) ->
+ Seq = ra_seq:floor(SmallestIdx, Seq0),
+ Pid ! {ra_log_event, {written, Term, Seq}},
+ update_ranges(Ranges, UId, MtTid, SmallestIdx, Seq);
complete_batch_writer(Pid, #batch_writer{old = #batch_writer{} = OldBw} = Bw,
Ranges0) ->
Ranges = complete_batch_writer(Pid, OldBw, Ranges0),
@@ -794,9 +784,10 @@ dump_records(<<_:1/unsigned, 1:1/unsigned, _:22/unsigned,
dump_records(<<>>, Entries) ->
Entries.
-recover_wal_chunks(#conf{} = Conf, Fd, Mode) ->
+recover_wal_chunks(#conf{} = Conf, Fd, Writers, Mode) ->
Chunk = read_wal_chunk(Fd, Conf#conf.recovery_chunk_size),
- recover_records(Conf, Fd, Chunk, #{}, #recovery{mode = Mode}).
+ recover_records(Conf, Fd, Chunk, #{}, #recovery{mode = Mode,
+ writers = Writers}).
% All zeros indicates end of a pre-allocated wal file
recover_records(_, _Fd, <<0:1/unsigned, 0:1/unsigned, 0:22/unsigned,
IdDataLen:16/unsigned, _:IdDataLen/binary,
@@ -816,13 +807,13 @@ recover_records(#conf{names = Names} = Conf, Fd,
case ra_directory:is_registered_uid(Names, UId) of
true ->
Cache = Cache0#{IdRef => {UId, <<1:1/unsigned, IdRef:22/unsigned>>}},
- SnapIdx = recover_snap_idx(Conf, UId, Trunc == 1, Idx),
+ SmallestIdx = recover_smallest_idx(Conf, UId, Trunc == 1, Idx),
case validate_checksum(Checksum, Idx, Term, EntryData) of
- ok when Idx > SnapIdx ->
+ ok when Idx >= SmallestIdx ->
State1 = handle_trunc(Trunc == 1, UId, Idx, State0),
case recover_entry(Names, UId,
{Idx, Term, binary_to_term(EntryData)},
- SnapIdx, State1) of
+ SmallestIdx, State1) of
{ok, State} ->
recover_records(Conf, Fd, Rest, Cache, State);
{retry, State} ->
@@ -831,20 +822,24 @@ recover_records(#conf{names = Names} = Conf, Fd,
ok ->
%% best the the snapshot index as the last
%% writer index
- Writers = case State0#recovery.writers of
- #{UId := {in_seq, SnapIdx}} = W ->
- W;
- W ->
- W#{UId => {in_seq, SnapIdx}}
- end,
+ % Writers = case State0#recovery.writers of
+ % #{UId := {in_seq, SmallestIdx}} = W ->
+ % W;
+ % W ->
+ % W#{UId => {in_seq, SmallestIdx}}
+ % end,
+ Writers = State0#recovery.writers,
+ % Writers = W#{UId => {in_seq, SmallestIdx - 1}},
recover_records(Conf, Fd, Rest, Cache,
- State0#recovery{writers = Writers});
+ State0#recovery{writers =
+ maps:remove(UId, Writers)});
error ->
- ?DEBUG("WAL: record failed CRC check. If this is the last record"
- " recovery can resume", []),
+ System = Conf#conf.system,
+ ?DEBUG("WAL in ~ts: record failed CRC check. If this is the last record"
+ " recovery can resume", [System]),
%% if this is the last entry in the wal we can just drop the
%% record;
- ok = is_last_record(Fd, Rest),
+ ok = is_last_record(Fd, Rest, Conf),
State0
end;
false ->
@@ -860,13 +855,13 @@ recover_records(#conf{names = Names} = Conf, Fd,
Cache, State0) ->
case Cache of
#{IdRef := {UId, _}} ->
- SnapIdx = recover_snap_idx(Conf, UId, Trunc == 1, Idx),
+ SmallestIdx = recover_smallest_idx(Conf, UId, Trunc == 1, Idx),
case validate_checksum(Checksum, Idx, Term, EntryData) of
- ok when Idx > SnapIdx ->
+ ok when Idx >= SmallestIdx ->
State1 = handle_trunc(Trunc == 1, UId, Idx, State0),
case recover_entry(Names, UId,
{Idx, Term, binary_to_term(EntryData)},
- SnapIdx, State1) of
+ SmallestIdx, State1) of
{ok, State} ->
recover_records(Conf, Fd, Rest, Cache, State);
{retry, State} ->
@@ -875,11 +870,12 @@ recover_records(#conf{names = Names} = Conf, Fd,
ok ->
recover_records(Conf, Fd, Rest, Cache, State0);
error ->
- ?DEBUG("WAL: record failed CRC check. If this is the last record"
- " recovery can resume", []),
+ System = Conf#conf.system,
+ ?DEBUG("WAL in ~ts: record failed CRC check. If this is the last record"
+ " recovery can resume", [System]),
%% if this is the last entry in the wal we can just drop the
%% record;
- ok = is_last_record(Fd, Rest),
+ ok = is_last_record(Fd, Rest, Conf),
State0
end;
_ ->
@@ -900,28 +896,28 @@ recover_records(Conf, Fd, Chunk, Cache, State) ->
recover_records(Conf, Fd, Chunk0, Cache, State)
end.
-recover_snap_idx(Conf, UId, Trunc, CurIdx) ->
+recover_smallest_idx(Conf, UId, Trunc, CurIdx) ->
case Trunc of
true ->
- max(CurIdx-1, snap_idx(Conf, UId));
+ max(CurIdx, smallest_live_index(Conf, UId));
false ->
- snap_idx(Conf, UId)
+ smallest_live_index(Conf, UId)
end.
-is_last_record(_Fd, <<0:104, _/binary>>) ->
+is_last_record(_Fd, <<0:104, _/binary>>, _) ->
ok;
-is_last_record(Fd, Rest) ->
+is_last_record(Fd, Rest, Conf) ->
case byte_size(Rest) < 13 of
true ->
case read_wal_chunk(Fd, 256) of
<<>> ->
ok;
Next ->
- is_last_record(Fd, <>)
+ is_last_record(Fd, <>, Conf)
end;
false ->
- ?ERROR("WAL: record failed CRC check during recovery. "
- "Unable to recover WAL data safely", []),
+ ?ERROR("WAL in ~ts: record failed CRC check during recovery. "
+ "Unable to recover WAL data safely", [Conf#conf.system]),
throw(wal_checksum_validation_failure)
end.
@@ -956,7 +952,6 @@ merge_conf_defaults(Conf) ->
recovery_chunk_size => ?WAL_RECOVERY_CHUNK_SIZE,
compute_checksums => true,
pre_allocate => false,
- write_strategy => default,
garbage_collect => false,
sync_method => datasync,
min_bin_vheap_size => ?MIN_BIN_VHEAP_SIZE,
@@ -983,29 +978,27 @@ should_roll_wal(#state{conf = #conf{max_entries = MaxEntries},
Count + 1 > MaxEntries
end.
-snap_idx(#conf{ra_log_snapshot_state_tid = Tid}, ServerUId) ->
- ets:lookup_element(Tid, ServerUId, 2, -1).
+smallest_live_index(#conf{ra_log_snapshot_state_tid = Tid}, ServerUId) ->
+ ra_log_snapshot_state:smallest(Tid, ServerUId).
-update_ranges(Ranges, UId, MtTid, SnapIdx, {Start, _} = AddRange) ->
+update_ranges(Ranges, UId, MtTid, _SmallestIdx, AddSeq) ->
case Ranges of
- #{UId := [{MtTid, Range0} | Rem]} ->
- %% SnapIdx might have moved to we truncate the old range first
+ #{UId := [{MtTid, Seq0} | Seqs]} ->
+ %% SmallestIdx might have moved to we truncate the old range first
%% before extending
- Range1 = ra_range:truncate(SnapIdx, Range0),
+ % Seq1 = ra_seq:floor(SmallestIdx, Seq0),
%% limit the old range by the add end start as in some resend
%% cases we may have got back before the prior range.
- Range = ra_range:add(AddRange, ra_range:limit(Start, Range1)),
- Ranges#{UId => [{MtTid, Range} | Rem]};
- #{UId := [{OldMtTid, OldMtRange} | Rem]} ->
+ Seq = ra_seq:add(AddSeq, Seq0),
+ Ranges#{UId => [{MtTid, Seq} | Seqs]};
+ #{UId := Seqs} ->
%% new Tid, need to add a new range record for this
- Ranges#{UId => [{MtTid, AddRange},
- ra_range:truncate(SnapIdx, {OldMtTid, OldMtRange})
- | Rem]};
+ Ranges#{UId => [{MtTid, AddSeq} | Seqs]};
_ ->
- Ranges#{UId => [{MtTid, AddRange}]}
+ Ranges#{UId => [{MtTid, AddSeq}]}
end.
-recover_entry(Names, UId, {Idx, _, _} = Entry, SnapIdx,
+recover_entry(Names, UId, {Idx, _, _} = Entry, SmallestIdx,
#recovery{mode = initial,
ranges = Ranges0,
writers = Writers,
@@ -1016,19 +1009,29 @@ recover_entry(Names, UId, {Idx, _, _} = Entry, SnapIdx,
{ok, M} = ra_log_ets:mem_table_please(Names, UId),
M
end,
- case ra_mt:insert(Entry, Mt0) of
+ %% always use write_sparse as there is nothing to indicate in the wal
+ %% data if an entry was written as such. this way we recover all writes
+ %% so should be ok for all types of writes
+ PrevIdx = case Writers of
+ #{UId := {in_seq, I}} ->
+ I;
+ _ ->
+ undefined
+ end,
+ case ra_mt:insert_sparse(Entry, PrevIdx, Mt0) of
{ok, Mt1} ->
Ranges = update_ranges(Ranges0, UId, ra_mt:tid(Mt1),
- SnapIdx, ra_range:new(Idx)),
+ SmallestIdx, [Idx]),
{ok, State#recovery{ranges = Ranges,
writers = Writers#{UId => {in_seq, Idx}},
tables = Tables#{UId => Mt1}}};
{error, overwriting} ->
%% create successor memtable
{ok, Mt1} = ra_log_ets:new_mem_table_please(Names, UId, Mt0),
- {retry, State#recovery{tables = Tables#{UId => Mt1}}}
+ {retry, State#recovery{tables = Tables#{UId => Mt1},
+ writers = maps:remove(UId, Writers)}}
end;
-recover_entry(Names, UId, {Idx, Term, _}, SnapIdx,
+recover_entry(Names, UId, {Idx, Term, _}, SmallestIdx,
#recovery{mode = post_boot,
ranges = Ranges0,
writers = Writers,
@@ -1051,7 +1054,7 @@ recover_entry(Names, UId, {Idx, Term, _}, SnapIdx,
tables = Tables#{UId => Mt0}}};
Tid ->
Ranges = update_ranges(Ranges0, UId, Tid,
- SnapIdx, ra_range:new(Idx)),
+ SmallestIdx, [Idx]),
{ok, State#recovery{ranges = Ranges,
writers = Writers#{UId => {in_seq, Idx}},
tables = Tables#{UId => Mt0}}}
@@ -1060,15 +1063,28 @@ recover_entry(Names, UId, {Idx, Term, _}, SnapIdx,
handle_trunc(false, _UId, _Idx, State) ->
State;
handle_trunc(true, UId, Idx, #recovery{mode = Mode,
+ ranges = Ranges0,
+ writers = Writers,
tables = Tbls} = State) ->
case Tbls of
#{UId := Mt0} when Mode == initial ->
%% only meddle with mem table data in initial mode
{Specs, Mt} = ra_mt:set_first(Idx-1, Mt0),
[_ = ra_mt:delete(Spec) || Spec <- Specs],
- State#recovery{tables = Tbls#{UId => Mt}};
+ Ranges = case Ranges0 of
+ #{UId := Seqs0} ->
+ Seqs = [{T, ra_seq:floor(Idx, Seq)}
+ || {T, Seq} <- Seqs0],
+ Ranges0#{UId => Seqs};
+ _ ->
+ Ranges0
+ end,
+
+ State#recovery{tables = Tbls#{UId => Mt},
+ writers = maps:remove(UId, Writers),
+ ranges = Ranges};
_ ->
- State
+ State#recovery{writers = maps:remove(UId, Writers)}
end.
named_cast(To, Msg) when is_pid(To) ->
diff --git a/src/ra_machine.erl b/src/ra_machine.erl
index 33533c0e..8178a8db 100644
--- a/src/ra_machine.erl
+++ b/src/ra_machine.erl
@@ -72,6 +72,7 @@
snapshot_installed/5,
state_enter/3,
overview/2,
+ live_indexes/2,
query/3,
module/1,
init_aux/2,
@@ -222,6 +223,7 @@
handle_aux/5,
handle_aux/6,
overview/1,
+ live_indexes/1,
snapshot_module/0,
version/0,
which_module/1
@@ -287,6 +289,8 @@
-callback overview(state()) -> map().
+-callback live_indexes(state()) -> [ra:index()].
+
-callback snapshot_module() -> module().
-callback version() -> version().
@@ -344,11 +348,17 @@ state_enter(Mod, RaftState, State) ->
overview(Mod, State) ->
?OPT_CALL(Mod:overview(State), State).
+-spec live_indexes(module(), state()) -> [ra:index()].
+live_indexes(Mod, State) ->
+ ?OPT_CALL(Mod:live_indexes(State), []).
+
%% @doc used to discover the latest machine version supported by the current
%% code
--spec version(machine()) -> version().
+-spec version(machine() | module()) -> version().
+version(Mod) when is_atom(Mod) ->
+ ?OPT_CALL(assert_version(Mod:version()), ?DEFAULT_VERSION);
version({machine, Mod, _}) ->
- ?OPT_CALL(assert_version(Mod:version()), ?DEFAULT_VERSION).
+ version(Mod).
-spec is_versioned(machine()) -> boolean().
is_versioned({machine, Mod, _}) ->
diff --git a/src/ra_mt.erl b/src/ra_mt.erl
index df25476e..55d3d011 100644
--- a/src/ra_mt.erl
+++ b/src/ra_mt.erl
@@ -6,6 +6,7 @@
%% @hidden
-module(ra_mt).
+-include_lib("stdlib/include/assert.hrl").
-include("ra.hrl").
-export([
@@ -13,9 +14,10 @@
init/2,
init_successor/3,
insert/2,
+ insert_sparse/3,
stage/2,
commit/1,
- abort/1,
+ % abort/1,
lookup/2,
lookup_term/2,
tid_for/3,
@@ -35,26 +37,27 @@
-define(MAX_MEMTBL_ENTRIES, 1_000_000).
--define(IN_RANGE(Idx, Range),
- (is_tuple(Range) andalso
- Idx >= element(1, Range) andalso
- Idx =< element(2, Range))).
+% -define(IN_RANGE(Idx, Range),
+% (is_tuple(Range) andalso
+% Idx >= element(1, Range) andalso
+% Idx =< element(2, Range))).
--define(IS_BEFORE_RANGE(Idx, Range),
- (is_tuple(Range) andalso
- Idx < element(1, Range))).
+% -define(IS_BEFORE_RANGE(Idx, Range),
+% (is_tuple(Range) andalso
+% Idx < element(1, Range))).
% -define(IS_AFTER_RANGE(Idx, Range),
% (is_tuple(Range) andalso
% Idx > element(2, Range))).
--define(IS_NEXT_IDX(Idx, Range),
- (Range == undefined orelse
- Idx == element(2, Range) + 1)).
+-define(IS_NEXT_IDX(Idx, Seq),
+ (Seq == [] orelse
+ (is_integer(hd(Seq)) andalso hd(Seq) + 1 == Idx) orelse
+ (Idx == element(2, hd(Seq)) + 1))).
-record(?MODULE,
{tid :: ets:tid(),
- range :: undefined | {ra:index(), ra:index()},
+ indexes :: ra_seq:state(),
staged :: undefined | {NumStaged :: non_neg_integer(), [log_entry()]},
prev :: undefined | #?MODULE{}
}).
@@ -64,7 +67,7 @@
-type delete_spec() :: undefined |
{'<', ets:tid(), ra:index()} |
{delete, ets:tid()} |
- {range, ets:tid(), ra:range()}.
+ {indexes, ets:tid(), ra_seq:state()}.
-export_type([
state/0,
delete_spec/0
@@ -72,20 +75,17 @@
-spec init(ets:tid(), read | read_write) -> state().
init(Tid, Mode) ->
- Range = case Mode of
+ Seq = case Mode of
read ->
- undefined;
+ [];
read_write ->
%% TODO: can this be optimised further?
- ets:foldl(fun ({I, _, _}, undefined) ->
- {I, I};
- ({I, _, _}, {S, E}) ->
- {min(I, S), max(I, E)}
- end, undefined, Tid)
+ ra_seq:from_list(ets:foldl(fun ({I, _, _}, Acc) ->
+ [I | Acc]
+ end, [], Tid))
end,
#?MODULE{tid = Tid,
- range = Range
- }.
+ indexes = Seq}.
-spec init(ets:tid()) -> state().
init(Tid) ->
@@ -100,47 +100,79 @@ init_successor(Tid, Mode, #?MODULE{} = State) ->
{ok, state()} | {error, overwriting | limit_reached}.
insert({Idx, _, _} = Entry,
#?MODULE{tid = Tid,
- range = Range} = State)
- when ?IS_NEXT_IDX(Idx, Range) ->
- case ra_range:size(Range) > ?MAX_MEMTBL_ENTRIES of
+ indexes = Seq} = State)
+ when ?IS_NEXT_IDX(Idx, Seq) ->
+ %% TODO ra_seq:length can be slow for sparse ra_seqs
+ case ra_seq:length(Seq) > ?MAX_MEMTBL_ENTRIES of
true ->
{error, limit_reached};
false ->
true = ets:insert(Tid, Entry),
- {ok, State#?MODULE{range = update_range_end(Idx, Range)}}
+ {ok, State#?MODULE{indexes = update_ra_seq(Idx, Seq)}}
end;
insert({Idx, _, _} = _Entry,
- #?MODULE{range = Range} = _State0)
- when ?IN_RANGE(Idx, Range) orelse
- ?IS_BEFORE_RANGE(Idx, Range) ->
- {error, overwriting}.
+ #?MODULE{indexes = Seq}) ->
+ case Idx =< ra_seq:last(Seq) of
+ true ->
+ {error, overwriting};
+ false ->
+ exit({unexpected_sparse_insert, Idx, Seq})
+ end.
+
+-spec insert_sparse(log_entry(), undefined | ra:index(), state()) ->
+ {ok, state()} | {error, overwriting | gap_detected | limit_reached}.
+insert_sparse({Idx, _, _} = Entry, LastIdx,
+ #?MODULE{tid = Tid,
+ indexes = Seq} = State) ->
+ LastSeq = ra_seq:last(Seq),
+ IsOverwriting = Idx =< LastSeq andalso is_integer(LastSeq),
+ case LastSeq == LastIdx andalso not IsOverwriting of
+ true ->
+ case ra_seq:length(Seq) > ?MAX_MEMTBL_ENTRIES of
+ true ->
+ {error, limit_reached};
+ false ->
+ true = ets:insert(Tid, Entry),
+ {ok, State#?MODULE{indexes = ra_seq:append(Idx, Seq)}}
+ end;
+ false ->
+ case IsOverwriting of
+ true ->
+ {error, overwriting};
+ false ->
+ {error, gap_detected}
+ end
+ end.
-spec stage(log_entry(), state()) ->
{ok, state()} | {error, overwriting | limit_reached}.
stage({Idx, _, _} = Entry,
#?MODULE{staged = {FstIdx, Staged},
- range = Range} = State)
+ indexes = Range} = State)
when ?IS_NEXT_IDX(Idx, Range) ->
{ok, State#?MODULE{staged = {FstIdx, [Entry | Staged]},
- range = update_range_end(Idx, Range)}};
+ indexes = update_ra_seq(Idx, Range)}};
stage({Idx, _, _} = Entry,
#?MODULE{tid = _Tid,
staged = undefined,
- range = Range} = State)
- when ?IS_NEXT_IDX(Idx, Range) ->
- case ra_range:size(Range) > ?MAX_MEMTBL_ENTRIES of
+ indexes = Seq} = State)
+ when ?IS_NEXT_IDX(Idx, Seq) ->
+ case ra_seq:length(Seq) > ?MAX_MEMTBL_ENTRIES of
true ->
%% the limit cannot be reached during transaction
{error, limit_reached};
false ->
{ok, State#?MODULE{staged = {Idx, [Entry]},
- range = update_range_end(Idx, Range)}}
+ indexes = update_ra_seq(Idx, Seq)}}
end;
stage({Idx, _, _} = _Entry,
- #?MODULE{range = Range} = _State0)
- when ?IN_RANGE(Idx, Range) orelse
- ?IS_BEFORE_RANGE(Idx, Range) ->
- {error, overwriting}.
+ #?MODULE{indexes = Seq}) ->
+ case Idx =< ra_seq:last(Seq) of
+ true ->
+ {error, overwriting};
+ false ->
+ exit({unexpected_sparse_stage, Idx, Seq})
+ end.
-spec commit(state()) -> {[log_entry()], state()}.
commit(#?MODULE{staged = undefined} = State) ->
@@ -160,22 +192,22 @@ commit(#?MODULE{tid = Tid,
{PrevStaged ++ Staged, State#?MODULE{staged = undefined,
prev = Prev}}.
--spec abort(state()) -> state().
-abort(#?MODULE{staged = undefined} = State) ->
- State;
-abort(#?MODULE{staged = {_, Staged},
- range = Range,
- prev = Prev0} = State) ->
- Prev = case Prev0 of
- undefined ->
- Prev0;
- _ ->
- abort(Prev0)
- end,
- {Idx, _, _} = lists:last(Staged),
- State#?MODULE{staged = undefined,
- range = ra_range:limit(Idx, Range),
- prev = Prev}.
+% -spec abort(state()) -> state().
+% abort(#?MODULE{staged = undefined} = State) ->
+% State;
+% abort(#?MODULE{staged = {_, Staged},
+% indexes = Range,
+% prev = Prev0} = State) ->
+% Prev = case Prev0 of
+% undefined ->
+% Prev0;
+% _ ->
+% abort(Prev0)
+% end,
+% {Idx, _, _} = lists:last(Staged),
+% State#?MODULE{staged = undefined,
+% indexes = ra_range:limit(Idx, Range),
+% prev = Prev}.
-spec lookup(ra:index(), state()) ->
log_entry() | undefined.
@@ -189,10 +221,12 @@ lookup(Idx, #?MODULE{staged = {FstStagedIdx, Staged}})
undefined
end;
lookup(Idx, #?MODULE{tid = Tid,
- range = Range,
+ indexes = Seq,
prev = Prev,
staged = undefined}) ->
- case ?IN_RANGE(Idx, Range) of
+ %% ra_seq:in/2 could be expensive for sparse mem tables,
+ %% TODO: consider checking ets table first
+ case ra_seq:in(Idx, Seq) of
true ->
[Entry] = ets:lookup(Tid, Idx),
Entry;
@@ -214,13 +248,21 @@ lookup_term(Idx, #?MODULE{staged = {FstStagedIdx, Staged}})
undefined
end;
lookup_term(Idx, #?MODULE{tid = Tid,
- range = Range})
- when ?IN_RANGE(Idx, Range) ->
- ets:lookup_element(Tid, Idx, 2);
-lookup_term(Idx, #?MODULE{prev = #?MODULE{} = Prev}) ->
- lookup_term(Idx, Prev);
-lookup_term(_Idx, _State) ->
- undefined.
+ prev = Prev,
+ indexes = _Seq}) ->
+ %% TODO: implement properly, checking Seq
+ case ets:lookup_element(Tid, Idx, 2, undefined) of
+ undefined when Prev =/= undefined ->
+ lookup_term(Idx, Prev);
+ Term ->
+ Term
+ end.
+ % when ?IN_RANGE(Idx, Seq) ->
+ % ets:lookup_element(Tid, Idx, 2);
+% lookup_term(Idx, #?MODULE{prev = #?MODULE{} = Prev}) ->
+ % lookup_term(Idx, Prev);
+% lookup_term(_Idx, _State) ->
+ % undefined.
-spec tid_for(ra:index(), ra_term(), state()) ->
undefined | ets:tid().
@@ -243,6 +285,7 @@ fold(To, To, Fun, Acc, State) ->
fold(From, To, Fun, Acc, State)
when To > From ->
E = lookup(From, State),
+ ?assert(E =/= undefined),
fold(From + 1, To, Fun, Fun(E, Acc), State).
-spec get_items([ra:index()], state()) ->
@@ -256,8 +299,11 @@ get_items(Indexes, #?MODULE{} = State) ->
non_neg_integer().
delete(undefined) ->
0;
-delete({range, Tid, {Start, End}}) ->
- NumToDelete = End - Start + 1,
+delete({indexes, Tid, Seq}) ->
+ NumToDelete = ra_seq:length(Seq),
+ Start = ra_seq:first(Seq),
+ End = ra_seq:last(Seq),
+ % NumToDelete = End - Start + 1,
Limit = ets:info(Tid, size) div 2,
%% check if there is an entry below the start of the deletion range,
%% if there is we've missed a segment event at some point and need
@@ -268,8 +314,11 @@ delete({range, Tid, {Start, End}}) ->
%% more than half the table is to be deleted
delete({'<', Tid, End + 1});
false ->
- delete(Start, End, Tid),
- End - Start + 1
+ _ = ra_seq:fold(fun (I, Acc) ->
+ _ = ets:delete(Tid, I),
+ Acc
+ end, undefined, Seq),
+ NumToDelete
end;
delete({Op, Tid, Idx})
when is_integer(Idx) and is_atom(Op) ->
@@ -297,11 +346,15 @@ range_overlap(ReqRange, #?MODULE{} = State) ->
-spec range(state()) ->
undefined | {ra:index(), ra:index()}.
-range(#?MODULE{range = Range,
+range(#?MODULE{indexes = Seq,
prev = undefined}) ->
- Range;
-range(#?MODULE{range = {_, End} = Range,
+ ra_seq:range(Seq);
+range(#?MODULE{indexes = []}) ->
+ undefined;
+range(#?MODULE{indexes = Seq,
prev = Prev}) ->
+ End = ra_seq:last(Seq),
+ Range = ra_seq:range(Seq),
PrevRange = ra_range:limit(End, range(Prev)),
ra_range:add(Range, PrevRange);
range(_State) ->
@@ -335,27 +388,28 @@ info(#?MODULE{tid = Tid,
has_previous => Prev =/= undefined
}.
--spec record_flushed(ets:tid(), ra:range(), state()) ->
+-spec record_flushed(ets:tid(), ra_seq:state(), state()) ->
{delete_spec(), state()}.
-record_flushed(TID = Tid, {Start, End},
+record_flushed(TID = Tid, FlushedSeq,
#?MODULE{tid = TID,
- range = Range} = State) ->
- HasExtraEntries = ets:info(Tid, size) > ra_range:size(Range),
- case ?IN_RANGE(End, Range) of
- true when HasExtraEntries ->
- {{'<', Tid, End + 1},
- State#?MODULE{range = ra_range:truncate(End, Range)}};
+ indexes = Seq} = State) ->
+ % HasExtraEntries = ets:info(Tid, size) > ra_seq:length(Seq),
+ End = ra_seq:last(FlushedSeq),
+ case ra_seq:in(End, Seq) of
+ % true when HasExtraEntries ->
+ % {{'<', Tid, End + 1},
+ % State#?MODULE{indexes = ra_range:truncate(End, Seq)}};
true ->
- {{range, Tid, {Start, End}},
- State#?MODULE{range = ra_range:truncate(End, Range)}};
+ {{indexes, Tid, FlushedSeq},
+ State#?MODULE{indexes = ra_seq:floor(End + 1, Seq)}};
false ->
{undefined, State}
end;
-record_flushed(_Tid, _Range, #?MODULE{prev = undefined} = State) ->
+record_flushed(_Tid, _FlushedSeq, #?MODULE{prev = undefined} = State) ->
{undefined, State};
-record_flushed(Tid, Range, #?MODULE{prev = Prev0} = State) ->
+record_flushed(Tid, FlushedSeq, #?MODULE{prev = Prev0} = State) ->
%% TODO: test many levels deep flushes
- {Spec, Prev} = record_flushed(Tid, Range, Prev0),
+ {Spec, Prev} = record_flushed(Tid, FlushedSeq, Prev0),
case range(Prev) of
undefined ->
%% the prev table is now empty and can be deleted,
@@ -367,59 +421,51 @@ record_flushed(Tid, Range, #?MODULE{prev = Prev0} = State) ->
-spec set_first(ra:index(), state()) ->
{[delete_spec()], state()}.
set_first(Idx, #?MODULE{tid = Tid,
- range = Range,
- prev = Prev0} = State)
- when (is_tuple(Range) andalso
- Idx > element(1, Range)) orelse
- Range == undefined ->
- {PrevSpecs, Prev} = case Prev0 of
+ indexes = Seq,
+ prev = Prev0} = State) ->
+ {PrevSpecs, Prev} =
+ case Prev0 of
+ undefined ->
+ {[], undefined};
+ _ ->
+ case set_first(Idx, Prev0) of
+ {[{indexes, PTID, _} | Rem],
+ #?MODULE{tid = PTID} = P} = Res ->
+ %% set_first/2 returned a range spec for
+ %% prev and prev is now empty,
+ %% upgrade to delete spec of whole tid
+ case range(P) of
undefined ->
- {[], undefined};
+ {[{delete, tid(P)} | Rem],
+ prev(P)};
_ ->
- case set_first(Idx, Prev0) of
- {[{range, PTID, _} | Rem],
- #?MODULE{tid = PTID} = P} = Res ->
- %% set_first/2 returned a range spec for
- %% prev and prev is now empty,
- %% upgrade to delete spec of whole tid
- case range(P) of
- undefined ->
- {[{delete, tid(P)} | Rem],
- prev(P)};
- _ ->
- Res
- end;
- Res ->
- Res
- end
- end,
- Specs = case Range of
- {Start, End} ->
- [{range, Tid, {Start, min(Idx - 1, End)}} | PrevSpecs];
- undefined ->
- PrevSpecs
+ Res
+ end;
+ Res ->
+ Res
+ end
+ end,
+ Specs = case Seq of
+ [] ->
+ PrevSpecs;
+ _ ->
+ DeleteSeq = ra_seq:limit(Idx - 1, Seq),
+ [{indexes, Tid, DeleteSeq} | PrevSpecs]
end,
{Specs,
- State#?MODULE{range = ra_range:truncate(Idx - 1, Range),
- prev = Prev}};
-set_first(_Idx, State) ->
- {[], State}.
-
-
-%% internal
-
-update_range_end(Idx, {Start, End})
- when Idx =< End orelse
- Idx == End + 1 ->
- {Start, Idx};
-update_range_end(Idx, undefined) ->
- {Idx, Idx}.
-
-delete(End, End, Tid) ->
- ets:delete(Tid, End);
-delete(Start, End, Tid) ->
- _ = ets:delete(Tid, Start),
- delete(Start+1, End, Tid).
+ State#?MODULE{indexes = ra_seq:floor(Idx, Seq),
+ prev = Prev}}.
+
+
+%% Internal
+
+update_ra_seq(Idx, Seq) ->
+ case ra_seq:last(Seq) of
+ undefined ->
+ ra_seq:append(Idx, Seq);
+ LastIdx when LastIdx == Idx - 1 ->
+ ra_seq:append(Idx, Seq)
+ end.
read_sparse(Indexes, State, Acc) ->
read_sparse(Indexes, State, 0, Acc).
diff --git a/src/ra_range.erl b/src/ra_range.erl
index 7bb53965..7e995b81 100644
--- a/src/ra_range.erl
+++ b/src/ra_range.erl
@@ -25,6 +25,13 @@
-export_type([range/0]).
+-define(IS_RANGE(R), ((is_tuple(R) andalso
+ tuple_size(R) == 2 andalso
+ is_integer(element(1, R)) andalso
+ is_integer(element(2, R))) orelse
+ R == undefined)).
+
+
-spec new(ra:index()) -> range().
new(Start) when is_integer(Start) ->
{Start, Start}.
@@ -35,7 +42,9 @@ new(Start, End)
is_integer(End) andalso
Start =< End ->
{Start, End};
-new(_Start, _End) ->
+new(Start, End)
+ when is_integer(Start) andalso
+ is_integer(End) ->
undefined.
-spec add(AddRange :: range(), CurRange :: range()) -> range().
@@ -71,14 +80,17 @@ limit(CeilExcl, Range)
-spec truncate(ra:index(), range()) -> range().
truncate(UpToIncl, {_Start, End})
when is_integer(UpToIncl) andalso
+ is_integer(End) andalso
UpToIncl >= End ->
undefined;
truncate(UpToIncl, {Start, End})
when is_integer(UpToIncl) andalso
+ is_integer(Start) andalso
UpToIncl >= Start ->
{UpToIncl + 1, End};
truncate(UpToIncl, Range)
- when is_integer(UpToIncl) ->
+ when is_integer(UpToIncl) andalso
+ ?IS_RANGE(Range) ->
Range.
size(undefined) ->
diff --git a/src/ra_seq.erl b/src/ra_seq.erl
new file mode 100644
index 00000000..18703590
--- /dev/null
+++ b/src/ra_seq.erl
@@ -0,0 +1,246 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2017-2025 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries.
+-module(ra_seq).
+
+%% open type
+%% sequences are ordered high -> low but ranges are ordered
+%% {low, high} so a typical sequence could look like
+%% [55, {20, 52}, 3]
+-type state() :: [ra:index() | ra:range()].
+
+-record(i, {seq :: state()}).
+-opaque iter() :: #i{}.
+
+-export_type([state/0,
+ iter/0]).
+
+
+-export([
+ append/2,
+ from_list/1,
+ floor/2,
+ limit/2,
+ add/2,
+ fold/3,
+ expand/1,
+ subtract/2,
+ remove_prefix/2,
+ first/1,
+ last/1,
+ iterator/1,
+ next/1,
+ length/1,
+ in/2,
+ range/1,
+ in_range/2
+ ]).
+
+-spec append(ra:index(), state()) -> state().
+append(Idx, [IdxN1, IdxN2 | Rem])
+ when Idx == IdxN1 + 1 andalso
+ Idx == IdxN2 + 2 ->
+ %% we can compact into a range
+ [{IdxN2, Idx} | Rem];
+append(Idx, [{IdxN, IdxN1} | Rem])
+ when Idx == IdxN1 + 1 ->
+ %% Extend the raage
+ [{IdxN, Idx} | Rem];
+append(Idx, [])
+ when is_integer(Idx) ->
+ [Idx];
+append(Idx, [Prev | _] = Seq)
+ when is_integer(Idx) andalso
+ ((is_tuple(Prev) andalso
+ Idx > element(2, Prev)) orelse
+ Idx > Prev) ->
+ [Idx | Seq].
+
+-spec from_list([ra:index()]) -> state().
+from_list(L) ->
+ lists:foldl(fun append/2, [], lists:sort(L)).
+
+%% @doc This operation is O(n)
+-spec floor(ra:index(), state()) -> state().
+floor(FloorIdxIncl, Seq) when is_list(Seq) ->
+ %% TODO: assert appendable
+ %% for now assume appendable
+ floor0(FloorIdxIncl, Seq, []).
+
+
+-spec limit(ra:index(), state()) -> state().
+limit(CeilIdx, [Last | Rem])
+ when is_integer(Last) andalso
+ Last > CeilIdx ->
+ limit(CeilIdx, Rem);
+limit(CeilIdx, [{_, _} = T | Rem]) ->
+ case ra_range:limit(CeilIdx + 1, T) of
+ undefined ->
+ limit(CeilIdx, Rem);
+ {I, I} ->
+ [I | Rem];
+ {I, I2} when I == I2 - 1 ->
+ [I2, I | Rem];
+ NewRange ->
+ [NewRange | Rem]
+ end;
+limit(_CeilIdx, Seq) ->
+ Seq.
+
+-spec add(state(), state()) -> state().
+add([], Seq2) ->
+ Seq2;
+add(Seq1, Seq2) ->
+ Fst = case lists:last(Seq1) of
+ {I, _} -> I;
+ I -> I
+ end,
+ fold(fun append/2, limit(Fst - 1, Seq2), Seq1).
+
+-spec fold(fun ((ra:index(), Acc) -> Acc), Acc, state()) ->
+ Acc when Acc :: term().
+fold(Fun, Acc0, Seq) ->
+ %% TODO: factor out the lists:seq/2
+ lists:foldr(
+ fun ({S, E}, Acc) ->
+ lists:foldl(Fun, Acc, lists:seq(S, E));
+ (Idx, Acc) ->
+ Fun(Idx, Acc)
+ end, Acc0, Seq).
+
+-spec expand(state()) -> [ra:index()].
+expand(Seq) ->
+ fold(fun (I, Acc) -> [I | Acc] end, [], Seq).
+
+-spec subtract(Min :: state(), Sub :: state()) -> Diff :: state().
+subtract(SeqA, SeqB) ->
+ %% TODO: not efficient at all but good enough for now
+ %% optimise if we end up using this in critical path
+ A = expand(SeqA),
+ B = expand(SeqB),
+ from_list(A -- B).
+
+-spec first(state()) -> undefined | ra:index().
+first([]) ->
+ undefined;
+first(Seq) ->
+ case lists:last(Seq) of
+ {I, _} ->
+ I;
+ I ->
+ I
+ end.
+
+-spec last(state()) -> undefined | ra:index().
+last([]) ->
+ undefined;
+last(Seq) ->
+ case hd(Seq) of
+ {_, I} ->
+ I;
+ I ->
+ I
+ end.
+
+-spec remove_prefix(state(), state()) ->
+ {ok, state()} | {error, not_prefix}.
+remove_prefix(Prefix, Seq) ->
+ P = iterator(Prefix),
+ S = iterator(Seq),
+ drop_prefix(next(P), next(S)).
+
+-spec iterator(state()) -> iter() | end_of_seq.
+iterator(Seq) when is_list(Seq) ->
+ #i{seq = lists:reverse(Seq)}.
+
+-spec next(iter()) -> {ra:index(), iter()} | end_of_seq.
+next(#i{seq = []}) ->
+ end_of_seq;
+next(#i{seq = [Next | Rem]})
+ when is_integer(Next) ->
+ {Next, #i{seq = Rem}};
+next(#i{seq = [{Next, End} | Rem]}) ->
+ case ra_range:new(Next + 1, End) of
+ undefined ->
+ {Next, #i{seq = Rem}};
+ NextRange ->
+ {Next, #i{seq = [NextRange | Rem]}}
+ end.
+
+length(Seq) ->
+ lists:foldl(
+ fun (Idx, Acc) when is_integer(Idx) ->
+ Acc + 1;
+ (Range, Acc) when is_tuple(Range) ->
+ Acc + ra_range:size(Range)
+ end, 0, Seq).
+
+in(_Idx, []) ->
+ false;
+in(Idx, [Idx | _]) ->
+ true;
+in(Idx, [Next | Rem])
+ when is_integer(Next) ->
+ in(Idx, Rem);
+in(Idx, [Range | Rem]) ->
+ case ra_range:in(Idx, Range) of
+ true ->
+ true;
+ false ->
+ in(Idx, Rem)
+ end.
+
+-spec range(state()) -> ra:range().
+range([]) ->
+ undefined;
+range(Seq) ->
+ ra_range:new(first(Seq), last(Seq)).
+
+
+in_range(_Range, []) ->
+ [];
+in_range(undefined, _) ->
+ [];
+in_range({Start, End}, Seq0) ->
+ %% TODO: optimise
+ floor(Start, limit(End, Seq0)).
+
+
+%% Internal functions
+
+drop_prefix({IDX, PI}, {IDX, SI}) ->
+ drop_prefix(next(PI), next(SI));
+drop_prefix(_, end_of_seq) ->
+ %% TODO: is this always right as it includes the case where there is
+ %% more prefex left to drop but nothing in the target?
+ {ok, []};
+drop_prefix(end_of_seq, {Idx, #i{seq = RevSeq}}) ->
+ {ok, add(lists:reverse(RevSeq), [Idx])};
+drop_prefix({PrefIdx, PI}, {Idx, _SI} = I)
+ when PrefIdx < Idx ->
+ drop_prefix(next(PI), I);
+drop_prefix({PrefIdx, _PI}, {Idx, _SI})
+ when Idx < PrefIdx ->
+ {error, not_prefix}.
+
+
+
+floor0(FloorIdx, [Last | Rem], Acc)
+ when is_integer(Last) andalso
+ Last >= FloorIdx ->
+ floor0(FloorIdx, Rem, [Last | Acc]);
+floor0(FloorIdx, [{_, _} = T | Rem], Acc) ->
+ case ra_range:truncate(FloorIdx - 1, T) of
+ undefined ->
+ lists:reverse(Acc);
+ {I, I} ->
+ floor0(FloorIdx, Rem, [I | Acc]);
+ {I, I2} when I == I2 - 1 ->
+ floor0(FloorIdx, Rem, [I, I2 | Acc]);
+ NewRange ->
+ floor0(FloorIdx, Rem, [NewRange | Acc])
+ end;
+floor0(_FloorIdx, _Seq, Acc) ->
+ lists:reverse(Acc).
diff --git a/src/ra_server.erl b/src/ra_server.erl
index 5d820d5b..9860306d 100644
--- a/src/ra_server.erl
+++ b/src/ra_server.erl
@@ -52,7 +52,6 @@
checkpoint/3,
persist_last_applied/1,
update_peer/3,
- register_external_log_reader/2,
update_disconnected_peers/3,
handle_down/5,
handle_node_status/6,
@@ -178,7 +177,8 @@
{notify, #{pid() => [term()]}} |
%% used for tracking valid leader messages
{record_leader_msg, ra_server_id()} |
- start_election_timeout.
+ start_election_timeout |
+ {bg_work, fun(() -> ok) | mfargs()}.
-type effects() :: [effect()].
@@ -232,7 +232,8 @@
counter => counters:counters_ref(),
membership => ra_membership(),
system_config => ra_system:config(),
- has_changed => boolean()
+ has_changed => boolean(),
+ parent => term() %% the supervisor
}.
-type ra_server_info_key() :: machine_version | atom().
@@ -927,9 +928,6 @@ handle_leader({transfer_leadership, ServerId},
transition_to => leader}}},
[{reply, ok}, {send_msg, ServerId, election_timeout, cast}]}
end;
-handle_leader({register_external_log_reader, Pid}, #{log := Log0} = State) ->
- {Log, Effs} = ra_log:register_reader(Pid, Log0),
- {leader, State#{log => Log}, Effs};
handle_leader(force_member_change, State0) ->
{follower, State0#{votes => 0}, [{next_event, force_member_change}]};
handle_leader(Msg, State) ->
@@ -1038,9 +1036,6 @@ handle_candidate({ra_log_event, Evt}, State = #{log := Log0}) ->
{candidate, State#{log => Log}, Effects};
handle_candidate(election_timeout, State) ->
call_for_election(candidate, State);
-handle_candidate({register_external_log_reader, Pid}, #{log := Log0} = State) ->
- {Log, Effs} = ra_log:register_reader(Pid, Log0),
- {candidate, State#{log => Log}, Effs};
handle_candidate(force_member_change, State0) ->
{follower, State0#{votes => 0}, [{next_event, force_member_change}]};
handle_candidate(#info_rpc{term = Term} = Msg,
@@ -1139,9 +1134,6 @@ handle_pre_vote({ra_log_event, Evt}, State = #{log := Log0}) ->
% simply forward all other events to ra_log
{Log, Effects} = ra_log:handle_event(Evt, Log0),
{pre_vote, State#{log => Log}, Effects};
-handle_pre_vote({register_external_log_reader, Pid}, #{log := Log0} = State) ->
- {Log, Effs} = ra_log:register_reader(Pid, Log0),
- {pre_vote, State#{log => Log}, Effs};
handle_pre_vote(force_member_change, State0) ->
{follower, State0#{votes => 0}, [{next_event, force_member_change}]};
handle_pre_vote(#info_rpc{term = Term} = Msg,
@@ -1490,9 +1482,6 @@ handle_follower(election_timeout, State) ->
call_for_election(pre_vote, State);
handle_follower(try_become_leader, State) ->
handle_follower(election_timeout, State);
-handle_follower({register_external_log_reader, Pid}, #{log := Log0} = State) ->
- {Log, Effs} = ra_log:register_reader(Pid, Log0),
- {follower, State#{log => Log}, Effs};
handle_follower(force_member_change,
#{cfg := #cfg{id = Id,
uid = Uid,
@@ -1543,16 +1532,13 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term,
?DEBUG("~ts: receiving snapshot chunk: ~b / ~w, index ~b, term ~b",
[LogId, Num, ChunkFlag, SnapIndex, SnapTerm]),
SnapState0 = ra_log:snapshot_state(Log0),
- {ok, SnapState} = ra_snapshot:accept_chunk(Data, Num, ChunkFlag,
- SnapState0),
+ {ok, SnapState, Effs0} = ra_snapshot:accept_chunk(Data, Num, ChunkFlag,
+ SnapState0),
Reply = #install_snapshot_result{term = CurTerm,
last_term = SnapTerm,
last_index = SnapIndex},
case ChunkFlag of
last ->
- %% this is the last chunk so we can "install" it
- {Log, Effs} = ra_log:install_snapshot({SnapIndex, SnapTerm},
- SnapState, Log0),
%% if the machine version of the snapshot is higher
%% we also need to update the current effective machine configuration
EffMacMod = ra_machine:which_module(Machine, SnapMacVer),
@@ -1568,9 +1554,11 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term,
false ->
Cfg0
end,
-
- {#{cluster := ClusterIds}, MacState} = ra_log:recover_snapshot(Log),
-
+ %% this is the last chunk so we can "install" it
+ {#{cluster := ClusterIds},
+ MacState, Log, Effs} = ra_log:install_snapshot({SnapIndex, SnapTerm},
+ SnapState,
+ EffMacMod, Log0),
OldServerIds = maps:map(fun (_, V) ->
maps:with([voter_status], V)
end, Cluster),
@@ -1593,16 +1581,18 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term,
cluster_index_term => {SnapIndex,
SnapTerm},
cluster => make_cluster(Id, ClusterIds),
- membership => get_membership(ClusterIds, State0),
+ membership =>
+ get_membership(ClusterIds, State0),
machine_state => MacState}),
%% it was the last snapshot chunk so we can revert back to
%% follower status
{follower, persist_last_applied(State), [{reply, Reply} |
- Effs ++ SnapInstalledEffs]};
+ Effs0 ++ Effs ++
+ SnapInstalledEffs]};
next ->
Log = ra_log:set_snapshot_state(SnapState, Log0),
State = update_term(Term, State0#{log => Log}),
- {receive_snapshot, State, [{reply, Reply}]}
+ {receive_snapshot, State, [{reply, Reply} | Effs0]}
end;
handle_receive_snapshot(#append_entries_rpc{term = Term} = Msg,
#{current_term := CurTerm,
@@ -1632,9 +1622,6 @@ handle_receive_snapshot(receive_snapshot_timeout, #{log := Log0} = State) ->
SnapState = ra_snapshot:abort_accept(SnapState0),
Log = ra_log:set_snapshot_state(SnapState, Log0),
{follower, State#{log => Log}, []};
-handle_receive_snapshot({register_external_log_reader, Pid}, #{log := Log0} = State) ->
- {Log, Effs} = ra_log:register_reader(Pid, Log0),
- {receive_snapshot, State#{log => Log}, Effs};
handle_receive_snapshot(#info_rpc{term = Term} = Msg,
#{current_term := CurTerm,
cfg := #cfg{log_id = LogId},
@@ -1706,9 +1693,6 @@ handle_await_condition({ra_log_event, Evt}, State = #{log := Log0}) ->
% simply forward all other events to ra_log
{Log, Effects} = ra_log:handle_event(Evt, Log0),
{await_condition, State#{log => Log}, Effects};
-handle_await_condition({register_external_log_reader, Pid}, #{log := Log0} = State) ->
- {Log, Effs} = ra_log:register_reader(Pid, Log0),
- {await_condition, State#{log => Log}, Effs};
handle_await_condition(Msg, #{condition := #{predicate_fun := Pred} = Cond} = State0) ->
case Pred(Msg, State0) of
{true, State1} ->
@@ -2204,21 +2188,26 @@ log_fold_cache(From, _To, _Cache, Acc) ->
term(), ra_server_state()) ->
{ra_server_state(), effects()}.
update_release_cursor(Index, MacState,
- State = #{log := Log0, cluster := Cluster}) ->
+ #{cfg := #cfg{machine = Machine},
+ log := Log0,
+ cluster := Cluster} = State) ->
MacVersion = index_machine_version(Index, State),
+ MacMod = ra_machine:which_module(Machine, MacVersion),
% simply pass on release cursor index to log
{Log, Effects} = ra_log:update_release_cursor(Index, Cluster,
- MacVersion,
+ MacMod,
MacState, Log0),
{State#{log => Log}, Effects}.
-spec checkpoint(ra_index(), term(), ra_server_state()) ->
{ra_server_state(), effects()}.
checkpoint(Index, MacState,
- State = #{log := Log0, cluster := Cluster}) ->
+ #{cfg := #cfg{machine = Machine},
+ log := Log0, cluster := Cluster} = State) ->
MacVersion = index_machine_version(Index, State),
+ MacMod = ra_machine:which_module(Machine, MacVersion),
{Log, Effects} = ra_log:checkpoint(Index, Cluster,
- MacVersion, MacState, Log0),
+ MacMod, MacState, Log0),
{State#{log => Log}, Effects}.
-spec promote_checkpoint(ra_index(), ra_server_state()) ->
@@ -2254,12 +2243,6 @@ update_peer(PeerId, Update, #{cluster := Peers} = State)
Peer = maps:merge(maps:get(PeerId, Peers), Update),
put_peer(PeerId, Peer, State).
--spec register_external_log_reader(pid(), ra_server_state()) ->
- {ra_server_state(), effects()}.
-register_external_log_reader(Pid, #{log := Log0} = State) ->
- {Log, Effs} = ra_log:register_reader(Pid, Log0),
- {State#{log => Log}, Effs}.
-
-spec update_disconnected_peers(node(), nodeup | nodedown, ra_server_state()) ->
ra_server_state().
update_disconnected_peers(Node, nodeup, #{cluster := Peers} = State) ->
@@ -2308,20 +2291,20 @@ handle_down(RaftState, snapshot_sender, Pid, Info,
"~ts: Snapshot sender process ~w exited with ~W",
[LogId, Pid, Info, 10]),
{leader, peer_snapshot_process_exited(Pid, State), []};
-handle_down(RaftState, snapshot_writer, Pid, Info,
- #{cfg := #cfg{log_id = LogId}, log := Log0} = State)
- when is_pid(Pid) ->
- case Info of
- noproc -> ok;
- normal -> ok;
- _ ->
- ?WARN("~ts: Snapshot write process ~w exited with ~w",
- [LogId, Pid, Info])
- end,
- SnapState0 = ra_log:snapshot_state(Log0),
- SnapState = ra_snapshot:handle_down(Pid, Info, SnapState0),
- Log = ra_log:set_snapshot_state(SnapState, Log0),
- {RaftState, State#{log => Log}, []};
+% handle_down(RaftState, snapshot_writer, Pid, Info,
+% #{cfg := #cfg{log_id = LogId}, log := Log0} = State)
+% when is_pid(Pid) ->
+% case Info of
+% noproc -> ok;
+% normal -> ok;
+% _ ->
+% ?WARN("~ts: Snapshot write process ~w exited with ~w",
+% [LogId, Pid, Info])
+% end,
+% SnapState0 = ra_log:snapshot_state(Log0),
+% SnapState = ra_snapshot:handle_error(Pid, Info, SnapState0),
+% Log = ra_log:set_snapshot_state(SnapState, Log0),
+% {RaftState, State#{log => Log}, []};
handle_down(RaftState, log, Pid, Info, #{log := Log0} = State) ->
{Log, Effects} = ra_log:handle_event({down, Pid, Info}, Log0),
{RaftState, State#{log => Log}, Effects};
diff --git a/src/ra_server_proc.erl b/src/ra_server_proc.erl
index 12eefe74..f98e495a 100644
--- a/src/ra_server_proc.erl
+++ b/src/ra_server_proc.erl
@@ -146,7 +146,8 @@
receive_snapshot_timeout = ?DEFAULT_RECEIVE_SNAPSHOT_TIMEOUT :: non_neg_integer(),
install_snap_rpc_timeout :: non_neg_integer(),
aten_poll_interval = 1000 :: non_neg_integer(),
- counter :: undefined | counters:counters_ref()
+ counter :: undefined | counters:counters_ref(),
+ worker_pid :: pid()
}).
-record(state, {conf :: #conf{},
@@ -301,18 +302,14 @@ multi_statem_call([ServerId | ServerIds], Msg, Errs, Timeout) ->
%%%===================================================================
init(#{reply_to := ReplyTo} = Config) ->
- %% we have a reply to key, perform init async
{ok, post_init, maps:remove(reply_to, Config),
- [{next_event, internal, {go, ReplyTo}}]};
-init(Config) ->
- %% no reply_to key, must have been started by an older node run synchronous
- %% init
- State = do_init(Config),
- {ok, recover, State, [{next_event, cast, go}]}.
+ [{next_event, internal, {go, ReplyTo}}]}.
do_init(#{id := Id,
uid := UId,
- cluster_name := ClusterName} = Config0) ->
+ parent := ParentPid,
+ cluster_name := ClusterName} = Config0)
+ when is_pid(ParentPid) ->
Key = ra_lib:ra_server_id_to_local_name(Id),
true = ets:insert(ra_state, {Key, init, unknown}),
process_flag(trap_exit, true),
@@ -361,6 +358,16 @@ do_init(#{id := Id,
?DEFAULT_RECEIVE_SNAPSHOT_TIMEOUT),
AtenPollInt = application:get_env(aten, poll_interval, 1000),
LogId = ra_server:log_id(ServerState),
+ %% TODO: full error handling
+ WorkerPid = case ra_server_sup:start_ra_worker(ParentPid, Config) of
+ {ok, P} -> P;
+ {error, {already_started, P}} ->
+ P
+ end,
+ ra_env:configure_logger(logger),
+ %% monitor worker process, it is easier to handle than linking as we're
+ %% already processing all downs
+ _ = monitor(process, WorkerPid),
State = #state{conf = #conf{log_id = LogId,
cluster_name = ClusterName,
name = Key,
@@ -372,7 +379,8 @@ do_init(#{id := Id,
install_snap_rpc_timeout = InstallSnapRpcTimeout,
receive_snapshot_timeout = ReceiveSnapshotTimeout,
aten_poll_interval = AtenPollInt,
- counter = Counter},
+ counter = Counter,
+ worker_pid = WorkerPid},
low_priority_commands = ra_ets_queue:new(),
server_state = ServerState},
ok = net_kernel:monitor_nodes(true, [nodedown_reason]),
@@ -919,8 +927,9 @@ receive_snapshot(EventType, Msg, State0) ->
TimeoutActions = case Msg of
#install_snapshot_rpc{} ->
%% Reset timeout only on receive snapshot progress.
- [{state_timeout, Conf#conf.receive_snapshot_timeout,
- receive_snapshot_timeout}];
+ [{state_timeout,
+ Conf#conf.receive_snapshot_timeout,
+ receive_snapshot_timeout}];
_ ->
[]
end,
@@ -1550,7 +1559,7 @@ handle_effect(leader, {send_snapshot, {_, ToNode} = To, {SnapState, Id, Term}},
SS = ra_server:update_peer(To, #{status => disconnected}, SS0),
{State0#state{server_state = SS}, Actions}
end;
-handle_effect(_, {delete_snapshot, Dir, SnapshotRef}, _, State0, Actions) ->
+handle_effect(_, {delete_snapshot, Dir, SnapshotRef}, _, State0, Actions) ->
%% delete snapshots in separate process
_ = spawn(fun() ->
ra_snapshot:delete(Dir, SnapshotRef)
@@ -1641,6 +1650,11 @@ handle_effect(follower, {record_leader_msg, _LeaderId}, _, State0, Actions) ->
handle_effect(_, {record_leader_msg, _LeaderId}, _, State0, Actions) ->
%% non follower states don't need to reset state timeout after an effect
{State0, Actions};
+handle_effect(_, {bg_work, FunOrMfa, ErrFun}, _,
+ #state{conf = #conf{worker_pid = WorkerPid}} = State0, Actions) ->
+ %% non follower states don't need to reset state timeout after an effect
+ ra_worker:queue_work(WorkerPid, FunOrMfa, ErrFun),
+ {State0, Actions};
handle_effect(_, _, _, State0, Actions) ->
{State0, Actions}.
@@ -2055,6 +2069,11 @@ handle_node_status_change(Node, Status, InfoList, RaftState,
monitors = Monitors}),
{keep_state, State, Actions}.
+handle_process_down(Pid, Info, _RaftState,
+ #state{conf = #conf{worker_pid = Pid}} = State) ->
+ ?WARN("~ts: worker exited with ~w",
+ [log_id(State), Info]),
+ {stop, Info, State};
handle_process_down(Pid, Info, RaftState,
#state{monitors = Monitors0,
pending_notifys = Nots,
diff --git a/src/ra_server_sup.erl b/src/ra_server_sup.erl
index 3857996c..11c39516 100644
--- a/src/ra_server_sup.erl
+++ b/src/ra_server_sup.erl
@@ -11,6 +11,7 @@
%% API functions
-export([start_link/1]).
+-export([start_ra_worker/2]).
%% Supervisor callbacks
-export([init/1]).
@@ -20,7 +21,18 @@
%%%===================================================================
start_link(Config) ->
- supervisor:start_link(?MODULE, [Config]).
+ supervisor:start_link(?MODULE, Config).
+
+-spec start_ra_worker(pid(), ra_server:config()) ->
+ supervisor:startchild_ret().
+start_ra_worker(SupPid, Config)
+ when is_pid(SupPid) andalso
+ is_map(Config) ->
+ RaWorker = #{id => ra_worker,
+ type => worker,
+ restart => transient,
+ start => {ra_worker, start_link, [Config]}},
+ supervisor:start_child(SupPid, RaWorker).
%%%===================================================================
%%% Supervisor callbacks
@@ -28,20 +40,20 @@ start_link(Config) ->
%%--------------------------------------------------------------------
-init([Config0]) ->
+init(Config0) ->
Id = maps:get(id, Config0),
Config = Config0#{parent => self()},
Name = ra_lib:ra_server_id_to_local_name(Id),
- SupFlags = #{strategy => one_for_one,
+ SupFlags = #{strategy => one_for_all,
intensity => 2,
period => 5},
- ChildSpec = #{id => Name,
- type => worker,
- % needs to be transient as may shut itself down by returning
- % {stop, normal, State}
- restart => transient,
- start => {ra_server_proc, start_link, [Config]}},
- {ok, {SupFlags, [ChildSpec]}}.
+ RaServer = #{id => Name,
+ type => worker,
+ % needs to be transient as may shut itself down by returning
+ % {stop, normal, State}
+ restart => transient,
+ start => {ra_server_proc, start_link, [Config]}},
+ {ok, {SupFlags, [RaServer]}}.
%%%===================================================================
%%% Internal functions
diff --git a/src/ra_snapshot.erl b/src/ra_snapshot.erl
index be6ce2af..f4bd0456 100644
--- a/src/ra_snapshot.erl
+++ b/src/ra_snapshot.erl
@@ -28,9 +28,9 @@
directory/2,
last_index_for/1,
- begin_snapshot/4,
+ begin_snapshot/5,
promote_checkpoint/2,
- complete_snapshot/3,
+ complete_snapshot/4,
begin_accept/2,
accept_chunk/4,
@@ -38,16 +38,21 @@
context/2,
- handle_down/3,
+ handle_error/3,
current_snapshot_dir/1,
latest_checkpoint/1,
take_older_checkpoints/2,
- take_extra_checkpoints/1
+ take_extra_checkpoints/1,
+
+ make_snapshot_dir/3,
+ write_indexes/2,
+ indexes/1
]).
--type effect() :: {monitor, process, snapshot_writer, pid()}.
+-type effect() :: {monitor, process, snapshot_writer, pid()} |
+ {bg_work, fun(), fun()}.
-type kind() :: snapshot | checkpoint.
@@ -79,7 +84,7 @@
%% like snapshots, these are also stored in subdirs
%% as /checkpoints/Term_Index
checkpoint_directory :: file:filename_all(),
- pending :: option({pid(), ra_idxterm(), kind()}),
+ pending :: option({ra_idxterm(), kind()}),
accepting :: option(#accept{}),
current :: option(ra_idxterm()),
checkpoints = [] :: list(checkpoint()),
@@ -192,12 +197,13 @@ find_snapshots(#?MODULE{uid = UId,
ok = delete_snapshots(SnapshotsDir, Snaps),
%% initialise snapshots table even if no snapshots have been taken
%% this ensure these is an entry when the WAL queries it
- true = ets:insert(?ETSTBL, {UId, -1}),
+ ok = ra_log_snapshot_state:insert(?ETSTBL, UId, -1, 0, []),
State;
Current0 ->
Current = filename:join(SnapshotsDir, Current0),
{ok, #{index := Idx, term := Term}} = Module:read_meta(Current),
- true = ets:insert(?ETSTBL, {UId, Idx}),
+ %% TODO: recover live indexes and record that
+ ok = ra_log_snapshot_state:insert(?ETSTBL, UId, Idx, Idx+1, []),
ok = delete_snapshots(SnapshotsDir, lists:delete(Current0, Snaps)),
%% delete old snapshots if any
@@ -329,7 +335,7 @@ current(#?MODULE{current = Current}) -> Current.
latest_checkpoint(#?MODULE{checkpoints = [Current | _]}) -> Current;
latest_checkpoint(#?MODULE{checkpoints = _}) -> undefined.
--spec pending(state()) -> option({pid(), ra_idxterm(), kind()}).
+-spec pending(state()) -> option({ra_idxterm(), kind()}).
pending(#?MODULE{pending = Pending}) ->
Pending.
@@ -345,16 +351,17 @@ directory(#?MODULE{checkpoint_directory = Dir}, checkpoint) -> Dir.
-spec last_index_for(ra_uid()) -> option(ra_index()).
last_index_for(UId) ->
- case ets:lookup(?ETSTBL, UId) of
- [{_, Index}] when Index >= 0 ->
+ case ra_log_snapshot_state:snapshot(?ETSTBL, UId) of
+ Index when Index >= 0 ->
Index;
_ ->
undefined
end.
--spec begin_snapshot(meta(), ReleaseCursorRef :: term(), kind(), state()) ->
+-spec begin_snapshot(meta(), MacModule :: module(),
+ MacStateb :: term(), kind(), state()) ->
{state(), [effect()]}.
-begin_snapshot(#{index := Idx, term := Term} = Meta, MacRef, SnapKind,
+begin_snapshot(#{index := Idx, term := Term} = Meta, MacMod, MacState, SnapKind,
#?MODULE{module = Mod,
counter = Counter,
snapshot_directory = SnapshotDir,
@@ -371,13 +378,18 @@ begin_snapshot(#{index := Idx, term := Term} = Meta, MacRef, SnapKind,
Sync = SnapKind =:= snapshot,
%% create directory for this snapshot
SnapDir = make_snapshot_dir(Dir, Idx, Term),
- %% call prepare then write_snapshot
+ %% TODO: really we'd like to run this in the ra worker as good potentially
+ %% be quite large and a touch expensive to compute but also we don't want
+ %% to close over both the MacState and the MacRef
+ LiveIndexes = ra_seq:from_list(ra_machine:live_indexes(MacMod, MacState)),
+ %% call prepare then write_snapshotS
%% This needs to be called in the current process to "lock" potentially
%% mutable machine state
- Ref = Mod:prepare(Meta, MacRef),
+ Ref = Mod:prepare(Meta, MacState),
%% write the snapshot in a separate process
Self = self(),
- Pid = spawn(fun () ->
+ IdxTerm = {Idx, Term},
+ BgWorkFun = fun () ->
ok = ra_lib:make_dir(SnapDir),
case Mod:write(SnapDir, Meta, Ref, Sync) of
ok -> ok;
@@ -386,15 +398,23 @@ begin_snapshot(#{index := Idx, term := Term} = Meta, MacRef, SnapKind,
BytesWritten),
ok
end,
+ %% write the live indexes, if any
+ case LiveIndexes of
+ [] -> ok;
+ _ ->
+ ok = write_indexes(SnapDir, LiveIndexes),
+ ok
+ end,
Self ! {ra_log_event,
- {snapshot_written, {Idx, Term}, SnapKind}},
+ {snapshot_written, IdxTerm,
+ LiveIndexes, SnapKind}},
ok
- end),
+ end,
%% record snapshot in progress
%% emit an effect that monitors the current snapshot attempt
- {State#?MODULE{pending = {Pid, {Idx, Term}, SnapKind}},
- [{monitor, process, snapshot_writer, Pid}]}.
+ {State#?MODULE{pending = {{Idx, Term}, SnapKind}},
+ [{bg_work, BgWorkFun, err_fun(IdxTerm, SnapKind)}]}.
-spec promote_checkpoint(Idx :: ra_index(), State0 :: state()) ->
{boolean(), State :: state(), Effects :: [effect()]}.
@@ -411,20 +431,28 @@ promote_checkpoint(PromotionIdx,
Checkpoint = make_snapshot_dir(CheckpointDir, Idx, Term),
Snapshot = make_snapshot_dir(SnapDir, Idx, Term),
Self = self(),
- Pid = spawn(fun() ->
- %% Checkpoints are created without calling
- %% fsync. Snapshots must be fsync'd though, so
- %% sync the checkpoint before promoting it
- %% into a snapshot.
- ok = Mod:sync(Checkpoint),
- ok = prim_file:rename(Checkpoint, Snapshot),
- Self ! {ra_log_event,
- {snapshot_written,
- {Idx, Term}, snapshot}}
- end),
- State = State0#?MODULE{pending = {Pid, {Idx, Term}, snapshot},
+ Fun = fun() ->
+ %% Checkpoints are created without calling
+ %% fsync. Snapshots must be fsync'd though, so
+ %% sync the checkpoint before promoting it
+ %% into a snapshot.
+ ok = Mod:sync(Checkpoint),
+ ok = prim_file:rename(Checkpoint, Snapshot),
+ F = filename:join(SnapDir, <<"indexes">>),
+ Indexes = case file:read_file(F) of
+ {ok, Bin} ->
+ binary_to_term(Bin);
+ _ ->
+ []
+ end,
+ Self ! {ra_log_event,
+ {snapshot_written, {Idx, Term},
+ Indexes, snapshot}}
+ end,
+
+ State = State0#?MODULE{pending = {{Idx, Term}, snapshot},
checkpoints = Checkpoints},
- {true, State, [{monitor, process, snapshot_writer, Pid}]};
+ {true, State, [{bg_work, Fun, err_fun({Idx, Term}, snapshot)}]};
undefined ->
{false, State0, []}
end.
@@ -448,14 +476,27 @@ find_promotable_checkpoint(Idx, [CP | Rest], Acc) ->
find_promotable_checkpoint(_Idx, [], _Acc) ->
undefined.
--spec complete_snapshot(ra_idxterm(), kind(), state()) ->
+-spec complete_snapshot(ra_idxterm(), kind(), ra_seq:state(), state()) ->
state().
-complete_snapshot({Idx, _} = IdxTerm, snapshot,
+complete_snapshot(_IdxTerm, snapshot, _LiveIndexes,
+ #?MODULE{pending = undefined} = State) ->
+ %% if pending=undefined it means and snapshot installation with a higher
+ %% index was accepted concurrently
+ State;
+complete_snapshot({Idx, _} = IdxTerm, snapshot, LiveIndexes,
#?MODULE{uid = UId} = State) ->
- true = ets:insert(?ETSTBL, {UId, Idx}),
+ SmallestIdx = case ra_seq:first(LiveIndexes) of
+ undefined ->
+ Idx + 1;
+ I ->
+ I
+ end,
+ %% TODO live indexes
+ ok = ra_log_snapshot_state:insert(?ETSTBL, UId, Idx, SmallestIdx,
+ LiveIndexes),
State#?MODULE{pending = undefined,
current = IdxTerm};
-complete_snapshot(IdxTerm, checkpoint,
+complete_snapshot(IdxTerm, checkpoint, _LiveIndexes,
#?MODULE{checkpoints = Checkpoints0} = State) ->
State#?MODULE{pending = undefined,
checkpoints = [IdxTerm | Checkpoints0]}.
@@ -472,12 +513,13 @@ begin_accept(#{index := Idx, term := Term} = Meta,
state = AcceptState}}}.
-spec accept_chunk(term(), non_neg_integer(), chunk_flag(), state()) ->
- {ok, state()}.
+ {ok, state(), [effect()]}.
accept_chunk(Chunk, Num, last,
#?MODULE{uid = UId,
module = Mod,
snapshot_directory = Dir,
current = Current,
+ pending = Pending,
accepting = #accept{next = Num,
idxterm = {Idx, _} = IdxTerm,
state = AccState}} = State) ->
@@ -485,13 +527,26 @@ accept_chunk(Chunk, Num, last,
ok = Mod:complete_accept(Chunk, AccState),
%% run validate here?
%% delete the current snapshot if any
- _ = spawn(fun () -> delete(Dir, Current) end),
+ Dels = case Pending of
+ undefined ->
+ [Current];
+ {PendIdxTerm, _} ->
+ [Current, PendIdxTerm]
+ end,
+ Eff = {bg_work,
+ fun() -> [delete(Dir, Del) || Del <- Dels] end,
+ fun (_) -> ok end},
+
%% update ets table
- true = ets:insert(?ETSTBL, {UId, Idx}),
+ % true = ets:insert(?ETSTBL, {UId, Idx}),
+
+ %% TODO: move this to install_snapshot so we can work out the
+ %% live indexes
+ ok = ra_log_snapshot_state:insert(?ETSTBL, UId, Idx, Idx+1, []),
{ok, State#?MODULE{accepting = undefined,
%% reset any pending snapshot writes
pending = undefined,
- current = IdxTerm}};
+ current = IdxTerm}, [Eff]};
accept_chunk(Chunk, Num, next,
#?MODULE{module = Mod,
accepting =
@@ -499,12 +554,13 @@ accept_chunk(Chunk, Num, next,
next = Num} = Accept} = State) ->
{ok, AccState} = Mod:accept_chunk(Chunk, AccState0),
{ok, State#?MODULE{accepting = Accept#accept{state = AccState,
- next = Num + 1}}};
+ next = Num + 1}},
+ []};
accept_chunk(_Chunk, Num, _ChunkFlag,
#?MODULE{accepting = #accept{next = Next}} = State)
when Next > Num ->
%% this must be a resend - we can just ignore it
- {ok, State}.
+ {ok, State, []}.
-spec abort_accept(state()) -> state().
abort_accept(#?MODULE{accepting = undefined} = State) ->
@@ -527,27 +583,22 @@ context(#?MODULE{module = Mod}, Node) ->
--spec handle_down(pid(), Info :: term(), state()) ->
+-spec handle_error({ra:index(), ra_term()}, Error :: term(), state()) ->
state().
-handle_down(_Pid, _Info, #?MODULE{pending = undefined} = State) ->
- State;
-handle_down(_Pid, normal, State) ->
- State;
-handle_down(_Pid, noproc, State) ->
- %% this could happen if the monitor was set up after the process had
- %% finished
- State;
-handle_down(Pid, _Info,
+handle_error(IDX_TERM = IdxTerm, _Error,
#?MODULE{snapshot_directory = SnapshotDir,
checkpoint_directory = CheckpointDir,
- pending = {Pid, IdxTerm, SnapKind}} = State) ->
+ pending = {IDX_TERM, SnapKind}} = State) ->
%% delete the pending snapshot/checkpoint directory
Dir = case SnapKind of
snapshot -> SnapshotDir;
checkpoint -> CheckpointDir
end,
ok = delete(Dir, IdxTerm),
- State#?MODULE{pending = undefined}.
+ State#?MODULE{pending = undefined};
+handle_error(_IdxTerm, _Error, #?MODULE{} = State) ->
+ %% ignore if not referring to the current pending, if any
+ State.
delete(_, undefined) ->
ok;
@@ -656,6 +707,27 @@ take_extra_checkpoints(#?MODULE{checkpoints = Checkpoints0,
{State0, Checks}
end.
+-spec write_indexes(file:filename_all(), ra_seq:state()) ->
+ ok | {error, file:posix()}.
+write_indexes(Dir, Indexes) ->
+ File = filename:join(Dir, <<"indexes">>),
+ ra_lib:write_file(File, term_to_binary(Indexes)).
+
+-spec indexes(file:filename_all()) ->
+ {ok, ra_seq:state()} | {error, file:posix()}.
+indexes(Dir) ->
+ File = filename:join(Dir, <<"indexes">>),
+ case prim_file:read_file(File) of
+ {ok, Bin} ->
+ {ok, binary_to_term(Bin)};
+ {error, enoent} ->
+ %% no indexes
+ {ok, []};
+ Err ->
+ Err
+ end.
+
+
%% Utility
-define(MAX_DIFF, 65_536).
@@ -683,7 +755,12 @@ counters_add(undefined, _, _) ->
counters_add(Counter, Ix, Incr) ->
counters:add(Counter, Ix, Incr).
-
+err_fun(IdxTerm, Kind) ->
+ Self = self(),
+ fun (Error) ->
+ Self ! {ra_log_event,
+ {snapshot_error, IdxTerm, Kind, Error}}
+ end.
-ifdef(TEST).
-include_lib("eunit/include/eunit.hrl").
diff --git a/src/ra_system.erl b/src/ra_system.erl
index 303b8480..6b2080ca 100644
--- a/src/ra_system.erl
+++ b/src/ra_system.erl
@@ -36,7 +36,6 @@
wal_compute_checksums => boolean(),
wal_max_batch_size => non_neg_integer(),
wal_max_entries => undefined | non_neg_integer(),
- wal_write_strategy => default | o_sync | sync_after_notify,
wal_sync_method => datasync | sync | none,
wal_hibernate_after => non_neg_integer(),
wal_garbage_collect => boolean(),
@@ -92,7 +91,6 @@ default_config() ->
WalMaxBatchSize = application:get_env(ra, wal_max_batch_size,
?WAL_DEFAULT_MAX_BATCH_SIZE),
WalMaxEntries = application:get_env(ra, wal_max_entries, undefined),
- WalWriteStrategy = application:get_env(ra, wal_write_strategy, default),
WalSyncMethod = application:get_env(ra, wal_sync_method, datasync),
DataDir = ra_env:data_dir(),
WalDataDir = application:get_env(ra, wal_data_dir, DataDir),
@@ -127,7 +125,6 @@ default_config() ->
wal_compute_checksums => WalComputeChecksums,
wal_max_batch_size => WalMaxBatchSize,
wal_max_entries => WalMaxEntries,
- wal_write_strategy => WalWriteStrategy,
wal_garbage_collect => WalGarbageCollect,
wal_pre_allocate => WalPreAllocate,
wal_sync_method => WalSyncMethod,
diff --git a/src/ra_worker.erl b/src/ra_worker.erl
new file mode 100644
index 00000000..cddd2e03
--- /dev/null
+++ b/src/ra_worker.erl
@@ -0,0 +1,83 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2017-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries.
+%%
+%% @hidden
+-module(ra_worker).
+-behaviour(gen_server).
+
+-export([start_link/1,
+ queue_work/3]).
+
+-export([init/1,
+ handle_call/3,
+ handle_cast/2,
+ handle_info/2,
+ terminate/2,
+ code_change/3]).
+
+-record(state, {}).
+
+%%% ra worker responsible for doing background work for a ra server.
+%%%
+%%% this could include, writing snapshots or checkpoints or log
+%%% compaction
+
+%%%===================================================================
+%%% API functions
+%%%===================================================================
+
+start_link(Config) ->
+ gen_server:start_link(?MODULE, Config, []).
+
+queue_work(Pid, FunOrMfa, ErrFun) when is_pid(Pid) ->
+ gen_server:cast(Pid, {work, FunOrMfa, ErrFun}).
+
+%%%===================================================================
+%%% gen_server callbacks
+%%%===================================================================
+
+init(Config) when is_map(Config) ->
+ process_flag(trap_exit, true),
+ {ok, #state{}}.
+
+handle_call(_, _From, State) ->
+ {reply, ok, State}.
+
+handle_cast({work, FunOrMfa, ErrFun}, State) ->
+ case FunOrMfa of
+ {M, F, Args} ->
+ try erlang:apply(M, F, Args) of
+ _ ->
+ ok
+ catch Type:Err:_Stack ->
+ %% TODO: log
+ ErrFun({Type, Err}),
+ ok
+ end;
+ _ when is_function(FunOrMfa) ->
+ try FunOrMfa() of
+ _ ->
+ ok
+ catch Type:Err:_Stack ->
+ %% TODO: log
+ ErrFun({Type, Err})
+ end,
+ ok
+ end,
+ {noreply, State}.
+
+handle_info(_Info, State) ->
+ {noreply, State}.
+
+terminate(_Reason, _State) ->
+ ok.
+
+code_change(_OldVsn, State, _Extra) ->
+ {ok, State}.
+
+%%%===================================================================
+%%% Internal functions
+%%%===================================================================
diff --git a/test/coordination_SUITE.erl b/test/coordination_SUITE.erl
index e338b52d..24220914 100644
--- a/test/coordination_SUITE.erl
+++ b/test/coordination_SUITE.erl
@@ -1004,15 +1004,15 @@ segment_writer_or_wal_crash_follower(Config) ->
end || I <- lists:seq(1, 10)],
%% stop and restart the follower
- ok = ra:stop_server(Follower),
- ok = ra:restart_server(Follower),
+ ok = ra:stop_server(?SYS, Follower),
+ ok = ra:restart_server(?SYS, Follower),
await_condition(AwaitReplicated, 100),
_ = ct_rpc:call(FollowerNode, ra_log_wal, force_rollover, [ra_log_wal]),
- ok = ra:stop_server(Follower),
- ok = ra:restart_server(Follower),
+ ok = ra:stop_server(?SYS, Follower),
+ ok = ra:restart_server(?SYS, Follower),
await_condition(AwaitReplicated, 100),
@@ -1114,15 +1114,15 @@ segment_writer_or_wal_crash_leader(Config) ->
end || I <- lists:seq(1, 10)],
%% stop and restart the leader
- ok = ra:stop_server(Leader),
- ok = ra:restart_server(Leader),
+ ok = ra:stop_server(?SYS, Leader),
+ ok = ra:restart_server(?SYS, Leader),
await_condition(AwaitReplicated, 100),
_ = ct_rpc:call(LeaderNode, ra_log_wal, force_rollover, [ra_log_wal]),
- ok = ra:stop_server(Leader),
- ok = ra:restart_server(Leader),
+ ok = ra:stop_server(?SYS, Leader),
+ ok = ra:restart_server(?SYS, Leader),
await_condition(AwaitReplicated, 100),
diff --git a/test/ra_2_SUITE.erl b/test/ra_2_SUITE.erl
index 5d6a5fef..ac6740fa 100644
--- a/test/ra_2_SUITE.erl
+++ b/test/ra_2_SUITE.erl
@@ -43,8 +43,6 @@ all_tests() ->
custom_ra_event_formatter,
config_modification_at_restart,
segment_writer_handles_server_deletion,
- %% TODO: mt decide on whether to support this
- % external_reader,
add_member_without_quorum,
force_start_follower_as_single_member,
force_start_follower_as_single_member_nonvoter,
@@ -649,33 +647,6 @@ segment_writer_handles_server_deletion(Config) ->
end,
ok.
-external_reader(Config) ->
- ok = logger:set_primary_config(level, all),
- ServerId = ?config(server_id, Config),
- ClusterName = ?config(cluster_name, Config),
- ok = start_cluster(ClusterName, [ServerId]),
- ra:members(ServerId),
- ok = enqueue(ServerId, msg1),
- [begin
- _ = ra:pipeline_command(ServerId, {enq, N}, no_correlation, normal)
- end || N <- lists:seq(1, 1023)],
- _ = enqueue(ServerId, final),
- R0 = ra:register_external_log_reader(ServerId),
- ok = force_roll_over(),
- receive
- {ra_event, _, {machine, {ra_log_update, _, _, _} = E}} ->
- R1 = ra_log_reader:handle_log_update(E, R0),
- {Entries, _R2} = ra_log_reader:sparse_read(R1, lists:seq(0, 1026), []),
- ct:pal("read ~w ~w", [length(Entries), lists:last(Entries)]),
- %% read all entries
- ok
- after 3000 ->
- flush(),
- exit(ra_log_update_timeout)
- end,
- ra:delete_cluster([ServerId]),
- ok.
-
add_member_without_quorum(Config) ->
ok = logger:set_primary_config(level, all),
%% ra:start_server should fail if the node already exists
diff --git a/test/ra_SUITE.erl b/test/ra_SUITE.erl
index fa3e0bf8..c8b441f6 100644
--- a/test/ra_SUITE.erl
+++ b/test/ra_SUITE.erl
@@ -132,7 +132,6 @@ single_server_processes_command(Config) ->
{_RaName, _} = N1 = nth_server_name(Config, 1),
ok = ra:start_server(default, Name, N1, add_machine(), []),
ok = ra:trigger_election(N1),
- monitor(process, element(1, N1)),
% index is 2 as leaders commit a no-op entry on becoming leaders
{ok, 5, _} = ra:process_command(N1, 5, 2000),
{ok, 10, _} = ra:process_command(N1, 5, 2000),
@@ -789,8 +788,8 @@ consistent_query_after_restart(Config) ->
{ok, _, _} = ra:process_command(A, N, ?PROCESS_COMMAND_TIMEOUT),
application:stop(ra),
restart_ra(DataDir),
- ok = ra:restart_server(A),
- ok = ra:restart_server(B),
+ ok = ra:restart_server(?SYS, A),
+ ok = ra:restart_server(?SYS, B),
?assertMatch({ok, N, _}, ra:consistent_query(A, fun(S) -> S end))
end || N <- lists:seq(1, 30)],
@@ -803,15 +802,15 @@ consistent_query_minority(Config) ->
{ok, _, Leader} = ra:process_command(A, 9,
?PROCESS_COMMAND_TIMEOUT),
[F1, F2] = Cluster -- [Leader],
- ra:stop_server(F1),
- ra:stop_server(F2),
+ ra:stop_server(?SYS, F1),
+ ra:stop_server(?SYS, F2),
{timeout, _} = ra:consistent_query(Leader, fun(S) -> S end),
%% restart after a short sleep so that quorum is restored whilst the next
%% query is executing
_ = spawn(fun() ->
timer:sleep(1000),
- ra:restart_server(F1),
+ ra:restart_server(?SYS, F1),
ok
end),
{ok, 9, _} = ra:consistent_query(Leader, fun(S) -> S end, 10000),
@@ -830,11 +829,11 @@ consistent_query_leader_change(Config) ->
{ok, _, A} = ra:process_command(A, 9, ?PROCESS_COMMAND_TIMEOUT),
%% do two consistent queries, this will put query_index == 2 everywhere
{ok, 9, A} = ra:consistent_query(A, fun(S) -> S end),
- ok = ra:stop_server(E),
+ ok = ra:stop_server(?SYS, E),
{ok, 9, A} = ra:consistent_query(A, fun(S) -> S end),
%% restart B
- ok = ra:stop_server(B),
- ok = ra:restart_server(B),
+ ok = ra:stop_server(?SYS, B),
+ ok = ra:restart_server(?SYS, B),
%% Wait for B to recover and catch up.
{ok, #{log := #{last_written_index_term := CurrentIdxTerm}}, _} =
ra:member_overview(A),
@@ -852,12 +851,12 @@ consistent_query_leader_change(Config) ->
ra_leaderboard:lookup_leader(ClusterName) =:= B
end, 20),
%% restart E
- ok = ra:restart_server(E),
+ ok = ra:restart_server(?SYS, E),
{ok, 9, B} = ra:consistent_query(B, fun(S) -> S end),
- ok = ra:stop_server(A),
- ok = ra:stop_server(C),
- ok = ra:stop_server(D),
+ ok = ra:stop_server(?SYS, A),
+ ok = ra:stop_server(?SYS, C),
+ ok = ra:stop_server(?SYS, D),
%% there is no quorum now so this should time out
case ra:consistent_query(B, fun(S) -> S end, 500) of
@@ -867,9 +866,9 @@ consistent_query_leader_change(Config) ->
ct:fail("consistent query should time out"),
ok
end,
- ok = ra:restart_server(A),
- ok = ra:restart_server(C),
- ok = ra:restart_server(D),
+ ok = ra:restart_server(?SYS, A),
+ ok = ra:restart_server(?SYS, C),
+ ok = ra:restart_server(?SYS, D),
{ok, 9, _} = ra:consistent_query(A, fun(S) -> S end),
{ok, 9, _} = ra:consistent_query(B, fun(S) -> S end),
{ok, 9, _} = ra:consistent_query(C, fun(S) -> S end),
diff --git a/test/ra_checkpoint_SUITE.erl b/test/ra_checkpoint_SUITE.erl
index 8640d7f5..375be3cb 100644
--- a/test/ra_checkpoint_SUITE.erl
+++ b/test/ra_checkpoint_SUITE.erl
@@ -13,6 +13,8 @@
-include_lib("eunit/include/eunit.hrl").
-include("src/ra.hrl").
+-define(MACMOD, ?MODULE).
+
%%%===================================================================
%%% Common Test callbacks
%%%===================================================================
@@ -82,14 +84,17 @@ take_checkpoint(Config) ->
State0 = init_state(Config),
Meta = meta(55, 2, [node()]),
- MacRef = ?FUNCTION_NAME,
- {State1, [{monitor, process, snapshot_writer, Pid}]} =
- ra_snapshot:begin_snapshot(Meta, MacRef, checkpoint, State0),
+ MacState = ?FUNCTION_NAME,
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(Meta, ?MACMOD, MacState, checkpoint, State0),
+
undefined = ra_snapshot:latest_checkpoint(State1),
- {Pid, {55, 2}, checkpoint} = ra_snapshot:pending(State1),
+ {{55, 2}, checkpoint} = ra_snapshot:pending(State1),
+ Fun(),
receive
- {ra_log_event, {snapshot_written, {55, 2} = IdxTerm, checkpoint}} ->
- State = ra_snapshot:complete_snapshot(IdxTerm, checkpoint, State1),
+ {ra_log_event, {snapshot_written, {55, 2} = IdxTerm, Indexes, checkpoint}} ->
+ State = ra_snapshot:complete_snapshot(IdxTerm, checkpoint,
+ Indexes, State1),
undefined = ra_snapshot:pending(State),
{55, 2} = ra_snapshot:latest_checkpoint(State),
ok
@@ -102,20 +107,20 @@ take_checkpoint(Config) ->
take_checkpoint_crash(Config) ->
State0 = init_state(Config),
Meta = meta(55, 2, [node()]),
- MacRef = ?FUNCTION_NAME,
- {State1, [{monitor, process, snapshot_writer, Pid}]} =
- ra_snapshot:begin_snapshot(Meta, MacRef, checkpoint, State0),
+ MacState = ?FUNCTION_NAME,
+ {State1, [{bg_work, _Fun, ErrFun}]} =
+ ra_snapshot:begin_snapshot(Meta, ?MODULE, MacState, checkpoint, State0),
undefined = ra_snapshot:latest_checkpoint(State1),
- {Pid, {55, 2}, checkpoint} = ra_snapshot:pending(State1),
+ {{55, 2}, checkpoint} = ra_snapshot:pending(State1),
+ ErrFun(it_failed),
+ {snapshot_error, {55,2}, checkpoint, Err} =
receive
- {ra_log_event, _} ->
- %% Just pretend the snapshot event didn't happen
- %% and the process instead crashed.
- ok
+ {ra_log_event, Evt} ->
+ Evt
after 10 -> ok
end,
- State = ra_snapshot:handle_down(Pid, it_crashed_dawg, State1),
+ State = ra_snapshot:handle_error({55,2}, Err, State1),
%% If the checkpoint process crashed we just have to consider the
%% checkpoint as faulty and clear it up.
undefined = ra_snapshot:pending(State),
@@ -131,11 +136,14 @@ recover_from_checkpoint_only(Config) ->
{error, no_current_snapshot} = ra_snapshot:recover(State0),
Meta = meta(55, 2, [node()]),
- {State1, [{monitor, process, snapshot_writer, _}]} =
- ra_snapshot:begin_snapshot(Meta, ?FUNCTION_NAME, checkpoint, State0),
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(Meta, ?MODULE, ?FUNCTION_NAME,
+ checkpoint, State0),
+ Fun(),
receive
- {ra_log_event, {snapshot_written, IdxTerm, checkpoint}} ->
- _ = ra_snapshot:complete_snapshot(IdxTerm, checkpoint, State1),
+ {ra_log_event, {snapshot_written, IdxTerm, Indexes, checkpoint}} ->
+ _ = ra_snapshot:complete_snapshot(IdxTerm, checkpoint,
+ Indexes, State1),
ok
after 1000 ->
error(snapshot_event_timeout)
@@ -157,22 +165,28 @@ recover_from_checkpoint_and_snapshot(Config) ->
%% Snapshot.
SnapMeta = meta(55, 2, [node()]),
- {State1, [{monitor, process, snapshot_writer, _}]} =
- ra_snapshot:begin_snapshot(SnapMeta, ?FUNCTION_NAME, snapshot, State0),
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(SnapMeta, ?MODULE, ?FUNCTION_NAME,
+ snapshot, State0),
+ Fun(),
State2 = receive
- {ra_log_event, {snapshot_written, IdxTerm1, snapshot}} ->
- ra_snapshot:complete_snapshot(IdxTerm1, snapshot, State1)
+ {ra_log_event, {snapshot_written, IdxTerm1, Indexes, snapshot}} ->
+ ra_snapshot:complete_snapshot(IdxTerm1, snapshot,
+ Indexes, State1)
after 1000 ->
error(snapshot_event_timeout)
end,
%% Checkpoint at a later index.
CPMeta = meta(105, 3, [node()]),
- {State3, [{monitor, process, snapshot_writer, _}]} =
- ra_snapshot:begin_snapshot(CPMeta, ?FUNCTION_NAME, checkpoint, State2),
+ {State3, [{bg_work, Fun2, _}]} =
+ ra_snapshot:begin_snapshot(CPMeta, ?MODULE, ?FUNCTION_NAME,
+ checkpoint, State2),
+ Fun2(),
receive
- {ra_log_event, {snapshot_written, IdxTerm2, checkpoint}} ->
- _ = ra_snapshot:complete_snapshot(IdxTerm2, checkpoint, State3),
+ {ra_log_event, {snapshot_written, IdxTerm2, Indexes2, checkpoint}} ->
+ _ = ra_snapshot:complete_snapshot(IdxTerm2, checkpoint,
+ Indexes2, State3),
ok
after 1000 ->
error(snapshot_event_timeout)
@@ -195,44 +209,56 @@ newer_snapshot_deletes_older_checkpoints(Config) ->
%% Checkpoint at 25.
CP1Meta = meta(25, 2, [node()]),
- {State1, [{monitor, process, snapshot_writer, _}]} =
- ra_snapshot:begin_snapshot(CP1Meta, ?FUNCTION_NAME, checkpoint, State0),
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(CP1Meta, ?MODULE, ?FUNCTION_NAME,
+ checkpoint, State0),
+ Fun(),
State2 = receive
- {ra_log_event, {snapshot_written, IdxTerm1, checkpoint}} ->
- ra_snapshot:complete_snapshot(IdxTerm1, checkpoint, State1)
+ {ra_log_event, {snapshot_written, IdxTerm1, Indexes, checkpoint}} ->
+ ra_snapshot:complete_snapshot(IdxTerm1, checkpoint,
+ Indexes, State1)
after 1000 ->
error(snapshot_event_timeout)
end,
%% Checkpoint at 35.
CP2Meta = meta(35, 3, [node()]),
- {State3, [{monitor, process, snapshot_writer, _}]} =
- ra_snapshot:begin_snapshot(CP2Meta, ?FUNCTION_NAME, checkpoint, State2),
+ {State3, [{bg_work, Fun2, _}]} =
+ ra_snapshot:begin_snapshot(CP2Meta, ?MODULE, ?FUNCTION_NAME,
+ checkpoint, State2),
+ Fun2(),
State4 = receive
- {ra_log_event, {snapshot_written, IdxTerm2, checkpoint}} ->
- ra_snapshot:complete_snapshot(IdxTerm2, checkpoint, State3)
+ {ra_log_event, {snapshot_written, IdxTerm2, Indexes2, checkpoint}} ->
+ ra_snapshot:complete_snapshot(IdxTerm2, checkpoint,
+ Indexes2, State3)
after 1000 ->
error(snapshot_event_timeout)
end,
%% Checkpoint at 55.
CP3Meta = meta(55, 5, [node()]),
- {State5, [{monitor, process, snapshot_writer, _}]} =
- ra_snapshot:begin_snapshot(CP3Meta, ?FUNCTION_NAME, checkpoint, State4),
+ {State5, [{bg_work, Fun3, _}]} =
+ ra_snapshot:begin_snapshot(CP3Meta, ?MODULE, ?FUNCTION_NAME,
+ checkpoint, State4),
+ Fun3(),
State6 = receive
- {ra_log_event, {snapshot_written, IdxTerm3, checkpoint}} ->
- ra_snapshot:complete_snapshot(IdxTerm3, checkpoint, State5)
+ {ra_log_event, {snapshot_written, IdxTerm3, Indexes3, checkpoint}} ->
+ ra_snapshot:complete_snapshot(IdxTerm3, checkpoint,
+ Indexes3, State5)
after 1000 ->
error(snapshot_event_timeout)
end,
%% Snapshot at 45.
SnapMeta = meta(45, 4, [node()]),
- {State7, [{monitor, process, snapshot_writer, _}]} =
- ra_snapshot:begin_snapshot(SnapMeta, ?FUNCTION_NAME, snapshot, State6),
+ {State7, [{bg_work, Fun4, _}]} =
+ ra_snapshot:begin_snapshot(SnapMeta, ?MODULE, ?FUNCTION_NAME,
+ snapshot, State6),
+ Fun4(),
State8 = receive
- {ra_log_event, {snapshot_written, IdxTerm4, snapshot}} ->
- ra_snapshot:complete_snapshot(IdxTerm4, snapshot, State7)
+ {ra_log_event, {snapshot_written, IdxTerm4, Indexes4, snapshot}} ->
+ ra_snapshot:complete_snapshot(IdxTerm4, snapshot,
+ Indexes4, State7)
after 1000 ->
error(snapshot_event_timeout)
end,
@@ -262,20 +288,26 @@ init_recover_corrupt(Config) ->
%% Take a checkpoint.
Meta1 = meta(55, 2, [node()]),
- {State1, _} = ra_snapshot:begin_snapshot(Meta1, ?FUNCTION_NAME, checkpoint, State0),
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(Meta1, ?MODULE, ?FUNCTION_NAME,
+ checkpoint, State0),
+ Fun(),
State2 = receive
- {ra_log_event, {snapshot_written, {55, 2} = IdxTerm1, checkpoint}} ->
- ra_snapshot:complete_snapshot(IdxTerm1, checkpoint, State1)
+ {ra_log_event, {snapshot_written, {55, 2} = IdxTerm1, Indexes, checkpoint}} ->
+ ra_snapshot:complete_snapshot(IdxTerm1, checkpoint, Indexes, State1)
after 1000 ->
error(snapshot_event_timeout)
end,
%% Take another checkpoint.
Meta2 = meta(165, 2, [node()]),
- {State3, _} = ra_snapshot:begin_snapshot(Meta2, ?FUNCTION_NAME, checkpoint, State2),
+ {State3, [{bg_work, Fun2, _}]} =
+ ra_snapshot:begin_snapshot(Meta2, ?MODULE, ?FUNCTION_NAME,
+ checkpoint, State2),
+ Fun2(),
receive
- {ra_log_event, {snapshot_written, {165, 2} = IdxTerm2, checkpoint}} ->
- _ = ra_snapshot:complete_snapshot(IdxTerm2, checkpoint, State3),
+ {ra_log_event, {snapshot_written, {165, 2} = IdxTerm2, Indexes2, checkpoint}} ->
+ _ = ra_snapshot:complete_snapshot(IdxTerm2, checkpoint, Indexes2, State3),
ok
after 1000 ->
error(snapshot_event_timeout)
@@ -303,22 +335,27 @@ init_recover_multi_corrupt(Config) ->
%% Checkpoint at 55.
CP1Meta = meta(55, 2, [node()]),
- {State1, _} =
- ra_snapshot:begin_snapshot(CP1Meta, ?FUNCTION_NAME, checkpoint, State0),
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(CP1Meta, ?MODULE, ?FUNCTION_NAME,
+ checkpoint, State0),
+ Fun(),
State2 = receive
- {ra_log_event, {snapshot_written, IdxTerm1, checkpoint}} ->
- ra_snapshot:complete_snapshot(IdxTerm1, checkpoint, State1)
+ {ra_log_event, {snapshot_written, IdxTerm1, Indexes, checkpoint}} ->
+ ra_snapshot:complete_snapshot(IdxTerm1, checkpoint, Indexes, State1)
after 1000 ->
error(snapshot_event_timeout)
end,
%% Checkpoint at 165.
CP2Meta = meta(165, 2, [node()]),
- {State3, _} =
- ra_snapshot:begin_snapshot(CP2Meta, ?FUNCTION_NAME, checkpoint, State2),
+ {State3, [{bg_work, Fun2, _}]} =
+ ra_snapshot:begin_snapshot(CP2Meta, ?MODULE, ?FUNCTION_NAME,
+ checkpoint, State2),
+ Fun2(),
State4 = receive
- {ra_log_event, {snapshot_written, IdxTerm2, checkpoint}} ->
- ra_snapshot:complete_snapshot(IdxTerm2, checkpoint, State3)
+ {ra_log_event, {snapshot_written, IdxTerm2, Indexes2, checkpoint}} ->
+ ra_snapshot:complete_snapshot(IdxTerm2, checkpoint,
+ Indexes2, State3)
after 1000 ->
error(snapshot_event_timeout)
end,
@@ -365,3 +402,7 @@ list_checkpoint_dirs(Config) ->
list_snap_dirs(Config) ->
SnapDir = ?config(snap_dir, Config),
filelib:wildcard(filename:join(SnapDir, "*")).
+
+%% ra_machine fakes
+version() -> 1.
+live_indexes(_) -> [].
diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl
index 6c5816c2..78e88e35 100644
--- a/test/ra_log_2_SUITE.erl
+++ b/test/ra_log_2_SUITE.erl
@@ -19,6 +19,7 @@ all() ->
all_tests() ->
[
resend_write_lost_in_wal_crash,
+ resend_after_written_event_lost_in_wal_crash,
resend_write_after_tick,
handle_overwrite,
handle_overwrite_append,
@@ -65,9 +66,8 @@ all_tests() ->
recover_after_snapshot,
updated_segment_can_be_read,
open_segments_limit,
- %% TODO mt: do or deprecate in current minor
- % external_reader,
- write_config
+ write_config,
+ sparse_write
].
groups() ->
@@ -130,7 +130,7 @@ handle_overwrite(Config) ->
{ok, Log1} = ra_log:write([{1, 1, "value"},
{2, 1, "value"}], Log0),
receive
- {ra_log_event, {written, 1, {1, 2}}} -> ok
+ {ra_log_event, {written, 1, [2, 1]}} -> ok
after 2000 ->
exit(written_timeout)
end,
@@ -140,11 +140,11 @@ handle_overwrite(Config) ->
{ok, Log4} = ra_log:write([{2, 2, "value"}], Log3),
% simulate the first written event coming after index 20 has already
% been written in a new term
- {Log, _} = ra_log:handle_event({written, 1, {1, 2}}, Log4),
+ {Log, _} = ra_log:handle_event({written, 1, [2, 1]}, Log4),
% ensure last written has not been incremented
{0, 0} = ra_log:last_written(Log),
{2, 2} = ra_log:last_written(
- element(1, ra_log:handle_event({written, 2, {1, 2}}, Log))),
+ element(1, ra_log:handle_event({written, 2, [2, 1]}, Log))),
ok = ra_log_wal:force_roll_over(ra_log_wal),
_ = deliver_all_log_events(Log, 100),
ra_log:close(Log),
@@ -159,7 +159,7 @@ handle_overwrite_append(Config) ->
{ok, Log1} = ra_log:write([{1, 1, "value"},
{2, 1, "value"}], Log0),
receive
- {ra_log_event, {written, 1, {1, 2}}} -> ok
+ {ra_log_event, {written, 1, [2, 1]}} -> ok
after 2000 ->
flush(),
exit(written_timeout)
@@ -174,11 +174,11 @@ handle_overwrite_append(Config) ->
{3, 3} = ra_log:last_index_term(Log4),
% simulate the first written event coming after index has already
% been written in a new term
- {Log, _} = ra_log:handle_event({written, 1, {1, 2}}, Log4),
+ {Log, _} = ra_log:handle_event({written, 1, [2, 1]}, Log4),
% ensure last written has not been incremented
{1, 1} = ra_log:last_written(Log),
{3, 3} = ra_log:last_written(
- element(1, ra_log:handle_event({written, 3, {2, 3}}, Log))),
+ element(1, ra_log:handle_event({written, 3, [3, 2]}, Log))),
ok = ra_log_wal:force_roll_over(ra_log_wal),
_ = deliver_all_log_events(Log, 100),
ra_log:close(Log),
@@ -202,7 +202,7 @@ receive_segment(Config) ->
{3, 1} = ra_log:last_written(Log2),
% force wal roll over
ok = ra_log_wal:force_roll_over(ra_log_wal),
- % Log3 = deliver_all_log_events(Log2, 1500),
+
Log3 = deliver_log_events_cond(
Log2, fun (L) ->
#{mem_table_range := MtRange} = ra_log:overview(L),
@@ -399,16 +399,18 @@ sparse_read_out_of_range_2(Config) ->
write_and_roll(1, 10, 2, Log0)), 50),
SnapIdx = 10,
%% do snapshot in
- {Log2, _} = ra_log:update_release_cursor(SnapIdx, #{}, 2,
- <<"snap@10">>, Log1),
- {Log3, _} = receive
- {ra_log_event, {snapshot_written, {10, 2},
- snapshot} = Evt} ->
- ra_log:handle_event(Evt, Log2)
- after 5000 ->
- flush(),
- exit(snapshot_written_timeout)
- end,
+ {Log2, Effs} = ra_log:update_release_cursor(SnapIdx, #{}, ?MODULE,
+ <<"snap@10">>, Log1),
+ run_effs(Effs),
+ {Log3, Effs3} = receive
+ {ra_log_event, {snapshot_written, {10, 2}, _,
+ snapshot} = Evt} ->
+ ra_log:handle_event(Evt, Log2)
+ after 5000 ->
+ flush(),
+ exit(snapshot_written_timeout)
+ end,
+ run_effs(Effs3),
Log4 = deliver_all_log_events(Log3, 100),
{SnapIdx, 2} = ra_log:snapshot_index_term(Log4),
@@ -534,26 +536,30 @@ written_event_after_snapshot(Config) ->
Log0 = ra_log_init(Config, #{min_snapshot_interval => 1}),
Log1 = ra_log:append({1, 1, <<"one">>}, Log0),
Log1b = ra_log:append({2, 1, <<"two">>}, Log1),
- {Log2, _} = ra_log:update_release_cursor(2, #{}, 1,
- <<"one+two">>, Log1b),
+ {Log2, Effs} = ra_log:update_release_cursor(2, #{}, ?MODULE,
+ <<"one+two">>, Log1b),
+ run_effs(Effs),
{Log3, _} = receive
- {ra_log_event, {snapshot_written, {2, 1},
+ {ra_log_event, {snapshot_written, {2, 1}, _,
snapshot} = Evt} ->
ra_log:handle_event(Evt, Log2)
after 500 ->
exit(snapshot_written_timeout)
end,
+ %% the written events for indexes [1,2] are delivered after
Log4 = deliver_all_log_events(Log3, 100),
+ ct:pal("Log4 ~p", [ra_log:overview(Log4)]),
% true = filelib:is_file(Snap1),
Log5 = ra_log:append({3, 1, <<"three">>}, Log4),
Log6 = ra_log:append({4, 1, <<"four">>}, Log5),
Log6b = deliver_all_log_events(Log6, 100),
- {Log7, _} = ra_log:update_release_cursor(4, #{}, 1,
- <<"one+two+three+four">>,
- Log6b),
+ {Log7, Effs2} = ra_log:update_release_cursor(4, #{}, ?MODULE,
+ <<"one+two+three+four">>,
+ Log6b),
+ run_effs(Effs2),
_ = receive
- {ra_log_event, {snapshot_written, {4, 1}, snapshot} = E} ->
+ {ra_log_event, {snapshot_written, {4, 1}, _, snapshot} = E} ->
ra_log:handle_event(E, Log7)
after 500 ->
exit(snapshot_written_timeout)
@@ -569,8 +575,9 @@ recover_after_snapshot(Config) ->
Log0 = ra_log_init(Config, #{min_snapshot_interval => 1}),
Log1 = ra_log:append({1, 1, <<"one">>}, Log0),
Log2 = ra_log:append({2, 1, <<"two">>}, Log1),
- {Log3, _} = ra_log:update_release_cursor(2, #{}, 1,
- <<"one+two">>, Log2),
+ {Log3, Effs} = ra_log:update_release_cursor(2, #{}, ?MODULE,
+ <<"one+two">>, Log2),
+ run_effs(Effs),
Log4 = deliver_all_log_events(Log3, 100),
ra_log:close(Log4),
restart_wal(),
@@ -591,7 +598,9 @@ writes_lower_than_snapshot_index_are_dropped(Config) ->
Log1b = deliver_all_log_events(ra_log:append({2, 1, <<"two">>}, Log1), 500),
true = erlang:suspend_process(whereis(ra_log_wal)),
Log2 = write_n(3, 500, 1, Log1b),
- {Log3, _} = ra_log:update_release_cursor(100, #{}, 1, <<"100">>, Log2),
+ {Log3, Effs0} = ra_log:update_release_cursor(100, #{}, ?MODULE,
+ <<"100">>, Log2),
+ run_effs(Effs0),
Log4 = deliver_all_log_events(Log3, 500),
Overview = ra_log:overview(Log4),
@@ -605,7 +614,7 @@ writes_lower_than_snapshot_index_are_dropped(Config) ->
%% no written notifications for anything lower than the snapshot should
%% be received
Log5 = receive
- {ra_log_event, {written, _Term, {From, _To}} = E}
+ {ra_log_event, {written, _Term, [{From, _To}]} = E}
when From == 101 ->
{Log4b, Effs} = ra_log:handle_event(E, Log4),
Log4c = lists:foldl(
@@ -892,6 +901,7 @@ resend_write_lost_in_wal_crash(Config) ->
Log3 = append_n(11, 13, 2, Log2b),
Log4 = receive
{ra_log_event, {resend_write, 10} = Evt} ->
+ ct:pal("resend"),
element(1, ra_log:handle_event(Evt, Log3));
{ra_log_event, {written, 2, {11, 12}}} ->
ct:fail("unexpected gappy write!!")
@@ -908,6 +918,37 @@ resend_write_lost_in_wal_crash(Config) ->
ok.
+resend_after_written_event_lost_in_wal_crash(Config) ->
+ Log0 = ra_log_init(Config),
+ {0, 0} = ra_log:last_index_term(Log0),
+ %% write 1..9
+ Log1 = append_n(1, 10, 2, Log0),
+ Log2 = assert_log_events(Log1, fun (L) ->
+ {9, 2} == ra_log:last_written(L)
+ end),
+ WalPid = whereis(ra_log_wal),
+ %% suspend wal, write an entry then kill it
+ Log2b = append_n(10, 11, 2, Log2),
+ receive
+ {ra_log_event, {written, 2, [10]}} ->
+ %% drop written event to simulate being lost in wal crash
+ ok
+ after 500 ->
+ flush(),
+ ct:fail(resend_write_timeout)
+ end,
+ %% restart wal to get a new pid, shouldn't matter
+ exit(WalPid, kill),
+ wait_for_wal(WalPid),
+ %% write 11..12 which should trigger resend
+ Log3 = append_n(11, 12, 2, Log2b),
+ Log6 = assert_log_events(Log3, fun (L) ->
+ {11, 2} == ra_log:last_written(L)
+ end),
+ {[_, _, _], _} = ra_log_take(9, 11, Log6),
+ ra_log:close(Log6),
+ ok.
+
resend_write_after_tick(Config) ->
meck:new(ra_log_wal, [passthrough]),
WalPid = whereis(ra_log_wal),
@@ -1059,10 +1100,11 @@ snapshot_written_after_installation(Config) ->
Log0 = ra_log_init(Config, #{min_snapshot_interval => 2}),
%% log 1 .. 9, should create a single segment
Log1 = write_and_roll(1, 10, 1, Log0),
- {Log2, _} = ra_log:update_release_cursor(5, #{}, 1,
- <<"one-five">>, Log1),
+ {Log2, Effs} = ra_log:update_release_cursor(5, #{}, ?MODULE,
+ <<"one-five">>, Log1),
+ run_effs(Effs),
DelayedSnapWritten = receive
- {ra_log_event, {snapshot_written, {5, 1},
+ {ra_log_event, {snapshot_written, {5, 1}, _,
snapshot} = Evt} ->
Evt
after 1000 ->
@@ -1075,8 +1117,9 @@ snapshot_written_after_installation(Config) ->
Chunk = create_snapshot_chunk(Config, Meta, Context),
SnapState0 = ra_log:snapshot_state(Log2),
{ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0),
- {ok, SnapState} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1),
- {Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, Log2),
+ {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1),
+ run_effs(AEffs),
+ {_, _, Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log2),
%% write some more to create another segment
Log4 = write_and_roll(16, 20, 2, Log3),
{Log5, Efx4} = ra_log:handle_event(DelayedSnapWritten, Log4),
@@ -1104,9 +1147,10 @@ oldcheckpoints_deleted_after_snapshot_install(Config) ->
min_checkpoint_interval => 2}),
%% log 1 .. 9, should create a single segment
Log1 = write_and_roll(1, 10, 1, Log0),
- {Log2, _} = ra_log:checkpoint(5, #{}, 1, <<"one-five">>, Log1),
+ {Log2, Effs} = ra_log:checkpoint(5, #{}, ?MODULE, <<"one-five">>, Log1),
+ run_effs(Effs),
DelayedSnapWritten = receive
- {ra_log_event, {snapshot_written, {5, 1},
+ {ra_log_event, {snapshot_written, {5, 1}, _,
checkpoint} = Evt} ->
Evt
after 1000 ->
@@ -1120,8 +1164,10 @@ oldcheckpoints_deleted_after_snapshot_install(Config) ->
Chunk = create_snapshot_chunk(Config, Meta, Context),
SnapState0 = ra_log:snapshot_state(Log3),
{ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0),
- {ok, SnapState} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1),
- {Log4, Effs4} = ra_log:install_snapshot({15, 2}, SnapState, Log3),
+ {ok, SnapState, AcceptEffs} =
+ ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1),
+ run_effs(AcceptEffs),
+ {_, _, Log4, Effs4} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log3),
?assert(lists:any(fun (E) -> element(1, E) == delete_snapshot end, Effs4)),
%% write some more to create another segment
Log5 = write_and_roll(16, 20, 2, Log4),
@@ -1164,8 +1210,9 @@ snapshot_installation(Config) ->
Chunk = create_snapshot_chunk(Config, Meta, #{}),
SnapState0 = ra_log:snapshot_state(Log2),
{ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0),
- {ok, SnapState} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1),
- {Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, Log2),
+ {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1),
+ run_effs(AEffs),
+ {_, _, Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log2),
{15, _} = ra_log:last_index_term(Log3),
{15, _} = ra_log:last_written(Log3),
@@ -1213,8 +1260,9 @@ append_after_snapshot_installation(Config) ->
Chunk = create_snapshot_chunk(Config, Meta, #{}),
SnapState0 = ra_log:snapshot_state(Log1),
{ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0),
- {ok, SnapState} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1),
- {Log2, _} = ra_log:install_snapshot({15, 2}, SnapState, Log1),
+ {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1),
+ run_effs(AEffs),
+ {_, _, Log2, _} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log1),
{15, _} = ra_log:last_index_term(Log2),
{15, _} = ra_log:last_written(Log2),
@@ -1244,8 +1292,9 @@ written_event_after_snapshot_installation(Config) ->
Chunk = create_snapshot_chunk(Config, Meta, #{}),
SnapState0 = ra_log:snapshot_state(Log1),
{ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0),
- {ok, SnapState} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1),
- {Log2, _} = ra_log:install_snapshot({SnapIdx, 2}, SnapState, Log1),
+ {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1),
+ run_effs(AEffs),
+ {_, _, Log2, _} = ra_log:install_snapshot({SnapIdx, 2}, SnapState, ?MODULE, Log1),
{SnapIdx, _} = ra_log:last_index_term(Log2),
{SnapIdx, _} = ra_log:last_written(Log2),
NextIdx = SnapIdx + 1,
@@ -1281,33 +1330,34 @@ update_release_cursor(Config) ->
% assert there are two segments at this point
[_, _] = find_segments(Config),
% update release cursor to the last entry of the first segment
- {Log2, _} = ra_log:update_release_cursor(127, #{?N1 => new_peer(),
- ?N2 => new_peer()},
- 1, initial_state, Log1),
+ {Log2, Effs} = ra_log:update_release_cursor(127, #{?N1 => new_peer(),
+ ?N2 => new_peer()},
+ ?MODULE, initial_state, Log1),
+ run_effs(Effs),
+ %% ensure snapshot index has been updated and 1 segment deleted
Log3 = assert_log_events(Log2,
fun (L) ->
- {127, 2} == ra_log:snapshot_index_term(L)
+ {127, 2} == ra_log:snapshot_index_term(L) andalso
+ length(find_segments(Config)) == 1
end),
%% now the snapshot_written should have been delivered and the
%% snapshot state table updated
- [{UId, 127}] = ets:lookup(ra_log_snapshot_state, ?config(uid, Config)),
+ UId = ?config(uid, Config),
+ 127 = ra_log_snapshot_state:snapshot(ra_log_snapshot_state, UId),
% this should delete a single segment
- ra_lib:retry(fun () ->
- Segments = find_segments(Config),
- 1 == length(Segments)
- end, 10, 100),
Log3b = validate_fold(128, 149, 2, Log3),
% update the release cursor all the way
- {Log4, _} = ra_log:update_release_cursor(149, #{?N1 => new_peer(),
- ?N2 => new_peer()},
- 1, initial_state, Log3b),
+ {Log4, Effs2} = ra_log:update_release_cursor(149, #{?N1 => new_peer(),
+ ?N2 => new_peer()},
+ ?MODULE, initial_state, Log3b),
+ run_effs(Effs2),
Log5 = assert_log_events(Log4,
fun (L) ->
{149, 2} == ra_log:snapshot_index_term(L)
end),
- [{UId, 149}] = ets:lookup(ra_log_snapshot_state, UId),
+ 149 = ra_log_snapshot_state:snapshot(ra_log_snapshot_state, UId),
% only one segment should remain as the segment writer always keeps
% at least one segment for each
@@ -1317,9 +1367,9 @@ update_release_cursor(Config) ->
% append a few more items
Log = assert_log_events(append_and_roll_no_deliver(150, 155, 2, Log5),
- fun (L) ->
- {154, 2} == ra_log:last_written(L)
- end),
+ fun (L) ->
+ {154, 2} == ra_log:last_written(L)
+ end),
validate_fold(150, 154, 2, Log),
% assert there is only one segment - the current
% snapshot has been confirmed.
@@ -1337,11 +1387,11 @@ update_release_cursor_with_machine_version(Config) ->
% assert there are two segments at this point
[_, _] = find_segments(Config),
% update release cursor to the last entry of the first segment
- MacVer = 2,
- {Log2, _} = ra_log:update_release_cursor(127, #{?N1 => new_peer(),
- ?N2 => new_peer()},
- MacVer,
- initial_state, Log1),
+ {Log2, Effs} = ra_log:update_release_cursor(127, #{?N1 => new_peer(),
+ ?N2 => new_peer()},
+ ?MODULE,
+ initial_state, Log1),
+ run_effs(Effs),
Log = assert_log_events(Log2,
fun (L) ->
{127, 2} == ra_log:snapshot_index_term(L)
@@ -1350,7 +1400,7 @@ update_release_cursor_with_machine_version(Config) ->
%% assert the version is in the snapshot state meta data
CurrentDir = ra_snapshot:current_snapshot_dir(SnapState),
{ok, Meta} = ra_snapshot:read_meta(ra_log_snapshot, CurrentDir),
- ?assertMatch(#{index := 127, machine_version := MacVer}, Meta),
+ ?assertMatch(#{index := 127, machine_version := 1}, Meta),
ok.
missed_mem_table_entries_are_deleted_at_next_opportunity(Config) ->
@@ -1388,9 +1438,12 @@ missed_mem_table_entries_are_deleted_at_next_opportunity(Config) ->
Log5 = validate_fold(1, 154, 2, Log4),
% then update the release cursor
- {Log6, _Effs} = ra_log:update_release_cursor(154, #{?N1 => new_peer(),
+ {Log6, Effs2} = ra_log:update_release_cursor(154, #{?N1 => new_peer(),
?N2 => new_peer()},
- 1, initial_state, Log5),
+ ?MODULE, initial_state, Log5),
+ run_effs(Effs2),
+ ct:pal("Effs2 ~p", [Effs2]),
+ ct:pal("find segments ~p", [find_segments(Config)]),
Log7 = deliver_log_events_cond(Log6,
fun (_) ->
case find_segments(Config) of
@@ -1452,69 +1505,6 @@ open_segments_limit(Config) ->
?assert(Open =< Max),
ok.
-external_reader(Config) ->
- %% external readers should be notified of all new segments
- %% and the lower bound of the log
- %% The upper bound needs to be discovered by querying
- logger:set_primary_config(level, all),
- Log0 = ra_log_init(Config),
- {0, 0} = ra_log:last_index_term(Log0),
-
- Log1 = write_n(200, 220, 2,
- write_and_roll(1, 200, 2, Log0)),
-
- Self = self(),
- Pid = spawn(
- fun () ->
- receive
- {ra_log_reader_state, R1} = Evt ->
- {Es, R2} = ra_log_reader:sparse_read(
- R1, lists:seq(0, 220), []),
- Len1 = length(Es),
- Self ! {got, Evt, Es},
- receive
- {ra_log_update, _, F, _} = Evt2 ->
- %% reader before update has been processed
- %% should work
- Indexes = lists:seq(F, 220),
- {Stale, _} = ra_log_reader:sparse_read(R2, Indexes, []),
- ?assertEqual(Len1, length(Stale)),
- R3 = ra_log_reader:handle_log_update(Evt2, R2),
- {Es2, _R4} = ra_log_reader:sparse_read(R3, Indexes, []),
- ct:pal("Es2 ~w", [length(Es2)]),
- ?assertEqual(Len1, length(Es2)),
- Self ! {got, Evt2, Es2}
- end
- end
- end),
- {Log2, [{reply, {ok, Reader}} | _]} =
- ra_log:register_reader(Pid, Log1),
- Pid ! {ra_log_reader_state, Reader},
- %% TODO: validate there is monitor effect
- receive
- {got, Evt, Entries} ->
- ct:pal("got segs: ~w ~w", [Evt, length(Entries)]),
- ok
- after 2000 ->
- flush(),
- exit(got_timeout)
- end,
- ra_log_wal:force_roll_over(ra_log_wal),
-
- deliver_log_events_cond(
- Log2, fun (_L) ->
- %% this should result in a segment update
- receive
- {got, Evt2, Entries1} ->
- ct:pal("got segs: ~p ~p", [Evt2, length(Entries1)]),
- true
- after 10 ->
- false
- end
- end, 100),
- flush(),
- ok.
-
write_config(Config) ->
C = #{cluster_name => ?MODULE,
@@ -1530,6 +1520,70 @@ write_config(Config) ->
ok.
+sparse_write(Config) ->
+ Log00 = ra_log_init(Config),
+ % assert there are no segments at this point
+ [] = find_segments(Config),
+
+ % create a segment
+
+ Indexes = lists:seq(1, 10, 2),
+ Log0 = write_sparse(Indexes, 0, Log00),
+ Log0b = assert_log_events(Log0,
+ fun (L) ->
+ #{num_pending := Num} = ra_log:overview(L),
+ Num == 0
+ end),
+ {Res0, _Log} = ra_log:sparse_read(Indexes, Log0b),
+ ?assertMatch([{1, _, _},
+ {3, _, _},
+ {5, _, _},
+ {7, _, _},
+ {9, _, _}], Res0),
+
+ %% roll wal and assert we can read sparsely from segments
+ ra_log_wal:force_roll_over(ra_log_wal),
+ Log1 = assert_log_events(Log0b,
+ fun (L) ->
+ #{num_segments := Segs} = ra_log:overview(L),
+ Segs > 0
+ end),
+
+ {Res, Log2} = ra_log:sparse_read(Indexes, Log1),
+ ?assertMatch([{1, _, _},
+ {3, _, _},
+ {5, _, _},
+ {7, _, _},
+ {9, _, _}], Res),
+
+ ct:pal("ov: ~p", [ra_log:overview(Log2)]),
+
+ %% the snapshot is written after live index replication
+ Meta = meta(15, 2, [?N1]),
+ Context = #{},
+ %% passing all Indexes but first one as snapshot state
+ LiveIndexes = tl(Indexes),
+ Chunk = create_snapshot_chunk(Config, Meta, LiveIndexes, Context),
+ SnapState0 = ra_log:snapshot_state(Log2),
+ {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0),
+ {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1),
+ run_effs(AEffs),
+ {_, _, Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log2),
+ {ok, Log} = ra_log:write([{16, 1, <<>>}], Log3),
+ {ResFinal, _} = ra_log:sparse_read(LiveIndexes, Log),
+ ?assertMatch([{3, _, _},
+ {5, _, _},
+ {7, _, _},
+ {9, _, _}], ResFinal),
+
+ ReInitLog= ra_log_init(Config),
+ {ResReInit, _} = ra_log:sparse_read(LiveIndexes, ReInitLog),
+ ?assertMatch([{3, _, _},
+ {5, _, _},
+ {7, _, _},
+ {9, _, _}], ResReInit),
+ ok.
+
validate_fold(From, To, Term, Log0) ->
{Entries0, Log} = ra_log:fold(From, To,
fun (E, A) -> [E | A] end,
@@ -1583,6 +1637,13 @@ write_n(From, To, Term, Log0) ->
{ok, Log} = ra_log:write(Entries, Log0),
Log.
+write_sparse([], _, Log0) ->
+ Log0;
+write_sparse([I | Rem], LastIdx, Log0) ->
+ ct:pal("write_sparse index ~b last ~w", [I, LastIdx]),
+ {ok, Log} = ra_log:write_sparse({I, 1, <>}, LastIdx, Log0),
+ write_sparse(Rem, I, Log).
+
%% Utility functions
deliver_log_events_cond(Log0, _CondFun, 0) ->
@@ -1599,12 +1660,15 @@ deliver_log_events_cond(Log0, CondFun, N) ->
P ! E,
Acc;
({next_event, {ra_log_event, E}}, Acc0) ->
- {Acc, _} = ra_log:handle_event(E, Acc0),
+ {Acc, Effs} = ra_log:handle_event(E, Acc0),
+ run_effs(Effs),
+ Acc;
+ ({bg_work, Fun, _}, Acc) ->
+ Fun(),
Acc;
(_, Acc) ->
Acc
end, Log1, Effs),
- [P ! E || {send_msg, P, E, _} <- Effs],
case CondFun(Log2) of
{false, Log} ->
deliver_log_events_cond(Log, CondFun, N-1);
@@ -1647,7 +1711,6 @@ deliver_all_log_events(Log0, Timeout) ->
(_, Acc) ->
Acc
end, Log1, Effs),
- [P ! E || {send_msg, P, E, _} <- Effs],
% ct:pal("log evt effs: ~p", [Effs]),
deliver_all_log_events(Log, Timeout)
after Timeout ->
@@ -1662,11 +1725,12 @@ assert_log_events(Log0, AssertPred, Timeout) ->
{ra_log_event, Evt} ->
ct:pal("log evt: ~p", [Evt]),
{Log1, Effs} = ra_log:handle_event(Evt, Log0),
+ run_effs(Effs),
%% handle any next events
Log = lists:foldl(
fun ({next_event, {ra_log_event, E}}, Acc0) ->
- ct:pal("eff log evt: ~p", [E]),
- {Acc, _Effs} = ra_log:handle_event(E, Acc0),
+ {Acc, Effs} = ra_log:handle_event(E, Acc0),
+ run_effs(Effs),
Acc;
(_, Acc) ->
Acc
@@ -1771,18 +1835,25 @@ meta(Idx, Term, Cluster) ->
machine_version => 1}.
create_snapshot_chunk(Config, #{index := Idx} = Meta, Context) ->
+ create_snapshot_chunk(Config, #{index := Idx} = Meta, <<"9">>, Context).
+
+create_snapshot_chunk(Config, #{index := Idx} = Meta, MacState, Context) ->
OthDir = filename:join(?config(work_dir, Config), "snapshot_installation"),
CPDir = filename:join(?config(work_dir, Config), "checkpoints"),
ok = ra_lib:make_dir(OthDir),
ok = ra_lib:make_dir(CPDir),
Sn0 = ra_snapshot:init(<<"someotheruid_adsfasdf">>, ra_log_snapshot,
OthDir, CPDir, undefined, ?DEFAULT_MAX_CHECKPOINTS),
- MacRef = <<"9">>,
- {Sn1, _} = ra_snapshot:begin_snapshot(Meta, MacRef, snapshot, Sn0),
+ LiveIndexes = [],
+ {Sn1, [{bg_work, Fun, _ErrFun}]} =
+ ra_snapshot:begin_snapshot(Meta, ?MODULE, MacState, snapshot, Sn0),
+ Fun(),
Sn2 =
receive
- {ra_log_event, {snapshot_written, {Idx, 2} = IdxTerm, snapshot}} ->
- ra_snapshot:complete_snapshot(IdxTerm, snapshot, Sn1)
+ {ra_log_event, {snapshot_written, {Idx, 2} = IdxTerm, _, snapshot}} ->
+ ra_snapshot:complete_snapshot(IdxTerm, snapshot,
+
+ LiveIndexes, Sn1)
after 1000 ->
exit(snapshot_timeout)
end,
@@ -1821,3 +1892,13 @@ wait_for_wal(OldPid) ->
P = whereis(ra_log_wal),
is_pid(P) andalso P =/= OldPid
end, 100, 100).
+run_effs(Effs) ->
+ [Fun() || {bg_work, Fun, _} <- Effs].
+
+%% ra_machine fakes
+version() -> 1.
+live_indexes(MacState) when is_list(MacState) ->
+ %% fake returning live indexes
+ MacState;
+live_indexes(_) ->
+ [].
diff --git a/test/ra_log_SUITE.erl b/test/ra_log_SUITE.erl
index 46c1a0cc..648d8fe0 100644
--- a/test/ra_log_SUITE.erl
+++ b/test/ra_log_SUITE.erl
@@ -22,6 +22,7 @@ all_tests() ->
fetch_not_found,
append_then_fetch,
write_then_fetch,
+ write_sparse_then_fetch,
append_then_fetch_no_wait,
write_then_overwrite,
append_integrity_error,
@@ -98,12 +99,33 @@ write_then_fetch(Config) ->
{Term, _} = ra_log:fetch_term(Idx, Log),
ok.
+write_sparse_then_fetch(Config) ->
+ Log0 = ?config(ra_log, Config),
+ Term = 1,
+ Idx = ra_log:next_index(Log0),
+ Idx5 = Idx + 5,
+ Entry1 = {Idx, Term, "entry"},
+ %% sparse
+ Entry2 = {Idx5, Term, "entry+5"},
+
+ {LastIdx0, _} = ra_log:last_index_term(Log0),
+ {ok, Log1} = ra_log:write_sparse(Entry1, LastIdx0, Log0),
+ {{Idx, Term, "entry"}, Log2} = ra_log:fetch(Idx, Log1),
+ {ok, Log3} = ra_log:write_sparse(Entry2, Idx, Log2),
+ Log = await_written_idx(Idx5, Term, Log3),
+ {Idx5, Term} = ra_log:last_written(Log),
+ {Idx5, _} = ra_log:last_index_term(Log),
+ {{Idx5, Term, "entry+5"}, Log} = ra_log:fetch(Idx5, Log),
+ ok.
+
append_then_fetch_no_wait(Config) ->
Log0 = ?config(ra_log, Config),
Term = 1,
Idx = ra_log:next_index(Log0),
Entry = {Idx, Term, "entry"},
+ ?assertMatch(#{num_pending := 0}, ra_log:overview(Log0)),
Log1 = ra_log:append(Entry, Log0),
+ ?assertMatch(#{num_pending := 1}, ra_log:overview(Log1)),
% check last written hasn't been incremented
{0, 0} = ra_log:last_written(Log1),
% log entry should be immediately visible to allow
@@ -115,10 +137,13 @@ append_then_fetch_no_wait(Config) ->
% results in the last written being updated
receive
{ra_log_event, {written, _, _} = Evt} ->
+ ct:pal("written ~p", [Evt]),
{Log, _} = ra_log:handle_event(Evt, Log3),
- {Idx, Term} = ra_log:last_written(Log)
- after 0 ->
- ok
+ {Idx, Term} = ra_log:last_written(Log),
+ ?assertMatch(#{num_pending := 0}, ra_log:overview(Log))
+ after 1000 ->
+ flush(),
+ ct:pal("fail written event not received")
end,
ok.
@@ -277,3 +302,28 @@ append_in(Term, Data, Log0) ->
ra_log_take(From, To, Log0) ->
{Acc, Log} = ra_log:fold(From, To, fun (E, Acc) -> [E | Acc] end, [], Log0),
{lists:reverse(Acc), Log}.
+
+flush() ->
+ receive
+ Any ->
+ ct:pal("flush ~p", [Any]),
+ flush()
+ after 0 ->
+ ok
+ end.
+
+await_written_idx(Idx, Term, Log0) ->
+ receive
+ {ra_log_event, {written, Term, _Seq} = Evt} ->
+ ct:pal("written ~p", [Evt]),
+ {Log, _} = ra_log:handle_event(Evt, Log0),
+ case ra_log:last_written(Log) of
+ {Idx, Term} ->
+ Log;
+ _ ->
+ await_written_idx(Idx, Term, Log)
+ end
+ after 1000_000 ->
+ flush(),
+ throw(ra_log_append_timeout)
+ end.
diff --git a/test/ra_log_memory.erl b/test/ra_log_memory.erl
index a090b6ac..cad46764 100644
--- a/test/ra_log_memory.erl
+++ b/test/ra_log_memory.erl
@@ -22,7 +22,7 @@
next_index/1,
snapshot_state/1,
set_snapshot_state/2,
- install_snapshot/3,
+ install_snapshot/4,
read_snapshot/1,
recover_snapshot/1,
snapshot_index_term/1,
@@ -41,6 +41,8 @@
-include("src/ra.hrl").
+-dialyzer({nowarn_function, install_snapshot/4}).
+
-type ra_log_memory_meta() :: #{atom() => term()}.
-record(state, {last_index = 0 :: ra_index(),
@@ -50,7 +52,7 @@
meta = #{} :: ra_log_memory_meta(),
snapshot :: option({ra_snapshot:meta(), term()})}).
--type ra_log_memory_state() :: #state{} | ra_log:state().
+-opaque ra_log_memory_state() :: #state{} | ra_log:state().
-export_type([ra_log_memory_state/0]).
@@ -170,17 +172,18 @@ last_written(#state{last_written = LastWritten}) ->
-spec handle_event(ra_log:event_body(), ra_log_memory_state()) ->
{ra_log_memory_state(), list()}.
-handle_event({written, Term, {_From, Idx} = Range0}, State0) ->
+handle_event({written, Term, Seq0}, State0) when is_list(Seq0) ->
+ Idx = ra_seq:last(Seq0),
case fetch_term(Idx, State0) of
{Term, State} ->
{State#state{last_written = {Idx, Term}}, []};
_ ->
- case ra_range:limit(Idx, Range0) of
- undefined ->
+ case ra_seq:limit(Idx - 1, Seq0) of
+ [] ->
% if the term doesn't match we just ignore it
{State0, []};
- Range ->
- handle_event({written, Term, Range}, State0)
+ Seq ->
+ handle_event({written, Term, Seq}, State0)
end
end;
handle_event(_Evt, State0) ->
@@ -210,17 +213,19 @@ fetch_term(Idx, #state{entries = Log} = State) ->
flush(_Idx, Log) -> Log.
-install_snapshot({Index, Term}, Data, #state{entries = Log0} = State) ->
- % Index = maps:get(index, Meta),
- % Term = maps:get(term, Meta),
- % discard log
+-spec install_snapshot(ra_idxterm(), term(), module(), ra_log_memory_state()) ->
+ {ra_snapshot:meta(), term(), ra_log_memory_state(), list()}.
+install_snapshot({Index, Term}, Data, _MacMod,
+ #state{entries = Log0} = State0)
+ when is_tuple(Data) ->
+ % discard log entries below snapshot index
Log = maps:filter(fun (K, _) -> K > Index end, Log0),
- {State#state{entries = Log,
- last_index = Index,
- last_written = {Index, Term},
- snapshot = Data}, []};
-install_snapshot(_Meta, _Data, State) ->
- {State, []}.
+ State = State0#state{entries = Log,
+ last_index = Index,
+ last_written = {Index, Term},
+ snapshot = Data},
+ {Meta, MacState} = Data,
+ {Meta, MacState, State, []}.
-spec read_snapshot(State :: ra_log_memory_state()) ->
{ok, ra_snapshot:meta(), term()}.
diff --git a/test/ra_log_meta_SUITE.erl b/test/ra_log_meta_SUITE.erl
index ae2c7bee..8ce74607 100644
--- a/test/ra_log_meta_SUITE.erl
+++ b/test/ra_log_meta_SUITE.erl
@@ -38,6 +38,7 @@ init_per_group(_, Config) ->
Config.
end_per_group(_, Config) ->
+ application:stop(ra),
Config.
init_per_testcase(TestCase, Config) ->
diff --git a/test/ra_log_props_SUITE.erl b/test/ra_log_props_SUITE.erl
deleted file mode 100644
index 38551e3d..00000000
--- a/test/ra_log_props_SUITE.erl
+++ /dev/null
@@ -1,858 +0,0 @@
-%% This Source Code Form is subject to the terms of the Mozilla Public
-%% License, v. 2.0. If a copy of the MPL was not distributed with this
-%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
-%%
-%% Copyright (c) 2017-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries.
-%%
--module(ra_log_props_SUITE).
--compile(nowarn_export_all).
--compile(export_all).
-
--include_lib("proper/include/proper.hrl").
--include_lib("common_test/include/ct.hrl").
--include_lib("eunit/include/eunit.hrl").
-
-all() ->
- [
- {group, tests}
- ].
-
-%% these tests were useful during the early days but it isn't clear how
-%% much they now contribute
-%% TODO: consider refactoring using a more condensed set of properties
-%% that only test clear log invariants (e.g. overwritten entries are never read).
-all_tests() ->
- [
- write,
- % write_missing_entry,
- % multi_write_missing_entry,
- write_overwrite_entry,
- % write_index_starts_zero,
- append
- % append_missing_entry,
- % append_overwrite_entry,
- % append_index_starts_one,
- % take,
- % take_out_of_range,
- % fetch,
- % fetch_out_of_range,
- % last_index_term,
- % fetch_term,
- % fetch_out_of_range_term,
- % next_index_term,
- % last_written,
- % last_written_with_wal,
- % last_written_with_segment_writer,
- % last_written_with_crashing_segment_writer
- ].
-
-groups() ->
- [
- {tests, [], all_tests()}
- ].
-
-init_per_suite(Config) ->
- {ok, _} = ra:start([{data_dir, ?config(priv_dir, Config)},
- {segment_max_entries, 128}]),
- Config.
-
-end_per_suite(Config) ->
- application:stop(ra),
- Config.
-
-init_per_testcase(TestCase, Config) ->
- application:stop(ra),
- PrivDir = ?config(priv_dir, Config),
- Dir = filename:join(PrivDir, TestCase),
- ra:start_in(Dir),
- % register(TestCase, self()),
- UId = atom_to_binary(TestCase, utf8),
- ok = ra_directory:register_name(default, UId, self(), undefined,
- TestCase, TestCase),
- [{test_case, UId}, {wal_dir, Dir} | Config].
-
-%%------------------
-%% Generators
-%%------------------
-
-log_entries_gen(N) ->
- ?LET(Length, choose(N, 100),
- ?LET(Terms, term_sequence_gen(Length),
- [{Idx, Term, <>}
- || {Idx, Term} <- lists:zip(lists:seq(1, Length),
- Terms)])).
-
-term_sequence_gen(N) ->
- ?LET(List, vector(N, non_neg_integer()),
- lists:sort(List)).
-
-wait_sequence_gen(N) ->
- ?LET(List, vector(N, wait_gen()), List).
-
-wait_gen() ->
- frequency([{8, 0}, {5, choose(0, 20)}, {1, choose(25, 150)}]).
-
-consume_gen(N) ->
- ?LET(List, vector(N, boolean()), List).
-
-log_entry_but_one_gen() ->
- ?LET(Idx, ?SUCHTHAT(Int, integer(), Int =/= 1),
- {Idx, 1, <>}).
-
-log_entry_but_one_zero_gen() ->
- ?LET(Idx, ?SUCHTHAT(Int, integer(), (Int =/= 1) and (Int =/= 0)),
- {Idx, 1, <>}).
-
-slice_gen(Entries) ->
- %% Head might be an empty list
- ?LET(N, choose(1, max_length(Entries)),
- begin
- {Head, [NEntry | Tail]} = lists:split(N - 1, Entries),
- {Head, NEntry, Tail}
- end).
-
-sorted_subset_gen(Entries) ->
- ?LET(Subset, list(elements(Entries)), lists:sort(Subset)).
-
-max_length(Entries) when length(Entries) > 1 ->
- length(Entries) - 1;
-max_length(_) ->
- 1.
-
-range_gen(Entries) ->
- %% Range can finish anywhere after total number of entries
- ?LET(Start, between_gen(1, length(Entries)),
- ?LET(Num, greater_than_gen(1),
- {Start, Num})).
-
-out_of_range_begin_gen() ->
- %% The range starts before the initial index
- ?LET(Start, less_than_gen(0),
- ?LET(Num, greater_than_gen(0),
- {Start, Num})).
-
-out_of_range_end_gen(Entries) ->
- %% The range starts after the last index
- ?LET(Start, greater_than_gen(length(Entries)),
- ?LET(Num, non_neg_integer(),
- {Start, Num})).
-
-between_gen(N, M) ->
- choose(N, M).
-
-greater_than_gen(N) ->
- integer(N + 1, inf).
-
-less_than_gen(N) ->
- integer(inf, N - 1).
-
-out_of_range_gen(Entries) ->
- oneof([out_of_range_begin_gen(),
- out_of_range_end_gen(Entries)]).
-
-%%------------------
-%% Properties
-%%------------------
-
-write(Config) ->
- %% There is no way to create a log file from a list of entries without the write
- %% API. We have to prove first that writing a consecutive log file succeeds,
- %% so we can use it as a base for our tests
- TestCase = ?config(test_case, Config),
- run_proper(fun write_prop/1, [TestCase], 100).
-
-write_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(1),
- begin
- {ok, Log0} = ra_log:write(
- Entries,
- ra_log_init(#{uid => TestCase})),
- {LogEntries, Log} = ra_log_take(1, length(Entries), Log0),
- ?WHENFAIL(io:format("Entries taken from the log: ~p~nRa log state: ~p",
- [LogEntries, Log]),
- Entries == LogEntries)
- end).
-
-append_all([], Log) ->
- Log;
-append_all([Entry | Entries], Log0) ->
- Log = ra_log:append(Entry, Log0),
- append_all(Entries, Log).
-
-write_missing_entry(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun write_missing_entry_prop/1, [TestCase], 100).
-
-write_missing_entry_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(3),
- ?FORALL(
- {Head, _Entry, Tail}, slice_gen(Entries),
- begin
- Log = ra_log_init(#{uid => TestCase}),
- Reply = ra_log:write(Head ++ Tail, Log),
- reset(Log),
- ?WHENFAIL(ct:pal("Reply: ~p", [Reply]),
- case Reply of
- {error, {integrity_error, _}} -> true;
- _ -> false
- end)
- end)).
-
-write_overwrite_entry(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun write_overwrite_entry_prop/1, [TestCase], 250).
-
-write_overwrite_entry_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(3),
- ?FORALL(
- {Head, {Idx, Term, _Value} = _Entry, _Tail}, slice_gen(Entries),
- begin
- {ok, Log0} = ra_log:write(
- Entries,
- ra_log_init(#{uid => TestCase})),
- NewEntry = [{Idx, Term, <<"overwrite">>}],
- {ok, Log} = ra_log:write(NewEntry, Log0),
- {LogEntries, Log1} = ra_log_take(1, length(Entries), Log),
- reset(Log1),
- ?WHENFAIL(io:format("Head: ~p~n New entry: ~p~n"
- "Entries taken from the log: ~p~n"
- "Ra log state: ~p",
- [Head, NewEntry, LogEntries, Log1]),
- ((Head ++ NewEntry) == LogEntries))
- end)).
-
-multi_write_missing_entry(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun multi_write_missing_entry_prop/1, [TestCase], 100).
-
-multi_write_missing_entry_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(3),
- ?FORALL(
- {Head, _Entry, Tail}, slice_gen(Entries),
- begin
- {ok, Log0} = ra_log:write(
- Head,
- ra_log_init(#{uid => TestCase})),
- Reply = ra_log:write(Tail, Log0),
- reset(Log0),
- ?WHENFAIL(io:format("Reply: ~p", [Reply]),
- case Reply of
- {error, {integrity_error, _}} -> true;
- _ -> false
- end)
- end)).
-
-append_missing_entry(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun append_missing_entry_prop/1, [TestCase], 100).
-
-append_missing_entry_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(3),
- ?FORALL(
- {Head, _Entry, Tail}, slice_gen(Entries),
- begin
- Log0 = append_all(Head,
- ra_log_init(#{uid => TestCase})),
- Failed = try
- ra_log:append(hd(Tail), Log0),
- false
- catch
- exit:{integrity_error, _} ->
- true
- end,
- {LogEntries, Log} = ra_log_take(1, length(Head), Log0),
- reset(Log),
- ?WHENFAIL(io:format("Failed: ~p~nHead: ~p~n Tail: ~p~n"
- "Entries taken from the log: ~p~n"
- "Ra log state: ~p",
- [Failed, Head, Tail, LogEntries, Log]),
- (Head == LogEntries) and Failed)
- end)).
-
-write_index_starts_zero(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun write_index_starts_zero_prop/1, [TestCase], 100).
-
-write_index_starts_zero_prop(TestCase) ->
- ?FORALL(
- Entry, log_entry_but_one_zero_gen(),
- begin
- Log = ra_log_init(#{uid => TestCase}),
- Reply = ra_log:write([Entry], Log),
- reset(Log),
- ?WHENFAIL(io:format("Reply: ~p", [Reply]),
- case Reply of
- {error, {integrity_error, _}} -> true;
- _ -> false
- end)
- end).
-
-append(Config) ->
- %% There is no way to create a log file from a list of entries without the
- %% write
- %% API. We have to prove first that writing a consecutive log file succeeds,
- %% so we can use it as a base for our tests
- TestCase = ?config(test_case, Config),
- run_proper(fun append_prop/1, [TestCase], 100).
-
-append_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(1),
- begin
- Log0 = append_all(
- Entries,
- ra_log_init(#{uid => TestCase})),
- {LogEntries, Log} = ra_log_take(1, length(Entries), Log0),
- reset(Log),
- ?WHENFAIL(io:format("Entries taken from the log: ~p~nRa log state: ~p",
- [LogEntries, Log]),
- Entries == LogEntries)
- end).
-
-append_overwrite_entry(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun append_overwrite_entry_prop/1, [TestCase], 100).
-
-append_overwrite_entry_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(3),
- ?FORALL(
- {_Head, {Idx, Term, _Value} = _Entry, _Tail}, slice_gen(Entries),
- begin
- {ok, Log} = ra_log:write(
- Entries,
- ra_log_init(#{uid => TestCase})),
- Failed = try
- ra_log:append({Idx, Term, <<"overwrite">>}, Log),
- false
- catch
- exit:{integrity_error, _} ->
- true
- end,
- reset(Log),
- ?WHENFAIL(io:format("Failed: ~p", [Failed]),
- Failed)
- end)).
-
-append_index_starts_one(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun append_index_starts_one_prop/1, [TestCase], 100).
-
-append_index_starts_one_prop(TestCase) ->
- ?FORALL(
- Entry, log_entry_but_one_gen(),
- begin
- Log = ra_log_init(#{uid => TestCase}),
- Failed = try
- ra_log:append(Entry, Log),
- false
- catch
- exit:{integrity_error, _} ->
- true
- end,
- reset(Log),
- ?WHENFAIL(io:format("Failed: ~p Entry: ~p", [Failed, Entry]), Failed)
- end).
-
-take(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun take_prop/1, [TestCase], 100).
-
-take_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(1),
- ?FORALL(
- {Start, Num}, range_gen(Entries),
- begin
- {ok, Log0} = ra_log:write(
- Entries,
- ra_log_init(#{uid => TestCase})),
- {Selected, Log} = ra_log_take(Start, Start + Num - 1, Log0),
- Expected = lists:sublist(Entries, Start, Num),
- reset(Log),
- ?WHENFAIL(io:format("Selected: ~p~nExpected: ~p",
- [Selected, Expected]),
- Selected == Expected)
- end)).
-
-take_out_of_range(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun take_out_of_range_prop/1, [TestCase], 100).
-
-take_out_of_range_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(1),
- ?FORALL(
- {Start, Num}, out_of_range_gen(Entries),
- begin
- {ok, Log0} = ra_log:write(
- Entries,
- ra_log_init(#{uid => TestCase})),
- {Reply, Log} = ra_log_take(Start, Start + Num - 1, Log0),
- reset(Log),
- ?WHENFAIL(io:format("Start: ~p Num: ~p~nReply: ~p", [Start, Num, Reply]),
- Reply == [])
- end)).
-
-fetch(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun fetch_prop/1, [TestCase], 100).
-
-fetch_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(1),
- ?FORALL(
- {_Head, {Idx, _Term, _Value} = Entry, _Tail}, slice_gen(Entries),
- begin
- {ok, Log0} = ra_log:write(
- Entries,
- ra_log_init(#{uid => TestCase})),
- {Got, Log} = ra_log:fetch(Idx, Log0),
- reset(Log),
- ?WHENFAIL(io:format("Got: ~p Expected: ~p", [Got, Entry]),
- Entry == Got)
- end)).
-
-fetch_out_of_range(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun fetch_out_of_range_prop/1, [TestCase], 100).
-
-fetch_out_of_range_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(1),
- ?FORALL(
- {Start, _Num}, out_of_range_gen(Entries),
- begin
- {ok, Log0} = ra_log:write(
- Entries,
- ra_log_init(#{uid => TestCase})),
- {Reply, Log} = ra_log:fetch(Start, Log0),
- reset(Log),
- ?WHENFAIL(io:format("Got: ~p Expected: undefined", [Reply]),
- Reply == undefined)
- end)).
-
-last_index_term(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun last_index_term_prop/1, [TestCase], 100).
-
-last_index_term_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(0),
- begin
- {ok, Log} = ra_log:write(
- Entries,
- ra_log_init(#{uid => TestCase})),
- {LastIdx, LastTerm} = case Entries of
- [] ->
- {0, 0};
- _ ->
- {LI, LT, _} = lists:last(Entries),
- {LI, LT}
- end,
- {Idx, Term} = ra_log:last_index_term(Log),
- reset(Log),
- ?WHENFAIL(io:format("Got: ~p Expected: ~p", [{Idx, Term}, {LastIdx, LastTerm}]),
- (LastIdx == Idx) and (LastTerm == Term))
- end).
-
-fetch_term(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun fetch_term_prop/1, [TestCase], 100).
-
-fetch_term_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(1),
- ?FORALL(
- {_Head, {Idx, ExpectedTerm, _}, _Tail}, slice_gen(Entries),
- begin
- {ok, Log0} = ra_log:write(
- Entries,
- ra_log_init(#{uid => TestCase})),
- {Term, Log} = ra_log:fetch_term(Idx, Log0),
- reset(Log),
- ?WHENFAIL(io:format("Got: ~p Expected: ~p", [Term, ExpectedTerm]),
- (ExpectedTerm == Term))
- end)).
-
-fetch_out_of_range_term(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun fetch_out_of_range_term_prop/1, [TestCase], 100).
-
-fetch_out_of_range_term_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(1),
- ?FORALL(
- {Start, _}, out_of_range_gen(Entries),
- begin
- {ok, Log0} = ra_log:write(
- Entries,
- ra_log_init(#{uid => TestCase})),
- {Term, Log} = ra_log:fetch_term(Start, Log0),
- reset(Log),
- ?WHENFAIL(io:format("Got: ~p for index: ~p", [Term, Start]),
- (undefined == Term) orelse ((0 == Term) and (Start == 0)))
- end)).
-
-next_index_term(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun last_index_term_prop/1, [TestCase], 100).
-
-next_index_term_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(1),
- begin
- {ok, Log} = ra_log:write(
- Entries,
- ra_log_init(#{uid => TestCase})),
- {LastIdx, _LastTerm, _} = lists:last(Entries),
- Idx = ra_log:next_index(Log),
- reset(Log),
- ?WHENFAIL(io:format("Got: ~p Expected: ~p", [Idx, LastIdx + 1]),
- LastIdx + 1 == Idx)
- end).
-
-
-last_written_with_wal(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun last_written_with_wal_prop/1, [TestCase], 15).
-
-build_action_list(Entries, Actions) ->
- lists:flatten(lists:map(fun(Index) ->
- E = lists:nth(Index, Entries),
- A = lists:foldl(fun({A0, I}, Acc) when I == Index ->
- [A0 | Acc];
- (_, Acc) ->
- Acc
- end, [], Actions),
- [E | A]
- end, lists:seq(1, length(Entries)))).
-
-position(Entries) ->
- choose(1, length(Entries)).
-
-last_written_with_wal_prop(TestCase) ->
- ok = logger:set_primary_config(level, all),
- ?FORALL(
- Entries, log_entries_gen(1),
- ?FORALL(
- Actions, list(frequency([{5, {{wait, wait_gen()}, position(Entries)}},
- {3, {consume, position(Entries)}},
- {2, {roll_wal, position(Entries)}},
- {2, {stop_wal, position(Entries)}},
- {2, {start_wal, position(Entries)}}])),
- begin
- flush(),
- All = build_action_list(Entries, Actions),
- Log0 = ra_log_init(#{uid => TestCase}),
- {Log, Last, LastIdx, _Status} =
- lists:foldl(fun({wait, Wait}, Acc) ->
- timer:sleep(Wait),
- Acc;
- (consume, {Acc0, Last0, LastIdx, St}) ->
- {Acc1, Last1} = consume_events(Acc0, Last0),
- {Acc1, Last1, LastIdx, St};
- (roll_wal, {_, _, _, wal_down} = Acc) ->
- Acc;
- (roll_wal, Acc) ->
- ra_log_wal:force_roll_over(ra_log_wal),
- Acc;
- (stop_wal, {Acc0, Last0, LastIdx, wal_up}) ->
- ok = supervisor:terminate_child(wal_sup(), ra_log_wal),
- {Acc0, Last0, LastIdx, wal_down};
- (stop_wal, {_, _, _, wal_down} = Acc) ->
- Acc;
- (start_wal, {Acc0, Last0, LastIdx, wal_down}) ->
- supervisor:restart_child(wal_sup(), ra_log_wal),
- {Acc0, Last0, LastIdx, wal_up};
- (start_wal, {_, _, _, wal_up} = Acc) ->
- Acc;
- ({Idx, _, _} = Entry, {Acc0, _, LastIdx, _} = Acc) when Idx > LastIdx + 1 ->
- {error, {integrity_error, _}} = ra_log:write([Entry], Acc0),
- Acc;
- (Entry, {Acc0, _, _, wal_down} = Acc) ->
- {error, wal_down} = ra_log:write([Entry], Acc0),
- Acc;
- ({Idx, _, _} = Entry, {Acc0, Last0, _LastIdx, St}) ->
- {ok, Acc} = ra_log:write([Entry], Acc0),
- {Acc, Last0, Idx, St}
- end, {Log0, {0, 0}, 0, wal_up}, All),
- Got = ra_log:last_written(Log),
- {Written, Log1} = ra_log_take(1, LastIdx, Log),
- reset(Log1),
- ?WHENFAIL(io:format("Got: ~p, Expected: ~p Written: ~p~n Actions: ~p",
- [Got, Last, Written, All]),
- (Got == Last) and (Written == lists:sublist(Entries, 1, LastIdx)))
- end)).
-
-last_written_with_segment_writer(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun last_written_with_segment_writer_prop/1, [TestCase], 25).
-
-last_written_with_segment_writer_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(1),
- ?FORALL(
- Actions, list(frequency([{5, {{wait, wait_gen()}, position(Entries)}},
- {3, {consume, position(Entries)}},
- {2, {stop_segment_writer, position(Entries)}},
- {2, {start_segment_writer, position(Entries)}}])),
- begin
- flush(),
- All = build_action_list(Entries, Actions),
- _ = supervisor:restart_child(ra_log_sup, ra_log_segment_writer),
- Log0 = ra_log_init(#{uid => TestCase}),
- {Log, Last, LastIdx, _Status} =
- lists:foldl(fun({wait, Wait}, Acc) ->
- timer:sleep(Wait),
- Acc;
- (consume, {Acc0, Last0, LastIdx, St}) ->
- {Acc1, Last1} = consume_events(Acc0, Last0),
- {Acc1, Last1, LastIdx, St};
- (stop_segment_writer, {Acc0, Last0, LastIdx, sw_up}) ->
- ok = supervisor:terminate_child(ra_log_sup, ra_log_segment_writer),
- {Acc0, Last0, LastIdx, sw_down};
- (stop_segment_writer, {_, _, _, sw_down} = Acc) ->
- Acc;
- (start_segment_writer, {Acc0, Last0, LastIdx, sw_down}) ->
- {ok, _} = supervisor:restart_child(ra_log_sup, ra_log_segment_writer),
- {Acc0, Last0, LastIdx, sw_up};
- (start_segment_writer, {_, _, _, sw_up} = Acc) ->
- Acc;
- ({Idx, _, _} = Entry, {Acc0, _, LastIdx, _} = Acc) when Idx > LastIdx + 1 ->
- {error, {integrity_error, _}} = ra_log:write([Entry], Acc0),
- Acc;
- ({Idx, _, _} = Entry, {Acc0, Last0, _LastIdx, St}) ->
- {ok, Acc} = ra_log:write([Entry], Acc0),
- {Acc, Last0, Idx, St}
- end, {Log0, {0, 0}, 0, sw_up}, All),
- Got = ra_log:last_written(Log),
- {Written, Log1} = ra_log_take(1, LastIdx, Log),
- reset(Log1),
- ?WHENFAIL(ct:pal("Got: ~p, Expected: ~p Written: ~p~n Actions: ~p",
- [Got, Last, Written, All]),
- (Got == Last) and (Written == lists:sublist(Entries, 1, LastIdx)))
- end)).
-
-last_written_with_crashing_segment_writer(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper_noshrink(fun last_written_with_crashing_segment_writer_prop/1,
- [TestCase], 1).
-
-last_written_with_crashing_segment_writer_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(1),
- ?FORALL(
- Actions, list(frequency([{5, {{wait, wait_gen()}, position(Entries)}},
- {3, {consume, position(Entries)}},
- {2, {crash_segment_writer, position(Entries)}}])),
- begin
- flush(),
- All = build_action_list(Entries, Actions),
- _ = supervisor:restart_child(ra_log_sup, ra_log_segment_writer),
- Log0 = ra_log_init(#{uid => TestCase,
- resend_window => 2}),
- ra_log_take(1, 10, Log0),
- {Log, _Last, Ts} =
- lists:foldl(fun({wait, Wait}, Acc) ->
- timer:sleep(Wait),
- Acc;
- (consume, {Acc0, Last0, Ts}) ->
- Acc1 = deliver_log_events(Acc0, 500),
- {Acc1, Last0, Ts};
- (crash_segment_writer, {Acc0, Last0, _Ts}) ->
- Acc = case whereis(ra_log_segment_writer) of
- undefined ->
- Acc0;
- P ->
- Acc1 = deliver_log_events(Acc0, 500),
- exit(P, kill),
- Acc1
- end,
- {Acc, Last0, get_timestamp()};
- (Entry, {Acc0, Last0, Ts}) ->
- case ra_log:write([Entry], Acc0) of
- {ok, Acc} ->
- {Acc, Last0, Ts};
- {error, wal_down} ->
- wait_for_wal(50, 0),
- {ok, Acc} = ra_log:write([Entry], Acc0),
- {Acc, Last0, Ts}
- end
- end, {Log0, {0, 0}, get_timestamp()}, All),
- %% We want to check that eventually we get the last written as the last entry,
- %% despite the segment writer crash. The log file might have to resend
- %% some entries after it, so it needs time to recover.
- timer:sleep(time_diff_to(Ts, 3000)),
- % write an entry to trigger resend protocol if required
- {LastIdx, LastTerm} = ra_log:last_index_term(Log),
- E = {LastIdx+1, LastTerm, <<>>},
- ActuallyLastIdxTerm = {LastIdx+1, LastTerm},
- {ok, Log1a} = ra_log:write([E], Log),
- Log1 = deliver_log_events(Log1a, 500),
- % Log1c = deliver_log_events(Log1b, 500),
- %% Consume all events
- % {Log1, Last1} = consume_events(Log1b, Last),
- %% Request last written
- LastWritten = ra_log:last_written(Log1),
- %% Request entries available, which should be all generated by this test
- {EIdx, ETerm, _} = lists:last(Entries),
- LastEntry = {EIdx, ETerm},
- ct:pal("Log1 ~p~nopen ~p~nclosed~p", [Log1,
- ets:tab2list(ra_log_open_mem_tables),
- ets:tab2list(ra_log_closed_mem_tables)
- ]),
- {Written, Log2} = ra_log_take(1, EIdx, Log1),
- %% We got all the data, can reset now
- basic_reset(Log2),
- ?WHENFAIL(ct:pal("Last written entry: ~p; actually last idx term: ~p;"
- " last entry written: ~p~nEntries taken: ~p~n Actions: ~p",
- [LastWritten, ActuallyLastIdxTerm, LastEntry, Written, Entries]),
- (LastWritten == ActuallyLastIdxTerm)
- and (Written == Entries))
- end)).
-
-get_timestamp() ->
- {Mm, S, Mi} = os:timestamp(),
- (Mm * 1000000 + S) * 1000 + round(Mi / 1000).
-
-time_diff_to(Ts, To) ->
- Tnow = get_timestamp(),
- case To - (Tnow - Ts) of
- T when T < 0 ->
- 0;
- T ->
- T
- end.
-
-wait_for_wal(N, N) ->
- exit(wait_for_wal_timeout);
-wait_for_wal(M, N) ->
- timer:sleep(100),
- case whereis(ra_log_wal) of
- undefined ->
- wait_for_wal(M, N+1);
- _ -> ok
- end.
-
-
-last_written(Config) ->
- TestCase = ?config(test_case, Config),
- run_proper(fun last_written_prop/1, [TestCase], 10).
-
-last_written_prop(TestCase) ->
- ?FORALL(
- Entries, log_entries_gen(1),
- ?FORALL(
- {Waits, Consumes}, {wait_sequence_gen(length(Entries)), consume_gen(length(Entries))},
- begin
- flush(),
- Actions = lists:zip3(Entries, Waits, Consumes),
- Log0 = ra_log_init(#{uid => TestCase}),
- {Log, Last} = lists:foldl(fun({Entry, Wait, Consume}, {Acc0, Last0}) ->
- {ok, Acc} = ra_log:write([Entry], Acc0),
- timer:sleep(Wait),
- case Consume of
- true ->
- consume_events(Acc, Last0);
- false ->
- {Acc, Last0}
- end
- end, {Log0, {0, 0}}, Actions),
- Got = ra_log:last_written(Log),
- reset(Log),
- ?WHENFAIL(io:format("Got: ~p, Expected: ~p~n Actions: ~p",
- [Got, Last, Actions]),
- Got == Last)
- end)).
-
-flush() ->
- receive
- {ra_log_event, _} ->
- flush()
- after 100 ->
- ok
- end.
-
-deliver_log_events(Log0, Timeout) ->
- receive
- {ra_log_event, Evt} ->
- ct:pal("ra_log_evt: ~w", [Evt]),
- {Log, _} = ra_log:handle_event(Evt, Log0),
- deliver_log_events(Log, Timeout)
- after Timeout ->
- Log0
- end.
-
-consume_events(Log0, Last) ->
- receive
- {ra_log_event, {written, Term, {_, To}} = Evt} ->
- {Log, _} = ra_log:handle_event(Evt, Log0),
- consume_events(Log, {To, Term})
- after 0 ->
- {Log0, Last}
- end.
-
-consume_all_events(Log0, Last) ->
- receive
- {ra_log_event, {written, Term, {_, To}} = Evt} ->
- {Log, _} = ra_log:handle_event(Evt, Log0),
- consume_events(Log, {To, Term})
- after 15000 ->
- {Log0, Last}
- end.
-
-last_idx_term([]) ->
- {0,0};
-last_idx_term(List) ->
- {Idx, Term, _} = lists:last(lists:sort(List)),
- {Idx, Term}.
-
-%% TODO refactor
-run_proper(Fun, Args, NumTests) ->
- ?assertEqual(
- true,
- proper:counterexample(erlang:apply(Fun, Args),
- [{numtests, NumTests},
- {on_output, fun(".", _) -> ok; % don't print the '.'s on new lines
- (F, A) -> ct:pal(?LOW_IMPORTANCE, F, A) end}])).
-
-run_proper_noshrink(Fun, Args, NumTests) ->
- ?assertEqual(
- true,
- proper:counterexample(erlang:apply(Fun, Args),
- [{numtests, NumTests},
- noshrink,
- {on_output, fun(".", _) -> ok; % don't print the '.'s on new lines
- (F, A) -> ct:pal(?LOW_IMPORTANCE, F, A) end}])).
-basic_reset(Log) ->
- ra_log:write([{0, 0, empty}], Log),
- receive
- {ra_log_event, {written, _, {0, 0}}} ->
- ok
- end,
- ra_log:close(Log).
-
-reset(Log) ->
- WalSup = wal_sup(),
- supervisor:restart_child(WalSup, ra_log_segment_writer),
- supervisor:restart_child(WalSup, ra_log_wal),
- basic_reset(Log).
-
-wal_sup() ->
- [WalSup] = [P || {ra_log_wal_sup, P, _, _}
- <- supervisor:which_children(ra_log_sup)],
- WalSup.
-
-ra_log_init(Cfg) ->
- %% augment with default system config
- ra_log:init(Cfg#{system_config => ra_system:default_config()}).
-
-ra_log_take(From, To, Log0) ->
- {Acc, Log} = ra_log:fold(From, To, fun (E, Acc) -> [E | Acc] end, [], Log0),
- {lists:reverse(Acc), Log}.
diff --git a/test/ra_log_segment_writer_SUITE.erl b/test/ra_log_segment_writer_SUITE.erl
index 5aed1caa..9cddcd87 100644
--- a/test/ra_log_segment_writer_SUITE.erl
+++ b/test/ra_log_segment_writer_SUITE.erl
@@ -91,16 +91,13 @@ accept_mem_tables(Config) ->
Entries = [{1, 42, a}, {2, 42, b}, {3, 43, c}],
Mt = make_mem_table(UId, Entries),
Tid = ra_mt:tid(Mt),
- TidRanges = [{Tid, ra_mt:range(Mt)}],
- Ranges = #{UId => TidRanges},
- % WalFile = "0000001.wal",
- % FullWalFile = filename:join(Dir, WalFile),
- % ok = file:write_file(FullWalFile, <<"waldata">>),
+ TidSeqs = [{Tid, [ra_mt:range(Mt)]}],
+ Ranges = #{UId => TidSeqs},
make_wal(Config, "w1.wal"),
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges,
make_wal(Config, "w1.wal")),
receive
- {ra_log_event, {segments, TidRanges, [{{1, 3}, SegFile}]}} ->
+ {ra_log_event, {segments, TidSeqs, [{{1, 3}, SegFile}]}} ->
SegmentFile = filename:join(?config(server_dir, Config), SegFile),
{ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}),
% assert Entries have been fully transferred
@@ -129,18 +126,18 @@ accept_mem_tables_append(Config) ->
Entries = [{1, 42, a}, {2, 42, b}, {3, 43, c}],
Tid = ets:new(?FUNCTION_NAME, []),
_ = make_mem_table(UId, Tid, Entries),
- Ranges = #{UId => [{Tid, {1, 3}}]},
- ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges,
+ FlushSpec = #{UId => [{Tid, [{1, 3}]}]},
+ ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, FlushSpec,
make_wal(Config, "w1.wal")),
% second batch
Entries2 = [{4, 43, d}, {5, 43, e}],
_ = make_mem_table(UId, Tid, Entries2),
- Ranges2 = #{UId => [{Tid, {4, 5}}]},
- ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2,
+ FlushSpec2 = #{UId => [{Tid, [{4, 5}]}]},
+ ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, FlushSpec2,
make_wal(Config, "w2.wal")),
AllEntries = Entries ++ Entries2,
receive
- {ra_log_event, {segments, [{Tid, {4, 5}}], [{{1, 5}, Fn}]}} ->
+ {ra_log_event, {segments, [{Tid, [{4, 5}]}], [{{1, 5}, Fn}]}} ->
SegmentFile = filename:join(?config(server_dir, Config), Fn),
{ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}),
% assert Entries have been fully transferred
@@ -150,6 +147,7 @@ accept_mem_tables_append(Config) ->
flush(),
throw(ra_log_event_timeout)
end,
+ flush(),
ok = gen_server:stop(TblWriterPid),
ok.
@@ -161,11 +159,11 @@ accept_mem_tables_overwrite(Config) ->
UId = ?config(uid, Config),
Entries = [{3, 42, c}, {4, 42, d}, {5, 42, e}],
Tid = ra_mt:tid(make_mem_table(UId, Entries)),
- Ranges = #{UId => [{Tid, {3, 5}}]},
+ Ranges = #{UId => [{Tid, [{3, 5}]}]},
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges,
make_wal(Config, "w1.wal")),
receive
- {ra_log_event, {segments, [{Tid, {3, 5}}], [{{3, 5}, Fn}]}} ->
+ {ra_log_event, {segments, [{Tid, [{3, 5}]}], [{{3, 5}, Fn}]}} ->
SegmentFile = filename:join(?config(server_dir, Config), Fn),
{ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}),
?assertMatch({{3, 5}, _}, ra_log_segment:segref(Seg)),
@@ -178,11 +176,11 @@ accept_mem_tables_overwrite(Config) ->
% second batch
Entries2 = [{1, 43, a}, {2, 43, b}, {3, 43, c2}],
Tid2 = ra_mt:tid(make_mem_table(UId, Entries2)),
- Ranges2 = #{UId => [{Tid2, {1, 3}}]},
+ Ranges2 = #{UId => [{Tid2, [{1, 3}]}]},
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2,
make_wal(Config, "w2.wal")),
receive
- {ra_log_event, {segments, [{Tid2, {1, 3}}], [{{1, 3}, Fn2}]}} ->
+ {ra_log_event, {segments, [{Tid2, [{1, 3}]}], [{{1, 3}, Fn2}]}} ->
SegmentFile2 = filename:join(?config(server_dir, Config), Fn2),
{ok, Seg2} = ra_log_segment:open(SegmentFile2, #{mode => read}),
?assertMatch({{1, 3}, _}, ra_log_segment:segref(Seg2)),
@@ -210,15 +208,12 @@ accept_mem_tables_overwrite_same_wal(Config) ->
% second batch
Entries2 = [{4, 43, d2}, {5, 43, e2}, {6, 43, f}],
Tid2 = ra_mt:tid(make_mem_table(UId, Entries2)),
- Ranges2 = #{UId => [{Tid2, {4, 6}}, {Tid, {2, 5}}]},
- % debugger:start(),
- % int:i(ra_log_segment_writer),
- % int:break(ra_log_segment_writer, 240),
+ Ranges2 = #{UId => [{Tid2, [{4, 6}]}, {Tid, [{2, 5}]}]},
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2,
make_wal(Config, "w2.wal")),
receive
{ra_log_event,
- {segments, [{Tid2, {4, 6}}, {Tid, {2, 5}}], [{{2, 6}, Fn}]}} ->
+ {segments, [{Tid2, [{4, 6}]}, {Tid, [{2, 5}]}], [{{2, 6}, Fn}]}} ->
SegmentFile = filename:join(?config(server_dir, Config), Fn),
{ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}),
?assertMatch({{2, 6}, _}, ra_log_segment:segref(Seg)),
@@ -249,7 +244,8 @@ accept_mem_tables_multi_segment(Config) ->
Entries = [{I, 2, x} || I <- lists:seq(1, 10)],
Mt = make_mem_table(UId, Entries),
Tid = ra_mt:tid(Mt),
- TidRanges = [{Tid, ra_mt:range(Mt)}],
+ TidSeq = {Tid, [ra_mt:range(Mt)]},
+ TidRanges = [TidSeq],
Ranges = #{UId => TidRanges},
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges,
make_wal(Config, "w.wal")),
@@ -276,7 +272,7 @@ accept_mem_tables_multi_segment_max_size(Config) ->
Entries = [{I, 2, crypto:strong_rand_bytes(120)} || I <- lists:seq(1, 10)],
Mt = make_mem_table(UId, Entries),
Tid = ra_mt:tid(Mt),
- TidRanges = [{Tid, ra_mt:range(Mt)}],
+ TidRanges = [{Tid, [ra_mt:range(Mt)]}],
Ranges = #{UId => TidRanges},
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges,
make_wal(Config, "w.wal")),
@@ -303,7 +299,7 @@ accept_mem_tables_multi_segment_overwrite(Config) ->
Entries = [{I, 2, x} || I <- lists:seq(1, 10)],
Mt = make_mem_table(UId, Entries),
Tid = ra_mt:tid(Mt),
- TidRanges = [{Tid, ra_mt:range(Mt)}],
+ TidRanges = [{Tid, [ra_mt:range(Mt)]}],
Ranges = #{UId => TidRanges},
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges,
make_wal(Config, "w.wal")),
@@ -320,7 +316,7 @@ accept_mem_tables_multi_segment_overwrite(Config) ->
Entries2 = [{I, 3, x} || I <- lists:seq(7, 15)],
Mt2 = make_mem_table(UId, Entries2),
Tid2 = ra_mt:tid(Mt2),
- TidRanges2 = [{Tid2, ra_mt:range(Mt2)}],
+ TidRanges2 = [{Tid2, [ra_mt:range(Mt2)]}],
Ranges2 = #{UId => TidRanges2},
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2,
make_wal(Config, "w2.wal")),
@@ -360,13 +356,13 @@ accept_mem_tables_for_down_server(Config) ->
Mt2 = make_mem_table(UId, Entries),
Tid = ra_mt:tid(Mt),
Tid2 = ra_mt:tid(Mt2),
- Ranges = #{DownUId => [{Tid, {1, 3}}],
- UId => [{Tid2, {1, 3}}]},
+ Ranges = #{DownUId => [{Tid, [{1, 3}]}],
+ UId => [{Tid2, [{1, 3}]}]},
WalFile = filename:join(Dir, "00001.wal"),
ok = file:write_file(WalFile, <<"waldata">>),
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile),
receive
- {ra_log_event, {segments, [{Tid2, {1, 3}}], [{{1, 3}, Fn}]}} ->
+ {ra_log_event, {segments, [{Tid2, [{1, 3}]}], [{{1, 3}, Fn}]}} ->
SegmentFile = filename:join(?config(server_dir, Config), Fn),
{ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}),
% assert Entries have been fully transferred
@@ -376,6 +372,7 @@ accept_mem_tables_for_down_server(Config) ->
flush(),
throw(ra_log_event_timeout)
end,
+ flush(),
%% validate fake uid entries were written
ra_log_segment_writer:await(?SEGWR),
DownFn = ra_lib:zpad_filename("", "segment", 1),
@@ -389,6 +386,7 @@ accept_mem_tables_for_down_server(Config) ->
%% if the server is down at the time the segment writer send the segments
%% the segment writer should clear up the ETS mem tables
+ timer:sleep(500),
FakeMt = ra_mt:init(Tid),
?assertMatch(#{size := 0}, ra_mt:info(FakeMt)),
@@ -421,12 +419,12 @@ accept_mem_tables_with_deleted_server(Config) ->
Mt2 = make_mem_table(UId, Entries),
Tid = ra_mt:tid(Mt),
Tid2 = ra_mt:tid(Mt2),
- Ranges = #{DeletedUId => [{Tid, {1, 3}}],
- UId => [{Tid2, {1, 3}}]},
+ Ranges = #{DeletedUId => [{Tid, [{1, 3}]}],
+ UId => [{Tid2, [{1, 3}]}]},
WalFile = make_wal(Config, "00001.wal"),
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile),
receive
- {ra_log_event, {segments, [{Tid2, {1, 3}}], [{{1, 3}, Fn}]}} ->
+ {ra_log_event, {segments, [{Tid2, [{1, 3}]}], [{{1, 3}, Fn}]}} ->
SegmentFile = filename:join(?config(server_dir, Config), Fn),
{ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}),
% assert Entries have been fully transferred
@@ -468,7 +466,7 @@ accept_mem_tables_with_corrupt_segment(Config) ->
Entries = [{1, 42, a}, {2, 42, b}, {3, 43, c}],
Mt = make_mem_table(UId, Entries),
Tid = ra_mt:tid(Mt),
- TidRanges = [{Tid, ra_mt:range(Mt)}],
+ TidRanges = [{Tid, [ra_mt:range(Mt)]}],
Ranges = #{UId => TidRanges},
WalFile = make_wal(Config, "0000001.wal"),
%% write an empty file to simulate corrupt segment
@@ -508,14 +506,15 @@ accept_mem_tables_multiple_ranges(Config)->
Mt = make_mem_table(UId, Entries),
Entries2 = [{N, 42, N} || N <- lists:seq(33, 64)],
Mt2 = make_mem_table(UId, Entries2),
- Ranges = #{UId => [
- {ra_mt:tid(Mt2), ra_mt:range(Mt2)},
- {ra_mt:tid(Mt), ra_mt:range(Mt)}
- ]},
+ TidRanges = [
+ {ra_mt:tid(Mt2), [ra_mt:range(Mt2)]},
+ {ra_mt:tid(Mt), [ra_mt:range(Mt)]}
+ ],
+ Ranges = #{UId => TidRanges},
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges,
make_wal(Config, "w1.wal")),
receive
- {ra_log_event, {segments, _TidRanges, SegRefs}} ->
+ {ra_log_event, {segments, TidRanges, SegRefs}} ->
?assertMatch([
{{49, 64}, _},
{{33, 48}, _},
@@ -542,16 +541,17 @@ accept_mem_tables_multiple_ranges_snapshot(Config)->
Mt = make_mem_table(UId, Entries),
Entries2 = [{N, 42, N} || N <- lists:seq(33, 64)],
Mt2 = make_mem_table(UId, Entries2),
- Ranges = #{UId => [
- {ra_mt:tid(Mt2), ra_mt:range(Mt2)},
- {ra_mt:tid(Mt), ra_mt:range(Mt)}
- ]},
- ets:insert(ra_log_snapshot_state, {UId, 64}),
+ TidRanges = [
+ {ra_mt:tid(Mt2), [ra_mt:range(Mt2)]},
+ {ra_mt:tid(Mt), [ra_mt:range(Mt)]}
+ ],
+ Ranges = #{UId => TidRanges},
+ ra_log_snapshot_state:insert(ra_log_snapshot_state, UId, 64, 65, []),
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges,
make_wal(Config, "w1.wal")),
receive
- {ra_log_event, {segments, _TidRanges, SegRefs}} ->
+ {ra_log_event, {segments, TidRanges, SegRefs}} ->
?assertMatch([], SegRefs),
ok
after 3000 ->
@@ -572,7 +572,7 @@ truncate_segments(Config) ->
Entries = [{N, 42, N} || N <- lists:seq(1, 32)],
Mt = make_mem_table(UId, Entries),
Tid = ra_mt:tid(Mt),
- TidRanges = [{Tid, ra_mt:range(Mt)}],
+ TidRanges = [{Tid, [ra_mt:range(Mt)]}],
Ranges = #{UId => TidRanges},
WalFile = make_wal(Config, "0000001.wal"),
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile),
@@ -607,18 +607,18 @@ truncate_segments_with_pending_update(Config) ->
UId = ?config(uid, Config),
Entries = [{N, 42, N} || N <- lists:seq(1, 32)],
Mt = make_mem_table(UId, Entries),
- Ranges = #{UId => [{ra_mt:tid(Mt), ra_mt:range(Mt)}]},
+ Ranges = #{UId => [{ra_mt:tid(Mt), [ra_mt:range(Mt)]}]},
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges,
make_wal(Config, "w1.wal")),
ra_log_segment_writer:await(?SEGWR),
%% write another range
Entries2 = [{N, 42, N} || N <- lists:seq(33, 40)],
Mt2 = make_mem_table(UId, Entries2),
- Ranges2 = #{UId => [{ra_mt:tid(Mt2), ra_mt:range(Mt2)}]},
+ Ranges2 = #{UId => [{ra_mt:tid(Mt2), [ra_mt:range(Mt2)]}]},
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2,
make_wal(Config, "w2.erl")),
receive
- {ra_log_event, {segments, _Tid, [{{25, 32}, S} = Cur | Rem]}} ->
+ {ra_log_event, {segments, _TidRanges, [{{25, 32}, S} = Cur | Rem]}} ->
% this is the event from the first call to accept_mem_tables,
% the Cur segments has been appended to since so should _not_
% be deleted when it is provided as the cutoff segref for
@@ -654,13 +654,13 @@ truncate_segments_with_pending_overwrite(Config) ->
% fake up a mem segment for Self
Entries = [{N, 42, N} || N <- lists:seq(1, 32)],
Mt = make_mem_table(UId, Entries),
- Ranges = #{UId => [{ra_mt:tid(Mt), ra_mt:range(Mt)}]},
+ Ranges = #{UId => [{ra_mt:tid(Mt), [ra_mt:range(Mt)]}]},
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges,
make_wal(Config, "w1.wal")),
%% write one more entry separately
Entries2 = [{N, 43, N} || N <- lists:seq(12, 25)],
Mt2 = make_mem_table(UId, Entries2),
- Ranges2 = #{UId => [{ra_mt:tid(Mt2), ra_mt:range(Mt2)}]},
+ Ranges2 = #{UId => [{ra_mt:tid(Mt2), [ra_mt:range(Mt2)]}]},
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2,
make_wal(Config, "w2.wal")),
receive
@@ -715,7 +715,7 @@ my_segments(Config) ->
% fake up a mem segment for Self
Entries = [{1, 42, a}, {2, 42, b}, {3, 43, c}],
Mt = make_mem_table(UId, Entries),
- Ranges = #{UId => [{ra_mt:tid(Mt), ra_mt:range(Mt)}]},
+ Ranges = #{UId => [{ra_mt:tid(Mt), [ra_mt:range(Mt)]}]},
TidRanges = maps:get(UId, Ranges),
WalFile = make_wal(Config, "00001.wal"),
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile),
@@ -741,7 +741,7 @@ upgrade_segment_name_format(Config) ->
% fake up a mem segment for Self
Entries = [{1, 42, a}, {2, 42, b}, {3, 43, c}],
Mt = make_mem_table(UId, Entries),
- Ranges = #{UId => [{ra_mt:tid(Mt), ra_mt:range(Mt)}]},
+ Ranges = #{UId => [{ra_mt:tid(Mt), [ra_mt:range(Mt)]}]},
TidRanges = maps:get(UId, Ranges),
WalFile = make_wal(Config, "00001.wal"),
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile),
@@ -788,9 +788,9 @@ skip_entries_lower_than_snapshot_index(Config) ->
{5, 43, e}
],
Mt = make_mem_table(UId, Entries),
- Ranges = #{UId => [{ra_mt:tid(Mt), ra_mt:range(Mt)}]},
+ Ranges = #{UId => [{ra_mt:tid(Mt), [ra_mt:range(Mt)]}]},
%% update snapshot state table
- ets:insert(ra_log_snapshot_state, {UId, 3}),
+ ra_log_snapshot_state:insert(ra_log_snapshot_state, UId, 3, 4, []),
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges,
make_wal(Config, "w1.wal")),
receive
@@ -821,9 +821,9 @@ skip_all_entries_lower_than_snapshot_index(Config) ->
{3, 43, e}
],
Mt = make_mem_table(UId, Entries),
- Ranges = #{UId => [{ra_mt:tid(Mt), ra_mt:range(Mt)}]},
+ Ranges = #{UId => [{ra_mt:tid(Mt), [ra_mt:range(Mt)]}]},
%% update snapshot state table
- ets:insert(ra_log_snapshot_state, {UId, 3}),
+ ra_log_snapshot_state:insert(ra_log_snapshot_state, UId, 3, 4, []),
ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges,
make_wal(Config, "w1.wal")),
TIDRANGES = maps:get(UId, Ranges),
diff --git a/test/ra_log_wal_SUITE.erl b/test/ra_log_wal_SUITE.erl
index a538a52a..31ee6be0 100644
--- a/test/ra_log_wal_SUITE.erl
+++ b/test/ra_log_wal_SUITE.erl
@@ -17,8 +17,6 @@ all() ->
[
{group, default},
{group, fsync},
- {group, o_sync},
- {group, sync_after_notify},
{group, no_sync}
].
@@ -26,6 +24,11 @@ all() ->
all_tests() ->
[
basic_log_writes,
+ sparse_writes,
+ sparse_write_same_batch,
+ sparse_write_overwrite,
+ sparse_write_recover,
+ sparse_write_recover_with_mt,
wal_filename_upgrade,
same_uid_different_process,
consecutive_terms_in_batch_should_result_in_two_written_events,
@@ -63,9 +66,8 @@ groups() ->
[
{default, [], all_tests()},
%% uses fsync instead of the default fdatasync
- {fsync, [], all_tests()},
- {o_sync, [], all_tests()},
- {sync_after_notify, [], all_tests()},
+ %% just testing that the configuration and dispatch works
+ {fsync, [], [basic_log_writes]},
{no_sync, [], all_tests()}
].
@@ -83,16 +85,16 @@ init_per_group(Group, Config) ->
ra_directory:init(?SYS),
ra_counters:init(),
% application:ensure_all_started(lg),
- {SyncMethod, WriteStrat} =
+ SyncMethod =
case Group of
fsync ->
- {sync, default};
+ sync;
no_sync ->
- {none, default};
+ none;
_ ->
- {datasync, Group}
+ datasync
end,
- [{write_strategy, WriteStrat},
+ [
{sys_cfg, SysCfg},
{sync_method, SyncMethod} | Config].
@@ -101,10 +103,9 @@ end_per_group(_, Config) ->
init_per_testcase(TestCase, Config) ->
PrivDir = ?config(priv_dir, Config),
- G = ?config(write_strategy, Config),
M = ?config(sync_method, Config),
Sys = ?config(sys_cfg, Config),
- Dir = filename:join([PrivDir, G, M, TestCase]),
+ Dir = filename:join([PrivDir, M, TestCase]),
{ok, Ets} = ra_log_ets:start_link(Sys),
ra_counters:init(),
UId = atom_to_binary(TestCase, utf8),
@@ -112,9 +113,8 @@ init_per_testcase(TestCase, Config) ->
TestCase, TestCase),
Names = maps:get(names, Sys),
WalConf = #{dir => Dir,
- name => ra_log_wal,
- names => Names,
- write_strategy => G,
+ system => default,
+ names => Names#{segment_writer => self()},
max_size_bytes => ?MAX_SIZE_BYTES},
_ = ets:new(ra_log_snapshot_state,
[named_table, public, {write_concurrency, true}]),
@@ -136,24 +136,147 @@ basic_log_writes(Config) ->
Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
Tid = ets:new(?FUNCTION_NAME, []),
- {ok, Pid} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 12, 1, "value"),
- ok = await_written(WriterId, 1, {12, 12}),
+ ok = await_written(WriterId, 1, [12]),
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 13, 1, "value2"),
- ok = await_written(WriterId, 1, {13, 13}),
+ ok = await_written(WriterId, 1, [13]),
ra_log_wal:force_roll_over(Pid),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid, {12, 13}}]}, "0000000000000001.wal"}} ->
+ {mem_tables, #{UId := [{Tid, [13, 12]}]}, "0000000000000001.wal"}} ->
ok
after 5000 ->
flush(),
- ct:fail("receiving mem tables timed out")
+ ct:fail("receiving mem table ranges timed out")
end,
proc_lib:stop(Pid),
meck:unload(),
ok.
+sparse_writes(Config) ->
+ meck:new(ra_log_segment_writer, [passthrough]),
+ meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
+ Conf = ?config(wal_conf, Config),
+ {UId, _} = WriterId = ?config(writer_id, Config),
+ Tid = ets:new(?FUNCTION_NAME, []),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
+ {ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 11, 12, 1, "value"),
+ ok = await_written(WriterId, 1, [12]),
+ %% write a "sparse write" at index 15 but reference 12 as the last
+ %% one
+ {ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 12, 15, 1, "value2"),
+ ok = await_written(WriterId, 1, [15]),
+ ra_log_wal:force_roll_over(Pid),
+ receive
+ {'$gen_cast',
+ {mem_tables, #{UId := [{Tid, [15, 12]}]}, _}} ->
+ ok
+ after 5000 ->
+ flush(),
+ ct:fail("receiving mem table ranges timed out")
+ end,
+ proc_lib:stop(Pid),
+ meck:unload(),
+ ok.
+
+sparse_write_same_batch(Config) ->
+ meck:new(ra_log_segment_writer, [passthrough]),
+ meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
+ Conf = ?config(wal_conf, Config),
+ {UId, _} = WriterId = ?config(writer_id, Config),
+ Tid = ets:new(?FUNCTION_NAME, []),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
+
+ suspend_process(Pid),
+ {ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 11, 12, 1, "value"),
+ {ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 12, 15, 1, "value2"),
+ erlang:resume_process(Pid),
+
+ ok = await_written(WriterId, 1, [15, 12]),
+ ra_log_wal:force_roll_over(Pid),
+ receive
+ {'$gen_cast',
+ {mem_tables, #{UId := [{Tid, [15, 12]}]}, _}} ->
+ ok
+ after 5000 ->
+ flush(),
+ ct:fail("receiving mem table ranges timed out")
+ end,
+ proc_lib:stop(Pid),
+ meck:unload(),
+ ok.
+
+sparse_write_recover(Config) ->
+ %% no mt case
+ meck:new(ra_log_segment_writer, [passthrough]),
+ meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
+ Conf = ?config(wal_conf, Config),
+ {UId, _} = WriterId = ?config(writer_id, Config),
+ Names = ?config(names, Config),
+ %% create a tid that isn't registered as mt
+ Tid = ets:new(?MODULE, [set]),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
+ {ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 11, 12, 1, "value"),
+ ok = await_written(WriterId, 1, [12]),
+ {ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 12, 15, 1, "value2"),
+ ok = await_written(WriterId, 1, [15]),
+ ?assert(is_process_alive(Pid)),
+ ok = proc_lib:stop(Pid),
+ {ok, Pid2} = ra_log_wal:start_link(Conf),
+ ?assert(is_process_alive(Pid2)),
+ receive
+ {'$gen_cast',
+ {mem_tables, #{UId := [{MtTid, [15, 12]}]}, _}} ->
+ {ok, Mt0} = ra_log_ets:mem_table_please(Names, UId),
+ ?assertEqual(MtTid, ra_mt:tid(Mt0)),
+ ok
+ after 5000 ->
+ flush(),
+ ct:fail("receiving mem table ranges timed out")
+ end,
+ flush(),
+ proc_lib:stop(Pid2),
+ meck:unload(),
+ ok.
+
+sparse_write_recover_with_mt(Config) ->
+ meck:new(ra_log_segment_writer, [passthrough]),
+ meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
+ Conf = ?config(wal_conf, Config),
+ {UId, _} = WriterId = ?config(writer_id, Config),
+ Names = ?config(names, Config),
+ {ok, Mt0} = ra_log_ets:mem_table_please(Names, UId),
+ Tid = ra_mt:tid(Mt0),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
+ {ok, Mt1} = ra_mt:insert_sparse({12, 1, "value"}, undefined, Mt0),
+ {ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 11, 12, 1, "value"),
+ ok = await_written(WriterId, 1, [12]),
+ {ok, _Mt} = ra_mt:insert_sparse({15, 1, "value"}, 12, Mt1),
+ {ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 12, 15, 1, "value2"),
+ ok = await_written(WriterId, 1, [15]),
+ ?assert(is_process_alive(Pid)),
+ ok = proc_lib:stop(Pid),
+ {ok, Pid2} = ra_log_wal:start_link(Conf),
+ ?assert(is_process_alive(Pid2)),
+ receive
+ {'$gen_cast',
+ {mem_tables, #{UId := [{Tid, [15, 12]}]}, _}} ->
+ ok
+ after 5000 ->
+ flush(),
+ ct:fail("receiving mem table ranges timed out")
+ end,
+ flush(),
+ proc_lib:stop(Pid2),
+ meck:unload(),
+ ok.
+
+%% TODO: as sparse writes are pre committed I dont
+%% think we'll ever overwrite anything.
+sparse_write_overwrite(_Config) ->
+ ok.
+
wal_filename_upgrade(Config) ->
meck:new(ra_log_segment_writer, [passthrough]),
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
@@ -161,23 +284,20 @@ wal_filename_upgrade(Config) ->
#{dir := Dir} = Conf,
{UId, _} = WriterId = ?config(writer_id, Config),
Tid = ets:new(?FUNCTION_NAME, []),
- {ok, Pid} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 12, 1, "value"),
- ok = await_written(WriterId, 1, {12, 12}),
+ ok = await_written(WriterId, 1, [12]),
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 13, 1, "value2"),
- ok = await_written(WriterId, 1, {13, 13}),
+ ok = await_written(WriterId, 1, [13]),
proc_lib:stop(Pid),
%% rename file to old 8 character format
Fn = filename:join(Dir, "0000000000000001.wal"),
FnOld = filename:join(Dir, "00000001.wal"),
ok = file:rename(Fn, FnOld),
- % debugger:start(),
- % int:i(ra_log_wal),
- % int:break(ra_log_wal, 373),
- {ok, Pid2} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ {ok, Pid2} = ra_log_wal:start_link(Conf),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{_Tid, {12, 13}}]}, "0000000000000001.wal"}} ->
+ {mem_tables, #{UId := [{_Tid, [13, 12]}]}, "0000000000000001.wal"}} ->
ok
after 5000 ->
flush(),
@@ -193,25 +313,26 @@ same_uid_different_process(Config) ->
Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
Tid = ets:new(?FUNCTION_NAME, []),
- {ok, Pid} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 12, 1, "value"),
- ok = await_written(WriterId, 1, {12, 12}),
+ ok = await_written(WriterId, 1, [12]),
Self = self(),
_ = spawn(fun() ->
Wid = {UId, self()},
{ok, _} = ra_log_wal:write(Pid, Wid, Tid, 13, 1, "value2"),
- ok = await_written(Wid, 1, {13, 13}),
+ ok = await_written(Wid, 1, [13]),
Self ! go
end),
receive
go -> ok
after 250 ->
+ flush(),
exit(go_timeout)
end,
ra_log_wal:force_roll_over(Pid),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid, {12, 13}}]}, _WalFile}} ->
+ {mem_tables, #{UId := [{Tid, [13, 12]}]}, _WalFile}} ->
ok
after 5000 ->
flush(),
@@ -226,23 +347,23 @@ consecutive_terms_in_batch_should_result_in_two_written_events(Config) ->
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- {ok, Pid} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
Data = <<"data">>,
Tid = ets:new(?FUNCTION_NAME, []),
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, I, 1, Data)
|| I <- lists:seq(1, 3)],
- await_written(WriterId, 1, {1, 3}),
+ await_written(WriterId, 1, [{1, 3}]),
flush(),
suspend_process(Pid),
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 4, 1, Data),
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 5, 2, Data),
erlang:resume_process(Pid),
- await_written(WriterId, 1, {4, 4}),
- await_written(WriterId, 2, {5, 5}),
+ await_written(WriterId, 1, [4]),
+ await_written(WriterId, 2, [5]),
ra_log_wal:force_roll_over(Pid),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid, {1, 5}}]}, _WalFile}} ->
+ {mem_tables, #{UId := [{Tid, [{1, 5}]}]}, _WalFile}} ->
ok
after 5000 ->
flush(),
@@ -257,21 +378,21 @@ writes_snapshot_idx_overtakes(Config) ->
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- {ok, Pid} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
Data = <<"data">>,
Tid = ets:new(?FUNCTION_NAME, []),
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, I, 1, Data)
|| I <- lists:seq(1, 3)],
- await_written(WriterId, 1, {1, 3}),
+ await_written(WriterId, 1, [{1, 3}]),
% snapshot idx overtakes
- ets:insert(ra_log_snapshot_state, {UId, 5}),
+ ok = ra_log_snapshot_state:insert(ra_log_snapshot_state, UId, 5, 6, []),
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, I, 1, Data)
|| I <- lists:seq(4, 7)],
- await_written(WriterId, 1, {6, 7}),
+ await_written(WriterId, 1, [{6, 7}]),
ra_log_wal:force_roll_over(Pid),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid, {6, 7}}]}, _WalFile}} ->
+ {mem_tables, #{UId := [{Tid, [7, 6]}]}, _WalFile}} ->
ok
after 5000 ->
flush(),
@@ -287,24 +408,24 @@ writes_implicit_truncate_write(Config) ->
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- {ok, Pid} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
Data = <<"data">>,
Tid = ets:new(?FUNCTION_NAME, []),
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, I, 1, Data)
|| I <- lists:seq(1, 3)],
- await_written(WriterId, 1, {1, 3}),
+ await_written(WriterId, 1, [{1, 3}]),
% snapshot idx updated and we follow that with the next index after the
% snapshot.
% before we had to detect this and send a special {truncate, append request
% but this is not necessary anymore
- ets:insert(ra_log_snapshot_state, {UId, 5}),
+ ok = ra_log_snapshot_state:insert(ra_log_snapshot_state, UId, 5, 6, []),
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, I, 1, Data)
|| I <- lists:seq(6, 7)],
- await_written(WriterId, 1, {6, 7}),
+ await_written(WriterId, 1, [7, 6]),
ra_log_wal:force_roll_over(Pid),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid, {6, 7}}]}, _WalFile}} ->
+ {mem_tables, #{UId := [{Tid, [7, 6]}]}, _WalFile}} ->
ok
after 5000 ->
flush(),
@@ -320,7 +441,7 @@ writes_snapshot_idx_overtakes_same_batch(Config) ->
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- {ok, Pid} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
Data = <<"data">>,
Tid = ets:new(?FUNCTION_NAME, []),
erlang:suspend_process(Pid),
@@ -329,21 +450,24 @@ writes_snapshot_idx_overtakes_same_batch(Config) ->
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 3, 1, Data),
%% this ensures the snapshot state is updated within the processing of a
%% single batch
- gen_batch_server:cast(Pid, {query,
- fun (_) ->
- ets:insert(ra_log_snapshot_state, {UId, 5})
- end}),
+ gen_batch_server:cast(
+ Pid, {query,
+ fun (_) ->
+
+ ok = ra_log_snapshot_state:insert(ra_log_snapshot_state, UId,
+ 5, 6, [])
+ end}),
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 4, 1, Data),
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 5, 1, Data),
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 6, 1, Data),
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 7, 1, Data),
erlang:resume_process(Pid),
% await_written(WriterId, {1, 3, 1}),
- await_written(WriterId, 1, {6, 7}),
+ await_written(WriterId, 1, [{6, 7}]),
ra_log_wal:force_roll_over(Pid),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid, {6, 7}}]}, _WalFile}} ->
+ {mem_tables, #{UId := [{Tid, [7, 6]}]}, _WalFile}} ->
ok
after 5000 ->
flush(),
@@ -358,13 +482,13 @@ overwrite_in_same_batch(Config) ->
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- {ok, Pid} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
Data = <<"data">>,
Tid = ets:new(?FUNCTION_NAME, []),
Tid2 = ets:new(?FUNCTION_NAME, []),
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, I, 1, Data)
|| I <- lists:seq(1, 3)],
- await_written(WriterId, 1, {1, 3}),
+ await_written(WriterId, 1, [{1, 3}]),
% write next index then immediately overwrite
suspend_process(Pid),
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 4, 1, Data),
@@ -378,13 +502,13 @@ overwrite_in_same_batch(Config) ->
% TODO: mt: find a way to avoid this, ideally we'd like to know the ranges
% for each term such that we can walk back until the first index that matches
% the term and set that as the last_written_index
- await_written(WriterId, 1, {4, 5}),
- await_written(WriterId, 2, {5, 5}),
+ await_written(WriterId, 1, [{4, 5}]),
+ await_written(WriterId, 2, [{5, 5}]),
ra_log_wal:force_roll_over(Pid),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid2, {5, 5}},%% the range to flush from the new table
- {Tid, {1, 5}}] %% this is the old table
+ {mem_tables, #{UId := [{Tid2, [5]},%% the range to flush from the new table
+ {Tid, [{1, 5}]}] %% this is the old table
}, _WalFile}} ->
ok
after 5000 ->
@@ -400,22 +524,22 @@ overwrite_completely(Config) ->
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- {ok, Pid} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
Data = <<"data">>,
Tid = ets:new(?FUNCTION_NAME, []),
Tid2 = ets:new(?FUNCTION_NAME, []),
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, I, 1, Data)
|| I <- lists:seq(3, 5)],
- await_written(WriterId, 1, {3, 5}),
+ await_written(WriterId, 1, [{3, 5}]),
% overwrite it all
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid2, I, 2, Data)
|| I <- lists:seq(3, 5)],
- await_written(WriterId, 2, {3, 5}),
+ await_written(WriterId, 2, [{3, 5}]),
ra_log_wal:force_roll_over(Pid),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid2, {3, 5}},
- {Tid, {3, 5}}]}, _WalFile}} ->
+ {mem_tables, #{UId := [{Tid2, [{3, 5}]},
+ {Tid, [{3, 5}]}]}, _WalFile}} ->
ok
after 5000 ->
flush(),
@@ -430,22 +554,22 @@ overwrite_inside(Config) ->
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- {ok, Pid} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
Data = <<"data">>,
Tid = ets:new(?FUNCTION_NAME, []),
Tid2 = ets:new(?FUNCTION_NAME, []),
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, I, 1, Data)
|| I <- lists:seq(1, 5)],
- await_written(WriterId, 1, {1, 5}),
+ await_written(WriterId, 1, [{1, 5}]),
% overwrite it all
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid2, I, 2, Data)
|| I <- lists:seq(3, 4)],
- await_written(WriterId, 2, {3, 4}),
+ await_written(WriterId, 2, [{3, 4}]),
ra_log_wal:force_roll_over(Pid),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid2, {3, 4}},
- {Tid, {1, 5}}]}, _WalFile}} ->
+ {mem_tables, #{UId := [{Tid2, [4, 3]},
+ {Tid, [{1, 5}]}]}, _WalFile}} ->
ok
after 5000 ->
flush(),
@@ -485,7 +609,8 @@ write_many(Config) ->
ok.
test_write_many(Name, NumWrites, ComputeChecksums, BatchSize, DataSize, Config) ->
- Conf0 = #{dir := Dir0} = ?config(wal_conf, Config),
+ Conf0 = #{dir := Dir0} = set_segment_writer(?config(wal_conf, Config),
+ spawn(fun () -> ok end)),
Dir = filename:join(Dir0, Name),
Conf = Conf0#{dir => Dir},
WriterId = ?config(writer_id, Config),
@@ -508,7 +633,8 @@ test_write_many(Name, NumWrites, ComputeChecksums, BatchSize, DataSize, Config)
fun () ->
[{ok, _} = ra_log_wal:write(ra_log_wal, WriterId, Tid, Idx, 1,
{data, Data}) || Idx <- Writes],
- await_written(WriterId, 1, {1, NumWrites}, fun ra_lib:ignore/2)
+
+ await_written(WriterId, 1, [{1, NumWrites}], fun ra_lib:ignore/2)
end),
timer:sleep(100),
{_, BinAfter} = erlang:process_info(WalPid, binary),
@@ -539,7 +665,8 @@ test_write_many(Name, NumWrites, ComputeChecksums, BatchSize, DataSize, Config)
write_many_by_many(Config) ->
NumWrites = 100,
NumWriters = 100,
- Conf = ?config(wal_conf, Config),
+ Conf = set_segment_writer(?config(wal_conf, Config),
+ spawn(fun() -> ok end)),
% {_UId, _} = WriterId = ?config(writer_id, Config),
{ok, WalPid} = ra_log_wal:start_link(Conf#{compute_checksums => false}),
Data = crypto:strong_rand_bytes(1024),
@@ -559,7 +686,7 @@ write_many_by_many(Config) ->
put(wid, WId),
[{ok, _} = ra_log_wal:write(ra_log_wal, WId, Tid, Idx, 1,
{data, Data}) || Idx <- Writes],
- await_written(WId, 1, {1, NumWrites}, fun ra_lib:ignore/2),
+ await_written(WId, 1, [{1, NumWrites}], fun ra_lib:ignore/2),
Self ! wal_write_done
end) || I <- lists:seq(1, NumWriters)],
[begin
@@ -602,7 +729,8 @@ out_of_seq_writes(Config) ->
% it will notify the write of the missing index and the writer can resend
% writes from that point
% the wal will discard all subsequent writes until it receives the missing one
- Conf = ?config(wal_conf, Config),
+ Conf = set_segment_writer(?config(wal_conf, Config),
+ spawn(fun() -> ok end)),
{_UId, _} = WriterId = ?config(writer_id, Config),
{ok, Pid} = ra_log_wal:start_link(Conf),
Data = crypto:strong_rand_bytes(1024),
@@ -610,7 +738,7 @@ out_of_seq_writes(Config) ->
% write 1-3
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, I, 1, Data)
|| I <- lists:seq(1, 3)],
- await_written(WriterId, 1, {1, 3}),
+ await_written(WriterId, 1, [{1, 3}]),
% then write 5
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 5, 1, Data),
% ensure an out of sync notification is received
@@ -625,9 +753,9 @@ out_of_seq_writes(Config) ->
% then write 4 and 5
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 4, 1, Data),
- await_written(WriterId, 1, {4, 4}),
+ await_written(WriterId, 1, [4]),
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 5, 1, Data),
- await_written(WriterId, 1, {5, 5}),
+ await_written(WriterId, 1, [5]),
% perform another out of sync write
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 7, 1, Data),
@@ -644,13 +772,13 @@ out_of_seq_writes(Config) ->
% ensure a written event is _NOT_ received
% when a roll-over happens after out of sync write
receive
- {ra_log_event, {written, 1, {8, 8}}} ->
+ {ra_log_event, {written, 1, [8]}} ->
throw(unexpected_written_event)
after 500 -> ok
end,
% write the missing one
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 6, 1, Data),
- await_written(WriterId, 1, {6, 6}),
+ await_written(WriterId, 1, [6]),
proc_lib:stop(Pid),
ok.
@@ -662,8 +790,7 @@ roll_over_max_size(Config) ->
meck:expect(ra_log_segment_writer, await,
fun(_) -> ok end),
% configure max_wal_size_bytes
- {ok, Pid} = ra_log_wal:start_link(Conf#{max_size_bytes => 1024 * NumWrites,
- segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf#{max_size_bytes => 1024 * NumWrites}),
%% DO this to ensure the actual max size bytes config is in place and not
%% the randomised value
ra_log_wal:force_roll_over(Pid),
@@ -673,11 +800,11 @@ roll_over_max_size(Config) ->
[begin
{ok, _} = ra_log_wal:write(ra_log_wal, WriterId, Tid, Idx, 1, Data)
end || Idx <- lists:seq(1, NumWrites)],
- await_written(UId, 1, {1, NumWrites}),
+ await_written(UId, 1, [{1, NumWrites}]),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid, {1, 97}}]}, _Wal}} ->
+ {mem_tables, #{UId := [{Tid, [{1, 97}]}]}, _Wal}} ->
%% TODO: do we realy need the hard coded 97 or just assert that
%% the wal was rolled, not exactly at which point?
ok
@@ -697,19 +824,18 @@ roll_over_with_data_larger_than_max_size(Config) ->
meck:expect(ra_log_segment_writer, await,
fun(_) -> ok end),
% configure max_wal_size_bytes
- {ok, Pid} = ra_log_wal:start_link(Conf#{max_size_bytes => 1024 * NumWrites * 10,
- segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf#{max_size_bytes => 1024 * NumWrites * 10}),
% write entries each larger than the WAL max size to trigger roll over
Data = crypto:strong_rand_bytes(64 * 1024),
Tid = ets:new(?FUNCTION_NAME, []),
[begin
{ok, _} = ra_log_wal:write(ra_log_wal, WriterId, Tid, Idx, 1, Data)
end || Idx <- lists:seq(1, NumWrites)],
- await_written(UId, 1, {1, NumWrites}),
+ await_written(UId, 1, [{1, NumWrites}]),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid, {1, 1}}]}, _Wal}} ->
+ {mem_tables, #{UId := [{Tid, [1]}]}, _Wal}} ->
ok
after 2000 ->
flush(),
@@ -729,19 +855,18 @@ roll_over_entry_limit(Config) ->
meck:expect(ra_log_segment_writer, await,
fun(_) -> ok end),
% configure max_wal_entries
- {ok, Pid} = ra_log_wal:start_link(Conf#{max_entries => 1000,
- segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf#{max_entries => 1000}),
% write enough entries to trigger roll over
Data = crypto:strong_rand_bytes(1024),
Tid = ets:new(?FUNCTION_NAME, []),
[begin
{ok, _} = ra_log_wal:write(ra_log_wal, WriterId, Tid, Idx, 1, Data)
end || Idx <- lists:seq(1, NumWrites)],
- await_written(UId, 1, {1, NumWrites}),
+ await_written(UId, 1, [{1, NumWrites}]),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid, {1, 1000}}]}, _Wal}} ->
+ {mem_tables, #{UId := [{Tid, [{1, 1000}]}]}, _Wal}} ->
%% 1000 is the last entry before the limit was reached
ok
after 2000 ->
@@ -755,19 +880,27 @@ roll_over_entry_limit(Config) ->
sys_get_status(Config) ->
- Conf = ?config(wal_conf, Config),
+ Conf = set_segment_writer(?config(wal_conf, Config),
+ spawn(fun () -> ok end)),
{_UId, _} = ?config(writer_id, Config),
{ok, Pid} = ra_log_wal:start_link(Conf),
- {_, _, _, [_, _, _, _, [_, _ ,S]]} = sys:get_status(ra_log_wal),
- #{write_strategy := _} = S,
+ {_, _, _, [_, _, _, _, [_, _ , S]]} = sys:get_status(ra_log_wal),
+ ?assert(is_map(S)),
+
+ ?assertMatch(#{sync_method := _,
+ compute_checksums := _,
+ writers := _,
+ filename := _,
+ current_size := _,
+ max_size_bytes := _,
+ counters := _ }, S),
proc_lib:stop(Pid),
ok.
recover(Config) ->
ok = logger:set_primary_config(level, all),
- Conf0 = ?config(wal_conf, Config),
+ Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- Conf = Conf0#{segment_writer => self()},
Data = <<42:256/unit:8>>,
meck:new(ra_log_segment_writer, [passthrough]),
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
@@ -777,14 +910,14 @@ recover(Config) ->
%% write some in one term
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, Idx, 1, Data)
|| Idx <- lists:seq(1, 100)],
- _ = await_written(WriterId, 1, {1, 100}),
+ _ = await_written(WriterId, 1, [{1, 100}]),
ra_log_wal:force_roll_over(ra_log_wal),
%% then some more in another
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, Idx, 2, Data)
|| Idx <- lists:seq(101, 200)],
- _ = await_written(WriterId, 2, {101, 200}),
+ _ = await_written(WriterId, 2, [{101, 200}]),
flush(),
ok = proc_lib:stop(ra_log_wal),
@@ -797,7 +930,7 @@ recover(Config) ->
MtTid = ra_mt:tid(Mt),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{MtTid, {1, 100}}]}, _Wal}} ->
+ {mem_tables, #{UId := [{MtTid, [{1, 100}]}]}, _Wal}} ->
ok
after 2000 ->
flush(),
@@ -805,7 +938,7 @@ recover(Config) ->
end,
receive
{'$gen_cast',
- {mem_tables, #{UId := [{MtTid, {101, 200}}]}, _}} ->
+ {mem_tables, #{UId := [{MtTid, [{101, 200}]}]}, _}} ->
ok
after 2000 ->
flush(),
@@ -828,9 +961,8 @@ recover(Config) ->
recover_with_snapshot_index(Config) ->
ok = logger:set_primary_config(level, all),
- Conf0 = ?config(wal_conf, Config),
+ Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- Conf = Conf0#{segment_writer => self()},
Data = <<42:256/unit:8>>,
meck:new(ra_log_segment_writer, [passthrough]),
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
@@ -839,11 +971,12 @@ recover_with_snapshot_index(Config) ->
%% write some in one term
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, Idx, 1, Data)
|| Idx <- lists:seq(1, 100)],
- _ = await_written(WriterId, 1, {1, 100}),
+ _ = await_written(WriterId, 1, [{1, 100}]),
flush(),
ok = proc_lib:stop(ra_log_wal),
- ets:insert(ra_log_snapshot_state, {UId, 50}),
+
+ ok = ra_log_snapshot_state:insert(ra_log_snapshot_state, UId, 50, 51, []),
{ok, Pid2} = ra_log_wal:start_link(Conf),
{ok, Mt} = ra_log_ets:mem_table_please(?config(names, Config), UId),
@@ -851,7 +984,7 @@ recover_with_snapshot_index(Config) ->
MtTid = ra_mt:tid(Mt),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{MtTid, {51, 100}}]}, _Wal}} ->
+ {mem_tables, #{UId := [{MtTid, [{51, 100}]}]}, _Wal}} ->
ok
after 2000 ->
flush(),
@@ -864,9 +997,8 @@ recover_with_snapshot_index(Config) ->
recover_overwrite(Config) ->
ok = logger:set_primary_config(level, all),
- Conf0 = ?config(wal_conf, Config),
+ Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- Conf = Conf0#{segment_writer => self()},
Data = <<42:256/unit:8>>,
meck:new(ra_log_segment_writer, [passthrough]),
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
@@ -876,15 +1008,15 @@ recover_overwrite(Config) ->
%% write some in one term
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, Idx, 1, Data)
|| Idx <- lists:seq(1, 10)],
- _ = await_written(WriterId, 1, {1, 10}),
+ _ = await_written(WriterId, 1, [{1, 10}]),
%% then some more in another
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid2, Idx, 2, Data)
|| Idx <- lists:seq(5, 20)],
- _ = await_written(WriterId, 2, {5, 20}),
+ _ = await_written(WriterId, 2, [{5, 20}]),
flush(),
- ok = proc_lib:stop(ra_log_wal),
+ ok = proc_lib:stop(ra_log_wal, normal, 5000),
{ok, Pid2} = ra_log_wal:start_link(Conf),
{ok, Mt} = ra_log_ets:mem_table_please(?config(names, Config), UId),
@@ -893,8 +1025,8 @@ recover_overwrite(Config) ->
PrevMtTid = ra_mt:tid(ra_mt:prev(Mt)),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{MtTid, {5, 20}},
- {PrevMtTid, {1, 10}}
+ {mem_tables, #{UId := [{MtTid, [{5, 20}]},
+ {PrevMtTid, [{1, 10}]}
]}, _Wal}} ->
ok
after 2000 ->
@@ -907,9 +1039,8 @@ recover_overwrite(Config) ->
recover_overwrite_rollover(Config) ->
ok = logger:set_primary_config(level, all),
- Conf0 = ?config(wal_conf, Config),
+ Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- Conf = Conf0#{segment_writer => self()},
Data = <<42:256/unit:8>>,
meck:new(ra_log_segment_writer, [passthrough]),
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
@@ -919,14 +1050,14 @@ recover_overwrite_rollover(Config) ->
%% write some in one term
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, Idx, 1, Data)
|| Idx <- lists:seq(1, 10)],
- _ = await_written(WriterId, 1, {1, 10}),
+ _ = await_written(WriterId, 1, [{1, 10}]),
ra_log_wal:force_roll_over(ra_log_wal),
%% then some more in another
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid2, Idx, 2, Data)
|| Idx <- lists:seq(5, 20)],
- _ = await_written(WriterId, 2, {5, 20}),
+ _ = await_written(WriterId, 2, [{5, 20}]),
flush(),
ok = proc_lib:stop(ra_log_wal),
@@ -937,7 +1068,7 @@ recover_overwrite_rollover(Config) ->
PrevMtTid = ra_mt:tid(ra_mt:prev(Mt)),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{PrevMtTid, {1, 10}}]}, _Wal}} ->
+ {mem_tables, #{UId := [{PrevMtTid, [{1, 10}]}]}, _Wal}} ->
ok
after 2000 ->
flush(),
@@ -945,7 +1076,7 @@ recover_overwrite_rollover(Config) ->
end,
receive
{'$gen_cast',
- {mem_tables, #{UId := [{MtTid, {5, 20}}]}, _}} ->
+ {mem_tables, #{UId := [{MtTid, [{5, 20}]}]}, _}} ->
ok
after 2000 ->
flush(),
@@ -958,9 +1089,8 @@ recover_overwrite_rollover(Config) ->
recover_existing_mem_table(Config) ->
ok = logger:set_primary_config(level, all),
- Conf0 = ?config(wal_conf, Config),
+ Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- Conf = Conf0#{segment_writer => self()},
Data = <<42:256/unit:8>>,
meck:new(ra_log_segment_writer, [passthrough]),
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
@@ -974,7 +1104,7 @@ recover_existing_mem_table(Config) ->
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, Idx, 1, Data),
Acc
end, Mt0, lists:seq(1, 100)),
- _ = await_written(WriterId, 1, {1, 100}),
+ _ = await_written(WriterId, 1, [{1, 100}]),
flush(),
ok = proc_lib:stop(ra_log_wal),
{ok, Pid2} = ra_log_wal:start_link(Conf),
@@ -983,7 +1113,7 @@ recover_existing_mem_table(Config) ->
?assertMatch({1, 100}, ra_mt:range(Mt)),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid, {1, 100}}]}, _}} ->
+ {mem_tables, #{UId := [{Tid, [{1, 100}]}]}, _}} ->
ok
after 2000 ->
flush(),
@@ -1006,9 +1136,8 @@ recover_existing_mem_table(Config) ->
recover_existing_mem_table_with_deletes(Config) ->
%% tests dirty recovery with partial mem table
ok = logger:set_primary_config(level, all),
- Conf0 = ?config(wal_conf, Config),
+ Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- Conf = Conf0#{segment_writer => self()},
Data = <<42:256/unit:8>>,
meck:new(ra_log_segment_writer, [passthrough]),
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
@@ -1022,7 +1151,7 @@ recover_existing_mem_table_with_deletes(Config) ->
{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, Idx, 1, Data),
Acc
end, Mt0, lists:seq(1, 100)),
- _ = await_written(WriterId, 1, {1, 100}),
+ _ = await_written(WriterId, 1, [{1, 100}]),
%% the delete comes in before recovery
{[Spec], _Mt2} = ra_mt:set_first(50, Mt1),
?assert(0 < ra_mt:delete(Spec)),
@@ -1033,7 +1162,7 @@ recover_existing_mem_table_with_deletes(Config) ->
?assertMatch({50, 100}, ra_mt:range(Mt)),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid, {50, 100}}]}, _}} ->
+ {mem_tables, #{UId := [{Tid, [{50, 100}]}]}, _}} ->
ok
after 2000 ->
flush(),
@@ -1045,9 +1174,8 @@ recover_existing_mem_table_with_deletes(Config) ->
recover_existing_mem_table_overwrite(Config) ->
ok = logger:set_primary_config(level, all),
- Conf0 = ?config(wal_conf, Config),
+ Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- Conf = Conf0#{segment_writer => self()},
Data = <<42:256/unit:8>>,
meck:new(ra_log_segment_writer, [passthrough]),
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
@@ -1061,7 +1189,7 @@ recover_existing_mem_table_overwrite(Config) ->
ra_mt:tid(Acc0), Idx, 1, Data),
Acc
end, Mt0, lists:seq(1, 100)),
- _ = await_written(WriterId, 1, {1, 100}),
+ _ = await_written(WriterId, 1, [{1, 100}]),
Mt2 = lists:foldl(
fun (Idx, Acc0) ->
{ok, Acc} = ra_mt:insert({Idx, 2, Data}, Acc0),
@@ -1072,7 +1200,7 @@ recover_existing_mem_table_overwrite(Config) ->
ra_log_ets:new_mem_table_please(?config(names, Config),
UId, Mt1)),
lists:seq(50, 200)),
- _ = await_written(WriterId, 2, {50, 200}),
+ _ = await_written(WriterId, 2, [{50, 200}]),
flush(),
ok = proc_lib:stop(ra_log_wal),
{ok, Pid2} = ra_log_wal:start_link(Conf),
@@ -1084,8 +1212,8 @@ recover_existing_mem_table_overwrite(Config) ->
Tid2 = ra_mt:tid(Mt2),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{Tid2, {50, 200}},
- {Tid, {1, 100}}]}, _}} ->
+ {mem_tables, #{UId := [{Tid2, [{50, 200}]},
+ {Tid, [{1, 100}]}]}, _}} ->
ok
after 2000 ->
flush(),
@@ -1101,36 +1229,33 @@ recover_implicit_truncate(Config) ->
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- {ok, Pid} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
Data = <<"data">>,
Tid = ets:new(?FUNCTION_NAME, []),
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, I, 1, Data)
|| I <- lists:seq(1, 3)],
- await_written(WriterId, 1, {1, 3}),
+ await_written(WriterId, 1, [{1, 3}]),
% snapshot idx updated and we follow that with the next index after the
% snapshot.
% before we had to detect this and send a special {truncate, append request
% but this is not necessary anymore
- ets:insert(ra_log_snapshot_state, {UId, 5}),
+ ok = ra_log_snapshot_state:insert(ra_log_snapshot_state, UId, 5, 6, []),
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, I, 1, Data)
|| I <- lists:seq(6, 7)],
- await_written(WriterId, 1, {6, 7}),
+ await_written(WriterId, 1, [{6, 7}]),
flush(),
ok = proc_lib:stop(Pid),
%% this could happen potentially in some edge cases??
- ets:delete(ra_log_snapshot_state, UId),
- % debugger:start(),
- % int:i(ra_log_wal),
- % int:break(ra_log_wal, 900),
- {ok, Pid2} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ ra_log_snapshot_state:delete(ra_log_snapshot_state, UId),
+ {ok, Pid2} = ra_log_wal:start_link(Conf),
{ok, Mt} = ra_log_ets:mem_table_please(?config(names, Config), UId),
?assertMatch(#{size := 2}, ra_mt:info(Mt)),
MtTid = ra_mt:tid(Mt),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{MtTid, {6, 7}}]}, _Wal}} ->
+ {mem_tables, #{UId := [{MtTid, [7, 6]}]}, _Wal}} ->
ok
after 2000 ->
flush(),
@@ -1146,23 +1271,23 @@ recover_delete_uid(Config) ->
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- {ok, Pid} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ {ok, Pid} = ra_log_wal:start_link(Conf),
{UId2, _} = WriterId2 = {<<"DELETEDUID">>, self()},
Data = <<"data">>,
Tid = ets:new(?FUNCTION_NAME, []),
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, I, 1, Data)
|| I <- lists:seq(1, 3)],
- await_written(WriterId, 1, {1, 3}),
+ await_written(WriterId, 1, [{1, 3}]),
Tid2 = ets:new(?FUNCTION_NAME, []),
[{ok, _} = ra_log_wal:write(Pid, WriterId2, Tid2, I, 9, Data)
|| I <- lists:seq(5, 9)],
- await_written(WriterId, 9, {5, 9}),
+ await_written(WriterId, 9, [{5, 9}]),
_ = ra_directory:unregister_name(default, UId2),
flush(),
ok = proc_lib:stop(Pid),
- {ok, Pid2} = ra_log_wal:start_link(Conf#{segment_writer => self()}),
+ {ok, Pid2} = ra_log_wal:start_link(Conf),
{ok, Mt} = ra_log_ets:mem_table_please(?config(names, Config), UId),
?assertMatch(#{size := 3}, ra_mt:info(Mt)),
@@ -1170,7 +1295,7 @@ recover_delete_uid(Config) ->
MtTid = ra_mt:tid(Mt),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{MtTid, {1, 3}}]} = Tables, _Wal}}
+ {mem_tables, #{UId := [{MtTid, [{1, 3}]}]} = Tables, _Wal}}
when not is_map_key(UId2, Tables) ->
ok
after 2000 ->
@@ -1185,8 +1310,7 @@ recover_delete_uid(Config) ->
recover_empty(Config) ->
ok = logger:set_primary_config(level, all),
- Conf0 = ?config(wal_conf, Config),
- Conf = Conf0#{segment_writer => self()},
+ Conf = ?config(wal_conf, Config),
meck:new(ra_log_segment_writer, [passthrough]),
meck:expect(ra_log_segment_writer, await,
fun(_) -> ok end),
@@ -1199,9 +1323,8 @@ recover_empty(Config) ->
recover_with_partial_last_entry(Config) ->
ok = logger:set_primary_config(level, all),
- #{dir := Dir} = Conf0 = ?config(wal_conf, Config),
+ #{dir := Dir} = Conf = ?config(wal_conf, Config),
{UId, _} = WriterId = ?config(writer_id, Config),
- Conf = Conf0#{segment_writer => self()},
Data = crypto:strong_rand_bytes(1000),
meck:new(ra_log_segment_writer, [passthrough]),
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
@@ -1209,7 +1332,7 @@ recover_with_partial_last_entry(Config) ->
Tid = ets:new(?FUNCTION_NAME, []),
[{ok, _} = ra_log_wal:write(ra_log_wal, WriterId, Tid, Idx, 1, Data)
|| Idx <- lists:seq(1, 100)],
- _ = await_written(WriterId, 1, {1, 100}),
+ _ = await_written(WriterId, 1, [{1, 100}]),
empty_mailbox(),
ok = proc_lib:stop(ra_log_wal),
@@ -1228,7 +1351,7 @@ recover_with_partial_last_entry(Config) ->
MtTid = ra_mt:tid(Mt),
receive
{'$gen_cast',
- {mem_tables, #{UId := [{MtTid, {1, 99}}]}, _File}} ->
+ {mem_tables, #{UId := [{MtTid, [{1, 99}]}]}, _File}} ->
ok
after 5000 ->
flush(),
@@ -1243,8 +1366,7 @@ recover_with_last_entry_corruption(Config) ->
ok = logger:set_primary_config(level, all),
#{dir := Dir} = Conf0 = ?config(wal_conf, Config),
WriterId = ?config(writer_id, Config),
- Conf = Conf0#{segment_writer => spawn(fun () -> ok end),
- pre_allocate => false},
+ Conf = set_segment_writer(Conf0, spawn(fun () -> ok end)),
Data = crypto:strong_rand_bytes(1000),
meck:new(ra_log_segment_writer, [passthrough]),
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
@@ -1252,7 +1374,7 @@ recover_with_last_entry_corruption(Config) ->
{ok, Pid} = ra_log_wal:start_link(Conf),
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, Idx, 1, Data)
|| Idx <- lists:seq(1, 100)],
- _ = await_written(WriterId, 1, {1, 100}),
+ _ = await_written(WriterId, 1, [{1, 100}]),
flush(),
ok = proc_lib:stop(ra_log_wal),
@@ -1274,8 +1396,7 @@ recover_with_last_entry_corruption_pre_allocate(Config) ->
ok = logger:set_primary_config(level, all),
#{dir := Dir} = Conf0 = ?config(wal_conf, Config),
WriterId = ?config(writer_id, Config),
- Conf = Conf0#{segment_writer => spawn(fun () -> ok end),
- pre_allocate => true},
+ Conf = set_segment_writer(Conf0, spawn(fun () -> ok end)),
Data = crypto:strong_rand_bytes(1000),
meck:new(ra_log_segment_writer, [passthrough]),
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
@@ -1283,7 +1404,7 @@ recover_with_last_entry_corruption_pre_allocate(Config) ->
Tid = ets:new(?FUNCTION_NAME, []),
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, Idx, 1, Data)
|| Idx <- lists:seq(1, 100)],
- _ = await_written(WriterId, 1, {1, 100}),
+ _ = await_written(WriterId, 1, [{1, 100}]),
empty_mailbox(),
ok = proc_lib:stop(ra_log_wal),
@@ -1307,8 +1428,7 @@ checksum_failure_in_middle_of_file_should_fail(Config) ->
ok = logger:set_primary_config(level, all),
#{dir := Dir} = Conf0 = ?config(wal_conf, Config),
WriterId = ?config(writer_id, Config),
- Conf = Conf0#{segment_writer => spawn(fun () -> ok end),
- pre_allocate => false},
+ Conf = set_segment_writer(Conf0, spawn(fun () -> ok end)),
Data = crypto:strong_rand_bytes(1000),
meck:new(ra_log_segment_writer, [passthrough]),
meck:expect(ra_log_segment_writer, await, fun(_) -> ok end),
@@ -1316,7 +1436,7 @@ checksum_failure_in_middle_of_file_should_fail(Config) ->
Tid = ets:new(?FUNCTION_NAME, []),
[{ok, _} = ra_log_wal:write(Pid, WriterId, Tid, Idx, 1, Data)
|| Idx <- lists:seq(1, 100)],
- _ = await_written(WriterId, 1, {1, 100}),
+ _ = await_written(WriterId, 1, [{1, 100}]),
empty_mailbox(),
ok = proc_lib:stop(ra_log_wal),
@@ -1341,25 +1461,33 @@ empty_mailbox() ->
ok
end.
-await_written(Id, Term, {From, To} = Written) ->
- await_written(Id, Term, {From, To} = Written, fun ct:pal/2).
+await_written(Id, Term, Written) when is_list(Written) ->
+ await_written(Id, Term, Written, fun ct:pal/2).
-await_written(Id, Term, {From, To} = Written, LogFun) ->
+await_written(Id, Term, Expected, LogFun) ->
receive
+ {ra_log_event, {written, Term, Written}}
+ when Written == Expected ->
+ %% consumed all of expected
+ LogFun("~s, got all ~b ~w", [?FUNCTION_NAME, Term, Written]),
+ ok;
{ra_log_event, {written, Term, Written}} ->
LogFun("~s, got ~b ~w", [?FUNCTION_NAME, Term, Written]),
- ok;
- {ra_log_event, {written, Term, {From, To} = W}} ->
- LogFun("~s, got ~b ~w", [?FUNCTION_NAME, Term, W]),
- %% indexes are the same but term is different,
- %% lets wait for the original
- await_written(Id, Term, Written, LogFun);
- {ra_log_event, {written, Term, {From, To0} = W}} ->
- LogFun("~s, got ~b ~w", [?FUNCTION_NAME, Term, W]),
- await_written(Id, Term, {To0+1, To}, LogFun)
+ case ra_seq:subtract(Expected, Written) of
+ [] ->
+ %% we're done
+ ok;
+ Rem ->
+ await_written(Id, Term, Rem, LogFun)
+ end;
+ {ra_log_event, {written, OthTerm, Written}}
+ when OthTerm =/= Term ->
+ %% different term
+ LogFun("~s, got oth term ~b ~w", [?FUNCTION_NAME, Term, Written]),
+ await_written(Id, Term, Expected, LogFun)
after 5000 ->
flush(),
- throw({written_timeout, To})
+ throw({written_timeout, Expected})
end.
% mem table read functions
@@ -1433,3 +1561,5 @@ suspend_process(Pid) ->
end
end.
+set_segment_writer(#{names := Names} = Conf, Writer) ->
+ Conf#{names => maps:put(segment_writer, Writer, Names)}.
diff --git a/test/ra_machine_ets_SUITE.erl b/test/ra_machine_ets_SUITE.erl
index ceb7e53d..249fecf3 100644
--- a/test/ra_machine_ets_SUITE.erl
+++ b/test/ra_machine_ets_SUITE.erl
@@ -12,7 +12,6 @@
-export([
]).
--include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
%%%===================================================================
diff --git a/test/ra_machine_int_SUITE.erl b/test/ra_machine_int_SUITE.erl
index 0d74b0f8..09f752db 100644
--- a/test/ra_machine_int_SUITE.erl
+++ b/test/ra_machine_int_SUITE.erl
@@ -610,6 +610,7 @@ log_effect(Config) ->
{[], ok,
{log, lists:reverse(Idxs),
fun (Cmds) ->
+ ct:pal("LOG!!"),
Datas = [D || {_, D} <- Cmds],
%% using a plain send here to
%% ensure this effect is only
diff --git a/test/ra_mt_SUITE.erl b/test/ra_mt_SUITE.erl
index 4aff51ef..6d8df7af 100644
--- a/test/ra_mt_SUITE.erl
+++ b/test/ra_mt_SUITE.erl
@@ -33,8 +33,9 @@ all_tests() ->
successor_below,
stage_commit,
range_overlap,
- stage_commit_abort,
- perf
+ stage_commit_2,
+ perf,
+ sparse
].
groups() ->
@@ -73,7 +74,7 @@ basics(_Config) ->
499 = ra_mt:delete(Spec),
?assertEqual({500, 1000}, ra_mt:range(Mt2)),
?assertEqual(501, ets:info(Tid, size)),
- {Spec2, Mt3} = ra_mt:record_flushed(Tid, {1, 999}, Mt2),
+ {Spec2, Mt3} = ra_mt:record_flushed(Tid, [{1, 999}], Mt2),
500 = ra_mt:delete(Spec2),
?assertEqual(1, ra_mt:lookup_term(1000, Mt3)),
ok.
@@ -86,15 +87,15 @@ record_flushed(_Config) ->
fun (I, Acc) ->
element(2, ra_mt:insert({I, 1, <<"banana">>}, Acc))
end, Mt0, lists:seq(1, 100)),
- {Spec, Mt2} = ra_mt:record_flushed(Tid, {1, 49}, Mt1),
- ?assertMatch({range, _, {1, 49}}, Spec),
+ {Spec, Mt2} = ra_mt:record_flushed(Tid, [{1, 49}], Mt1),
+ ?assertMatch({indexes, _, [{1, 49}]}, Spec),
?assertMatch({50, 100}, ra_mt:range(Mt2)),
_ = ra_mt:delete(Spec),
- {Spec2, Mt3} = ra_mt:record_flushed(Tid, {1, 49}, Mt2),
+ {Spec2, Mt3} = ra_mt:record_flushed(Tid, [{1, 49}], Mt2),
?assertMatch(undefined, Spec2),
_ = ra_mt:delete(Spec2),
- {Spec3, Mt4} = ra_mt:record_flushed(Tid, {50, 100}, Mt3),
- ?assertMatch({range, _, {50, 100}}, Spec3),
+ {Spec3, Mt4} = ra_mt:record_flushed(Tid, [{50, 100}], Mt3),
+ ?assertMatch({indexes, _, [{50, 100}]}, Spec3),
?assertEqual(undefined, ra_mt:range(Mt4)),
_ = ra_mt:delete(Spec3),
?assertMatch(#{size := 0}, ra_mt:info(Mt4)),
@@ -107,14 +108,14 @@ record_flushed_after_set_first(_Config) ->
fun (I, Acc) ->
element(2, ra_mt:insert({I, 1, <<"banana">>}, Acc))
end, Mt0, lists:seq(1, 100)),
- {Spec, Mt2} = ra_mt:record_flushed(Tid, {1, 49}, Mt1),
- ?assertMatch({range, _, {1, 49}}, Spec),
+ {Spec, Mt2} = ra_mt:record_flushed(Tid, [{1, 49}], Mt1),
+ ?assertMatch({indexes, _, [{1, 49}]}, Spec),
?assertMatch({50, 100}, ra_mt:range(Mt2)),
_ = ra_mt:delete(Spec),
{[Spec2], Mt3} = ra_mt:set_first(150, Mt2),
- ?assertMatch({range, Tid, {50, 100}}, Spec2),
+ ?assertMatch({indexes, Tid, [{50, 100}]}, Spec2),
?assertMatch(undefined, ra_mt:range(Mt3)),
- {undefined, Mt4} = ra_mt:record_flushed(Tid, {1, 49}, Mt3),
+ {undefined, Mt4} = ra_mt:record_flushed(Tid, [{1, 49}], Mt3),
?assertMatch(undefined, ra_mt:range(Mt4)),
ok.
@@ -134,14 +135,14 @@ record_flushed_prev(_Config) ->
end, Mt2, lists:seq(50, 80)),
?assertMatch({1, 100}, ra_mt:range(ra_mt:prev(Mt3))),
%%
- {Spec, Mt4} = ra_mt:record_flushed(Tid, {1, 49}, Mt3),
- ?assertMatch({range, Tid, {1, 49}}, Spec),
+ {Spec, Mt4} = ra_mt:record_flushed(Tid, [{1, 49}], Mt3),
+ ?assertMatch({indexes, Tid, [{1, 49}]}, Spec),
?assertMatch({50, 80}, ra_mt:range(Mt4)),
?assertMatch({50, 100}, ra_mt:range(ra_mt:prev(Mt4))),
_ = ra_mt:delete(Spec),
%% delete the remainder of the old mt
- {Spec2, Mt5} = ra_mt:record_flushed(Tid, {50, 100}, Mt4),
+ {Spec2, Mt5} = ra_mt:record_flushed(Tid, [{50, 100}], Mt4),
?assertMatch({delete, Tid}, Spec2),
?assertEqual(undefined, ra_mt:prev(Mt5)),
?assertMatch({50, 80}, ra_mt:range(Mt5)),
@@ -163,12 +164,12 @@ set_first(_Config) ->
element(2, ra_mt:insert({I, 2, <<"banana">>}, Acc))
end, Mt2, lists:seq(50, 120)),
{[Spec1, Spec2], Mt4} = ra_mt:set_first(75, Mt3),
- ?assertMatch({range, Tid2, {50, 74}}, Spec1),
- ?assertMatch({range, Tid, {1, 74}}, Spec2),
+ ?assertMatch({indexes, Tid2, [{50, 74}]}, Spec1),
+ ?assertMatch({indexes, Tid, [{1, 74}]}, Spec2),
?assertMatch({75, 120}, ra_mt:range(Mt4)),
{[Spec3, Spec4], Mt5} = ra_mt:set_first(105, Mt4),
- ?assertMatch({range, Tid2, {75, 104}}, Spec3),
+ ?assertMatch({indexes, Tid2, [{75, 104}]}, Spec3),
?assertMatch({delete, Tid}, Spec4),
?assertMatch({105, 120}, ra_mt:range(Mt5)),
?assertMatch(undefined, ra_mt:prev(Mt5)),
@@ -198,11 +199,11 @@ set_first_with_multi_prev(_Config) ->
?assertEqual({1, 200}, ra_mt:range(Mt3)),
- {[{range, Tid3, {75, 79}},
- {range, Tid2, {50, 79}},
- {range, Tid1, {1, 79}}], Mt4} = ra_mt:set_first(80, Mt3),
+ {[{indexes, Tid3, [{75, 79}]},
+ {indexes, Tid2, [{50, 79}]},
+ {indexes, Tid1, [{1, 79}]}], Mt4} = ra_mt:set_first(80, Mt3),
- {[{range, Tid3, {80, 159}},
+ {[{indexes, Tid3, [{80, 159}]},
{delete, Tid2},
{delete, Tid1}], _Mt5} = ra_mt:set_first(160, Mt4),
ok.
@@ -233,12 +234,12 @@ set_first_with_middle_small_range(_Config) ->
?assertEqual({1, 200}, ra_mt:range(Mt3)),
- {[{range, Tid3, {75, 84}},
+ {[{indexes, Tid3, [{75, 84}]},
{delete, Tid2},
- {range, Tid1, {1, 84}}], Mt4} = ra_mt:set_first(85, Mt3),
+ {indexes, Tid1, [{1, 84}]}], Mt4} = ra_mt:set_first(85, Mt3),
?assertEqual({85, 200}, ra_mt:range(Mt4)),
- {[{range, Tid3, {85, 100}},
+ {[{indexes, Tid3, [{85, 100}]},
{delete, Tid1}], Mt5} = ra_mt:set_first(101, Mt4),
?assertEqual({101, 200}, ra_mt:range(Mt5)),
?assertEqual(undefined, ra_mt:prev(Mt5)),
@@ -261,11 +262,16 @@ set_first_with_old_larger_range(_Config) ->
element(2, ra_mt:insert({I, 2, <<"banana">>}, Acc))
end, ra_mt:init_successor(Tid2, read_write, Mt1),
lists:seq(50, 75)),
- {[{range, Tid2, {50, 75}},
- {range, Tid1, {1, 84}}], Mt3} = ra_mt:set_first(85, Mt2),
+ {[{indexes, Tid2, [{50, 75}]},
+ {indexes, Tid1, [{1, 84}]}], Mt3} = ra_mt:set_first(85, Mt2),
+ ct:pal("Mt3 ~p", [Mt3]),
?assertEqual(undefined, ra_mt:range(Mt3)),
%% eventually when set_first passes the end of the old range it gets
%% deleted
+ % debugger:start(),
+ % int:i(ra_mt),
+ % int:break(ra_mt, 419),
+
{[{delete, Tid1}], Mt4} = ra_mt:set_first(101, Mt3),
?assertEqual(undefined, ra_mt:prev(Mt4)),
ok.
@@ -286,7 +292,7 @@ set_first_with_old_smaller_range(_Config) ->
lists:seq(1, 100)),
?assertEqual({1, 100}, ra_mt:range(Mt2)),
- {[{range, Tid2, {1, 84}},
+ {[{indexes, Tid2, [{1, 84}]},
{delete, Tid1}], Mt3} = ra_mt:set_first(85, Mt2),
?assertEqual({85, 100}, ra_mt:range(Mt3)),
ok.
@@ -312,10 +318,10 @@ successor(_Config) ->
?assertMatch({I, T, _}, ra_mt:lookup(I, Mt3))
end, lists:seq(1, 100)),
- {{range, Tid, {1, 20}}, Mt4a} = ra_mt:record_flushed(Tid, {1, 20}, Mt3),
+ {{indexes, Tid, [{1, 20}]}, Mt4a} = ra_mt:record_flushed(Tid, [{1, 20}], Mt3),
?assertMatch({21, 120}, ra_mt:range(Mt4a)),
- {{range, Tid, {1, 60}}, Mt4b} = ra_mt:record_flushed(Tid, {1, 60}, Mt3),
+ {{indexes, Tid, [{1, 60}]}, Mt4b} = ra_mt:record_flushed(Tid, [{1, 60}], Mt3),
?assertMatch({50, 120}, ra_mt:range(Mt4b)),
ok.
@@ -336,16 +342,16 @@ successor_below(_Config) ->
end, Mt2, lists:seq(50, 75)),
?assertMatch({50, 75}, ra_mt:range(Mt3)),
- {{range, Tid, {100, 150}}, Mt4a} =
- ra_mt:record_flushed(Tid, {100, 150}, Mt3),
+ {{indexes, Tid, [{100, 150}]}, Mt4a} =
+ ra_mt:record_flushed(Tid, [{100, 150}], Mt3),
?assertMatch({50, 75}, ra_mt:range(Mt4a)),
- {{range, Tid2, {50, 60}}, Mt4b} =
- ra_mt:record_flushed(Tid2, {50, 60}, Mt3),
+ {{indexes, Tid2, [{50, 60}]}, Mt4b} =
+ ra_mt:record_flushed(Tid2, [{50, 60}], Mt3),
?assertMatch({61, 75}, ra_mt:range(Mt4b)),
{{delete, Tid}, Mt4c} =
- ra_mt:record_flushed(Tid, {100, 200}, Mt3),
+ ra_mt:record_flushed(Tid, [{100, 200}], Mt3),
?assertMatch({50, 75}, ra_mt:range(Mt4c)),
?assertMatch(#{has_previous := false}, ra_mt:info(Mt4c)),
ok.
@@ -386,7 +392,7 @@ range_overlap(_Config) ->
% {{20, 30}, {31, 40}} = ra_mt:range_overlap({10, 40}, Mt1),
ok.
-stage_commit_abort(_Config) ->
+stage_commit_2(_Config) ->
Tid = ets:new(t1, [set, public]),
Mt0 = ra_mt:init(Tid),
Mt1 = lists:foldl(
@@ -396,8 +402,6 @@ stage_commit_abort(_Config) ->
?assertMatch(#{size := 0,
range := {20, 30}}, ra_mt:info(Mt1)),
- ?assertMatch(#{size := 0,
- range := undefined}, ra_mt:info(ra_mt:abort(Mt1))),
{[{20, _, _} | _] = Entries, Mt} = ra_mt:commit(Mt1),
?assertEqual(11, length(Entries)),
@@ -468,6 +472,19 @@ perf(_Config) ->
ok.
+%% TODO: expand sparse tests
+sparse(_Config) ->
+ Tid = ets:new(t1, [set, public]),
+ Mt0 = ra_mt:init(Tid),
+ {ok, Mt1} = ra_mt:insert_sparse({2, 1, <<"banana">>}, undefined, Mt0),
+ {ok, Mt2} = ra_mt:insert_sparse({5, 1, <<"banana">>}, 2, Mt1),
+ ?assertEqual({2, 5}, ra_mt:range(Mt2)),
+ {Spec, Mt3} = ra_mt:record_flushed(Tid, ra_seq:from_list([2, 5]), Mt2),
+ 2 = ra_mt:delete(Spec),
+ ?assertMatch(#{size := 0,
+ range := undefined}, ra_mt:info(Mt3)),
+ ?assertEqual(0, ets:info(Tid, size)),
+ ok.
%%% Util
diff --git a/test/ra_seq_SUITE.erl b/test/ra_seq_SUITE.erl
new file mode 100644
index 00000000..1a1ca0f1
--- /dev/null
+++ b/test/ra_seq_SUITE.erl
@@ -0,0 +1,151 @@
+-module(ra_seq_SUITE).
+
+-compile(nowarn_export_all).
+-compile(export_all).
+
+-export([
+ ]).
+
+-include_lib("eunit/include/eunit.hrl").
+
+%%%===================================================================
+%%% Common Test callbacks
+%%%===================================================================
+
+all() ->
+ [
+ {group, tests}
+ ].
+
+
+all_tests() ->
+ [
+ append,
+ floor,
+ limit,
+ add,
+ subtract,
+ iter,
+ remove_prefix,
+ remove_prefix_2
+ ].
+
+groups() ->
+ [{tests, [], all_tests()}].
+
+init_per_suite(Config) ->
+ Config.
+
+end_per_suite(_Config) ->
+ ok.
+
+init_per_group(_Group, Config) ->
+ Config.
+
+end_per_group(_Group, _Config) ->
+ ok.
+
+init_per_testcase(_TestCase, Config) ->
+ Config.
+
+end_per_testcase(_TestCase, _Config) ->
+ ok.
+
+%%%===================================================================
+%%% Test cases
+%%%===================================================================
+
+append(_Config) ->
+ S1 = [1] = ra_seq:append(1, []),
+ S2 = [2, 1] = ra_seq:append(2, S1),
+ S3 = [{1, 3}] = ra_seq:append(3, S2),
+ S4 = [{1, 4}] = ra_seq:append(4, S3),
+ S5 = [6, {1, 4}] = ra_seq:append(6, S4),
+
+ ?assertError(function_clause, ra_seq:append(2, S4)),
+ ?assertError(function_clause, ra_seq:append(6, S5)),
+
+ ok.
+
+floor(_Config) ->
+ S = ra_seq:from_list([1, 2, 3, 5, 6, 7, 8, 9, 11]),
+ [11] = ra_seq:floor(11, S),
+ [11, 9] = ra_seq:floor(9, S),
+ [11, 9, 8] = ra_seq:floor(8, S),
+ [11, {7, 9}] = ra_seq:floor(7, S),
+ [11, {6, 9}] = ra_seq:floor(6, S),
+ [11, {5, 9}] = ra_seq:floor(5, S),
+ [11, {5, 9}] = ra_seq:floor(4, S),
+ [11, {5, 9}, 3] = ra_seq:floor(3, S),
+ [11, {5, 9}, 3, 2] = ra_seq:floor(2, S),
+ [11, {5, 9}, {1, 3}] = ra_seq:floor(1, S),
+ [11, {5, 9}, {1, 3}] = ra_seq:floor(0, S),
+ ok.
+
+limit(_Config) ->
+ S = ra_seq:from_list([1, 2, 3, 5, 6, 7, 8, 9, 11]),
+ [11, {5, 9}, {1, 3}] = ra_seq:limit(11, S),
+ [{5, 9}, {1, 3}] = ra_seq:limit(10, S),
+ [{5, 9}, {1, 3}] = ra_seq:limit(9, S),
+ [{5, 8}, {1, 3}] = ra_seq:limit(8, S),
+ [{5, 7}, {1, 3}] = ra_seq:limit(7, S),
+ [6, 5, {1, 3}] = ra_seq:limit(6, S),
+ [5, {1, 3}] = ra_seq:limit(5, S),
+ [{1, 3}] = ra_seq:limit(4, S),
+ [{1, 3}] = ra_seq:limit(3, S),
+ [2, 1] = ra_seq:limit(2, S),
+ [1] = ra_seq:limit(1, S),
+ [] = ra_seq:limit(0, S),
+ ok.
+
+add(_Config) ->
+ S1 = ra_seq:from_list([1, 2, 3, 5, 6]),
+ S2 = ra_seq:from_list([7, 8, 9, 11]),
+ [11, {5, 9}, {1, 3}] = ra_seq:add(S2, S1),
+
+ S3 = ra_seq:from_list([1, 2, 3, 5, 6, 7, 8]),
+ S4 = ra_seq:from_list([7, 8, 9, 11]),
+ [11, {5, 9}, {1, 3}] = ra_seq:add(S4, S3),
+ ok.
+
+subtract(_Config) ->
+ [11, {7, 9}, {1, 3}] = ra_seq:subtract([{1, 11}], [10, {4, 6}]),
+ ok.
+
+iter(_Config) ->
+ S = ra_seq:from_list([1, 2, 3, 5, 6, 8, 9, 10, 12]),
+ I0 = ra_seq:iterator(S),
+ {1, I1} = ra_seq:next(I0),
+ {2, I2} = ra_seq:next(I1),
+ {3, I3} = ra_seq:next(I2),
+ {5, I4} = ra_seq:next(I3),
+ {6, I5} = ra_seq:next(I4),
+ {8, I6} = ra_seq:next(I5),
+ {9, I7} = ra_seq:next(I6),
+ {10, I8} = ra_seq:next(I7),
+ {12, I9} = ra_seq:next(I8),
+ end_of_seq = ra_seq:next(I9),
+ ok.
+
+remove_prefix(_Config) ->
+ S0 = ra_seq:from_list([2, 3, 5, 6, 8, 9, 10, 12]),
+ Pref1 = ra_seq:from_list([2, 3, 5]),
+ {ok, S1} = ra_seq:remove_prefix(Pref1, S0),
+ [12, 10, 9, 8, 6] = ra_seq:expand(S1),
+
+ %% prefix includes already removed items
+ Pref2 = ra_seq:from_list([1, 2, 3, 5]),
+ {ok, S2} = ra_seq:remove_prefix(Pref2, S0),
+ [12, 10, 9, 8, 6] = ra_seq:expand(S2),
+ %% not a prefix
+ Pref3 = ra_seq:from_list([5, 6, 8]),
+ {error, not_prefix} = ra_seq:remove_prefix(Pref3, S0),
+
+ {ok, []} = ra_seq:remove_prefix(S0, S0),
+ ok.
+
+remove_prefix_2(_Config) ->
+ S1 = ra_seq:from_list([2, 3, 4, 5]),
+ S2 = ra_seq:from_list([1, 2, 3]),
+ {ok, [5, 4]} = ra_seq:remove_prefix(S2, S1),
+ ok.
diff --git a/test/ra_server_SUITE.erl b/test/ra_server_SUITE.erl
index f811fb38..044d30ab 100644
--- a/test/ra_server_SUITE.erl
+++ b/test/ra_server_SUITE.erl
@@ -11,6 +11,12 @@
-include("src/ra_server.hrl").
-include_lib("eunit/include/eunit.hrl").
+%% TODO: make so this is not needed
+-dialyzer({nowarn_function,
+ [init_test/1,
+ higher_term_detected/1,
+ follower_aer_term_mismatch_snapshot/1]}).
+
all() ->
[
init_test,
@@ -152,27 +158,27 @@ setup_log() ->
ra_lib:default(get({U, K}), D)
end),
meck:expect(ra_snapshot, begin_accept,
- fun(_Meta, SS) ->
- {ok, SS}
+ fun(Meta, undefined) ->
+ {ok, {Meta, undefined}}
end),
meck:expect(ra_snapshot, accept_chunk,
- fun(_Data, _OutOf, _Flag, SS) ->
- {ok, SS}
+ fun(Data, _OutOf, _Flag, {Meta, _}) ->
+ {ok, {Meta, Data}, []}
end),
meck:expect(ra_snapshot, abort_accept, fun(SS) -> SS end),
meck:expect(ra_snapshot, accepting, fun(_SS) -> undefined end),
meck:expect(ra_log, snapshot_state, fun ra_log_memory:snapshot_state/1),
meck:expect(ra_log, set_snapshot_state, fun ra_log_memory:set_snapshot_state/2),
- meck:expect(ra_log, install_snapshot, fun ra_log_memory:install_snapshot/3),
+ meck:expect(ra_log, install_snapshot, fun ra_log_memory:install_snapshot/4),
meck:expect(ra_log, recover_snapshot, fun ra_log_memory:recover_snapshot/1),
meck:expect(ra_log, snapshot_index_term, fun ra_log_memory:snapshot_index_term/1),
meck:expect(ra_log, fold, fun ra_log_memory:fold/5),
meck:expect(ra_log, release_resources, fun ra_log_memory:release_resources/3),
meck:expect(ra_log, append_sync,
- fun({Idx, Term, _} = E, L) ->
- L1 = ra_log_memory:append(E, L),
- {LX, _} = ra_log_memory:handle_event({written, Term, {Idx, Idx}}, L1),
- LX
+ fun({Idx, Term, _} = E, L0) ->
+ L1 = ra_log_memory:append(E, L0),
+ {L, _} = ra_log_memory:handle_event({written, Term, [Idx]}, L1),
+ L
end),
meck:expect(ra_log, write_config, fun ra_log_memory:write_config/2),
meck:expect(ra_log, next_index, fun ra_log_memory:next_index/1),
@@ -222,8 +228,9 @@ init_test(_Config) ->
cluster => dehydrate_cluster(Cluster),
machine_version => 1},
SnapshotData = "hi1+2+3",
- {LogS, _} = ra_log_memory:install_snapshot({3, 5}, {SnapshotMeta,
- SnapshotData}, Log0),
+ {_, _, LogS, _} = ra_log_memory:install_snapshot({3, 5}, {SnapshotMeta,
+ SnapshotData},
+ ?MODULE, Log0),
meck:expect(ra_log, init, fun (_) -> LogS end),
#{current_term := 5,
commit_index := 3,
@@ -722,7 +729,7 @@ follower_aer_term_mismatch_snapshot(_Config) ->
cluster => #{},
machine_version => 1},
Data = <<"hi3">>,
- {Log,_} = ra_log_memory:install_snapshot({3, 5}, {Meta, Data}, Log0),
+ {_, _, Log,_} = ra_log_memory:install_snapshot({3, 5}, {Meta, Data}, ?MODULE, Log0),
State = maps:put(log, Log, State0),
AE = #append_entries_rpc{term = 6,
@@ -1867,8 +1874,8 @@ follower_cluster_change(_Config) ->
ok.
-written_evt(Term, Range) ->
- {ra_log_event, {written, Term, Range}}.
+written_evt(Term, Range) when is_tuple(Range) ->
+ {ra_log_event, {written, Term, [Range]}}.
leader_applies_new_cluster(_Config) ->
N1 = ?N1, N2 = ?N2, N3 = ?N3, N4 = ?N4,
@@ -2252,14 +2259,6 @@ follower_installs_snapshot(_Config) ->
[{next_event, ISRpc}, {record_leader_msg, _}]} =
ra_server:handle_follower(ISRpc, FState),
- meck:expect(ra_log, recover_snapshot,
- fun (_) ->
- {#{index => Idx,
- term => Term,
- cluster => dehydrate_cluster(Config),
- machine_version => 0},
- []}
- end),
{follower, #{current_term := Term,
commit_index := Idx,
last_applied := Idx,
@@ -3153,7 +3152,7 @@ base_state(NumServers, MacMod) ->
[{1, 1, usr(<<"hi1">>)},
{2, 3, usr(<<"hi2">>)},
{3, 5, usr(<<"hi3">>)}]),
- {Log, _} = ra_log:handle_event({written, 5, {1, 3}}, Log0),
+ {Log, _} = ra_log:handle_event({written, 5, [{1, 3}]}, Log0),
Servers = lists:foldl(fun(N, Acc) ->
Name = {list_to_atom("n" ++ integer_to_list(N)), node()},
diff --git a/test/ra_snapshot_SUITE.erl b/test/ra_snapshot_SUITE.erl
index 18d16403..a0bac05d 100644
--- a/test/ra_snapshot_SUITE.erl
+++ b/test/ra_snapshot_SUITE.erl
@@ -16,6 +16,8 @@
-include_lib("eunit/include/eunit.hrl").
-include("src/ra.hrl").
+-define(MACMOD, ?MODULE).
+
%%%===================================================================
%%% Common Test callbacks
%%%===================================================================
@@ -38,7 +40,8 @@ all_tests() ->
read_snapshot,
accept_snapshot,
abort_accept,
- accept_receives_snapshot_written_with_lower_index
+ accept_receives_snapshot_written_with_higher_index,
+ accept_receives_snapshot_written_with_higher_index_2
].
groups() ->
@@ -92,14 +95,17 @@ take_snapshot(Config) ->
UId = ?config(uid, Config),
State0 = init_state(Config),
Meta = meta(55, 2, [node()]),
- MacRef = ?FUNCTION_NAME,
- {State1, [{monitor, process, snapshot_writer, Pid}]} =
- ra_snapshot:begin_snapshot(Meta, MacRef, snapshot, State0),
+ MacState = ?FUNCTION_NAME,
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(Meta, ?MACMOD,MacState, snapshot, State0),
undefined = ra_snapshot:current(State1),
- {Pid, {55, 2}, snapshot} = ra_snapshot:pending(State1),
+ Fun(),
+ {{55, 2}, snapshot} = ra_snapshot:pending(State1),
receive
- {ra_log_event, {snapshot_written, {55, 2} = IdxTerm, snapshot}} ->
- State = ra_snapshot:complete_snapshot(IdxTerm, snapshot, State1),
+ {ra_log_event,
+ {snapshot_written, {55, 2} = IdxTerm, Indexes, snapshot}} ->
+ State = ra_snapshot:complete_snapshot(IdxTerm, snapshot,
+ Indexes, State1),
undefined = ra_snapshot:pending(State),
{55, 2} = ra_snapshot:current(State),
55 = ra_snapshot:last_index_for(UId),
@@ -114,28 +120,28 @@ take_snapshot_crash(Config) ->
SnapDir = ?config(snap_dir, Config),
State0 = init_state(Config),
Meta = meta(55, 2, [node()]),
- MacRef = ?FUNCTION_NAME,
- {State1, [{monitor, process, snapshot_writer, Pid}]} =
- ra_snapshot:begin_snapshot(Meta, MacRef, snapshot, State0),
+ MacState = ?FUNCTION_NAME,
+ {State1, [{bg_work, _Fun, ErrFun}]} =
+ ra_snapshot:begin_snapshot(Meta, ?MACMOD, MacState, snapshot, State0),
+ ErrFun({error, blah}),
undefined = ra_snapshot:current(State1),
- {Pid, {55, 2}, snapshot} = ra_snapshot:pending(State1),
+ {{55, 2}, snapshot} = ra_snapshot:pending(State1),
receive
- {ra_log_event, _} ->
- %% just pretend the snapshot event didn't happen
- %% and the process instead crashed
+ {ra_log_event, {snapshot_error, {55, 2} = IdxTerm, snapshot, Err}} ->
+ State = ra_snapshot:handle_error(IdxTerm, Err, State1),
+ undefined = ra_snapshot:pending(State),
+ undefined = ra_snapshot:current(State),
+ undefined = ra_snapshot:last_index_for(UId),
+
+ %% assert there are no snapshots now
+ ?assertEqual([], filelib:wildcard(filename:join(SnapDir, "*"))),
ok
- after 10 -> ok
+ after 10 ->
+ ct:fail("no log event")
end,
- State = ra_snapshot:handle_down(Pid, it_crashed_dawg, State1),
%% if the snapshot process crashed we just have to consider the
%% snapshot as faulty and clear it up
- undefined = ra_snapshot:pending(State),
- undefined = ra_snapshot:current(State),
- undefined = ra_snapshot:last_index_for(UId),
-
- %% assert there are no snapshots now
- ?assertEqual([], filelib:wildcard(filename:join(SnapDir, "*"))),
ok.
@@ -143,11 +149,13 @@ init_recover(Config) ->
UId = ?config(uid, Config),
State0 = init_state(Config),
Meta = meta(55, 2, [node()]),
- {State1, [{monitor, process, snapshot_writer, _}]} =
- ra_snapshot:begin_snapshot(Meta, ?FUNCTION_NAME, snapshot, State0),
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(Meta, ?MACMOD, ?FUNCTION_NAME, snapshot, State0),
+ Fun(),
receive
- {ra_log_event, {snapshot_written, IdxTerm, snapshot}} ->
- _ = ra_snapshot:complete_snapshot(IdxTerm, snapshot, State1),
+ {ra_log_event, {snapshot_written, IdxTerm, Indexes, snapshot}} ->
+ _ = ra_snapshot:complete_snapshot(IdxTerm, snapshot,
+ Indexes, State1),
ok
after 1000 ->
error(snapshot_event_timeout)
@@ -169,11 +177,12 @@ init_recover_voter_status(Config) ->
UId = ?config(uid, Config),
State0 = init_state(Config),
Meta = meta(55, 2, #{node() => #{voter_status => test}}),
- {State1, [{monitor, process, snapshot_writer, _}]} =
- ra_snapshot:begin_snapshot(Meta, ?FUNCTION_NAME, snapshot, State0),
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(Meta, ?MACMOD, ?FUNCTION_NAME, snapshot, State0),
+ Fun(),
receive
- {ra_log_event, {snapshot_written, IdxTerm, snapshot}} ->
- _ = ra_snapshot:complete_snapshot(IdxTerm, snapshot, State1),
+ {ra_log_event, {snapshot_written, IdxTerm, Indexes, snapshot}} ->
+ _ = ra_snapshot:complete_snapshot(IdxTerm, snapshot, Indexes, State1),
ok
after 1000 ->
error(snapshot_event_timeout)
@@ -196,16 +205,21 @@ init_multi(Config) ->
State0 = init_state(Config),
Meta1 = meta(55, 2, [node()]),
Meta2 = meta(165, 2, [node()]),
- {State1, _} = ra_snapshot:begin_snapshot(Meta1, ?FUNCTION_NAME, snapshot,
- State0),
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(Meta1, ?MACMOD, ?FUNCTION_NAME,
+ snapshot, State0),
+ %% simulate ra worker execution
+ Fun(),
receive
- {ra_log_event, {snapshot_written, IdxTerm, snapshot}} ->
- State2 = ra_snapshot:complete_snapshot(IdxTerm, snapshot, State1),
- {State3, _} = ra_snapshot:begin_snapshot(Meta2, ?FUNCTION_NAME,
- snapshot, State2),
- {_, {165, 2}, snapshot} = ra_snapshot:pending(State3),
+ {ra_log_event, {snapshot_written, IdxTerm, Indexes, snapshot}} ->
+ State2 = ra_snapshot:complete_snapshot(IdxTerm, snapshot, Indexes, State1),
+ {State3, [{bg_work, Fun2, _}]} =
+ ra_snapshot:begin_snapshot(Meta2, ?MACMOD, ?FUNCTION_NAME,
+ snapshot, State2),
+ {{165, 2}, snapshot} = ra_snapshot:pending(State3),
{55, 2} = ra_snapshot:current(State3),
55 = ra_snapshot:last_index_for(UId),
+ Fun2(),
receive
{ra_log_event, _} ->
%% don't complete snapshot
@@ -235,16 +249,20 @@ init_recover_multi_corrupt(Config) ->
State0 = init_state(Config),
Meta1 = meta(55, 2, [node()]),
Meta2 = meta(165, 2, [node()]),
- {State1, _} = ra_snapshot:begin_snapshot(Meta1, ?FUNCTION_NAME, snapshot,
- State0),
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(Meta1, ?MACMOD, ?FUNCTION_NAME,
+ snapshot, State0),
+ Fun(),
receive
- {ra_log_event, {snapshot_written, IdxTerm, snapshot}} ->
- State2 = ra_snapshot:complete_snapshot(IdxTerm, snapshot, State1),
- {State3, _} = ra_snapshot:begin_snapshot(Meta2, ?FUNCTION_NAME,
- snapshot, State2),
- {_, {165, 2}, snapshot} = ra_snapshot:pending(State3),
+ {ra_log_event, {snapshot_written, IdxTerm, Indexes, snapshot}} ->
+ State2 = ra_snapshot:complete_snapshot(IdxTerm, snapshot, Indexes, State1),
+ {State3, [{bg_work, Fun2, _}]} =
+ ra_snapshot:begin_snapshot(Meta2, ?MACMOD, ?FUNCTION_NAME,
+ snapshot, State2),
+ {{165, 2}, snapshot} = ra_snapshot:pending(State3),
{55, 2} = ra_snapshot:current(State3),
55 = ra_snapshot:last_index_for(UId),
+ Fun2(),
receive
{ra_log_event, _} ->
%% don't complete snapshot
@@ -280,11 +298,13 @@ init_recover_corrupt(Config) ->
Meta = meta(55, 2, [node()]),
SnapsDir = ?config(snap_dir, Config),
State0 = init_state(Config),
- {State1, _} = ra_snapshot:begin_snapshot(Meta, ?FUNCTION_NAME, snapshot,
- State0),
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(Meta, ?MACMOD, ?FUNCTION_NAME,
+ snapshot, State0),
+ Fun(),
_ = receive
- {ra_log_event, {snapshot_written, IdxTerm, snapshot}} ->
- ra_snapshot:complete_snapshot(IdxTerm, snapshot, State1)
+ {ra_log_event, {snapshot_written, IdxTerm, Indexes, snapshot}} ->
+ ra_snapshot:complete_snapshot(IdxTerm, snapshot, Indexes, State1)
after 1000 ->
error(snapshot_event_timeout)
end,
@@ -310,22 +330,23 @@ read_snapshot(Config) ->
State0 = init_state(Config),
Meta = meta(55, 2, [node()]),
MacRef = crypto:strong_rand_bytes(1024 * 4),
- {State1, _} =
- ra_snapshot:begin_snapshot(Meta, MacRef, snapshot, State0),
- State = receive
- {ra_log_event, {snapshot_written, IdxTerm, snapshot}} ->
- ra_snapshot:complete_snapshot(IdxTerm, snapshot, State1)
- after 1000 ->
- error(snapshot_event_timeout)
- end,
- Context = #{},
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(Meta, ?MACMOD, MacRef, snapshot, State0),
+ Fun(),
+ State = receive
+ {ra_log_event, {snapshot_written, IdxTerm, Indexes, snapshot}} ->
+ ra_snapshot:complete_snapshot(IdxTerm, snapshot, Indexes, State1)
+ after 1000 ->
+ error(snapshot_event_timeout)
+ end,
+ Context = #{},
- {ok, Meta, InitChunkState} = ra_snapshot:begin_read(State, Context),
+ {ok, Meta, InitChunkState} = ra_snapshot:begin_read(State, Context),
- <<_:32/integer, Data/binary>> = read_all_chunks(InitChunkState, State, 1024, <<>>),
- ?assertEqual(MacRef, binary_to_term(Data)),
+ <<_:32/integer, Data/binary>> = read_all_chunks(InitChunkState, State, 1024, <<>>),
+ ?assertEqual(MacRef, binary_to_term(Data)),
- ok.
+ ok.
read_all_chunks(ChunkState, State, Size, Acc) ->
case ra_snapshot:read_chunk(ChunkState, Size, State) of
@@ -355,11 +376,11 @@ accept_snapshot(Config) ->
undefined = ra_snapshot:accepting(State0),
{ok, S1} = ra_snapshot:begin_accept(Meta, State0),
{55, 2} = ra_snapshot:accepting(S1),
- {ok, S2} = ra_snapshot:accept_chunk(A, 1, next, S1),
- {ok, S3} = ra_snapshot:accept_chunk(B, 2, next, S2),
- {ok, S4} = ra_snapshot:accept_chunk(C, 3, next, S3),
- {ok, S5} = ra_snapshot:accept_chunk(D, 4, next, S4),
- {ok, S} = ra_snapshot:accept_chunk(E, 5, last, S5),
+ {ok, S2, _} = ra_snapshot:accept_chunk(A, 1, next, S1),
+ {ok, S3, _} = ra_snapshot:accept_chunk(B, 2, next, S2),
+ {ok, S4, _} = ra_snapshot:accept_chunk(C, 3, next, S3),
+ {ok, S5, _} = ra_snapshot:accept_chunk(D, 4, next, S4),
+ {ok, S, _} = ra_snapshot:accept_chunk(E, 5, last, S5),
undefined = ra_snapshot:accepting(S),
undefined = ra_snapshot:pending(S),
@@ -383,8 +404,8 @@ abort_accept(Config) ->
undefined = ra_snapshot:accepting(State0),
{ok, S1} = ra_snapshot:begin_accept(Meta, State0),
{55, 2} = ra_snapshot:accepting(S1),
- {ok, S2} = ra_snapshot:accept_chunk(A, 1, next, S1),
- {ok, S3} = ra_snapshot:accept_chunk(B, 2, next, S2),
+ {ok, S2, _} = ra_snapshot:accept_chunk(A, 1, next, S1),
+ {ok, S3, _} = ra_snapshot:accept_chunk(B, 2, next, S2),
S = ra_snapshot:abort_accept(S3),
undefined = ra_snapshot:accepting(S),
undefined = ra_snapshot:pending(S),
@@ -392,15 +413,16 @@ abort_accept(Config) ->
undefined = ra_snapshot:last_index_for(UId),
ok.
-accept_receives_snapshot_written_with_lower_index(Config) ->
+accept_receives_snapshot_written_with_higher_index(Config) ->
UId = ?config(uid, Config),
State0 = init_state(Config),
- MetaLocal = meta(55, 2, [node()]),
- MetaRemote = meta(165, 2, [node()]),
- MetaRemoteBin = term_to_binary(MetaRemote),
+ MetaLow = meta(55, 2, [node()]),
+ MetaHigh = meta(165, 2, [node()]),
+ MetaRemoteBin = term_to_binary(MetaHigh),
%% begin a local snapshot
- {State1, _} = ra_snapshot:begin_snapshot(MetaLocal, ?FUNCTION_NAME,
- snapshot, State0),
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(MetaLow, ?MACMOD, ?FUNCTION_NAME, snapshot, State0),
+ Fun(),
MacRef = crypto:strong_rand_bytes(1024),
MacBin = term_to_binary(MacRef),
Crc = erlang:crc32([<<(size(MetaRemoteBin)):32/unsigned>>,
@@ -411,19 +433,19 @@ accept_receives_snapshot_written_with_lower_index(Config) ->
B/binary>> = <>,
%% then begin an accept for a higher index
- {ok, State2} = ra_snapshot:begin_accept(MetaRemote, State1),
+ {ok, State2} = ra_snapshot:begin_accept(MetaHigh, State1),
{165, 2} = ra_snapshot:accepting(State2),
- {ok, State3} = ra_snapshot:accept_chunk(A, 1, next, State2),
+ {ok, State3, _} = ra_snapshot:accept_chunk(A, 1, next, State2),
%% then the snapshot written event is received
receive
- {ra_log_event, {snapshot_written, {55, 2} = IdxTerm, snapshot}} ->
- State4 = ra_snapshot:complete_snapshot(IdxTerm, snapshot, State3),
+ {ra_log_event, {snapshot_written, {55, 2} = IdxTerm, Indexes, snapshot}} ->
+ State4 = ra_snapshot:complete_snapshot(IdxTerm, snapshot, Indexes, State3),
undefined = ra_snapshot:pending(State4),
{55, 2} = ra_snapshot:current(State4),
55 = ra_snapshot:last_index_for(UId),
%% then accept the last chunk
- {ok, State} = ra_snapshot:accept_chunk(B, 2, last, State4),
+ {ok, State, _} = ra_snapshot:accept_chunk(B, 2, last, State4),
undefined = ra_snapshot:accepting(State),
{165, 2} = ra_snapshot:current(State),
ok
@@ -432,14 +454,16 @@ accept_receives_snapshot_written_with_lower_index(Config) ->
end,
ok.
-accept_receives_snapshot_written_with_higher_index(Config) ->
+accept_receives_snapshot_written_with_higher_index_2(Config) ->
UId = ?config(uid, Config),
State0 = init_state(Config),
- MetaRemote = meta(55, 2, [node()]),
- MetaLocal = meta(165, 2, [node()]),
+ MetaLow = meta(55, 2, [node()]),
+ MetaHigh = meta(165, 2, [node()]),
%% begin a local snapshot
- {State1, _} = ra_snapshot:begin_snapshot(MetaLocal, ?FUNCTION_NAME,
- snapshot, State0),
+ {State1, [{bg_work, Fun, _}]} =
+ ra_snapshot:begin_snapshot(MetaLow, ?MACMOD, ?FUNCTION_NAME,
+ snapshot, State0),
+ Fun(),
MacRef = crypto:strong_rand_bytes(1024),
MacBin = term_to_binary(MacRef),
%% split into 1024 max byte chunks
@@ -447,22 +471,24 @@ accept_receives_snapshot_written_with_higher_index(Config) ->
B/binary>> = MacBin,
%% then begin an accept for a higher index
- {ok, State2} = ra_snapshot:begin_accept(MetaRemote, State1),
- undefined = ra_snapshot:accepting(State2),
- {ok, State3} = ra_snapshot:accept_chunk(A, 1, next, State2),
- undefined = ra_snapshot:accepting(State3),
-
- %% then the snapshot written event is received
+ {ok, State2} = ra_snapshot:begin_accept(MetaHigh, State1),
+ {165, 2} = ra_snapshot:accepting(State2),
+ {ok, State3, _} = ra_snapshot:accept_chunk(A, 1, next, State2),
+ {165, 2} = ra_snapshot:accepting(State3),
+
+ {ok, State4, _} = ra_snapshot:accept_chunk(B, 2, last, State3),
+ undefined = ra_snapshot:accepting(State4),
+ {165, 2} = ra_snapshot:current(State4),
+ undefined = ra_snapshot:pending(State4),
+ %% then the snapshot written event is received after the higher index
+ %% has been received
receive
- {ra_log_event, {snapshot_written, {55, 2} = IdxTerm, snapshot}} ->
- State4 = ra_snapshot:complete_snapshot(IdxTerm, snapshot, State3),
- undefined = ra_snapshot:pending(State4),
- {55, 2} = ra_snapshot:current(State4),
- 55 = ra_snapshot:last_index_for(UId),
+ {ra_log_event, {snapshot_written, {55, 2} = IdxTerm, Indexes, snapshot}} ->
+ State5 = ra_snapshot:complete_snapshot(IdxTerm, snapshot, Indexes, State4),
+ undefined = ra_snapshot:pending(State5),
+ {165, 2} = ra_snapshot:current(State5),
+ 165 = ra_snapshot:last_index_for(UId),
%% then accept the last chunk
- {ok, State} = ra_snapshot:accept_chunk(B, 2, last, State4),
- undefined = ra_snapshot:accepting(State),
- {165, 2} = ra_snapshot:current(State),
ok
after 1000 ->
error(snapshot_event_timeout)
@@ -480,3 +506,7 @@ meta(Idx, Term, Cluster) ->
term => Term,
cluster => Cluster,
machine_version => 1}.
+
+%% ra_machine fakes
+version() -> 1.
+live_indexes(_) -> [].
|