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. Indicate whether the wal should compute and validate checksums. Default: `true` Boolean - - wal_write_strategy - - - - Enumeration: default | o_sync - 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](compaction1.jpg) + + +## 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. + +![segments](compaction2.jpg) + +### 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(_) -> [].